Browse Source

[yt/cpp/mapreduce] YT-21595: Use gtest instead of ytest in all mapreduce tests
85671f0cf4f45b4f015fa2cc0d195b81c16c6e8a

nadya73 8 months ago
parent
commit
5ea9afc5ee

+ 199 - 0
yt/cpp/mapreduce/http/ut/connection_pool_ut.cpp

@@ -0,0 +1,199 @@
+#include "simple_server.h"
+
+#include <yt/cpp/mapreduce/http/http.h>
+
+#include <yt/cpp/mapreduce/interface/config.h>
+
+#include <library/cpp/threading/future/async.h>
+
+#include <library/cpp/http/io/stream.h>
+
+#include <library/cpp/testing/gtest/gtest.h>
+
+#include <library/cpp/testing/common/network.h>
+
+#include <util/string/builder.h>
+#include <util/stream/tee.h>
+#include <util/system/thread.h>
+
+using namespace NYT;
+
+namespace {
+    void ParseFirstLine(const TString firstLine, TString& method, TString& host , ui64& port, TString& command)
+    {
+        size_t idx = firstLine.find_first_of(' ');
+        method = firstLine.substr(0, idx);
+        size_t idx2 = firstLine.find_first_of(':', idx + 1);
+        host = firstLine.substr(idx + 1, idx2 - idx - 1);
+        idx = firstLine.find_first_of('/', idx2 + 1);
+        port = std::atoi(firstLine.substr(idx2 + 1, idx - idx2 - 1).c_str());
+        idx2 = firstLine.find_first_of(' ', idx + 1);
+        command = firstLine.substr(idx, idx2 - idx);
+    }
+} // namespace
+
+THolder<TSimpleServer> CreateSimpleHttpServer()
+{
+    auto port = NTesting::GetFreePort();
+    return MakeHolder<TSimpleServer>(
+        port,
+        [] (IInputStream* input, IOutputStream* output) {
+            try {
+                while (true) {
+                    THttpInput httpInput(input);
+                    httpInput.ReadAll();
+
+                    THttpOutput httpOutput(output);
+                    httpOutput.EnableKeepAlive(true);
+                    httpOutput << "HTTP/1.1 200 OK\r\n";
+                    httpOutput << "\r\n";
+                    for (size_t i = 0; i != 10000; ++i) {
+                        httpOutput << "The grass was greener";
+                    }
+                    httpOutput.Flush();
+                }
+            } catch (const std::exception&) {
+            }
+        });
+}
+
+THolder<TSimpleServer> CreateProxyHttpServer()
+{
+    auto port = NTesting::GetFreePort();
+    return MakeHolder<TSimpleServer>(
+        port,
+        [] (IInputStream* input, IOutputStream* output) {
+            try {
+                while (true) {
+                    THttpInput httpInput(input);
+                    const TString inputStr = httpInput.FirstLine();
+                    auto headers = httpInput.Headers();
+                    TString method, command, host;
+                    ui64 port;
+                    ParseFirstLine(inputStr, method, host, port, command);
+
+                    THttpRequest request;
+                    const TString hostName = ::TStringBuilder() << host << ":" << port;
+                    request.Connect(hostName);
+                    auto header = THttpHeader(method, command);
+                    request.StartRequest(header);
+                    request.FinishRequest();
+                    auto res = request.GetResponseStream();
+                    THttpOutput httpOutput(output);
+                    httpOutput.EnableKeepAlive(true);
+                    auto strRes = res->ReadAll();
+                    httpOutput << "HTTP/1.1 200 OK\r\n";
+                    httpOutput << "\r\n";
+                    httpOutput << strRes;
+                    httpOutput.Flush();
+                }
+            } catch (const std::exception&) {
+            }
+        });
+}
+
+
+class TConnectionPoolConfigGuard
+{
+public:
+    TConnectionPoolConfigGuard(int newSize)
+    {
+        OldValue_ = TConfig::Get()->ConnectionPoolSize;
+        TConfig::Get()->ConnectionPoolSize = newSize;
+    }
+
+    ~TConnectionPoolConfigGuard()
+    {
+        TConfig::Get()->ConnectionPoolSize = OldValue_;
+    }
+
+private:
+    int OldValue_;
+};
+
+class TFuncThread
+    : public ISimpleThread
+{
+public:
+    using TFunc = std::function<void()>;
+
+public:
+    TFuncThread(const TFunc& func)
+        : Func_(func)
+    { }
+
+    void* ThreadProc() noexcept override {
+        Func_();
+        return nullptr;
+    }
+
+private:
+    TFunc Func_;
+};
+
+TEST(TConnectionPool, TestReleaseUnread)
+{
+    auto simpleServer = CreateSimpleHttpServer();
+
+    const TString hostName = ::TStringBuilder() << "localhost:" << simpleServer->GetPort();
+
+    for (size_t i = 0; i != 10; ++i) {
+        THttpRequest request;
+        request.Connect(hostName);
+        request.StartRequest(THttpHeader("GET", "foo"));
+        request.FinishRequest();
+        request.GetResponseStream();
+    }
+}
+
+TEST(TConnectionPool, TestProxy)
+{
+    auto simpleServer = CreateSimpleHttpServer();
+    auto simpleServer2 = CreateProxyHttpServer();
+
+    const TString hostName = ::TStringBuilder() << "localhost:" << simpleServer->GetPort();
+    const TString hostName2 = ::TStringBuilder() << "localhost:" << simpleServer2->GetPort();
+
+    for (size_t i = 0; i != 10; ++i) {
+        THttpRequest request;
+        request.Connect(hostName2);
+        auto header = THttpHeader("GET", "foo");
+        header.SetProxyAddress(hostName2);
+        header.SetHostPort(hostName);
+        request.StartRequest(header);
+        request.FinishRequest();
+        request.GetResponseStream();
+    }
+}
+
+TEST(TConnectionPool, TestConcurrency)
+{
+    TConnectionPoolConfigGuard g(1);
+
+    auto simpleServer = CreateSimpleHttpServer();
+    const TString hostName = ::TStringBuilder() << "localhost:" << simpleServer->GetPort();
+    auto threadPool = CreateThreadPool(20);
+
+    const auto func = [&] {
+        for (int i = 0; i != 100; ++i) {
+            THttpRequest request;
+            request.Connect(hostName);
+            request.StartRequest(THttpHeader("GET", "foo"));
+            request.FinishRequest();
+            auto res = request.GetResponseStream();
+            res->ReadAll();
+        }
+    };
+
+    TVector<THolder<TFuncThread>> threads;
+    for (int i = 0; i != 10; ++i) {
+        threads.push_back(MakeHolder<TFuncThread>(func));
+    };
+
+    for (auto& t : threads) {
+        t->Start();
+    }
+    for (auto& t : threads) {
+        t->Join();
+    }
+}

+ 94 - 0
yt/cpp/mapreduce/http/ut/http_ut.cpp

@@ -0,0 +1,94 @@
+#include "simple_server.h"
+
+#include <yt/cpp/mapreduce/http/http.h>
+
+#include <yt/cpp/mapreduce/interface/config.h>
+
+#include <library/cpp/testing/common/network.h>
+
+#include <library/cpp/testing/gtest/gtest.h>
+
+#include <util/system/byteorder.h>
+
+using namespace NYT;
+
+void WriteDataFrame(TStringBuf string, IOutputStream* stream)
+{
+    stream->Write("\x01");
+    ui32 size = string.Size();
+    auto littleEndianSize = HostToLittle(size);
+    stream->Write(&littleEndianSize, sizeof(littleEndianSize));
+    stream->Write(string);
+}
+
+THolder<TSimpleServer> CreateFramingEchoServer()
+{
+    auto port = NTesting::GetFreePort();
+    return MakeHolder<TSimpleServer>(
+        port,
+        [] (IInputStream* input, IOutputStream* output) {
+            try {
+                THttpInput httpInput(input);
+                if (!httpInput.Headers().FindHeader("X-YT-Accept-Framing")) {
+                    FAIL() << "X-YT-Accept-Framing header not found";
+                }
+                auto input = httpInput.ReadAll();
+
+                THttpOutput httpOutput(output);
+                httpOutput << "HTTP/1.1 200 OK\r\n";
+                httpOutput << "X-YT-Framing: 1\r\n";
+                httpOutput << "\r\n";
+                httpOutput << "\x02\x02"; // Two KeepAlive frames.
+                WriteDataFrame("", &httpOutput);
+                WriteDataFrame(TStringBuf(input).SubString(0, 10), &httpOutput);
+                httpOutput << "\x02"; // KeepAlive.
+                WriteDataFrame("", &httpOutput);
+                WriteDataFrame(TStringBuf(input).SubString(10, std::string::npos), &httpOutput);
+                httpOutput << "\x02"; // KeepAlive.
+
+                httpOutput.Flush();
+            } catch (const std::exception& exc) {
+            }
+        });
+}
+
+TEST(THttpHeaderTest, AddParameter)
+{
+    THttpHeader header("POST", "/foo");
+    header.AddMutationId();
+
+    auto id1 = header.GetParameters()["mutation_id"].AsString();
+
+    header.AddMutationId();
+
+    auto id2 = header.GetParameters()["mutation_id"].AsString();
+
+    EXPECT_TRUE(id1 != id2);
+}
+
+TEST(TFramingTest, FramingSimple)
+{
+    auto server = CreateFramingEchoServer();
+
+    THttpRequest request;
+    request.Connect(server->GetAddress());
+    auto requestStream = request.StartRequest(THttpHeader("POST", "concatenate"));
+    *requestStream << "Some funny data";
+    request.FinishRequest();
+    auto response = request.GetResponseStream()->ReadAll();
+    EXPECT_EQ(response, "Some funny data");
+}
+
+TEST(TFramingTest, FramingLarge)
+{
+    auto server = CreateFramingEchoServer();
+
+    THttpRequest request;
+    request.Connect(server->GetAddress());
+    auto requestStream = request.StartRequest(THttpHeader("POST", "concatenate"));
+    auto data = TString(100000, 'x');
+    *requestStream << data;
+    request.FinishRequest();
+    auto response = request.GetResponseStream()->ReadAll();
+    EXPECT_EQ(response, data);
+}

+ 90 - 0
yt/cpp/mapreduce/http/ut/simple_server.cpp

@@ -0,0 +1,90 @@
+#include "simple_server.h"
+
+#include <util/network/pair.h>
+#include <util/network/poller.h>
+#include <util/network/sock.h>
+#include <util/string/builder.h>
+#include <util/system/thread.h>
+#include <util/thread/pool.h>
+
+TSimpleServer::TSimpleServer(int port, TRequestHandler requestHandler)
+    : Port_(port)
+{
+    auto listenSocket = MakeAtomicShared<TInetStreamSocket>();
+    TSockAddrInet addr((TIpHost)INADDR_ANY, Port_);
+    SetSockOpt(*listenSocket, SOL_SOCKET, SO_REUSEADDR, 1);
+    int ret = listenSocket->Bind(&addr);
+    Y_ENSURE_EX(ret == 0, TSystemError() << "Can not bind");
+
+    SOCKET socketPair[2];
+    ret = SocketPair(socketPair);
+    Y_ENSURE_EX(ret == 0, TSystemError() << "Can not create socket pair");
+
+    ret = listenSocket->Listen(10);
+    Y_ENSURE_EX(ret == 0, TSystemError() << "Can not listen socket");
+
+    SendFinishSocket_ = MakeHolder<TInetStreamSocket>(socketPair[1]);
+
+    ThreadPool_ = MakeHolder<TAdaptiveThreadPool>();
+    ThreadPool_->Start(1);
+
+    auto receiveFinish = MakeAtomicShared<TInetStreamSocket>(socketPair[0]);
+    ListenerThread_ = ThreadPool_->Run([listenSocket, receiveFinish, requestHandler] {
+        TSocketPoller socketPoller;
+        socketPoller.WaitRead(*receiveFinish, nullptr);
+        socketPoller.WaitRead(*listenSocket, (void*)1);
+
+        bool running = true;
+        while (running) {
+            void* cookies[2];
+            size_t cookieCount = socketPoller.WaitI(cookies, 2);
+            for (size_t i = 0; i != cookieCount; ++i) {
+                if (!cookies[i]) {
+                    running = false;
+                } else {
+                    TSockAddrInet addr;
+                    TAtomicSharedPtr<TStreamSocket> socket = MakeAtomicShared<TInetStreamSocket>();
+                    int ret = listenSocket->Accept(socket.Get(), &addr);
+                    Y_ENSURE_EX(ret == 0, TSystemError() << "Can not accept connection");
+
+                    SystemThreadFactory()->Run(
+                        [socket, requestHandler] {
+                            TStreamSocketInput input(socket.Get());
+                            TStreamSocketOutput output(socket.Get());
+                            requestHandler(&input, &output);
+                            socket->Close();
+                        });
+                }
+            }
+        }
+    });
+}
+
+TSimpleServer::~TSimpleServer()
+{
+    try {
+        if (ThreadPool_) {
+            Stop();
+        }
+    } catch (...) {
+    }
+}
+
+void TSimpleServer::Stop()
+{
+    // Just send something to indicate shutdown.
+    SendFinishSocket_->Send("X", 1);
+    ListenerThread_->Join();
+    ThreadPool_->Stop();
+    ThreadPool_.Destroy();
+}
+
+int TSimpleServer::GetPort() const
+{
+    return Port_;
+}
+
+TString TSimpleServer::GetAddress() const
+{
+    return TStringBuilder() << "localhost:" << Port_;
+}

+ 35 - 0
yt/cpp/mapreduce/http/ut/simple_server.h

@@ -0,0 +1,35 @@
+#pragma once
+
+#include <util/generic/ptr.h>
+
+#include <util/stream/input.h>
+#include <util/stream/output.h>
+
+#include <util/thread/pool.h>
+
+#include <functional>
+
+class TInetStreamSocket;
+
+// Simple server listens on the specified port and launches
+// requestHandler in the separate thread for each incoming connection.
+class TSimpleServer
+{
+public:
+    using TRequestHandler = std::function<void(IInputStream* input, IOutputStream* output)>;
+
+public:
+    TSimpleServer(int port, TRequestHandler requestHandler);
+    ~TSimpleServer();
+
+    void Stop();
+
+    int GetPort() const;
+    TString GetAddress() const;
+
+private:
+    const int Port_;
+    THolder<IThreadPool> ThreadPool_;
+    THolder<IThreadFactory::IThread> ListenerThread_;
+    THolder<TInetStreamSocket> SendFinishSocket_;
+};

+ 14 - 0
yt/cpp/mapreduce/http/ut/ya.make

@@ -0,0 +1,14 @@
+GTEST()
+
+SRCS(
+    connection_pool_ut.cpp
+    http_ut.cpp
+    simple_server.cpp
+)
+
+PEERDIR(
+    yt/cpp/mapreduce/http
+    library/cpp/testing/common
+)
+
+END()

+ 4 - 0
yt/cpp/mapreduce/http/ya.make

@@ -27,3 +27,7 @@ PEERDIR(
 )
 
 END()
+
+RECURSE_FOR_TESTS(
+    ut
+)

+ 0 - 357
yt/cpp/mapreduce/interface/common_ut.cpp

@@ -1,357 +0,0 @@
-#include "common_ut.h"
-
-#include "fluent.h"
-
-#include <yt/cpp/mapreduce/interface/common.h>
-
-#include <yt/cpp/mapreduce/tests/yt_unittest_lib/yt_unittest_lib.h>
-
-#include <library/cpp/testing/unittest/registar.h>
-
-#include <library/cpp/yson/node/node_io.h>
-#include <library/cpp/yson/node/node_builder.h>
-
-#include <util/generic/xrange.h>
-
-using namespace NYT;
-
-template <class T>
-TString SaveToString(const T& obj)
-{
-    TString s;
-    TStringOutput out(s);
-    ::Save(&out, obj);
-    return s;
-}
-
-template <class T>
-T LoadFromString(TStringBuf s)
-{
-    TMemoryInput in(s);
-    T obj;
-    ::Load(&in, obj);
-    return obj;
-}
-
-template <class T>
-T SaveLoad(const T& obj)
-{
-    return LoadFromString<T>(SaveToString(obj));
-}
-
-Y_UNIT_TEST_SUITE(Common)
-{
-    Y_UNIT_TEST(SortColumnsLegacy)
-    {
-        TSortColumns keys1("a", "b");
-        UNIT_ASSERT((keys1.Parts_ == TSortColumns{"a", "b"}));
-
-        keys1.Add("c", "d");
-        UNIT_ASSERT((keys1.Parts_ == TSortColumns{"a", "b", "c", "d"}));
-
-        auto keys2 = TSortColumns(keys1).Add("e", "f");
-        UNIT_ASSERT((keys1.Parts_ == TSortColumns{"a", "b", "c", "d"}));
-        UNIT_ASSERT((keys2.Parts_ == TSortColumns{"a", "b", "c", "d", "e", "f"}));
-
-        auto keys3 = TSortColumns(keys1).Add("e").Add("f").Add("g");
-        UNIT_ASSERT((keys1.Parts_ == TSortColumns{"a", "b", "c", "d"}));
-        UNIT_ASSERT((keys3.Parts_ == TSortColumns{"a", "b", "c", "d", "e", "f", "g"}));
-    }
-
-    Y_UNIT_TEST(SortColumn)
-    {
-        auto ascending = TSortColumn("a");
-        UNIT_ASSERT_VALUES_EQUAL(ascending.Name(), "a");
-        UNIT_ASSERT_VALUES_EQUAL(ascending.SortOrder(), ESortOrder::SO_ASCENDING);
-        UNIT_ASSERT_VALUES_EQUAL(ascending, TSortColumn("a", ESortOrder::SO_ASCENDING));
-        UNIT_ASSERT_VALUES_UNEQUAL(ascending, TSortColumn("a", ESortOrder::SO_DESCENDING));
-
-        UNIT_ASSERT_NO_EXCEPTION(ascending.EnsureAscending());
-        UNIT_ASSERT_VALUES_EQUAL(static_cast<TString>(ascending), "a");
-        UNIT_ASSERT_VALUES_EQUAL(ascending, "a");
-
-        auto another = ascending;
-        UNIT_ASSERT_NO_EXCEPTION(another = "another");
-        UNIT_ASSERT_VALUES_EQUAL(another.Name(), "another");
-        UNIT_ASSERT_VALUES_EQUAL(another.SortOrder(), ESortOrder::SO_ASCENDING);
-        UNIT_ASSERT_VALUES_EQUAL(another, TSortColumn("another", ESortOrder::SO_ASCENDING));
-        UNIT_ASSERT_VALUES_UNEQUAL(another, TSortColumn("another", ESortOrder::SO_DESCENDING));
-
-        auto ascendingNode = BuildYsonNodeFluently().Value(ascending);
-        UNIT_ASSERT_VALUES_EQUAL(ascendingNode, TNode("a"));
-
-        UNIT_ASSERT_VALUES_EQUAL(SaveLoad(ascending), ascending);
-        UNIT_ASSERT_VALUES_UNEQUAL(SaveToString(ascending), SaveToString(TString("a")));
-
-        auto descending = TSortColumn("a", ESortOrder::SO_DESCENDING);
-        UNIT_ASSERT_VALUES_EQUAL(descending.Name(), "a");
-        UNIT_ASSERT_VALUES_EQUAL(descending.SortOrder(), ESortOrder::SO_DESCENDING);
-        UNIT_ASSERT_VALUES_EQUAL(descending, TSortColumn("a", ESortOrder::SO_DESCENDING));
-        UNIT_ASSERT_VALUES_UNEQUAL(descending, TSortColumn("a", ESortOrder::SO_ASCENDING));
-
-        UNIT_ASSERT_EXCEPTION(descending.EnsureAscending(), yexception);
-        UNIT_ASSERT_EXCEPTION(static_cast<TString>(descending), yexception);
-        UNIT_ASSERT_EXCEPTION(descending == "a", yexception);
-        UNIT_ASSERT_EXCEPTION(descending = "a", yexception);
-
-        auto descendingNode = BuildYsonNodeFluently().Value(descending);
-        UNIT_ASSERT_VALUES_EQUAL(descendingNode, TNode()("name", "a")("sort_order", "descending"));
-
-        UNIT_ASSERT_VALUES_EQUAL(SaveLoad(descending), descending);
-        UNIT_ASSERT_VALUES_UNEQUAL(SaveToString(descending), SaveToString("a"));
-
-        UNIT_ASSERT_VALUES_EQUAL(ToString(TSortColumn("blah")), "blah");
-        UNIT_ASSERT_VALUES_EQUAL(ToString(TSortColumn("blah", ESortOrder::SO_DESCENDING)), "{\"name\"=\"blah\";\"sort_order\"=\"descending\"}");
-    }
-
-    Y_UNIT_TEST(SortColumns)
-    {
-        TSortColumns ascending("a", "b");
-        UNIT_ASSERT(ascending.Parts_ == (TSortColumns{"a", "b"}));
-        UNIT_ASSERT_NO_EXCEPTION(ascending.EnsureAscending());
-        UNIT_ASSERT_VALUES_EQUAL(static_cast<TColumnNames>(ascending).Parts_, (TVector<TString>{"a", "b"}));
-        UNIT_ASSERT_VALUES_EQUAL(ascending.GetNames(), (TVector<TString>{"a", "b"}));
-
-        auto mixed = ascending;
-        mixed.Add(TSortColumn("c", ESortOrder::SO_DESCENDING), "d");
-        UNIT_ASSERT((mixed.Parts_ != TVector<TSortColumn>{"a", "b", "c", "d"}));
-        UNIT_ASSERT((mixed.Parts_ == TVector<TSortColumn>{"a", "b", TSortColumn("c", ESortOrder::SO_DESCENDING), "d"}));
-        UNIT_ASSERT_VALUES_EQUAL(mixed.GetNames(), (TVector<TString>{"a", "b", "c", "d"}));
-        UNIT_ASSERT_EXCEPTION(mixed.EnsureAscending(), yexception);
-        UNIT_ASSERT_EXCEPTION(static_cast<TColumnNames>(mixed), yexception);
-    }
-
-    Y_UNIT_TEST(KeyBound)
-    {
-        auto keyBound = TKeyBound(ERelation::Greater, TKey(7, "a", TNode()("x", "y")));
-        UNIT_ASSERT_VALUES_EQUAL(keyBound.Relation(), ERelation::Greater);
-        UNIT_ASSERT_EQUAL(keyBound.Key(), TKey(7, "a", TNode()("x", "y")));
-
-        auto keyBound1 = TKeyBound().Relation(ERelation::Greater).Key(TKey(7, "a", TNode()("x", "y")));
-        auto expectedNode = TNode()
-            .Add(">")
-            .Add(TNode().Add(7).Add("a").Add(TNode()("x", "y")));
-
-        UNIT_ASSERT_VALUES_EQUAL(expectedNode, BuildYsonNodeFluently().Value(keyBound));
-        UNIT_ASSERT_VALUES_EQUAL(expectedNode, BuildYsonNodeFluently().Value(keyBound1));
-
-        keyBound.Relation(ERelation::LessOrEqual);
-        keyBound.Key(TKey("A", 7));
-        UNIT_ASSERT_VALUES_EQUAL(keyBound.Relation(), ERelation::LessOrEqual);
-        UNIT_ASSERT_EQUAL(keyBound.Key(), TKey("A", 7));
-
-        UNIT_ASSERT_VALUES_EQUAL(
-            BuildYsonNodeFluently().Value(keyBound),
-            TNode()
-                .Add("<=")
-                .Add(TNode().Add("A").Add(7)));
-    }
-
-    Y_UNIT_TEST(TTableSchema)
-    {
-        TTableSchema schema;
-        schema
-            .AddColumn(TColumnSchema().Name("a").Type(EValueType::VT_STRING).SortOrder(SO_ASCENDING))
-            .AddColumn(TColumnSchema().Name("b").Type(EValueType::VT_UINT64))
-            .AddColumn(TColumnSchema().Name("c").Type(EValueType::VT_INT64));
-        auto checkSortBy = [](TTableSchema schema, const TVector<TString>& columns) {
-            auto initialSchema = schema;
-            schema.SortBy(columns);
-            for (auto i: xrange(columns.size())) {
-                UNIT_ASSERT_VALUES_EQUAL(schema.Columns()[i].Name(), columns[i]);
-                UNIT_ASSERT_VALUES_EQUAL(schema.Columns()[i].SortOrder(), ESortOrder::SO_ASCENDING);
-            }
-            for (auto i: xrange(columns.size(), (size_t)initialSchema.Columns().size())) {
-                UNIT_ASSERT_VALUES_EQUAL(schema.Columns()[i].SortOrder(), Nothing());
-            }
-            UNIT_ASSERT_VALUES_EQUAL(initialSchema.Columns().size(), schema.Columns().size());
-            return schema;
-        };
-        auto newSchema = checkSortBy(schema, {"b"});
-        UNIT_ASSERT_VALUES_EQUAL(newSchema.Columns()[1].Name(), TString("a"));
-        UNIT_ASSERT_VALUES_EQUAL(newSchema.Columns()[2].Name(), TString("c"));
-        checkSortBy(schema, {"b", "c"});
-        checkSortBy(schema, {"c", "a"});
-        UNIT_ASSERT_EXCEPTION(checkSortBy(schema, {"b", "b"}), yexception);
-        UNIT_ASSERT_EXCEPTION(checkSortBy(schema, {"a", "junk"}), yexception);
-    }
-
-    Y_UNIT_TEST(TTableSchema_Decimal)
-    {
-        NYT::TTableSchema tableSchema;
-
-        tableSchema.AddColumn("a", NTi::Decimal(35, 18));
-        tableSchema.AddColumn("b", NTi::Optional(NTi::Decimal(35, 18)));
-        tableSchema.AddColumn("c", NTi::List(NTi::Decimal(35, 18)));
-
-        auto tableSchemaNode = tableSchema.ToNode();
-        const auto& tableSchemaNodeList = tableSchemaNode.AsList();
-
-        // There was a bug in the serialization of decimal type: https://github.com/ytsaurus/ytsaurus/issues/173
-        {
-            const auto& currentType = tableSchemaNodeList[0];
-            UNIT_ASSERT_VALUES_EQUAL(currentType.ChildAsString("type"), "string");
-            UNIT_ASSERT(currentType.ChildAsBool("required"));
-            UNIT_ASSERT(currentType.HasKey("type_v3"));
-            UNIT_ASSERT_VALUES_EQUAL(currentType.At("type_v3").ChildAsString("type_name"), "decimal");
-        }
-        {
-            const auto& currentType = tableSchemaNodeList[1];
-            UNIT_ASSERT_VALUES_EQUAL(currentType.ChildAsString("type"), "string");
-            UNIT_ASSERT(!currentType.ChildAsBool("required"));
-            UNIT_ASSERT(currentType.HasKey("type_v3"));
-            UNIT_ASSERT_VALUES_EQUAL(currentType.At("type_v3").ChildAsString("type_name"), "optional");
-            UNIT_ASSERT_VALUES_EQUAL(currentType.At("type_v3").At("item").ChildAsString("type_name"), "decimal");
-        }
-        {
-            const auto& currentType = tableSchemaNodeList[2];
-            UNIT_ASSERT_VALUES_EQUAL(currentType.ChildAsString("type"), "any");
-            UNIT_ASSERT(currentType.ChildAsBool("required"));
-            UNIT_ASSERT(currentType.HasKey("type_v3"));
-            UNIT_ASSERT_VALUES_EQUAL(currentType.At("type_v3").ChildAsString("type_name"), "list");
-            UNIT_ASSERT_VALUES_EQUAL(currentType.At("type_v3").At("item").ChildAsString("type_name"), "decimal");
-        }
-
-        UNIT_ASSERT_EQUAL(tableSchema, TTableSchema::FromNode(tableSchemaNode));
-    }
-
-    Y_UNIT_TEST(TColumnSchema_TypeV3)
-    {
-        {
-            auto column = TColumnSchema().Type(NTi::Interval());
-            UNIT_ASSERT_VALUES_EQUAL(column.Required(), true);
-            UNIT_ASSERT_VALUES_EQUAL(column.Type(), VT_INTERVAL);
-        }
-        {
-            auto column = TColumnSchema().Type(NTi::Optional(NTi::Date()));
-            UNIT_ASSERT_VALUES_EQUAL(column.Required(), false);
-            UNIT_ASSERT_VALUES_EQUAL(column.Type(), VT_DATE);
-        }
-        {
-            auto column = TColumnSchema().Type(NTi::Interval64());
-            UNIT_ASSERT_VALUES_EQUAL(column.Required(), true);
-            UNIT_ASSERT_VALUES_EQUAL(column.Type(), VT_INTERVAL64);
-        }
-        {
-            auto column = TColumnSchema().Type(NTi::Optional(NTi::Date32()));
-            UNIT_ASSERT_VALUES_EQUAL(column.Required(), false);
-            UNIT_ASSERT_VALUES_EQUAL(column.Type(), VT_DATE32);
-        }
-        {
-            auto column = TColumnSchema().Type(NTi::Null());
-            UNIT_ASSERT_VALUES_EQUAL(column.Required(), false);
-            UNIT_ASSERT_VALUES_EQUAL(column.Type(), VT_NULL);
-        }
-        {
-            auto column = TColumnSchema().Type(NTi::Optional(NTi::Null()));
-            UNIT_ASSERT_VALUES_EQUAL(column.Required(), false);
-            UNIT_ASSERT_VALUES_EQUAL(column.Type(), VT_ANY);
-        }
-        {
-            auto column = TColumnSchema().Type(NTi::Decimal(35, 18));
-            UNIT_ASSERT_VALUES_EQUAL(column.Required(), true);
-            UNIT_ASSERT_VALUES_EQUAL(column.Type(), VT_STRING);
-        }
-    }
-
-    Y_UNIT_TEST(ToTypeV3)
-    {
-        UNIT_ASSERT_VALUES_EQUAL(*ToTypeV3(VT_INT32, true), *NTi::Int32());
-        UNIT_ASSERT_VALUES_EQUAL(*ToTypeV3(VT_UTF8, false), *NTi::Optional(NTi::Utf8()));
-    }
-
-    Y_UNIT_TEST(DeserializeColumn)
-    {
-        auto deserialize = [] (TStringBuf yson) {
-            auto node = NodeFromYsonString(yson);
-            TColumnSchema column;
-            Deserialize(column, node);
-            return column;
-        };
-
-        auto column = deserialize("{name=foo; type=int64; required=%false}");
-        UNIT_ASSERT_VALUES_EQUAL(column.Name(), "foo");
-        UNIT_ASSERT_VALUES_EQUAL(*column.TypeV3(), *NTi::Optional(NTi::Int64()));
-
-        column = deserialize("{name=bar; type=utf8; required=%true; type_v3=utf8}");
-        UNIT_ASSERT_VALUES_EQUAL(column.Name(), "bar");
-        UNIT_ASSERT_VALUES_EQUAL(*column.TypeV3(), *NTi::Utf8());
-    }
-
-    Y_UNIT_TEST(ColumnSchemaEquality)
-    {
-        auto base = TColumnSchema()
-            .Name("col")
-            .TypeV3(NTi::Optional(NTi::List(NTi::String())))
-            .SortOrder(ESortOrder::SO_ASCENDING)
-            .Lock("lock")
-            .Expression("x + 12")
-            .Aggregate("sum")
-            .Group("group");
-
-        auto other = base;
-        ASSERT_SERIALIZABLES_EQUAL(other, base);
-        other.Name("other");
-        ASSERT_SERIALIZABLES_UNEQUAL(other, base);
-
-        other = base;
-        other.TypeV3(NTi::List(NTi::String()));
-        ASSERT_SERIALIZABLES_UNEQUAL(other, base);
-
-        other = base;
-        other.ResetSortOrder();
-        ASSERT_SERIALIZABLES_UNEQUAL(other, base);
-
-        other = base;
-        other.Lock("lock1");
-        ASSERT_SERIALIZABLES_UNEQUAL(other, base);
-
-        other = base;
-        other.Expression("x + 13");
-        ASSERT_SERIALIZABLES_UNEQUAL(other, base);
-
-        other = base;
-        other.ResetAggregate();
-        ASSERT_SERIALIZABLES_UNEQUAL(other, base);
-
-        other = base;
-        other.Group("group1");
-        ASSERT_SERIALIZABLES_UNEQUAL(other, base);
-    }
-
-    Y_UNIT_TEST(TableSchemaEquality)
-    {
-        auto col1 = TColumnSchema()
-            .Name("col1")
-            .TypeV3(NTi::Optional(NTi::List(NTi::String())))
-            .SortOrder(ESortOrder::SO_ASCENDING);
-
-        auto col2 = TColumnSchema()
-            .Name("col2")
-            .TypeV3(NTi::Uint32());
-
-        auto schema = TTableSchema()
-            .AddColumn(col1)
-            .AddColumn(col2)
-            .Strict(true)
-            .UniqueKeys(true);
-
-        auto other = schema;
-        ASSERT_SERIALIZABLES_EQUAL(other, schema);
-
-        other.Strict(false);
-        ASSERT_SERIALIZABLES_UNEQUAL(other, schema);
-
-        other = schema;
-        other.MutableColumns()[0].TypeV3(NTi::List(NTi::String()));
-        ASSERT_SERIALIZABLES_UNEQUAL(other, schema);
-
-        other = schema;
-        other.MutableColumns().push_back(col1);
-        ASSERT_SERIALIZABLES_UNEQUAL(other, schema);
-
-        other = schema;
-        other.UniqueKeys(false);
-        ASSERT_SERIALIZABLES_UNEQUAL(other, schema);
-    }
-}

+ 0 - 20
yt/cpp/mapreduce/interface/config_ut.cpp

@@ -1,20 +0,0 @@
-#include <library/cpp/testing/unittest/registar.h>
-
-#include <yt/cpp/mapreduce/interface/config.h>
-
-using namespace NYT;
-
-Y_UNIT_TEST_SUITE(ConfigSuite)
-{
-    Y_UNIT_TEST(TestReset) {
-        // very limited test, checks only one config field
-
-        auto origConfig = *TConfig::Get();
-        TConfig::Get()->Reset();
-        UNIT_ASSERT_VALUES_EQUAL(origConfig.Hosts, TConfig::Get()->Hosts);
-
-        TConfig::Get()->Hosts = "hosts/fb867";
-        TConfig::Get()->Reset();
-        UNIT_ASSERT_VALUES_EQUAL(origConfig.Hosts, TConfig::Get()->Hosts);
-    }
-}

+ 0 - 81
yt/cpp/mapreduce/interface/error_ut.cpp

@@ -1,81 +0,0 @@
-#include <library/cpp/testing/unittest/registar.h>
-
-#include <library/cpp/json/json_reader.h>
-
-#include <yt/cpp/mapreduce/interface/errors.h>
-#include <yt/cpp/mapreduce/common/helpers.h>
-
-using namespace NYT;
-
-template<>
-void Out<NYT::TNode>(IOutputStream& s, const NYT::TNode& node)
-{
-    s << "TNode:" << NodeToYsonString(node);
-}
-
-Y_UNIT_TEST_SUITE(ErrorSuite)
-{
-    Y_UNIT_TEST(TestParseJson)
-    {
-        // Scary real world error! Бу!
-        const char* jsonText =
-            R"""({)"""
-                R"""("code":500,)"""
-                R"""("message":"Error resolving path //home/user/link",)"""
-                R"""("attributes":{)"""
-                    R"""("fid":18446484571700269066,)"""
-                    R"""("method":"Create",)"""
-                    R"""("tid":17558639495721339338,)"""
-                    R"""("datetime":"2017-04-07T13:38:56.474819Z",)"""
-                    R"""("pid":414529,)"""
-                    R"""("host":"build01-01g.yt.yandex.net"},)"""
-                R"""("inner_errors":[{)"""
-                    R"""("code":1,)"""
-                    R"""("message":"Node //tt cannot have children",)"""
-                    R"""("attributes":{)"""
-                        R"""("fid":18446484571700269066,)"""
-                        R"""("tid":17558639495721339338,)"""
-                        R"""("datetime":"2017-04-07T13:38:56.474725Z",)"""
-                        R"""("pid":414529,)"""
-                        R"""("host":"build01-01g.yt.yandex.net"},)"""
-                    R"""("inner_errors":[]}]})""";
-
-        NJson::TJsonValue jsonValue;
-        ReadJsonFastTree(jsonText, &jsonValue, /*throwOnError=*/ true);
-
-        TYtError error(jsonValue);
-        UNIT_ASSERT_VALUES_EQUAL(error.GetCode(), 500);
-        UNIT_ASSERT_VALUES_EQUAL(error.GetMessage(), R"""(Error resolving path //home/user/link)""");
-        UNIT_ASSERT_VALUES_EQUAL(error.InnerErrors().size(), 1);
-        UNIT_ASSERT_VALUES_EQUAL(error.InnerErrors()[0].GetCode(), 1);
-
-        UNIT_ASSERT_VALUES_EQUAL(error.HasAttributes(), true);
-        UNIT_ASSERT_VALUES_EQUAL(error.GetAttributes().at("method"), TNode("Create"));
-
-        UNIT_ASSERT_VALUES_EQUAL(error.GetAllErrorCodes(), TSet<int>({500, 1}));
-    }
-
-    Y_UNIT_TEST(TestGetYsonText) {
-        const char* jsonText =
-            R"""({)"""
-                R"""("code":500,)"""
-                R"""("message":"outer error",)"""
-                R"""("attributes":{)"""
-                    R"""("method":"Create",)"""
-                    R"""("pid":414529},)"""
-                R"""("inner_errors":[{)"""
-                    R"""("code":1,)"""
-                    R"""("message":"inner error",)"""
-                    R"""("attributes":{},)"""
-                    R"""("inner_errors":[])"""
-                R"""(}]})""";
-        TYtError error;
-        error.ParseFrom(jsonText);
-        TString ysonText = error.GetYsonText();
-        TYtError error2(NodeFromYsonString(ysonText));
-        UNIT_ASSERT_EQUAL(
-            ysonText,
-            R"""({"code"=500;"message"="outer error";"attributes"={"method"="Create";"pid"=414529};"inner_errors"=[{"code"=1;"message"="inner error"}]})""");
-        UNIT_ASSERT_EQUAL(error2.GetYsonText(), ysonText);
-    }
-}

+ 0 - 235
yt/cpp/mapreduce/interface/format_ut.cpp

@@ -1,235 +0,0 @@
-#include "common.h"
-#include "errors.h"
-#include "format.h"
-#include "common_ut.h"
-
-#include <yt/cpp/mapreduce/interface/proto3_ut.pb.h>
-#include <yt/cpp/mapreduce/interface/protobuf_table_schema_ut.pb.h>
-
-#include <library/cpp/testing/unittest/registar.h>
-
-using namespace NYT;
-
-static TNode GetColumns(const TFormat& format, int tableIndex = 0)
-{
-    return format.Config.GetAttributes()["tables"][tableIndex]["columns"];
-}
-
-Y_UNIT_TEST_SUITE(ProtobufFormat)
-{
-    Y_UNIT_TEST(TIntegral)
-    {
-        const auto format = TFormat::Protobuf<NUnitTesting::TIntegral>();
-        auto columns = GetColumns(format);
-
-        struct TColumn
-        {
-            TString Name;
-            TString ProtoType;
-            int FieldNumber;
-        };
-
-        auto expected = TVector<TColumn>{
-            {"DoubleField", "double", 1},
-            {"FloatField", "float", 2},
-            {"Int32Field", "int32", 3},
-            {"Int64Field", "int64", 4},
-            {"Uint32Field", "uint32", 5},
-            {"Uint64Field", "uint64", 6},
-            {"Sint32Field", "sint32", 7},
-            {"Sint64Field", "sint64", 8},
-            {"Fixed32Field", "fixed32", 9},
-            {"Fixed64Field", "fixed64", 10},
-            {"Sfixed32Field", "sfixed32", 11},
-            {"Sfixed64Field", "sfixed64", 12},
-            {"BoolField", "bool", 13},
-            {"EnumField", "enum_string", 14},
-        };
-
-        UNIT_ASSERT_VALUES_EQUAL(columns.Size(), expected.size());
-        for (int i = 0; i < static_cast<int>(columns.Size()); ++i) {
-            UNIT_ASSERT_VALUES_EQUAL(columns[i]["name"], expected[i].Name);
-            UNIT_ASSERT_VALUES_EQUAL(columns[i]["proto_type"], expected[i].ProtoType);
-            UNIT_ASSERT_VALUES_EQUAL(columns[i]["field_number"], expected[i].FieldNumber);
-        }
-    }
-
-    Y_UNIT_TEST(TRowFieldSerializationOption)
-    {
-        const auto format = TFormat::Protobuf<NUnitTesting::TRowFieldSerializationOption>();
-        auto columns = GetColumns(format);
-
-        UNIT_ASSERT_VALUES_EQUAL(columns[0]["name"], "UrlRow_1");
-        UNIT_ASSERT_VALUES_EQUAL(columns[0]["proto_type"], "structured_message");
-        UNIT_ASSERT_VALUES_EQUAL(columns[0]["field_number"], 1);
-        const auto& fields = columns[0]["fields"];
-        UNIT_ASSERT_VALUES_EQUAL(fields[0]["name"], "Host");
-        UNIT_ASSERT_VALUES_EQUAL(fields[0]["proto_type"], "string");
-        UNIT_ASSERT_VALUES_EQUAL(fields[0]["field_number"], 1);
-
-        UNIT_ASSERT_VALUES_EQUAL(fields[1]["name"], "Path");
-        UNIT_ASSERT_VALUES_EQUAL(fields[1]["proto_type"], "string");
-        UNIT_ASSERT_VALUES_EQUAL(fields[1]["field_number"], 2);
-
-        UNIT_ASSERT_VALUES_EQUAL(fields[2]["name"], "HttpCode");
-        UNIT_ASSERT_VALUES_EQUAL(fields[2]["proto_type"], "sint32");
-        UNIT_ASSERT_VALUES_EQUAL(fields[2]["field_number"], 3);
-
-        UNIT_ASSERT_VALUES_EQUAL(columns[1]["name"], "UrlRow_2");
-        UNIT_ASSERT_VALUES_EQUAL(columns[1]["proto_type"], "message");
-        UNIT_ASSERT_VALUES_EQUAL(columns[1]["field_number"], 2);
-    }
-
-    Y_UNIT_TEST(Packed)
-    {
-        const auto format = TFormat::Protobuf<NUnitTesting::TPacked>();
-        auto column = GetColumns(format)[0];
-
-        UNIT_ASSERT_VALUES_EQUAL(column["name"], "PackedListInt64");
-        UNIT_ASSERT_VALUES_EQUAL(column["proto_type"], "int64");
-        UNIT_ASSERT_VALUES_EQUAL(column["field_number"], 1);
-        UNIT_ASSERT_VALUES_EQUAL(column["packed"], true);
-        UNIT_ASSERT_VALUES_EQUAL(column["repeated"], true);
-    }
-
-    Y_UNIT_TEST(Cyclic)
-    {
-        UNIT_ASSERT_EXCEPTION(TFormat::Protobuf<NUnitTesting::TCyclic>(), TApiUsageError);
-        UNIT_ASSERT_EXCEPTION(TFormat::Protobuf<NUnitTesting::TCyclic::TA>(), TApiUsageError);
-        UNIT_ASSERT_EXCEPTION(TFormat::Protobuf<NUnitTesting::TCyclic::TB>(), TApiUsageError);
-        UNIT_ASSERT_EXCEPTION(TFormat::Protobuf<NUnitTesting::TCyclic::TC>(), TApiUsageError);
-        UNIT_ASSERT_EXCEPTION(TFormat::Protobuf<NUnitTesting::TCyclic::TD>(), TApiUsageError);
-
-        const auto format = TFormat::Protobuf<NUnitTesting::TCyclic::TE>();
-        auto column = GetColumns(format)[0];
-        UNIT_ASSERT_VALUES_EQUAL(column["name"], "d");
-        UNIT_ASSERT_VALUES_EQUAL(column["proto_type"], "message");
-        UNIT_ASSERT_VALUES_EQUAL(column["field_number"], 1);
-    }
-
-    Y_UNIT_TEST(Map)
-    {
-        const auto format = TFormat::Protobuf<NUnitTesting::TWithMap>();
-        auto columns = GetColumns(format);
-
-        UNIT_ASSERT_VALUES_EQUAL(columns.Size(), 5);
-        {
-            const auto& column = columns[0];
-            UNIT_ASSERT_VALUES_EQUAL(column["name"], "MapDefault");
-            UNIT_ASSERT_VALUES_EQUAL(column["proto_type"], "structured_message");
-            UNIT_ASSERT_VALUES_EQUAL(column["fields"].Size(), 2);
-            UNIT_ASSERT_VALUES_EQUAL(column["fields"][0]["proto_type"], "int64");
-            UNIT_ASSERT_VALUES_EQUAL(column["fields"][1]["proto_type"], "message");
-        }
-        {
-            const auto& column = columns[1];
-            UNIT_ASSERT_VALUES_EQUAL(column["name"], "MapListOfStructsLegacy");
-            UNIT_ASSERT_VALUES_EQUAL(column["proto_type"], "structured_message");
-            UNIT_ASSERT_VALUES_EQUAL(column["fields"].Size(), 2);
-            UNIT_ASSERT_VALUES_EQUAL(column["fields"][0]["proto_type"], "int64");
-            UNIT_ASSERT_VALUES_EQUAL(column["fields"][1]["proto_type"], "message");
-        }
-        {
-            const auto& column = columns[2];
-            UNIT_ASSERT_VALUES_EQUAL(column["name"], "MapListOfStructs");
-            UNIT_ASSERT_VALUES_EQUAL(column["proto_type"], "structured_message");
-            UNIT_ASSERT_VALUES_EQUAL(column["fields"].Size(), 2);
-            UNIT_ASSERT_VALUES_EQUAL(column["fields"][0]["proto_type"], "int64");
-            UNIT_ASSERT_VALUES_EQUAL(column["fields"][1]["proto_type"], "structured_message");
-        }
-        {
-            const auto& column = columns[3];
-            UNIT_ASSERT_VALUES_EQUAL(column["name"], "MapOptionalDict");
-            UNIT_ASSERT_VALUES_EQUAL(column["proto_type"], "structured_message");
-            UNIT_ASSERT_VALUES_EQUAL(column["fields"].Size(), 2);
-            UNIT_ASSERT_VALUES_EQUAL(column["fields"][0]["proto_type"], "int64");
-            UNIT_ASSERT_VALUES_EQUAL(column["fields"][1]["proto_type"], "structured_message");
-        }
-        {
-            const auto& column = columns[4];
-            UNIT_ASSERT_VALUES_EQUAL(column["name"], "MapDict");
-            UNIT_ASSERT_VALUES_EQUAL(column["proto_type"], "structured_message");
-            UNIT_ASSERT_VALUES_EQUAL(column["fields"].Size(), 2);
-            UNIT_ASSERT_VALUES_EQUAL(column["fields"][0]["proto_type"], "int64");
-            UNIT_ASSERT_VALUES_EQUAL(column["fields"][1]["proto_type"], "structured_message");
-        }
-    }
-
-    Y_UNIT_TEST(Oneof)
-    {
-        const auto format = TFormat::Protobuf<NUnitTesting::TWithOneof>();
-        auto columns = GetColumns(format);
-
-        UNIT_ASSERT_VALUES_EQUAL(columns.Size(), 4);
-        auto check = [] (const TNode& column, TStringBuf name, TStringBuf oneof2Name) {
-            UNIT_ASSERT_VALUES_EQUAL(column["name"], name);
-            UNIT_ASSERT_VALUES_EQUAL(column["proto_type"], "structured_message");
-            UNIT_ASSERT_VALUES_EQUAL(column["fields"].Size(), 5);
-            UNIT_ASSERT_VALUES_EQUAL(column["fields"][0]["name"], "field");
-
-            const auto& oneof2 = column["fields"][1];
-            UNIT_ASSERT_VALUES_EQUAL(oneof2["name"], oneof2Name);
-            UNIT_ASSERT_VALUES_EQUAL(oneof2["proto_type"], "oneof");
-            UNIT_ASSERT_VALUES_EQUAL(oneof2["fields"][0]["name"], "y2");
-            UNIT_ASSERT_VALUES_EQUAL(oneof2["fields"][1]["name"], "z2");
-            UNIT_ASSERT_VALUES_EQUAL(oneof2["fields"][1]["proto_type"], "structured_message");
-            const auto& embeddedOneof = oneof2["fields"][1]["fields"][0];
-            UNIT_ASSERT_VALUES_EQUAL(embeddedOneof["name"], "Oneof");
-            UNIT_ASSERT_VALUES_EQUAL(embeddedOneof["fields"][0]["name"], "x");
-            UNIT_ASSERT_VALUES_EQUAL(embeddedOneof["fields"][1]["name"], "y");
-            UNIT_ASSERT_VALUES_EQUAL(oneof2["fields"][2]["name"], "x2");
-
-            UNIT_ASSERT_VALUES_EQUAL(column["fields"][2]["name"], "x1");
-            UNIT_ASSERT_VALUES_EQUAL(column["fields"][3]["name"], "y1");
-            UNIT_ASSERT_VALUES_EQUAL(column["fields"][4]["name"], "z1");
-        };
-
-        check(columns[0], "DefaultSeparateFields", "variant_field_name");
-        check(columns[1], "NoDefault", "Oneof2");
-
-        {
-            const auto& column = columns[2];
-            UNIT_ASSERT_VALUES_EQUAL(column["name"], "SerializationProtobuf");
-            UNIT_ASSERT_VALUES_EQUAL(column["proto_type"], "structured_message");
-            UNIT_ASSERT_VALUES_EQUAL(column["fields"].Size(), 3);
-            UNIT_ASSERT_VALUES_EQUAL(column["fields"][0]["name"], "x1");
-            UNIT_ASSERT_VALUES_EQUAL(column["fields"][1]["name"], "y1");
-            UNIT_ASSERT_VALUES_EQUAL(column["fields"][2]["name"], "z1");
-        }
-        {
-            const auto& column = columns[3];
-            UNIT_ASSERT_VALUES_EQUAL(column["name"], "TopLevelOneof");
-            UNIT_ASSERT_VALUES_EQUAL(column["proto_type"], "oneof");
-            UNIT_ASSERT_VALUES_EQUAL(column["fields"].Size(), 1);
-            UNIT_ASSERT_VALUES_EQUAL(column["fields"][0]["name"], "MemberOfTopLevelOneof");
-        }
-    }
-}
-
-Y_UNIT_TEST_SUITE(Proto3)
-{
-    Y_UNIT_TEST(TWithOptional)
-    {
-        const auto format = TFormat::Protobuf<NTestingProto3::TWithOptional>();
-        auto columns = GetColumns(format);
-
-        UNIT_ASSERT_VALUES_EQUAL(columns[0]["name"], "x");
-        UNIT_ASSERT_VALUES_EQUAL(columns[0]["proto_type"], "int64");
-        UNIT_ASSERT_VALUES_EQUAL(columns[0]["field_number"], 1);
-    }
-
-    Y_UNIT_TEST(TWithOptionalMessage)
-    {
-        const auto format = TFormat::Protobuf<NTestingProto3::TWithOptionalMessage>();
-        auto columns = GetColumns(format);
-
-        UNIT_ASSERT_VALUES_EQUAL(columns[0]["name"], "x");
-        UNIT_ASSERT_VALUES_EQUAL(columns[0]["proto_type"], "structured_message");
-        UNIT_ASSERT_VALUES_EQUAL(columns[0]["field_number"], 1);
-
-        UNIT_ASSERT_VALUES_EQUAL(columns[0]["fields"].Size(), 1);
-        UNIT_ASSERT_VALUES_EQUAL(columns[0]["fields"][0]["name"], "x");
-        UNIT_ASSERT_VALUES_EQUAL(columns[0]["fields"][0]["proto_type"], "int64");
-        UNIT_ASSERT_VALUES_EQUAL(columns[0]["fields"][0]["field_number"], 1);
-    }
-}

Some files were not shown because too many files changed in this diff