diff options
author | aneporada <aneporada@yandex-team.com> | 2024-12-02 16:10:41 +0300 |
---|---|---|
committer | aneporada <aneporada@yandex-team.com> | 2024-12-02 16:27:04 +0300 |
commit | 64894f0aef4182e171cd8b84e1520b894e3f7119 (patch) | |
tree | 8d29ced5dec93f67887814f094bfa75bd20d6d75 | |
parent | 399c0870221aa4217fe09d54fd6700fd41ae2687 (diff) | |
download | ydb-64894f0aef4182e171cd8b84e1520b894e3f7119.tar.gz |
Add yt provider to piglet conf
После переноса YT провайдера из папки contriby/ydb в папку yt/yql добавляем его в конфигурацию синка.
Ранее yt провайдер жил в публичном гитхабе YDB
commit_hash:a530faa0bbc496a5c44854372948645872bf9882
278 files changed, 86382 insertions, 0 deletions
diff --git a/yt/yql/providers/yt/codec/codegen/no_llvm/ya.make b/yt/yql/providers/yt/codec/codegen/no_llvm/ya.make new file mode 100644 index 0000000000..09a9f0b057 --- /dev/null +++ b/yt/yql/providers/yt/codec/codegen/no_llvm/ya.make @@ -0,0 +1,22 @@ +LIBRARY() + +ADDINCL( + yt/yql/providers/yt/codec/codegen +) + +SRCDIR( + yt/yql/providers/yt/codec/codegen +) + +SRCS( + yt_codec_cg_dummy.cpp +) + +PEERDIR( +) + +PROVIDES(YT_CODEC_CODEGEN) + +YQL_LAST_ABI_VERSION() + +END() diff --git a/yt/yql/providers/yt/codec/codegen/ut/ya.make b/yt/yql/providers/yt/codec/codegen/ut/ya.make new file mode 100644 index 0000000000..cb6befcbf8 --- /dev/null +++ b/yt/yql/providers/yt/codec/codegen/ut/ya.make @@ -0,0 +1,30 @@ +UNITTEST_FOR(yt/yql/providers/yt/codec/codegen) + +SRCS( + yt_codec_cg_ut.cpp +) + +IF (SANITIZER_TYPE OR WITH_VALGRIND) + SIZE(MEDIUM) +ELSE() + SIZE(SMALL) +ENDIF() + + +PEERDIR( + yql/essentials/minikql/computation/llvm14 + yql/essentials/public/udf/service/exception_policy + yql/essentials/sql + yql/essentials/sql/pg_dummy + yt/yql/providers/yt/codec +) + +YQL_LAST_ABI_VERSION() + +IF (MKQL_DISABLE_CODEGEN) + CFLAGS( + -DMKQL_DISABLE_CODEGEN + ) +ENDIF() + +END() diff --git a/yt/yql/providers/yt/codec/codegen/ut/yt_codec_cg_ut.cpp b/yt/yql/providers/yt/codec/codegen/ut/yt_codec_cg_ut.cpp new file mode 100644 index 0000000000..6ee03c1ecc --- /dev/null +++ b/yt/yql/providers/yt/codec/codegen/ut/yt_codec_cg_ut.cpp @@ -0,0 +1,958 @@ +#ifndef MKQL_DISABLE_CODEGEN +#include "yt_codec_cg.h" +#include <yql/essentials/providers/common/codec/yql_codec_buf.h> + +#include <yql/essentials/minikql/computation/mkql_computation_node_holders.h> +#include <yql/essentials/minikql/mkql_node.h> +#include <yql/essentials/minikql/mkql_string_util.h> + +#include <yt/yql/providers/yt/codec/yt_codec_io.h> +#include <yql/essentials/minikql/codegen/codegen.h> + +#include <llvm/IR/Module.h> + +#include <library/cpp/testing/unittest/registar.h> + +namespace NYql { + +using namespace NCommon; +using namespace NCodegen; +using namespace NKikimr; +using namespace NKikimr::NMiniKQL; +using namespace std::string_view_literals; + +class TTestWriter : public IBlockWriter { +public: + TTestWriter() + { + Buffer_.ReserveAndResize(1024); + } + + void SetRecordBoundaryCallback(std::function<void()> /*callback*/) override { + } + + std::pair<char*, char*> NextEmptyBlock() override { + if (FirstBuffer_) { + FirstBuffer_ = false; + auto ptr = Buffer_.Detach(); + return {ptr, ptr + Buffer_.capacity()}; + } else { + return {nullptr, nullptr}; + } + } + + void ReturnBlock(size_t avail, std::optional<size_t> /*lastRecordBoundary*/) override { + Buffer_.resize(avail); + } + + void Finish() override { + } + + TString Str() const { + return Buffer_; + } + +private: + TString Buffer_; + bool FirstBuffer_ = true; +}; + +struct TWriteSetup { + TWriteSetup(const char* funcName) + : Buf_(TestWriter_, nullptr) + { + Codegen_ = ICodegen::Make(ETarget::Native); + Codegen_->LoadBitCode(GetYtCodecBitCode(), "YtCodecFuncs"); + Func_ = Codegen_->GetModule().getFunction(funcName); + UNIT_ASSERT(Func_); + Codegen_->Verify(); + YtCodecAddMappings(*Codegen_); + Codegen_->ExportSymbol(Func_); + Codegen_->Compile(); + //Codegen_->GetModule().dump(); + } + + ICodegen::TPtr Codegen_; + llvm::Function* Func_; + TTestWriter TestWriter_; + TOutputBuf Buf_; +}; + +class TTestReader : public IBlockReader { +public: + TTestReader(TStringBuf data) + : Data_(data) + {} + + void SetDeadline(TInstant deadline) override { + Y_UNUSED(deadline); + } + + std::pair<const char*, const char*> NextFilledBlock() override { + if (FirstBuffer_) { + FirstBuffer_ = false; + return std::make_pair(Data_.begin(), Data_.end()); + } else { + return { nullptr, nullptr }; + } + } + + void ReturnBlock() override { + YQL_ENSURE(!FirstBuffer_); + } + + bool Retry(const TMaybe<ui32>& rangeIndex, const TMaybe<ui64>& rowIndex, const std::exception_ptr& error) override { + Y_UNUSED(rangeIndex); + Y_UNUSED(rowIndex); + Y_UNUSED(error); + return false; + } + +private: + TStringBuf Data_; + bool FirstBuffer_ = true; +}; + +struct TReadSetup { + TReadSetup(const char* funcName, TStringBuf readData) + : TestReader_(readData) + , Buf_(TestReader_, nullptr) + { + Codegen_ = ICodegen::Make(ETarget::Native); + Codegen_->LoadBitCode(GetYtCodecBitCode(), "YtCodecFuncs"); + Func_ = Codegen_->GetModule().getFunction(funcName); + UNIT_ASSERT(Func_); + Codegen_->Verify(); + YtCodecAddMappings(*Codegen_); + Codegen_->ExportSymbol(Func_); + Codegen_->Compile(); + //Codegen_->GetModule().dump(); + } + + ICodegen::TPtr Codegen_; + llvm::Function* Func_; + TTestReader TestReader_; + TInputBuf Buf_; +}; + +Y_UNIT_TEST_SUITE(TYtCodegenCodec) { + Y_UNIT_TEST(TestWriteJust) { + TWriteSetup setup("WriteJust"); + typedef void(*TFunc)(TOutputBuf&); + auto funcPtr = (TFunc)setup.Codegen_->GetPointerToFunction(setup.Func_); + funcPtr(setup.Buf_); + setup.Buf_.Finish(); + UNIT_ASSERT_STRINGS_EQUAL(setup.TestWriter_.Str().Quote(), R"("\1")"); + } + + Y_UNIT_TEST(TestWriteNothing) { + TWriteSetup setup("WriteNothing"); + typedef void(*TFunc)(TOutputBuf&); + auto funcPtr = (TFunc)setup.Codegen_->GetPointerToFunction(setup.Func_); + funcPtr(setup.Buf_); + setup.Buf_.Finish(); + UNIT_ASSERT_STRINGS_EQUAL(setup.TestWriter_.Str().Quote(), R"("\0")"); + } + + Y_UNIT_TEST(TestWriteBool) { + TWriteSetup setup("WriteBool"); + typedef void(*TFunc)(TOutputBuf&, bool); + auto funcPtr = (TFunc)setup.Codegen_->GetPointerToFunction(setup.Func_); + funcPtr(setup.Buf_, true); + funcPtr(setup.Buf_, false); + setup.Buf_.Finish(); + UNIT_ASSERT_STRINGS_EQUAL(setup.TestWriter_.Str().Quote(), R"("\1\0")"); + } + + Y_UNIT_TEST(TestWrite8) { + TWriteSetup setup("Write8"); + typedef void(*TFunc)(TOutputBuf&, ui8); + auto funcPtr = (TFunc)setup.Codegen_->GetPointerToFunction(setup.Func_); + funcPtr(setup.Buf_, 3); + setup.Buf_.Finish(); + UNIT_ASSERT_STRINGS_EQUAL(setup.TestWriter_.Str().Quote(), R"("\3")"); + } +#ifndef _win_ + Y_UNIT_TEST(TestWrite16) { + TWriteSetup setup("Write16"); + typedef void(*TFunc)(TOutputBuf&, ui16); + auto funcPtr = (TFunc)setup.Codegen_->GetPointerToFunction(setup.Func_); + funcPtr(setup.Buf_, 258); + setup.Buf_.Finish(); + UNIT_ASSERT_STRINGS_EQUAL(setup.TestWriter_.Str().Quote(), R"("\2\1")"); + } + + Y_UNIT_TEST(TestWrite32) { + TWriteSetup setup("Write32"); + typedef void(*TFunc)(TOutputBuf&, ui32); + auto funcPtr = (TFunc)setup.Codegen_->GetPointerToFunction(setup.Func_); + funcPtr(setup.Buf_, 258); + setup.Buf_.Finish(); + UNIT_ASSERT_STRINGS_EQUAL(setup.TestWriter_.Str().Quote(), R"("\2\1\0\0")"); + } + + Y_UNIT_TEST(TestWrite64) { + TWriteSetup setup("Write64"); + typedef void(*TFunc)(TOutputBuf&, ui64); + auto funcPtr = (TFunc)setup.Codegen_->GetPointerToFunction(setup.Func_); + funcPtr(setup.Buf_, 258); + setup.Buf_.Finish(); + UNIT_ASSERT_STRINGS_EQUAL(setup.TestWriter_.Str().Quote(), R"("\2\1\0\0\0\0\0\0")"); + } +#endif + Y_UNIT_TEST(TestWriteString) { + TWriteSetup setup("WriteString"); + typedef void(*TFunc)(TOutputBuf&, const char*, ui32); + auto funcPtr = (TFunc)setup.Codegen_->GetPointerToFunction(setup.Func_); + funcPtr(setup.Buf_, "foo", 3); + setup.Buf_.Finish(); + UNIT_ASSERT_STRINGS_EQUAL(setup.TestWriter_.Str().Quote(), R"("foo")"); + } +#ifndef _win_ + Y_UNIT_TEST(TestWriteDataRowSmall) { + auto codegen = ICodegen::Make(ETarget::Native); + codegen->LoadBitCode(GetYtCodecBitCode(), "YtCodecFuncs"); + auto writer = MakeYtCodecCgWriter<false>(codegen); + TScopedAlloc alloc(__LOCATION__); + TTypeEnvironment env(alloc); + auto boolType = TDataType::Create(NUdf::TDataType<bool>::Id, env); + auto int8Type = TDataType::Create(NUdf::TDataType<i8>::Id, env); + auto uint8Type = TDataType::Create(NUdf::TDataType<ui8>::Id, env); + auto int16Type = TDataType::Create(NUdf::TDataType<i16>::Id, env); + auto uint16Type = TDataType::Create(NUdf::TDataType<ui16>::Id, env); + auto int32Type = TDataType::Create(NUdf::TDataType<i32>::Id, env); + auto uint32Type = TDataType::Create(NUdf::TDataType<ui32>::Id, env); + auto int64Type = TDataType::Create(NUdf::TDataType<i64>::Id, env); + auto uint64Type = TDataType::Create(NUdf::TDataType<ui64>::Id, env); + auto floatType = TDataType::Create(NUdf::TDataType<float>::Id, env); + auto doubleType = TDataType::Create(NUdf::TDataType<double>::Id, env); + + writer->AddField(boolType, false); + writer->AddField(int8Type, false); + writer->AddField(uint8Type, false); + writer->AddField(int16Type, false); + writer->AddField(uint16Type, false); + writer->AddField(int32Type, false); + writer->AddField(uint32Type, false); + writer->AddField(int64Type, false); + writer->AddField(uint64Type, false); + writer->AddField(floatType, false); + writer->AddField(doubleType, false); + + auto func = writer->Build(); + codegen->Verify(); + YtCodecAddMappings(*codegen); + codegen->Compile(); + + TMemoryUsageInfo memInfo("test"); + THolderFactory holderFactory(alloc.Ref(), memInfo); + NUdf::TUnboxedValue* items; + NUdf::TUnboxedValue row = holderFactory.CreateDirectArrayHolder(11, items); + items[0] = NUdf::TUnboxedValuePod(true); + items[1] = NUdf::TUnboxedValuePod(i8(-1)); + items[2] = NUdf::TUnboxedValuePod(ui8(2)); + items[3] = NUdf::TUnboxedValuePod(i16(-3)); + items[4] = NUdf::TUnboxedValuePod(ui16(4)); + items[5] = NUdf::TUnboxedValuePod(i32(-5)); + items[6] = NUdf::TUnboxedValuePod(ui32(6)); + items[7] = NUdf::TUnboxedValuePod(i64(-7)); + items[8] = NUdf::TUnboxedValuePod(ui64(8)); + items[9] = NUdf::TUnboxedValuePod(float(9)); + items[10] = NUdf::TUnboxedValuePod(double(10)); + typedef void(*TFunc)(const NUdf::TUnboxedValuePod, TOutputBuf&); + auto funcPtr = (TFunc)codegen->GetPointerToFunction(func); + + TTestWriter testWriter; + TOutputBuf buf(testWriter, nullptr); + funcPtr(row, buf); + buf.Finish(); + UNIT_ASSERT_STRINGS_EQUAL(testWriter.Str().Quote(), R"("\1\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF\2\0\0\0\0\0\0\0\xFD\xFF\xFF\xFF\xFF\xFF\xFF\xFF\4\0\0\0\0\0\0\0\xFB\xFF\xFF\xFF\xFF\xFF\xFF\xFF\6\0\0\0\0\0\0\0\xF9\xFF\xFF\xFF\xFF\xFF\xFF\xFF\x08\0\0\0\0\0\0\0\0\0\0\0\0\0\"@\0\0\0\0\0\0$@")"); + } + + Y_UNIT_TEST(TestWriteDataRowString) { + auto codegen = ICodegen::Make(ETarget::Native); + codegen->LoadBitCode(GetYtCodecBitCode(), "YtCodecFuncs"); + auto writer = MakeYtCodecCgWriter<false>(codegen); + TScopedAlloc alloc(__LOCATION__); + TTypeEnvironment env(alloc); + auto stringType = TDataType::Create(NUdf::TDataType<char*>::Id, env); + auto utf8Type = TDataType::Create(NUdf::TDataType<NUdf::TUtf8>::Id, env); + auto jsonType = TDataType::Create(NUdf::TDataType<NUdf::TJson>::Id, env); + auto ysonType = TDataType::Create(NUdf::TDataType<NUdf::TYson>::Id, env); + + writer->AddField(stringType, false); + writer->AddField(utf8Type, false); + writer->AddField(jsonType, false); + writer->AddField(ysonType, false); + + auto func = writer->Build(); + codegen->Verify(); + YtCodecAddMappings(*codegen); + codegen->Compile(); + + TMemoryUsageInfo memInfo("test"); + THolderFactory holderFactory(alloc.Ref(), memInfo); + NUdf::TUnboxedValue* items; + NUdf::TUnboxedValue row = holderFactory.CreateDirectArrayHolder(4, items); + items[0] = NUdf::TUnboxedValue(MakeString("aaa")); + items[1] = NUdf::TUnboxedValue(MakeString("VERY LOOOONG STRING")); + items[2] = NUdf::TUnboxedValue(MakeString("[1,2]")); + items[3] = NUdf::TUnboxedValue(MakeString("{foo=bar}")); + typedef void(*TFunc)(const NUdf::TUnboxedValuePod, TOutputBuf&); + auto funcPtr = (TFunc)codegen->GetPointerToFunction(func); + + TTestWriter testWriter; + TOutputBuf buf(testWriter, nullptr); + funcPtr(row, buf); + buf.Finish(); + UNIT_ASSERT_STRINGS_EQUAL(testWriter.Str().Quote(), R"("\3\0\0\0aaa\x13\0\0\0VERY LOOOONG STRING\5\0\0\0[1,2]\t\0\0\0{foo=bar}")"); + } + + Y_UNIT_TEST(TestWriteDataRowDecimal) { + auto codegen = ICodegen::Make(ETarget::Native); + codegen->LoadBitCode(GetYtCodecBitCode(), "YtCodecFuncs"); + auto writer = MakeYtCodecCgWriter<false>(codegen); + TScopedAlloc alloc(__LOCATION__); + TTypeEnvironment env(alloc); + auto longintType = TDataDecimalType::Create(10, 4, env); + + writer->AddField(longintType, false); + + auto func = writer->Build(); + codegen->Verify(); + YtCodecAddMappings(*codegen); + codegen->Compile(); + + TMemoryUsageInfo memInfo("test"); + THolderFactory holderFactory(alloc.Ref(), memInfo); + NUdf::TUnboxedValue* items; + NUdf::TUnboxedValue row = holderFactory.CreateDirectArrayHolder(1, items); + items[0] = NUdf::TUnboxedValuePod(NDecimal::TInt128(-5)); + typedef void(*TFunc)(const NUdf::TUnboxedValuePod, TOutputBuf&); + auto funcPtr = (TFunc)codegen->GetPointerToFunction(func); + + TTestWriter testWriter; + TOutputBuf buf(testWriter, nullptr); + funcPtr(row, buf); + buf.Finish(); + UNIT_ASSERT_STRINGS_EQUAL(testWriter.Str().Quote(), R"("\2\0\0\0~\xFB")"); + } + + Y_UNIT_TEST(TestWriteDataRowDecimalNan) { + auto codegen = ICodegen::Make(ETarget::Native); + codegen->LoadBitCode(GetYtCodecBitCode(), "YtCodecFuncs"); + auto writer = MakeYtCodecCgWriter<false>(codegen); + TScopedAlloc alloc(__LOCATION__); + TTypeEnvironment env(alloc); + auto longintType = TDataDecimalType::Create(10, 4, env); + + writer->AddField(longintType, false); + + auto func = writer->Build(); + codegen->Verify(); + YtCodecAddMappings(*codegen); + codegen->Compile(); + + TMemoryUsageInfo memInfo("test"); + THolderFactory holderFactory(alloc.Ref(), memInfo); + NUdf::TUnboxedValue* items; + NUdf::TUnboxedValue row = holderFactory.CreateDirectArrayHolder(1, items); + items[0] = NUdf::TUnboxedValuePod(NDecimal::Nan()); + typedef void(*TFunc)(const NUdf::TUnboxedValuePod, TOutputBuf&); + auto funcPtr = (TFunc)codegen->GetPointerToFunction(func); + + TTestWriter testWriter; + TOutputBuf buf(testWriter, nullptr); + funcPtr(row, buf); + buf.Finish(); + UNIT_ASSERT_STRINGS_EQUAL(testWriter.Str().Quote(), R"("\1\0\0\0\xFF")"); + } + + Y_UNIT_TEST(TestWriteOptionalRow) { + auto codegen = ICodegen::Make(ETarget::Native); + codegen->LoadBitCode(GetYtCodecBitCode(), "YtCodecFuncs"); + auto writer = MakeYtCodecCgWriter<false>(codegen); + TScopedAlloc alloc(__LOCATION__); + TTypeEnvironment env(alloc); + auto int32Type = TDataType::Create(NUdf::TDataType<i32>::Id, env); + auto optInt32Type = TOptionalType::Create(int32Type, env); + + writer->AddField(optInt32Type, false); + writer->AddField(optInt32Type, false); + + auto func = writer->Build(); + codegen->Verify(); + YtCodecAddMappings(*codegen); + codegen->Compile(); + + TMemoryUsageInfo memInfo("test"); + THolderFactory holderFactory(alloc.Ref(), memInfo); + NUdf::TUnboxedValue* items; + NUdf::TUnboxedValue row = holderFactory.CreateDirectArrayHolder(2, items); + items[0] = NUdf::TUnboxedValuePod(); + items[1] = NUdf::TUnboxedValuePod(i32(5)); + typedef void(*TFunc)(const NUdf::TUnboxedValuePod, TOutputBuf&); + auto funcPtr = (TFunc)codegen->GetPointerToFunction(func); + + TTestWriter testWriter; + TOutputBuf buf(testWriter, nullptr); + funcPtr(row, buf); + buf.Finish(); + UNIT_ASSERT_STRINGS_EQUAL(testWriter.Str().Quote(), R"("\0\1\5\0\0\0\0\0\0\0")"); + } + + Y_UNIT_TEST(TestWriteContainerRow) { + auto codegen = ICodegen::Make(ETarget::Native); + codegen->LoadBitCode(GetYtCodecBitCode(), "YtCodecFuncs"); + auto writer = MakeYtCodecCgWriter<false>(codegen); + TScopedAlloc alloc(__LOCATION__); + TTypeEnvironment env(alloc); + auto uint32Type = TDataType::Create(NUdf::TDataType<ui32>::Id, env); + auto optUint32Type = TOptionalType::Create(uint32Type, env); + auto list1Type = TListType::Create(optUint32Type, env); + auto list2Type = TOptionalType::Create(TListType::Create(optUint32Type, env), env); + + writer->AddField(list1Type, false); + writer->AddField(list2Type, false); + writer->AddField(list2Type, false); + + auto func = writer->Build(); + codegen->Verify(); + YtCodecAddMappings(*codegen); + codegen->Compile(); + + TMemoryUsageInfo memInfo("test"); + THolderFactory holderFactory(alloc.Ref(), memInfo); + + NUdf::TUnboxedValue* list1items; + NUdf::TUnboxedValue list1 = holderFactory.CreateDirectArrayHolder(2, list1items); + list1items[0] = NUdf::TUnboxedValuePod(ui32(2)); + list1items[1] = NUdf::TUnboxedValuePod(ui32(3)); + + NUdf::TUnboxedValue* list2items; + NUdf::TUnboxedValue list2 = holderFactory.CreateDirectArrayHolder(3, list2items); + list2items[0] = NUdf::TUnboxedValuePod(ui32(4)); + list2items[1] = NUdf::TUnboxedValuePod(ui32(5)); + list2items[2] = NUdf::TUnboxedValue(); + + NUdf::TUnboxedValue* items; + NUdf::TUnboxedValue row = holderFactory.CreateDirectArrayHolder(3, items); + items[0] = list1; + items[1] = NUdf::TUnboxedValue(); + items[2] = list2; + typedef void(*TFunc)(const NUdf::TUnboxedValuePod, TOutputBuf&); + auto funcPtr = (TFunc)codegen->GetPointerToFunction(func); + + TTestWriter testWriter; + TOutputBuf buf(testWriter, nullptr); + funcPtr(row, buf); + buf.Finish(); + UNIT_ASSERT_STRINGS_EQUAL(testWriter.Str().Quote(), R"("\x0E\0\0\0[[\6\2;];[\6\3;];]\0\1\x10\0\0\0[[\6\4;];[\6\5;];#;]")"); + } +#endif + Y_UNIT_TEST(TestReadBool) { + TReadSetup setup("ReadBool", "\1"); + typedef void(*TFunc)(TInputBuf&, void*); + auto funcPtr = (TFunc)setup.Codegen_->GetPointerToFunction(setup.Func_); + NUdf::TUnboxedValue val; + funcPtr(setup.Buf_, &val); + UNIT_ASSERT(val.Get<bool>()); + } +#ifndef _win_ + Y_UNIT_TEST(TestReadInt8) { + TReadSetup setup("ReadInt8", "\xff\xff\xff\xff\xff\xff\xff\xff"); + typedef void(*TFunc)(TInputBuf&, void*); + auto funcPtr = (TFunc)setup.Codegen_->GetPointerToFunction(setup.Func_); + NUdf::TUnboxedValue val; + funcPtr(setup.Buf_, &val); + UNIT_ASSERT_VALUES_EQUAL(val.Get<i8>(), -1); + } + + Y_UNIT_TEST(TestReadUint8) { + TReadSetup setup("ReadUint8", "\2\0\0\0\0\0\0\0"sv); + typedef void(*TFunc)(TInputBuf&, void*); + auto funcPtr = (TFunc)setup.Codegen_->GetPointerToFunction(setup.Func_); + NUdf::TUnboxedValue val; + funcPtr(setup.Buf_, &val); + UNIT_ASSERT_VALUES_EQUAL(val.Get<ui8>(), 2); + } + + Y_UNIT_TEST(TestReadInt16) { + TReadSetup setup("ReadInt16", "\xff\xff\xff\xff\xff\xff\xff\xff"); + typedef void(*TFunc)(TInputBuf&, void*); + auto funcPtr = (TFunc)setup.Codegen_->GetPointerToFunction(setup.Func_); + NUdf::TUnboxedValue val; + funcPtr(setup.Buf_, &val); + UNIT_ASSERT_VALUES_EQUAL(val.Get<i16>(), -1); + } + + Y_UNIT_TEST(TestReadUint16) { + TReadSetup setup("ReadUint16", "\2\1\0\0\0\0\0\0"sv); + typedef void(*TFunc)(TInputBuf&, void*); + auto funcPtr = (TFunc)setup.Codegen_->GetPointerToFunction(setup.Func_); + NUdf::TUnboxedValue val; + funcPtr(setup.Buf_, &val); + UNIT_ASSERT_VALUES_EQUAL(val.Get<ui16>(), 258); + } + + Y_UNIT_TEST(TestReadInt32) { + TReadSetup setup("ReadInt32", "\xff\xff\xff\xff\xff\xff\xff\xff"); + typedef void(*TFunc)(TInputBuf&, void*); + auto funcPtr = (TFunc)setup.Codegen_->GetPointerToFunction(setup.Func_); + NUdf::TUnboxedValue val; + funcPtr(setup.Buf_, &val); + UNIT_ASSERT_VALUES_EQUAL(val.Get<i32>(), -1); + } + + Y_UNIT_TEST(TestReadUint32) { + TReadSetup setup("ReadUint32", "\2\1\0\0\0\0\0\0"sv); + typedef void(*TFunc)(TInputBuf&, void*); + auto funcPtr = (TFunc)setup.Codegen_->GetPointerToFunction(setup.Func_); + NUdf::TUnboxedValue val; + funcPtr(setup.Buf_, &val); + UNIT_ASSERT_VALUES_EQUAL(val.Get<ui32>(), 258); + } + + Y_UNIT_TEST(TestReadInt64) { + TReadSetup setup("ReadInt64", "\xff\xff\xff\xff\xff\xff\xff\xff"); + typedef void(*TFunc)(TInputBuf&, void*); + auto funcPtr = (TFunc)setup.Codegen_->GetPointerToFunction(setup.Func_); + NUdf::TUnboxedValue val; + funcPtr(setup.Buf_, &val); + UNIT_ASSERT_VALUES_EQUAL(val.Get<i32>(), -1); + } + + Y_UNIT_TEST(TestReadUint64) { + TReadSetup setup("ReadUint64", "\2\1\0\0\0\0\0\0"sv); + typedef void(*TFunc)(TInputBuf&, void*); + auto funcPtr = (TFunc)setup.Codegen_->GetPointerToFunction(setup.Func_); + NUdf::TUnboxedValue val; + funcPtr(setup.Buf_, &val); + UNIT_ASSERT_VALUES_EQUAL(val.Get<ui64>(), 258); + } + + Y_UNIT_TEST(TestReadDataRowSmall) { + auto codegen = ICodegen::Make(ETarget::Native); + codegen->LoadBitCode(GetYtCodecBitCode(), "YtCodecFuncs"); + TScopedAlloc alloc(__LOCATION__); + TMemoryUsageInfo memInfo("test"); + THolderFactory holderFactory(alloc.Ref(), memInfo); + + auto reader = MakeYtCodecCgReader(codegen, holderFactory); + TTypeEnvironment env(alloc); + auto boolType = TDataType::Create(NUdf::TDataType<bool>::Id, env); + auto int8Type = TDataType::Create(NUdf::TDataType<i8>::Id, env); + auto uint8Type = TDataType::Create(NUdf::TDataType<ui8>::Id, env); + auto int16Type = TDataType::Create(NUdf::TDataType<i16>::Id, env); + auto uint16Type = TDataType::Create(NUdf::TDataType<ui16>::Id, env); + auto int32Type = TDataType::Create(NUdf::TDataType<i32>::Id, env); + auto uint32Type = TDataType::Create(NUdf::TDataType<ui32>::Id, env); + auto int64Type = TDataType::Create(NUdf::TDataType<i64>::Id, env); + auto uint64Type = TDataType::Create(NUdf::TDataType<ui64>::Id, env); + auto floatType = TDataType::Create(NUdf::TDataType<float>::Id, env); + auto doubleType = TDataType::Create(NUdf::TDataType<double>::Id, env); + + reader->AddField(boolType, {}, false); + reader->AddField(int8Type, {}, false); + reader->AddField(uint8Type, {}, false); + reader->AddField(int16Type, {}, false); + reader->AddField(uint16Type, {}, false); + reader->AddField(int32Type, {}, false); + reader->AddField(uint32Type, {}, false); + reader->AddField(int64Type, {}, false); + reader->AddField(uint64Type, {}, false); + reader->AddField(floatType, {}, false); + reader->AddField(doubleType, {}, false); + + auto func = reader->Build(); + codegen->Verify(); + YtCodecAddMappings(*codegen); + codegen->Compile(); + + NUdf::TUnboxedValue* items; + NUdf::TUnboxedValue row = holderFactory.CreateDirectArrayHolder(11, items); + typedef void(*TFunc)(NUdf::TUnboxedValue*, TInputBuf&); + auto funcPtr = (TFunc)codegen->GetPointerToFunction(func); + + TTestReader testReader("\1\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF\2\0\0\0\0\0\0\0\xFD\xFF\xFF\xFF\xFF\xFF\xFF\xFF\4\0\0\0\0\0\0\0\xFB\xFF\xFF\xFF\xFF\xFF\xFF\xFF\6\0\0\0\0\0\0\0\xF9\xFF\xFF\xFF\xFF\xFF\xFF\xFF\x08\0\0\0\0\0\0\0\0\0\0\0\0\0\"@\0\0\0\0\0\0$@"sv); + TInputBuf buf(testReader, nullptr); + funcPtr(items, buf); + UNIT_ASSERT(items[0].Get<bool>()); + UNIT_ASSERT_VALUES_EQUAL(items[1].Get<i8>(), i8(-1)); + UNIT_ASSERT_VALUES_EQUAL(items[2].Get<ui8>(), ui8(2)); + UNIT_ASSERT_VALUES_EQUAL(items[3].Get<i16>(), i16(-3)); + UNIT_ASSERT_VALUES_EQUAL(items[4].Get<ui16>(), ui16(4)); + UNIT_ASSERT_VALUES_EQUAL(items[5].Get<i32>(), i32(-5)); + UNIT_ASSERT_VALUES_EQUAL(items[6].Get<ui32>(), ui32(6)); + UNIT_ASSERT_VALUES_EQUAL(items[7].Get<i64>(), i64(-7)); + UNIT_ASSERT_VALUES_EQUAL(items[8].Get<ui64>(), ui64(8)); + UNIT_ASSERT_VALUES_EQUAL(items[9].Get<float>(), float(9)); + UNIT_ASSERT_VALUES_EQUAL(items[10].Get<double>(), double(10)); + } +#endif + Y_UNIT_TEST(TestReadDataRowString) { + auto codegen = ICodegen::Make(ETarget::Native); + codegen->LoadBitCode(GetYtCodecBitCode(), "YtCodecFuncs"); + TMemoryUsageInfo memInfo("test"); + TScopedAlloc alloc(__LOCATION__); + THolderFactory holderFactory(alloc.Ref(), memInfo); + + auto reader = MakeYtCodecCgReader(codegen, holderFactory); + TTypeEnvironment env(alloc); + auto stringType = TDataType::Create(NUdf::TDataType<char*>::Id, env); + auto utf8Type = TDataType::Create(NUdf::TDataType<NUdf::TUtf8>::Id, env); + auto jsonType = TDataType::Create(NUdf::TDataType<NUdf::TJson>::Id, env); + auto ysonType = TDataType::Create(NUdf::TDataType<NUdf::TYson>::Id, env); + + reader->AddField(stringType, {}, false); + reader->AddField(utf8Type, {}, false); + reader->AddField(jsonType, {}, false); + reader->AddField(ysonType, {}, false); + + auto func = reader->Build(); + codegen->Verify(); + YtCodecAddMappings(*codegen); + codegen->Compile(); + + NUdf::TUnboxedValue* items; + NUdf::TUnboxedValue row = holderFactory.CreateDirectArrayHolder(4, items); + typedef void(*TFunc)(NUdf::TUnboxedValue*, TInputBuf&); + auto funcPtr = (TFunc)codegen->GetPointerToFunction(func); + + TTestReader testReader("\3\0\0\0aaa\x13\0\0\0VERY LOOOONG STRING\5\0\0\0[1,2]\t\0\0\0{foo=bar}"sv); + TInputBuf buf(testReader, nullptr); + funcPtr(items, buf); + UNIT_ASSERT_STRINGS_EQUAL(TStringBuf(items[0].AsStringRef()), "aaa"); + UNIT_ASSERT_STRINGS_EQUAL(TStringBuf(items[1].AsStringRef()), "VERY LOOOONG STRING"); + UNIT_ASSERT_STRINGS_EQUAL(TStringBuf(items[2].AsStringRef()), "[1,2]"); + UNIT_ASSERT_STRINGS_EQUAL(TStringBuf(items[3].AsStringRef()), "{foo=bar}"); + } +#ifndef _win_ + Y_UNIT_TEST(TestReadDataRowDecimal) { + auto codegen = ICodegen::Make(ETarget::Native); + codegen->LoadBitCode(GetYtCodecBitCode(), "YtCodecFuncs"); + TMemoryUsageInfo memInfo("test"); + TScopedAlloc alloc(__LOCATION__); + THolderFactory holderFactory(alloc.Ref(), memInfo); + + auto reader = MakeYtCodecCgReader(codegen, holderFactory); + TTypeEnvironment env(alloc); + auto longintType = TDataDecimalType::Create(10, 4, env); + + reader->AddField(longintType, {}, false); + + auto func = reader->Build(); + codegen->Verify(); + YtCodecAddMappings(*codegen); + codegen->Compile(); + + NUdf::TUnboxedValue* items; + NUdf::TUnboxedValue row = holderFactory.CreateDirectArrayHolder(1, items); + typedef void(*TFunc)(NUdf::TUnboxedValue*, TInputBuf&); + auto funcPtr = (TFunc)codegen->GetPointerToFunction(func); + + TTestReader testReader("\2\0\0\0~\xFB"sv); + TInputBuf buf(testReader, nullptr); + funcPtr(items, buf); + UNIT_ASSERT_EQUAL(items[0].GetInt128(), -5); + } + + Y_UNIT_TEST(TestReadDataRowDecimalNan) { + auto codegen = ICodegen::Make(ETarget::Native); + codegen->LoadBitCode(GetYtCodecBitCode(), "YtCodecFuncs"); + TMemoryUsageInfo memInfo("test"); + TScopedAlloc alloc(__LOCATION__); + THolderFactory holderFactory(alloc.Ref(), memInfo); + + auto reader = MakeYtCodecCgReader(codegen, holderFactory); + TTypeEnvironment env(alloc); + auto longintType = TDataDecimalType::Create(10, 4, env); + + reader->AddField(longintType, {}, false); + + auto func = reader->Build(); + codegen->Verify(); + YtCodecAddMappings(*codegen); + codegen->Compile(); + + NUdf::TUnboxedValue* items; + NUdf::TUnboxedValue row = holderFactory.CreateDirectArrayHolder(1, items); + typedef void(*TFunc)(NUdf::TUnboxedValue*, TInputBuf&); + auto funcPtr = (TFunc)codegen->GetPointerToFunction(func); + + TTestReader testReader("\1\0\0\0\xFF"sv); + TInputBuf buf(testReader, nullptr); + funcPtr(items, buf); + UNIT_ASSERT_EQUAL(items[0].GetInt128(), NDecimal::Nan()); + } + + Y_UNIT_TEST(TestReadOptionalRow) { + auto codegen = ICodegen::Make(ETarget::Native); + codegen->LoadBitCode(GetYtCodecBitCode(), "YtCodecFuncs"); + TMemoryUsageInfo memInfo("test"); + TScopedAlloc alloc(__LOCATION__); + THolderFactory holderFactory(alloc.Ref(), memInfo); + + auto reader = MakeYtCodecCgReader(codegen, holderFactory); + TTypeEnvironment env(alloc); + auto int32Type = TDataType::Create(NUdf::TDataType<i32>::Id, env); + auto optInt32Type = TOptionalType::Create(int32Type, env); + + reader->AddField(optInt32Type, {}, false); + reader->AddField(optInt32Type, {}, false); + + auto func = reader->Build(); + codegen->Verify(); + YtCodecAddMappings(*codegen); + codegen->Compile(); + + NUdf::TUnboxedValue* items; + NUdf::TUnboxedValue row = holderFactory.CreateDirectArrayHolder(2, items); + typedef void(*TFunc)(NUdf::TUnboxedValue*, TInputBuf&); + auto funcPtr = (TFunc)codegen->GetPointerToFunction(func); + + TTestReader testReader("\0\1\5\0\0\0\0\0\0\0"sv); + TInputBuf buf(testReader, nullptr); + funcPtr(items, buf); + UNIT_ASSERT(!items[0]); + UNIT_ASSERT_VALUES_EQUAL(items[1].Get<i32>(), 5); + } +#endif + Y_UNIT_TEST(TestReadContainerRow) { + auto codegen = ICodegen::Make(ETarget::Native); + codegen->LoadBitCode(GetYtCodecBitCode(), "YtCodecFuncs"); + TMemoryUsageInfo memInfo("test"); + TScopedAlloc alloc(__LOCATION__); + THolderFactory holderFactory(alloc.Ref(), memInfo); + + auto reader = MakeYtCodecCgReader(codegen, holderFactory); + TTypeEnvironment env(alloc); + + auto uint32Type = TDataType::Create(NUdf::TDataType<ui32>::Id, env); + auto optUint32Type = TOptionalType::Create(uint32Type, env); + auto list1Type = TListType::Create(optUint32Type, env); + auto list2Type = TOptionalType::Create(TListType::Create(optUint32Type, env), env); + + reader->AddField(list1Type, {}, false); + reader->AddField(list2Type, {}, false); + reader->AddField(list2Type, {}, false); + + auto func = reader->Build(); + codegen->Verify(); + YtCodecAddMappings(*codegen); + codegen->Compile(); + + NUdf::TUnboxedValue* items; + NUdf::TUnboxedValue row = holderFactory.CreateDirectArrayHolder(3, items); + typedef void(*TFunc)(NUdf::TUnboxedValue*, TInputBuf&); + auto funcPtr = (TFunc)codegen->GetPointerToFunction(func); + + TTestReader testReader("\x0E\0\0\0[[\6\2;];[\6\3;];]\0\1\x10\0\0\0[[\6\4;];[\6\5;];#;]"sv); + TInputBuf buf(testReader, nullptr); + funcPtr(items, buf); + + UNIT_ASSERT_VALUES_EQUAL(items[0].GetListLength(), 2); + UNIT_ASSERT_VALUES_EQUAL(items[0].GetElements()[0].Get<ui32>(), 2); + UNIT_ASSERT_VALUES_EQUAL(items[0].GetElements()[1].Get<ui32>(), 3); + + UNIT_ASSERT(!items[1]); + + UNIT_ASSERT_VALUES_EQUAL(items[2].GetListLength(), 3); + UNIT_ASSERT_VALUES_EQUAL(items[2].GetElements()[0].Get<ui32>(), 4); + UNIT_ASSERT_VALUES_EQUAL(items[2].GetElements()[1].Get<ui32>(), 5); + UNIT_ASSERT(!items[2].GetElements()[2]); + } +#ifndef _win_ + Y_UNIT_TEST(TestReadDefaultRow) { + auto codegen = ICodegen::Make(ETarget::Native); + codegen->LoadBitCode(GetYtCodecBitCode(), "YtCodecFuncs"); + TMemoryUsageInfo memInfo("test"); + TScopedAlloc alloc(__LOCATION__); + THolderFactory holderFactory(alloc.Ref(), memInfo); + + auto reader = MakeYtCodecCgReader(codegen, holderFactory); + TTypeEnvironment env(alloc); + auto int32Type = TDataType::Create(NUdf::TDataType<i32>::Id, env); + + reader->AddField(int32Type, NUdf::TUnboxedValuePod(7), false); + reader->AddField(int32Type, NUdf::TUnboxedValuePod(7), false); + + auto func = reader->Build(); + codegen->Verify(); + YtCodecAddMappings(*codegen); + codegen->Compile(); + + NUdf::TUnboxedValue* items; + NUdf::TUnboxedValue row = holderFactory.CreateDirectArrayHolder(2, items); + typedef void(*TFunc)(NUdf::TUnboxedValue*, TInputBuf&); + auto funcPtr = (TFunc)codegen->GetPointerToFunction(func); + + TTestReader testReader("\0\1\5\0\0\0\0\0\0\0"sv); + TInputBuf buf(testReader, nullptr); + funcPtr(items, buf); + UNIT_ASSERT_VALUES_EQUAL(items[0].Get<i32>(), 7); + UNIT_ASSERT_VALUES_EQUAL(items[1].Get<i32>(), 5); + } +#endif + Y_UNIT_TEST(TestReadDefaultRowString) { + auto codegen = ICodegen::Make(ETarget::Native); + codegen->LoadBitCode(GetYtCodecBitCode(), "YtCodecFuncs"); + TMemoryUsageInfo memInfo("test"); + TScopedAlloc alloc(__LOCATION__); + THolderFactory holderFactory(alloc.Ref(), memInfo); + + auto reader = MakeYtCodecCgReader(codegen, holderFactory); + TTypeEnvironment env(alloc); + auto stringType = TDataType::Create(NUdf::TDataType<char*>::Id, env); + + const NUdf::TUnboxedValue defStr = MakeString(NUdf::TStringRef::Of("VERY LOOONG STRING")); + reader->AddField(stringType, defStr, false); + reader->AddField(stringType, defStr, false); + + auto func = reader->Build(); + codegen->Verify(); + YtCodecAddMappings(*codegen); + codegen->Compile(); + + NUdf::TUnboxedValue* items; + NUdf::TUnboxedValue row = holderFactory.CreateDirectArrayHolder(2, items); + typedef void(*TFunc)(NUdf::TUnboxedValue*, TInputBuf&); + auto funcPtr = (TFunc)codegen->GetPointerToFunction(func); + + TTestReader testReader("\0\1\3\0\0\0foo"sv); + TInputBuf buf(testReader, nullptr); + funcPtr(items, buf); + UNIT_ASSERT_STRINGS_EQUAL(TStringBuf(items[0].AsStringRef()), "VERY LOOONG STRING"); + UNIT_ASSERT_STRINGS_EQUAL(TStringBuf(items[1].AsStringRef()), "foo"); + } +#ifndef _win_ + Y_UNIT_TEST(TestReadTzDate) { + // full size = 2 + 2 = 4 + TStringBuf buf = "\x04\0\0\0\1\2\0\1"sv; + TReadSetup setup("ReadTzDate", buf); + typedef void(*TFunc)(TInputBuf&, void*); + auto funcPtr = (TFunc)setup.Codegen_->GetPointerToFunction(setup.Func_); + NUdf::TUnboxedValue val; + funcPtr(setup.Buf_, &val); + UNIT_ASSERT_VALUES_EQUAL(val.Get<ui16>(), 258); + UNIT_ASSERT_VALUES_EQUAL(val.GetTimezoneId(), 1); + } + + Y_UNIT_TEST(TestReadTzDatetime) { + // full size = 4 + 2 = 6 + TStringBuf buf = "\x06\0\0\0\0\0\1\3\0\1"sv; + TReadSetup setup("ReadTzDatetime", buf); + typedef void(*TFunc)(TInputBuf&, void*); + auto funcPtr = (TFunc)setup.Codegen_->GetPointerToFunction(setup.Func_); + NUdf::TUnboxedValue val; + funcPtr(setup.Buf_, &val); + UNIT_ASSERT_VALUES_EQUAL(val.Get<ui32>(), 259); + UNIT_ASSERT_VALUES_EQUAL(val.GetTimezoneId(), 1); + } + + Y_UNIT_TEST(TestReadTzTimestamp) { + // full size = 8 + 2 = 10 + TStringBuf buf = "\x0a\0\0\0\0\0\0\0\0\0\1\4\0\1"sv; + TReadSetup setup("ReadTzTimestamp", buf); + typedef void(*TFunc)(TInputBuf&, void*); + auto funcPtr = (TFunc)setup.Codegen_->GetPointerToFunction(setup.Func_); + NUdf::TUnboxedValue val; + funcPtr(setup.Buf_, &val); + UNIT_ASSERT_VALUES_EQUAL(val.Get<ui64>(), 260); + UNIT_ASSERT_VALUES_EQUAL(val.GetTimezoneId(), 1); + } + + Y_UNIT_TEST(TestReadTzDate32) { + // full size = 4 + 2 = 6 + TStringBuf buf = "\6\0\0\0\x7F\xFF\xFE\xFE\0\1"sv; + TReadSetup setup("ReadTzDate32", buf); + typedef void(*TFunc)(TInputBuf&, void*); + auto funcPtr = (TFunc)setup.Codegen_->GetPointerToFunction(setup.Func_); + NUdf::TUnboxedValue val; + funcPtr(setup.Buf_, &val); + UNIT_ASSERT_VALUES_EQUAL(val.Get<i32>(), -258); + UNIT_ASSERT_VALUES_EQUAL(val.GetTimezoneId(), 1); + } + + Y_UNIT_TEST(TestReadTzDatetime64) { + // full size = 8 + 2 = 10 + TStringBuf buf = "\n\0\0\0\x7F\xFF\xFF\xFF\xFF\xFF\xFE\xFD\0\1"sv; + TReadSetup setup("ReadTzDatetime64", buf); + typedef void(*TFunc)(TInputBuf&, void*); + auto funcPtr = (TFunc)setup.Codegen_->GetPointerToFunction(setup.Func_); + NUdf::TUnboxedValue val; + funcPtr(setup.Buf_, &val); + UNIT_ASSERT_VALUES_EQUAL(val.Get<i64>(), -259); + UNIT_ASSERT_VALUES_EQUAL(val.GetTimezoneId(), 1); + } + + Y_UNIT_TEST(TestReadTzTimestamp64) { + // full size = 8 + 2 = 10 + TStringBuf buf = "\n\0\0\0\x7F\xFF\xFF\xFF\xFF\xFF\xFE\xFC\0\1"sv; + TReadSetup setup("ReadTzTimestamp64", buf); + typedef void(*TFunc)(TInputBuf&, void*); + auto funcPtr = (TFunc)setup.Codegen_->GetPointerToFunction(setup.Func_); + NUdf::TUnboxedValue val; + funcPtr(setup.Buf_, &val); + UNIT_ASSERT_VALUES_EQUAL(val.Get<i64>(), -260); + UNIT_ASSERT_VALUES_EQUAL(val.GetTimezoneId(), 1); + } + + Y_UNIT_TEST(TestWriteTzDate) { + TWriteSetup setup("WriteTzDate"); + typedef void(*TFunc)(TOutputBuf&, ui16, ui16); + auto funcPtr = (TFunc)setup.Codegen_->GetPointerToFunction(setup.Func_); + funcPtr(setup.Buf_, 258, 1); + setup.Buf_.Finish(); + UNIT_ASSERT_STRINGS_EQUAL(setup.TestWriter_.Str().Quote(), TString("\x04\0\0\0\1\2\0\1"sv).Quote()); + } + + Y_UNIT_TEST(TestWriteTzDatetime) { + TWriteSetup setup("WriteTzDatetime"); + typedef void(*TFunc)(TOutputBuf&, ui32, ui16); + auto funcPtr = (TFunc)setup.Codegen_->GetPointerToFunction(setup.Func_); + funcPtr(setup.Buf_, 259, 1); + setup.Buf_.Finish(); + UNIT_ASSERT_STRINGS_EQUAL(setup.TestWriter_.Str().Quote(), TString("\x06\0\0\0\0\0\1\3\0\1"sv).Quote()); + } + + Y_UNIT_TEST(TestWriteTzTimestamp) { + TWriteSetup setup("WriteTzTimestamp"); + typedef void(*TFunc)(TOutputBuf&, ui64, ui16); + auto funcPtr = (TFunc)setup.Codegen_->GetPointerToFunction(setup.Func_); + funcPtr(setup.Buf_, 260, 1); + setup.Buf_.Finish(); + UNIT_ASSERT_STRINGS_EQUAL(setup.TestWriter_.Str().Quote(), TString("\x0a\0\0\0\0\0\0\0\0\0\1\4\0\1"sv).Quote()); + } + + Y_UNIT_TEST(TestWriteTzDate32) { + TWriteSetup setup("WriteTzDate32"); + typedef void(*TFunc)(TOutputBuf&, i32, ui16); + auto funcPtr = (TFunc)setup.Codegen_->GetPointerToFunction(setup.Func_); + funcPtr(setup.Buf_, -258, 1); + setup.Buf_.Finish(); + UNIT_ASSERT_STRINGS_EQUAL(setup.TestWriter_.Str().Quote(), TString("\6\0\0\0\x7F\xFF\xFE\xFE\0\1"sv).Quote()); + } + + Y_UNIT_TEST(TestWriteTzDatetime64) { + TWriteSetup setup("WriteTzDatetime64"); + typedef void(*TFunc)(TOutputBuf&, i64, ui16); + auto funcPtr = (TFunc)setup.Codegen_->GetPointerToFunction(setup.Func_); + funcPtr(setup.Buf_, -259, 1); + setup.Buf_.Finish(); + UNIT_ASSERT_STRINGS_EQUAL(setup.TestWriter_.Str().Quote(), TString("\n\0\0\0\x7F\xFF\xFF\xFF\xFF\xFF\xFE\xFD\0\1"sv).Quote()); + } + + Y_UNIT_TEST(TestWriteTzTimestamp64) { + TWriteSetup setup("WriteTzTimestamp64"); + typedef void(*TFunc)(TOutputBuf&, i64, ui16); + auto funcPtr = (TFunc)setup.Codegen_->GetPointerToFunction(setup.Func_); + funcPtr(setup.Buf_, -260, 1); + setup.Buf_.Finish(); + UNIT_ASSERT_STRINGS_EQUAL(setup.TestWriter_.Str().Quote(), TString("\n\0\0\0\x7F\xFF\xFF\xFF\xFF\xFF\xFE\xFC\0\1"sv).Quote()); + } +#endif +} + +} +#endif diff --git a/yt/yql/providers/yt/codec/codegen/ya.make b/yt/yql/providers/yt/codec/codegen/ya.make new file mode 100644 index 0000000000..15ed0bb53a --- /dev/null +++ b/yt/yql/providers/yt/codec/codegen/ya.make @@ -0,0 +1,92 @@ +LIBRARY() + +SRCS( + yt_codec_cg.cpp + yt_codec_cg.h +) + +PEERDIR( + library/cpp/resource + yql/essentials/types/binary_json + yql/essentials/minikql/computation/llvm14 + yql/essentials/parser/pg_wrapper/interface + yql/essentials/utils +) + +IF (NOT MKQL_DISABLE_CODEGEN) + PEERDIR( + yql/essentials/minikql/codegen/llvm14 + ) + USE_LLVM_BC14() + LLVM_BC( + yt_codec_bc.cpp + NAME + YtCodecFuncs + SYMBOLS + WriteJust + WriteNothing + WriteBool + Write8 + Write16 + Write32 + Write64 + Write120 + WriteDecimal32 + WriteDecimal64 + WriteDecimal128 + WriteFloat + WriteDouble + WriteString + ReadBool + ReadInt8 + ReadUint8 + ReadInt16 + ReadUint16 + ReadInt32 + ReadUint32 + ReadInt64 + ReadUint64 + ReadInt120 + ReadDecimal32 + ReadDecimal64 + ReadDecimal128 + ReadFloat + ReadDouble + ReadOptional + ReadVariantData + SkipFixedData + SkipVarData + ReadTzDate + ReadTzDatetime + ReadTzTimestamp + ReadTzDate32 + ReadTzDatetime64 + ReadTzTimestamp64 + WriteTzDate + WriteTzDatetime + WriteTzTimestamp + WriteTzDate32 + WriteTzDatetime64 + WriteTzTimestamp64 + GetWrittenBytes + FillZero + ) +ELSE() + CFLAGS( + -DMKQL_DISABLE_CODEGEN + ) +ENDIF() + +YQL_LAST_ABI_VERSION() + +PROVIDES(YT_CODEC_CODEGEN) + +END() + +RECURSE( + no_llvm +) + +RECURSE_FOR_TESTS( + ut +) diff --git a/yt/yql/providers/yt/codec/codegen/yt_codec_bc.cpp b/yt/yql/providers/yt/codec/codegen/yt_codec_bc.cpp new file mode 100644 index 0000000000..a778f2afd0 --- /dev/null +++ b/yt/yql/providers/yt/codec/codegen/yt_codec_bc.cpp @@ -0,0 +1,376 @@ +#define LLVM_BC +#include "yt_codec_cg.h" +#include <yql/essentials/utils/swap_bytes.h> +#include <yql/essentials/providers/common/codec/yql_codec_buf.h> +#include <yql/essentials/public/decimal/yql_decimal_serialize.h> +#include <yql/essentials/public/decimal/yql_decimal.h> +#include <yql/essentials/public/udf/udf_value.h> + +#include <yql/essentials/public/decimal/yql_decimal_serialize.cpp> +#include <yql/essentials/public/decimal/yql_decimal.cpp> + +using namespace NYql; + +extern "C" void WriteJust(void* vbuf) { + NCommon::TOutputBuf& buf = *(NCommon::TOutputBuf*)vbuf; + buf.Write('\1'); +} + +extern "C" void WriteNothing(void* vbuf) { + NCommon::TOutputBuf& buf = *(NCommon::TOutputBuf*)vbuf; + buf.Write('\0'); +} + +extern "C" void WriteBool(void* vbuf, bool value) { + NCommon::TOutputBuf& buf = *(NCommon::TOutputBuf*)vbuf; + buf.Write(value ? '\1' : '\0'); +} + +extern "C" void Write8(void* vbuf, ui8 value) { + NCommon::TOutputBuf& buf = *(NCommon::TOutputBuf*)vbuf; + buf.Write(value); +} + +extern "C" void Write16(void* vbuf, ui16 value) { + NCommon::TOutputBuf& buf = *(NCommon::TOutputBuf*)vbuf; + buf.WriteMany((const char*)&value, sizeof(value)); +} + +extern "C" void Write32(void* vbuf, ui32 value) { + NCommon::TOutputBuf& buf = *(NCommon::TOutputBuf*)vbuf; + buf.WriteMany((const char*)&value, sizeof(value)); +} + +extern "C" void Write64(void* vbuf, ui64 value) { + NCommon::TOutputBuf& buf = *(NCommon::TOutputBuf*)vbuf; + buf.WriteMany((const char*)&value, sizeof(value)); +} +extern "C" void Write120(void* vbuf, const void* decimal) { + auto value = reinterpret_cast<const NDecimal::TInt128*>(decimal); + char b[sizeof(*value)]; + const ui32 size = NDecimal::Serialize(*value, b); + NCommon::TOutputBuf& buf = *(NCommon::TOutputBuf*)vbuf; + buf.WriteMany(reinterpret_cast<const char*>(&size), sizeof(size)); + buf.WriteMany(b, size); +} + +extern "C" void WriteDecimal32(void* vbuf, const void* decimal) { + auto value = reinterpret_cast<const NDecimal::TInt128*>(decimal); + NCommon::TOutputBuf& buf = *(NCommon::TOutputBuf*)vbuf; + i32 data = NDecimal::ToYtDecimal<i32>(*value); + buf.WriteMany(reinterpret_cast<const char*>(&data), sizeof(data)); +} +extern "C" void WriteDecimal64(void* vbuf, const void* decimal) { + auto value = reinterpret_cast<const NDecimal::TInt128*>(decimal); + NCommon::TOutputBuf& buf = *(NCommon::TOutputBuf*)vbuf; + i64 data = NDecimal::ToYtDecimal<i64>(*value); + buf.WriteMany(reinterpret_cast<const char*>(&data), sizeof(data)); +} +extern "C" void WriteDecimal128(void* vbuf, const void* decimal) { + auto value = reinterpret_cast<const NDecimal::TInt128*>(decimal); + NCommon::TOutputBuf& buf = *(NCommon::TOutputBuf*)vbuf; + NDecimal::TInt128 data = NDecimal::ToYtDecimal<NDecimal::TInt128>(*value); + buf.WriteMany(reinterpret_cast<const char*>(&data), sizeof(data)); +} +extern "C" void WriteFloat(void* vbuf, ui32 value) { + NCommon::TOutputBuf& buf = *(NCommon::TOutputBuf*)vbuf; + double data = (double)*(const float*)&value; + buf.WriteMany((const char*)&data, sizeof(data)); +} + +extern "C" void WriteDouble(void* vbuf, ui64 value) { + NCommon::TOutputBuf& buf = *(NCommon::TOutputBuf*)vbuf; + buf.WriteMany((const char*)&value, sizeof(value)); +} + +extern "C" void WriteString(void* vbuf, const char* buffer, ui32 len) { + NCommon::TOutputBuf& buf = *(NCommon::TOutputBuf*)vbuf; + buf.WriteMany(buffer, len); +} + +extern "C" void ReadBool(void* vbuf, void* vpod) { + NCommon::TInputBuf& buf = *(NCommon::TInputBuf*)vbuf; + char cmd = buf.Read(); + new (vpod) NUdf::TUnboxedValuePod(cmd != 0); +} + +extern "C" void ReadInt8(void* vbuf, void* vpod) { + NCommon::TInputBuf& buf = *(NCommon::TInputBuf*)vbuf; + i64 data; + buf.ReadMany((char*)&data, sizeof(data)); + new (vpod) NUdf::TUnboxedValuePod(i8(data)); +} + +extern "C" void ReadUint8(void* vbuf, void* vpod) { + NCommon::TInputBuf& buf = *(NCommon::TInputBuf*)vbuf; + ui64 data; + buf.ReadMany((char*)&data, sizeof(data)); + new (vpod) NUdf::TUnboxedValuePod(ui8(data)); +} + +extern "C" void ReadInt16(void* vbuf, void* vpod) { + NCommon::TInputBuf& buf = *(NCommon::TInputBuf*)vbuf; + i64 data; + buf.ReadMany((char*)&data, sizeof(data)); + new (vpod) NUdf::TUnboxedValuePod(i16(data)); +} + +extern "C" void ReadUint16(void* vbuf, void* vpod) { + NCommon::TInputBuf& buf = *(NCommon::TInputBuf*)vbuf; + ui64 data; + buf.ReadMany((char*)&data, sizeof(data)); + new (vpod) NUdf::TUnboxedValuePod(ui16(data)); +} + +extern "C" void ReadInt32(void* vbuf, void* vpod) { + NCommon::TInputBuf& buf = *(NCommon::TInputBuf*)vbuf; + i64 data; + buf.ReadMany((char*)&data, sizeof(data)); + new (vpod) NUdf::TUnboxedValuePod(i32(data)); +} + +extern "C" void ReadUint32(void* vbuf, void* vpod) { + NCommon::TInputBuf& buf = *(NCommon::TInputBuf*)vbuf; + ui64 data; + buf.ReadMany((char*)&data, sizeof(data)); + new (vpod) NUdf::TUnboxedValuePod(ui32(data)); +} + +extern "C" void ReadInt64(void* vbuf, void* vpod) { + NCommon::TInputBuf& buf = *(NCommon::TInputBuf*)vbuf; + i64 data; + buf.ReadMany((char*)&data, sizeof(data)); + new (vpod) NUdf::TUnboxedValuePod(data); +} + +extern "C" void ReadUint64(void* vbuf, void* vpod) { + NCommon::TInputBuf& buf = *(NCommon::TInputBuf*)vbuf; + ui64 data; + buf.ReadMany((char*)&data, sizeof(data)); + new (vpod) NUdf::TUnboxedValuePod(data); +} +extern "C" void ReadInt120(void* vbuf, void* vpod) { + NCommon::TInputBuf& buf = *(NCommon::TInputBuf*)vbuf; + ui32 size; + buf.ReadMany(reinterpret_cast<char*>(&size), sizeof(size)); + + if (size <= sizeof(NDecimal::TInt128)) { + char data[sizeof(NDecimal::TInt128)]; + buf.ReadMany(data, size); + auto v = NDecimal::Deserialize(data, size).first; + if (v == NDecimal::Err()) { + ThrowBadDecimal(); + } else { + new (vpod) NUdf::TUnboxedValuePod(v); + } + } else { + ThrowBadDecimal(); + } +} + +extern "C" void ReadDecimal32(void* vbuf, void* vpod) { + NCommon::TInputBuf& buf = *(NCommon::TInputBuf*)vbuf; + i32 data; + buf.ReadMany((char*)&data, sizeof(data)); + new (vpod) NUdf::TUnboxedValuePod(NDecimal::FromYtDecimal(data)); +} +extern "C" void ReadDecimal64(void* vbuf, void* vpod) { + NCommon::TInputBuf& buf = *(NCommon::TInputBuf*)vbuf; + i64 data; + buf.ReadMany((char*)&data, sizeof(data)); + new (vpod) NUdf::TUnboxedValuePod(NDecimal::FromYtDecimal(data)); +} +extern "C" void ReadDecimal128(void* vbuf, void* vpod) { + NCommon::TInputBuf& buf = *(NCommon::TInputBuf*)vbuf; + NDecimal::TInt128 data; + buf.ReadMany((char*)&data, sizeof(data)); + new (vpod) NUdf::TUnboxedValuePod(NDecimal::FromYtDecimal(data)); +} +extern "C" void ReadFloat(void* vbuf, void* vpod) { + NCommon::TInputBuf& buf = *(NCommon::TInputBuf*)vbuf; + double data; + buf.ReadMany((char*)&data, sizeof(data)); + new (vpod) NUdf::TUnboxedValuePod(float(data)); +} + +extern "C" void ReadDouble(void* vbuf, void* vpod) { + NCommon::TInputBuf& buf = *(NCommon::TInputBuf*)vbuf; + double data; + buf.ReadMany((char*)&data, sizeof(data)); + new (vpod) NUdf::TUnboxedValuePod(data); +} + +extern "C" ui8 ReadOptional(void* vbuf) { + NCommon::TInputBuf& buf = *(NCommon::TInputBuf*)vbuf; + return buf.Read(); +} + +extern "C" ui16 ReadVariantData(void* vbuf, ui8 oneByte) { + NCommon::TInputBuf& buf = *(NCommon::TInputBuf*)vbuf; + if (oneByte) { + return buf.Read(); + } else { + ui16 data = 0; + buf.ReadMany((char*)&data, sizeof(data)); + return data; + } +} + +extern "C" void SkipFixedData(void* vbuf, ui64 size) { + NCommon::TInputBuf& buf = *(NCommon::TInputBuf*)vbuf; + buf.SkipMany(size); +} + +extern "C" void SkipVarData(void* vbuf) { + NCommon::TInputBuf& buf = *(NCommon::TInputBuf*)vbuf; + ui32 size; + buf.ReadMany((char*)&size, sizeof(size)); + buf.SkipMany(size); +} + +extern "C" void ReadTzDate(void* vbuf, void* vpod) { + NCommon::TInputBuf& buf = *(NCommon::TInputBuf*)vbuf; + ui32 size; + buf.ReadMany((char*)&size, sizeof(size)); + ui16 data; + buf.ReadMany((char*)&data, sizeof(data)); + ui16 tzId; + buf.ReadMany((char*)&tzId, sizeof(tzId)); + data = SwapBytes(data); + tzId = SwapBytes(tzId); + (new (vpod) NUdf::TUnboxedValuePod(data))->SetTimezoneId(tzId); +} + +extern "C" void ReadTzDatetime(void* vbuf, void* vpod) { + NCommon::TInputBuf& buf = *(NCommon::TInputBuf*)vbuf; + ui32 size; + buf.ReadMany((char*)&size, sizeof(size)); + ui32 data; + buf.ReadMany((char*)&data, sizeof(data)); + ui16 tzId; + buf.ReadMany((char*)&tzId, sizeof(tzId)); + data = SwapBytes(data); + tzId = SwapBytes(tzId); + (new (vpod) NUdf::TUnboxedValuePod(data))->SetTimezoneId(tzId); +} + +extern "C" void ReadTzTimestamp(void* vbuf, void* vpod) { + NCommon::TInputBuf& buf = *(NCommon::TInputBuf*)vbuf; + ui32 size; + buf.ReadMany((char*)&size, sizeof(size)); + ui64 data; + buf.ReadMany((char*)&data, sizeof(data)); + ui16 tzId; + buf.ReadMany((char*)&tzId, sizeof(tzId)); + data = SwapBytes(data); + tzId = SwapBytes(tzId); + (new (vpod) NUdf::TUnboxedValuePod(data))->SetTimezoneId(tzId); +} + +extern "C" void ReadTzDate32(void* vbuf, void* vpod) { + NCommon::TInputBuf& buf = *(NCommon::TInputBuf*)vbuf; + ui32 size; + buf.ReadMany((char*)&size, sizeof(size)); + ui32 data; + buf.ReadMany((char*)&data, sizeof(data)); + ui16 tzId; + buf.ReadMany((char*)&tzId, sizeof(tzId)); + i32 value = SwapBytes(0x80 ^ data); + tzId = SwapBytes(tzId); + (new (vpod) NUdf::TUnboxedValuePod(value))->SetTimezoneId(tzId); +} + +extern "C" void ReadTzDatetime64(void* vbuf, void* vpod) { + NCommon::TInputBuf& buf = *(NCommon::TInputBuf*)vbuf; + ui32 size; + buf.ReadMany((char*)&size, sizeof(size)); + ui64 data; + buf.ReadMany((char*)&data, sizeof(data)); + ui16 tzId; + buf.ReadMany((char*)&tzId, sizeof(tzId)); + i64 value = SwapBytes(0x80 ^ data); + tzId = SwapBytes(tzId); + (new (vpod) NUdf::TUnboxedValuePod(value))->SetTimezoneId(tzId); +} + +extern "C" void ReadTzTimestamp64(void* vbuf, void* vpod) { + NCommon::TInputBuf& buf = *(NCommon::TInputBuf*)vbuf; + ui32 size; + buf.ReadMany((char*)&size, sizeof(size)); + ui64 data; + buf.ReadMany((char*)&data, sizeof(data)); + ui16 tzId; + buf.ReadMany((char*)&tzId, sizeof(tzId)); + i64 value = SwapBytes(0x80 ^ data); + tzId = SwapBytes(tzId); + (new (vpod) NUdf::TUnboxedValuePod(value))->SetTimezoneId(tzId); +} + +extern "C" void WriteTzDate(void* vbuf, ui16 value, ui16 tzId) { + value = SwapBytes(value); + tzId = SwapBytes(tzId); + NCommon::TOutputBuf& buf = *(NCommon::TOutputBuf*)vbuf; + const ui32 size = sizeof(value) + sizeof(tzId); + buf.WriteMany((const char*)&size, sizeof(size)); + buf.WriteMany((const char*)&value, sizeof(value)); + buf.WriteMany((const char*)&tzId, sizeof(tzId)); +} + +extern "C" void WriteTzDatetime(void* vbuf, ui32 value, ui16 tzId) { + value = SwapBytes(value); + tzId = SwapBytes(tzId); + NCommon::TOutputBuf& buf = *(NCommon::TOutputBuf*)vbuf; + const ui32 size = sizeof(value) + sizeof(tzId); + buf.WriteMany((const char*)&size, sizeof(size)); + buf.WriteMany((const char*)&value, sizeof(value)); + buf.WriteMany((const char*)&tzId, sizeof(tzId)); +} + +extern "C" void WriteTzTimestamp(void* vbuf, ui64 value, ui16 tzId) { + value = SwapBytes(value); + tzId = SwapBytes(tzId); + NCommon::TOutputBuf& buf = *(NCommon::TOutputBuf*)vbuf; + const ui32 size = sizeof(value) + sizeof(tzId); + buf.WriteMany((const char*)&size, sizeof(size)); + buf.WriteMany((const char*)&value, sizeof(value)); + buf.WriteMany((const char*)&tzId, sizeof(tzId)); +} + +extern "C" void WriteTzDate32(void* vbuf, i32 value, ui16 tzId) { + ui32 data = 0x80 ^ SwapBytes((ui32)value); + tzId = SwapBytes(tzId); + NCommon::TOutputBuf& buf = *(NCommon::TOutputBuf*)vbuf; + const ui32 size = sizeof(data) + sizeof(tzId); + buf.WriteMany((const char*)&size, sizeof(size)); + buf.WriteMany((const char*)&data, sizeof(data)); + buf.WriteMany((const char*)&tzId, sizeof(tzId)); +} + +extern "C" void WriteTzDatetime64(void* vbuf, i64 value, ui16 tzId) { + ui64 data = 0x80 ^ SwapBytes((ui64)value); + tzId = SwapBytes(tzId); + NCommon::TOutputBuf& buf = *(NCommon::TOutputBuf*)vbuf; + const ui32 size = sizeof(data) + sizeof(tzId); + buf.WriteMany((const char*)&size, sizeof(size)); + buf.WriteMany((const char*)&data, sizeof(data)); + buf.WriteMany((const char*)&tzId, sizeof(tzId)); +} + +extern "C" void WriteTzTimestamp64(void* vbuf, i64 value, ui16 tzId) { + ui64 data = 0x80 ^ SwapBytes((ui64)value); + tzId = SwapBytes(tzId); + NCommon::TOutputBuf& buf = *(NCommon::TOutputBuf*)vbuf; + const ui32 size = sizeof(data) + sizeof(tzId); + buf.WriteMany((const char*)&size, sizeof(size)); + buf.WriteMany((const char*)&data, sizeof(data)); + buf.WriteMany((const char*)&tzId, sizeof(tzId)); +} + +extern "C" ui64 GetWrittenBytes(void* vbuf) { + NCommon::TOutputBuf& buf = *(NCommon::TOutputBuf*)vbuf; + return buf.GetWrittenBytes(); +} + +extern "C" void FillZero(void* vpod) { + new (vpod) NUdf::TUnboxedValuePod(NUdf::TUnboxedValuePod::Zero()); +} diff --git a/yt/yql/providers/yt/codec/codegen/yt_codec_cg.cpp b/yt/yql/providers/yt/codec/codegen/yt_codec_cg.cpp new file mode 100644 index 0000000000..cdf9588f94 --- /dev/null +++ b/yt/yql/providers/yt/codec/codegen/yt_codec_cg.cpp @@ -0,0 +1,1115 @@ +#include "yt_codec_cg.h" + +#include <yql/essentials/providers/common/codec/yql_codec.h> +#include <yql/essentials/parser/pg_wrapper/interface/codec.h> +#include <yql/essentials/providers/common/codec/yql_codec_buf.h> +#include <yql/essentials/providers/common/codec/yql_codec_type_flags.h> + +#ifndef MKQL_DISABLE_CODEGEN +#include <yql/essentials/minikql/mkql_node.h> +#include <yql/essentials/minikql/mkql_node_builder.h> +#include <yql/essentials/minikql/mkql_string_util.h> +#include <yql/essentials/minikql/mkql_type_ops.h> +#include <yql/essentials/minikql/computation/mkql_computation_node_codegen.h> +#include <yql/essentials/minikql/codegen/codegen.h> + +#include <yql/essentials/types/binary_json/read.h> +#include <yql/essentials/types/binary_json/write.h> + +#include <library/cpp/resource/resource.h> + +#include <llvm/IR/Module.h> +#include <llvm/IR/Instructions.h> + +#endif + +namespace NYql { + +#ifndef MKQL_DISABLE_CODEGEN + +using namespace llvm; +using namespace NKikimr; +using namespace NKikimr::NMiniKQL; + +extern "C" void ThrowBadDecimal() { + throw yexception() << "Invalid decimal data"; +} + +extern "C" void YtCodecReadString(void* vbuf, void* vpod) { + NYql::NCommon::TInputBuf& buf = *(NYql::NCommon::TInputBuf*)vbuf; + NUdf::TUnboxedValue& pod = *(NUdf::TUnboxedValue*)vpod; + ui32 size; + buf.ReadMany((char*)&size, sizeof(size)); + CHECK_STRING_LENGTH_UNSIGNED(size); + auto str = NUdf::TUnboxedValue(MakeStringNotFilled(size)); + buf.ReadMany(str.AsStringRef().Data(), size); + pod = std::move(str); +} + +extern "C" void YtCodecWriteJsonDocument(void* vbuf, const char* buffer, ui32 len) { + NCommon::TOutputBuf& buf = *(NCommon::TOutputBuf*)vbuf; + TStringBuf binaryJson(buffer, len); + const TString json = NBinaryJson::SerializeToJson(binaryJson); + const ui32 size = json.size(); + buf.WriteMany((const char*)&size, sizeof(size)); + buf.WriteMany(json.data(), size); +} + +extern "C" void YtCodecReadJsonDocument(void* vbuf, void* vpod) { + NYql::NCommon::TInputBuf& buf = *(NYql::NCommon::TInputBuf*)vbuf; + NUdf::TUnboxedValue& pod = *(NUdf::TUnboxedValue*)vpod; + + ui32 size; + buf.ReadMany((char*)&size, sizeof(size)); + CHECK_STRING_LENGTH_UNSIGNED(size); + + auto json = NUdf::TUnboxedValue(MakeStringNotFilled(size)); + buf.ReadMany(json.AsStringRef().Data(), size); + + const auto binaryJson = NBinaryJson::SerializeToBinaryJson(json.AsStringRef()); + if (std::holds_alternative<TString>(binaryJson)) { + YQL_ENSURE(false, "Invalid JSON stored for JsonDocument type"); + } + + const auto& value = std::get<NBinaryJson::TBinaryJson>(binaryJson); + TStringBuf binaryJsonRef(value.Data(), value.Size()); + pod = MakeString(binaryJsonRef); +} + +template<bool Flat> +class TYtCodecCgWriter : public IYtCodecCgWriter { +public: + TYtCodecCgWriter(const std::unique_ptr<NCodegen::ICodegen>& codegen, const void* cookie) + : Codegen_(codegen) + { + auto& module = Codegen_->GetModule(); + auto& context = Codegen_->GetContext(); + // input - pointer to struct UnboxedValue as int128 and instance of buffer, output - void + const auto funcType = Flat || Codegen_->GetEffectiveTarget() == NYql::NCodegen::ETarget::Windows ? + FunctionType::get(Type::getVoidTy(context), {PointerType::getUnqual(Type::getInt128Ty(context)), PointerType::getUnqual(Type::getInt8Ty(context))}, false): + FunctionType::get(Type::getVoidTy(context), {Type::getInt128Ty(context), PointerType::getUnqual(Type::getInt8Ty(context))}, false); + Func_ = cast<Function>(module.getOrInsertFunction((TStringBuilder() << (Flat ? "YtCodecCgWriterFlat." : "YtCodecCgWriter.") << cookie).data(), funcType).getCallee()); + Block_ = BasicBlock::Create(context, "EntryBlock", Func_); + } + + void AddField(NKikimr::NMiniKQL::TType* type, ui64 nativeYtTypeFlags) override { + auto& module = Codegen_->GetModule(); + auto& context = Codegen_->GetContext(); + auto args = Func_->arg_begin(); + const auto valueType = Type::getInt128Ty(context); + const auto valueArg = &*args++; + const auto buf = &*args++; + const auto value = Flat || !valueArg->getType()->isPointerTy() ? + static_cast<Value*>(valueArg) : new LoadInst(valueType, valueArg, "row", Block_); + const auto index = ConstantInt::get(Type::getInt32Ty(context), Index_++); + + const auto elemPtr = Flat ? + GetElementPtrInst::CreateInBounds(valueType, value, { index }, "elemPtr", Block_): + static_cast<Value*>(new AllocaInst(valueType, 0U, "elemPtr", Block_)); + + if constexpr (!Flat) { + CallBoxedValueVirtualMethod<NUdf::TBoxedValueAccessor::EMethod::GetElement>(elemPtr, value, *Codegen_, Block_, index); + } + + bool isOptional; + auto unwrappedType = UnpackOptional(type, isOptional); + if (!isOptional) { + GenerateRequired(elemPtr, buf, type, nativeYtTypeFlags, false); + } else { + const auto just = BasicBlock::Create(context, "just", Func_); + const auto nothing = BasicBlock::Create(context, "nothing", Func_); + const auto done = BasicBlock::Create(context, "done", Func_); + + const auto zero = ConstantInt::get(valueType, 0ULL); + const auto elem = new LoadInst(valueType, elemPtr, "elem", Block_); + const auto check = CmpInst::Create(Instruction::ICmp, ICmpInst::ICMP_NE, elem, zero, "exists", Block_); + BranchInst::Create(just, nothing, check, Block_); + + { + Block_ = just; + CallInst::Create(module.getFunction("WriteJust"), { buf }, "", Block_); + if (unwrappedType->IsOptional() || unwrappedType->IsPg()) { + const auto unwrappedElem = GetOptionalValue(context, elem, Block_); + const auto unwrappedElemPtr = new AllocaInst(valueType, 0U, "unwrapped", Block_); + new StoreInst(unwrappedElem, unwrappedElemPtr, Block_); + GenerateRequired(unwrappedElemPtr, buf, unwrappedType, nativeYtTypeFlags, true); + } else { + GenerateRequired(elemPtr, buf, unwrappedType, nativeYtTypeFlags, true); + } + + BranchInst::Create(done, Block_); + } + + { + Block_ = nothing; + CallInst::Create(module.getFunction("WriteNothing"), { buf }, "", Block_); + BranchInst::Create(done, Block_); + } + + Block_ = done; + } + } + + Function* Build() override { + ReturnInst::Create(Codegen_->GetContext(), Block_); + Codegen_->ExportSymbol(Func_); + return Func_; + } + + void GenerateRequired(Value* elemPtr, Value* buf, NKikimr::NMiniKQL::TType* type, ui64 nativeYtTypeFlags, bool wasOptional) { + if (type->IsData()) { + return GenerateData(elemPtr, buf, type, nativeYtTypeFlags); + } + + if (!wasOptional && type->IsPg()) { + return GeneratePg(elemPtr, buf, static_cast<NKikimr::NMiniKQL::TPgType*>(type)); + } + + // call external writer + auto& context = Codegen_->GetContext(); + const auto typeConst = ConstantInt::get(Type::getInt64Ty(context), (ui64)type); + const auto valType = Type::getInt128Ty(context); + const auto flagsConst = ConstantInt::get(Type::getInt64Ty(context), nativeYtTypeFlags); + if (nativeYtTypeFlags) { + const auto funcAddr = ConstantInt::get(Type::getInt64Ty(context), (ui64)&NYql::NCommon::WriteContainerNativeYtValue); + const auto funType = FunctionType::get(Type::getVoidTy(context), { + Type::getInt64Ty(context), Type::getInt64Ty(context), PointerType::getUnqual(valType), + PointerType::getUnqual(Type::getInt8Ty(context)) + }, false); + + const auto funcPtr = CastInst::Create(Instruction::IntToPtr, funcAddr, PointerType::getUnqual(funType), "ptr", Block_); + CallInst::Create(funType, funcPtr, { typeConst, flagsConst, elemPtr, buf }, "", Block_); + } else { + const auto funcAddr = ConstantInt::get(Type::getInt64Ty(context), (ui64)&NYql::NCommon::WriteYsonContainerValue); + const auto funType = FunctionType::get(Type::getVoidTy(context), { + Type::getInt64Ty(context), Type::getInt64Ty(context), PointerType::getUnqual(valType), + PointerType::getUnqual(Type::getInt8Ty(context)) + }, false); + + const auto funcPtr = CastInst::Create(Instruction::IntToPtr, funcAddr, PointerType::getUnqual(funType), "ptr", Block_); + CallInst::Create(funType, funcPtr, { typeConst, flagsConst, elemPtr, buf }, "", Block_); + } + if constexpr (!Flat) { + TCodegenContext ctx(*Codegen_); + ctx.Func = Func_; + ValueUnRef(GetValueRepresentation(type), elemPtr, ctx, Block_); + } + } + + void GenerateData(Value* elemPtr, Value* buf, NKikimr::NMiniKQL::TType* type, ui64 nativeYtTypeFlags) { + auto& module = Codegen_->GetModule(); + auto& context = Codegen_->GetContext(); + const auto elem = new LoadInst(Type::getInt128Ty(context), elemPtr, "elem", Block_); + const auto schemeType = static_cast<TDataType*>(type)->GetSchemeType(); + switch (schemeType) { + case NUdf::TDataType<bool>::Id: { + const auto data = CastInst::Create(Instruction::Trunc, elem, Type::getInt1Ty(context), "data", Block_); + CallInst::Create(module.getFunction("WriteBool"), { buf, data }, "", Block_); + break; + } + + case NUdf::TDataType<i8>::Id: { + const auto data = CastInst::Create(Instruction::Trunc, elem, Type::getInt8Ty(context), "data", Block_); + const auto ext = CastInst::Create(Instruction::SExt, data, Type::getInt64Ty(context), "ext", Block_); + CallInst::Create(module.getFunction("Write64"), { buf, ext }, "", Block_); + break; + } + + case NUdf::TDataType<ui8>::Id: { + const auto data = CastInst::Create(Instruction::Trunc, elem, Type::getInt8Ty(context), "data", Block_); + const auto ext = CastInst::Create(Instruction::ZExt, data, Type::getInt64Ty(context), "ext", Block_); + CallInst::Create(module.getFunction("Write64"), { buf, ext }, "", Block_); + break; + } + + case NUdf::TDataType<i16>::Id: { + const auto data = CastInst::Create(Instruction::Trunc, elem, Type::getInt16Ty(context), "data", Block_); + const auto ext = CastInst::Create(Instruction::SExt, data, Type::getInt64Ty(context), "ext", Block_); + CallInst::Create(module.getFunction("Write64"), { buf, ext }, "", Block_); + break; + } + + case NUdf::TDataType<NUdf::TDate>::Id: + case NUdf::TDataType<ui16>::Id: { + const auto data = CastInst::Create(Instruction::Trunc, elem, Type::getInt16Ty(context), "data", Block_); + const auto ext = CastInst::Create(Instruction::ZExt, data, Type::getInt64Ty(context), "ext", Block_); + CallInst::Create(module.getFunction("Write64"), { buf, ext }, "", Block_); + break; + } + + case NUdf::TDataType<NUdf::TDate32>::Id: + case NUdf::TDataType<i32>::Id: { + const auto data = CastInst::Create(Instruction::Trunc, elem, Type::getInt32Ty(context), "data", Block_); + const auto ext = CastInst::Create(Instruction::SExt, data, Type::getInt64Ty(context), "ext", Block_); + CallInst::Create(module.getFunction("Write64"), { buf, ext }, "", Block_); + break; + } + + case NUdf::TDataType<NUdf::TDatetime>::Id: + case NUdf::TDataType<ui32>::Id: { + const auto data = CastInst::Create(Instruction::Trunc, elem, Type::getInt32Ty(context), "data", Block_); + const auto ext = CastInst::Create(Instruction::ZExt, data, Type::getInt64Ty(context), "ext", Block_); + CallInst::Create(module.getFunction("Write64"), { buf, ext }, "", Block_); + break; + } + + case NUdf::TDataType<float>::Id: { + const auto data = CastInst::Create(Instruction::Trunc, elem, Type::getInt32Ty(context), "data", Block_); + CallInst::Create(module.getFunction("WriteFloat"), { buf, data }, "", Block_); + break; + } + + case NUdf::TDataType<double>::Id: { + const auto data = CastInst::Create(Instruction::Trunc, elem, Type::getInt64Ty(context), "data", Block_); + CallInst::Create(module.getFunction("WriteDouble"), { buf, data }, "", Block_); + break; + } + + case NUdf::TDataType<NUdf::TTimestamp>::Id: + case NUdf::TDataType<NUdf::TInterval>::Id: + case NUdf::TDataType<NUdf::TInterval64>::Id: + case NUdf::TDataType<NUdf::TDatetime64>::Id: + case NUdf::TDataType<NUdf::TTimestamp64>::Id: + case NUdf::TDataType<ui64>::Id: + case NUdf::TDataType<i64>::Id: { + const auto data = CastInst::Create(Instruction::Trunc, elem, Type::getInt64Ty(context), "data", Block_); + CallInst::Create(module.getFunction("Write64"), { buf, data }, "", Block_); + break; + } + + case NUdf::TDataType<NUdf::TDecimal>::Id: { + const auto ptr = new AllocaInst(Type::getInt128Ty(context), 0U, "ptr", Block_); + new StoreInst(GetterForInt128(elem, Block_), ptr, Block_); + const auto velemPtr = CastInst::Create(Instruction::BitCast, ptr, PointerType::getUnqual(Type::getInt8Ty(context)), "cast", Block_); + if (nativeYtTypeFlags & NTCF_DECIMAL) { + auto const params = static_cast<TDataDecimalType*>(type)->GetParams(); + if (params.first < 10) { + CallInst::Create(module.getFunction("WriteDecimal32"), { buf, velemPtr }, "", Block_); + } else if (params.first < 19) { + CallInst::Create(module.getFunction("WriteDecimal64"), { buf, velemPtr }, "", Block_); + } else { + CallInst::Create(module.getFunction("WriteDecimal128"), { buf, velemPtr }, "", Block_); + } + } else { + CallInst::Create(module.getFunction("Write120"), { buf, velemPtr }, "", Block_); + } + break; + } + + case NUdf::TDataType<char*>::Id: + case NUdf::TDataType<NUdf::TUtf8>::Id: + case NUdf::TDataType<NUdf::TJson>::Id: + case NUdf::TDataType<NUdf::TYson>::Id: + case NUdf::TDataType<NUdf::TUuid>::Id: + case NUdf::TDataType<NUdf::TDyNumber>::Id: + case NUdf::TDataType<NUdf::TJsonDocument>::Id: { + const auto type = Type::getInt8Ty(context); + const auto embType = FixedVectorType::get(type, 16); + const auto cast = CastInst::Create(Instruction::BitCast, elem, embType, "cast", Block_); + const auto mark = ExtractElementInst::Create(cast, ConstantInt::get(type, 15), "mark", Block_); + + const auto bsize = ExtractElementInst::Create(cast, ConstantInt::get(type, 14), "bsize", Block_); + const auto esize = CastInst::Create(Instruction::ZExt, bsize, Type::getInt32Ty(context), "esize", Block_); + + const auto sizeType = Type::getInt32Ty(context); + const auto strType = FixedVectorType::get(sizeType, 4); + const auto four = CastInst::Create(Instruction::BitCast, elem, strType, "four", Block_); + const auto ssize = ExtractElementInst::Create(four, ConstantInt::get(type, 2), "ssize", Block_); + + const auto cemb = CastInst::Create(Instruction::Trunc, mark, Type::getInt1Ty(context), "cemb", Block_); + const auto size = SelectInst::Create(cemb, esize, ssize, "size", Block_); + + const auto emb = BasicBlock::Create(context, "emb", Func_); + const auto str = BasicBlock::Create(context, "str", Func_); + const auto done = BasicBlock::Create(context, "done", Func_); + + const auto scond = CmpInst::Create(Instruction::ICmp, ICmpInst::ICMP_EQ, mark, ConstantInt::get(type, 2), "scond", Block_); + BranchInst::Create(str, emb, scond, Block_); + + { + Block_ = emb; + + const auto bytePtr = CastInst::Create(Instruction::BitCast, elemPtr, PointerType::getUnqual(Type::getInt8Ty(context)), "cast", Block_); + if (schemeType == NUdf::TDataType<NUdf::TJsonDocument>::Id) { + const auto fnType = FunctionType::get(Type::getVoidTy(context), { + PointerType::getUnqual(Type::getInt8Ty(context)), + PointerType::getUnqual(Type::getInt8Ty(context)), + sizeType, + }, false); + const auto func = module.getOrInsertFunction("YtCodecWriteJsonDocument", fnType); + CallInst::Create(func, { buf, bytePtr, size }, "", Block_); + } else { + CallInst::Create(module.getFunction("Write32"), { buf, size }, "", Block_); + CallInst::Create(module.getFunction("WriteString"), { buf, bytePtr, size }, "", Block_); + } + BranchInst::Create(done, Block_); + } + + { + Block_ = str; + + const auto foffs = ExtractElementInst::Create(four, ConstantInt::get(type, 3), "foffs", Block_); + const auto offs = BinaryOperator::CreateAnd(foffs, ConstantInt::get(foffs->getType(), 0xFFFFFF), "offs", Block_); + const auto skip = BinaryOperator::CreateAdd(offs, ConstantInt::get(offs->getType(), 16), "skip", Block_); + + const auto half = CastInst::Create(Instruction::Trunc, elem, Type::getInt64Ty(context), "half", Block_); + const auto ptr = CastInst::Create(Instruction::IntToPtr, half, PointerType::getUnqual(type), "ptr", Block_); + + const auto bytePtr = GetElementPtrInst::CreateInBounds(Type::getInt8Ty(context), ptr, { skip }, "bptr", Block_); + + if (schemeType == NUdf::TDataType<NUdf::TJsonDocument>::Id) { + const auto fnType = FunctionType::get(Type::getVoidTy(context), { + PointerType::getUnqual(Type::getInt8Ty(context)), + PointerType::getUnqual(Type::getInt8Ty(context)), + sizeType, + }, false); + const auto func = module.getOrInsertFunction("YtCodecWriteJsonDocument", fnType); + CallInst::Create(func, { buf, bytePtr, size }, "", Block_); + } else { + CallInst::Create(module.getFunction("Write32"), { buf, size }, "", Block_); + CallInst::Create(module.getFunction("WriteString"), { buf, bytePtr, size }, "", Block_); + } + + if constexpr (!Flat) { + TCodegenContext ctx(*Codegen_); + ctx.Func = Func_; + ValueUnRef(EValueRepresentation::String, elemPtr, ctx, Block_); + } + + BranchInst::Create(done, Block_); + } + + Block_ = done; + break; + } + + case NUdf::TDataType<NUdf::TTzDate>::Id: { + const auto data = CastInst::Create(Instruction::Trunc, elem, Type::getInt16Ty(context), "data", Block_); + const auto sizeType = Type::getInt16Ty(context); + const auto strType = FixedVectorType::get(sizeType, 8); + const auto eight = CastInst::Create(Instruction::BitCast, elem, strType, "eight", Block_); + const auto type = Type::getInt8Ty(context); + const auto tzId = ExtractElementInst::Create(eight, ConstantInt::get(type, 4), "id", Block_); + CallInst::Create(module.getFunction("WriteTzDate"), { buf, data, tzId }, "", Block_); + break; + } + + case NUdf::TDataType<NUdf::TTzDatetime>::Id: { + const auto data = CastInst::Create(Instruction::Trunc, elem, Type::getInt32Ty(context), "data", Block_); + const auto sizeType = Type::getInt16Ty(context); + const auto strType = FixedVectorType::get(sizeType, 8); + const auto eight = CastInst::Create(Instruction::BitCast, elem, strType, "eight", Block_); + const auto type = Type::getInt8Ty(context); + const auto tzId = ExtractElementInst::Create(eight, ConstantInt::get(type, 4), "id", Block_); + CallInst::Create(module.getFunction("WriteTzDatetime"), { buf, data, tzId }, "", Block_); + break; + } + + case NUdf::TDataType<NUdf::TTzTimestamp>::Id: { + const auto data = CastInst::Create(Instruction::Trunc, elem, Type::getInt64Ty(context), "data", Block_); + const auto sizeType = Type::getInt16Ty(context); + const auto strType = FixedVectorType::get(sizeType, 8); + const auto eight = CastInst::Create(Instruction::BitCast, elem, strType, "eight", Block_); + const auto type = Type::getInt8Ty(context); + const auto tzId = ExtractElementInst::Create(eight, ConstantInt::get(type, 4), "id", Block_); + CallInst::Create(module.getFunction("WriteTzTimestamp"), { buf, data, tzId }, "", Block_); + break; + } + + case NUdf::TDataType<NUdf::TTzDate32>::Id: { + const auto data = CastInst::Create(Instruction::Trunc, elem, Type::getInt32Ty(context), "data", Block_); + const auto sizeType = Type::getInt16Ty(context); + const auto strType = FixedVectorType::get(sizeType, 8); + const auto eight = CastInst::Create(Instruction::BitCast, elem, strType, "eight", Block_); + const auto type = Type::getInt8Ty(context); + const auto tzId = ExtractElementInst::Create(eight, ConstantInt::get(type, 4), "id", Block_); + CallInst::Create(module.getFunction("WriteTzDate32"), { buf, data, tzId }, "", Block_); + break; + } + + case NUdf::TDataType<NUdf::TTzDatetime64>::Id: { + const auto data = CastInst::Create(Instruction::Trunc, elem, Type::getInt64Ty(context), "data", Block_); + const auto sizeType = Type::getInt16Ty(context); + const auto strType = FixedVectorType::get(sizeType, 8); + const auto eight = CastInst::Create(Instruction::BitCast, elem, strType, "eight", Block_); + const auto type = Type::getInt8Ty(context); + const auto tzId = ExtractElementInst::Create(eight, ConstantInt::get(type, 4), "id", Block_); + CallInst::Create(module.getFunction("WriteTzDatetime64"), { buf, data, tzId }, "", Block_); + break; + } + + case NUdf::TDataType<NUdf::TTzTimestamp64>::Id: { + const auto data = CastInst::Create(Instruction::Trunc, elem, Type::getInt64Ty(context), "data", Block_); + const auto sizeType = Type::getInt16Ty(context); + const auto strType = FixedVectorType::get(sizeType, 8); + const auto eight = CastInst::Create(Instruction::BitCast, elem, strType, "eight", Block_); + const auto type = Type::getInt8Ty(context); + const auto tzId = ExtractElementInst::Create(eight, ConstantInt::get(type, 4), "id", Block_); + CallInst::Create(module.getFunction("WriteTzTimestamp64"), { buf, data, tzId }, "", Block_); + break; + } + + default: + YQL_ENSURE(false, "Unsupported data type: " << schemeType); + } + } + + void GeneratePg(Value* elemPtr, Value* buf, NKikimr::NMiniKQL::TPgType* type) { + auto& context = Codegen_->GetContext(); + const auto typeConst = ConstantInt::get(Type::getInt64Ty(context), (ui64)type); + const auto valType = Type::getInt128Ty(context); + const auto funcAddr = ConstantInt::get(Type::getInt64Ty(context), (ui64)&NYql::NCommon::WriteSkiffPgValue); + const auto funType = FunctionType::get(Type::getVoidTy(context), { + Type::getInt64Ty(context), PointerType::getUnqual(valType), + PointerType::getUnqual(Type::getInt8Ty(context)) + }, false); + + const auto funcPtr = CastInst::Create(Instruction::IntToPtr, funcAddr, PointerType::getUnqual(funType), "ptr", Block_); + CallInst::Create(funType, funcPtr, { typeConst, elemPtr, buf }, "", Block_); + if constexpr (!Flat) { + TCodegenContext ctx(*Codegen_); + ctx.Func = Func_; + ValueUnRef(EValueRepresentation::Any, elemPtr, ctx, Block_); + } + } + +private: + const std::unique_ptr<NCodegen::ICodegen>& Codegen_; + Function* Func_; + BasicBlock* Block_; + ui32 Index_ = 0; +}; + +class TYtCodecCgReader : public IYtCodecCgReader { +public: + TYtCodecCgReader(const std::unique_ptr<NCodegen::ICodegen>& codegen, + const NKikimr::NMiniKQL::THolderFactory& holderFactory, + const void* cookie) + : Codegen_(codegen) + , HolderFactory_(holderFactory) + { + auto& module = Codegen_->GetModule(); + auto& context = Codegen_->GetContext(); + // input - pointer to array of UnboxedValue as int128 and instance of buffer, output - void + const auto funcType = FunctionType::get(Type::getVoidTy(context), {PointerType::getUnqual(Type::getInt128Ty(context)), PointerType::getUnqual(Type::getInt8Ty(context))}, false); + Func_ = cast<Function>(module.getOrInsertFunction((TStringBuilder() << "YtCodecCgReader." << cookie).data(), funcType).getCallee()); + Block_ = BasicBlock::Create(context, "EntryBlock", Func_); + } + + void AddField(NKikimr::NMiniKQL::TType* type, const NKikimr::NUdf::TUnboxedValuePod& defValue, ui64 nativeYtTypeFlags) override { + auto& module = Codegen_->GetModule(); + auto& context = Codegen_->GetContext(); + auto args = Func_->arg_begin(); + const auto valuesPtr = &*args++; + const auto buf = &*args++; + const auto valueType = Type::getInt128Ty(context); + const auto indexVal = ConstantInt::get(Type::getInt32Ty(context), Index_); + const auto elemPtr = GetElementPtrInst::CreateInBounds(valueType, valuesPtr, { indexVal }, "elemPtr", Block_); + const auto velemPtr = CastInst::Create(Instruction::BitCast, elemPtr, PointerType::getUnqual(Type::getInt8Ty(context)), "cast", Block_); + + bool isOptional; + auto unwrappedType = UnpackOptional(type, isOptional); + if (isOptional || defValue) { + const auto just = BasicBlock::Create(context, "just", Func_); + const auto nothing = BasicBlock::Create(context, "nothing", Func_); + const auto done = BasicBlock::Create(context, "done", Func_); + + const auto optMarker = CallInst::Create(module.getFunction("ReadOptional"), { buf }, "optMarker", Block_); + const auto zero = ConstantInt::get(Type::getInt8Ty(context), 0); + const auto check = CmpInst::Create(Instruction::ICmp, ICmpInst::ICMP_NE, optMarker, zero, "exists", Block_); + BranchInst::Create(just, nothing, check, Block_); + + { + Block_ = just; + if (unwrappedType->IsData()) { + GenerateData(velemPtr, buf, static_cast<TDataType*>(unwrappedType), nativeYtTypeFlags); + } else { + GenerateContainer(velemPtr, buf, unwrappedType, true, nativeYtTypeFlags); + } + + BranchInst::Create(done, Block_); + } + + { + Block_ = nothing; + BranchInst::Create(done, Block_); + } + + Block_ = done; + } else { + if (unwrappedType->IsData()) { + GenerateData(velemPtr, buf, static_cast<TDataType*>(unwrappedType), nativeYtTypeFlags); + } else if (unwrappedType->IsPg()) { + GeneratePg(velemPtr, buf, static_cast<TPgType*>(unwrappedType)); + } else { + GenerateContainer(velemPtr, buf, unwrappedType, false, nativeYtTypeFlags); + } + } + + if (defValue) { + // load value from elemPtr and use default if that is empty + const auto empty = BasicBlock::Create(context, "empty", Func_); + const auto ok = BasicBlock::Create(context, "ok", Func_); + const auto done = BasicBlock::Create(context, "done", Func_); + + auto value = new LoadInst(valueType, elemPtr, "elem", Block_); + const auto zero = ConstantInt::get(valueType, 0ULL); + const auto check = CmpInst::Create(Instruction::ICmp, ICmpInst::ICMP_EQ, value, zero, "empty", Block_); + BranchInst::Create(empty, ok, check, Block_); + + { + Block_ = empty; + // copy def value and ref it + ArrayRef<uint64_t> bits((uint64_t*)&defValue, 2); + APInt defInt(128, bits); + const auto defValData = ConstantInt::get(valueType, defInt); + new StoreInst(defValData, elemPtr, Block_); + TCodegenContext ctx(*Codegen_); + ctx.Func = Func_; + ValueAddRef(EValueRepresentation::Any, elemPtr, ctx, Block_); + BranchInst::Create(done, Block_); + } + + { + Block_ = ok; + BranchInst::Create(done, Block_); + } + + Block_ = done; + } + + ++Index_; + } + + void SkipField(NKikimr::NMiniKQL::TType* type, ui64 nativeYtTypeFlags) override { + auto& module = Codegen_->GetModule(); + auto& context = Codegen_->GetContext(); + auto args = Func_->arg_begin(); + const auto valuesPtr = &*args++; + Y_UNUSED(valuesPtr); + const auto buf = &*args++; + + bool isOptional; + auto unwrappedType = UnpackOptional(type, isOptional); + if (isOptional) { + const auto just = BasicBlock::Create(context, "just", Func_); + const auto nothing = BasicBlock::Create(context, "nothing", Func_); + const auto done = BasicBlock::Create(context, "done", Func_); + + const auto optMarker = CallInst::Create(module.getFunction("ReadOptional"), { buf }, "optMarker", Block_); + const auto zero = ConstantInt::get(Type::getInt8Ty(context), 0); + const auto check = CmpInst::Create(Instruction::ICmp, ICmpInst::ICMP_NE, optMarker, zero, "exists", Block_); + BranchInst::Create(just, nothing, check, Block_); + + { + Block_ = just; + GenerateSkip(buf, unwrappedType, nativeYtTypeFlags); + BranchInst::Create(done, Block_); + } + + { + Block_ = nothing; + BranchInst::Create(done, Block_); + } + + Block_ = done; + } else { + GenerateSkip(buf, unwrappedType, nativeYtTypeFlags); + } + + ++Index_; + } + + void SkipOther() override { + ++Index_; + } + + void SkipVirtual() override { + auto& module = Codegen_->GetModule(); + auto& context = Codegen_->GetContext(); + auto args = Func_->arg_begin(); + const auto valuesPtr = &*args++; + const auto indexVal = ConstantInt::get(Type::getInt32Ty(context), Index_); + const auto elemPtr = GetElementPtrInst::CreateInBounds(Type::getInt128Ty(context), valuesPtr, { indexVal }, "elemPtr", Block_); + const auto velemPtr = CastInst::Create(Instruction::BitCast, elemPtr, PointerType::getUnqual(Type::getInt8Ty(context)), "cast", Block_); + + CallInst::Create(module.getFunction("FillZero"), { velemPtr }, "", Block_); + + ++Index_; + } + + llvm::Function* Build() override { + ReturnInst::Create(Codegen_->GetContext(), Block_); + Codegen_->ExportSymbol(Func_); + return Func_; + } + +private: + void GenerateData(Value* velemPtr, Value* buf, TDataType* dataType, ui64 nativeYtTypeFlags) { + auto& module = Codegen_->GetModule(); + auto& context = Codegen_->GetContext(); + const auto schemeType = dataType->GetSchemeType(); + switch (schemeType) { + case NUdf::TDataType<bool>::Id: { + CallInst::Create(module.getFunction("ReadBool"), { buf, velemPtr }, "", Block_); + break; + } + case NUdf::TDataType<i8>::Id: { + CallInst::Create(module.getFunction("ReadInt8"), { buf, velemPtr }, "", Block_); + break; + } + case NUdf::TDataType<ui8>::Id: { + CallInst::Create(module.getFunction("ReadUint8"), { buf, velemPtr }, "", Block_); + break; + } + case NUdf::TDataType<i16>::Id: { + CallInst::Create(module.getFunction("ReadInt16"), { buf, velemPtr }, "", Block_); + break; + } + case NUdf::TDataType<NUdf::TDate>::Id: + case NUdf::TDataType<ui16>::Id: { + CallInst::Create(module.getFunction("ReadUint16"), { buf, velemPtr }, "", Block_); + break; + } + case NUdf::TDataType<NUdf::TDate32>::Id: + case NUdf::TDataType<i32>::Id: { + CallInst::Create(module.getFunction("ReadInt32"), { buf, velemPtr }, "", Block_); + break; + } + case NUdf::TDataType<NUdf::TDatetime>::Id: + case NUdf::TDataType<ui32>::Id: { + CallInst::Create(module.getFunction("ReadUint32"), { buf, velemPtr }, "", Block_); + break; + } + case NUdf::TDataType<NUdf::TInterval>::Id: + case NUdf::TDataType<NUdf::TInterval64>::Id: + case NUdf::TDataType<NUdf::TDatetime64>::Id: + case NUdf::TDataType<NUdf::TTimestamp64>::Id: + case NUdf::TDataType<i64>::Id: { + CallInst::Create(module.getFunction("ReadInt64"), { buf, velemPtr }, "", Block_); + break; + } + case NUdf::TDataType<NUdf::TTimestamp>::Id: + case NUdf::TDataType<ui64>::Id: { + CallInst::Create(module.getFunction("ReadUint64"), { buf, velemPtr }, "", Block_); + break; + } + case NUdf::TDataType<float>::Id: { + CallInst::Create(module.getFunction("ReadFloat"), { buf, velemPtr }, "", Block_); + break; + } + case NUdf::TDataType<double>::Id: { + CallInst::Create(module.getFunction("ReadDouble"), { buf, velemPtr }, "", Block_); + break; + } + case NUdf::TDataType<NUdf::TDecimal>::Id: { + if (nativeYtTypeFlags & NTCF_DECIMAL) { + auto const params = static_cast<TDataDecimalType*>(dataType)->GetParams(); + if (params.first < 10) { + CallInst::Create(module.getFunction("ReadDecimal32"), { buf, velemPtr }, "", Block_); + } else if (params.first < 19) { + CallInst::Create(module.getFunction("ReadDecimal64"), { buf, velemPtr }, "", Block_); + } else { + CallInst::Create(module.getFunction("ReadDecimal128"), { buf, velemPtr }, "", Block_); + } + } else { + CallInst::Create(module.getFunction("ReadInt120"), { buf, velemPtr }, "", Block_); + } + break; + } + + case NUdf::TDataType<char*>::Id: + case NUdf::TDataType<NUdf::TUtf8>::Id: + case NUdf::TDataType<NUdf::TJson>::Id: + case NUdf::TDataType<NUdf::TYson>::Id: + case NUdf::TDataType<NUdf::TDyNumber>::Id: + case NUdf::TDataType<NUdf::TUuid>::Id: + case NUdf::TDataType<NUdf::TJsonDocument>::Id: { + const auto fnType = FunctionType::get(Type::getVoidTy(context), { + PointerType::getUnqual(Type::getInt8Ty(context)), + PointerType::getUnqual(Type::getInt8Ty(context)) + }, false); + + const char* funcName = "YtCodecReadString"; + if (schemeType == NUdf::TDataType<NUdf::TJsonDocument>::Id) { + funcName = "YtCodecReadJsonDocument"; + } + + const auto func = module.getOrInsertFunction(funcName, fnType); + CallInst::Create(func, { buf, velemPtr }, "", Block_); + break; + } + + case NUdf::TDataType<NUdf::TTzDate>::Id: { + CallInst::Create(module.getFunction("ReadTzDate"), { buf, velemPtr }, "", Block_); + break; + } + + case NUdf::TDataType<NUdf::TTzDatetime>::Id: { + CallInst::Create(module.getFunction("ReadTzDatetime"), { buf, velemPtr }, "", Block_); + break; + } + + case NUdf::TDataType<NUdf::TTzTimestamp>::Id: { + CallInst::Create(module.getFunction("ReadTzTimestamp"), { buf, velemPtr }, "", Block_); + break; + } + + case NUdf::TDataType<NUdf::TTzDate32>::Id: { + CallInst::Create(module.getFunction("ReadTzDate32"), { buf, velemPtr }, "", Block_); + break; + } + + case NUdf::TDataType<NUdf::TTzDatetime64>::Id: { + CallInst::Create(module.getFunction("ReadTzDatetime64"), { buf, velemPtr }, "", Block_); + break; + } + + case NUdf::TDataType<NUdf::TTzTimestamp64>::Id: { + CallInst::Create(module.getFunction("ReadTzTimestamp64"), { buf, velemPtr }, "", Block_); + break; + } + + default: + YQL_ENSURE(false, "Unknown data type: " << schemeType); + } + } + + void GeneratePg(Value* velemPtr, Value* buf, TPgType* type) { + auto& context = Codegen_->GetContext(); + const auto funcAddr = ConstantInt::get(Type::getInt64Ty(context), (ui64)&NCommon::ReadSkiffPgValue); + const auto typeConst = ConstantInt::get(Type::getInt64Ty(context), (ui64)type); + + const auto funType = FunctionType::get(Type::getVoidTy(context), { + Type::getInt64Ty(context), PointerType::getUnqual(Type::getInt8Ty(context)), + PointerType::getUnqual(Type::getInt8Ty(context)) + }, false); + + const auto funcPtr = CastInst::Create(Instruction::IntToPtr, funcAddr, PointerType::getUnqual(funType), "ptr", Block_); + CallInst::Create(funType, funcPtr, { typeConst, velemPtr, buf}, "", Block_); + } + + void GenerateContainer(Value* velemPtr, Value* buf, TType* type, bool wrapOptional, ui64 nativeYtTypeFlags) { + auto& context = Codegen_->GetContext(); + + const auto funcAddr = ConstantInt::get(Type::getInt64Ty(context), nativeYtTypeFlags ? (ui64)&NCommon::ReadContainerNativeYtValue : (ui64)&NCommon::ReadYsonContainerValue); + const auto typeConst = ConstantInt::get(Type::getInt64Ty(context), (ui64)type); + const auto holderFactoryConst = ConstantInt::get(Type::getInt64Ty(context), (ui64)&HolderFactory_); + const auto wrapConst = ConstantInt::get(Type::getInt1Ty(context), wrapOptional); + const auto flagsConst = ConstantInt::get(Type::getInt64Ty(context), nativeYtTypeFlags); + if (nativeYtTypeFlags) { + const auto funType = FunctionType::get(Type::getVoidTy(context), { + Type::getInt64Ty(context), Type::getInt64Ty(context), Type::getInt64Ty(context), PointerType::getUnqual(Type::getInt8Ty(context)), + PointerType::getUnqual(Type::getInt8Ty(context)), Type::getInt1Ty(context) + }, false); + + const auto funcPtr = CastInst::Create(Instruction::IntToPtr, funcAddr, PointerType::getUnqual(funType), "ptr", Block_); + CallInst::Create(funType, funcPtr, { typeConst, flagsConst, holderFactoryConst, velemPtr, buf, wrapConst }, "", Block_); + } else { + const auto funType = FunctionType::get(Type::getVoidTy(context), { + Type::getInt64Ty(context), Type::getInt64Ty(context), Type::getInt64Ty(context), PointerType::getUnqual(Type::getInt8Ty(context)), + PointerType::getUnqual(Type::getInt8Ty(context)), Type::getInt1Ty(context) + }, false); + + const auto funcPtr = CastInst::Create(Instruction::IntToPtr, funcAddr, PointerType::getUnqual(funType), "ptr", Block_); + CallInst::Create(funType, funcPtr, { typeConst, flagsConst, holderFactoryConst, velemPtr, buf, wrapConst }, "", Block_); + } + } + + void GenerateSkip(Value* buf, TType* type, ui64 nativeYtTypeFlags) { + auto& module = Codegen_->GetModule(); + auto& context = Codegen_->GetContext(); + + if (type->IsData()) { + auto schemeType = static_cast<TDataType*>(type)->GetSchemeType(); + switch (schemeType) { + case NUdf::TDataType<bool>::Id: { + const auto sizeConst = ConstantInt::get(Type::getInt64Ty(context), (ui64)sizeof(ui8)); + CallInst::Create(module.getFunction("SkipFixedData"), { buf, sizeConst }, "", Block_); + break; + } + + case NUdf::TDataType<ui8>::Id: + case NUdf::TDataType<ui16>::Id: + case NUdf::TDataType<ui32>::Id: + case NUdf::TDataType<ui64>::Id: + case NUdf::TDataType<NUdf::TDate>::Id: + case NUdf::TDataType<NUdf::TDatetime>::Id: + case NUdf::TDataType<NUdf::TTimestamp>::Id: { + const auto sizeConst = ConstantInt::get(Type::getInt64Ty(context), (ui64)sizeof(ui64)); + CallInst::Create(module.getFunction("SkipFixedData"), { buf, sizeConst }, "", Block_); + break; + } + case NUdf::TDataType<i8>::Id: + case NUdf::TDataType<i16>::Id: + case NUdf::TDataType<i32>::Id: + case NUdf::TDataType<i64>::Id: + case NUdf::TDataType<NUdf::TDate32>::Id: + case NUdf::TDataType<NUdf::TDatetime64>::Id: + case NUdf::TDataType<NUdf::TTimestamp64>::Id: + case NUdf::TDataType<NUdf::TInterval64>::Id: + case NUdf::TDataType<NUdf::TInterval>::Id: { + const auto sizeConst = ConstantInt::get(Type::getInt64Ty(context), (ui64)sizeof(i64)); + CallInst::Create(module.getFunction("SkipFixedData"), { buf, sizeConst }, "", Block_); + break; + } + case NUdf::TDataType<float>::Id: + case NUdf::TDataType<double>::Id: { + const auto sizeConst = ConstantInt::get(Type::getInt64Ty(context), (ui64)sizeof(double)); + CallInst::Create(module.getFunction("SkipFixedData"), { buf, sizeConst }, "", Block_); + break; + } + case NUdf::TDataType<NUdf::TUtf8>::Id: + case NUdf::TDataType<char*>::Id: + case NUdf::TDataType<NUdf::TJson>::Id: + case NUdf::TDataType<NUdf::TYson>::Id: + case NUdf::TDataType<NUdf::TUuid>::Id: + case NUdf::TDataType<NUdf::TJsonDocument>::Id: + case NUdf::TDataType<NUdf::TTzDate>::Id: + case NUdf::TDataType<NUdf::TTzDatetime>::Id: + case NUdf::TDataType<NUdf::TTzTimestamp>::Id: + case NUdf::TDataType<NUdf::TTzDate32>::Id: + case NUdf::TDataType<NUdf::TTzDatetime64>::Id: + case NUdf::TDataType<NUdf::TTzTimestamp64>::Id: { + CallInst::Create(module.getFunction("SkipVarData"), { buf }, "", Block_); + break; + } + case NUdf::TDataType<NUdf::TDecimal>::Id: { + if (nativeYtTypeFlags & NTCF_DECIMAL) { + auto const params = static_cast<TDataDecimalType*>(type)->GetParams(); + if (params.first < 10) { + const auto sizeConst = ConstantInt::get(Type::getInt64Ty(context), (ui64)sizeof(i32)); + CallInst::Create(module.getFunction("SkipFixedData"), { buf, sizeConst }, "", Block_); + } else if (params.first < 19) { + const auto sizeConst = ConstantInt::get(Type::getInt64Ty(context), (ui64)sizeof(i64)); + CallInst::Create(module.getFunction("SkipFixedData"), { buf, sizeConst }, "", Block_); + } else { + const auto sizeConst = ConstantInt::get(Type::getInt64Ty(context), (ui64)sizeof(NDecimal::TInt128)); + CallInst::Create(module.getFunction("SkipFixedData"), { buf, sizeConst }, "", Block_); + } + } else { + CallInst::Create(module.getFunction("SkipVarData"), { buf }, "", Block_); + } + break; + } + + default: + YQL_ENSURE(false, "Unknown data type: " << schemeType); + } + return; + } + + if (type->IsStruct()) { + auto structType = static_cast<TStructType*>(type); + const std::vector<size_t>* reorder = nullptr; + if (auto cookie = structType->GetCookie()) { + reorder = ((const std::vector<size_t>*)cookie); + } + for (ui32 i = 0; i < structType->GetMembersCount(); ++i) { + GenerateSkip(buf, structType->GetMemberType(reorder ? reorder->at(i) : i), nativeYtTypeFlags); + } + return; + } + + if (type->IsTuple()) { + auto tupleType = static_cast<TTupleType*>(type); + for (ui32 i = 0; i < tupleType->GetElementsCount(); ++i) { + GenerateSkip(buf, tupleType->GetElementType(i), nativeYtTypeFlags); + } + return; + } + + if (type->IsList()) { + auto itemType = static_cast<TListType*>(type)->GetItemType(); + const auto done = BasicBlock::Create(context, "done", Func_); + const auto listEndMarker = ConstantInt::get(Type::getInt8Ty(context), 0xFF); + const auto innerSkip = BasicBlock::Create(context, "innerSkip", Func_); + const auto listContinue = BasicBlock::Create(context, "listContinue", Func_); + BranchInst::Create(listContinue, Block_); + + { + Block_ = innerSkip; + GenerateSkip(buf, itemType, nativeYtTypeFlags); + BranchInst::Create(listContinue, Block_); + } + { + Block_ = listContinue; + const auto marker = CallInst::Create(module.getFunction("ReadOptional"), { buf }, "optMarker", Block_); + const auto check = CmpInst::Create(Instruction::ICmp, ICmpInst::ICMP_EQ, marker, listEndMarker, "exists", Block_); + BranchInst::Create(done, innerSkip, check, Block_); + } + + Block_ = done; + return; + } + + if (type->IsVariant()) { + auto varType = static_cast<TVariantType*>(type); + const auto isOneByte = ConstantInt::get(Type::getInt8Ty(context), varType->GetAlternativesCount() < 256); + const auto data = CallInst::Create(module.getFunction("ReadVariantData"), { buf, isOneByte }, "data", Block_); + + std::function<TType*(size_t)> getType; + std::function<void(size_t, size_t)> genLR = [&] (size_t l, size_t r){ + size_t m = (l + r) >> 1; + if (l == r) { + GenerateSkip(buf, getType(m), nativeYtTypeFlags); + return; + } + auto fn = std::to_string(l) + "_" + std::to_string(r); + const auto currIdx = ConstantInt::get(Type::getInt16Ty(context), m); + const auto isCurrent = CmpInst::Create(Instruction::ICmp, ICmpInst::ICMP_ULE, data, currIdx, "isUs" + fn, Block_); + auto lessEq = BasicBlock::Create(context, "le" + fn, Func_); + auto greater = BasicBlock::Create(context, "g" + fn, Func_); + auto out = BasicBlock::Create(context, "o" + fn, Func_); + BranchInst::Create(lessEq, greater, isCurrent, Block_); + { + Block_ = lessEq; + genLR(l, m); + BranchInst::Create(out, Block_); + } + { + Block_ = greater; + genLR(m + 1, r); + BranchInst::Create(out, Block_); + } + Block_ = out; + }; + + size_t elemCount = 0; + if (varType->GetUnderlyingType()->IsTuple()) { + auto tupleType = static_cast<TTupleType*>(varType->GetUnderlyingType()); + elemCount = tupleType->GetElementsCount(); + getType = [tupleType=tupleType] (size_t i) { + return tupleType->GetElementType(i); + }; + } else { + auto structType = static_cast<TStructType*>(varType->GetUnderlyingType()); + + const std::vector<size_t>* reorder = nullptr; + if (auto cookie = structType->GetCookie()) { + reorder = ((const std::vector<size_t>*)cookie); + } + + elemCount = structType->GetMembersCount(); + + getType = [reorder = reorder, structType=structType] (size_t i) { + return structType->GetMemberType(reorder ? reorder->at(i) : i); + }; + } + genLR(0, elemCount - 1); + return; + } + + if (type->IsVoid()) { + return; + } + + if (type->IsNull()) { + return; + } + + if (type->IsEmptyList() || type->IsEmptyDict()) { + return; + } + + if (type->IsDict()) { + auto dictType = static_cast<TDictType*>(type); + auto keyType = dictType->GetKeyType(); + auto payloadType = dictType->GetPayloadType(); + const auto done = BasicBlock::Create(context, "done", Func_); + + const auto innerSkip = BasicBlock::Create(context, "innerSkip", Func_); + + const auto listContinue = BasicBlock::Create(context, "listContinue", Func_); + + const auto listEndMarker = ConstantInt::get(Type::getInt8Ty(context), 0xFF); + + BranchInst::Create(listContinue, Block_); + + { + Block_ = innerSkip; + GenerateSkip(buf, keyType, nativeYtTypeFlags); + GenerateSkip(buf, payloadType, nativeYtTypeFlags); + BranchInst::Create(listContinue, Block_); + } + { + Block_ = listContinue; + const auto marker = CallInst::Create(module.getFunction("ReadOptional"), { buf }, "optMarker", Block_); + const auto check = CmpInst::Create(Instruction::ICmp, ICmpInst::ICMP_EQ, marker, listEndMarker, "exists", Block_); + BranchInst::Create(done, innerSkip, check, Block_); + } + + Block_ = done; + return; + } + + YQL_ENSURE(false, "Unsupported type for skip: " << type->GetKindAsStr()); + } + +private: + const std::unique_ptr<NCodegen::ICodegen>& Codegen_; + const NKikimr::NMiniKQL::THolderFactory& HolderFactory_; + Function* Func_; + BasicBlock* Block_; + ui32 Index_ = 0; +}; + +TString GetYtCodecBitCode() { + auto bitcode = NResource::Find("/llvm_bc/YtCodecFuncs"); + return bitcode; +} + +void YtCodecAddMappings(NCodegen::ICodegen& codegen) { + codegen.AddGlobalMapping("OutputBufFlushThunk", (const void*)&NCommon::OutputBufFlushThunk); + codegen.AddGlobalMapping("OutputBufWriteManySlowThunk", (const void*)&NCommon::OutputBufWriteManySlowThunk); + codegen.AddGlobalMapping("InputBufReadSlowThunk", (const void*)&NCommon::InputBufReadSlowThunk); + codegen.AddGlobalMapping("InputBufReadManySlowThunk", (const void*)&NCommon::InputBufReadManySlowThunk); + codegen.AddGlobalMapping("InputBufSkipManySlowThunk", (const void*)&NCommon::InputBufSkipManySlowThunk); + codegen.AddGlobalMapping("YtCodecReadString", (const void*)&YtCodecReadString); + codegen.AddGlobalMapping("YtCodecWriteJsonDocument", (const void*)&YtCodecWriteJsonDocument); + codegen.AddGlobalMapping("YtCodecReadJsonDocument", (const void*)&YtCodecReadJsonDocument); + codegen.AddGlobalMapping("ThrowBadDecimal", (const void*)&ThrowBadDecimal); +} + +template<bool Flat> +THolder<IYtCodecCgWriter> MakeYtCodecCgWriter(const std::unique_ptr<NCodegen::ICodegen>& codegen, const void* cookie) { + return MakeHolder<TYtCodecCgWriter<Flat>>(codegen, cookie); +} + +template THolder<IYtCodecCgWriter> MakeYtCodecCgWriter<true>(const std::unique_ptr<NCodegen::ICodegen>& codegen, const void* cookie); +template THolder<IYtCodecCgWriter> MakeYtCodecCgWriter<false>(const std::unique_ptr<NCodegen::ICodegen>& codegen, const void* cookie); + +THolder<IYtCodecCgReader> MakeYtCodecCgReader(const std::unique_ptr<NCodegen::ICodegen>& codegen, + const NKikimr::NMiniKQL::THolderFactory& holderFactory, const void* cookie) { + return MakeHolder<TYtCodecCgReader>(codegen, holderFactory, cookie); +} + +#else +TString GetYtCodecBitCode() { + ythrow yexception() << "No Codegen"; +} + +void YtCodecAddMappings(NCodegen::ICodegen& codegen) { + Y_UNUSED(codegen); + ythrow yexception() << "No Codegen"; +} + +THolder<IYtCodecCgWriter> MakeYtCodecCgWriter(const std::unique_ptr<NCodegen::ICodegen>& codegen, + const void* cookie) { + Y_UNUSED(codegen); + Y_UNUSED(cookie); + ythrow yexception() << "No Codegen"; +} + +THolder<IYtCodecCgReader> MakeYtCodecCgReader(const std::unique_ptr<NCodegen::ICodegen>& codegen, + const NKikimr::NMiniKQL::THolderFactory& holderFactory, + const void* cookie) { + Y_UNUSED(codegen); + Y_UNUSED(holderFactory); + Y_UNUSED(cookie); + ythrow yexception() << "No Codegen"; +} + +#endif + +} diff --git a/yt/yql/providers/yt/codec/codegen/yt_codec_cg.h b/yt/yql/providers/yt/codec/codegen/yt_codec_cg.h new file mode 100644 index 0000000000..d89dc9772b --- /dev/null +++ b/yt/yql/providers/yt/codec/codegen/yt_codec_cg.h @@ -0,0 +1,61 @@ +#pragma once +#include <util/generic/string.h> + +namespace llvm { + class Function; +} + +namespace NKikimr { +namespace NMiniKQL { + class TType; + class THolderFactory; +} + +} + +namespace NYql { + +namespace NUdf { +class TUnboxedValue; +class TUnboxedValuePod; +} + +namespace NCodegen { + class ICodegen; +} + +namespace NCommon { + class TInputBuf; + class TOutputBuf; +} + +TString GetYtCodecBitCode(); +void YtCodecAddMappings(NCodegen::ICodegen& codegen); + +class IYtCodecCgWriter { +public: + virtual ~IYtCodecCgWriter() = default; + virtual void AddField(NKikimr::NMiniKQL::TType* type, ui64 nativeYtTypeFlags) = 0; + virtual llvm::Function* Build() = 0; +}; + +template<bool Flat> +THolder<IYtCodecCgWriter> MakeYtCodecCgWriter(const std::unique_ptr<NCodegen::ICodegen>& codegen, + const void* cookie = nullptr); + +class IYtCodecCgReader { +public: + virtual ~IYtCodecCgReader() = default; + virtual void AddField(NKikimr::NMiniKQL::TType* type, const NYql::NUdf::TUnboxedValuePod& defValue, ui64 nativeYtTypeFlags) = 0; + virtual void SkipField(NKikimr::NMiniKQL::TType* type, ui64 nativeYtTypeFlags) = 0; + virtual void SkipOther() = 0; + virtual void SkipVirtual() = 0; + virtual llvm::Function* Build() = 0; +}; + +THolder<IYtCodecCgReader> MakeYtCodecCgReader(const std::unique_ptr<NCodegen::ICodegen>& codegen, + const NKikimr::NMiniKQL::THolderFactory& holderFactory, const void* cookie = nullptr); + +extern "C" void ThrowBadDecimal(); + +} diff --git a/yt/yql/providers/yt/codec/codegen/yt_codec_cg_dummy.cpp b/yt/yql/providers/yt/codec/codegen/yt_codec_cg_dummy.cpp new file mode 100644 index 0000000000..b9bdb23c35 --- /dev/null +++ b/yt/yql/providers/yt/codec/codegen/yt_codec_cg_dummy.cpp @@ -0,0 +1,38 @@ +#include "yt_codec_cg.h" + +#include <util/generic/yexception.h> + +namespace NYql { + +TString GetYtCodecBitCode() { + throw yexception() << "Codegen is not available"; +} + +void YtCodecAddMappings(NCodegen::ICodegen& codegen) { + Y_UNUSED(codegen); + throw yexception() << "Codegen is not available"; +} + +template<bool Flat> +THolder<IYtCodecCgWriter> MakeYtCodecCgWriter(const std::unique_ptr<NCodegen::ICodegen>& codegen, const void* cookie) { + Y_UNUSED(codegen); + Y_UNUSED(cookie); + throw yexception() << "Codegen is not available"; +} + +template THolder<IYtCodecCgWriter> MakeYtCodecCgWriter<true>(const std::unique_ptr<NCodegen::ICodegen>& codegen, const void* cookie); +template THolder<IYtCodecCgWriter> MakeYtCodecCgWriter<false>(const std::unique_ptr<NCodegen::ICodegen>& codegen, const void* cookie); + +THolder<IYtCodecCgReader> MakeYtCodecCgReader(const std::unique_ptr<NCodegen::ICodegen>& codegen, + const NKikimr::NMiniKQL::THolderFactory& holderFactory, const void* cookie) { + Y_UNUSED(codegen); + Y_UNUSED(holderFactory); + Y_UNUSED(cookie); + throw yexception() << "Codegen is not available"; +} + +extern "C" void ThrowBadDecimal() { + throw yexception() << "Codegen is not available"; +} + +} diff --git a/yt/yql/providers/yt/codec/ut/no_llvm/ya.make b/yt/yql/providers/yt/codec/ut/no_llvm/ya.make new file mode 100644 index 0000000000..6b470f57bf --- /dev/null +++ b/yt/yql/providers/yt/codec/ut/no_llvm/ya.make @@ -0,0 +1,26 @@ +UNITTEST_FOR(yt/yql/providers/yt/codec) + +SRCDIR( + yt/yql/providers/yt/codec/ut +) + +SRCS( + yt_codec_io_ut.cpp +) + +PEERDIR( + library/cpp/yson/node + yql/essentials/minikql/codegen/no_llvm + yql/essentials/minikql/computation/no_llvm + yql/essentials/public/udf/service/exception_policy + yql/essentials/sql + yql/essentials/sql/pg_dummy + yql/essentials/providers/common/codec + yql/essentials/providers/common/mkql + yt/yql/providers/yt/lib/yson_helpers + yt/yql/providers/yt/codec/codegen/no_llvm +) + +YQL_LAST_ABI_VERSION() + +END() diff --git a/yt/yql/providers/yt/codec/ut/ya.make b/yt/yql/providers/yt/codec/ut/ya.make new file mode 100644 index 0000000000..c58c4b1d9b --- /dev/null +++ b/yt/yql/providers/yt/codec/ut/ya.make @@ -0,0 +1,21 @@ +UNITTEST_FOR(yt/yql/providers/yt/codec) + +SRCS( + yt_codec_io_ut.cpp +) + +PEERDIR( + library/cpp/yson/node + yql/essentials/minikql/computation/llvm14 + yql/essentials/public/udf/service/exception_policy + yql/essentials/sql + yql/essentials/sql/pg_dummy + yql/essentials/providers/common/codec + yql/essentials/providers/common/mkql + yt/yql/providers/yt/lib/yson_helpers + yt/yql/providers/yt/codec/codegen +) + +YQL_LAST_ABI_VERSION() + +END() diff --git a/yt/yql/providers/yt/codec/ya.make b/yt/yql/providers/yt/codec/ya.make new file mode 100644 index 0000000000..235b854255 --- /dev/null +++ b/yt/yql/providers/yt/codec/ya.make @@ -0,0 +1,51 @@ +LIBRARY() + +SRCS( + yt_arrow_converter.cpp + yt_codec_io.cpp + yt_codec_io.h + yt_codec_job.cpp + yt_codec_job.h + yt_codec.cpp + yt_codec.h +) + +PEERDIR( + library/cpp/streams/brotli + library/cpp/yson + library/cpp/yson/node + yt/cpp/mapreduce/interface + yt/cpp/mapreduce/io + contrib/libs/apache/arrow + yql/essentials/minikql + yql/essentials/public/udf + yql/essentials/utils + yql/essentials/public/result_format + yql/essentials/public/udf/arrow + yql/essentials/providers/common/codec + yql/essentials/providers/common/codec/arrow + yql/essentials/providers/common/schema/mkql + yql/essentials/providers/common/schema/parser + yt/yql/providers/yt/common + yt/yql/providers/yt/lib/mkql_helpers + yt/yql/providers/yt/lib/skiff +) + +IF (MKQL_DISABLE_CODEGEN) + CFLAGS( + -DMKQL_DISABLE_CODEGEN + ) +ENDIF() + +YQL_LAST_ABI_VERSION() + +END() + +RECURSE( + codegen +) + +RECURSE_FOR_TESTS( + ut + ut/no_llvm +) diff --git a/yt/yql/providers/yt/codec/yt_arrow_converter.cpp b/yt/yql/providers/yt/codec/yt_arrow_converter.cpp new file mode 100644 index 0000000000..f7bfc41dba --- /dev/null +++ b/yt/yql/providers/yt/codec/yt_arrow_converter.cpp @@ -0,0 +1,729 @@ +#include "yt_arrow_converter.h" + +#include <yql/essentials/public/udf/arrow/defs.h> +#include <yql/essentials/public/udf/arrow/block_builder.h> +#include <yql/essentials/public/udf/arrow/block_reader.h> +#include <yql/essentials/utils/yql_panic.h> +#include <yql/essentials/minikql/mkql_type_builder.h> +#include <yql/essentials/minikql/mkql_type_ops.h> +#include <yql/essentials/minikql/mkql_node_cast.h> + +#include <library/cpp/yson/node/node_io.h> +#include <library/cpp/yson/detail.h> +#include <library/cpp/yson/varint.h> +#include <util/stream/mem.h> + +#include <arrow/array/data.h> +#include <arrow/type.h> +#include <arrow/type_traits.h> +#include <arrow/compute/cast.h> + +namespace NYql { + +template<typename T> +struct TypeHelper { + using Type = T; +}; + +#define GEN_TYPE(type)\ + NumericConverterImpl<arrow::type ## Type> + +#define GEN_TYPE_STR(type)\ + StringConverterImpl<arrow::type ## Type> + +template<typename T> +arrow::Datum NumericConverterImpl(NUdf::IArrayBuilder* builder, std::shared_ptr<arrow::ArrayData> block) { + arrow::DictionaryArray dict(block); + typename ::arrow::TypeTraits<T>::ArrayType val(dict.dictionary()->data()); + auto data = dict.indices()->data()->GetValues<ui32>(1); + if (dict.null_count()) { + for (i64 i = 0; i < block->length; ++i) { + if (dict.IsNull(i)) { + builder->Add(NUdf::TBlockItem{}); + } else { + if constexpr (std::is_same_v<decltype(val.Value(data[i])), bool>) { + builder->Add(NUdf::TBlockItem((ui8)val.Value(data[i]))); +#if defined(_darwin_) && defined(_64_) + } else if constexpr (std::is_same_v<decltype(val.Value(data[i])), unsigned long long>) { + builder->Add(NUdf::TBlockItem((ui64)val.Value(data[i]))); + } else if constexpr (std::is_same_v<decltype(val.Value(data[i])), long long>) { + builder->Add(NUdf::TBlockItem((i64)val.Value(data[i]))); +#endif + } else { + builder->Add(NUdf::TBlockItem(val.Value(data[i]))); + } + } + } + } else { + for (i64 i = 0; i < block->length; ++i) { + if constexpr (std::is_same_v<decltype(val.Value(data[i])), bool>) { + builder->Add(NUdf::TBlockItem((ui8)val.Value(data[i]))); +#if defined(_darwin_) && defined(_64_) + } else if constexpr (std::is_same_v<decltype(val.Value(data[i])), unsigned long long>) { + builder->Add(NUdf::TBlockItem((ui64)val.Value(data[i]))); + } else if constexpr (std::is_same_v<decltype(val.Value(data[i])), long long>) { + builder->Add(NUdf::TBlockItem((i64)val.Value(data[i]))); +#endif + } else { + builder->Add(NUdf::TBlockItem(val.Value(data[i]))); + } + } + } + return builder->Build(false); +} + +template<typename T> +arrow::Datum StringConverterImpl(NUdf::IArrayBuilder* builder, std::shared_ptr<arrow::ArrayData> block) { + arrow::DictionaryArray dict(block); + typename ::arrow::TypeTraits<T>::ArrayType val(dict.dictionary()->data()); + auto data = dict.indices()->data()->GetValues<ui32>(1); + if (dict.null_count()) { + for (i64 i = 0; i < block->length; ++i) { + if (dict.IsNull(i)) { + builder->Add(NUdf::TBlockItem{}); + } else { + i32 len; + auto ptr = reinterpret_cast<const char*>(val.GetValue(data[i], &len)); + builder->Add(NUdf::TBlockItem(std::string_view(ptr, len))); + } + } + } else { + for (i64 i = 0; i < block->length; ++i) { + i32 len; + auto ptr = reinterpret_cast<const char*>(val.GetValue(data[i], &len)); + builder->Add(NUdf::TBlockItem(std::string_view(ptr, len))); + } + } + return builder->Build(false); +} + +using namespace NKikimr::NMiniKQL; +using namespace NYson::NDetail; + +class TYsonReaderDetails { +public: + TYsonReaderDetails(const std::string_view& s) : Data_(s.data()), Available_(s.size()) {} + + constexpr char Next() { + YQL_ENSURE(Available_-- > 0); + return *(++Data_); + } + + constexpr char Current() { + return *Data_; + } + + template<typename T> + constexpr T ReadVarSlow() { + T shift = 0; + T value = Current() & 0x7f; + for (;;) { + shift += 7; + value |= T(Next() & 0x7f) << shift; + if (!(Current() & 0x80)) { + break; + } + } + Next(); + return value; + } + + ui32 ReadVarUI32() { + char prev = Current(); + if (Y_LIKELY(!(prev & 0x80))) { + Next(); + return prev; + } + + return ReadVarSlow<ui32>(); + } + + ui64 ReadVarUI64() { + char prev = Current(); + if (Y_LIKELY(!(prev & 0x80))) { + Next(); + return prev; + } + + return ReadVarSlow<ui64>(); + } + + i32 ReadVarI32() { + return NYson::ZigZagDecode32(ReadVarUI32()); + } + + i64 ReadVarI64() { + return NYson::ZigZagDecode64(ReadVarUI64()); + } + + double NextDouble() { + double val = *reinterpret_cast<const double*>(Data_); + Data_ += sizeof(double); + return val; + } + + void Skip(i32 cnt) { + Data_ += cnt; + } + + const char* Data() { + return Data_; + } + + size_t Available() const { + return Available_; + } +private: + const char* Data_; + size_t Available_; +}; + +namespace { +void SkipYson(TYsonReaderDetails& buf) { + switch (buf.Current()) { + case BeginListSymbol: { + buf.Next(); + for (;;) { + SkipYson(buf); + if (buf.Current() == ListItemSeparatorSymbol) { + buf.Next(); + } + if (buf.Current() == EndListSymbol) { + break; + } + } + buf.Next(); + break; + } + case BeginAttributesSymbol: + case BeginMapSymbol: { + auto originalEnd = buf.Current() == BeginMapSymbol ? EndMapSymbol : EndAttributesSymbol; + buf.Next(); + for (;;) { + SkipYson(buf); + YQL_ENSURE(buf.Current() == KeyValueSeparatorSymbol); + buf.Next(); + SkipYson(buf); + if (buf.Current() == KeyedItemSeparatorSymbol) { + buf.Next(); + } + if (buf.Current() == originalEnd) { + break; + } + } + buf.Next(); + break; + } + case StringMarker: + buf.Next(); + buf.Skip(buf.ReadVarI32()); + break; + case Uint64Marker: + case Int64Marker: + buf.Next(); + Y_UNUSED(buf.ReadVarI64()); + break; + case TrueMarker: + case FalseMarker: + buf.Next(); + break; + case DoubleMarker: + buf.Next(); + Y_UNUSED(buf.NextDouble()); + break; + default: + YQL_ENSURE(false, "Unexpected char: " + std::string{buf.Current()}); + } +} + +NUdf::TBlockItem ReadYson(TYsonReaderDetails& buf) { + const char* beg = buf.Data(); + SkipYson(buf); + return NUdf::TBlockItem(std::string_view(beg, buf.Data() - beg)); +} +}; + +class IYsonBlockReader { +public: + virtual NUdf::TBlockItem GetItem(TYsonReaderDetails& buf) = 0; + virtual ~IYsonBlockReader() = default; +}; + +template<bool Native> +class IYsonBlockReaderWithNativeFlag : public IYsonBlockReader { +public: + virtual NUdf::TBlockItem GetNotNull(TYsonReaderDetails&) = 0; + NUdf::TBlockItem GetNullableItem(TYsonReaderDetails& buf) { + char prev = buf.Current(); + if constexpr (Native) { + if (prev == EntitySymbol) { + buf.Next(); + return NUdf::TBlockItem(); + } + return GetNotNull(buf).MakeOptional(); + } + buf.Next(); + if (prev == EntitySymbol) { + return NUdf::TBlockItem(); + } + YQL_ENSURE(prev == BeginListSymbol); + if (buf.Current() == EndListSymbol) { + buf.Next(); + return NUdf::TBlockItem(); + } + auto result = GetNotNull(buf); + if (buf.Current() == ListItemSeparatorSymbol) { + buf.Next(); + } + YQL_ENSURE(buf.Current() == EndListSymbol); + buf.Next(); + return result.MakeOptional(); + } +private: +}; + +template<bool Nullable, bool Native> +class TYsonTupleBlockReader final : public IYsonBlockReaderWithNativeFlag<Native> { +public: + TYsonTupleBlockReader(TVector<std::unique_ptr<IYsonBlockReader>>&& children) + : Children_(std::move(children)) + , Items_(Children_.size()) + {} + + NUdf::TBlockItem GetItem(TYsonReaderDetails& buf) override final { + if constexpr (Nullable) { + return this->GetNullableItem(buf); + } + return GetNotNull(buf); + } + NUdf::TBlockItem GetNotNull(TYsonReaderDetails& buf) override final { + YQL_ENSURE(buf.Current() == BeginListSymbol); + buf.Next(); + for (ui32 i = 0; i < Children_.size(); ++i) { + Items_[i] = Children_[i]->GetItem(buf); + if (buf.Current() == ListItemSeparatorSymbol) { + buf.Next(); + } + } + YQL_ENSURE(buf.Current() == EndListSymbol); + buf.Next(); + return NUdf::TBlockItem(Items_.data()); + } +private: + const TVector<std::unique_ptr<IYsonBlockReader>> Children_; + TVector<NUdf::TBlockItem> Items_; +}; + +template<typename T, bool Nullable, NKikimr::NUdf::EDataSlot OriginalT, bool Native> +class TYsonStringBlockReader final : public IYsonBlockReaderWithNativeFlag<Native> { +public: + NUdf::TBlockItem GetItem(TYsonReaderDetails& buf) override final { + if constexpr (Nullable) { + return this->GetNullableItem(buf); + } + return GetNotNull(buf); + } + + NUdf::TBlockItem GetNotNull(TYsonReaderDetails& buf) override final { + if constexpr (NUdf::EDataSlot::Yson != OriginalT) { + YQL_ENSURE(buf.Current() == StringMarker); + buf.Next(); + const i32 length = buf.ReadVarI32(); + auto res = NUdf::TBlockItem(NUdf::TStringRef(buf.Data(), length)); + buf.Skip(length); + return res; + } else { + return ReadYson(buf); + } + } +}; + +template<typename T, bool Nullable, bool Native> +class TYsonTzDateBlockReader final : public IYsonBlockReaderWithNativeFlag<Native> { +public: + NUdf::TBlockItem GetItem(TYsonReaderDetails& buf) override final { + if constexpr (Nullable) { + return this->GetNullableItem(buf); + } + return GetNotNull(buf); + } + + NUdf::TBlockItem GetNotNull(TYsonReaderDetails& buf) override final { + using TLayout = typename NUdf::TDataType<T>::TLayout; + size_t length = sizeof(TLayout) + sizeof(NUdf::TTimezoneId); + Y_ASSERT(buf.Available() == length); + + TLayout date; + NUdf::TTimezoneId tz; + + if constexpr (std::is_same_v<T, NUdf::TTzDate>) { + DeserializeTzDate({buf.Data(), length}, date, tz); + } else if constexpr (std::is_same_v<T, NUdf::TTzDatetime>) { + DeserializeTzDatetime({buf.Data(), length}, date, tz); + } else if constexpr (std::is_same_v<T, NUdf::TTzTimestamp>) { + DeserializeTzTimestamp({buf.Data(), length}, date, tz); + } else if constexpr (std::is_same_v<T, NUdf::TTzDate32>) { + DeserializeTzDate32({buf.Data(), length}, date, tz); + } else if constexpr (std::is_same_v<T, NUdf::TTzDatetime64>) { + DeserializeTzDatetime64({buf.Data(), length}, date, tz); + } else if constexpr (std::is_same_v<T, NUdf::TTzTimestamp64>) { + DeserializeTzTimestamp64({buf.Data(), length}, date, tz); + } else { + static_assert(sizeof(T) == 0, "Unsupported tz date type"); + } + + buf.Skip(length); + NUdf::TBlockItem res {date}; + res.SetTimezoneId(tz); + return res; + } +}; + +namespace { +struct TYtColumnConverterSettings { + TYtColumnConverterSettings(NKikimr::NMiniKQL::TType* type, const NUdf::IPgBuilder* pgBuilder, arrow::MemoryPool& pool, bool isNative); + NKikimr::NMiniKQL::TType* Type; + const NUdf::IPgBuilder* PgBuilder; + arrow::MemoryPool& Pool; + const bool IsNative; + const bool IsTopOptional; + std::shared_ptr<arrow::DataType> ArrowType; + std::unique_ptr<NKikimr::NUdf::IArrayBuilder> Builder; +}; +} + +template<typename T, bool Nullable, bool Native> +class TYsonFixedSizeBlockReader final : public IYsonBlockReaderWithNativeFlag<Native> { +public: + NUdf::TBlockItem GetItem(TYsonReaderDetails& buf) override final { + if constexpr (Nullable) { + return this->GetNullableItem(buf); + } + return GetNotNull(buf); + } + + NUdf::TBlockItem GetNotNull(TYsonReaderDetails& buf) override final { + if constexpr (std::is_same_v<T, bool>) { + YQL_ENSURE(buf.Current() == FalseMarker || buf.Current() == TrueMarker); + bool res = buf.Current() == TrueMarker; + buf.Next(); + return NUdf::TBlockItem(res); + } + + if constexpr (std::is_same_v<T, unsigned char>) { + if (buf.Current() == FalseMarker || buf.Current() == TrueMarker) { + bool res = buf.Current() == TrueMarker; + buf.Next(); + return NUdf::TBlockItem(T(res)); + } + } + + if constexpr (std::is_integral_v<T> && !std::is_same_v<T, NYql::NDecimal::TInt128>) { + if constexpr (std::is_signed_v<T>) { + YQL_ENSURE(buf.Current() == Int64Marker); + buf.Next(); + return NUdf::TBlockItem(T(buf.ReadVarI64())); + } else { + YQL_ENSURE(buf.Current() == Uint64Marker); + buf.Next(); + return NUdf::TBlockItem(T(buf.ReadVarUI64())); + } + } else if constexpr (std::is_floating_point_v<T>) { + YQL_ENSURE(buf.Current() == DoubleMarker); + buf.Next(); + return NUdf::TBlockItem(T(buf.NextDouble())); + } else { + static_assert(std::is_floating_point_v<T>); + } + } +}; + +template<bool Native> +class TYsonExternalOptBlockReader final : public IYsonBlockReaderWithNativeFlag<Native> { +public: + TYsonExternalOptBlockReader(std::unique_ptr<IYsonBlockReader>&& inner) + : Inner_(std::move(inner)) + {} + + NUdf::TBlockItem GetItem(TYsonReaderDetails& buf) final { + char prev = buf.Current(); + buf.Next(); + if (prev == EntitySymbol) { + return NUdf::TBlockItem(); + } + YQL_ENSURE(prev == BeginListSymbol); + if constexpr (!Native) { + if (buf.Current() == EndListSymbol) { + buf.Next(); + return NUdf::TBlockItem(); + } + } + auto result = Inner_->GetItem(buf); + if (buf.Current() == ListItemSeparatorSymbol) { + buf.Next(); + } + YQL_ENSURE(buf.Current() == EndListSymbol); + buf.Next(); + return result.MakeOptional(); + } + + NUdf::TBlockItem GetNotNull(TYsonReaderDetails&) override final { + YQL_ENSURE(false, "Can't be called"); + } +private: + std::unique_ptr<IYsonBlockReader> Inner_; +}; + +template<bool Native> +struct TYsonBlockReaderTraits { + using TResult = IYsonBlockReader; + template <bool Nullable> + using TTuple = TYsonTupleBlockReader<Nullable, Native>; + // TODO: Implement reader for decimals + template <typename T, bool Nullable, typename = std::enable_if_t<!std::is_same_v<T, NYql::NDecimal::TInt128> && (std::is_integral_v<T> || std::is_floating_point_v<T>)>> + using TFixedSize = TYsonFixedSizeBlockReader<T, Nullable, Native>; + template <typename TStringType, bool Nullable, NKikimr::NUdf::EDataSlot OriginalT> + using TStrings = TYsonStringBlockReader<TStringType, Nullable, OriginalT, Native>; + using TExtOptional = TYsonExternalOptBlockReader<Native>; + + static std::unique_ptr<TResult> MakePg(const NUdf::TPgTypeDescription& desc, const NUdf::IPgBuilder* pgBuilder) { + Y_UNUSED(pgBuilder); + if (desc.PassByValue) { + return std::make_unique<TFixedSize<ui64, true>>(); + } else { + return std::make_unique<TStrings<arrow::BinaryType, true, NKikimr::NUdf::EDataSlot::String>>(); + } + } + + static std::unique_ptr<TResult> MakeResource(bool isOptional) { + Y_UNUSED(isOptional); + ythrow yexception() << "Yson reader not implemented for block resources"; + } + + template<typename TTzDate> + static std::unique_ptr<TResult> MakeTzDate(bool isOptional) { + Y_UNUSED(isOptional); + if (isOptional) { + using TTzDateReader = TYsonTzDateBlockReader<TTzDate, true, Native>; + return std::make_unique<TTzDateReader>(); + } else { + using TTzDateReader = TYsonTzDateBlockReader<TTzDate, false, Native>; + return std::make_unique<TTzDateReader>(); + } + } +}; + +template<bool IsDictionary> +class TPrimitiveColumnConverter { +public: + TPrimitiveColumnConverter(TYtColumnConverterSettings& settings) : Settings_(settings) { + if constexpr (IsDictionary) { + switch (Settings_.ArrowType->id()) { + case arrow::Type::BOOL: PrimitiveConverterImpl_ = GEN_TYPE(Boolean); break; + case arrow::Type::INT8: PrimitiveConverterImpl_ = GEN_TYPE(Int8); break; + case arrow::Type::UINT8: PrimitiveConverterImpl_ = GEN_TYPE(UInt8); break; + case arrow::Type::INT16: PrimitiveConverterImpl_ = GEN_TYPE(Int16); break; + case arrow::Type::UINT16: PrimitiveConverterImpl_ = GEN_TYPE(UInt16); break; + case arrow::Type::INT32: PrimitiveConverterImpl_ = GEN_TYPE(Int32); break; + case arrow::Type::UINT32: PrimitiveConverterImpl_ = GEN_TYPE(UInt32); break; + case arrow::Type::INT64: PrimitiveConverterImpl_ = GEN_TYPE(Int64); break; + case arrow::Type::UINT64: PrimitiveConverterImpl_ = GEN_TYPE(UInt64); break; + case arrow::Type::DOUBLE: PrimitiveConverterImpl_ = GEN_TYPE(Double); break; + case arrow::Type::FLOAT: PrimitiveConverterImpl_ = GEN_TYPE(Float); break; + case arrow::Type::STRING: PrimitiveConverterImpl_ = GEN_TYPE_STR(Binary); break; // all strings from yt is in binary format + case arrow::Type::BINARY: PrimitiveConverterImpl_ = GEN_TYPE_STR(Binary); break; + default: + return; // will check in runtime + }; + } + } + arrow::Datum Convert(std::shared_ptr<arrow::ArrayData> block) { + if constexpr (IsDictionary) { + return PrimitiveConverterImpl_(Settings_.Builder.get(), block); + } + return block; + } +private: + TYtColumnConverterSettings& Settings_; + arrow::Datum (*PrimitiveConverterImpl_)(NUdf::IArrayBuilder*, std::shared_ptr<arrow::ArrayData>); +}; + +template<bool Native, bool IsTopOptional, bool IsDictionary> +class TYtYsonColumnConverter { +public: + TYtYsonColumnConverter(TYtColumnConverterSettings& settings) : Settings_(settings) { + Reader_ = NUdf::MakeBlockReaderImpl<TYsonBlockReaderTraits<Native>>(TTypeInfoHelper(), settings.Type, settings.PgBuilder); + } + + arrow::Datum Convert(std::shared_ptr<arrow::ArrayData> block) { + if constexpr(!IsDictionary) { + arrow::BinaryArray binary(block); + if (block->GetNullCount()) { + for (i64 i = 0; i < block->length; ++i) { + if (binary.IsNull(i)) { + Settings_.Builder->Add(NUdf::TBlockItem{}); + } else { + i32 len; + auto ptr = reinterpret_cast<const char*>(binary.GetValue(i, &len)); + TYsonReaderDetails inp(std::string_view(ptr, len)); + auto res = Reader_->GetItem(inp); + if constexpr (!Native && IsTopOptional) { + res = res.MakeOptional(); + } + Settings_.Builder->Add(std::move(res)); + } + } + } else { + for (i64 i = 0; i < block->length; ++i) { + i32 len; + auto ptr = reinterpret_cast<const char*>(binary.GetValue(i, &len)); + TYsonReaderDetails inp(std::string_view(ptr, len)); + auto res = Reader_->GetItem(inp); + if constexpr (!Native && IsTopOptional) { + res = res.MakeOptional(); + } + Settings_.Builder->Add(std::move(res)); + } + } + return Settings_.Builder->Build(false); + } + arrow::DictionaryArray dict(block); + arrow::BinaryArray binary(block->dictionary); + auto data = dict.indices()->data()->GetValues<ui32>(1); + if (dict.null_count()) { + for (i64 i = 0; i < block->length; ++i) { + if (dict.IsNull(i)) { + Settings_.Builder->Add(NUdf::TBlockItem{}); + } else { + i32 len; + auto ptr = reinterpret_cast<const char*>(binary.GetValue(data[i], &len)); + TYsonReaderDetails inp(std::string_view(ptr, len)); + auto res = Reader_->GetItem(inp); + if constexpr (!Native && IsTopOptional) { + res = res.MakeOptional(); + } + Settings_.Builder->Add(std::move(res)); + } + } + } else { + for (i64 i = 0; i < block->length; ++i) { + i32 len; + auto ptr = reinterpret_cast<const char*>(binary.GetValue(data[i], &len)); + TYsonReaderDetails inp(std::string_view(ptr, len)); + auto res = Reader_->GetItem(inp); + if constexpr (!Native && IsTopOptional) { + res = res.MakeOptional(); + } + Settings_.Builder->Add(std::move(res)); + } + } + return Settings_.Builder->Build(false); + } + +private: + std::shared_ptr<typename TYsonBlockReaderTraits<Native>::TResult> Reader_; + TYtColumnConverterSettings& Settings_; +}; + +template<bool Native, bool IsTopOptional> +class TYtColumnConverter final : public IYtColumnConverter { +public: + TYtColumnConverter(TYtColumnConverterSettings&& settings) + : Settings_(std::move(settings)) + , DictYsonConverter_(Settings_) + , YsonConverter_(Settings_) + , DictPrimitiveConverter_(Settings_) + { + auto type = Settings_.Type; + IsJson_ = type->IsData() && AS_TYPE(TDataType, type)->GetDataSlot() == NUdf::EDataSlot::Json + || (Native && type->IsOptional() && AS_TYPE(TOptionalType, type)->GetItemType()->IsData() + && AS_TYPE(TDataType, AS_TYPE(TOptionalType, type)->GetItemType())->GetDataSlot() == NUdf::EDataSlot::Json); + } + + arrow::Datum Convert(std::shared_ptr<arrow::ArrayData> block) override { + if (arrow::Type::DICTIONARY == block->type->id()) { + auto valType = static_cast<const arrow::DictionaryType&>(*block->type).value_type(); + if (valType->Equals(Settings_.ArrowType)) { + // just unpack + return DictPrimitiveConverter_.Convert(block); + } else if (arrow::Type::UINT8 == Settings_.ArrowType->id() && arrow::Type::BOOL == valType->id()) { + // unpack an cast + auto result = arrow::compute::Cast(DictPrimitiveConverter_.Convert(block), Settings_.ArrowType); + YQL_ENSURE(result.ok()); + return *result; + } else if (IsJson_ && arrow::Type::STRING == Settings_.ArrowType->id() && arrow::Type::BINARY == valType->id()) + { + auto result = arrow::compute::Cast(DictPrimitiveConverter_.Convert(block), Settings_.ArrowType); + YQL_ENSURE(result.ok()); + return *result; + } else { + return DictYsonConverter_.Convert(block); + } + } else { + auto blockType = block->type; + auto noConvert = blockType->Equals(Settings_.ArrowType); + if (noConvert) { + return block; + } else if (arrow::Type::UINT8 == Settings_.ArrowType->id() && arrow::Type::BOOL == blockType->id()) { + auto result = arrow::compute::Cast(arrow::Datum(*block), Settings_.ArrowType); + YQL_ENSURE(result.ok()); + return *result; + } else if (IsJson_ && arrow::Type::STRING == Settings_.ArrowType->id() && arrow::Type::BINARY == blockType->id()) + { + auto result = arrow::compute::Cast(arrow::Datum(*block), Settings_.ArrowType); + YQL_ENSURE(result.ok()); + return *result; + } else { + YQL_ENSURE(arrow::Type::BINARY == blockType->id()); + return YsonConverter_.Convert(block); + } + } + } +private: + TYtColumnConverterSettings Settings_; + TYtYsonColumnConverter<Native, IsTopOptional, true> DictYsonConverter_; + TYtYsonColumnConverter<Native, IsTopOptional, false> YsonConverter_; + TPrimitiveColumnConverter<true> DictPrimitiveConverter_; + bool IsJson_; +}; + +TYtColumnConverterSettings::TYtColumnConverterSettings(NKikimr::NMiniKQL::TType* type, const NUdf::IPgBuilder* pgBuilder, arrow::MemoryPool& pool, bool isNative) + : Type(type), PgBuilder(pgBuilder), Pool(pool), IsNative(isNative), IsTopOptional(!isNative && type->IsOptional()) +{ + if (!isNative) { + if (Type->IsOptional()) { + Type = static_cast<NKikimr::NMiniKQL::TOptionalType*>(Type)->GetItemType(); + } + } + YQL_ENSURE(ConvertArrowType(type, ArrowType), "Can't convert type to arrow"); + size_t maxBlockItemSize = CalcMaxBlockItemSize(type); + size_t maxBlockLen = CalcBlockLen(maxBlockItemSize); + Builder = std::move(NUdf::MakeArrayBuilder( + TTypeInfoHelper(), type, + pool, + maxBlockLen, + pgBuilder + )); +} + +template<typename Common, template <bool...> typename T, typename Args, bool... Acc> +struct TBoolDispatcher { + + std::unique_ptr<Common> Dispatch(Args&& args) const { + return std::make_unique<T<Acc...>>(std::forward<Args&&>(args)); + } + + template <typename... Bools> + auto Dispatch(Args&& args, bool head, Bools... tail) const { + return head ? + TBoolDispatcher<Common, T, Args, Acc..., true >().Dispatch(std::forward<Args&&>(args), tail...) : + TBoolDispatcher<Common, T, Args, Acc..., false>().Dispatch(std::forward<Args&&>(args), tail...); + } +}; + +std::unique_ptr<IYtColumnConverter> MakeYtColumnConverter(NKikimr::NMiniKQL::TType* type, const NUdf::IPgBuilder* pgBuilder, arrow::MemoryPool& pool, bool isNative) { + TYtColumnConverterSettings settings(type, pgBuilder, pool, isNative); + bool isTopOptional = settings.IsTopOptional; + return TBoolDispatcher<IYtColumnConverter, TYtColumnConverter, TYtColumnConverterSettings>().Dispatch(std::move(settings), isNative, isTopOptional); +} +} diff --git a/yt/yql/providers/yt/codec/yt_arrow_converter.h b/yt/yql/providers/yt/codec/yt_arrow_converter.h new file mode 100644 index 0000000000..a6f5c5a52e --- /dev/null +++ b/yt/yql/providers/yt/codec/yt_arrow_converter.h @@ -0,0 +1,22 @@ +#include <yql/essentials/public/udf/arrow/block_builder.h> +#include <yql/essentials/minikql/mkql_node.h> + +#include <arrow/io/interfaces.h> +#include <arrow/io/memory.h> +#include <arrow/ipc/reader.h> +#include <arrow/array.h> +#include <arrow/record_batch.h> +#include <arrow/type.h> +#include <arrow/result.h> +#include <arrow/buffer.h> + +namespace NYql { + +class IYtColumnConverter { +public: + virtual arrow::Datum Convert(std::shared_ptr<arrow::ArrayData> block) = 0; + virtual ~IYtColumnConverter() = default; +}; + +std::unique_ptr<IYtColumnConverter> MakeYtColumnConverter(NKikimr::NMiniKQL::TType* type, const NUdf::IPgBuilder* pgBuilder, arrow::MemoryPool& pool, bool isNative); +} diff --git a/yt/yql/providers/yt/codec/yt_codec.cpp b/yt/yql/providers/yt/codec/yt_codec.cpp new file mode 100644 index 0000000000..61e415d6ff --- /dev/null +++ b/yt/yql/providers/yt/codec/yt_codec.cpp @@ -0,0 +1,667 @@ +#include "yt_codec.h" + +#include <yt/yql/providers/yt/common/yql_names.h> +#include <yt/yql/providers/yt/lib/skiff/yql_skiff_schema.h> +#include <yt/yql/providers/yt/lib/mkql_helpers/mkql_helpers.h> +#include <yql/essentials/providers/common/codec/yql_codec_type_flags.h> +#include <yql/essentials/providers/common/schema/parser/yql_type_parser.h> +#include <yql/essentials/providers/common/schema/mkql/yql_mkql_schema.h> +#include <yql/essentials/minikql/mkql_node_cast.h> +#include <yql/essentials/minikql/mkql_node_builder.h> +#include <yql/essentials/minikql/mkql_string_util.h> +#include <yql/essentials/utils/yql_panic.h> + +#include <library/cpp/yson/node/node_io.h> + +#include <util/generic/hash_set.h> +#include <util/generic/map.h> +#include <util/generic/xrange.h> +#include <util/generic/ylimits.h> +#include <util/generic/ptr.h> +#include <util/stream/str.h> +#include <util/string/builder.h> +#include <util/digest/numeric.h> +#include <util/str_stl.h> + +namespace NYql { + +using namespace NKikimr; +using namespace NKikimr::NMiniKQL; + +////////////////////////////////////////////////////////////////////////////////////////////////////////// + +namespace { + +NYT::TNode ParseYsonSpec(bool inputSpec, const TString& spec) { + YQL_ENSURE(!spec.empty()); + + NYT::TNode attrs; + TStringStream err; + if (!NCommon::ParseYson(attrs, spec, err)) { + ythrow yexception() << "Invalid " + << (inputSpec ? "input" : "output") + << " attrs: " << err.Str(); + } + + return attrs; +} + +} + +////////////////////////////////////////////////////////////////////////////////////////////////////////// +void TMkqlIOSpecs::Init(NCommon::TCodecContext& codecCtx, + const TString& inputSpecs, + const TVector<ui32>& inputGroups, + const TVector<TString>& tableNames, + TType* itemType, + const THashSet<TString>& auxColumns, + const TString& outSpecs, + NKikimr::NMiniKQL::IStatsRegistry* jobStats +) { + NYT::TNode inAttrs, outAttrs; + + if (inputSpecs) { + inAttrs = ParseYsonSpec(true, inputSpecs); + } + + if (outSpecs) { + outAttrs = ParseYsonSpec(false, outSpecs); + } + + Init( + codecCtx, inAttrs, inputGroups, tableNames, + itemType, auxColumns, outAttrs, jobStats + ); +} + +void TMkqlIOSpecs::Init(NCommon::TCodecContext& codecCtx, + const NYT::TNode& inAttrs, + const TVector<ui32>& inputGroups, + const TVector<TString>& tableNames, + TType* itemType, + const THashSet<TString>& auxColumns, + const NYT::TNode& outAttrs, + NKikimr::NMiniKQL::IStatsRegistry* jobStats +) { + if (!inAttrs.IsUndefined()) { + InitInput(codecCtx, inAttrs, inputGroups, tableNames, itemType, {}, auxColumns); + } + if (!outAttrs.IsUndefined()) { + InitOutput(codecCtx, outAttrs); + } + JobStats_ = jobStats; +} + +void TMkqlIOSpecs::Init(NCommon::TCodecContext& codecCtx, + const TString& inputSpecs, + const TVector<TString>& tableNames, + const TMaybe<TVector<TString>>& columns +) { + Init(codecCtx, ParseYsonSpec(true, inputSpecs), tableNames, columns); +} + +void TMkqlIOSpecs::Init(NCommon::TCodecContext& codecCtx, + const NYT::TNode& inAttrs, + const TVector<TString>& tableNames, + const TMaybe<TVector<TString>>& columns +) { + InitInput(codecCtx, inAttrs, {}, tableNames, nullptr, columns, {}); +} + +void TMkqlIOSpecs::Init(NCommon::TCodecContext& codecCtx, const TString& outSpecs) { + Init(codecCtx, ParseYsonSpec(false, outSpecs)); +} + +void TMkqlIOSpecs::Init(NCommon::TCodecContext& codecCtx, const NYT::TNode& outAttrs) { + InitOutput(codecCtx, outAttrs); +} + +void TMkqlIOSpecs::SetTableOffsets(const TVector<ui64>& offsets) { + YQL_ENSURE(Inputs.size() == offsets.size()); + TableOffsets = offsets; +} + +void TMkqlIOSpecs::Clear() { + JobStats_ = nullptr; + Inputs.clear(); + InputGroups.clear(); + Outputs.clear(); + SystemFields_ = TSystemFields(); +} + +void TMkqlIOSpecs::LoadSpecInfo(bool inputSpec, const NYT::TNode& attrs, NCommon::TCodecContext& codecCtx, TSpecInfo& info) { + YQL_ENSURE(inputSpec || attrs.HasKey(YqlRowSpecAttribute), "Missing mandatory " + << TString{YqlRowSpecAttribute}.Quote() << " attribute"); + + if (attrs.HasKey(YqlRowSpecAttribute)) { + auto& rowSpec = attrs[YqlRowSpecAttribute]; + TStringStream err; + info.Type = NCommon::ParseOrderAwareTypeFromYson(rowSpec[RowSpecAttrType], codecCtx, err); + YQL_ENSURE(info.Type, "Invalid row spec type: " << err.Str()); + if (inputSpec && rowSpec.HasKey(RowSpecAttrStrictSchema)) { + info.StrictSchema = rowSpec[RowSpecAttrStrictSchema].IsInt64() + ? rowSpec[RowSpecAttrStrictSchema].AsInt64() != 0 + : NYT::GetBool(rowSpec[RowSpecAttrStrictSchema]); + + if (!info.StrictSchema) { + auto stringType = TDataType::Create(NUdf::TDataType<char*>::Id, codecCtx.Env); + auto othersDictType = TDictType::Create(stringType, stringType, codecCtx.Env); + info.Type = codecCtx.Builder.NewStructType(info.Type, YqlOthersColumnName, othersDictType); + + // Extend input record type by weak fields with 'Yson' type + if (rowSpec.HasKey(RowSpecAttrWeakFields)) { + auto& weakFields = rowSpec[RowSpecAttrWeakFields].AsList(); + auto weakType = codecCtx.Builder.NewOptionalType(codecCtx.Builder.NewDataType(NUdf::EDataSlot::Yson)); + auto structType = AS_TYPE(TStructType, info.Type); + for (auto& field: weakFields) { + if (!structType->FindMemberIndex(field.AsString())) { + info.Type = codecCtx.Builder.NewStructType(info.Type, field.AsString(), weakType); + } + } + } + } + } + if (rowSpec.HasKey(RowSpecAttrDefaultValues)) { + YQL_ENSURE(inputSpec, "Unexpected DefaultValues attribute in output spec"); + for (auto& value : rowSpec[RowSpecAttrDefaultValues].AsMap()) { + auto val = NYT::NodeFromYsonString(value.second.AsString()); + YQL_ENSURE(val.IsString(), "DefaultValues contains non-string value: " << value.second.AsString()); + info.DefaultValues[value.first] = val.AsString(); + } + } + + if (rowSpec.HasKey(RowSpecAttrNativeYtTypeFlags)) { + info.NativeYtTypeFlags = rowSpec[RowSpecAttrNativeYtTypeFlags].AsUint64(); + } else { + if (rowSpec.HasKey(RowSpecAttrUseNativeYtTypes)) { + info.NativeYtTypeFlags = rowSpec[RowSpecAttrUseNativeYtTypes].AsBool() ? NTCF_LEGACY : NTCF_NONE; + } else if (rowSpec.HasKey(RowSpecAttrUseTypeV2)) { + info.NativeYtTypeFlags = rowSpec[RowSpecAttrUseTypeV2].AsBool() ? NTCF_LEGACY : NTCF_NONE; + } + } + + if (rowSpec.HasKey(RowSpecAttrSortedBy)) { + auto structType = AS_TYPE(TStructType, info.Type); + auto& sortedBy = rowSpec[RowSpecAttrSortedBy].AsList(); + auto& sortedByType = rowSpec[RowSpecAttrSortedByTypes].AsList(); + auto& sortedDirections = rowSpec[RowSpecAttrSortDirections].AsList(); + for (size_t i: xrange(sortedBy.size())) { + if (!structType->FindMemberIndex(sortedBy[i].AsString())) { + auto fieldType = NCommon::ParseTypeFromYson(sortedByType[i], codecCtx.Builder, err); + YQL_ENSURE(fieldType, "Invalid SortedByTypes type: " << err.Str()); + if (!sortedDirections.empty() && !sortedDirections[i].AsInt64()) { + fieldType = codecCtx.Builder.NewDataType(NUdf::EDataSlot::String); + } + info.AuxColumns.emplace(sortedBy[i].AsString(), fieldType); + } + } + } + + if (rowSpec.HasKey(RowSpecAttrExplicitYson)) { + YQL_ENSURE(inputSpec, "Unexpected ExplicitYson attribute in output spec"); + for (auto& value : rowSpec[RowSpecAttrExplicitYson].AsList()) { + YQL_ENSURE(value.IsString(), "ExplicitYson contains non-string value: " << NYT::NodeToYsonString(value)); + info.ExplicitYson.emplace(value.AsString()); + } + } + + if (inputSpec && AS_TYPE(TStructType, info.Type)->GetMembersCount() == 0) { + auto fieldType = codecCtx.Builder.NewDataType(NUdf::EDataSlot::Bool); + fieldType = codecCtx.Builder.NewOptionalType(fieldType); + info.AuxColumns.emplace(YqlFakeColumnName, fieldType); + } + } + else { + info.Type = codecCtx.Builder.NewEmptyStructType(); + auto stringType = TDataType::Create(NUdf::TDataType<char*>::Id, codecCtx.Env); + for (auto field: YAMR_FIELDS) { + info.Type = codecCtx.Builder.NewStructType(info.Type, field, stringType); + } + } + if (attrs.HasKey(YqlDynamicAttribute)) { + info.Dynamic = attrs[YqlDynamicAttribute].AsBool(); + } + if (attrs.HasKey(YqlSysColumnPrefix)) { + for (auto& n: attrs[YqlSysColumnPrefix].AsList()) { + const TString sys = n.AsString(); + const auto uniq = info.SysColumns.insert(sys).second; + YQL_ENSURE(uniq, "Duplicate system column: " << sys); + info.Type = codecCtx.Builder.NewStructType(info.Type, TString(YqlSysColumnPrefix).append(sys), TDataType::Create(GetSysColumnTypeId(sys), codecCtx.Env)); + } + } +} + +void TMkqlIOSpecs::InitDecoder(NCommon::TCodecContext& codecCtx, + const TSpecInfo& specInfo, + const THashMap<TString, ui32>& structColumns, + const THashSet<TString>& auxColumns, + TDecoderSpec& decoder +) { + TStructType* inStruct = AS_TYPE(TStructType, specInfo.Type); + for (auto& col: structColumns) { + YQL_ENSURE(inStruct->FindMemberIndex(col.first) || specInfo.AuxColumns.contains(col.first), "Bad column " << col.first); + } + TStructTypeBuilder extendedStruct(codecCtx.Env); + + for (ui32 index = 0; index < inStruct->GetMembersCount(); ++index) { + auto name = inStruct->GetMemberNameStr(index); + if (structColumns.contains(name.Str())) { + auto type = inStruct->GetMemberType(index); + extendedStruct.Add(name.Str(), type); + } + } + for (auto& aux: specInfo.AuxColumns) { + if (structColumns.contains(aux.first)) { + extendedStruct.Add(aux.first, aux.second); + } + } + + auto rowType = extendedStruct.Build(); + decoder.FieldsVec.resize(rowType->GetMembersCount()); + decoder.DefaultValues.resize(rowType->GetMembersCount()); + decoder.StructSize = structColumns.size(); + decoder.NativeYtTypeFlags = specInfo.NativeYtTypeFlags; + decoder.Dynamic = specInfo.Dynamic; + THashSet<ui32> virtualColumns; + + if (specInfo.SysColumns.contains("path")) { + if (auto pos = rowType->FindMemberIndex(YqlSysColumnPath)) { + virtualColumns.insert(*pos); + decoder.FillSysColumnPath = pos; + } + } + if (specInfo.SysColumns.contains("record")) { + if (auto pos = rowType->FindMemberIndex(YqlSysColumnRecord)) { + virtualColumns.insert(*pos); + decoder.FillSysColumnRecord = pos; + } + } + if (specInfo.SysColumns.contains("index")) { + if (auto pos = rowType->FindMemberIndex(YqlSysColumnIndex)) { + virtualColumns.insert(*pos); + decoder.FillSysColumnIndex = pos; + } + } + if (specInfo.SysColumns.contains("num")) { + if (auto pos = rowType->FindMemberIndex(YqlSysColumnNum)) { + virtualColumns.insert(*pos); + decoder.FillSysColumnNum = pos; + } + } + if (specInfo.SysColumns.contains("keyswitch")) { + if (auto pos = rowType->FindMemberIndex(YqlSysColumnKeySwitch)) { + virtualColumns.insert(*pos); + decoder.FillSysColumnKeySwitch = pos; + } + } + + THashSet<ui32> usedPos; + for (ui32 index = 0; index < rowType->GetMembersCount(); ++index) { + auto name = rowType->GetMemberNameStr(index); + auto ndx = structColumns.FindPtr(name.Str()); + YQL_ENSURE(ndx); + ui32 pos = *ndx; + + YQL_ENSURE(usedPos.insert(pos).second, "Reused column position"); + + auto fieldType = rowType->GetMemberType(index); + TMkqlIOSpecs::TDecoderSpec::TDecodeField field; + field.StructIndex = pos; + field.Type = fieldType; + field.Name = name.Str(); + field.Virtual = virtualColumns.contains(index); + field.ExplicitYson = specInfo.ExplicitYson.contains(name.Str()); + decoder.FieldsVec[pos] = field; + decoder.Fields.emplace(field.Name, field); + auto it = specInfo.DefaultValues.find(name.Str()); + if (it != specInfo.DefaultValues.end()) { + YQL_ENSURE(fieldType->GetKind() == TType::EKind::Data && + AS_TYPE(TDataType, fieldType)->GetSchemeType() == NUdf::TDataType<char*>::Id, + "Default values are supported only for string fields"); + decoder.DefaultValues[pos] = MakeString(it->second); + } + + if (!specInfo.StrictSchema && name.Str() == YqlOthersColumnName) { + decoder.OthersStructIndex = pos; + bool isTuple; + bool encoded; + bool useIHash; + GetDictionaryKeyTypes(TDataType::Create(NUdf::TDataType<char*>::Id, codecCtx.Env), + decoder.OthersKeyTypes, isTuple, encoded, useIHash); + } + } + + // Store all unused fields with Max<ui32>() position to correctly fill _other or skip aux + if (decoder.OthersStructIndex || !auxColumns.empty()) { + for (ui32 index = 0; index < inStruct->GetMembersCount(); ++index) { + auto name = inStruct->GetMemberNameStr(index); + if (!structColumns.contains(name.Str())) { + if (decoder.OthersStructIndex || auxColumns.contains(name.Str())) { + TMkqlIOSpecs::TDecoderSpec::TDecodeField field; + field.StructIndex = Max<ui32>(); + field.Name = name.Str(); + field.Type = inStruct->GetMemberType(index); + decoder.FieldsVec.push_back(field); + decoder.Fields.emplace(name.Str(), field); + if (!specInfo.StrictSchema && name.Str() == YqlOthersColumnName && !decoder.OthersStructIndex) { + decoder.OthersStructIndex = Max<ui32>(); + bool isTuple; + bool encoded; + bool useIHash; + GetDictionaryKeyTypes(TDataType::Create(NUdf::TDataType<char*>::Id, codecCtx.Env), + decoder.OthersKeyTypes, isTuple, encoded, useIHash); + } + } + } + } + } + decoder.SkiffSize = decoder.FieldsVec.size(); + + for (auto& col: specInfo.AuxColumns) { + if (!structColumns.contains(col.first)) { + TMkqlIOSpecs::TDecoderSpec::TDecodeField field; + field.StructIndex = Max<ui32>(); + field.Name = col.first; + field.Type = col.second; + decoder.FieldsVec.push_back(field); + auto res = decoder.Fields.emplace(col.first, field); + YQL_ENSURE(res.second, "Aux column " << col.first << " already added"); + } + } +} + +void TMkqlIOSpecs::PrepareInput(const TVector<ui32>& inputGroups) { + InputGroups = inputGroups; + THashSet<ui32> groups; + for (auto& x : InputGroups) { + groups.emplace(x); + } + + for (ui32 group = 0; group < groups.size(); ++group) { + YQL_ENSURE(groups.contains(group), "Missing group: " << group); + } +} + +void TMkqlIOSpecs::InitInput(NCommon::TCodecContext& codecCtx, + const NYT::TNode& inAttrs, + const TVector<ui32>& inputGroups, + const TVector<TString>& tableNames, + TType* itemType, + const TMaybe<TVector<TString>>& columns, + const THashSet<TString>& auxColumns +) { + PrepareInput(inputGroups); + + Y_ENSURE(inAttrs.IsMap(), "Expect Map type of input meta attrs, but got type " << inAttrs.GetType()); + Y_ENSURE(inAttrs.HasKey(YqlIOSpecTables), "Expect " << TString{YqlIOSpecTables}.Quote() << " key"); + + InputSpec = inAttrs; + + TVariantType* itemVarType = nullptr; + if (!InputGroups.empty()) { + YQL_ENSURE(itemType, "Expect non-null item type"); + YQL_ENSURE(TType::EKind::Variant == itemType->GetKind(), "Expect Variant item type, but got " << itemType->GetKindAsStr()); + itemVarType = static_cast<TVariantType*>(itemType); + } + + auto& inputSpecs = inAttrs[YqlIOSpecTables].AsList(); + Inputs.resize(inputSpecs.size()); + YQL_ENSURE(InputGroups.empty() || InputGroups.size() == Inputs.size()); + + bool useCommonColumns = true; + THashMap<TString, ui32> structColumns; + if (columns.Defined()) { + for (size_t i = 0; i < columns->size(); ++i) { + structColumns.insert({columns->at(i), (ui32)i}); + } + } + else if (itemType && InputGroups.empty()) { + TStructType* itemTypeStruct = AS_TYPE(TStructType, itemType); + for (ui32 index = 0; index < itemTypeStruct->GetMembersCount(); ++index) { + structColumns.emplace(itemTypeStruct->GetMemberName(index), index); + } + } + else { + useCommonColumns = false; + } + + THashMap<TString, TSpecInfo> specInfoRegistry; + + for (size_t inputIndex = 0; inputIndex < inputSpecs.size(); ++inputIndex) { + try { + auto group = InputGroups.empty() ? 0 : inputGroups.at(inputIndex); + TSpecInfo localSpecInfo; + TSpecInfo* specInfo = &localSpecInfo; + TString decoderRefName = TStringBuilder() << "_internal" << inputIndex; + if (inputSpecs[inputIndex].IsString()) { + auto refName = inputSpecs[inputIndex].AsString(); + decoderRefName = refName; + if (auto p = specInfoRegistry.FindPtr(refName)) { + specInfo = p; + } else { + Y_ENSURE(inAttrs.HasKey(YqlIOSpecRegistry) && inAttrs[YqlIOSpecRegistry].HasKey(refName), "Bad input registry reference: " << refName); + specInfo = &specInfoRegistry[refName]; + LoadSpecInfo(true, inAttrs[YqlIOSpecRegistry][refName], codecCtx, *specInfo); + } + } else { + LoadSpecInfo(true, inputSpecs[inputIndex], codecCtx, localSpecInfo); + } + + TStructType* inStruct = AS_TYPE(TStructType, specInfo->Type); + if (itemType) { // itemType may be null for operations without graph (TopSort f.e.) + inStruct = itemVarType + ? AS_TYPE(TStructType, itemVarType->GetAlternativeType(group)) + : AS_TYPE(TStructType, itemType); + } + + if (!useCommonColumns) { + structColumns.clear(); + for (ui32 index = 0; index < inStruct->GetMembersCount(); ++index) { + auto col = inStruct->GetMemberName(index); + structColumns.emplace(col, index); + decoderRefName.append(';').append(ToString(col.length())).append(':').append(col); // Make decoder unique by input type and set of used columns + } + } + if (specInfo->Dynamic) { + decoderRefName.append(";dynamic"); + } + + if (auto p = Decoders.FindPtr(decoderRefName)) { + // Reuse already initialized decoder for the same schema and column set + Inputs[inputIndex] = p; + } + else { + TDecoderSpec* decoder = &Decoders[decoderRefName]; + InitDecoder(codecCtx, *specInfo, structColumns, auxColumns, *decoder); + Inputs[inputIndex] = decoder; + } + } + catch (const yexception& e) { + ythrow yexception() << "Invalid decoder spec for " << inputIndex << " input: " << e; + } + } + + if (!tableNames.empty()) { + YQL_ENSURE(tableNames.size() == Inputs.size()); + for (auto& name: tableNames) { + TableNames.push_back(MakeString(name)); + } + } else { + TableNames.resize(Inputs.size(), NUdf::TUnboxedValuePod::Zero()); + } + TableOffsets.resize(Inputs.size(), 0ull); +} + +void TMkqlIOSpecs::InitOutput(NCommon::TCodecContext& codecCtx, const NYT::TNode& outAttrs) { + Y_ENSURE(outAttrs.IsMap(), "Expect Map type of output meta attrs, but got type " << outAttrs.GetType()); + Y_ENSURE(outAttrs.HasKey(YqlIOSpecTables), "Expect " << TString{YqlIOSpecTables}.Quote() << " key"); + + OutputSpec = outAttrs; + + auto& outputSpecs = outAttrs[YqlIOSpecTables].AsList(); + Outputs.resize(outputSpecs.size()); + + THashMap<TString, TSpecInfo> specInfoRegistry; + THashMap<TString, TStructType*> outTypeRegistry; + + for (size_t i = 0; i < outputSpecs.size(); ++i) { + try { + TSpecInfo localSpecInfo; + TSpecInfo* specInfo = &localSpecInfo; + TString refName; + if (outputSpecs[i].IsString()) { + refName = outputSpecs[i].AsString(); + + if (auto p = specInfoRegistry.FindPtr(refName)) { + specInfo = p; + } + else { + Y_ENSURE(outAttrs.HasKey(YqlIOSpecRegistry) && outAttrs[YqlIOSpecRegistry].HasKey(refName), "Bad output registry reference: " << refName); + specInfo = &specInfoRegistry[refName]; + LoadSpecInfo(false, outAttrs[YqlIOSpecRegistry][refName], codecCtx, *specInfo); + } + + if (auto p = outTypeRegistry.FindPtr(refName)) { + Outputs[i].RowType = *p; + Outputs[i].NativeYtTypeFlags = specInfo->NativeYtTypeFlags; + continue; + } + } else { + LoadSpecInfo(false, outputSpecs[i], codecCtx, localSpecInfo); + } + + auto structType = AS_TYPE(TStructType, specInfo->Type); + // Extend struct by aux columns + for (auto& col: specInfo->AuxColumns) { + structType = AS_TYPE(TStructType, codecCtx.Builder.NewStructType(structType, col.first, col.second)); + } + if (refName) { + outTypeRegistry[refName] = structType; + } + + Outputs[i].RowType = structType; + Outputs[i].NativeYtTypeFlags = specInfo->NativeYtTypeFlags; + } catch (const yexception& e) { + ythrow yexception() << "Invalid encoder spec for " << i << " output: " << e; + } + } +} + +NYT::TFormat TMkqlIOSpecs::MakeOutputFormat() const { + if (!UseSkiff_ || Outputs.empty()) { + return NYT::TFormat::YsonBinary(); + } + + YQL_ENSURE(!OutputSpec.IsUndefined()); + NYT::TNode formatConfig = TablesSpecToOutputSkiff(OutputSpec); + return NYT::TFormat(formatConfig); +} + +NYT::TFormat TMkqlIOSpecs::MakeOutputFormat(size_t tableIndex) const { + Y_ENSURE(tableIndex < Outputs.size(), "Invalid output table index: " << tableIndex); + + if (!UseSkiff_) { + return NYT::TFormat::YsonBinary(); + } + + YQL_ENSURE(!OutputSpec.IsUndefined()); + NYT::TNode formatConfig = SingleTableSpecToOutputSkiff(OutputSpec, tableIndex); + return NYT::TFormat(formatConfig); +} + +NYT::TFormat TMkqlIOSpecs::MakeInputFormat(const THashSet<TString>& auxColumns) const { + if (UseBlockInput_) { + YQL_ENSURE(auxColumns.empty()); + return NYT::TFormat(NYT::TNode("arrow")); + } + + if (!UseSkiff_ || Inputs.empty()) { + return NYT::TFormat::YsonBinary(); + } + + NYT::TNode formatConfig = NYT::TNode("skiff"); + auto& skiffConfig = formatConfig.Attributes(); + auto schemas = NYT::TNode::CreateList(); + THashMap<const TDecoderSpec*, size_t> uniqDecoders; + for (size_t i = 0; i < Inputs.size(); ++i) { + auto input = Inputs[i]; + size_t schemaId = uniqDecoders.size(); + auto p = uniqDecoders.emplace(input, schemaId); + if (p.second) { + THashMap<TString, ui32> structColumns; + for (size_t f = 0; f < input->StructSize; ++f) { + structColumns[input->FieldsVec[f].Name] = f; + } + + NYT::TNode tableSchema = SingleTableSpecToInputSkiffSchema(InputSpec, i, structColumns, auxColumns, + SystemFields_.HasFlags(ESystemField::RowIndex), + SystemFields_.HasFlags(ESystemField::RangeIndex), + SystemFields_.HasFlags(ESystemField::KeySwitch)); + skiffConfig["skiff_schema_registry"][TStringBuilder() << "table" << schemaId] = tableSchema; + } + else { + schemaId = p.first->second; + } + + schemas.Add(NYT::TNode(TStringBuilder() << "$table" << schemaId)); + } + + skiffConfig["table_skiff_schemas"] = schemas; + //Cerr << NYT::NodeToYsonString(skiffConfig) << Endl; + return NYT::TFormat(formatConfig); +} + +NYT::TFormat TMkqlIOSpecs::MakeInputFormat(size_t tableIndex) const { + Y_ENSURE(tableIndex < Inputs.size(), "Invalid output table index: " << tableIndex); + + if (UseBlockInput_) { + YQL_ENSURE(tableIndex == 0); + return NYT::TFormat(NYT::TNode("arrow")); + } + + if (!UseSkiff_) { + return NYT::TFormat::YsonBinary(); + } + + NYT::TNode formatConfig = NYT::TNode("skiff"); + + auto input = Inputs[tableIndex]; + THashMap<TString, ui32> structColumns; + for (size_t f = 0; f < input->StructSize; ++f) { + structColumns[input->FieldsVec[f].Name] = f; + } + + NYT::TNode tableSchema = SingleTableSpecToInputSkiffSchema(InputSpec, tableIndex, structColumns, {}, + SystemFields_.HasFlags(ESystemField::RowIndex), + SystemFields_.HasFlags(ESystemField::RangeIndex), + SystemFields_.HasFlags(ESystemField::KeySwitch)); + + auto& skiffConfig = formatConfig.Attributes(); + skiffConfig["table_skiff_schemas"] = NYT::TNode::CreateList().Add(std::move(tableSchema)); + + return NYT::TFormat(formatConfig); +} + +TMkqlIOCache::TMkqlIOCache(const TMkqlIOSpecs& specs, const THolderFactory& holderFactory) + : Specs_(specs) + , HolderFactory(holderFactory) +{ + THashSet<ui32> groups; + for (auto& x : Specs_.InputGroups) { + groups.emplace(x); + } + + for (ui32 i = 0; i < Max<ui32>(1, groups.size()); ++i) { + RowCache_.emplace_back(MakeHolder<TPlainContainerCache>()); + } + + DecoderCache_.resize(Specs_.Inputs.size()); + for (size_t i: xrange(Specs_.Inputs.size())) { + DecoderCache_[i].LastFields_.reserve(Specs_.Inputs[i]->FieldsVec.size()); + for (auto& field: Specs_.Inputs[i]->FieldsVec) { + DecoderCache_[i].LastFields_.push_back(&field); + } + } +} + +} // NYql diff --git a/yt/yql/providers/yt/codec/yt_codec.h b/yt/yql/providers/yt/codec/yt_codec.h new file mode 100644 index 0000000000..0b8a3b9d31 --- /dev/null +++ b/yt/yql/providers/yt/codec/yt_codec.h @@ -0,0 +1,255 @@ +#pragma once + +#include <yql/essentials/providers/common/codec/yql_codec.h> +#include <yql/essentials/public/udf/udf_value.h> + +#include <yql/essentials/minikql/computation/mkql_computation_node_holders.h> +#include <yql/essentials/minikql/mkql_stats_registry.h> +#include <yql/essentials/minikql/mkql_node.h> + +#include <yt/cpp/mapreduce/interface/io.h> + +#include <library/cpp/yson/node/node.h> + +#include <util/generic/ptr.h> +#include <util/generic/maybe.h> +#include <util/generic/hash.h> +#include <util/generic/hash_set.h> +#include <util/generic/vector.h> +#include <util/generic/string.h> +#include <util/generic/flags.h> + +namespace NYql { + +class TMkqlIOSpecs { +public: + enum class ESystemField: ui32 { + KeySwitch = 1 << 0, + RowIndex = 1 << 1, + RangeIndex = 1 << 2, + }; + + Y_DECLARE_FLAGS(TSystemFields, ESystemField); + + struct TSpecInfo { + NKikimr::NMiniKQL::TType* Type = nullptr; + bool StrictSchema = true; + THashMap<TString, TString> DefaultValues; + THashMap<TString, NKikimr::NMiniKQL::TType*> AuxColumns; + ui64 NativeYtTypeFlags = 0; + bool Dynamic = false; + THashSet<TString> SysColumns; + THashSet<TString> ExplicitYson; + }; + + struct TDecoderSpec { + struct TDecodeField { + TString Name; + ui32 StructIndex = 0; + NKikimr::NMiniKQL::TType* Type = nullptr; + bool Virtual = false; + bool ExplicitYson = false; + }; + + TMaybe<ui32> OthersStructIndex; // filled if scheme is not strict + NKikimr::NMiniKQL::TKeyTypes OthersKeyTypes; + THashMap<TString, TDecodeField> Fields; + TVector<TDecodeField> FieldsVec; + ui32 StructSize = 0; // Number of visible columns after decoding (excludes all aux columns) + ui32 SkiffSize = 0; // Number of columns expected by skiff (includes all visible columns and explicitly requested aux columns) + TVector<NKikimr::NUdf::TUnboxedValue> DefaultValues; + ui64 NativeYtTypeFlags = 0; + bool Dynamic = false; + TMaybe<ui32> FillSysColumnPath; + TMaybe<ui32> FillSysColumnRecord; + TMaybe<ui32> FillSysColumnIndex; + TMaybe<ui32> FillSysColumnNum; + TMaybe<ui32> FillSysColumnKeySwitch; + }; + + struct TEncoderSpec { + NKikimr::NMiniKQL::TStructType* RowType = nullptr; + ui64 NativeYtTypeFlags = 0; + }; + +public: + // Job specific initialization + void Init(NCommon::TCodecContext& codecCtx, + const TString& inputSpecs, + const TVector<ui32>& inputGroups, + const TVector<TString>& tableNames, + NKikimr::NMiniKQL::TType* itemType, + const THashSet<TString>& auxColumns, + const TString& outSpecs, + NKikimr::NMiniKQL::IStatsRegistry* jobStats = nullptr + ); + + // Job specific initialization + void Init(NCommon::TCodecContext& codecCtx, + const NYT::TNode& inAttrs, + const TVector<ui32>& inputGroups, + const TVector<TString>& tableNames, + NKikimr::NMiniKQL::TType* itemType, + const THashSet<TString>& auxColumns, + const NYT::TNode& outAttrs, + NKikimr::NMiniKQL::IStatsRegistry* jobStats = nullptr + ); + + // Pull specific initialization + void Init(NCommon::TCodecContext& codecCtx, + const TString& inputSpecs, + const TVector<TString>& tableNames, + const TMaybe<TVector<TString>>& columns // Use Nothing to select all columns in original order + ); + + // Pull specific initialization + void Init(NCommon::TCodecContext& codecCtx, + const NYT::TNode& inAttrs, + const TVector<TString>& tableNames, + const TMaybe<TVector<TString>>& columns // Use Nothing to select all columns in original order + ); + + // Fill specific initialization + void Init(NCommon::TCodecContext& codecCtx, + const TString& outSpecs + ); + + // Fill specific initialization + void Init(NCommon::TCodecContext& codecCtx, + const NYT::TNode& outAttrs + ); + + void SetUseSkiff(const TString& optLLVM, TSystemFields sysFields = {}) { + UseSkiff_ = true; + OptLLVM_ = optLLVM; + SystemFields_ = sysFields; + } + + void SetUseBlockInput() { + UseBlockInput_ = true; + } + + void SetTableOffsets(const TVector<ui64>& offsets); + + void Clear(); + + static void LoadSpecInfo(bool inputSpec, const NYT::TNode& attrs, NCommon::TCodecContext& codecCtx, TSpecInfo& info); + + NYT::TFormat MakeInputFormat(const THashSet<TString>& auxColumns) const; // uses Inputs + NYT::TFormat MakeInputFormat(size_t tableIndex) const; // uses Inputs + NYT::TFormat MakeOutputFormat() const; // uses Outputs + NYT::TFormat MakeOutputFormat(size_t tableIndex) const; // uses Outputs + +public: + bool UseSkiff_ = false; + bool UseBlockInput_ = false; + TString OptLLVM_; + TSystemFields SystemFields_; + + NKikimr::NMiniKQL::IStatsRegistry* JobStats_ = nullptr; + THashMap<TString, TDecoderSpec> Decoders; + TVector<const TDecoderSpec*> Inputs; + TVector<ui32> InputGroups; // translation of tableindex->index of Inputs + TVector<TEncoderSpec> Outputs; + + NYT::TNode InputSpec; + NYT::TNode OutputSpec; + TVector<NKikimr::NUdf::TUnboxedValue> TableNames; + TVector<ui64> TableOffsets; + +protected: + void PrepareInput(const TVector<ui32>& inputGroups); + + void InitInput(NCommon::TCodecContext& codecCtx, + const NYT::TNode& inAttrs, + const TVector<ui32>& inputGroups, + const TVector<TString>& tableNames, + NKikimr::NMiniKQL::TType* itemType, + const TMaybe<TVector<TString>>& columns, + const THashSet<TString>& auxColumns + ); + + void InitDecoder(NCommon::TCodecContext& codecCtx, + const TSpecInfo& specInfo, + const THashMap<TString, ui32>& structColumns, + const THashSet<TString>& auxColumns, + TDecoderSpec& decoder + ); + + void InitOutput(NCommon::TCodecContext& codecCtx, + const NYT::TNode& outAttrs + ); +}; + +Y_DECLARE_OPERATORS_FOR_FLAGS(TMkqlIOSpecs::TSystemFields); + +////////////////////////////////////////////////////////////////////////////////////////////////////////// + +class TMkqlIOCache { +public: + TMkqlIOCache(const TMkqlIOSpecs& specs, const NKikimr::NMiniKQL::THolderFactory& holderFactory); + + NKikimr::NUdf::TUnboxedValue NewRow(size_t tableIndex, NKikimr::NUdf::TUnboxedValue*& items, bool wideBlock = false) { + const auto group = Specs_.InputGroups.empty() ? 0 : Specs_.InputGroups[tableIndex]; + auto structSize = Specs_.Inputs[tableIndex]->StructSize; + if (wideBlock) { + structSize++; + } + return RowCache_[group]->NewArray(HolderFactory, structSize, items); + } + + const TMkqlIOSpecs& GetSpecs() const { + return Specs_; + } + const NKikimr::NMiniKQL::THolderFactory& GetHolderFactory() { + return HolderFactory; + } + + ui32 GetMaxOthersFields(size_t tableIndex) const { + return DecoderCache_.at(tableIndex).MaxOthersFields_; + } + + void UpdateMaxOthersFields(size_t tableIndex, ui32 maxOthersFields) { + DecoderCache_[tableIndex].MaxOthersFields_ = Max<ui32>(DecoderCache_.at(tableIndex).MaxOthersFields_, maxOthersFields); + } + + TVector<const TMkqlIOSpecs::TDecoderSpec::TDecodeField*>& GetLastFields(size_t tableIndex) { + return DecoderCache_.at(tableIndex).LastFields_; + } + + +private: + const TMkqlIOSpecs& Specs_; + const NKikimr::NMiniKQL::THolderFactory& HolderFactory; + TVector<THolder<NKikimr::NMiniKQL::TPlainContainerCache>> RowCache_; + + struct TDecoderCache { + ui32 MaxOthersFields_ = 0; + TVector<const TMkqlIOSpecs::TDecoderSpec::TDecodeField*> LastFields_; + }; + TVector<TDecoderCache> DecoderCache_; +}; + + +////////////////////////////////////////////////////////////////////////////////////////////////////////// + +class IMkqlReaderImpl : public NYT::IReaderImplBase { +public: + virtual ~IMkqlReaderImpl() = default; + virtual NKikimr::NUdf::TUnboxedValue GetRow() const = 0; +}; + +using IMkqlReaderImplPtr = TIntrusivePtr<IMkqlReaderImpl>; + +class IMkqlWriterImpl : public TThrRefBase { +public: + virtual ~IMkqlWriterImpl() = default; + virtual void AddRow(const NUdf::TUnboxedValuePod row) = 0; + virtual void AddFlatRow(const NUdf::TUnboxedValuePod* row) = 0; + virtual void Finish() = 0; + virtual void Abort() = 0; +}; + +using IMkqlWriterImplPtr = TIntrusivePtr<IMkqlWriterImpl>; + +} // NYql diff --git a/yt/yql/providers/yt/codec/yt_codec_io.cpp b/yt/yql/providers/yt/codec/yt_codec_io.cpp new file mode 100644 index 0000000000..d96be778ee --- /dev/null +++ b/yt/yql/providers/yt/codec/yt_codec_io.cpp @@ -0,0 +1,2454 @@ +#include "yt_codec_io.h" + +#include <yql/essentials/public/udf/arrow/args_dechunker.h> +#include <yql/essentials/providers/common/codec/arrow/yql_codec_buf_input_stream.h> +#include <yt/yql/providers/yt/codec/yt_arrow_converter.h> +#include <yql/essentials/public/result_format/yql_restricted_yson.h> +#include <yql/essentials/providers/common/codec/yql_codec_type_flags.h> +#include <yql/essentials/providers/common/codec/yql_codec.h> +#include <yt/yql/providers/yt/common/yql_names.h> +#include <exception> +#ifndef MKQL_DISABLE_CODEGEN +#include <yt/yql/providers/yt/codec/codegen/yt_codec_cg.h> +#endif +#include <yql/essentials/minikql/mkql_alloc.h> +#include <yql/essentials/minikql/mkql_stats_registry.h> +#include <yql/essentials/minikql/mkql_string_util.h> +#include <yql/essentials/minikql/mkql_node_cast.h> +#include <yql/essentials/minikql/aligned_page_pool.h> +#include <yql/essentials/utils/yql_panic.h> +#include <yql/essentials/utils/swap_bytes.h> +#include <yql/essentials/utils/log/log.h> +#include <yql/essentials/public/decimal/yql_decimal_serialize.h> +#include <yql/essentials/public/udf/arrow/memory_pool.h> + +#include <library/cpp/yson/node/node_io.h> +#include <library/cpp/yson/detail.h> +#include <library/cpp/yson/varint.h> +#include <library/cpp/streams/brotli/brotli.h> + +#include <util/generic/yexception.h> +#include <util/generic/string.h> +#include <util/generic/queue.h> +#include <util/generic/xrange.h> +#include <util/system/guard.h> +#include <util/system/yassert.h> +#include <util/system/condvar.h> +#include <util/system/mutex.h> +#include <util/system/thread.h> +#include <util/stream/file.h> + +#include <arrow/util/key_value_metadata.h> + +#include <functional> + +namespace NYql { + +using namespace NCommon; +using namespace NKikimr; +using namespace NKikimr::NMiniKQL; +using namespace NYT; +using namespace ::NYson::NDetail; + +////////////////////////////////////////////////////////////////////////////////////////////////////////// + +TStatKey InputDecodeTime("Job_InputDecodeTime", true); +TStatKey InputReadTime("Job_InputReadTime", true); +TStatKey OutputEncodeTime("Job_OutputEncodeTime", true); +TStatKey OutputWriteTime("Job_OutputWriteTime", true); + +struct TBlock : public TThrRefBase { + size_t Avail_ = 0; + std::optional<size_t> LastRecordBoundary_; + const size_t Capacity_; + TArrayHolder<char> Buffer_; + TMaybe<yexception> Error_; + + explicit TBlock(size_t capacity) + : Capacity_(capacity) + , Buffer_(new char[Capacity_]) + {} +}; + +using TBlockPtr = TIntrusivePtr<TBlock>; + +struct TBufferManager { + const size_t BlockCount_; + const size_t BlockSize_; + TQueue<TBlockPtr> FreeBlocks_; + TQueue<TBlockPtr> FilledBlocks_; + + TBufferManager(size_t blockCount, size_t blockSize) + : BlockCount_(blockCount) + , BlockSize_(blockSize) + { + // mark all blocks as free + for (size_t i = 0; i < BlockCount_; ++i) { + FreeBlocks_.push(MakeIntrusive<TBlock>(BlockSize_)); + } + } +}; + +size_t LoadWithTimeout(IInputStream& source, char* buffer_in, size_t len, const TMaybe<TInstant>& deadline) { + char* buf = buffer_in; + while (len) { + const size_t ret = source.Read(buf, len); + + buf += ret; + len -= ret; + + if (ret == 0) { + break; + } + + if (deadline && Now() > deadline) { + ythrow TTimeoutException() << "Read timeout"; + } + } + + return buf - buffer_in; +} + +////////////////////////////////////////////////////////////////////////////////////////////////////////// + +class TAsyncBlockReader: public IBlockReader { +public: + TAsyncBlockReader(NYT::TRawTableReader& source, size_t blockCount, size_t blockSize) + : Source_(source) + , BufMan_(blockCount, blockSize) + , Thread_(TThread::TParams(&ThreadFunc, this).SetName("reader")) + { + Thread_.Start(); + } + + ~TAsyncBlockReader() { + with_lock(Mutex_) { + Shutdown_ = true; + CondRestore_.Signal(); + CondFill_.Signal(); + } + + Thread_.Join(); + } + + void SetDeadline(TInstant deadline) override { + Deadline_ = deadline; + } + + std::pair<const char*, const char*> NextFilledBlock() override { + TBlockPtr firstBlock; + with_lock(Mutex_) { + while (BufMan_.FilledBlocks_.empty()) { + CondRead_.WaitI(Mutex_); + } + + firstBlock = BufMan_.FilledBlocks_.front(); + } + + if (firstBlock->Error_.Defined()) { + YQL_LOG_CTX_THROW *firstBlock->Error_; + } + return { firstBlock->Buffer_.Get(), firstBlock->Buffer_.Get() + firstBlock->Avail_ }; + } + + void ReturnBlock() override { + auto guard = Guard(Mutex_); + + auto firstBlock = BufMan_.FilledBlocks_.front(); + firstBlock->Avail_ = 0; + firstBlock->Error_.Clear(); + BufMan_.FilledBlocks_.pop(); + BufMan_.FreeBlocks_.push(firstBlock); + + CondFill_.Signal(); + } + + bool Retry(const TMaybe<ui32>& rangeIndex, const TMaybe<ui64>& rowIndex, const std::exception_ptr& error) override { + auto guard = Guard(Mutex_); + + // Clean all filled blocks + while (!BufMan_.FilledBlocks_.empty()) { + auto block = BufMan_.FilledBlocks_.front(); + block->Avail_ = 0; + block->Error_.Clear(); + BufMan_.FilledBlocks_.pop(); + BufMan_.FreeBlocks_.push(block); + } + + if (!Source_.Retry(rangeIndex, rowIndex, error)) { + Shutdown_ = true; + CondRestore_.Signal(); + return false; + } + CondRestore_.Signal(); + return true; + } + +private: + static void* ThreadFunc(void* param) { + static_cast<TAsyncBlockReader*>(param)->Fetch(); + return nullptr; + } + + void Fetch() { + for (;;) { + TBlockPtr freeBlock; + with_lock(Mutex_) { + while (BufMan_.FreeBlocks_.empty() && !Shutdown_) { + CondFill_.WaitI(Mutex_); + } + + if (Shutdown_) { + return; + } + + freeBlock = BufMan_.FreeBlocks_.front(); + BufMan_.FreeBlocks_.pop(); + } + + bool hasError = false; + try { + freeBlock->Avail_ = LoadWithTimeout(Source_, freeBlock->Buffer_.Get(), freeBlock->Capacity_, Deadline_); + } + catch (yexception& e) { + freeBlock->Error_ = std::move(e); + hasError = true; + } + catch (...) { + freeBlock->Error_ = (yexception() << CurrentExceptionMessage()); + hasError = true; + } + + const bool lastBlock = !freeBlock->Avail_; + with_lock(Mutex_) { + BufMan_.FilledBlocks_.push(freeBlock); + CondRead_.Signal(); + if (hasError) { + CondRestore_.WaitI(Mutex_); + if (Shutdown_) { + return; + } + } else if (lastBlock) { + break; + } + } + } + } + +private: + TMaybe<TInstant> Deadline_; + NYT::TRawTableReader& Source_; + TBufferManager BufMan_; + TMutex Mutex_; + TCondVar CondFill_; + TCondVar CondRead_; + TCondVar CondRestore_; + bool Shutdown_ = false; + TThread Thread_; +}; + +////////////////////////////////////////////////////////////////////////////////////////////////////////// + +class TSyncBlockReader: public IBlockReader { +public: + TSyncBlockReader(NYT::TRawTableReader& source, size_t blockSize) + : Source_(source) + , Block_(blockSize) + { + } + + ~TSyncBlockReader() = default; + + void SetDeadline(TInstant deadline) override { + Deadline_ = deadline; + } + + std::pair<const char*, const char*> NextFilledBlock() override { + Block_.Avail_ = LoadWithTimeout(Source_, Block_.Buffer_.Get(), Block_.Capacity_, Deadline_); + return { Block_.Buffer_.Get(), Block_.Buffer_.Get() + Block_.Avail_ }; + } + + void ReturnBlock() override { + Block_.Avail_ = 0; + } + + bool Retry(const TMaybe<ui32>& rangeIndex, const TMaybe<ui64>& rowIndex, const std::exception_ptr& error) override { + Block_.Avail_ = 0; + return Source_.Retry(rangeIndex, rowIndex, error); + } + +private: + NYT::TRawTableReader& Source_; + TBlock Block_; + TMaybe<TInstant> Deadline_; +}; + +////////////////////////////////////////////////////////////////////////////////////////////////////////// + +THolder<IBlockReader> MakeBlockReader(NYT::TRawTableReader& source, size_t blockCount, size_t blockSize) { + THolder<IBlockReader> res; + if (blockCount > 1) { + res.Reset(new TAsyncBlockReader(source, blockCount, blockSize)); + } else { + res.Reset(new TSyncBlockReader(source, blockSize)); + } + return res; +} + +////////////////////////////////////////////////////////////////////////////////////////////////////////// + +class TAsyncBlockWriter: public IBlockWriter { +public: + TAsyncBlockWriter(IOutputStream& target, size_t blockCount, size_t blockSize) + : Target_(target) + , BufMan_(blockCount, blockSize) + , Thread_(TThread::TParams(&ThreadFunc, this).SetName("writer")) + { + Thread_.Start(); + } + + ~TAsyncBlockWriter() { + Stop(); + } + + void SetRecordBoundaryCallback(std::function<void()> callback) override { + OnRecordBoundaryCallback_ = std::move(callback); + } + + std::pair<char*, char*> NextEmptyBlock() override { + TBlockPtr firstBlock; + with_lock(Mutex_) { + while (BufMan_.FreeBlocks_.empty()) { + CondFill_.WaitI(Mutex_); + } + if (Error_.Defined()) { + YQL_LOG_CTX_THROW *Error_; + } + + firstBlock = BufMan_.FreeBlocks_.front(); + } + + firstBlock->Avail_ = 0; + return { firstBlock->Buffer_.Get(), firstBlock->Buffer_.Get() + firstBlock->Capacity_ }; + } + + void ReturnBlock(size_t avail, std::optional<size_t> lastRecordBoundary) override { + if (!avail) { + return; + } + + with_lock(Mutex_) { + auto firstBlock = BufMan_.FreeBlocks_.front(); + firstBlock->Avail_ = avail; + firstBlock->LastRecordBoundary_ = lastRecordBoundary; + BufMan_.FilledBlocks_.push(firstBlock); + BufMan_.FreeBlocks_.pop(); + } + + CondWrite_.Signal(); + } + + void Finish() override { + Stop(); + if (Error_.Defined()) { + YQL_LOG_CTX_THROW *Error_; + } + } + +private: + static void* ThreadFunc(void* param) { + static_cast<TAsyncBlockWriter*>(param)->Write(); + return nullptr; + } + + void Stop() { + with_lock(Mutex_) { + Shutdown_ = true; + } + + CondWrite_.Signal(); + Thread_.Join(); + } + + void Write() { + for (;;) { + TBlockPtr firstBlock; + bool needShutdown = false; + with_lock(Mutex_) { + while (BufMan_.FilledBlocks_.empty()) { + if (Shutdown_) { + needShutdown = true; + break; + } + + CondWrite_.WaitI(Mutex_); + } + + if (!needShutdown) { + firstBlock = BufMan_.FilledBlocks_.front(); + BufMan_.FilledBlocks_.pop(); + } + } + + if (needShutdown) { + try { + Target_.Finish(); + } + catch (yexception& e) { + auto guard = Guard(Mutex_); + Error_ = std::move(e); + } + catch (...) { + auto guard = Guard(Mutex_); + Error_ = (yexception() << CurrentExceptionMessage()); + } + return; + } + + try { + Target_.Write(firstBlock->Buffer_.Get(), firstBlock->LastRecordBoundary_.value_or(firstBlock->Avail_)); + if (firstBlock->LastRecordBoundary_) { + if (OnRecordBoundaryCallback_) { + OnRecordBoundaryCallback_(); + } + if (firstBlock->Avail_ > *firstBlock->LastRecordBoundary_) { + Target_.Write(firstBlock->Buffer_.Get() + *firstBlock->LastRecordBoundary_, firstBlock->Avail_ - *firstBlock->LastRecordBoundary_); + } + } + + } + catch (yexception& e) { + auto guard = Guard(Mutex_); + Error_ = std::move(e); + } + catch (...) { + auto guard = Guard(Mutex_); + Error_ = (yexception() << CurrentExceptionMessage()); + } + + with_lock(Mutex_) { + firstBlock->Avail_ = 0; + BufMan_.FreeBlocks_.push(firstBlock); + } + + CondFill_.Signal(); + } + } + +private: + IOutputStream& Target_; + TBufferManager BufMan_; + TMutex Mutex_; + TCondVar CondFill_; + TCondVar CondWrite_; + bool Shutdown_ = false; + TThread Thread_; + TMaybe<yexception> Error_; + std::function<void()> OnRecordBoundaryCallback_; +}; + +////////////////////////////////////////////////////////////////////////////////////////////////////////// + +class TSyncBlockWriter: public IBlockWriter { +public: + TSyncBlockWriter(IOutputStream& target, size_t blockSize) + : Target_(target) + , Block_(blockSize) + { + } + + void SetRecordBoundaryCallback(std::function<void()> callback) override { + OnRecordBoundaryCallback_ = std::move(callback); + } + + std::pair<char*, char*> NextEmptyBlock() override { + Block_.Avail_ = 0; + return { Block_.Buffer_.Get(), Block_.Buffer_.Get() + Block_.Capacity_ }; + } + + void ReturnBlock(size_t avail, std::optional<size_t> lastRecordBoundary) override { + if (!avail) { + return; + } + + Target_.Write(Block_.Buffer_.Get(), lastRecordBoundary.value_or(avail)); + if (lastRecordBoundary) { + if (OnRecordBoundaryCallback_) { + OnRecordBoundaryCallback_(); + } + if (avail > *lastRecordBoundary) { + Target_.Write(Block_.Buffer_.Get() + *lastRecordBoundary, avail - *lastRecordBoundary); + } + } + } + + void Finish() override { + Target_.Finish(); + Finished_ = true; + } + +private: + IOutputStream& Target_; + TBlock Block_; + bool Finished_ = false; + std::function<void()> OnRecordBoundaryCallback_; +}; + +////////////////////////////////////////////////////////////////////////////////////////////////////////// + +// TODO: share buffer manager between all writers (with at least <out table count> + 1 blocks) +THolder<IBlockWriter> MakeBlockWriter(IOutputStream& target, size_t blockCount, size_t blockSize) { + THolder<IBlockWriter> res; + if (blockCount > 1) { + res.Reset(new TAsyncBlockWriter(target, blockCount, blockSize)); + } else { + res.Reset(new TSyncBlockWriter(target, blockSize)); + } + return res; +} + +////////////////////////////////////////////////////////////////////////////////////////////////////////// + +namespace { + +inline void WriteWithLength(IOutputStream& out, TStringBuf val) { + ::NYson::WriteVarInt32(&out, val.size()); + out.Write(val.data(), val.size()); +} + +inline void WriteWithLength(IOutputStream& out, const NUdf::TUnboxedValue& val) { + WriteWithLength(out, val.AsStringRef()); +} + +void WriteRowItems(TMkqlIOCache& specsCache, size_t tableIndex, + const TVector<TString>& items, + const TMaybe<THashMap<TString, TString>>& others, + IOutputStream& out) +{ + auto& decoder = *specsCache.GetSpecs().Inputs[tableIndex]; + // Output + out.Write(BeginListSymbol); + for (ui32 index = 0; index < decoder.StructSize; ++index) { + if (!items[index].empty()) { + out.Write(items[index].data(), items[index].size()); + out.Write(ListItemSeparatorSymbol); + continue; + } + + if (decoder.OthersStructIndex && *decoder.OthersStructIndex == index) { + specsCache.UpdateMaxOthersFields(tableIndex, others->size()); + + out.Write(BeginListSymbol); + for (const auto& oField : *others) { + out.Write(BeginListSymbol); + out.Write(StringMarker); + WriteWithLength(out, oField.first); + out.Write(ListItemSeparatorSymbol); + out.Write(StringMarker); + WriteWithLength(out, oField.second); + out.Write(EndListSymbol); + + out.Write(ListItemSeparatorSymbol); + } + out.Write(EndListSymbol); + out.Write(ListItemSeparatorSymbol); + continue; + } + + // missing value + const auto& field = decoder.FieldsVec[index]; + if (field.Type->IsOptional() || field.Type->IsVoid() || field.Type->IsNull()) { + out.Write(EntitySymbol); + out.Write(ListItemSeparatorSymbol); + continue; + } + + const auto& defVal = decoder.DefaultValues[index]; + YQL_ENSURE(defVal, "Missing field data: " << field.Name); + + out.Write(StringMarker); + WriteWithLength(out, defVal); + out.Write(ListItemSeparatorSymbol); + } + out.Write(EndListSymbol); + out.Write(ListItemSeparatorSymbol); +} + +} // unnamed + +////////////////////////////////////////////////////////////////////////////////////////////////////////// +struct TMkqlReaderImpl::TDecoder { + TDecoder(TInputBuf& buf, const TMkqlIOSpecs& specs, const NKikimr::NMiniKQL::THolderFactory& holderFactory) + : Buf_(buf) + , SpecsCache_(specs, holderFactory) + { + } + + virtual ~TDecoder() = default; + virtual bool DecodeNext(NKikimr::NUdf::TUnboxedValue*& items, TMaybe<NKikimr::NMiniKQL::TValuesDictHashMap>& others) = 0; + + NUdf::TUnboxedValue ReadOtherField(char cmd) { + switch (cmd) { + case EntitySymbol: + return NUdf::TUnboxedValuePod::Embedded(TStringBuf("#")); + + case TrueMarker: + return NUdf::TUnboxedValuePod::Embedded(TStringBuf("%true")); + + case FalseMarker: + return NUdf::TUnboxedValuePod::Embedded(TStringBuf("%false")); + + case Int64Marker: { + auto val = Buf_.ReadVarI64(); + char format[100]; + auto len = ToString(val, format, Y_ARRAY_SIZE(format)); + return NUdf::TUnboxedValue(MakeString(NUdf::TStringRef(format, len))); + } + + case Uint64Marker: { + auto val = Buf_.ReadVarUI64(); + char format[100]; + auto len = ToString(val, format, Y_ARRAY_SIZE(format) - 1); + format[len] = 'u'; + return NUdf::TUnboxedValue(MakeString(NUdf::TStringRef(format, len + 1))); + } + + case StringMarker: { + const i32 length = Buf_.ReadVarI32(); + CHECK_STRING_LENGTH(length); + auto val = NUdf::TUnboxedValue(MakeStringNotFilled(length)); + const auto& buf = val.AsStringRef(); + Buf_.ReadMany(buf.Data(), buf.Size()); + return val; + } + + case DoubleMarker: { + double val; + Buf_.ReadMany((char*)&val, sizeof(val)); + char format[100]; + auto len = ToString(val, format, Y_ARRAY_SIZE(format)); + return NUdf::TUnboxedValue(MakeString(NUdf::TStringRef(format, len))); + } + + } + + auto& yson = Buf_.YsonBuffer(); + yson.clear(); + CopyYsonWithAttrs(cmd, Buf_, yson); + return NUdf::TUnboxedValue(MakeString(NUdf::TStringRef(yson.data(), yson.size()))); + } + + void EndStream() { + Finished_ = true; + Valid_ = false; + KeySwitch_ = false; + } + + void Reset(bool hasRangeIndices, ui32 tableIndex, bool ignoreStreamTableIndex) { + HasRangeIndices_ = hasRangeIndices; + TableIndex_ = tableIndex; + AtStart_ = true; + Valid_ = true; + Finished_ = false; + RowAlreadyRead_ = false; + RowIndex_.Clear(); + RangeIndex_.Clear(); + Row_.Clear(); + IgnoreStreamTableIndex = ignoreStreamTableIndex; + // Don't reset KeySwitch_ here, because this method is called when switching to a next table. It shouldn't touch key switch flag + } + +public: + bool HasRangeIndices_ = false; + bool AtStart_ = true; + bool Valid_ = true; + bool Finished_ = false; + bool RowAlreadyRead_ = false; + ui32 TableIndex_ = 0; + TMaybe<ui64> RowIndex_; + TMaybe<ui32> RangeIndex_; + NKikimr::NUdf::TUnboxedValue Row_; + bool IgnoreStreamTableIndex = false; + bool KeySwitch_ = true; + bool HandlesSysColumns_ = false; + +protected: + TInputBuf& Buf_; + TMkqlIOCache SpecsCache_; +}; + + +class TYsonDecoder: public TMkqlReaderImpl::TDecoder { +public: + TYsonDecoder(TInputBuf& buf, const TMkqlIOSpecs& specs, const NKikimr::NMiniKQL::THolderFactory& holderFactory) + : TMkqlReaderImpl::TDecoder(buf, specs, holderFactory) + { + } + + virtual ~TYsonDecoder() = default; + + bool DecodeNext(NUdf::TUnboxedValue*& items, TMaybe<TValuesDictHashMap>& others) final { + char cmd; + if (Finished_ || !Buf_.TryRead(cmd)) { + EndStream(); + return false; + } + + cmd = ReadAttrs(cmd); + if (!Valid_) { + return false; + } + + CHECK_EXPECTED(cmd, BeginMapSymbol); + AtStart_ = false; + auto& decoder = *SpecsCache_.GetSpecs().Inputs[TableIndex_]; + auto& lastFields = SpecsCache_.GetLastFields(TableIndex_); + Row_.Clear(); + Row_ = SpecsCache_.NewRow(TableIndex_, items); + if (decoder.OthersStructIndex) { + others.ConstructInPlace(SpecsCache_.GetMaxOthersFields(TableIndex_), + TValueHasher(decoder.OthersKeyTypes, false, nullptr), + TValueEqual(decoder.OthersKeyTypes, false, nullptr)); + } + + ui32 fieldNo = 0; + cmd = Buf_.Read(); + + for (;;) { + if (cmd == EndMapSymbol) { + break; + } + CHECK_EXPECTED(cmd, StringMarker); + auto name = Buf_.ReadYtString(2); // KeyValueSeparatorSymbol + subsequent cmd + EXPECTED(Buf_, KeyValueSeparatorSymbol); + const TMkqlIOSpecs::TDecoderSpec::TDecodeField* field = nullptr; + if (!decoder.OthersStructIndex || name != YqlOthersColumnName) { + if (fieldNo < lastFields.size()) { + auto lastField = lastFields[fieldNo]; + if (lastField->Name == name) { + field = lastField; + } + } + + if (!field) { + auto it = decoder.Fields.find(name); + if (it != decoder.Fields.end()) { + field = &it->second; + } + } + } + + if (!field) { + YQL_ENSURE(decoder.OthersStructIndex, "Unexpected field: " << name << " in strict scheme"); + cmd = Buf_.Read(); + // 'name' buffer may be invalidated in ReadOtherField() + auto key = NUdf::TUnboxedValue(MakeString(name)); + auto isStringKey = (cmd != StringMarker) ? NUdf::TUnboxedValue() : + NUdf::TUnboxedValue(MakeString(TStringBuilder() << "_yql_" << name)); + try { + auto value = ReadOtherField(cmd); + if (isStringKey) { + auto empty = NUdf::TUnboxedValue::Zero(); + others->emplace(std::move(isStringKey), std::move(empty)); + } + others->emplace(std::move(key), std::move(value)); + } + catch (const TYqlPanic& e) { + ythrow TYqlPanic() << "Failed to read field: '" << TStringBuf(key.AsStringRef()) << "'\n" << e.what(); + } + catch (...) { + ythrow yexception() << "Failed to read field: '" << TStringBuf(key.AsStringRef()) << "'\n" << CurrentExceptionMessage(); + } + } else { + try { + if (Y_LIKELY(field->StructIndex != Max<ui32>())) { + items[field->StructIndex] = ReadField(field->Type); + } else { + SkipField(field->Type); + } + } + catch (const TYqlPanic& e) { + ythrow TYqlPanic() << "Failed to read field: '" << name << "'\n" << e.what(); + } + catch (...) { + ythrow yexception() << "Failed to read field: '" << name << "'\n" << CurrentExceptionMessage(); + } + + if (fieldNo < lastFields.size()) { + lastFields[fieldNo] = field; + } + + ++fieldNo; + } + + cmd = Buf_.Read(); + if (cmd == KeyedItemSeparatorSymbol) { + cmd = Buf_.Read(); + } + } + + if (!Buf_.TryRead(cmd)) { + // don't EndStream here as it would throw away last record + // (GetRow throws on invalidated stream) + // instead it would be invalidated on next call + // Finished_ is set to not call TryRead next time + // because else TryRead causes an infinite loop + Finished_ = true; + return true; + } + + CHECK_EXPECTED(cmd, ListItemSeparatorSymbol); + + if (others) { + SpecsCache_.UpdateMaxOthersFields(TableIndex_, others->size()); + } + + return true; + } + +protected: + char ReadAttrs(char cmd) { + while (cmd == BeginAttributesSymbol) { + TMaybe<ui64> rowIndex; + TMaybe<ui32> rangeIndex; + cmd = Buf_.Read(); + + for (;;) { + if (cmd == EndAttributesSymbol) { + EXPECTED(Buf_, EntitySymbol); + // assume there's no control record at the end of the stream + EXPECTED(Buf_, ListItemSeparatorSymbol); + + if (Valid_) { + if (!Buf_.TryRead(cmd)) { + EndStream(); + } + } + + break; + } + + CHECK_EXPECTED(cmd, StringMarker); + auto name = Buf_.ReadYtString(); + if (name == TStringBuf("row_index")) { + EXPECTED(Buf_, KeyValueSeparatorSymbol); + EXPECTED(Buf_, Int64Marker); + rowIndex = Buf_.ReadVarI64(); + } + else if (name == TStringBuf("table_index")) { + EXPECTED(Buf_, KeyValueSeparatorSymbol); + EXPECTED(Buf_, Int64Marker); + const auto tableIndex = Buf_.ReadVarI64(); + if (!IgnoreStreamTableIndex) { + TableIndex_ = tableIndex; + YQL_ENSURE(TableIndex_ < SpecsCache_.GetSpecs().Inputs.size()); + } + } + else if (name == TStringBuf("key_switch")) { + EXPECTED(Buf_, KeyValueSeparatorSymbol); + EXPECTED(Buf_, TrueMarker); + if (!AtStart_) { + Valid_ = false; + KeySwitch_ = true; + } + } + else if (name == TStringBuf("range_index")) { + EXPECTED(Buf_, KeyValueSeparatorSymbol); + EXPECTED(Buf_, Int64Marker); + rangeIndex = Buf_.ReadVarI64(); + } + else { + YQL_ENSURE(false, "Unsupported annotation:" << name); + } + + cmd = Buf_.Read(); + if (cmd == ListItemSeparatorSymbol) { + cmd = Buf_.Read(); + } + } + if (rowIndex) { + if (HasRangeIndices_) { + if (rangeIndex) { + RowIndex_ = rowIndex; + RangeIndex_ = rangeIndex; + } + } else { + RowIndex_ = rowIndex; + } + } + + } + return cmd; + } + + NUdf::TUnboxedValue ReadField(TType* type) { + auto cmd = Buf_.Read(); + if (type->IsNull()) { + return NUdf::TUnboxedValue(); + } + + const bool isOptional = type->IsOptional(); + if (isOptional) { + TType* uwrappedType = static_cast<TOptionalType*>(type)->GetItemType(); + if (cmd == EntitySymbol) { + return NUdf::TUnboxedValue(); + } + auto& decoder = *SpecsCache_.GetSpecs().Inputs[TableIndex_]; + auto val = ReadYsonValue((decoder.NativeYtTypeFlags & ENativeTypeCompatFlags::NTCF_COMPLEX) ? type : uwrappedType, decoder.NativeYtTypeFlags, SpecsCache_.GetHolderFactory(), cmd, Buf_, true); + return (decoder.NativeYtTypeFlags & ENativeTypeCompatFlags::NTCF_COMPLEX) ? val : val.Release().MakeOptional(); + } else { + if (Y_LIKELY(cmd != EntitySymbol)) { + auto& decoder = *SpecsCache_.GetSpecs().Inputs[TableIndex_]; + return ReadYsonValue(type, decoder.NativeYtTypeFlags, SpecsCache_.GetHolderFactory(), cmd, Buf_, true); + } + + if (type->GetKind() == TType::EKind::Data && static_cast<TDataType*>(type)->GetSchemeType() == NUdf::TDataType<NUdf::TYson>::Id) { + return NUdf::TUnboxedValue::Embedded(TStringBuf("#")); + } + + // Don't fail right here because field can be filled from DefaultValues + return NUdf::TUnboxedValue(); + } + } + + void SkipField(TType* type) { + auto cmd = Buf_.Read(); + if (cmd != EntitySymbol) { + SkipValue(type->IsOptional() ? static_cast<TOptionalType*>(type)->GetItemType() : type, cmd); + } + } + + void SkipValue(TType* type, char cmd) { + switch (type->GetKind()) { + case TType::EKind::Variant: { + auto varType = static_cast<TVariantType*>(type); + CHECK_EXPECTED(cmd, BeginListSymbol); + cmd = Buf_.Read(); + YQL_ENSURE(cmd == Int64Marker || cmd == Uint64Marker, "Excepted [U]Int64 marker, but got: " << int(cmd)); + auto underlyingType = varType->GetUnderlyingType(); + YQL_ENSURE(underlyingType->IsTuple() || underlyingType->IsStruct(), "Wrong underlying type"); + TType* itemType; + i64 index; + + if (cmd == Int64Marker) { + index = Buf_.ReadVarI64(); + } else { + index = Buf_.ReadVarUI64(); + } + + YQL_ENSURE(index > -1 && index < varType->GetAlternativesCount(), "Bad variant alternative: " << index << ", only " << + varType->GetAlternativesCount() << " are available"); + if (underlyingType->IsTuple()) { + itemType = static_cast<TTupleType*>(underlyingType)->GetElementType(index); + } else { + itemType = static_cast<TStructType*>(underlyingType)->GetMemberType(index); + } + + + EXPECTED(Buf_, ListItemSeparatorSymbol); + cmd = Buf_.Read(); + SkipValue(itemType, cmd); + + cmd = Buf_.Read(); + if (cmd == ListItemSeparatorSymbol) { + cmd = Buf_.Read(); + } + + CHECK_EXPECTED(cmd, EndListSymbol); + break; + } + + case TType::EKind::Data: { + auto schemeType = static_cast<TDataType*>(type)->GetSchemeType(); + switch (schemeType) { + case NUdf::TDataType<bool>::Id: + YQL_ENSURE(cmd == FalseMarker || cmd == TrueMarker, "Expected either true or false, but got: " << cmd); + break; + + case NUdf::TDataType<ui8>::Id: + case NUdf::TDataType<ui16>::Id: + case NUdf::TDataType<ui32>::Id: + case NUdf::TDataType<ui64>::Id: + case NUdf::TDataType<NUdf::TDate>::Id: + case NUdf::TDataType<NUdf::TDatetime>::Id: + case NUdf::TDataType<NUdf::TTimestamp>::Id: + CHECK_EXPECTED(cmd, Uint64Marker); + Buf_.ReadVarUI64(); + break; + + case NUdf::TDataType<i8>::Id: + case NUdf::TDataType<i16>::Id: + case NUdf::TDataType<i32>::Id: + case NUdf::TDataType<i64>::Id: + case NUdf::TDataType<NUdf::TInterval>::Id: + case NUdf::TDataType<NUdf::TDate32>::Id: + case NUdf::TDataType<NUdf::TDatetime64>::Id: + case NUdf::TDataType<NUdf::TTimestamp64>::Id: + case NUdf::TDataType<NUdf::TInterval64>::Id: + CHECK_EXPECTED(cmd, Int64Marker); + Buf_.ReadVarI64(); + break; + + case NUdf::TDataType<float>::Id: + case NUdf::TDataType<double>::Id: + CHECK_EXPECTED(cmd, DoubleMarker); + Buf_.SkipMany(sizeof(double)); + break; + + case NUdf::TDataType<NUdf::TDecimal>::Id: + case NUdf::TDataType<NUdf::TUtf8>::Id: + case NUdf::TDataType<char*>::Id: + case NUdf::TDataType<NUdf::TJson>::Id: + case NUdf::TDataType<NUdf::TTzDate>::Id: + case NUdf::TDataType<NUdf::TTzDatetime>::Id: + case NUdf::TDataType<NUdf::TTzTimestamp>::Id: + case NUdf::TDataType<NUdf::TTzDate32>::Id: + case NUdf::TDataType<NUdf::TTzDatetime64>::Id: + case NUdf::TDataType<NUdf::TTzTimestamp64>::Id: + case NUdf::TDataType<NUdf::TDyNumber>::Id: + case NUdf::TDataType<NUdf::TUuid>::Id: + case NUdf::TDataType<NUdf::TJsonDocument>::Id: { + CHECK_EXPECTED(cmd, StringMarker); + const i32 length = Buf_.ReadVarI32(); + CHECK_STRING_LENGTH(length); + Buf_.SkipMany(length); + break; + } + + case NUdf::TDataType<NUdf::TYson>::Id: { + auto& yson = Buf_.YsonBuffer(); + yson.clear(); + CopyYsonWithAttrs(cmd, Buf_, yson); + break; + } + + default: + YQL_ENSURE(false, "Unsupported data type: " << schemeType); + } + break; + } + + case TType::EKind::Struct: { + auto structType = static_cast<TStructType*>(type); + YQL_ENSURE(cmd == BeginMapSymbol || cmd == BeginListSymbol); + if (cmd == BeginListSymbol) { + cmd = Buf_.Read(); + for (ui32 i = 0; i < structType->GetMembersCount(); ++i) { + SkipValue(structType->GetMemberType(i), cmd); + + cmd = Buf_.Read(); + if (cmd == ListItemSeparatorSymbol) { + cmd = Buf_.Read(); + } + } + CHECK_EXPECTED(cmd, EndMapSymbol); + break; + } + + cmd = Buf_.Read(); + for (ui32 i = 0; i < structType->GetMembersCount(); ++i) { + CHECK_EXPECTED(cmd, StringMarker); + const i32 length = Buf_.ReadVarI32(); + CHECK_STRING_LENGTH(length); + TString name(length, '\0'); + Buf_.ReadMany((char*)name.data(), length); + cmd = Buf_.Read(); + CHECK_EXPECTED(cmd, KeyValueSeparatorSymbol); + + auto idx = structType->FindMemberIndex(name); + YQL_ENSURE(idx); + cmd = Buf_.Read(); + SkipValue(structType->GetMemberType(*idx), cmd); + + cmd = Buf_.Read(); + if (cmd == ListItemSeparatorSymbol) { + cmd = Buf_.Read(); + } + } + CHECK_EXPECTED(cmd, EndMapSymbol); + break; + } + + case TType::EKind::List: { + auto itemType = static_cast<TListType*>(type)->GetItemType(); + CHECK_EXPECTED(cmd, BeginListSymbol); + cmd = Buf_.Read(); + + for (;;) { + if (cmd == EndListSymbol) { + break; + } + + SkipValue(itemType, cmd); + cmd = Buf_.Read(); + if (cmd == ListItemSeparatorSymbol) { + cmd = Buf_.Read(); + } + } + break; + } + + case TType::EKind::Optional: { + if (cmd == EntitySymbol) { + return; + } + + CHECK_EXPECTED(cmd, BeginListSymbol); + cmd = Buf_.Read(); + if (cmd == EndListSymbol) { + return; + } + + SkipValue(static_cast<TOptionalType*>(type)->GetItemType(), cmd); + + cmd = Buf_.Read(); + if (cmd == ListItemSeparatorSymbol) { + cmd = Buf_.Read(); + } + + CHECK_EXPECTED(cmd, EndListSymbol); + break; + } + + case TType::EKind::Dict: { + auto dictType = static_cast<TDictType*>(type); + auto keyType = dictType->GetKeyType(); + auto payloadType = dictType->GetPayloadType(); + CHECK_EXPECTED(cmd, BeginListSymbol); + cmd = Buf_.Read(); + + for (;;) { + if (cmd == EndListSymbol) { + break; + } + + CHECK_EXPECTED(cmd, BeginListSymbol); + cmd = Buf_.Read(); + SkipValue(keyType, cmd); + EXPECTED(Buf_, ListItemSeparatorSymbol); + cmd = Buf_.Read(); + SkipValue(payloadType, cmd); + + cmd = Buf_.Read(); + // skip inner list separator + if (cmd == ListItemSeparatorSymbol) { + cmd = Buf_.Read(); + } + + CHECK_EXPECTED(cmd, EndListSymbol); + + cmd = Buf_.Read(); + // skip outer list separator + if (cmd == ListItemSeparatorSymbol) { + cmd = Buf_.Read(); + } + } + break; + } + + case TType::EKind::Tuple: { + auto tupleType = static_cast<TTupleType*>(type); + CHECK_EXPECTED(cmd, BeginListSymbol); + cmd = Buf_.Read(); + + for (ui32 i = 0; i < tupleType->GetElementsCount(); ++i) { + SkipValue(tupleType->GetElementType(i), cmd); + + cmd = Buf_.Read(); + if (cmd == ListItemSeparatorSymbol) { + cmd = Buf_.Read(); + } + + } + + CHECK_EXPECTED(cmd, EndListSymbol); + break; + } + + case TType::EKind::Void: { + if (cmd == EntitySymbol) { + return; + } + + CHECK_EXPECTED(cmd, StringMarker); + i32 length = Buf_.ReadVarI32(); + YQL_ENSURE(length == 4, "Expected Void"); + char buffer[4]; + Buf_.ReadMany(buffer, 4); + YQL_ENSURE(TStringBuf(buffer, 4) == TStringBuf("Void"), "Expected Void"); + break; + } + + default: + YQL_ENSURE(false, "Unsupported type: " << type->GetKindAsStr()); + } + } +}; + +class TSkiffDecoderBase: public TMkqlReaderImpl::TDecoder { +public: + TSkiffDecoderBase(TInputBuf& buf, const TMkqlIOSpecs& specs, const NKikimr::NMiniKQL::THolderFactory& holderFactory) + : TMkqlReaderImpl::TDecoder(buf, specs, holderFactory) + { + } + + bool DecodeNext(NUdf::TUnboxedValue*& items, TMaybe<TValuesDictHashMap>& others) final { + char byte1; + if (!Buf_.TryRead(byte1)) { + EndStream(); + return false; + } + + char byte2 = Buf_.Read(); + if (!IgnoreStreamTableIndex) { + ui16 tableIndex = (ui16(ui8(byte2)) << 8) | ui8(byte1); + TableIndex_ = tableIndex; + } + auto& spec = SpecsCache_.GetSpecs(); + YQL_ENSURE(TableIndex_ < spec.Inputs.size()); + + auto& decoder = *spec.Inputs[TableIndex_]; + + if (spec.SystemFields_.HasFlags(TMkqlIOSpecs::ESystemField::RangeIndex)) { + auto cmd = Buf_.Read(); + if (cmd) { + i64 value; + Buf_.ReadMany((char*)&value, sizeof(value)); + YQL_ENSURE(HasRangeIndices_); + RangeIndex_ = value; + } + } + + if (decoder.Dynamic) { + RowIndex_.Clear(); + } else if (spec.SystemFields_.HasFlags(TMkqlIOSpecs::ESystemField::RowIndex)) { + auto cmd = Buf_.Read(); + if (cmd) { + i64 value; + Buf_.ReadMany((char*)&value, sizeof(value)); + RowIndex_ = value; + } + } + + if (spec.SystemFields_.HasFlags(TMkqlIOSpecs::ESystemField::KeySwitch)) { + auto cmd = Buf_.Read(); + if (!AtStart_ && cmd) { + Valid_ = false; + RowAlreadyRead_ = true; + KeySwitch_ = true; + } + } + + AtStart_ = false; + + Row_ = NUdf::TUnboxedValue(); + Row_ = SpecsCache_.NewRow(TableIndex_, items); + + DecodeItems(items); + + if (decoder.OthersStructIndex) { + // parse yson with other fields + try { + others.ConstructInPlace(SpecsCache_.GetMaxOthersFields(TableIndex_), + TValueHasher(decoder.OthersKeyTypes, false, nullptr), + TValueEqual(decoder.OthersKeyTypes, false, nullptr)); + ReadOtherSkiffFields(*others); + SpecsCache_.UpdateMaxOthersFields(TableIndex_, others->size()); + } catch (const TYqlPanic& e) { + ythrow TYqlPanic() << "Failed to read others\n" << e.what(); + } catch (...) { + ythrow yexception() << "Failed to read others\n" << CurrentExceptionMessage(); + } + } + return true; + } + +protected: + virtual void DecodeItems(NUdf::TUnboxedValue* items) = 0; + + void ReadOtherSkiffFields(TValuesDictHashMap& others) { + ui32 size; + Buf_.ReadMany((char*)&size, sizeof(size)); + YQL_ENSURE(size > 0); + EXPECTED(Buf_, BeginMapSymbol); + char cmd = Buf_.Read(); + + for (;;) { + if (cmd == EndMapSymbol) { + break; + } + + auto name = Buf_.ReadYtString(2); // KeyValueSeparatorSymbol + subsequent cmd + EXPECTED(Buf_, KeyValueSeparatorSymbol); + cmd = Buf_.Read(); + // 'name' buffer may be invalidated in ReadOtherField() + auto key = NUdf::TUnboxedValue(MakeString(name)); + auto isStringKey = (cmd != StringMarker) ? NUdf::TUnboxedValue() : + NUdf::TUnboxedValue(MakeString(TStringBuilder() << "_yql_" << name)); + try { + auto value = ReadOtherField(cmd); + if (isStringKey) { + auto empty = NUdf::TUnboxedValue::Zero(); + others.emplace(std::move(isStringKey), std::move(empty)); + } + others.emplace(std::move(key), std::move(value)); + } catch (const TYqlPanic& e) { + ythrow TYqlPanic() << "Failed to read field: '" << TStringBuf(key.AsStringRef()) << "'\n" << e.what(); + } catch (...) { + ythrow yexception() << "Failed to read field: '" << TStringBuf(key.AsStringRef()) << "'\n" << CurrentExceptionMessage(); + } + + cmd = Buf_.Read(); + if (cmd == KeyedItemSeparatorSymbol) { + cmd = Buf_.Read(); + } + } + } +}; + +class TSkiffDecoder: public TSkiffDecoderBase { +public: + TSkiffDecoder(TInputBuf& buf, const TMkqlIOSpecs& specs, const NKikimr::NMiniKQL::THolderFactory& holderFactory) + : TSkiffDecoderBase(buf, specs, holderFactory) + { + } + +protected: + void DecodeItems(NUdf::TUnboxedValue* items) final { + auto& decoder = *SpecsCache_.GetSpecs().Inputs[TableIndex_]; + for (ui32 i = 0; i < decoder.SkiffSize; ++i) { + if (decoder.OthersStructIndex && i == *decoder.OthersStructIndex) { + continue; + } + + if (decoder.FieldsVec[i].Virtual) { + items[i] = NUdf::TUnboxedValuePod::Zero(); + continue; + } + + try { + if (Y_UNLIKELY(decoder.FieldsVec[i].StructIndex == Max<ui32>())) { + SkipSkiffField(decoder.FieldsVec[i].Type, decoder.NativeYtTypeFlags); + } else if (decoder.NativeYtTypeFlags && !decoder.FieldsVec[i].ExplicitYson) { + items[i] = ReadSkiffFieldNativeYt(decoder.FieldsVec[i].Type, decoder.NativeYtTypeFlags); + } else if (decoder.DefaultValues[i]) { + auto val = ReadSkiffField(decoder.FieldsVec[i].Type, true); + items[i] = val ? NUdf::TUnboxedValue(val.Release().GetOptionalValue()) : decoder.DefaultValues[i]; + } else { + items[i] = ReadSkiffField(decoder.FieldsVec[i].Type, false); + } + } catch (const TYqlPanic& e) { + ythrow TYqlPanic() << "Failed to read field: '" << decoder.FieldsVec[i].Name << "'\n" << e.what(); + } catch (...) { + ythrow yexception() << "Failed to read field: '" << decoder.FieldsVec[i].Name << "'\n" << CurrentExceptionMessage(); + } + } + } + + NUdf::TUnboxedValue ReadSkiffField(TType* type, bool withDefVal) { + const bool isOptional = withDefVal || type->IsOptional(); + TType* uwrappedType = type; + if (type->IsOptional()) { + uwrappedType = static_cast<TOptionalType*>(type)->GetItemType(); + } + + if (isOptional) { + auto marker = Buf_.Read(); + if (!marker) { + return NUdf::TUnboxedValue(); + } + } + + if (uwrappedType->IsData()) { + return NCommon::ReadSkiffData(uwrappedType, 0, Buf_); + } else if (!isOptional && uwrappedType->IsPg()) { + return NCommon::ReadSkiffPg(static_cast<TPgType*>(uwrappedType), Buf_); + } else { + // yson content + ui32 size; + Buf_.ReadMany((char*)&size, sizeof(size)); + CHECK_STRING_LENGTH_UNSIGNED(size); + // parse binary yson... + YQL_ENSURE(size > 0); + char cmd = Buf_.Read(); + auto value = ReadYsonValue(uwrappedType, 0, SpecsCache_.GetHolderFactory(), cmd, Buf_, true); + return isOptional ? value.Release().MakeOptional() : value; + } + } + + NUdf::TUnboxedValue ReadSkiffFieldNativeYt(TType* type, ui64 nativeYtTypeFlags) { + return NCommon::ReadSkiffNativeYtValue(type, nativeYtTypeFlags, SpecsCache_.GetHolderFactory(), Buf_); + } + + void SkipSkiffField(TType* type, ui64 nativeYtTypeFlags) { + return NCommon::SkipSkiffField(type, nativeYtTypeFlags, Buf_); + } +}; + +#ifndef MKQL_DISABLE_CODEGEN + +class TSkiffLLVMDecoder: public TSkiffDecoderBase { +public: + TSkiffLLVMDecoder(TInputBuf& buf, const TMkqlIOSpecs& specs, const NKikimr::NMiniKQL::THolderFactory& holderFactory) + : TSkiffDecoderBase(buf, specs, holderFactory) + { + Codegen_ = NCodegen::ICodegen::Make(NCodegen::ETarget::Native); + Codegen_->LoadBitCode(GetYtCodecBitCode(), "YtCodecFuncs"); + TVector<llvm::Function*> funcs; + THashMap<const TMkqlIOSpecs::TDecoderSpec*, llvm::Function*> processedDecoders; + for (auto x : specs.Inputs) { + llvm::Function*& func = processedDecoders[x]; + if (!func) { + auto rowReaderBuilder = MakeYtCodecCgReader(Codegen_, holderFactory, x); + for (ui32 i = 0; i < x->SkiffSize; ++i) { + if (x->OthersStructIndex && i == *x->OthersStructIndex) { + rowReaderBuilder->SkipOther(); + continue; + } + + if (x->FieldsVec[i].Virtual) { + rowReaderBuilder->SkipVirtual(); + continue; + } + + if (x->FieldsVec[i].StructIndex != Max<ui32>()) { + rowReaderBuilder->AddField(x->FieldsVec[i].Type, x->DefaultValues[i], x->FieldsVec[i].ExplicitYson ? 0 : x->NativeYtTypeFlags); + } else { + rowReaderBuilder->SkipField(x->FieldsVec[i].Type, x->NativeYtTypeFlags); + } + } + + func = rowReaderBuilder->Build(); + } + funcs.push_back(func); + } + + Codegen_->Verify(); + YtCodecAddMappings(*Codegen_); + Codegen_->Compile(); + for (const auto& func : funcs) { + RowReaders_.push_back((TRowReader)Codegen_->GetPointerToFunction(func)); + } + } + +protected: + void DecodeItems(NUdf::TUnboxedValue* items) final { + RowReaders_.at(TableIndex_)(items, Buf_); + } + + NCodegen::ICodegen::TPtr Codegen_; + typedef void(*TRowReader)(NKikimr::NUdf::TUnboxedValue*, TInputBuf&); + TVector<TRowReader> RowReaders_; +}; + +#endif + +class TArrowDecoder : public TMkqlReaderImpl::TDecoder { +public: + TArrowDecoder( + TInputBuf& buf, + const TMkqlIOSpecs& specs, + const NKikimr::NMiniKQL::THolderFactory& holderFactory, + arrow::MemoryPool* pool + ) + : TMkqlReaderImpl::TDecoder(buf, specs, holderFactory) + , Specs_(specs) + , Pool_(pool) + { + InputStream_ = std::make_unique<TInputBufArrowInputStream>(buf, pool); + ResetColumnConverters(); + + HandlesSysColumns_ = true; + } + + bool DecodeNext(NKikimr::NUdf::TUnboxedValue*& items, TMaybe<NKikimr::NMiniKQL::TValuesDictHashMap>&) override { + YQL_ENSURE(!RangeIndex_); + AtStart_ = false; + + if (Chunks_.empty()) { + bool read = ReadNext(); + if (!read) { + EndStream(); + return false; + } + + YQL_ENSURE(!Chunks_.empty()); + } + + auto& inputFields = SpecsCache_.GetSpecs().Inputs[TableIndex_]->FieldsVec; + Row_ = SpecsCache_.NewRow(TableIndex_, items, true); + + auto& [chunkRowIndex, chunkLen, chunk] = Chunks_.front(); + for (size_t i = 0; i < inputFields.size(); i++) { + items[inputFields[i].StructIndex] = SpecsCache_.GetHolderFactory().CreateArrowBlock(std::move(chunk[i])); + } + items[inputFields.size()] = SpecsCache_.GetHolderFactory().CreateArrowBlock(arrow::Datum(static_cast<uint64_t>(chunkLen))); + RowIndex_ = chunkRowIndex; + + Chunks_.pop_front(); + return true; + } + + bool ReadNext() { + if (!StreamReader_) { + auto streamReaderResult = arrow::ipc::RecordBatchStreamReader::Open(InputStream_.get()); + if (!streamReaderResult.ok() && InputStream_->EOSReached() && InputStream_->Tell().ValueOrDie() == 0) { + // Workaround for YT-23495 + return false; + } + StreamReader_ = ARROW_RESULT(streamReaderResult); + + auto oldTableIndex = TableIndex_; + if (!IgnoreStreamTableIndex) { + auto tableIdKey = StreamReader_->schema()->metadata()->Get("TableId"); + if (tableIdKey.ok()) { + TableIndex_ = std::stoi(tableIdKey.ValueOrDie()); + YQL_ENSURE(TableIndex_ < Specs_.Inputs.size()); + } + } + + if (TableIndex_ != oldTableIndex) { + ResetColumnConverters(); + } + } + + std::shared_ptr<arrow::RecordBatch> batch; + ARROW_OK(StreamReader_->ReadNext(&batch)); + if (!batch) { + if (InputStream_->EOSReached()) { + return false; + } + + // InputStream EOS hasn't reached yet - next Arrow IPC stream must be present + StreamReader_.reset(); + return ReadNext(); + } + + auto& decoder = *Specs_.Inputs[TableIndex_]; + auto& inputFields = decoder.FieldsVec; + YQL_ENSURE(inputFields.size() == ColumnConverters_.size()); + + auto rowIndices = batch->GetColumnByName("$row_index"); + YQL_ENSURE(rowIndices || decoder.Dynamic); + + arrow::compute::ExecContext execContext(Pool_); + std::vector<arrow::Datum> convertedBatch; + for (size_t i = 0; i < inputFields.size(); i++) { + auto batchColumn = batch->GetColumnByName(inputFields[i].Name); + if (!batchColumn) { + arrow::Datum convertedColumn; + + if (decoder.FillSysColumnPath == inputFields[i].StructIndex) { + auto tableName = Specs_.TableNames.at(TableIndex_).AsStringRef(); + auto tableNameScalar = arrow::BinaryScalar(std::make_shared<arrow::Buffer>(reinterpret_cast<const uint8_t*>(tableName.Data()), tableName.Size())); + convertedColumn = ARROW_RESULT(arrow::MakeArrayFromScalar(tableNameScalar, batch->num_rows(), Pool_)); + + } else if (decoder.FillSysColumnRecord == inputFields[i].StructIndex || decoder.FillSysColumnNum == inputFields[i].StructIndex) { + if (rowIndices) { + auto addFirst = ARROW_RESULT(arrow::compute::Cast(rowIndices, arrow::uint64(), arrow::compute::CastOptions::Safe(), &execContext)); + auto addSecond = arrow::Datum(std::make_shared<arrow::UInt64Scalar>(1)); + convertedColumn = ARROW_RESULT(arrow::compute::Add(addFirst, addSecond, arrow::compute::ArithmeticOptions(), &execContext)); + + if (decoder.FillSysColumnNum == inputFields[i].StructIndex) { + auto addThird = arrow::Datum(std::make_shared<arrow::UInt64Scalar>(Specs_.TableOffsets.at(TableIndex_))); + convertedColumn = ARROW_RESULT(arrow::compute::Add(convertedColumn, addThird, arrow::compute::ArithmeticOptions(), &execContext)); + } + } else { + convertedColumn = ARROW_RESULT(arrow::MakeArrayFromScalar(arrow::UInt64Scalar(0), batch->num_rows(), Pool_)); + } + } else if (decoder.FillSysColumnIndex == inputFields[i].StructIndex) { + convertedColumn = ARROW_RESULT(arrow::MakeArrayFromScalar(arrow::UInt32Scalar(TableIndex_), batch->num_rows())); + } else { + YQL_ENSURE(false, "unexpected column: " << inputFields[i].Name); + } + + convertedBatch.emplace_back(convertedColumn); + continue; + } + + convertedBatch.emplace_back(ColumnConverters_[i]->Convert(batchColumn->data())); + } + + // index of the first row in the block + ui64 blockRowIndex = rowIndices ? std::dynamic_pointer_cast<arrow::Int64Scalar>(ARROW_RESULT(rowIndices->GetScalar(0)))->value : 0; + + NUdf::TArgsDechunker dechunker(std::move(convertedBatch)); + std::vector<arrow::Datum> chunk; + ui64 chunkLen = 0; + while (dechunker.Next(chunk, chunkLen)) { + Chunks_.emplace_back(rowIndices ? blockRowIndex : 0, chunkLen, std::move(chunk)); + blockRowIndex += chunkLen; + } + + return true; + } + + void ResetColumnConverters() { + auto& fields = Specs_.Inputs[TableIndex_]->FieldsVec; + ColumnConverters_.clear(); + ColumnConverters_.reserve(fields.size()); + for (auto& field: fields) { + YQL_ENSURE(!field.Type->IsPg()); + bool native = Specs_.Inputs[TableIndex_]->NativeYtTypeFlags && !field.ExplicitYson; + ColumnConverters_.emplace_back(MakeYtColumnConverter(field.Type, nullptr, *Pool_, native)); + } + } + +private: + std::unique_ptr<TInputBufArrowInputStream> InputStream_; + std::shared_ptr<arrow::ipc::RecordBatchStreamReader> StreamReader_; + std::vector<std::unique_ptr<IYtColumnConverter>> ColumnConverters_; + + TDeque<std::tuple<ui64, ui64, std::vector<arrow::Datum>>> Chunks_; + + const TMkqlIOSpecs& Specs_; + arrow::MemoryPool* Pool_; +}; + +////////////////////////////////////////////////////////////////////////////////////////////////////////// + +TMkqlReaderImpl::TMkqlReaderImpl(NYT::TRawTableReader& source, size_t blockCount, size_t blockSize, ui32 tableIndex, bool ignoreStreamTableIndex) + : TMkqlReaderImpl() +{ + SetReader(source, blockCount, blockSize, tableIndex, ignoreStreamTableIndex); +} + +TMkqlReaderImpl::TMkqlReaderImpl() + : TimerDecode_(InputDecodeTime, 100) + , TimerRead_(InputReadTime, 100) + , Buf_(&TimerRead_) +{ +} + +TMkqlReaderImpl::~TMkqlReaderImpl() { +} + +void TMkqlReaderImpl::SetReader(NYT::TRawTableReader& source, size_t blockCount, size_t blockSize, ui32 tableIndex, bool ignoreStreamTableIndex, TMaybe<ui64> currentRow, TMaybe<ui32> currentRange) { + Reader_ = MakeBlockReader(source, blockCount, blockSize); + Buf_.SetSource(*Reader_); + HasRangeIndices_ = source.HasRangeIndices(); + InitialTableIndex_ = tableIndex; + IgnoreStreamTableIndex_ = ignoreStreamTableIndex; + if (Decoder_) { + Decoder_->Reset(HasRangeIndices_, InitialTableIndex_, IgnoreStreamTableIndex_); + if (currentRow) { + Decoder_->RowIndex_ = currentRow; + } + if (currentRange) { + Decoder_->RangeIndex_ = currentRange; + } + } +} + +void TMkqlReaderImpl::SetSpecs(const TMkqlIOSpecs& specs, const NKikimr::NMiniKQL::THolderFactory& holderFactory) { + Specs_ = &specs; + HolderFactoryPtr = &holderFactory; + JobStats_ = specs.JobStats_; + Buf_.SetStats(JobStats_); + if (Specs_->UseBlockInput_) { + Decoder_.Reset(new TArrowDecoder(Buf_, *Specs_, holderFactory, NUdf::GetYqlMemoryPool())); + } else if (Specs_->UseSkiff_) { +#ifndef MKQL_DISABLE_CODEGEN + if (Specs_->OptLLVM_ != "OFF" && NCodegen::ICodegen::IsCodegenAvailable()) { + Decoder_.Reset(new TSkiffLLVMDecoder(Buf_, *Specs_, holderFactory)); + } + else +#endif + { + Decoder_.Reset(new TSkiffDecoder(Buf_, *Specs_, holderFactory)); + } + } else { + Decoder_.Reset(new TYsonDecoder(Buf_, *Specs_, holderFactory)); + } + Decoder_->TableIndex_ = InitialTableIndex_; + Decoder_->HasRangeIndices_ = HasRangeIndices_; + Decoder_->IgnoreStreamTableIndex = IgnoreStreamTableIndex_; +} + +void TMkqlReaderImpl::Finish() { + if (Decoder_) { + Decoder_->Row_.Clear(); + } + TimerDecode_.Report(JobStats_); + TimerRead_.Report(JobStats_); +} + +void TMkqlReaderImpl::OnError(const std::exception_ptr& error, TStringBuf msg) { + YQL_LOG(ERROR) << "Reader error: " << msg; + Buf_.Reset(); + if (!Reader_->Retry(Decoder_->RangeIndex_, Decoder_->RowIndex_, error)) { + ythrow yexception() << "Failed to read row, table index: " << Decoder_->TableIndex_ << ", row index: " << + (Decoder_->RowIndex_.Defined() ? ToString(*Decoder_->RowIndex_) : "?") << "\n" << msg; + } +} + +NUdf::TUnboxedValue TMkqlReaderImpl::GetRow() const { + CheckValidity(); + CheckReadRow(); + return Decoder_->Row_; +} + +bool TMkqlReaderImpl::IsValid() const { + return Reader_ && Decoder_ && Decoder_->Valid_; +} + +TMaybe<ui64> TMkqlReaderImpl::GetRowIndexUnchecked() const { + return Decoder_ ? Decoder_->RowIndex_ : TMaybe<ui64>(); +} + +TMaybe<ui32> TMkqlReaderImpl::GetRangeIndexUnchecked() const { + return Decoder_ ? Decoder_->RangeIndex_ : TMaybe<ui32>(); +} + +ui32 TMkqlReaderImpl::GetTableIndex() const { + CheckValidity(); + CheckReadRow(); + return Decoder_->TableIndex_; +} + +ui32 TMkqlReaderImpl::GetRangeIndex() const { + CheckValidity(); + CheckReadRow(); + return Decoder_->RangeIndex_.GetOrElse(0); +} + +ui64 TMkqlReaderImpl::GetRowIndex() const { + CheckValidity(); + CheckReadRow(); + return Decoder_->RowIndex_.GetOrElse(0UL); +} + +void TMkqlReaderImpl::Next() { + auto guard = Guard(TimerDecode_); + CheckValidity(); + + if (Decoder_->RowAlreadyRead_) { + Decoder_->RowAlreadyRead_ = false; + return; + } + + if (Decoder_->RowIndex_ && !Decoder_->HandlesSysColumns_) { + ++*Decoder_->RowIndex_; + } + + NUdf::TUnboxedValue* items = nullptr; + TMaybe<TValuesDictHashMap> others; + + // Retrieable part + while (true) { + try { + if (Decoder_->DecodeNext(items, others)) { + break; // Retry loop + } else { + return; + } + } catch (const TYqlPanic& e) { + ythrow TYqlPanic() << "Failed to read row, table index: " << Decoder_->TableIndex_ << ", row index: " << + (Decoder_->RowIndex_.Defined() ? ToString(*Decoder_->RowIndex_) : "?") << "\n" << e.what(); + } catch (const TTimeoutException&) { + throw; + } catch (const yexception& e) { + OnError(std::make_exception_ptr(e), e.AsStrBuf()); + } catch (...) { + OnError(std::current_exception(), CurrentExceptionMessage()); + } + } + + // Unretrieable part + auto& decoder = *Specs_->Inputs[Decoder_->TableIndex_]; + if (Specs_->UseSkiff_) { + if (decoder.OthersStructIndex && *decoder.OthersStructIndex != Max<ui32>()) { + items[*decoder.OthersStructIndex] = BuildOthers(decoder, *others); + } + } else { + for (ui32 index = 0; index < decoder.StructSize; ++index) { + if (items[index]) { + continue; + } + + if (decoder.OthersStructIndex && *decoder.OthersStructIndex == index) { + items[index] = BuildOthers(decoder, *others); + continue; + } + + // missing value + const auto& field = decoder.FieldsVec[index]; + if (field.Type->IsOptional() || field.Type->IsNull() || field.Type->IsPg()) { + items[index] = NUdf::TUnboxedValuePod(); + continue; + } + + if (field.Type->IsVoid()) { + items[index] = NUdf::TUnboxedValue::Void(); + continue; + } + + if (field.Virtual) { + items[index] = NUdf::TUnboxedValue::Zero(); + continue; + } + + const auto& defVal = decoder.DefaultValues[index]; + YQL_ENSURE(defVal, "Failed to read row, table index: " << Decoder_->TableIndex_ << ", row index: " << + (Decoder_->RowIndex_.Defined() ? ToString(*Decoder_->RowIndex_) : "?") << ": missing field data: " << field.Name); + items[index] = defVal; + } + } + + if (Decoder_->HandlesSysColumns_) { + return; + } + + if (decoder.FillSysColumnPath) { + items[*decoder.FillSysColumnPath] = Specs_->TableNames.at(Decoder_->TableIndex_); + } + if (decoder.FillSysColumnIndex) { + items[*decoder.FillSysColumnIndex] = NUdf::TUnboxedValuePod(Decoder_->TableIndex_); + } + if (decoder.FillSysColumnKeySwitch) { + items[*decoder.FillSysColumnKeySwitch] = NUdf::TUnboxedValuePod(Decoder_->KeySwitch_); + } + if (auto row = Decoder_->RowIndex_) { + if (decoder.FillSysColumnRecord) { + items[*decoder.FillSysColumnRecord] = NUdf::TUnboxedValuePod(*row + 1); + } + if (decoder.FillSysColumnNum) { + items[*decoder.FillSysColumnNum] = NUdf::TUnboxedValuePod(Specs_->TableOffsets.at(Decoder_->TableIndex_) + *row + 1); + } + } + Decoder_->KeySwitch_ = false; +} + +NUdf::TUnboxedValue TMkqlReaderImpl::BuildOthers(const TMkqlIOSpecs::TDecoderSpec& decoder, TValuesDictHashMap& others) { + if (others.empty()) { + return HolderFactoryPtr->GetEmptyContainerLazy(); + } else { + auto filler = [&others](TValuesDictHashMap& map) { + map.swap(others); + }; + + return HolderFactoryPtr->CreateDirectHashedDictHolder(filler, decoder.OthersKeyTypes, false, true, nullptr, nullptr, nullptr); + } +} + +void TMkqlReaderImpl::NextKey() { + while (Decoder_->Valid_) { + Next(); + } + + if (Decoder_->Finished_) { + return; + } + + Decoder_->Valid_ = true; + if (Decoder_->RowIndex_ && !Decoder_->RowAlreadyRead_) { + --*Decoder_->RowIndex_; + } +} + +void TMkqlReaderImpl::CheckValidity() const { + YQL_ENSURE(Decoder_ && Decoder_->Valid_, "Iterator is not valid"); +} + +void TMkqlReaderImpl::CheckReadRow() const { + YQL_ENSURE(Decoder_ && !Decoder_->AtStart_, "Next() must be called"); +} + +////////////////////////////////////////////////////////////////////////////////////////////////////////// + +struct TMkqlWriterImpl::TEncoder { + TEncoder(TOutputBuf& buf, const TMkqlIOSpecs& specs) + : Buf_(buf) + , Specs_(specs) + { + } + virtual ~TEncoder() = default; + + virtual void EncodeNext(const NUdf::TUnboxedValuePod row) = 0; + virtual void EncodeNext(const NUdf::TUnboxedValuePod* row) = 0; + +protected: + struct TField { + TStringBuf Name; + TType* Type; + bool Optional; + }; + + static TVector<TField> GetFields(TStructType* type, const TVector<TString>& columns = {}) { + TVector<TField> res; + + THashMap<TString, ui32> columnsOrder; + if (columns.size() > 0) { + for (ui32 index = 0; index < columns.size(); index++) { + columnsOrder[columns[index]] = index; + } + res.resize(columns.size()); + } else { + res.reserve(type->GetMembersCount()); + } + + for (ui32 index = 0; index < type->GetMembersCount(); ++index) { + auto name = type->GetMemberName(index); + auto fieldType = type->GetMemberType(index); + const bool isOptional = fieldType->IsOptional(); + if (isOptional) { + fieldType = static_cast<TOptionalType*>(fieldType)->GetItemType(); + } + + if (!columnsOrder.empty() && columnsOrder.contains(name)) { + res[columnsOrder[name]] = TField{name, fieldType, isOptional}; + } else { + res.push_back(TField{name, fieldType, isOptional}); + } + } + return res; + } + +protected: + TOutputBuf& Buf_; + const TMkqlIOSpecs& Specs_; +}; + +class TYsonEncoder: public TMkqlWriterImpl::TEncoder { +public: + TYsonEncoder(TOutputBuf& buf, const TMkqlIOSpecs& specs, size_t tableIndex) + : TMkqlWriterImpl::TEncoder(buf, specs) + { + Fields_ = GetFields(Specs_.Outputs[tableIndex].RowType); + NativeYtTypeFlags_ = Specs_.Outputs[tableIndex].NativeYtTypeFlags; + } + + void EncodeNext(const NUdf::TUnboxedValuePod row) final { + Buf_.Write(BeginMapSymbol); + + for (size_t index = 0; index < Fields_.size(); ++index) { + const TField& field = Fields_[index]; + auto value = row.GetElement(index); + if (field.Optional || field.Type->GetKind() == TTypeBase::EKind::Pg) { + if (!value) { + continue; + } + value = value.Release().GetOptionalValue(); + } + + Buf_.Write(StringMarker); + Buf_.WriteVarI32(field.Name.size()); + Buf_.WriteMany(field.Name.data(), field.Name.size()); + Buf_.Write(KeyValueSeparatorSymbol); + + bool isOptionalFieldTypeV3 = field.Optional && (NativeYtTypeFlags_ & ENativeTypeCompatFlags::NTCF_COMPLEX); + bool wrapOptionalTypeV3 = isOptionalFieldTypeV3 && + (field.Type->GetKind() == TTypeBase::EKind::Optional || field.Type->GetKind() == TTypeBase::EKind::Pg); + if (wrapOptionalTypeV3) { + Buf_.Write(BeginListSymbol); + } + + WriteYsonValueInTableFormat(Buf_, field.Type, NativeYtTypeFlags_, std::move(value), true); + + if (wrapOptionalTypeV3) { + Buf_.Write(ListItemSeparatorSymbol); + Buf_.Write(EndListSymbol); + } + + Buf_.Write(KeyedItemSeparatorSymbol); + } + Buf_.Write(EndMapSymbol); + Buf_.Write(ListItemSeparatorSymbol); + Buf_.OnRecordBoundary(); + } + + void EncodeNext(const NUdf::TUnboxedValuePod* row) final { + Buf_.Write(BeginMapSymbol); + + for (size_t index = 0; index < Fields_.size(); ++index) { + const TField& field = Fields_[index]; + auto value = row[index]; + if (field.Optional) { + if (!value) { + continue; + } + value = value.GetOptionalValue(); + } + + Buf_.Write(StringMarker); + Buf_.WriteVarI32(field.Name.size()); + Buf_.WriteMany(field.Name.data(), field.Name.size()); + Buf_.Write(KeyValueSeparatorSymbol); + + WriteYsonValueInTableFormat(Buf_, field.Type, NativeYtTypeFlags_, std::move(value), true); + + Buf_.Write(KeyedItemSeparatorSymbol); + } + Buf_.Write(EndMapSymbol); + Buf_.Write(ListItemSeparatorSymbol); + Buf_.OnRecordBoundary(); + } +private: + TVector<TField> Fields_; + ui64 NativeYtTypeFlags_; +}; + +class TSkiffEncoderBase: public TMkqlWriterImpl::TEncoder { +public: + TSkiffEncoderBase(TOutputBuf& buf, const TMkqlIOSpecs& specs) + : TMkqlWriterImpl::TEncoder(buf, specs) + { + } + + void EncodeNext(const NUdf::TUnboxedValuePod row) final { + const ui16 tableIndexVal = 0; // Always should be zero + Buf_.WriteMany((const char*)&tableIndexVal, sizeof(tableIndexVal)); + EncodeData(row); + Buf_.OnRecordBoundary(); + } + + void EncodeNext(const NUdf::TUnboxedValuePod* row) final { + const ui16 tableIndexVal = 0; // Always should be zero + Buf_.WriteMany((const char*)&tableIndexVal, sizeof(tableIndexVal)); + EncodeData(row); + Buf_.OnRecordBoundary(); + } + +protected: + virtual void EncodeData(const NUdf::TUnboxedValuePod row) = 0; + virtual void EncodeData(const NUdf::TUnboxedValuePod* row) = 0; +}; + +class TSkiffEncoder: public TSkiffEncoderBase { +public: + TSkiffEncoder(TOutputBuf& buf, const TMkqlIOSpecs& specs, size_t tableIndex, const TVector<TString>& columns) + : TSkiffEncoderBase(buf, specs) + { + Fields_ = GetFields(Specs_.Outputs[tableIndex].RowType, columns); + NativeYtTypeFlags_ = Specs_.Outputs[tableIndex].NativeYtTypeFlags; + } + +protected: + void EncodeData(const NUdf::TUnboxedValuePod row) final { + for (size_t index = 0; index < Fields_.size(); ++index) { + const TField& field = Fields_[index]; + auto value = row.GetElement(index); + if (field.Optional) { + if (!value) { + Buf_.Write('\0'); + continue; + } + Buf_.Write('\1'); + value = value.Release().GetOptionalValue(); + } + WriteSkiffValue(field.Type, value, field.Optional); + } + } + + void EncodeData(const NUdf::TUnboxedValuePod* row) final { + for (size_t index = 0; index < Fields_.size(); ++index) { + const TField& field = Fields_[index]; + auto value = row[index]; + if (field.Optional) { + if (!value) { + Buf_.Write('\0'); + continue; + } + Buf_.Write('\1'); + value = value.GetOptionalValue(); + } + WriteSkiffValue(field.Type, value, field.Optional); + } + } + + void WriteSkiffValue(TType* type, const NUdf::TUnboxedValuePod& value, bool wasOptional) { + if (NativeYtTypeFlags_) { + NCommon::WriteSkiffNativeYtValue(type, NativeYtTypeFlags_, value, Buf_); + } else if (type->IsData()) { + NCommon::WriteSkiffData(type, 0, value, Buf_); + } else if (!wasOptional && type->IsPg()) { + NCommon::WriteSkiffPg(static_cast<TPgType*>(type), value, Buf_); + } else { + WriteYsonContainerValue(type, 0, value, Buf_); + } + } + +protected: + TVector<TField> Fields_; + ui64 NativeYtTypeFlags_; +}; + +class TSkiffEmptySchemaEncoder: public TSkiffEncoderBase { +public: + TSkiffEmptySchemaEncoder(TOutputBuf& buf, const TMkqlIOSpecs& specs) + : TSkiffEncoderBase(buf, specs) + { + } + +protected: + void EncodeData(const NUdf::TUnboxedValuePod row) final { + Y_UNUSED(row); + Buf_.Write('\0'); // Empty optional "_yql_fake_column" + } + + void EncodeData(const NUdf::TUnboxedValuePod* row) final { + Y_UNUSED(row); + Buf_.Write('\0'); // Empty optional "_yql_fake_column" + } +}; + +class TSkiffLLVMEncoder: public TSkiffEncoderBase { +public: + typedef void (*TRowWriter)(const NUdf::TUnboxedValuePod, TOutputBuf&); + typedef void (*TRowFlatWriter)(const NUdf::TUnboxedValuePod*, TOutputBuf&); + + TSkiffLLVMEncoder(TOutputBuf& buf, const TMkqlIOSpecs& specs, TRowWriter rowWriter, TRowFlatWriter flatWriter) + : TSkiffEncoderBase(buf, specs) + , RowWriter_(rowWriter), RowFlatWriter_(flatWriter) + { + } + +protected: + void EncodeData(const NUdf::TUnboxedValuePod row) final { + RowWriter_(row, Buf_); + } + + void EncodeData(const NUdf::TUnboxedValuePod* row) final { + RowFlatWriter_(row, Buf_); + } +private: + TRowWriter RowWriter_; + TRowFlatWriter RowFlatWriter_; +}; + +////////////////////////////////////////////////////////////////////////////////////////////////////////// + +TMkqlWriterImpl::TOutput::TOutput(IOutputStream& stream, size_t blockCount, size_t blockSize, TStatTimer& timerWrite) + : Writer_(MakeBlockWriter(stream, blockCount, blockSize)) + , Buf_(*Writer_, &timerWrite) +{ +} + +TMkqlWriterImpl::TOutput::TOutput(NYT::TRawTableWriterPtr stream, size_t blockSize, NKikimr::NMiniKQL::TStatTimer& timerWrite) + : Writer_(MakeBlockWriter(*stream, 0, blockSize)) + , Buf_(*Writer_, &timerWrite) +{ + Writer_->SetRecordBoundaryCallback([stream](){ stream->NotifyRowEnd(); }); +} + +TMkqlWriterImpl::TMkqlWriterImpl(const TVector<IOutputStream*>& streams, size_t blockCount, size_t blockSize) + : TimerEncode_(OutputEncodeTime) + , TimerWrite_(OutputWriteTime) +{ + for (IOutputStream* stream: streams) { + Outputs_.push_back(MakeHolder<TOutput>(*stream, blockCount, blockSize, TimerWrite_)); + } +} + +TMkqlWriterImpl::TMkqlWriterImpl(IOutputStream& stream, size_t blockCount, size_t blockSize) + : TimerEncode_(OutputEncodeTime) + , TimerWrite_(OutputWriteTime) +{ + Outputs_.push_back(MakeHolder<TOutput>(stream, blockCount, blockSize, TimerWrite_)); +} + +TMkqlWriterImpl::TMkqlWriterImpl(const TVector<NYT::TRawTableWriterPtr>& streams, size_t blockSize) + : TimerEncode_(OutputEncodeTime) + , TimerWrite_(OutputWriteTime) +{ + for (auto& stream: streams) { + Outputs_.push_back(MakeHolder<TOutput>(stream, blockSize, TimerWrite_)); + } +} + +TMkqlWriterImpl::TMkqlWriterImpl(NYT::TRawTableWriterPtr stream, size_t blockSize) + : TimerEncode_(OutputEncodeTime) + , TimerWrite_(OutputWriteTime) +{ + Outputs_.push_back(MakeHolder<TOutput>(stream, blockSize, TimerWrite_)); +} + +TMkqlWriterImpl::~TMkqlWriterImpl() { +} + +void TMkqlWriterImpl::SetSpecs(const TMkqlIOSpecs& specs, const TVector<TString>& columns) { + // In the case of the "skiff" format, the ordering of columns in data and in spec are assumed to be identical during encoding + // To provide this, a "columns" field is used to change the alphabetical order of columns in spec + + Specs_ = &specs; + JobStats_ = specs.JobStats_; + +#ifndef MKQL_DISABLE_CODEGEN + THashMap<TStructType*, std::pair<llvm::Function*, llvm::Function*>> llvmFunctions; + if (Specs_->UseSkiff_ && Specs_->OptLLVM_ != "OFF" && NCodegen::ICodegen::IsCodegenAvailable()) { + for (size_t i: xrange(Specs_->Outputs.size())) { + auto rowType = Specs_->Outputs[i].RowType; + if (rowType->GetMembersCount() != 0 && !llvmFunctions.contains(rowType)) { + if (!Codegen_) { + Codegen_ = NCodegen::ICodegen::Make(NCodegen::ETarget::Native); + Codegen_->LoadBitCode(GetYtCodecBitCode(), "YtCodecFuncs"); + } + + const auto writer1 = MakeYtCodecCgWriter<false>(Codegen_, rowType); + const auto writer2 = MakeYtCodecCgWriter<true>(Codegen_, rowType); + + for (ui32 index = 0; index < rowType->GetMembersCount(); ++index) { + ui32 column = index; + if (columns) { + column = rowType->GetMemberIndex(columns[index]); + } + + auto fieldType = rowType->GetMemberType(column); + writer1->AddField(fieldType, Specs_->Outputs[i].NativeYtTypeFlags); + writer2->AddField(fieldType, Specs_->Outputs[i].NativeYtTypeFlags); + } + + llvmFunctions.emplace(rowType, std::make_pair(writer1->Build(), writer2->Build())); + } + } + if (!llvmFunctions.empty()) { + Codegen_->Verify(); + YtCodecAddMappings(*Codegen_); + Codegen_->Compile(); + } + else { + Codegen_.reset(); + } + } +#endif + + for (size_t i: xrange(Outputs_.size())) { + auto& out = Outputs_[i]; + out->Buf_.SetStats(JobStats_); + if (Specs_->UseSkiff_) { + if (Specs_->Outputs[i].RowType->GetMembersCount() == 0) { + YQL_ENSURE(columns.empty()); + Encoders_.emplace_back(new TSkiffEmptySchemaEncoder(out->Buf_, *Specs_)); + } +#ifndef MKQL_DISABLE_CODEGEN + else if (auto p = llvmFunctions.FindPtr(Specs_->Outputs[i].RowType)) { + Encoders_.emplace_back(new TSkiffLLVMEncoder(out->Buf_, *Specs_, + (TSkiffLLVMEncoder::TRowWriter)Codegen_->GetPointerToFunction(p->first), + (TSkiffLLVMEncoder::TRowFlatWriter)Codegen_->GetPointerToFunction(p->second) + )); + } +#endif + else { + Encoders_.emplace_back(new TSkiffEncoder(out->Buf_, *Specs_, i, columns)); + } + } else { + YQL_ENSURE(columns.empty()); + Encoders_.emplace_back(new TYsonEncoder(out->Buf_, *Specs_, i)); + } + } +} + +void TMkqlWriterImpl::AddRow(const NUdf::TUnboxedValuePod row) { + const auto guard = Guard<TStatTimer>(TimerEncode_); + if (Encoders_.size() == 1U) { + Encoders_.front()->EncodeNext(row); + } else { + const auto tableIndex = row.GetVariantIndex(); + YQL_ENSURE(tableIndex < Encoders_.size(), "Wrong table index: " << tableIndex + << ", there are only " << Encoders_.size() << " outputs"); + const auto item = row.GetVariantItem().Release(); + Encoders_[tableIndex]->EncodeNext(item); + } + + if (WriteLimit) { + ui64 res = 0; + for (auto& x : Outputs_) { + res += x->Buf_.GetWrittenBytes(); + } + if (res > *WriteLimit) { + throw TMemoryLimitExceededException(); + } + } +} + +void TMkqlWriterImpl::AddFlatRow(const NUdf::TUnboxedValuePod* row) { + YQL_ENSURE(Encoders_.size() == 1U, "Expected single table."); + const auto guard = Guard<TStatTimer>(TimerEncode_); + Encoders_.front()->EncodeNext(row); + if (WriteLimit) { + ui64 res = 0; + for (auto& x : Outputs_) { + res += x->Buf_.GetWrittenBytes(); + } + if (res > *WriteLimit) { + throw TMemoryLimitExceededException(); + } + } +} + +void TMkqlWriterImpl::Finish() { + if (!IsFinished_) { + IsFinished_ = true; + DoFinish(false); + } +} + +void TMkqlWriterImpl::Abort() { + if (!IsFinished_) { + IsFinished_ = true; + DoFinish(true); + } +} + +void TMkqlWriterImpl::DoFinish(bool abort) { + auto guard = Guard<TStatTimer>(TimerEncode_); + if (!abort) { + for (auto& x : Outputs_) { + x->Buf_.Finish(); + } + } + + Report(); +} + +void TMkqlWriterImpl::Report() { + TimerEncode_.Report(JobStats_); + TimerWrite_.Report(JobStats_); +} + +////////////////////////////////////////////////////////////////////////////////////////////////////////// + +THolder<IInputStream> MakeStringInput(const TString& str, bool decompress) { + if (decompress) { + return MakeHolder<THoldingStream<TBrotliDecompress, TStringInput>>(MakeHolder<TStringInput>(str)); + } else { + return MakeHolder<TStringInput>(str); + } +} + +THolder<IInputStream> MakeFileInput(const TString& file, bool decompress) { + if (decompress) { + return MakeHolder<THoldingStream<TBrotliDecompress, TUnbufferedFileInput>>(MakeHolder<TUnbufferedFileInput>(file)); + } else { + return MakeHolder<TUnbufferedFileInput>(file); + } +} + +////////////////////////////////////////////////////////////////////////////////////////////////////////// + +NUdf::TUnboxedValue DecodeYamr(TMkqlIOCache& specsCache, size_t tableIndex, const NYT::TYaMRRow& row) { + NUdf::TUnboxedValue* items; + auto ret = specsCache.NewRow(tableIndex, items); + auto& decoder = *specsCache.GetSpecs().Inputs.at(tableIndex); + auto& lastFields = specsCache.GetLastFields(tableIndex); + + TVector<TStringBuf> values{row.Key, row.SubKey, row.Value}; + for (size_t i = 0; i < YAMR_FIELDS.size(); ++i) { + const TMkqlIOSpecs::TDecoderSpec::TDecodeField* field = nullptr; + if (i < lastFields.size()) { + auto lastField = lastFields[i]; + if (lastField->Name == YAMR_FIELDS[i]) { + field = lastField; + } + } + if (!field) { + auto it = decoder.Fields.find(YAMR_FIELDS[i]); + if (it != decoder.Fields.end()) { + field = &it->second; + } + } + if (field) { + items[field->StructIndex] = MakeString(NUdf::TStringRef(values[i].data(), values[i].size())); + } + } + + return ret; +} + +void DecodeToYson(TMkqlIOCache& specsCache, size_t tableIndex, const NYT::TNode& value, IOutputStream& ysonOut) { + auto& decoder = *specsCache.GetSpecs().Inputs.at(tableIndex); + + TVector<TString> items(decoder.StructSize); + TMaybe<THashMap<TString, TString>> others; + if (decoder.OthersStructIndex) { + others.ConstructInPlace(); + others->reserve(specsCache.GetMaxOthersFields(tableIndex)); + } + + for (auto& node: value.AsMap()) { + auto& name = node.first; + const TMkqlIOSpecs::TDecoderSpec::TDecodeField* field = nullptr; + if (!decoder.OthersStructIndex || name != YqlOthersColumnName) { + auto it = decoder.Fields.find(name); + if (it != decoder.Fields.end()) { + field = &it->second; + } + } + + if (!field && decoder.OthersStructIndex) { + if (node.second.IsString()) { + others->emplace(name, node.second.AsString()); + } else if (node.second.IsEntity() || node.second.IsBool() || node.second.IsInt64() || node.second.IsUint64() || node.second.IsDouble()) { + others->emplace(name, NYT::NodeToYsonString(node.second, NYT::NYson::EYsonFormat::Text)); + } else { + others->emplace(name, NYT::NodeToYsonString(node.second, NYT::NYson::EYsonFormat::Binary)); + } + continue; + } + + if (field->StructIndex != Max<ui32>()) { + NYT::TNode res = node.second; + auto dataType = field->Type; + if (field->Type->IsOptional()) { + dataType = static_cast<TOptionalType*>(field->Type)->GetItemType(); + if (res.IsEntity()) { + res = NYT::TNode::CreateList(); + } else { + res = NYT::TNode::CreateList().Add(node.second); + } + } else if (res.IsEntity()) { + res = NYT::TNode(); + } + if (res.GetType() != NYT::TNode::Undefined) { + if (dataType->GetKind() == TType::EKind::Data && static_cast<TDataType*>(dataType)->GetSchemeType() == NUdf::TDataType<NUdf::TYson>::Id) { + items[field->StructIndex] = NResult::EncodeRestrictedYson(res, NYT::NYson::EYsonFormat::Binary); + } else { + items[field->StructIndex] = NYT::NodeToYsonString(res, NYT::NYson::EYsonFormat::Binary); + } + } + } + } + + WriteRowItems(specsCache, tableIndex, items, others, ysonOut); +} + +void DecodeToYson(TMkqlIOCache& specsCache, size_t tableIndex, const NYT::TYaMRRow& value, IOutputStream& ysonOut) { + auto& decoder = *specsCache.GetSpecs().Inputs.at(tableIndex); + auto& lastFields = specsCache.GetLastFields(tableIndex); + + TVector<TString> items(decoder.StructSize); + TVector<TStringBuf> values{value.Key, value.SubKey, value.Value}; + for (size_t i = 0; i < YAMR_FIELDS.size(); ++i) { + const TMkqlIOSpecs::TDecoderSpec::TDecodeField* field = nullptr; + if (i < lastFields.size()) { + auto lastField = lastFields[i]; + if (lastField->Name == YAMR_FIELDS[i]) { + field = lastField; + } + } + if (!field) { + auto it = decoder.Fields.find(YAMR_FIELDS[i]); + if (it != decoder.Fields.end()) { + field = &it->second; + } + } + if (field) { + items[field->StructIndex] = NYT::NodeToYsonString(NYT::TNode(values[i]), NYT::NYson::EYsonFormat::Binary); + } + } + + WriteRowItems(specsCache, tableIndex, items, {}, ysonOut); +} + +void DecodeToYson(TMkqlIOCache& specsCache, size_t tableIndex, const NUdf::TUnboxedValuePod& value, IOutputStream& ysonOut) { + auto& decoder = *specsCache.GetSpecs().Inputs.at(tableIndex); + TVector<TString> items(decoder.StructSize); + for (ui32 i = 0; i < decoder.StructSize; ++i) { + items[i] = NCommon::WriteYsonValue(value.GetElement(decoder.FieldsVec[i].StructIndex), decoder.FieldsVec[i].Type, nullptr, NYT::NYson::EYsonFormat::Binary); + } + WriteRowItems(specsCache, tableIndex, items, {}, ysonOut); +} + +////////////////////////////////////////////////////////////////////////////////////////////////////////// + +} // NYql diff --git a/yt/yql/providers/yt/codec/yt_codec_io.h b/yt/yql/providers/yt/codec/yt_codec_io.h new file mode 100644 index 0000000000..6d9c2ae572 --- /dev/null +++ b/yt/yql/providers/yt/codec/yt_codec_io.h @@ -0,0 +1,163 @@ +#pragma once + +#include "yt_codec.h" +#include <yql/essentials/providers/common/codec/yql_codec_buf.h> + +#include <yql/essentials/minikql/computation/mkql_computation_node_holders.h> +#include <yql/essentials/minikql/aligned_page_pool.h> + +#ifndef MKQL_DISABLE_CODEGEN +#include <yql/essentials/minikql/codegen/codegen.h> +#endif + +#include <yt/cpp/mapreduce/interface/io.h> +#include <yt/cpp/mapreduce/io/stream_table_reader.h> + +#include <library/cpp/yson/zigzag.h> + +#include <util/generic/strbuf.h> +#include <util/generic/ptr.h> +#include <util/generic/xrange.h> +#include <util/generic/yexception.h> +#include <util/generic/maybe.h> +#include <util/stream/output.h> +#include <util/stream/input.h> +#include <util/stream/holder.h> +#include <util/system/compiler.h> + +namespace NYql { + +////////////////////////////////////////////////////////////////////////////////////////////////////////// + +class TMkqlReaderImpl : public IMkqlReaderImpl { +public: + struct TDecoder; + + // Input source must be valid while TMkqlReaderImpl exists + TMkqlReaderImpl(); + TMkqlReaderImpl(NYT::TRawTableReader& source, size_t blockCount, size_t blockSize, ui32 tableIndex = 0, bool ignoreStreamTableIndex = false); + ~TMkqlReaderImpl(); + + void SetReader(NYT::TRawTableReader& source, size_t blockCount, size_t blockSize, ui32 tableIndex = 0, bool ignoreStreamTableIndex = false, TMaybe<ui64> currentRow={}, TMaybe<ui32> currentRange={}); + void SetSpecs(const TMkqlIOSpecs& specs, const NKikimr::NMiniKQL::THolderFactory& holderFactory); + void SetDeadline(TInstant deadline) { + Reader_->SetDeadline(deadline); + } + void SetNextBlockCallback(std::function<void()> cb) { + Buf_.SetNextBlockCallback(cb); + } + + NKikimr::NUdf::TUnboxedValue GetRow() const override; + bool IsValid() const override; + void Next() override; + TMaybe<ui32> GetRangeIndexUnchecked() const; + TMaybe<ui64> GetRowIndexUnchecked() const; + ui32 GetTableIndex() const override; + ui32 GetRangeIndex() const override; + ui64 GetRowIndex() const override; + void NextKey() override; + + void Finish(); + +protected: + NKikimr::NUdf::TUnboxedValue BuildOthers(const TMkqlIOSpecs::TDecoderSpec& decoder, NKikimr::NMiniKQL::TValuesDictHashMap& others); + + void CheckValidity() const; + void CheckReadRow() const; + + void OnError(const std::exception_ptr& error, TStringBuf msg); + +protected: + NKikimr::NMiniKQL::TSamplingStatTimer TimerDecode_; + NKikimr::NMiniKQL::TSamplingStatTimer TimerRead_; + THolder<NCommon::IBlockReader> Reader_; + NCommon::TInputBuf Buf_; + THolder<TDecoder> Decoder_; + + bool HasRangeIndices_ = false; + ui32 InitialTableIndex_ = 0; + bool IgnoreStreamTableIndex_ = false; + + const TMkqlIOSpecs* Specs_ = nullptr; + const NKikimr::NMiniKQL::THolderFactory* HolderFactoryPtr = nullptr; + NKikimr::NMiniKQL::IStatsRegistry* JobStats_ = nullptr; +}; + +////////////////////////////////////////////////////////////////////////////////////////////////////////// + +class TMkqlWriterImpl: public IMkqlWriterImpl { +protected: + struct TOutput { + TOutput(IOutputStream& streams, size_t blockCount, size_t blockSize, NKikimr::NMiniKQL::TStatTimer& timerWrite); + TOutput(NYT::TRawTableWriterPtr stream, size_t blockSize, NKikimr::NMiniKQL::TStatTimer& timerWrite); + + THolder<NCommon::IBlockWriter> Writer_; + NCommon::TOutputBuf Buf_; + }; + +public: + struct TEncoder; + + // Output streams must be valid while TMkqlWriterImpl exists + TMkqlWriterImpl(const TVector<IOutputStream*>& streams, size_t blockCount, size_t blockSize); + TMkqlWriterImpl(IOutputStream& stream, size_t blockCount, size_t blockSize); + // client writer + TMkqlWriterImpl(const TVector<NYT::TRawTableWriterPtr>& streams, size_t blockSize); + TMkqlWriterImpl(NYT::TRawTableWriterPtr stream, size_t blockSize); + + ~TMkqlWriterImpl(); + + void SetSpecs(const TMkqlIOSpecs& specs, const TVector<TString>& columns = {}); + + void AddRow(const NKikimr::NUdf::TUnboxedValuePod row) override; + void AddFlatRow(const NUdf::TUnboxedValuePod* row) override; + + void SetWriteLimit(ui64 limit) { + WriteLimit = limit; + } + + void Finish() override; + void Abort() override; + + void Report(); + +protected: + void WriteSkiffValue(NCommon::TOutputBuf& buf, NKikimr::NMiniKQL::TType* type, const NKikimr::NUdf::TUnboxedValuePod& value); + virtual void DoFinish(bool abort); + +protected: + NKikimr::NMiniKQL::TStatTimer TimerEncode_; + NKikimr::NMiniKQL::TStatTimer TimerWrite_; + const TMkqlIOSpecs* Specs_ = nullptr; + NKikimr::NMiniKQL::IStatsRegistry* JobStats_ = nullptr; + TVector<THolder<TOutput>> Outputs_; + bool IsFinished_ = false; + TVector<THolder<TEncoder>> Encoders_; +#ifndef MKQL_DISABLE_CODEGEN + NCodegen::ICodegen::TPtr Codegen_; +#endif + TMaybe<ui64> WriteLimit; +}; + +////////////////////////////////////////////////////////////////////////////////////////////////////////// + +class TMkqlInput: public THoldingStream<NYT::NDetail::TInputStreamProxy, IInputStream> { + using TBase = THoldingStream<NYT::NDetail::TInputStreamProxy, IInputStream>; +public: + TMkqlInput(::THolder<IInputStream> h) + : TBase(std::move(h)) + { + } +}; + +THolder<IInputStream> MakeStringInput(const TString& str, bool decompress); +THolder<IInputStream> MakeFileInput(const TString& file, bool decompress); + +////////////////////////////////////////////////////////////////////////////////////////////////////////// + +NKikimr::NUdf::TUnboxedValue DecodeYamr(TMkqlIOCache& specsCache, size_t tableIndex, const NYT::TYaMRRow& row); +void DecodeToYson(TMkqlIOCache& specsCache, size_t tableIndex, const NYT::TNode& value, IOutputStream& ysonOut); +void DecodeToYson(TMkqlIOCache& specsCache, size_t tableIndex, const NYT::TYaMRRow& value, IOutputStream& ysonOut); +void DecodeToYson(TMkqlIOCache& specsCache, size_t tableIndex, const NKikimr::NUdf::TUnboxedValuePod& value, IOutputStream& ysonOut); + +} // NYql diff --git a/yt/yql/providers/yt/codec/yt_codec_io_ut.cpp b/yt/yql/providers/yt/codec/yt_codec_io_ut.cpp new file mode 100644 index 0000000000..033950204d --- /dev/null +++ b/yt/yql/providers/yt/codec/yt_codec_io_ut.cpp @@ -0,0 +1,308 @@ +#include "yt_codec_io.h" + +#include <yt/yql/providers/yt/lib/yson_helpers/yson_helpers.h> +#include <yql/essentials/providers/common/codec/yql_codec.h> +#include <yql/essentials/minikql/mkql_function_registry.h> +#include <yql/essentials/minikql/mkql_alloc.h> +#include <yql/essentials/minikql/mkql_node.h> +#include <yql/essentials/minikql/mkql_mem_info.h> +#include <yql/essentials/minikql/computation/mkql_computation_node_holders.h> + +#include <library/cpp/yson/node/node_visitor.h> + +#include <library/cpp/testing/unittest/registar.h> + +#include <util/stream/buffer.h> +#include <util/generic/maybe.h> +#include <util/generic/string.h> +#include <util/generic/vector.h> +#include <util/generic/buffer.h> +#include <util/generic/ylimits.h> + +using namespace NYql; +using namespace NKikimr; +using namespace NKikimr::NMiniKQL; + +namespace { + +class TTestInput: public NYT::TRawTableReader { +public: + TTestInput(const NYT::TNode::TListType& records, size_t fails, size_t retries, bool failAtStart, bool omitLastSemicolon) + : Records(records) + , Fails(fails) + , Retries(retries) + , FailAtStart(failAtStart) + , OmitLastSemicolon(omitLastSemicolon) + { + Prepare(); + } + + bool Retry(const TMaybe<ui32>& /*rangeIndex*/, const TMaybe<ui64>& rowIndex, const std::exception_ptr& /*error*/) override { + if (0 == Retries) { + return false; + } + --Retries; + Prepare(rowIndex.GetOrElse(0)); + CurrentOffset = 0; + return true; + } + + void ResetRetries() override { + } + + // Returns 'true' if the input stream may contain table ranges. + // The TRawTableReader user is responsible to track active range index in this case + // in order to pass it to Retry(). + bool HasRangeIndices() const override { + return false; + } + +protected: + void Prepare(size_t fromRowIndex = 0) { + Y_ABORT_UNLESS(fromRowIndex < Records.size()); + AtStart = true; + Data.Buffer().Clear(); + Data.Rewind(); + TBinaryYsonWriter writer(&Data); + NYT::TNodeVisitor visitor(&writer); + auto entity = NYT::TNode::CreateEntity(); + entity.Attributes()("row_index", i64(fromRowIndex)); + visitor.Visit(entity); + for (size_t i = fromRowIndex; i < Records.size(); ++i) { + Data.Write(';'); + visitor.Visit(Records[i]); + } + if (!OmitLastSemicolon) { + Data.Write(';'); + } + } + + size_t DoRead(void* buf, size_t len) override { + size_t read = Data.Read(buf, len); + if (AtStart == FailAtStart && Fails > 0 && read > 0) { + --Fails; + throw yexception() << "Fail"; + } + AtStart = false; + return read; + } + +private: + const NYT::TNode::TListType& Records; + TBufferStream Data; + size_t Fails; + size_t Retries; + const bool FailAtStart; + size_t CurrentOffset = 0; + bool AtStart = true; + bool OmitLastSemicolon = false; +}; + +struct TMkqlCodecFixture { + TIntrusivePtr<IFunctionRegistry> FunctionRegistry; + TScopedAlloc Alloc; + TTypeEnvironment Env; + TMemoryUsageInfo MemInfo; + THolderFactory HolderFactory; + NCommon::TCodecContext CodecCtx; + TMkqlIOSpecs Specs; + + static const TString INPUT_SPEC; + static const TString VALUE; + + TMkqlCodecFixture() + : FunctionRegistry(CreateFunctionRegistry(IBuiltinFunctionRegistry::TPtr())) + , Alloc(__LOCATION__) + , Env(Alloc) + , MemInfo("Test") + , HolderFactory(Alloc.Ref(), MemInfo, FunctionRegistry.Get()) + , CodecCtx(Env, *FunctionRegistry, &HolderFactory) + { + Specs.Init(CodecCtx, INPUT_SPEC, {}, Nothing()); + } + + static TVector<NYT::TNode> Generate(size_t numRecords) { + TVector<NYT::TNode> data; + for (size_t i = 0; i < numRecords; i++) { + data.push_back(NYT::TNode()("key", i)("value", VALUE)); + } + return data; + } + + static void Validate(size_t rowIndex, const NUdf::TUnboxedValue& row) { + UNIT_ASSERT_VALUES_EQUAL(row.GetElement(0).Get<ui64>(), rowIndex + 1); + UNIT_ASSERT_VALUES_EQUAL(row.GetElement(1).Get<ui64>(), rowIndex); + auto value = row.GetElement(2); + UNIT_ASSERT_VALUES_EQUAL(TStringBuf(value.AsStringRef()), VALUE); + } +}; + +const TString TMkqlCodecFixture::INPUT_SPEC = R"({ + tables = [{ + "_yql_row_spec" = { + "Type" = [ + "StructType"; [ + ["key"; ["DataType"; "Uint64"]]; + ["value"; ["DataType"; "String"]] + ] + ] + }; + "_yql_sys_table" = [ + "record" + ] + }] +})"; + +const TString TMkqlCodecFixture::VALUE = TString().append(100, 'z'); + +} // unnamed + +Y_UNIT_TEST_SUITE(TMkqlCodec) { + + void TestRead(size_t blockCount) { + TMkqlCodecFixture fixture; + auto data = TMkqlCodecFixture::Generate(10); + // With semicolon in end of input + { + TTestInput input(data, 0, 0, false, false); + TMkqlReaderImpl reader(input, blockCount, TMkqlCodecFixture::VALUE.size()); + reader.SetSpecs(fixture.Specs, fixture.HolderFactory); + reader.Next(); + + for (size_t i = 0; i < 10; reader.Next(), i++) { + UNIT_ASSERT(reader.IsValid()); + TMkqlCodecFixture::Validate(i, reader.GetRow()); + } + UNIT_ASSERT(!reader.IsValid()); + } + // Without semicolon in end of input + { + TTestInput input(data, 0, 0, false, true); + TMkqlReaderImpl reader(input, blockCount, TMkqlCodecFixture::VALUE.size()); + reader.SetSpecs(fixture.Specs, fixture.HolderFactory); + reader.Next(); + + for (size_t i = 0; i < 10; reader.Next(), i++) { + UNIT_ASSERT(reader.IsValid()); + TMkqlCodecFixture::Validate(i, reader.GetRow()); + } + UNIT_ASSERT(!reader.IsValid()); + } + } + + Y_UNIT_TEST(ReadSync) { + TestRead(0); + } + + Y_UNIT_TEST(ReadAsync) { + TestRead(4); + } + + void TestReadFail(size_t blockCount) { + TMkqlCodecFixture fixture; + auto data = TMkqlCodecFixture::Generate(10); + // Before first record + { + TTestInput input(data, 1, 0, true, false); + TMkqlReaderImpl reader(input, blockCount, TMkqlCodecFixture::VALUE.size()); + reader.SetSpecs(fixture.Specs, fixture.HolderFactory); + UNIT_ASSERT_EXCEPTION(reader.Next(), yexception); + } + // In the middle + { + TTestInput input(data, 1, 0, false, false); + TMkqlReaderImpl reader(input, blockCount, 2 * TMkqlCodecFixture::VALUE.size()); + reader.SetSpecs(fixture.Specs, fixture.HolderFactory); + reader.Next(); + UNIT_ASSERT(reader.IsValid()); + UNIT_ASSERT_EXCEPTION(reader.Next(), yexception); + } + } + + Y_UNIT_TEST(ReadSyncFail) { + TestReadFail(0); + } + + Y_UNIT_TEST(ReadAsyncFail) { + TestReadFail(4); + } + + void TestReadRetry(size_t blockCount) { + TMkqlCodecFixture fixture; + auto data = TMkqlCodecFixture::Generate(10); + // Before first record + { + TTestInput input(data, 1, 1, true, false); + TMkqlReaderImpl reader(input, blockCount, TMkqlCodecFixture::VALUE.size()); + reader.SetSpecs(fixture.Specs, fixture.HolderFactory); + reader.Next(); + + for (size_t i = 0; i < 10; reader.Next(), i++) { + UNIT_ASSERT(reader.IsValid()); + TMkqlCodecFixture::Validate(i, reader.GetRow()); + } + UNIT_ASSERT(!reader.IsValid()); + } + // In the middle + { + TTestInput input(data, 3, 3, false, false); + TMkqlReaderImpl reader(input, blockCount, TMkqlCodecFixture::VALUE.size()); + reader.SetSpecs(fixture.Specs, fixture.HolderFactory); + reader.Next(); + + for (size_t i = 0; i < 10; reader.Next(), i++) { + UNIT_ASSERT(reader.IsValid()); + TMkqlCodecFixture::Validate(i, reader.GetRow()); + } + UNIT_ASSERT(!reader.IsValid()); + } + // Fail through time + { + TTestInput input(data, Max(), Max(), false, false); + TMkqlReaderImpl reader(input, blockCount, 2 * TMkqlCodecFixture::VALUE.size()); + reader.SetSpecs(fixture.Specs, fixture.HolderFactory); + reader.Next(); + + for (size_t i = 0; i < 10; reader.Next(), i++) { + UNIT_ASSERT(reader.IsValid()); + TMkqlCodecFixture::Validate(i, reader.GetRow()); + } + UNIT_ASSERT(!reader.IsValid()); + } + // Small buffer + { + TTestInput input(data, blockCount + 1, blockCount + 1, false, false); + TMkqlReaderImpl reader(input, blockCount, TMkqlCodecFixture::VALUE.size() / 2); + reader.SetSpecs(fixture.Specs, fixture.HolderFactory); + reader.Next(); + + for (size_t i = 0; i < 10; reader.Next(), i++) { + UNIT_ASSERT(reader.IsValid()); + TMkqlCodecFixture::Validate(i, reader.GetRow()); + } + UNIT_ASSERT(!reader.IsValid()); + } + // Large buffer + { + TTestInput input(data, blockCount + 1, blockCount + 1, false, false); + TMkqlReaderImpl reader(input, blockCount, TMkqlCodecFixture::VALUE.size() * 2); + reader.SetSpecs(fixture.Specs, fixture.HolderFactory); + reader.Next(); + + for (size_t i = 0; i < 10; reader.Next(), i++) { + UNIT_ASSERT(reader.IsValid()); + TMkqlCodecFixture::Validate(i, reader.GetRow()); + } + UNIT_ASSERT(!reader.IsValid()); + } + } + + Y_UNIT_TEST(ReadSyncRetry) { + TestReadRetry(0); + } + + Y_UNIT_TEST(ReadAsyncRetry) { + TestReadRetry(4); + } + +} diff --git a/yt/yql/providers/yt/codec/yt_codec_job.cpp b/yt/yql/providers/yt/codec/yt_codec_job.cpp new file mode 100644 index 0000000000..47d1feb734 --- /dev/null +++ b/yt/yql/providers/yt/codec/yt_codec_job.cpp @@ -0,0 +1,65 @@ +#include "yt_codec_job.h" + +#include <yt/yql/providers/yt/common/yql_configuration.h> + +#include <yt/cpp/mapreduce/io/job_reader.h> +#include <yt/cpp/mapreduce/io/job_writer.h> + +#include <util/system/file.h> +#include <util/stream/output.h> +#include <util/generic/xrange.h> +#include <util/generic/vector.h> + +namespace NYql { + +using namespace NKikimr; +using namespace NKikimr::NMiniKQL; + +namespace NPrivate { + +TInStreamHolder::TInStreamHolder(const TFile& inHandle) + : Input(inHandle) + , Proxy(&Input) +{ +} + +TOutStreamsHolder::TOutStreamsHolder(const TVector<TFile>& outHandles) { + Outputs.reserve(outHandles.size()); + for (auto& h: outHandles) { + Outputs.emplace_back(h); + } +} + +TVector<IOutputStream*> TOutStreamsHolder::GetVectorOfStreams() { + TVector<IOutputStream*> res(Reserve(Outputs.size())); + for (auto& s: Outputs) { + res.push_back(&s); + } + return res; +} + +} // NPrivate + +TJobMkqlReaderImpl::TJobMkqlReaderImpl(const TFile& in) + : NPrivate::TInStreamHolder(in) + , TMkqlReaderImpl(GetYtStream(), YQL_JOB_CODEC_BLOCK_COUNT, YQL_JOB_CODEC_BLOCK_SIZE) +{ +} + +TJobMkqlWriterImpl::TJobMkqlWriterImpl(const TMkqlIOSpecs& specs, const TVector<TFile>& outHandles) + : NPrivate::TOutStreamsHolder(outHandles) + , TMkqlWriterImpl(GetVectorOfStreams(), YQL_JOB_CODEC_BLOCK_COUNT, YQL_JOB_CODEC_BLOCK_SIZE) +{ + SetSpecs(specs); +} + +void TJobMkqlWriterImpl::DoFinish(bool abort) { + TMkqlWriterImpl::DoFinish(abort); + if (!abort) { + for (auto& out: Outputs) { + out.Flush(); + } + } +} + +} // NYql diff --git a/yt/yql/providers/yt/codec/yt_codec_job.h b/yt/yql/providers/yt/codec/yt_codec_job.h new file mode 100644 index 0000000000..e2a8232d2c --- /dev/null +++ b/yt/yql/providers/yt/codec/yt_codec_job.h @@ -0,0 +1,52 @@ +#pragma once + +#include "yt_codec_io.h" + +#include <yt/cpp/mapreduce/io/stream_table_reader.h> + +#include <util/system/file.h> +#include <util/stream/file.h> + + +namespace NYql { + +namespace NPrivate { + class TInStreamHolder { + public: + TInStreamHolder(const TFile& inHandle); + NYT::TRawTableReader& GetYtStream() { + return Proxy; + } + + protected: + TUnbufferedFileInput Input; + NYT::NDetail::TInputStreamProxy Proxy; + }; + + class TOutStreamsHolder { + public: + TOutStreamsHolder(const TVector<TFile>& outHandles); + TVector<IOutputStream*> GetVectorOfStreams(); + + protected: + TVector<TUnbufferedFileOutput> Outputs; + }; +} + +class TJobMkqlReaderImpl: protected NPrivate::TInStreamHolder, public TMkqlReaderImpl { +public: + TJobMkqlReaderImpl(const TFile& in); + ~TJobMkqlReaderImpl() = default; +}; + + +class TJobMkqlWriterImpl: protected NPrivate::TOutStreamsHolder, public TMkqlWriterImpl { +public: + TJobMkqlWriterImpl(const TMkqlIOSpecs& specs, const TVector<TFile>& outHandles); + ~TJobMkqlWriterImpl() = default; + +private: + void DoFinish(bool abort) override; +}; + +} // NYql diff --git a/yt/yql/providers/yt/common/ya.make b/yt/yql/providers/yt/common/ya.make new file mode 100644 index 0000000000..7a9b170a71 --- /dev/null +++ b/yt/yql/providers/yt/common/ya.make @@ -0,0 +1,24 @@ +LIBRARY() + +SRCS( + yql_configuration.cpp + yql_names.cpp + yql_yt_settings.cpp +) + +PEERDIR( + library/cpp/regex/pcre + library/cpp/string_utils/parse_size + library/cpp/yson/node + yt/cpp/mapreduce/interface + yql/essentials/ast + yql/essentials/utils/log + yql/essentials/providers/common/codec + yql/essentials/providers/common/config +) + +YQL_LAST_ABI_VERSION() + +GENERATE_ENUM_SERIALIZATION(yql_yt_settings.h) + +END() diff --git a/yt/yql/providers/yt/common/yql_configuration.cpp b/yt/yql/providers/yt/common/yql_configuration.cpp new file mode 100644 index 0000000000..02d5a2dd57 --- /dev/null +++ b/yt/yql/providers/yt/common/yql_configuration.cpp @@ -0,0 +1 @@ +#include "yql_configuration.h" diff --git a/yt/yql/providers/yt/common/yql_configuration.h b/yt/yql/providers/yt/common/yql_configuration.h new file mode 100644 index 0000000000..804d6dcadc --- /dev/null +++ b/yt/yql/providers/yt/common/yql_configuration.h @@ -0,0 +1,105 @@ +#pragma once + +#include <yql/essentials/public/udf/udf_data_type.h> + +#include <util/system/types.h> +#include <util/datetime/base.h> +#include <util/generic/size_literals.h> +#include <util/generic/set.h> + +namespace NYql { + +constexpr size_t YQL_JOB_CODEC_BLOCK_COUNT = 16; +constexpr size_t YQL_JOB_CODEC_BLOCK_SIZE = 1_MB; + +constexpr size_t YQL_JOB_CODEC_MEM = YQL_JOB_CODEC_BLOCK_COUNT * YQL_JOB_CODEC_BLOCK_SIZE + (30_MB); + +constexpr ui64 DEFAULT_TOP_SORT_LIMIT = 1000ULL; + +constexpr ui16 DEFAULT_WIDE_FLOW_LIMIT = 101U; + +constexpr bool DEFAULT_USE_FLOW = true; + +constexpr bool DEFAULT_USE_NATIVE_YT_TYPES = false; + +constexpr bool DEFAULT_USE_INTERMEDIATE_STREAMS = false; + +constexpr bool DEFAULT_USE_SKIFF = true; + +constexpr bool DEFAULT_USE_SYS_COLUMNS = true; + +constexpr bool DEFAULT_USE_MULTISET_ATTRS = true; + +constexpr bool DEFAULT_MAP_JOIN_USE_FLOW = true; + +constexpr bool DefaultHybridDqExecution = false; +constexpr auto DefaultHybridDqDataSizeLimitForOrdered = 1_GB; +constexpr auto DefaultHybridDqDataSizeLimitForUnordered = 16_GB; + +constexpr bool DEFAULT_ROW_SPEC_COMPACT_FORM = false; + +constexpr bool DEFAULT_USE_NEW_PREDICATE_EXTRACTION = false; +constexpr bool DEFAULT_PRUNE_KEY_FILTER_LAMBDA = false; +constexpr bool DEFAULT_DQ_PRUNE_KEY_FILTER_LAMBDA = false; +constexpr bool DEFAULT_MERGE_ADJACENT_POINT_RANGES = true; +constexpr bool DEFAULT_KEY_FILTER_FOR_STARTS_WITH = true; +constexpr ui64 DEFAULT_MAX_KEY_RANGE_COUNT = 1000ULL; + +constexpr bool DEFAULT_USE_NATIVE_DESC_SORT = false; + +constexpr ui64 DEFAULT_MAX_CHUNKS_FOR_DQ_READ = 500; + +constexpr bool DEFAULT_USE_KEY_BOUND_API = false; + +constexpr ui32 DEFAULT_MAX_OPERATION_FILES = 1000; + +constexpr bool DEFAULT_JOIN_COMMON_USE_MULTI_OUT = false; + +constexpr bool DEFAULT_USE_RPC_READER_IN_DQ = false; +constexpr size_t DEFAULT_RPC_READER_INFLIGHT = 1; +constexpr TDuration DEFAULT_RPC_READER_TIMEOUT = TDuration::Seconds(120); +const TSet<TString> DEFAULT_BLOCK_READER_SUPPORTED_TYPES = {"pg", "tuple"}; +const TSet<NUdf::EDataSlot> DEFAULT_BLOCK_READER_SUPPORTED_DATA_TYPES = + { + NUdf::EDataSlot::Int8, NUdf::EDataSlot::Uint8, + NUdf::EDataSlot::Int16, NUdf::EDataSlot::Uint16, + NUdf::EDataSlot::Int32, NUdf::EDataSlot::Uint32, + NUdf::EDataSlot::Int64, NUdf::EDataSlot::Uint64, + NUdf::EDataSlot::Bool, NUdf::EDataSlot::Double, + NUdf::EDataSlot::String, NUdf::EDataSlot::Json, + NUdf::EDataSlot::Yson, NUdf::EDataSlot::Utf8 + }; +const TSet<TString> DEFAULT_BLOCK_INPUT_SUPPORTED_TYPES = {"tuple"}; +const TSet<NUdf::EDataSlot> DEFAULT_BLOCK_INPUT_SUPPORTED_DATA_TYPES = + { + NUdf::EDataSlot::Int8, NUdf::EDataSlot::Uint8, + NUdf::EDataSlot::Int16, NUdf::EDataSlot::Uint16, + NUdf::EDataSlot::Int32, NUdf::EDataSlot::Uint32, + NUdf::EDataSlot::Int64, NUdf::EDataSlot::Uint64, + NUdf::EDataSlot::Bool, NUdf::EDataSlot::Double, + NUdf::EDataSlot::String, NUdf::EDataSlot::Utf8, + NUdf::EDataSlot::Yson + }; + +constexpr auto DEFAULT_SWITCH_MEMORY_LIMIT = 128_MB; + +constexpr ui32 DEFAULT_MAX_INPUT_TABLES = 3000; +constexpr ui32 DEFAULT_MAX_OUTPUT_TABLES = 100; +constexpr ui64 DEFAULT_APPLY_STORED_CONSTRAINTS = 0ULL; + +constexpr ui64 DEFAULT_TABLE_CONTENT_LOCAL_EXEC = 0; + +constexpr ui32 DEFAULT_BATCH_LIST_FOLDER_CONCURRENCY = 5; + +constexpr bool DEFAULT_PARTITION_BY_CONSTANT_KEYS_VIA_MAP = false; + +constexpr ui64 DEFAULT_LLVM_NODE_COUNT_LIMIT = 200000; + +constexpr ui16 DEFAULT_MIN_COLUMN_GROUP_SIZE = 2; +constexpr ui16 DEFAULT_MAX_COLUMN_GROUPS = 64; + +constexpr bool DEFAULT_DISABLE_FUSE_OPERATIONS = false; + +constexpr bool DEFAULT_ENABLE_DQ_WRITE_CONSTRAINTS = false; + +} // NYql diff --git a/yt/yql/providers/yt/common/yql_names.cpp b/yt/yql/providers/yt/common/yql_names.cpp new file mode 100644 index 0000000000..3cb0193565 --- /dev/null +++ b/yt/yql/providers/yt/common/yql_names.cpp @@ -0,0 +1,11 @@ +#include "yql_names.h" + +namespace NYql { + +const TVector<TStringBuf> YAMR_FIELDS = { + TStringBuf("key"), + TStringBuf("subkey"), + TStringBuf("value"), +}; + +} diff --git a/yt/yql/providers/yt/common/yql_names.h b/yt/yql/providers/yt/common/yql_names.h new file mode 100644 index 0000000000..e8ca0c8222 --- /dev/null +++ b/yt/yql/providers/yt/common/yql_names.h @@ -0,0 +1,73 @@ +#pragma once + +#include <util/generic/strbuf.h> +#include <util/generic/vector.h> + +namespace NYql { + +const TStringBuf YqlRowSpecAttribute = "_yql_row_spec"; + +const TStringBuf RowSpecAttrType = "Type"; +const TStringBuf RowSpecAttrTypePatch = "TypePatch"; +const TStringBuf RowSpecAttrStrictSchema = "StrictSchema"; +const TStringBuf RowSpecAttrUniqueKeys = "UniqueKeys"; +const TStringBuf RowSpecAttrSortDirections = "SortDirections"; +const TStringBuf RowSpecAttrSortMembers = "SortMembers"; +const TStringBuf RowSpecAttrSortedBy = "SortedBy"; +const TStringBuf RowSpecAttrSortedByTypes = "SortedByTypes"; +const TStringBuf RowSpecAttrDefaultValues = "DefaultValues"; +const TStringBuf RowSpecAttrWeakFields = "WeakFields"; +const TStringBuf RowSpecAttrUseTypeV2 = "UseTypeV2"; +const TStringBuf RowSpecAttrUseNativeYtTypes = "UseNativeYtTypes"; +const TStringBuf RowSpecAttrNativeYtTypeFlags = "NativeYtTypeFlags"; +const TStringBuf RowSpecAttrExplicitYson = "ExplicitYson"; +const TStringBuf RowSpecAttrConstraints = "Constraints"; + +const TStringBuf YqlReadUdfAttribute = "_yql_read_udf"; +const TStringBuf YqlReadUdfTypeConfigAttribute = "_yql_read_udf_type_config"; +const TStringBuf YqlReadUdfRunConfigAttribute = "_yql_read_udf_run_config"; +const TStringBuf YqlViewPrefixAttribute = "_yql_view_"; +const TStringBuf YqlProtoFieldPrefixAttribute = "_yql_proto_field_"; +const TStringBuf YqlDynamicAttribute = "_yql_dynamic"; + +const TStringBuf YqlSysColumnPrefix = "_yql_sys_table"; +const TStringBuf YqlSysColumnPath = "_yql_sys_tablepath"; +const TStringBuf YqlSysColumnRecord = "_yql_sys_tablerecord"; +const TStringBuf YqlSysColumnIndex = "_yql_sys_tableindex"; +const TStringBuf YqlSysColumnNum = "_yql_sys_tablenum"; +const TStringBuf YqlSysColumnKeySwitch = "_yql_sys_tablekeyswitch"; + +const TStringBuf YqlOthersColumnName = "_other"; +const TStringBuf YqlFakeColumnName = "_yql_fake_column"; +const TStringBuf YqlTypeAttribute = "_yql_type"; +const TStringBuf YqlTypeView = "view"; + +const TStringBuf QB2Premapper = "_qb2_premapper"; + +// Generated by SQL parser +const TStringBuf MrTableRangeName = "MrTableRange"; +const TStringBuf MrTableRangeStrictName = "MrTableRangeStrict"; +const TStringBuf MrTableConcatName = "MrTableConcat"; +const TStringBuf MrFolderName = "MrFolder"; +const TStringBuf MrWalkFoldersName = "MrWalkFolders"; +const TStringBuf MrWalkFoldersImplName = "MrWalkFoldersImpl"; +const TStringBuf MrRangeInputListInternal = "MrRangeInputListInternal"; + +// YT related names +const TStringBuf READ_SCHEMA_ATTR_NAME = "_read_schema"; +const TStringBuf INFER_SCHEMA_ATTR_NAME = "_infer_schema"; +const TStringBuf SCHEMA_ATTR_NAME = "schema"; +const TStringBuf SCHEMA_MODE_ATTR_NAME = "schema_mode"; +const TStringBuf FORMAT_ATTR_NAME = "_format"; + +const TStringBuf KeyFilterName = "keyFilter"; +const TStringBuf CurrentYtClusterShortcut = "current"; + +const TStringBuf YqlIOSpecTables = "tables"; +const TStringBuf YqlIOSpecRegistry = "registry"; + +const TStringBuf SecurityTagsName = "security_tags"; + +extern const TVector<TStringBuf> YAMR_FIELDS; + +} diff --git a/yt/yql/providers/yt/common/yql_yt_settings.cpp b/yt/yql/providers/yt/common/yql_yt_settings.cpp new file mode 100644 index 0000000000..d74a0e6044 --- /dev/null +++ b/yt/yql/providers/yt/common/yql_yt_settings.cpp @@ -0,0 +1,570 @@ +#include "yql_yt_settings.h" + +#include <yql/essentials/providers/common/codec/yql_codec_type_flags.h> +#include <yql/essentials/utils/log/log.h> +#include <yql/essentials/public/udf/udf_data_type.h> + +#include <library/cpp/yson/node/node_io.h> +#include <library/cpp/regex/pcre/regexp.h> +#include <library/cpp/string_utils/parse_size/parse_size.h> + +#include <util/generic/yexception.h> +#include <util/generic/size_literals.h> +#include <util/generic/vector.h> +#include <util/string/cast.h> +#include <util/string/split.h> +#include <util/system/compiler.h> + +namespace NYql { + +using namespace NCommon; + +namespace { + +// See https://wiki.yandex-team.ru/yt/userdoc/compression/#podderzhivaemyealgoritmyszhatija +const TRegExMatch CODECS("none|snappy|zlib_[1-9]|lz4(_high_compression)?|quick_lz|zstd_([1-9]|1[0-9]|2[0-1])|brotli_([1-9]|1[0-1])|lzma_[0-9]|bzip2_[1-9]"); + +} // namespace + +bool ValidateCompressionCodecValue(const TStringBuf& codec) { + return CODECS.Match(codec.data()); +} + +void MediaValidator(const NYT::TNode& value) { + if (!value.IsMap()) { + throw yexception() << "Expected yson map, but got " << value.GetType(); + } + for (auto& p: value.AsMap()) { + if (!p.second.IsMap()) { + throw yexception() << "Expected yson map, but got " << p.second.GetType() << "\" for the " << p.first.Quote() << " key"; + } + for (auto& pp: p.second.AsMap()) { + if (pp.first == "replication_factor") { + if (!pp.second.IsInt64()) { + throw yexception() << "Expected int64, but got \"" << pp.second.GetType() << "\" for the " << p.first << "." << pp.first << " key"; + } + } else if (pp.first == "data_parts_only") { + if (!pp.second.IsBool()) { + throw yexception() << "Expected bool, but got \"" << pp.second.GetType() << "\" for the " << p.first << "." << pp.first << " key"; + } + } else { + throw yexception() << "Expected key in "<< p.first <<".[replication_factor, data_parts_only], but got " << p.first << "." << pp.first << " key"; + } + } + } +} + +TYtConfiguration::TYtConfiguration(TTypeAnnotationContext& typeCtx) +{ + const auto codecValidator = [] (const TString&, TString str) { + if (!ValidateCompressionCodecValue(str)) { + throw yexception() << "Bad codec value"; + } + }; + + const auto mediaValidator = [] (const TString&, const NYT::TNode& value) { + MediaValidator(value); + }; + + REGISTER_SETTING(*this, Auth) + .ValueSetter([this](const TString&, const TString& value) { + Auth = value; + for (auto& x: Tokens) { + x.second = value; + } + }); + REGISTER_SETTING(*this, ExternalTx); + REGISTER_SETTING(*this, TmpFolder); + REGISTER_SETTING(*this, TablesTmpFolder); + REGISTER_SETTING(*this, TempTablesTtl); + REGISTER_SETTING(*this, KeepTempTables) + .ValueSetter([this](const TString& cluster, bool value) { + Y_UNUSED(cluster); + if (value) { + ReleaseTempData = EReleaseTempDataMode::Never; + } + }); + REGISTER_SETTING(*this, InflightTempTablesLimit); + REGISTER_SETTING(*this, ReleaseTempData).Parser([](const TString& v) { return FromString<EReleaseTempDataMode>(v); }); + REGISTER_SETTING(*this, IgnoreYamrDsv); + REGISTER_SETTING(*this, IgnoreWeakSchema); + REGISTER_SETTING(*this, InferSchema) + .Lower(1) + .Upper(1000) + .Parser([](const TString& v) { return v.empty() ? 1 : Max(1u, FromString<ui32>(v)); }) + .Validator([this](const TString&, ui32) { + if (ForceInferSchema.Get()) { + throw yexception() << "InferSchema cannot be used together with ForceInferSchema"; + } + }); + + REGISTER_SETTING(*this, ForceInferSchema) + .Lower(1) + .Upper(1000) + .Parser([](const TString& v) { return v.empty() ? 1 : Max(1u, FromString<ui32>(v)); }) + .Validator([this](const TString&, ui32) { + if (InferSchema.Get()) { + throw yexception() << "ForceInferSchema cannot be used together with InferSchema"; + } + }); + + REGISTER_SETTING(*this, InferSchemaTableCountThreshold); + + REGISTER_SETTING(*this, QueryCacheMode).Parser([](const TString& v) { return FromString<EQueryCacheMode>(v); }); + REGISTER_SETTING(*this, QueryCacheIgnoreTableRevision); + REGISTER_SETTING(*this, QueryCacheSalt); + REGISTER_SETTING(*this, QueryCacheTtl); + REGISTER_SETTING(*this, QueryCacheUseForCalc); + REGISTER_SETTING(*this, QueryCacheUseExpirationTimeout); + + REGISTER_SETTING(*this, DefaultMemoryLimit); + REGISTER_SETTING(*this, DefaultMemoryReserveFactor).Lower(0.0).Upper(1.0); + REGISTER_SETTING(*this, DefaultMemoryDigestLowerBound).Lower(0.0).Upper(1.0); + REGISTER_SETTING(*this, DefaultMaxJobFails); + REGISTER_SETTING(*this, MaxRowWeight).Lower(1).Upper(128_MB); + REGISTER_SETTING(*this, MaxKeyWeight).Lower(1).Upper(256_KB); + REGISTER_SETTING(*this, BufferRowCount).Lower(1); + REGISTER_SETTING(*this, DataSizePerJob); + REGISTER_SETTING(*this, DataSizePerSortJob).Lower(10_MB); + REGISTER_SETTING(*this, DataSizePerMapJob); + REGISTER_SETTING(*this, DataSizePerPartition); + REGISTER_SETTING(*this, DefaultLocalityTimeout); + REGISTER_SETTING(*this, MapLocalityTimeout); + REGISTER_SETTING(*this, ReduceLocalityTimeout); + REGISTER_SETTING(*this, SortLocalityTimeout); + REGISTER_SETTING(*this, MinLocalityInputDataWeight); + REGISTER_SETTING(*this, MaxJobCount).Lower(1); + REGISTER_SETTING(*this, UserSlots).Lower(1); + REGISTER_SETTING(*this, Pool).NonEmpty(); + REGISTER_SETTING(*this, DefaultOperationWeight).Lower(0.0); + REGISTER_SETTING(*this, DefaultMapSelectivityFactor).Lower(0.0); + REGISTER_SETTING(*this, NightlyCompress); + REGISTER_SETTING(*this, PublishedCompressionCodec).Validator(codecValidator); + REGISTER_SETTING(*this, TemporaryCompressionCodec).Validator(codecValidator); + // See https://wiki.yandex-team.ru/yt/userdoc/chunkowners/#replikacija + REGISTER_SETTING(*this, PublishedErasureCodec).Parser([](const TString& v) { return FromString<NYT::EErasureCodecAttr>(v); }); + REGISTER_SETTING(*this, TemporaryErasureCodec).Parser([](const TString& v) { return FromString<NYT::EErasureCodecAttr>(v); }); + REGISTER_SETTING(*this, ClientMapTimeout).Deprecated(); + REGISTER_SETTING(*this, CoreDumpPath).NonEmpty(); + REGISTER_SETTING(*this, UseTmpfs); + REGISTER_SETTING(*this, SuspendIfAccountLimitExceeded); + REGISTER_SETTING(*this, ExtraTmpfsSize); + REGISTER_SETTING(*this, OptimizeFor) + .Parser([](const TString& v) { + return FromString<NYT::EOptimizeForAttr>(v); + }); + + REGISTER_SETTING(*this, DefaultCluster) + .Validator([this] (const TString&, TString value) { + if (!ValidClusters.contains(value)) { + throw yexception() << "Unknown cluster name: " << value; + } + }); + REGISTER_SETTING(*this, UseTypeV2) + .ValueSetter([this](const TString& cluster, bool value) { + Y_UNUSED(cluster); + UseNativeYtTypes = value; + }) + .Warning("Pragma UseTypeV2 is deprecated. Use UseNativeYtTypes instead"); + REGISTER_SETTING(*this, UseNativeYtTypes); + REGISTER_SETTING(*this, UseNativeDescSort); + REGISTER_SETTING(*this, UseIntermediateSchema).Deprecated(); + REGISTER_SETTING(*this, UseIntermediateStreams); + REGISTER_SETTING(*this, StaticPool); + REGISTER_SETTING(*this, UseFlow) + .ValueSetter([this](const TString&, bool value) { + UseFlow = value; + if (value) { + UseSystemColumns = true; + } else { + MapJoinUseFlow = false; + } + }); + REGISTER_SETTING(*this, WideFlowLimit) + .ValueSetter([this](const TString&, ui16 value) { + WideFlowLimit = value; + if (value > 0) { + UseSystemColumns = true; + } + }); + REGISTER_SETTING(*this, ExpirationDeadline) + .Lower(typeCtx.QContext.CanRead() ? TInstant::Zero() : Now()) + .ValueSetter([this] (const TString& cluster, TInstant value) { + ExpirationDeadline[cluster] = value; + }); + REGISTER_SETTING(*this, ExpirationInterval) + .ValueSetter([this] (const TString& cluster, TDuration value) { + ExpirationInterval[cluster] = value; + }); + REGISTER_SETTING(*this, ScriptCpu).Lower(1.0).GlobalOnly(); + REGISTER_SETTING(*this, PythonCpu).Lower(1.0).GlobalOnly(); + REGISTER_SETTING(*this, JavascriptCpu).Lower(1.0).GlobalOnly(); + REGISTER_SETTING(*this, ErasureCodecCpu).Lower(1.0); + REGISTER_SETTING(*this, ErasureCodecCpuForDq).Lower(1.0); + + REGISTER_SETTING(*this, Owners) + .NonEmpty() + .ValueSetterWithRestore([this] (const TString& cluster, TSet<TString> owners) { + if (ALL_CLUSTERS == cluster) { + Owners.UpdateAll([&owners] (const TString&, TSet<TString>& val) { + val.insert(owners.begin(), owners.end()); + }); + } else { + Owners[cluster].insert(owners.begin(), owners.end()); + } + }); + REGISTER_SETTING(*this, OperationReaders).NonEmpty(); + REGISTER_SETTING(*this, SchedulingTag); + REGISTER_SETTING(*this, SchedulingTagFilter); + REGISTER_SETTING(*this, PoolTrees) + .NonEmpty() + .ValueSetter([this] (const TString& cluster, TSet<TString> trees) { + HybridDqExecution = false; + PoolTrees[cluster] = trees; + }); + REGISTER_SETTING(*this, TentativePoolTrees).NonEmpty(); + REGISTER_SETTING(*this, TentativeTreeEligibilitySampleJobCount); + REGISTER_SETTING(*this, TentativeTreeEligibilityMaxJobDurationRatio); + REGISTER_SETTING(*this, TentativeTreeEligibilityMinJobDuration); + REGISTER_SETTING(*this, UseDefaultTentativePoolTrees); + REGISTER_SETTING(*this, IntermediateAccount).NonEmpty(); + REGISTER_SETTING(*this, IntermediateReplicationFactor).Lower(1).Upper(10); + REGISTER_SETTING(*this, PublishedReplicationFactor).Lower(1).Upper(10); + REGISTER_SETTING(*this, TemporaryReplicationFactor).Lower(1).Upper(10); + REGISTER_SETTING(*this, AutoMerge).Enum({"relaxed", "economy", "disabled"}) + .ValueSetter([this](const TString& cluster, const TString& value) { + PublishedAutoMerge[cluster] = value; + TemporaryAutoMerge[cluster] = value; + }); + + REGISTER_SETTING(*this, PublishedAutoMerge).Enum({ "relaxed", "economy", "disabled" }); + REGISTER_SETTING(*this, TemporaryAutoMerge).Enum({ "relaxed", "economy", "disabled" }); + REGISTER_SETTING(*this, UseSkiff) + .ValueSetter([this](const TString& cluster, bool value) { + UseSkiff[cluster] = value; + if (!value) { + UseNativeYtTypes = false; + } + }); + REGISTER_SETTING(*this, TableContentCompressLevel).Upper(11); + REGISTER_SETTING(*this, TableContentDeliveryMode).Parser([](const TString& v) { return FromString<ETableContentDeliveryMode>(v); }); + REGISTER_SETTING(*this, TableContentMaxChunksForNativeDelivery).Upper(1000); + REGISTER_SETTING(*this, TableContentTmpFolder); + REGISTER_SETTING(*this, TableContentColumnarStatistics); + REGISTER_SETTING(*this, TableContentUseSkiff); + REGISTER_SETTING(*this, TableContentLocalExecution) + .Parser([](const TString& v) { + // backward compatible parse from bool + bool value = true; + if (!v || TryFromString<bool>(v, value)) { + return value ? 10_MB : 0_MB; + } else { + return NSize::ParseSize(v); + } + }) + .Upper(5_GB); + REGISTER_SETTING(*this, DisableJobSplitting); + REGISTER_SETTING(*this, UseColumnarStatistics) + .Parser([](const TString& v) { + // backward compatible parse from bool + bool value = true; + if (!v || TryFromString<bool>(v, value)) { + return value ? EUseColumnarStatisticsMode::Force : EUseColumnarStatisticsMode::Disable; + } else { + return FromString<EUseColumnarStatisticsMode>(v); + } + }) + ; + REGISTER_SETTING(*this, ParallelOperationsLimit).Lower(1); + REGISTER_SETTING(*this, LocalCalcLimit).Lower(1); + REGISTER_SETTING(*this, DefaultCalcMemoryLimit); + REGISTER_SETTING(*this, LayerPaths).NonEmpty() + .ValueSetter([this](const TString& cluster, const TVector<TString>& value) { + LayerPaths[cluster] = value; + HybridDqExecution = false; + }); + REGISTER_SETTING(*this, DockerImage).NonEmpty() + .ValueSetter([this](const TString& cluster, const TString& value) { + DockerImage[cluster] = value; + HybridDqExecution = false; + }); + REGISTER_SETTING(*this, _EnableDq); + // Deprecated. Use MaxInputTables instead + REGISTER_SETTING(*this, ExtendTableLimit).Lower(2).Upper(3000) + .ValueSetter([this] (const TString& cluster, ui32 value) { + Y_UNUSED(cluster); + MaxInputTables = value; + }) + .Warning("Pragma ExtendTableLimit is deprecated. Use MaxInputTables instead"); + REGISTER_SETTING(*this, CommonJoinCoreLimit); + REGISTER_SETTING(*this, CombineCoreLimit).Lower(1_MB); // Min 1Mb + REGISTER_SETTING(*this, SwitchLimit).Lower(1_MB); // Min 1Mb + REGISTER_SETTING(*this, JoinMergeTablesLimit); + REGISTER_SETTING(*this, JoinMergeUseSmallAsPrimary); + REGISTER_SETTING(*this, JoinMergeReduceJobMaxSize).Lower(1); // YT requires max_data_size_per_job to be > 0, YT default is 200GB + REGISTER_SETTING(*this, JoinMergeUnsortedFactor).Lower(0.0); + REGISTER_SETTING(*this, JoinMergeForce); + REGISTER_SETTING(*this, MapJoinLimit); + REGISTER_SETTING(*this, MapJoinShardMinRows); + REGISTER_SETTING(*this, MapJoinShardCount).Lower(1).Upper(10); + REGISTER_SETTING(*this, MapJoinUseFlow); + REGISTER_SETTING(*this, EvaluationTableSizeLimit).Upper(10_MB); // Max 10Mb + REGISTER_SETTING(*this, LookupJoinLimit).Upper(10_MB); // Same as EvaluationTableSizeLimit + REGISTER_SETTING(*this, LookupJoinMaxRows).Upper(10000); + REGISTER_SETTING(*this, DisableOptimizers); + REGISTER_SETTING(*this, MaxInputTables).Lower(2).Upper(3000); // 3000 - default max limit on YT clusters + REGISTER_SETTING(*this, MaxOutputTables).Lower(1).Upper(100); // https://ml.yandex-team.ru/thread/yt/166633186212752141/ + REGISTER_SETTING(*this, MaxInputTablesForSortedMerge).Lower(2).Upper(1000); // https://st.yandex-team.ru/YTADMINREQ-16742 + REGISTER_SETTING(*this, DisableFuseOperations); + REGISTER_SETTING(*this, MaxExtraJobMemoryToFuseOperations); + REGISTER_SETTING(*this, MaxReplicationFactorToFuseOperations).Lower(1.0); + REGISTER_SETTING(*this, MaxOperationFiles).Lower(2).Upper(1000); + REGISTER_SETTING(*this, GeobaseDownloadUrl); + REGISTER_SETTING(*this, MinPublishedAvgChunkSize); + REGISTER_SETTING(*this, MinTempAvgChunkSize); + REGISTER_SETTING(*this, TopSortMaxLimit); + REGISTER_SETTING(*this, TopSortSizePerJob).Lower(1); + REGISTER_SETTING(*this, TopSortRowMultiplierPerJob).Lower(1); + REGISTER_SETTING(*this, JoinUseColumnarStatistics) + .ValueSetter([this](const TString& arg, bool value) { + Y_UNUSED(arg); + if (!value) { + JoinCollectColumnarStatistics = EJoinCollectColumnarStatisticsMode::Disable; + } + }) + .Warning("Pragma JoinUseColumnarStatistics is deprecated. Use JoinCollectColumnarStatistics instead"); + REGISTER_SETTING(*this, JoinCollectColumnarStatistics) + .Parser([](const TString& v) { return FromString<EJoinCollectColumnarStatisticsMode>(v); }); + REGISTER_SETTING(*this, JoinColumnarStatisticsFetcherMode) + .Parser([](const TString& v) { return FromString<NYT::EColumnarStatisticsFetcherMode>(v); }); + REGISTER_SETTING(*this, JoinWaitAllInputs); + REGISTER_SETTING(*this, JoinAllowColumnRenames); + REGISTER_SETTING(*this, JoinMergeSetTopLevelFullSort); + REGISTER_SETTING(*this, JoinEnableStarJoin); + REGISTER_SETTING(*this, JobEnv) + .Parser([](const TString& v) { return NYT::NodeFromYsonString(v, ::NYson::EYsonType::Node); }) + .Validator([] (const TString&, const NYT::TNode& value) { + if (!value.IsMap()) { + throw yexception() << "Expected yson map, but got " << value.GetType(); + } + for (auto& p: value.AsMap()) { + if (!p.second.IsString()) { + throw yexception() << "Expected string, but got \"" << p.second.GetType() << "\" for the " << p.first.Quote() << " key"; + } + } + }); + REGISTER_SETTING(*this, OperationSpec) + .Parser([](const TString& v) { return NYT::NodeFromYsonString(v, ::NYson::EYsonType::Node); }) + .Validator([] (const TString&, const NYT::TNode& value) { + if (!value.IsMap()) { + throw yexception() << "Expected yson map, but got " << value.GetType(); + } + }) + .ValueSetter([this](const TString& cluster, const NYT::TNode& spec) { + OperationSpec[cluster] = spec; + HybridDqExecution = false; + }); + REGISTER_SETTING(*this, Annotations) + .Parser([](const TString& v) { return NYT::NodeFromYsonString(v); }) + .Validator([] (const TString&, const NYT::TNode& value) { + if (!value.IsMap()) { + throw yexception() << "Expected yson map, but got " << value.GetType(); + } + }); + REGISTER_SETTING(*this, Description) + .Parser([](const TString& v) { return NYT::NodeFromYsonString(v); }) + .Validator([] (const TString&, const NYT::TNode& value) { + if (!value.IsMap()) { + throw yexception() << "Expected yson map, but got " << value.GetType(); + } + }); + REGISTER_SETTING(*this, StartedBy) + .Parser([](const TString& v) { return NYT::NodeFromYsonString(v); }) + .Validator([] (const TString&, const NYT::TNode& value) { + if (!value.IsMap()) { + throw yexception() << "Expected yson map, but got " << value.GetType(); + } + }); + REGISTER_SETTING(*this, MaxSpeculativeJobCountPerTask); + REGISTER_SETTING(*this, LLVMMemSize); + REGISTER_SETTING(*this, LLVMPerNodeMemSize); + REGISTER_SETTING(*this, LLVMNodeCountLimit); + REGISTER_SETTING(*this, SamplingIoBlockSize); + REGISTER_SETTING(*this, BinaryTmpFolder); + REGISTER_SETTING(*this, _BinaryCacheFolder); + REGISTER_SETTING(*this, BinaryExpirationInterval); + REGISTER_SETTING(*this, FolderInlineDataLimit); + REGISTER_SETTING(*this, FolderInlineItemsLimit); + REGISTER_SETTING(*this, TableContentMinAvgChunkSize); + REGISTER_SETTING(*this, TableContentMaxInputTables); + REGISTER_SETTING(*this, UseSystemColumns) + .ValueSetter([this](const TString&, bool value) { + UseSystemColumns = value; + if (!value) { + UseFlow = false; + WideFlowLimit = 0; + } + }); + REGISTER_SETTING(*this, PublishedMedia) + .Parser([](const TString& v) { return NYT::NodeFromYsonString(v, ::NYson::EYsonType::Node); }) + .Validator(mediaValidator); + REGISTER_SETTING(*this, TemporaryMedia) + .Parser([](const TString& v) { return NYT::NodeFromYsonString(v, ::NYson::EYsonType::Node); }) + .Validator(mediaValidator); + REGISTER_SETTING(*this, PublishedPrimaryMedium); + REGISTER_SETTING(*this, TemporaryPrimaryMedium); + REGISTER_SETTING(*this, IntermediateDataMedium); + REGISTER_SETTING(*this, PrimaryMedium).ValueSetter([this](const TString& cluster, const TString& value) { + PublishedPrimaryMedium[cluster] = value; + TemporaryPrimaryMedium[cluster] = value; + IntermediateDataMedium[cluster] = value; + }); + REGISTER_SETTING(*this, QueryCacheChunkLimit); + REGISTER_SETTING(*this, IgnoreTypeV3); + REGISTER_SETTING(*this, HybridDqExecution); + REGISTER_SETTING(*this, HybridDqDataSizeLimitForOrdered); + REGISTER_SETTING(*this, HybridDqDataSizeLimitForUnordered); + REGISTER_SETTING(*this, HybridDqExecutionFallback); + REGISTER_SETTING(*this, NativeYtTypeCompatibility) + .Parser([](const TString& v) { + ui64 res = 0; + TVector<TString> vec; + StringSplitter(v).SplitBySet(",;| ").AddTo(&vec); + for (auto& s: vec) { + if (s.empty()) { + throw yexception() << "Empty value item"; + } + res |= FromString<ENativeTypeCompatFlags>(s); + } + return res; + }); + REGISTER_SETTING(*this, _UseKeyBoundApi); + REGISTER_SETTING(*this, UseYqlRowSpecCompactForm); + REGISTER_SETTING(*this, UseNewPredicateExtraction); + REGISTER_SETTING(*this, PruneKeyFilterLambda); + REGISTER_SETTING(*this, DqPruneKeyFilterLambda); + REGISTER_SETTING(*this, MergeAdjacentPointRanges); + REGISTER_SETTING(*this, KeyFilterForStartsWith); + REGISTER_SETTING(*this, MaxKeyRangeCount).Upper(10000); + REGISTER_SETTING(*this, MaxChunksForDqRead).Lower(1); + REGISTER_SETTING(*this, NetworkProject); + REGISTER_SETTING(*this, FileCacheTtl); + REGISTER_SETTING(*this, _ImpersonationUser); + REGISTER_SETTING(*this, InferSchemaMode).Parser([](const TString& v) { return FromString<EInferSchemaMode>(v); }); + REGISTER_SETTING(*this, BatchListFolderConcurrency).Lower(1); // Upper bound on concurrent batch folder list requests https://yt.yandex-team.ru/docs/api/commands#execute_batch + REGISTER_SETTING(*this, ForceTmpSecurity); + REGISTER_SETTING(*this, JoinCommonUseMapMultiOut); + REGISTER_SETTING(*this, _EnableYtPartitioning); + REGISTER_SETTING(*this, UseAggPhases); + REGISTER_SETTING(*this, UsePartitionsByKeysForFinalAgg); + REGISTER_SETTING(*this, ForceJobSizeAdjuster); + REGISTER_SETTING(*this, EnforceJobUtc); + REGISTER_SETTING(*this, UseRPCReaderInDQ); + REGISTER_SETTING(*this, DQRPCReaderInflight).Lower(1); + REGISTER_SETTING(*this, DQRPCReaderTimeout); + REGISTER_SETTING(*this, BlockReaderSupportedTypes); + REGISTER_SETTING(*this, BlockReaderSupportedDataTypes) + .Parser([](const TString& v) { + TSet<TString> vec; + StringSplitter(v).SplitBySet(",").AddTo(&vec); + TSet<NUdf::EDataSlot> res; + for (auto& s: vec) { + res.emplace(NUdf::GetDataSlot(s)); + } + return res; + }); + REGISTER_SETTING(*this, JobBlockInputSupportedTypes); + REGISTER_SETTING(*this, JobBlockInputSupportedDataTypes) + .Parser([](const TString& v) { + TSet<TString> vec; + StringSplitter(v).SplitBySet(",").AddTo(&vec); + TSet<NUdf::EDataSlot> res; + for (auto& s: vec) { + res.emplace(NUdf::GetDataSlot(s)); + } + return res; + }); + REGISTER_SETTING(*this, MaxCpuUsageToFuseMultiOuts).Lower(1.0); + REGISTER_SETTING(*this, MaxReplicationFactorToFuseMultiOuts).Lower(1.0); + REGISTER_SETTING(*this, ApplyStoredConstraints) + .Parser([](const TString& v) { + ui64 res = 0; + TVector<TString> vec; + StringSplitter(v).SplitBySet(",;| ").AddTo(&vec); + for (auto& s: vec) { + if (s.empty()) { + throw yexception() << "Empty value item"; + } + res |= ui64(FromString<EStoredConstraint>(s)); + } + return res; + }); + REGISTER_SETTING(*this, ViewIsolation); + REGISTER_SETTING(*this, PartitionByConstantKeysViaMap); + REGISTER_SETTING(*this, ColumnGroupMode) + .Parser([](const TString& v) { + return FromString<EColumnGroupMode>(v); + }); + REGISTER_SETTING(*this, MinColumnGroupSize).Lower(2); + REGISTER_SETTING(*this, MaxColumnGroups); + REGISTER_SETTING(*this, ExtendedStatsMaxChunkCount); + REGISTER_SETTING(*this, JobBlockInput); + REGISTER_SETTING(*this, _EnableYtDqProcessWriteConstraints); +} + +EReleaseTempDataMode GetReleaseTempDataMode(const TYtSettings& settings) { + return settings.ReleaseTempData.Get().GetOrElse(EReleaseTempDataMode::Finish); +} + +EJoinCollectColumnarStatisticsMode GetJoinCollectColumnarStatisticsMode(const TYtSettings& settings) { + return settings.JoinCollectColumnarStatistics.Get().GetOrElse(EJoinCollectColumnarStatisticsMode::Async); +} + +TYtSettings::TConstPtr TYtConfiguration::Snapshot() const { + return std::make_shared<const TYtSettings>(*this); +} + +size_t TYtVersionedConfiguration::FindNodeVer(const TExprNode& node) { + auto it = NodeIdToVer.find(node.UniqueId()); + if (it != NodeIdToVer.end()) { + return it->second; + } + + size_t ver = 0; + for (auto& child: node.Children()) { + ver = Max<size_t>(ver, FindNodeVer(*child)); + } + NodeIdToVer.emplace(node.UniqueId(), ver); + return ver; +} + +void TYtVersionedConfiguration::FreezeZeroVersion() { + if (Y_UNLIKELY(FrozenSettings.empty())) { + FrozenSettings.push_back(Snapshot()); + } +} + +void TYtVersionedConfiguration::PromoteVersion(const TExprNode& node) { + NodeIdToVer[node.UniqueId()] = FrozenSettings.size(); + FrozenSettings.push_back(Snapshot()); +} + +TYtSettings::TConstPtr TYtVersionedConfiguration::GetSettingsForNode(const TExprNode& node) { + FreezeZeroVersion(); + size_t ver = FindNodeVer(node); + YQL_CLOG(DEBUG, ProviderYt) << "Using settings ver." << ver; + return FrozenSettings.at(ver); +} + +TYtSettings::TConstPtr TYtVersionedConfiguration::GetSettingsVer(size_t ver) { + FreezeZeroVersion(); + YQL_CLOG(DEBUG, ProviderYt) << "Using settings ver." << ver; + return FrozenSettings.at(ver); +} + +void TYtVersionedConfiguration::ClearVersions() { + FrozenSettings.clear(); + NodeIdToVer.clear(); +} + +} diff --git a/yt/yql/providers/yt/common/yql_yt_settings.h b/yt/yql/providers/yt/common/yql_yt_settings.h new file mode 100644 index 0000000000..724f9fb7f3 --- /dev/null +++ b/yt/yql/providers/yt/common/yql_yt_settings.h @@ -0,0 +1,377 @@ +#pragma once + +#include <yql/essentials/providers/common/config/yql_dispatch.h> +#include <yql/essentials/providers/common/config/yql_setting.h> +#include <yql/essentials/ast/yql_expr.h> + +#include <library/cpp/string_utils/parse_size/parse_size.h> + +#include <yt/cpp/mapreduce/interface/common.h> +#include <yt/cpp/mapreduce/interface/client_method_options.h> +#include <library/cpp/yson/node/node.h> + +#include <util/datetime/base.h> +#include <util/generic/guid.h> +#include <util/generic/string.h> +#include <util/generic/ptr.h> +#include <util/generic/set.h> +#include <util/generic/vector.h> + +#include <unordered_map> + +namespace NYql { + +enum class EQueryCacheMode { + Disable /* "disable" */, + Readonly /* "readonly" */, + Refresh /* "refresh" */, + Normal /* "normal" */, +}; + +enum class EReleaseTempDataMode { + Never /* "never" */, + Immediate /* "immediate" */, + Finish /* "finish" */, +}; + +enum class ETableContentDeliveryMode { + Native /* "native" */, + File /* "file" */, +}; + +enum class EJoinCollectColumnarStatisticsMode { + Disable /* "disable" */, + Sync /* "sync" */, + Async /* "async" */, +}; + +enum class EUseColumnarStatisticsMode { + Disable /* "disable" */, + Auto /* "auto" */, + Force /* "force" */, +}; + +enum class EStoredConstraint : ui64 { + None = 0ULL /* "None" */, + Sorted = 1ULL /* "Sorted" */, + Unique = 2ULL /* "Unique" */, + Distinct = 4ULL /* "Distinct" */, +}; + +enum class EInferSchemaMode { + Sequential = 0ULL /* "sequential" */, + Parallel = 1ULL /* "parallel" */, + RPC = 2ULL /* "rpc" */, +}; + +enum class EColumnGroupMode { + Disable /* "disable" */, + Single /* "single" */, + PerUsage /* "perusage", "per-usage" */, +}; + +struct TYtSettings { + using TConstPtr = std::shared_ptr<const TYtSettings>; + + // should be static, because are used on earlier stages + NCommon::TConfSetting<TString, false> Auth; + NCommon::TConfSetting<TGUID, false> ExternalTx; + NCommon::TConfSetting<TString, false> TmpFolder; + NCommon::TConfSetting<TString, false> TablesTmpFolder; + NCommon::TConfSetting<TDuration, false> TempTablesTtl; + NCommon::TConfSetting<bool, false> KeepTempTables; + NCommon::TConfSetting<ui32, false> InflightTempTablesLimit; + NCommon::TConfSetting<EReleaseTempDataMode, false> ReleaseTempData; + NCommon::TConfSetting<bool, false> IgnoreYamrDsv; + NCommon::TConfSetting<bool, false> IgnoreWeakSchema; + NCommon::TConfSetting<ui32, false> InferSchema; + NCommon::TConfSetting<ui32, false> ForceInferSchema; + NCommon::TConfSetting<ui32, false> InferSchemaTableCountThreshold; + NCommon::TConfSetting<NSize::TSize, false> DefaultCalcMemoryLimit; + NCommon::TConfSetting<ui32, false> ParallelOperationsLimit; + NCommon::TConfSetting<ui32, false> LocalCalcLimit; + NCommon::TConfSetting<EQueryCacheMode, false> QueryCacheMode; + NCommon::TConfSetting<bool, false> QueryCacheIgnoreTableRevision; + NCommon::TConfSetting<TString, false> QueryCacheSalt; + NCommon::TConfSetting<TDuration, false> QueryCacheTtl; + NCommon::TConfSetting<bool, false> QueryCacheUseExpirationTimeout; + NCommon::TConfSetting<bool, false> QueryCacheUseForCalc; + NCommon::TConfSetting<ui32, false> DefaultMaxJobFails; + NCommon::TConfSetting<TString, false> CoreDumpPath; + NCommon::TConfSetting<TString, false> DefaultCluster; + NCommon::TConfSetting<TString, false> StaticPool; + NCommon::TConfSetting<TString, false> BinaryTmpFolder; + NCommon::TConfSetting<TDuration, false> BinaryExpirationInterval; + NCommon::TConfSetting<bool, false> IgnoreTypeV3; + NCommon::TConfSetting<bool, false> _UseMultisetAttributes; + NCommon::TConfSetting<TDuration, false> FileCacheTtl; + NCommon::TConfSetting<TString, false> _ImpersonationUser; + NCommon::TConfSetting<EInferSchemaMode, false> InferSchemaMode; + NCommon::TConfSetting<ui32, false> BatchListFolderConcurrency; + NCommon::TConfSetting<bool, false> ForceTmpSecurity; + + // Job runtime + NCommon::TConfSetting<TString, true> Pool; + NCommon::TConfSetting<NSize::TSize, true> DefaultMemoryLimit; + NCommon::TConfSetting<double, true> DefaultMemoryReserveFactor; + NCommon::TConfSetting<double, true> DefaultMemoryDigestLowerBound; + NCommon::TConfSetting<NSize::TSize, true> MaxRowWeight; + NCommon::TConfSetting<NSize::TSize, true> MaxKeyWeight; + NCommon::TConfSetting<ui32, true> BufferRowCount; + NCommon::TConfSetting<NSize::TSize, true> DataSizePerJob; + NCommon::TConfSetting<NSize::TSize, true> DataSizePerSortJob; + NCommon::TConfSetting<NSize::TSize, true> DataSizePerMapJob; + NCommon::TConfSetting<NSize::TSize, true> DataSizePerPartition; + NCommon::TConfSetting<TDuration, true> DefaultLocalityTimeout; + NCommon::TConfSetting<TDuration, true> MapLocalityTimeout; + NCommon::TConfSetting<TDuration, true> ReduceLocalityTimeout; + NCommon::TConfSetting<TDuration, true> SortLocalityTimeout; + NCommon::TConfSetting<NSize::TSize, true> MinLocalityInputDataWeight; + NCommon::TConfSetting<ui64, true> MaxJobCount; + NCommon::TConfSetting<ui64, true> UserSlots; + NCommon::TConfSetting<double, true> DefaultOperationWeight; + NCommon::TConfSetting<double, true> DefaultMapSelectivityFactor; + NCommon::TConfSetting<bool, true> NightlyCompress; + NCommon::TConfSetting<TString, true> PublishedCompressionCodec; + NCommon::TConfSetting<TString, true> TemporaryCompressionCodec; + NCommon::TConfSetting<NYT::EErasureCodecAttr, true> PublishedErasureCodec; + NCommon::TConfSetting<NYT::EErasureCodecAttr, true> TemporaryErasureCodec; + NCommon::TConfSetting<TDuration, true> ClientMapTimeout; // TODO: yt_native + NCommon::TConfSetting<bool, true> UseTmpfs; + NCommon::TConfSetting<bool, true> SuspendIfAccountLimitExceeded; + NCommon::TConfSetting<NSize::TSize, true> ExtraTmpfsSize; + NCommon::TConfSetting<NYT::EOptimizeForAttr, true> OptimizeFor; // {scan, lookup} + NCommon::TConfSetting<TInstant, true> ExpirationDeadline; + NCommon::TConfSetting<TDuration, true> ExpirationInterval; + NCommon::TConfSetting<double, true> ScriptCpu; + NCommon::TConfSetting<double, true> PythonCpu; + NCommon::TConfSetting<double, true> JavascriptCpu; + NCommon::TConfSetting<double, true> ErasureCodecCpu; + NCommon::TConfSetting<double, true> ErasureCodecCpuForDq; + NCommon::TConfSetting<TSet<TString>, true> Owners; + NCommon::TConfSetting<TSet<TString>, true> OperationReaders; + NCommon::TConfSetting<TString, true> SchedulingTag; + NCommon::TConfSetting<TString, true> SchedulingTagFilter; + NCommon::TConfSetting<TSet<TString>, true> PoolTrees; + NCommon::TConfSetting<TSet<TString>, true> TentativePoolTrees; + NCommon::TConfSetting<ui32, true> TentativeTreeEligibilitySampleJobCount; + NCommon::TConfSetting<double, true> TentativeTreeEligibilityMaxJobDurationRatio; + NCommon::TConfSetting<ui32, true> TentativeTreeEligibilityMinJobDuration; + NCommon::TConfSetting<bool, true> UseDefaultTentativePoolTrees; + NCommon::TConfSetting<TString, true> IntermediateAccount; + NCommon::TConfSetting<ui32, true> IntermediateReplicationFactor; + NCommon::TConfSetting<ui32, true> PublishedReplicationFactor; + NCommon::TConfSetting<ui32, true> TemporaryReplicationFactor; + NCommon::TConfSetting<TString, true> AutoMerge; // {relaxed, economy, disabled} + NCommon::TConfSetting<TString, true> PublishedAutoMerge; + NCommon::TConfSetting<TString, true> TemporaryAutoMerge; + NCommon::TConfSetting<TVector<TString>, true> LayerPaths; + NCommon::TConfSetting<TString, true> DockerImage; + NCommon::TConfSetting<NYT::TNode, true> JobEnv; + NCommon::TConfSetting<NYT::TNode, true> OperationSpec; + NCommon::TConfSetting<NYT::TNode, true> Annotations; + NCommon::TConfSetting<NYT::TNode, true> StartedBy; + NCommon::TConfSetting<NYT::TNode, true> Description; + NCommon::TConfSetting<bool, true> UseSkiff; + NCommon::TConfSetting<ui32, true> TableContentCompressLevel; + NCommon::TConfSetting<bool, true> DisableJobSplitting; + NCommon::TConfSetting<EUseColumnarStatisticsMode, true> UseColumnarStatistics; + NCommon::TConfSetting<ETableContentDeliveryMode, true> TableContentDeliveryMode; + NCommon::TConfSetting<bool, true> TableContentUseSkiff; + NCommon::TConfSetting<TString, true> TableContentTmpFolder; + NCommon::TConfSetting<bool, true> TableContentColumnarStatistics; + NCommon::TConfSetting<TString, true> GeobaseDownloadUrl; + NCommon::TConfSetting<ui32, true> MaxSpeculativeJobCountPerTask; + NCommon::TConfSetting<NSize::TSize, true> LLVMMemSize; + NCommon::TConfSetting<NSize::TSize, true> LLVMPerNodeMemSize; + NCommon::TConfSetting<ui64, true> LLVMNodeCountLimit; + NCommon::TConfSetting<NSize::TSize, true> SamplingIoBlockSize; + NCommon::TConfSetting<NYT::TNode, true> PublishedMedia; + NCommon::TConfSetting<NYT::TNode, true> TemporaryMedia; + NCommon::TConfSetting<TString, true> PublishedPrimaryMedium; + NCommon::TConfSetting<TString, true> TemporaryPrimaryMedium; + NCommon::TConfSetting<TString, true> IntermediateDataMedium; + NCommon::TConfSetting<TString, true> PrimaryMedium; + NCommon::TConfSetting<ui64, true> QueryCacheChunkLimit; + NCommon::TConfSetting<ui64, true> NativeYtTypeCompatibility; + NCommon::TConfSetting<bool, true> _UseKeyBoundApi; + NCommon::TConfSetting<TString, true> NetworkProject; + NCommon::TConfSetting<bool, true> _EnableYtPartitioning; + NCommon::TConfSetting<bool, true> ForceJobSizeAdjuster; + NCommon::TConfSetting<bool, true> EnforceJobUtc; + NCommon::TConfSetting<bool, true> UseRPCReaderInDQ; + NCommon::TConfSetting<size_t, true> DQRPCReaderInflight; + NCommon::TConfSetting<TDuration, true> DQRPCReaderTimeout; + NCommon::TConfSetting<TSet<TString>, true> BlockReaderSupportedTypes; + NCommon::TConfSetting<TSet<NUdf::EDataSlot>, true> BlockReaderSupportedDataTypes; + NCommon::TConfSetting<TSet<TString>, true> JobBlockInputSupportedTypes; + NCommon::TConfSetting<TSet<NUdf::EDataSlot>, true> JobBlockInputSupportedDataTypes; + NCommon::TConfSetting<TString, true> _BinaryCacheFolder; + + // Optimizers + NCommon::TConfSetting<bool, true> _EnableDq; + NCommon::TConfSetting<ui32, false> ExtendTableLimit; // Deprecated. Use MaxInputTables instead + NCommon::TConfSetting<NSize::TSize, false> CommonJoinCoreLimit; + NCommon::TConfSetting<NSize::TSize, false> CombineCoreLimit; + NCommon::TConfSetting<NSize::TSize, false> SwitchLimit; + NCommon::TConfSetting<ui64, false> JoinMergeTablesLimit; + NCommon::TConfSetting<bool, false> JoinMergeUseSmallAsPrimary; + NCommon::TConfSetting<NSize::TSize, false> JoinMergeReduceJobMaxSize; + NCommon::TConfSetting<double, false> JoinMergeUnsortedFactor; // (>=0.0) + NCommon::TConfSetting<bool, false> JoinMergeForce; + NCommon::TConfSetting<NSize::TSize, false> MapJoinLimit; + NCommon::TConfSetting<ui64, false> MapJoinShardMinRows; + NCommon::TConfSetting<ui64, false> MapJoinShardCount; // [1-10] + NCommon::TConfSetting<bool, false> MapJoinUseFlow; + NCommon::TConfSetting<NSize::TSize, false> LookupJoinLimit; + NCommon::TConfSetting<ui64, false> LookupJoinMaxRows; + NCommon::TConfSetting<NSize::TSize, false> EvaluationTableSizeLimit; + NCommon::TConfSetting<TSet<TString>, false> DisableOptimizers; + NCommon::TConfSetting<ui32, false> MaxInputTables; + NCommon::TConfSetting<ui32, false> MaxInputTablesForSortedMerge; + NCommon::TConfSetting<ui32, false> MaxOutputTables; + NCommon::TConfSetting<bool, false> DisableFuseOperations; + NCommon::TConfSetting<NSize::TSize, false> MaxExtraJobMemoryToFuseOperations; + NCommon::TConfSetting<double, false> MaxReplicationFactorToFuseOperations; + NCommon::TConfSetting<ui32, false> MaxOperationFiles; + NCommon::TConfSetting<NSize::TSize, false> MinPublishedAvgChunkSize; + NCommon::TConfSetting<NSize::TSize, false> MinTempAvgChunkSize; + NCommon::TConfSetting<ui32, false> TopSortMaxLimit; + NCommon::TConfSetting<NSize::TSize, false> TopSortSizePerJob; + NCommon::TConfSetting<ui32, false> TopSortRowMultiplierPerJob; + NCommon::TConfSetting<bool, false> JoinUseColumnarStatistics; // Deprecated. Use JoinCollectColumnarStatistics instead + NCommon::TConfSetting<EJoinCollectColumnarStatisticsMode, false> JoinCollectColumnarStatistics; + NCommon::TConfSetting<NYT::EColumnarStatisticsFetcherMode, false> JoinColumnarStatisticsFetcherMode; + NCommon::TConfSetting<bool, false> JoinWaitAllInputs; + NCommon::TConfSetting<bool, false> JoinAllowColumnRenames; + NCommon::TConfSetting<bool, false> JoinMergeSetTopLevelFullSort; + NCommon::TConfSetting<bool, false> JoinEnableStarJoin; + NCommon::TConfSetting<NSize::TSize, false> FolderInlineDataLimit; + NCommon::TConfSetting<ui32, false> FolderInlineItemsLimit; + NCommon::TConfSetting<NSize::TSize, false> TableContentMinAvgChunkSize; + NCommon::TConfSetting<ui32, false> TableContentMaxInputTables; + NCommon::TConfSetting<ui32, false> TableContentMaxChunksForNativeDelivery; + NCommon::TConfSetting<NSize::TSize, false> TableContentLocalExecution; + NCommon::TConfSetting<bool, false> UseTypeV2; + NCommon::TConfSetting<bool, false> UseNativeYtTypes; + NCommon::TConfSetting<bool, false> UseNativeDescSort; + NCommon::TConfSetting<bool, false> UseIntermediateSchema; + NCommon::TConfSetting<bool, false> UseIntermediateStreams; + NCommon::TConfSetting<bool, false> UseFlow; + NCommon::TConfSetting<ui16, false> WideFlowLimit; + NCommon::TConfSetting<bool, false> UseSystemColumns; + NCommon::TConfSetting<bool, false> HybridDqExecution; + NCommon::TConfSetting<TDuration, false> HybridDqTimeSpentLimit; + NCommon::TConfSetting<NSize::TSize, false> HybridDqDataSizeLimitForOrdered; + NCommon::TConfSetting<NSize::TSize, false> HybridDqDataSizeLimitForUnordered; + NCommon::TConfSetting<bool, false> HybridDqExecutionFallback; + NCommon::TConfSetting<bool, false> UseYqlRowSpecCompactForm; + NCommon::TConfSetting<bool, false> UseNewPredicateExtraction; + NCommon::TConfSetting<bool, false> PruneKeyFilterLambda; + NCommon::TConfSetting<bool, false> DqPruneKeyFilterLambda; + NCommon::TConfSetting<bool, false> MergeAdjacentPointRanges; + NCommon::TConfSetting<bool, false> KeyFilterForStartsWith; + NCommon::TConfSetting<ui64, false> MaxKeyRangeCount; + NCommon::TConfSetting<ui64, false> MaxChunksForDqRead; + NCommon::TConfSetting<bool, false> JoinCommonUseMapMultiOut; + NCommon::TConfSetting<bool, false> UseAggPhases; + NCommon::TConfSetting<bool, false> UsePartitionsByKeysForFinalAgg; + NCommon::TConfSetting<double, false> MaxCpuUsageToFuseMultiOuts; + NCommon::TConfSetting<double, false> MaxReplicationFactorToFuseMultiOuts; + NCommon::TConfSetting<ui64, false> ApplyStoredConstraints; + NCommon::TConfSetting<bool, false> ViewIsolation; + NCommon::TConfSetting<bool, false> PartitionByConstantKeysViaMap; + NCommon::TConfSetting<EColumnGroupMode, false> ColumnGroupMode; + NCommon::TConfSetting<ui16, false> MinColumnGroupSize; + NCommon::TConfSetting<ui16, false> MaxColumnGroups; + NCommon::TConfSetting<ui64, false> ExtendedStatsMaxChunkCount; + NCommon::TConfSetting<bool, false> JobBlockInput; + NCommon::TConfSetting<bool, false> _EnableYtDqProcessWriteConstraints; +}; + +EReleaseTempDataMode GetReleaseTempDataMode(const TYtSettings& settings); +EJoinCollectColumnarStatisticsMode GetJoinCollectColumnarStatisticsMode(const TYtSettings& settings); +inline TString GetTablesTmpFolder(const TYtSettings& settings) { + return settings.TablesTmpFolder.Get().GetOrElse(settings.TmpFolder.Get().GetOrElse({})); +} + +struct TYtConfiguration : public TYtSettings, public NCommon::TSettingDispatcher { + using TPtr = TIntrusivePtr<TYtConfiguration>; + + TYtConfiguration(TTypeAnnotationContext& typeCtx); + TYtConfiguration(const TYtConfiguration&) = delete; + + template <class TProtoConfig, typename TFilter> + void Init(const TProtoConfig& config, const TFilter& filter, TTypeAnnotationContext& typeCtx) { + TVector<TString> clusters(Reserve(config.ClusterMappingSize())); + for (auto& cluster: config.GetClusterMapping()) { + clusters.push_back(cluster.GetName()); + Tokens[cluster.GetName()] = typeCtx.Credentials->FindCredentialContent("cluster:default_" + cluster.GetName(), "default_yt", cluster.GetYTToken()); + } + + auto impersonationUser = typeCtx.Credentials->FindCredential("impersonation_user_yt"); + if (impersonationUser) { + _ImpersonationUser = impersonationUser->Content; + } + + this->SetValidClusters(clusters); + + // Init settings from config + this->Dispatch(config.GetDefaultSettings(), filter); + for (auto& cluster: config.GetClusterMapping()) { + this->Dispatch(cluster.GetName(), cluster.GetSettings(), filter); + } + this->FreezeDefaults(); + } + + TYtSettings::TConstPtr Snapshot() const; + + THashMap<TString, TString> Tokens; +}; + +class TYtVersionedConfiguration: public TYtConfiguration { +public: + using TPtr = TIntrusivePtr<TYtVersionedConfiguration>; + + struct TState { + TVector<TYtSettings::TConstPtr> FrozenSettings; + std::unordered_map<ui64, size_t> NodeIdToVer; + TYtSettings::TConstPtr Snapshot; + }; + + TYtVersionedConfiguration(TTypeAnnotationContext& types) + : TYtConfiguration(types) + { + } + + ~TYtVersionedConfiguration() = default; + + size_t FindNodeVer(const TExprNode& node); + void FreezeZeroVersion(); + void PromoteVersion(const TExprNode& node); + size_t GetLastVersion() const { + return FrozenSettings.empty() ? 0 : FrozenSettings.size() - 1; + } + TYtSettings::TConstPtr GetSettingsForNode(const TExprNode& node); + TYtSettings::TConstPtr GetSettingsVer(size_t ver); + void ClearVersions(); + + TState GetState() const { + return TState{FrozenSettings, NodeIdToVer, Snapshot()}; + } + + void RestoreState(TState&& state) { + FrozenSettings = std::move(state.FrozenSettings); + NodeIdToVer = std::move(state.NodeIdToVer); + *((TYtSettings*)this) = *state.Snapshot; + } + +private: + TVector<TYtSettings::TConstPtr> FrozenSettings; + std::unordered_map<ui64, size_t> NodeIdToVer; +}; + +bool ValidateCompressionCodecValue(const TStringBuf& codec); +void MediaValidator(const NYT::TNode& value); + +} // NYql diff --git a/yt/yql/providers/yt/comp_nodes/dq/dq_yt_block_reader.cpp b/yt/yql/providers/yt/comp_nodes/dq/dq_yt_block_reader.cpp new file mode 100644 index 0000000000..abdcb46ead --- /dev/null +++ b/yt/yql/providers/yt/comp_nodes/dq/dq_yt_block_reader.cpp @@ -0,0 +1,671 @@ +#include "dq_yt_block_reader.h" +#include "stream_decoder.h" +#include "dq_yt_rpc_helpers.h" + +#include <yql/essentials/public/udf/arrow/block_builder.h> + +#include <yt/yql/providers/yt/codec/yt_arrow_converter.h> +#include <yt/yql/providers/yt/codec/yt_codec.h> +#include <yql/essentials/providers/common/codec/yql_codec.h> +#include <yql/essentials/minikql/mkql_node_cast.h> +#include <yql/essentials/minikql/computation/mkql_computation_node_codegen.h> +#include <yql/essentials/minikql/computation/mkql_computation_node_impl.h> +#include <yql/essentials/minikql/computation/mkql_computation_node.h> +#include <yql/essentials/minikql/computation/mkql_block_impl.h> +#include <yql/essentials/minikql/mkql_stats_registry.h> +#include <yql/essentials/minikql/mkql_node.h> +#include <yql/essentials/minikql/mkql_type_builder.h> + +#include <yt/yt/core/concurrency/thread_pool.h> +#include <yt/yt/core/threading/thread.h> +#include <yt/cpp/mapreduce/interface/common.h> +#include <yt/cpp/mapreduce/interface/errors.h> +#include <yt/cpp/mapreduce/interface/client.h> +#include <yt/cpp/mapreduce/interface/serialize.h> +#include <yt/cpp/mapreduce/interface/config.h> +#include <yt/cpp/mapreduce/common/wait_proxy.h> + +#include <arrow/compute/cast.h> +#include <arrow/compute/api_vector.h> +#include <arrow/compute/api.h> +#include <arrow/io/interfaces.h> +#include <arrow/io/memory.h> +#include <arrow/ipc/reader.h> +#include <arrow/array.h> +#include <arrow/chunked_array.h> +#include <arrow/record_batch.h> +#include <arrow/type.h> +#include <arrow/result.h> +#include <arrow/buffer.h> + +#include <library/cpp/yson/node/node.h> +#include <library/cpp/yson/node/node_io.h> + +#include <util/generic/size_literals.h> +#include <util/stream/output.h> + +namespace NYql::NDqs { + +using namespace NKikimr::NMiniKQL; + +TStatKey FallbackCount("YtBlockReader_Fallbacks", true); +TStatKey BlockCount("YtBlockReader_Blocks", true); + +namespace { +struct TResultBatch { + using TPtr = std::shared_ptr<TResultBatch>; + size_t RowsCnt; + std::vector<arrow::Datum> Columns; + TResultBatch(int64_t cnt) : RowsCnt(cnt) {} + TResultBatch(int64_t cnt, decltype(Columns)&& columns) : RowsCnt(cnt), Columns(std::move(columns)) {} +}; + +template<typename T> +class TBlockingQueueWithLimit { + struct Poison { + TString Error; + }; + struct TFallbackNotify {}; + using TPoisonOr = std::variant<T, Poison, TFallbackNotify>; +public: + TBlockingQueueWithLimit(size_t limit) : Limit_(limit) {} + void Push(T&& val, bool imm) { + PushInternal(std::move(val), imm); + } + + void PushPoison(const TString& err) { + PushInternal(Poison{err}, true); + } + + void PushNotify() { + PushInternal(TFallbackNotify{}, true); + } + + TMaybe<T> Get() { + auto res = GetInternal(); + if (std::holds_alternative<Poison>(res)) { + throw std::runtime_error(std::get<Poison>(res).Error); + } + if (std::holds_alternative<TFallbackNotify>(res)) { + return {}; + } + return std::move(std::get<T>(res)); + } +private: + template<typename X> + void PushInternal(X&& val, bool imm) { + for (;;) { + NYT::TPromise<void> promise; + { + std::lock_guard guard(Mtx_); + if (!Awaiting_.empty()) { + Awaiting_.front().Set(std::move(val)); + Awaiting_.pop(); + return; + } + if (!imm && Ready_.size() >= Limit_) { + promise = NYT::NewPromise<void>(); + BlockedPushes_.push(promise); + } else { + Ready_.emplace(std::move(val)); + return; + } + } + YQL_ENSURE(NYT::NConcurrency::WaitFor(promise.ToFuture()).IsOK()); + } + } + + TPoisonOr GetInternal() { + NYT::TPromise<TPoisonOr> awaiter; + { + std::lock_guard guard(Mtx_); + if (!BlockedPushes_.empty()) { + BlockedPushes_.front().Set(); + BlockedPushes_.pop(); + } + if (!Ready_.empty()) { + auto res = std::move(Ready_.front()); + Ready_.pop(); + return res; + } + awaiter = NYT::NewPromise<TPoisonOr>(); + Awaiting_.push(awaiter); + } + auto awaitResult = NYT::NConcurrency::WaitFor(awaiter.ToFuture()); + if (!awaitResult.IsOK()) { + throw std::runtime_error(awaitResult.GetMessage()); + } + return std::move(awaitResult.Value()); + } + std::mutex Mtx_; + std::queue<TPoisonOr> Ready_; + std::queue<NYT::TPromise<void>> BlockedPushes_; + std::queue<NYT::TPromise<TPoisonOr>> Awaiting_; + size_t Limit_; +}; + +class TListener { + using TBatchPtr = std::shared_ptr<arrow::RecordBatch>; +public: + using TPromise = NYT::TPromise<TResultBatch>; + using TPtr = std::shared_ptr<TListener>; + TListener(size_t initLatch, size_t inflight) + : Latch_(initLatch) + , Queue_(inflight) {} + + void OnEOF() { + bool excepted = 0; + if (GotEOF_.compare_exchange_strong(excepted, 1)) { + // block poining to nullptr is marker of EOF + HandleResult(nullptr); + } else { + // can't get EOF more than one time + HandleError("EOS already got"); + } + } + + // Handles result + void HandleResult(TResultBatch::TPtr&& res, bool immediatly = false) { + Queue_.Push(std::move(res), immediatly); + } + + TMaybe<TResultBatch::TPtr> Get() { + return Queue_.Get(); + } + + void HandleError(const TString& msg) { + Queue_.PushPoison(msg); + } + + void HandleFallback(TResultBatch::TPtr&& block) { + HandleResult(std::move(block), true); + } + + void NotifyFallback() { + Queue_.PushNotify(); + } + + void InputDone() { + // EOF comes when all inputs got EOS + if (!--Latch_) { + OnEOF(); + } + } +private: + std::atomic<size_t> Latch_; + std::atomic<bool> GotEOF_; + TBlockingQueueWithLimit<TResultBatch::TPtr> Queue_; +}; + +class TBlockBuilder { +public: + void Init(std::shared_ptr<std::vector<TType*>> columnTypes, arrow::MemoryPool& pool, const NUdf::IPgBuilder* pgBuilder) { + ColumnTypes_ = columnTypes; + ColumnBuilders_.reserve(ColumnTypes_->size()); + size_t maxBlockItemSize = 0; + for (auto& type: *ColumnTypes_) { + maxBlockItemSize = std::max(maxBlockItemSize, CalcMaxBlockItemSize(type)); + } + size_t maxBlockLen = CalcBlockLen(maxBlockItemSize); + for (size_t i = 0; i < ColumnTypes_->size(); ++i) { + ColumnBuilders_.push_back( + std::move(NUdf::MakeArrayBuilder( + TTypeInfoHelper(), ColumnTypes_->at(i), + pool, + maxBlockLen, + pgBuilder + )) + ); + } + } + + void Add(const NUdf::TUnboxedValue& val) { + for (ui32 i = 0; i < ColumnBuilders_.size(); ++i) { + auto v = val.GetElement(i); + ColumnBuilders_[i]->Add(v); + } + ++RowsCnt_; + } + + std::vector<TResultBatch::TPtr> Build() { + std::vector<arrow::Datum> columns; + columns.reserve(ColumnBuilders_.size()); + for (size_t i = 0; i < ColumnBuilders_.size(); ++i) { + columns.emplace_back(std::move(ColumnBuilders_[i]->Build(false))); + } + std::vector<std::shared_ptr<TResultBatch>> blocks; + int64_t offset = 0; + std::vector<int64_t> currentChunk(columns.size()), inChunkOffset(columns.size()); + while (RowsCnt_) { + int64_t max_curr_len = RowsCnt_; + for (size_t i = 0; i < columns.size(); ++i) { + if (arrow::Datum::Kind::CHUNKED_ARRAY == columns[i].kind()) { + auto& c_arr = columns[i].chunked_array(); + while (currentChunk[i] < c_arr->num_chunks() && !c_arr->chunk(currentChunk[i])) { + ++currentChunk[i]; + } + YQL_ENSURE(currentChunk[i] < c_arr->num_chunks()); + max_curr_len = std::min(max_curr_len, c_arr->chunk(currentChunk[i])->length() - inChunkOffset[i]); + } + } + RowsCnt_ -= max_curr_len; + decltype(columns) result_columns; + result_columns.reserve(columns.size()); + offset += max_curr_len; + for (size_t i = 0; i < columns.size(); ++i) { + auto& e = columns[i]; + if (arrow::Datum::Kind::CHUNKED_ARRAY == e.kind()) { + result_columns.emplace_back(e.chunked_array()->chunk(currentChunk[i])->Slice(inChunkOffset[i], max_curr_len)); + if (max_curr_len + inChunkOffset[i] == e.chunked_array()->chunk(currentChunk[i])->length()) { + ++currentChunk[i]; + inChunkOffset[i] = 0; + } else { + inChunkOffset[i] += max_curr_len; + } + } else { + result_columns.emplace_back(e.array()->Slice(offset - max_curr_len, max_curr_len)); + } + } + blocks.emplace_back(std::make_shared<TResultBatch>(max_curr_len, std::move(result_columns))); + } + return blocks; + } + +private: + int64_t RowsCnt_ = 0; + std::vector<std::unique_ptr<NUdf::IArrayBuilder>> ColumnBuilders_; + std::shared_ptr<std::vector<TType*>> ColumnTypes_; +}; + +class TLocalListener : public arrow::ipc::Listener { +public: + TLocalListener(std::shared_ptr<TListener> consumer + , std::unordered_map<std::string, ui32>& columnOrderMapping + , std::shared_ptr<std::vector<TType*>> columnTypes + , std::shared_ptr<std::vector<std::shared_ptr<arrow::DataType>>> arrowTypes + , arrow::MemoryPool& pool, const NUdf::IPgBuilder* pgBuilder + , bool isNative, NKikimr::NMiniKQL::IStatsRegistry* jobStats) + : Consumer_(consumer) + , ColumnTypes_(columnTypes) + , JobStats_(jobStats) + , ColumnOrderMapping(columnOrderMapping) + { + ColumnConverters_.reserve(columnTypes->size()); + for (size_t i = 0; i < columnTypes->size(); ++i) { + ColumnConverters_.emplace_back(MakeYtColumnConverter(columnTypes->at(i), pgBuilder, pool, isNative)); + } + } + + void Init(std::shared_ptr<TLocalListener> self) { + Self_ = self; + Decoder_ = std::make_shared<arrow::ipc::NDqs::StreamDecoder2>(self, arrow::ipc::IpcReadOptions{.use_threads=false}); + } + + arrow::Status OnEOS() override { + Decoder_->Reset(); + return arrow::Status::OK(); + } + + arrow::Status OnRecordBatchDecoded(std::shared_ptr<arrow::RecordBatch> batch) override { + YQL_ENSURE(batch); + MKQL_ADD_STAT(JobStats_, BlockCount, 1); + std::vector<arrow::Datum> result; + YQL_ENSURE((size_t)batch->num_columns() == ColumnConverters_.size()); + result.resize(ColumnConverters_.size()); + size_t matchedColumns = 0; + for (size_t i = 0; i < ColumnConverters_.size(); ++i) { + auto columnIdxIt = ColumnOrderMapping.find(batch->schema()->field_names()[i]); + if (ColumnOrderMapping.end() == columnIdxIt) { + continue; + } + ++matchedColumns; + auto columnIdx = columnIdxIt->second; + result[columnIdx] = std::move(ColumnConverters_[columnIdx]->Convert(batch->column(i)->data())); + } + Y_ENSURE(matchedColumns == ColumnOrderMapping.size()); + Consumer_->HandleResult(std::make_shared<TResultBatch>(batch->num_rows(), std::move(result))); + return arrow::Status::OK(); + } + + void Consume(std::shared_ptr<arrow::Buffer> buff) { + ARROW_OK(Decoder_->Consume(buff)); + } + + void Finish() { + Self_ = nullptr; + } +private: + std::shared_ptr<TLocalListener> Self_; + std::shared_ptr<TListener> Consumer_; + std::shared_ptr<arrow::ipc::NDqs::StreamDecoder2> Decoder_; + std::shared_ptr<std::vector<TType*>> ColumnTypes_; + NKikimr::NMiniKQL::IStatsRegistry* JobStats_; + std::vector<std::unique_ptr<IYtColumnConverter>> ColumnConverters_; + std::unordered_map<std::string, ui32>& ColumnOrderMapping; +}; + +class TSource : public TNonCopyable { +public: + using TPtr = std::shared_ptr<TSource>; + TSource(std::unique_ptr<TSettingsHolder>&& settings, + size_t inflight, TType* type, std::shared_ptr<std::vector<std::shared_ptr<arrow::DataType>>> types, const THolderFactory& holderFactory, NKikimr::NMiniKQL::IStatsRegistry* jobStats) + : HolderFactory(holderFactory) + , Settings_(std::move(settings)) + , Inputs_(Settings_->Requests.size()) + , Listener_(std::make_shared<TListener>(Inputs_.size(), inflight)) + , JobStats_(jobStats) + { + auto structType = AS_TYPE(TStructType, type); + std::vector<TType*> columnTypes_(structType->GetMembersCount()); + for (ui32 i = 0; i < structType->GetMembersCount(); ++i) { + columnTypes_[i] = structType->GetMemberType(i); + } + auto ptr = std::make_shared<decltype(columnTypes_)>(std::move(columnTypes_)); + Inflight_ = std::min(inflight, Inputs_.size()); + + LocalListeners_.reserve(Inputs_.size()); + for (size_t i = 0; i < Inputs_.size(); ++i) { + auto& decoder = Settings_->Specs->Inputs[Settings_->OriginalIndexes[i]]; + bool native = decoder->NativeYtTypeFlags && !decoder->FieldsVec[i].ExplicitYson; + LocalListeners_.emplace_back(std::make_shared<TLocalListener>(Listener_, Settings_->ColumnNameMapping, ptr, types, *Settings_->Pool, Settings_->PgBuilder, native, jobStats)); + LocalListeners_.back()->Init(LocalListeners_.back()); + } + BlockBuilder_.Init(ptr, *Settings_->Pool, Settings_->PgBuilder); + FallbackReader_.SetSpecs(*Settings_->Specs, HolderFactory); + } + + void RunRead() { + size_t inputIdx; + { + std::lock_guard guard(Mtx_); + if (InputsQueue_.empty()) { + if (NextFreeInputIdx >= Inputs_.size()) { + return; + } + inputIdx = NextFreeInputIdx++; + } else { + inputIdx = InputsQueue_.front(); + InputsQueue_.pop(); + } + } + if (!Inputs_[inputIdx]) { + CreateInputStream(Settings_->Requests[inputIdx]).SubscribeUnique(BIND([self = Self_, inputIdx] (NYT::TErrorOr<NYT::NConcurrency::IAsyncZeroCopyInputStreamPtr>&& stream) { + self->Pool_->GetInvoker()->Invoke(BIND([inputIdx, self, stream = std::move(stream)]() mutable { + try { + self->Inputs_[inputIdx] = std::move(stream.ValueOrThrow()); + self->InputDone(inputIdx); + self->RunRead(); + } catch (...) { + self->Listener_->HandleError(CurrentExceptionMessage()); + } + })); + })); + return; + } + Inputs_[inputIdx]->Read().SubscribeUnique(BIND([inputIdx = inputIdx, self = Self_](NYT::TErrorOr<NYT::TSharedRef>&& res) { + self->Pool_->GetInvoker()->Invoke(BIND([inputIdx, self, res = std::move(res)]() mutable { + try { + self->Accept(inputIdx, std::move(res)); + self->RunRead(); + } catch (...) { + self->Listener_->HandleError(CurrentExceptionMessage()); + } + })); + })); + } + + void Accept(size_t inputIdx, NYT::TErrorOr<NYT::TSharedRef>&& res) { + if (res.IsOK() && !res.Value()) { + // End Of Stream + Listener_->InputDone(); + return; + } + + if (!res.IsOK()) { + // Propagate error + Listener_->HandleError(res.GetMessage()); + return; + } + + NYT::NApi::NRpcProxy::NProto::TRowsetDescriptor descriptor; + NYT::NApi::NRpcProxy::NProto::TRowsetStatistics statistics; + NYT::TSharedRef currentPayload = NYT::NApi::NRpcProxy::DeserializeRowStreamBlockEnvelope(res.Value(), &descriptor, &statistics); + if (descriptor.rowset_format() != NYT::NApi::NRpcProxy::NProto::RF_ARROW) { + if (currentPayload.Size()) { + std::lock_guard guard(FallbackMtx_); + Fallbacks_.push({inputIdx, currentPayload}); + } else { + InputDone(inputIdx); + } + Listener_->NotifyFallback(); + return; + } + + if (!currentPayload.Size()) { + // EOS + Listener_->InputDone(); + return; + } + // TODO(): support row and range indexes + auto payload = TMemoryInput(currentPayload.Begin(), currentPayload.Size()); + arrow::BufferBuilder bb; + ARROW_OK(bb.Reserve(currentPayload.Size())); + ARROW_OK(bb.Append((const uint8_t*)payload.Buf(), currentPayload.Size())); + LocalListeners_[inputIdx]->Consume(*bb.Finish()); + InputDone(inputIdx); + } + + // Return input back to queue + void InputDone(auto input) { + std::lock_guard guard(Mtx_); + InputsQueue_.emplace(input); + } + + TResultBatch::TPtr Next() { + for(;;) { + size_t inputIdx = 0; + NYT::TSharedRef payload; + { + std::lock_guard guard(FallbackMtx_); + if (Fallbacks_.size()) { + inputIdx = Fallbacks_.front().first; + payload = Fallbacks_.front().second; + Fallbacks_.pop(); + } + } + if (payload) { + for (auto &e: FallbackHandler(inputIdx, payload)) { + Listener_->HandleFallback(std::move(e)); + } + InputDone(inputIdx); + RunRead(); + } + auto result = Listener_->Get(); + if (!result) { // Falled back + continue; + } + return *result; + } + } + + std::vector<TResultBatch::TPtr> FallbackHandler(size_t idx, NYT::TSharedRef payload) { + if (!payload.Size()) { + return {}; + } + auto currentReader_ = std::make_shared<TPayloadRPCReader>(std::move(payload)); + MKQL_ADD_STAT(JobStats_, FallbackCount, 1); + // TODO(): save and recover row indexes + FallbackReader_.SetReader(*currentReader_, 1, 4_MB, ui32(Settings_->OriginalIndexes[idx]), true); + // If we don't save the reader, after exiting FallbackHandler it will be destroyed, + // but FallbackReader points on it yet. + Reader_ = currentReader_; + FallbackReader_.Next(); + while (FallbackReader_.IsValid()) { + auto currentRow = std::move(FallbackReader_.GetRow()); + if (!Settings_->Specs->InputGroups.empty()) { + currentRow = std::move(HolderFactory.CreateVariantHolder(currentRow.Release(), Settings_->Specs->InputGroups.at(Settings_->OriginalIndexes[idx]))); + } + BlockBuilder_.Add(currentRow); + FallbackReader_.Next(); + } + return BlockBuilder_.Build(); + } + + void Finish() { + FallbackReader_.Finish(); + Pool_->Shutdown(); + for (auto& e: LocalListeners_) { + e->Finish(); + } + Self_ = nullptr; + } + + void SetSelfAndRun(TPtr self) { + Self_ = self; + Pool_ = NYT::NConcurrency::CreateThreadPool(Inflight_, "block_reader"); + // Run Inflight_ reads at the same time + for (size_t i = 0; i < Inflight_; ++i) { + RunRead(); + } + } + + const THolderFactory& HolderFactory; +private: + NYT::NConcurrency::IThreadPoolPtr Pool_; + std::mutex Mtx_; + std::mutex FallbackMtx_; + std::queue<std::pair<size_t, NYT::TSharedRef>> Fallbacks_; + std::queue<TResultBatch::TPtr> FallbackBlocks_; + std::unique_ptr<TSettingsHolder> Settings_; + std::vector<std::shared_ptr<TLocalListener>> LocalListeners_; + std::vector<NYT::NConcurrency::IAsyncZeroCopyInputStreamPtr> Inputs_; + TMkqlReaderImpl FallbackReader_; + TBlockBuilder BlockBuilder_; + std::shared_ptr<TPayloadRPCReader> Reader_; + std::queue<size_t> InputsQueue_; + TListener::TPtr Listener_; + TPtr Self_; + size_t Inflight_; + NKikimr::NMiniKQL::IStatsRegistry* JobStats_; + size_t NextFreeInputIdx = 0; +}; + +class TReaderState: public TComputationValue<TReaderState> { + using TBase = TComputationValue<TReaderState>; +public: + TReaderState(TMemoryUsageInfo* memInfo, TSource::TPtr source, size_t width, std::shared_ptr<std::vector<std::shared_ptr<arrow::DataType>>> arrowTypes) + : TBase(memInfo) + , Source_(std::move(source)) + , Width_(width) + , Types_(arrowTypes) + , Result_(width) + { + } + + NUdf::EFetchStatus WideFetch(NUdf::TUnboxedValue* output, ui32 width) { + if (GotFinish_) { + return NUdf::EFetchStatus::Finish; + } + YQL_ENSURE(width == Width_ + 1); + try { + auto batch = Source_->Next(); + if (!batch) { + GotFinish_ = 1; + Source_->Finish(); + return NUdf::EFetchStatus::Finish; + } + + for (size_t i = 0; i < Width_; ++i) { + YQL_ENSURE(batch->Columns[i].type()->Equals(Types_->at(i))); + output[i] = Source_->HolderFactory.CreateArrowBlock(std::move(batch->Columns[i])); + } + output[Width_] = Source_->HolderFactory.CreateArrowBlock(arrow::Datum(ui64(batch->RowsCnt))); + } catch (...) { + Cerr << "YT RPC Reader exception:\n"; + throw; + } + return NUdf::EFetchStatus::Ok; + } + +private: + TSource::TPtr Source_; + const size_t Width_; + std::shared_ptr<std::vector<std::shared_ptr<arrow::DataType>>> Types_; + std::vector<NUdf::TUnboxedValue*> Result_; + bool GotFinish_ = 0; +}; +}; + +class TDqYtReadBlockWrapper : public TMutableComputationNode<TDqYtReadBlockWrapper> { +using TBaseComputation = TMutableComputationNode<TDqYtReadBlockWrapper>; +public: + + TDqYtReadBlockWrapper(const TComputationNodeFactoryContext& ctx, const TString& clusterName, + const TString& token, const NYT::TNode& inputSpec, const NYT::TNode& samplingSpec, + const TVector<ui32>& inputGroups, + TType* itemType, const TVector<TString>& tableNames, TVector<std::pair<NYT::TRichYPath, NYT::TFormat>>&& tables, + NKikimr::NMiniKQL::IStatsRegistry* jobStats, size_t inflight, size_t timeout, const TVector<ui64>& tableOffsets) + : TBaseComputation(ctx.Mutables, EValueRepresentation::Boxed) + , Width_(AS_TYPE(TStructType, itemType)->GetMembersCount()) + , CodecCtx_(ctx.Env, ctx.FunctionRegistry, &ctx.HolderFactory) + , ClusterName_(clusterName) + , Token_(token) + , SamplingSpec_(samplingSpec) + , Tables_(std::move(tables)) + , Inflight_(inflight) + , Timeout_(timeout) + , Type_(itemType) + , JobStats_(jobStats) + { + // TODO() Enable range indexes + row indexes + Specs_.SetUseSkiff("", 0); + Specs_.Init(CodecCtx_, inputSpec, inputGroups, tableNames, itemType, {}, {}, jobStats); + Specs_.SetTableOffsets(tableOffsets); + } + + void MakeState(TComputationContext& ctx, NUdf::TUnboxedValue& state) const { + } + + NUdf::TUnboxedValuePod DoCalculate(TComputationContext& ctx) const { + auto settings = CreateInputStreams(true, Token_, ClusterName_, Timeout_, Inflight_ > 1, Tables_, SamplingSpec_); + settings->Specs = &Specs_; + settings->Pool = arrow::default_memory_pool(); + settings->PgBuilder = &ctx.Builder->GetPgBuilder(); + auto types = std::make_shared<std::vector<std::shared_ptr<arrow::DataType>>>(Width_); + TVector<TString> columnNames; + for (size_t i = 0; i < Width_; ++i) { + columnNames.emplace_back(AS_TYPE(TStructType, Type_)->GetMemberName(i)); + YQL_ENSURE(ConvertArrowType(AS_TYPE(TStructType, Type_)->GetMemberType(i), types->at(i)), "Can't convert type to arrow"); + } + settings->SetColumns(columnNames); + auto source = std::make_shared<TSource>(std::move(settings), Inflight_, Type_, types, ctx.HolderFactory, JobStats_); + source->SetSelfAndRun(source); + return ctx.HolderFactory.Create<TReaderState>(source, Width_, types); + } + + void RegisterDependencies() const final {} +private: + const ui32 Width_; + NCommon::TCodecContext CodecCtx_; + TMkqlIOSpecs Specs_; + + TString ClusterName_; + TString Token_; + NYT::TNode SamplingSpec_; + TVector<std::pair<NYT::TRichYPath, NYT::TFormat>> Tables_; + size_t Inflight_; + size_t Timeout_; + TType* Type_; + NKikimr::NMiniKQL::IStatsRegistry* JobStats_; +}; + +IComputationNode* CreateDqYtReadBlockWrapper(const TComputationNodeFactoryContext& ctx, const TString& clusterName, + const TString& token, const NYT::TNode& inputSpec, const NYT::TNode& samplingSpec, + const TVector<ui32>& inputGroups, + TType* itemType, const TVector<TString>& tableNames, TVector<std::pair<NYT::TRichYPath, NYT::TFormat>>&& tables, + NKikimr::NMiniKQL::IStatsRegistry* jobStats, size_t inflight, size_t timeout, const TVector<ui64>& tableOffsets) +{ + return new TDqYtReadBlockWrapper(ctx, clusterName, token, inputSpec, samplingSpec, inputGroups, itemType, + tableNames, std::move(tables), jobStats, inflight, timeout, tableOffsets); +} +} diff --git a/yt/yql/providers/yt/comp_nodes/dq/dq_yt_block_reader.h b/yt/yql/providers/yt/comp_nodes/dq/dq_yt_block_reader.h new file mode 100644 index 0000000000..85a9b178ca --- /dev/null +++ b/yt/yql/providers/yt/comp_nodes/dq/dq_yt_block_reader.h @@ -0,0 +1,18 @@ +#include <yql/essentials/minikql/computation/mkql_computation_node.h> +#include <yql/essentials/minikql/mkql_stats_registry.h> +#include <yql/essentials/minikql/mkql_node.h> + +#include <yt/yql/providers/yt/comp_nodes/yql_mkql_file_input_state.h> +#include <yt/yql/providers/yt/codec/yt_codec.h> +#include <yql/essentials/providers/common/codec/yql_codec.h> +#include <yql/essentials/minikql/mkql_node_cast.h> +#include <yql/essentials/minikql/computation/mkql_computation_node_codegen.h> + +namespace NYql::NDqs { +NKikimr::NMiniKQL::IComputationNode* CreateDqYtReadBlockWrapper( + const NKikimr::NMiniKQL::TComputationNodeFactoryContext& ctx, const TString& clusterName, + const TString& token, const NYT::TNode& inputSpec, const NYT::TNode& samplingSpec, + const TVector<ui32>& inputGroups, NKikimr::NMiniKQL::TType* itemType, const TVector<TString>& tableNames, + TVector<std::pair<NYT::TRichYPath, NYT::TFormat>>&& tables, NKikimr::NMiniKQL::IStatsRegistry* jobStats, + size_t inflight, size_t timeout, const TVector<ui64>& tableOffsets); +} diff --git a/yt/yql/providers/yt/comp_nodes/dq/dq_yt_factory.cpp b/yt/yql/providers/yt/comp_nodes/dq/dq_yt_factory.cpp new file mode 100644 index 0000000000..4d88adfc6e --- /dev/null +++ b/yt/yql/providers/yt/comp_nodes/dq/dq_yt_factory.cpp @@ -0,0 +1,24 @@ +#include "dq_yt_factory.h" +#include "dq_yt_reader.h" +#include "dq_yt_writer.h" + +namespace NYql { + +using namespace NKikimr::NMiniKQL; + +TComputationNodeFactory GetDqYtFactory(NKikimr::NMiniKQL::IStatsRegistry* jobStats) { + return [=] (TCallable& callable, const TComputationNodeFactoryContext& ctx) -> IComputationNode* { + TStringBuf name = callable.GetType()->GetName(); + if (name == "DqYtRead" || name == "DqYtBlockRead") { + return NDqs::WrapDqYtRead(callable, jobStats, ctx, name == "DqYtBlockRead"); + } + + if (name == "YtDqRowsWideWrite") { + return NDqs::WrapYtDqRowsWideWrite(callable, ctx); + } + + return nullptr; + }; +} + +} // NYql diff --git a/yt/yql/providers/yt/comp_nodes/dq/dq_yt_factory.h b/yt/yql/providers/yt/comp_nodes/dq/dq_yt_factory.h new file mode 100644 index 0000000000..77aea930f4 --- /dev/null +++ b/yt/yql/providers/yt/comp_nodes/dq/dq_yt_factory.h @@ -0,0 +1,10 @@ +#pragma once + +#include <yql/essentials/minikql/computation/mkql_computation_node.h> +#include <yql/essentials/minikql/mkql_stats_registry.h> + +namespace NYql { + +NKikimr::NMiniKQL::TComputationNodeFactory GetDqYtFactory(NKikimr::NMiniKQL::IStatsRegistry* jobStats = nullptr); + +} diff --git a/yt/yql/providers/yt/comp_nodes/dq/dq_yt_reader.cpp b/yt/yql/providers/yt/comp_nodes/dq/dq_yt_reader.cpp new file mode 100644 index 0000000000..5ef6be6942 --- /dev/null +++ b/yt/yql/providers/yt/comp_nodes/dq/dq_yt_reader.cpp @@ -0,0 +1,138 @@ +#include "dq_yt_reader.h" + +#include "dq_yt_reader_impl.h" +#include "dq_yt_block_reader.h" +#include "dq_yt_rpc_reader.h" + +namespace NYql::NDqs { + +using namespace NKikimr::NMiniKQL; + +class TDqYtReadWrapperHttp : public TDqYtReadWrapperBase<TDqYtReadWrapperHttp, TFileInputState> { +public: +using TInputType = NYT::TRawTableReaderPtr; + TDqYtReadWrapperHttp(const TComputationNodeFactoryContext& ctx, const TString& clusterName, + const TString& token, const NYT::TNode& inputSpec, const NYT::TNode& samplingSpec, + const TVector<ui32>& inputGroups, TType* itemType, const TVector<TString>& tableNames, + TVector<std::pair<NYT::TRichYPath, NYT::TFormat>>&& tables, + NKikimr::NMiniKQL::IStatsRegistry* jobStats, size_t inflight, size_t timeout, + const TVector<ui64>& tableOffsets) + : TDqYtReadWrapperBase<TDqYtReadWrapperHttp, TFileInputState>(ctx, clusterName, token, + inputSpec, samplingSpec, inputGroups, itemType, tableNames, std::move(tables), + jobStats, inflight, timeout, tableOffsets) {} + + void MakeState(TComputationContext& ctx, NUdf::TUnboxedValue& state) const { + TVector<NYT::TRawTableReaderPtr> rawReaders; + + NYT::TCreateClientOptions createOpts; + if (Token) { + createOpts.Token(Token); + } + + auto client = NYT::CreateClient(ClusterName, createOpts); + NYT::TTableReaderOptions readerOptions; + if (!SamplingSpec.IsUndefined()) { + readerOptions.Config(SamplingSpec); + } + + for (auto [richYPath, format]: Tables) { + NYT::TRawTableReaderPtr reader; + const int lastAttempt = NYT::TConfig::Get()->ReadRetryCount - 1; + for (int attempt = 0; attempt <= lastAttempt; ++attempt) { + try { + if (richYPath.TransactionId_) { + auto transaction = client->AttachTransaction(richYPath.TransactionId_.GetRef(), NYT::TAttachTransactionOptions().AutoPingable(true)); + richYPath.TransactionId_.Clear(); + reader = transaction->CreateRawReader(richYPath, format, readerOptions.CreateTransaction(false)); + } else { + reader = client->CreateRawReader(richYPath, format, readerOptions.CreateTransaction(true)); + } + break; + } catch (const NYT::TErrorResponse& e) { + Cerr << "Error creating reader for " << richYPath.Path_ << ": " << e.what(); + // Already retried inside CreateRawReader + throw; + } catch (const yexception& e) { + Cerr << "Error creating reader for " << richYPath.Path_ << ": " << e.what(); + if (attempt == lastAttempt) { + throw; + } + NYT::NDetail::TWaitProxy::Get()->Sleep(NYT::TConfig::Get()->RetryInterval); + } + } + rawReaders.push_back(reader); + } + + state = ctx.HolderFactory.Create<TDqYtReadWrapperBase<TDqYtReadWrapperHttp, TFileInputState>::TState>(Specs, ctx.HolderFactory, std::move(rawReaders), 4, 4_MB); + } +}; + +IComputationNode* WrapDqYtRead(TCallable& callable, NKikimr::NMiniKQL::IStatsRegistry* jobStats, const TComputationNodeFactoryContext& ctx, bool useBlocks) { + MKQL_ENSURE(callable.GetInputsCount() == 8 || callable.GetInputsCount() == 9, "Expected 8 or 9 arguments."); + + TString clusterName(AS_VALUE(TDataLiteral, callable.GetInput(0))->AsValue().AsStringRef()); + TString tokenName(AS_VALUE(TDataLiteral, callable.GetInput(1))->AsValue().AsStringRef()); + TString inputSpec(AS_VALUE(TDataLiteral, callable.GetInput(2))->AsValue().AsStringRef()); + TString samplingSpec(AS_VALUE(TDataLiteral, callable.GetInput(3))->AsValue().AsStringRef()); + TString token; + if (auto sec = ctx.SecureParamsProvider) { + NUdf::TStringRef val; + if (sec->GetSecureParam(tokenName, val)) { + token = val; + } + } + + TVector<ui32> inputGroups; + TVector<TString> tableNames; + TVector<std::pair<NYT::TRichYPath, NYT::TFormat>> tables; // richpath, skiff + TListLiteral* groupList = AS_VALUE(TListLiteral, callable.GetInput(4)); + TVector<ui64> tableOffsets; + for (ui32 grp = 0; grp < groupList->GetItemsCount(); ++grp) { + TListLiteral* tableList = AS_VALUE(TListLiteral, groupList->GetItems()[grp]); + for (ui32 i = 0; i < tableList->GetItemsCount(); ++i) { + TTupleLiteral* tableTuple = AS_VALUE(TTupleLiteral, tableList->GetItems()[i]); + YQL_ENSURE(tableTuple->GetValuesCount() == 4); + tableNames.emplace_back(AS_VALUE(TDataLiteral, tableTuple->GetValue(0))->AsValue().AsStringRef()); + inputGroups.push_back(grp); + + NYT::TRichYPath richYPath; + NYT::Deserialize(richYPath, NYT::NodeFromYsonString(TString(AS_VALUE(TDataLiteral, tableTuple->GetValue(1))->AsValue().AsStringRef()))); + tables.emplace_back(richYPath, NYT::TFormat(NYT::NodeFromYsonString(AS_VALUE(TDataLiteral, tableTuple->GetValue(2))->AsValue().AsStringRef()))); + tableOffsets.push_back(AS_VALUE(TDataLiteral, tableTuple->GetValue(3))->AsValue().Get<ui64>()); + } + } + if (1 == groupList->GetItemsCount()) { + inputGroups.clear(); + } + size_t timeout(AS_VALUE(TDataLiteral, callable.GetInput(7))->AsValue().Get<size_t>()); +#ifdef __linux__ + size_t inflight(AS_VALUE(TDataLiteral, callable.GetInput(6))->AsValue().Get<size_t>()); + if (inflight) { + if (useBlocks) { + return CreateDqYtReadBlockWrapper(ctx, clusterName, token, + NYT::NodeFromYsonString(inputSpec), samplingSpec ? NYT::NodeFromYsonString(samplingSpec) : NYT::TNode(), + inputGroups, static_cast<TType*>(callable.GetInput(5).GetNode()), tableNames, std::move(tables), jobStats, + inflight, timeout, tableOffsets); + } else { + return new TDqYtReadWrapperRPC(ctx, clusterName, token, + NYT::NodeFromYsonString(inputSpec), samplingSpec ? NYT::NodeFromYsonString(samplingSpec) : NYT::TNode(), + inputGroups, static_cast<TType*>(callable.GetInput(5).GetNode()), tableNames, std::move(tables), jobStats, + inflight, timeout, tableOffsets); + } + } else { + YQL_ENSURE(!useBlocks); + return new TDqYtReadWrapperHttp(ctx, clusterName, token, + NYT::NodeFromYsonString(inputSpec), samplingSpec ? NYT::NodeFromYsonString(samplingSpec) : NYT::TNode(), + inputGroups, static_cast<TType*>(callable.GetInput(5).GetNode()), tableNames, std::move(tables), jobStats, + inflight, timeout, tableOffsets); + } +#else + YQL_ENSURE(!useBlocks); + return new TDqYtReadWrapperHttp(ctx, clusterName, token, + NYT::NodeFromYsonString(inputSpec), samplingSpec ? NYT::NodeFromYsonString(samplingSpec) : NYT::TNode(), + inputGroups, static_cast<TType*>(callable.GetInput(5).GetNode()), tableNames, std::move(tables), jobStats, 0, timeout, + tableOffsets); +#endif +} + +} // NYql diff --git a/yt/yql/providers/yt/comp_nodes/dq/dq_yt_reader.h b/yt/yql/providers/yt/comp_nodes/dq/dq_yt_reader.h new file mode 100644 index 0000000000..4bf763d042 --- /dev/null +++ b/yt/yql/providers/yt/comp_nodes/dq/dq_yt_reader.h @@ -0,0 +1,11 @@ +#pragma once + +#include <yql/essentials/minikql/computation/mkql_computation_node.h> +#include <yql/essentials/minikql/mkql_stats_registry.h> +#include <yql/essentials/minikql/mkql_node.h> + +namespace NYql::NDqs { + +NKikimr::NMiniKQL::IComputationNode* WrapDqYtRead(NKikimr::NMiniKQL::TCallable& callable, NKikimr::NMiniKQL::IStatsRegistry* jobStats, const NKikimr::NMiniKQL::TComputationNodeFactoryContext& ctx, bool useBlocks); + +} // NYql diff --git a/yt/yql/providers/yt/comp_nodes/dq/dq_yt_reader_impl.h b/yt/yql/providers/yt/comp_nodes/dq/dq_yt_reader_impl.h new file mode 100644 index 0000000000..e10df53d2a --- /dev/null +++ b/yt/yql/providers/yt/comp_nodes/dq/dq_yt_reader_impl.h @@ -0,0 +1,207 @@ +#pragma once + +#include "dq_yt_reader.h" + +#include <yt/yql/providers/yt/comp_nodes/yql_mkql_file_input_state.h> +#include <yt/yql/providers/yt/codec/yt_codec.h> +#include <yql/essentials/providers/common/codec/yql_codec.h> +#include <yql/essentials/minikql/mkql_node_cast.h> +#include <yql/essentials/minikql/computation/mkql_computation_node_codegen.h> + +#include <yt/cpp/mapreduce/interface/common.h> +#include <yt/cpp/mapreduce/interface/errors.h> +#include <yt/cpp/mapreduce/interface/client.h> +#include <yt/cpp/mapreduce/interface/serialize.h> +#include <yt/cpp/mapreduce/interface/config.h> +#include <yt/cpp/mapreduce/common/wait_proxy.h> + +#include <library/cpp/yson/node/node.h> +#include <library/cpp/yson/node/node_io.h> + +#include <util/generic/size_literals.h> +#include <util/stream/output.h> + +namespace NYql::NDqs { + +using namespace NKikimr::NMiniKQL; + +template<typename T, typename IS> +class TDqYtReadWrapperBase : public TStatefulWideFlowCodegeneratorNode<TDqYtReadWrapperBase<T, IS>> { +using TBaseComputation = TStatefulWideFlowCodegeneratorNode<TDqYtReadWrapperBase<T, IS>>; +public: + TDqYtReadWrapperBase(const TComputationNodeFactoryContext& ctx, const TString& clusterName, + const TString& token, const NYT::TNode& inputSpec, const NYT::TNode& samplingSpec, + const TVector<ui32>& inputGroups, + TType* itemType, const TVector<TString>& tableNames, TVector<std::pair<NYT::TRichYPath, NYT::TFormat>>&& tables, + NKikimr::NMiniKQL::IStatsRegistry* jobStats, size_t inflight, size_t timeout, const TVector<ui64>& tableOffsets) + : TBaseComputation(ctx.Mutables, this, EValueRepresentation::Boxed, EValueRepresentation::Boxed) + , Width(AS_TYPE(TStructType, itemType)->GetMembersCount()) + , CodecCtx(ctx.Env, ctx.FunctionRegistry, &ctx.HolderFactory) + , ClusterName(clusterName) + , Token(token) + , SamplingSpec(samplingSpec) + , Tables(std::move(tables)) + , Inflight(inflight) + , Timeout(timeout) + { + Specs.SetUseSkiff("", TMkqlIOSpecs::ESystemField::RowIndex | TMkqlIOSpecs::ESystemField::RangeIndex); + Specs.Init(CodecCtx, inputSpec, inputGroups, tableNames, itemType, {}, {}, jobStats); + Specs.SetTableOffsets(tableOffsets); + } + + class TState: public TComputationValue<TState>, public IS { + public: + + template<typename... Args> + TState(TMemoryUsageInfo* memInfo, Args&&... args) + : TComputationValue<TState>(memInfo) + , IS(std::forward<Args>(args)...) + { + IS::SetNextBlockCallback([this]() { Yield_ = true; }); + } + + virtual ~TState() = default; + + NUdf::TUnboxedValuePod FetchRecord() { + if (!AtStart_) { + IS::Next(); + } + AtStart_ = false; + + if (!IS::IsValid()) { + IS::Finish(); + return NUdf::TUnboxedValuePod::MakeFinish(); + } + + if (Yield_) { + Yield_ = false; + AtStart_ = true; + return NUdf::TUnboxedValuePod::MakeYield(); + } + + return IS::GetCurrent().Release(); + } + + private: + bool AtStart_ = true; + bool Yield_ = false; + }; + + void MakeState(TComputationContext& ctx, NUdf::TUnboxedValue& state) const { + static_cast<const T*>(this)->MakeState(ctx, state); + } + + EFetchResult DoCalculate(NUdf::TUnboxedValue& state, TComputationContext& ctx, NUdf::TUnboxedValue*const* output) const { + if (state.IsInvalid()) { + MakeState(ctx, state); + } + + if (const auto value = static_cast<TState&>(*state.AsBoxed()).FetchRecord(); value.IsFinish()) + return EFetchResult::Finish; + else if (value.IsYield()) + return EFetchResult::Yield; + else { + const auto elements = value.GetElements(); + for (ui32 i = 0U; i < Width; ++i) + if (const auto out = *output++) + *out = elements[i]; + + } + + return EFetchResult::One; + } + +#ifndef MKQL_DISABLE_CODEGEN + ICodegeneratorInlineWideNode::TGenerateResult DoGenGetValues(const TCodegenContext& ctx, Value* statePtr, BasicBlock*& block) const { + auto& context = ctx.Codegen.GetContext(); + + const auto valueType = Type::getInt128Ty(context); + const auto arrayType = ArrayType::get(valueType, Width); + const auto pointerType = PointerType::getUnqual(arrayType); + const auto structPtrType = PointerType::getUnqual(StructType::get(context)); + const auto statusType = Type::getInt32Ty(context); + + const auto stateType = StructType::get(context, { + structPtrType, // vtbl + Type::getInt32Ty(context), // ref + Type::getInt16Ty(context), // abi + Type::getInt16Ty(context), // reserved + structPtrType // meminfo + }); + + const auto statePtrType = PointerType::getUnqual(stateType); + + const auto placeholder = new AllocaInst(pointerType, 0U, "paceholder", &ctx.Func->getEntryBlock().back()); + + const auto make = BasicBlock::Create(context, "make", ctx.Func); + const auto main = BasicBlock::Create(context, "main", ctx.Func); + const auto good = BasicBlock::Create(context, "good", ctx.Func); + const auto done = BasicBlock::Create(context, "done", ctx.Func); + + BranchInst::Create(make, main, IsInvalid(statePtr, block), block); + block = make; + + const auto self = CastInst::Create(Instruction::IntToPtr, ConstantInt::get(Type::getInt64Ty(context), uintptr_t(static_cast<const TDqYtReadWrapperBase<T, IS>*>(this))), structPtrType, "self", block); + const auto makeFunc = ConstantInt::get(Type::getInt64Ty(context), GetMethodPtr(&TDqYtReadWrapperBase<T, IS>::MakeState)); + const auto makeType = FunctionType::get(Type::getVoidTy(context), {self->getType(), ctx.Ctx->getType(), statePtr->getType()}, false); + const auto makeFuncPtr = CastInst::Create(Instruction::IntToPtr, makeFunc, PointerType::getUnqual(makeType), "function", block); + CallInst::Create(makeType, makeFuncPtr, {self, ctx.Ctx, statePtr}, "", block); + BranchInst::Create(main, block); + + block = main; + + const auto state = new LoadInst(valueType, statePtr, "state", block); + const auto half = CastInst::Create(Instruction::Trunc, state, Type::getInt64Ty(context), "half", block); + const auto stateArg = CastInst::Create(Instruction::IntToPtr, half, statePtrType, "state_arg", block); + + const auto func = ConstantInt::get(Type::getInt64Ty(context), GetMethodPtr(&TState::FetchRecord)); + const auto funcType = FunctionType::get(valueType, { statePtrType }, false); + const auto funcPtr = CastInst::Create(Instruction::IntToPtr, func, PointerType::getUnqual(funcType), "fetch_func", block); + const auto fetch = CallInst::Create(funcType, funcPtr, { stateArg }, "fetch", block); + + const auto result = PHINode::Create(statusType, 2U, "result", done); + const auto special = SelectInst::Create(IsYield(fetch, block), ConstantInt::get(statusType, static_cast<i32>(EFetchResult::Yield)), ConstantInt::get(statusType, static_cast<i32>(EFetchResult::Finish)), "special", block); + result->addIncoming(special, block); + + BranchInst::Create(done, good, IsSpecial(fetch, block), block); + + block = good; + + const auto elements = CallBoxedValueVirtualMethod<NUdf::TBoxedValueAccessor::EMethod::GetElements>(pointerType, fetch, ctx.Codegen, block); + new StoreInst(elements, placeholder, block); + + result->addIncoming(ConstantInt::get(statusType, static_cast<i32>(EFetchResult::One)), block); + + BranchInst::Create(done, block); + + block = done; + + ICodegeneratorInlineWideNode::TGettersList getters; + getters.reserve(Width); + for (ui32 i = 0U; i < Width; ++i) { + getters.emplace_back([i, placeholder, pointerType, arrayType, valueType](const TCodegenContext& ctx, BasicBlock*& block) { + const auto indexType = Type::getInt32Ty(ctx.Codegen.GetContext()); + const auto pointer = new LoadInst(pointerType, placeholder, (TString("pointer_") += ToString(i)).c_str(), block); + const auto ptr = GetElementPtrInst::CreateInBounds(arrayType, pointer, {ConstantInt::get(indexType, 0), ConstantInt::get(indexType, i)}, (TString("ptr_") += ToString(i)).c_str(), block); + const auto load = new LoadInst(valueType, ptr, (TString("load_") += ToString(i)).c_str(), block); + return load; + }); + } + + return {result, std::move(getters)}; + } +#endif + void RegisterDependencies() const final {} + + const ui32 Width; + NCommon::TCodecContext CodecCtx; + TMkqlIOSpecs Specs; + + TString ClusterName; + TString Token; + NYT::TNode SamplingSpec; + TVector<std::pair<NYT::TRichYPath, NYT::TFormat>> Tables; + size_t Inflight; + size_t Timeout; +}; +} diff --git a/yt/yql/providers/yt/comp_nodes/dq/dq_yt_rpc_helpers.cpp b/yt/yql/providers/yt/comp_nodes/dq/dq_yt_rpc_helpers.cpp new file mode 100644 index 0000000000..6e5e5e829b --- /dev/null +++ b/yt/yql/providers/yt/comp_nodes/dq/dq_yt_rpc_helpers.cpp @@ -0,0 +1,95 @@ +#include "dq_yt_rpc_helpers.h" + +#include <yt/yql/providers/yt/lib/yt_rpc_helpers/yt_convert_helpers.h> + +namespace NYql::NDqs { + +NYT::NYPath::TRichYPath ConvertYPathFromOld(const NYT::TRichYPath& richYPath) { + TStringStream ss; + NYT::PathToNode(richYPath).Save(&ss); + NYT::NYPath::TRichYPath path; + NYT::NYPath::Deserialize(path, NYT::NYTree::ConvertToNode(NYT::NYson::TYsonString(ss.Str()))); + return path; +} + +std::unique_ptr<TSettingsHolder> CreateInputStreams(bool isArrow, const TString& token, const TString& clusterName, const ui64 timeout, bool unordered, const TVector<std::pair<NYT::TRichYPath, NYT::TFormat>>& tables, NYT::TNode samplingSpec) { + auto connectionConfig = NYT::New<NYT::NApi::NRpcProxy::TConnectionConfig>(); + connectionConfig->ClusterUrl = clusterName; + connectionConfig->EnableRetries = true; + connectionConfig->DefaultPingPeriod = TDuration::MilliSeconds(5000); + + auto connection = CreateConnection(connectionConfig); + auto clientOptions = NYT::NApi::TClientOptions(); + + if (token) { + clientOptions.Token = token; + } + + auto client = DynamicPointerCast<NYT::NApi::NRpcProxy::TClient>(connection->CreateClient(clientOptions)); + Y_ABORT_UNLESS(client); + auto apiServiceProxy = client->CreateApiServiceProxy(); + + TVector<NYT::NApi::NRpcProxy::TApiServiceProxy::TReqReadTablePtr> requests; + + size_t inputIdx = 0; + TVector<size_t> originalIndexes; + + for (auto [richYPath, format]: tables) { + if (richYPath.GetRanges() && richYPath.GetRanges()->empty()) { + ++inputIdx; + continue; + } + originalIndexes.emplace_back(inputIdx++); + + auto request = apiServiceProxy.ReadTable(); + client->InitStreamingRequest(*request); + request->ServerAttachmentsStreamingParameters().WriteTimeout = TDuration::MilliSeconds(timeout); + request->ClientAttachmentsStreamingParameters().ReadTimeout = TDuration::MilliSeconds(timeout); + + TString ppath; + auto tableYPath = ConvertYPathFromOld(richYPath); + + NYT::NYPath::ToProto(&ppath, tableYPath); + request->set_path(ppath); + request->set_desired_rowset_format(isArrow ? NYT::NApi::NRpcProxy::NProto::ERowsetFormat::RF_ARROW : NYT::NApi::NRpcProxy::NProto::ERowsetFormat::RF_FORMAT); + if (isArrow) { + request->set_arrow_fallback_rowset_format(NYT::NApi::NRpcProxy::NProto::ERowsetFormat::RF_FORMAT); + } + + // TODO() Enable row indexes + request->set_enable_row_index(!isArrow); + request->set_enable_table_index(true); + // TODO() Enable range indexes + request->set_enable_range_index(!isArrow); + + request->set_unordered(unordered); + + // https://a.yandex-team.ru/arcadia/yt/yt_proto/yt/client/api/rpc_proxy/proto/api_service.proto?rev=r11519304#L2338 + if (!samplingSpec.IsUndefined()) { + TStringStream ss; + samplingSpec.Save(&ss); + request->set_config(ss.Str()); + } + + ConfigureTransaction(request, richYPath); + // Get skiff format yson string + TStringStream fmt; + format.Config.Save(&fmt); + request->set_format(fmt.Str()); + + requests.emplace_back(std::move(request)); + } + return std::make_unique<TSettingsHolder>(std::move(connection), std::move(client), std::move(requests), std::move(originalIndexes)); +} + +NYT::TFuture<NYT::NConcurrency::IAsyncZeroCopyInputStreamPtr> CreateInputStream(NYT::NApi::NRpcProxy::TApiServiceProxy::TReqReadTablePtr request) { + return CreateRpcClientInputStream(std::move(request)).ApplyUnique(BIND([](NYT::NConcurrency::IAsyncZeroCopyInputStreamPtr&& stream) { + // first packet contains meta, skip it + return stream->Read().ApplyUnique(BIND([stream = std::move(stream)](NYT::TSharedRef&&) { + return std::move(stream); + })); + })); +} + + +} diff --git a/yt/yql/providers/yt/comp_nodes/dq/dq_yt_rpc_helpers.h b/yt/yql/providers/yt/comp_nodes/dq/dq_yt_rpc_helpers.h new file mode 100644 index 0000000000..d089494138 --- /dev/null +++ b/yt/yql/providers/yt/comp_nodes/dq/dq_yt_rpc_helpers.h @@ -0,0 +1,80 @@ +#pragma once + +#include <yt/yql/providers/yt/codec/yt_codec.h> +#include <yql/essentials/public/udf/udf_value_builder.h> + +#include <yt/cpp/mapreduce/interface/io.h> +#include <yt/cpp/mapreduce/common/helpers.h> +#include <yt/cpp/mapreduce/interface/client.h> +#include <yt/cpp/mapreduce/interface/serialize.h> +#include <yt/cpp/mapreduce/interface/config.h> +#include <yt/cpp/mapreduce/interface/common.h> + +#include <yt/yt/library/auth/auth.h> +#include <yt/yt/client/api/client.h> +#include <yt/yt/client/api/rpc_proxy/client_impl.h> +#include <yt/yt/client/api/rpc_proxy/config.h> +#include <yt/yt/client/api/rpc_proxy/connection.h> +#include <yt/yt/client/api/rpc_proxy/row_stream.h> + +#include <arrow/memory_pool.h> + +namespace NYql::NDqs { +NYT::NYPath::TRichYPath ConvertYPathFromOld(const NYT::TRichYPath& richYPath); +class TPayloadRPCReader : public NYT::TRawTableReader { +public: + TPayloadRPCReader(NYT::TSharedRef&& payload) : Payload_(std::move(payload)), PayloadStream_(Payload_.Begin(), Payload_.Size()) {} + + bool Retry(const TMaybe<ui32>&, const TMaybe<ui64>&, const std::exception_ptr&) override { + return false; + } + + void ResetRetries() override { + + } + + bool HasRangeIndices() const override { + return true; + }; + + size_t DoRead(void* buf, size_t len) override { + if (!PayloadStream_.Exhausted()) { + return PayloadStream_.Read(buf, len); + } + return 0; + }; + + virtual ~TPayloadRPCReader() override { + } +private: + NYT::TSharedRef Payload_; + TMemoryInput PayloadStream_; +}; + +struct TSettingsHolder : public TNonCopyable { + TSettingsHolder(NYT::NApi::IConnectionPtr&& connection, NYT::TIntrusivePtr<NYT::NApi::NRpcProxy::TClient>&& client, + TVector<NYT::NApi::NRpcProxy::TApiServiceProxy::TReqReadTablePtr>&& requests, TVector<size_t>&& originalIndexes) + : Connection(std::move(connection)) + , Client(std::move(client)) + , Requests(std::move(requests)) + , OriginalIndexes(std::move(originalIndexes)) {}; + void SetColumns(const TVector<TString>& columnNames) { + for (ui32 i = 0; i < columnNames.size(); ++i) { + ColumnNameMapping[columnNames[i]] = i; + } + } + NYT::NApi::IConnectionPtr Connection; + NYT::TIntrusivePtr<NYT::NApi::NRpcProxy::TClient> Client; + const TMkqlIOSpecs* Specs = nullptr; + arrow::MemoryPool* Pool = nullptr; + const NUdf::IPgBuilder* PgBuilder = nullptr; + TVector<NYT::NApi::NRpcProxy::TApiServiceProxy::TReqReadTablePtr> Requests; + + TVector<size_t> OriginalIndexes; + std::unordered_map<std::string, ui32> ColumnNameMapping; +}; + +std::unique_ptr<TSettingsHolder> CreateInputStreams(bool isArrow, const TString& token, const TString& clusterName, const ui64 timeout, bool unordered, const TVector<std::pair<NYT::TRichYPath, NYT::TFormat>>& tables, NYT::TNode samplingSpec); +NYT::TFuture<NYT::NConcurrency::IAsyncZeroCopyInputStreamPtr> CreateInputStream(NYT::NApi::NRpcProxy::TApiServiceProxy::TReqReadTablePtr requestPtr); + +}; diff --git a/yt/yql/providers/yt/comp_nodes/dq/dq_yt_rpc_reader.cpp b/yt/yql/providers/yt/comp_nodes/dq/dq_yt_rpc_reader.cpp new file mode 100644 index 0000000000..a41c1b8761 --- /dev/null +++ b/yt/yql/providers/yt/comp_nodes/dq/dq_yt_rpc_reader.cpp @@ -0,0 +1,245 @@ +#include "dq_yt_rpc_reader.h" +#include "dq_yt_rpc_helpers.h" + +#include <yql/essentials/utils/failure_injector/failure_injector.h> + +#include "yt/cpp/mapreduce/common/helpers.h" + +#include <yt/yt/library/auth/auth.h> + +#include <yt/yt/client/api/client.h> + +#include <yt/yt/client/api/rpc_proxy/client_impl.h> +#include <yt/yt/client/api/rpc_proxy/config.h> +#include <yt/yt/client/api/rpc_proxy/connection.h> +#include <yt/yt/client/api/rpc_proxy/row_stream.h> + +namespace NYql::NDqs { + +using namespace NKikimr::NMiniKQL; + +namespace { +TStatKey RPCReaderAwaitingStallTime("Job_RPCReaderAwaitingStallTime", true); +} +#ifdef RPC_PRINT_TIME +int cnt = 0; +auto beg = std::chrono::steady_clock::now(); +std::mutex mtx; +void print_add(int x) { + std::lock_guard l(mtx); + Cerr << (cnt += x) << " (" << std::chrono::duration_cast<std::chrono::milliseconds>(std::chrono::steady_clock::now() - beg).count() << "ms gone from process start)\n"; +} +#endif + +TParallelFileInputState::TParallelFileInputState(const TMkqlIOSpecs& spec, + const NKikimr::NMiniKQL::THolderFactory& holderFactory, + size_t blockSize, + size_t inflight, + std::unique_ptr<TSettingsHolder>&& settings) + : InnerState_(new TInnerState (settings->Requests.size(), inflight)) + , StateByReader_(settings->Requests.size()) + , Spec_(&spec) + , HolderFactory_(holderFactory) + , BlockSize_(blockSize) + , TimerAwaiting_(RPCReaderAwaitingStallTime, 100) + , OriginalIndexes_(std::move(settings->OriginalIndexes)) + , Settings_(std::move(settings)) +{ +#ifdef RPC_PRINT_TIME + print_add(1); +#endif + YQL_ENSURE(Settings_->Requests.size() == OriginalIndexes_.size()); + MkqlReader_.SetSpecs(*Spec_, HolderFactory_); + Valid_ = NextValue(); +} + +size_t TParallelFileInputState::GetTableIndex() const { + return OriginalIndexes_[CurrentInput_]; +} + +size_t TParallelFileInputState::GetRecordIndex() const { + Y_ABORT("Not implemented"); + return CurrentRecord_; // returns 1-based index +} + +void TParallelFileInputState::SetNextBlockCallback(std::function<void()> cb) { + MkqlReader_.SetNextBlockCallback(cb); + OnNextBlockCallback_ = std::move(cb); +} + +bool TParallelFileInputState::IsValid() const { + return Valid_; +} + +NUdf::TUnboxedValue TParallelFileInputState::GetCurrent() { + return CurrentValue_; +} + +void TParallelFileInputState::Next() { + Valid_ = NextValue(); +} + +void TParallelFileInputState::Finish() { + TimerAwaiting_.Report(Spec_->JobStats_); + MkqlReader_.Finish(); +} + +void TParallelFileInputState::CheckError() const { + if (!InnerState_->Error.IsOK()) { + Cerr << "YT RPC Reader exception:\n" << InnerState_->Error.GetMessage(); + InnerState_->Error.ThrowOnError(); + } +} + +// Call when reader created, or when previous batch was successfully decoded +void TParallelFileInputState::TInnerState::InputReady(size_t idx) { + std::lock_guard guard(Lock); + ReadyReaders.emplace(idx); + --ReadersInflight; +} + +TMaybe<size_t> TParallelFileInputState::TInnerState::GetFreeReader() { + std::lock_guard guard(Lock); + if (ReadersInflight >= Inflight) { + return {}; + } + if (ReadyReaders.size()) { + size_t res = ReadyReaders.front(); + ReadyReaders.pop(); + ++ReadersInflight; + return res; + } + if (NextFreeReaderIdx >= RawInputs.size()) { + return {}; + } + ++ReadersInflight; + return NextFreeReaderIdx++; +} + +void TParallelFileInputState::TInnerState::InputDone() { + std::lock_guard guard(Lock); + --ReadersInflight; +} + +bool TParallelFileInputState::TInnerState::AllReadersDone() { + std::lock_guard guard(Lock); + return NextFreeReaderIdx == RawInputs.size() && ReadyReaders.empty() && !ReadersInflight; +} + +namespace { + +void ReadCallback(NYT::TErrorOr<NYT::TSharedRef>&& res_, std::shared_ptr<TParallelFileInputState::TInnerState> state, size_t inputIdx) { + TFailureInjector::Reach("dq_rpc_reader_read_err_when_empty", [&res_] { + res_ = NYT::TErrorOr<NYT::TSharedRef>(NYT::TError("Failure_On_Read_Callback")); + }); + + if (!res_.IsOK()) { + std::lock_guard lock(state->Lock); + state->Error = std::move(res_); + state->WaitPromise.TrySet(); + return; + } + + auto block = std::move(res_.Value()); + NYT::NApi::NRpcProxy::NProto::TRowsetDescriptor descriptor; + NYT::NApi::NRpcProxy::NProto::TRowsetStatistics statistics; + + auto CurrentPayload_ = std::move(NYT::NApi::NRpcProxy::DeserializeRowStreamBlockEnvelope(block, &descriptor, &statistics)); + // skip no-skiff data + if (descriptor.rowset_format() != NYT::NApi::NRpcProxy::NProto::RF_FORMAT) { + state->WaitPromise.TrySet(); + return; + } + + if (CurrentPayload_.Empty()) { + state->WaitPromise.TrySet(); + state->InputDone(); + return; + } + std::lock_guard lock(state->Lock); + + state->Results.emplace(std::move(TParallelFileInputState::TResult{inputIdx, std::move(CurrentPayload_)})); + state->WaitPromise.TrySet(); +} + +void ExecuteRead(size_t inputIdx, std::shared_ptr<TParallelFileInputState::TInnerState> state) { + state->RawInputs[inputIdx]->Read().SubscribeUnique(BIND([state, inputIdx](NYT::TErrorOr<NYT::TSharedRef>&& res_){ + ReadCallback(std::move(res_), state, inputIdx); + })); +} +} + +void TParallelFileInputState::RunNext() { + while (auto idx = InnerState_->GetFreeReader()) { + auto inputIdx = *idx; + if (!InnerState_->RawInputs[inputIdx]) { + CreateInputStream(Settings_->Requests[inputIdx]).SubscribeUnique(BIND([state = InnerState_, inputIdx](NYT::TErrorOr<NYT::NConcurrency::IAsyncZeroCopyInputStreamPtr>&& stream) { + if (!stream.IsOK()) { + state->Error = stream; + return; + } + state->RawInputs[inputIdx] = std::move(stream.Value()); + ExecuteRead(inputIdx, state); + })); + continue; + } + ExecuteRead(inputIdx, InnerState_); + } +} + +bool TParallelFileInputState::NextValue() { + for (;;) { + if (!InnerState_->AllReadersDone()) { + RunNext(); + } + if (MkqlReader_.IsValid()) { + CurrentValue_ = std::move(MkqlReader_.GetRow()); + if (!Spec_->InputGroups.empty()) { + CurrentValue_ = HolderFactory_.CreateVariantHolder(CurrentValue_.Release(), Spec_->InputGroups.at(OriginalIndexes_[CurrentInput_])); + } + MkqlReader_.Next(); + return true; + } + if (!Settings_->Requests.empty()) { + if (MkqlReader_.GetRowIndexUnchecked()) { + StateByReader_[CurrentInput_].CurrentRow = *MkqlReader_.GetRowIndexUnchecked() - 1; + } + StateByReader_[CurrentInput_].CurrentRange = MkqlReader_.GetRangeIndexUnchecked(); + } + bool needWait = false; + { + std::lock_guard lock(InnerState_->Lock); + needWait = InnerState_->Results.empty(); + } + if (needWait && InnerState_->AllReadersDone()) { + CheckError(); + return false; + } + if (needWait) { + auto guard = Guard(TimerAwaiting_); + YQL_ENSURE(NYT::NConcurrency::WaitFor(InnerState_->WaitPromise.ToFuture()).IsOK()); + } + TResult result; + { + std::lock_guard lock(InnerState_->Lock); + CheckError(); + if (InnerState_->Results.empty()) { + continue; + } + result = std::move(InnerState_->Results.front()); + InnerState_->Results.pop(); + InnerState_->WaitPromise = NYT::NewPromise<void>(); + } + CurrentInput_ = result.Input_; + CurrentReader_ = MakeIntrusive<TPayloadRPCReader>(std::move(result.Value_)); + MkqlReader_.SetReader(*CurrentReader_, 1, BlockSize_, ui32(OriginalIndexes_[CurrentInput_]), true, StateByReader_[CurrentInput_].CurrentRow, StateByReader_[CurrentInput_].CurrentRange); + MkqlReader_.Next(); + InnerState_->InputReady(CurrentInput_); + } +} + +void TDqYtReadWrapperRPC::MakeState(TComputationContext& ctx, NUdf::TUnboxedValue& state) const { + auto settings = CreateInputStreams(false, Token, ClusterName, Timeout, Inflight > 1, Tables, SamplingSpec); + state = ctx.HolderFactory.Create<TDqYtReadWrapperBase<TDqYtReadWrapperRPC, TParallelFileInputState>::TState>(Specs, ctx.HolderFactory, 4_MB, Inflight, std::move(settings)); +} +} diff --git a/yt/yql/providers/yt/comp_nodes/dq/dq_yt_rpc_reader.h b/yt/yql/providers/yt/comp_nodes/dq/dq_yt_rpc_reader.h new file mode 100644 index 0000000000..9288e98fd4 --- /dev/null +++ b/yt/yql/providers/yt/comp_nodes/dq/dq_yt_rpc_reader.h @@ -0,0 +1,103 @@ +#pragma once + +#include "dq_yt_reader_impl.h" + +#include <yt/yt/core/actions/future.h> +#include <mutex> + +namespace NYql::NDqs { + +using namespace NKikimr::NMiniKQL; + +struct TSettingsHolder; + +class TParallelFileInputState: public IInputState { +public: + struct TResult { + size_t Input_ = 0; + NYT::TSharedRef Value_; + }; + struct TReaderState { + TMaybe<ui64> CurrentRow; + TMaybe<ui32> CurrentRange; + }; + // Used to pass struct in lambdas. std::shared_ptr copying is thread-safe + struct TInnerState { + TInnerState(size_t inputsCount, size_t inflight) : RawInputs(inputsCount), Inflight(inflight) {}; + void InputReady(size_t idx); + void InputDone(); + TMaybe<size_t> GetFreeReader(); + bool AllReadersDone(); + + std::mutex Lock; + std::queue<TResult> Results; + NYT::TError Error{}; + + NYT::TPromise<void> WaitPromise = NYT::NewPromise<void>(); + TVector<NYT::NConcurrency::IAsyncZeroCopyInputStreamPtr> RawInputs; + std::queue<size_t> ReadyReaders; + size_t NextFreeReaderIdx = 0; + size_t ReadersInflight = 0; + size_t Inflight; + }; + TParallelFileInputState(const TMkqlIOSpecs& spec, + const NKikimr::NMiniKQL::THolderFactory& holderFactory, + size_t blockSize, + size_t inflight, + std::unique_ptr<TSettingsHolder>&& settings); + + size_t GetTableIndex() const; + + size_t GetRecordIndex() const; + + void SetNextBlockCallback(std::function<void()> cb); +protected: + virtual bool IsValid() const override; + + virtual NUdf::TUnboxedValue GetCurrent() override; + + virtual void Next() override; + + void Finish(); + + void RunNext(); + + bool NextValue(); + +private: + void CheckError() const; + std::shared_ptr<TInnerState> InnerState_; + std::vector<TReaderState> StateByReader_; + NYT::TRawTableReaderPtr CurrentReader_ = nullptr; + bool Eof_ = false; + const TMkqlIOSpecs* Spec_; + const NKikimr::NMiniKQL::THolderFactory& HolderFactory_; + const size_t BlockSize_; + TMkqlReaderImpl MkqlReader_; + size_t CurrentInput_ = 0; + size_t CurrentRecord_ = 1; + bool Valid_ = true; + NUdf::TUnboxedValue CurrentValue_; + std::function<void()> OnNextBlockCallback_; + NKikimr::NMiniKQL::TSamplingStatTimer TimerAwaiting_; + TVector<size_t> OriginalIndexes_; + std::unique_ptr<TSettingsHolder> Settings_; +}; + + +class TDqYtReadWrapperRPC : public TDqYtReadWrapperBase<TDqYtReadWrapperRPC, TParallelFileInputState> { +public: +using TInputType = NYT::NConcurrency::IAsyncZeroCopyInputStreamPtr; + TDqYtReadWrapperRPC(const TComputationNodeFactoryContext& ctx, const TString& clusterName, + const TString& token, const NYT::TNode& inputSpec, const NYT::TNode& samplingSpec, + const TVector<ui32>& inputGroups, TType* itemType, const TVector<TString>& tableNames, + TVector<std::pair<NYT::TRichYPath, NYT::TFormat>>&& tables, + NKikimr::NMiniKQL::IStatsRegistry* jobStats, size_t inflight, size_t timeout, + const TVector<ui64>& tableOffsets) + : TDqYtReadWrapperBase<TDqYtReadWrapperRPC, TParallelFileInputState>(ctx, clusterName, token, + inputSpec, samplingSpec, inputGroups, itemType, tableNames, std::move(tables), jobStats, + inflight, timeout, tableOffsets) {} + + void MakeState(TComputationContext& ctx, NUdf::TUnboxedValue& state) const; +}; +} diff --git a/yt/yql/providers/yt/comp_nodes/dq/dq_yt_writer.cpp b/yt/yql/providers/yt/comp_nodes/dq/dq_yt_writer.cpp new file mode 100644 index 0000000000..65773c8365 --- /dev/null +++ b/yt/yql/providers/yt/comp_nodes/dq/dq_yt_writer.cpp @@ -0,0 +1,282 @@ +#include "dq_yt_writer.h" + +#include <library/cpp/yson/node/node_io.h> +#include <util/generic/size_literals.h> +#include <yt/cpp/mapreduce/interface/client.h> +#include <yt/cpp/mapreduce/interface/common.h> +#include <yt/yql/providers/yt/codec/yt_codec_io.h> +#include <yql/essentials/minikql/computation/mkql_computation_node_codegen.h> +#include <yql/essentials/minikql/mkql_node_cast.h> +#include <yt/yql/providers/yt/common/yql_names.h> + +#include <utility> + +namespace NYql::NDqs { +namespace { + +using namespace NKikimr::NMiniKQL; +using namespace NYT; + +class TYtDqWideWriteWrapper final : public TStatefulFlowCodegeneratorNode<TYtDqWideWriteWrapper> { + using TBaseComputation = TStatefulFlowCodegeneratorNode<TYtDqWideWriteWrapper>; + + class TWriterState : public TComputationValue<TWriterState> { + public: + TWriterState( + TMemoryUsageInfo* memInfo, + IClientPtr&& client, + ITransactionPtr&& transaction, + THolder<TMkqlIOSpecs>&& specs, + TRawTableWriterPtr&& outStream, + THolder<TMkqlWriterImpl>&& writer + ) + : TComputationValue<TWriterState>(memInfo) + , Client(std::move(client)), Transaction(std::move(transaction)) + , Specs(std::move(specs)), OutStream(std::move(outStream)), Writer(std::move(writer)) + {} + + ~TWriterState() override { + try { + Finish(); + } catch (...) { + } + } + + void AddRow(const NUdf::TUnboxedValuePod* row) const { + Writer->AddFlatRow(row); + } + + void Finish() { + if (!Finished) { + Writer->Finish(); + OutStream->Finish(); + } + Finished = true; + } + private: + bool Finished = false; + const IClientPtr Client; + const ITransactionPtr Transaction; + const THolder<TMkqlIOSpecs> Specs; + const TRawTableWriterPtr OutStream; + const THolder<TMkqlWriterImpl> Writer; + }; + +public: + TYtDqWideWriteWrapper(TComputationMutables& mutables, + IComputationWideFlowNode* flow, + std::vector<EValueRepresentation>&& representations, + const std::string_view& clusterName, + const std::string_view& token, + const TRichYPath& path, + const NYT::TNode& outSpec, + const NYT::TNode& writerOptions, + THolder<NCommon::TCodecContext>&& codecCtx + ) + : TBaseComputation(mutables, flow, EValueRepresentation::Embedded, EValueRepresentation::Boxed) + , Flow(flow) + , Representations(std::move(representations)) + , ClusterName(clusterName) + , Token(token) + , Path(path) + , OutSpec(outSpec) + , WriterOptions(writerOptions) + , CodecCtx(std::move(codecCtx)) + , Values(Representations.size()), Fields(GetPointers(Values)) + {} + + NUdf::TUnboxedValuePod DoCalculate(NUdf::TUnboxedValue& state, TComputationContext& ctx) const { + if (state.IsFinish()) { + return NUdf::TUnboxedValuePod::MakeFinish(); + } else if (state.IsInvalid()) + MakeState(ctx, state); + + switch (const auto ptr = static_cast<TWriterState*>(state.AsBoxed().Get()); Flow->FetchValues(ctx, Fields.data())) { + case EFetchResult::One: + ptr->AddRow(Values.data()); + return NUdf::TUnboxedValuePod::Void(); + case EFetchResult::Yield: + return NUdf::TUnboxedValuePod::MakeYield(); + case EFetchResult::Finish: + ptr->Finish(); + state = NUdf::TUnboxedValuePod::MakeFinish(); + return NUdf::TUnboxedValuePod::MakeFinish(); + } + } +#ifndef MKQL_DISABLE_CODEGEN + Value* DoGenerateGetValue(const TCodegenContext& ctx, Value* statePtr, BasicBlock*& block) const { + auto& context = ctx.Codegen.GetContext(); + + const auto valueType = Type::getInt128Ty(context); + const auto indexType = Type::getInt32Ty(context); + const auto structPtrType = PointerType::getUnqual(StructType::get(context)); + const auto arrayType = ArrayType::get(valueType, Representations.size()); + const auto values = new AllocaInst(arrayType, 0U, "values", &ctx.Func->getEntryBlock().back()); + + const auto init = BasicBlock::Create(context, "init", ctx.Func); + const auto next = BasicBlock::Create(context, "next", ctx.Func); + const auto work = BasicBlock::Create(context, "work", ctx.Func); + const auto done = BasicBlock::Create(context, "done", ctx.Func); + const auto exit = BasicBlock::Create(context, "exit", ctx.Func); + + const auto output = PHINode::Create(valueType, 4U, "output", exit); + output->addIncoming(GetFinish(context), block); + + const auto check = new LoadInst(valueType, statePtr, "check", block); + const auto choise = SwitchInst::Create(check, next, 2U, block); + choise->addCase(GetInvalid(context), init); + choise->addCase(GetFinish(context), exit); + + block = init; + + const auto ptrType = PointerType::getUnqual(StructType::get(context)); + const auto self = CastInst::Create(Instruction::IntToPtr, ConstantInt::get(Type::getInt64Ty(context), uintptr_t(this)), ptrType, "self", block); + const auto makeFunc = ConstantInt::get(Type::getInt64Ty(context), GetMethodPtr(&TYtDqWideWriteWrapper::MakeState)); + const auto makeType = FunctionType::get(Type::getVoidTy(context), {self->getType(), ctx.Ctx->getType(), statePtr->getType()}, false); + const auto makeFuncPtr = CastInst::Create(Instruction::IntToPtr, makeFunc, PointerType::getUnqual(makeType), "function", block); + CallInst::Create(makeType, makeFuncPtr, {self, ctx.Ctx, statePtr}, "", block); + + BranchInst::Create(next, block); + + block = next; + + const auto result = GetNodeValues(Flow, ctx, block); + + output->addIncoming(GetYield(context), block); + + const auto way = SwitchInst::Create(result.first, work, 2U, block); + way->addCase(ConstantInt::get(indexType, static_cast<i32>(EFetchResult::Yield)), exit); + way->addCase(ConstantInt::get(indexType, static_cast<i32>(EFetchResult::Finish)), done); + + { + block = work; + + TSmallVec<Value*> fields; + fields.reserve(Representations.size()); + for (ui32 i = 0U; i < Representations.size(); ++i) { + const auto pointer = GetElementPtrInst::CreateInBounds(arrayType, values, {ConstantInt::get(indexType, 0), ConstantInt::get(indexType, i)}, (TString("ptr_") += ToString(i)).c_str(), block); + fields.emplace_back(result.second[i](ctx, block)); + new StoreInst(fields.back(), pointer, block); + } + + const auto state = new LoadInst(valueType, statePtr, "state", block); + const auto half = CastInst::Create(Instruction::Trunc, state, Type::getInt64Ty(context), "half", block); + const auto stateArg = CastInst::Create(Instruction::IntToPtr, half, structPtrType, "state_arg", block); + + const auto addFunc = ConstantInt::get(Type::getInt64Ty(context), GetMethodPtr(&TWriterState::AddRow)); + const auto addType = FunctionType::get(Type::getVoidTy(context), {stateArg->getType(), values->getType()}, false); + const auto addPtr = CastInst::Create(Instruction::IntToPtr, addFunc, PointerType::getUnqual(addType), "write", block); + CallInst::Create(addType, addPtr, {stateArg, values}, "", block); + + for (ui32 i = 0U; i < Representations.size(); ++i) { + ValueCleanup(Representations[i], fields[i], ctx, block); + } + + output->addIncoming(GetFalse(context), block); + BranchInst::Create(exit, block); + } + + { + block = done; + + const auto state = new LoadInst(valueType, statePtr, "state", block); + const auto half = CastInst::Create(Instruction::Trunc, state, Type::getInt64Ty(context), "half", block); + const auto stateArg = CastInst::Create(Instruction::IntToPtr, half, structPtrType, "state_arg", block); + + const auto finishFunc = ConstantInt::get(Type::getInt64Ty(context), GetMethodPtr(&TWriterState::Finish)); + const auto finishType = FunctionType::get(Type::getVoidTy(context), {stateArg->getType()}, false); + const auto finishPtr = CastInst::Create(Instruction::IntToPtr, finishFunc, PointerType::getUnqual(finishType), "finish", block); + CallInst::Create(finishType, finishPtr, {stateArg}, "", block); + + UnRefBoxed(statePtr, ctx, block); + new StoreInst(GetFinish(context), statePtr, block); + + output->addIncoming(GetFinish(context), block); + BranchInst::Create(exit, block); + } + + block = exit; + return output; + } +#endif +private: + void MakeState(TComputationContext& ctx, NUdf::TUnboxedValue& state) const { + TString token; + if (NUdf::TStringRef tokenRef(Token); ctx.Builder->GetSecureParam(tokenRef, tokenRef)) { + token = tokenRef; + } + + NYT::TCreateClientOptions createOpts; + if (token) { + createOpts.Token(token); + } + + auto client = NYT::CreateClient(ClusterName, createOpts); + auto transaction = client->AttachTransaction(Path.TransactionId_.GetRef()); + auto specs = MakeHolder<TMkqlIOSpecs>(); + specs->SetUseSkiff(""); + specs->Init(*CodecCtx, OutSpec); + auto path = Path; + path.TransactionId_.Clear(); + NYT::TTableWriterOptions options; + options.Config(WriterOptions); + auto outStream = transaction->CreateRawWriter(path, specs->MakeOutputFormat(), options); + + auto writer = MakeHolder<TMkqlWriterImpl>(outStream, 4_MB); + writer->SetSpecs(*specs); + + state = ctx.HolderFactory.Create<TWriterState>(std::move(client), std::move(transaction), std::move(specs), std::move(outStream), std::move(writer)); + } + + void RegisterDependencies() const final { + FlowDependsOn(Flow); + } + + static std::vector<NUdf::TUnboxedValue*> GetPointers(std::vector<NUdf::TUnboxedValue>& array) { + std::vector<NUdf::TUnboxedValue*> pointers; + pointers.reserve(array.size()); + std::transform(array.begin(), array.end(), std::back_inserter(pointers), [](NUdf::TUnboxedValue& v) { return std::addressof(v); }); + return pointers; + } + + IComputationWideFlowNode* const Flow; + const std::vector<EValueRepresentation> Representations; + + const TString ClusterName; + const TString Token; + const TRichYPath Path; + const NYT::TNode OutSpec; + const NYT::TNode WriterOptions; + const THolder<NCommon::TCodecContext> CodecCtx; + + std::vector<NUdf::TUnboxedValue> Values; + const std::vector<NUdf::TUnboxedValue*> Fields; +}; + +} + +IComputationNode* WrapYtDqRowsWideWrite(TCallable& callable, const TComputationNodeFactoryContext& ctx) { + YQL_ENSURE(callable.GetInputsCount() == 6, "Expected six args."); + + const auto& clusterName = AS_VALUE(TDataLiteral, callable.GetInput(1))->AsValue().AsStringRef(); + const auto& token = AS_VALUE(TDataLiteral, callable.GetInput(2))->AsValue().AsStringRef(); + + TRichYPath richYPath; + Deserialize(richYPath, NYT::NodeFromYsonString(TString(AS_VALUE(TDataLiteral, callable.GetInput(3))->AsValue().AsStringRef()))); + + const NYT::TNode outSpec(NYT::NodeFromYsonString((AS_VALUE(TDataLiteral, callable.GetInput(4))->AsValue().AsStringRef()))); + const NYT::TNode writerOptions(NYT::NodeFromYsonString((AS_VALUE(TDataLiteral, callable.GetInput(5))->AsValue().AsStringRef()))); + const auto node = LocateNode(ctx.NodeLocator, callable, 0); + + std::vector<EValueRepresentation> representations; + auto wideComponents = GetWideComponents(AS_TYPE(TFlowType, callable.GetInput(0).GetStaticType())); + representations.reserve(wideComponents.size()); + for (ui32 i = 0U; i < wideComponents.size(); ++i) { + representations.emplace_back(GetValueRepresentation(wideComponents[i])); + } + + auto codecCtx = MakeHolder<NCommon::TCodecContext>(ctx.Env, ctx.FunctionRegistry, &ctx.HolderFactory); + return new TYtDqWideWriteWrapper(ctx.Mutables, static_cast<IComputationWideFlowNode*>(node), std::move(representations), clusterName, token, richYPath, outSpec, writerOptions, std::move(codecCtx)); +} + +} // NYql diff --git a/yt/yql/providers/yt/comp_nodes/dq/dq_yt_writer.h b/yt/yql/providers/yt/comp_nodes/dq/dq_yt_writer.h new file mode 100644 index 0000000000..222824c2eb --- /dev/null +++ b/yt/yql/providers/yt/comp_nodes/dq/dq_yt_writer.h @@ -0,0 +1,10 @@ +#pragma once + +#include <yql/essentials/minikql/computation/mkql_computation_node.h> +#include <yql/essentials/minikql/mkql_node.h> + +namespace NYql::NDqs { + +NKikimr::NMiniKQL::IComputationNode* WrapYtDqRowsWideWrite(NKikimr::NMiniKQL::TCallable& callable, const NKikimr::NMiniKQL::TComputationNodeFactoryContext& ctx); + +} // NYql diff --git a/yt/yql/providers/yt/comp_nodes/dq/stream_decoder.cpp b/yt/yql/providers/yt/comp_nodes/dq/stream_decoder.cpp new file mode 100644 index 0000000000..920575a5b8 --- /dev/null +++ b/yt/yql/providers/yt/comp_nodes/dq/stream_decoder.cpp @@ -0,0 +1,1404 @@ +#include "stream_decoder.h" + +#include <algorithm> +#include <climits> +#include <cstdint> +#include <cstring> +#include <string> +#include <type_traits> +#include <utility> +#include <vector> + +#include <flatbuffers/flatbuffers.h> // IWYU pragma: export + +#include <arrow/array.h> +#include <arrow/buffer.h> +#include <arrow/extension_type.h> +#include <arrow/io/caching.h> +#include <arrow/io/interfaces.h> +#include <arrow/io/memory.h> +#include <arrow/ipc/message.h> +#include <arrow/ipc/metadata_internal.h> +#include <arrow/ipc/util.h> +#include <arrow/ipc/writer.h> +#include <arrow/record_batch.h> +#include <arrow/sparse_tensor.h> +#include <arrow/status.h> +#include <arrow/type.h> +#include <arrow/type_traits.h> +#include <arrow/util/bit_util.h> +#include <arrow/util/bitmap_ops.h> +#include <arrow/util/checked_cast.h> +#include <arrow/util/compression.h> +#include <arrow/util/endian.h> +#include <arrow/util/key_value_metadata.h> +#include <arrow/util/parallel.h> +#include <arrow/util/string.h> +#include <arrow/util/thread_pool.h> +#include <arrow/util/ubsan.h> +#include <arrow/visitor_inline.h> + +#include <generated/File.fbs.h> // IWYU pragma: export +#include <generated/Message.fbs.h> +#include <generated/Schema.fbs.h> +#include <generated/SparseTensor.fbs.h> + +namespace arrow { + +namespace flatbuf = org::apache::arrow::flatbuf; + +using internal::checked_cast; +using internal::checked_pointer_cast; +using internal::GetByteWidth; + +namespace ipc { + +using internal::FileBlock; +using internal::kArrowMagicBytes; + +namespace NDqs { +Status MaybeAlignMetadata(std::shared_ptr<Buffer>* metadata) { + if (reinterpret_cast<uintptr_t>((*metadata)->data()) % 8 != 0) { + ARROW_ASSIGN_OR_RAISE(*metadata, (*metadata)->CopySlice(0, (*metadata)->size())); + } + return Status::OK(); +} + +Status CheckMetadataAndGetBodyLength(const Buffer& metadata, int64_t* body_length) { + const flatbuf::Message* fb_message = nullptr; + RETURN_NOT_OK(internal::VerifyMessage(metadata.data(), metadata.size(), &fb_message)); + *body_length = fb_message->bodyLength(); + if (*body_length < 0) { + return Status::IOError("Invalid IPC message: negative bodyLength"); + } + return Status::OK(); +} + +std::string FormatMessageType(MessageType type) { + switch (type) { + case MessageType::SCHEMA: + return "schema"; + case MessageType::RECORD_BATCH: + return "record batch"; + case MessageType::DICTIONARY_BATCH: + return "dictionary"; + case MessageType::TENSOR: + return "tensor"; + case MessageType::SPARSE_TENSOR: + return "sparse tensor"; + default: + break; + } + return "unknown"; +} + +Status WriteMessage(const Buffer& message, const IpcWriteOptions& options, + io::OutputStream* file, int32_t* message_length) { + const int32_t prefix_size = options.write_legacy_ipc_format ? 4 : 8; + const int32_t flatbuffer_size = static_cast<int32_t>(message.size()); + + int32_t padded_message_length = static_cast<int32_t>( + PaddedLength(flatbuffer_size + prefix_size, options.alignment)); + + int32_t padding = padded_message_length - flatbuffer_size - prefix_size; + + *message_length = padded_message_length; + + if (!options.write_legacy_ipc_format) { + RETURN_NOT_OK(file->Write(&internal::kIpcContinuationToken, sizeof(int32_t))); + } + + int32_t padded_flatbuffer_size = + BitUtil::ToLittleEndian(padded_message_length - prefix_size); + RETURN_NOT_OK(file->Write(&padded_flatbuffer_size, sizeof(int32_t))); + + RETURN_NOT_OK(file->Write(message.data(), flatbuffer_size)); + if (padding > 0) { + RETURN_NOT_OK(file->Write(kPaddingBytes, padding)); + } + + return Status::OK(); +} + +static constexpr auto kMessageDecoderNextRequiredSizeInitial = sizeof(int32_t); +static constexpr auto kMessageDecoderNextRequiredSizeMetadataLength = sizeof(int32_t); + +class MessageDecoder2::MessageDecoderImpl { + public: + explicit MessageDecoderImpl(std::shared_ptr<MessageDecoderListener> listener, + State initial_state, int64_t initial_next_required_size, + MemoryPool* pool) + : listener_(std::move(listener)), + pool_(pool), + state_(initial_state), + next_required_size_(initial_next_required_size), + save_initial_size_(initial_next_required_size), + chunks_(), + buffered_size_(0), + metadata_(nullptr) {} + + Status ConsumeData(const uint8_t* data, int64_t size) { + if (buffered_size_ == 0) { + while (size > 0 && size >= next_required_size_) { + auto used_size = next_required_size_; + switch (state_) { + case State::INITIAL: + RETURN_NOT_OK(ConsumeInitialData(data, next_required_size_)); + break; + case State::METADATA_LENGTH: + RETURN_NOT_OK(ConsumeMetadataLengthData(data, next_required_size_)); + break; + case State::METADATA: { + auto buffer = std::make_shared<Buffer>(data, next_required_size_); + RETURN_NOT_OK(ConsumeMetadataBuffer(buffer)); + } break; + case State::BODY: { + auto buffer = std::make_shared<Buffer>(data, next_required_size_); + RETURN_NOT_OK(ConsumeBodyBuffer(buffer)); + } break; + case State::EOS: + return Status::OK(); + } + data += used_size; + size -= used_size; + } + } + + if (size == 0) { + return Status::OK(); + } + + chunks_.push_back(std::make_shared<Buffer>(data, size)); + buffered_size_ += size; + return ConsumeChunks(); + } + + Status ConsumeBuffer(std::shared_ptr<Buffer> buffer) { + if (buffered_size_ == 0) { + while (buffer->size() >= next_required_size_) { + auto used_size = next_required_size_; + switch (state_) { + case State::INITIAL: + RETURN_NOT_OK(ConsumeInitialBuffer(buffer)); + break; + case State::METADATA_LENGTH: + RETURN_NOT_OK(ConsumeMetadataLengthBuffer(buffer)); + break; + case State::METADATA: + if (buffer->size() == next_required_size_) { + return ConsumeMetadataBuffer(buffer); + } else { + auto sliced_buffer = SliceBuffer(buffer, 0, next_required_size_); + RETURN_NOT_OK(ConsumeMetadataBuffer(sliced_buffer)); + } + break; + case State::BODY: + if (buffer->size() == next_required_size_) { + return ConsumeBodyBuffer(buffer); + } else { + auto sliced_buffer = SliceBuffer(buffer, 0, next_required_size_); + RETURN_NOT_OK(ConsumeBodyBuffer(sliced_buffer)); + } + break; + case State::EOS: + return Status::OK(); + } + if (buffer->size() == used_size) { + return Status::OK(); + } + buffer = SliceBuffer(buffer, used_size); + } + } + + if (buffer->size() == 0) { + return Status::OK(); + } + + buffered_size_ += buffer->size(); + chunks_.push_back(std::move(buffer)); + return ConsumeChunks(); + } + + int64_t next_required_size() const { return next_required_size_ - buffered_size_; } + + MessageDecoder2::State state() const { return state_; } + void Reset() { + state_ = State::INITIAL; + next_required_size_ = save_initial_size_; + chunks_.clear(); + buffered_size_ = 0; + metadata_ = nullptr; + } + + private: + Status ConsumeChunks() { + while (state_ != State::EOS) { + if (buffered_size_ < next_required_size_) { + return Status::OK(); + } + + switch (state_) { + case State::INITIAL: + RETURN_NOT_OK(ConsumeInitialChunks()); + break; + case State::METADATA_LENGTH: + RETURN_NOT_OK(ConsumeMetadataLengthChunks()); + break; + case State::METADATA: + RETURN_NOT_OK(ConsumeMetadataChunks()); + break; + case State::BODY: + RETURN_NOT_OK(ConsumeBodyChunks()); + break; + case State::EOS: + return Status::OK(); + } + } + + return Status::OK(); + } + + Status ConsumeInitialData(const uint8_t* data, int64_t) { + return ConsumeInitial(BitUtil::FromLittleEndian(util::SafeLoadAs<int32_t>(data))); + } + + Status ConsumeInitialBuffer(const std::shared_ptr<Buffer>& buffer) { + ARROW_ASSIGN_OR_RAISE(auto continuation, ConsumeDataBufferInt32(buffer)); + return ConsumeInitial(BitUtil::FromLittleEndian(continuation)); + } + + Status ConsumeInitialChunks() { + int32_t continuation = 0; + RETURN_NOT_OK(ConsumeDataChunks(sizeof(int32_t), &continuation)); + return ConsumeInitial(BitUtil::FromLittleEndian(continuation)); + } + + Status ConsumeInitial(int32_t continuation) { + if (continuation == internal::kIpcContinuationToken) { + state_ = State::METADATA_LENGTH; + next_required_size_ = kMessageDecoderNextRequiredSizeMetadataLength; + RETURN_NOT_OK(listener_->OnMetadataLength()); + // Valid IPC message, read the message length now + return Status::OK(); + } else if (continuation == 0) { + state_ = State::EOS; + next_required_size_ = 0; + RETURN_NOT_OK(listener_->OnEOS()); + return Status::OK(); + } else if (continuation > 0) { + state_ = State::METADATA; + next_required_size_ = continuation; + RETURN_NOT_OK(listener_->OnMetadata()); + return Status::OK(); + } else { + return Status::IOError("Invalid IPC stream: negative continuation token"); + } + } + + Status ConsumeMetadataLengthData(const uint8_t* data, int64_t) { + return ConsumeMetadataLength( + BitUtil::FromLittleEndian(util::SafeLoadAs<int32_t>(data))); + } + + Status ConsumeMetadataLengthBuffer(const std::shared_ptr<Buffer>& buffer) { + ARROW_ASSIGN_OR_RAISE(auto metadata_length, ConsumeDataBufferInt32(buffer)); + return ConsumeMetadataLength(BitUtil::FromLittleEndian(metadata_length)); + } + + Status ConsumeMetadataLengthChunks() { + int32_t metadata_length = 0; + RETURN_NOT_OK(ConsumeDataChunks(sizeof(int32_t), &metadata_length)); + return ConsumeMetadataLength(BitUtil::FromLittleEndian(metadata_length)); + } + + Status ConsumeMetadataLength(int32_t metadata_length) { + if (metadata_length == 0) { + state_ = State::EOS; + next_required_size_ = 0; + RETURN_NOT_OK(listener_->OnEOS()); + return Status::OK(); + } else if (metadata_length > 0) { + state_ = State::METADATA; + next_required_size_ = metadata_length; + RETURN_NOT_OK(listener_->OnMetadata()); + return Status::OK(); + } else { + return Status::IOError("Invalid IPC message: negative metadata length"); + } + } + + Status ConsumeMetadataBuffer(const std::shared_ptr<Buffer>& buffer) { + if (buffer->is_cpu()) { + metadata_ = buffer; + } else { + ARROW_ASSIGN_OR_RAISE(metadata_, + Buffer::ViewOrCopy(buffer, CPUDevice::memory_manager(pool_))); + } + return ConsumeMetadata(); + } + + Status ConsumeMetadataChunks() { + if (chunks_[0]->size() >= next_required_size_) { + if (chunks_[0]->size() == next_required_size_) { + if (chunks_[0]->is_cpu()) { + metadata_ = std::move(chunks_[0]); + } else { + ARROW_ASSIGN_OR_RAISE( + metadata_, + Buffer::ViewOrCopy(chunks_[0], CPUDevice::memory_manager(pool_))); + } + chunks_.erase(chunks_.begin()); + } else { + metadata_ = SliceBuffer(chunks_[0], 0, next_required_size_); + if (!chunks_[0]->is_cpu()) { + ARROW_ASSIGN_OR_RAISE( + metadata_, Buffer::ViewOrCopy(metadata_, CPUDevice::memory_manager(pool_))); + } + chunks_[0] = SliceBuffer(chunks_[0], next_required_size_); + } + buffered_size_ -= next_required_size_; + } else { + ARROW_ASSIGN_OR_RAISE(auto metadata, AllocateBuffer(next_required_size_, pool_)); + metadata_ = std::shared_ptr<Buffer>(metadata.release()); + RETURN_NOT_OK(ConsumeDataChunks(next_required_size_, metadata_->mutable_data())); + } + return ConsumeMetadata(); + } + + Status ConsumeMetadata() { + RETURN_NOT_OK(MaybeAlignMetadata(&metadata_)); + int64_t body_length = -1; + RETURN_NOT_OK(CheckMetadataAndGetBodyLength(*metadata_, &body_length)); + + state_ = State::BODY; + next_required_size_ = body_length; + RETURN_NOT_OK(listener_->OnBody()); + if (next_required_size_ == 0) { + ARROW_ASSIGN_OR_RAISE(auto body, AllocateBuffer(0, pool_)); + std::shared_ptr<Buffer> shared_body(body.release()); + return ConsumeBody(&shared_body); + } else { + return Status::OK(); + } + } + + Status ConsumeBodyBuffer(std::shared_ptr<Buffer> buffer) { + return ConsumeBody(&buffer); + } + + Status ConsumeBodyChunks() { + if (chunks_[0]->size() >= next_required_size_) { + auto used_size = next_required_size_; + if (chunks_[0]->size() == next_required_size_) { + RETURN_NOT_OK(ConsumeBody(&chunks_[0])); + chunks_.erase(chunks_.begin()); + } else { + auto body = SliceBuffer(chunks_[0], 0, next_required_size_); + RETURN_NOT_OK(ConsumeBody(&body)); + chunks_[0] = SliceBuffer(chunks_[0], used_size); + } + buffered_size_ -= used_size; + return Status::OK(); + } else { + ARROW_ASSIGN_OR_RAISE(auto body, AllocateBuffer(next_required_size_, pool_)); + RETURN_NOT_OK(ConsumeDataChunks(next_required_size_, body->mutable_data())); + std::shared_ptr<Buffer> shared_body(body.release()); + return ConsumeBody(&shared_body); + } + } + + Status ConsumeBody(std::shared_ptr<Buffer>* buffer) { + ARROW_ASSIGN_OR_RAISE(std::unique_ptr<Message> message, + Message::Open(metadata_, *buffer)); + + RETURN_NOT_OK(listener_->OnMessageDecoded(std::move(message))); + state_ = State::INITIAL; + next_required_size_ = kMessageDecoderNextRequiredSizeInitial; + RETURN_NOT_OK(listener_->OnInitial()); + return Status::OK(); + } + + Result<int32_t> ConsumeDataBufferInt32(const std::shared_ptr<Buffer>& buffer) { + if (buffer->is_cpu()) { + return util::SafeLoadAs<int32_t>(buffer->data()); + } else { + ARROW_ASSIGN_OR_RAISE(auto cpu_buffer, + Buffer::ViewOrCopy(buffer, CPUDevice::memory_manager(pool_))); + return util::SafeLoadAs<int32_t>(cpu_buffer->data()); + } + } + + Status ConsumeDataChunks(int64_t nbytes, void* out) { + size_t offset = 0; + size_t n_used_chunks = 0; + auto required_size = nbytes; + std::shared_ptr<Buffer> last_chunk; + for (auto& chunk : chunks_) { + if (!chunk->is_cpu()) { + ARROW_ASSIGN_OR_RAISE( + chunk, Buffer::ViewOrCopy(chunk, CPUDevice::memory_manager(pool_))); + } + auto data = chunk->data(); + auto data_size = chunk->size(); + auto copy_size = std::min(required_size, data_size); + memcpy(static_cast<uint8_t*>(out) + offset, data, copy_size); + n_used_chunks++; + offset += copy_size; + required_size -= copy_size; + if (required_size == 0) { + if (data_size != copy_size) { + last_chunk = SliceBuffer(chunk, copy_size); + } + break; + } + } + chunks_.erase(chunks_.begin(), chunks_.begin() + n_used_chunks); + if (last_chunk.get() != nullptr) { + chunks_.insert(chunks_.begin(), std::move(last_chunk)); + } + buffered_size_ -= offset; + return Status::OK(); + } + + std::shared_ptr<MessageDecoderListener> listener_; + MemoryPool* pool_; + State state_; + int64_t next_required_size_, save_initial_size_; + std::vector<std::shared_ptr<Buffer>> chunks_; + int64_t buffered_size_; + std::shared_ptr<Buffer> metadata_; // Must be CPU buffer +}; + +MessageDecoder2::MessageDecoder2(std::shared_ptr<MessageDecoderListener> listener, + MemoryPool* pool) { + impl_.reset(new MessageDecoderImpl(std::move(listener), State::INITIAL, + kMessageDecoderNextRequiredSizeInitial, pool)); +} + +MessageDecoder2::MessageDecoder2(std::shared_ptr<MessageDecoderListener> listener, + State initial_state, int64_t initial_next_required_size, + MemoryPool* pool) { + impl_.reset(new MessageDecoderImpl(std::move(listener), initial_state, + initial_next_required_size, pool)); +} + +MessageDecoder2::~MessageDecoder2() {} + +Status MessageDecoder2::Consume(const uint8_t* data, int64_t size) { + return impl_->ConsumeData(data, size); +} + +void MessageDecoder2::Reset() { + impl_->Reset(); +} + +Status MessageDecoder2::Consume(std::shared_ptr<Buffer> buffer) { + return impl_->ConsumeBuffer(buffer); +} + +int64_t MessageDecoder2::next_required_size() const { return impl_->next_required_size(); } + +MessageDecoder2::State MessageDecoder2::state() const { return impl_->state(); } + +enum class DictionaryKind { New, Delta, Replacement }; + +Status InvalidMessageType(MessageType expected, MessageType actual) { + return Status::IOError("Expected IPC message of type ", ::arrow::ipc::FormatMessageType(expected), + " but got ", ::arrow::ipc::FormatMessageType(actual)); +} + +#define CHECK_MESSAGE_TYPE(expected, actual) \ + do { \ + if ((actual) != (expected)) { \ + return InvalidMessageType((expected), (actual)); \ + } \ + } while (0) + +#define CHECK_HAS_BODY(message) \ + do { \ + if ((message).body() == nullptr) { \ + return Status::IOError("Expected body in IPC message of type ", \ + ::arrow::ipc::FormatMessageType((message).type())); \ + } \ + } while (0) + +#define CHECK_HAS_NO_BODY(message) \ + do { \ + if ((message).body_length() != 0) { \ + return Status::IOError("Unexpected body in IPC message of type ", \ + ::arrow::ipc::FormatMessageType((message).type())); \ + } \ + } while (0) +struct IpcReadContext { + IpcReadContext(DictionaryMemo* memo, const IpcReadOptions& option, bool swap, + MetadataVersion version = MetadataVersion::V5, + Compression::type kind = Compression::UNCOMPRESSED) + : dictionary_memo(memo), + options(option), + metadata_version(version), + compression(kind), + swap_endian(swap) {} + + DictionaryMemo* dictionary_memo; + + const IpcReadOptions& options; + + MetadataVersion metadata_version; + + Compression::type compression; + + const bool swap_endian; +}; + + + +Result<std::shared_ptr<Buffer>> DecompressBuffer(const std::shared_ptr<Buffer>& buf, + const IpcReadOptions& options, + util::Codec* codec) { + if (buf == nullptr || buf->size() == 0) { + return buf; + } + + if (buf->size() < 8) { + return Status::Invalid( + "Likely corrupted message, compressed buffers " + "are larger than 8 bytes by construction"); + } + + const uint8_t* data = buf->data(); + int64_t compressed_size = buf->size() - sizeof(int64_t); + int64_t uncompressed_size = BitUtil::FromLittleEndian(util::SafeLoadAs<int64_t>(data)); + + ARROW_ASSIGN_OR_RAISE(auto uncompressed, + AllocateBuffer(uncompressed_size, options.memory_pool)); + + ARROW_ASSIGN_OR_RAISE( + int64_t actual_decompressed, + codec->Decompress(compressed_size, data + sizeof(int64_t), uncompressed_size, + uncompressed->mutable_data())); + if (actual_decompressed != uncompressed_size) { + return Status::Invalid("Failed to fully decompress buffer, expected ", + uncompressed_size, " bytes but decompressed ", + actual_decompressed); + } + + return std::move(uncompressed); +} + +Status DecompressBuffers(Compression::type compression, const IpcReadOptions& options, + ArrayDataVector* fields) { + struct BufferAccumulator { + using BufferPtrVector = std::vector<std::shared_ptr<Buffer>*>; + + void AppendFrom(const ArrayDataVector& fields) { + for (const auto& field : fields) { + for (auto& buffer : field->buffers) { + buffers_.push_back(&buffer); + } + AppendFrom(field->child_data); + } + } + + BufferPtrVector Get(const ArrayDataVector& fields) && { + AppendFrom(fields); + return std::move(buffers_); + } + + BufferPtrVector buffers_; + }; + + auto buffers = BufferAccumulator{}.Get(*fields); + + std::unique_ptr<util::Codec> codec; + ARROW_ASSIGN_OR_RAISE(codec, util::Codec::Create(compression)); + + return ::arrow::internal::OptionalParallelFor( + options.use_threads, static_cast<int>(buffers.size()), [&](int i) { + ARROW_ASSIGN_OR_RAISE(*buffers[i], + DecompressBuffer(*buffers[i], options, codec.get())); + return Status::OK(); + }); +} +class ArrayLoader { + public: + explicit ArrayLoader(const flatbuf::RecordBatch* metadata, + MetadataVersion metadata_version, const IpcReadOptions& options, + io::RandomAccessFile* file) + : metadata_(metadata), + metadata_version_(metadata_version), + file_(file), + max_recursion_depth_(options.max_recursion_depth) {} + + Status ReadBuffer(int64_t offset, int64_t length, std::shared_ptr<Buffer>* out) { + if (skip_io_) { + return Status::OK(); + } + if (offset < 0) { + return Status::Invalid("Negative offset for reading buffer ", buffer_index_); + } + if (length < 0) { + return Status::Invalid("Negative length for reading buffer ", buffer_index_); + } + if (!BitUtil::IsMultipleOf8(offset)) { + return Status::Invalid("Buffer ", buffer_index_, + " did not start on 8-byte aligned offset: ", offset); + } + return file_->ReadAt(offset, length).Value(out); + } + + Status LoadType(const DataType& type) { return VisitTypeInline(type, this); } + + Status Load(const Field* field, ArrayData* out) { + if (max_recursion_depth_ <= 0) { + return Status::Invalid("Max recursion depth reached"); + } + + field_ = field; + out_ = out; + out_->type = field_->type(); + return LoadType(*field_->type()); + } + + Status SkipField(const Field* field) { + ArrayData dummy; + skip_io_ = true; + Status status = Load(field, &dummy); + skip_io_ = false; + return status; + } + + Status GetBuffer(int buffer_index, std::shared_ptr<Buffer>* out) { + auto buffers = metadata_->buffers(); + CHECK_FLATBUFFERS_NOT_NULL(buffers, "RecordBatch.buffers"); + if (buffer_index >= static_cast<int>(buffers->size())) { + return Status::IOError("buffer_index out of range."); + } + const flatbuf::Buffer* buffer = buffers->Get(buffer_index); + if (buffer->length() == 0) { + // Should never return a null buffer here. + // (zero-sized buffer allocations are cheap) + return AllocateBuffer(0).Value(out); + } else { + return ReadBuffer(buffer->offset(), buffer->length(), out); + } + } + + Status GetFieldMetadata(int field_index, ArrayData* out) { + auto nodes = metadata_->nodes(); + CHECK_FLATBUFFERS_NOT_NULL(nodes, "Table.nodes"); + if (field_index >= static_cast<int>(nodes->size())) { + return Status::Invalid("Ran out of field metadata, likely malformed"); + } + const flatbuf::FieldNode* node = nodes->Get(field_index); + + out->length = node->length(); + out->null_count = node->null_count(); + out->offset = 0; + return Status::OK(); + } + + Status LoadCommon(Type::type type_id) { + RETURN_NOT_OK(GetFieldMetadata(field_index_++, out_)); + + if (internal::HasValidityBitmap(type_id, metadata_version_)) { + // Extract null_bitmap which is common to all arrays except for unions + // and nulls. + if (out_->null_count != 0) { + RETURN_NOT_OK(GetBuffer(buffer_index_, &out_->buffers[0])); + } + buffer_index_++; + } + return Status::OK(); + } + + template <typename TYPE> + Status LoadPrimitive(Type::type type_id) { + out_->buffers.resize(2); + + RETURN_NOT_OK(LoadCommon(type_id)); + if (out_->length > 0) { + RETURN_NOT_OK(GetBuffer(buffer_index_++, &out_->buffers[1])); + } else { + buffer_index_++; + out_->buffers[1].reset(new Buffer(nullptr, 0)); + } + return Status::OK(); + } + + template <typename TYPE> + Status LoadBinary(Type::type type_id) { + out_->buffers.resize(3); + + RETURN_NOT_OK(LoadCommon(type_id)); + RETURN_NOT_OK(GetBuffer(buffer_index_++, &out_->buffers[1])); + return GetBuffer(buffer_index_++, &out_->buffers[2]); + } + + template <typename TYPE> + Status LoadList(const TYPE& type) { + out_->buffers.resize(2); + + RETURN_NOT_OK(LoadCommon(type.id())); + RETURN_NOT_OK(GetBuffer(buffer_index_++, &out_->buffers[1])); + + const int num_children = type.num_fields(); + if (num_children != 1) { + return Status::Invalid("Wrong number of children: ", num_children); + } + + return LoadChildren(type.fields()); + } + + Status LoadChildren(const std::vector<std::shared_ptr<Field>>& child_fields) { + ArrayData* parent = out_; + + parent->child_data.resize(child_fields.size()); + for (int i = 0; i < static_cast<int>(child_fields.size()); ++i) { + parent->child_data[i] = std::make_shared<ArrayData>(); + --max_recursion_depth_; + RETURN_NOT_OK(Load(child_fields[i].get(), parent->child_data[i].get())); + ++max_recursion_depth_; + } + out_ = parent; + return Status::OK(); + } + + Status Visit(const NullType&) { + out_->buffers.resize(1); + + return GetFieldMetadata(field_index_++, out_); + } + + template <typename T> + enable_if_t<std::is_base_of<FixedWidthType, T>::value && + !std::is_base_of<FixedSizeBinaryType, T>::value && + !std::is_base_of<DictionaryType, T>::value, + Status> + Visit(const T& type) { + return LoadPrimitive<T>(type.id()); + } + + template <typename T> + enable_if_base_binary<T, Status> Visit(const T& type) { + return LoadBinary<T>(type.id()); + } + + Status Visit(const FixedSizeBinaryType& type) { + out_->buffers.resize(2); + RETURN_NOT_OK(LoadCommon(type.id())); + return GetBuffer(buffer_index_++, &out_->buffers[1]); + } + + template <typename T> + enable_if_var_size_list<T, Status> Visit(const T& type) { + return LoadList(type); + } + + Status Visit(const MapType& type) { + RETURN_NOT_OK(LoadList(type)); + return MapArray::ValidateChildData(out_->child_data); + } + + Status Visit(const FixedSizeListType& type) { + out_->buffers.resize(1); + + RETURN_NOT_OK(LoadCommon(type.id())); + + const int num_children = type.num_fields(); + if (num_children != 1) { + return Status::Invalid("Wrong number of children: ", num_children); + } + + return LoadChildren(type.fields()); + } + + Status Visit(const StructType& type) { + out_->buffers.resize(1); + RETURN_NOT_OK(LoadCommon(type.id())); + return LoadChildren(type.fields()); + } + + Status Visit(const UnionType& type) { + int n_buffers = type.mode() == UnionMode::SPARSE ? 2 : 3; + out_->buffers.resize(n_buffers); + + RETURN_NOT_OK(LoadCommon(type.id())); + + if (out_->null_count != 0 && out_->buffers[0] != nullptr) { + return Status::Invalid( + "Cannot read pre-1.0.0 Union array with top-level validity bitmap"); + } + out_->buffers[0] = nullptr; + out_->null_count = 0; + + if (out_->length > 0) { + RETURN_NOT_OK(GetBuffer(buffer_index_, &out_->buffers[1])); + if (type.mode() == UnionMode::DENSE) { + RETURN_NOT_OK(GetBuffer(buffer_index_ + 1, &out_->buffers[2])); + } + } + buffer_index_ += n_buffers - 1; + return LoadChildren(type.fields()); + } + + Status Visit(const DictionaryType& type) { + // out_->dictionary will be filled later in ResolveDictionaries() + return LoadType(*type.index_type()); + } + + Status Visit(const ExtensionType& type) { return LoadType(*type.storage_type()); } + + private: + const flatbuf::RecordBatch* metadata_; + const MetadataVersion metadata_version_; + io::RandomAccessFile* file_; + int max_recursion_depth_; + int buffer_index_ = 0; + int field_index_ = 0; + bool skip_io_ = false; + + const Field* field_; + ArrayData* out_; +}; + +Result<std::shared_ptr<RecordBatch>> LoadRecordBatchSubset( + const flatbuf::RecordBatch* metadata, const std::shared_ptr<Schema>& schema, + const std::vector<bool>* inclusion_mask, const IpcReadContext& context, + io::RandomAccessFile* file) { + ArrayLoader loader(metadata, context.metadata_version, context.options, file); + + ArrayDataVector columns(schema->num_fields()); + ArrayDataVector filtered_columns; + FieldVector filtered_fields; + std::shared_ptr<Schema> filtered_schema; + + for (int i = 0; i < schema->num_fields(); ++i) { + const Field& field = *schema->field(i); + if (!inclusion_mask || (*inclusion_mask)[i]) { + // Read field + auto column = std::make_shared<ArrayData>(); + RETURN_NOT_OK(loader.Load(&field, column.get())); + if (metadata->length() != column->length) { + return Status::IOError("Array length did not match record batch length"); + } + columns[i] = std::move(column); + if (inclusion_mask) { + filtered_columns.push_back(columns[i]); + filtered_fields.push_back(schema->field(i)); + } + } else { + RETURN_NOT_OK(loader.SkipField(&field)); + } + } + + RETURN_NOT_OK(ResolveDictionaries(columns, *context.dictionary_memo, + context.options.memory_pool)); + + if (inclusion_mask) { + filtered_schema = ::arrow::schema(std::move(filtered_fields), schema->metadata()); + columns.clear(); + } else { + filtered_schema = schema; + filtered_columns = std::move(columns); + } + if (context.compression != Compression::UNCOMPRESSED) { + RETURN_NOT_OK( + DecompressBuffers(context.compression, context.options, &filtered_columns)); + } + + if (context.swap_endian) { + for (int i = 0; i < static_cast<int>(filtered_columns.size()); ++i) { + ARROW_ASSIGN_OR_RAISE(filtered_columns[i], + arrow::internal::SwapEndianArrayData(filtered_columns[i])); + } + } + return RecordBatch::Make(std::move(filtered_schema), metadata->length(), + std::move(filtered_columns)); +} + +Result<std::shared_ptr<RecordBatch>> LoadRecordBatch( + const flatbuf::RecordBatch* metadata, const std::shared_ptr<Schema>& schema, + const std::vector<bool>& inclusion_mask, const IpcReadContext& context, + io::RandomAccessFile* file) { + if (inclusion_mask.size() > 0) { + return LoadRecordBatchSubset(metadata, schema, &inclusion_mask, context, file); + } else { + return LoadRecordBatchSubset(metadata, schema, /*param_name=*/nullptr, context, file); + } +} + +Status GetCompression(const flatbuf::RecordBatch* batch, Compression::type* out) { + *out = Compression::UNCOMPRESSED; + const flatbuf::BodyCompression* compression = batch->compression(); + if (compression != nullptr) { + if (compression->method() != flatbuf::BodyCompressionMethod::BUFFER) { + return Status::Invalid("This library only supports BUFFER compression method"); + } + + if (compression->codec() == flatbuf::CompressionType::LZ4_FRAME) { + *out = Compression::LZ4_FRAME; + } else if (compression->codec() == flatbuf::CompressionType::ZSTD) { + *out = Compression::ZSTD; + } else { + return Status::Invalid("Unsupported codec in RecordBatch::compression metadata"); + } + return Status::OK(); + } + return Status::OK(); +} + +Status GetCompressionExperimental(const flatbuf::Message* message, + Compression::type* out) { + *out = Compression::UNCOMPRESSED; + if (message->custom_metadata() != nullptr) { + std::shared_ptr<KeyValueMetadata> metadata; + RETURN_NOT_OK(internal::GetKeyValueMetadata(message->custom_metadata(), &metadata)); + int index = metadata->FindKey("ARROW:experimental_compression"); + if (index != -1) { + auto name = arrow::internal::AsciiToLower(metadata->value(index)); + ARROW_ASSIGN_OR_RAISE(*out, util::Codec::GetCompressionType(name)); + } + return internal::CheckCompressionSupported(*out); + } + return Status::OK(); +} + +static Status ReadContiguousPayload(io::InputStream* file, + std::unique_ptr<Message>* message) { + ARROW_ASSIGN_OR_RAISE(*message, ReadMessage(file)); + if (*message == nullptr) { + return Status::Invalid("Unable to read metadata at offset"); + } + return Status::OK(); +} + +Result<std::shared_ptr<RecordBatch>> ReadRecordBatch( + const std::shared_ptr<Schema>& schema, const DictionaryMemo* dictionary_memo, + const IpcReadOptions& options, io::InputStream* file) { + std::unique_ptr<Message> message; + RETURN_NOT_OK(ReadContiguousPayload(file, &message)); + CHECK_HAS_BODY(*message); + ARROW_ASSIGN_OR_RAISE(auto reader, Buffer::GetReader(message->body())); + return ReadRecordBatch(*message->metadata(), schema, dictionary_memo, options, + reader.get()); +} + +Result<std::shared_ptr<RecordBatch>> ReadRecordBatch( + const Message& message, const std::shared_ptr<Schema>& schema, + const DictionaryMemo* dictionary_memo, const IpcReadOptions& options) { + CHECK_MESSAGE_TYPE(MessageType::RECORD_BATCH, message.type()); + CHECK_HAS_BODY(message); + ARROW_ASSIGN_OR_RAISE(auto reader, Buffer::GetReader(message.body())); + return ReadRecordBatch(*message.metadata(), schema, dictionary_memo, options, + reader.get()); +} + +Result<std::shared_ptr<RecordBatch>> ReadRecordBatchInternal( + const Buffer& metadata, const std::shared_ptr<Schema>& schema, + const std::vector<bool>& inclusion_mask, IpcReadContext& context, + io::RandomAccessFile* file) { + const flatbuf::Message* message = nullptr; + RETURN_NOT_OK(internal::VerifyMessage(metadata.data(), metadata.size(), &message)); + auto batch = message->header_as_RecordBatch(); + if (batch == nullptr) { + return Status::IOError( + "Header-type of flatbuffer-encoded Message is not RecordBatch."); + } + + Compression::type compression; + RETURN_NOT_OK(GetCompression(batch, &compression)); + if (context.compression == Compression::UNCOMPRESSED && + message->version() == flatbuf::MetadataVersion::V4) { + RETURN_NOT_OK(GetCompressionExperimental(message, &compression)); + } + context.compression = compression; + context.metadata_version = internal::GetMetadataVersion(message->version()); + return LoadRecordBatch(batch, schema, inclusion_mask, context, file); +} + +Status GetInclusionMaskAndOutSchema(const std::shared_ptr<Schema>& full_schema, + const std::vector<int>& included_indices, + std::vector<bool>* inclusion_mask, + std::shared_ptr<Schema>* out_schema) { + inclusion_mask->clear(); + if (included_indices.empty()) { + *out_schema = full_schema; + return Status::OK(); + } + + inclusion_mask->resize(full_schema->num_fields(), false); + + auto included_indices_sorted = included_indices; + std::sort(included_indices_sorted.begin(), included_indices_sorted.end()); + + FieldVector included_fields; + for (int i : included_indices_sorted) { + if (i < 0 || i >= full_schema->num_fields()) { + return Status::Invalid("Out of bounds field index: ", i); + } + + if (inclusion_mask->at(i)) continue; + + inclusion_mask->at(i) = true; + included_fields.push_back(full_schema->field(i)); + } + + *out_schema = schema(std::move(included_fields), full_schema->endianness(), + full_schema->metadata()); + return Status::OK(); +} + +Status UnpackSchemaMessage(const void* opaque_schema, const IpcReadOptions& options, + DictionaryMemo* dictionary_memo, + std::shared_ptr<Schema>* schema, + std::shared_ptr<Schema>* out_schema, + std::vector<bool>* field_inclusion_mask, bool* swap_endian) { + RETURN_NOT_OK(internal::GetSchema(opaque_schema, dictionary_memo, schema)); + + RETURN_NOT_OK(GetInclusionMaskAndOutSchema(*schema, options.included_fields, + field_inclusion_mask, out_schema)); + *swap_endian = options.ensure_native_endian && !out_schema->get()->is_native_endian(); + if (*swap_endian) { + *schema = schema->get()->WithEndianness(Endianness::Native); + *out_schema = out_schema->get()->WithEndianness(Endianness::Native); + } + return Status::OK(); +} + +Status UnpackSchemaMessage(const Message& message, const IpcReadOptions& options, + DictionaryMemo* dictionary_memo, + std::shared_ptr<Schema>* schema, + std::shared_ptr<Schema>* out_schema, + std::vector<bool>* field_inclusion_mask, bool* swap_endian) { + CHECK_MESSAGE_TYPE(MessageType::SCHEMA, message.type()); + CHECK_HAS_NO_BODY(message); + + return UnpackSchemaMessage(message.header(), options, dictionary_memo, schema, + out_schema, field_inclusion_mask, swap_endian); +} + +Result<std::shared_ptr<RecordBatch>> ReadRecordBatch( + const Buffer& metadata, const std::shared_ptr<Schema>& schema, + const DictionaryMemo* dictionary_memo, const IpcReadOptions& options, + io::RandomAccessFile* file) { + std::shared_ptr<Schema> out_schema; + // Empty means do not use + std::vector<bool> inclusion_mask; + IpcReadContext context(const_cast<DictionaryMemo*>(dictionary_memo), options, false); + RETURN_NOT_OK(GetInclusionMaskAndOutSchema(schema, context.options.included_fields, + &inclusion_mask, &out_schema)); + return ReadRecordBatchInternal(metadata, schema, inclusion_mask, context, file); +} + +Status ReadDictionary(const Buffer& metadata, const IpcReadContext& context, + DictionaryKind* kind, io::RandomAccessFile* file) { + const flatbuf::Message* message = nullptr; + RETURN_NOT_OK(internal::VerifyMessage(metadata.data(), metadata.size(), &message)); + const auto dictionary_batch = message->header_as_DictionaryBatch(); + if (dictionary_batch == nullptr) { + return Status::IOError( + "Header-type of flatbuffer-encoded Message is not DictionaryBatch."); + } + + // The dictionary is embedded in a record batch with a single column + const auto batch_meta = dictionary_batch->data(); + + CHECK_FLATBUFFERS_NOT_NULL(batch_meta, "DictionaryBatch.data"); + + Compression::type compression; + RETURN_NOT_OK(GetCompression(batch_meta, &compression)); + if (compression == Compression::UNCOMPRESSED && + message->version() == flatbuf::MetadataVersion::V4) { + RETURN_NOT_OK(GetCompressionExperimental(message, &compression)); + } + + const int64_t id = dictionary_batch->id(); + + ARROW_ASSIGN_OR_RAISE(auto value_type, context.dictionary_memo->GetDictionaryType(id)); + + ArrayLoader loader(batch_meta, internal::GetMetadataVersion(message->version()), + context.options, file); + auto dict_data = std::make_shared<ArrayData>(); + const Field dummy_field("", value_type); + RETURN_NOT_OK(loader.Load(&dummy_field, dict_data.get())); + + if (compression != Compression::UNCOMPRESSED) { + ArrayDataVector dict_fields{dict_data}; + RETURN_NOT_OK(DecompressBuffers(compression, context.options, &dict_fields)); + } + + if (context.swap_endian) { + ARROW_ASSIGN_OR_RAISE(dict_data, ::arrow::internal::SwapEndianArrayData(dict_data)); + } + + if (dictionary_batch->isDelta()) { + if (kind != nullptr) { + *kind = DictionaryKind::Delta; + } + return context.dictionary_memo->AddDictionaryDelta(id, dict_data); + } + ARROW_ASSIGN_OR_RAISE(bool inserted, + context.dictionary_memo->AddOrReplaceDictionary(id, dict_data)); + if (kind != nullptr) { + *kind = inserted ? DictionaryKind::New : DictionaryKind::Replacement; + } + return Status::OK(); +} + +Status ReadDictionary(const Message& message, const IpcReadContext& context, + DictionaryKind* kind) { + DCHECK_EQ(message.type(), MessageType::DICTIONARY_BATCH); + CHECK_HAS_BODY(message); + ARROW_ASSIGN_OR_RAISE(auto reader, Buffer::GetReader(message.body())); + return ReadDictionary(*message.metadata(), context, kind, reader.get()); +} + + +class StreamDecoder2::StreamDecoder2Impl : public MessageDecoderListener { + private: + enum State { + SCHEMA, + INITIAL_DICTIONARIES, + RECORD_BATCHES, + EOS, + }; + + public: + explicit StreamDecoder2Impl(std::shared_ptr<Listener> listener, IpcReadOptions options) + : listener_(std::move(listener)), + options_(std::move(options)), + state_(State::SCHEMA), + message_decoder_(std::shared_ptr<StreamDecoder2Impl>(this, [](void*) {}), + options_.memory_pool), + n_required_dictionaries_(0), + dictionary_memo_(std::make_unique<DictionaryMemo>()) {} + + void Reset() { + state_ = State::SCHEMA; + field_inclusion_mask_.clear(); + n_required_dictionaries_ = 0; + dictionary_memo_ = std::make_unique<DictionaryMemo>(); + schema_ = out_schema_ = nullptr; + message_decoder_.Reset(); + } + + Status OnMessageDecoded(std::unique_ptr<Message> message) override { + ++stats_.num_messages; + switch (state_) { + case State::SCHEMA: + ARROW_RETURN_NOT_OK(OnSchemaMessageDecoded(std::move(message))); + break; + case State::INITIAL_DICTIONARIES: + ARROW_RETURN_NOT_OK(OnInitialDictionaryMessageDecoded(std::move(message))); + break; + case State::RECORD_BATCHES: + ARROW_RETURN_NOT_OK(OnRecordBatchMessageDecoded(std::move(message))); + break; + case State::EOS: + break; + } + return Status::OK(); + } + + Status OnEOS() override { + state_ = State::EOS; + return listener_->OnEOS(); + } + Status Consume(const uint8_t* data, int64_t size) { + return message_decoder_.Consume(data, size); + } + + Status Consume(std::shared_ptr<Buffer> buffer) { + return message_decoder_.Consume(std::move(buffer)); + } + + std::shared_ptr<Schema> schema() const { return out_schema_; } + + int64_t next_required_size() const { return message_decoder_.next_required_size(); } + + ReadStats stats() const { return stats_; } + + private: + Status OnSchemaMessageDecoded(std::unique_ptr<Message> message) { + RETURN_NOT_OK(UnpackSchemaMessage(*message, options_, dictionary_memo_.get(), &schema_, + &out_schema_, &field_inclusion_mask_, + &swap_endian_)); + + n_required_dictionaries_ = dictionary_memo_->fields().num_fields(); + if (n_required_dictionaries_ == 0) { + state_ = State::RECORD_BATCHES; + RETURN_NOT_OK(listener_->OnSchemaDecoded(schema_)); + } else { + state_ = State::INITIAL_DICTIONARIES; + } + return Status::OK(); + } + + Status OnInitialDictionaryMessageDecoded(std::unique_ptr<Message> message) { + if (message->type() != MessageType::DICTIONARY_BATCH) { + return Status::Invalid("IPC stream did not have the expected number (", + dictionary_memo_->fields().num_fields(), + ") of dictionaries at the start of the stream"); + } + RETURN_NOT_OK(ReadDictionary(*message)); + n_required_dictionaries_--; + if (n_required_dictionaries_ == 0) { + state_ = State::RECORD_BATCHES; + ARROW_RETURN_NOT_OK(listener_->OnSchemaDecoded(schema_)); + } + return Status::OK(); + } + + Status OnRecordBatchMessageDecoded(std::unique_ptr<Message> message) { + IpcReadContext context(dictionary_memo_.get(), options_, swap_endian_); + if (message->type() == MessageType::DICTIONARY_BATCH) { + return ReadDictionary(*message); + } else { + CHECK_HAS_BODY(*message); + ARROW_ASSIGN_OR_RAISE(auto reader, Buffer::GetReader(message->body())); + IpcReadContext context(dictionary_memo_.get(), options_, swap_endian_); + ARROW_ASSIGN_OR_RAISE( + auto batch, + ReadRecordBatchInternal(*message->metadata(), schema_, field_inclusion_mask_, + context, reader.get())); + ++stats_.num_record_batches; + return listener_->OnRecordBatchDecoded(std::move(batch)); + } + } + + Status ReadDictionary(const Message& message) { + DictionaryKind kind; + IpcReadContext context(dictionary_memo_.get(), options_, swap_endian_); + RETURN_NOT_OK(::arrow::ipc::NDqs::ReadDictionary(message, context, &kind)); + ++stats_.num_dictionary_batches; + switch (kind) { + case DictionaryKind::New: + break; + case DictionaryKind::Delta: + ++stats_.num_dictionary_deltas; + break; + case DictionaryKind::Replacement: + ++stats_.num_replaced_dictionaries; + break; + } + return Status::OK(); + } + + std::shared_ptr<Listener> listener_; + const IpcReadOptions options_; + State state_; + MessageDecoder2 message_decoder_; + std::vector<bool> field_inclusion_mask_; + int n_required_dictionaries_; + std::unique_ptr<DictionaryMemo> dictionary_memo_; + std::shared_ptr<Schema> schema_, out_schema_; + ReadStats stats_; + bool swap_endian_; +}; + +Result<std::shared_ptr<Schema>> ReadSchema(io::InputStream* stream, + DictionaryMemo* dictionary_memo) { + std::unique_ptr<MessageReader> reader = MessageReader::Open(stream); + ARROW_ASSIGN_OR_RAISE(std::unique_ptr<Message> message, reader->ReadNextMessage()); + if (!message) { + return Status::Invalid("Tried reading schema message, was null or length 0"); + } + CHECK_MESSAGE_TYPE(MessageType::SCHEMA, message->type()); + return ReadSchema(*message, dictionary_memo); +} + +Result<std::shared_ptr<Schema>> ReadSchema(const Message& message, + DictionaryMemo* dictionary_memo) { + std::shared_ptr<Schema> result; + RETURN_NOT_OK(internal::GetSchema(message.header(), dictionary_memo, &result)); + return result; +} + +Result<std::shared_ptr<Tensor>> ReadTensor(io::InputStream* file) { + std::unique_ptr<Message> message; + RETURN_NOT_OK(ReadContiguousPayload(file, &message)); + return ReadTensor(*message); +} + +Result<std::shared_ptr<Tensor>> ReadTensor(const Message& message) { + std::shared_ptr<DataType> type; + std::vector<int64_t> shape; + std::vector<int64_t> strides; + std::vector<std::string> dim_names; + CHECK_HAS_BODY(message); + RETURN_NOT_OK(internal::GetTensorMetadata(*message.metadata(), &type, &shape, &strides, + &dim_names)); + return Tensor::Make(type, message.body(), shape, strides, dim_names); +} + + +StreamDecoder2::StreamDecoder2(std::shared_ptr<Listener> listener, IpcReadOptions options) { + impl_.reset(new StreamDecoder2::StreamDecoder2Impl(std::move(listener), options)); +} + +StreamDecoder2::~StreamDecoder2() {} + +Status StreamDecoder2::Consume(const uint8_t* data, int64_t size) { + return impl_->Consume(data, size); +} + +void StreamDecoder2::Reset() { + impl_->Reset(); +} + +Status StreamDecoder2::Consume(std::shared_ptr<Buffer> buffer) { + return impl_->Consume(std::move(buffer)); +} + +std::shared_ptr<Schema> StreamDecoder2::schema() const { return impl_->schema(); } + +int64_t StreamDecoder2::next_required_size() const { return impl_->next_required_size(); } + +ReadStats StreamDecoder2::stats() const { return impl_->stats(); } + +class InputStreamMessageReader : public MessageReader, public MessageDecoderListener { + public: + explicit InputStreamMessageReader(io::InputStream* stream) + : stream_(stream), + owned_stream_(), + message_(), + decoder_(std::shared_ptr<InputStreamMessageReader>(this, [](void*) {})) {} + + explicit InputStreamMessageReader(const std::shared_ptr<io::InputStream>& owned_stream) + : InputStreamMessageReader(owned_stream.get()) { + owned_stream_ = owned_stream; + } + + ~InputStreamMessageReader() {} + + Status OnMessageDecoded(std::unique_ptr<Message> message) override { + message_ = std::move(message); + return Status::OK(); + } + + Result<std::unique_ptr<Message>> ReadNextMessage() override { + ARROW_RETURN_NOT_OK(DecodeMessage(&decoder_, stream_)); + return std::move(message_); + } + + private: + io::InputStream* stream_; + std::shared_ptr<io::InputStream> owned_stream_; + std::unique_ptr<Message> message_; + MessageDecoder decoder_; +}; + + +std::unique_ptr<MessageReader> MessageReader::Open(io::InputStream* stream) { + return std::unique_ptr<MessageReader>(new InputStreamMessageReader(stream)); +} + +std::unique_ptr<MessageReader> MessageReader::Open( + const std::shared_ptr<io::InputStream>& owned_stream) { + return std::unique_ptr<MessageReader>(new InputStreamMessageReader(owned_stream)); +} + +} + +} // namespace ipc::NDqs +} // namespace arrow diff --git a/yt/yql/providers/yt/comp_nodes/dq/stream_decoder.h b/yt/yql/providers/yt/comp_nodes/dq/stream_decoder.h new file mode 100644 index 0000000000..f5ba1be7fd --- /dev/null +++ b/yt/yql/providers/yt/comp_nodes/dq/stream_decoder.h @@ -0,0 +1,85 @@ +// almost copy of reader.cc + message.cc without comments +// TODO(): Remove when .Reset() will be added in contrib version +#pragma once + +#include <arrow/ipc/reader.h> +#include <arrow/ipc/message.h> +#include <cstddef> +#include <cstdint> +#include <memory> +#include <utility> +#include <vector> + +#include <arrow/io/caching.h> +#include <arrow/io/type_fwd.h> +#include <arrow/ipc/message.h> +#include <arrow/ipc/options.h> +#include <arrow/record_batch.h> +#include <arrow/result.h> +#include <arrow/type_fwd.h> +#include <arrow/util/async_generator.h> +#include <arrow/util/macros.h> +#include <arrow/util/visibility.h> + +namespace arrow { +namespace ipc::NDqs { + +class ARROW_EXPORT MessageDecoder2 { + public: + enum State { + INITIAL, + METADATA_LENGTH, + METADATA, + BODY, + EOS, + }; + + explicit MessageDecoder2(std::shared_ptr<MessageDecoderListener> listener, + MemoryPool* pool = default_memory_pool()); + + MessageDecoder2(std::shared_ptr<MessageDecoderListener> listener, State initial_state, + int64_t initial_next_required_size, + MemoryPool* pool = default_memory_pool()); + + virtual ~MessageDecoder2(); + Status Consume(const uint8_t* data, int64_t size); + Status Consume(std::shared_ptr<Buffer> buffer); + int64_t next_required_size() const; + State state() const; + void Reset(); + + private: + class MessageDecoderImpl; + std::unique_ptr<MessageDecoderImpl> impl_; + + ARROW_DISALLOW_COPY_AND_ASSIGN(MessageDecoder2); +}; + +class ARROW_EXPORT MessageReader { + public: + virtual ~MessageReader() = default; + static std::unique_ptr<MessageReader> Open(io::InputStream* stream); + static std::unique_ptr<MessageReader> Open( + const std::shared_ptr<io::InputStream>& owned_stream); + virtual Result<std::unique_ptr<Message>> ReadNextMessage() = 0; +}; + +class ARROW_EXPORT StreamDecoder2 { + public: + StreamDecoder2(std::shared_ptr<Listener> listener, + IpcReadOptions options = IpcReadOptions::Defaults()); + + virtual ~StreamDecoder2(); + Status Consume(const uint8_t* data, int64_t size); + Status Consume(std::shared_ptr<Buffer> buffer); + std::shared_ptr<Schema> schema() const; + int64_t next_required_size() const; + ReadStats stats() const; + void Reset(); + + private: + class StreamDecoder2Impl; + std::unique_ptr<StreamDecoder2Impl> impl_; +}; +} // namespace ipc::NDqs +} // namespace arrow diff --git a/yt/yql/providers/yt/comp_nodes/dq/ya.make b/yt/yql/providers/yt/comp_nodes/dq/ya.make new file mode 100644 index 0000000000..22b5d83e96 --- /dev/null +++ b/yt/yql/providers/yt/comp_nodes/dq/ya.make @@ -0,0 +1,51 @@ +LIBRARY() + +PEERDIR( + yql/essentials/minikql + yql/essentials/minikql/computation/llvm14 + yt/yql/providers/yt/comp_nodes + yt/yql/providers/yt/codec + yql/essentials/providers/common/codec + yql/essentials/utils/failure_injector + yt/cpp/mapreduce/interface + yt/cpp/mapreduce/common + library/cpp/yson/node + yt/yt/core + yql/essentials/public/udf/arrow + yql/essentials/public/udf + contrib/libs/apache/arrow + contrib/libs/flatbuffers +) + +ADDINCL( + contrib/libs/flatbuffers/include +) + +IF(LINUX) + PEERDIR( + yt/yt/client + yt/yt/client/arrow + yt/yql/providers/yt/lib/yt_rpc_helpers + ) + + SRCS( + stream_decoder.cpp + dq_yt_rpc_reader.cpp + dq_yt_rpc_helpers.cpp + dq_yt_block_reader.cpp + ) + CFLAGS( + -Wno-unused-parameter + ) +ENDIF() + +SRCS( + dq_yt_reader.cpp + dq_yt_factory.cpp + dq_yt_writer.cpp +) + +YQL_LAST_ABI_VERSION() + + +END() diff --git a/yt/yql/providers/yt/comp_nodes/llvm14/ya.make b/yt/yql/providers/yt/comp_nodes/llvm14/ya.make new file mode 100644 index 0000000000..47cfd1f49a --- /dev/null +++ b/yt/yql/providers/yt/comp_nodes/llvm14/ya.make @@ -0,0 +1,18 @@ +LIBRARY() + +NO_COMPILER_WARNINGS() + +PEERDIR( + yql/essentials/minikql/codegen/llvm14 + yql/essentials/minikql/invoke_builtins/llvm14 + contrib/libs/llvm14/lib/IR + contrib/libs/llvm14/lib/ExecutionEngine/MCJIT + contrib/libs/llvm14/lib/Linker + contrib/libs/llvm14/lib/Target/X86 + contrib/libs/llvm14/lib/Target/X86/AsmParser + contrib/libs/llvm14/lib/Transforms/IPO +) + +INCLUDE(../ya.make.inc) + +END() diff --git a/yt/yql/providers/yt/comp_nodes/no_llvm/ya.make b/yt/yql/providers/yt/comp_nodes/no_llvm/ya.make new file mode 100644 index 0000000000..fcc6ca106f --- /dev/null +++ b/yt/yql/providers/yt/comp_nodes/no_llvm/ya.make @@ -0,0 +1,11 @@ +LIBRARY() + +CXXFLAGS(-DMKQL_DISABLE_CODEGEN) + +ADDINCL(GLOBAL yql/essentials/minikql/codegen/llvm_stub) + +INCLUDE(../ya.make.inc) + +PEERDIR(yql/essentials/minikql/invoke_builtins/no_llvm) + +END() diff --git a/yt/yql/providers/yt/comp_nodes/ut/ya.make b/yt/yql/providers/yt/comp_nodes/ut/ya.make new file mode 100644 index 0000000000..9ec924b301 --- /dev/null +++ b/yt/yql/providers/yt/comp_nodes/ut/ya.make @@ -0,0 +1,29 @@ +UNITTEST_FOR(yt/yql/providers/yt/comp_nodes) + +FORK_SUBTESTS() + +IF (SANITIZER_TYPE == "thread" OR WITH_VALGRIND) + SIZE(LARGE) + TAG(ya:fat) +ELSE() + SIZE(MEDIUM) +ENDIF() + +SRCS( + yql_mkql_output_ut.cpp +) + +PEERDIR( + library/cpp/random_provider + library/cpp/time_provider + yql/essentials/minikql/comp_nodes/llvm14 + yql/essentials/public/udf/service/exception_policy + yt/yql/providers/yt/comp_nodes/llvm14 + yt/yql/providers/yt/codec + yt/yql/providers/yt/codec/codegen + yql/essentials/sql/pg_dummy +) + +YQL_LAST_ABI_VERSION() + +END() diff --git a/yt/yql/providers/yt/comp_nodes/ut/yql_mkql_output_ut.cpp b/yt/yql/providers/yt/comp_nodes/ut/yql_mkql_output_ut.cpp new file mode 100644 index 0000000000..9a2db9e918 --- /dev/null +++ b/yt/yql/providers/yt/comp_nodes/ut/yql_mkql_output_ut.cpp @@ -0,0 +1,319 @@ +#include <yql/essentials/minikql/comp_nodes/ut/mkql_computation_node_ut.h> +#include <yql/essentials/minikql/comp_nodes/mkql_factories.h> +#include <yql/essentials/minikql/computation/mkql_computation_node_impl.h> +#include <yql/essentials/minikql/computation/mkql_computation_node_holders.h> +#include <yql/essentials/minikql/computation/mkql_computation_node.h> +#include <yql/essentials/minikql/invoke_builtins/mkql_builtins.h> +#include <yql/essentials/minikql/mkql_mem_info.h> +#include <yql/essentials/minikql/mkql_node_cast.h> +#include <yql/essentials/minikql/mkql_node.h> +#include <yql/essentials/minikql/mkql_function_registry.h> +#include <yql/essentials/minikql/mkql_program_builder.h> +#include <yql/essentials/minikql/mkql_string_util.h> + +#include <library/cpp/testing/unittest/registar.h> +#include <library/cpp/random_provider/random_provider.h> +#include <library/cpp/time_provider/time_provider.h> + +#include <yt/yql/providers/yt/comp_nodes/yql_mkql_output.h> + +#include <util/generic/array_size.h> +#include <util/generic/array_ref.h> + +#include <cstring> + +namespace NKikimr { +namespace NMiniKQL { + +namespace { + +TIntrusivePtr<IRandomProvider> CreateRandomProvider() { + return CreateDeterministicRandomProvider(1); +} + +TIntrusivePtr<ITimeProvider> CreateTimeProvider() { + return CreateDeterministicTimeProvider(10000000); +} + +TComputationNodeFactory GetTestFactory(NYql::TMkqlWriterImpl& writer) { + return [&](TCallable& callable, const TComputationNodeFactoryContext& ctx) -> IComputationNode* { + if (callable.GetType()->GetName() == "YtOutput") { + return NYql::WrapYtOutput(callable, ctx, writer); + } + return GetBuiltinFactory()(callable, ctx); + }; +} + +template<bool UseLLVM> +struct TSetup_ { + TSetup_() + : FunctionRegistry(CreateFunctionRegistry(CreateBuiltinRegistry())) + , RandomProvider(CreateRandomProvider()) + , TimeProvider(CreateTimeProvider()) + , Alloc(__LOCATION__) + , Env(MakeHolder<TTypeEnvironment>(Alloc)) + , PgmBuilder(MakeHolder<TProgramBuilder>(*Env, *FunctionRegistry)) + {} + + TAutoPtr<IComputationGraph> BuildGraph(TRuntimeNode pgm, NYql::TMkqlWriterImpl& writer) { + Explorer.Walk(pgm.GetNode(), *Env); + TComputationPatternOpts opts(Alloc.Ref(), *Env, GetTestFactory(writer), FunctionRegistry.Get(), + NUdf::EValidateMode::None, NUdf::EValidatePolicy::Exception, UseLLVM ? "" : "OFF", EGraphPerProcess::Multi); + Pattern = MakeComputationPattern(Explorer, pgm, {}, opts); + return Pattern->Clone(opts.ToComputationOptions(*RandomProvider, *TimeProvider)); + } + + TIntrusivePtr<IFunctionRegistry> FunctionRegistry; + TIntrusivePtr<IRandomProvider> RandomProvider; + TIntrusivePtr<ITimeProvider> TimeProvider; + + TScopedAlloc Alloc; + THolder<TTypeEnvironment> Env; + THolder<TProgramBuilder> PgmBuilder; + + TExploringNodeVisitor Explorer; + IComputationPattern::TPtr Pattern; +}; + +template<bool LLVM> +TRuntimeNode MakeYtWrite(TSetup_<LLVM>& setup, TRuntimeNode item) { + TProgramBuilder& pb = *setup.PgmBuilder; + TCallableBuilder callableBuilder(*setup.Env, "YtOutput", pb.NewFlowType(pb.NewVoid().GetStaticType())); + callableBuilder.Add(item); + return TRuntimeNode(callableBuilder.Build(), false); +} + +} // unnamed + +Y_UNIT_TEST_SUITE(YtWriterTests) { + Y_UNIT_TEST_LLVM(SimpleYson) { + TSetup_<LLVM> setup; + TProgramBuilder& pb = *setup.PgmBuilder; + + const auto key1 = pb.NewDataLiteral<ui32>(1); + const auto key2 = pb.NewDataLiteral<ui32>(2); + const auto key3 = pb.NewDataLiteral<ui32>(3); + + const auto payload1 = pb.NewDataLiteral<NUdf::EDataSlot::String>("aaa"); + const auto payload2 = pb.NewDataLiteral<NUdf::EDataSlot::String>(""); + const auto payload3 = pb.NewDataLiteral<NUdf::EDataSlot::String>("qqq"); + + const auto keyType = pb.NewDataType(NUdf::TDataType<ui32>::Id); + const auto payloadType = pb.NewDataType(NUdf::TDataType<char*>::Id); + auto structType = pb.NewEmptyStructType(); + structType = pb.NewStructType(structType, "payload", payloadType); + structType = pb.NewStructType(structType, "key", keyType); + + std::vector<std::pair<std::string_view, TRuntimeNode>> map1 = { + { "key", key1 }, + { "payload", payload1 } + }; + + std::vector<std::pair<std::string_view, TRuntimeNode>> map2 = { + { "key", key2 }, + { "payload", payload2 } + }; + + std::vector<std::pair<std::string_view, TRuntimeNode>> map3 = { + { "key", key3 }, + { "payload", payload3 } + }; + + const auto list = pb.NewList(structType, { + pb.NewStruct(map2), + pb.NewStruct(map1), + pb.NewStruct(map3) + }); + + const auto pgmReturn = pb.Discard(pb.Map(pb.ToFlow(list), + [&](TRuntimeNode item) { + return MakeYtWrite(setup, item); + } + )); + + const TString spec = R"({ + tables = [{ + "_yql_row_spec" = { + "Type" = [ + "StructType"; [ + ["key"; ["DataType"; "Uint32"]]; + ["payload"; ["DataType"; "String"]] + ] + ] + } + }] + })"; + + NYql::NCommon::TCodecContext CodecCtx(*setup.Env, *setup.FunctionRegistry); + NYql::TMkqlIOSpecs specs; + specs.Init(CodecCtx, spec); + + TStringStream strm; + NYql::TMkqlWriterImpl writer(strm, 0ULL, 1ULL << 20ULL); + writer.SetSpecs(specs); + + const auto graph = setup.BuildGraph(pgmReturn, writer); + UNIT_ASSERT(graph->GetValue().IsFinish()); + writer.Finish(); + strm.Finish(); + + const auto& output = strm.Str(); + const std::string_view expected("{\1\6key=\6\2;\1\x0Epayload=\1\0;};{\1\6key=\6\1;\1\x0Epayload=\1\6aaa;};{\1\6key=\6\3;\1\x0Epayload=\1\6qqq;};", 81); + UNIT_ASSERT_STRINGS_EQUAL(output, expected); + } + + Y_UNIT_TEST_LLVM(SimpleSkiff) { + TSetup_<LLVM> setup; + TProgramBuilder& pb = *setup.PgmBuilder; + + const auto key1 = pb.NewDataLiteral<ui32>(1); + const auto key2 = pb.NewDataLiteral<ui32>(2); + const auto key3 = pb.NewDataLiteral<ui32>(3); + + const auto payload1 = pb.NewDataLiteral<NUdf::EDataSlot::String>("aaa"); + const auto payload2 = pb.NewDataLiteral<NUdf::EDataSlot::String>(""); + const auto payload3 = pb.NewDataLiteral<NUdf::EDataSlot::String>("qqq"); + + const auto keyType = pb.NewDataType(NUdf::TDataType<ui32>::Id); + const auto payloadType = pb.NewDataType(NUdf::TDataType<char*>::Id); + auto structType = pb.NewEmptyStructType(); + structType = pb.NewStructType(structType, "payload", payloadType); + structType = pb.NewStructType(structType, "key", keyType); + + std::vector<std::pair<std::string_view, TRuntimeNode>> map1 = { + { "key", key1 }, + { "payload", payload1 } + }; + + std::vector<std::pair<std::string_view, TRuntimeNode>> map2 = { + { "key", key2 }, + { "payload", payload2 } + }; + + std::vector<std::pair<std::string_view, TRuntimeNode>> map3 = { + { "key", key3 }, + { "payload", payload3 } + }; + + const auto list = pb.NewList(structType, { + pb.NewStruct(map2), + pb.NewStruct(map1), + pb.NewStruct(map3) + }); + + const auto pgmReturn = pb.Discard(MakeYtWrite(setup, pb.ToFlow(list))); + + const TString spec = R"({ + tables = [{ + "_yql_row_spec" = { + "Type" = [ + "StructType"; [ + ["key"; ["DataType"; "Uint32"]]; + ["payload"; ["DataType"; "String"]] + ] + ] + } + }] + })"; + + NYql::NCommon::TCodecContext CodecCtx(*setup.Env, *setup.FunctionRegistry); + NYql::TMkqlIOSpecs specs; + specs.SetUseSkiff(LLVM ? "": "OFF"); + specs.Init(CodecCtx, spec); + + TStringStream strm; + NYql::TMkqlWriterImpl writer(strm, 0ULL, 1ULL << 20ULL); + writer.SetSpecs(specs); + + const auto graph = setup.BuildGraph(pgmReturn, writer); + UNIT_ASSERT(graph->GetValue().IsFinish()); + writer.Finish(); + strm.Finish(); + + const auto& output = strm.Str(); + const std::string_view expected("\0\0\2\0\0\0\0\0\0\0\0\0\0\0\0\0\1\0\0\0\0\0\0\0\3\0\0\0aaa\0\0\3\0\0\0\0\0\0\0\3\0\0\0qqq", 48); + UNIT_ASSERT_STRINGS_EQUAL(output, expected); + } + + Y_UNIT_TEST_LLVM(FlattenSkiff) { + TSetup_<LLVM> setup; + TProgramBuilder& pb = *setup.PgmBuilder; + + const auto key1 = pb.NewDataLiteral<ui32>(1); + const auto key2 = pb.NewDataLiteral<ui32>(2); + const auto key3 = pb.NewDataLiteral<ui32>(3); + + const auto payload1 = pb.NewDataLiteral<NUdf::EDataSlot::String>("aaa"); + const auto payload2 = pb.NewDataLiteral<NUdf::EDataSlot::String>(""); + const auto payload3 = pb.NewDataLiteral<NUdf::EDataSlot::String>("qqq"); + + const auto keyType = pb.NewDataType(NUdf::TDataType<ui32>::Id); + const auto payloadType = pb.NewDataType(NUdf::TDataType<char*>::Id); + auto structType = pb.NewEmptyStructType(); + structType = pb.NewStructType(structType, "payload", payloadType); + structType = pb.NewStructType(structType, "key", keyType); + + std::vector<std::pair<std::string_view, TRuntimeNode>> map1 = { + { "key", key1 }, + { "payload", payload1 } + }; + + std::vector<std::pair<std::string_view, TRuntimeNode>> map2 = { + { "key", key2 }, + { "payload", payload2 } + }; + + std::vector<std::pair<std::string_view, TRuntimeNode>> map3 = { + { "key", key3 }, + { "payload", payload3 } + }; + + const auto list = pb.NewList(structType, { + pb.NewStruct(map2), + pb.NewStruct(map1), + pb.NewStruct(map3) + }); + + const auto flow = pb.ExpandMap(pb.ToFlow(list), + [&](TRuntimeNode item) -> TRuntimeNode::TList { + return { pb.Member(item, "key"), pb.Member(item, "payload")}; + } + ); + + const auto pgmReturn = pb.Discard(MakeYtWrite(setup, flow)); + + const TString spec = R"({ + tables = [{ + "_yql_row_spec" = { + "Type" = [ + "StructType"; [ + ["key"; ["DataType"; "Uint32"]]; + ["payload"; ["DataType"; "String"]] + ] + ] + } + }] + })"; + + NYql::NCommon::TCodecContext CodecCtx(*setup.Env, *setup.FunctionRegistry); + NYql::TMkqlIOSpecs specs; + specs.SetUseSkiff(LLVM ? "": "OFF"); + specs.Init(CodecCtx, spec); + + TStringStream strm; + NYql::TMkqlWriterImpl writer(strm, 0ULL, 1ULL << 20ULL); + writer.SetSpecs(specs); + + const auto graph = setup.BuildGraph(pgmReturn, writer); + UNIT_ASSERT(graph->GetValue().IsFinish()); + writer.Finish(); + strm.Finish(); + + const auto& output = strm.Str(); + const std::string_view expected("\0\0\2\0\0\0\0\0\0\0\0\0\0\0\0\0\1\0\0\0\0\0\0\0\3\0\0\0aaa\0\0\3\0\0\0\0\0\0\0\3\0\0\0qqq", 48); + UNIT_ASSERT_STRINGS_EQUAL(output, expected); + } +} + +} // NMiniKQL +} // NKikimr diff --git a/yt/yql/providers/yt/comp_nodes/ya.make b/yt/yql/providers/yt/comp_nodes/ya.make new file mode 100644 index 0000000000..2f1044fd1d --- /dev/null +++ b/yt/yql/providers/yt/comp_nodes/ya.make @@ -0,0 +1,12 @@ +LIBRARY() + +END() + +RECURSE( + llvm14 + no_llvm +) + +RECURSE_FOR_TESTS( + ut +) diff --git a/yt/yql/providers/yt/comp_nodes/yql_mkql_file_input_state.cpp b/yt/yql/providers/yt/comp_nodes/yql_mkql_file_input_state.cpp new file mode 100644 index 0000000000..023da36b58 --- /dev/null +++ b/yt/yql/providers/yt/comp_nodes/yql_mkql_file_input_state.cpp @@ -0,0 +1,74 @@ +#include "yql_mkql_file_input_state.h" + +#include <yql/essentials/utils/yql_panic.h> + +#include <util/system/fs.h> + +namespace NYql { + +TFileInputState::TFileInputState(const TMkqlIOSpecs& spec, + const NKikimr::NMiniKQL::THolderFactory& holderFactory, + TVector<NYT::TRawTableReaderPtr>&& rawReaders, + size_t blockCount, + size_t blockSize) + : Spec_(&spec) + , HolderFactory_(holderFactory) + , RawReaders_(std::move(rawReaders)) + , BlockCount_(blockCount) + , BlockSize_(blockSize) +{ + YQL_ENSURE(Spec_->Inputs.size() == RawReaders_.size()); + MkqlReader_.SetSpecs(*Spec_, HolderFactory_); + Valid_ = NextValue(); +} + +bool TFileInputState::NextValue() { + for (;;) { + if (CurrentInput_ >= RawReaders_.size()) { + return false; + } + + if (!MkqlReader_.IsValid()) { + if (!RawReaders_[CurrentInput_]) { + ++CurrentInput_; + continue; + } + + CurrentReader_ = std::move(RawReaders_[CurrentInput_]); + if (CurrentInput_ > 0 && OnNextBlockCallback_) { + OnNextBlockCallback_(); + } + MkqlReader_.SetReader(*CurrentReader_, BlockCount_, BlockSize_, ui32(CurrentInput_), true); + MkqlReader_.Next(); + CurrentRecord_ = 0; + + if (!MkqlReader_.IsValid()) { + ++CurrentInput_; + continue; + } + } + + CurrentValue_ = MkqlReader_.GetRow(); + if (!Spec_->InputGroups.empty()) { + CurrentValue_ = HolderFactory_.CreateVariantHolder(CurrentValue_.Release(), Spec_->InputGroups.at(CurrentInput_)); + } + + MkqlReader_.Next(); + ++CurrentRecord_; + return true; + } +} + +TVector<NYT::TRawTableReaderPtr> MakeMkqlFileInputs(const TVector<TString>& files, bool decompress) { + TVector<NYT::TRawTableReaderPtr> rawReaders; + for (auto& file: files) { + if (!NFs::Exists(file)) { + rawReaders.emplace_back(nullptr); + continue; + } + rawReaders.emplace_back(MakeIntrusive<TMkqlInput>(MakeFileInput(file, decompress))); + } + return rawReaders; +} + +} // NYql diff --git a/yt/yql/providers/yt/comp_nodes/yql_mkql_file_input_state.h b/yt/yql/providers/yt/comp_nodes/yql_mkql_file_input_state.h new file mode 100644 index 0000000000..d0f0541b95 --- /dev/null +++ b/yt/yql/providers/yt/comp_nodes/yql_mkql_file_input_state.h @@ -0,0 +1,74 @@ +#pragma once + +#include "yql_mkql_input_stream.h" + +#include <yql/essentials/providers/common/codec/yql_codec.h> +#include <yt/yql/providers/yt/codec/yt_codec_io.h> +#include <yql/essentials/public/udf/udf_value.h> +#include <yql/essentials/minikql/mkql_node.h> + +#include <util/generic/string.h> +#include <util/generic/vector.h> +#include <util/generic/ptr.h> + + +namespace NYql { + +class TFileInputState: public IInputState { +public: + TFileInputState(const TMkqlIOSpecs& spec, + const NKikimr::NMiniKQL::THolderFactory& holderFactory, + TVector<NYT::TRawTableReaderPtr>&& rawReaders, + size_t blockCount, + size_t blockSize); + + size_t GetTableIndex() const { + return CurrentInput_; + } + + size_t GetRecordIndex() const { + return CurrentRecord_; // returns 1-based index + } + + void SetNextBlockCallback(std::function<void()> cb) { + MkqlReader_.SetNextBlockCallback(cb); + OnNextBlockCallback_ = std::move(cb); + } + +protected: + virtual bool IsValid() const override { + return Valid_; + } + + virtual NUdf::TUnboxedValue GetCurrent() override { + return CurrentValue_; + } + + virtual void Next() override { + Valid_ = NextValue(); + } + + void Finish() { + MkqlReader_.Finish(); + } + + bool NextValue(); + +private: + const TMkqlIOSpecs* Spec_; + const NKikimr::NMiniKQL::THolderFactory& HolderFactory_; + TVector<NYT::TRawTableReaderPtr> RawReaders_; + const size_t BlockCount_; + const size_t BlockSize_; + size_t CurrentInput_ = 0; + size_t CurrentRecord_ = 0; + bool Valid_ = true; + NUdf::TUnboxedValue CurrentValue_; + NYT::TRawTableReaderPtr CurrentReader_; + TMkqlReaderImpl MkqlReader_; // Should be deleted before CurrentReader_ + std::function<void()> OnNextBlockCallback_; +}; + +TVector<NYT::TRawTableReaderPtr> MakeMkqlFileInputs(const TVector<TString>& files, bool decompress); + +} // NYql diff --git a/yt/yql/providers/yt/comp_nodes/yql_mkql_file_list.cpp b/yt/yql/providers/yt/comp_nodes/yql_mkql_file_list.cpp new file mode 100644 index 0000000000..5c65e85c8d --- /dev/null +++ b/yt/yql/providers/yt/comp_nodes/yql_mkql_file_list.cpp @@ -0,0 +1,41 @@ +#include "yql_mkql_file_list.h" +#include "yql_mkql_file_input_state.h" + +namespace NYql { + +using namespace NKikimr::NMiniKQL; + +TFileListValueBase::TIterator::TIterator(TMemoryUsageInfo* memInfo, THolder<IInputState>&& state, std::optional<ui64> length) + : TComputationValue(memInfo) + , State_(std::move(state)) + , ExpectedLength_(std::move(length)) +{ +} + +bool TFileListValueBase::TIterator::Next(NUdf::TUnboxedValue& value) { + if (!AtStart_) { + State_->Next(); + } + AtStart_ = false; + if (!State_->IsValid()) { + MKQL_ENSURE(!ExpectedLength_ || *ExpectedLength_ == 0, "Invalid file length"); + return false; + } + + if (ExpectedLength_) { + MKQL_ENSURE(*ExpectedLength_ > 0, "Invalid file length"); + --(*ExpectedLength_); + } + value = State_->GetCurrent(); + return true; +} + +NUdf::TUnboxedValue TFileListValueBase::GetListIterator() const { + return NUdf::TUnboxedValuePod(new TIterator(GetMemInfo(), MakeState(), Length)); +} + +THolder<IInputState> TFileListValue::MakeState() const { + return MakeHolder<TFileInputState>(Spec, HolderFactory, MakeMkqlFileInputs(FilePaths, Decompress), BlockCount, BlockSize); +} + +} diff --git a/yt/yql/providers/yt/comp_nodes/yql_mkql_file_list.h b/yt/yql/providers/yt/comp_nodes/yql_mkql_file_list.h new file mode 100644 index 0000000000..912a083efe --- /dev/null +++ b/yt/yql/providers/yt/comp_nodes/yql_mkql_file_list.h @@ -0,0 +1,78 @@ +#pragma once + +#include "yql_mkql_input_stream.h" + +#include <yt/yql/providers/yt/codec/yt_codec.h> +#include <yql/essentials/minikql/computation/mkql_computation_node.h> +#include <yql/essentials/minikql/computation/mkql_custom_list.h> + +#include <util/generic/ptr.h> +#include <util/generic/vector.h> +#include <util/generic/string.h> + +#include <optional> + +namespace NYql { + +class TFileListValueBase : public NKikimr::NMiniKQL::TCustomListValue { +public: + TFileListValueBase(NKikimr::NMiniKQL::TMemoryUsageInfo* memInfo, const TMkqlIOSpecs& spec, + const NKikimr::NMiniKQL::THolderFactory& holderFactory, std::optional<ui64> length) + : TCustomListValue(memInfo) + , Spec(spec) + , HolderFactory(holderFactory) + { + Length = length; + } + +protected: + class TIterator : public NKikimr::NMiniKQL::TComputationValue<TIterator> { + public: + TIterator(NKikimr::NMiniKQL::TMemoryUsageInfo* memInfo, THolder<IInputState>&& state, std::optional<ui64> length); + + private: + bool Next(NUdf::TUnboxedValue& value) override; + + bool AtStart_ = true; + THolder<IInputState> State_; + std::optional<ui64> ExpectedLength_; + }; + + NUdf::TUnboxedValue GetListIterator() const override; + + virtual THolder<IInputState> MakeState() const = 0; + +protected: + const TMkqlIOSpecs& Spec; + const NKikimr::NMiniKQL::THolderFactory& HolderFactory; +}; + +class TFileListValue : public TFileListValueBase { +public: + TFileListValue(NKikimr::NMiniKQL::TMemoryUsageInfo* memInfo, + const TMkqlIOSpecs& spec, + const NKikimr::NMiniKQL::THolderFactory& holderFactory, + const TVector<TString>& filePaths, + bool decompress, + size_t blockCount, + size_t blockSize, + std::optional<ui64> length) + : TFileListValueBase(memInfo, spec, holderFactory, length) + , FilePaths(filePaths) + , Decompress(decompress) + , BlockCount(blockCount) + , BlockSize(blockSize) + { + } + +protected: + THolder<IInputState> MakeState() const override; + +private: + const TVector<TString> FilePaths; + const bool Decompress; + const size_t BlockCount; + const size_t BlockSize; +}; + +} diff --git a/yt/yql/providers/yt/comp_nodes/yql_mkql_input.cpp b/yt/yql/providers/yt/comp_nodes/yql_mkql_input.cpp new file mode 100644 index 0000000000..743d3c546e --- /dev/null +++ b/yt/yql/providers/yt/comp_nodes/yql_mkql_input.cpp @@ -0,0 +1,395 @@ +#include "yql_mkql_input.h" +#include "yql_mkql_table.h" + +#include <yt/yql/providers/yt/codec/yt_codec_io.h> +#include <yql/essentials/minikql/computation/mkql_computation_node_codegen.h> // Y_IGNORE +#include <yql/essentials/minikql/mkql_node_cast.h> +#include <yql/essentials/public/udf/udf_value.h> +#include <yql/essentials/public/udf/udf_value_builder.h> +#include <yql/essentials/utils/yql_panic.h> + +#include <algorithm> +#include <functional> +#include <array> + +namespace NYql { + +using namespace NKikimr::NMiniKQL; + +class TInputStateBase: public TComputationValue<TInputStateBase> { +public: + TInputStateBase(TMemoryUsageInfo* memInfo, + const TMkqlIOSpecs& specs, NYT::IReaderImplBase* input, TComputationContext& ctx, + const std::array<IComputationExternalNode*, 5>& argNodes) + : TComputationValue<TInputStateBase>(memInfo) + , SpecsCache_(specs, ctx.HolderFactory) + , TableState_(specs.TableNames, specs.TableOffsets, ctx, argNodes) + , Input_(input) + , IsValid_(input->IsValid()) + { + } + virtual ~TInputStateBase() = default; + + NUdf::TUnboxedValuePod FetchRecord() { + if (AtStart_) { + UpdateTableState(true); + } else { + ReadNext(); + } + AtStart_ = false; + + if (!IsValid_) { + return NUdf::TUnboxedValuePod(); + } + + const size_t tableIndex = Input_->GetTableIndex(); + NUdf::TUnboxedValue result = GetCurrent(tableIndex); + + auto& specs = SpecsCache_.GetSpecs(); + if (!specs.InputGroups.empty()) { + result = SpecsCache_.GetHolderFactory().CreateVariantHolder(result.Release(), specs.InputGroups.at(tableIndex)); + } + return result.Release().MakeOptional(); + } + +protected: + virtual NUdf::TUnboxedValue GetCurrent(size_t tableIndex) = 0; + + void ReadNext() { + if (Y_LIKELY(IsValid_)) { + Input_->Next(); + IsValid_ = Input_->IsValid(); + bool keySwitch = false; + if (!IsValid_) { + Input_->NextKey(); + if (Input_->IsValid()) { + Input_->Next(); + keySwitch = true; + IsValid_ = Input_->IsValid(); + } + } + UpdateTableState(keySwitch); + } + } + + void UpdateTableState(bool keySwitch) { + if (IsValid_) { + TableState_.Update(Input_->GetTableIndex(), Input_->GetRowIndex() + 1, keySwitch); + } else { + TableState_.Reset(); + } + } + +protected: + TMkqlIOCache SpecsCache_; + TTableState TableState_; + NYT::IReaderImplBase* Input_; + bool IsValid_; + bool AtStart_ = true; +}; + +class TYamrInputState: public TInputStateBase { +public: + TYamrInputState(TMemoryUsageInfo* memInfo, const TMkqlIOSpecs& specs, NYT::IYaMRReaderImpl* input, TComputationContext& ctx, + const std::array<IComputationExternalNode*, 5>& argNodes) + : TInputStateBase(memInfo, specs, input, ctx, argNodes) + { + } + +protected: + NUdf::TUnboxedValue GetCurrent(size_t tableIndex) final { + return NYql::DecodeYamr(SpecsCache_, tableIndex, static_cast<NYT::IYaMRReaderImpl*>(Input_)->GetRow()); + } +}; + +class TYtInputState: public TInputStateBase { +public: + TYtInputState(TMemoryUsageInfo* memInfo, const TMkqlIOSpecs& specs, TMkqlReaderImpl* input, TComputationContext& ctx, + const std::array<IComputationExternalNode*, 5>& argNodes) + : TInputStateBase(memInfo, specs, input, ctx, argNodes) + { + } + +protected: + NUdf::TUnboxedValue GetCurrent(size_t tableIndex) final { + Y_UNUSED(tableIndex); + return static_cast<TMkqlReaderImpl*>(Input_)->GetRow(); + } +}; + +//////////////////////////////////////////////////////////////////////////////////////////////////////////////////////// + +class TYtInputWrapper : public TMutableComputationNode<TYtInputWrapper> { + typedef TMutableComputationNode<TYtInputWrapper> TBaseComputation; +public: + TYtInputWrapper(TComputationMutables& mutables, const TMkqlIOSpecs& specs, NYT::IReaderImplBase* input, + std::array<IComputationExternalNode*, 5>&& argNodes, TComputationNodePtrVector&& dependentNodes) + : TBaseComputation(mutables) + , Spec_(specs) + , Input_(input) + , ArgNodes_(std::move(argNodes)) + , DependentNodes_(std::move(dependentNodes)) + , StateIndex_(mutables.CurValueIndex++) + {} + + NUdf::TUnboxedValuePod DoCalculate(TComputationContext& ctx) const { + auto& state = ctx.MutableValues[StateIndex_]; + if (state.IsInvalid()) { + MakeState(ctx, state); + } + + return static_cast<TInputStateBase&>(*state.AsBoxed()).FetchRecord(); + } +private: + void MakeState(TComputationContext& ctx, NUdf::TUnboxedValue& state) const { + if (const auto mkqlReader = dynamic_cast<TMkqlReaderImpl*>(Input_)) { + state = ctx.HolderFactory.Create<TYtInputState>(Spec_, mkqlReader, ctx, ArgNodes_); + } else if (const auto yamrReader = dynamic_cast<NYT::IYaMRReaderImpl*>(Input_)) { + state = ctx.HolderFactory.Create<TYamrInputState>(Spec_, yamrReader, ctx, ArgNodes_); + } + } + + void RegisterDependencies() const final { + std::for_each(ArgNodes_.cbegin(), ArgNodes_.cend(), std::bind(&TYtInputWrapper::Own, this, std::placeholders::_1)); + std::for_each(DependentNodes_.cbegin(), DependentNodes_.cend(), std::bind(&TYtInputWrapper::DependsOn, this, std::placeholders::_1)); + } + + const TMkqlIOSpecs& Spec_; + NYT::IReaderImplBase*const Input_; + const std::array<IComputationExternalNode*, 5> ArgNodes_; + const TComputationNodePtrVector DependentNodes_; + const ui32 StateIndex_; +}; + +class TYtBaseInputWrapper { +protected: + TYtBaseInputWrapper(const TMkqlIOSpecs& specs, NYT::IReaderImplBase* input) + : Spec_(specs), Input_(input) + {} + + void MakeState(TComputationContext& ctx, NUdf::TUnboxedValue& state) const { + std::array<IComputationExternalNode*, 5U> stub; + stub.fill(nullptr); + if (const auto mkqlReader = dynamic_cast<TMkqlReaderImpl*>(Input_)) { + state = ctx.HolderFactory.Create<TYtInputState>(Spec_, mkqlReader, ctx, stub); + } else if (const auto yamrReader = dynamic_cast<NYT::IYaMRReaderImpl*>(Input_)) { + state = ctx.HolderFactory.Create<TYamrInputState>(Spec_, yamrReader, ctx, stub); + } + } +private: + const TMkqlIOSpecs& Spec_; + NYT::IReaderImplBase*const Input_; +}; + +class TYtFlowInputWrapper : public TStatefulFlowCodegeneratorNode<TYtFlowInputWrapper>, private TYtBaseInputWrapper { +using TBaseComputation = TStatefulFlowCodegeneratorNode<TYtFlowInputWrapper>; +public: + TYtFlowInputWrapper(TComputationMutables& mutables, EValueRepresentation kind, const TMkqlIOSpecs& specs, NYT::IReaderImplBase* input) + : TBaseComputation(mutables, this, kind, EValueRepresentation::Boxed), TYtBaseInputWrapper(specs, input) + {} + + NUdf::TUnboxedValuePod DoCalculate(NUdf::TUnboxedValue& state, TComputationContext& ctx) const { + if (state.IsInvalid()) { + MakeState(ctx, state); + } + + if (const auto value = static_cast<TInputStateBase&>(*state.AsBoxed()).FetchRecord()) + return value; + else + return NUdf::TUnboxedValuePod::MakeFinish(); + } +#ifndef MKQL_DISABLE_CODEGEN + Value* DoGenerateGetValue(const TCodegenContext& ctx, Value* statePtr, BasicBlock*& block) const { + auto& context = ctx.Codegen.GetContext(); + + const auto valueType = Type::getInt128Ty(context); + const auto structPtrType = PointerType::getUnqual(StructType::get(context)); + + const auto stateType = StructType::get(context, { + structPtrType, // vtbl + Type::getInt32Ty(context), // ref + Type::getInt16Ty(context), // abi + Type::getInt16Ty(context), // reserved + structPtrType // meminfo + }); + + const auto statePtrType = PointerType::getUnqual(stateType); + + const auto make = BasicBlock::Create(context, "make", ctx.Func); + const auto main = BasicBlock::Create(context, "main", ctx.Func); + + BranchInst::Create(make, main, IsInvalid(statePtr, block), block); + block = make; + + const auto self = CastInst::Create(Instruction::IntToPtr, ConstantInt::get(Type::getInt64Ty(context), uintptr_t(static_cast<const TYtBaseInputWrapper*>(this))), structPtrType, "self", block); + const auto makeFunc = ConstantInt::get(Type::getInt64Ty(context), GetMethodPtr(&TYtFlowInputWrapper::MakeState)); + const auto makeType = FunctionType::get(Type::getVoidTy(context), {self->getType(), ctx.Ctx->getType(), statePtr->getType()}, false); + const auto makeFuncPtr = CastInst::Create(Instruction::IntToPtr, makeFunc, PointerType::getUnqual(makeType), "function", block); + CallInst::Create(makeType, makeFuncPtr, {self, ctx.Ctx, statePtr}, "", block); + BranchInst::Create(main, block); + + block = main; + + const auto state = new LoadInst(valueType, statePtr, "state", block); + const auto half = CastInst::Create(Instruction::Trunc, state, Type::getInt64Ty(context), "half", block); + const auto stateArg = CastInst::Create(Instruction::IntToPtr, half, statePtrType, "state_arg", block); + + const auto func = ConstantInt::get(Type::getInt64Ty(context), GetMethodPtr(&TInputStateBase::FetchRecord)); + const auto funcType = FunctionType::get(valueType, { statePtrType }, false); + const auto funcPtr = CastInst::Create(Instruction::IntToPtr, func, PointerType::getUnqual(funcType), "fetch_func", block); + const auto fetch = CallInst::Create(funcType, funcPtr, { stateArg }, "fetch", block); + const auto result = SelectInst::Create(IsExists(fetch, block), fetch, GetFinish(context), "result", block); + + return result; + } +#endif +private: + void RegisterDependencies() const final {} +}; + +class TYtWideInputWrapper : public TPairStateWideFlowCodegeneratorNode<TYtWideInputWrapper>, private TYtBaseInputWrapper { +using TBaseComputation = TPairStateWideFlowCodegeneratorNode<TYtWideInputWrapper>; +public: + TYtWideInputWrapper(TComputationMutables& mutables, ui32 width, const TMkqlIOSpecs& specs, NYT::IReaderImplBase* input) + : TBaseComputation(mutables, this, EValueRepresentation::Boxed, EValueRepresentation::Embedded) + , TYtBaseInputWrapper(specs, input) + , Width(width) + { + } + + EFetchResult DoCalculate(NUdf::TUnboxedValue& state, NUdf::TUnboxedValue& current, TComputationContext& ctx, NUdf::TUnboxedValue*const* output) const { + if (state.IsInvalid()) { + MakeState(ctx, state); + } + + if (const auto value = static_cast<TInputStateBase&>(*state.AsBoxed()).FetchRecord()) { + const auto elements = value.GetElements(); + current = NUdf::TUnboxedValuePod(reinterpret_cast<ui64>(elements)); + for (ui32 i = 0U; i < Width; ++i) + if (const auto out = *output++) + *out = elements[i]; + + return EFetchResult::One; + } + + return EFetchResult::Finish; + } +#ifndef MKQL_DISABLE_CODEGEN + ICodegeneratorInlineWideNode::TGenerateResult DoGenGetValues(const TCodegenContext& ctx, Value* statePtr, Value* currentPtr, BasicBlock*& block) const { + auto& context = ctx.Codegen.GetContext(); + + const auto valueType = Type::getInt128Ty(context); + const auto arrayType = ArrayType::get(valueType, Width); + const auto pointerType = PointerType::getUnqual(arrayType); + const auto structPtrType = PointerType::getUnqual(StructType::get(context)); + const auto statusType = Type::getInt32Ty(context); + + const auto stateType = StructType::get(context, { + structPtrType, // vtbl + Type::getInt32Ty(context), // ref + Type::getInt16Ty(context), // abi + Type::getInt16Ty(context), // reserved + structPtrType // meminfo + }); + + const auto statePtrType = PointerType::getUnqual(stateType); + + const auto placeholder = GetElementPtrInst::CreateInBounds(valueType, ctx.GetMutables(), {ConstantInt::get(statusType, static_cast<const IComputationNode*>(this)->GetIndex() + 1U)}, "placeholder", &ctx.Func->getEntryBlock().back()); + + const auto make = BasicBlock::Create(context, "make", ctx.Func); + const auto main = BasicBlock::Create(context, "main", ctx.Func); + const auto good = BasicBlock::Create(context, "good", ctx.Func); + const auto done = BasicBlock::Create(context, "done", ctx.Func); + + BranchInst::Create(make, main, IsInvalid(statePtr, block), block); + block = make; + + const auto self = CastInst::Create(Instruction::IntToPtr, ConstantInt::get(Type::getInt64Ty(context), uintptr_t(static_cast<const TYtBaseInputWrapper*>(this))), structPtrType, "self", block); + const auto makeFunc = ConstantInt::get(Type::getInt64Ty(context), GetMethodPtr(&TYtWideInputWrapper::MakeState)); + const auto makeType = FunctionType::get(Type::getVoidTy(context), {self->getType(), ctx.Ctx->getType(), statePtr->getType()}, false); + const auto makeFuncPtr = CastInst::Create(Instruction::IntToPtr, makeFunc, PointerType::getUnqual(makeType), "function", block); + CallInst::Create(makeType, makeFuncPtr, {self, ctx.Ctx, statePtr}, "", block); + BranchInst::Create(main, block); + + block = main; + + const auto state = new LoadInst(valueType, statePtr, "state", block); + const auto half = CastInst::Create(Instruction::Trunc, state, Type::getInt64Ty(context), "half", block); + const auto stateArg = CastInst::Create(Instruction::IntToPtr, half, statePtrType, "state_arg", block); + + const auto func = ConstantInt::get(Type::getInt64Ty(context), GetMethodPtr(&TInputStateBase::FetchRecord)); + const auto funcType = FunctionType::get(valueType, { statePtrType }, false); + const auto funcPtr = CastInst::Create(Instruction::IntToPtr, func, PointerType::getUnqual(funcType), "fetch_func", block); + const auto fetch = CallInst::Create(funcType, funcPtr, { stateArg }, "fetch", block); + + const auto result = PHINode::Create(statusType, 2U, "result", done); + + result->addIncoming(ConstantInt::get(statusType, static_cast<i32>(EFetchResult::Finish)), block); + + BranchInst::Create(good, done, IsExists(fetch, block), block); + + block = good; + + const auto elements = CallBoxedValueVirtualMethod<NUdf::TBoxedValueAccessor::EMethod::GetElements>(pointerType, fetch, ctx.Codegen, block); + const auto integer = CastInst::Create(Instruction::PtrToInt, elements, Type::getInt64Ty(context), "integer", block); + const auto stored = SetterFor<ui64>(integer, context, block); + new StoreInst(stored, currentPtr, block); + + result->addIncoming(ConstantInt::get(statusType, static_cast<i32>(EFetchResult::One)), block); + + BranchInst::Create(done, block); + + block = done; + + TGettersList getters; + getters.reserve(Width); + for (ui32 i = 0U; i < Width; ++i) { + getters.emplace_back([i, placeholder, pointerType, valueType, arrayType](const TCodegenContext& ctx, BasicBlock*& block) { + auto& context = ctx.Codegen.GetContext(); + const auto current = new LoadInst(valueType, placeholder, (TString("current_") += ToString(i)).c_str(), block); + const auto integer = GetterFor<ui64>(current, context, block); + const auto pointer = CastInst::Create(Instruction::IntToPtr, integer, pointerType, (TString("pointer_") += ToString(i)).c_str(), block); + const auto indexType = Type::getInt32Ty(context); + const auto ptr = GetElementPtrInst::CreateInBounds(arrayType, pointer, {ConstantInt::get(indexType, 0), ConstantInt::get(indexType, i)}, (TString("ptr_") += ToString(i)).c_str(), block); + const auto load = new LoadInst(valueType, ptr, (TString("load_") += ToString(i)).c_str(), block); + return load; + }); + } + + return {result, std::move(getters)}; + } +#endif +private: + void RegisterDependencies() const final {} + const ui32 Width; +}; + +IComputationNode* WrapYtInput(TCallable& callable, const TComputationNodeFactoryContext& ctx, const TMkqlIOSpecs& specs, NYT::IReaderImplBase* input) +{ + if (!callable.GetInputsCount()) { + if (const auto type = AS_TYPE(TFlowType, callable.GetType()->GetReturnType())->GetItemType(); type->IsTuple()) + return new TYtWideInputWrapper(ctx.Mutables, AS_TYPE(TTupleType, type)->GetElementsCount(), specs, input); + else if (type->IsMulti()) + return new TYtWideInputWrapper(ctx.Mutables, AS_TYPE(TMultiType, type)->GetElementsCount(), specs, input); + else if (type->IsStruct()) + return new TYtFlowInputWrapper(ctx.Mutables, GetValueRepresentation(type), specs, input); + + THROW yexception() << "Expected tuple or struct as flow item type."; + } + + YQL_ENSURE(callable.GetInputsCount() >= 5, "Expected at least 5 args"); + + std::array<IComputationExternalNode*, 5> argNodes; + for (size_t i = 0; i < argNodes.size(); ++i) { + argNodes[i] = LocateExternalNode(ctx.NodeLocator, callable, i, false); + } + + TComputationNodePtrVector dependentNodes(callable.GetInputsCount() - argNodes.size()); + for (ui32 i = argNodes.size(); i < callable.GetInputsCount(); ++i) { + dependentNodes[i - argNodes.size()] = LocateNode(ctx.NodeLocator, callable, i); + } + + return new TYtInputWrapper(ctx.Mutables, specs, input, std::move(argNodes), std::move(dependentNodes)); +} + +} // NYql diff --git a/yt/yql/providers/yt/comp_nodes/yql_mkql_input.h b/yt/yql/providers/yt/comp_nodes/yql_mkql_input.h new file mode 100644 index 0000000000..f9196eaf9e --- /dev/null +++ b/yt/yql/providers/yt/comp_nodes/yql_mkql_input.h @@ -0,0 +1,15 @@ +#pragma once + +#include <yt/yql/providers/yt/codec/yt_codec.h> + +#include <yql/essentials/minikql/computation/mkql_computation_node.h> +#include <yql/essentials/minikql/mkql_node.h> + +#include <yt/cpp/mapreduce/interface/io.h> + +namespace NYql { + +NKikimr::NMiniKQL::IComputationNode* WrapYtInput(NKikimr::NMiniKQL::TCallable& callable, const NKikimr::NMiniKQL::TComputationNodeFactoryContext& ctx, + const TMkqlIOSpecs& specs, NYT::IReaderImplBase* input); + +} // NYql diff --git a/yt/yql/providers/yt/comp_nodes/yql_mkql_input_stream.cpp b/yt/yql/providers/yt/comp_nodes/yql_mkql_input_stream.cpp new file mode 100644 index 0000000000..c0e1676ec9 --- /dev/null +++ b/yt/yql/providers/yt/comp_nodes/yql_mkql_input_stream.cpp @@ -0,0 +1,37 @@ +#include "yql_mkql_input_stream.h" + +#include <yql/essentials/utils/yql_panic.h> + +namespace NYql { + +using namespace NKikimr::NMiniKQL; + +//////////////////////////////////////////////////////////////////////////////////////////////// + +TInputStreamValue::TInputStreamValue(TMemoryUsageInfo* memInfo, IInputState* state) + : TComputationValue<TInputStreamValue>(memInfo) + , State_(state) +{ +} + +NUdf::IBoxedValuePtr TInputStreamValue::ToIndexDictImpl(const NUdf::IValueBuilder& builder) const { + Y_UNUSED(builder); + YQL_ENSURE(false, "Single-pass iterator cannot be used for index dict"); + return {}; +} + +NUdf::EFetchStatus TInputStreamValue::Fetch(NUdf::TUnboxedValue& result) { + if (!AtStart_) { + State_->Next(); + } + AtStart_ = false; + + if (!State_->IsValid()) { + return NUdf::EFetchStatus::Finish; + } + + result = State_->GetCurrent(); + return NUdf::EFetchStatus::Ok; +} + +} // NYql diff --git a/yt/yql/providers/yt/comp_nodes/yql_mkql_input_stream.h b/yt/yql/providers/yt/comp_nodes/yql_mkql_input_stream.h new file mode 100644 index 0000000000..ebd66b43f1 --- /dev/null +++ b/yt/yql/providers/yt/comp_nodes/yql_mkql_input_stream.h @@ -0,0 +1,42 @@ +#pragma once + +#include <yql/essentials/minikql/computation/mkql_computation_node_impl.h> +#include <yql/essentials/minikql/mkql_node.h> +#include <yql/essentials/public/udf/udf_value.h> + +#include <util/generic/ptr.h> + +namespace NYql { + +struct IInputState { + virtual ~IInputState() = default; + + virtual bool IsValid() const = 0; + virtual NUdf::TUnboxedValue GetCurrent() = 0; + virtual void Next() = 0; +}; + +class TInputStreamValue + : public NKikimr::NMiniKQL::TComputationValue<TInputStreamValue> +{ +public: + TInputStreamValue(NKikimr::NMiniKQL::TMemoryUsageInfo* memInfo, IInputState* state); + +private: + virtual NUdf::IBoxedValuePtr ToIndexDictImpl(const NUdf::IValueBuilder& builder) const override; + virtual NUdf::EFetchStatus Fetch(NUdf::TUnboxedValue& result) override; + + bool AtStart_ = true; + IInputState* State_; +}; + +class THoldingInputStreamValue : private THolder<IInputState>, public TInputStreamValue { +public: + inline THoldingInputStreamValue(NKikimr::NMiniKQL::TMemoryUsageInfo* memInfo, IInputState* state) + : THolder<IInputState>(state) + , TInputStreamValue(memInfo, this->Get()) + { + } +}; + +} // NYql diff --git a/yt/yql/providers/yt/comp_nodes/yql_mkql_output.cpp b/yt/yql/providers/yt/comp_nodes/yql_mkql_output.cpp new file mode 100644 index 0000000000..819a8f9fd7 --- /dev/null +++ b/yt/yql/providers/yt/comp_nodes/yql_mkql_output.cpp @@ -0,0 +1,226 @@ +#include "yql_mkql_output.h" + +#include <yql/essentials/minikql/computation/mkql_computation_node_codegen.h> // Y_IGNORE +#include <yql/essentials/minikql/mkql_node_cast.h> + +namespace NYql { + +namespace { + +using namespace NKikimr::NMiniKQL; + +class TYtOutputWrapper : public TDecoratorCodegeneratorNode<TYtOutputWrapper> { + using TBaseComputation = TDecoratorCodegeneratorNode<TYtOutputWrapper>; +public: + TYtOutputWrapper(IComputationNode* item, TMkqlWriterImpl& writer) + : TBaseComputation(item, EValueRepresentation::Embedded), Writer(writer) + {} + + NUdf::TUnboxedValuePod DoCalculate(TComputationContext&, const NUdf::TUnboxedValuePod& value) const { + AddRowImpl(value); + return NUdf::TUnboxedValuePod::Void(); + } + +#ifndef MKQL_DISABLE_CODEGEN + Value* DoGenerateGetValue(const TCodegenContext& ctx, Value* item, BasicBlock*& block) const { + auto& context = ctx.Codegen.GetContext(); + if (true /*|| TODO: !Writer.GenAddRow(item, ctx, block)*/) { + const auto addFunc = ConstantInt::get(Type::getInt64Ty(context), GetMethodPtr(&TYtOutputWrapper::AddRowImpl)); + const auto selfArg = ConstantInt::get(Type::getInt64Ty(context), ui64(this)); + const auto arg = WrapArgumentForWindows(item, ctx, block); + const auto addType = FunctionType::get(Type::getVoidTy(context), {selfArg->getType(), arg->getType()}, false); + const auto addPtr = CastInst::Create(Instruction::IntToPtr, addFunc, PointerType::getUnqual(addType), "write", block); + CallInst::Create(addType, addPtr, {selfArg, arg}, "", block); + } + if (Node->IsTemporaryValue()) + ValueCleanup(Node->GetRepresentation(), item, ctx, block); + return GetFalse(context); + } +#endif +private: + void AddRowImpl(NUdf::TUnboxedValuePod row) const { + Writer.AddRow(row); + } + + TMkqlWriterImpl& Writer; +}; + +class TYtFlowOutputWrapper : public TStatelessFlowCodegeneratorNode<TYtFlowOutputWrapper> { +using TBaseComputation = TStatelessFlowCodegeneratorNode<TYtFlowOutputWrapper>; +public: + TYtFlowOutputWrapper(IComputationNode* flow, TMkqlWriterImpl& writer) + : TBaseComputation(flow, EValueRepresentation::Embedded), Flow(flow), Writer(writer) + {} + + NUdf::TUnboxedValuePod DoCalculate(TComputationContext& ctx) const { + if (const auto value = Flow->GetValue(ctx); value.IsSpecial()) + return value; + else + AddRowImpl(value); + return NUdf::TUnboxedValuePod::Void(); + } +#ifndef MKQL_DISABLE_CODEGEN + Value* DoGenerateGetValue(const TCodegenContext& ctx, BasicBlock*& block) const { + auto& context = ctx.Codegen.GetContext(); + + const auto item = GetNodeValue(Flow, ctx, block); + + const auto work = BasicBlock::Create(context, "work", ctx.Func); + const auto pass = BasicBlock::Create(context, "pass", ctx.Func); + + const auto result = PHINode::Create(item->getType(), 2U, "result", pass); + result->addIncoming(item, block); + + BranchInst::Create(pass, work, IsSpecial(item, block), block); + + block = work; + + const auto addFunc = ConstantInt::get(Type::getInt64Ty(context), GetMethodPtr(&TYtFlowOutputWrapper::AddRowImpl)); + const auto selfArg = ConstantInt::get(Type::getInt64Ty(context), ui64(this)); + const auto arg = WrapArgumentForWindows(item, ctx, block); + const auto addType = FunctionType::get(Type::getVoidTy(context), {selfArg->getType(), arg->getType()}, false); + const auto addPtr = CastInst::Create(Instruction::IntToPtr, addFunc, PointerType::getUnqual(addType), "write", block); + CallInst::Create(addType, addPtr, {selfArg, arg}, "", block); + + ValueCleanup(Flow->GetRepresentation(), item, ctx, block); + + result->addIncoming(ConstantInt::get(item->getType(), 0), block); + + BranchInst::Create(pass, block); + + block = pass; + return result; + } +#endif +private: + void RegisterDependencies() const final { + FlowDependsOn(Flow); + } + + void AddRowImpl(NUdf::TUnboxedValuePod row) const { + Writer.AddRow(row); + } + + static std::vector<NUdf::TUnboxedValue*> GetPointers(std::vector<NUdf::TUnboxedValue>& array) { + std::vector<NUdf::TUnboxedValue*> pointers; + pointers.reserve(array.size()); + std::transform(array.begin(), array.end(), std::back_inserter(pointers), [](NUdf::TUnboxedValue& v) { return std::addressof(v); }); + return pointers; + } + + IComputationNode *const Flow; + const std::vector<EValueRepresentation> Representations; + + TMkqlWriterImpl& Writer; + + std::vector<NUdf::TUnboxedValue> Values; + const std::vector<NUdf::TUnboxedValue*> Fields; +}; + +class TYtWideOutputWrapper : public TStatelessWideFlowCodegeneratorNode<TYtWideOutputWrapper> { +using TBaseComputation = TStatelessWideFlowCodegeneratorNode<TYtWideOutputWrapper>; +public: + TYtWideOutputWrapper(IComputationWideFlowNode* flow, TMkqlWriterImpl& writer, std::vector<EValueRepresentation>&& representations) + : TBaseComputation(flow), Flow(flow), Representations(std::move(representations)), Writer(writer), Values(Representations.size()), Fields(GetPointers(Values)) + {} + + EFetchResult DoCalculate(TComputationContext& ctx, NUdf::TUnboxedValue*const*) const { + if (const auto result = Flow->FetchValues(ctx, Fields.data()); EFetchResult::One != result) + return result; + + AddRowImpl(static_cast<const NUdf::TUnboxedValuePod*>(Values.data())); + + return EFetchResult::One; + } +#ifndef MKQL_DISABLE_CODEGEN + TGenerateResult DoGenGetValues(const TCodegenContext& ctx, BasicBlock*& block) const { + auto& context = ctx.Codegen.GetContext(); + + const auto valueType = Type::getInt128Ty(context); + const auto indexType = Type::getInt32Ty(context); + const auto arrayType = ArrayType::get(valueType, Representations.size()); + + const auto values = new AllocaInst(arrayType, 0U, "values", &ctx.Func->getEntryBlock().back()); + + const auto result = GetNodeValues(Flow, ctx, block); + + const auto good = CmpInst::Create(Instruction::ICmp, ICmpInst::ICMP_SGT, result.first, ConstantInt::get(result.first->getType(), 0), "good", block); + + const auto work = BasicBlock::Create(context, "work", ctx.Func); + const auto pass = BasicBlock::Create(context, "pass", ctx.Func); + + BranchInst::Create(work, pass, good, block); + + block = work; + + TSmallVec<Value*> fields; + fields.reserve(Representations.size()); + for (ui32 i = 0U; i < Representations.size(); ++i) { + const auto pointer = GetElementPtrInst::CreateInBounds(arrayType, values, {ConstantInt::get(indexType, 0), ConstantInt::get(indexType, i)}, (TString("ptr_") += ToString(i)).c_str(), block); + fields.emplace_back(result.second[i](ctx, block)); + new StoreInst(fields.back(), pointer, block); + } + + const auto addFunc = ConstantInt::get(Type::getInt64Ty(context), GetMethodPtr(&TYtWideOutputWrapper::AddRowImpl)); + const auto selfArg = ConstantInt::get(Type::getInt64Ty(context), ui64(this)); + const auto addType = FunctionType::get(Type::getVoidTy(context), {selfArg->getType(), values->getType()}, false); + const auto addPtr = CastInst::Create(Instruction::IntToPtr, addFunc, PointerType::getUnqual(addType), "write", block); + CallInst::Create(addType, addPtr, {selfArg, values}, "", block); + + for (ui32 i = 0U; i < Representations.size(); ++i) { + ValueCleanup(Representations[i], fields[i], ctx, block); + } + + BranchInst::Create(pass, block); + + block = pass; + return {result.first, {}}; + + } +#endif +private: + void RegisterDependencies() const final { + FlowDependsOn(Flow); + } + + void AddRowImpl(const NUdf::TUnboxedValuePod* row) const { + Writer.AddFlatRow(row); + } + + static std::vector<NUdf::TUnboxedValue*> GetPointers(std::vector<NUdf::TUnboxedValue>& array) { + std::vector<NUdf::TUnboxedValue*> pointers; + pointers.reserve(array.size()); + std::transform(array.begin(), array.end(), std::back_inserter(pointers), [](NUdf::TUnboxedValue& v) { return std::addressof(v); }); + return pointers; + } + + IComputationWideFlowNode *const Flow; + const std::vector<EValueRepresentation> Representations; + + TMkqlWriterImpl& Writer; + + std::vector<NUdf::TUnboxedValue> Values; + const std::vector<NUdf::TUnboxedValue*> Fields; +}; + +} + +IComputationNode* WrapYtOutput(TCallable& callable, const TComputationNodeFactoryContext& ctx, TMkqlWriterImpl& writer) { + YQL_ENSURE(callable.GetInputsCount() == 1, "Expected 1 arg"); + const auto item = LocateNode(ctx.NodeLocator, callable, 0); + if (const auto inputType = callable.GetInput(0).GetStaticType(); inputType->IsFlow()) { + if (const auto wide = dynamic_cast<IComputationWideFlowNode*>(item)) { + std::vector<EValueRepresentation> inputRepresentations; + auto wideComponents = GetWideComponents(AS_TYPE(TFlowType, inputType)); + inputRepresentations.reserve(wideComponents.size()); + for (ui32 i = 0U; i < wideComponents.size(); ++i) + inputRepresentations.emplace_back(GetValueRepresentation(wideComponents[i])); + return new TYtWideOutputWrapper(wide, writer, std::move(inputRepresentations)); + } + return new TYtFlowOutputWrapper(item, writer); + } + + return new TYtOutputWrapper(item, writer); +} + +} // NYql diff --git a/yt/yql/providers/yt/comp_nodes/yql_mkql_output.h b/yt/yql/providers/yt/comp_nodes/yql_mkql_output.h new file mode 100644 index 0000000000..044fe64ec8 --- /dev/null +++ b/yt/yql/providers/yt/comp_nodes/yql_mkql_output.h @@ -0,0 +1,13 @@ +#pragma once + +#include <yt/yql/providers/yt/codec/yt_codec_io.h> + +#include <yql/essentials/minikql/computation/mkql_computation_node.h> +#include <yql/essentials/minikql/mkql_node.h> + +namespace NYql { + +NKikimr::NMiniKQL::IComputationNode* WrapYtOutput(NKikimr::NMiniKQL::TCallable& callable, + const NKikimr::NMiniKQL::TComputationNodeFactoryContext& ctx, TMkqlWriterImpl& writer); + +} // NYql diff --git a/yt/yql/providers/yt/comp_nodes/yql_mkql_table.cpp b/yt/yql/providers/yt/comp_nodes/yql_mkql_table.cpp new file mode 100644 index 0000000000..65f1bc67fb --- /dev/null +++ b/yt/yql/providers/yt/comp_nodes/yql_mkql_table.cpp @@ -0,0 +1,108 @@ +#include "yql_mkql_table.h" + +#include <yql/essentials/public/udf/udf_value.h> + +namespace NYql { + +using namespace NKikimr::NMiniKQL; + +TVector<NUdf::TUnboxedValue> MakeTablePaths(const TTypeEnvironment& env, const TVector<TString>& paths) { + TVector<NUdf::TUnboxedValue> tablePaths; + tablePaths.reserve(paths.size()); + for (auto& path: paths) { + if (path) { + tablePaths.emplace_back(env.NewStringValue(path)); + } else { + tablePaths.emplace_back(); + } + } + return tablePaths; +} + +TTableState::TTableState(const TTypeEnvironment& env, const TVector<TString>& paths, const TVector<ui64>& recordOffsets, + TComputationContext& ctx, const std::array<IComputationExternalNode*, 5>& nodes) + : TTableState(MakeTablePaths(env, paths), recordOffsets, ctx, nodes) +{ +} + +TTableState::TTableState(const TVector<NUdf::TUnboxedValue>& paths, const TVector<ui64>& rowOffsets, TComputationContext& ctx, + const std::array<IComputationExternalNode*, 5>& nodes) +{ + auto tableIndexNode = nodes[0]; + if (tableIndexNode) { + UpdateTableIndex = [&ctx, tableIndexNode] (bool valid, ui32 tableIndex) { + tableIndexNode->SetValue(ctx, valid ? NUdf::TUnboxedValuePod(tableIndex) : NUdf::TUnboxedValue::Zero()); + }; + } else { + UpdateTableIndex = [] (bool, ui32) {}; + } + + auto tablePathNode = nodes[1]; + if (tablePathNode && paths) { + UpdateTablePath = [&ctx, tablePathNode, paths] (bool valid, ui32 tableIndex) { + if (valid && paths.at(tableIndex)) { + tablePathNode->SetValue(ctx, NUdf::TUnboxedValue(paths.at(tableIndex))); + } else { + tablePathNode->SetValue(ctx, NUdf::TUnboxedValue::Zero()); + } + }; + } else { + UpdateTablePath = [] (bool, ui32) {}; + } + + auto tableRecordNode = nodes[2]; + if (tableRecordNode && paths) { + UpdateTableRecord = [&ctx, tableRecordNode, paths] (bool valid, ui32 tableIndex, ui64 rowNumber) { + if (valid && paths.at(tableIndex)) { + tableRecordNode->SetValue(ctx, NUdf::TUnboxedValuePod(rowNumber)); + } else { + tableRecordNode->SetValue(ctx, NUdf::TUnboxedValue::Zero()); + } + }; + } else { + UpdateTableRecord = [] (bool, ui32, ui64) {}; + } + + auto isKeySwitchNode = nodes[3]; + if (isKeySwitchNode) { + UpdateIsKeySwitch = [&ctx, isKeySwitchNode] (bool keySwitch) { +// TODO:correct set key switch (as hidden column, as example). +// isKeySwitchNode->SetValue(ctx, NUdf::TUnboxedValuePod(keySwitch)); + ctx.MutableValues[isKeySwitchNode->GetIndex()] = NUdf::TUnboxedValuePod(keySwitch); + }; + } else { + UpdateIsKeySwitch = [] (bool) {}; + } + + auto rowNumberNode = nodes[4]; + if (rowNumberNode && rowOffsets) { + UpdateRowNumber = [&ctx, rowNumberNode, rowOffsets] (bool valid, ui32 tableIndex, ui64 rowNumber) { + if (valid) { + rowNumberNode->SetValue(ctx, NUdf::TUnboxedValuePod(rowOffsets.at(tableIndex) + rowNumber)); + } else { + rowNumberNode->SetValue(ctx, NUdf::TUnboxedValue::Zero()); + } + }; + } else { + UpdateRowNumber = [] (bool, ui32, ui64) {}; + } +} + +void TTableState::Reset() { + UpdateTableIndex(false, 0); + UpdateTablePath(false, 0); + UpdateTableRecord(false, 0, 0); + UpdateIsKeySwitch(false); + UpdateRowNumber(false, 0, 0); +} + +void TTableState::Update(ui32 tableIndex, ui64 tableRecord, bool keySwitch) { + UpdateTableIndex(true, tableIndex); + UpdateTablePath(true, tableIndex); + UpdateTableRecord(true, tableIndex, tableRecord); + UpdateIsKeySwitch(keySwitch); + UpdateRowNumber(true, tableIndex, tableRecord); +} + + +} // NYql diff --git a/yt/yql/providers/yt/comp_nodes/yql_mkql_table.h b/yt/yql/providers/yt/comp_nodes/yql_mkql_table.h new file mode 100644 index 0000000000..c8344547ce --- /dev/null +++ b/yt/yql/providers/yt/comp_nodes/yql_mkql_table.h @@ -0,0 +1,42 @@ +#pragma once + +#include <yql/essentials/minikql/computation/mkql_computation_node.h> +#include <yql/essentials/minikql/mkql_node.h> +#include <yql/essentials/public/udf/udf_value.h> + +#include <util/generic/string.h> +#include <util/generic/vector.h> + +#include <functional> +#include <array> + +namespace NYql { + +TVector<NUdf::TUnboxedValue> MakeTablePaths(const NKikimr::NMiniKQL::TTypeEnvironment& env, const TVector<TString>& paths); + +class TTableState { +public: + TTableState(const NKikimr::NMiniKQL::TTypeEnvironment& env, const TVector<TString>& paths, + const TVector<ui64>& recordOffsets, NKikimr::NMiniKQL::TComputationContext& ctx, + const std::array<NKikimr::NMiniKQL::IComputationExternalNode*, 5>& nodes); + + TTableState(const TVector<NUdf::TUnboxedValue>& paths, const TVector<ui64>& recordOffsets, + NKikimr::NMiniKQL::TComputationContext& ctx, + const std::array<NKikimr::NMiniKQL::IComputationExternalNode*, 5>& nodes); + + TTableState(const TTableState&) = default; + TTableState(TTableState&&) = default; + ~TTableState() = default; + + void Reset(); + void Update(ui32 tableIndex, ui64 tableRecord, bool keySwitch = false); + +private: + std::function<void(bool, ui32)> UpdateTableIndex; + std::function<void(bool, ui32)> UpdateTablePath; + std::function<void(bool, ui32, ui64)> UpdateTableRecord; + std::function<void(bool)> UpdateIsKeySwitch; + std::function<void(bool, ui32, ui64)> UpdateRowNumber; +}; + +} // NYql diff --git a/yt/yql/providers/yt/comp_nodes/yql_mkql_table_content.cpp b/yt/yql/providers/yt/comp_nodes/yql_mkql_table_content.cpp new file mode 100644 index 0000000000..4dc5e57243 --- /dev/null +++ b/yt/yql/providers/yt/comp_nodes/yql_mkql_table_content.cpp @@ -0,0 +1,78 @@ +#include "yql_mkql_table_content.h" +#include "yql_mkql_file_input_state.h" +#include "yql_mkql_file_list.h" + +#include <yql/essentials/minikql/computation/mkql_computation_node_impl.h> +#include <yql/essentials/minikql/mkql_node_cast.h> +#include <yql/essentials/minikql/defs.h> + +#include <yql/essentials/public/udf/udf_value.h> + +#include <util/generic/vector.h> +#include <util/generic/string.h> +#include <util/generic/size_literals.h> + +#include <type_traits> + +namespace NYql { + +using namespace NKikimr; +using namespace NKikimr::NMiniKQL; + +class TYtTableContentWrapper : public TMutableComputationNode<TYtTableContentWrapper> { + typedef TMutableComputationNode<TYtTableContentWrapper> TBaseComputation; +public: + TYtTableContentWrapper(TComputationMutables& mutables, NCommon::TCodecContext& codecCtx, + TVector<TString>&& files, const TString& inputSpec, TType* listType, bool useSkiff, bool decompress, const TString& optLLVM, + std::optional<ui64> length) + : TBaseComputation(mutables) + , Files_(std::move(files)) + , Decompress_(decompress) + , Length_(std::move(length)) + { + if (useSkiff) { + Spec_.SetUseSkiff(optLLVM); + } + Spec_.Init(codecCtx, inputSpec, {}, {}, AS_TYPE(TListType, listType)->GetItemType(), {}, TString()); + } + + NUdf::TUnboxedValuePod DoCalculate(TComputationContext& ctx) const { + return ctx.HolderFactory.Create<TFileListValue>(Spec_, ctx.HolderFactory, Files_, Decompress_, 4, 1_MB, Length_); + } + +private: + void RegisterDependencies() const final {} + + TMkqlIOSpecs Spec_; + TVector<TString> Files_; + const bool Decompress_; + const std::optional<ui64> Length_; +}; + +IComputationNode* WrapYtTableContent(NCommon::TCodecContext& codecCtx, + TComputationMutables& mutables, TCallable& callable, const TString& optLLVM, TStringBuf pathPrefix) +{ + MKQL_ENSURE(callable.GetInputsCount() == 6, "Expected 6 arguments"); + TString uniqueId(AS_VALUE(TDataLiteral, callable.GetInput(0))->AsValue().AsStringRef()); + const ui32 tablesCount = AS_VALUE(TDataLiteral, callable.GetInput(1))->AsValue().Get<ui32>(); + TString inputSpec(AS_VALUE(TDataLiteral, callable.GetInput(2))->AsValue().AsStringRef()); + const bool useSkiff = AS_VALUE(TDataLiteral, callable.GetInput(3))->AsValue().Get<bool>(); + const bool decompress = AS_VALUE(TDataLiteral, callable.GetInput(4))->AsValue().Get<bool>(); + + std::optional<ui64> length; + TTupleLiteral* lengthTuple = AS_VALUE(TTupleLiteral, callable.GetInput(5)); + if (lengthTuple->GetValuesCount() > 0) { + YQL_ENSURE(lengthTuple->GetValuesCount() == 1, "Expect 1 element in the length tuple"); + length = AS_VALUE(TDataLiteral, lengthTuple->GetValue(0))->AsValue().Get<ui64>(); + } + + TVector<TString> files; + for (ui32 index = 0; index < tablesCount; ++index) { + files.push_back(TStringBuilder() << pathPrefix << uniqueId << '_' << index); + } + + return new TYtTableContentWrapper(mutables, codecCtx, std::move(files), inputSpec, + callable.GetType()->GetReturnType(), useSkiff, decompress, optLLVM, length); +} + +} // NYql diff --git a/yt/yql/providers/yt/comp_nodes/yql_mkql_table_content.h b/yt/yql/providers/yt/comp_nodes/yql_mkql_table_content.h new file mode 100644 index 0000000000..b3e0032b01 --- /dev/null +++ b/yt/yql/providers/yt/comp_nodes/yql_mkql_table_content.h @@ -0,0 +1,18 @@ +#pragma once + +#include <yql/essentials/providers/common/codec/yql_codec.h> + +#include <yql/essentials/minikql/mkql_node.h> +#include <yql/essentials/minikql/computation/mkql_computation_node.h> + +#include <util/generic/string.h> +#include <util/generic/strbuf.h> + +namespace NYql { + +NKikimr::NMiniKQL::IComputationNode* WrapYtTableContent( + NYql::NCommon::TCodecContext& codecCtx, + NKikimr::NMiniKQL::TComputationMutables& mutables, + NKikimr::NMiniKQL::TCallable& callable, const TString& optLLVM, TStringBuf pathPrefix); + +} // NYql diff --git a/yt/yql/providers/yt/comp_nodes/yql_mkql_ungrouping_list.cpp b/yt/yql/providers/yt/comp_nodes/yql_mkql_ungrouping_list.cpp new file mode 100644 index 0000000000..2d49841f2a --- /dev/null +++ b/yt/yql/providers/yt/comp_nodes/yql_mkql_ungrouping_list.cpp @@ -0,0 +1,107 @@ +#include "yql_mkql_ungrouping_list.h" + +#include <yql/essentials/minikql/computation/mkql_computation_node_holders.h> +#include <yql/essentials/minikql/computation/mkql_custom_list.h> +#include <yql/essentials/minikql/mkql_node_cast.h> + +#include <yql/essentials/utils/yql_panic.h> + +namespace NYql { + +using namespace NKikimr::NMiniKQL; + +namespace { + +class TYtUngroupingListWrapper : public TMutableComputationNode<TYtUngroupingListWrapper> { + typedef TMutableComputationNode<TYtUngroupingListWrapper> TBaseComputation; +public: + class TListValue : public TCustomListValue { + public: + class TIterator : public TComputationValue<TIterator> { + public: + TIterator(TMemoryUsageInfo* memInfo, TComputationContext& ctx, NUdf::TUnboxedValue&& iter, IComputationExternalNode* isKeySwitchNode) + : TComputationValue<TIterator>(memInfo) + , ListIterator(std::move(iter)) + , IsKeySwitchNode(isKeySwitchNode) + , Ctx(ctx) + {} + + private: + bool Next(NUdf::TUnboxedValue& value) final { + for (;;) { + if (IsKeySwitchNode) { + Ctx.MutableValues[IsKeySwitchNode->GetIndex()] = NUdf::TUnboxedValuePod(!SubListIterator); + } + if (!SubListIterator) { + NUdf::TUnboxedValue pair; + if (!ListIterator.Next(pair)) { + return false; + } + SubListIterator = pair.GetListIterator(); + } + if (SubListIterator.Next(value)) { + return true; + } else { + SubListIterator.Clear(); + } + } + } + + const NUdf::TUnboxedValue ListIterator; + NUdf::TUnboxedValue SubListIterator; + IComputationExternalNode* const IsKeySwitchNode; + TComputationContext& Ctx; + }; + + TListValue(TMemoryUsageInfo* memInfo, NUdf::TUnboxedValue&& list, IComputationExternalNode* isKeySwitchNode, TComputationContext& ctx) + : TCustomListValue(memInfo), List(std::move(list)), IsKeySwitchNode(isKeySwitchNode), Ctx(ctx) + {} + + private: + NUdf::TUnboxedValue GetListIterator() const override { + return Ctx.HolderFactory.Create<TIterator>(Ctx, List.GetListIterator(), IsKeySwitchNode); + } + + private: + const NUdf::TUnboxedValue List; + IComputationExternalNode* const IsKeySwitchNode; + TComputationContext& Ctx; + }; + + TYtUngroupingListWrapper(TComputationMutables& mutables, IComputationNode* list, IComputationExternalNode* isKeySwitchNode) + : TBaseComputation(mutables) + , List_(list) + , IsKeySwitchNode_(isKeySwitchNode) + { + } + + NUdf::TUnboxedValuePod DoCalculate(TComputationContext& ctx) const { + return ctx.HolderFactory.Create<TListValue>(List_->GetValue(ctx), IsKeySwitchNode_, ctx); + } + +private: + void RegisterDependencies() const final { + DependsOn(List_); + Own(IsKeySwitchNode_); + } + + IComputationNode* const List_; + IComputationExternalNode* const IsKeySwitchNode_; +}; + +} // unnamed + +/////////////////////////////////////////////////////////////////////////////////////////////////////// + +IComputationNode* WrapYtUngroupingList(TCallable& callable, const TComputationNodeFactoryContext& ctx) { + YQL_ENSURE(callable.GetInputsCount() == 2, "Expected 2 args"); + const auto listItemType = AS_TYPE(TListType, callable.GetInput(0))->GetItemType(); + const auto subListItemType = AS_TYPE(TListType, listItemType)->GetItemType(); + YQL_ENSURE(subListItemType->IsStruct() || subListItemType->IsVariant()); + + const auto list = LocateNode(ctx.NodeLocator, callable, 0); + const auto isKeySwitch = LocateExternalNode(ctx.NodeLocator, callable, 1, false); + return new TYtUngroupingListWrapper(ctx.Mutables, list, isKeySwitch); +} + +} // NYql diff --git a/yt/yql/providers/yt/comp_nodes/yql_mkql_ungrouping_list.h b/yt/yql/providers/yt/comp_nodes/yql_mkql_ungrouping_list.h new file mode 100644 index 0000000000..0f35b5a9c4 --- /dev/null +++ b/yt/yql/providers/yt/comp_nodes/yql_mkql_ungrouping_list.h @@ -0,0 +1,10 @@ +#pragma once + +#include <yql/essentials/minikql/computation/mkql_computation_node.h> +#include <yql/essentials/minikql/mkql_node.h> + +namespace NYql { + +NKikimr::NMiniKQL::IComputationNode* WrapYtUngroupingList(NKikimr::NMiniKQL::TCallable& callable, const NKikimr::NMiniKQL::TComputationNodeFactoryContext& ctx); + +} // NYql diff --git a/yt/yql/providers/yt/expr_nodes/ya.make b/yt/yql/providers/yt/expr_nodes/ya.make new file mode 100644 index 0000000000..5c46e4ff03 --- /dev/null +++ b/yt/yql/providers/yt/expr_nodes/ya.make @@ -0,0 +1,52 @@ +LIBRARY() + +SRCS( + yql_yt_expr_nodes.cpp +) + +PEERDIR( + yql/essentials/core/expr_nodes + yql/essentials/providers/common/provider +) + +SRCDIR( + yql/essentials/core/expr_nodes_gen +) + +IF (EXPORT_CMAKE) + RUN_PYTHON3( + ${ARCADIA_ROOT}/yql/essentials/core/expr_nodes_gen/gen/__main__.py + yql_expr_nodes_gen.jnj + yql_yt_expr_nodes.json + yql_yt_expr_nodes.gen.h + yql_yt_expr_nodes.decl.inl.h + yql_yt_expr_nodes.defs.inl.h + IN yql_expr_nodes_gen.jnj + IN yql_yt_expr_nodes.json + OUT yql_yt_expr_nodes.gen.h + OUT yql_yt_expr_nodes.decl.inl.h + OUT yql_yt_expr_nodes.defs.inl.h + OUTPUT_INCLUDES + ${ARCADIA_ROOT}/yql/essentials/core/expr_nodes_gen/yql_expr_nodes_gen.h + ${ARCADIA_ROOT}/util/generic/hash_set.h + ) +ELSE() + RUN_PROGRAM( + yql/essentials/core/expr_nodes_gen/gen + yql_expr_nodes_gen.jnj + yql_yt_expr_nodes.json + yql_yt_expr_nodes.gen.h + yql_yt_expr_nodes.decl.inl.h + yql_yt_expr_nodes.defs.inl.h + IN yql_expr_nodes_gen.jnj + IN yql_yt_expr_nodes.json + OUT yql_yt_expr_nodes.gen.h + OUT yql_yt_expr_nodes.decl.inl.h + OUT yql_yt_expr_nodes.defs.inl.h + OUTPUT_INCLUDES + ${ARCADIA_ROOT}/yql/essentials/core/expr_nodes_gen/yql_expr_nodes_gen.h + ${ARCADIA_ROOT}/util/generic/hash_set.h + ) +ENDIF() + +END() diff --git a/yt/yql/providers/yt/expr_nodes/yql_yt_expr_nodes.cpp b/yt/yql/providers/yt/expr_nodes/yql_yt_expr_nodes.cpp new file mode 100644 index 0000000000..d6a0eb9563 --- /dev/null +++ b/yt/yql/providers/yt/expr_nodes/yql_yt_expr_nodes.cpp @@ -0,0 +1,7 @@ +#include "yql_yt_expr_nodes.h" + +namespace NYql { +namespace NNodes { + +} // namespace NNodes +} // namespace NYql diff --git a/yt/yql/providers/yt/expr_nodes/yql_yt_expr_nodes.h b/yt/yql/providers/yt/expr_nodes/yql_yt_expr_nodes.h new file mode 100644 index 0000000000..cfe5530c96 --- /dev/null +++ b/yt/yql/providers/yt/expr_nodes/yql_yt_expr_nodes.h @@ -0,0 +1,66 @@ +#pragma once + +#include <yt/yql/providers/yt/expr_nodes/yql_yt_expr_nodes.gen.h> +#include <yql/essentials/providers/common/provider/yql_provider_names.h> + +#include <yql/essentials/core/expr_nodes/yql_expr_nodes.h> + +namespace NYql { +namespace NNodes { + +#include <yt/yql/providers/yt/expr_nodes/yql_yt_expr_nodes.decl.inl.h> + +class TYtDSource: public NGenerated::TYtDSourceStub<TExprBase, TCallable, TCoAtom> { +public: + explicit TYtDSource(const TExprNode* node) + : TYtDSourceStub(node) + { + } + + explicit TYtDSource(const TExprNode::TPtr& node) + : TYtDSourceStub(node) + { + } + + static bool Match(const TExprNode* node) { + if (!TYtDSourceStub::Match(node)) { + return false; + } + + if (node->Child(0)->Content() != YtProviderName) { + return false; + } + + return true; + } +}; + +class TYtDSink: public NGenerated::TYtDSinkStub<TExprBase, TCallable, TCoAtom> { +public: + explicit TYtDSink(const TExprNode* node) + : TYtDSinkStub(node) + { + } + + explicit TYtDSink(const TExprNode::TPtr& node) + : TYtDSinkStub(node) + { + } + + static bool Match(const TExprNode* node) { + if (!TYtDSinkStub::Match(node)) { + return false; + } + + if (node->Child(0)->Content() != YtProviderName) { + return false; + } + + return true; + } +}; + +#include <yt/yql/providers/yt/expr_nodes/yql_yt_expr_nodes.defs.inl.h> + +} // namespace NNodes +} // namespace NYql diff --git a/yt/yql/providers/yt/expr_nodes/yql_yt_expr_nodes.json b/yt/yql/providers/yt/expr_nodes/yql_yt_expr_nodes.json new file mode 100644 index 0000000000..d6508aef6c --- /dev/null +++ b/yt/yql/providers/yt/expr_nodes/yql_yt_expr_nodes.json @@ -0,0 +1,497 @@ +{ + "NodeRootType": "TExprBase", + "NodeBuilderBase": "TNodeBuilderBase", + "ListBuilderBase": "TListBuilderBase", + "FreeArgCallableBase": "TFreeArgCallable", + "FreeArgBuilderBase": "TFreeArgCallableBuilderBase", + "Nodes": [ + { + "Name": "TYtDSource", + "Base": "TCallable", + "Definition": "Custom", + "Match": {"Type": "Callable", "Name": "DataSource"}, + "Children": [ + {"Index": 0, "Name": "Category", "Type": "TCoAtom"}, + {"Index": 1, "Name": "Cluster", "Type": "TCoAtom"} + ] + }, + { + "Name": "TYtDSink", + "Base": "TCallable", + "Definition": "Custom", + "Match": {"Type": "Callable", "Name": "DataSink"}, + "Children": [ + {"Index": 0, "Name": "Category", "Type": "TCoAtom"}, + {"Index": 1, "Name": "Cluster", "Type": "TCoAtom"} + ] + }, + { + "Name": "TYtNamedSettingsBase", + "VarArgBase": "TCoNameValueTuple", + "Builder": {"Generate": "None"} + }, + { + "Name": "TYqlRowSpec", + "Base": "TYtNamedSettingsBase", + "Match": {"Type": "Callable", "Name": "YqlRowSpec"} + }, + { + "Name": "TEpoch", + "Base": "TCallable", + "Match": {"Type": "Callable", "Name": "Epoch"}, + "Children": [ + {"Index": 0, "Name": "Value", "Type": "TCoAtom"} + ] + }, + { + "Name": "TYtMeta", + "Base": "TYtNamedSettingsBase", + "Match": {"Type": "Callable", "Name": "YtMeta"} + }, + { + "Name": "TYtStat", + "Base": "TYtNamedSettingsBase", + "Match": {"Type": "Callable", "Name": "YtStat"} + }, + { + "Name": "TYtTableBase", + "Base": "TCallable", + "Match": {"Type": "CallableBase"}, + "Builder": {"Generate": "None"}, + "Children": [ + {"Index": 0, "Name": "Name", "Type": "TCoAtom"}, + {"Index": 1, "Name": "RowSpec", "Type": "TExprBase"}, + {"Index": 2, "Name": "Meta", "Type": "TExprBase"}, + {"Index": 3, "Name": "Stat", "Type": "TExprBase"}, + {"Index": 4, "Name": "Settings", "Type": "TCoNameValueTupleList"} + ] + }, + { + "Name": "TYtTable", + "Base": "TYtTableBase", + "Match": {"Type": "Callable", "Name": "YtTable"}, + "Children": [ + {"Index": 5, "Name": "Epoch", "Type": "TExprBase"}, + {"Index": 6, "Name": "CommitEpoch", "Type": "TExprBase"}, + {"Index": 7, "Name": "Cluster", "Type": "TCoAtom"} + ] + }, + { + "Name": "TYtOutTable", + "Base": "TYtTableBase", + "Match": {"Type": "Callable", "Name": "YtOutTable"} + }, + { + "Name": "TYtOutSection", + "ListBase": "TYtOutTable" + }, + { + "Name": "TYtRangeItemBase", + "Base": "TCallable", + "Match": {"Type": "CallableBase"}, + "Builder": {"Generate": "None"} + }, + { + "Name": "TYtKeyExact", + "Base": "TYtRangeItemBase", + "Match": {"Type": "Callable", "Name": "YtKeyExact"}, + "Children": [ + {"Index": 0, "Name": "Key", "Type": "TExprList"} + ] + }, + { + "Name": "TYtKeyRange", + "Base": "TYtRangeItemBase", + "Match": {"Type": "Callable", "Name": "YtKeyRange"}, + "Children": [ + {"Index": 0, "Name": "Lower", "Type": "TExprList"}, + {"Index": 1, "Name": "Upper", "Type": "TExprList"}, + {"Index": 2, "Name": "Flags", "Type": "TCoAtomList", "Optional": true} + ] + }, + { + "Name": "TYtRow", + "Base": "TYtRangeItemBase", + "Match": {"Type": "Callable", "Name": "YtRow"}, + "Children": [ + {"Index": 0, "Name": "Index", "Type": "TCoUint64"} + ] + }, + { + "Name": "TYtRowRange", + "Base": "TYtRangeItemBase", + "Match": {"Type": "Callable", "Name": "YtRowRange"}, + "Children": [ + {"Index": 0, "Name": "Lower", "Type": "TExprBase"}, + {"Index": 1, "Name": "Upper", "Type": "TExprBase"} + ] + }, + { + "Name": "TYtPath", + "Base": "TCallable", + "Match": {"Type": "Callable", "Name": "YtPath"}, + "Children": [ + {"Index": 0, "Name": "Table", "Type": "TExprBase"}, + {"Index": 1, "Name": "Columns", "Type": "TExprBase"}, + {"Index": 2, "Name": "Ranges", "Type": "TExprBase"}, + {"Index": 3, "Name": "Stat", "Type": "TExprBase"}, + {"Index": 4, "Name": "AdditionalAttributes", "Type": "TCoAtom", "Optional": true} + ] + }, + { + "Name": "TYtPathList", + "ListBase": "TYtPath" + }, + { + "Name": "TYtSection", + "Base": "TCallable", + "Match": {"Type": "Callable", "Name": "YtSection"}, + "Children": [ + {"Index": 0, "Name": "Paths", "Type": "TYtPathList"}, + {"Index": 1, "Name": "Settings", "Type": "TCoNameValueTupleList"} + ] + }, + { + "Name": "TYtSectionList", + "ListBase": "TYtSection" + }, + { + "Name": "TYtRead", + "Base": "TFreeArgCallable", + "Match": {"Type": "Callable", "Name": "Read!"}, + "Children": [ + {"Index": 0, "Name": "World", "Type": "TExprBase"}, + {"Index": 1, "Name": "DataSource", "Type": "TYtDSource"} + ] + }, + { + "Name": "TYtReadTable", + "Base": "TCallable", + "Match": {"Type": "Callable", "Name": "YtReadTable!"}, + "Children": [ + {"Index": 0, "Name": "World", "Type": "TExprBase"}, + {"Index": 1, "Name": "DataSource", "Type": "TYtDSource"}, + {"Index": 2, "Name": "Input", "Type": "TYtSectionList"} + ] + }, + { + "Name": "TYtReadTableScheme", + "Base": "TCallable", + "Match": {"Type": "Callable", "Name": "YtReadTableScheme!"}, + "Children": [ + {"Index": 0, "Name": "World", "Type": "TExprBase"}, + {"Index": 1, "Name": "DataSource", "Type": "TYtDSource"}, + {"Index": 2, "Name": "Table", "Type": "TYtTable"}, + {"Index": 3, "Name": "Type", "Type": "TExprBase"} + ] + }, + { + "Name": "TYtWrite", + "Base": "TFreeArgCallable", + "Match": {"Type": "Callable", "Name": "Write!"}, + "Children": [ + {"Index": 0, "Name": "World", "Type": "TExprBase"}, + {"Index": 1, "Name": "DataSink", "Type": "TYtDSink"} + ] + }, + { + "Name": "TYtTableContent", + "Base": "TCallable", + "Match": {"Type": "Callable", "Name": "YtTableContent"}, + "Children": [ + {"Index": 0, "Name": "Input", "Type": "TExprBase"}, + {"Index": 1, "Name": "Settings", "Type": "TCoNameValueTupleList"} + ] + }, + { + "Name": "TYtLength", + "Base": "TCallable", + "Match": {"Type": "Callable", "Name": "YtLength"}, + "Children": [ + {"Index": 0, "Name": "Input", "Type": "TExprBase"} + ] + }, + { + "Name": "TYtConfigure", + "Base": "TFreeArgCallable", + "Match": {"Type": "Callable", "Name": "YtConfigure!"}, + "Children": [ + {"Index": 0, "Name": "World", "Type": "TExprBase"}, + {"Index": 1, "Name": "DataSource", "Type": "TYtDSource"} + ] + }, + { + "Name": "TYtTablePropBase", + "Base": "TCallable", + "Match": {"Type": "CallableBase"}, + "Builder": {"Generate": "None"}, + "Children": [ + {"Index": 0, "Name": "DependsOn", "Type": "TCoDependsOn"} + ] + }, + { + "Name": "TYtTablePath", + "Base": "TYtTablePropBase", + "Match": {"Type": "Callable", "Name": "YtTablePath"} + }, + { + "Name": "TYtTableRecord", + "Base": "TYtTablePropBase", + "Match": {"Type": "Callable", "Name": "YtTableRecord"} + }, + { + "Name": "TYtTableIndex", + "Base": "TYtTablePropBase", + "Match": {"Type": "Callable", "Name": "YtTableIndex"} + }, + { + "Name": "TYtIsKeySwitch", + "Base": "TYtTablePropBase", + "Match": {"Type": "Callable", "Name": "YtIsKeySwitch"} + }, + { + "Name": "TYtRowNumber", + "Base": "TYtTablePropBase", + "Match": {"Type": "Callable", "Name": "YtRowNumber"} + }, + { + "Name": "TYtTableName", + "Base": "TCallable", + "Match": {"Type": "Callable", "Name": "YtTableName"}, + "Children": [ + {"Index": 0, "Name": "Input", "Type": "TExprBase"} + ] + }, + { + "Name": "TYtOpBase", + "Base": "TCallable", + "Match": {"Type": "CallableBase"}, + "Builder": {"Generate": "None"}, + "Children": [ + {"Index": 0, "Name": "World", "Type": "TExprBase"}, + {"Index": 1, "Name": "DataSink", "Type": "TYtDSink"} + ] + }, + { + "Name": "TYtOutputOpBase", + "Base": "TYtOpBase", + "Match": {"Type": "CallableBase"}, + "Builder": {"Generate": "None"}, + "Children": [ + {"Index": 2, "Name": "Output", "Type": "TYtOutSection"} + ] + }, + { + "Name": "TYtTransientOpBase", + "Base": "TYtOutputOpBase", + "Match": {"Type": "CallableBase"}, + "Builder": {"Generate": "None"}, + "Children": [ + {"Index": 3, "Name": "Input", "Type": "TYtSectionList"}, + {"Index": 4, "Name": "Settings", "Type": "TCoNameValueTupleList"} + ] + }, + { + "Name": "TYtWithUserJobsOpBase", + "Base": "TYtTransientOpBase", + "Match": {"Type": "CallableBase"}, + "Builder": {"Generate": "None"} + }, + { + "Name": "TYtOutput", + "Base": "TCallable", + "Match": {"Type": "Callable", "Name": "YtOutput!"}, + "Children": [ + {"Index": 0, "Name": "Operation", "Type": "TExprBase"}, + {"Index": 1, "Name": "OutIndex", "Type": "TCoAtom"}, + {"Index": 2, "Name": "Mode", "Type": "TCoAtom", "Optional": true} + ] + }, + { + "Name": "TYtWriteTable", + "Base": "TYtOpBase", + "Match": {"Type": "Callable", "Name": "YtWriteTable!"}, + "Children": [ + {"Index": 2, "Name": "Table", "Type": "TYtTable"}, + {"Index": 3, "Name": "Content", "Type": "TExprBase"}, + {"Index": 4, "Name": "Settings", "Type": "TCoNameValueTupleList"} + ] + }, + { + "Name": "TYtStatOutTable", + "Base": "TFreeArgCallable", + "Match": {"Type": "Callable", "Name": "YtStatOutTable"}, + "Children": [ + {"Index": 0, "Name": "Name", "Type": "TCoAtom"}, + {"Index": 1, "Name": "Scale", "Type": "TCoAtom"}, + {"Index": 2, "Name": "Cluster", "Type": "TCoAtom"} + ] + }, + { + "Name": "TYtStatOut", + "Base": "TYtOpBase", + "Match": {"Type": "Callable", "Name": "YtStatOut!"}, + "Children": [ + {"Index": 2, "Name": "Input", "Type": "TYtOutput"}, + {"Index": 3, "Name": "Table", "Type": "TYtStatOutTable"}, + {"Index": 4, "Name": "ReplaceMask", "Type": "TCoAtomList"}, + {"Index": 5, "Name": "Settings", "Type": "TCoNameValueTupleList"} + ] + }, + { + "Name": "TYtFill", + "Base": "TYtOutputOpBase", + "Match": {"Type": "Callable", "Name": "YtFill!"}, + "Children": [ + {"Index": 3, "Name": "Content", "Type": "TCoLambda"}, + {"Index": 4, "Name": "Settings", "Type": "TCoNameValueTupleList"} + ] + }, + { + "Name": "TYtMaterialize", + "Base": "TYtOpBase", + "Match": {"Type": "Callable", "Name": "YtMaterialize!"}, + "Children": [ + {"Index": 2, "Name": "Input", "Type": "TExprBase"}, + {"Index": 3, "Name": "Settings", "Type": "TCoNameValueTupleList"} + ] + }, + { + "Name": "TYtTouch", + "Base": "TYtOutputOpBase", + "Match": {"Type": "Callable", "Name": "YtTouch!"} + }, + { + "Name": "TYtDqProcessWrite", + "Base": "TYtOutputOpBase", + "Match": {"Type": "Callable", "Name": "YtDqProcessWrite!"}, + "Children": [ + {"Index": 3, "Name": "Input", "Type": "TExprBase"}, + {"Index": 4, "Name": "Flags", "Type": "TCoAtomList", "Optional": true} + ] + }, + { + "Name": "TYtDropTable", + "Base": "TYtOpBase", + "Match": {"Type": "Callable", "Name": "YtDropTable!"}, + "Children": [ + {"Index": 2, "Name": "Table", "Type": "TYtTable"} + ] + }, + { + "Name": "TYtOutputList", + "ListBase": "TYtOutput" + }, + { + "Name": "TYtPublish", + "Base": "TYtOpBase", + "Match": {"Type": "Callable", "Name": "YtPublish!"}, + "Children": [ + {"Index": 2, "Name": "Input", "Type": "TYtOutputList"}, + {"Index": 3, "Name": "Publish", "Type": "TYtTable"}, + {"Index": 4, "Name": "Settings", "Type": "TCoNameValueTupleList"} + ] + }, + { + "Name": "TYtSort", + "Base": "TYtTransientOpBase", + "Match": {"Type": "Callable", "Name": "YtSort!"} + }, + { + "Name": "TYtMap", + "Base": "TYtWithUserJobsOpBase", + "Match": {"Type": "Callable", "Name": "YtMap!"}, + "Children": [ + {"Index": 5, "Name": "Mapper", "Type": "TCoLambda"} + ] + }, + { + "Name": "TYtReduce", + "Base": "TYtWithUserJobsOpBase", + "Match": {"Type": "Callable", "Name": "YtReduce!"}, + "Children": [ + {"Index": 5, "Name": "Reducer", "Type": "TCoLambda"} + ] + }, + { + "Name": "TYtMapReduce", + "Base": "TYtWithUserJobsOpBase", + "Match": {"Type": "Callable", "Name": "YtMapReduce!"}, + "Children": [ + {"Index": 5, "Name": "Mapper", "Type": "TExprBase"}, + {"Index": 6, "Name": "Reducer", "Type": "TCoLambda"} + ] + }, + { + "Name": "TYtCopy", + "Base": "TYtTransientOpBase", + "Match": {"Type": "Callable", "Name": "YtCopy!"} + }, + { + "Name": "TYtMerge", + "Base": "TYtTransientOpBase", + "Match": {"Type": "Callable", "Name": "YtMerge!"} + }, + { + "Name": "TYtEquiJoin", + "Base": "TYtTransientOpBase", + "Match": {"Type": "Callable", "Name": "YtEquiJoin!"}, + "Children": [ + {"Index": 5, "Name": "Joins", "Type": "TExprBase"}, + {"Index": 6, "Name": "JoinOptions", "Type": "TCoNameValueTupleList"} + ] + }, + { + "Name": "TYtDqWrite", + "Base": "TCallable", + "Match": {"Type": "Callable", "Name": "YtDqWrite"}, + "Children": [ + {"Index": 0, "Name": "Input", "Type": "TExprBase"}, + {"Index": 1, "Name": "Settings", "Type": "TCoNameValueTupleList"} + ] + }, + { + "Name": "TYtDqWideWrite", + "Base": "TCallable", + "Match": {"Type": "Callable", "Name": "YtDqWideWrite"}, + "Children": [ + {"Index": 0, "Name": "Input", "Type": "TExprBase"}, + {"Index": 1, "Name": "Settings", "Type": "TCoNameValueTupleList"} + ] + }, + { + "Name": "TYtTryFirst", + "Base": "TCallable", + "Match": {"Type": "Callable", "Name": "YtTryFirst!"}, + "Children": [ + {"Index": 0, "Name": "First", "Type": "TYtOutputOpBase"}, + {"Index": 1, "Name": "Second", "Type": "TYtOutputOpBase"} + ] + }, + { + "Name": "TYtWalkFolders", + "Base": "TCallable", + "Match": {"Type": "Callable", "Name": "MrWalkFolders"}, + "Children": [ + {"Index": 0, "Name": "Prefix", "Type": "TCoAtom"}, + {"Index": 1, "Name": "Attributes", "Type": "TCoAtom"}, + {"Index": 2, "Name": "PickledUserState", "Type": "TExprBase"}, + {"Index": 3, "Name": "UserStateType", "Type": "TExprBase"}, + {"Index": 4, "Name": "PreHandler", "Type": "TExprBase"}, + {"Index": 5, "Name": "ResolveHandler", "Type": "TExprBase"}, + {"Index": 6, "Name": "DiveHandler", "Type": "TExprBase"}, + {"Index": 7, "Name": "PostHandler", "Type": "TExprBase"} + ] + }, + { + "Name": "TYtWalkFoldersImpl", + "Base": "TCallable", + "Match": {"Type": "Callable", "Name": "MrWalkFoldersImpl"}, + "Children": [ + {"Index": 0, "Name": "PickledUserState", "Type": "TExprBase"}, + {"Index": 1, "Name": "UserStateType", "Type": "TExprBase"}, + {"Index": 2, "Name": "ProcessStateKey", "Type": "TCoAtom"} + ] + } + ] +} diff --git a/yt/yql/providers/yt/gateway/file/ya.make b/yt/yql/providers/yt/gateway/file/ya.make new file mode 100644 index 0000000000..2957d7703b --- /dev/null +++ b/yt/yql/providers/yt/gateway/file/ya.make @@ -0,0 +1,50 @@ +LIBRARY() + +SRCS( + yql_yt_file_comp_nodes.cpp + yql_yt_file_mkql_compiler.cpp + yql_yt_file_services.cpp + yql_yt_file.cpp + yql_yt_file_text_yson.cpp +) + +PEERDIR( + library/cpp/yson + library/cpp/yson/node + yt/cpp/mapreduce/common + yt/cpp/mapreduce/interface + yql/essentials/core/file_storage + yql/essentials/core/file_storage/proto + yql/essentials/core/file_storage/http_download + yql/essentials/minikql/comp_nodes + yql/essentials/public/udf + yql/essentials/utils + yql/essentials/utils/threading + yql/essentials/core + yql/essentials/core/expr_nodes + yql/essentials/core/type_ann + yql/essentials/providers/common/codec + yql/essentials/providers/common/comp_nodes + yql/essentials/providers/common/gateway + yql/essentials/providers/common/mkql + yql/essentials/providers/common/provider + yql/essentials/providers/common/schema/expr + yql/essentials/providers/common/schema/mkql + yql/essentials/providers/result/expr_nodes + yt/yql/providers/yt/common + yt/yql/providers/yt/comp_nodes + yt/yql/providers/yt/expr_nodes + yt/yql/providers/yt/gateway/lib + yt/yql/providers/yt/lib/infer_schema + yt/yql/providers/yt/lib/lambda_builder + yt/yql/providers/yt/lib/mkql_helpers + yt/yql/providers/yt/lib/res_pull + yt/yql/providers/yt/lib/schema + yt/yql/providers/yt/lib/yson_helpers + yt/yql/providers/yt/provider + yql/essentials/parser/pg_wrapper +) + +YQL_LAST_ABI_VERSION() + +END() diff --git a/yt/yql/providers/yt/gateway/file/yql_yt_file.cpp b/yt/yql/providers/yt/gateway/file/yql_yt_file.cpp new file mode 100644 index 0000000000..5eff2bc247 --- /dev/null +++ b/yt/yql/providers/yt/gateway/file/yql_yt_file.cpp @@ -0,0 +1,1656 @@ +#include "yql_yt_file.h" +#include "yql_yt_file_mkql_compiler.h" +#include "yql_yt_file_comp_nodes.h" + +#include <yql/essentials/providers/common/mkql/yql_provider_mkql.h> +#include <yql/essentials/providers/common/provider/yql_provider.h> +#include <yql/essentials/providers/common/provider/yql_provider_names.h> +#include <yql/essentials/providers/common/mkql/yql_provider_mkql.h> +#include <yql/essentials/providers/common/mkql/yql_type_mkql.h> +#include <yql/essentials/providers/common/schema/expr/yql_expr_schema.h> +#include <yql/essentials/providers/common/schema/mkql/yql_mkql_schema.h> +#include <yql/essentials/providers/common/codec/yql_codec.h> +#include <yql/essentials/providers/common/codec/yql_codec_type_flags.h> +#include <yql/essentials/providers/result/expr_nodes/yql_res_expr_nodes.h> +#include <yt/yql/providers/yt/common/yql_names.h> +#include <yt/yql/providers/yt/common/yql_configuration.h> +#include <yt/yql/providers/yt/lib/mkql_helpers/mkql_helpers.h> +#include <yt/yql/providers/yt/lib/res_pull/res_or_pull.h> +#include <yt/yql/providers/yt/lib/yson_helpers/yson_helpers.h> +#include <yt/yql/providers/yt/lib/lambda_builder/lambda_builder.h> +#include <yt/yql/providers/yt/lib/infer_schema/infer_schema.h> +#include <yt/yql/providers/yt/lib/schema/schema.h> +#include <yt/yql/providers/yt/expr_nodes/yql_yt_expr_nodes.h> +#include <yt/yql/providers/yt/provider/yql_yt_mkql_compiler.h> +#include <yt/yql/providers/yt/provider/yql_yt_helpers.h> +#include <yt/yql/providers/yt/gateway/lib/yt_helpers.h> +#include <yt/yql/providers/yt/gateway/lib/query_cache.h> +#include <yt/yql/providers/yt/comp_nodes/yql_mkql_table.h> +#include <yql/essentials/core/expr_nodes/yql_expr_nodes.h> +#include <yql/essentials/core/type_ann/type_ann_core.h> +#include <yql/essentials/core/type_ann/type_ann_expr.h> +#include <yql/essentials/core/yql_graph_transformer.h> +#include <yql/essentials/minikql/mkql_function_registry.h> +#include <yql/essentials/minikql/mkql_program_builder.h> +#include <yql/essentials/minikql/mkql_program_builder.h> +#include <yql/essentials/minikql/mkql_node_visitor.h> +#include <yql/essentials/minikql/mkql_node_cast.h> +#include <yql/essentials/minikql/mkql_node_builder.h> +#include <yql/essentials/minikql/comp_nodes/mkql_factories.h> +#include <yql/essentials/minikql/computation/mkql_computation_node.h> +#include <yql/essentials/minikql/computation/mkql_computation_node_pack.h> +#include <yql/essentials/utils/threading/async_queue.h> +#include <library/cpp/random_provider/random_provider.h> +#include <library/cpp/time_provider/time_provider.h> + +#include <yt/cpp/mapreduce/interface/config.h> +#include <yt/cpp/mapreduce/common/helpers.h> +#include <library/cpp/yson/node/node_io.h> +#include <library/cpp/yson/node/node_visitor.h> +#include <library/cpp/yson/node/node_builder.h> + +#include <library/cpp/yson/writer.h> +#include <library/cpp/yson/parser.h> + +#include <util/stream/file.h> +#include <util/stream/str.h> +#include <util/system/fs.h> +#include <util/system/fstat.h> +#include <util/string/split.h> +#include <util/string/builder.h> +#include <util/string/cast.h> +#include <util/folder/path.h> +#include <util/generic/yexception.h> +#include <util/generic/xrange.h> +#include <util/generic/ptr.h> +#include <util/random/random.h> + +#include <algorithm> +#include <iterator> +#include <cmath> + + +namespace NYql { + +using namespace NCommon; +using namespace NKikimr; +using namespace NKikimr::NMiniKQL; +using namespace NNodes; +using namespace NThreading; + +namespace NFile { + +/////////////////////////////////////////////////////////////////////////////////////////////////////// + +TIntrusivePtr<IFunctionRegistry> MakeFunctionRegistry( + const IFunctionRegistry& functionRegistry, const TUserDataTable& files, + const TFileStoragePtr& fileStorage, TVector<TFileLinkPtr>& externalFiles) { + auto cloned = functionRegistry.Clone(); + for (auto& d : files) { + if (d.first.IsFile() && d.second.Usage.Test(EUserDataBlockUsage::Udf)) { + if (d.second.Type == EUserDataType::PATH) { + cloned->LoadUdfs(d.second.Data, {}, 0, d.second.CustomUdfPrefix); + } else if (fileStorage && d.second.Type == EUserDataType::URL) { + auto externalFile = fileStorage->PutUrl(d.second.Data, ""); + externalFiles.push_back(externalFile); + cloned->LoadUdfs(externalFile->GetPath().GetPath(), {}, 0, d.second.CustomUdfPrefix); + } else { + MKQL_ENSURE(false, "Unsupported block type"); + } + } + } + return cloned; +} + + +/////////////////////////////////////////////////////////////////////////////////////////////////////// + +struct TSession { + TSession(const IYtGateway::TOpenSessionOptions& options, bool keepTempTables) + : RandomProvider_(options.RandomProvider()) + , TimeProvider_(options.TimeProvider()) + , KeepTempTables_(keepTempTables) + , InflightTempTablesLimit_(Max<ui32>()) + , ConfigInitDone_(false) + { + } + + ~TSession() { + if (!KeepTempTables_) { + for (auto& x : TempTables_) { + for (auto& path: x.second) { + try { + NFs::Remove(path); + NFs::Remove(path + ".attr"); + } catch (...) { + } + } + } + } + } + + void DeleteAtFinalize(const TYtSettings::TConstPtr& config, const TString& cluster, const TString& table) { + if (!ConfigInitDone_) { + InflightTempTablesLimit_ = config->InflightTempTablesLimit.Get().GetOrElse(Max<ui32>()); + if (GetReleaseTempDataMode(*config) == EReleaseTempDataMode::Never) { + KeepTempTables_ = true; + } + ConfigInitDone_ = true; + } + + auto& tempTables = TempTables_[cluster]; + tempTables.insert(table); + if (tempTables.size() > InflightTempTablesLimit_) { + ythrow yexception() << "Too many temporary tables registered - limit is " << InflightTempTablesLimit_; + } + } + + void CancelDeleteAtFinalize(const TString& cluster, const TString& table) { + TempTables_[cluster].erase(table); + } + + const TIntrusivePtr<IRandomProvider> RandomProvider_; + const TIntrusivePtr<ITimeProvider> TimeProvider_; + bool KeepTempTables_; + ui32 InflightTempTablesLimit_; + bool ConfigInitDone_; + + THashMap<TString, THashSet<TString>> TempTables_; +}; + +/////////////////////////////////////////////////////////////////////////////////////////////////////// + +struct TFileYtLambdaBuilder: public TLambdaBuilder { + TFileYtLambdaBuilder(TScopedAlloc& alloc, const TSession& /*session*/, + TIntrusivePtr<IFunctionRegistry> customFunctionRegistry, + const NUdf::ISecureParamsProvider* secureParamsProvider) + : TLambdaBuilder(customFunctionRegistry.Get(), alloc, nullptr, CreateDeterministicRandomProvider(1), CreateDeterministicTimeProvider(10000000), + nullptr, nullptr, secureParamsProvider) + , CustomFunctionRegistry_(customFunctionRegistry) + {} + + TIntrusivePtr<IFunctionRegistry> CustomFunctionRegistry_; +}; + +/////////////////////////////////////////////////////////////////////////////////////////////////////// + +class TFileTransformProvider { +public: + TFileTransformProvider(const TYtFileServices::TPtr& services, const TUserDataTable& userDataBlocks) + : Services(services) + , UserDataBlocks(userDataBlocks) + , ExtraArgs(std::make_shared<THashMap<TString, TRuntimeNode>>()) + { + } + + TCallableVisitFunc operator()(TInternName name) { + if (name == "FilePath") { + return [&](NMiniKQL::TCallable& callable, const TTypeEnvironment& env) { + MKQL_ENSURE(callable.GetInputsCount() == 1, "Expected 1 arguments"); + const TString name(AS_VALUE(TDataLiteral, callable.GetInput(0))->AsValue().AsStringRef()); + auto block = TUserDataStorage::FindUserDataBlock(UserDataBlocks, name); + MKQL_ENSURE(block, "File not found: " << name); + MKQL_ENSURE(block->Type == EUserDataType::PATH || block->FrozenFile, "File is not frozen, name: " + << name << ", block type: " << block->Type); + return TProgramBuilder(env, *Services->GetFunctionRegistry()).NewDataLiteral<NUdf::EDataSlot::String>( + block->Type == EUserDataType::PATH ? block->Data : block->FrozenFile->GetPath().GetPath() + ); + }; + } + + if (name == "FolderPath") { + return [&](NMiniKQL::TCallable& callable, const TTypeEnvironment& env) { + MKQL_ENSURE(callable.GetInputsCount() == 1, "Expected 1 arguments"); + const TString name(AS_VALUE(TDataLiteral, callable.GetInput(0))->AsValue().AsStringRef()); + auto folderName = TUserDataStorage::MakeFolderName(name); + TMaybe<TString> folderPath; + for (const auto& x : UserDataBlocks) { + if (!x.first.Alias().StartsWith(folderName)) { + continue; + } + + MKQL_ENSURE(x.second.Type == EUserDataType::PATH, "FolderPath not supported for non-file data block, name: " + << x.first.Alias() << ", block type: " << x.second.Type); + auto newFolderPath = x.second.Data.substr(0, x.second.Data.size() - (x.first.Alias().size() - folderName.size())); + if (!folderPath) { + folderPath = newFolderPath; + } else { + MKQL_ENSURE(*folderPath == newFolderPath, "File " << x.second.Data << " is out of directory " << *folderPath); + } + } + + return TProgramBuilder(env, *Services->GetFunctionRegistry()).NewDataLiteral<NUdf::EDataSlot::String>(*folderPath); + }; + } + + if (name == "FileContent") { + return [&](NMiniKQL::TCallable& callable, const TTypeEnvironment& env) { + MKQL_ENSURE(callable.GetInputsCount() == 1, "Expected 1 arguments"); + const TString name(AS_VALUE(TDataLiteral, callable.GetInput(0))->AsValue().AsStringRef()); + auto block = TUserDataStorage::FindUserDataBlock(UserDataBlocks, name); + MKQL_ENSURE(block, "File not found: " << name); + const TProgramBuilder pgmBuilder(env, *Services->GetFunctionRegistry()); + if (block->Type == EUserDataType::PATH) { + auto content = TFileInput(block->Data).ReadAll(); + return pgmBuilder.NewDataLiteral<NUdf::EDataSlot::String>(content); + } + else if (block->Type == EUserDataType::RAW_INLINE_DATA) { + return pgmBuilder.NewDataLiteral<NUdf::EDataSlot::String>(block->Data); + } + else if (block->FrozenFile && block->Type == EUserDataType::URL) { + auto content = TFileInput(block->FrozenFile->GetPath().GetPath()).ReadAll(); + return pgmBuilder.NewDataLiteral<NUdf::EDataSlot::String>(content); + } else { + MKQL_ENSURE(false, "Unsupported block type"); + } + }; + } + + if (name == TYtTableIndex::CallableName()) { + return [this, name](NMiniKQL::TCallable&, const TTypeEnvironment& env) { + return GetExtraArg(TString{name.Str()}, NUdf::EDataSlot::Uint32, env); + }; + } + if (name == TYtTablePath::CallableName()) { + return [this, name](NMiniKQL::TCallable&, const TTypeEnvironment& env) { + return GetExtraArg(TString{name.Str()}, NUdf::EDataSlot::String, env); + }; + } + if (name == TYtTableRecord::CallableName()) { + return [this, name](NMiniKQL::TCallable&, const TTypeEnvironment& env) { + return GetExtraArg(TString{name.Str()}, NUdf::EDataSlot::Uint64, env); + }; + } + if (name == TYtIsKeySwitch::CallableName()) { + return [this, name](NMiniKQL::TCallable&, const TTypeEnvironment& env) { + return GetExtraArg(TString{name.Str()}, NUdf::EDataSlot::Bool, env); + }; + } + if (name == TYtRowNumber::CallableName()) { + return [this, name](NMiniKQL::TCallable&, const TTypeEnvironment& env) { + return GetExtraArg(TString{name.Str()}, NUdf::EDataSlot::Uint64, env); + }; + } + + if (name == TYtTableContent::CallableName()) { + return [name](NMiniKQL::TCallable& callable, const TTypeEnvironment& env) { + TCallableBuilder callableBuilder(env, + TStringBuilder() << TYtTableContent::CallableName() << "File", + callable.GetType()->GetReturnType(), false); + for (ui32 i: xrange(callable.GetInputsCount())) { + callableBuilder.Add(callable.GetInput(i)); + } + return TRuntimeNode(callableBuilder.Build(), false); + }; + } + + if (name == "YtTableInput") { + return [this, name](NMiniKQL::TCallable& callable, const TTypeEnvironment& env) { + TCallableBuilder callableBuilder(env, "YtTableInputFile", callable.GetType()->GetReturnType(), false); + for (ui32 i: xrange(callable.GetInputsCount())) { + callableBuilder.Add(callable.GetInput(i)); + } + callableBuilder.Add(GetExtraArg(TString(TYtTableIndex::CallableName()), NUdf::EDataSlot::Uint32, env)); + callableBuilder.Add(GetExtraArg(TString(TYtTablePath::CallableName()), NUdf::EDataSlot::String, env)); + callableBuilder.Add(GetExtraArg(TString(TYtTableRecord::CallableName()), NUdf::EDataSlot::Uint64, env)); + callableBuilder.Add(GetExtraArg(TString(TYtRowNumber::CallableName()), NUdf::EDataSlot::Uint64, env)); + return TRuntimeNode(callableBuilder.Build(), false); + }; + } + + if (name == "YtTableInputNoCtx") { + return [name](NMiniKQL::TCallable& callable, const TTypeEnvironment& env) { + TCallableBuilder callableBuilder(env, "YtTableInputNoCtxFile", callable.GetType()->GetReturnType(), false); + for (ui32 i: xrange(callable.GetInputsCount())) { + callableBuilder.Add(callable.GetInput(i)); + } + return TRuntimeNode(callableBuilder.Build(), false); + }; + } + + if (name == "YtUngroupingList") { + return [this, name](NMiniKQL::TCallable& callable, const TTypeEnvironment& env) { + TCallableBuilder callableBuilder(env, "YtUngroupingListFile", callable.GetType()->GetReturnType(), false); + for (ui32 i: xrange(callable.GetInputsCount())) { + callableBuilder.Add(callable.GetInput(i)); + } + callableBuilder.Add(GetExtraArg(TString(TYtIsKeySwitch::CallableName()), NUdf::EDataSlot::Bool, env)); + return TRuntimeNode(callableBuilder.Build(), false); + }; + } + + return TCallableVisitFunc(); + } + +private: + TRuntimeNode GetExtraArg(const TString& name, NUdf::EDataSlot slot, const TTypeEnvironment& env) { + TRuntimeNode& node = (*ExtraArgs)[name]; + if (!node) { + TCallableBuilder builder(env, "Arg", TDataType::Create(NUdf::GetDataTypeInfo(slot).TypeId, env), true); + node = TRuntimeNode(builder.Build(), false); + } + return node; + } + +private: + TYtFileServices::TPtr Services; + const TUserDataTable& UserDataBlocks; + std::shared_ptr<THashMap<TString, TRuntimeNode>> ExtraArgs; +}; + +template <typename TType> +static inline TType OptionFromString(const TStringBuf value) { + if constexpr (std::is_same_v<TString, TType>) { + return TString{value}; + } else if constexpr (std::is_same_v<NYT::TNode, TType>) { + return NYT::NodeFromYsonString(value); + } else { + return FromString<TType>(value); + } +} + +template <typename TType> +static inline const TType& NoOp(const TType& value) { + return value; +} + + +/////////////////////////////////////////////////////////////////////////////////////////////////////// + +class TYtFileGateway : public IYtGateway { +public: + TYtFileGateway(const TYtFileServices::TPtr& services, bool* emulateOutputForMultirunPtr) + : Services_(services) + , MkqlCompiler_(MakeIntrusive<NCommon::TMkqlCommonCallableCompiler>()) + , EmulateOutputForMultirunPtr(emulateOutputForMultirunPtr) + , FakeQueue_(TAsyncQueue::Make(1, "FakePool")) + { + RegisterYtMkqlCompilers(*MkqlCompiler_); + RegisterYtFileMkqlCompilers(*MkqlCompiler_); + } + + void OpenSession(TOpenSessionOptions&& options) final { + if (!Sessions.emplace(options.SessionId(), TSession(options, Services_->GetKeepTempTables())).second) { + ythrow yexception() << "Session already exists: " << options.SessionId(); + } + } + + NThreading::TFuture<void> CloseSession(TCloseSessionOptions&& options) final { + Sessions.erase(options.SessionId()); + return MakeFuture(); + } + + NThreading::TFuture<void> CleanupSession(TCleanupSessionOptions&& options) final { + Y_UNUSED(options); + return MakeFuture(); + } + + template<typename T> + TSession* GetSession(const T& options) { + const auto session = Sessions.FindPtr(options.SessionId()); + YQL_ENSURE(session); + return session; + } + + template<typename T> + const TSession* GetSession(const T& options) const { + const auto session = Sessions.FindPtr(options.SessionId()); + YQL_ENSURE(session); + return session; + } + + TFuture<TFinalizeResult> Finalize(TFinalizeOptions&& /*options*/) final { + TFinalizeResult res; + res.SetSuccess(); + return MakeFuture(res); + } + + TFuture<TCanonizePathsResult> CanonizePaths(TCanonizePathsOptions&& options) final { + TCanonizePathsResult res; + std::transform( + options.Paths().begin(), options.Paths().end(), + std::back_inserter(res.Data), + [] (const TCanonizeReq& req) { + return CanonizedPath(req.Path()); + }); + res.SetSuccess(); + return MakeFuture(res); + } + + bool ShouldEmulateOutputForMultirun(const TTableReq& req) { + return EmulateOutputForMultirunPtr && *EmulateOutputForMultirunPtr && + req.Cluster() == "plato" && req.Table() == "Output"; + } + + TFuture<TTableInfoResult> GetTableInfo(TGetTableInfoOptions&& options) final { + TTableInfoResult res; + try { + for (const TTableReq& req: options.Tables()) { + auto path = Services_->GetTablePath(req.Cluster(), req.Table(), req.Anonymous(), true); + const bool exists = NFs::Exists(path) && !ShouldEmulateOutputForMultirun(req); + + res.Data.emplace_back(); + + res.Data.back().WriteLock = HasModifyIntents(req.Intents()); + + TYtTableMetaInfo::TPtr metaData = new TYtTableMetaInfo; + metaData->DoesExist = exists; + if (exists) { + try { + LoadTableMetaInfo(req, path, *metaData); + } catch (const yexception& e) { + throw yexception() << "Error loading " << req.Cluster() << '.' << req.Table() << " table metadata: " << e.what(); + } + } + res.Data.back().Meta = metaData; + + if (exists) { + TYtTableStatInfo::TPtr statData = new TYtTableStatInfo; + statData->Id = req.Table(); + if (metaData->SqlView.empty()) { + try { + LoadTableStatInfo(path, *statData); + } catch (const yexception& e) { + throw yexception() << "Error loading " << req.Cluster() << '.' << req.Table() << " table stat: " << e.what(); + } + + auto fullTableName = TString(YtProviderName).append('.').append(req.Cluster()).append('.').append(req.Table()); + Services_->LockPath(path, fullTableName); + } + res.Data.back().Stat = statData; + } + } + + res.SetSuccess(); + } catch (const yexception& e) { + res = NCommon::ResultFromException<TTableInfoResult>(e); + } + return MakeFuture(res); + } + + TFuture<TTableRangeResult> GetTableRange(TTableRangeOptions&& options) final { + auto pos = options.Pos(); + try { + TSession* session = GetSession(options); + + TSet<TString> uniqueTables; + const auto fullPrefix = options.Prefix().empty() ? TString() : (options.Prefix() + '/'); + const auto fullSuffix = options.Suffix().empty() ? TString() : ('/' + options.Suffix()); + for (const auto& [tableName, _] : Services_->GetTablesMapping()) { + TVector<TString> parts; + Split(tableName, ".", parts); + if (parts.size() != 3) { + continue; + } + if (parts[0] != YtProviderName || parts[1] != options.Cluster()) { + continue; + } + if (!parts[2].StartsWith(fullPrefix)) { + continue; + } + if (!parts[2].EndsWith(fullSuffix)) { + continue; + } + uniqueTables.insert(parts[2]); + } + + TTableRangeResult res; + res.SetSuccess(); + + if (!uniqueTables.empty()) { + if (auto filter = options.Filter()) { + auto exprCtx = options.ExprCtx(); + YQL_ENSURE(exprCtx); + TScopedAlloc alloc(__LOCATION__, TAlignedPagePoolCounters(), + Services_->GetFunctionRegistry()->SupportsSizedAllocators()); + alloc.SetLimit(options.Config()->DefaultCalcMemoryLimit.Get().GetOrElse(0)); + auto secureParamsProvider = MakeSimpleSecureParamsProvider(options.SecureParams()); + TVector<TFileLinkPtr> externalFiles; + TFileYtLambdaBuilder builder(alloc, *session, + MakeFunctionRegistry(*Services_->GetFunctionRegistry(), options.UserDataBlocks(), Services_->GetFileStorage(), externalFiles), secureParamsProvider.get()); + TProgramBuilder pgmBuilder(builder.GetTypeEnvironment(), *Services_->GetFunctionRegistry()); + + TVector<TRuntimeNode> strings; + for (auto& tableName: uniqueTables) { + auto stripped = TStringBuf(tableName); + stripped.SkipPrefix(fullPrefix); + stripped.ChopSuffix(fullSuffix); + strings.push_back(pgmBuilder.NewDataLiteral<NUdf::EDataSlot::String>(TString(stripped))); + } + + auto inputNode = pgmBuilder.AsList(strings); + auto data = pgmBuilder.Filter(inputNode, [&](TRuntimeNode item) { + TMkqlBuildContext ctx(*MkqlCompiler_, pgmBuilder, *exprCtx, filter->UniqueId(), {{&filter->Head().Head(), item}}); + return pgmBuilder.Coalesce(MkqlBuildExpr(filter->Tail(), ctx), pgmBuilder.NewDataLiteral(false)); + }); + + data = builder.TransformAndOptimizeProgram(data, TFileTransformProvider(Services_, options.UserDataBlocks())); + TExploringNodeVisitor explorer; + auto nodeFactory = GetYtFileFullFactory(Services_); + auto compGraph = builder.BuildGraph(nodeFactory, options.UdfValidateMode(), + NUdf::EValidatePolicy::Exception, options.OptLLVM(), EGraphPerProcess::Multi, explorer, data); + compGraph->Prepare(); + const TBindTerminator bind(compGraph->GetTerminator()); + const auto& value = compGraph->GetValue(); + const auto it = value.GetListIterator(); + for (NUdf::TUnboxedValue current; it.Next(current);) { + TString tableName = TString(current.AsStringRef()); + tableName.prepend(fullPrefix); + tableName.append(fullSuffix); + res.Tables.push_back(TCanonizedPath{std::move(tableName), Nothing(), {}, Nothing()}); + } + } + else { + std::transform( + uniqueTables.begin(), uniqueTables.end(), + std::back_inserter(res.Tables), + [] (const TString& path) { + return TCanonizedPath{path, Nothing(), {}, Nothing()}; + }); + } + } + + return MakeFuture(res); + } catch (const yexception& e) { + return MakeFuture(NCommon::ResultFromException<TTableRangeResult>(e, pos)); + } + } + + TFuture<TFolderResult> GetFolder(TFolderOptions&& options) final { + auto pos = options.Pos(); + try { + TSet<TString> uniqueTables; + const auto fullPrefix = options.Prefix().empty() ? "" : (options.Prefix() + '/'); + for (const auto& [tableName, _] : Services_->GetTablesMapping()) { + TVector<TString> parts; + Split(tableName, ".", parts); + if (parts.size() != 3) { + continue; + } + if (parts[0] != YtProviderName || parts[1] != options.Cluster()) { + continue; + } + if (!parts[2].StartsWith(fullPrefix)) { + continue; + } + uniqueTables.insert(parts[2]); + } + + TVector<TFolderResult::TFolderItem> items; + TFolderResult res; + res.SetSuccess(); + + for (auto& table : uniqueTables) { + TFolderResult::TFolderItem item; + item.Path = table; + item.Type = "table"; + auto allAttrs = LoadTableAttrs(Services_->GetTablePath(options.Cluster(), table, false, true)); + auto attrs = NYT::TNode::CreateMap(); + for (const auto& attrName : options.Attributes()) { + if (attrName && allAttrs.HasKey(attrName)) { + attrs[attrName] = allAttrs[attrName]; + } + } + + item.Attributes = NYT::NodeToYsonString(attrs); + items.push_back(std::move(item)); + } + res.ItemsOrFileLink = std::move(items); + return MakeFuture(res); + } catch (const yexception& e) { + return MakeFuture(NCommon::ResultFromException<TFolderResult>(e, pos)); + } + } + + TFuture<TBatchFolderResult> ResolveLinks(TResolveOptions&& options) final { + TBatchFolderResult res; + res.SetSuccess(); + for (auto&& [item, reqAttrs] : options.Items()) { + if (item.Type != "link") { + res.Items.push_back(item); + } + else { + if (item.Attributes.HasKey("broken") || item.Attributes["broken"].AsBool()) { + continue; + } + const TStringBuf targetPath = item.Attributes["target_path"].AsString(); + const auto folder = targetPath.RBefore('/'); + const auto folderContent = GetFolder(TFolderOptions(options.SessionId()) + .Attributes(reqAttrs) + .Cluster(options.Cluster()) + .Prefix(TString(folder)) + .Config(options.Config()) + .Pos(options.Pos())).GetValue(); + + if (std::holds_alternative<TFileLinkPtr>(folderContent.ItemsOrFileLink)) { + Y_ENSURE(false, "File link result from file gateway GetFolder() is unexpected"); + } + for (const auto& item: std::get<TVector<TFolderResult::TFolderItem>>(folderContent.ItemsOrFileLink)) { + if (item.Path == targetPath) { + res.Items.push_back({item.Path, item.Type, NYT::NodeFromYsonString(item.Attributes)}); + break; + } + } + } + } + return MakeFuture(res); + } + + TFuture<TBatchFolderResult> GetFolders(TBatchFolderOptions&& options) final { + TBatchFolderResult res; + res.SetSuccess(); + for (const auto& folder : options.Folders()) { + TFolderOptions folderOptions(options.SessionId()); + folderOptions.Attributes(folder.AttrKeys) + .Cluster(options.Cluster()) + .Prefix(folder.Prefix) + .Config(options.Config()) + .Pos(options.Pos()); + const auto folderContent = GetFolder(TFolderOptions(std::move(folderOptions))).GetValue(); + if (std::holds_alternative<TFileLinkPtr>(folderContent.ItemsOrFileLink)) { + Y_ENSURE(false, "File link result from file gateway GetFolder() is unexpected"); + } + for (const auto& item: std::get<TVector<TFolderResult::TFolderItem>>(folderContent.ItemsOrFileLink)) { + res.Items.push_back({item.Path, item.Type, NYT::NodeFromYsonString(item.Attributes)}); + } + } + return MakeFuture(res); + } + + TFuture<TResOrPullResult> ResOrPull(const TExprNode::TPtr& node, TExprContext& ctx, TResOrPullOptions&& options) final { + TResOrPullResult res; + auto nodePos = ctx.GetPosition(node->Pos()); + try { + TSession* session = GetSession(options); + TVector<TString> columns(NCommon::GetResOrPullColumnHints(*node)); + if (columns.empty()) { + columns = NCommon::GetStructFields(node->Child(0)->GetTypeAnn()); + } + + TStringStream out; + NYson::TYsonWriter writer(&out, NCommon::GetYsonFormat(options.FillSettings()), ::NYson::EYsonType::Node, false); + writer.OnBeginMap(); + if (NCommon::HasResOrPullOption(*node, "type")) { + writer.OnKeyedItem("Type"); + NCommon::WriteResOrPullType(writer, node->Child(0)->GetTypeAnn(), TColumnOrder(columns)); + } + + bool truncated = false; + if (TStringBuf("Result") == node->Content()) { + truncated = ExecuteResult(*session, writer, NNodes::TResult(node).Input(), ctx, std::move(options), columns); + } else if (TStringBuf("Pull") == node->Content()) { + truncated = ExecutePull(*session, writer, NNodes::TPull(node), ctx, std::move(options), columns); + } else { + ythrow yexception() << "Don't know how to execute " << node->Content(); + } + + if (truncated) { + writer.OnKeyedItem("Truncated"); + writer.OnBooleanScalar(true); + } + + writer.OnEndMap(); + res.Data = out.Str(); + res.SetSuccess(); + } catch (const yexception& e) { + res = NCommon::ResultFromException<TResOrPullResult>(e, nodePos); + } + + return MakeFuture(res); + } + + TFuture<TRunResult> Run(const TExprNode::TPtr& node, TExprContext& ctx, TRunOptions&& options) final { + TRunResult res; + auto nodePos = ctx.GetPosition(node->Pos()); + if (auto reduce = TMaybeNode<TYtReduce>(node)) { + auto maxDataSizePerJob = NYql::GetMaxJobSizeForFirstAsPrimary(reduce.Cast().Settings().Ref()); + // YT wants max_data_size_per_job > 0 + if (maxDataSizePerJob && *maxDataSizePerJob <= 1) { + TIssue rootIssue = YqlIssue(nodePos, TIssuesIds::YT_MAX_DATAWEIGHT_PER_JOB_EXCEEDED); + res.SetStatus(TIssuesIds::UNEXPECTED); + res.AddIssue(rootIssue); + return MakeFuture(res); + } + } + + try { + TSession* session = GetSession(options); + if (TYtTouch::Match(node.Get())) { + res.OutTableStats = ExecuteTouch(options.Config(), *session, TYtTouch(node)); + res.SetSuccess(); + } + else if (TYtOutputOpBase::Match(node.Get())) { + res.OutTableStats = ExecuteOpWithOutput(*session, node, ctx, std::move(options)); + res.SetSuccess(); + } + else if (TYtDropTable::Match(node.Get())) { + ExecuteDrop(node); + res.SetSuccess(); + } + else { + res.AddIssue(TIssue(nodePos, TStringBuilder() << "Unsupported function: " << node->Content())); + } + } + catch (const TNodeException& e) { + res.SetException(e, ctx.GetPosition(e.Pos())); + } + catch (const yexception& e) { + res.SetException(e, nodePos); + } + return MakeFuture(res); + } + + + TFuture<TRunResult> Prepare(const TExprNode::TPtr& node, TExprContext& ctx, TPrepareOptions&& options) const final { + TRunResult res; + auto nodePos = ctx.GetPosition(node->Pos()); + + try { + auto session = GetSession(options); + res.OutTableStats = ExecutePrepare(options, *session, TYtOutputOpBase(node)); + res.SetSuccess(); + } + catch (const TNodeException& e) { + res.SetException(e, ctx.GetPosition(e.Pos())); + } + catch (const yexception& e) { + res.SetException(e, nodePos); + } + return MakeFuture(res); + } + + TFuture<TRunResult> GetTableStat(const TExprNode::TPtr& node, TExprContext& ctx, TPrepareOptions&& options) override { + TRunResult res; + auto nodePos = ctx.GetPosition(node->Pos()); + + try { + auto session = GetSession(options); + const TYtOutputOpBase op(node); + const auto cluster = op.DataSink().Cluster().StringValue(); + Y_ENSURE(1U == op.Output().Size(), "Single output expected."); + const auto table = op.Output().Item(0); + + TYtOutTableInfo tableInfo(table); + auto outTablePath = Services_->GetTablePath(cluster, tableInfo.Name, true); + TFsQueryCacheItem queryCacheItem(*options.Config(), cluster, Services_->GetTmpDir(), options.OperationHash(), outTablePath); + + NYT::TNode outSpec = NYT::TNode::CreateList(); + tableInfo.RowSpec->FillCodecNode(outSpec.Add()[YqlRowSpecAttribute]); + outSpec = NYT::TNode::CreateMap()(TString{YqlIOSpecTables}, std::move(outSpec)); + + auto content = Services_->GetTableContent(tableInfo.Name); + TScopedAlloc alloc(__LOCATION__, TAlignedPagePoolCounters(), + Services_->GetFunctionRegistry()->SupportsSizedAllocators()); + TMemoryUsageInfo memInfo("Stat"); + TTypeEnvironment env(alloc); + NKikimr::NMiniKQL::TTypeBuilder typeBuilder(env); + THolderFactory holderFactory(alloc.Ref(), memInfo, Services_->GetFunctionRegistry()); + + NCommon::TCodecContext codecCtx(env, *Services_->GetFunctionRegistry(), &holderFactory); + TMkqlIOSpecs spec; + spec.Init(codecCtx, outSpec); + + TStringStream out; + TMkqlWriterImpl writer(out, 0, 4_MB); + writer.SetSpecs(spec); + + TStringStream err; + auto type = BuildType(*tableInfo.RowSpec->GetExtendedType(ctx), typeBuilder, err); + TValuePacker packer(true, type); + for (auto& c: content) { + auto val = packer.Unpack(c, holderFactory); + writer.AddRow(val); + } + writer.Finish(); + + WriteOutTable(options.Config(), *session, cluster, tableInfo, out.Str()); + queryCacheItem.Store(); + + auto statInfo = MakeIntrusive<TYtTableStatInfo>(); + LoadTableStatInfo(outTablePath, *statInfo); + statInfo->Id = tableInfo.Name; + + res.OutTableStats.emplace_back(statInfo->Id, statInfo); + res.SetSuccess(); + } + catch (const TNodeException& e) { + res.SetException(e, ctx.GetPosition(e.Pos())); + } + catch (const yexception& e) { + res.SetException(e, nodePos); + } + return MakeFuture(res); + } + + + TFuture<TCalcResult> Calc(const TExprNode::TListType& nodes, TExprContext& ctx, TCalcOptions&& options) final { + TCalcResult res; + Y_UNUSED(ctx); + // TODO: fixme + try { + TSession* session = GetSession(options); + TScopedAlloc alloc(__LOCATION__, TAlignedPagePoolCounters(), + Services_->GetFunctionRegistry()->SupportsSizedAllocators()); + alloc.SetLimit(options.Config()->DefaultCalcMemoryLimit.Get().GetOrElse(0)); + auto secureParamsProvider = MakeSimpleSecureParamsProvider(options.SecureParams()); + TVector<TFileLinkPtr> externalFiles; + TFileYtLambdaBuilder builder(alloc, *session, + MakeFunctionRegistry(*Services_->GetFunctionRegistry(), options.UserDataBlocks(), Services_->GetFileStorage(), externalFiles), secureParamsProvider.get()); + auto nodeFactory = GetYtFileFullFactory(Services_); + for (auto& node: nodes) { + auto data = builder.BuildLambda(*MkqlCompiler_, node, ctx); + auto transform = TFileTransformProvider(Services_, options.UserDataBlocks()); + data = builder.TransformAndOptimizeProgram(data, transform); + TExploringNodeVisitor explorer; + auto compGraph = builder.BuildGraph(nodeFactory, options.UdfValidateMode(), + NUdf::EValidatePolicy::Exception, options.OptLLVM(), EGraphPerProcess::Multi, explorer, data, {data.GetNode()}); + const TBindTerminator bind(compGraph->GetTerminator()); + compGraph->Prepare(); + auto value = compGraph->GetValue(); + res.Data.push_back(NCommon::ValueToNode(value, data.GetStaticType())); + } + res.SetSuccess(); + } catch (const yexception& e) { + res = NCommon::ResultFromException<TCalcResult>(e); + } + return MakeFuture(res); + } + + TFuture<TPublishResult> Publish(const TExprNode::TPtr& node, TExprContext& exprCtx, TPublishOptions&& options) final { + TPublishResult res; + try { + TSession* session = GetSession(options); + + auto publish = TYtPublish(node); + + EYtWriteMode mode = EYtWriteMode::Renew; + if (const auto modeSetting = NYql::GetSetting(publish.Settings().Ref(), EYtSettingType::Mode)) { + mode = FromString<EYtWriteMode>(modeSetting->Child(1)->Content()); + } + + bool append = mode == EYtWriteMode::Append; + auto cluster = TString{publish.DataSink().Cluster().Value()}; + + bool isAnonymous = NYql::HasSetting(publish.Publish().Settings().Ref(), EYtSettingType::Anonymous); + auto destFilePath = Services_->GetTablePath(cluster, publish.Publish().Name().Value(), isAnonymous, true); + + append = append && NFs::Exists(destFilePath); + + TScopedAlloc alloc(__LOCATION__, TAlignedPagePoolCounters(), + Services_->GetFunctionRegistry()->SupportsSizedAllocators()); + alloc.SetLimit(options.Config()->DefaultCalcMemoryLimit.Get().GetOrElse(0)); + TVector<TFileLinkPtr> externalFiles; + TFileYtLambdaBuilder builder(alloc, *session, + MakeFunctionRegistry(*Services_->GetFunctionRegistry(), {}, Services_->GetFileStorage(), externalFiles), nullptr); + + TProgramBuilder pgmBuilder(builder.GetTypeEnvironment(), builder.GetFunctionRegistry()); + TMkqlBuildContext ctx(*MkqlCompiler_, pgmBuilder, exprCtx); + + auto dstRowSpec = options.DestinationRowSpec(); + NYT::TNode spec; + dstRowSpec->FillCodecNode(spec[YqlRowSpecAttribute]); + + std::vector<std::pair<std::string_view, TType*>> members; + members.reserve(dstRowSpec->GetType()->GetItems().size()); + for (auto& item : dstRowSpec->GetType()->GetItems()) { + members.emplace_back(item->GetName(), NCommon::BuildType(publish.Ref(), *item->GetItemType(), pgmBuilder)); + } + for (size_t i: xrange(dstRowSpec->SortedBy.size())) { + if (!dstRowSpec->GetType()->FindItem(dstRowSpec->SortedBy[i])) { + members.emplace_back(dstRowSpec->SortedBy[i], NCommon::BuildType(publish.Ref(), *dstRowSpec->SortedByTypes[i], pgmBuilder)); + } + } + auto srcType = pgmBuilder.NewStructType(members); + + TVector<TRuntimeNode> inputs; + if (append) { + inputs.push_back(BuildRuntimeTableInput("YtTableInputNoCtx", srcType, cluster, publish.Publish().Name().Value(), NYT::NodeToYsonString(spec), isAnonymous, ctx)); + } + + for (auto out: publish.Input()) { + inputs.push_back(BuildTableContentCall("YtTableInputNoCtx", srcType, cluster, out.Ref(), Nothing(), ctx, false)); + } + + auto data = pgmBuilder.Extend(inputs); + if (inputs.size() > 1 && dstRowSpec->IsSorted()) { + data = SortListBy(data, dstRowSpec->GetForeignSort(), ctx); + } + data = BuildTableOutput(data, ctx); + + auto transform = TFileTransformProvider(Services_, {}); + data = builder.TransformAndOptimizeProgram(data, transform); + + TExploringNodeVisitor explorer; + auto nodeFactory = GetYtFileFullFactory(Services_); + auto compGraph = builder.BuildGraph(nodeFactory, NUdf::EValidateMode::None, + NUdf::EValidatePolicy::Exception, options.OptLLVM(), EGraphPerProcess::Multi, explorer, data, {data.GetNode()}); + const TBindTerminator bind(compGraph->GetTerminator()); + compGraph->Prepare(); + + NYT::TNode outSpec = NYT::TNode::CreateMap()(TString{YqlIOSpecTables}, NYT::TNode::CreateList().Add(spec)); + + TVector<TString> outTableContent = GetFileWriteResult( + builder.GetTypeEnvironment(), + builder.GetFunctionRegistry(), + compGraph->GetContext(), + compGraph->GetValue(), + outSpec); + YQL_ENSURE(1 == outTableContent.size()); + + { + TMemoryInput in(outTableContent.front()); + TOFStream of(destFilePath); + TDoubleHighPrecisionYsonWriter writer(&of, ::NYson::EYsonType::ListFragment); + NYson::TYsonParser parser(&writer, &in, ::NYson::EYsonType::ListFragment); + parser.Parse(); + } + + { + NYT::TNode attrs = NYT::TNode::CreateMap(); + TString srcFilePath = Services_->GetTablePath(cluster, GetOutTable(publish.Input().Item(0)).Cast<TYtOutTable>().Name().Value(), true); + if (NFs::Exists(srcFilePath + ".attr")) { + TIFStream input(srcFilePath + ".attr"); + attrs = NYT::NodeFromYsonStream(&input); + } + + const auto nativeYtTypeCompatibility = options.Config()->NativeYtTypeCompatibility.Get(cluster).GetOrElse(NTCF_LEGACY); + const bool rowSpecCompactForm = options.Config()->UseYqlRowSpecCompactForm.Get().GetOrElse(DEFAULT_ROW_SPEC_COMPACT_FORM); + dstRowSpec->FillAttrNode(attrs[YqlRowSpecAttribute], nativeYtTypeCompatibility, rowSpecCompactForm); + NYT::TNode columnGroupsSpec; + if (options.Config()->OptimizeFor.Get(cluster).GetOrElse(NYT::OF_LOOKUP_ATTR) != NYT::OF_LOOKUP_ATTR) { + if (auto setting = NYql::GetSetting(publish.Settings().Ref(), EYtSettingType::ColumnGroups)) { + columnGroupsSpec = NYT::NodeFromYsonString(setting->Tail().Content()); + } + } + if (!append || !attrs.HasKey("schema") || !columnGroupsSpec.IsUndefined() || dstRowSpec->IsSorted()) { + attrs["schema"] = RowSpecToYTSchema(spec[YqlRowSpecAttribute], nativeYtTypeCompatibility, columnGroupsSpec).ToNode(); + } + + if (EYtWriteMode::Renew == mode || EYtWriteMode::RenewKeepMeta == mode) { + bool isTimestamp = false, isDuration = false; + TInstant stamp; + TDuration duration; + if (auto e = NYql::GetSetting(publish.Settings().Ref(), EYtSettingType::Expiration)) { + isDuration = TDuration::TryParse(e->Tail().Content(), duration); + if (!isDuration) { + isTimestamp = TInstant::TryParseIso8601(e->Tail().Content(), stamp); + } + } + const TMaybe<TInstant> deadline = options.Config()->ExpirationDeadline.Get(cluster); + const TMaybe<TDuration> interval = options.Config()->ExpirationInterval.Get(cluster); + if (deadline || isTimestamp) { + attrs["expiration_time"] = isTimestamp ? stamp.ToStringUpToSeconds() : deadline->ToStringUpToSeconds(); + } + if (interval || isDuration) { + attrs["expiration_timeout"] = isDuration ? duration.MilliSeconds() : interval->MilliSeconds(); + } + if (options.Config()->NightlyCompress.Get(cluster).GetOrElse(false)) { + attrs["force_nightly_compress"] = true; + } + } + +#define HANDLE_OPT(name, attr, conv) \ + auto dst##name = isAnonymous \ + ? options.Config()->Temporary##name.Get(cluster) \ + : options.Config()->Published##name.Get(cluster); \ + if (auto s = NYql::GetSetting(publish.Settings().Ref(), EYtSettingType::name)) { \ + dst##name = OptionFromString<decltype(dst##name)::value_type>(s->Tail().Content()); \ + } \ + if (dst##name && dst##name != options.Config()->Temporary##name.Get(cluster)) { \ + attrs[attr] = conv(*dst##name); \ + } + + HANDLE_OPT(CompressionCodec, "compression_codec", NoOp); + HANDLE_OPT(ErasureCodec, "erasure_codec", ToString); + HANDLE_OPT(ReplicationFactor, "replication_factor", static_cast<i64>); + HANDLE_OPT(Media, "media", NoOp); + HANDLE_OPT(PrimaryMedium, "primary_medium", NoOp); +#undef DEFINE_OPT + + if (auto optimizeFor = options.Config()->OptimizeFor.Get(cluster)) { + if (dstRowSpec->GetType()->GetSize()) { + attrs["optimize_for"] = ToString(*optimizeFor); + } + } + + if (auto ua = NYql::GetSetting(publish.Settings().Ref(), EYtSettingType::UserAttrs)) { + const NYT::TNode mapNode = NYT::NodeFromYsonString(ua->Tail().Content()); + const auto& map = mapNode.AsMap(); + for (auto it = map.cbegin(); it != map.cend(); ++it) { + attrs[it->first] = it->second; + } + } + + TOFStream ofAttr(destFilePath + ".attr"); + ofAttr.Write(NYT::NodeToYsonString(attrs, NYson::EYsonFormat::Pretty)); + } + + if (isAnonymous) { + session->DeleteAtFinalize(options.Config(), cluster, destFilePath); + } + + res.SetSuccess(); + } + catch (const TNodeException& e) { + res.SetException(e, exprCtx.GetPosition(e.Pos())); + } + catch (const yexception& e) { + res.SetException(e, exprCtx.GetPosition(node->Pos())); + } + return MakeFuture(res); + } + + TFuture<TCommitResult> Commit(TCommitOptions&& options) final { + Y_UNUSED(options); + TCommitResult res; + res.SetSuccess(); + return MakeFuture(res); + } + + TFuture<TDropTrackablesResult> DropTrackables(TDropTrackablesOptions&& options) final { + TDropTrackablesResult res; + try { + TSession* session = GetSession(options); + // check for overrides from command line + if (session->KeepTempTables_) { + res.SetSuccess(); + return MakeFuture(res); + } + + for (const auto& i : options.Pathes()) { + + const TString& cluster = i.Cluster; + const TString& path = i.Path; + + auto tmpPath = Services_->GetTablePath(cluster, path, true); + + session->CancelDeleteAtFinalize(cluster, tmpPath); + + NFs::Remove(tmpPath); + NFs::Remove(tmpPath + ".attr"); + } + res.SetSuccess(); + } + catch (const yexception& e) { + res.SetException(e); + } + return MakeFuture(res); + } + + TFuture<TPathStatResult> PathStat(TPathStatOptions&& options) final { + bool onlyCached = false; + return MakeFuture(DoPathStat(std::move(options), onlyCached)); + } + + TPathStatResult TryPathStat(TPathStatOptions&& options) final { + bool onlyCached = true; + return DoPathStat(std::move(options), onlyCached); + } + + bool TryParseYtUrl(const TString& url, TString* cluster, TString* path) const final { + Y_UNUSED(url); + Y_UNUSED(cluster); + Y_UNUSED(path); + return false; + } + + TString GetClusterServer(const TString& cluster) const final { + return cluster; + } + + NYT::TRichYPath GetRealTable(const TString& sessionId, const TString& cluster, const TString& table, ui32 epoch, const TString& tmpFolder) const final { + Y_UNUSED(sessionId); + Y_UNUSED(cluster); + Y_UNUSED(epoch); + Y_UNUSED(tmpFolder); + return NYT::TRichYPath().Path(table); + } + + NYT::TRichYPath GetWriteTable(const TString& sessionId, const TString& cluster, const TString& table, const TString& tmpFolder) const final { + Y_UNUSED(sessionId); + Y_UNUSED(cluster); + auto realTableName = NYql::TransformPath(tmpFolder, table, true, ""); + realTableName = NYT::AddPathPrefix(realTableName, NYT::TConfig::Get()->Prefix); + NYT::TRichYPath res{realTableName}; + res.TransactionId(TGUID()); + return res; + } + + TFuture<TDownloadTablesResult> DownloadTables(TDownloadTablesOptions&& options) final { + Y_UNUSED(options); + return MakeFuture<TDownloadTablesResult>(); + } + + TFuture<TUploadTableResult> UploadTable(TUploadTableOptions&& options) final { + Y_UNUSED(options); + return MakeFuture<TUploadTableResult>(); + } + + TFullResultTableResult PrepareFullResultTable(TFullResultTableOptions&& options) final { + try { + TString cluster = options.Cluster(); + auto outTable = options.OutTable(); + TSession* session = GetSession(options); + + NYT::TNode attrs = NYT::TNode::CreateMap(); + for (auto& a: options.OutTable().Meta->Attrs) { + attrs[a.first] = a.second; + } + const auto nativeYtTypeCompatibility = options.Config()->NativeYtTypeCompatibility.Get(TString{cluster}).GetOrElse(NTCF_LEGACY); + const bool rowSpecCompactForm = options.Config()->UseYqlRowSpecCompactForm.Get().GetOrElse(DEFAULT_ROW_SPEC_COMPACT_FORM); + options.OutTable().RowSpec->FillAttrNode(attrs[YqlRowSpecAttribute], nativeYtTypeCompatibility, rowSpecCompactForm); + NYT::TNode rowSpecYson; + options.OutTable().RowSpec->FillCodecNode(rowSpecYson); + attrs["schema"] = RowSpecToYTSchema(rowSpecYson, nativeYtTypeCompatibility).ToNode(); + + NYT::TNode outSpec = NYT::TNode::CreateList(); + outSpec.Add(NYT::TNode::CreateMap()(TString{YqlRowSpecAttribute}, rowSpecYson)); + outSpec = NYT::TNode::CreateMap()(TString{YqlIOSpecTables}, std::move(outSpec)); + + TFullResultTableResult res; + + TString name = TStringBuilder() << "tmp/" << GetGuidAsString(session->RandomProvider_->GenGuid()); + TString path = Services_->GetTablePath(cluster, name, true); + + res.Server = cluster; + res.Path = path; + res.RefName = name; + res.CodecSpec = NYT::NodeToYsonString(outSpec); + res.TableAttrs = NYT::NodeToYsonString(attrs); + + res.SetSuccess(); + return res; + } catch (...) { + return ResultFromCurrentException<TFullResultTableResult>(); + } + } + + TString GetDefaultClusterName() const final { + return {}; + } + + void SetStatUploader(IStatUploader::TPtr statUploader) final { + Y_UNUSED(statUploader); + } + + void RegisterMkqlCompiler(NCommon::TMkqlCallableCompilerBase& compiler) override { + RegisterDqYtFileMkqlCompilers(compiler); + } + + TGetTablePartitionsResult GetTablePartitions(TGetTablePartitionsOptions&& options) override { + const TString tmpFolder = GetTablesTmpFolder(*options.Config()); + auto res = TGetTablePartitionsResult(); + TVector<NYT::TRichYPath> paths; + for (const auto& pathInfo: options.Paths()) { + const auto tablePath = TransformPath(tmpFolder, pathInfo->Table->Name, pathInfo->Table->IsTemp, options.SessionId()); + NYT::TRichYPath richYtPath{NYT::AddPathPrefix(tablePath, NYT::TConfig::Get()->Prefix)}; + pathInfo->FillRichYPath(richYtPath); // n.b. throws exception, if there is no RowSpec (we assume it is always there) + paths.push_back(std::move(richYtPath)); + } + res.Partitions.Partitions.push_back({}); + res.Partitions.Partitions.back().TableRanges = std::move(paths); + res.SetSuccess(); + return res; + } + + void AddCluster(const TYtClusterConfig&) override { + } + +private: + static NYT::TNode LoadTableAttrs(const TString& path) { + NYT::TNode attrs = NYT::TNode::CreateMap(); + if (NFs::Exists(path + ".attr")) { + attrs = NYT::NodeFromYsonString(TIFStream(path + ".attr").ReadAll()); + }; + return attrs; + } + + static void LoadTableMetaInfo(const TTableReq& req, const TString& path, TYtTableMetaInfo& info) { + NYT::TNode attrs = LoadTableAttrs(path); + + TransferTableAttributes(attrs, [&info] (const TString& name, const TString& val) { + info.Attrs[name] = val; + }); + + if (attrs.HasKey(YqlDynamicAttribute)) { + info.IsDynamic = attrs[YqlDynamicAttribute].AsBool(); + } + + if (attrs.HasKey(YqlTypeAttribute)) { + auto type = attrs[YqlTypeAttribute]; + YQL_ENSURE(type.AsString() == YqlTypeView); + info.SqlView = TIFStream(path).ReadAll(); + auto attrVer = type.Attributes()["syntax_version"]; + info.SqlViewSyntaxVersion = attrVer.IsUndefined() ? 1 : attrVer.AsInt64(); + info.CanWrite = false; + return; + } + + info.CanWrite = true; + info.YqlCompatibleScheme = ValidateTableSchema( + req.Table(), attrs, req.IgnoreYamrDsv(), req.IgnoreWeakSchema() + ); + + if (attrs.AsMap().contains("schema_mode") && attrs["schema_mode"].AsString() == "weak") { + info.Attrs["schema_mode"] = attrs["schema_mode"].AsString(); + } + + NYT::TNode schemaAttrs; + if (req.ForceInferSchema() && req.InferSchemaRows() > 0) { + info.Attrs.erase(YqlRowSpecAttribute); + if (!req.Intents().HasFlags( + TYtTableIntent::Override | TYtTableIntent::Append | TYtTableIntent::Drop | TYtTableIntent::Flush)) { + auto list = LoadTableContent(path); + if (!list.AsList().empty()) { + auto inferedSchemaAttrs = GetSchemaFromAttributes(attrs, true, req.IgnoreWeakSchema()); + inferedSchemaAttrs[INFER_SCHEMA_ATTR_NAME] = InferSchemaFromSample(list, req.Table(), req.InferSchemaRows()); + info.InferredScheme = true; + schemaAttrs = std::move(inferedSchemaAttrs); + } + } + } else { + if (info.YqlCompatibleScheme) { + schemaAttrs = GetSchemaFromAttributes(attrs, false, req.IgnoreWeakSchema()); + } + else if (!info.Attrs.contains(YqlRowSpecAttribute) + && req.InferSchemaRows() > 0 + && !req.Intents().HasFlags(TYtTableIntent::Override | TYtTableIntent::Append | TYtTableIntent::Drop | TYtTableIntent::Flush)) { + auto list = LoadTableContent(path); + if (!list.AsList().empty()) { + schemaAttrs[INFER_SCHEMA_ATTR_NAME] = InferSchemaFromSample(list, req.Table(), req.InferSchemaRows()); + info.InferredScheme = true; + } + } + } + + if (!schemaAttrs.IsUndefined()) { + for (auto& item: schemaAttrs.AsMap()) { + info.Attrs[item.first] = NYT::NodeToYsonString(item.second, NYson::EYsonFormat::Text); + } + } + } + + static void LoadTableStatInfo(const TString& path, TYtTableStatInfo& info) { + NYT::TNode inputList = LoadTableContent(path); + info.RecordsCount = inputList.AsList().size(); + if (!info.IsEmpty()) { + info.DataSize = TFileStat(path).Size; + info.ChunkCount = 1; + } + } + + static NYT::TNode LoadTableContent(const TString& path) { + NYT::TNode inputList = NYT::TNode::CreateList(); + if (TFileStat(path).Size) { + TIFStream input(path); + NYT::TNodeBuilder builder(&inputList); + NYson::TYsonParser parser(&builder, &input, ::NYson::EYsonType::ListFragment); + parser.Parse(); + } + return inputList; + } + + bool ExecuteResult(TSession& session, NYson::TYsonWriter& writer, TExprBase input, TExprContext& exprCtx, + TResOrPullOptions&& options, const TVector<TString>& columns) const + { + TScopedAlloc alloc(__LOCATION__, TAlignedPagePoolCounters(), + Services_->GetFunctionRegistry()->SupportsSizedAllocators()); + alloc.SetLimit(options.Config()->DefaultCalcMemoryLimit.Get().GetOrElse(0)); + auto secureParamsProvider = MakeSimpleSecureParamsProvider(options.SecureParams()); + TVector<TFileLinkPtr> externalFiles; + TFileYtLambdaBuilder builder(alloc, session, + MakeFunctionRegistry(*Services_->GetFunctionRegistry(), options.UserDataBlocks(), Services_->GetFileStorage(), externalFiles), secureParamsProvider.get()); + auto data = builder.BuildLambda(*MkqlCompiler_, input.Ptr(), exprCtx); + auto transform = TFileTransformProvider(Services_, options.UserDataBlocks()); + data = builder.TransformAndOptimizeProgram(data, transform); + + TExploringNodeVisitor explorer; + auto nodeFactory = GetYtFileFullFactory(Services_); + auto compGraph = builder.BuildGraph(nodeFactory, options.UdfValidateMode(), + NUdf::EValidatePolicy::Exception, options.OptLLVM(), EGraphPerProcess::Multi, explorer, data, {data.GetNode()}); + const TBindTerminator bind(compGraph->GetTerminator()); + compGraph->Prepare(); + + TYsonExecuteResOrPull resultData(options.FillSettings().RowsLimitPerWrite, + options.FillSettings().AllResultsBytesLimit, MakeMaybe(columns)); + + resultData.WriteValue(compGraph->GetValue(), data.GetStaticType()); + auto dataRes = resultData.Finish(); + + writer.OnKeyedItem("Data"); + writer.OnRaw(options.FillSettings().Discard ? "#" : dataRes); + + return resultData.IsTruncated(); + } + + bool ExecutePull(TSession& session, NYson::TYsonWriter& writer, TPull pull, TExprContext& exprCtx, + TResOrPullOptions&& options, const TVector<TString>& columns) const + { + bool truncated = false; + bool writeRef = NCommon::HasResOrPullOption(pull.Ref(), "ref"); + + if (!writeRef) { + truncated = ExecuteResult(session, writer, pull, exprCtx, std::move(options), columns); + writeRef = truncated && NCommon::HasResOrPullOption(pull.Ref(), "autoref"); + } + + if (writeRef && !options.FillSettings().Discard) { + auto cluster = GetClusterName(pull.Input()); + writer.OnKeyedItem("Ref"); + writer.OnBeginList(); + for (auto& tableInfo: GetInputTableInfos(pull.Input())) { + writer.OnListItem(); + if (tableInfo->IsTemp) { + auto outPath = Services_->GetTablePath(cluster, tableInfo->Name, true); + session.CancelDeleteAtFinalize(TString{cluster}, outPath); + } + NYql::WriteTableReference(writer, YtProviderName, cluster, tableInfo->Name, tableInfo->IsTemp, columns); + } + writer.OnEndList(); + } + + return truncated; + } + + TVector<std::pair<TString, TYtTableStatInfo::TPtr>> ExecuteOpWithOutput(TSession& session, + const TExprNode::TPtr& node, TExprContext& exprCtx, + TRunOptions&& options) const + { + TYtOutputOpBase op(node); + + auto cluster = TString{op.DataSink().Cluster().Value()}; + TVector<TString> outTablePaths; + TVector<TYtOutTableInfo> outTableInfos; + for (auto table: op.Output()) { + TString name = TStringBuilder() << "tmp/" << GetGuidAsString(session.RandomProvider_->GenGuid()); + + outTablePaths.push_back(Services_->GetTablePath(cluster, name, true)); + + outTableInfos.emplace_back(table); + outTableInfos.back().Name = name; + } + + TFsQueryCacheItem queryCacheItem(*options.Config(), cluster, Services_->GetTmpDir(), options.OperationHash(), outTablePaths); + if (!queryCacheItem.Lookup(FakeQueue_)) { + TScopedAlloc alloc(__LOCATION__, TAlignedPagePoolCounters(), + Services_->GetFunctionRegistry()->SupportsSizedAllocators()); + alloc.SetLimit(options.Config()->DefaultCalcMemoryLimit.Get().GetOrElse(0)); + auto secureParamsProvider = MakeSimpleSecureParamsProvider(options.SecureParams()); + TVector<TFileLinkPtr> externalFiles; + TFileYtLambdaBuilder builder(alloc, session, + MakeFunctionRegistry(*Services_->GetFunctionRegistry(), options.UserDataBlocks(), Services_->GetFileStorage(), externalFiles), secureParamsProvider.get()); + auto data = builder.BuildLambda(*MkqlCompiler_, node, exprCtx); + auto transform = TFileTransformProvider(Services_, options.UserDataBlocks()); + data = builder.TransformAndOptimizeProgram(data, transform); + + TExploringNodeVisitor explorer; + auto nodeFactory = GetYtFileFullFactory(Services_); + auto compGraph = builder.BuildGraph(nodeFactory, options.UdfValidateMode(), + NUdf::EValidatePolicy::Exception, options.OptLLVM(), EGraphPerProcess::Multi, explorer, data, {data.GetNode()}); + const TBindTerminator bind(compGraph->GetTerminator()); + compGraph->Prepare(); + + WriteOutTables(builder, options.Config(), session, cluster, outTableInfos, compGraph.Get()); + queryCacheItem.Store(); + } + + TVector<std::pair<TString, TYtTableStatInfo::TPtr>> outStat; + for (size_t i: xrange(outTableInfos.size())) { + TYtTableStatInfo::TPtr statInfo = MakeIntrusive<TYtTableStatInfo>(); + statInfo->Id = outTableInfos[i].Name; + LoadTableStatInfo(outTablePaths.at(i), *statInfo); + + outStat.emplace_back(statInfo->Id, statInfo); + } + return outStat; + } + + TVector<std::pair<TString, TYtTableStatInfo::TPtr>> ExecuteTouch(const TYtSettings::TConstPtr& config, TSession& session, const TYtTouch& op) const { + auto cluster = op.DataSink().Cluster().StringValue(); + TVector<std::pair<TString, TYtTableStatInfo::TPtr>> outStat; + for (auto table: op.Output()) { + TString name = TStringBuilder() << "tmp/" << GetGuidAsString(session.RandomProvider_->GenGuid()); + + TYtOutTableInfo tableInfo(table); + tableInfo.Name = name; + WriteOutTable(config, session, cluster, tableInfo, {}); + + TYtTableStatInfo::TPtr statInfo = MakeIntrusive<TYtTableStatInfo>(); + statInfo->Id = name; + LoadTableStatInfo(Services_->GetTablePath(cluster, name, true), *statInfo); + + outStat.emplace_back(statInfo->Id, statInfo); + } + + return outStat; + } + + TVector<std::pair<TString, TYtTableStatInfo::TPtr>> ExecutePrepare(const TPrepareOptions& options, const TSession& session, const TYtOutputOpBase& op) const { + const auto cluster = op.DataSink().Cluster().StringValue(); + YQL_ENSURE(op.Output().Size() == 1U); + + const TString name = TStringBuilder() << "tmp/" << GetGuidAsString(session.RandomProvider_->GenGuid()); + const auto path = Services_->GetTablePath(cluster, name, true); + + TFsQueryCacheItem queryCacheItem(*options.Config(), cluster, Services_->GetTmpDir(), options.OperationHash(), path); + if (queryCacheItem.Lookup(FakeQueue_)) { + TYtTableStatInfo::TPtr statInfo = MakeIntrusive<TYtTableStatInfo>(); + statInfo->Id = name; + LoadTableStatInfo(path, *statInfo); + + return {{statInfo->Id, std::move(statInfo)}}; + } + + TYtOutTableInfo tableInfo(op.Output().Item(0)); + tableInfo.Name = name; + WriteOutTable(options.Config(), const_cast<TSession&>(session), cluster, tableInfo, {}); + return {{name, nullptr}}; + } + + void ExecuteDrop(const TExprNode::TPtr& node) const { + TYtDropTable op(node); + auto table = op.Table(); + bool isAnonymous = NYql::HasSetting(table.Settings().Ref(), EYtSettingType::Anonymous); + auto path = Services_->GetTablePath(op.DataSink().Cluster().Value(), table.Name().Value(), isAnonymous, true); + + NFs::Remove(path); + NFs::Remove(path + ".attr"); + } + + void WriteOutTables(TLambdaBuilder& builder, const TYtSettings::TConstPtr& config, TSession& session, const TString& cluster, + const TVector<TYtOutTableInfo>& outTableInfos, IComputationGraph* compGraph) const + { + NYT::TNode outSpec = NYT::TNode::CreateList(); + for (auto table: outTableInfos) { + table.RowSpec->FillCodecNode(outSpec.Add()[YqlRowSpecAttribute]); + } + outSpec = NYT::TNode::CreateMap()(TString{YqlIOSpecTables}, std::move(outSpec)); + + TVector<TString> outTableContent = GetFileWriteResult( + builder.GetTypeEnvironment(), + builder.GetFunctionRegistry(), + compGraph->GetContext(), + compGraph->GetValue(), + outSpec); + YQL_ENSURE(outTableInfos.size() == outTableContent.size()); + + for (size_t i: xrange(outTableInfos.size())) { + WriteOutTable(config, session, cluster, outTableInfos[i], outTableContent[i]); + } + } + + void WriteOutTable(const TYtSettings::TConstPtr& config, TSession& session, const TString& cluster, + const TYtOutTableInfo& outTableInfo, TStringBuf binaryYson) const + { + auto outPath = Services_->GetTablePath(cluster, outTableInfo.Name, true); + session.DeleteAtFinalize(config, cluster, outPath); + if (binaryYson) { + TMemoryInput in(binaryYson); + TOFStream of(outPath); + TDoubleHighPrecisionYsonWriter writer(&of, ::NYson::EYsonType::ListFragment); + NYson::TYsonParser parser(&writer, &in, ::NYson::EYsonType::ListFragment); + parser.Parse(); + } + else { + YQL_ENSURE(TFile(outPath, CreateAlways | WrOnly).IsOpen(), "Failed to create " << outPath.Quote() << " file"); + } + + { + NYT::TNode attrs = NYT::TNode::CreateMap(); + for (auto& a: outTableInfo.Meta->Attrs) { + attrs[a.first] = a.second; + } + const auto nativeYtTypeCompatibility = config->NativeYtTypeCompatibility.Get(cluster).GetOrElse(NTCF_LEGACY); + const bool rowSpecCompactForm = config->UseYqlRowSpecCompactForm.Get().GetOrElse(DEFAULT_ROW_SPEC_COMPACT_FORM); + const bool optimizeForScan = config->OptimizeFor.Get(cluster).GetOrElse(NYT::EOptimizeForAttr::OF_LOOKUP_ATTR) != NYT::EOptimizeForAttr::OF_LOOKUP_ATTR; + outTableInfo.RowSpec->FillAttrNode(attrs[YqlRowSpecAttribute], nativeYtTypeCompatibility, rowSpecCompactForm); + NYT::TNode rowSpecYson; + outTableInfo.RowSpec->FillCodecNode(rowSpecYson); + + attrs["schema"] = RowSpecToYTSchema(rowSpecYson, nativeYtTypeCompatibility, optimizeForScan ? outTableInfo.GetColumnGroups() : NYT::TNode{}).ToNode(); + TOFStream ofAttr(outPath + ".attr"); + NYson::TYsonWriter writer(&ofAttr, NYson::EYsonFormat::Pretty, ::NYson::EYsonType::Node); + NYT::TNodeVisitor visitor(&writer); + visitor.Visit(attrs); + } + } + + TSet<TString>& GetColumnarStatHistory(NYT::TRichYPath ytPath) { + YQL_ENSURE(ytPath.Columns_.Defined()); + ytPath.Columns_.Clear(); + return ColumnarStatHistory[NYT::NodeToCanonicalYsonString(NYT::PathToNode(ytPath), NYson::EYsonFormat::Text)]; + } + + TPathStatResult DoPathStat(TPathStatOptions&& options, bool onlyCached) { + TPathStatResult res; + res.DataSize.reserve(options.Paths().size()); + + auto extractSysColumns = [] (NYT::TRichYPath& ytPath) -> TVector<TString> { + TVector<TString> res; + if (ytPath.Columns_) { + auto it = std::remove_if( + ytPath.Columns_->Parts_.begin(), + ytPath.Columns_->Parts_.end(), + [] (const TString& col) { return col.StartsWith(YqlSysColumnPrefix); } + ); + res.assign(it, ytPath.Columns_->Parts_.end()); + ytPath.Columns_->Parts_.erase(it, ytPath.Columns_->Parts_.end()); + } + return res; + }; + + for (auto& req: options.Paths()) { + auto path = Services_->GetTablePath(options.Cluster(), req.Path().Path_, req.IsTemp()); + + const NYT::TNode attrs = LoadTableAttrs(path); + bool inferSchema = attrs.HasKey("infer_schema") && attrs["infer_schema"].AsBool(); + + res.DataSize.push_back(0); + auto ytPath = req.Path(); + if (auto sysColumns = extractSysColumns(ytPath)) { + NYT::TNode inputList = LoadTableContent(path); + auto records = inputList.AsList().size(); + records = GetUsedRows(ytPath, records).GetOrElse(records); + for (auto col: sysColumns) { + auto size = 0; + if (col == YqlSysColumnNum || col == YqlSysColumnRecord) { + size = sizeof(ui64); + } else if (col == YqlSysColumnIndex) { + size = sizeof(ui32); + } else if (col == YqlSysColumnPath && !req.IsTemp()) { + size = req.Path().Path_.size(); + } + res.DataSize.back() += size * records; + } + } + + if (ytPath.Columns_ && !inferSchema) { + TSet<TString>& columnarStatsHistory = GetColumnarStatHistory(ytPath); + + if (onlyCached) { + bool allColumnsPresent = AllOf(ytPath.Columns_->Parts_, [&](const auto& c) { + return columnarStatsHistory.contains(c); + }); + + if (!allColumnsPresent) { + return res; + } + } + + TIFStream in(path); + TStringStream out; + TSet<TStringBuf> columns(ytPath.Columns_->Parts_.begin(), ytPath.Columns_->Parts_.end()); + THashMap<TStringBuf, TStringBuf> renames; + if (ytPath.RenameColumns_) { + renames.insert(ytPath.RenameColumns_->begin(), ytPath.RenameColumns_->end()); + } + + TBinaryYsonWriter writer(&out, ::NYson::EYsonType::ListFragment); + TColumnFilteringConsumer filter(&writer, columns, renames); + NYson::TYsonParser parser(&filter, &in, ::NYson::EYsonType::ListFragment); + parser.Parse(); + + for (auto& c : columns) { + columnarStatsHistory.insert(TString(c)); + } + res.DataSize.back() += out.Str().size(); + + if (options.Extended()) { + if (attrs.HasKey("extended_stats")) { + THashMap<TString, i64> dataWeight; + THashMap<TString, ui64> estimatedUniqueCounts; + auto extendedStats = attrs["extended_stats"].AsList(); + std::sort(extendedStats.begin(), extendedStats.end(), + [](NYT::TNode& lhs, NYT::TNode& rhs) { + return lhs.AsMap()["column_name"].AsString() < rhs.AsMap()["column_name"].AsString(); + }); + for (const auto& column : columns) { + auto pos = std::lower_bound(extendedStats.begin(), extendedStats.end(), nullptr, + [&column](NYT::TNode& item, nullptr_t) { + return item.AsMap()["column_name"].AsString() < column; + }); + if (pos != extendedStats.end() && pos->AsMap()["column_name"] == column) { + const auto& m = pos->AsMap(); + auto dataWeightPos = m.find("data_weight"); + if (dataWeightPos != m.end()) { + dataWeight[column] = dataWeightPos->second.ConvertTo<i64>(); + } + auto uniqueValPos = m.find("num_unique_vals"); + if (uniqueValPos != m.end()) { + estimatedUniqueCounts[column] = uniqueValPos->second.ConvertTo<ui64>(); + } + } + } + res.Extended.push_back(IYtGateway::TPathStatResult::TExtendedResult{ + .DataWeight = dataWeight, + .EstimatedUniqueCounts = estimatedUniqueCounts + }); + } else { + res.Extended.push_back(Nothing()); + } + } + } else { + res.DataSize.back() += TFileStat(path).Size; + } + } + res.SetSuccess(); + return res; + } + + +private: + TYtFileServices::TPtr Services_; + TIntrusivePtr<NCommon::TMkqlCommonCallableCompiler> MkqlCompiler_; + THashMap<TString, TSession> Sessions; + bool* EmulateOutputForMultirunPtr; + THashMap<TString, TSet<TString>> ColumnarStatHistory; + TAsyncQueue::TPtr FakeQueue_; +}; + +} // NFile + +IYtGateway::TPtr CreateYtFileGateway(const NFile::TYtFileServices::TPtr& services, bool* emulateOutputForMultirunPtr) { + return new NFile::TYtFileGateway(services, emulateOutputForMultirunPtr); +} + +} // NYql diff --git a/yt/yql/providers/yt/gateway/file/yql_yt_file.h b/yt/yql/providers/yt/gateway/file/yql_yt_file.h new file mode 100644 index 0000000000..bcff8bdccb --- /dev/null +++ b/yt/yql/providers/yt/gateway/file/yql_yt_file.h @@ -0,0 +1,29 @@ +#pragma once + +#include "yql_yt_file_services.h" + +#include <yt/yql/providers/yt/provider/yql_yt_gateway.h> +#include <yql/essentials/minikql/mkql_function_registry.h> + +#include <util/generic/hash.h> +#include <util/generic/string.h> +#include <util/generic/strbuf.h> +#include <util/generic/maybe.h> +#include <util/generic/ptr.h> +#include <util/folder/dirut.h> +#include <util/system/mutex.h> + +#include <vector> + +namespace NKikimr { +namespace NMiniKQL { +class IComputationPatternCache; +} +} + +namespace NYql { + +IYtGateway::TPtr CreateYtFileGateway(const NFile::TYtFileServices::TPtr& services, + bool* emulateOutputForMultirunPtr = nullptr); + +} diff --git a/yt/yql/providers/yt/gateway/file/yql_yt_file_comp_nodes.cpp b/yt/yql/providers/yt/gateway/file/yql_yt_file_comp_nodes.cpp new file mode 100644 index 0000000000..78af8c3d70 --- /dev/null +++ b/yt/yql/providers/yt/gateway/file/yql_yt_file_comp_nodes.cpp @@ -0,0 +1,470 @@ +#include "yql_yt_file_comp_nodes.h" +#include "yql_yt_file.h" +#include "yql_yt_file_text_yson.h" + +#include <yt/yql/providers/yt/expr_nodes/yql_yt_expr_nodes.h> +#include <yt/yql/providers/yt/comp_nodes/yql_mkql_file_input_state.h> +#include <yt/yql/providers/yt/comp_nodes/yql_mkql_file_list.h> +#include <yt/yql/providers/yt/comp_nodes/yql_mkql_table.h> +#include <yt/yql/providers/yt/comp_nodes/yql_mkql_ungrouping_list.h> +#include <yt/yql/providers/yt/codec/yt_codec_io.h> +#include <yt/yql/providers/yt/codec/yt_codec.h> +#include <yt/yql/providers/yt/lib/mkql_helpers/mkql_helpers.h> +#include <yt/yql/providers/yt/lib/yson_helpers/yson_helpers.h> +#include <yt/yql/providers/yt/common/yql_names.h> +#include <yql/essentials/providers/common/codec/yql_codec.h> +#include <yql/essentials/providers/common/comp_nodes/yql_factory.h> +#include <yql/essentials/parser/pg_wrapper/interface/comp_factory.h> +#include <yql/essentials/public/udf/udf_version.h> +#include <yql/essentials/utils/yql_panic.h> + +#include <yql/essentials/minikql/computation/mkql_computation_node_impl.h> +#include <yql/essentials/minikql/computation/mkql_computation_node_pack.h> +#include <yql/essentials/minikql/computation/mkql_custom_list.h> +#include <yql/essentials/minikql/comp_nodes/mkql_factories.h> +#include <yql/essentials/minikql/mkql_node_cast.h> +#include <yql/essentials/minikql/mkql_node.h> + + +#include <library/cpp/yson/node/node_io.h> +#include <yt/cpp/mapreduce/interface/serialize.h> + +#include <library/cpp/yson/parser.h> +#include <library/cpp/yson/writer.h> + +#include <util/stream/str.h> +#include <util/stream/file.h> +#include <util/stream/buffer.h> +#include <util/system/fs.h> +#include <util/generic/vector.h> +#include <util/generic/string.h> +#include <util/generic/xrange.h> +#include <util/generic/ylimits.h> +#include <util/generic/maybe.h> +#include <util/generic/size_literals.h> + +#include <utility> +#include <algorithm> +#include <iterator> +#include <array> + +namespace NYql::NFile { + +using namespace NKikimr; +using namespace NKikimr::NMiniKQL; + +namespace { + +/////////////////////////////////////////////////////////////////////////////////////////////////////// + +class TFileInputStateWithTableState: public TFileInputState { +public: + TFileInputStateWithTableState(const TMkqlIOSpecs& spec, const NKikimr::NMiniKQL::THolderFactory& holderFactory, + TVector<NYT::TRawTableReaderPtr>&& rawReaders, + size_t blockCount, size_t blockSize, TTableState&& tableState) + : TFileInputState(spec, holderFactory, std::move(rawReaders), blockCount, blockSize) + , TableState_(std::move(tableState)) + { + UpdateTableState(); + } + +protected: + void Next() override { + TFileInputState::Next(); + UpdateTableState(); + } + + void UpdateTableState() { + if (IsValid()) { + TableState_.Update(GetTableIndex(), GetRecordIndex()); + } else { + TableState_.Reset(); + } + } + +private: + TTableState TableState_; +}; + +/////////////////////////////////////////////////////////////////////////////////////////////////////// + +class TTextYsonFileListValue : public TFileListValueBase { +public: + TTextYsonFileListValue(NKikimr::NMiniKQL::TMemoryUsageInfo* memInfo, + const TMkqlIOSpecs& spec, + const NKikimr::NMiniKQL::THolderFactory& holderFactory, + const TVector<std::pair<TString, TColumnsInfo>>& tablePaths, + TTableState&& tableState, std::optional<ui64> length) + : TFileListValueBase(memInfo, spec, holderFactory, length) + , TablePaths_(tablePaths) + , TableState_(std::move(tableState)) + { + } + +protected: + THolder<IInputState> MakeState() const override { + return MakeHolder<TFileInputStateWithTableState>(Spec, HolderFactory, MakeTextYsonInputs(TablePaths_), + 0u, 1_MB, TTableState(TableState_)); + } + +private: + TVector<std::pair<TString, TColumnsInfo>> TablePaths_; + TTableState TableState_; +}; + +/////////////////////////////////////////////////////////////////////////////////////////////////////// + +template <bool TableContent> +class TYtTableFileWrapper : public TMutableComputationNode<TYtTableFileWrapper<TableContent>> { + typedef TMutableComputationNode<TYtTableFileWrapper<TableContent>> TBaseComputation; +public: + TYtTableFileWrapper(TComputationMutables& mutables, NCommon::TCodecContext& codecCtx, + TVector<std::pair<TString, TColumnsInfo>>&& tablePaths, + const NYT::TNode& inputSpecs, const TVector<ui32>& groups, TType* itemType, + TVector<TString>&& tableNames, TVector<ui64>&& rowOffsets, THashSet<TString>&& auxColumns, + std::array<IComputationExternalNode*, 5>&& argNodes, std::optional<ui64> length) + : TBaseComputation(mutables) + , TablePaths_(std::move(tablePaths)) + , ArgNodes_(std::move(argNodes)) + , Length_(std::move(length)) + { + Spec_.Init(codecCtx, inputSpecs, groups, tableNames, itemType, auxColumns, NYT::TNode()); + if (!rowOffsets.empty()) { + Spec_.SetTableOffsets(rowOffsets); + } + } + + NUdf::TUnboxedValuePod DoCalculate(TComputationContext& ctx) const { + if (TableContent) { + return ctx.HolderFactory.Create<TTextYsonFileListValue>(Spec_, ctx.HolderFactory, TablePaths_, + TTableState(Spec_.TableNames, Spec_.TableOffsets, ctx, ArgNodes_), Length_); + } + else { + THolder<TFileInputState> inputState(new TFileInputStateWithTableState(Spec_, ctx.HolderFactory, MakeTextYsonInputs(TablePaths_), + 0, 1_MB, TTableState(Spec_.TableNames, Spec_.TableOffsets, ctx, ArgNodes_))); + NUdf::TUnboxedValue singlePassIter(ctx.HolderFactory.Create<THoldingInputStreamValue>(inputState.Release())); + return ctx.HolderFactory.Create<TForwardListValue>(std::move(singlePassIter)); + } + } + +private: + void RegisterDependencies() const final { + for (auto node: ArgNodes_) { + TBaseComputation::Own(node); + } + } + + TMkqlIOSpecs Spec_; + const TVector<std::pair<TString, TColumnsInfo>> TablePaths_; + const std::array<IComputationExternalNode*, 5> ArgNodes_; + std::optional<ui64> Length_; +}; + +/////////////////////////////////////////////////////////////////////////////////////////////////////// + +class TFileWriteApplyContext : public NUdf::IApplyContext { +public: + TFileWriteApplyContext(TMkqlWriterImpl& writer) + : Writer(writer) + { + } + + void WriteStream(const NUdf::TUnboxedValue& res) { + NUdf::TUnboxedValue value; + for (auto status = res.Fetch(value); status != NUdf::EFetchStatus::Finish; status = res.Fetch(value)) { + if (status != NUdf::EFetchStatus::Yield) + Writer.AddRow(value); + } + } + +private: + TMkqlWriterImpl& Writer; +}; + +/////////////////////////////////////////////////////////////////////////////////////////////////////// + +class TFileWriteWrapper : public TMutableComputationNode<TFileWriteWrapper> { + typedef TMutableComputationNode<TFileWriteWrapper> TBaseComputation; +public: + class TResult : public TComputationValue<TResult> { + public: + TResult(TMemoryUsageInfo* memInfo, const NUdf::TUnboxedValue& result) + : TComputationValue(memInfo) + , Result(result) + { + } + + private: + void Apply(NUdf::IApplyContext& applyContext) const override { + CheckedCast<TFileWriteApplyContext*>(&applyContext)->WriteStream(Result); + } + + const NUdf::TUnboxedValue& Result; + }; + + + TFileWriteWrapper(TComputationMutables& mutables, IComputationNode* result) + : TBaseComputation(mutables) + , Result(result) + , StreamValueIndex(mutables.CurValueIndex++) + { + } + + NUdf::TUnboxedValuePod DoCalculate(TComputationContext& ctx) const { + auto& streamRef = ctx.MutableValues[StreamValueIndex]; + streamRef = Result->GetValue(ctx); + return ctx.HolderFactory.Create<TResult>(streamRef); + } + +private: + void RegisterDependencies() const final { + DependsOn(Result); + } + + IComputationNode* const Result; + const ui32 StreamValueIndex; +}; + +/////////////////////////////////////////////////////////////////////////////////////////////////////// + +class TDqWriteWrapper : public TMutableComputationNode<TDqWriteWrapper> { + typedef TMutableComputationNode<TDqWriteWrapper> TBaseComputation; +public: + TDqWriteWrapper(TComputationMutables& mutables, IComputationNode* item, const TString& path, TType* itemType, const TYtFileServices::TPtr& services) + : TBaseComputation(mutables) + , Item(item) + , Path(path) + , Packer(true, itemType) + , Services(services) + { + } + + NUdf::TUnboxedValuePod DoCalculate(TComputationContext& ctx) const { + auto value = Item->GetValue(ctx); + Services->PushTableContent(Path, TString{Packer.Pack(value)}); + return {}; + } + +private: + void RegisterDependencies() const final { + DependsOn(Item); + } + + IComputationNode* const Item; + const TString Path; + TValuePacker Packer; + TYtFileServices::TPtr Services; +}; + +/////////////////////////////////////////////////////////////////////////////////////////////////////// + +template <bool TableContent> +IComputationNode* WrapYtTableFile(NMiniKQL::TCallable& callable, const TComputationNodeFactoryContext& ctx, + const TYtFileServices::TPtr& services, bool noLocks) +{ + if (TableContent) { + YQL_ENSURE(callable.GetInputsCount() == 4, "Expected 4 args"); + } else { + YQL_ENSURE(callable.GetInputsCount() == 8 || callable.GetInputsCount() == 4, "Expected 8 or 4 args"); + } + const TString cluster(AS_VALUE(TDataLiteral, callable.GetInput(0))->AsValue().AsStringRef()); + + TType* itemType = AS_TYPE(NMiniKQL::TListType, callable.GetType()->GetReturnType())->GetItemType(); + TVector<std::pair<TString, TColumnsInfo>> tablePaths; + TVector<TString> tableNames; + TVector<ui64> rowOffsets; + ui64 currentRowOffset = 0; + TVector<ui32> groups; + NYT::TNode inputAttrs = NYT::TNode::CreateList(); + THashSet<TString> auxColumns; + + NCommon::TCodecContext codecCtx(ctx.Env, ctx.FunctionRegistry, &ctx.HolderFactory); + + TListLiteral* groupList = AS_VALUE(TListLiteral, callable.GetInput(1)); + const bool multiGroup = groupList->GetItemsCount() > 1; + + TVector<THashSet<TStringBuf>> outColumnGroups; + if (multiGroup) { + TVariantType* itemVarType = AS_TYPE(TVariantType, itemType); + for (ui32 g = 0; g < groupList->GetItemsCount(); ++g) { + NMiniKQL::TStructType* itemStruct = AS_TYPE(TStructType, itemVarType->GetAlternativeType(g)); + auto& outColumns = outColumnGroups.emplace_back(); + for (ui32 index = 0; index < itemStruct->GetMembersCount(); ++index) { + outColumns.insert(itemStruct->GetMemberName(index)); + } + } + } + else { + NMiniKQL::TStructType* itemStruct = AS_TYPE(NMiniKQL::TStructType, itemType); + auto& outColumns = outColumnGroups.emplace_back(); + for (ui32 index = 0; index < itemStruct->GetMembersCount(); ++index) { + outColumns.insert(itemStruct->GetMemberName(index)); + } + } + + for (ui32 g = 0; g < groupList->GetItemsCount(); ++g) { + TListLiteral* tableList = AS_VALUE(TListLiteral, groupList->GetItems()[g]); + auto& outColumns = outColumnGroups[g]; + currentRowOffset = 0; + for (ui32 t = 0; t < tableList->GetItemsCount(); ++t) { + TTupleLiteral* tuple = AS_VALUE(TTupleLiteral, tableList->GetItems()[t]); + YQL_ENSURE(tuple->GetValuesCount() == 7, "Expect 7 elements in the table tuple"); + + NYT::TRichYPath richYPath; + NYT::Deserialize(richYPath, NYT::NodeFromYsonString(TString(AS_VALUE(TDataLiteral, tuple->GetValue(0))->AsValue().AsStringRef()))); + + const bool isTemporary = AS_VALUE(TDataLiteral, tuple->GetValue(1))->AsValue().Get<bool>(); + auto tableMeta = NYT::NodeFromYsonString(TString(AS_VALUE(TDataLiteral, tuple->GetValue(2))->AsValue().AsStringRef())); + TMkqlIOSpecs::TSpecInfo specInfo; + TMkqlIOSpecs::LoadSpecInfo(true, tableMeta, codecCtx, specInfo); + NMiniKQL::TStructType* itemStruct = AS_TYPE(NMiniKQL::TStructType, specInfo.Type); + for (ui32 index = 0; index < itemStruct->GetMembersCount(); ++index) { + // Ignore extra columns, which are not selected from the table + if (!outColumns.contains(itemStruct->GetMemberName(index))) { + auxColumns.insert(TString{itemStruct->GetMemberName(index)}); + } + } + for (auto& aux: specInfo.AuxColumns) { + if (!outColumns.contains(aux.first)) { + auxColumns.insert(aux.first); + } + } + + inputAttrs.Add(tableMeta); + auto path = services->GetTablePath(cluster, richYPath.Path_, isTemporary, noLocks); + tableNames.push_back(isTemporary ? TString() : richYPath.Path_); + tablePaths.emplace_back(path, TColumnsInfo{richYPath.Columns_, richYPath.RenameColumns_}); + if (!richYPath.Columns_ && !isTemporary && specInfo.StrictSchema) { + TVector<TString> columns; + for (ui32 index = 0; index < itemStruct->GetMembersCount(); ++index) { + columns.emplace_back(itemStruct->GetMemberName(index)); + } + for (auto& aux: specInfo.AuxColumns) { + columns.push_back(aux.first); + } + tablePaths.back().second.Columns.ConstructInPlace(std::move(columns)); + } + if (multiGroup) { + groups.push_back(g); + } + rowOffsets.push_back(currentRowOffset); + currentRowOffset += AS_VALUE(TDataLiteral, tuple->GetValue(4))->AsValue().Get<ui64>(); + } + } + + std::optional<ui64> length; + TTupleLiteral* lengthTuple = AS_VALUE(TTupleLiteral, callable.GetInput(3)); + if (lengthTuple->GetValuesCount() > 0) { + YQL_ENSURE(lengthTuple->GetValuesCount() == 1, "Expect 1 element in the length tuple"); + length = AS_VALUE(TDataLiteral, lengthTuple->GetValue(0))->AsValue().Get<ui64>(); + } + + std::array<IComputationExternalNode*, 5> argNodes; + argNodes.fill(nullptr); + + if (!TableContent && callable.GetInputsCount() == 8) { + argNodes[0] = LocateExternalNode(ctx.NodeLocator, callable, 4, false); // TableIndex + argNodes[1] = LocateExternalNode(ctx.NodeLocator, callable, 5, false); // TablePath + argNodes[2] = LocateExternalNode(ctx.NodeLocator, callable, 6, false); // TableRecord + argNodes[4] = LocateExternalNode(ctx.NodeLocator, callable, 7, false); // RowNumber + } + + // sampling arg is ignored in the file provider + return new TYtTableFileWrapper<TableContent>(ctx.Mutables, codecCtx, std::move(tablePaths), + NYT::TNode::CreateMap()(TString{YqlIOSpecTables}, std::move(inputAttrs)), + groups, itemType, std::move(tableNames), std::move(rowOffsets), std::move(auxColumns), std::move(argNodes), length); +} + +IComputationNode* WrapFileWrite(NMiniKQL::TCallable& callable, const TComputationNodeFactoryContext& ctx) { + YQL_ENSURE(callable.GetInputsCount() == 1, "Expected 1 arg"); + const auto list = LocateNode(ctx.NodeLocator, callable, 0); + return new TFileWriteWrapper(ctx.Mutables, list); +} + +IComputationNode* WrapDqWrite(NMiniKQL::TCallable& callable, const TComputationNodeFactoryContext& ctx, const TYtFileServices::TPtr& services) { + YQL_ENSURE(callable.GetInputsCount() == 2, "Expected 2 args"); + const auto item = LocateNode(ctx.NodeLocator, callable, 0); + TType* itemType = callable.GetInput(0).GetStaticType(); + auto path = TString(AS_VALUE(TDataLiteral, callable.GetInput(1))->AsValue().AsStringRef()); + return new TDqWriteWrapper(ctx.Mutables, item, path, itemType, services); +} + +} // unnamed + +/////////////////////////////////////////////////////////////////////////////////////////////////////// + +class TYtFileFactory { +public: + TYtFileFactory(const TYtFileServices::TPtr& services) + : Services_(services) + { + } + + IComputationNode* operator() (NMiniKQL::TCallable& callable, const TComputationNodeFactoryContext& ctx) { + auto name = callable.GetType()->GetName(); + if (name.ChopSuffix("File")) { + if (name == TStringBuf("YtWrite")) { + return WrapFileWrite(callable, ctx); + } + if (name == TStringBuf("DqWrite")) { + return WrapDqWrite(callable, ctx, Services_); + } + if (name == NNodes::TYtTableContent::CallableName()) { + return WrapYtTableFile<true>(callable, ctx, Services_, false); + } + if (name == "YtUngroupingList") { + return WrapYtUngroupingList(callable, ctx); + } + if (name == "YtTableInput") { + return WrapYtTableFile<false>(callable, ctx, Services_, false); + } + if (name == "YtTableInputNoCtx") { + return WrapYtTableFile<false>(callable, ctx, Services_, true); + } + } + + return nullptr; + } +private: + TYtFileServices::TPtr Services_; + TMaybe<ui32> ExprContextObject; +}; + +TComputationNodeFactory GetYtFileFactory(const TYtFileServices::TPtr& services) { + return TYtFileFactory(services); +} + +NKikimr::NMiniKQL::TComputationNodeFactory GetYtFileFullFactory(const TYtFileServices::TPtr& services) { + return NMiniKQL::GetCompositeWithBuiltinFactory({ + GetYtFileFactory(services), + NMiniKQL::GetYqlFactory(), + GetPgFactory() + }); +} + +TVector<TString> GetFileWriteResult(const TTypeEnvironment& env, const IFunctionRegistry& functionRegistry, TComputationContext& ctx, const NKikimr::NUdf::TUnboxedValue& value, + const NYT::TNode& outSpecs) +{ + NCommon::TCodecContext codecCtx(env, functionRegistry, &ctx.HolderFactory); + TMkqlIOSpecs spec; + spec.Init(codecCtx, outSpecs); + + TVector<TStringStream> streams(spec.Outputs.size()); + { + TVector<IOutputStream*> out(Reserve(spec.Outputs.size())); + std::transform(streams.begin(), streams.end(), std::back_inserter(out), [] (TStringStream& s) { return &s; }); + TMkqlWriterImpl writer(out, 0, 4_MB); + writer.SetSpecs(spec); + + TFileWriteApplyContext applyCtx(writer); + ApplyChanges(value, applyCtx); + writer.Finish(); + } + TVector<TString> res; + res.reserve(spec.Outputs.size()); + std::transform(streams.begin(), streams.end(), std::back_inserter(res), [] (TStringStream& s) { return s.Str(); }); + return res; +} + + +} // NYql::NFile diff --git a/yt/yql/providers/yt/gateway/file/yql_yt_file_comp_nodes.h b/yt/yql/providers/yt/gateway/file/yql_yt_file_comp_nodes.h new file mode 100644 index 0000000000..c588e475e7 --- /dev/null +++ b/yt/yql/providers/yt/gateway/file/yql_yt_file_comp_nodes.h @@ -0,0 +1,18 @@ +#pragma once + +#include "yql_yt_file_services.h" + +#include <yql/essentials/minikql/computation/mkql_computation_node.h> +#include <yql/essentials/minikql/computation/mkql_computation_node_impl.h> +#include <yql/essentials/public/udf/udf_value.h> + +#include <library/cpp/yson/node/node.h> + +namespace NYql::NFile { + +NKikimr::NMiniKQL::TComputationNodeFactory GetYtFileFactory(const TYtFileServices::TPtr& services); +NKikimr::NMiniKQL::TComputationNodeFactory GetYtFileFullFactory(const TYtFileServices::TPtr& services); +TVector<TString> GetFileWriteResult(const NKikimr::NMiniKQL::TTypeEnvironment& env, const NKikimr::NMiniKQL::IFunctionRegistry& functionRegistry, + NKikimr::NMiniKQL::TComputationContext& ctx, const NKikimr::NUdf::TUnboxedValue& value, const NYT::TNode& outSpecs); + +} // NYql::NFile diff --git a/yt/yql/providers/yt/gateway/file/yql_yt_file_mkql_compiler.cpp b/yt/yql/providers/yt/gateway/file/yql_yt_file_mkql_compiler.cpp new file mode 100644 index 0000000000..b4ff5c5ca4 --- /dev/null +++ b/yt/yql/providers/yt/gateway/file/yql_yt_file_mkql_compiler.cpp @@ -0,0 +1,1112 @@ +#include "yql_yt_file_mkql_compiler.h" + +#include <yt/yql/providers/yt/expr_nodes/yql_yt_expr_nodes.h> +#include <yt/yql/providers/yt/provider/yql_yt_table.h> +#include <yt/yql/providers/yt/provider/yql_yt_helpers.h> +#include <yt/yql/providers/yt/provider/yql_yt_mkql_compiler.h> + +#include <yql/essentials/providers/result/expr_nodes/yql_res_expr_nodes.h> +#include <yql/essentials/providers/common/mkql/yql_type_mkql.h> +#include <yql/essentials/providers/common/schema/mkql/yql_mkql_schema.h> +#include <yql/essentials/core/dq_expr_nodes/dq_expr_nodes.h> +#include <yql/essentials/core/dqs_expr_nodes/dqs_expr_nodes.h> +#include <yql/essentials/core/expr_nodes/yql_expr_nodes.h> +#include <yql/essentials/core/yql_opt_utils.h> +#include <yql/essentials/utils/yql_panic.h> +#include <yql/essentials/minikql/mkql_program_builder.h> +#include <yql/essentials/minikql/mkql_node_cast.h> +#include <yql/essentials/minikql/defs.h> + +#include <yt/cpp/mapreduce/interface/node.h> +#include <yt/cpp/mapreduce/interface/serialize.h> +#include <yt/cpp/mapreduce/common/helpers.h> + +#include <library/cpp/yson/node/node_io.h> + +#include <util/generic/xrange.h> +#include <util/string/cast.h> +#include <util/stream/str.h> + +namespace NYql { + +using namespace NKikimr; +using namespace NKikimr::NMiniKQL; +using namespace NNodes; +using namespace NNodes::NDq; + +namespace { + +bool IsValidKeyNode(TExprBase node) { + if (node.Maybe<TCoNull>() || node.Maybe<TCoNothing>()) { + return true; + } + + if (auto maybeJust = node.Maybe<TCoJust>()) { + node = maybeJust.Cast().Input(); + } + + return TCoDataCtor::Match(node.Raw()); +} + +bool HasRanges(const TExprNode& input, bool withKeyUsage) { + if (input.IsCallable(TYtOutTable::CallableName())) { + return false; + } + auto sectionList = TYtSectionList(&input); + return AnyOf(sectionList, [&](const TYtSection& section) { + return AnyOf(section.Paths(), [&](const TYtPath& path) { + TYtPathInfo pathInfo(path); + auto ranges = pathInfo.Ranges; + return ranges && (!withKeyUsage || ranges->GetUsedKeyPrefixLength()); + }); + }); +} + +bool HasRangesWithKeyColumns(const TExprNode& input) { + return HasRanges(input, /*withKeyUsage*/true); +} + +TRuntimeNode ApplyPathRanges(TRuntimeNode inputList, const TExprNode& input, NCommon::TMkqlBuildContext& ctx) { + if (!HasRanges(input, false)) { + return inputList; + } + + ui32 tableIndex = 0; + auto makeSectionFilter = [&](TYtSection section, TRuntimeNode item) { + ui64 currentOffset = 0; + TRuntimeNode filter = ctx.ProgramBuilder.NewDataLiteral(true); + + TRuntimeNode rowNumber = ctx.ProgramBuilder.Member(item, YqlSysColumnNum); + rowNumber = ctx.ProgramBuilder.Sub( + ctx.ProgramBuilder.Max( + rowNumber, + ctx.ProgramBuilder.NewDataLiteral(ui64(1)) + ), + ctx.ProgramBuilder.NewDataLiteral(ui64(1)) + ); + + TRuntimeNode tableIndexCall = ctx.ProgramBuilder.Member(item, YqlSysColumnIndex); + + TVector<std::pair<TRuntimeNode, TRuntimeNode>> tableIndexDictItems; + + for (auto p: section.Paths()) { + TYtPathInfo pathInfo(p); + if (pathInfo.Ranges) { + TRuntimeNode condition = ctx.ProgramBuilder.NewDataLiteral(false); + for (auto& range: pathInfo.Ranges->GetRanges()) { + TRuntimeNode data; + // We can't use switch by index there since msvc deny to compile it. + // Anyway, visit works faster because we're making only one check instead of two. + std::visit([&](const auto& val) { + using TValue = std::decay_t<decltype(val)>; + if constexpr (std::is_same_v<TValue, TYtRangesInfo::TRowSingle>) { + data = ctx.ProgramBuilder.AggrEquals(rowNumber, ctx.ProgramBuilder.NewDataLiteral(currentOffset + val.Offset)); + } else if constexpr (std::is_same_v<TValue, TYtRangesInfo::TRowRange>) { + if (val.Lower) { + data = ctx.ProgramBuilder.AggrGreaterOrEqual(rowNumber, ctx.ProgramBuilder.NewDataLiteral(currentOffset + *val.Lower)); + } + if (val.Upper) { + auto upper = ctx.ProgramBuilder.AggrLess(rowNumber, ctx.ProgramBuilder.NewDataLiteral(currentOffset + *val.Upper)); + data = data ? ctx.ProgramBuilder.And({data, upper}) : upper; + } + } else if constexpr (std::is_same_v<TValue, TYtRangesInfo::TKeySingle>) { + YQL_ENSURE(pathInfo.Table->RowSpec); + auto sortedBy = pathInfo.Table->RowSpec->SortedBy; + YQL_ENSURE(val.Key.size() <= sortedBy.size()); + size_t sortedByIndex = 0; + for (auto& node : val.Key) { + TRuntimeNode keyPart; + YQL_ENSURE(IsValidKeyNode(node), "Range should be calculated"); + if (TCoNull::Match(node.Raw())) { + keyPart = ctx.ProgramBuilder.Not(ctx.ProgramBuilder.Exists( + ctx.ProgramBuilder.Member(item, sortedBy[sortedByIndex++]))); + } else if (TCoJust::Match(node.Raw()) || TCoNothing::Match(node.Raw())) { + keyPart = ctx.ProgramBuilder.AggrEquals( + ctx.ProgramBuilder.Member(item, sortedBy[sortedByIndex++]), + NCommon::MkqlBuildExpr(node.Ref(), ctx)); + } else { + keyPart = ctx.ProgramBuilder.Equals( + ctx.ProgramBuilder.Member(item, sortedBy[sortedByIndex++]), + NCommon::MkqlBuildExpr(node.Ref(), ctx)); + } + data = data ? ctx.ProgramBuilder.And({data, keyPart}) : keyPart; + } + } else if constexpr (std::is_same_v<TValue, TYtRangesInfo::TKeyRange>) { + YQL_ENSURE(pathInfo.Table->RowSpec); + auto sortedBy = pathInfo.Table->RowSpec->SortedBy; + if (!val.Lower.empty()) { + YQL_ENSURE(val.Lower.size() <= sortedBy.size()); + for (size_t i: xrange(val.Lower.size() - 1)) { + YQL_ENSURE(IsValidKeyNode(val.Lower[i]), "Lower range should be calculated"); + TRuntimeNode keyPart; + TRuntimeNode member = ctx.ProgramBuilder.Member(item, sortedBy[i]); + TRuntimeNode expr = NCommon::MkqlBuildExpr(val.Lower[i].Ref(), ctx); + + if (TCoNull::Match(val.Lower[i].Raw())) { + keyPart = ctx.ProgramBuilder.Not(ctx.ProgramBuilder.Exists(member)); + } else if (TCoJust::Match(val.Lower[i].Raw()) || TCoNothing::Match(val.Lower[i].Raw())) { + keyPart = ctx.ProgramBuilder.AggrGreaterOrEqual(member, expr); + } else { + keyPart = ctx.ProgramBuilder.GreaterOrEqual(member, expr); + } + data = data ? ctx.ProgramBuilder.And({data, keyPart}) : keyPart; + } + TRuntimeNode keyPart; + TRuntimeNode member = ctx.ProgramBuilder.Member(item, sortedBy[val.Lower.size() - 1]); + TRuntimeNode expr = NCommon::MkqlBuildExpr(val.Lower.back().Ref(), ctx); + if (TCoJust::Match(val.Lower.back().Raw()) || TCoNothing::Match(val.Lower.back().Raw())) { + keyPart = val.LowerInclude + ? ctx.ProgramBuilder.AggrGreaterOrEqual(member, expr) + : ctx.ProgramBuilder.AggrGreater(member, expr); + } else { + keyPart = val.LowerInclude + ? ctx.ProgramBuilder.GreaterOrEqual(member, expr) + : ctx.ProgramBuilder.Greater(member, expr); + } + data = data ? ctx.ProgramBuilder.And({data, keyPart}) : keyPart; + } + if (!val.Upper.empty()) { + YQL_ENSURE(val.Upper.size() <= sortedBy.size()); + for (size_t i: xrange(val.Upper.size() - 1)) { + YQL_ENSURE(IsValidKeyNode(val.Upper[i]), "Upper range should be calculated"); + TRuntimeNode keyPart; + TRuntimeNode member = ctx.ProgramBuilder.Member(item, sortedBy[i]); + TRuntimeNode expr = NCommon::MkqlBuildExpr(val.Upper[i].Ref(), ctx); + if (TCoNull::Match(val.Upper[i].Raw())) { + keyPart = ctx.ProgramBuilder.Not(ctx.ProgramBuilder.Exists(member)); + } else if (TCoJust::Match(val.Upper[i].Raw()) || TCoNothing::Match(val.Upper[i].Raw())) { + keyPart = ctx.ProgramBuilder.AggrLessOrEqual(member, expr); + } else { + keyPart = ctx.ProgramBuilder.LessOrEqual(member, expr); + } + data = data ? ctx.ProgramBuilder.And({data, keyPart}) : keyPart; + } + TRuntimeNode keyPart; + TRuntimeNode member = ctx.ProgramBuilder.Member(item, sortedBy[val.Upper.size() - 1]); + TRuntimeNode expr = NCommon::MkqlBuildExpr(val.Upper.back().Ref(), ctx); + if (TCoJust::Match(val.Upper.back().Raw()) || TCoNothing::Match(val.Upper.back().Raw())) { + keyPart = val.UpperInclude + ? ctx.ProgramBuilder.AggrLessOrEqual(member, expr) + : ctx.ProgramBuilder.AggrLess(member, expr); + } else { + keyPart = val.UpperInclude + ? ctx.ProgramBuilder.LessOrEqual(member, expr) + : ctx.ProgramBuilder.Less(member, expr); + } + data = data ? ctx.ProgramBuilder.And({data, keyPart}) : keyPart; + } + } + }, range); + condition = ctx.ProgramBuilder.Or({condition, data}); + } + if (condition.GetStaticType()->IsOptional() != filter.GetStaticType()->IsOptional()) { + if (condition.GetStaticType()->IsOptional()) { + filter = ctx.ProgramBuilder.NewOptional(filter); + } else { + condition = ctx.ProgramBuilder.NewOptional(condition); + } + } + + filter = ctx.ProgramBuilder.If( + ctx.ProgramBuilder.AggrEquals(tableIndexCall, ctx.ProgramBuilder.NewDataLiteral(tableIndex)), + condition, + filter); + + tableIndexDictItems.emplace_back(ctx.ProgramBuilder.NewDataLiteral<ui32>(tableIndex), ctx.ProgramBuilder.NewVoid()); + } + ++tableIndex; + currentOffset += pathInfo.Table->Stat->RecordsCount; + } + + if (!tableIndexDictItems.empty()) { + auto dictType = ctx.ProgramBuilder.NewDictType( + ctx.ProgramBuilder.NewDataType(NUdf::TDataType<ui32>::Id), + ctx.ProgramBuilder.GetTypeEnvironment().GetTypeOfVoidLazy(), + false + ); + auto dict = ctx.ProgramBuilder.NewDict(dictType, tableIndexDictItems); + filter = ctx.ProgramBuilder.Or({filter, ctx.ProgramBuilder.Not(ctx.ProgramBuilder.Contains(dict, tableIndexCall))}); + } + + if (filter.GetStaticType()->IsOptional()) { + filter = ctx.ProgramBuilder.Coalesce(filter, ctx.ProgramBuilder.NewDataLiteral(false)); + } + + return filter; + }; + + auto sectionList = TYtSectionList(&input); + if (sectionList.Size() > 1) { + inputList = ctx.ProgramBuilder.OrderedFilter(inputList, [&](TRuntimeNode varItem) -> TRuntimeNode { + YQL_ENSURE(AS_TYPE(TVariantType, varItem)->GetAlternativesCount() == sectionList.Size()); + return ctx.ProgramBuilder.VisitAll(varItem, [&](ui32 index, TRuntimeNode item) -> TRuntimeNode { + return makeSectionFilter(sectionList.Item(index), item); + }); + }); + } else { + inputList = ctx.ProgramBuilder.OrderedFilter(inputList, [&](TRuntimeNode item) -> TRuntimeNode { + return makeSectionFilter(sectionList.Item(0), item); + }); + } + + return inputList; +} + +TRuntimeNode ApplySampling(TRuntimeNode list, const TExprNode& input, NCommon::TMkqlBuildContext& ctx) +{ + if (input.IsCallable(TYtOutTable::CallableName())) { + return list; + } + + auto getSamplingPercent = [](TYtSection section) -> TMaybe<double> { + if (auto setting = NYql::GetSetting(section.Settings().Ref(), EYtSettingType::Sample)) { + return FromString<double>(setting->Child(1)->Child(1)->Content()); + } + return Nothing(); + }; + + // At least one table has ranges + auto sectonList = TYtSectionList(&input); + TMaybe<double> percent = getSamplingPercent(sectonList.Item(0)); + + if (!percent) { + return list; + } + + auto oneHundredNode = ctx.ProgramBuilder.NewDataLiteral<double>(100.); + auto percentNode = ctx.ProgramBuilder.NewDataLiteral<double>(*percent); + auto trueNode = ctx.ProgramBuilder.NewDataLiteral<bool>(true); + auto falseNode = ctx.ProgramBuilder.NewDataLiteral<bool>(false); + list = ctx.ProgramBuilder.ChainMap(list, oneHundredNode, [&] (TRuntimeNode item, TRuntimeNode state) -> TRuntimeNodePair { + auto tuple = ctx.ProgramBuilder.If(ctx.ProgramBuilder.AggrGreaterOrEqual(state, oneHundredNode), + ctx.ProgramBuilder.NewTuple({trueNode, ctx.ProgramBuilder.Add(ctx.ProgramBuilder.Sub(state, oneHundredNode), percentNode)}), + ctx.ProgramBuilder.NewTuple({falseNode, ctx.ProgramBuilder.Add(state, percentNode)}) + ); + return { + ctx.ProgramBuilder.NewTuple({item, ctx.ProgramBuilder.Nth(tuple, 0)}), + ctx.ProgramBuilder.Nth(tuple, 1) + }; + }); + list = ctx.ProgramBuilder.Filter(list, [&] (TRuntimeNode item) { + return ctx.ProgramBuilder.Nth(item, 1); + }); + list = ctx.ProgramBuilder.Map(list, [&] (TRuntimeNode item) { + return ctx.ProgramBuilder.Nth(item, 0); + }); + return list; +} + +TRuntimeNode ApplyPathRangesAndSampling(TRuntimeNode inputList, TType* itemType, const TExprNode& input, NCommon::TMkqlBuildContext& ctx) { + inputList = ApplyPathRanges(inputList, input, ctx); + TType* actualItemType = inputList.GetStaticType(); + if (actualItemType->IsStream()) { + actualItemType = AS_TYPE(TStreamType, actualItemType)->GetItemType(); + } else if (actualItemType->IsFlow()) { + actualItemType = AS_TYPE(TFlowType, actualItemType)->GetItemType(); + } else { + YQL_ENSURE(actualItemType->IsList()); + actualItemType = AS_TYPE(TListType, actualItemType)->GetItemType(); + } + + auto dropExtraMembers = [&](TRuntimeNode item, TStructType* structType, TStructType* actualStructType) { + TVector<TStringBuf> toDrop; + for (ui32 i = 0; i < actualStructType->GetMembersCount(); ++i) { + auto member = actualStructType->GetMemberName(i); + if (!structType->FindMemberIndex(member)) { + toDrop.push_back(member); + } + } + + for (auto& member : toDrop) { + item = ctx.ProgramBuilder.RemoveMember(item, member, true); + } + + return item; + }; + + if (itemType->IsStruct()) { + YQL_ENSURE(actualItemType->IsStruct()); + const auto structType = AS_TYPE(TStructType, itemType); + const auto actualStructType = AS_TYPE(TStructType, actualItemType); + YQL_ENSURE(actualStructType->GetMembersCount() >= structType->GetMembersCount()); + + if (actualStructType->GetMembersCount() > structType->GetMembersCount()) { + inputList = ctx.ProgramBuilder.Map(inputList, [&](TRuntimeNode item) { + return dropExtraMembers(item, structType, actualStructType); + }); + } + } else { + const auto varType = AS_TYPE(TVariantType, itemType); + const auto actualVarType = AS_TYPE(TVariantType, actualItemType); + + const auto tupleType = AS_TYPE(TTupleType, varType->GetUnderlyingType()); + const auto actualTupleType = AS_TYPE(TTupleType, actualVarType->GetUnderlyingType()); + + inputList = ctx.ProgramBuilder.Map(inputList, [&](TRuntimeNode varItem) { + return ctx.ProgramBuilder.VisitAll(varItem, [&](ui32 index, TRuntimeNode item) { + const auto structType = AS_TYPE(TStructType, tupleType->GetElementType(index)); + const auto actualStructType = AS_TYPE(TStructType, actualTupleType->GetElementType(index)); + YQL_ENSURE(actualStructType->GetMembersCount() >= structType->GetMembersCount()); + + item = dropExtraMembers(item, structType, actualStructType); + return ctx.ProgramBuilder.NewVariant(item, index, varType); + }); + }); + } + inputList = ApplySampling(inputList, input, ctx); + return inputList; +} + +TRuntimeNode ToList(TRuntimeNode list, NCommon::TMkqlBuildContext& ctx) { + const auto listType = list.GetStaticType(); + if (listType->IsOptional()) { + return ctx.ProgramBuilder.ToList(list); + } else if (listType->IsList()) { + return list; + } else if (listType->IsFlow() || listType->IsStream()) { + return ctx.ProgramBuilder.ForwardList(list); + } else { + YQL_ENSURE(false, "Expected list, stream or optional"); + } +} + +TType* BuildInputType(TYtSectionList input, NCommon::TMkqlBuildContext& ctx) { + TVector<TType*> items; + for (auto section: input) { + items.push_back(NCommon::BuildType(input.Ref(), *section.Ref().GetTypeAnn()->Cast<TListExprType>()->GetItemType(), ctx.ProgramBuilder)); + } + return items.size() == 1 + ? items.front() + : ctx.ProgramBuilder.NewVariantType(ctx.ProgramBuilder.NewTupleType(items)); +} + +TType* BuildOutputType(TYtOutSection output, NCommon::TMkqlBuildContext& ctx) { + TVector<TType*> items; + for (auto table: output) { + items.push_back(NCommon::BuildType(output.Ref(), *table.Ref().GetTypeAnn()->Cast<TListExprType>()->GetItemType(), ctx.ProgramBuilder)); + } + return items.size() == 1 + ? items.front() + : ctx.ProgramBuilder.NewVariantType(ctx.ProgramBuilder.NewTupleType(items)); +} + +TRuntimeNode ExpandFlow(TRuntimeNode flow, NCommon::TMkqlBuildContext& ctx) { + const auto structType = AS_TYPE(TStructType, AS_TYPE(TFlowType, flow.GetStaticType())->GetItemType()); + return ctx.ProgramBuilder.ExpandMap(flow, + [&](TRuntimeNode item) { + TRuntimeNode::TList fields; + fields.reserve(structType->GetMembersCount()); + auto i = 0U; + std::generate_n(std::back_inserter(fields), structType->GetMembersCount(), [&](){ return ctx.ProgramBuilder.Member(item, structType->GetMemberName(i++)); }); + return fields; + }); +} + +TRuntimeNode NarrowFlow(TRuntimeNode flow, const TStructType& structType, NCommon::TMkqlBuildContext& ctx) { + return ctx.ProgramBuilder.NarrowMap(flow, + [&](TRuntimeNode::TList items) { + TSmallVec<const std::pair<std::string_view, TRuntimeNode>> fields; + fields.reserve(structType.GetMembersCount()); + auto i = 0U; + std::transform(items.cbegin(), items.cend(), std::back_inserter(fields), [&](TRuntimeNode item) { + return std::make_pair(structType.GetMemberName(i++), item); + }); + return ctx.ProgramBuilder.NewStruct(fields); + }); +} + +TRuntimeNode NarrowFlowOutput(TPositionHandle pos, TRuntimeNode flow, const TStructExprType* type, NCommon::TMkqlBuildContext& ctx) { + if (const auto width = GetWideComponentsCount(AS_TYPE(TFlowType, flow.GetStaticType())); type->GetSize() < width) { + auto items = type->GetItems(); + auto i = 0U; + do items.emplace_back(ctx.ExprCtx.MakeType<TItemExprType>(TString("_yql_column_") += ToString(i++), ctx.ExprCtx.MakeType<TDataExprType>(EDataSlot::String))); + while (items.size() < width); + type = ctx.ExprCtx.MakeType<TStructExprType>(items); + } + + return NarrowFlow(flow, *AS_TYPE(TStructType, NCommon::BuildType(pos, *type, ctx.ProgramBuilder)), ctx); +} + +TRuntimeNode NarrowFlow(TRuntimeNode flow, TYtOutputOpBase op, NCommon::TMkqlBuildContext& ctx) { + auto type = op.Ref().GetTypeAnn()->Cast<TTupleExprType>()->GetItems().back()->Cast<TListExprType>()->GetItemType()->Cast<TStructExprType>(); + return NarrowFlowOutput(op.Pos(), flow, type, ctx); +} + +TRuntimeNode BuildTableInput(TType* outItemType, TStringBuf clusterName, const TExprNode& input, NCommon::TMkqlBuildContext& ctx, + const THashSet<TString>& extraSysColumns, bool forceKeyColumns) +{ + return BuildTableContentCall("YtTableInput", outItemType, clusterName, input, Nothing(), ctx, false, extraSysColumns, forceKeyColumns); +} + +} // unnamed + +TRuntimeNode ToStream(TRuntimeNode list, NCommon::TMkqlBuildContext& ctx) { + const auto listType = list.GetStaticType(); + if (listType->IsFlow()) { + return ctx.ProgramBuilder.FromFlow(list); + } else if (listType->IsOptional()) { + return ctx.ProgramBuilder.Iterator(ctx.ProgramBuilder.ToList(list), {}); + } else if (listType->IsList()) { + return ctx.ProgramBuilder.Iterator(list, {}); + } else if (listType->IsStream()) { + return list; + } else { + YQL_ENSURE(false, "Expected list, stream or optional"); + } +} + +TRuntimeNode SortListBy(TRuntimeNode list, const TVector<std::pair<TString, bool>>& sortBy, NCommon::TMkqlBuildContext& ctx) { + TRuntimeNode sortDirections; + std::function<TRuntimeNode(TRuntimeNode item)> keySelector; + if (sortBy.size() == 1) { + sortDirections = ctx.ProgramBuilder.NewDataLiteral(sortBy.front().second); + keySelector = [&ctx, &sortBy](TRuntimeNode item) { + return ctx.ProgramBuilder.Member(item, sortBy.front().first); + }; + } else { + TVector<TRuntimeNode> tupleItems{sortBy.size()}; + std::transform(sortBy.cbegin(), sortBy.cend(), tupleItems.begin(), [&ctx](const auto& it) { return ctx.ProgramBuilder.NewDataLiteral(it.second); }); + sortDirections = ctx.ProgramBuilder.NewTuple(tupleItems); + keySelector = [&ctx, &sortBy](TRuntimeNode item) { + TVector<TRuntimeNode> members; + for (auto& it: sortBy) { + members.push_back(ctx.ProgramBuilder.Member(item, it.first)); + } + return ctx.ProgramBuilder.NewTuple(members); + }; + } + + return ctx.ProgramBuilder.Sort(list, sortDirections, keySelector); +} + +TRuntimeNode BuildTableOutput(TRuntimeNode list, NCommon::TMkqlBuildContext& ctx) { + list = ToStream(list, ctx); + + TCallableBuilder fileWriteCall(ctx.ProgramBuilder.GetTypeEnvironment(), "YtWriteFile", ctx.ProgramBuilder.GetTypeEnvironment().GetTypeOfVoidLazy()); + fileWriteCall.Add(list); + + return ctx.ProgramBuilder.AsList(TRuntimeNode(fileWriteCall.Build(), false)); +} + +TRuntimeNode BuildDqWrite(TRuntimeNode item, TStringBuf path, NCommon::TMkqlBuildContext& ctx) { + TCallableBuilder fileWriteCall(ctx.ProgramBuilder.GetTypeEnvironment(), "DqWriteFile", ctx.ProgramBuilder.GetTypeEnvironment().GetTypeOfVoidLazy()); + fileWriteCall.Add(item); + fileWriteCall.Add(ctx.ProgramBuilder.NewDataLiteral<NUdf::EDataSlot::String>(path)); + + return TRuntimeNode(fileWriteCall.Build(), false); +} + +TRuntimeNode BuildRuntimeTableInput(TStringBuf callName, + TType* outItemType, + TStringBuf clusterName, + TStringBuf tableName, + TStringBuf spec, + bool isTemp, + NCommon::TMkqlBuildContext& ctx) +{ + auto outListType = ctx.ProgramBuilder.NewListType(outItemType); + TType* const strType = ctx.ProgramBuilder.NewDataType(NUdf::TDataType<char*>::Id); + TType* const boolType = ctx.ProgramBuilder.NewDataType(NUdf::TDataType<bool>::Id); + TType* const ui64Type = ctx.ProgramBuilder.NewDataType(NUdf::TDataType<ui64>::Id); + TType* const ui32Type = ctx.ProgramBuilder.NewDataType(NUdf::TDataType<ui32>::Id); + TType* const tupleTypeTables = ctx.ProgramBuilder.NewTupleType({strType, boolType, strType, ui64Type, ui64Type, boolType, ui32Type}); + TType* const listTypeGroup = ctx.ProgramBuilder.NewListType(tupleTypeTables); + + TCallableBuilder call(ctx.ProgramBuilder.GetTypeEnvironment(), callName, outListType); + call.Add(ctx.ProgramBuilder.NewDataLiteral<NUdf::EDataSlot::String>(clusterName)); // cluster name + + TVector<TRuntimeNode> groups; + groups.push_back( + ctx.ProgramBuilder.NewList(tupleTypeTables, {ctx.ProgramBuilder.NewTuple(tupleTypeTables, { + ctx.ProgramBuilder.NewDataLiteral<NUdf::EDataSlot::String>(NYT::NodeToYsonString(NYT::PathToNode(NYT::TRichYPath(TString{tableName})))), + ctx.ProgramBuilder.NewDataLiteral(isTemp), + ctx.ProgramBuilder.NewDataLiteral<NUdf::EDataSlot::String>(spec), + ctx.ProgramBuilder.NewDataLiteral(ui64(1)), + ctx.ProgramBuilder.NewDataLiteral(ui64(1)), + ctx.ProgramBuilder.NewDataLiteral(false), + ctx.ProgramBuilder.NewDataLiteral(ui32(0)), + })}) + ); + + call.Add(ctx.ProgramBuilder.NewList(listTypeGroup, groups)); + call.Add(ctx.ProgramBuilder.NewEmptyTuple()); // Sampling + call.Add(ctx.ProgramBuilder.NewEmptyTuple()); // length + + return TRuntimeNode(call.Build(), false); +} + +void RegisterYtFileMkqlCompilers(NCommon::TMkqlCallableCompilerBase& compiler) { + compiler.OverrideCallable(TYtTableContent::CallableName(), + [](const TExprNode& node, NCommon::TMkqlBuildContext& ctx) { + TYtTableContent tableContent(&node); + TMaybe<ui64> itemsCount; + if (auto setting = NYql::GetSetting(tableContent.Settings().Ref(), EYtSettingType::ItemsCount)) { + itemsCount = FromString<ui64>(setting->Child(1)->Content()); + } + const auto itemType = NCommon::BuildType(node, *node.GetTypeAnn()->Cast<TListExprType>()->GetItemType(), ctx.ProgramBuilder); + TRuntimeNode values; + if (auto maybeRead = tableContent.Input().Maybe<TYtReadTable>()) { + auto read = maybeRead.Cast(); + + const bool hasRangesOrSampling = AnyOf(read.Input(), [](const TYtSection& s) { + return NYql::HasSetting(s.Settings().Ref(), EYtSettingType::Sample) + || AnyOf(s.Paths(), [](const TYtPath& p) { return !p.Ranges().Maybe<TCoVoid>(); }); + }); + if (hasRangesOrSampling) { + itemsCount.Clear(); + } + + const bool forceKeyColumns = HasRangesWithKeyColumns(read.Input().Ref()); + values = BuildTableContentCall( + TYtTableContent::CallableName(), + itemType, + read.DataSource().Cluster().Value(), read.Input().Ref(), itemsCount, ctx, true, THashSet<TString>{"num", "index"}, forceKeyColumns); + values = ApplyPathRangesAndSampling(values, itemType, read.Input().Ref(), ctx); + } else { + auto output = tableContent.Input().Cast<TYtOutput>(); + values = BuildTableContentCall( + TYtTableContent::CallableName(), + itemType, + GetOutputOp(output).DataSink().Cluster().Value(), output.Ref(), itemsCount, ctx, true); + } + + return values; + }); + + compiler.AddCallable({TYtSort::CallableName(), TYtCopy::CallableName(), TYtMerge::CallableName()}, + [](const TExprNode& node, NCommon::TMkqlBuildContext& ctx) { + + TYtTransientOpBase ytOp(&node); + TYtOutTableInfo outTableInfo(ytOp.Output().Item(0)); + + TMaybe<ui64> limit; + if (ytOp.Maybe<TYtSort>()) { + limit = GetLimit(ytOp.Settings().Ref()); + } + + const TStructExprType* inputType = ytOp.Input().Item(0).Ref().GetTypeAnn()->Cast<TListExprType>()->GetItemType()->Cast<TStructExprType>(); + auto mkqlInputType = NCommon::BuildType(ytOp.Input().Ref(), *inputType, ctx.ProgramBuilder); + for (size_t i: xrange(outTableInfo.RowSpec->SortedBy.size())) { + if (!inputType->FindItem(outTableInfo.RowSpec->SortedBy[i])) { + mkqlInputType = ctx.ProgramBuilder.NewStructType(mkqlInputType, outTableInfo.RowSpec->SortedBy[i], + NCommon::BuildType(ytOp.Input().Ref(), *outTableInfo.RowSpec->SortedByTypes[i], ctx.ProgramBuilder)); + } + } + + const bool forceKeyColumns = HasRangesWithKeyColumns(ytOp.Input().Ref()); + TRuntimeNode values = BuildTableInput(mkqlInputType, + ytOp.DataSink().Cluster().Value(), ytOp.Input().Ref(), ctx, THashSet<TString>{"num", "index"}, forceKeyColumns); + + values = ApplyPathRangesAndSampling(values, mkqlInputType, ytOp.Input().Ref(), ctx); + + if ((ytOp.Maybe<TYtMerge>() && outTableInfo.RowSpec->IsSorted() && ytOp.Input().Item(0).Paths().Size() > 1) + || ytOp.Maybe<TYtSort>()) + { + values = SortListBy(values, outTableInfo.RowSpec->GetForeignSort(), ctx); + } + if (limit) { + values = ctx.ProgramBuilder.Take(values, ctx.ProgramBuilder.NewDataLiteral(*limit)); + } + auto res = BuildTableOutput(values, ctx); + return res; + }); + + compiler.AddCallable(TYtMap::CallableName(), + [](const TExprNode& node, NCommon::TMkqlBuildContext& ctx) { + + TYtMap ytMap(&node); + + const auto itemType = BuildInputType(ytMap.Input(), ctx); + const bool forceKeyColumns = HasRangesWithKeyColumns(ytMap.Input().Ref()); + TRuntimeNode values = BuildTableInput( + itemType, + ytMap.DataSink().Cluster().Value(), ytMap.Input().Ref(), ctx, + THashSet<TString>{"num", "index"}, forceKeyColumns); + + const auto arg = ytMap.Mapper().Args().Arg(0).Raw(); + values = arg->GetTypeAnn()->GetKind() == ETypeAnnotationKind::Flow ? + ctx.ProgramBuilder.ToFlow(values) : ctx.ProgramBuilder.Iterator(values, {}); + values = ApplyPathRangesAndSampling(values, itemType, ytMap.Input().Ref(), ctx); + + auto& lambdaInputType = GetSeqItemType(*ytMap.Mapper().Args().Arg(0).Ref().GetTypeAnn()); + + if (lambdaInputType.GetKind() == ETypeAnnotationKind::Multi) { + values = ExpandFlow(values, ctx); + } + + if (IsWideBlockType(lambdaInputType)) { + values = ctx.ProgramBuilder.WideToBlocks(values); + } + + NCommon::TMkqlBuildContext innerCtx(ctx, {{arg, values}}, ytMap.Mapper().Ref().UniqueId()); + values = NCommon::MkqlBuildExpr(ytMap.Mapper().Body().Ref(), innerCtx); + + if (ETypeAnnotationKind::Multi == GetSeqItemType(*ytMap.Mapper().Body().Ref().GetTypeAnn()).GetKind()) + values = NarrowFlow(values, ytMap, ctx); + + auto res = BuildTableOutput(values, ctx); + return res; + }); + + compiler.AddCallable(TYtReduce::CallableName(), + [](const TExprNode& node, NCommon::TMkqlBuildContext& ctx) { + + TYtReduce ytReduce(&node); + + TType* itemType = BuildInputType(ytReduce.Input(), ctx); + const bool multiSection = itemType->GetKind() == TType::EKind::Variant; + + const bool forceKeyColumns = HasRangesWithKeyColumns(ytReduce.Input().Ref()); + TRuntimeNode values = BuildTableInput(itemType, + ytReduce.DataSink().Cluster().Value(), ytReduce.Input().Ref(), ctx, + THashSet<TString>{"num", "index"}, forceKeyColumns); + + values = ApplyPathRangesAndSampling(values, itemType, ytReduce.Input().Ref(), ctx); + + TVector<TString> reduceBy = NYql::GetSettingAsColumnList(ytReduce.Settings().Ref(), EYtSettingType::ReduceBy); + TVector<std::pair<TString, bool>> sortBy = NYql::GetSettingAsColumnPairList(ytReduce.Settings().Ref(), EYtSettingType::SortBy); + TVector<bool> opt; + if (multiSection) { + opt.resize(Max(reduceBy.size(), sortBy.size()), false); + auto varType = AS_TYPE(TVariantType, AS_TYPE(TListType, values)->GetItemType()); + for (ui32 i = 0; i < varType->GetAlternativesCount(); ++i) { + TStructType* structType = AS_TYPE(TStructType, varType->GetAlternativeType(i)); + for (size_t c: xrange(reduceBy.size())) { + opt[c] = opt[c] || structType->GetMemberType(structType->GetMemberIndex(reduceBy[c]))->IsOptional(); + } + if (!sortBy.empty() && sortBy.size() > reduceBy.size()) { + for (size_t c: xrange(reduceBy.size(), sortBy.size())) { + opt[c] = opt[c] || structType->GetMemberType(structType->GetMemberIndex(sortBy[c].first))->IsOptional(); + } + } + } + } + + auto dict = ctx.ProgramBuilder.ToHashedDict(values, true, [&](TRuntimeNode item) { + if (multiSection) { + return ctx.ProgramBuilder.VisitAll(item, [&](ui32, TRuntimeNode varItem) { + TVector<TRuntimeNode> keyItems; + for (size_t c: xrange(reduceBy.size())) { + auto key = ctx.ProgramBuilder.Member(varItem, reduceBy[c]); + if (opt[c] && !key.GetStaticType()->IsOptional()) { + key = ctx.ProgramBuilder.NewOptional(key); + } + keyItems.push_back(key); + } + return keyItems.size() == 1 + ? keyItems.front() + : ctx.ProgramBuilder.NewTuple(keyItems); + }); + } + TVector<TRuntimeNode> keyItems; + for (auto& column: reduceBy) { + keyItems.push_back(ctx.ProgramBuilder.Member(item, column)); + } + return keyItems.size() == 1 + ? keyItems.front() + : ctx.ProgramBuilder.NewTuple(keyItems); + }, [&](TRuntimeNode item) { + return item; + }); + + values = ctx.ProgramBuilder.DictPayloads(dict); + if (!sortBy.empty() && sortBy.size() > reduceBy.size()) { + sortBy.erase(sortBy.begin(), sortBy.begin() + reduceBy.size()); + } + + // sort partial lists + if (!sortBy.empty() || multiSection) { + size_t keySize = sortBy.size() + multiSection; + + TRuntimeNode sortDirections; + if (keySize > 1) { + TVector<TRuntimeNode> tupleItems(keySize, ctx.ProgramBuilder.NewDataLiteral(true)); + std::transform(sortBy.cbegin(), sortBy.cend(), tupleItems.begin(), [&ctx](const auto& it) { return ctx.ProgramBuilder.NewDataLiteral(it.second); }); + sortDirections = ctx.ProgramBuilder.NewTuple(tupleItems); + } else { + sortDirections = ctx.ProgramBuilder.NewDataLiteral(sortBy.empty() || sortBy.front().second); + } + + values = ctx.ProgramBuilder.Map(values, [&](TRuntimeNode list) { + list = ctx.ProgramBuilder.Sort(list, sortDirections, [&](TRuntimeNode item) { + if (multiSection) { + return ctx.ProgramBuilder.VisitAll(item, [&](ui32 ndx, TRuntimeNode varItem) { + TVector<TRuntimeNode> keyItems; + for (size_t c: xrange(sortBy.size())) { + auto key = ctx.ProgramBuilder.Member(varItem, sortBy[c].first); + if (opt[c + reduceBy.size()] && !key.GetStaticType()->IsOptional()) { + key = ctx.ProgramBuilder.NewOptional(key); + } + keyItems.push_back(key); + } + keyItems.push_back(ctx.ProgramBuilder.NewDataLiteral(ndx)); + return keyItems.size() == 1 + ? keyItems.front() + : ctx.ProgramBuilder.NewTuple(keyItems); + }); + } + TVector<TRuntimeNode> keyItems; + for (auto& column: sortBy) { + keyItems.push_back(ctx.ProgramBuilder.Member(item, column.first)); + } + return keyItems.size() == 1 + ? keyItems.front() + : ctx.ProgramBuilder.NewTuple(keyItems); + }); + + return list; + }); + } + + const auto arg = ytReduce.Reducer().Args().Arg(0).Raw(); + if (NYql::HasSetting(ytReduce.Settings().Ref(), EYtSettingType::KeySwitch)) { + itemType = multiSection ? + NCommon::BuildType(ytReduce.Reducer().Ref(), GetSeqItemType(*arg->GetTypeAnn()), ctx.ProgramBuilder): + ctx.ProgramBuilder.NewStructType(itemType, YqlSysColumnKeySwitch, ctx.ProgramBuilder.NewDataType(EDataSlot::Bool)); + values = ctx.ProgramBuilder.Map(values, [&](TRuntimeNode list) { + list = ctx.ProgramBuilder.Enumerate(list); + list = ctx.ProgramBuilder.Map(list, [&](TRuntimeNode item) { + auto grpSwitch = ctx.ProgramBuilder.Equals(ctx.ProgramBuilder.Nth(item, 0), ctx.ProgramBuilder.NewDataLiteral(ui64(0))); + if (multiSection) { + return ctx.ProgramBuilder.VisitAll(ctx.ProgramBuilder.Nth(item, 1), [&](ui32 ndx, TRuntimeNode varItem) { + return ctx.ProgramBuilder.NewVariant(ctx.ProgramBuilder.AddMember(varItem, YqlSysColumnKeySwitch, grpSwitch), ndx, itemType); + }); + } + return ctx.ProgramBuilder.AddMember(ctx.ProgramBuilder.Nth(item, 1), YqlSysColumnKeySwitch, grpSwitch); + }); + return list; + }); + } + + TCallableBuilder callableBuilder(ctx.ProgramBuilder.GetTypeEnvironment(), "YtUngroupingList", + ctx.ProgramBuilder.NewListType(itemType)); + callableBuilder.Add(values); + values = TRuntimeNode(callableBuilder.Build(), false); + + values = arg->GetTypeAnn()->GetKind() == ETypeAnnotationKind::Flow ? + ctx.ProgramBuilder.ToFlow(values) : ctx.ProgramBuilder.Iterator(values, {}); + + if (ETypeAnnotationKind::Multi == GetSeqItemType(*ytReduce.Reducer().Args().Arg(0).Ref().GetTypeAnn()).GetKind()) + values = ExpandFlow(values, ctx); + + NCommon::TMkqlBuildContext innerCtx(ctx, {{arg, values}}, ytReduce.Reducer().Ref().UniqueId()); + + values = NCommon::MkqlBuildExpr(ytReduce.Reducer().Body().Ref(), innerCtx); + + if (ETypeAnnotationKind::Multi == GetSeqItemType(*ytReduce.Reducer().Body().Ref().GetTypeAnn()).GetKind()) + values = NarrowFlow(values, ytReduce, ctx); + + // TODO: preserve sorting in reduce processing instead of sorting according to output spec + TYtOutTableInfo outTableInfo(ytReduce.Output().Item(0)); + if (outTableInfo.RowSpec->IsSorted()) { + values = SortListBy(values, outTableInfo.RowSpec->GetForeignSort(), ctx); + } + + auto res = BuildTableOutput(values, ctx); + return res; + }); + + compiler.AddCallable(TYtMapReduce::CallableName(), + [](const TExprNode& node, NCommon::TMkqlBuildContext& ctx) { + + TYtMapReduce ytMapReduce(&node); + + bool hasMap = !ytMapReduce.Mapper().Maybe<TCoVoid>(); + const auto itemType = BuildInputType(ytMapReduce.Input(), ctx); + const bool forceKeyColumns = HasRangesWithKeyColumns(ytMapReduce.Input().Ref()); + TRuntimeNode values = BuildTableInput(itemType, + ytMapReduce.DataSink().Cluster().Value(), ytMapReduce.Input().Ref(), ctx, + THashSet<TString>{"num", "index"}, forceKeyColumns); + + values = ApplyPathRangesAndSampling(values, itemType, ytMapReduce.Input().Ref(), ctx); + + const auto outputItemType = BuildOutputType(ytMapReduce.Output(), ctx); + const size_t outputsCount = ytMapReduce.Output().Ref().ChildrenSize(); + + size_t mapDirectOutputsCount = 0; + TRuntimeNode mapDirectOutputs; + if (hasMap) { + const auto& mapper = ytMapReduce.Mapper().Cast<TCoLambda>(); + if (const auto arg = mapper.Args().Arg(0).Raw(); arg != mapper.Body().Raw()) { + values = arg->GetTypeAnn()->GetKind() == ETypeAnnotationKind::Flow ? + ctx.ProgramBuilder.ToFlow(values) : ctx.ProgramBuilder.Iterator(values, {}); + + if (ETypeAnnotationKind::Multi == GetSeqItemType(*ytMapReduce.Mapper().Cast<TCoLambda>().Args().Arg(0).Ref().GetTypeAnn()).GetKind()) + values = ExpandFlow(values, ctx); + + NCommon::TMkqlBuildContext innerCtx(ctx, {{arg, values}}, ytMapReduce.Mapper().Ref().UniqueId()); + + const auto& body = ytMapReduce.Mapper().Cast<TCoLambda>().Body().Ref(); + values = NCommon::MkqlBuildExpr(body, innerCtx); + + const auto& mapOutItemType = GetSeqItemType(*body.GetTypeAnn()); + if (const auto mapOutputTypeSetting = NYql::GetSetting(ytMapReduce.Settings().Ref(), EYtSettingType::MapOutputType)) { + if (ETypeAnnotationKind::Multi == mapOutItemType.GetKind()) { + values = NarrowFlow(values, *AS_TYPE(TStructType, NCommon::BuildType(body, *mapOutputTypeSetting->Tail().GetTypeAnn()->Cast<TTypeExprType>()->GetType(), ctx.ProgramBuilder)), ctx); + } + } + + values = ToList(values, ctx); + + if (mapOutItemType.GetKind() == ETypeAnnotationKind::Variant) { + auto tupleType = mapOutItemType.Cast<TVariantExprType>()->GetUnderlyingType()->Cast<TTupleExprType>(); + YQL_ENSURE(tupleType->GetSize() > 0); + mapDirectOutputsCount = tupleType->GetSize() - 1; + YQL_ENSURE(mapDirectOutputsCount < outputsCount); + + values = ctx.ProgramBuilder.Collect(values); + + mapDirectOutputs = ctx.ProgramBuilder.OrderedFlatMap(values, [&](TRuntimeNode mapOut) { + return ctx.ProgramBuilder.VisitAll(mapOut, [&](ui32 index, TRuntimeNode varitem) { + if (index == 0) { + return ctx.ProgramBuilder.NewEmptyOptional(ctx.ProgramBuilder.NewOptionalType(outputItemType)); + } + return ctx.ProgramBuilder.NewOptional(ctx.ProgramBuilder.NewVariant(varitem, index - 1, outputItemType)); + }); + }); + + auto toReduceType = NCommon::BuildType(body, *tupleType->GetItems().front(), ctx.ProgramBuilder); + values = ctx.ProgramBuilder.OrderedFlatMap(values, [&](TRuntimeNode mapOut) { + return ctx.ProgramBuilder.VisitAll(mapOut, [&](ui32 index, TRuntimeNode varitem) { + if (index == 0) { + return ctx.ProgramBuilder.NewOptional(varitem); + } + return ctx.ProgramBuilder.NewEmptyOptional(ctx.ProgramBuilder.NewOptionalType(toReduceType)); + }); + }); + } + } + } + + TVector<TString> reduceBy = NYql::GetSettingAsColumnList(ytMapReduce.Settings().Ref(), EYtSettingType::ReduceBy); + auto dict = ctx.ProgramBuilder.ToHashedDict(values, true, [&](TRuntimeNode item) { + TVector<TRuntimeNode> keyItems; + for (auto& column: reduceBy) { + keyItems.push_back(ctx.ProgramBuilder.Member(item, column)); + } + return keyItems.size() == 1 + ? keyItems.front() + : ctx.ProgramBuilder.NewTuple(keyItems); + }, [&](TRuntimeNode item) { + return item; + }); + + values = ctx.ProgramBuilder.DictPayloads(dict); + TVector<std::pair<TString, bool>> sortBy = NYql::GetSettingAsColumnPairList(ytMapReduce.Settings().Ref(), EYtSettingType::SortBy); + TVector<TString> filterBy = NYql::GetSettingAsColumnList(ytMapReduce.Settings().Ref(), EYtSettingType::ReduceFilterBy); + if (!sortBy.empty() && sortBy.size() > reduceBy.size()) { + // sort partial lists + sortBy.erase(sortBy.begin(), sortBy.begin() + reduceBy.size()); + + TRuntimeNode sortDirections; + if (sortBy.size() > 1) { + TVector<TRuntimeNode> tupleItems(sortBy.size(), ctx.ProgramBuilder.NewDataLiteral(true)); + std::transform(sortBy.cbegin(), sortBy.cend(), tupleItems.begin(), [&ctx](const auto& it) { return ctx.ProgramBuilder.NewDataLiteral(it.second); }); + sortDirections = ctx.ProgramBuilder.NewTuple(tupleItems); + } else { + sortDirections = ctx.ProgramBuilder.NewDataLiteral(sortBy.front().second); + } + + values = ctx.ProgramBuilder.Map(values, [&](TRuntimeNode list) { + list = ctx.ProgramBuilder.Sort(list, sortDirections, [&](TRuntimeNode item) { + TVector<TRuntimeNode> keyItems; + for (auto& column: sortBy) { + keyItems.push_back(ctx.ProgramBuilder.Member(item, column.first)); + } + return keyItems.size() == 1 + ? keyItems.front() + : ctx.ProgramBuilder.NewTuple(keyItems); + }); + + if (NYql::HasSetting(ytMapReduce.Settings().Ref(), EYtSettingType::ReduceFilterBy)) { + list = ctx.ProgramBuilder.OrderedMap(list, [&filterBy, &ctx] (TRuntimeNode item) { + TRuntimeNode res = ctx.ProgramBuilder.NewEmptyStruct(); + for (auto& column: filterBy) { + res = ctx.ProgramBuilder.AddMember(res, column, ctx.ProgramBuilder.Member(item, column)); + } + return res; + }); + } + + return list; + }); + } + else if (!filterBy.empty()) { + values = ctx.ProgramBuilder.Map(values, [&](TRuntimeNode list) { + list = ctx.ProgramBuilder.OrderedMap(list, [&filterBy, &ctx] (TRuntimeNode item) { + TRuntimeNode res = ctx.ProgramBuilder.NewEmptyStruct(); + for (auto& column: filterBy) { + res = ctx.ProgramBuilder.AddMember(res, column, ctx.ProgramBuilder.Member(item, column)); + } + return res; + }); + + return list; + }); + } + + const auto arg = ytMapReduce.Reducer().Args().Arg(0).Raw(); + const auto reduceInputTypeSetting = NYql::GetSetting(ytMapReduce.Settings().Ref(), EYtSettingType::ReduceInputType); + const auto reduceInputType = reduceInputTypeSetting ? reduceInputTypeSetting->Tail().GetTypeAnn()->Cast<TTypeExprType>()->GetType() : &GetSeqItemType(*arg->GetTypeAnn()); + TType* mkqlItemType = NCommon::BuildType(ytMapReduce.Reducer().Ref(), *reduceInputType, ctx.ProgramBuilder); + if (NYql::HasSetting(ytMapReduce.Settings().Ref(), EYtSettingType::KeySwitch)) { + values = ctx.ProgramBuilder.Map(values, [&](TRuntimeNode list) { + list = ctx.ProgramBuilder.Enumerate(list); + list = ctx.ProgramBuilder.Map(list, [&](TRuntimeNode item) { + auto grpSwitch = ctx.ProgramBuilder.Equals(ctx.ProgramBuilder.Nth(item, 0), ctx.ProgramBuilder.NewDataLiteral(ui64(0))); + return ctx.ProgramBuilder.AddMember(ctx.ProgramBuilder.Nth(item, 1), YqlSysColumnKeySwitch, grpSwitch); + }); + return list; + }); + } + + TCallableBuilder callableBuilder(ctx.ProgramBuilder.GetTypeEnvironment(), "YtUngroupingList", + ctx.ProgramBuilder.NewListType(mkqlItemType)); + callableBuilder.Add(values); + values = TRuntimeNode(callableBuilder.Build(), false); + + values = arg->GetTypeAnn()->GetKind() == ETypeAnnotationKind::Flow ? + ctx.ProgramBuilder.ToFlow(values) : ctx.ProgramBuilder.Iterator(values, {}); + + if (ETypeAnnotationKind::Multi == GetSeqItemType(*arg->GetTypeAnn()).GetKind()) + values = ExpandFlow(values, ctx); + + NCommon::TMkqlBuildContext innerCtx(ctx, {{arg, values}}, ytMapReduce.Reducer().Ref().UniqueId()); + + values = NCommon::MkqlBuildExpr(ytMapReduce.Reducer().Body().Ref(), innerCtx); + + const auto& reduceOutItemType = GetSeqItemType(*ytMapReduce.Reducer().Body().Ref().GetTypeAnn()); + + if (ETypeAnnotationKind::Multi == reduceOutItemType.GetKind()) { + auto type = ytMapReduce.Ref().GetTypeAnn()->Cast<TTupleExprType>()->GetItems().back()->Cast<TListExprType>()->GetItemType(); + if (type->GetKind() == ETypeAnnotationKind::Variant) { + type = type->Cast<TVariantExprType>()->GetUnderlyingType()->Cast<TTupleExprType>()->GetItems().back(); + } + values = NarrowFlowOutput(ytMapReduce.Pos(), values, type->Cast<TStructExprType>(), ctx); + } + + if (mapDirectOutputsCount > 0) { + // remap reduce output to new indexes + values = ctx.ProgramBuilder.OrderedMap(values, [&](TRuntimeNode redueOutItem) { + if (reduceOutItemType.GetKind() == ETypeAnnotationKind::Variant) { + return ctx.ProgramBuilder.VisitAll(redueOutItem, [&](ui32 idx, TRuntimeNode item) { + return ctx.ProgramBuilder.NewVariant(item, idx + mapDirectOutputsCount, outputItemType); + }); + } + return ctx.ProgramBuilder.NewVariant(redueOutItem, mapDirectOutputsCount, outputItemType); + }); + + // prepend with map output + values = ctx.ProgramBuilder.Extend({mapDirectOutputs, ToList(values, ctx)}); + } + + auto res = BuildTableOutput(values, ctx); + return res; + }); + + compiler.AddCallable(TYtFill::CallableName(), + [](const TExprNode& node, NCommon::TMkqlBuildContext& ctx) { + + TYtFill ytFill(&node); + + auto values = NCommon::MkqlBuildExpr(ytFill.Content().Body().Ref(), ctx); + + if (ETypeAnnotationKind::Multi == GetSeqItemType(*ytFill.Content().Body().Ref().GetTypeAnn()).GetKind()) + values = NarrowFlow(values, ytFill, ctx); + + auto res = BuildTableOutput(values, ctx); + return res; + }); + + compiler.AddCallable("Pull", + [](const TExprNode& node, NCommon::TMkqlBuildContext& ctx) { + TPull pull(&node); + auto clusterName = GetClusterName(pull.Input()); + const auto itemType = NCommon::BuildType(pull.Input().Ref(), *pull.Input().Ref().GetTypeAnn()->Cast<TListExprType>()->GetItemType(), ctx.ProgramBuilder); + if (auto out = pull.Input().Maybe<TYtOutput>()) { + + return BuildTableInput( + itemType, + clusterName, pull.Input().Ref(), ctx, THashSet<TString>{}, false); + + } else { + auto read = pull.Input().Maybe<TCoRight>().Input().Maybe<TYtReadTable>(); + YQL_ENSURE(read, "Unknown operation input"); + + const bool forceKeyColumns = HasRangesWithKeyColumns(read.Cast().Input().Ref()); + TRuntimeNode values = BuildTableInput( + itemType, + clusterName, read.Cast().Input().Ref(), ctx, + THashSet<TString>{"num", "index"}, forceKeyColumns); + + values = ApplyPathRangesAndSampling(values, itemType, read.Cast().Input().Ref(), ctx); + + return values; + } + }); +} // RegisterYtFileMkqlCompilers + + +void RegisterDqYtFileMkqlCompilers(NCommon::TMkqlCallableCompilerBase& compiler) { + compiler.OverrideCallable(TDqReadWideWrap::CallableName(), + [](const TExprNode& node, NCommon::TMkqlBuildContext& ctx) { + const auto wrapper = TDqReadWrapBase(&node); + if (wrapper.Input().Maybe<TYtReadTable>().IsValid()) { + auto ytRead = wrapper.Input().Cast<TYtReadTable>(); + auto cluster = TString{ytRead.DataSource().Cluster().Value()}; + const auto outputType = NCommon::BuildType(wrapper.Ref(), + *ytRead.Input().Ref().GetTypeAnn()->Cast<TTupleExprType>()->GetItems()[0]->Cast<TListExprType>()->GetItemType(), ctx.ProgramBuilder); + + const bool forceKeyColumns = HasRangesWithKeyColumns(ytRead.Input().Ref()); + auto values = BuildTableContentCall("YtTableInputFile", outputType, cluster, + ytRead.Input().Ref(), Nothing(), ctx, false, THashSet<TString>{"num", "index"}, forceKeyColumns); + values = ApplyPathRangesAndSampling(values, outputType, ytRead.Input().Ref(), ctx); + + return ExpandFlow(ctx.ProgramBuilder.ToFlow(values), ctx); + } + + return TRuntimeNode(); + }); + + compiler.OverrideCallable(TDqReadBlockWideWrap::CallableName(), + [](const TExprNode& node, NCommon::TMkqlBuildContext& ctx) { + const auto wrapper = TDqReadWrapBase(&node); + if (wrapper.Input().Maybe<TYtReadTable>().IsValid()) { + auto ytRead = wrapper.Input().Cast<TYtReadTable>(); + auto cluster = TString{ytRead.DataSource().Cluster().Value()}; + const auto outputType = NCommon::BuildType(wrapper.Ref(), + *ytRead.Input().Ref().GetTypeAnn()->Cast<TTupleExprType>()->GetItems()[0]->Cast<TListExprType>()->GetItemType(), ctx.ProgramBuilder); + + const bool forceKeyColumns = HasRangesWithKeyColumns(ytRead.Input().Ref()); + auto values = BuildTableContentCall("YtTableInputFile", outputType, cluster, + ytRead.Input().Ref(), Nothing(), ctx, false, THashSet<TString>{"num", "index"}, forceKeyColumns); + values = ApplyPathRangesAndSampling(values, outputType, ytRead.Input().Ref(), ctx); + return ctx.ProgramBuilder.FromFlow(ctx.ProgramBuilder.WideToBlocks(ExpandFlow(ctx.ProgramBuilder.ToFlow(values), ctx))); + } + + return TRuntimeNode(); + }); + + compiler.OverrideCallable(TYtDqWideWrite::CallableName(), + [](const TExprNode& node, NCommon::TMkqlBuildContext& ctx) { + const auto wideWrite = TYtDqWideWrite(&node); + + auto values = NCommon::MkqlBuildExpr(wideWrite.Input().Ref(), ctx); + + auto tableName = GetSetting(wideWrite.Settings().Ref(), "tableName")->Child(1)->Content(); + auto tableType = GetSetting(wideWrite.Settings().Ref(), "tableType")->Child(1)->Content(); + + TStringStream err; + auto inputItemType = NCommon::ParseTypeFromYson(tableType, ctx.ProgramBuilder, err); + YQL_ENSURE(inputItemType, "Parse type error: " << err.Str()); + + auto structType = AS_TYPE(TStructType, inputItemType); + values = NarrowFlow(values, *structType, ctx); + values = ctx.ProgramBuilder.Map(values, [&](TRuntimeNode item) { + return BuildDqWrite(item, tableName, ctx); + }); + return values; + }); +} + +} // NYql diff --git a/yt/yql/providers/yt/gateway/file/yql_yt_file_mkql_compiler.h b/yt/yql/providers/yt/gateway/file/yql_yt_file_mkql_compiler.h new file mode 100644 index 0000000000..f98bdc7139 --- /dev/null +++ b/yt/yql/providers/yt/gateway/file/yql_yt_file_mkql_compiler.h @@ -0,0 +1,24 @@ +#pragma once + +#include <yql/essentials/providers/common/mkql/yql_provider_mkql.h> +#include <yql/essentials/minikql/mkql_node.h> +#include <yql/essentials/ast/yql_expr.h> + +#include <util/generic/strbuf.h> +#include <util/generic/vector.h> +#include <util/generic/string.h> + +#include <utility> + +namespace NYql { + +NKikimr::NMiniKQL::TRuntimeNode BuildTableOutput(NKikimr::NMiniKQL::TRuntimeNode list, NCommon::TMkqlBuildContext& ctx); +NKikimr::NMiniKQL::TRuntimeNode BuildRuntimeTableInput(TStringBuf callName, NKikimr::NMiniKQL::TType* outItemType, TStringBuf clusterName, + TStringBuf tableName, TStringBuf spec, bool isTemp, NCommon::TMkqlBuildContext& ctx); + +NKikimr::NMiniKQL::TRuntimeNode SortListBy(NKikimr::NMiniKQL::TRuntimeNode list, const TVector<std::pair<TString, bool>>& sortBy, NCommon::TMkqlBuildContext& ctx); + +void RegisterYtFileMkqlCompilers(NCommon::TMkqlCallableCompilerBase& compiler); +void RegisterDqYtFileMkqlCompilers(NCommon::TMkqlCallableCompilerBase& compiler); + +} diff --git a/yt/yql/providers/yt/gateway/file/yql_yt_file_services.cpp b/yt/yql/providers/yt/gateway/file/yql_yt_file_services.cpp new file mode 100644 index 0000000000..cc8458ce0d --- /dev/null +++ b/yt/yql/providers/yt/gateway/file/yql_yt_file_services.cpp @@ -0,0 +1,92 @@ +#include "yql_yt_file_services.h" + +#include <yql/essentials/providers/common/provider/yql_provider_names.h> + +#include <util/system/guard.h> +#include <util/system/fs.h> +#include <util/generic/yexception.h> +#include <util/folder/path.h> +#include <util/string/cast.h> +#include <util/random/random.h> +#include <util/stream/file.h> +#include <util/stream/input.h> + +#include <algorithm> +#include <iterator> + +namespace NYql::NFile { + +TYtFileServices::~TYtFileServices() { + for (auto& x: Locks) { + try { + NFs::Remove(x.second); + NFs::Remove(x.second + ".attr"); + } catch (...) { + } + } +} + +TString TYtFileServices::GetTablePath(TStringBuf cluster, TStringBuf table, bool isTemp, bool noLocks) { + if (isTemp) { + return TString(TFsPath(TmpDir) / TString(table.substr(4)).append(TStringBuf(".tmp"))); + } + + const auto tablePrefix = TString(YtProviderName).append('.').append(cluster); + const auto fullTableName = TString(tablePrefix).append('.').append(table); + if (!noLocks) { + auto guard = Guard(Mutex); + if (auto p = Locks.FindPtr(fullTableName)) { + return *p; + } + } + if (auto p = TablesMapping.FindPtr(fullTableName)) { + return *p; + } + if (auto dirPtr = TablesDirMapping.FindPtr(tablePrefix)) { + return TFsPath(*dirPtr) / TString(table).append(".txt"); + } + ythrow yexception() << "Table \"" << table << "\" does not exist"; +} + +void TYtFileServices::LockPath(const TString& path, const TString& fullTableName) { + auto name = TFsPath(path).GetName(); + auto lockPath = TFsPath(TmpDir) / (name + ToString(RandomNumber<float>())); + while (true) { + try { + TUnbufferedFileInput src(path); + TUnbufferedFileOutput dst(TFile(lockPath, CreateNew | WrOnly | Seq)); + TransferData(&src, &dst); + break; + } catch (const TFileError& e) { + lockPath = TFsPath(TmpDir) / (name + ToString(RandomNumber<float>())); + } + } + if (NFs::Exists(path + ".attr")) { + NFs::Copy(path + ".attr", lockPath.GetPath() + ".attr"); + } + auto guard = Guard(Mutex); + if (auto p = Locks.FindPtr(fullTableName)) { + try { + NFs::Remove(*p); + NFs::Remove(*p + ".attr"); + } catch (...) { + } + } + Locks[fullTableName] = lockPath; +} + +void TYtFileServices::PushTableContent(const TString& path, const TString& content) { + auto guard = Guard(Mutex); + Contents.emplace(path, content); +} + +std::vector<TString> TYtFileServices::GetTableContent(const TString& path) { + auto guard = Guard(Mutex); + auto range = Contents.equal_range(path); + std::vector<TString> res; + std::transform(range.first, range.second, std::back_inserter(res), [](const auto& p) { return p.second; }); + Contents.erase(path); + return res; +} + +} // NYql::NFile diff --git a/yt/yql/providers/yt/gateway/file/yql_yt_file_services.h b/yt/yql/providers/yt/gateway/file/yql_yt_file_services.h new file mode 100644 index 0000000000..a06ff5eba6 --- /dev/null +++ b/yt/yql/providers/yt/gateway/file/yql_yt_file_services.h @@ -0,0 +1,91 @@ +#pragma once + +#include <yql/essentials/minikql/mkql_function_registry.h> +#include <yql/essentials/core/file_storage/file_storage.h> +#include <yql/essentials/core/file_storage/proto/file_storage.pb.h> + +#include <util/generic/hash.h> +#include <util/generic/string.h> +#include <util/generic/strbuf.h> +#include <util/generic/ptr.h> +#include <util/system/spinlock.h> +#include <util/folder/dirut.h> + +#include <vector> +#include <unordered_map> + +namespace NYql::NFile { + +class TYtFileServices: public TThrRefBase { +public: + using TPtr = TIntrusivePtr<TYtFileServices>; + ~TYtFileServices(); + + static TPtr Make(const NKikimr::NMiniKQL::IFunctionRegistry* registry, const THashMap<TString, TString>& mapping = {}, + TFileStoragePtr fileStorage = {}, const TString& tmpDir = {}, bool keepTempTables = false, const THashMap<TString, TString>& dirMapping = {}) + { + return new TYtFileServices(registry, mapping, fileStorage, tmpDir.empty() ? GetSystemTempDir() : tmpDir, keepTempTables, dirMapping); + } + + const NKikimr::NMiniKQL::IFunctionRegistry* GetFunctionRegistry() const { + return FunctionRegistry; + } + + const TString& GetTmpDir() const { + return TmpDir; + } + + THashMap<TString, TString>& GetTablesMapping() { + return TablesMapping; + } + + bool GetKeepTempTables() const { + return KeepTempTables; + } + + TString GetTablePath(TStringBuf cluster, TStringBuf table, bool isTemp, bool noLocks = false); + + void LockPath(const TString& path, const TString& fullTableName); + + void PushTableContent(const TString& path, const TString& content); + std::vector<TString> GetTableContent(const TString& path); + + TFileStoragePtr GetFileStorage() const { + return FileStorage; + } + +private: + TYtFileServices( + const NKikimr::NMiniKQL::IFunctionRegistry* registry, + const THashMap<TString, TString>& mapping, + TFileStoragePtr fileStorage, + const TString& tmpDir, + bool keepTempTables, + const THashMap<TString, TString>& dirMapping + ) + : FunctionRegistry(registry) + , TablesMapping(mapping) + , TablesDirMapping(dirMapping) + , TmpDir(tmpDir) + , KeepTempTables(keepTempTables) + { + FileStorage = fileStorage; + if (!FileStorage) { + TFileStorageConfig params; + FileStorage = CreateFileStorage(params); + } + } + + TFileStoragePtr FileStorage; + const NKikimr::NMiniKQL::IFunctionRegistry* FunctionRegistry; + THashMap<TString, TString> TablesMapping; // [cluster].[name] -> [file path] + THashMap<TString, TString> TablesDirMapping; // [cluster] -> [dir path] + TString TmpDir; + bool KeepTempTables; + + std::unordered_multimap<TString, TString> Contents; // path -> pickled content + THashMap<TString, TString> Locks; + TAdaptiveLock Mutex; +}; + +} // NYql::NFile diff --git a/yt/yql/providers/yt/gateway/file/yql_yt_file_text_yson.cpp b/yt/yql/providers/yt/gateway/file/yql_yt_file_text_yson.cpp new file mode 100644 index 0000000000..ec42b70bfd --- /dev/null +++ b/yt/yql/providers/yt/gateway/file/yql_yt_file_text_yson.cpp @@ -0,0 +1,74 @@ +#include "yql_yt_file_text_yson.h" +#include <yt/yql/providers/yt/lib/yson_helpers/yson_helpers.h> +#include <yt/cpp/mapreduce/interface/io.h> +#include <library/cpp/yson/parser.h> +#include <util/stream/buffer.h> +#include <util/stream/file.h> +#include <util/system/fs.h> + +namespace NYql::NFile { + +class TTextYsonInput: public NYT::TRawTableReader { +public: + TTextYsonInput(const TString& file, const TColumnsInfo& columnsInfo) { + TIFStream in(file); + + TBinaryYsonWriter writer(&Input_, ::NYson::EYsonType::ListFragment); + writer.OnBeginAttributes(); + writer.OnKeyedItem("row_index"); + writer.OnInt64Scalar(0); + writer.OnEndAttributes(); + writer.OnEntity(); + writer.OnListItem(); + NYT::NYson::IYsonConsumer* consumer = &writer; + THolder<TColumnFilteringConsumer> filter; + if (columnsInfo.Columns || columnsInfo.RenameColumns) { + TMaybe<TSet<TStringBuf>> columns; + TMaybe<THashMap<TStringBuf, TStringBuf>> renames; + if (columnsInfo.Columns) { + columns.ConstructInPlace(columnsInfo.Columns->Parts_.begin(), columnsInfo.Columns->Parts_.end()); + } + if (columnsInfo.RenameColumns) { + renames.ConstructInPlace(columnsInfo.RenameColumns->begin(), columnsInfo.RenameColumns->end()); + } + + filter.Reset(new TColumnFilteringConsumer(consumer, columns, renames)); + consumer = filter.Get(); + } + NYson::TYsonParser parser(consumer, &in, ::NYson::EYsonType::ListFragment); + parser.Parse(); + } + + bool Retry(const TMaybe<ui32>& /* rangeIndex */, const TMaybe<ui64>& /* rowIndex */, const std::exception_ptr& /* error */) override { + return false; + } + + void ResetRetries() override { + } + + bool HasRangeIndices() const override { + return false; + } + +protected: + size_t DoRead(void* buf, size_t len) override { + return Input_.Read(buf, len); + } + +private: + TBufferStream Input_; +}; + +TVector<NYT::TRawTableReaderPtr> MakeTextYsonInputs(const TVector<std::pair<TString, TColumnsInfo>>& files) { + TVector<NYT::TRawTableReaderPtr> rawReaders; + for (auto& file: files) { + if (!NFs::Exists(file.first)) { + rawReaders.emplace_back(nullptr); + continue; + } + rawReaders.emplace_back(MakeIntrusive<TTextYsonInput>(file.first, file.second)); + } + return rawReaders; +} + +} //namespace NYql::NFile
\ No newline at end of file diff --git a/yt/yql/providers/yt/gateway/file/yql_yt_file_text_yson.h b/yt/yql/providers/yt/gateway/file/yql_yt_file_text_yson.h new file mode 100644 index 0000000000..85b903c84b --- /dev/null +++ b/yt/yql/providers/yt/gateway/file/yql_yt_file_text_yson.h @@ -0,0 +1,14 @@ +#pragma once +#include <yt/cpp/mapreduce/interface/fwd.h> +#include <yt/cpp/mapreduce/interface/common.h> + +namespace NYql::NFile { + +struct TColumnsInfo { + TMaybe<NYT::TSortColumns> Columns; + TMaybe<NYT::TRichYPath::TRenameColumnsDescriptor> RenameColumns; +}; + +TVector<NYT::TRawTableReaderPtr> MakeTextYsonInputs(const TVector<std::pair<TString, TColumnsInfo>>& files); + +}//namespace NYql::NFile
\ No newline at end of file diff --git a/yt/yql/providers/yt/gateway/lib/query_cache.cpp b/yt/yql/providers/yt/gateway/lib/query_cache.cpp new file mode 100644 index 0000000000..0846b13539 --- /dev/null +++ b/yt/yql/providers/yt/gateway/lib/query_cache.cpp @@ -0,0 +1,380 @@ +#include "query_cache.h" +#include "yt_helpers.h" + +#include <yt/yql/providers/yt/lib/hash/yql_hash_builder.h> +#include <yql/essentials/utils/log/log.h> +#include <yql/essentials/utils/yql_panic.h> + +#include <yt/cpp/mapreduce/interface/errors.h> +#include <yt/cpp/mapreduce/interface/error_codes.h> +#include <yt/cpp/mapreduce/common/helpers.h> + +#include <util/generic/algorithm.h> +#include <util/string/hex.h> +#include <util/system/fs.h> +#include <util/system/error.h> + +#include <errno.h> + +namespace NYql { + +using namespace NYT; + +namespace { + bool IsRace(const NYT::TYtError& e) { + if (e.ContainsErrorCode(NClusterErrorCodes::NTransactionClient::NoSuchTransaction)) { + return false; + } + + return e.ContainsErrorCode(NClusterErrorCodes::NYTree::ResolveError) || + e.ContainsErrorCode(NClusterErrorCodes::NCypressClient::ConcurrentTransactionLockConflict); + } +} + +TFsQueryCacheItem::TFsQueryCacheItem(const TYtSettings& config, const TString& cluster, const TString& tmpDir, + const TString& hash, const TString& outputTablePath) + : TQueryCacheItemBase<TFsQueryCacheItem>(hash.empty() ? EQueryCacheMode::Disable : config.QueryCacheMode.Get().GetOrElse(EQueryCacheMode::Disable)) + , OutputTablePaths(1, outputTablePath) + , CachedPaths(1, TFsPath(tmpDir) / "query_cache" / cluster / (HexEncode(hash) + ".tmp")) +{ +} + +TFsQueryCacheItem::TFsQueryCacheItem(const TYtSettings& config, const TString& cluster, const TString& tmpDir, + const TString& hash, const TVector<TString>& outputTablePaths) + : TQueryCacheItemBase<TFsQueryCacheItem>(hash.empty() ? EQueryCacheMode::Disable : config.QueryCacheMode.Get().GetOrElse(EQueryCacheMode::Disable)) + , OutputTablePaths(outputTablePaths) +{ + for (size_t i = 0; i < outputTablePaths.size(); ++i) { + auto outputHash = (THashBuilder() << hash << i).Finish(); + CachedPaths.push_back(TFsPath(tmpDir) / "query_cache" / cluster / (HexEncode(outputHash) + ".tmp")); + } +} + +NThreading::TFuture<bool> TFsQueryCacheItem::LookupImpl(const TAsyncQueue::TPtr& /*queue*/) { + for (auto cachePath: CachedPaths) { + Cerr << "Check path: " << cachePath << "\n"; + if (!cachePath.Exists()) { + return NThreading::MakeFuture<bool>(false); + } + } + + for (size_t i = 0; i < CachedPaths.size(); ++i) { + Cerr << "Copy from: " << CachedPaths.at(i) << " to " << OutputTablePaths.at(i) << "\n"; + NFs::HardLinkOrCopy(CachedPaths.at(i), OutputTablePaths.at(i)); + NFs::HardLinkOrCopy(CachedPaths.at(i).GetPath() + ".attr", OutputTablePaths.at(i) + ".attr"); + } + + return NThreading::MakeFuture<bool>(true); +} + +void TFsQueryCacheItem::StoreImpl() { + for (size_t i = 0; i < CachedPaths.size(); ++i) { + Cerr << "Make dir: " << CachedPaths.at(i).Parent() << "\n"; + CachedPaths.at(i).Parent().MkDirs(); + if (CachedPaths.at(i).Exists()) { + if (Mode != EQueryCacheMode::Refresh) { + continue; + } + + Cerr << "Remove: " << CachedPaths.at(i) << "\n"; + if (!NFs::Remove(CachedPaths.at(i)) && LastSystemError() != ENOENT) { + ythrow TSystemError(); + } + + if (!NFs::Remove(CachedPaths.at(i).GetPath() + ".attr") && LastSystemError() != ENOENT) { + ythrow TSystemError(); + } + } + + Cerr << "Copy from: " << OutputTablePaths.at(i) << " to " << CachedPaths.at(i) << "\n"; + if (!NFs::HardLink(OutputTablePaths.at(i), CachedPaths.at(i))) { + if (LastSystemError() != EEXIST || Mode != EQueryCacheMode::Normal) { + ythrow TSystemError(); + } + } + + if (!NFs::HardLink(OutputTablePaths.at(i) + ".attr", CachedPaths.at(i).GetPath() + ".attr")) { + if (LastSystemError() != EEXIST || Mode != EQueryCacheMode::Normal) { + ythrow TSystemError(); + } + } + } +} + +TYtQueryCacheItem::TYtQueryCacheItem(EQueryCacheMode mode, const TTransactionCache::TEntry::TPtr& entry, const TString& hash, + const TVector<TString>& dstTables, const TVector<NYT::TNode>& dstSpecs, + const TString& userName, const TString& tmpFolder, const NYT::TNode& mergeSpec, + const NYT::TNode& tableAttrs, ui64 chunkLimit, bool useExpirationTimeout, bool useMultiSet, + const std::pair<TString, TString>& logCtx) + : TQueryCacheItemBase<TYtQueryCacheItem>(hash.empty() ? EQueryCacheMode::Disable : mode) + , Entry(entry) + , DstTables(dstTables) + , DstSpecs(dstSpecs) + , CachePath(GetCachePath(userName, tmpFolder)) + , ChunkLimit(chunkLimit) + , UseExpirationTimeout(useExpirationTimeout) + , UseMultiSet(useMultiSet) + , LogCtx(logCtx) + , MergeSpec(mergeSpec) + , TableAttrs(tableAttrs) +{ + if (!hash.empty()) { + for (size_t i = 0; i < dstTables.size(); ++i) { + auto outputHash = (THashBuilder() << hash << i).Finish(); + auto path = MakeCachedPath(outputHash); + CachedPaths.push_back(path); + SortedCachedPaths[path] = i; + } + } +} + +TString TYtQueryCacheItem::GetCachePath(const TString& userName, const TString& tmpFolder) { + auto path = tmpFolder; + if (path.empty()) { + path = "tmp/yql/" + userName; + } + + if (!path.EndsWith('/')) { + path += "/"; + } + + return path + "query_cache"; +} + +TString TYtQueryCacheItem::MakeCachedPath(const TString& hash) { + YQL_ENSURE(hash.size() == 32); + + auto hex = HexEncode(hash); + return TString::Join(CachePath, "/", hex.substr(0, 2), "/", hex.substr(2, 2), "/", hex); +} + +NThreading::TFuture<bool> TYtQueryCacheItem::LookupImpl(const TAsyncQueue::TPtr& queue) { + if (Entry->CacheTxId) { + return NThreading::MakeFuture<bool>(false); + } + + Entry->CreateDefaultTmpFolder(); + CreateParents(DstTables, Entry->CacheTx); + + NThreading::TFuture futureLock = NThreading::MakeFuture(); + if (Mode == EQueryCacheMode::Refresh || Mode == EQueryCacheMode::Normal) { + CreateParents(CachedPaths, Entry->CacheTx); + // get a lock for all paths before return + if (!LockTx) { + LockTx = Entry->Tx->StartTransaction(TStartTransactionOptions().Attributes(Entry->TransactionSpec)); + } + for (const auto& sortedEntry : SortedCachedPaths) { + TString cachedPath = CachedPaths[sortedEntry.second]; + futureLock = futureLock.Apply([cachedPath, lockTx = LockTx, logCtx = LogCtx](const auto& f) { + YQL_LOG_CTX_ROOT_SESSION_SCOPE(logCtx); + if (f.HasException()) { + return f; + } + auto pos = cachedPath.rfind("/"); + auto dir = cachedPath.substr(0, pos); + auto childKey = cachedPath.substr(pos + 1) + ".lock"; + YQL_CLOG(INFO, ProviderYt) << "Wait for " << cachedPath; + for (ui32 retriesLeft = 10; retriesLeft != 0; --retriesLeft) { + try { + return lockTx->Lock(dir, NYT::ELockMode::LM_SHARED, NYT::TLockOptions() + .Waitable(true).ChildKey(childKey))->GetAcquiredFuture(); + } catch (const TErrorResponse& e) { + if (!IsRace(e.GetError())) { + throw; + } + } + YQL_CLOG(INFO, ProviderYt) << "Retry, retriesLeft: " << retriesLeft; + } + YQL_CLOG(INFO, ProviderYt) << "Wait complete"; + return NThreading::MakeFuture(); + }); + } + } + + return futureLock.Apply([queue, cachedPaths = CachedPaths, dstTables = DstTables, entry = Entry, + useExpirationTimeout = UseExpirationTimeout, logCtx = LogCtx](const auto& f) { + f.GetValue(); + return queue->Async([=]() { + YQL_LOG_CTX_ROOT_SESSION_SCOPE(logCtx); + bool hit = true; + try { + for (size_t i = 0; i < cachedPaths.size(); ++i) { + YQL_CLOG(INFO, ProviderYt) << "Check path: " << cachedPaths.at(i); + entry->Tx->Copy(cachedPaths.at(i), dstTables.at(i), TCopyOptions().Force(true)); + } + } catch (const TErrorResponse& e) { + // Yt returns NoSuchTransaction as inner issue for ResolveError + if (!e.IsResolveError() || e.IsNoSuchTransaction()) { + throw; + } + + YQL_CLOG(INFO, ProviderYt) << "Query cache miss"; + hit = false; + } + if (!hit) { + return false; + } + + YQL_CLOG(INFO, ProviderYt) << "Query cache hit"; + if (useExpirationTimeout) { + return true; + } + for (const auto& cachedPath: cachedPaths) { + try { + entry->CacheTx->Set(cachedPath + "/@touched", TNode(true)); + } catch (const TErrorResponse& e) { + if (!IsRace(e.GetError())) { + throw; + } + } + try { + entry->CacheTx->Set(cachedPath + "/@expiration_time", + TNode((Now() + entry->CacheTtl).ToStringUpToSeconds())); + } catch (const TErrorResponse& e) { + if (!IsRace(e.GetError())) { + throw; + } + } + } + return true; + }); + }); +} + +void TYtQueryCacheItem::StoreImpl() { + if (Entry->CacheTxId) { + return; + } + + CreateParents(CachedPaths, Entry->CacheTx); + try { + Entry->CacheTx->Set(CachePath + "/@prune_empty_map_nodes", TNode(true)); + } catch (const TErrorResponse& e) { + if (!IsRace(e.GetError())) { + throw; + } + } + + NYT::ITransactionPtr nestedReadTx; + NYT::ITransactionPtr nestedWriteTx; + + auto handle = [&](const NYT::TYtError& e) { + if (nestedReadTx) { + nestedReadTx->Abort(); + } + if (nestedWriteTx) { + nestedWriteTx->Abort(); + } + + if (!IsRace(e) || Mode == EQueryCacheMode::Refresh) { + throw; + } + }; + + try { + nestedReadTx = Entry->Tx->StartTransaction(TStartTransactionOptions().Attributes(Entry->TransactionSpec)); + nestedWriteTx = Entry->CacheTx->StartTransaction(TStartTransactionOptions().Attributes(Entry->TransactionSpec)); + + TVector<NThreading::TFuture<void>> futures; + futures.reserve(CachedPaths.size()); + for (size_t i = 0; i < CachedPaths.size(); ++i) { + YQL_CLOG(INFO, ProviderYt) << "Store to query cache, from " << DstTables.at(i) << " into " << CachedPaths.at(i); + if (ChunkLimit) { + NYT::TNode attrs = nestedReadTx->Get(DstTables.at(i) + "&/@", NYT::TGetOptions().AttributeFilter( + NYT::TAttributeFilter() + .AddAttribute("chunk_count") + .AddAttribute("compression_codec") + .AddAttribute("erasure_codec") + .AddAttribute("optimize_for") + .AddAttribute("media") + .AddAttribute("primary_medium") + .AddAttribute("schema") + )); + + auto inputChunks = attrs["chunk_count"].AsInt64(); + if ((ui64)inputChunks <= ChunkLimit) { + YQL_CLOG(INFO, ProviderYt) << "Use Concatenate to store cache (chunks: " << + inputChunks << ", limit: " << ChunkLimit << ")"; + + attrs.AsMap().erase("chunk_count"); + nestedWriteTx->Create(CachedPaths.at(i), NYT::NT_TABLE, TCreateOptions().Force(true).Attributes(attrs)); + nestedWriteTx->Concatenate( + { NYT::TRichYPath(DstTables.at(i)).TransactionId(nestedReadTx->GetId()) }, + NYT::TRichYPath(CachedPaths.at(i)), + TConcatenateOptions() + ); + continue; + } else { + YQL_CLOG(INFO, ProviderYt) << "Use Merge to store cache (chunks: " << + inputChunks << ", limit: " << ChunkLimit << ")"; + } + } + + auto operation = nestedWriteTx->Merge(TMergeOperationSpec() + .AddInput(NYT::TRichYPath(DstTables.at(i)) + .TransactionId(nestedReadTx->GetId())) + .Output(CachedPaths.at(i)) + .Mode(EMergeMode::MM_ORDERED), + TOperationOptions().Spec(MergeSpec).Wait(false)); + futures.push_back(operation->Watch()); + } + + auto all = NThreading::WaitExceptionOrAll(futures); + all.GetValueSync(); + + nestedReadTx->Abort(); + nestedReadTx = {}; + + nestedWriteTx->Commit(); + if (LockTx) { + LockTx->Abort(); + } + + if (UseExpirationTimeout) { + TableAttrs["expiration_timeout"] = TNode(Entry->CacheTtl.MilliSeconds()); + } else { + TableAttrs["expiration_time"] = TNode((Now() + Entry->CacheTtl).ToStringUpToSeconds()); + } + + for (size_t i = 0; i < CachedPaths.size(); ++i) { + SetTableAttrs(DstSpecs[i], CachedPaths[i]); + } + } catch (const TErrorResponse& e) { + handle(e.GetError()); + } catch (const NYT::TOperationFailedError& e) { + handle(e.GetError()); + } +} + +void TYtQueryCacheItem::SetTableAttrs(const NYT::TNode& spec, const TString& cachedPath) { + NYT::TNode attrs = spec; + NYT::MergeNodes(attrs, TableAttrs); + if (UseMultiSet) { + try { + Entry->CacheTx->MultisetAttributes(cachedPath + "/@", attrs.AsMap(), NYT::TMultisetAttributesOptions()); + } catch (const TErrorResponse& e) { + if (!IsRace(e.GetError())) { + throw; + } + } + } else { + auto batchSet = Entry->CacheTx->CreateBatchRequest(); + TVector<NThreading::TFuture<void>> batchSetRes; + for (auto& attr : attrs.AsMap()) { + batchSetRes.push_back(batchSet->Set(TStringBuilder() << cachedPath << "/@" << attr.first, attr.second)); + } + + batchSet->ExecuteBatch(); + for (auto& x : batchSetRes) { + try { + x.GetValueSync(); + } catch (const TErrorResponse& e) { + if (!IsRace(e.GetError())) { + throw; + } + } + } + } +} + +} // NYql diff --git a/yt/yql/providers/yt/gateway/lib/query_cache.h b/yt/yql/providers/yt/gateway/lib/query_cache.h new file mode 100644 index 0000000000..e2196ab706 --- /dev/null +++ b/yt/yql/providers/yt/gateway/lib/query_cache.h @@ -0,0 +1,112 @@ +#pragma once + +#include <yt/yql/providers/yt/common/yql_yt_settings.h> +#include <yt/yql/providers/yt/gateway/lib/transaction_cache.h> +#include <yql/essentials/utils/threading/async_queue.h> + +#include <yt/cpp/mapreduce/interface/fwd.h> + +#include <library/cpp/threading/future/future.h> + +#include <util/folder/path.h> +#include <util/thread/pool.h> + +namespace NYql { + +template <class TDerived> +class TQueryCacheItemBase { +public: + TQueryCacheItemBase(EQueryCacheMode mode) + : Mode(mode) + { + } + + // returns true if cache was used + bool Lookup(const TAsyncQueue::TPtr& queue) { + return LookupAsync(queue).GetValueSync(); + } + + [[nodiscard]] + NThreading::TFuture<bool> LookupAsync(const TAsyncQueue::TPtr& queue) { + if (Mode == EQueryCacheMode::Disable || Mode == EQueryCacheMode::Refresh) { + return NThreading::MakeFuture<bool>(false); + } + if (Found) { + return *Found; + } + + Found = static_cast<TDerived*>(this)->LookupImpl(queue); + return *Found; + + } + + void Store() { + if (Mode == EQueryCacheMode::Disable || Mode == EQueryCacheMode::Readonly) { + return; + } + static_cast<TDerived*>(this)->StoreImpl(); + } + + bool Hit() const { + return Found && Found->GetValueSync(); + } + +protected: + const EQueryCacheMode Mode; + +private: + TMaybe<NThreading::TFuture<bool>> Found; +}; + + +class TFsQueryCacheItem: public TQueryCacheItemBase<TFsQueryCacheItem> { +public: + TFsQueryCacheItem(const TYtSettings& config, const TString& cluster, const TString& tmpDir, const TString& hash, + const TString& outputTablePath); + TFsQueryCacheItem(const TYtSettings& config, const TString& cluster, const TString& tmpDir, const TString& hash, + const TVector<TString>& outputTablePaths); + + // returns true if cache was used + NThreading::TFuture<bool> LookupImpl(const TAsyncQueue::TPtr& queue); + void StoreImpl(); + +private: + const TVector<TString> OutputTablePaths; + TVector<TFsPath> CachedPaths; +}; + +class TYtQueryCacheItem: public TQueryCacheItemBase<TYtQueryCacheItem> { +public: + TYtQueryCacheItem(EQueryCacheMode mode, const TTransactionCache::TEntry::TPtr& entry, const TString& hash, + const TVector<TString>& dstTables, const TVector<NYT::TNode>& dstSpecs, const TString& userName, + const TString& tmpFolder, const NYT::TNode& mergeSpec, + const NYT::TNode& tableAttrs, ui64 chunkLimit, bool useExpirationTimeout, bool useMultiSet, + const std::pair<TString, TString>& logCtx); + + // returns true if cache was used + NThreading::TFuture<bool> LookupImpl(const TAsyncQueue::TPtr& queue); + void StoreImpl(); + +private: + static TString GetCachePath(const TString& userName, const TString& tmpFolder); + TString MakeCachedPath(const TString& hash); + static void PrepareParentFolders(const TString& path, NYT::IClientBasePtr tx); + void SetTableAttrs(const NYT::TNode& spec, const TString& cachedPath); + +private: + const TTransactionCache::TEntry::TPtr Entry; + const TVector<TString> DstTables; + const TVector<NYT::TNode> DstSpecs; + const TString CachePath; + TVector<TString> CachedPaths; + const ui64 ChunkLimit; + const bool UseExpirationTimeout; + const bool UseMultiSet; + const std::pair<TString, TString> LogCtx; + TMap<TString, ui32> SortedCachedPaths; + const NYT::TNode MergeSpec; + NYT::TNode TableAttrs; + NYT::ITransactionPtr LockTx; +}; + +} // NYql diff --git a/yt/yql/providers/yt/gateway/lib/temp_files.cpp b/yt/yql/providers/yt/gateway/lib/temp_files.cpp new file mode 100644 index 0000000000..afaffe0dc3 --- /dev/null +++ b/yt/yql/providers/yt/gateway/lib/temp_files.cpp @@ -0,0 +1,19 @@ +#include "temp_files.h" + +#include <yql/essentials/utils/yql_panic.h> + +namespace NYql { + +TTempFiles::TTempFiles(const TString& tmpDir) + : TmpDir(tmpDir) +{ +} + +TString TTempFiles::AddFile(const TString& fileName) { + TFsPath filePath = TmpDir / fileName; + YQL_ENSURE(!filePath.Exists(), "Twice usage of the " << fileName << " temp file"); + Files.emplace_back(MakeHolder<TTempFile>(filePath)); + return filePath; +} + +} // NYql diff --git a/yt/yql/providers/yt/gateway/lib/temp_files.h b/yt/yql/providers/yt/gateway/lib/temp_files.h new file mode 100644 index 0000000000..98cdd17634 --- /dev/null +++ b/yt/yql/providers/yt/gateway/lib/temp_files.h @@ -0,0 +1,20 @@ +#pragma once + +#include <util/folder/path.h> +#include <util/system/tempfile.h> +#include <util/generic/string.h> +#include <util/generic/ptr.h> +#include <util/generic/vector.h> + +namespace NYql { + +struct TTempFiles { + TTempFiles(const TString& tmpDir); + + TString AddFile(const TString& fileName); + + const TFsPath TmpDir; + TVector<THolder<TTempFile>> Files; +}; + +} // NYql diff --git a/yt/yql/providers/yt/gateway/lib/transaction_cache.cpp b/yt/yql/providers/yt/gateway/lib/transaction_cache.cpp new file mode 100644 index 0000000000..0b89666117 --- /dev/null +++ b/yt/yql/providers/yt/gateway/lib/transaction_cache.cpp @@ -0,0 +1,600 @@ +#include "transaction_cache.h" +#include "yt_helpers.h" + +#include <yt/cpp/mapreduce/common/helpers.h> +#include <yt/cpp/mapreduce/interface/config.h> + +#include <yql/essentials/utils/log/log.h> + +#include <util/system/guard.h> +#include <util/generic/yexception.h> +#include <util/generic/guid.h> +#include <util/generic/scope.h> +#include <util/folder/path.h> + +namespace NYql { + +using namespace NYT; + +void TTransactionCache::TEntry::DeleteAtFinalizeUnlocked(const TString& table, bool isInternal) +{ + auto inserted = TablesToDeleteAtFinalize.insert(table); + if (!isInternal && inserted.second) { + if (++ExternalTempTablesCount > InflightTempTablesLimit) { + YQL_LOG_CTX_THROW yexception() << "Too many temporary tables registered - limit is " << InflightTempTablesLimit; + } + } +} + +bool TTransactionCache::TEntry::CancelDeleteAtFinalizeUnlocked(const TString& table, bool isInternal) +{ + auto erased = TablesToDeleteAtFinalize.erase(table); + if (!isInternal) { + YQL_ENSURE(erased <= ExternalTempTablesCount); + ExternalTempTablesCount -= erased; + } + return erased != 0; +} + + +void TTransactionCache::TEntry::RemoveInternal(const TString& table) { + bool existed; + with_lock(Lock_) { + existed = CancelDeleteAtFinalizeUnlocked(table, true); + } + if (existed) { + DoRemove(table); + } +} + +void TTransactionCache::TEntry::DoRemove(const TString& table) { + if (!KeepTables) { + YQL_CLOG(INFO, ProviderYt) << "Removing " << table.Quote() << " on " << Server; + Tx->Remove(table, TRemoveOptions().Force(true)); + } +} + +void TTransactionCache::TEntry::Finalize(const TString& clusterName) { + NYT::ITransactionPtr binarySnapshotTx; + decltype(SnapshotTxs) snapshotTxs; + THashSet<TString> toDelete; + decltype(CheckpointTxs) checkpointTxs; + decltype(WriteTxs) writeTxs; + with_lock(Lock_) { + binarySnapshotTx.Swap(BinarySnapshotTx); + snapshotTxs.swap(SnapshotTxs); + LastSnapshotTx.Drop(); + toDelete.swap(TablesToDeleteAtFinalize); + ExternalTempTablesCount = 0; + checkpointTxs.swap(CheckpointTxs); + writeTxs.swap(WriteTxs); + } + + for (auto& item: writeTxs) { + item.second->Abort(); + } + + for (auto& item: checkpointTxs) { + item.second->Abort(); + } + + if (binarySnapshotTx) { + binarySnapshotTx->Abort(); + } + + for (auto& item: snapshotTxs) { + item.second->Abort(); + } + + for (auto i : toDelete) { + DoRemove(i); + } + + YQL_CLOG(INFO, ProviderYt) << "Committing tx " << GetGuidAsString(Tx->GetId()) << " on " << clusterName; + Tx->Commit(); +} + +TMaybe<ui64> TTransactionCache::TEntry::GetColumnarStat(NYT::TRichYPath ytPath) const { + YQL_ENSURE(ytPath.Columns_.Defined()); + TVector<TString> columns(std::move(ytPath.Columns_->Parts_)); + ytPath.Columns_.Clear(); + + auto guard = Guard(Lock_); + if (auto p = StatisticsCache.FindPtr(NYT::NodeToCanonicalYsonString(NYT::PathToNode(ytPath), NYT::NYson::EYsonFormat::Text))) { + ui64 sum = p->ColumnarStat.LegacyChunksDataWeight; + for (auto& column: columns) { + if (auto c = p->ColumnarStat.ColumnDataWeight.FindPtr(column)) { + sum += *c; + } else { + return Nothing(); + } + } + return sum; + } + return Nothing(); +} + +TMaybe<NYT::TTableColumnarStatistics> TTransactionCache::TEntry::GetExtendedColumnarStat(NYT::TRichYPath ytPath) const { + TVector<TString> columns(std::move(ytPath.Columns_->Parts_)); + ytPath.Columns_.Clear(); + auto cacheKey = NYT::NodeToCanonicalYsonString(NYT::PathToNode(ytPath), NYT::NYson::EYsonFormat::Text); + + auto guard = Guard(Lock_); + auto p = StatisticsCache.FindPtr(cacheKey); + if (!p) { + return Nothing(); + } + + NYT::TTableColumnarStatistics res; + for (auto& column: columns) { + if (p->ExtendedStatColumns.count(column) == 0) { + return Nothing(); + } + if (auto c = p->ColumnarStat.ColumnDataWeight.FindPtr(column)) { + res.ColumnDataWeight[column] = *c; + } + if (auto c = p->ColumnarStat.ColumnEstimatedUniqueCounts.FindPtr(column)) { + res.ColumnEstimatedUniqueCounts[column] = *c; + } + } + return res; +} + +void TTransactionCache::TEntry::UpdateColumnarStat(NYT::TRichYPath ytPath, ui64 size) { + YQL_ENSURE(ytPath.Columns_.Defined()); + TVector<TString> columns(std::move(ytPath.Columns_->Parts_)); + ytPath.Columns_.Clear(); + auto cacheKey = NYT::NodeToCanonicalYsonString(NYT::PathToNode(ytPath), NYT::NYson::EYsonFormat::Text); + + auto guard = Guard(Lock_); + auto& cacheEntry = StatisticsCache[cacheKey]; + cacheEntry.ColumnarStat.LegacyChunksDataWeight = size; + for (auto& c: cacheEntry.ColumnarStat.ColumnDataWeight) { + c.second = 0; + } + for (auto& c: columns) { + cacheEntry.ColumnarStat.ColumnDataWeight[c] = 0; + } +} + +void TTransactionCache::TEntry::UpdateColumnarStat(NYT::TRichYPath ytPath, const NYT::TTableColumnarStatistics& columnStat, bool extended) { + TVector<TString> columns(std::move(ytPath.Columns_->Parts_)); + ytPath.Columns_.Clear(); + auto guard = Guard(Lock_); + auto& cacheEntry = StatisticsCache[NYT::NodeToCanonicalYsonString(NYT::PathToNode(ytPath), NYT::NYson::EYsonFormat::Text)]; + if (extended) { + std::copy(columns.begin(), columns.end(), std::inserter(cacheEntry.ExtendedStatColumns, cacheEntry.ExtendedStatColumns.end())); + } + cacheEntry.ColumnarStat.LegacyChunksDataWeight = columnStat.LegacyChunksDataWeight; + cacheEntry.ColumnarStat.TimestampTotalWeight = columnStat.TimestampTotalWeight; + for (auto& c: columnStat.ColumnDataWeight) { + cacheEntry.ColumnarStat.ColumnDataWeight[c.first] = c.second; + } + if (extended) { + for (auto& c : columnStat.ColumnEstimatedUniqueCounts) { + cacheEntry.ColumnarStat.ColumnEstimatedUniqueCounts[c.first] = c.second; + } + } +} + +ITransactionPtr TTransactionCache::TEntry::GetSnapshotTx(bool createTx) { + auto guard = Guard(Lock_); + if (createTx || !LastSnapshotTx) { + LastSnapshotTx = Tx->StartTransaction(TStartTransactionOptions().Attributes(TransactionSpec).PingAncestors(true)); + SnapshotTxs.emplace(LastSnapshotTx->GetId(), LastSnapshotTx); + } + return LastSnapshotTx; +} + +NYT::ITransactionPtr TTransactionCache::TEntry::GetSnapshotTx(const NYT::TTransactionId& id) const { + auto guard = Guard(Lock_); + auto p = SnapshotTxs.FindPtr(id); + YQL_ENSURE(p, "Unknown snapshot transaction id=" << GetGuidAsString(id)); + return *p; +} + +NYT::ITransactionPtr TTransactionCache::TEntry::GetCheckpointTx(const TString& tablePath) const { + auto guard = Guard(Lock_); + auto p = CheckpointTxs.FindPtr(tablePath); + YQL_ENSURE(p, "No transaction found for checkpoint " << tablePath.Quote()); + return *p; +} + +NYT::ITransactionPtr TTransactionCache::TEntry::GetOrCreateCheckpointTx(const TString& tablePath) { + auto guard = Guard(Lock_); + auto& tx = CheckpointTxs[tablePath]; + if (!tx) { + tx = Client->StartTransaction(TStartTransactionOptions().Attributes(TransactionSpec)); + YQL_CLOG(INFO, ProviderYt) << "Started checkpoint tx " << GetGuidAsString(tx->GetId()); + } + return tx; +} + +void TTransactionCache::TEntry::CommitCheckpointTx(const TString& tablePath) { + auto guard = Guard(Lock_); + auto p = CheckpointTxs.FindPtr(tablePath); + YQL_ENSURE(p, "No transaction found for checkpoint " << tablePath.Quote()); + YQL_CLOG(INFO, ProviderYt) << "Commiting checkpoint tx " << GetGuidAsString((*p)->GetId()); + (*p)->Commit(); + CheckpointTxs.erase(tablePath); +} + +NYT::TTransactionId TTransactionCache::TEntry::AllocWriteTx() { + auto guard = Guard(Lock_); + auto writeTx = Tx->StartTransaction(TStartTransactionOptions().Attributes(TransactionSpec)); + WriteTxs.emplace(writeTx->GetId(), writeTx); + YQL_CLOG(INFO, ProviderYt) << "Allocated write tx " << GetGuidAsString(writeTx->GetId()); + return writeTx->GetId(); + +} + +void TTransactionCache::TEntry::CompleteWriteTx(const NYT::TTransactionId& id, bool abort) { + auto guard = Guard(Lock_); + auto p = WriteTxs.FindPtr(id); + YQL_ENSURE(p, "No transaction found: " << GetGuidAsString(id)); + YQL_CLOG(INFO, ProviderYt) << (abort ? "Aborting" : "Commiting") << " write tx " << GetGuidAsString(id); + if (abort) { + (*p)->Abort(); + } else { + (*p)->Commit(); + } + WriteTxs.erase(id); +} + +std::pair<TString, NYT::TTransactionId> TTransactionCache::TEntry::GetBinarySnapshot(TString remoteTmpFolder, const TString& md5, const TString& localPath, TDuration expirationInterval) { + if (remoteTmpFolder.StartsWith(NYT::TConfig::Get()->Prefix)) { + remoteTmpFolder = remoteTmpFolder.substr(NYT::TConfig::Get()->Prefix.size()); + } + TString remotePath = TFsPath(remoteTmpFolder) / md5; + + ITransactionPtr snapshotTx; + with_lock(Lock_) { + if (!BinarySnapshotTx) { + BinarySnapshotTx = Client->StartTransaction(TStartTransactionOptions().Attributes(TransactionSpec)); + } + snapshotTx = BinarySnapshotTx; + if (auto p = BinarySnapshots.FindPtr(remotePath)) { + return std::make_pair(*p, snapshotTx->GetId()); + } + } + CreateParents({remotePath}, Client); + + NYT::ILockPtr fileLock; + ITransactionPtr lockTx; + NYT::ILockPtr waitLock; + + for (bool uploaded = false; ;) { + try { + YQL_CLOG(INFO, ProviderYt) << "Taking snapshot of " << remotePath; + fileLock = snapshotTx->Lock(remotePath, NYT::ELockMode::LM_SNAPSHOT); + break; + } catch (const TErrorResponse& e) { + // Yt returns NoSuchTransaction as inner issue for ResolveError + if (!e.IsResolveError() || e.IsNoSuchTransaction()) { + throw; + } + } + YQL_ENSURE(!uploaded, "Fail to take snapshot"); + if (!lockTx) { + auto pos = remotePath.rfind("/"); + auto dir = remotePath.substr(0, pos); + auto childKey = remotePath.substr(pos + 1) + ".lock"; + + lockTx = Client->StartTransaction(TStartTransactionOptions().Attributes(TransactionSpec)); + YQL_CLOG(INFO, ProviderYt) << "Waiting for " << dir << '/' << childKey; + waitLock = lockTx->Lock(dir, NYT::ELockMode::LM_SHARED, TLockOptions().Waitable(true).ChildKey(childKey)); + waitLock->GetAcquiredFuture().GetValueSync(); + // Try to take snapshot again after waiting lock. Someone else may complete uploading the file at the moment + continue; + } + // Lock is already taken and file still doesn't exist + YQL_CLOG(INFO, ProviderYt) << "Start uploading " << localPath << " to " << remotePath; + Y_SCOPE_EXIT(localPath, remotePath) { + YQL_CLOG(INFO, ProviderYt) << "Complete uploading " << localPath << " to " << remotePath; + }; + auto uploadTx = Client->StartTransaction(TStartTransactionOptions().Attributes(TransactionSpec)); + try { + auto out = uploadTx->CreateFileWriter(TRichYPath(remotePath).Executable(true), TFileWriterOptions().CreateTransaction(false)); + TIFStream in(localPath); + TransferData(&in, out.Get()); + out->Finish(); + uploadTx->Commit(); + } catch (...) { + uploadTx->Abort(); + throw; + } + // Continue with taking snapshot lock after uploading + uploaded = true; + } + + TString snapshotPath = TStringBuilder() << '#' << GetGuidAsString(fileLock->GetLockedNodeId()); + YQL_CLOG(INFO, ProviderYt) << "Snapshot of " << remotePath << ": " << snapshotPath; + with_lock(Lock_) { + BinarySnapshots[remotePath] = snapshotPath; + } + + if (expirationInterval) { + TString expirationTime = (Now() + expirationInterval).ToStringUpToSeconds(); + try { + YQL_CLOG(INFO, ProviderYt) << "Prolonging expiration time for " << remotePath << " up to " << expirationTime; + Client->Set(remotePath + "/@expiration_time", expirationTime); + } catch (...) { + // log and ignore the error + YQL_CLOG(ERROR, ProviderYt) << "Error setting expiration time for " << remotePath << ": " << CurrentExceptionMessage(); + } + } + + return std::make_pair(snapshotPath, snapshotTx->GetId()); +} + +void TTransactionCache::TEntry::UpdateCacheMetrics(const TString& fileName, ECacheStatus status) { + static const TString cacheHitMrjob = "CacheHitMrjob"; + static const TString cacheMissMrjob = "CacheMissMrjob"; + static const TString cacheOtherMrjob = "CacheOtherMrjob"; + static const TString cacheHitUdf = "CacheHitUdf"; + static const TString cacheMissUdf = "CacheMissUdf"; + static const TString cacheOtherUdf = "CacheOtherUdf"; + + if (Metrics) { + bool isMrJob = fileName == "mrjob"; + switch(status) { + case ECacheStatus::Hit: + isMrJob ? Metrics->IncCounter(cacheHitMrjob, Server) : Metrics->IncCounter(cacheHitUdf, Server); + break; + case ECacheStatus::Miss: + isMrJob ? Metrics->IncCounter(cacheMissMrjob, Server) : Metrics->IncCounter(cacheMissUdf, Server); + break; + default: + isMrJob ? Metrics->IncCounter(cacheOtherMrjob, Server) : Metrics->IncCounter(cacheOtherUdf, Server); + } + } +}; + +TMaybe<std::pair<TString, NYT::TTransactionId>> TTransactionCache::TEntry::GetBinarySnapshotFromCache(TString binaryCacheFolder, const TString& md5, const TString& fileName) { + if (binaryCacheFolder.StartsWith(NYT::TConfig::Get()->Prefix)) { + binaryCacheFolder = binaryCacheFolder.substr(NYT::TConfig::Get()->Prefix.size()); + } + YQL_ENSURE(md5.size() > 4); + TString remotePath = TFsPath(binaryCacheFolder) / md5.substr(0, 2) / md5.substr(2, 2) / md5; + + ITransactionPtr snapshotTx; + with_lock(Lock_) { + if (!BinarySnapshotTx) { + BinarySnapshotTx = Client->StartTransaction(TStartTransactionOptions().Attributes(TransactionSpec)); + } + snapshotTx = BinarySnapshotTx; + if (auto p = BinarySnapshots.FindPtr(remotePath)) { + UpdateCacheMetrics(fileName, ECacheStatus::Hit); + return std::make_pair(*p, snapshotTx->GetId()); + } + } + TString snapshotPath; + try { + NYT::ILockPtr fileLock = snapshotTx->Lock(remotePath, NYT::ELockMode::LM_SNAPSHOT); + snapshotPath = TStringBuilder() << '#' << GetGuidAsString(fileLock->GetLockedNodeId()); + } catch (const TErrorResponse& e) { + YQL_CLOG(WARN, ProviderYt) << "Can't load binary for \"" << fileName << "\" from BinaryCacheFolder: " << e.what(); + if (e.IsResolveError()) { + UpdateCacheMetrics(fileName, ECacheStatus::Miss); + } else { + UpdateCacheMetrics(fileName, ECacheStatus::Other); + } + return Nothing(); + } + with_lock(Lock_) { + BinarySnapshots[remotePath] = snapshotPath; + } + YQL_CLOG(DEBUG, ProviderYt) << "Snapshot \"" + << fileName << "\" -> \"" << remotePath << "\" -> " + << snapshotPath << ", tx=" << GetGuidAsString(snapshotTx->GetId()); + UpdateCacheMetrics(fileName, ECacheStatus::Hit); + + return std::make_pair(snapshotPath, snapshotTx->GetId()); +} + +void TTransactionCache::TEntry::CreateDefaultTmpFolder() { + if (DefaultTmpFolder) { + Client->Create(DefaultTmpFolder, NYT::NT_MAP, NYT::TCreateOptions().Recursive(true).IgnoreExisting(true)); + } +} + +TTransactionCache::TTransactionCache(const TString& userName) + : UserName_(userName) +{} + +TTransactionCache::TEntry::TPtr TTransactionCache::GetEntry(const TString& server) { + auto res = TryGetEntry(server); + if (!res) { + YQL_LOG_CTX_THROW yexception() << "GetEntry() failed for " << server; + } + return res; +} + +TTransactionCache::TEntry::TPtr TTransactionCache::TryGetEntry(const TString& server) { + auto guard = Guard(Lock_); + auto it = TxMap_.find(server); + if (it != TxMap_.end()) { + return it->second; + } + return {}; +} + +TTransactionCache::TEntry::TPtr TTransactionCache::GetOrCreateEntry(const TString& server, const TString& token, + const TMaybe<TString>& impersonationUser, const TSpecProvider& specProvider, const TYtSettings::TConstPtr& config, IMetricsRegistryPtr metrics) +{ + TEntry::TPtr createdEntry = nullptr; + NYT::TTransactionId externalTx = config->ExternalTx.Get().GetOrElse(TGUID()); + with_lock(Lock_) { + auto it = TxMap_.find(server); + if (it != TxMap_.end()) { + return it->second; + } + + TString tmpFolder = GetTablesTmpFolder(*config); + + createdEntry = MakeIntrusive<TEntry>(); + createdEntry->Server = server; + auto createClientOptions = TCreateClientOptions().Token(token); + if (impersonationUser) { + createClientOptions = createClientOptions.ImpersonationUser(*impersonationUser); + } + createdEntry->Client = CreateClient(server, createClientOptions); + createdEntry->TransactionSpec = specProvider(); + if (externalTx) { + createdEntry->ExternalTx = createdEntry->Client->AttachTransaction(externalTx); + createdEntry->Tx = createdEntry->ExternalTx->StartTransaction(TStartTransactionOptions().Attributes(createdEntry->TransactionSpec)); + } else { + createdEntry->Tx = createdEntry->Client->StartTransaction(TStartTransactionOptions().Attributes(createdEntry->TransactionSpec)); + } + createdEntry->CacheTx = createdEntry->Client; + createdEntry->CacheTtl = config->QueryCacheTtl.Get().GetOrElse(TDuration::Days(7)); + if (!tmpFolder.empty()) { + auto fullTmpFolder = AddPathPrefix(tmpFolder, NYT::TConfig::Get()->Prefix); + bool existsGlobally = createdEntry->Client->Exists(fullTmpFolder); + bool existsInTx = externalTx && createdEntry->ExternalTx->Exists(fullTmpFolder); + if (!existsGlobally && existsInTx) { + createdEntry->CacheTx = createdEntry->ExternalTx; + createdEntry->CacheTxId = createdEntry->ExternalTx->GetId(); + } + } else { + createdEntry->DefaultTmpFolder = NYT::AddPathPrefix("tmp/yql/" + UserName_, NYT::TConfig::Get()->Prefix); + } + createdEntry->InflightTempTablesLimit = config->InflightTempTablesLimit.Get().GetOrElse(Max<ui32>()); + createdEntry->KeepTables = GetReleaseTempDataMode(*config) == EReleaseTempDataMode::Never; + createdEntry->Metrics = metrics; + + TxMap_.emplace(server, createdEntry); + } + if (externalTx) { + YQL_CLOG(INFO, ProviderYt) << "Attached to external tx " << GetGuidAsString(externalTx); + } + YQL_CLOG(INFO, ProviderYt) << "Created tx " << GetGuidAsString(createdEntry->Tx->GetId()) << " on " << server; + return createdEntry; +} + +void TTransactionCache::Commit(const TString& server) { + ITransactionPtr tx; + THashSet<TString> tablesToDelete; + with_lock(Lock_) { + auto it = TxMap_.find(server); + if (it != TxMap_.end()) { + auto entry = it->second; + tablesToDelete.swap(entry->TablesToDeleteAtCommit); + if (!tablesToDelete.empty()) { + tx = entry->Tx; + } + } + } + if (tx) { + for (auto& table : tablesToDelete) { + YQL_CLOG(INFO, ProviderYt) << "Removing " << table.Quote() << " on " << server; + tx->Remove(table, TRemoveOptions().Force(true)); + } + } +} + +void TTransactionCache::Finalize() { + THashMap<TString, TEntry::TPtr> txMap; + with_lock(Lock_) { + txMap.swap(TxMap_); + } + for (auto& item: txMap) { + item.second->Finalize(item.first); + } +} + +void TTransactionCache::AbortAll() { + THashMap<TString, TEntry::TPtr> txMap; + with_lock(Lock_) { + txMap.swap(TxMap_); + } + + TString error; + auto abortTx = [&] (const ITransactionPtr& tx) { + try { + tx->Abort(); + } catch (...) { + YQL_CLOG(ERROR, ProviderYt) << CurrentExceptionMessage(); + + // Store first abort error. + if (error.empty()) { + error = "Failed to abort transaction " + GetGuidAsString(tx->GetId()) + ": " + CurrentExceptionMessage(); + } + } + }; + + for (auto& item : txMap) { + auto entry = item.second; + + for (auto& item: entry->SnapshotTxs) { + YQL_CLOG(DEBUG, ProviderYt) << "AbortAll(): Aborting Snapshot tx " << GetGuidAsString(item.second->GetId()); + abortTx(item.second); + } + for (auto& item : entry->CheckpointTxs) { + YQL_CLOG(DEBUG, ProviderYt) << "AbortAll(): Aborting Checkpoint tx " << GetGuidAsString(item.second->GetId()); + abortTx(item.second); + } + for (auto& item: entry->WriteTxs) { + YQL_CLOG(DEBUG, ProviderYt) << "AbortAll(): Aborting Write tx " << GetGuidAsString(item.second->GetId()); + abortTx(item.second); + } + if (entry->BinarySnapshotTx) { + YQL_CLOG(INFO, ProviderYt) << "AbortAll(): Aborting BinarySnapshot tx " << GetGuidAsString(entry->BinarySnapshotTx->GetId()); + abortTx(entry->BinarySnapshotTx); + } + if (entry->Tx) { + YQL_CLOG(INFO, ProviderYt) << "Aborting tx " << GetGuidAsString(entry->Tx->GetId()) << " on " << item.first; + abortTx(entry->Tx); + } + + if (entry->Client) { + YQL_CLOG(INFO, ProviderYt) << "Shutting down client"; + try { + entry->Client->Shutdown(); + } catch (...) { + if (!error) { + error = "Failed to shut down client: " + CurrentExceptionMessage(); + } + } + } + } + + if (error) { + ythrow yexception() << error; + } +} + +void TTransactionCache::DetachSnapshotTxs() { + TString error; + auto detachTx = [&] (const ITransactionPtr& tx) { + try { + tx->Detach(); + } catch (...) { + YQL_CLOG(ERROR, ProviderYt) << CurrentExceptionMessage(); + + // Store first detach error. + if (error.empty()) { + error = "Failed to detach transaction " + GetGuidAsString(tx->GetId()) + ": " + CurrentExceptionMessage(); + } + } + }; + + for (auto& item : TxMap_) { + auto entry = item.second; + + for (auto& item : entry->SnapshotTxs) { + YQL_CLOG(DEBUG, ProviderYt) << "DetachSnapshotTxs(): Detaching Snapshot tx " << GetGuidAsString(item.second->GetId()); + detachTx(item.second); + } + if (entry->Tx) { + YQL_CLOG(INFO, ProviderYt) << "Detaching tx " << GetGuidAsString(entry->Tx->GetId()) << " on " << item.first; + detachTx(entry->Tx); + } + } + + if (error) { + ythrow yexception() << error; + } +} + +} // NYql diff --git a/yt/yql/providers/yt/gateway/lib/transaction_cache.h b/yt/yql/providers/yt/gateway/lib/transaction_cache.h new file mode 100644 index 0000000000..2feba09286 --- /dev/null +++ b/yt/yql/providers/yt/gateway/lib/transaction_cache.h @@ -0,0 +1,169 @@ +#pragma once + +#include <yt/yql/providers/yt/common/yql_yt_settings.h> + +#include <yql/essentials/core/file_storage/storage.h> +#include <yql/essentials/providers/common/metrics/metrics_registry.h> + +#include <yt/cpp/mapreduce/interface/client.h> +#include <yt/cpp/mapreduce/interface/fwd.h> +#include <yt/cpp/mapreduce/interface/common.h> + +#include <util/generic/string.h> +#include <util/generic/hash_set.h> +#include <util/generic/hash.h> +#include <util/generic/ptr.h> +#include <util/generic/variant.h> +#include <util/generic/vector.h> +#include <util/generic/maybe.h> +#include <util/datetime/base.h> +#include <util/system/mutex.h> + +#include <utility> +#include <functional> +#include <tuple> +#include <vector> +#include <exception> + +namespace NYql { + +class TTransactionCache { +public: + using TSpecProvider = std::function<NYT::TNode()>; + + struct TEntry : public TThrRefBase { + TString Server; + NYT::IClientPtr Client; + NYT::ITransactionPtr Tx; + NYT::ITransactionPtr ExternalTx; + NYT::IClientBasePtr CacheTx; + NYT::TTransactionId CacheTxId; + TDuration CacheTtl; + THashMap<NYT::TTransactionId, NYT::ITransactionPtr> SnapshotTxs; + THashMap<NYT::TTransactionId, NYT::ITransactionPtr> WriteTxs; + NYT::ITransactionPtr LastSnapshotTx; + THashSet<TString> TablesToDeleteAtFinalize; + THashSet<TString> TablesToDeleteAtCommit; + ui32 InflightTempTablesLimit = Max<ui32>(); + bool KeepTables = false; + THashMap<std::pair<TString, ui32>, std::tuple<TString, NYT::TTransactionId, ui64>> Snapshots; // {tablepath, epoch} -> {table_id, transaction_id, revision} + NYT::TNode TransactionSpec; + THashMap<TString, TString> BinarySnapshots; // remote path -> snapshot path + NYT::ITransactionPtr BinarySnapshotTx; + THashMap<TString, NYT::ITransactionPtr> CheckpointTxs; + TString DefaultTmpFolder; + THashMap<std::tuple<TString, TString, TString>, std::vector<NYT::TRichYPath>> RangeCache; + THashMap<TString, std::pair<std::vector<TString>, std::vector<std::exception_ptr>>> PartialRangeCache; + + using TFolderCache = THashMap<TString, std::vector<std::tuple<TString, TString, NYT::TNode>>>; + TFolderCache FolderCache; + + THashMap<TString, TFileLinkPtr> FolderFilePtrCache; + + TMutex Lock_; + IMetricsRegistryPtr Metrics; + + inline void DeleteAtFinalize(const TString& table) { + with_lock(Lock_) { + DeleteAtFinalizeUnlocked(table, false); + } + } + + inline void DeleteAtFinalizeInternal(const TString& table) { + with_lock(Lock_) { + DeleteAtFinalizeUnlocked(table, true); + } + } + + inline void CancelDeleteAtFinalize(const TString& table) { + with_lock(Lock_) { + CancelDeleteAtFinalizeUnlocked(table, false); + } + } + + void RemoveInternal(const TString& table); + void Finalize(const TString& clusterName); + + template<typename T> + T CancelDeleteAtFinalize(const T& range) { + T filteredRange; + with_lock(Lock_) { + for (const auto& i : range) { + if (CancelDeleteAtFinalizeUnlocked(i, false)) { + filteredRange.insert(filteredRange.end(), i); + } + } + } + return filteredRange; + } + + inline NYT::IClientBasePtr GetRoot() const { + if (ExternalTx) { + return ExternalTx; + } + return Client; + } + + NYT::ITransactionPtr GetSnapshotTx(bool createTx); + NYT::ITransactionPtr GetSnapshotTx(const NYT::TTransactionId& id) const; + + NYT::ITransactionPtr GetCheckpointTx(const TString& tablePath) const; + NYT::ITransactionPtr GetOrCreateCheckpointTx(const TString& tablePath); + void CommitCheckpointTx(const TString& tablePath); + + NYT::TTransactionId AllocWriteTx(); + void CompleteWriteTx(const NYT::TTransactionId& id, bool abort); + + TMaybe<ui64> GetColumnarStat(NYT::TRichYPath ytPath) const; + TMaybe<NYT::TTableColumnarStatistics> GetExtendedColumnarStat(NYT::TRichYPath ytPath) const; + + void UpdateColumnarStat(NYT::TRichYPath ytPath, ui64 size); + void UpdateColumnarStat(NYT::TRichYPath ytPath, const NYT::TTableColumnarStatistics& columnStat, bool extended = false); + + std::pair<TString, NYT::TTransactionId> GetBinarySnapshot(TString remoteTmpFolder, const TString& md5, const TString& localPath, TDuration expirationInterval); + TMaybe<std::pair<TString, NYT::TTransactionId>> GetBinarySnapshotFromCache(TString binaryCacheFolder, const TString& md5, const TString& fileName); + + enum class ECacheStatus { + Hit, + Miss, + Other + }; + void UpdateCacheMetrics(const TString& fileName, ECacheStatus status); + + void CreateDefaultTmpFolder(); + + using TPtr = TIntrusivePtr<TEntry>; + + private: + struct TStatisticsCacheEntry { + std::unordered_set<TString> ExtendedStatColumns; + NYT::TTableColumnarStatistics ColumnarStat; + }; + + THashMap<TString, TStatisticsCacheEntry> StatisticsCache; + + void DeleteAtFinalizeUnlocked(const TString& table, bool isInternal); + bool CancelDeleteAtFinalizeUnlocked(const TString& table, bool isInternal); + void DoRemove(const TString& table); + + size_t ExternalTempTablesCount = 0; + }; + + TTransactionCache(const TString& userName); + + TEntry::TPtr GetEntry(const TString& server); + TEntry::TPtr GetOrCreateEntry(const TString& server, const TString& token, const TMaybe<TString>& impersonationUser, const TSpecProvider& specProvider, const TYtSettings::TConstPtr& config, IMetricsRegistryPtr metrics); + TEntry::TPtr TryGetEntry(const TString& server); + + void Commit(const TString& server); + void Finalize(); + void AbortAll(); + void DetachSnapshotTxs(); + +private: + TMutex Lock_; + THashMap<TString, TEntry::TPtr> TxMap_; + const TString UserName_; +}; + +} // NYql diff --git a/yt/yql/providers/yt/gateway/lib/user_files.cpp b/yt/yql/providers/yt/gateway/lib/user_files.cpp new file mode 100644 index 0000000000..1990ccadfb --- /dev/null +++ b/yt/yql/providers/yt/gateway/lib/user_files.cpp @@ -0,0 +1,107 @@ +#include "user_files.h" + +#include <yt/yql/providers/yt/common/yql_names.h> +#include <yql/essentials/providers/common/provider/yql_provider.h> +#include <yql/essentials/utils/log/log.h> +#include <yql/essentials/utils/yql_panic.h> + +#include <util/system/guard.h> +#include <library/cpp/string_utils/url/url.h> + + +namespace NYql { + +TUserFiles::TUserFiles(const TYtUrlMapper& urlMapper, const TString& activeCluster) + : UrlMapper(urlMapper) + , ActiveCluster(activeCluster) +{ +} + +void TUserFiles::AddFile(const TUserDataKey& key, const TUserDataBlock& block) { + with_lock(Mutex) { + if (Files.contains(key.Alias())) { + return; + } + } + + + TFileInfo userFile; + userFile.IsUdf = block.Usage.Test(EUserDataBlockUsage::Udf); + userFile.IsPgExt = block.Usage.Test(EUserDataBlockUsage::PgExt); + userFile.IsPgCatalog = (key.Alias() == NCommon::PgCatalogFileName); + + if (block.Options.contains("bypass_artifact_cache")) { + auto option = block.Options.at(TString("bypass_artifact_cache")); + try { + userFile.BypassArtifactCache = FromString<bool>(option); + } catch (const TFromStringException &) { + YQL_LOG_CTX_THROW yexception() << "FileOption: invalid value for option bypass_artifact_cache: " << option; + } + } + + // we can optimize file copy if file resides on the same cluster + // and provide only link + TString cluster; + TString remotePath; + if ((block.Type == EUserDataType::URL) && + UrlMapper.MapYtUrl(block.Data, &cluster, &remotePath) && + (cluster == CurrentYtClusterShortcut || cluster == ActiveCluster)) { + userFile.RemotePath = remotePath; + userFile.RemoteMemoryFactor = 1.0; + YQL_CLOG(INFO, Default) << "Using remote file " << userFile.RemotePath.Quote() << " from " << ActiveCluster.Quote(); + } else { + if (!block.FrozenFile) { + YQL_LOG_CTX_THROW yexception() << "File with key " << key << " is not frozen"; + } + + userFile.Path = block.FrozenFile; + userFile.InMemorySize = userFile.Path->GetSize(); + } + + with_lock(Mutex) { + Files[key.Alias()] = std::move(userFile); + } +} + +bool TUserFiles::HasFilePath(const TString& name) const { + auto guard = Guard(Mutex); + return Files.FindPtr(name) != nullptr; +} + +TString TUserFiles::GetFilePath(const TString& name) const { + auto guard = Guard(Mutex); + auto x = Files.FindPtr(name); + YQL_ENSURE(x); + return x->Path->GetPath(); +} + +bool TUserFiles::FindFolder(const TString& name, TVector<TString>& files) const { + auto guard = Guard(Mutex); + auto prefix = TUserDataStorage::MakeFolderName(name); + for (auto& x : Files) { + if (x.first.StartsWith(prefix)) { + files.push_back(x.first); + } + } + + return !files.empty(); +} + +const TUserFiles::TFileInfo* TUserFiles::GetFile(const TString& name) const { + auto guard = Guard(Mutex); + return Files.FindPtr(name); +} + +THashMap<TString, TUserFiles::TFileInfo> TUserFiles::GetFiles() const { + auto guard = Guard(Mutex); + return Files; +} + +inline bool TUserFiles::IsEmpty() const { + auto guard = Guard(Mutex); + return Files.empty(); +} + +} // NYql + + diff --git a/yt/yql/providers/yt/gateway/lib/user_files.h b/yt/yql/providers/yt/gateway/lib/user_files.h new file mode 100644 index 0000000000..3b24f2b6da --- /dev/null +++ b/yt/yql/providers/yt/gateway/lib/user_files.h @@ -0,0 +1,53 @@ +#pragma once + +#include <yt/yql/providers/yt/lib/url_mapper/yql_yt_url_mapper.h> +#include <yql/essentials/core/file_storage/storage.h> +#include <yql/essentials/core/type_ann/type_ann_core.h> +#include <yql/essentials/core/yql_user_data.h> + +#include <util/system/mutex.h> +#include <util/generic/ptr.h> +#include <util/generic/hash.h> +#include <util/generic/string.h> + +class IThreadPool; + +namespace NYql { + +class TYtGatewayConfig; + +class TUserFiles: public TThrRefBase { +public: + using TPtr = TIntrusivePtr<TUserFiles>; + + struct TFileInfo { + TFileLinkPtr Path; // Real path in storage + bool IsUdf = false; + bool IsPgExt = false; + bool IsPgCatalog = false; + ui64 InMemorySize = 0; + TString RemotePath; + double RemoteMemoryFactor = 0.; + bool BypassArtifactCache = false; + }; + +public: + TUserFiles(const TYtUrlMapper& urlMapper, const TString& activeCluster); + + void AddFile(const TUserDataKey& key, const TUserDataBlock& block); + + bool HasFilePath(const TString& name) const; + bool FindFolder(const TString& name, TVector<TString>& files) const; + TString GetFilePath(const TString& name) const; + const TFileInfo* GetFile(const TString& name) const; + THashMap<TString, TFileInfo> GetFiles() const; + bool IsEmpty() const; + +private: + const TYtUrlMapper& UrlMapper; + const TString ActiveCluster; + THashMap<TString, TFileInfo> Files; + TMutex Mutex; +}; + +} // NYql diff --git a/yt/yql/providers/yt/gateway/lib/ya.make b/yt/yql/providers/yt/gateway/lib/ya.make new file mode 100644 index 0000000000..bbd5e0b1d3 --- /dev/null +++ b/yt/yql/providers/yt/gateway/lib/ya.make @@ -0,0 +1,43 @@ +LIBRARY() + +SRCS( + query_cache.cpp + query_cache.h + temp_files.cpp + temp_files.h + transaction_cache.cpp + transaction_cache.h + user_files.cpp + user_files.h + yt_helpers.cpp + yt_helpers.h +) + +PEERDIR( + library/cpp/regex/pcre + library/cpp/string_utils/url + library/cpp/threading/future + library/cpp/yson/node + yt/cpp/mapreduce/client + yt/cpp/mapreduce/common + yt/cpp/mapreduce/interface + yql/essentials/core/file_storage + yql/essentials/public/issue + yql/essentials/utils + yql/essentials/utils/log + yql/essentials/utils/threading + yql/essentials/core/type_ann + yql/essentials/providers/common/codec + yql/essentials/providers/common/gateway + yql/essentials/providers/common/metrics + yt/yql/providers/yt/provider + yt/yql/providers/yt/common + yt/yql/providers/yt/lib/hash + yt/yql/providers/yt/lib/res_pull + yt/yql/providers/yt/lib/url_mapper + yt/yql/providers/yt/lib/yson_helpers +) + +YQL_LAST_ABI_VERSION() + +END() diff --git a/yt/yql/providers/yt/gateway/lib/yt_helpers.cpp b/yt/yql/providers/yt/gateway/lib/yt_helpers.cpp new file mode 100644 index 0000000000..1f20fc7dae --- /dev/null +++ b/yt/yql/providers/yt/gateway/lib/yt_helpers.cpp @@ -0,0 +1,686 @@ +#include "yt_helpers.h" + +#include <yt/yql/providers/yt/lib/res_pull/table_limiter.h> +#include <yt/yql/providers/yt/lib/res_pull/res_or_pull.h> +#include <yt/yql/providers/yt/lib/yson_helpers/yson_helpers.h> +#include <yt/yql/providers/yt/common/yql_names.h> +#include <yt/yql/providers/yt/codec/yt_codec.h> +#include <yql/essentials/providers/common/gateway/yql_provider_gateway.h> +#include <yql/essentials/core/issue/yql_issue.h> +#include <yql/essentials/core/yql_type_annotation.h> +#include <yql/essentials/minikql/aligned_page_pool.h> +#include <yql/essentials/utils/log/log.h> + +#include <yt/cpp/mapreduce/common/helpers.h> +#include <yt/cpp/mapreduce/interface/config.h> +#include <yt/cpp/mapreduce/interface/error_codes.h> +#include <yt/cpp/mapreduce/interface/errors.h> +#include <yt/cpp/mapreduce/interface/serialize.h> + +#include <library/cpp/yson/node/node_io.h> +#include <library/cpp/threading/future/future.h> + +#include <util/string/split.h> +#include <util/system/env.h> +#include <util/generic/hash_set.h> +#include <util/generic/hash.h> +#include <util/generic/map.h> +#include <util/generic/set.h> +#include <util/generic/string.h> +#include <util/generic/strbuf.h> +#include <util/generic/yexception.h> +#include <util/generic/algorithm.h> + +namespace NYql { + +namespace { +EYqlIssueCode IssueCodeForYtError(const NYT::TYtError& error) { + if (error.ContainsErrorCode(NYT::NClusterErrorCodes::NSecurityClient::AuthorizationError) || + error.ContainsErrorCode(NYT::NClusterErrorCodes::NSecurityClient::AuthenticationError)) { + return TIssuesIds::YT_ACCESS_DENIED; + } + + if (error.ContainsErrorCode(NYT::NClusterErrorCodes::NChunkPools::MaxDataWeightPerJobExceeded)) { + return TIssuesIds::YT_MAX_DATAWEIGHT_PER_JOB_EXCEEDED; + } + + return TIssuesIds::DEFAULT_ERROR; +} +} + +TMaybe<ui64> GetUsedRows(const NYT::TRichYPath& table, ui64 tableRowCount) { + TMaybe<ui64> rows; + if (auto ranges = table.GetRanges()) { + rows = 0; + for (const NYT::TReadRange& readRange: *ranges) { + if (readRange.Exact_.RowIndex_) { + rows = rows.GetOrElse(0) + 1; + } else if (readRange.LowerLimit_.RowIndex_ || readRange.UpperLimit_.RowIndex_) { + ui64 range = tableRowCount; + if (readRange.UpperLimit_.RowIndex_) { + range = *readRange.UpperLimit_.RowIndex_; + } + if (readRange.LowerLimit_.RowIndex_) { + range -= Min<ui64>(range, *readRange.LowerLimit_.RowIndex_); + } + rows = rows.GetOrElse(0) + range; + } else { + return Nothing(); + } + } + } + return rows; +} + +TMaybe<ui64> GetUncompressedFileSize(NYT::ITransactionPtr tx, const TString& path) { + if (!tx->Exists(path)) { + return 0; + } + + NYT::TNode attrs = tx->Get(path + "/@", NYT::TGetOptions().AttributeFilter( + NYT::TAttributeFilter() + .AddAttribute(TString("uncompressed_data_size")) + )); + + return GetDataWeight(attrs); +} + +TString TransformPath(TStringBuf tmpFolder, TStringBuf name, bool isTempTable, TStringBuf userName) { + TString path; + path.assign(name); + if (isTempTable && tmpFolder) { + path = tmpFolder; + if (!tmpFolder.EndsWith('/')) { + path += '/'; + } + path += name; + } + if (path.StartsWith("//")) { + return path.substr(2); + } + + if (isTempTable && !tmpFolder && path.StartsWith("tmp/")) { + TStringBuilder builder; + builder << "tmp/yql/"; + if (userName) { + builder << userName << '/'; + } + builder << path.substr(4); + path = builder; + } + + return path; +} + +namespace { + +THashSet<TStringBuf> DEPRECATED_YQL_ATTRS = { + TStringBuf("_yql_key_meta"), + TStringBuf("_yql_subkey_meta"), + TStringBuf("_yql_value_meta"), +}; + +THashSet<TStringBuf> TEST_YQL_ATTRS = { + YqlDynamicAttribute, +}; + +THashSet<TStringBuf> SERVICE_YQL_ATTRS = { + TStringBuf("_yql_runner"), + TStringBuf("_yql_op_id"), + TStringBuf("_yql_op_title"), + TStringBuf("_yql_query_name"), +}; + +THashSet<TString> SUPPORTED_RICH_YPATH_ATTRS = { + "timestamp" +}; + +} + +TMaybe<TString> SerializeRichYPathAttrs(const NYT::TRichYPath& richPath) { + NYT::TNode pathNode; + NYT::TNodeBuilder builder(&pathNode); + NYT::Serialize(richPath, &builder); + if (!pathNode.HasAttributes() || pathNode.GetAttributes().Empty()) { + return Nothing(); + } + auto attrMap = pathNode.GetAttributes().AsMap(); + attrMap.erase("columns"); + attrMap.erase("ranges"); + for (const auto& [attr, _] : attrMap) { + if (!SUPPORTED_RICH_YPATH_ATTRS.contains(attr)) { + throw yexception() << "Unsupported YPath attribute: '" << attr << "'"; + } + } + pathNode.Attributes() = attrMap; + return NYT::NodeToYsonString(pathNode.GetAttributes()); +} + +void DeserializeRichYPathAttrs(const TString& serializedAttrs, NYT::TRichYPath& richPath) { + NYT::TNode pathNode; + NYT::TNodeBuilder pathNodeBuilder(&pathNode); + NYT::Serialize(richPath, &pathNodeBuilder); + NYT::MergeNodes(pathNode.Attributes(), NYT::NodeFromYsonString(serializedAttrs)); + NYT::Deserialize(richPath, pathNode); +} + +IYtGateway::TCanonizedPath CanonizedPath(const TString& path) { + NYT::TRichYPath richYPath(path); + if (path.StartsWith('<')) { + NYT::Deserialize(richYPath, NYT::NodeFromYsonString(path)); + } + const auto additionalAttrs = SerializeRichYPathAttrs(richYPath); + size_t pos = 0; + if ((pos = richYPath.Path_.find('{')) != TString::npos) { + size_t end = richYPath.Path_.find('}'); + YQL_ENSURE(end != TString::npos && end > pos); + TVector<TString> columns; + StringSplitter(richYPath.Path_.substr(pos + 1, end - pos - 1)).Split(',').AddTo(&columns); + richYPath.Columns(columns); + richYPath.Path_ = richYPath.Path_.substr(0, pos); + } + + if ((pos = richYPath.Path_.find('[')) != TString::npos) { + size_t end = richYPath.Path_.find(']'); + YQL_ENSURE(end != TString::npos && end > pos); + TString rangeString = richYPath.Path_.substr(pos + 1, end - pos - 1); + richYPath.Path_ = richYPath.Path_.substr(0, pos); + TVector<TString> ranges; + size_t startPos = 0; + int insideParens = 0; + for (size_t i = 0; i < rangeString.length(); ++i) { + switch (rangeString.at(i)) { + case '(': + ++insideParens; + break; + case ')': + --insideParens; + break; + case ',': + if (0 == insideParens) { + ranges.push_back(rangeString.substr(startPos, i - startPos)); + startPos = i + 1; + } + break; + } + } + if (startPos < rangeString.length()) { + ranges.push_back(rangeString.substr(startPos)); + } + auto toReadLimit = [] (const TString& s) -> NYT::TReadLimit { + if (s.StartsWith('#')) { + return NYT::TReadLimit().RowIndex(FromString<i64>(s.substr(1))); + } else if (s.StartsWith('(')) { + YQL_ENSURE(s.EndsWith(')')); + TVector<TString> keys; + StringSplitter(s.substr(1, s.length() - 2)).Split(',').AddTo(&keys); + NYT::TKey complexKey; + for (auto& key: keys) { + complexKey.Add(NYT::NodeFromYsonString(key)); + } + return NYT::TReadLimit().Key(complexKey); + } else { + return NYT::TReadLimit().Key(NYT::NodeFromYsonString(s)); + } + }; + + richYPath.MutableRanges().ConstructInPlace(); + for (TString& r: ranges) { + if ((pos = r.find(':')) != TString::npos) { + NYT::TReadRange range; + if (TString lower = r.substr(0, pos)) { + range.LowerLimit(toReadLimit(lower)); + } + if (TString upper = r.substr(pos + 1)) { + range.UpperLimit(toReadLimit(upper)); + } + richYPath.AddRange(range); + } else { + richYPath.AddRange(NYT::TReadRange().Exact(toReadLimit(r))); + } + } + } + return { + richYPath.Path_, + richYPath.Columns_.Defined() ? richYPath.Columns_->Parts_ : TMaybe<TVector<TString>>(), + richYPath.GetRanges(), + additionalAttrs + }; +}; + +NYT::TNode GetUserAttributes(NYT::ITransactionPtr tx, TString path, TMaybe<bool> includeYqlAttrs) { + path.append("/@"); + NYT::TNode attrs = tx->Get(path, NYT::TGetOptions() + .AttributeFilter(NYT::TAttributeFilter() + .AddAttribute("user_attribute_keys") + ) + ); + if (attrs.HasKey("user_attribute_keys")) { + NYT::TAttributeFilter filter; + for (auto key: attrs["user_attribute_keys"].AsList()) { + if (key.AsString().StartsWith("_yql") && includeYqlAttrs) { + if (!*includeYqlAttrs + || TEST_YQL_ATTRS.contains(key.AsString()) + || DEPRECATED_YQL_ATTRS.contains(key.AsString()) + || SERVICE_YQL_ATTRS.contains(key.AsString())) { + continue; + } + } + filter.AddAttribute(key.AsString()); + } + if (!filter.Attributes_.empty()) { + return tx->Get(path, NYT::TGetOptions().AttributeFilter(filter)); + } + } + return NYT::TNode::CreateMap(); +} + +void TransferTableAttributes(const NYT::TNode& attributes, const std::function<void(const TString&,const TString&)>& receiver) +{ + for (const auto& attr : attributes.AsMap()) { + const TString& attrName = attr.first; + const NYT::TNode& attrValue = attr.second; + + if (attrName == FORMAT_ATTR_NAME) { + receiver(attrName, NYT::NodeToYsonString(attrValue)); + } + else if (attrName.StartsWith(TStringBuf("_yql"))) { + if (attrName == YqlRowSpecAttribute) { + receiver(attrName, NYT::NodeToYsonString(attrValue)); + } else if (!TEST_YQL_ATTRS.contains(attrName) + && !DEPRECATED_YQL_ATTRS.contains(attrName) + && !SERVICE_YQL_ATTRS.contains(attr.first)) { + try { + receiver(attrName, attrValue.ConvertTo<TString>()); + } catch (const NYT::TNode::TTypeError&) { + throw yexception() << "Unexpected value of '" << attrName << "' attribute: " << NYT::NodeToYsonString(attrValue); + } + } + } + } +} + +NYT::TNode FilterYqlAttributes(const NYT::TNode& attributes) +{ + NYT::TNode res = NYT::TNode::CreateMap(); + for (const auto& attr : attributes.AsMap()) { + if (attr.first.StartsWith(TStringBuf("_yql")) + && !TEST_YQL_ATTRS.contains(attr.first) + && !DEPRECATED_YQL_ATTRS.contains(attr.first) + && !SERVICE_YQL_ATTRS.contains(attr.first)) { + + res[attr.first] = attr.second; + } + } + return res; +} + +template <bool YAMRED_DSV> +static bool IterateRows(NYT::ITransactionPtr tx, + NYT::TRichYPath path, + ui32 tableIndex, + TMkqlIOCache& specsCache, + IExecuteResOrPull& exec, + const TTableLimiter& limiter, + const TMaybe<TSampleParams>& sampling) +{ + const ui64 startRecordInTable = limiter.GetTableStart(); + const ui64 endRecordInTable = limiter.GetTableZEnd(); // 0 means the entire table usage + + if (startRecordInTable || endRecordInTable) { + YQL_ENSURE(path.GetRanges().Empty()); + NYT::TReadRange readRange; + if (startRecordInTable) { + readRange.LowerLimit(NYT::TReadLimit().RowIndex(startRecordInTable)); + } + if (endRecordInTable) { + readRange.UpperLimit(NYT::TReadLimit().RowIndex(endRecordInTable)); + } + path.AddRange(readRange); + } + + NYT::TTableReaderOptions readerOptions; + if (sampling) { + NYT::TNode spec = NYT::TNode::CreateMap(); + spec["sampling_rate"] = sampling->Percentage / 100.; + if (sampling->Repeat) { + spec["sampling_seed"] = static_cast<i64>(sampling->Repeat); + } + if (sampling->Mode == EYtSampleMode::System) { + spec["sampling_mode"] = "block"; + } + readerOptions.Config(spec); + } + + if (!YAMRED_DSV && exec.GetColumns()) { + if (!specsCache.GetSpecs().Inputs[tableIndex]->OthersStructIndex) { + path.Columns(*exec.GetColumns()); + } + } + + if (YAMRED_DSV) { + path.Columns_.Clear(); + auto reader = tx->CreateTableReader<NYT::TYaMRRow>(path, readerOptions); + for (; reader->IsValid(); reader->Next()) { + if (!exec.WriteNext(specsCache, reader->GetRow(), tableIndex)) { + return true; + } + } + } else { + auto format = specsCache.GetSpecs().MakeInputFormat(tableIndex); + auto rawReader = tx->CreateRawReader(path, format, readerOptions); + TMkqlReaderImpl reader(*rawReader, 0, 4 << 10, tableIndex); + reader.SetSpecs(specsCache.GetSpecs(), specsCache.GetHolderFactory()); + + for (reader.Next(); reader.IsValid(); reader.Next()) { + if (!exec.WriteNext(specsCache, reader.GetRow(), tableIndex)) { + return true; + } + } + } + + return false; +} + +bool IterateYamredRows(NYT::ITransactionPtr tx, + const NYT::TRichYPath& table, + ui32 tableIndex, + TMkqlIOCache& specsCache, + IExecuteResOrPull& exec, + const TTableLimiter& limiter, + const TMaybe<TSampleParams>& sampling) +{ + return IterateRows<true>(tx, table, tableIndex, specsCache, exec, limiter, sampling); +} + +bool IterateYsonRows(NYT::ITransactionPtr tx, + const NYT::TRichYPath& table, + ui32 tableIndex, + TMkqlIOCache& specsCache, + IExecuteResOrPull& exec, + const TTableLimiter& limiter, + const TMaybe<TSampleParams>& sampling) +{ + return IterateRows<false>(tx, table, tableIndex, specsCache, exec, limiter, sampling); +} + +bool SelectRows(NYT::IClientPtr client, + const TString& table, + ui32 tableIndex, + TMkqlIOCache& specsCache, + IExecuteResOrPull& exec, + TTableLimiter& limiter) +{ + ui64 startRecordInTable = limiter.GetTableStart(); + const ui64 endRecordInTable = limiter.GetTableZEnd(); // 0 means the entire table usage + TStringStream sqlBuilder; + const auto& columns = exec.GetColumns(); + if (columns) { + bool isFirstColumn = true; + for (auto& x : *columns) { + if (!isFirstColumn) { + sqlBuilder << ", "; + } + + isFirstColumn = false; + sqlBuilder << "[" << x << "]"; + } + } else { + sqlBuilder << "*"; + } + + sqlBuilder << " FROM ["; + sqlBuilder << NYT::AddPathPrefix(table, NYT::TConfig::Get()->Prefix); + sqlBuilder << "]"; + if (exec.GetRowsLimit()) { + ui64 effectiveLimit = endRecordInTable; + if (!effectiveLimit) { + effectiveLimit = startRecordInTable + *exec.GetRowsLimit() + 1; + } else { + effectiveLimit = Min(effectiveLimit, *exec.GetRowsLimit() + 1); + } + + sqlBuilder << " LIMIT " << effectiveLimit; + } + + ui64 processed = 0; + bool ret = false; + auto rows = client->SelectRows(sqlBuilder.Str()); + for (const auto& row : rows) { + ++processed; + if (processed <= startRecordInTable) { + continue; + } + + if (!exec.WriteNext(specsCache, row, tableIndex)) { + ret = true; + break; + } + + if (endRecordInTable) { + if (processed >= endRecordInTable) { + break; + } + } + } + + limiter.Skip(processed); + return ret; +} + +NYT::TNode YqlOpOptionsToSpec(const TYqlOperationOptions& opOpts, const TString& userName, const TVector<std::pair<TString, TString>>& code) +{ + NYT::TNode spec = NYT::TNode::CreateMap(); + + if (auto title = opOpts.Title.GetOrElse(TString())) { + spec["title"] = title; + } else { + TStringBuilder titleBuilder; + titleBuilder << "YQL operation ("; + if (opOpts.QueryName) { + titleBuilder << *opOpts.QueryName; + } + if (opOpts.Id) { + if (opOpts.QueryName) { + titleBuilder << ", "; + } + titleBuilder << *opOpts.Id; + } + titleBuilder << " by " << userName << ')'; + spec["title"] = titleBuilder; + } + + NYT::TNode& description = spec["description"]; + description["yql_runner"] = opOpts.Runner; + + if (auto id = opOpts.Id.GetOrElse(TString())) { + description["yql_op_id"] = id; + } + + if (auto url = opOpts.Url.GetOrElse(TString())) { + NYT::TNode& urlNode = description["yql_op_url"]; + urlNode = url; + // Mark as URL for YT UI (see https://clubs.at.yandex-team.ru/yt/2364) + urlNode.Attributes()["_type_tag"] = "url"; + } + + if (auto title = opOpts.Title.GetOrElse(TString())) { + description["yql_op_title"] = title; + } + + if (auto name = opOpts.QueryName.GetOrElse(TString())) { + description["yql_query_name"] = name; + } + + static constexpr size_t OP_CODE_LIMIT = 1ul << 17; // 128Kb + + if (!code.empty()) { + size_t remaining = OP_CODE_LIMIT; + NYT::TNode& codeNode = description["yql_op_code"]; + for (auto& c: code) { + TString snippet = c.second; + if (!remaining) { + snippet = "__truncated__"; + } else if (snippet.length() > remaining) { + // Keep the end part of the code as more interesting + snippet = TStringBuilder() << "__truncated__\n" << TStringBuf(snippet).Last(remaining) << "\n__truncated__"; + } + codeNode[c.first] = snippet; + remaining -= Min(remaining, snippet.length()); + } + } + + if (auto attrs = opOpts.AttrsYson.GetOrElse(TString())) { + NYT::TNode userAttrs = NYT::NodeFromYsonString(attrs); + for (const auto& item: userAttrs.AsMap()) { + const TString& key = item.first; + const NYT::TNode& value = item.second; + + if (key != TStringBuf("runner") && + key != TStringBuf("op_id") && + key != TStringBuf("op_url") && + key != TStringBuf("op_title") && + key != TStringBuf("query_name") && + key != TStringBuf("op_code")) + { + // do not allow to override specific attrs + description[TString("yql_") + key] = value; + } + } + } + + return spec; +} + +NYT::TNode YqlOpOptionsToAttrs(const TYqlOperationOptions& opOpts) { + NYT::TNode attrs = NYT::TNode::CreateMap(); + + attrs["_yql_runner"] = opOpts.Runner; + if (auto id = opOpts.Id.GetOrElse(TString())) { + attrs["_yql_op_id"] = id; + } + if (auto title = opOpts.Title.GetOrElse(TString())) { + attrs["_yql_op_title"] = title; + } + if (auto name = opOpts.QueryName.GetOrElse(TString())) { + attrs["_yql_query_name"] = name; + } + return attrs; +} + +void CreateParents(const TVector<TString>& tables, NYT::IClientBasePtr tx) { + auto batchExists = tx->CreateBatchRequest(); + TVector<NThreading::TFuture<void>> batchExistsRes; + + THashSet<TString> uniqFolders; + auto batchCreateParent = tx->CreateBatchRequest(); + TVector<NThreading::TFuture<NYT::TLockId>> batchCreateParentRes; + + for (auto& table: tables) { + auto slash = table.rfind('/'); + if (TString::npos != slash) { + TString folder = table.substr(0, slash); + if (uniqFolders.insert(folder).second) { + batchExistsRes.push_back( + batchExists->Exists(folder).Apply([&batchCreateParentRes, &batchCreateParent, folder](const NThreading::TFuture<bool>& f) { + if (!f.GetValue()) { + batchCreateParentRes.push_back(batchCreateParent->Create(folder, NYT::NT_MAP, + NYT::TCreateOptions().Recursive(true).IgnoreExisting(true))); + } + }) + ); + } + } + } + + batchExists->ExecuteBatch(); + ForEach(batchExistsRes.begin(), batchExistsRes.end(), [] (const NThreading::TFuture<void>& f) { + f.GetValue(); + }); + + if (!batchCreateParentRes.empty()) { + batchCreateParent->ExecuteBatch(); + ForEach(batchCreateParentRes.begin(), batchCreateParentRes.end(), [] (const NThreading::TFuture<NYT::TLockId>& f) { + f.GetValue(); + }); + } +} + +TIssue MakeIssueFromYtError(const NYT::TYtError& e, TStringBuf what, TPosition pos, bool shortErrors) { + TString errMsg = shortErrors || GetEnv("YQL_DETERMINISTIC_MODE") ? e.ShortDescription() : TString(what); + EYqlIssueCode rootIssueCode = IssueCodeForYtError(e); + return YqlIssue(pos, rootIssueCode, errMsg); +} + +namespace { + +void FillResultFromOperationError(NCommon::TOperationResult& result, const NYT::TOperationFailedError& e, TPosition pos, bool shortErrors) { + TIssue rootIssue = MakeIssueFromYtError(e.GetError(), e.what(), pos, shortErrors); + + if (!e.GetFailedJobInfo().empty()) { + TSet<TString> uniqueErrors; + for (auto& failedJob: e.GetFailedJobInfo()) { + TStringBuf message = failedJob.Stderr; + auto parsedPos = TryParseTerminationMessage(message); + if (message.size() < failedJob.Stderr.size()) { + if (uniqueErrors.emplace(message).second) { + rootIssue.AddSubIssue(MakeIntrusive<TIssue>(YqlIssue(parsedPos.GetOrElse(pos), TIssuesIds::DEFAULT_ERROR, TString{message}))); + } + } else { + TString errorDescription = failedJob.Error.ShortDescription(); + if (uniqueErrors.insert(errorDescription).second) { + rootIssue.AddSubIssue(MakeIntrusive<TIssue>(YqlIssue(pos, TIssuesIds::UNEXPECTED, errorDescription))); + } + } + } + } + + result.SetStatus(TIssuesIds::DEFAULT_ERROR); + result.AddIssue(rootIssue); +} + +void FillResultFromErrorResponse(NCommon::TOperationResult& result, const NYT::TErrorResponse& e, TPosition pos, bool shortErrors) { + TIssue rootIssue = MakeIssueFromYtError(e.GetError(), e.what(), pos, shortErrors); + + result.SetStatus(TIssuesIds::DEFAULT_ERROR); + result.AddIssue(rootIssue); +} + +} // unnamed + +void FillResultFromCurrentException(NCommon::TOperationResult& result, TPosition pos, bool shortErrors) { + try { + throw; + } catch (const NYT::TOperationFailedError& e) { + FillResultFromOperationError(result, e, pos, shortErrors); + } catch (const NYT::TErrorResponse& e) { + FillResultFromErrorResponse(result, e, pos, shortErrors); + } catch (const std::exception& e) { + result.SetException(e, pos); + } catch (const NKikimr::TMemoryLimitExceededException&) { + result.SetStatus(TIssuesIds::UNEXPECTED); + result.AddIssue(TIssue(pos, "Memory limit exceeded in MKQL runtime")); + } catch (...) { + result.SetStatus(TIssuesIds::UNEXPECTED); + result.AddIssue(TIssue(pos, CurrentExceptionMessage())); + } +} + +void EnsureSpecDoesntUseNativeYtTypes(const NYT::TNode& spec, TStringBuf tableName, bool read) { + if (spec.HasKey(YqlRowSpecAttribute)) { + const auto& rowSpec = spec[YqlRowSpecAttribute]; + bool useNativeYtTypes = false; + if (rowSpec.HasKey(RowSpecAttrUseNativeYtTypes)) { + useNativeYtTypes = rowSpec[RowSpecAttrUseNativeYtTypes].AsBool(); + } else if (rowSpec.HasKey(RowSpecAttrUseTypeV2)) { + useNativeYtTypes = rowSpec[RowSpecAttrUseTypeV2].AsBool(); + } else if (rowSpec.HasKey(RowSpecAttrNativeYtTypeFlags)) { + useNativeYtTypes = rowSpec[RowSpecAttrNativeYtTypeFlags].AsUint64() > 0; + } + if (useNativeYtTypes) { + throw yexception() << "Cannot " << (read ? "read" : "modify") << " table \"" << tableName << "\" with type_v3 schema using yson codec"; + } + } +} + +} // NYql diff --git a/yt/yql/providers/yt/gateway/lib/yt_helpers.h b/yt/yql/providers/yt/gateway/lib/yt_helpers.h new file mode 100644 index 0000000000..f4a7622e08 --- /dev/null +++ b/yt/yql/providers/yt/gateway/lib/yt_helpers.h @@ -0,0 +1,75 @@ +#pragma once + +#include <yql/essentials/public/issue/yql_issue.h> +#include <yql/essentials/providers/common/gateway/yql_provider_gateway.h> +#include <yt/yql/providers/yt/provider/yql_yt_op_settings.h> +#include <yt/yql/providers/yt/provider/yql_yt_gateway.h> + +#include <yt/cpp/mapreduce/interface/client.h> +#include <yt/cpp/mapreduce/interface/common.h> + +#include <library/cpp/yson/node/node.h> + +#include <util/generic/maybe.h> +#include <util/generic/vector.h> +#include <util/generic/string.h> +#include <util/generic/map.h> + +#include <functional> +#include <utility> + +namespace NYql { + +class TMkqlIOCache; +class IExecuteResOrPull; +class TTableLimiter; +struct TYqlOperationOptions; + +namespace NCommon { + +class TOperationResult; + +} + +TMaybe<ui64> GetUsedRows(const NYT::TRichYPath& table, ui64 tableRowCount); +TMaybe<ui64> GetUncompressedFileSize(NYT::ITransactionPtr tx, const TString& path); + +TString TransformPath(TStringBuf tmpFolder, TStringBuf name, bool isTempTable, TStringBuf userName); + +NYT::TNode GetUserAttributes(NYT::ITransactionPtr tx, TString path, TMaybe<bool> includeYqlAttrs = Nothing()); +void TransferTableAttributes(const NYT::TNode& attributes, const std::function<void(const TString&,const TString&)>& receiver); +NYT::TNode FilterYqlAttributes(const NYT::TNode& attributes); + +bool IterateYamredRows(NYT::ITransactionPtr tx, const NYT::TRichYPath& table, ui32 tableIndex, TMkqlIOCache& specsCache, + IExecuteResOrPull& exec, const TTableLimiter& limiter, const TMaybe<TSampleParams>& sampling = {}); +bool IterateYsonRows(NYT::ITransactionPtr tx, const NYT::TRichYPath& table, ui32 tableIndex, TMkqlIOCache& specsCache, + IExecuteResOrPull& exec, const TTableLimiter& limiter, const TMaybe<TSampleParams>& sampling = {}); +bool SelectRows(NYT::IClientPtr client, const TString& table, ui32 tableIndex, TMkqlIOCache& specsCache, + IExecuteResOrPull& exec, TTableLimiter& limiter); + +NYT::TNode YqlOpOptionsToSpec(const TYqlOperationOptions& opOpts, const TString& userName, const TVector<std::pair<TString, TString>>& code = {}); +NYT::TNode YqlOpOptionsToAttrs(const TYqlOperationOptions& opOpts); + +void CreateParents(const TVector<TString>& tables, NYT::IClientBasePtr tx); + +// must be used inside 'catch' because it rethrows current exception to analyze it's type +void FillResultFromCurrentException(NCommon::TOperationResult& result, TPosition pos = {}, bool shortErrors = false); + +// must be used inside 'catch' because it rethrows current exception to analyze it's type +template<typename TResult> +static TResult ResultFromCurrentException(TPosition pos = {}, bool shortErrors = false) { + TResult result; + FillResultFromCurrentException(result, pos, shortErrors); + return result; +} + +TMaybe<TString> SerializeRichYPathAttrs(const NYT::TRichYPath& richPath); +void DeserializeRichYPathAttrs(const TString& serializedAttrs, NYT::TRichYPath& richPath); + +IYtGateway::TCanonizedPath CanonizedPath(const TString& path); + +void EnsureSpecDoesntUseNativeYtTypes(const NYT::TNode& spec, TStringBuf tableName, bool read); + +TIssue MakeIssueFromYtError(const NYT::TYtError& e, TStringBuf what, TPosition pos = {}, bool shortErrors = false); + +} // NYql diff --git a/yt/yql/providers/yt/gateway/native/ut/ya.make b/yt/yql/providers/yt/gateway/native/ut/ya.make new file mode 100644 index 0000000000..702a53d5dd --- /dev/null +++ b/yt/yql/providers/yt/gateway/native/ut/ya.make @@ -0,0 +1,28 @@ +IF (NOT OPENSOURCE) + +UNITTEST() + +SRCS( + yql_yt_native_folders_ut.cpp +) + +PEERDIR( + yt/yql/providers/yt/gateway/native + yt/yql/providers/yt/gateway/file + yt/yql/providers/yt/codec/codegen + yt/yql/providers/yt/comp_nodes/llvm14 + yql/essentials/core/ut_common + library/cpp/testing/mock_server + library/cpp/testing/common + yql/essentials/public/udf/service/terminate_policy + yql/essentials/sql/pg + yql/essentials/minikql/comp_nodes/llvm14 + yql/essentials/minikql/invoke_builtins/llvm14 +) + +YQL_LAST_ABI_VERSION() + +END() + +ENDIF() + diff --git a/yt/yql/providers/yt/gateway/native/ya.make b/yt/yql/providers/yt/gateway/native/ya.make new file mode 100644 index 0000000000..c0bad18eeb --- /dev/null +++ b/yt/yql/providers/yt/gateway/native/ya.make @@ -0,0 +1,69 @@ +LIBRARY() + +SRCS( + yql_yt_exec_ctx.cpp + yql_yt_lambda_builder.cpp + yql_yt_native.cpp + yql_yt_native_folders.cpp + yql_yt_op_tracker.cpp + yql_yt_qb2.cpp + yql_yt_session.cpp + yql_yt_spec.cpp + yql_yt_transform.cpp +) + +PEERDIR( + library/cpp/containers/sorted_vector + library/cpp/digest/md5 + library/cpp/random_provider + library/cpp/streams/brotli + library/cpp/threading/future + library/cpp/time_provider + library/cpp/yson + library/cpp/yson/node + yt/cpp/mapreduce/common + yt/cpp/mapreduce/interface + yql/essentials/ast + yql/essentials/core/file_storage + yql/essentials/minikql/comp_nodes + yql/essentials/utils + yql/essentials/utils/log + yql/essentials/utils/threading + yql/essentials/core + yql/essentials/core/expr_nodes + yql/essentials/core/issue + yql/essentials/providers/common/codec + yql/essentials/providers/common/comp_nodes + yql/essentials/providers/common/metrics + yql/essentials/providers/common/mkql + yql/essentials/providers/common/proto + yql/essentials/providers/common/provider + yql/essentials/providers/common/schema/expr + yql/essentials/providers/result/expr_nodes + yt/yql/providers/yt/codec + yt/yql/providers/yt/common + yt/yql/providers/yt/expr_nodes + yt/yql/providers/yt/gateway/lib + yt/yql/providers/yt/job + yt/yql/providers/yt/lib/expr_traits + yt/yql/providers/yt/lib/infer_schema + yt/yql/providers/yt/lib/lambda_builder + yt/yql/providers/yt/lib/log + yt/yql/providers/yt/lib/mkql_helpers + yt/yql/providers/yt/lib/res_pull + yt/yql/providers/yt/lib/schema + yt/yql/providers/yt/lib/skiff + yt/yql/providers/yt/lib/url_mapper + yt/yql/providers/yt/lib/yson_helpers + yt/yql/providers/yt/lib/init_yt_api + yt/yql/providers/yt/lib/config_clusters + yt/yql/providers/yt/provider +) + +YQL_LAST_ABI_VERSION() + +END() + +RECURSE_FOR_TESTS( + ut +) diff --git a/yt/yql/providers/yt/gateway/native/yql_yt_exec_ctx.cpp b/yt/yql/providers/yt/gateway/native/yql_yt_exec_ctx.cpp new file mode 100644 index 0000000000..2e10daa628 --- /dev/null +++ b/yt/yql/providers/yt/gateway/native/yql_yt_exec_ctx.cpp @@ -0,0 +1,416 @@ +#include "yql_yt_exec_ctx.h" + +#include "yql_yt_spec.h" + +#include <yt/yql/providers/yt/gateway/lib/yt_helpers.h> +#include <yt/yql/providers/yt/provider/yql_yt_op_settings.h> +#include <yt/yql/providers/yt/provider/yql_yt_table.h> +#include <yt/yql/providers/yt/codec/yt_codec.h> +#include <yt/yql/providers/yt/lib/schema/schema.h> +#include <yt/yql/providers/yt/common/yql_names.h> +#include <yt/yql/providers/yt/common/yql_configuration.h> +#include <yql/essentials/providers/common/codec/yql_codec_type_flags.h> +#include <yql/essentials/providers/common/proto/gateways_config.pb.h> + +#include <yql/essentials/utils/log/log.h> +#include <yql/essentials/utils/yql_panic.h> + +#include <library/cpp/yson/node/node_io.h> +#include <yt/cpp/mapreduce/interface/config.h> +#include <yt/cpp/mapreduce/common/helpers.h> + +#include <util/generic/hash.h> +#include <util/generic/strbuf.h> +#include <util/generic/yexception.h> +#include <util/generic/xrange.h> +#include <util/string/builder.h> +#include <util/system/guard.h> +#include <util/system/platform.h> + +#include <type_traits> + +namespace NYql { + +namespace NNative { + +using namespace NNodes; + +TExecContextBase::TExecContextBase(const TYtNativeServices& services, + const TConfigClusters::TPtr& clusters, + const TIntrusivePtr<NCommon::TMkqlCommonCallableCompiler>& mkqlCompiler, + const TSession::TPtr& session, + const TString& cluster, + const TYtUrlMapper& urlMapper, + IMetricsRegistryPtr metrics) + : FunctionRegistry_(services.FunctionRegistry) + , FileStorage_(services.FileStorage) + , Config_(services.Config) + , Clusters_(clusters) + , MkqlCompiler_(mkqlCompiler) + , Session_(session) + , Cluster_(cluster) + , UrlMapper_(urlMapper) + , DisableAnonymousClusterAccess_(services.DisableAnonymousClusterAccess) + , Hidden(session->SessionId_.EndsWith("_hidden")) + , Metrics(std::move(metrics)) +{ + YtServer_ = Clusters_->GetServer(Cluster_); + LogCtx_ = NYql::NLog::CurrentLogContextPath(); +} + + +void TExecContextBase::MakeUserFiles(const TUserDataTable& userDataBlocks) { + const TString& activeYtCluster = Clusters_->GetYtName(Cluster_); + UserFiles_ = MakeIntrusive<TUserFiles>(UrlMapper_, activeYtCluster); + for (const auto& file: userDataBlocks) { + auto block = file.second; + if (!Config_->GetMrJobUdfsDir().empty() && block.Usage.Test(EUserDataBlockUsage::Udf) && block.Type == EUserDataType::PATH) { + TFsPath path = block.Data; + TString fileName = path.Basename(); +#ifdef _win_ + TStringBuf changedName(fileName); + changedName.ChopSuffix(".dll"); + fileName = TString("lib") + changedName + ".so"; +#endif + block.Data = TFsPath(Config_->GetMrJobUdfsDir()) / fileName; + TString md5; + if (block.FrozenFile) { + md5 = block.FrozenFile->GetMd5(); + } + block.FrozenFile = CreateFakeFileLink(block.Data, md5); + } + + UserFiles_->AddFile(file.first, block); + } +} + +void TExecContextBase::SetInput(TExprBase input, bool forcePathColumns, const THashSet<TString>& extraSysColumns, const TYtSettings::TConstPtr& settings) { + const TString tmpFolder = GetTablesTmpFolder(*settings); + + NYT::TNode extraSysColumnsNode; + for (auto sys: extraSysColumns) { + extraSysColumnsNode.Add(sys); + } + + if (auto out = input.Maybe<TYtOutput>()) { // Pull case + auto tableInfo = TYtTableBaseInfo::Parse(out.Cast()); + YQL_CLOG(INFO, ProviderYt) << "Input: " << Cluster_ << '.' << tableInfo->Name; + NYT::TRichYPath richYPath(NYql::TransformPath(tmpFolder, tableInfo->Name, true, Session_->UserName_)); + + auto spec = tableInfo->GetCodecSpecNode(); + if (!extraSysColumnsNode.IsUndefined()) { + spec[YqlSysColumnPrefix] = extraSysColumnsNode; + } + + InputTables_.emplace_back( + richYPath.Path_, + richYPath, + true, + true, + *tableInfo, + spec, + 0 + ); + } + else { + TMaybe<bool> hasScheme; + size_t loggedTable = 0; + const auto entry = Config_->GetLocalChainTest() ? TTransactionCache::TEntry::TPtr() : GetEntry(); + + auto fillSection = [&] (TYtSection section, ui32 group) { + TVector<TStringBuf> columns; + auto sysColumnsSetting = NYql::GetSettingAsColumnList(section.Settings().Ref(), EYtSettingType::SysColumns); + if (forcePathColumns) { + for (auto& colType: section.Ref().GetTypeAnn()->Cast<TListExprType>()->GetItemType()->Cast<TStructExprType>()->GetItems()) { + if (auto name = colType->GetName(); !name.SkipPrefix(YqlSysColumnPrefix) || Find(sysColumnsSetting, name) == sysColumnsSetting.cend()) { + columns.push_back(colType->GetName()); + } + } + } + NYT::TNode sysColumns = extraSysColumnsNode; + for (auto sys: sysColumnsSetting) { + if (!extraSysColumns.contains(sys)) { + sysColumns.Add(sys); + } + } + for (auto path: section.Paths()) { + TYtPathInfo pathInfo(path); + if (loggedTable++ < 10) { + YQL_CLOG(INFO, ProviderYt) << "Input: " << Cluster_ << '.' << pathInfo.Table->Name << '[' << group << ']'; + } + // Table may have aux columns. Exclude them by specifying explicit columns from the type + if (forcePathColumns && pathInfo.Table->RowSpec && !pathInfo.HasColumns()) { + pathInfo.SetColumns(columns); + } + auto name = NYql::TransformPath(tmpFolder, pathInfo.Table->Name, pathInfo.Table->IsTemp, Session_->UserName_); + NYT::TRichYPath richYPath; + if ((pathInfo.Table->IsTemp && !pathInfo.Table->IsAnonymous) || !entry) { + richYPath.Path(name); + } else { + auto p = entry->Snapshots.FindPtr(std::make_pair(name, pathInfo.Table->Epoch.GetOrElse(0))); + YQL_ENSURE(p, "Table " << pathInfo.Table->Name << " has no snapshot"); + richYPath.Path(std::get<0>(*p)).TransactionId(std::get<1>(*p)).OriginalPath(NYT::AddPathPrefix(name, NYT::TConfig::Get()->Prefix)); + } + pathInfo.FillRichYPath(richYPath); + + auto spec = pathInfo.GetCodecSpecNode(); + if (!sysColumns.IsUndefined()) { + spec[YqlSysColumnPrefix] = sysColumns; + } + + InputTables_.emplace_back( + name, + richYPath, + pathInfo.Table->IsTemp, + !pathInfo.Table->RowSpec || pathInfo.Table->RowSpec->StrictSchema, + *pathInfo.Table, + spec, + group + ); + if (NYql::HasSetting(pathInfo.Table->Settings.Ref(), EYtSettingType::WithQB)) { + auto p = pathInfo.Table->Meta->Attrs.FindPtr(QB2Premapper); + YQL_ENSURE(p, "Expect " << QB2Premapper << " in meta attrs"); + InputTables_.back().QB2Premapper = NYT::NodeFromYsonString(*p); + } + const bool tableHasScheme = InputTables_.back().Spec.HasKey(YqlRowSpecAttribute); + if (!hasScheme) { + hasScheme = tableHasScheme; + } else { + YQL_ENSURE(*hasScheme == tableHasScheme, "Mixed Yamr/Yson input table formats"); + } + } + if (0 == group) { + Sampling = NYql::GetSampleParams(section.Settings().Ref()); + } else { + YQL_ENSURE(NYql::GetSampleParams(section.Settings().Ref()) == Sampling, "Different sampling settings"); + } + }; + + if (entry) { + with_lock(entry->Lock_) { + ui32 group = 0; + for (auto section: input.Cast<TYtSectionList>()) { + fillSection(section, group); + ++group; + } + } + } else { + ui32 group = 0; + for (auto section: input.Cast<TYtSectionList>()) { + fillSection(section, group); + ++group; + } + } + + if (hasScheme && !*hasScheme) { + YamrInput = true; + } + if (loggedTable > 10) { + YQL_CLOG(INFO, ProviderYt) << "...total input tables=" << loggedTable; + } + } +} + +void TExecContextBase::SetOutput(TYtOutSection output, const TYtSettings::TConstPtr& settings, const TString& opHash) { + const TString tmpFolder = GetTablesTmpFolder(*settings); + const auto nativeYtTypeCompatibility = settings->NativeYtTypeCompatibility.Get(Cluster_).GetOrElse(NTCF_LEGACY); + const bool rowSpecCompactForm = settings->UseYqlRowSpecCompactForm.Get().GetOrElse(DEFAULT_ROW_SPEC_COMPACT_FORM); + const bool optimizeForScan = settings->OptimizeFor.Get(Cluster_).GetOrElse(NYT::EOptimizeForAttr::OF_LOOKUP_ATTR) != NYT::EOptimizeForAttr::OF_LOOKUP_ATTR; + size_t loggedTable = 0; + TVector<TString> outTablePaths; + TVector<NYT::TNode> outTableSpecs; + for (auto table: output) { + TYtOutTableInfo tableInfo(table); + TString outTableName = tableInfo.Name; + if (outTableName.empty()) { + outTableName = TStringBuilder() << "tmp/" << GetGuidAsString(Session_->RandomProvider_->GenGuid()); + } + TString outTablePath = NYql::TransformPath(tmpFolder, outTableName, true, Session_->UserName_); + auto attrSpec = tableInfo.GetAttrSpecNode(nativeYtTypeCompatibility, rowSpecCompactForm); + OutTables_.emplace_back( + outTableName, + outTablePath, + tableInfo.GetCodecSpecNode(), + attrSpec, + ToYTSortColumns(tableInfo.RowSpec->GetForeignSort()), + optimizeForScan ? tableInfo.GetColumnGroups() : NYT::TNode{} + ); + outTablePaths.push_back(outTablePath); + outTableSpecs.push_back(std::move(attrSpec)); + if (loggedTable++ < 10) { + YQL_CLOG(INFO, ProviderYt) << "Output: " << Cluster_ << '.' << outTableName; + } + } + if (loggedTable > 10) { + YQL_CLOG(INFO, ProviderYt) << "...total output tables=" << loggedTable; + } + + SetCacheItem(outTablePaths, outTableSpecs, tmpFolder, settings, opHash); +} + +void TExecContextBase::SetSingleOutput(const TYtOutTableInfo& outTable, const TYtSettings::TConstPtr& settings) { + const TString tmpFolder = GetTablesTmpFolder(*settings); + TString outTableName = TStringBuilder() << "tmp/" << GetGuidAsString(Session_->RandomProvider_->GenGuid()); + TString outTablePath = NYql::TransformPath(tmpFolder, outTableName, true, Session_->UserName_); + + const auto nativeYtTypeCompatibility = settings->NativeYtTypeCompatibility.Get(Cluster_).GetOrElse(NTCF_LEGACY); + const bool rowSpecCompactForm = settings->UseYqlRowSpecCompactForm.Get().GetOrElse(DEFAULT_ROW_SPEC_COMPACT_FORM); + const bool optimizeForScan = settings->OptimizeFor.Get(Cluster_).GetOrElse(NYT::EOptimizeForAttr::OF_LOOKUP_ATTR) != NYT::EOptimizeForAttr::OF_LOOKUP_ATTR; + + OutTables_.emplace_back( + outTableName, + outTablePath, + outTable.GetCodecSpecNode(), + outTable.GetAttrSpecNode(nativeYtTypeCompatibility, rowSpecCompactForm), + ToYTSortColumns(outTable.RowSpec->GetForeignSort()), + optimizeForScan ? outTable.GetColumnGroups() : NYT::TNode{} + ); + + YQL_CLOG(INFO, ProviderYt) << "Output: " << Cluster_ << '.' << outTableName; +} + +void TExecContextBase::SetCacheItem(const TVector<TString>& outTablePaths, const TVector<NYT::TNode>& outTableSpecs, + const TString& tmpFolder, const TYtSettings::TConstPtr& settings, const TString& opHash) { + const bool testRun = Config_->GetLocalChainTest(); + if (!testRun && !Hidden) { + NYT::TNode mergeSpec = Session_->CreateSpecWithDesc(); + NYT::TNode tableAttrs = Session_->CreateTableAttrs(); + + auto entry = GetOrCreateEntry(settings); + FillSpec(mergeSpec, *this, settings, entry, 0., Nothing()); + auto chunkLimit = settings->QueryCacheChunkLimit.Get(Cluster_).GetOrElse(0); + + QueryCacheItem.Reset(new TYtQueryCacheItem(settings->QueryCacheMode.Get().GetOrElse(EQueryCacheMode::Disable), + entry, opHash, outTablePaths, outTableSpecs, Session_->UserName_, tmpFolder, mergeSpec, tableAttrs, chunkLimit, + settings->QueryCacheUseExpirationTimeout.Get().GetOrElse(false), + settings->_UseMultisetAttributes.Get().GetOrElse(DEFAULT_USE_MULTISET_ATTRS), LogCtx_)); + } +} + +TString TExecContextBase::GetInputSpec(bool ensureOldTypesOnly, ui64 nativeTypeCompatibilityFlags, bool intermediateInput) const { + return GetSpecImpl(InputTables_, 0, InputTables_.size(), {}, ensureOldTypesOnly, nativeTypeCompatibilityFlags, intermediateInput); +} + +TString TExecContextBase::GetOutSpec(bool ensureOldTypesOnly, ui64 nativeTypeCompatibilityFlags) const { + return GetSpecImpl(OutTables_, 0, OutTables_.size(), {}, ensureOldTypesOnly, nativeTypeCompatibilityFlags, false); +} + +TString TExecContextBase::GetOutSpec(size_t beginIdx, size_t endIdx, NYT::TNode initialOutSpec, bool ensureOldTypesOnly, ui64 nativeTypeCompatibilityFlags) const { + return GetSpecImpl(OutTables_, beginIdx, endIdx, initialOutSpec, ensureOldTypesOnly, nativeTypeCompatibilityFlags, false); +} + +template <class TTableType> +TString TExecContextBase::GetSpecImpl(const TVector<TTableType>& tables, size_t beginIdx, size_t endIdx, NYT::TNode initialSpec, bool ensureOldTypesOnly, ui64 nativeTypeCompatibilityFlags, bool intermediateInput) { + YQL_ENSURE(beginIdx <= endIdx); + YQL_ENSURE(endIdx <= tables.size()); + NYT::TNode specNode = initialSpec; + if (initialSpec.IsUndefined()) { + specNode = NYT::TNode::CreateMap(); + } + NYT::TNode& tablesNode = specNode[YqlIOSpecTables]; + + auto updateFlags = [nativeTypeCompatibilityFlags](NYT::TNode& spec) { + if (spec.HasKey(YqlRowSpecAttribute)) { + auto& rowSpec = spec[YqlRowSpecAttribute]; + ui64 nativeYtTypeFlags = 0; + if (rowSpec.HasKey(RowSpecAttrNativeYtTypeFlags)) { + nativeYtTypeFlags = rowSpec[RowSpecAttrNativeYtTypeFlags].AsUint64(); + } else { + if (rowSpec.HasKey(RowSpecAttrUseNativeYtTypes)) { + nativeYtTypeFlags = rowSpec[RowSpecAttrUseNativeYtTypes].AsBool() ? NTCF_LEGACY : NTCF_NONE; + } else if (rowSpec.HasKey(RowSpecAttrUseTypeV2)) { + nativeYtTypeFlags = rowSpec[RowSpecAttrUseTypeV2].AsBool() ? NTCF_LEGACY : NTCF_NONE; + } + } + rowSpec[RowSpecAttrNativeYtTypeFlags] = (nativeYtTypeFlags & nativeTypeCompatibilityFlags); + } + }; + + if (!intermediateInput && (endIdx - beginIdx) > 1) { + NYT::TNode& registryNode = specNode[YqlIOSpecRegistry]; + THashMap<TString, TString> uniqSpecs; + for (size_t i = beginIdx; i < endIdx; ++i) { + auto& table = tables[i]; + TString refName = TStringBuilder() << "$table" << uniqSpecs.size(); + auto spec = table.Spec; + if (ensureOldTypesOnly) { + EnsureSpecDoesntUseNativeYtTypes(spec, table.Name, std::is_same<TTableType, TInputInfo>::value); + } else { + updateFlags(spec); + } + auto res = uniqSpecs.emplace(NYT::NodeToCanonicalYsonString(spec), refName); + if (res.second) { + registryNode[refName] = std::move(spec); + } + else { + refName = res.first->second; + } + tablesNode.Add(refName); + } + } + else { + auto& table = tables[beginIdx]; + auto spec = table.Spec; + if (ensureOldTypesOnly) { + EnsureSpecDoesntUseNativeYtTypes(spec, table.Name, std::is_same<TTableType, TInputInfo>::value); + } else { + updateFlags(spec); + } + + tablesNode.Add(std::move(spec)); + } + return NYT::NodeToYsonString(specNode); +} + +TTransactionCache::TEntry::TPtr TExecContextBase::GetOrCreateEntry(const TYtSettings::TConstPtr& settings) const { + auto token = GetAuth(settings); + auto impersonationUser = GetImpersonationUser(settings); + if (!token && DisableAnonymousClusterAccess_) { + // do not use ythrow here for better error message + throw yexception() << "Accessing YT cluster " << Cluster_ << " without OAuth token is not allowed"; + } + + return Session_->TxCache_.GetOrCreateEntry(YtServer_, token, impersonationUser, [s = Session_]() { return s->CreateSpecWithDesc(); }, settings, Metrics); +} + +TExpressionResorceUsage TExecContextBase::ScanExtraResourceUsageImpl(const TExprNode& node, const TYtSettings::TConstPtr& config, bool withInput) { + auto extraUsage = ScanExtraResourceUsage(node, *config); + if (withInput && AnyOf(InputTables_, [](const auto& input) { return input.Erasure; })) { + if (auto codecCpu = config->ErasureCodecCpu.Get(Cluster_)) { + extraUsage.Cpu *= *codecCpu; + } + } + return extraUsage; +} + +TString TExecContextBase::GetAuth(const TYtSettings::TConstPtr& config) const { + auto auth = config->Auth.Get(); + if (!auth || auth->empty()) { + auth = Clusters_->GetAuth(Cluster_); + } + + return auth.GetOrElse(TString()); +} + +TMaybe<TString> TExecContextBase::GetImpersonationUser(const TYtSettings::TConstPtr& config) const { + return config->_ImpersonationUser.Get(); +} + +ui64 TExecContextBase::EstimateLLVMMem(size_t nodes, const TString& llvmOpt, const TYtSettings::TConstPtr& config) const { + ui64 memUsage = 0; + if (llvmOpt != "OFF") { + if (auto usage = config->LLVMMemSize.Get(Cluster_)) { + memUsage += *usage; + } + if (auto usage = config->LLVMPerNodeMemSize.Get(Cluster_)) { + memUsage += *usage * nodes; + } + } + + return memUsage; +} + + +} // NNative + +} // NYql diff --git a/yt/yql/providers/yt/gateway/native/yql_yt_exec_ctx.h b/yt/yql/providers/yt/gateway/native/yql_yt_exec_ctx.h new file mode 100644 index 0000000000..cec3838522 --- /dev/null +++ b/yt/yql/providers/yt/gateway/native/yql_yt_exec_ctx.h @@ -0,0 +1,304 @@ +#pragma once + +#include "yql_yt_native.h" +#include "yql_yt_session.h" + +#include <yt/yql/providers/yt/lib/config_clusters/config_clusters.h> +#include <yt/yql/providers/yt/gateway/lib/query_cache.h> +#include <yt/yql/providers/yt/gateway/lib/user_files.h> + +#include <yt/yql/providers/yt/common/yql_yt_settings.h> +#include <yt/yql/providers/yt/lib/url_mapper/yql_yt_url_mapper.h> +#include <yt/yql/providers/yt/lib/expr_traits/yql_expr_traits.h> +#include <yt/yql/providers/yt/expr_nodes/yql_yt_expr_nodes.h> +#include <yt/yql/providers/yt/provider/yql_yt_table.h> + +#include <yql/essentials/providers/common/mkql/yql_provider_mkql.h> + +#include <yql/essentials/core/yql_user_data.h> +#include <yql/essentials/core/expr_nodes/yql_expr_nodes.h> +#include <yql/essentials/core/file_storage/file_storage.h> +#include <yql/essentials/minikql/mkql_function_registry.h> +#include <yql/essentials/utils/log/context.h> + +#include <yt/cpp/mapreduce/interface/common.h> +#include <library/cpp/yson/node/node.h> +#include <library/cpp/threading/future/future.h> +#include <library/cpp/threading/future/async.h> + +#include <util/generic/maybe.h> +#include <util/generic/ptr.h> +#include <util/generic/string.h> +#include <util/generic/vector.h> +#include <util/generic/size_literals.h> +#include <util/generic/hash_set.h> + +#include <utility> + + +namespace NYql { + +namespace NNative { + +struct TInputInfo { + TInputInfo() = default; + TInputInfo(const TString& name, const NYT::TRichYPath& path, bool temp, bool strict, const TYtTableBaseInfo& info, const NYT::TNode& spec, ui32 group = 0) + : Name(name) + , Path(path) + , Temp(temp) + , Dynamic(info.Meta->IsDynamic) + , Strict(strict) + , Records(info.Stat->RecordsCount) + , DataSize(info.Stat->DataSize) + , Spec(spec) + , Group(group) + , Lookup(info.Meta->Attrs.Value("optimize_for", "scan") != "scan") + , Erasure(info.Meta->Attrs.Value("erasure_codec", "none") != "none") + { + } + + TString Name; + NYT::TRichYPath Path; + bool Temp = false; + bool Dynamic = false; + bool Strict = true; + ui64 Records = 0; + ui64 DataSize = 0; + NYT::TNode Spec; + NYT::TNode QB2Premapper; + ui32 Group = 0; + bool Lookup = false; + bool Erasure = false; +}; + +struct TOutputInfo { + TOutputInfo() = default; + TOutputInfo(const TString& name, const TString& path, const NYT::TNode& codecSpec, const NYT::TNode& attrSpec, + const NYT::TSortColumns& sortedBy, NYT::TNode columnGroups) + : Name(name) + , Path(path) + , Spec(codecSpec) + , AttrSpec(attrSpec) + , SortedBy(sortedBy) + , ColumnGroups(std::move(columnGroups)) + { + } + TString Name; + TString Path; + NYT::TNode Spec; + NYT::TNode AttrSpec; + NYT::TSortColumns SortedBy; + NYT::TNode ColumnGroups; +}; + +class TExecContextBase: public TThrRefBase { +protected: + TExecContextBase(const TYtNativeServices& services, + const TConfigClusters::TPtr& clusters, + const TIntrusivePtr<NCommon::TMkqlCommonCallableCompiler>& mkqlCompiler, + const TSession::TPtr& session, + const TString& cluster, + const TYtUrlMapper& urlMapper, + IMetricsRegistryPtr metrics); + +public: + TString GetInputSpec(bool ensureOldTypesOnly, ui64 nativeTypeCompatibilityFlags, bool intermediateInput) const; + TString GetOutSpec(bool ensureOldTypesOnly, ui64 nativeTypeCompatibilityFlags) const; + TString GetOutSpec(size_t beginIdx, size_t endIdx, NYT::TNode initialOutSpec, bool ensureOldTypesOnly, ui64 nativeTypeCompatibilityFlags) const; + + TTransactionCache::TEntry::TPtr GetEntry() const { + return Session_->TxCache_.GetEntry(YtServer_); + } + + TTransactionCache::TEntry::TPtr TryGetEntry() const { + return Session_->TxCache_.TryGetEntry(YtServer_); + } + + TTransactionCache::TEntry::TPtr GetOrCreateEntry(const TYtSettings::TConstPtr& settings) const; + +protected: + void MakeUserFiles(const TUserDataTable& userDataBlocks); + + void SetInput(NNodes::TExprBase input, bool forcePathColumns, const THashSet<TString>& extraSysColumns, const TYtSettings::TConstPtr& settings); + void SetOutput(NNodes::TYtOutSection output, const TYtSettings::TConstPtr& settings, const TString& opHash); + void SetSingleOutput(const TYtOutTableInfo& outTable, const TYtSettings::TConstPtr& settings); + void SetCacheItem(const TVector<TString>& outTablePaths, const TVector<NYT::TNode>& outTableSpecs, + const TString& tmpFolder, const TYtSettings::TConstPtr& settings, const TString& opHash); + + template <class TTableType> + static TString GetSpecImpl(const TVector<TTableType>& tables, size_t beginIdx, size_t endIdx, NYT::TNode initialOutSpec, bool ensureOldTypesOnly, ui64 nativeTypeCompatibilityFlags, bool intermediateInput); + + NThreading::TFuture<void> MakeOperationWaiter(const NYT::IOperationPtr& op, const TMaybe<ui32>& publicId) const { + if (const auto& opTracker = Session_->OpTracker_) { + return opTracker->MakeOperationWaiter(op, publicId, YtServer_, Cluster_, Session_->ProgressWriter_, Session_->StatWriter_); + } + return NThreading::MakeErrorFuture<void>(std::make_exception_ptr(yexception() << "Cannot run operations in session without operation tracker")); + } + + TString GetAuth(const TYtSettings::TConstPtr& config) const; + TMaybe<TString> GetImpersonationUser(const TYtSettings::TConstPtr& config) const; + + ui64 EstimateLLVMMem(size_t nodes, const TString& llvmOpt, const TYtSettings::TConstPtr& config) const; + + TExpressionResorceUsage ScanExtraResourceUsageImpl(const TExprNode& node, const TYtSettings::TConstPtr& config, bool withInput); + + NThreading::TFuture<NThreading::TAsyncSemaphore::TPtr> AcquireOperationLock() { + return Session_->OperationSemaphore->AcquireAsync(); + } + +public: + const NKikimr::NMiniKQL::IFunctionRegistry* FunctionRegistry_ = nullptr; + TFileStoragePtr FileStorage_; + TYtGatewayConfigPtr Config_; + TConfigClusters::TPtr Clusters_; + TIntrusivePtr<NCommon::TMkqlCommonCallableCompiler> MkqlCompiler_; + TSession::TPtr Session_; + TString Cluster_; + TString YtServer_; + TUserFiles::TPtr UserFiles_; + TVector<std::pair<TString, TString>> CodeSnippets_; + std::pair<TString, TString> LogCtx_; + TVector<TInputInfo> InputTables_; + bool YamrInput = false; + TMaybe<TSampleParams> Sampling; + TVector<TOutputInfo> OutTables_; + THolder<TYtQueryCacheItem> QueryCacheItem; + const TYtUrlMapper& UrlMapper_; + bool DisableAnonymousClusterAccess_; + bool Hidden = false; + IMetricsRegistryPtr Metrics; +}; + + +template <class T> +class TExecContext: public TExecContextBase { +public: + using TPtr = ::TIntrusivePtr<TExecContext>; + using TOptions = T; + + TExecContext(const TYtNativeServices& services, + const TConfigClusters::TPtr& clusters, + const TIntrusivePtr<NCommon::TMkqlCommonCallableCompiler>& mkqlCompiler, + TOptions&& options, + const TSession::TPtr& session, + const TString& cluster, + const TYtUrlMapper& urlMapper, + IMetricsRegistryPtr metrics) + : TExecContextBase(services, clusters, mkqlCompiler, session, cluster, urlMapper, std::move(metrics)) + , Options_(std::move(options)) + { + } + + void MakeUserFiles() { + TExecContextBase::MakeUserFiles(Options_.UserDataBlocks()); + } + + void SetInput(NNodes::TExprBase input, bool forcePathColumns, const THashSet<TString>& extraSysColumns) { + TExecContextBase::SetInput(input, forcePathColumns, extraSysColumns, Options_.Config()); + } + + void SetOutput(NNodes::TYtOutSection output) { + TExecContextBase::SetOutput(output, Options_.Config(), Options_.OperationHash()); + } + + void SetSingleOutput(const TYtOutTableInfo& outTable) { + TExecContextBase::SetSingleOutput(outTable, Options_.Config()); + } + + void SetCacheItem(const TVector<TString>& outTablePaths, const TVector<NYT::TNode>& outTableSpecs, const TString& tmpFolder) { + TExecContextBase::SetCacheItem(outTablePaths, outTableSpecs, tmpFolder, Options_.Config(), Options_.OperationHash()); + } + + TExpressionResorceUsage ScanExtraResourceUsage(const TExprNode& node, bool withInput) { + return TExecContextBase::ScanExtraResourceUsageImpl(node, Options_.Config(), withInput); + } + + TTransactionCache::TEntry::TPtr GetOrCreateEntry() const { + return TExecContextBase::GetOrCreateEntry(Options_.Config()); + } + + TString GetAuth() const { + return TExecContextBase::GetAuth(Options_.Config()); + } + + ui64 EstimateLLVMMem(size_t nodes) const { + return TExecContextBase::EstimateLLVMMem(nodes, Options_.OptLLVM(), Options_.Config()); + } + + void SetNodeExecProgress(const TString& stage) { + auto publicId = Options_.PublicId(); + if (!publicId) { + return; + } + auto progress = TOperationProgress(TString(YtProviderName), *publicId, + TOperationProgress::EState::InProgress, stage); + Session_->ProgressWriter_(progress); + } + + [[nodiscard]] + NThreading::TFuture<bool> LookupQueryCacheAsync() { + if (QueryCacheItem) { + SetNodeExecProgress("Awaiting cache"); + return QueryCacheItem->LookupAsync(Session_->Queue_); + } + return NThreading::MakeFuture(false); + } + + void StoreQueryCache() { + if (QueryCacheItem) { + SetNodeExecProgress("Storing to cache"); + QueryCacheItem->Store(); + } + } + + template <bool WithState = true> + [[nodiscard]] + NThreading::TFuture<void> RunOperation(std::function<NYT::IOperationPtr()>&& opFactory) { + if constexpr (WithState) { + SetNodeExecProgress("Waiting for concurrency limit"); + } + Session_->EnsureInitializedSemaphore(Options_.Config()); + return TExecContextBase::AcquireOperationLock().Apply([opFactory = std::move(opFactory), self = TIntrusivePtr(this)](const auto& f) { + YQL_LOG_CTX_ROOT_SESSION_SCOPE(self->LogCtx_); + auto lock = f.GetValue()->MakeAutoRelease(); + auto op = opFactory(); + op->GetPreparedFuture().Subscribe([op, self](const auto& f) { + YQL_LOG_CTX_ROOT_SESSION_SCOPE(self->LogCtx_); + if (!f.HasException()) { + if constexpr (WithState) { + self->SetNodeExecProgress("Starting operation"); + } + try { + op->Start(); + } catch (...) { + // Promise will be initialized with exception inside of TOperation::Start() + } + } + }); + // opFactory factory may contain locked resources. Explicitly wait preparation before destroying it + op->GetPreparedFuture().GetValueSync(); + + NThreading::TFuture<void> res; + if constexpr (WithState) { + res = self->TExecContextBase::MakeOperationWaiter(op, self->Options_.PublicId()); + } else { + res = self->TExecContextBase::MakeOperationWaiter(op, Nothing()); + } + return res.Apply([queue = self->Session_->Queue_, unlock = lock.DeferRelease()](const auto& f) { + if (f.HasException()) { + unlock(f); + f.TryRethrow(); + } + return queue->Async([unlock = std::move(unlock), f]() { + return unlock(f); + }); + }); + }); + } + + TOptions Options_; +}; + +} // NNative + +} // NYql diff --git a/yt/yql/providers/yt/gateway/native/yql_yt_lambda_builder.cpp b/yt/yql/providers/yt/gateway/native/yql_yt_lambda_builder.cpp new file mode 100644 index 0000000000..6feb3542dc --- /dev/null +++ b/yt/yql/providers/yt/gateway/native/yql_yt_lambda_builder.cpp @@ -0,0 +1,168 @@ +#include "yql_yt_lambda_builder.h" + +#include "yql_yt_native.h" +#include "yql_yt_session.h" + +#include <yql/essentials/core/yql_opt_utils.h> +#include <yt/yql/providers/yt/expr_nodes/yql_yt_expr_nodes.h> +#include <yt/yql/providers/yt/comp_nodes/yql_mkql_output.h> +#include <yt/yql/providers/yt/comp_nodes/yql_mkql_table_content.h> + +#include <yql/essentials/providers/common/mkql/yql_provider_mkql.h> +#include <yql/essentials/providers/common/mkql/yql_type_mkql.h> +#include <yql/essentials/providers/common/comp_nodes/yql_factory.h> +#include <yql/essentials/parser/pg_wrapper/interface/comp_factory.h> + +#include <yql/essentials/public/udf/udf_value.h> +#include <yql/essentials/minikql/mkql_alloc.h> +#include <yql/essentials/minikql/mkql_function_registry.h> +#include <yql/essentials/minikql/mkql_node.h> +#include <yql/essentials/minikql/mkql_node_cast.h> +#include <yql/essentials/minikql/mkql_node_builder.h> +#include <yql/essentials/minikql/mkql_node_serialization.h> +#include <yql/essentials/minikql/mkql_program_builder.h> +#include <yql/essentials/minikql/mkql_string_util.h> +#include <yql/essentials/minikql/comp_nodes/mkql_factories.h> +#include <yql/essentials/minikql/computation/mkql_computation_node.h> +#include <yql/essentials/minikql/computation/mkql_computation_node_holders.h> +#include <yql/essentials/minikql/computation/mkql_computation_node_holders_codegen.h> + +#include <yql/essentials/utils/yql_panic.h> + +#include <util/generic/maybe.h> +#include <util/generic/ylimits.h> + +namespace NYql { + +namespace NNative { + +using namespace NNodes; +using namespace NCommon; +using namespace NKikimr; +using namespace NKikimr::NMiniKQL; + +NKikimr::NMiniKQL::TComputationNodeFactory GetGatewayNodeFactory(TCodecContext* codecCtx, TMkqlWriterImpl* writer, TUserFiles::TPtr files, TStringBuf filePrefix) { + TMaybe<ui32> exprContextObject; + return [exprContextObject, codecCtx, writer, files, filePrefix](NMiniKQL::TCallable& callable, const TComputationNodeFactoryContext& ctx) mutable -> IComputationNode* { + if (callable.GetType()->GetName() == TYtTablePath::CallableName() + || callable.GetType()->GetName() == TYtTableIndex::CallableName() + || callable.GetType()->GetName() == TYtTableRecord::CallableName() + || callable.GetType()->GetName() == TYtIsKeySwitch::CallableName() + || callable.GetType()->GetName() == TYtRowNumber::CallableName() + ) { + return ctx.NodeFactory.CreateImmutableNode(NUdf::TUnboxedValuePod::Zero()); + } + + if (files) { + if (callable.GetType()->GetName() == "FilePathJob" || callable.GetType()->GetName() == "FileContentJob") { + const TString fullFileName(AS_VALUE(TDataLiteral, callable.GetInput(0))->AsValue().AsStringRef()); + const auto fileInfo = files->GetFile(fullFileName); + YQL_ENSURE(fileInfo, "Unknown file path " << fullFileName); + const auto path = fileInfo->Path->GetPath(); + const auto content = callable.GetType()->GetName() == "FileContentJob" ? TFileInput(path).ReadAll() : path.GetPath(); + return ctx.NodeFactory.CreateImmutableNode(MakeString(content)); + } + } + + if (callable.GetType()->GetName() == "YtOutput") { + YQL_ENSURE(writer); + return WrapYtOutput(callable, ctx, *writer); + } + + if (callable.GetType()->GetName() == "YtTableContentJob") { + YQL_ENSURE(codecCtx); + return WrapYtTableContent(*codecCtx, ctx.Mutables, callable, "OFF" /* no LLVM for local exec */, filePrefix); + } + + if (!exprContextObject) { + exprContextObject = ctx.Mutables.CurValueIndex++; + } + + auto yql = GetYqlFactory(*exprContextObject)(callable, ctx); + if (yql) { + return yql; + } + + auto pg = GetPgFactory()(callable, ctx); + if (pg) { + return pg; + } + + return GetBuiltinFactory()(callable, ctx); + }; +} + + +TNativeYtLambdaBuilder::TNativeYtLambdaBuilder(TScopedAlloc& alloc, const IFunctionRegistry* functionRegistry, const TSession& session, + const NKikimr::NUdf::ISecureParamsProvider* secureParamsProvider) + : TLambdaBuilder(functionRegistry, alloc, nullptr, + session.RandomProvider_, session.TimeProvider_, nullptr, nullptr, secureParamsProvider) +{ +} + +TNativeYtLambdaBuilder::TNativeYtLambdaBuilder(TScopedAlloc& alloc, const TYtNativeServices& services, const TSession& session) + : TNativeYtLambdaBuilder(alloc, services.FunctionRegistry, session) +{ +} + +TString TNativeYtLambdaBuilder::BuildLambdaWithIO(const IMkqlCallableCompiler& compiler, TCoLambda lambda, TExprContext& exprCtx) { + TProgramBuilder pgmBuilder(GetTypeEnvironment(), *FunctionRegistry); + TArgumentsMap arguments(1U); + if (lambda.Args().Size() > 0) { + const auto arg = lambda.Args().Arg(0); + const auto argType = arg.Ref().GetTypeAnn(); + auto inputItemType = NCommon::BuildType(arg.Ref(), *GetSeqItemType(argType), pgmBuilder); + switch (bool isStream = true; argType->GetKind()) { + case ETypeAnnotationKind::Flow: + if (ETypeAnnotationKind::Multi == argType->Cast<TFlowExprType>()->GetItemType()->GetKind()) { + arguments.emplace(arg.Raw(), TRuntimeNode(TCallableBuilder(GetTypeEnvironment(), "YtInput", pgmBuilder.NewFlowType(inputItemType)).Build(), false)); + break; + } + isStream = false; + [[fallthrough]]; // AUTOGENERATED_FALLTHROUGH_FIXME + case ETypeAnnotationKind::Stream: { + auto inputStream = pgmBuilder.SourceOf(isStream ? + pgmBuilder.NewStreamType(pgmBuilder.GetTypeEnvironment().GetTypeOfVoidLazy()) : pgmBuilder.NewFlowType(pgmBuilder.GetTypeEnvironment().GetTypeOfVoidLazy())); + + inputItemType = pgmBuilder.NewOptionalType(inputItemType); + inputStream = pgmBuilder.Map(inputStream, [&] (TRuntimeNode item) { + TCallableBuilder inputCall(GetTypeEnvironment(), "YtInput", inputItemType); + inputCall.Add(item); + return TRuntimeNode(inputCall.Build(), false); + }); + inputStream = pgmBuilder.TakeWhile(inputStream, [&] (TRuntimeNode item) { + return pgmBuilder.Exists(item); + }); + + inputStream = pgmBuilder.FlatMap(inputStream, [&] (TRuntimeNode item) { + return item; + }); + + arguments[arg.Raw()] = inputStream; + break; + } + default: + YQL_ENSURE(false, "Unsupported lambda argument type: " << arg.Ref().GetTypeAnn()->GetKind()); + } + } + TMkqlBuildContext ctx(compiler, pgmBuilder, exprCtx, lambda.Ref().UniqueId(), std::move(arguments)); + TRuntimeNode outStream = MkqlBuildExpr(lambda.Body().Ref(), ctx); + if (outStream.GetStaticType()->IsFlow()) { + TCallableBuilder outputCall(GetTypeEnvironment(), "YtOutput", pgmBuilder.NewFlowType(GetTypeEnvironment().GetTypeOfVoidLazy())); + outputCall.Add(outStream); + outStream = TRuntimeNode(outputCall.Build(), false); + } else { + outStream = pgmBuilder.Map(outStream, [&] (TRuntimeNode item) { + TCallableBuilder outputCall(GetTypeEnvironment(), "YtOutput", GetTypeEnvironment().GetTypeOfVoidLazy()); + outputCall.Add(item); + return TRuntimeNode(outputCall.Build(), false); + }); + } + outStream = pgmBuilder.Discard(outStream); + + return SerializeRuntimeNode(outStream, GetTypeEnvironment()); +} + +} // NNative + +} // NYql diff --git a/yt/yql/providers/yt/gateway/native/yql_yt_lambda_builder.h b/yt/yql/providers/yt/gateway/native/yql_yt_lambda_builder.h new file mode 100644 index 0000000000..d147ad4c65 --- /dev/null +++ b/yt/yql/providers/yt/gateway/native/yql_yt_lambda_builder.h @@ -0,0 +1,46 @@ +#pragma once + +#include <yt/yql/providers/yt/lib/lambda_builder/lambda_builder.h> +#include <yt/yql/providers/yt/codec/yt_codec_io.h> +#include <yql/essentials/core/expr_nodes/yql_expr_nodes.h> +#include <yql/essentials/minikql/computation/mkql_computation_node.h> +#include <yt/yql/providers/yt/gateway/lib/user_files.h> + +namespace NKikimr::NMiniKQL { + +class TScopedAlloc; +class IFunctionRegistry; + +} + +namespace NYql { + +namespace NUdf { +class ISecureParamsProvider; +} + +namespace NCommon { +class IMkqlCallableCompiler; +} + +struct TYtNativeServices; + +namespace NNative { + +struct TSession; + +struct TNativeYtLambdaBuilder: public TLambdaBuilder { + TNativeYtLambdaBuilder(NKikimr::NMiniKQL::TScopedAlloc& alloc, const NKikimr::NMiniKQL::IFunctionRegistry* functionRegistry, + const TSession& session, const NKikimr::NUdf::ISecureParamsProvider* secureParamsProvider = nullptr); + + TNativeYtLambdaBuilder(NKikimr::NMiniKQL::TScopedAlloc& alloc, const TYtNativeServices& services, const TSession& session); + + TString BuildLambdaWithIO(const NCommon::IMkqlCallableCompiler& compiler, NNodes::TCoLambda lambda, TExprContext& exprCtx); +}; + +NKikimr::NMiniKQL::TComputationNodeFactory GetGatewayNodeFactory(NYql::NCommon::TCodecContext* codecCtx, + TMkqlWriterImpl* writer, TUserFiles::TPtr files, TStringBuf filePrefix); + +} // NNative + +} // NYql diff --git a/yt/yql/providers/yt/gateway/native/yql_yt_native.cpp b/yt/yql/providers/yt/gateway/native/yql_yt_native.cpp new file mode 100644 index 0000000000..fb33e571aa --- /dev/null +++ b/yt/yql/providers/yt/gateway/native/yql_yt_native.cpp @@ -0,0 +1,5462 @@ +#include "yql_yt_native.h" + +#include "yql_yt_lambda_builder.h" +#include "yql_yt_qb2.h" +#include "yql_yt_session.h" +#include "yql_yt_spec.h" +#include "yql_yt_transform.h" +#include "yql_yt_native_folders.h" + +#include <yt/yql/providers/yt/gateway/lib/yt_helpers.h> +#include <yt/yql/providers/yt/lib/config_clusters/config_clusters.h> +#include <yt/yql/providers/yt/lib/log/yt_logger.h> + +#include <yt/yql/providers/yt/lib/mkql_helpers/mkql_helpers.h> +#include <yt/yql/providers/yt/lib/res_pull/res_or_pull.h> +#include <yt/yql/providers/yt/lib/res_pull/table_limiter.h> +#include <yt/yql/providers/yt/lib/yson_helpers/yson_helpers.h> +#include <yt/yql/providers/yt/lib/infer_schema/infer_schema.h> +#include <yt/yql/providers/yt/lib/infer_schema/infer_schema_rpc.h> +#include <yt/yql/providers/yt/lib/schema/schema.h> +#include <yt/yql/providers/yt/lib/skiff/yql_skiff_schema.h> +#include <yt/yql/providers/yt/common/yql_names.h> +#include <yt/yql/providers/yt/common/yql_configuration.h> +#include <yt/yql/providers/yt/job/yql_job_base.h> +#include <yt/yql/providers/yt/job/yql_job_calc.h> +#include <yt/yql/providers/yt/job/yql_job_registry.h> +#include <yt/yql/providers/yt/provider/yql_yt_helpers.h> +#include <yt/yql/providers/yt/provider/yql_yt_mkql_compiler.h> + +#include <yql/essentials/providers/common/mkql/yql_provider_mkql.h> +#include <yql/essentials/providers/common/provider/yql_provider.h> +#include <yql/essentials/providers/common/codec/yql_codec_type_flags.h> +#include <yql/essentials/providers/common/schema/expr/yql_expr_schema.h> +#include <yql/essentials/providers/common/proto/gateways_config.pb.h> +#include <yql/essentials/providers/result/expr_nodes/yql_res_expr_nodes.h> + +#include <yql/essentials/ast/yql_expr.h> +#include <yql/essentials/core/issue/yql_issue.h> +#include <yql/essentials/core/yql_type_helpers.h> +#include <yql/essentials/core/yql_graph_transformer.h> +#include <yql/essentials/utils/log/log.h> +#include <yql/essentials/minikql/mkql_node.h> +#include <yql/essentials/minikql/mkql_node_cast.h> +#include <yql/essentials/minikql/mkql_node_serialization.h> +#include <yql/essentials/minikql/computation/mkql_computation_node.h> + +#include <yt/cpp/mapreduce/interface/config.h> +#include <yt/cpp/mapreduce/common/helpers.h> +#include <yt/cpp/mapreduce/interface/error_codes.h> + +#include <library/cpp/yson/node/node_io.h> +#include <library/cpp/threading/future/async.h> +#include <library/cpp/threading/future/future.h> +#include <library/cpp/containers/sorted_vector/sorted_vector.h> +#include <library/cpp/streams/brotli/brotli.h> + +#include <util/folder/tempdir.h> +#include <util/generic/ptr.h> +#include <util/generic/yexception.h> +#include <util/generic/xrange.h> +#include <util/generic/size_literals.h> +#include <util/generic/scope.h> +#include <util/stream/null.h> +#include <util/stream/str.h> +#include <util/stream/input.h> +#include <util/stream/file.h> +#include <util/system/execpath.h> +#include <util/system/guard.h> +#include <util/system/shellcommand.h> +#include <util/ysaveload.h> + +#include <algorithm> +#include <iterator> +#include <variant> +#include <exception> + +namespace NYql { + +using namespace NYT; +using namespace NCommon; +using namespace NKikimr; +using namespace NKikimr::NMiniKQL; +using namespace NNodes; +using namespace NThreading; + +namespace NNative { + +namespace { + THashMap<TString, std::pair<NYT::TNode, TString>> TestTables; +} + +/////////////////////////////////////////////////////////////////////////////////////////////////////// + +void DumpLocalTable(const TString& tableContent, const TString& path) { + if (!path) { + return; + } + + TFileOutput out(path); + out.Write(tableContent.data(), tableContent.size()); + out.Flush(); +} + +/////////////////////////////////////////////////////////////////////////////////////////////////////// + +template<typename TOptions> +struct TMetaPerServerRequest { + TVector<size_t> TableIndicies; + typename TExecContext<TOptions>::TPtr ExecContext; +}; + +/////////////////////////////////////////////////////////////////////////////////////////////////////// + +namespace { + +TString ToColumn(const NYT::TSortColumn& item) { + return TStringBuilder() << item.Name() << '(' << (item.SortOrder() == NYT::ESortOrder::SO_ASCENDING) << ')'; +} + +TString ToColumn(const std::pair<TString, int>& item) { + return TStringBuilder() << item.first << '(' << bool(item.second) << ')'; +} + +template <typename T> +TString ToColumnList(const TVector<T>& list) { + TStringBuilder builder; + builder << '['; + for (auto& col: list) { + builder << ToColumn(col) << ','; + } + if (!list.empty()) { + builder.pop_back(); + } + return (builder << ']'); +} + +const NYT::TJobBinaryConfig GetJobBinary(const NYT::TRawMapOperationSpec& spec) { + return spec.MapperSpec_.GetJobBinary(); +} + +const NYT::TJobBinaryConfig GetJobBinary(const NYT::TRawMapReduceOperationSpec& spec) { + return spec.MapperSpec_.GetJobBinary(); +} + +const NYT::TJobBinaryConfig GetJobBinary(const NYT::TRawReduceOperationSpec& spec) { + return spec.ReducerSpec_.GetJobBinary(); +} + +const TVector<std::tuple<TLocalFilePath, TAddLocalFileOptions>> GetLocalFiles(const NYT::TRawMapOperationSpec& spec) { + return spec.MapperSpec_.GetLocalFiles(); +} + +const TVector<std::tuple<TLocalFilePath, TAddLocalFileOptions>> GetLocalFiles(const NYT::TRawMapReduceOperationSpec& spec) { + return spec.MapperSpec_.GetLocalFiles(); +} + +const TVector<std::tuple<TLocalFilePath, TAddLocalFileOptions>> GetLocalFiles(const NYT::TRawReduceOperationSpec& spec) { + return spec.ReducerSpec_.GetLocalFiles(); +} + +const TVector<TRichYPath> GetRemoteFiles(const NYT::TRawMapOperationSpec& spec) { + return spec.MapperSpec_.Files_; +} + +const TVector<TRichYPath> GetRemoteFiles(const NYT::TRawMapReduceOperationSpec& spec) { + return spec.MapperSpec_.Files_; +} + +const TVector<TRichYPath> GetRemoteFiles(const NYT::TRawReduceOperationSpec& spec) { + return spec.ReducerSpec_.Files_; +} + +template <typename TType> +inline TType OptionFromString(const TString& value) { + if constexpr (std::is_same_v<TString, TType>) { + return value; + } else if constexpr (std::is_same_v<NYT::TNode, TType>) { + return NYT::NodeFromYsonString(value); + } else { + return FromString<TType>(value); + } +} + +template <typename TType> +inline TType OptionFromNode(const NYT::TNode& value) { + if constexpr (std::is_same_v<NYT::TNode, TType>) { + return value; + } else if constexpr (std::is_integral_v<TType>) { + return static_cast<TType>(value.AsInt64()); + } else { + return FromString<TType>(value.AsString()); + } +} + +void PopulatePathStatResult(IYtGateway::TPathStatResult& out, int index, NYT::TTableColumnarStatistics& extendedStat) { + for (const auto& entry : extendedStat.ColumnDataWeight) { + out.DataSize[index] += entry.second; + } + out.Extended[index] = IYtGateway::TPathStatResult::TExtendedResult{ + .DataWeight = extendedStat.ColumnDataWeight, + .EstimatedUniqueCounts = extendedStat.ColumnEstimatedUniqueCounts + }; +} + +TString DebugPath(NYT::TRichYPath path) { + constexpr int maxDebugColumns = 20; + if (!path.Columns_ || std::ssize(path.Columns_->Parts_) <= maxDebugColumns) { + return NYT::NodeToCanonicalYsonString(NYT::PathToNode(path), NYT::NYson::EYsonFormat::Text); + } + int numColumns = std::ssize(path.Columns_->Parts_); + path.Columns_->Parts_.erase(path.Columns_->Parts_.begin() + maxDebugColumns, path.Columns_->Parts_.end()); + path.Columns_->Parts_.push_back("..."); + return NYT::NodeToCanonicalYsonString(NYT::PathToNode(path), NYT::NYson::EYsonFormat::Text) + " (" + std::to_string(numColumns) + " columns)"; +} + +} // unnamed + +/////////////////////////////////////////////////////////////////////////////////////////////////////// + +class TYtNativeGateway : public IYtGateway { +public: + TYtNativeGateway(const TYtNativeServices& services) + : Services_(services) + , Clusters_(MakeIntrusive<TConfigClusters>(*Services_.Config)) + , MkqlCompiler_(MakeIntrusive<NCommon::TMkqlCommonCallableCompiler>()) + , UrlMapper_(*Services_.Config) + { + RegisterYtMkqlCompilers(*MkqlCompiler_); + SetYtLoggerGlobalBackend( + Services_.Config->HasYtLogLevel() ? Services_.Config->GetYtLogLevel() : -1, + Services_.Config->GetYtDebugLogSize(), + Services_.Config->GetYtDebugLogFile(), + Services_.Config->GetYtDebugLogAlwaysWrite() + ); + } + + ~TYtNativeGateway() { + } + + void OpenSession(TOpenSessionOptions&& options) final { + TString sessionId = options.SessionId(); + YQL_LOG_CTX_SCOPE(TStringBuf("Gateway"), __FUNCTION__); + TSession::TPtr session = MakeIntrusive<TSession>(std::move(options), Services_.Config->GetGatewayThreads()); + with_lock(Mutex_) { + if (!Sessions_.insert({sessionId, session}).second) { + YQL_LOG_CTX_THROW yexception() << "Session already exists: " << sessionId; + } + } + } + + TFuture<void> CloseSession(TCloseSessionOptions&& options) final { + YQL_LOG_CTX_SCOPE(TStringBuf("Gateway"), __FUNCTION__); + + TSession::TPtr session; + with_lock(Mutex_) { + auto it = Sessions_.find(options.SessionId()); + if (it != Sessions_.end()) { + session = it->second; + Sessions_.erase(it); + } + } + + // Do final destruction outside of mutex, because it may do some transaction aborts on YT clusters + if (session) { + try { + session->Close(); + } catch (...) { + YQL_CLOG(ERROR, ProviderYt) << CurrentExceptionMessage(); + return MakeErrorFuture<void>(std::current_exception()); + } + } + + return MakeFuture(); + } + + TFuture<void> CleanupSession(TCleanupSessionOptions&& options) final { + YQL_LOG_CTX_SCOPE(TStringBuf("Gateway"), __FUNCTION__); + + if (auto session = GetSession(options.SessionId(), false)) { + if (session->OperationSemaphore) { + session->OperationSemaphore->Cancel(); + session->OperationSemaphore.Drop(); + } + auto logCtx = NYql::NLog::CurrentLogContextPath(); + return session->Queue_->Async([session, logCtx] { + YQL_LOG_CTX_ROOT_SESSION_SCOPE(logCtx); + try { + session->TxCache_.AbortAll(); + } catch (...) { + YQL_CLOG(ERROR, ProviderYt) << CurrentExceptionMessage(); + return MakeErrorFuture<void>(std::current_exception()); + } + + return MakeFuture(); + }); + } + + return MakeFuture(); + } + + TFuture<TFinalizeResult> Finalize(TFinalizeOptions&& options) final { + YQL_LOG_CTX_SCOPE(TStringBuf("Gateway"), __FUNCTION__); + try { + TSession::TPtr session = GetSession(options.SessionId()); + auto logCtx = NYql::NLog::CurrentLogContextPath(); + return session->Queue_->Async([session, logCtx, abort=options.Abort(), detachSnapshotTxs=options.DetachSnapshotTxs()] () { + YQL_LOG_CTX_ROOT_SESSION_SCOPE(logCtx); + return ExecFinalize(session, abort, detachSnapshotTxs); + }); + } catch (...) { + return MakeFuture(ResultFromCurrentException<TFinalizeResult>()); + } + } + + TFuture<TCanonizePathsResult> CanonizePaths(TCanonizePathsOptions&& options) final { + if (Services_.Config->GetLocalChainTest()) { + TCanonizePathsResult res; + std::transform( + options.Paths().begin(), options.Paths().end(), + std::back_inserter(res.Data), + [] (const TCanonizeReq& req) { + return CanonizedPath(req.Path()); + }); + res.SetSuccess(); + return MakeFuture(res); + } + + YQL_LOG_CTX_SCOPE(TStringBuf("Gateway"), __FUNCTION__); + try { + TSession::TPtr session = GetSession(options.SessionId()); + + TVector<TCanonizeReq> paths(std::move(options.Paths())); + if (YQL_CLOG_ACTIVE(INFO, ProviderYt)) { + for (size_t i: xrange(Min<size_t>(paths.size(), 10))) { + YQL_CLOG(INFO, ProviderYt) << paths[i].Cluster() << '.' << paths[i].Path(); + } + if (paths.size() > 10) { + YQL_CLOG(INFO, ProviderYt) << "...total tables=" << paths.size(); + } + } + + THashMap<TString, TMetaPerServerRequest<TCanonizePathsOptions>> reqPerServer; + for (size_t i: xrange(paths.size())) { + TCanonizeReq& path = paths[i]; + auto cluster = path.Cluster(); + auto ytServer = Clusters_->GetServer(cluster); + auto& r = reqPerServer[ytServer]; + if (r.TableIndicies.empty()) { + r.ExecContext = MakeExecCtx(TCanonizePathsOptions(options), session, cluster, nullptr, nullptr); + } + + // Use absolute path to prevent adding YT_PREFFIX + if (!path.Path().StartsWith("//") && !path.Path().StartsWith("<")) { + path.Path(TString("//").append(path.Path())); + } + + r.TableIndicies.push_back(i); + } + + auto logCtx = NYql::NLog::CurrentLogContextPath(); + return session->Queue_->Async([session, paths = std::move(paths), reqPerServer = std::move(reqPerServer), logCtx] () { + YQL_LOG_CTX_ROOT_SESSION_SCOPE(logCtx); + return ExecCanonizePaths(paths, reqPerServer); + }); + } catch (...) { + return MakeFuture(ResultFromCurrentException<TCanonizePathsResult>()); + } + } + + TFuture<TTableInfoResult> GetTableInfo(TGetTableInfoOptions&& options) final { + if (Services_.Config->GetLocalChainTest()) { + TTableInfoResult result; + for (const auto& t : options.Tables()) { + const auto it = TestTables.find(t.Table()); + result.Data.emplace_back(); + auto& table = result.Data.back(); + table.Meta = MakeIntrusive<TYtTableMetaInfo>(); + if (table.Meta->DoesExist = TestTables.cend() != it) { + table.Meta->Attrs.emplace(YqlRowSpecAttribute, NYT::NodeToYsonString(it->second.first)); + table.Stat = MakeIntrusive<TYtTableStatInfo>(); + table.Stat->Id = "stub"; + // Prevent empty table optimizations + table.Stat->RecordsCount = 1; + table.Stat->DataSize = 1; + table.Stat->ChunkCount = 1; + } + + table.WriteLock = HasModifyIntents(t.Intents()); + } + result.SetSuccess(); + return MakeFuture<TTableInfoResult>(std::move(result)); + } + YQL_LOG_CTX_SCOPE(TStringBuf("Gateway"), __FUNCTION__); + try { + TSession::TPtr session = GetSession(options.SessionId()); + session->EnsureInitializedSemaphore(options.Config()); + + TString tmpFolder = GetTablesTmpFolder(*options.Config()); + + YQL_CLOG(INFO, ProviderYt) << "ReadOnly=" << options.ReadOnly() << ", Epoch=" << options.Epoch(); + TVector<TTableReq> tables(std::move(options.Tables())); + if (YQL_CLOG_ACTIVE(INFO, ProviderYt)) { + for (size_t i: xrange(Min<size_t>(tables.size(), 10))) { + YQL_CLOG(INFO, ProviderYt) << tables[i].Cluster() << '.' << tables[i].Table() + << ", LockOnly=" << tables[i].LockOnly() << ", Intents=" << tables[i].Intents(); + } + if (tables.size() > 10) { + YQL_CLOG(INFO, ProviderYt) << "...total tables=" << tables.size(); + } + } + + THashMap<TString, TMetaPerServerRequest<TGetTableInfoOptions>> reqPerServer; + for (size_t i: xrange(tables.size())) { + TTableReq& table = tables[i]; + auto cluster = table.Cluster(); + auto ytServer = Clusters_->GetServer(cluster); + auto& r = reqPerServer[ytServer]; + if (r.TableIndicies.empty()) { + r.ExecContext = MakeExecCtx(TGetTableInfoOptions(options), session, cluster, nullptr, nullptr); + } + table.Table(NYql::TransformPath(tmpFolder, table.Table(), table.Anonymous(), session->UserName_)); + r.TableIndicies.push_back(i); + } + + auto logCtx = NYql::NLog::CurrentLogContextPath(); + bool readOnly = options.ReadOnly(); + ui32 epoch = options.Epoch(); + return session->Queue_->Async([session, tables = std::move(tables), reqPerServer = std::move(reqPerServer), readOnly, epoch, logCtx] () { + YQL_LOG_CTX_ROOT_SESSION_SCOPE(logCtx); + return ExecGetTableInfo(tables, reqPerServer, readOnly, epoch); + }); + } catch (...) { + return MakeFuture(ResultFromCurrentException<TTableInfoResult>()); + } + } + + TFuture<TTableRangeResult> GetTableRange(TTableRangeOptions&& options) final { + YQL_LOG_CTX_SCOPE(TStringBuf("Gateway"), __FUNCTION__); + + YQL_CLOG(INFO, ProviderYt) << "Server=" << options.Cluster() << ", Prefix=" << options.Prefix() + << ", Suffix=" << options.Suffix() << ", Filter=" << (options.Filter() ? "(present)" : "null"); + + auto pos = options.Pos(); + + try { + TSession::TPtr session = GetSession(options.SessionId()); + session->EnsureInitializedSemaphore(options.Config()); + + TString tmpFolder = GetTablesTmpFolder(*options.Config()); + TString tmpTablePath = NYql::TransformPath(tmpFolder, + TStringBuilder() << "tmp/" << GetGuidAsString(session->RandomProvider_->GenGuid()), true, session->UserName_); + + auto cluster = options.Cluster(); + auto filter = options.Filter(); + auto exprCtx = options.ExprCtx(); + + TExpressionResorceUsage extraUsage; + TString lambda; + if (filter) { + YQL_ENSURE(exprCtx); + TScopedAlloc alloc(__LOCATION__, NKikimr::TAlignedPagePoolCounters(), + Services_.FunctionRegistry->SupportsSizedAllocators()); + alloc.SetLimit(options.Config()->DefaultCalcMemoryLimit.Get().GetOrElse(0)); + NKikimr::NMiniKQL::TTypeEnvironment typeEnv(alloc); + TProgramBuilder pgmBuilder(typeEnv, *Services_.FunctionRegistry); + + auto returnType = pgmBuilder.NewListType(pgmBuilder.NewTupleType({ + pgmBuilder.NewDataType(NUdf::TDataType<char*>::Id), + pgmBuilder.NewDataType(NUdf::TDataType<ui64>::Id) + })); + TCallableBuilder inputNodeBuilder(typeEnv, MrRangeInputListInternal, returnType); + auto inputNode = TRuntimeNode(inputNodeBuilder.Build(), false); + auto pgm = pgmBuilder.Filter(inputNode, [&](TRuntimeNode item) { + TMkqlBuildContext ctx(*MkqlCompiler_, pgmBuilder, *exprCtx, filter->UniqueId(), {{&filter->Head().Head(), pgmBuilder.Nth(item, 0)}}); + return pgmBuilder.Coalesce(MkqlBuildExpr(filter->Tail(), ctx), pgmBuilder.NewDataLiteral(false)); + }); + lambda = SerializeRuntimeNode(pgm, typeEnv); + extraUsage = ScanExtraResourceUsage(filter->Tail(), *options.Config()); + } + + auto execCtx = MakeExecCtx(std::move(options), session, cluster, filter, exprCtx); + if (lambda) { + return session->Queue_->Async([execCtx, tmpTablePath, lambda, extraUsage] () { + YQL_LOG_CTX_ROOT_SESSION_SCOPE(execCtx->LogCtx_); + execCtx->MakeUserFiles(); + return ExecGetTableRange(execCtx, tmpTablePath, lambda, extraUsage); + }); + } + + return session->Queue_->Async([execCtx] () { + YQL_LOG_CTX_ROOT_SESSION_SCOPE(execCtx->LogCtx_); + return ExecGetTableRange(execCtx); + }); + } catch (...) { + return MakeFuture(ResultFromCurrentException<TTableRangeResult>(pos)); + } + } + + TFuture<TFolderResult> GetFolder(TFolderOptions&& options) final { + YQL_LOG_CTX_SCOPE(TStringBuf("Gateway"), __FUNCTION__); + + YQL_CLOG(INFO, ProviderYt) << "Server=" << options.Cluster() << ", Prefix=" << options.Prefix(); + + try { + TSession::TPtr session = GetSession(options.SessionId()); + + auto batchOptions = TBatchFolderOptions(options.SessionId()) + .Cluster(options.Cluster()) + .Pos(options.Pos()) + .Config(options.Config()) + .Folders({{options.Prefix(), options.Attributes()}}); + auto execCtx = MakeExecCtx(std::move(batchOptions), session, options.Cluster(), nullptr, nullptr); + + if (auto filePtr = MaybeGetFilePtrFromCache(execCtx->GetOrCreateEntry(), execCtx->Options_.Folders().front())) { + TFolderResult res; + res.SetSuccess(); + res.ItemsOrFileLink = *filePtr; + return MakeFuture(res); + } + + auto getFolderFuture = session->Queue_->Async([execCtx] () { + YQL_LOG_CTX_ROOT_SESSION_SCOPE(execCtx->LogCtx_); + return ExecGetFolder(execCtx); + }); + auto resolvedLinksFuture = getFolderFuture.Apply([options, this, session] (const TFuture<TBatchFolderResult>& f) { + TVector<IYtGateway::TResolveOptions::TItemWithReqAttrs> resolveItems; + auto res = f.GetValue(); + + if (!res.Success()) { + YQL_CLOG(INFO, ProviderYt) << "Skipping resolve for unsuccessful batch folder list call"; + return res; + } + if (res.Items.empty()) { + YQL_CLOG(INFO, ProviderYt) << "Skipping resolve for empty batch folder result"; + return res; + } + + for (auto&& item : res.Items) { + YQL_CLOG(DEBUG, ProviderYt) << "Adding to batch get command: " << item.Path; + IYtGateway::TResolveOptions::TItemWithReqAttrs resolveItem { + .Item = item, + .AttrKeys = options.Attributes() + }; + resolveItems.push_back(std::move(resolveItem)); + } + + auto resolveOptions = TResolveOptions(options.SessionId()) + .Cluster(options.Cluster()) + .Pos(options.Pos()) + .Config(options.Config()) + .Items(resolveItems); + auto execCtx = MakeExecCtx(std::move(resolveOptions), session, options.Cluster(), nullptr, nullptr); + return ExecResolveLinks(execCtx); + }); + + return resolvedLinksFuture.Apply([execCtx] (const TFuture<TBatchFolderResult>& f) { + const ui32 countLimit = execCtx->Options_.Config()->FolderInlineItemsLimit.Get().GetOrElse(100); + const ui64 sizeLimit = execCtx->Options_.Config()->FolderInlineDataLimit.Get().GetOrElse(100_KB); + + auto resolveRes = f.GetValue(); + + TFolderResult res; + res.AddIssues(resolveRes.Issues()); + + if (!resolveRes.Success()) { + res.SetStatus(resolveRes.Status()); + return res; + } + res.SetSuccess(); + + YQL_CLOG(INFO, ProviderYt) << "Batch get command got: " << resolveRes.Items.size() << " items"; + + TVector<TFolderResult::TFolderItem> items; + for (auto& batchItem : resolveRes.Items) { + TFolderResult::TFolderItem item { + .Path = std::move(batchItem.Path), + .Type = std::move(batchItem.Type), + .Attributes = NYT::NodeToYsonString(batchItem.Attributes) + }; + items.emplace_back(std::move(item)); + } + if (items.size() > countLimit) { + res.ItemsOrFileLink = SaveItemsToTempFile(execCtx, items); + return res; + } + ui64 total_size = std::accumulate(items.begin(), items.end(), 0, [] (ui64 size, const TFolderResult::TFolderItem& i) { + return size + i.Type.length() + i.Path.length() + i.Attributes.length(); + }); + if (total_size > sizeLimit) { + res.ItemsOrFileLink = SaveItemsToTempFile(execCtx, items); + return res; + } + res.ItemsOrFileLink = std::move(items); + return res; + }); + } catch (...) { + return MakeFuture(ResultFromCurrentException<TFolderResult>(options.Pos())); + } + } + + TFuture<TBatchFolderResult> GetFolders(TBatchFolderOptions&& options) final { + YQL_LOG_CTX_SCOPE(TStringBuf("Gateway"), __FUNCTION__); + + YQL_CLOG(INFO, ProviderYt) << "Server=" << options.Cluster(); + + auto pos = options.Pos(); + try { + TSession::TPtr session = GetSession(options.SessionId()); + + auto cluster = options.Cluster(); + auto execCtx = MakeExecCtx(std::move(options), session, cluster, nullptr, nullptr); + + return session->Queue_->Async([execCtx] () { + YQL_LOG_CTX_ROOT_SESSION_SCOPE(execCtx->LogCtx_); + return ExecGetFolder(execCtx); + }); + } catch (...) { + return MakeFuture(ResultFromCurrentException<TBatchFolderResult>(pos)); + } + } + + + TFuture<TBatchFolderResult> ResolveLinks(TResolveOptions&& options) final { + YQL_LOG_CTX_SCOPE(TStringBuf("Gateway"), __FUNCTION__); + + YQL_CLOG(INFO, ProviderYt) << "Server=" << options.Cluster(); + + auto pos = options.Pos(); + try { + TSession::TPtr session = GetSession(options.SessionId()); + + auto cluster = options.Cluster(); + auto execCtx = MakeExecCtx(std::move(options), session, cluster, nullptr, nullptr); + + return session->Queue_->Async([execCtx] () { + YQL_LOG_CTX_ROOT_SESSION_SCOPE(execCtx->LogCtx_); + return ExecResolveLinks(execCtx); + }); + } catch (...) { + return MakeFuture(ResultFromCurrentException<TBatchFolderResult>(pos)); + } + } + + TFuture<TResOrPullResult> ResOrPull(const TExprNode::TPtr& node, TExprContext& ctx, TResOrPullOptions&& options) final { + YQL_LOG_CTX_SCOPE(TStringBuf("Gateway"), __FUNCTION__); + try { + YQL_CLOG(INFO, ProviderYt) << node->Content(); + TSession::TPtr session = GetSession(options.SessionId()); + session->EnsureInitializedSemaphore(options.Config()); + if (auto pull = TMaybeNode<TPull>(node)) { + return DoPull(session, pull.Cast(), ctx, std::move(options)); + } else if (auto result = TMaybeNode<TResult>(node)) { + return DoResult(session, result.Cast(), ctx, std::move(options)); + } else { + YQL_LOG_CTX_THROW yexception() << "Don't know how to execute " << node->Content(); + } + } catch (...) { + return MakeFuture(ResultFromCurrentException<TResOrPullResult>()); + } + } + + TFuture<TRunResult> Run(const TExprNode::TPtr& node, TExprContext& ctx, TRunOptions&& options) final { + YQL_LOG_CTX_SCOPE(TStringBuf("Gateway"), __FUNCTION__); + auto nodePos = ctx.GetPosition(node->Pos()); + try { + YQL_CLOG(INFO, ProviderYt) << node->Content(); + TSession::TPtr session = GetSession(options.SessionId()); + session->EnsureInitializedSemaphore(options.Config()); + + TYtOpBase opBase(node); + + auto cluster = TString{opBase.DataSink().Cluster().Value()}; + + auto execCtx = MakeExecCtx(std::move(options), session, cluster, opBase.Raw(), &ctx); + + if (auto transientOp = opBase.Maybe<TYtTransientOpBase>()) { + THashSet<TString> extraSysColumns; + if (NYql::HasSetting(transientOp.Settings().Ref(), EYtSettingType::KeySwitch) + && !transientOp.Maybe<TYtMapReduce>().Mapper().Maybe<TCoLambda>().IsValid()) { + extraSysColumns.insert("keyswitch"); + } + execCtx->SetInput(transientOp.Cast().Input(), opBase.Maybe<TYtWithUserJobsOpBase>().IsValid(), extraSysColumns); + } + if (auto outputOp = opBase.Maybe<TYtOutputOpBase>()) { + execCtx->SetOutput(outputOp.Cast().Output()); + } + + TFuture<void> future; + if (auto op = opBase.Maybe<TYtSort>()) { + future = DoSort(op.Cast(), execCtx); + } else if (auto op = opBase.Maybe<TYtCopy>()) { + future = DoCopy(op.Cast(), execCtx); + } else if (auto op = opBase.Maybe<TYtMerge>()) { + future = DoMerge(op.Cast(), execCtx); + } else if (auto op = opBase.Maybe<TYtMap>()) { + future = DoMap(op.Cast(), execCtx, ctx); + } else if (auto op = opBase.Maybe<TYtReduce>()) { + future = DoReduce(op.Cast(), execCtx, ctx); + } else if (auto op = opBase.Maybe<TYtMapReduce>()) { + future = DoMapReduce(op.Cast(), execCtx, ctx); + } else if (auto op = opBase.Maybe<TYtFill>()) { + future = DoFill(op.Cast(), execCtx, ctx); + } else if (auto op = opBase.Maybe<TYtTouch>()) { + future = DoTouch(op.Cast(), execCtx); + } else if (auto op = opBase.Maybe<TYtDropTable>()) { + future = DoDrop(op.Cast(), execCtx); + } else if (auto op = opBase.Maybe<TYtStatOut>()) { + future = DoStatOut(op.Cast(), execCtx); + } else if (auto op = opBase.Maybe<TYtDqProcessWrite>()) { + future = DoTouch(op.Cast(), execCtx); // Do touch just for creating temporary tables. + } else { + ythrow yexception() << "Don't know how to execute " << node->Content(); + } + + if (Services_.Config->GetLocalChainTest()) { + return future.Apply([execCtx](const TFuture<void>&) { + TRunResult result; + result.OutTableStats.reserve(execCtx->OutTables_.size()); + for (const auto& table : execCtx->OutTables_) { + result.OutTableStats.emplace_back(table.Name, MakeIntrusive<TYtTableStatInfo>()); + result.OutTableStats.back().second->Id = "stub"; + } + result.SetSuccess(); + return MakeFuture<TRunResult>(std::move(result)); + }); + } + + return future.Apply([execCtx, pos = nodePos](const TFuture<void>& f) { + YQL_LOG_CTX_ROOT_SESSION_SCOPE(execCtx->LogCtx_); + try { + f.GetValue(); // rethrow error if any + execCtx->StoreQueryCache(); + execCtx->SetNodeExecProgress("Fetching attributes of output tables"); + return MakeRunResult(execCtx->OutTables_, execCtx->GetEntry()); + } catch (...) { + return ResultFromCurrentException<TRunResult>(pos); + } + }); + + } catch (...) { + return MakeFuture(ResultFromCurrentException<TRunResult>(nodePos)); + } + } + + TFuture<TCalcResult> Calc(const TExprNode::TListType& nodes, TExprContext& ctx, TCalcOptions&& options) final { + YQL_LOG_CTX_SCOPE(TStringBuf("Gateway"), __FUNCTION__); + try { + TSession::TPtr session = GetSession(options.SessionId()); + session->EnsureInitializedSemaphore(options.Config()); + auto cluster = options.Cluster(); + + if (YQL_CLOG_ACTIVE(INFO, ProviderYt)) { + for (size_t i: xrange(Min<size_t>(nodes.size(), 10))) { + YQL_CLOG(INFO, ProviderYt) << "Cluster=" << cluster << ": " << nodes[i]->Content(); + } + if (nodes.size() > 10) { + YQL_CLOG(INFO, ProviderYt) << "...total nodes to calc=" << nodes.size(); + } + } + + TExpressionResorceUsage extraUsage; + TString lambda; + { + TScopedAlloc alloc(__LOCATION__, NKikimr::TAlignedPagePoolCounters(), + Services_.FunctionRegistry->SupportsSizedAllocators()); + alloc.SetLimit(options.Config()->DefaultCalcMemoryLimit.Get().GetOrElse(0)); + TNativeYtLambdaBuilder builder(alloc, Services_, *session); + TVector<TRuntimeNode> tupleNodes; + for (auto& node: nodes) { + tupleNodes.push_back(builder.BuildLambda(*MkqlCompiler_, node, ctx)); + auto nodeUsage = ScanExtraResourceUsage(*node, *options.Config()); + extraUsage.Cpu = Max<double>(extraUsage.Cpu, nodeUsage.Cpu); + extraUsage.Memory = Max<ui64>(extraUsage.Memory, nodeUsage.Memory); + } + lambda = SerializeRuntimeNode(builder.MakeTuple(tupleNodes), builder.GetTypeEnvironment()); + } + + TString tmpFolder = GetTablesTmpFolder(*options.Config()); + TString tmpTablePath = NYql::TransformPath(tmpFolder, + TStringBuilder() << "tmp/" << GetGuidAsString(session->RandomProvider_->GenGuid()), true, session->UserName_); + + auto execCtx = MakeExecCtx(std::move(options), session, cluster, nullptr, nullptr); + + return session->Queue_->Async([execCtx, lambda, extraUsage, tmpTablePath] () { + YQL_LOG_CTX_ROOT_SESSION_SCOPE(execCtx->LogCtx_); + execCtx->MakeUserFiles(); + return ExecCalc(lambda, extraUsage, tmpTablePath, execCtx, {}, TNodeResultFactory()); + }) + .Apply([execCtx](const TFuture<NYT::TNode>& f) { + YQL_LOG_CTX_ROOT_SESSION_SCOPE(execCtx->LogCtx_); + try { + const NYT::TNode& node = f.GetValue(); + YQL_ENSURE(node.IsList()); + TCalcResult res; + for (auto& n: node.AsList()) { + res.Data.push_back(n); + } + res.SetSuccess(); + return res; + } catch (...) { + return ResultFromCurrentException<TCalcResult>(); + } + }); + + } catch (...) { + return MakeFuture(ResultFromCurrentException<TCalcResult>()); + } + } + + TFuture<TPublishResult> Publish(const TExprNode::TPtr& node, TExprContext& ctx, TPublishOptions&& options) final { + YQL_LOG_CTX_SCOPE(TStringBuf("Gateway"), __FUNCTION__); + auto nodePos = ctx.GetPosition(node->Pos()); + try { + auto publish = TYtPublish(node); + EYtWriteMode mode = EYtWriteMode::Renew; + if (const auto modeSetting = NYql::GetSetting(publish.Settings().Ref(), EYtSettingType::Mode)) { + mode = FromString<EYtWriteMode>(modeSetting->Child(1)->Content()); + } + const bool initial = NYql::HasSetting(publish.Settings().Ref(), EYtSettingType::Initial); + + std::unordered_map<EYtSettingType, TString> strOpts; + for (const auto& setting : publish.Settings().Ref().Children()) { + if (setting->ChildrenSize() == 2) { + strOpts.emplace(FromString<EYtSettingType>(setting->Head().Content()), setting->Tail().Content()); + } else if (setting->ChildrenSize() == 1) { + strOpts.emplace(FromString<EYtSettingType>(setting->Head().Content()), TString());; + } + } + + YQL_CLOG(INFO, ProviderYt) << "Mode: " << mode << ", IsInitial: " << initial; + + TSession::TPtr session = GetSession(options.SessionId()); + session->EnsureInitializedSemaphore(options.Config()); + + auto cluster = publish.DataSink().Cluster().StringValue(); + + TVector<TString> src; + ui64 chunksCount = 0; + ui64 dataSize = 0; + std::unordered_set<TString> columnGroups; + for (auto out: publish.Input()) { + auto outTable = GetOutTable(out).Cast<TYtOutTable>(); + src.emplace_back(outTable.Name().Value()); + if (auto columnGroupSetting = NYql::GetSetting(outTable.Settings().Ref(), EYtSettingType::ColumnGroups)) { + columnGroups.emplace(columnGroupSetting->Tail().Content()); + } else { + columnGroups.emplace(); + } + auto stat = TYtTableStatInfo(outTable.Stat()); + chunksCount += stat.ChunkCount; + dataSize += stat.DataSize; + if (src.size() <= 10) { + YQL_CLOG(INFO, ProviderYt) << "Input: " << cluster << '.' << src.back(); + } + } + if (src.size() > 10) { + YQL_CLOG(INFO, ProviderYt) << "...total input tables=" << src.size(); + } + TString srcColumnGroups = columnGroups.size() == 1 ? *columnGroups.cbegin() : TString(); + + bool combineChunks = false; + if (auto minChunkSize = options.Config()->MinPublishedAvgChunkSize.Get()) { + combineChunks = *minChunkSize == 0 + || (chunksCount > 1 && dataSize > chunksCount && (dataSize / chunksCount) < minChunkSize->GetValue()); + } + + const auto dst = publish.Publish().Name().StringValue(); + YQL_CLOG(INFO, ProviderYt) << "Output: " << cluster << '.' << dst; + if (combineChunks) { + YQL_CLOG(INFO, ProviderYt) << "Use chunks combining"; + } + if (Services_.Config->GetLocalChainTest()) { + if (!src.empty()) { + const auto& path = NYql::TransformPath(GetTablesTmpFolder(*options.Config()), src.front(), true, session->UserName_); + const auto it = TestTables.find(path); + YQL_ENSURE(TestTables.cend() != it); + YQL_ENSURE(TestTables.emplace(dst, it->second).second); + } + + TPublishResult result; + result.SetSuccess(); + return MakeFuture<TPublishResult>(std::move(result)); + } + + + bool isAnonymous = NYql::HasSetting(publish.Publish().Settings().Ref(), EYtSettingType::Anonymous); + const ui32 dstEpoch = TEpochInfo::Parse(publish.Publish().Epoch().Ref()).GetOrElse(0); + auto execCtx = MakeExecCtx(std::move(options), session, cluster, node.Get(), &ctx); + + return session->Queue_->Async([execCtx, src = std::move(src), dst, dstEpoch, isAnonymous, mode, initial, srcColumnGroups, combineChunks, strOpts = std::move(strOpts)] () { + YQL_LOG_CTX_ROOT_SESSION_SCOPE(execCtx->LogCtx_); + return ExecPublish(execCtx, src, dst, dstEpoch, isAnonymous, mode, initial, srcColumnGroups, combineChunks, strOpts); + }) + .Apply([nodePos] (const TFuture<void>& f) { + try { + f.GetValue(); + TPublishResult res; + res.SetSuccess(); + return res; + } catch (...) { + return ResultFromCurrentException<TPublishResult>(nodePos); + } + }); + + } catch (...) { + return MakeFuture(ResultFromCurrentException<TPublishResult>(nodePos)); + } + } + + TFuture<TCommitResult> Commit(TCommitOptions&& options) final { + YQL_LOG_CTX_SCOPE(TStringBuf("Gateway"), __FUNCTION__); + try { + TSession::TPtr session = GetSession(options.SessionId()); + auto cluster = options.Cluster(); + + auto execCtx = MakeExecCtx(std::move(options), session, cluster, nullptr, nullptr); + return session->Queue_->Async([execCtx] () { + YQL_LOG_CTX_ROOT_SESSION_SCOPE(execCtx->LogCtx_); + try { + execCtx->Session_->TxCache_.Commit(execCtx->YtServer_); + + TCommitResult res; + res.SetSuccess(); + return res; + } catch (...) { + return ResultFromCurrentException<TCommitResult>(); + } + }); + } catch (...) { + return MakeFuture(ResultFromCurrentException<TCommitResult>()); + } + } + + TFuture<TDropTrackablesResult> DropTrackables(TDropTrackablesOptions&& options) final { + YQL_LOG_CTX_SCOPE(TStringBuf("Gateway"), __FUNCTION__); + try { + TSession::TPtr session = GetSession(options.SessionId()); + + if (YQL_CLOG_ACTIVE(INFO, ProviderYt)) { + for (size_t i: xrange(Min<size_t>(options.Pathes().size(), 10))) { + const auto& path = options.Pathes()[i].Path; + const auto& cluster = options.Pathes()[i].Cluster; + YQL_CLOG(INFO, ProviderYt) << "Dropping temporary table '" << path << "' on cluster '" << cluster << "'"; + } + if (options.Pathes().size() > 10) { + YQL_CLOG(INFO, ProviderYt) << "...total dropping tables=" << options.Pathes().size(); + } + } + + THashMap<TString, TVector<TString>> pathsByCluster; + for (const auto& i : options.Pathes()) { + pathsByCluster[i.Cluster].push_back(i.Path); + } + + TVector<TFuture<void>> futures; + for (const auto& i : pathsByCluster) { + auto cluster = i.first; + auto paths = i.second; + + + auto execCtx = MakeExecCtx(TDropTrackablesOptions(options), session, cluster, nullptr, nullptr); + + futures.push_back(session->Queue_->Async([execCtx, paths] () { + YQL_LOG_CTX_ROOT_SESSION_SCOPE(execCtx->LogCtx_); + return ExecDropTrackables(paths, execCtx); + })); + } + + return WaitExceptionOrAll(futures).Apply([] (const TFuture<void>& f) { + try { + f.GetValue(); // rethrow error if any + + TDropTrackablesResult res; + res.SetSuccess(); + return res; + } catch (...) { + return ResultFromCurrentException<TDropTrackablesResult>(); + } + }); + } catch (...) { + return MakeFuture(ResultFromCurrentException<TDropTrackablesResult>()); + } + } + + TFuture<TPathStatResult> PathStat(TPathStatOptions&& options) final { + YQL_LOG_CTX_SCOPE(TStringBuf("Gateway"), __FUNCTION__); + try { + TSession::TPtr session = GetSession(options.SessionId()); + auto logCtx = NYql::NLog::CurrentLogContextPath(); + auto cluster = options.Cluster(); + + if (YQL_CLOG_ACTIVE(INFO, ProviderYt)) { + for (size_t i: xrange(Min<size_t>(options.Paths().size(), 10))) { + YQL_CLOG(INFO, ProviderYt) << "Cluster: " << cluster << ", table: " << NYT::NodeToYsonString(NYT::PathToNode(options.Paths()[i].Path())); + } + if (options.Paths().size() > 10) { + YQL_CLOG(INFO, ProviderYt) << "...total tables=" << options.Paths().size(); + } + } + + auto execCtx = MakeExecCtx(std::move(options), session, cluster, nullptr, nullptr); + + return session->Queue_->Async([execCtx, logCtx] () { + YQL_LOG_CTX_ROOT_SESSION_SCOPE(logCtx); + bool onlyCached = false; + return ExecPathStat(execCtx, onlyCached); + }); + } catch (...) { + return MakeFuture(ResultFromCurrentException<TPathStatResult>()); + } + } + + TPathStatResult TryPathStat(TPathStatOptions&& options) final { + YQL_LOG_CTX_SCOPE(TStringBuf("Gateway"), __FUNCTION__); + + TSession::TPtr session = GetSession(options.SessionId()); + auto logCtx = NYql::NLog::CurrentLogContextPath(); + auto cluster = options.Cluster(); + + auto execCtx = MakeExecCtx(std::move(options), session, cluster, nullptr, nullptr); + + YQL_LOG_CTX_ROOT_SESSION_SCOPE(logCtx); + bool onlyCached = true; + return ExecPathStat(execCtx, onlyCached); + } + + bool TryParseYtUrl(const TString& url, TString* cluster, TString* path) const final { + TString server; + if (!UrlMapper_.MapYtUrl(url, &server, path)) { + return false; + } + + if (cluster) { + *cluster = Clusters_->GetNameByYtName(server); + } + return true; + } + + TString GetDefaultClusterName() const final { + return Clusters_->GetDefaultClusterName(); + } + + TString GetClusterServer(const TString& cluster) const final { + return Clusters_->TryGetServer(cluster); + } + + NYT::TRichYPath GetRealTable(const TString& sessionId, const TString& cluster, const TString& table, ui32 epoch, const TString& tmpFolder) const final { + auto richYPath = NYT::TRichYPath(table); + if (TSession::TPtr session = GetSession(sessionId, true)) { + if (auto ytServer = Clusters_->TryGetServer(cluster)) { + auto entry = session->TxCache_.GetEntry(ytServer); + if (auto p = entry->Snapshots.FindPtr(std::make_pair(table, epoch))) { + richYPath.Path(std::get<0>(*p)).TransactionId(std::get<1>(*p)).OriginalPath(NYT::AddPathPrefix(table, NYT::TConfig::Get()->Prefix)); + } else { + auto realTableName = NYql::TransformPath(tmpFolder, table, true, session->UserName_); + realTableName = NYT::AddPathPrefix(realTableName, NYT::TConfig::Get()->Prefix); + richYPath = NYT::TRichYPath(realTableName); + richYPath.TransactionId(session->TxCache_.GetEntry(ytServer)->Tx->GetId()); + } + } + } + YQL_CLOG(DEBUG, ProviderYt) << "Real table path: " << NYT::NodeToYsonString(NYT::PathToNode(richYPath), NYT::NYson::EYsonFormat::Text); + return richYPath; + } + + NYT::TRichYPath GetWriteTable(const TString& sessionId, const TString& cluster, const TString& table, const TString& tmpFolder) const final { + if (TSession::TPtr session = GetSession(sessionId, true)) { + if (auto ytServer = Clusters_->TryGetServer(cluster)) { + auto entry = session->TxCache_.GetEntry(ytServer); + auto realTableName = NYql::TransformPath(tmpFolder, table, true, session->UserName_); + realTableName = NYT::AddPathPrefix(realTableName, NYT::TConfig::Get()->Prefix); + auto richYPath = NYT::TRichYPath(realTableName); + richYPath.TransactionId(entry->Tx->GetId()); + YQL_CLOG(DEBUG, ProviderYt) << "Write table path: " << NYT::NodeToYsonString(NYT::PathToNode(richYPath), NYT::NYson::EYsonFormat::Text); + return richYPath; + } + } + YQL_CLOG(DEBUG, ProviderYt) << "(Alternative) Write table path: " << NYT::NodeToYsonString(NYT::PathToNode(NYT::TRichYPath(table)), NYT::NYson::EYsonFormat::Text); + return NYT::TRichYPath(table); + } + + TFuture<TDownloadTablesResult> DownloadTables(TDownloadTablesOptions&& options) final { + YQL_LOG_CTX_SCOPE(TStringBuf("Gateway"), __FUNCTION__); + try { + TSession::TPtr session = GetSession(options.SessionId()); + auto logCtx = NYql::NLog::CurrentLogContextPath(); + + const auto epoch = options.Epoch(); + THashMap<TString, TTransactionCache::TEntry::TPtr> entries; + TVector<TFuture<void>> waits; + for (auto& req: options.Tables()) { + const auto cluster = req.Cluster(); + const auto table = req.Table(); + const auto anon = req.Anonymous(); + const auto targetPath = req.TargetPath(); + YQL_CLOG(DEBUG, ProviderYt) << "Downloading " << cluster << '.' << table << " to " << targetPath; + + TTransactionCache::TEntry::TPtr& entry = entries[cluster]; + if (!entry) { + auto ytServer = Clusters_->TryGetServer(cluster); + YQL_ENSURE(ytServer); + entry = session->TxCache_.GetEntry(ytServer); + } + + auto richYPath = NYT::TRichYPath(table); + if (auto p = entry->Snapshots.FindPtr(std::make_pair(table, epoch))) { + richYPath.Path(std::get<0>(*p)).TransactionId(std::get<1>(*p)); + } else { + auto realTableName = NYql::TransformPath(GetTablesTmpFolder(*options.Config()), table, anon, session->UserName_); + realTableName = NYT::AddPathPrefix(realTableName, NYT::TConfig::Get()->Prefix); + richYPath = NYT::TRichYPath(realTableName); + richYPath.TransactionId(entry->Tx->GetId()); + } + + waits.push_back(session->Queue_->Async([entry, richYPath, targetPath, logCtx] () { + YQL_LOG_CTX_ROOT_SESSION_SCOPE(logCtx); + + auto reader = entry->Tx->CreateRawReader( + richYPath, + NYT::TFormat::YsonText(), + NYT::TTableReaderOptions() + .CreateTransaction(false) + .ControlAttributes( + NYT::TControlAttributes() + .EnableRowIndex(false) + .EnableRangeIndex(false) + ) + ); + + TOFStream out(targetPath); + TransferData(reader.Get(), &out); + out.Finish(); + })); + } + return WaitExceptionOrAll(waits).Apply([](const TFuture<void>& f) { + try { + f.TryRethrow(); + TDownloadTablesResult res; + res.SetSuccess(); + return res; + } catch (...) { + YQL_CLOG(ERROR, ProviderYt) << CurrentExceptionMessage(); + return ResultFromCurrentException<TDownloadTablesResult>(); + } + }); + } catch (...) { + YQL_CLOG(ERROR, ProviderYt) << CurrentExceptionMessage(); + return MakeFuture(ResultFromCurrentException<TDownloadTablesResult>()); + } + } + + TFuture<TUploadTableResult> UploadTable(TUploadTableOptions&& options) final { + YQL_LOG_CTX_SCOPE(TStringBuf("Gateway"), __FUNCTION__); + try { + TSession::TPtr session = GetSession(options.SessionId()); + auto logCtx = NYql::NLog::CurrentLogContextPath(); + + const auto cluster = options.Cluster(); + const auto table = options.Table(); + const auto path = options.Path(); + auto attrs = NYT::NodeFromYsonString(options.Attrs()); + const auto config = options.Config(); + YQL_CLOG(DEBUG, ProviderYt) << "Uploading " << path << " to " << cluster << '.' << table; + + auto ytServer = Clusters_->TryGetServer(cluster); + YQL_ENSURE(ytServer); + auto entry = session->TxCache_.GetEntry(ytServer); + + NYT::TTableWriterOptions writerOptions; + auto maxRowWeight = config->MaxRowWeight.Get(cluster); + auto maxKeyWeight = config->MaxKeyWeight.Get(cluster); + + if (maxRowWeight || maxKeyWeight) { + NYT::TNode writeConfig; + if (maxRowWeight) { + writeConfig["max_row_weight"] = static_cast<i64>(*maxRowWeight); + } + if (maxKeyWeight) { + writeConfig["max_key_weight"] = static_cast<i64>(*maxKeyWeight); + } + writerOptions.Config(writeConfig); + } + + NYT::MergeNodes(attrs, YqlOpOptionsToAttrs(session->OperationOptions_)); + + return session->Queue_->Async([entry, table, path, attrs, writerOptions, logCtx] () { + YQL_LOG_CTX_ROOT_SESSION_SCOPE(logCtx); + try { + entry->Tx->Create(table, NT_TABLE, NYT::TCreateOptions().Force(true).Attributes(attrs)); + + TRawTableWriterPtr writer = entry->Tx->CreateRawWriter(table, NYT::TFormat::YsonText(), writerOptions); + TIFStream in(path); + TransferData(&in, writer.Get()); + writer->Finish(); + TUploadTableResult res; + res.SetSuccess(); + return res; + } catch (...) { + YQL_CLOG(ERROR, ProviderYt) << CurrentExceptionMessage(); + return ResultFromCurrentException<TUploadTableResult>(); + } + }); + } catch (...) { + YQL_CLOG(ERROR, ProviderYt) << CurrentExceptionMessage(); + return MakeFuture(ResultFromCurrentException<TUploadTableResult>()); + } + } + + TFuture<TRunResult> Prepare(const TExprNode::TPtr& node, TExprContext& ctx, TPrepareOptions&& options) const final { + YQL_LOG_CTX_SCOPE(TStringBuf("Gateway"), __FUNCTION__); + auto nodePos = ctx.GetPosition(node->Pos()); + try { + YQL_CLOG(INFO, ProviderYt) << node->Content(); + const auto session = GetSession(options.SessionId()); + session->EnsureInitializedSemaphore(options.Config()); + + TYtOutputOpBase opBase(node); + + const auto cluster = TString{opBase.DataSink().Cluster().Value()}; + const auto execCtx = MakeExecCtx(std::move(options), session, cluster, opBase.Raw(), &ctx); + execCtx->SetOutput(opBase.Output()); + + auto future = DoPrepare(opBase, execCtx); + + return future.Apply([execCtx, pos = nodePos](const TFuture<bool>& f) { + try { + YQL_LOG_CTX_ROOT_SESSION_SCOPE(execCtx->LogCtx_); + if (f.GetValue()) { + return MakeRunResult(execCtx->OutTables_, execCtx->GetEntry()); + } else { + TRunResult res; + res.SetSuccess(); + std::transform(execCtx->OutTables_.cbegin(), execCtx->OutTables_.cend(), std::back_inserter(res.OutTableStats), + [](const TOutputInfo& info) -> std::pair<TString, TYtTableStatInfo::TPtr> { return { info.Name, nullptr }; }); + return res; + } + } catch (...) { + return ResultFromCurrentException<TRunResult>(pos); + } + }); + } catch (...) { + return MakeFuture(ResultFromCurrentException<TRunResult>(nodePos)); + } + } + + NThreading::TFuture<TRunResult> GetTableStat(const TExprNode::TPtr& node, TExprContext& ctx, TPrepareOptions&& options) final { + if (TSession::TPtr session = GetSession(options.SessionId(), false)) { + const TYtOutputOpBase opBase(node); + if (const auto cluster = TString{opBase.DataSink().Cluster().Value()}; auto ytServer = Clusters_->TryGetServer(cluster)) { + auto entry = session->TxCache_.GetEntry(ytServer); + auto execCtx = MakeExecCtx(std::move(options), session, cluster, opBase.Raw(), &ctx); + execCtx->SetOutput(opBase.Output()); + YQL_ENSURE(execCtx->OutTables_.size() == 1U, "TODO: Support multi out."); + const auto tableName = execCtx->OutTables_.front().Name; + const auto tmpFolder = GetTablesTmpFolder(*execCtx->Options_.Config()); + const auto realTableName = NYT::AddPathPrefix(NYql::TransformPath(tmpFolder, execCtx->OutTables_.front().Name, true, session->UserName_), NYT::TConfig::Get()->Prefix); + auto batchGet = entry->Tx->CreateBatchRequest(); + auto f = batchGet->Get(realTableName + "/@", TGetOptions() + .AttributeFilter(TAttributeFilter() + .AddAttribute(TString("id")) + .AddAttribute(TString("dynamic")) + .AddAttribute(TString("row_count")) + .AddAttribute(TString("chunk_row_count")) + .AddAttribute(TString("uncompressed_data_size")) + .AddAttribute(TString("data_weight")) + .AddAttribute(TString("chunk_count")) + .AddAttribute(TString("modification_time")) + .AddAttribute(TString("sorted_by")) + .AddAttribute(TString("revision")) + .AddAttribute(TString("content_revision")) + .AddAttribute(TString(SecurityTagsName)) + ) + ).Apply([tableName, execCtx = std::move(execCtx)](const TFuture<NYT::TNode>& f) { + execCtx->StoreQueryCache(); + auto attrs = f.GetValue(); + auto statInfo = MakeIntrusive<TYtTableStatInfo>(); + statInfo->Id = tableName; + statInfo->RecordsCount = GetTableRowCount(attrs); + statInfo->DataSize = GetDataWeight(attrs).GetOrElse(0); + statInfo->ChunkCount = attrs["chunk_count"].AsInt64(); + TString strModifyTime = attrs["modification_time"].AsString(); + statInfo->ModifyTime = TInstant::ParseIso8601(strModifyTime).Seconds(); + statInfo->TableRevision = attrs["revision"].IntCast<ui64>(); + statInfo->SecurityTags = {}; + for (const auto& tagNode : attrs[SecurityTagsName].AsList()) { + statInfo->SecurityTags.insert(tagNode.AsString()); + } + statInfo->Revision = GetContentRevision(attrs); + TRunResult result; + result.OutTableStats.emplace_back(statInfo->Id, statInfo); + result.SetSuccess(); + return result; + }); + + batchGet->ExecuteBatch(); + + return f; + } + } + + return MakeFuture(TRunResult()); + } + + TFullResultTableResult PrepareFullResultTable(TFullResultTableOptions&& options) override { + try { + TString cluster = options.Cluster(); + auto outTable = options.OutTable(); + TSession::TPtr session = GetSession(options.SessionId(), true); + + auto execCtx = MakeExecCtx(std::move(options), session, cluster, nullptr, nullptr); + execCtx->SetSingleOutput(outTable); + + const auto& out = execCtx->OutTables_.front(); + NYT::TNode attrs = NYT::TNode::CreateMap(); + PrepareAttributes(attrs, out, execCtx, cluster, true); + + TFullResultTableResult res; + if (auto entry = execCtx->TryGetEntry()) { + res.RootTransactionId = GetGuidAsString(entry->Tx->GetId()); + if (entry->CacheTxId) { + res.ExternalTransactionId = GetGuidAsString(entry->CacheTxId); + } + } + + const auto nativeTypeCompat = execCtx->Options_.Config()->NativeYtTypeCompatibility.Get(cluster).GetOrElse(NTCF_LEGACY); + res.Server = execCtx->YtServer_; + res.Path = NYT::AddPathPrefix(out.Path, NYT::TConfig::Get()->Prefix); + res.RefName = out.Path; + res.CodecSpec = execCtx->GetOutSpec(false, nativeTypeCompat); + res.TableAttrs = NYT::NodeToYsonString(attrs); + + res.SetSuccess(); + return res; + } catch (...) { + return ResultFromCurrentException<TFullResultTableResult>(); + } + } + + void SetStatUploader(IStatUploader::TPtr statUploader) final { + YQL_ENSURE(!StatUploader_, "StatUploader already set"); + StatUploader_ = statUploader; + } + + void RegisterMkqlCompiler(NCommon::TMkqlCallableCompilerBase& compiler) override { + Y_UNUSED(compiler); + } + + TGetTablePartitionsResult GetTablePartitions(TGetTablePartitionsOptions&& options) override { + try { + TSession::TPtr session = GetSession(options.SessionId()); + const TString tmpFolder = GetTablesTmpFolder(*options.Config()); + + auto execCtx = MakeExecCtx(std::move(options), session, options.Cluster(), nullptr, nullptr); + auto entry = execCtx->GetOrCreateEntry(); + + TVector<NYT::TRichYPath> paths; + for (const auto& pathInfo: execCtx->Options_.Paths()) { + const auto tablePath = TransformPath(tmpFolder, pathInfo->Table->Name, pathInfo->Table->IsTemp, session->UserName_); + NYT::TRichYPath richYtPath{NYT::AddPathPrefix(tablePath, NYT::TConfig::Get()->Prefix)}; + if (auto p = entry->Snapshots.FindPtr(std::make_pair(pathInfo->Table->Name, pathInfo->Table->Epoch.GetOrElse(0)))) { + richYtPath.Path(std::get<0>(*p)).TransactionId(std::get<1>(*p)).OriginalPath(NYT::AddPathPrefix(pathInfo->Table->Name, NYT::TConfig::Get()->Prefix)); + } + pathInfo->FillRichYPath(richYtPath); // n.b. throws exception, if there is no RowSpec (we assume it is always there) + paths.push_back(std::move(richYtPath)); + } + + auto apiOptions = NYT::TGetTablePartitionsOptions() + .PartitionMode(NYT::ETablePartitionMode::Unordered) + .DataWeightPerPartition(execCtx->Options_.DataSizePerJob()) + .MaxPartitionCount(execCtx->Options_.MaxPartitions()) + .AdjustDataWeightPerPartition(execCtx->Options_.AdjustDataWeightPerPartition()); + auto res = TGetTablePartitionsResult(); + res.Partitions = entry->Tx->GetTablePartitions(paths, apiOptions); + res.SetSuccess(); + return res; + } catch (...) { + return ResultFromCurrentException<TGetTablePartitionsResult>({}, true); + } + } + + void AddCluster(const TYtClusterConfig& cluster) override { + Clusters_->AddCluster(cluster, false); + } + +private: + class TNodeResultBuilder { + public: + void WriteValue(const NUdf::TUnboxedValue& value, TType* type) { + if (type->IsTuple()) { + auto tupleType = AS_TYPE(NMiniKQL::TTupleType, type); + for (ui32 i: xrange(tupleType->GetElementsCount())) { + Node_.Add(NCommon::ValueToNode(value.GetElement(i), tupleType->GetElementType(i))); + } + } else if (type->IsList()) { + auto itemType = AS_TYPE(NMiniKQL::TListType, type)->GetItemType(); + const auto it = value.GetListIterator(); + for (NUdf::TUnboxedValue item; it.Next(item);) { + Node_.Add(NCommon::ValueToNode(item, itemType)); + } + } else { + Node_.Add(NCommon::ValueToNode(value, type)); + } + } + + bool WriteNext(const NYT::TNode& item) { + Node_.Add(item); + return true; + } + + NYT::TNode Make() { + if (Node_.IsUndefined()) { + return NYT::TNode::CreateList(); + } + return std::move(Node_); + } + private: + NYT::TNode Node_; + }; + + struct TNodeResultFactory { + using TResult = NYT::TNode; + + bool UseResultYson() const { + return false; + } + + THolder<TNodeResultBuilder> Create(const TCodecContext& codecCtx, const NKikimr::NMiniKQL::THolderFactory& holderFactory) const { + Y_UNUSED(codecCtx); + Y_UNUSED(holderFactory); + + return Create(); + } + + THolder<TNodeResultBuilder> Create() const { + return MakeHolder<TNodeResultBuilder>(); + } + }; + + class TYsonExprResultFactory { + public: + using TResult = std::pair<TString, bool>; + + TYsonExprResultFactory(TMaybe<ui64> rowLimit, TMaybe<ui64> byteLimit, const TVector<TString>& columns, bool hasListResult) + : RowLimit_(rowLimit) + , ByteLimit_(byteLimit) + , Columns_(columns) + , HasListResult_(hasListResult) + { + } + + bool UseResultYson() const { + return true; + } + + THolder<TYsonExecuteResOrPull> Create(TCodecContext& codecCtx, const NKikimr::NMiniKQL::THolderFactory& holderFactory) const { + Y_UNUSED(codecCtx); + Y_UNUSED(holderFactory); + + return Create(); + } + + THolder<TYsonExecuteResOrPull> Create() const { + THolder<TYsonExecuteResOrPull> res; + + res = MakeHolder<TYsonExecuteResOrPull>(RowLimit_, ByteLimit_, Columns_); + if (HasListResult_) { + res->SetListResult(); + } + return res; + } + + private: + const TMaybe<ui64> RowLimit_; + const TMaybe<ui64> ByteLimit_; + const TVector<TString> Columns_; + const bool HasListResult_; + }; + + class TSkiffExprResultFactory { + public: + using TResult = std::pair<TString, bool>; + + TSkiffExprResultFactory(TMaybe<ui64> rowLimit, TMaybe<ui64> byteLimit, bool hasListResult, const NYT::TNode& attrs, const TString& optLLVM, const TVector<TString>& columns) + : RowLimit_(rowLimit) + , ByteLimit_(byteLimit) + , HasListResult_(hasListResult) + , Attrs_(attrs) + , OptLLVM_(optLLVM) + , Columns_(columns) + { + } + + bool UseResultYson() const { + return true; + } + + THolder<TSkiffExecuteResOrPull> Create(TCodecContext& codecCtx, const NKikimr::NMiniKQL::THolderFactory& holderFactory) const { + THolder<TSkiffExecuteResOrPull> res; + + res = MakeHolder<TSkiffExecuteResOrPull>(RowLimit_, ByteLimit_, codecCtx, holderFactory, Attrs_, OptLLVM_, Columns_); + if (HasListResult_) { + res->SetListResult(); + } + + return res; + } + + THolder<TSkiffExecuteResOrPull> Create() const { + YQL_ENSURE(false, "Unexpected skiff result builder creation"); + } + private: + const TMaybe<ui64> RowLimit_; + const TMaybe<ui64> ByteLimit_; + const bool HasListResult_; + const NYT::TNode Attrs_; + const TString OptLLVM_; + const TVector<TString> Columns_; + }; + + static TFinalizeResult ExecFinalize(const TSession::TPtr& session, bool abort, bool detachSnapshotTxs) { + try { + TFinalizeResult res; + if (detachSnapshotTxs) { + YQL_CLOG(INFO, ProviderYt) << "Detaching all snapshot transactions"; + session->TxCache_.DetachSnapshotTxs(); + } + if (abort) { + YQL_CLOG(INFO, ProviderYt) << "Aborting all transactions for hidden query"; + session->TxCache_.AbortAll(); + } else { + session->TxCache_.Finalize(); + } + res.SetSuccess(); + return res; + } catch (...) { + return ResultFromCurrentException<TFinalizeResult>(); + } + } + + static TCanonizePathsResult ExecCanonizePaths(const TVector<TCanonizeReq>& paths, + const THashMap<TString, TMetaPerServerRequest<TCanonizePathsOptions>>& reqPerServer) + { + try { + TCanonizePathsResult res; + res.SetSuccess(); + res.Data.resize(paths.size()); + + for (auto& grp: reqPerServer) { + auto entry = grp.second.ExecContext->GetOrCreateEntry(); + auto batch = entry->Tx->CreateBatchRequest(); + TVector<TFuture<void>> batchRes(Reserve(grp.second.TableIndicies.size())); + + for (auto idx: grp.second.TableIndicies) { + const TCanonizeReq& canonReq = paths[idx]; + batchRes.push_back(batch->CanonizeYPath(canonReq.Path()).Apply([idx, &res] (const TFuture<TRichYPath>& f) { + auto& normalizedPath = f.GetValue(); + TString path = normalizedPath.Path_; + + // Convert back from absolute path to relative + // All futhrer YT operations will use the path with YT_PREFIX + if (path.StartsWith("//")) { + path = path.substr(2); + } + res.Data[idx].Path = path; + if (normalizedPath.Columns_) { + res.Data[idx].Columns.ConstructInPlace(normalizedPath.Columns_->Parts_); + } + res.Data[idx].Ranges = normalizedPath.GetRanges(); + res.Data[idx].AdditionalAttributes = SerializeRichYPathAttrs(normalizedPath); + })); + + } + batch->ExecuteBatch(); + for (size_t i: xrange(batchRes.size())) { + try { + batchRes[i].GetValue(); + } + catch (...) { + FillResultFromCurrentException(res, paths.at(grp.second.TableIndicies.at(i)).Pos()); + } + } + } + + return res; + } catch (...) { + return ResultFromCurrentException<TCanonizePathsResult>(); + } + } + + static TVector<std::pair<size_t, TString>> BatchLockTables(const NYT::ITransactionPtr& tx, const TVector<TTableReq>& tables, + const TVector<size_t>& tablesToLock, TMaybe<ELockMode> lockMode = {}) + { + auto batchLock = tx->CreateBatchRequest(); + TVector<TFuture<std::pair<size_t, TString>>> batchLockRes; + batchLockRes.reserve(tablesToLock.size()); + + for (auto idx: tablesToLock) { + const TTableReq& tableReq = tables[idx]; + + auto tablePath = tableReq.Table(); + ELockMode mode = lockMode.GetOrElse(HasExclusiveModifyIntents(tableReq.Intents()) ? LM_EXCLUSIVE : LM_SHARED); + + batchLockRes.push_back(batchLock->Lock(tablePath, mode).Apply([idx](const TFuture<ILockPtr>& res) { + try { + auto lock = res.GetValue(); + TString lockId = TStringBuilder() << '#' << GetGuidAsString(lock->GetId()); + return std::make_pair(idx, lockId); + } catch (const TErrorResponse& e) { + // Yt returns NoSuchTransaction as inner issue for ResolveError + if (!e.IsResolveError() || e.IsNoSuchTransaction()) { + throw; + } + return std::make_pair(idx, TString()); + } + })); + } + + batchLock->ExecuteBatch(); + + auto batchGet = tx->CreateBatchRequest(); + TVector<TFuture<std::pair<size_t, TString>>> batchGetRes; + batchGetRes.reserve(tablesToLock.size()); + + for (auto& f: batchLockRes) { + auto lockRes = f.GetValue(); + size_t idx = lockRes.first; + TString lockId = lockRes.second; + if (lockId) { + batchGetRes.push_back(batchGet->Get(lockId + "/@node_id").Apply([idx](const TFuture<NYT::TNode>& res) { + TString id = TStringBuilder() << '#' << res.GetValue().AsString(); + return std::make_pair(idx, id); + })); + } else { + batchGetRes.push_back(MakeFuture(lockRes)); + } + } + batchGet->ExecuteBatch(); + + TVector<std::pair<size_t, TString>> res; + res.reserve(tablesToLock.size()); + + std::transform(batchGetRes.begin(), batchGetRes.end(), std::back_inserter(res), + [] (const TFuture<std::pair<size_t, TString>>& f) { return f.GetValue(); }); + + return res; + } + + // Returns tables, which require additional snapshot lock + static TVector<size_t> ProcessTablesToXLock( + const TExecContext<TGetTableInfoOptions>::TPtr& execCtx, + const TTransactionCache::TEntry::TPtr& entry, + const NYT::ITransactionPtr& lockTx, + const TVector<TTableReq>& tables, + const TVector<size_t>& tablesToXLock, + ui32 epoch, + TTableInfoResult& res) + { + NSorted::TSimpleMap<size_t, TString> existingIdxs; + + auto lockIds = BatchLockTables(lockTx, tables, tablesToXLock); + + if (0 == epoch) { + auto batchGet = lockTx->CreateBatchRequest(); + TVector<TFuture<void>> batchGetRes; + with_lock(entry->Lock_) { + for (auto& lockRes: lockIds) { + const TTableReq& tableReq = tables[lockRes.first]; + auto tablePath = tableReq.Table(); + if (auto p = entry->Snapshots.FindPtr(std::make_pair(tablePath, epoch))) { + const ui64 revision = std::get<2>(*p); + if (lockRes.second) { + batchGetRes.push_back(batchGet->Get(lockRes.second + "/@revision").Apply([revision, tablePath](const TFuture<NYT::TNode>& f) { + const NYT::TNode& attr = f.GetValue(); + if (attr.IntCast<ui64>() != revision) { + YQL_LOG_CTX_THROW TErrorException(TIssuesIds::YT_CONCURRENT_TABLE_MODIF) + << "Table " << tablePath.Quote() + << " was modified before taking exclusive lock for it." + << " Aborting query to prevent data lost"; + } + })); + } else { + YQL_LOG_CTX_THROW TErrorException(TIssuesIds::YT_CONCURRENT_TABLE_MODIF) + << "Table " << tablePath.Quote() + << " was dropped before taking exclusive lock for it." + << " Aborting query to prevent data lost"; + + } + } + } + } + if (batchGetRes) { + batchGet->ExecuteBatch(); + WaitExceptionOrAll(batchGetRes).GetValue(); + } + } + + auto batchGetSort = lockTx->CreateBatchRequest(); + TVector<TFuture<std::pair<size_t, bool>>> batchGetSortRes; + TVector<TString> ensureParents; + TVector<TString> ensureParentsTmp; + auto batchLock = lockTx->CreateBatchRequest(); + TVector<TFuture<void>> batchLockRes; + + for (auto& lockRes: lockIds) { + size_t idx = lockRes.first; + TString id = lockRes.second; + const TTableReq& tableReq = tables[idx]; + auto tablePath = tableReq.Table(); + TYtTableMetaInfo::TPtr metaRes; + if (!tableReq.LockOnly()) { + metaRes = res.Data[idx].Meta = MakeIntrusive<TYtTableMetaInfo>(); + } + const bool loadMeta = !tableReq.LockOnly(); + const bool exclusive = HasExclusiveModifyIntents(tableReq.Intents()); + if (id) { + if (metaRes) { + metaRes->DoesExist = true; + } + YQL_CLOG(INFO, ProviderYt) << "Lock " << tablePath.Quote() << " with " + << (exclusive ? LM_EXCLUSIVE : LM_SHARED) + << " mode (" << id << ')'; + + if (loadMeta) { + existingIdxs.emplace_back(idx, id); + } + if (!exclusive) { + batchGetSortRes.push_back(batchGetSort->Get(id + "/@", TGetOptions().AttributeFilter(TAttributeFilter().AddAttribute("sorted_by"))) + .Apply([idx](const TFuture<NYT::TNode>& f) { + const NYT::TNode& attrs = f.GetValue(); + return std::make_pair(idx, attrs.HasKey("sorted_by") && !attrs["sorted_by"].AsList().empty()); + }) + ); + } + } else { + if (metaRes) { + metaRes->DoesExist = false; + } + tablePath = NYT::AddPathPrefix(tablePath, NYT::TConfig::Get()->Prefix); + TString folder; + TString tableName = tablePath; + auto slash = tableName.rfind('/'); + if (TString::npos != slash) { + folder = tableName.substr(0, slash); + tableName = tableName.substr(slash + 1); + if (folder == "/") { + folder = "#" + lockTx->Get("//@id").AsString(); + } else { + (tableReq.Anonymous() ? ensureParentsTmp : ensureParents).push_back(tablePath); + } + } + YQL_CLOG(INFO, ProviderYt) << "Lock " << tableName.Quote() << " child of " + << folder.Quote() << " with " << LM_SHARED << " mode"; + batchLockRes.push_back(batchLock->Lock(folder, LM_SHARED, + TLockOptions().ChildKey(tableName)). Apply([] (const TFuture<ILockPtr>& f) { f.GetValue(); })); + } + } + + TVector<size_t> tablesToUpgradeLock; + if (batchGetSortRes) { + batchGetSort->ExecuteBatch(); + for (auto& f: batchGetSortRes) { + auto& sortRes = f.GetValue(); + if (sortRes.second) { + tablesToUpgradeLock.push_back(sortRes.first); + } + } + if (tablesToUpgradeLock) { + auto upgradeLockRes = BatchLockTables(lockTx, tables, tablesToUpgradeLock, LM_EXCLUSIVE); + for (auto& upgradedRes: upgradeLockRes) { + size_t idx = upgradedRes.first; + TString id = upgradedRes.second; + const TTableReq& tableReq = tables[idx]; + YQL_CLOG(INFO, ProviderYt) << "Upgrade " << tableReq.Table().Quote() << " lock to " << LM_EXCLUSIVE << " mode (" << id << ')'; + if (!tableReq.LockOnly()) { + existingIdxs[idx] = id; // Override existing record + } + } + } + } + + if (ensureParentsTmp) { + CreateParents(ensureParentsTmp, entry->CacheTx); + } + if (ensureParents) { + CreateParents(ensureParents, entry->GetRoot()); + } + + if (batchLockRes) { + batchLock->ExecuteBatch(); + WaitExceptionOrAll(batchLockRes).GetValue(); + } + + if (existingIdxs) { + FillMetadataResult(execCtx, entry, lockTx, existingIdxs, tables, res); + } + + return tablesToUpgradeLock; + } + + static TTableInfoResult ExecGetTableInfo(const TVector<TTableReq>& tables, + const THashMap<TString, TMetaPerServerRequest<TGetTableInfoOptions>>& reqPerServer, bool readOnly, ui32 epoch) + { + try { + TTableInfoResult res; + res.Data.resize(tables.size()); + + for (auto& grp: reqPerServer) { + auto entry = grp.second.ExecContext->GetOrCreateEntry(); + + NSorted::TSimpleMap<size_t, TString> existingIdxs; + + TVector<size_t> checkpointsToXLock; + TVector<size_t> tablesToXLock; + for (auto idx: grp.second.TableIndicies) { + const TTableReq& tableReq = tables[idx]; + if (HasModifyIntents(tableReq.Intents())) { + if (tableReq.Intents().HasFlags(TYtTableIntent::Flush)) { + checkpointsToXLock.push_back(idx); + } else { + tablesToXLock.push_back(idx); + } + res.Data[idx].WriteLock = true; + } + } + + TVector<size_t> tablesToSLock; + bool makeUniqSLock = false; + if (!readOnly) { + if (tablesToXLock || checkpointsToXLock) { + entry->CreateDefaultTmpFolder(); + } + if (tablesToXLock) { + tablesToSLock = ProcessTablesToXLock(grp.second.ExecContext, entry, entry->Tx, tables, tablesToXLock, epoch, res); + makeUniqSLock = !tablesToSLock.empty(); + } + + // each checkpoint has unique transaction + for (auto idx: checkpointsToXLock) { + auto lockTx = entry->GetOrCreateCheckpointTx(tables[idx].Table()); + ProcessTablesToXLock(grp.second.ExecContext, entry, lockTx, tables, {idx}, epoch, res); + } + } + + for (auto idx: grp.second.TableIndicies) { + const TTableReq& tableReq = tables[idx]; + if (!tableReq.LockOnly() && (readOnly || HasReadIntents(tableReq.Intents()))) { + auto metaRes = res.Data[idx].Meta; + if (!metaRes || metaRes->DoesExist) { + tablesToSLock.push_back(idx); + } + } + } + + if (tablesToSLock) { + if (makeUniqSLock) { + std::sort(tablesToSLock.begin(), tablesToSLock.end()); + tablesToSLock.erase(std::unique(tablesToSLock.begin(), tablesToSLock.end()), tablesToSLock.end()); + } + + auto snapshotTx = entry->GetSnapshotTx(epoch != 0); + auto snapshotTxId = snapshotTx->GetId(); + auto snapshotTxIdStr = GetGuidAsString(snapshotTxId); + + auto lockIds = BatchLockTables(snapshotTx, tables, tablesToSLock, LM_SNAPSHOT); + + TVector<std::tuple<TString, TString, size_t>> locks; + + for (auto& lockRes: lockIds) { + size_t idx = lockRes.first; + TString id = lockRes.second; + + const TTableReq& tableReq = tables[idx]; + + bool loadMeta = false; + auto metaRes = res.Data[idx].Meta; + if (!metaRes) { + metaRes = res.Data[idx].Meta = MakeIntrusive<TYtTableMetaInfo>(); + loadMeta = true; + } + + auto tablePath = tableReq.Table(); + if (id) { + if (loadMeta) { + metaRes->DoesExist = true; + existingIdxs.emplace_back(idx, id); + } + locks.emplace_back(tablePath, id, idx); + YQL_CLOG(INFO, ProviderYt) << "Snapshot " << tablePath.Quote() << " -> " << id << ", tx=" << snapshotTxIdStr; + } else { + YQL_ENSURE(loadMeta); + metaRes->DoesExist = false; + } + } + + if (existingIdxs) { + FillMetadataResult(grp.second.ExecContext, entry, snapshotTx, existingIdxs, tables, res); + } + + if (locks) { + with_lock(entry->Lock_) { + for (auto& l: locks) { + entry->Snapshots[std::make_pair(std::get<0>(l), epoch)] = std::make_tuple(std::get<1>(l), snapshotTxId, res.Data[std::get<2>(l)].Stat->TableRevision); + } + } + } + } + } + + res.SetSuccess(); + return res; + } catch (...) { + return ResultFromCurrentException<TTableInfoResult>(); + } + } + + static TFuture<TTableRangeResult> ExecGetTableRange(const TExecContext<TTableRangeOptions>::TPtr& execCtx, + const TString& tmpTablePath = {}, TString filterLambda = {}, const TExpressionResorceUsage& extraUsage = {}) + { + auto pos = execCtx->Options_.Pos(); + try { + auto entry = execCtx->GetOrCreateEntry(); + + TString prefix = execCtx->Options_.Prefix(); + TString suffix = execCtx->Options_.Suffix(); + + auto cacheKey = std::make_tuple(prefix, suffix, filterLambda); + with_lock(entry->Lock_) { + if (auto p = entry->RangeCache.FindPtr(cacheKey)) { + YQL_CLOG(INFO, ProviderYt) << "Found range in cache for key ('" << prefix << "','" << suffix << "',<filter with size " << filterLambda.size() << ">) - number of items " << p->size(); + return MakeFuture(MakeTableRangeResult(*p)); + } + } + + if (!prefix.empty() && !entry->Tx->Exists(prefix)) { + YQL_CLOG(INFO, ProviderYt) << "Storing empty range to cache with key ('" << std::get<0>(cacheKey) << "','" << std::get<1>(cacheKey) << "',<filter with size " << std::get<2>(cacheKey).size() << ">)"; + with_lock(entry->Lock_) { + entry->RangeCache.emplace(std::move(cacheKey), std::vector<NYT::TRichYPath>{}); + } + + TTableRangeResult rangeRes; + rangeRes.SetSuccess(); + return MakeFuture(rangeRes); + } + + std::vector<TString> names; + std::vector<std::exception_ptr> errors; + + bool foundInPartialCache = false; + with_lock(entry->Lock_) { + if (auto p = entry->PartialRangeCache.FindPtr(prefix)) { + std::tie(names, errors) = *p; + foundInPartialCache = true; + } + } + + if (!foundInPartialCache) { + auto typeAttrFilter = TAttributeFilter().AddAttribute("type").AddAttribute("_yql_type").AddAttribute("broken"); + auto nodeList = entry->Tx->List(prefix, + TListOptions().AttributeFilter(typeAttrFilter)); + TVector< + std::pair< + TString, //name + std::variant<TString, std::exception_ptr> //type or exception + > + > items(nodeList.size()); + { + auto batchGet = entry->Tx->CreateBatchRequest(); + TVector<TFuture<void>> batchRes; + for (size_t i: xrange(nodeList.size())) { + const auto& node = nodeList[i]; + items[i].first = node.AsString(); + items[i].second = GetTypeFromNode(node, true); + if (std::get<TString>(items[i].second) == "link") { + if (!node.GetAttributes().HasKey("broken") || !node.GetAttributes()["broken"].AsBool()) { + batchRes.push_back(batchGet->Get(prefix + "/" + node.AsString() + "/@", TGetOptions().AttributeFilter(typeAttrFilter)) + .Apply([i, &items](const TFuture<NYT::TNode> &f) { + try { + items[i].second = GetTypeFromAttributes(f.GetValue(), true); + } catch (...) { + items[i].second = std::current_exception(); + } + })); + } + } + } + batchGet->ExecuteBatch(); + WaitExceptionOrAll(batchRes).GetValue(); + } + + names.reserve(items.size()); + errors.reserve(items.size()); + for (const auto& item: items) { + if (const auto* type = std::get_if<TString>(&item.second)) { + if ( + (suffix.empty() && ("table" == *type || "view" == *type)) || + (!suffix.empty() && "map_node" == *type) + ) { + names.push_back(item.first); + errors.emplace_back(); + } + } else { + auto exptr = std::get<std::exception_ptr>(item.second); + if (filterLambda) { + // Delayed error processing + names.push_back(item.first); + errors.push_back(std::move(exptr)); + } else { + std::rethrow_exception(exptr); + } + } + } + YQL_ENSURE(names.size() == errors.size()); + YQL_CLOG(INFO, ProviderYt) << "Got " << names.size() << " items in folder '" << prefix << "'. Storing to partial cache"; + with_lock(entry->Lock_) { + entry->PartialRangeCache.emplace(prefix, std::make_pair(names, errors)); + } + } else { + YQL_CLOG(INFO, ProviderYt) << "Found range in partial cache for '" << prefix << "' - number of items " << names.size(); + } + + if (filterLambda && !names.empty()) { + YQL_CLOG(DEBUG, ProviderYt) << "Executing range filter"; + { + TScopedAlloc alloc(__LOCATION__, NKikimr::TAlignedPagePoolCounters(), + execCtx->FunctionRegistry_->SupportsSizedAllocators()); + alloc.SetLimit(execCtx->Options_.Config()->DefaultCalcMemoryLimit.Get().GetOrElse(0)); + TNativeYtLambdaBuilder builder(alloc, execCtx->FunctionRegistry_, *execCtx->Session_); + TProgramBuilder pgmBuilder(builder.GetTypeEnvironment(), *execCtx->FunctionRegistry_); + + TRuntimeNode root = DeserializeRuntimeNode(filterLambda, builder.GetTypeEnvironment()); + + root = builder.TransformAndOptimizeProgram(root, [&](TInternName name)->TCallableVisitFunc { + if (name == MrRangeInputListInternal) { + return [&](NMiniKQL::TCallable& callable, const TTypeEnvironment& env)->TRuntimeNode { + Y_UNUSED(callable); + Y_UNUSED(env); + TVector<TRuntimeNode> inputs; + for (size_t i = 0; i < names.size(); ++i) { + inputs.push_back(pgmBuilder.NewTuple({ + pgmBuilder.NewDataLiteral<NUdf::EDataSlot::String>(names[i]), + pgmBuilder.NewDataLiteral(ui64(i)) + })); + } + auto inputNode = pgmBuilder.AsList(inputs); + return inputNode; + }; + } + return TCallableVisitFunc(); + }); + filterLambda = SerializeRuntimeNode(root, builder.GetTypeEnvironment()); + } + + auto logCtx = execCtx->LogCtx_; + return ExecCalc(filterLambda, extraUsage, tmpTablePath, execCtx, entry, TNodeResultFactory()) + .Apply([logCtx, prefix, suffix, entry, pos, errors = std::move(errors), cacheKey = std::move(cacheKey)](const TFuture<NYT::TNode>& f) mutable { + YQL_LOG_CTX_ROOT_SESSION_SCOPE(logCtx); + std::vector<TString> names; + try { + const NYT::TNode& node = f.GetValue(); + YQL_ENSURE(node.IsList()); + for (auto& n: node.AsList()) { + YQL_ENSURE(n.IsList()); + if (auto err = errors.at(n.AsList().at(1).AsUint64())) { + std::rethrow_exception(err); + } + names.push_back(n.AsList().at(0).AsString()); + } + return MakeTableRangeResult(std::move(names), std::move(cacheKey), prefix, suffix, entry); + } catch (...) { + return ResultFromCurrentException<TTableRangeResult>(pos); + } + }); + } + return MakeFuture(MakeTableRangeResult(std::move(names), std::move(cacheKey), prefix, suffix, entry)); + + } catch (...) { + return MakeFuture(ResultFromCurrentException<TTableRangeResult>(pos)); + } + } + + static TTableRangeResult MakeTableRangeResult(const std::vector<NYT::TRichYPath>& paths) { + TTableRangeResult rangeRes; + rangeRes.SetSuccess(); + + for (auto& normalizedPath: paths) { + TCanonizedPath canonPath; + canonPath.Path = normalizedPath.Path_; + if (normalizedPath.Columns_) { + canonPath.Columns.ConstructInPlace(normalizedPath.Columns_->Parts_); + } + canonPath.Ranges = normalizedPath.GetRanges(); + rangeRes.Tables.push_back(std::move(canonPath)); + } + + SortBy(rangeRes.Tables, [] (const TCanonizedPath& path) { return path.Path; }); + + return rangeRes; + } + + static TTableRangeResult MakeTableRangeResult(std::vector<TString>&& names, std::tuple<TString, TString, TString>&& cacheKey, + TString prefix, TString suffix, const TTransactionCache::TEntry::TPtr& entry) + { + TTableRangeResult rangeRes; + rangeRes.SetSuccess(); + std::vector<NYT::TRichYPath> cached; + if (prefix) { + prefix.append('/'); + } + if (suffix) { + if (!names.empty()) { + auto batchCanonize = entry->Tx->CreateBatchRequest(); + auto batchExists = entry->Tx->CreateBatchRequest(); + TVector<TFuture<void>> batchCanonizeRes; + TVector<TFuture<void>> batchExistsRes; + for (TString& name: names) { + name.prepend(prefix).append('/').append(suffix); + batchCanonizeRes.push_back(batchCanonize->CanonizeYPath(name) + .Apply([&batchExists, &batchExistsRes, &rangeRes, &cached] (const TFuture<TRichYPath>& f) { + TCanonizedPath canonPath; + auto normalizedPath = f.GetValue(); + if (normalizedPath.Path_.StartsWith(TConfig::Get()->Prefix)) { + normalizedPath.Path_ = normalizedPath.Path_.substr(TConfig::Get()->Prefix.size()); + } + canonPath.Path = normalizedPath.Path_; + if (normalizedPath.Columns_) { + canonPath.Columns.ConstructInPlace(normalizedPath.Columns_->Parts_); + } + canonPath.Ranges = normalizedPath.GetRanges(); + batchExistsRes.push_back(batchExists->Exists(canonPath.Path) + .Apply([canonPath = std::move(canonPath), normalizedPath = std::move(normalizedPath), &rangeRes, &cached] (const NThreading::TFuture<bool>& f) { + if (f.GetValue()) { + rangeRes.Tables.push_back(std::move(canonPath)); + cached.push_back(std::move(normalizedPath)); + } + })); + })); + } + batchCanonize->ExecuteBatch(); + WaitExceptionOrAll(batchCanonizeRes).GetValue(); + + batchExists->ExecuteBatch(); + WaitExceptionOrAll(batchExistsRes).GetValue(); + } + } + else { + if (prefix.StartsWith(TConfig::Get()->Prefix)) { + prefix = prefix.substr(TConfig::Get()->Prefix.size()); + } + for (auto& name: names) { + auto fullName = prefix + name; + rangeRes.Tables.push_back(TCanonizedPath{fullName, Nothing(), {}, Nothing()}); + cached.push_back(NYT::TRichYPath(fullName)); + } + } + + YQL_CLOG(INFO, ProviderYt) << "Storing " << cached.size() << " items to range cache with key ('" << std::get<0>(cacheKey) << "','" << std::get<1>(cacheKey) << "',<filter with size " << std::get<2>(cacheKey).size() << ">)"; + with_lock(entry->Lock_) { + entry->RangeCache.emplace(std::move(cacheKey), std::move(cached)); + } + + SortBy(rangeRes.Tables, [] (const TCanonizedPath& path) { return path.Path; }); + + return rangeRes; + } + + static TFuture<void> ExecPublish( + const TExecContext<TPublishOptions>::TPtr& execCtx, + const TVector<TString>& src, + const TString& dst, + const ui32 dstEpoch, + const bool isAnonymous, + EYtWriteMode mode, + const bool initial, + const TString& srcColumnGroups, + const bool combineChunks, + const std::unordered_map<EYtSettingType, TString>& strOpts) + { + TString tmpFolder = GetTablesTmpFolder(*execCtx->Options_.Config()); + auto cluster = execCtx->Cluster_; + auto entry = execCtx->GetEntry(); + + TVector<TString> srcPaths; + for (auto& p: src) { + srcPaths.push_back(NYql::TransformPath(tmpFolder, p, true, execCtx->Session_->UserName_)); + } + + auto dstPath = NYql::TransformPath(tmpFolder, dst, isAnonymous, execCtx->Session_->UserName_); + if (execCtx->Hidden) { + const auto origDstPath = dstPath; + dstPath = NYql::TransformPath( + tmpFolder, + TStringBuilder() << "tmp/" << GetGuidAsString(CreateDefaultRandomProvider()->GenGuid()), + true, + execCtx->Session_->UserName_); + if (entry->Tx->Exists(origDstPath) && EYtWriteMode::Flush != mode) { + entry->Tx->Copy( + origDstPath, + dstPath, + TCopyOptions().Force(true)); + entry->DeleteAtFinalizeInternal(dstPath); + } + if (EYtWriteMode::Flush == mode) { + mode = EYtWriteMode::Renew; + } + YQL_CLOG(INFO, ProviderYt) << "Hidden query publish destination: " << dstPath; + } + + auto publishTx = EYtWriteMode::Flush == mode ? entry->GetCheckpointTx(dstPath) : entry->Tx; + + if (isAnonymous) { + entry->CreateDefaultTmpFolder(); + } + CreateParents(TVector<TString>{dstPath}, entry->GetRoot()); + + const bool exists = entry->Tx->Exists(dstPath); + if ((EYtWriteMode::Append == mode || EYtWriteMode::RenewKeepMeta == mode) && !exists) { + mode = EYtWriteMode::Renew; + } + if (isAnonymous) { + entry->DeleteAtFinalize(dstPath); + } + + TYqlRowSpecInfo::TPtr rowSpec = execCtx->Options_.DestinationRowSpec(); + + bool appendToSorted = false; + if (EYtWriteMode::Append == mode && !strOpts.contains(EYtSettingType::MonotonicKeys)) { + NYT::TNode attrs = entry->Tx->Get(dstPath + "/@", TGetOptions() + .AttributeFilter(TAttributeFilter() + .AddAttribute(TString("sorted_by")) + ) + ); + appendToSorted = attrs.HasKey("sorted_by") && !attrs["sorted_by"].AsList().empty(); + } + + auto yqlAttrs = NYT::TNode::CreateMap(); + + auto storageAttrs = NYT::TNode::CreateMap(); + if (appendToSorted || EYtWriteMode::RenewKeepMeta == mode) { + yqlAttrs = GetUserAttributes(entry->Tx, dstPath, false); + storageAttrs = entry->Tx->Get(dstPath + "/@", TGetOptions() + .AttributeFilter(TAttributeFilter() + .AddAttribute("compression_codec") + .AddAttribute("erasure_codec") + .AddAttribute("replication_factor") + .AddAttribute("media") + .AddAttribute("primary_medium") + ) + ); + } + + bool forceMerge = combineChunks; + + NYT::MergeNodes(yqlAttrs, GetUserAttributes(entry->Tx, srcPaths.back(), true)); + NYT::MergeNodes(yqlAttrs, YqlOpOptionsToAttrs(execCtx->Session_->OperationOptions_)); + if (EYtWriteMode::RenewKeepMeta == mode) { + auto dstAttrs = entry->Tx->Get(dstPath + "/@", TGetOptions() + .AttributeFilter(TAttributeFilter() + .AddAttribute("annotation") + .AddAttribute("expiration_time") + .AddAttribute("expiration_timeout") + .AddAttribute("tablet_cell_bundle") + .AddAttribute("enable_dynamic_store_read") + ) + ); + if (dstAttrs.AsMap().contains("tablet_cell_bundle") && dstAttrs["tablet_cell_bundle"] != "default") { + forceMerge = true; + } + dstAttrs.AsMap().erase("tablet_cell_bundle"); + if (dstAttrs.AsMap().contains("enable_dynamic_store_read")) { + forceMerge = true; + } + dstAttrs.AsMap().erase("enable_dynamic_store_read"); + NYT::MergeNodes(yqlAttrs, dstAttrs); + } + NYT::TNode& rowSpecNode = yqlAttrs[YqlRowSpecAttribute]; + const auto nativeYtTypeCompatibility = execCtx->Options_.Config()->NativeYtTypeCompatibility.Get(cluster).GetOrElse(NTCF_LEGACY); + const bool rowSpecCompactForm = execCtx->Options_.Config()->UseYqlRowSpecCompactForm.Get().GetOrElse(DEFAULT_ROW_SPEC_COMPACT_FORM); + rowSpec->FillAttrNode(rowSpecNode, nativeYtTypeCompatibility, rowSpecCompactForm); + + const auto multiSet = execCtx->Options_.Config()->_UseMultisetAttributes.Get().GetOrElse(DEFAULT_USE_MULTISET_ATTRS); + + auto commitCheckpoint = [entry, dstPath, mode] (const TFuture<void>& f) { + f.GetValue(); + if (EYtWriteMode::Flush == mode) { + entry->CommitCheckpointTx(dstPath); + } + }; + + if (EYtWriteMode::Renew == mode || EYtWriteMode::RenewKeepMeta == mode) { + const auto expirationIt = strOpts.find(EYtSettingType::Expiration); + bool isTimestamp = false, isDuration = false; + TInstant stamp; + TDuration duration; + if (expirationIt != strOpts.cend()) { + isDuration = TDuration::TryParse(expirationIt->second, duration); + if (!isDuration) { + isTimestamp = TInstant::TryParseIso8601(expirationIt->second, stamp); + } + } + const TMaybe<TInstant> deadline = + execCtx->Options_.Config()->ExpirationDeadline.Get(cluster); + const TMaybe<TDuration> interval = + execCtx->Options_.Config()->ExpirationInterval.Get(cluster); + if (deadline || isTimestamp) { + yqlAttrs["expiration_time"] = isTimestamp ? stamp.ToStringUpToSeconds() + : deadline->ToStringUpToSeconds(); + } + if (interval || isDuration) { + yqlAttrs["expiration_timeout"] = isDuration ? duration.MilliSeconds() + : (*interval).MilliSeconds(); + } + if (execCtx->Options_.Config()->NightlyCompress.Get(cluster).GetOrElse(false)) { + yqlAttrs["force_nightly_compress"] = true; + } + } + + NYT::TNode securityTagsNode; + if (strOpts.contains(EYtSettingType::SecurityTags)) { + securityTagsNode = NYT::NodeFromYsonString(strOpts.at(EYtSettingType::SecurityTags)); + } + + if (EYtWriteMode::Append != mode && !securityTagsNode.IsUndefined()) { + yqlAttrs[SecurityTagsName] = securityTagsNode; + } + + const auto userAttrsIt = strOpts.find(EYtSettingType::UserAttrs); + if (userAttrsIt != strOpts.cend()) { + const NYT::TNode mapNode = NYT::NodeFromYsonString(userAttrsIt->second); + const auto& map = mapNode.AsMap(); + for (auto it = map.cbegin(); it != map.cend(); ++it) { + yqlAttrs[it->first] = it->second; + } + } + + bool forceTransform = false; + +#define DEFINE_OPT(name, attr, transform) \ + auto dst##name = isAnonymous \ + ? execCtx->Options_.Config()->Temporary##name.Get(cluster) \ + : execCtx->Options_.Config()->Published##name.Get(cluster); \ + if (EYtWriteMode::RenewKeepMeta == mode && storageAttrs.HasKey(attr) \ + && execCtx->Options_.Config()->Temporary##name.Get(cluster)) { \ + dst##name = OptionFromNode<decltype(dst##name)::value_type>(storageAttrs[attr]); \ + } \ + if (const auto it = strOpts.find(EYtSettingType::name); it != strOpts.cend()) { \ + dst##name = OptionFromString<decltype(dst##name)::value_type>(it->second); \ + } \ + if (dst##name && dst##name != execCtx->Options_.Config()->Temporary##name.Get(cluster)) { \ + forceMerge = true; \ + forceTransform = forceTransform || transform; \ + YQL_CLOG(INFO, ProviderYt) << "Option " #name " forces merge"; \ + } + + DEFINE_OPT(CompressionCodec, "compression_codec", true); + DEFINE_OPT(ErasureCodec, "erasure_codec", true); + DEFINE_OPT(ReplicationFactor, "replication_factor", false); + DEFINE_OPT(Media, "media", true); + DEFINE_OPT(PrimaryMedium, "primary_medium", true); + +#undef DEFINE_OPT + + NYT::TNode columnGroupsSpec; + if (const auto it = strOpts.find(EYtSettingType::ColumnGroups); it != strOpts.cend() && execCtx->Options_.Config()->OptimizeFor.Get(cluster).GetOrElse(NYT::OF_LOOKUP_ATTR) != NYT::OF_LOOKUP_ATTR) { + columnGroupsSpec = NYT::NodeFromYsonString(it->second); + if (it->second != srcColumnGroups) { + forceMerge = forceTransform = true; + } + } + + TFuture<void> res; + if (EYtWriteMode::Flush == mode || EYtWriteMode::Append == mode || srcPaths.size() > 1 || forceMerge) { + TFuture<bool> cacheCheck = MakeFuture<bool>(false); + if (EYtWriteMode::Flush != mode && isAnonymous) { + execCtx->SetCacheItem({dstPath}, {NYT::TNode::CreateMap()}, tmpFolder); + cacheCheck = execCtx->LookupQueryCacheAsync(); + } + res = cacheCheck.Apply([mode, srcPaths, execCtx, rowSpec, forceTransform, + appendToSorted, initial, entry, dstPath, dstEpoch, yqlAttrs, combineChunks, + dstCompressionCodec, dstErasureCodec, dstReplicationFactor, dstMedia, dstPrimaryMedium, + nativeYtTypeCompatibility, publishTx, cluster, + commitCheckpoint, columnGroupsSpec = std::move(columnGroupsSpec), + securityTagsNode] (const auto& f) mutable + { + if (f.GetValue()) { + execCtx->QueryCacheItem.Destroy(); + return MakeFuture(); + } + // Use explicit columns for source tables to cut aux columns + TVector<TString> columns; + for (auto item: rowSpec->GetType()->GetItems()) { + columns.emplace_back(item->GetName()); + } + for (auto item: rowSpec->GetAuxColumns()) { + columns.emplace_back(item.first); + } + + TMergeOperationSpec mergeSpec; + if (appendToSorted) { + if (initial) { + auto p = entry->Snapshots.FindPtr(std::make_pair(dstPath, dstEpoch)); + YQL_ENSURE(p, "Table " << dstPath << " has no snapshot"); + mergeSpec.AddInput(TRichYPath(std::get<0>(*p)).TransactionId(std::get<1>(*p)).OriginalPath(NYT::AddPathPrefix(dstPath, NYT::TConfig::Get()->Prefix)).Columns(columns)); + } else { + mergeSpec.AddInput(TRichYPath(dstPath).Columns(columns)); + } + } + for (auto& s: srcPaths) { + auto path = TRichYPath(s).Columns(columns); + if (EYtWriteMode::Flush == mode) { + path.TransactionId(entry->Tx->GetId()); + } + mergeSpec.AddInput(path); + } + + NYT::TNode spec = execCtx->Session_->CreateSpecWithDesc(); + + if (EYtWriteMode::Append == mode && !securityTagsNode.IsUndefined()) { + spec["additional_security_tags"] = securityTagsNode; + } + + auto ytDst = TRichYPath(dstPath); + if (EYtWriteMode::Append == mode && !appendToSorted) { + ytDst.Append(true); + } else { + NYT::TNode fullSpecYson; + rowSpec->FillCodecNode(fullSpecYson); + const auto schema = RowSpecToYTSchema(fullSpecYson, nativeYtTypeCompatibility, columnGroupsSpec); + ytDst.Schema(schema); + + if (EYtWriteMode::Append != mode && EYtWriteMode::RenewKeepMeta != mode) { + yqlAttrs["schema"] = schema.ToNode(); + if (dstCompressionCodec) { + yqlAttrs["compression_codec"] = *dstCompressionCodec; + } + if (dstErasureCodec) { + yqlAttrs["erasure_codec"] = ToString(*dstErasureCodec); + } + if (dstReplicationFactor) { + yqlAttrs["replication_factor"] = static_cast<i64>(*dstReplicationFactor); + } + if (dstMedia) { + yqlAttrs["media"] = *dstMedia; + } + if (dstPrimaryMedium) { + yqlAttrs["primary_medium"] = *dstPrimaryMedium; + } + if (auto optimizeFor = execCtx->Options_.Config()->OptimizeFor.Get(cluster)) { + if (schema.Columns().size()) { + yqlAttrs["optimize_for"] = ToString(*optimizeFor); + } + } + + YQL_CLOG(INFO, ProviderYt) << "Creating " << dstPath << " with attrs: " << NYT::NodeToYsonString(yqlAttrs); + publishTx->Create(dstPath, NT_TABLE, TCreateOptions().Force(true).Attributes(yqlAttrs)); + + yqlAttrs.Clear(); + } + } + mergeSpec.Output(ytDst); + mergeSpec.ForceTransform(forceTransform); + + if (rowSpec->IsSorted()) { + mergeSpec.Mode(MM_SORTED); + mergeSpec.MergeBy(ToYTSortColumns(rowSpec->GetForeignSort())); + } else { + mergeSpec.Mode(MM_ORDERED); + } + + EYtOpProps flags = EYtOpProp::PublishedAutoMerge; + if (combineChunks) { + flags |= EYtOpProp::PublishedChunkCombine; + } + + FillSpec(spec, *execCtx, entry, 0., Nothing(), flags); + + if (combineChunks) { + mergeSpec.CombineChunks(true); + } + + return execCtx->RunOperation([publishTx, mergeSpec = std::move(mergeSpec), spec = std::move(spec)]() { + return publishTx->Merge(mergeSpec, TOperationOptions().StartOperationMode(TOperationOptions::EStartOperationMode::AsyncPrepare).CreateOutputTables(false).Spec(spec)); + }) + .Apply([execCtx](const auto& f){ + f.GetValue(); + execCtx->StoreQueryCache(); + }); + }); + } + else { + publishTx->Copy(srcPaths.front(), dstPath, TCopyOptions().Force(true)); + res = MakeFuture(); + } + + std::function<void(const TFuture<void>&)> setAttrs = [logCtx = execCtx->LogCtx_, entry, publishTx, dstPath, mode, yqlAttrs, multiSet] (const TFuture<void>& f) { + YQL_LOG_CTX_ROOT_SESSION_SCOPE(logCtx); + f.GetValue(); + if (yqlAttrs.IsUndefined()) { + return; + } + YQL_CLOG(INFO, ProviderYt) << "Setting attrs for " << dstPath << ": " << NYT::NodeToYsonString(yqlAttrs); + if (multiSet) { + try { + publishTx->MultisetAttributes(dstPath + "/@", yqlAttrs.AsMap(), NYT::TMultisetAttributesOptions()); + } + catch (const TErrorResponse& e) { + if (EYtWriteMode::Append != mode || !e.IsConcurrentTransactionLockConflict()) { + throw; + } + } + } else { + auto batch = publishTx->CreateBatchRequest(); + + TVector<TFuture<void>> batchRes; + + for (auto& attr: yqlAttrs.AsMap()) { + batchRes.push_back(batch->Set(TStringBuilder() << dstPath << "/@" << attr.first, attr.second)); + } + + batch->ExecuteBatch(); + ForEach(batchRes.begin(), batchRes.end(), [mode] (const TFuture<void>& f) { + try { + f.GetValue(); + } + catch (const TErrorResponse& e) { + if (EYtWriteMode::Append != mode || !e.IsConcurrentTransactionLockConflict()) { + throw; + } + } + }); + } + }; + return res.Apply(setAttrs).Apply(commitCheckpoint); + } + + static TFuture<void> ExecDropTrackables(const TVector<TString>& paths, + const TExecContext<TDropTrackablesOptions>::TPtr& execCtx) + { + if (paths.empty()) { + return MakeFuture(); + } + + const auto tmpFolder = GetTablesTmpFolder(*execCtx->Options_.Config()); + TVector<TString> toRemove; + for (const auto& p : paths) { + toRemove.push_back(NYql::TransformPath(tmpFolder, p, true, execCtx->Session_->UserName_)); + } + + if (execCtx->Config_->GetLocalChainTest()) { + for (const auto& path : toRemove) { + YQL_ENSURE(TestTables.erase(path)); + } + return MakeFuture(); + } + + const auto entry = execCtx->GetEntry(); + + toRemove = entry->CancelDeleteAtFinalize(toRemove); + if (toRemove.empty()) { + return MakeFuture(); + } + + auto batch = entry->Tx->CreateBatchRequest(); + TVector<TFuture<void>> batchResults; + for (const auto& p : toRemove) { + batchResults.push_back(batch->Remove(p, TRemoveOptions().Force(true))); + } + batch->ExecuteBatch(); + return WaitExceptionOrAll(batchResults); + } + + static void FillMetadataResult( + const TExecContext<TGetTableInfoOptions>::TPtr& execCtx, + const TTransactionCache::TEntry::TPtr& entry, + const ITransactionPtr& tx, + const NSorted::TSimpleMap<size_t, TString>& idxs, + const TVector<TTableReq>& tables, + TTableInfoResult& result) + { + TVector<NYT::TNode> attributes(tables.size()); + TVector<TMaybe<NYT::TNode>> linkAttributes(tables.size()); + std::atomic<bool> linksPresent = false; + { + auto batchGet = tx->CreateBatchRequest(); + TVector<TFuture<void>> batchRes(Reserve(idxs.size())); + for (auto& idx: idxs) { + batchRes.push_back(batchGet->Get(tables[idx.first].Table() + "&/@").Apply( + [&attributes, &linkAttributes, &linksPresent, idx] (const TFuture<NYT::TNode>& res) { + try { + NYT::TNode attrs = res.GetValue(); + auto type = GetTypeFromAttributes(attrs, false); + if (type == "link") { + linkAttributes[idx.first] = attrs; + linksPresent.store(true); + } else { + attributes[idx.first] = attrs; + } + } catch (const TErrorResponse& e) { + // Yt returns NoSuchTransaction as inner issue for ResolveError + if (!e.IsResolveError() || e.IsNoSuchTransaction()) { + throw; + } + // Just ignore. Original table path may be deleted at this time + } + })); + } + batchGet->ExecuteBatch(); + WaitExceptionOrAll(batchRes).GetValue(); + } + if (linksPresent.load()) { + auto batchGet = tx->CreateBatchRequest(); + TVector<TFuture<void>> batchRes; + for (auto& idx : idxs) { + if (!linkAttributes[idx.first]) { + continue; + } + const auto& linkAttr = *linkAttributes[idx.first]; + batchRes.push_back(batchGet->Get(idx.second + "/@").Apply( + [idx, &linkAttr, &attributes](const TFuture<NYT::TNode>& f) { + NYT::TNode attrs = f.GetValue(); + attributes[idx.first] = attrs; + // override some attributes by the link ones + if (linkAttr.HasKey(QB2Premapper)) { + attributes[idx.first][QB2Premapper] = linkAttr[QB2Premapper]; + } + if (linkAttr.HasKey(YqlRowSpecAttribute)) { + attributes[idx.first][YqlRowSpecAttribute] = linkAttr[YqlRowSpecAttribute]; + } + })); + } + batchGet->ExecuteBatch(); + WaitExceptionOrAll(batchRes).GetValue(); + } + auto batchGet = tx->CreateBatchRequest(); + TVector<TFuture<void>> batchRes; + + TVector<std::pair<size_t, TString>> idxsToInferFromContent; + + for (auto& idx: idxs) { + try { + NYT::TNode& attrs = attributes[idx.first]; + + TYtTableMetaInfo::TPtr metaInfo = result.Data[idx.first].Meta; + TYtTableStatInfo::TPtr statInfo = MakeIntrusive<TYtTableStatInfo>(); + result.Data[idx.first].Stat = statInfo; + + auto type = GetTypeFromAttributes(attrs, false); + ui16 viewSyntaxVersion = 1; + if (type == "document") { + if (attrs.HasKey(YqlTypeAttribute)) { + auto typeAttr = attrs[YqlTypeAttribute]; + type = typeAttr.AsString(); + auto verAttr = typeAttr.Attributes()["syntax_version"]; + viewSyntaxVersion = verAttr.IsUndefined() ? 1 : verAttr.AsInt64(); + } + } + + if (type != "table" && type != "replicated_table" && type != YqlTypeView) { + YQL_LOG_CTX_THROW TErrorException(TIssuesIds::YT_ENTRY_NOT_TABLE_OR_VIEW) << "Input " << tables[idx.first].Table() << " is not a table or a view, got: " << type; + } + + statInfo->Id = attrs["id"].AsString(); + statInfo->TableRevision = attrs["revision"].IntCast<ui64>(); + statInfo->Revision = GetContentRevision(attrs); + + if (type == YqlTypeView) { + batchRes.push_back(batchGet->Get(idx.second).Apply([metaInfo, viewSyntaxVersion](const TFuture<NYT::TNode>& f) { + metaInfo->SqlView = f.GetValue().AsString(); + metaInfo->SqlViewSyntaxVersion = viewSyntaxVersion; + metaInfo->CanWrite = false; + })); + continue; + } + + bool isDynamic = attrs.AsMap().contains("dynamic") && NYT::GetBool(attrs["dynamic"]); + auto rowCount = attrs[isDynamic ? "chunk_row_count" : "row_count"].AsInt64(); + statInfo->RecordsCount = rowCount; + statInfo->DataSize = GetDataWeight(attrs).GetOrElse(0); + statInfo->ChunkCount = attrs["chunk_count"].AsInt64(); + TString strModifyTime = attrs["modification_time"].AsString(); + statInfo->ModifyTime = TInstant::ParseIso8601(strModifyTime).Seconds(); + metaInfo->IsDynamic = isDynamic; + if (statInfo->IsEmpty()) { + YQL_CLOG(INFO, ProviderYt) << "Empty table : " << tables[idx.first].Table() << ", modify time: " << strModifyTime << ", revision: " << statInfo->Revision; + } + + bool schemaValid = ValidateTableSchema( + tables[idx.first].Table(), attrs, + tables[idx.first].IgnoreYamrDsv(), tables[idx.first].IgnoreWeakSchema() + ); + + metaInfo->YqlCompatibleScheme = schemaValid; + + TransferTableAttributes(attrs, [metaInfo] (const TString& name, const TString& value) { + metaInfo->Attrs[name] = value; + }); + + if (attrs.AsMap().contains("erasure_codec") && attrs["erasure_codec"].AsString() != "none") { + metaInfo->Attrs["erasure_codec"] = attrs["erasure_codec"].AsString(); + } + if (attrs.AsMap().contains("optimize_for") && attrs["optimize_for"].AsString() != "scan") { + metaInfo->Attrs["optimize_for"] = attrs["optimize_for"].AsString(); + } + if (attrs.AsMap().contains("schema_mode") && attrs["schema_mode"].AsString() == "weak") { + metaInfo->Attrs["schema_mode"] = attrs["schema_mode"].AsString(); + } + if (attrs.AsMap().contains(SecurityTagsName)) { + TVector<TString> securityTags; + for (const auto& tag : attrs[SecurityTagsName].AsList()) { + securityTags.push_back(tag.AsString()); + } + statInfo->SecurityTags = {securityTags.begin(), securityTags.end()}; + } + + NYT::TNode schemaAttrs; + if (tables[idx.first].ForceInferSchema() && tables[idx.first].InferSchemaRows() > 0) { + metaInfo->Attrs.erase(YqlRowSpecAttribute); + if (isDynamic) { + schemaAttrs = GetSchemaFromAttributes(attrs, false, tables[idx.first].IgnoreWeakSchema()); + } else if (!statInfo->IsEmpty()) { + idxsToInferFromContent.push_back(idx); + } + } else { + if (attrs.HasKey(QB2Premapper)) { + metaInfo->Attrs[QB2Premapper] = NYT::NodeToYsonString(attrs[QB2Premapper], NYT::NYson::EYsonFormat::Text); + metaInfo->Attrs[TString{YqlRowSpecAttribute}.append("_qb2")] = NYT::NodeToYsonString( + QB2PremapperToRowSpec(attrs[QB2Premapper], attrs[SCHEMA_ATTR_NAME]), NYT::NYson::EYsonFormat::Text); + } + + if (schemaValid) { + schemaAttrs = GetSchemaFromAttributes(attrs, false, tables[idx.first].IgnoreWeakSchema()); + } else if (!attrs.HasKey(YqlRowSpecAttribute) && !isDynamic && tables[idx.first].InferSchemaRows() > 0 && !statInfo->IsEmpty()) { + idxsToInferFromContent.push_back(idx); + } + } + + if (!schemaAttrs.IsUndefined()) { + for (auto& item: schemaAttrs.AsMap()) { + metaInfo->Attrs[item.first] = NYT::NodeToYsonString(item.second, NYT::NYson::EYsonFormat::Text); + } + } + } catch (const TErrorException& e) { + throw; + } catch (...) { + throw yexception() << "Error loading '" << tables[idx.first].Table() << "' table metadata: " << CurrentExceptionMessage(); + } + } + if (batchRes) { + batchGet->ExecuteBatch(); + WaitExceptionOrAll(batchRes).GetValue(); + } + + if (idxsToInferFromContent) { + TString tmpFolder = GetTablesTmpFolder(*execCtx->Options_.Config()); + TString tmpTablePath = NYql::TransformPath(tmpFolder, + TStringBuilder() << "tmp/" << GetGuidAsString(execCtx->Session_->RandomProvider_->GenGuid()), true, execCtx->Session_->UserName_); + + auto inferResult = ExecInferSchema(tmpTablePath, execCtx, entry, tx, idxsToInferFromContent, tables); + for (size_t i : xrange(idxsToInferFromContent.size())) { + size_t idx = idxsToInferFromContent[i].first; + NYT::TNode& attrs = attributes[idx]; + TYtTableMetaInfo::TPtr metaInfo = result.Data[idx].Meta; + + if (auto inferSchema = inferResult[i]) { + NYT::TNode schemaAttrs; + if (tables[idx].ForceInferSchema()) { + schemaAttrs = GetSchemaFromAttributes(attrs, true, tables[idx].IgnoreWeakSchema()); + } + schemaAttrs[INFER_SCHEMA_ATTR_NAME] = *inferSchema; + for (auto& item: schemaAttrs.AsMap()) { + metaInfo->Attrs[item.first] = NYT::NodeToYsonString(item.second, NYT::NYson::EYsonFormat::Text); + } + metaInfo->InferredScheme = true; + } + } + } + } + + using TMaybeSchema = TMaybe<NYT::TNode>; + static TVector<TMaybeSchema> ExecInferSchema(const TString& tmpTable, + const TExecContext<TGetTableInfoOptions>::TPtr& execCtx, + const TTransactionCache::TEntry::TPtr& entry, + const ITransactionPtr& tx, + const TVector<std::pair<size_t, TString>>& idxs, + const TVector<TTableReq>& tables) + { + size_t jobThreshold = execCtx->Options_.Config()->InferSchemaTableCountThreshold.Get().GetOrElse(Max<ui32>()); + + TVector<TMaybeSchema> result; + if (idxs.size() <= jobThreshold) { + result.reserve(idxs.size()); + auto mode = execCtx->Options_.Config()->InferSchemaMode.Get().GetOrElse(EInferSchemaMode::Sequential); + if (EInferSchemaMode::Sequential == mode) { + for (auto& idx : idxs) { + YQL_ENSURE(tables[idx.first].InferSchemaRows() > 0); + result.push_back(InferSchemaFromTableContents(tx, idx.second, tables[idx.first].Table(), tables[idx.first].InferSchemaRows())); + } + return result; + } + if (EInferSchemaMode::RPC == mode) { +#ifdef __linux__ + std::vector<TTableInferSchemaRequest> requests; + requests.reserve(idxs.size()); + for (auto& idx : idxs) { + YQL_ENSURE(tables[idx.first].InferSchemaRows() > 0); + requests.push_back({idx.second, tables[idx.first].Table(), tables[idx.first].InferSchemaRows()}); + } + return InferSchemaFromTablesContents(execCtx->YtServer_, execCtx->GetAuth(), tx->GetId(), requests, execCtx->Session_->Queue_); +#else + ythrow yexception() << "Unimplemented RPC reader on non-linux platforms"; +#endif + } + result.resize(idxs.size()); + std::vector<NThreading::TFuture<void>> futures; + futures.reserve(idxs.size()); + size_t i = 0; + for (auto& idx : idxs) { + YQL_ENSURE(tables[idx.first].InferSchemaRows() > 0); + futures.push_back(execCtx->Session_->Queue_->Async([i, idx, &result, &tables, &tx](){ + YQL_CLOG(INFO, ProviderYt) << "Infering schema for table '" << tables[idx.first].Table() << "'"; + result[i] = InferSchemaFromTableContents(tx, idx.second, tables[idx.first].Table(), tables[idx.first].InferSchemaRows()); + })); + ++i; + } + (NThreading::WaitAll(futures)).Wait(); + return result; + } + + YQL_ENSURE(!idxs.empty()); + + TRawMapOperationSpec mapOpSpec; + mapOpSpec.Format(TFormat::YsonBinary()); + auto job = MakeIntrusive<TYqlInferSchemaJob>(); + + { + TUserJobSpec userJobSpec; + FillUserJobSpec(userJobSpec, execCtx, {}, 0, 0, false); + mapOpSpec.MapperSpec(userJobSpec); + } + + TVector<TString> inputTables; + for (auto& idx : idxs) { + YQL_ENSURE(tables[idx.first].InferSchemaRows() > 0); + inputTables.push_back(tables[idx.first].Table()); + auto path = NYT::TRichYPath(idx.second) + .AddRange(NYT::TReadRange::FromRowIndices(0, tables[idx.first].InferSchemaRows())); + mapOpSpec.AddInput(path); + } + mapOpSpec.AddOutput(tmpTable); + job->SetTableNames(inputTables); + + FillOperationSpec(mapOpSpec, execCtx); + + NYT::TNode spec = execCtx->Session_->CreateSpecWithDesc(execCtx->CodeSnippets_); + FillSpec(spec, *execCtx, entry, 0., Nothing(), EYtOpProp::WithMapper); + spec["job_count"] = 1; + + TOperationOptions opOpts; + FillOperationOptions(opOpts, execCtx, entry); + opOpts.StartOperationMode(TOperationOptions::EStartOperationMode::AsyncPrepare).Spec(spec); + + auto tmpTx = tx->StartTransaction(); + PrepareTempDestination(tmpTable, execCtx, entry, tmpTx); + + execCtx->RunOperation<false>([tmpTx, job, mapOpSpec = std::move(mapOpSpec), opOpts = std::move(opOpts)](){ + return tmpTx->RawMap(mapOpSpec, job, opOpts); + }).GetValueSync(); + + result.resize(idxs.size()); + auto reader = tmpTx->CreateTableReader<NYT::TNode>(tmpTable); + for (; reader->IsValid(); reader->Next()) { + auto& row = reader->GetRow(); + size_t tableIdx = row["index"].AsUint64(); + YQL_ENSURE(tableIdx < idxs.size()); + + auto schema = NYT::NodeFromYsonString(row["schema"].AsString()); + if (schema.IsString()) { + YQL_LOG_CTX_THROW yexception() << schema.AsString(); + } + result[tableIdx] = schema; + } + reader.Drop(); + tmpTx->Abort(); + return result; + } + + static std::pair<TString, NYT::TNode> ParseYTType(const TExprNode& node, + TExprContext& ctx, + const TExecContext<TResOrPullOptions>::TPtr& execCtx, + const TMaybe<NYql::TColumnOrder>& columns = Nothing()) + { + const auto sequenceItemType = GetSequenceItemType(node.Pos(), node.GetTypeAnn(), false, ctx); + + auto rowSpecInfo = MakeIntrusive<TYqlRowSpecInfo>(); + rowSpecInfo->SetType(sequenceItemType->Cast<TStructExprType>(), execCtx->Options_.Config()->UseNativeYtTypes.Get().GetOrElse(DEFAULT_USE_NATIVE_YT_TYPES) ? NTCF_ALL : NTCF_NONE); + if (columns) { + rowSpecInfo->SetColumnOrder(columns); + } + + NYT::TNode tableSpec = NYT::TNode::CreateMap(); + rowSpecInfo->FillCodecNode(tableSpec[YqlRowSpecAttribute]); + + auto resultYTType = NodeToYsonString(RowSpecToYTSchema(tableSpec[YqlRowSpecAttribute], execCtx->Options_.Config()->NativeYtTypeCompatibility.Get(execCtx->Cluster_).GetOrElse(NTCF_LEGACY)).ToNode()); + auto resultRowSpec = NYT::TNode::CreateMap()(TString{YqlIOSpecTables}, NYT::TNode::CreateList().Add(tableSpec)); + return {resultYTType, resultRowSpec}; + } + + TFuture<TResOrPullResult> DoPull(const TSession::TPtr& session, NNodes::TPull pull, TExprContext& ctx, TResOrPullOptions&& options) { + if (options.FillSettings().Discard) { + TResOrPullResult res; + res.SetSuccess(); + return MakeFuture(res); + } + TVector<TString> columns(NCommon::GetResOrPullColumnHints(pull.Ref())); + if (columns.empty()) { + columns = NCommon::GetStructFields(pull.Input().Ref().GetTypeAnn()); + } + + bool ref = NCommon::HasResOrPullOption(pull.Ref(), "ref"); + bool autoRef = NCommon::HasResOrPullOption(pull.Ref(), "autoref"); + + auto cluster = TString{GetClusterName(pull.Input())}; + auto execCtx = MakeExecCtx(std::move(options), session, cluster, pull.Raw(), &ctx); + + if (auto read = pull.Input().Maybe<TCoRight>().Input().Maybe<TYtReadTable>()) { + execCtx->SetInput(read.Cast().Input(), false, {}); + } else { + execCtx->SetInput(pull.Input(), false, {}); + } + + TRecordsRange range; + if (!ref) { + if (auto read = pull.Input().Maybe<TCoRight>().Input().Maybe<TYtReadTable>()) { + YQL_ENSURE(read.Cast().Input().Size() == 1); + range.Fill(read.Cast().Input().Item(0).Settings().Ref()); + } + } + + TString type; + NYT::TNode rowSpec; + if (execCtx->Options_.FillSettings().Format == IDataProvider::EResultFormat::Skiff) { + auto ytType = ParseYTType(pull.Input().Ref(), ctx, execCtx, TColumnOrder(columns)); + + type = ytType.first; + rowSpec = ytType.second; + } else if (NCommon::HasResOrPullOption(pull.Ref(), "type")) { + TStringStream typeYson; + ::NYson::TYsonWriter typeWriter(&typeYson); + NCommon::WriteResOrPullType(typeWriter, pull.Input().Ref().GetTypeAnn(), TColumnOrder(columns)); + type = typeYson.Str(); + } + + auto pos = ctx.GetPosition(pull.Pos()); + + return session->Queue_->Async([rowSpec, type, ref, range, autoRef, execCtx, columns, pos] () { + YQL_LOG_CTX_ROOT_SESSION_SCOPE(execCtx->LogCtx_); + execCtx->MakeUserFiles(); + try { + TResOrPullResult res; + TStringStream out; + + auto fillSettings = execCtx->Options_.FillSettings(); + fillSettings.Format = IDataProvider::EResultFormat::Yson; + + ::NYson::TYsonWriter writer(&out, NCommon::GetYsonFormat(fillSettings), ::NYson::EYsonType::Node, false); + writer.OnBeginMap(); + + if (type) { + writer.OnKeyedItem("Type"); + writer.OnRaw(type); + } + + bool truncated = false; + if (!ref) { + truncated = ExecPull(execCtx, writer, range, rowSpec, columns); + } + + if (ref || (truncated && autoRef)) { + writer.OnKeyedItem("Ref"); + writer.OnBeginList(); + TVector<TString> keepTables; + for (auto& table: execCtx->InputTables_) { + writer.OnListItem(); + if (table.Temp) { + keepTables.push_back(table.Name); + } + NYql::WriteTableReference(writer, YtProviderName, execCtx->Cluster_, table.Name, table.Temp, columns); + } + writer.OnEndList(); + if (!keepTables.empty()) { + auto entry = execCtx->GetEntry(); + // TODO: check anonymous tables + entry->CancelDeleteAtFinalize(keepTables); + } + } + + if (truncated) { + writer.OnKeyedItem("Truncated"); + writer.OnBooleanScalar(true); + } + + writer.OnEndMap(); + res.Data = out.Str(); + res.SetSuccess(); + + return res; + } catch (...) { + return ResultFromCurrentException<TResOrPullResult>(pos); + } + }); + } + + static bool ExecPull(const TExecContext<TResOrPullOptions>::TPtr& execCtx, + ::NYson::TYsonWriter& writer, + const TRecordsRange& range, + const NYT::TNode& rowSpec, + const TVector<TString>& columns) + { + TScopedAlloc alloc(__LOCATION__, NKikimr::TAlignedPagePoolCounters(), + execCtx->FunctionRegistry_->SupportsSizedAllocators()); + alloc.SetLimit(execCtx->Options_.Config()->DefaultCalcMemoryLimit.Get().GetOrElse(0)); + TMemoryUsageInfo memInfo("Pull"); + TTypeEnvironment env(alloc); + THolderFactory holderFactory(alloc.Ref(), memInfo, execCtx->FunctionRegistry_); + NCommon::TCodecContext codecCtx(env, *execCtx->FunctionRegistry_, &holderFactory); + + bool useSkiff = execCtx->Options_.Config()->UseSkiff.Get(execCtx->Cluster_).GetOrElse(DEFAULT_USE_SKIFF); + + const bool testRun = execCtx->Config_->GetLocalChainTest(); + + TVector<TString> tables; + for (const TInputInfo& table: execCtx->InputTables_) { + auto tablePath = table.Path; + tables.push_back(table.Temp ? TString() : table.Name); + } + + TMkqlIOSpecs specs; + if (useSkiff) { + specs.SetUseSkiff(execCtx->Options_.OptLLVM(), testRun ? TMkqlIOSpecs::ESystemField(0) : TMkqlIOSpecs::ESystemField::RangeIndex | TMkqlIOSpecs::ESystemField::RowIndex); + } + const auto nativeTypeCompat = execCtx->Options_.Config()->NativeYtTypeCompatibility.Get(execCtx->Cluster_).GetOrElse(NTCF_LEGACY); + specs.Init(codecCtx, execCtx->GetInputSpec(!useSkiff, nativeTypeCompat, false), tables, columns); + + auto run = [&] (IExecuteResOrPull& pullData) { + TMkqlIOCache specsCache(specs, holderFactory); + + if (testRun) { + YQL_ENSURE(execCtx->InputTables_.size() == 1U, "Support single input only."); + const auto itI = TestTables.find(execCtx->InputTables_.front().Path.Path_); + YQL_ENSURE(TestTables.cend() != itI); + + TMkqlInput input(MakeStringInput(std::move(itI->second.second), false)); + TMkqlReaderImpl reader(input, 0, 4 << 10, 0); + reader.SetSpecs(specs, holderFactory); + for (reader.Next(); reader.IsValid(); reader.Next()) { + if (!pullData.WriteNext(specsCache, reader.GetRow(), 0)) { + return true; + } + } + } else if (auto limiter = TTableLimiter(range)) { + auto entry = execCtx->GetEntry(); + bool stop = false; + for (size_t i = 0; i < execCtx->InputTables_.size(); ++i) { + TString srcTableName = execCtx->InputTables_[i].Name; + NYT::TRichYPath srcTable = execCtx->InputTables_[i].Path; + bool isDynamic = execCtx->InputTables_[i].Dynamic; + ui64 recordsCount = execCtx->InputTables_[i].Records; + if (!isDynamic) { + if (!limiter.NextTable(recordsCount)) { + continue; + } + } else { + limiter.NextDynamicTable(); + } + + if (isDynamic) { + YQL_ENSURE(srcTable.GetRanges().Empty()); + stop = NYql::SelectRows(entry->Client, srcTableName, i, specsCache, pullData, limiter); + } else { + auto readTx = entry->Tx; + if (srcTable.TransactionId_) { + readTx = entry->GetSnapshotTx(*srcTable.TransactionId_); + srcTable.TransactionId_.Clear(); + } + if (execCtx->YamrInput) { + stop = NYql::IterateYamredRows(readTx, srcTable, i, specsCache, pullData, limiter, execCtx->Sampling); + } else { + stop = NYql::IterateYsonRows(readTx, srcTable, i, specsCache, pullData, limiter, execCtx->Sampling); + } + } + if (stop || limiter.Exceed()) { + break; + } + } + } + return false; + }; + + switch (execCtx->Options_.FillSettings().Format) { + case IDataProvider::EResultFormat::Yson: { + TYsonExecuteResOrPull pullData(execCtx->Options_.FillSettings().RowsLimitPerWrite, + execCtx->Options_.FillSettings().AllResultsBytesLimit, MakeMaybe(columns)); + + if (run(pullData)) { + return true; + } + specs.Clear(); + + writer.OnKeyedItem("Data"); + writer.OnBeginList(); + writer.OnRaw(pullData.Finish(), ::NYson::EYsonType::ListFragment); + writer.OnEndList(); + return pullData.IsTruncated(); + } + case IDataProvider::EResultFormat::Skiff: { + THashMap<TString, ui32> structColumns; + for (size_t index = 0; index < columns.size(); index++) { + structColumns.emplace(columns[index], index); + } + + auto skiffNode = TablesSpecToOutputSkiff(rowSpec); + + writer.OnKeyedItem("SkiffType"); + writer.OnRaw(NodeToYsonString(skiffNode), ::NYson::EYsonType::Node); + + writer.OnKeyedItem("Columns"); + writer.OnBeginList(); + for (auto& column : columns) { + writer.OnListItem(); + writer.OnStringScalar(column); + } + writer.OnEndList(); + + TSkiffExecuteResOrPull pullData(execCtx->Options_.FillSettings().RowsLimitPerWrite, + execCtx->Options_.FillSettings().AllResultsBytesLimit, + codecCtx, + holderFactory, + rowSpec, + execCtx->Options_.OptLLVM(), + columns); + + if (run(pullData)) { + return true; + } + specs.Clear(); + + writer.OnKeyedItem("Data"); + writer.OnStringScalar(pullData.Finish()); + + return pullData.IsTruncated(); + } + default: + YQL_LOG_CTX_THROW yexception() << "Invalid result type: " << execCtx->Options_.FillSettings().Format; + } + } + + TFuture<TResOrPullResult> DoResult(const TSession::TPtr& session, NNodes::TResult result, TExprContext& ctx, TResOrPullOptions&& options) { + TVector<TString> columns(NCommon::GetResOrPullColumnHints(result.Ref())); + if (columns.empty()) { + columns = NCommon::GetStructFields(result.Input().Ref().GetTypeAnn()); + } + + TString lambda; + bool hasListResult = false; + { + TScopedAlloc alloc(__LOCATION__, NKikimr::TAlignedPagePoolCounters(), + Services_.FunctionRegistry->SupportsSizedAllocators()); + alloc.SetLimit(options.Config()->DefaultCalcMemoryLimit.Get().GetOrElse(0)); + TNativeYtLambdaBuilder builder(alloc, Services_, *session); + auto rootNode = builder.BuildLambda(*MkqlCompiler_, result.Input().Ptr(), ctx); + hasListResult = rootNode.GetStaticType()->IsList(); + lambda = SerializeRuntimeNode(rootNode, builder.GetTypeEnvironment()); + } + + auto extraUsage = ScanExtraResourceUsage(result.Input().Ref(), *options.Config()); + + TString cluster = options.UsedCluster(); + if (cluster.empty()) { + cluster = options.Config()->DefaultCluster.Get().GetOrElse(TString()); + } + if (cluster.empty()) { + cluster = Clusters_->GetDefaultClusterName(); + } + TString tmpFolder = GetTablesTmpFolder(*options.Config()); + TString tmpTablePath = NYql::TransformPath(tmpFolder, + TStringBuilder() << "tmp/" << GetGuidAsString(session->RandomProvider_->GenGuid()), true, session->UserName_); + bool discard = options.FillSettings().Discard; + auto execCtx = MakeExecCtx(std::move(options), session, cluster, result.Input().Raw(), &ctx); + auto pos = ctx.GetPosition(result.Pos()); + + TString type, skiffType; + NYT::TNode rowSpec; + if (execCtx->Options_.FillSettings().Format == IDataProvider::EResultFormat::Skiff) { + auto ytType = ParseYTType(result.Input().Ref(), ctx, execCtx); + + type = ytType.first; + rowSpec = ytType.second; + skiffType = NodeToYsonString(TablesSpecToOutputSkiff(rowSpec)); + } else if (NCommon::HasResOrPullOption(result.Ref(), "type")) { + TStringStream typeYson; + ::NYson::TYsonWriter typeWriter(&typeYson); + NCommon::WriteResOrPullType(typeWriter, result.Input().Ref().GetTypeAnn(), TColumnOrder(columns)); + type = typeYson.Str(); + } + + return session->Queue_->Async([lambda, hasListResult, extraUsage, tmpTablePath, execCtx, columns, rowSpec] () { + YQL_LOG_CTX_ROOT_SESSION_SCOPE(execCtx->LogCtx_); + execCtx->MakeUserFiles(); + + switch (execCtx->Options_.FillSettings().Format) { + case IDataProvider::EResultFormat::Yson: + return ExecCalc(lambda, extraUsage, tmpTablePath, execCtx, {}, + TYsonExprResultFactory(execCtx->Options_.FillSettings().RowsLimitPerWrite, + execCtx->Options_.FillSettings().AllResultsBytesLimit, + columns, + hasListResult), + &columns, + execCtx->Options_.FillSettings().Format); + case IDataProvider::EResultFormat::Skiff: + return ExecCalc(lambda, extraUsage, tmpTablePath, execCtx, {}, + TSkiffExprResultFactory(execCtx->Options_.FillSettings().RowsLimitPerWrite, + execCtx->Options_.FillSettings().AllResultsBytesLimit, + hasListResult, + rowSpec, + execCtx->Options_.OptLLVM(), + columns), + &columns, + execCtx->Options_.FillSettings().Format); + default: + YQL_LOG_CTX_THROW yexception() << "Invalid result type: " << execCtx->Options_.FillSettings().Format; + } + }) + .Apply([skiffType, type, execCtx, discard, pos, columns] (const TFuture<std::pair<TString, bool>>& f) { + YQL_LOG_CTX_ROOT_SESSION_SCOPE(execCtx->LogCtx_); + try { + const std::pair<TString, bool>& value = f.GetValue(); + + TResOrPullResult res; + TStringStream out; + + auto fillSettings = execCtx->Options_.FillSettings(); + fillSettings.Format = IDataProvider::EResultFormat::Yson; + + ::NYson::TYsonWriter writer(discard ? (IOutputStream*)&Cnull : (IOutputStream*)&out, NCommon::GetYsonFormat(fillSettings), ::NYson::EYsonType::Node, true); + writer.OnBeginMap(); + + if (skiffType) { + writer.OnKeyedItem("SkiffType"); + writer.OnRaw(skiffType, ::NYson::EYsonType::Node); + + + writer.OnKeyedItem("Columns"); + writer.OnBeginList(); + for (auto& column: columns) { + writer.OnListItem(); + writer.OnStringScalar(column); + } + writer.OnEndList(); + } + + if (type) { + writer.OnKeyedItem("Type"); + writer.OnRaw(type); + } + + writer.OnKeyedItem("Data"); + switch (execCtx->Options_.FillSettings().Format) { + case IDataProvider::EResultFormat::Yson: + writer.OnRaw(value.first); + break; + case IDataProvider::EResultFormat::Skiff: + writer.OnStringScalar(value.first); + break; + default: + YQL_LOG_CTX_THROW yexception() << "Invalid result type: " << execCtx->Options_.FillSettings().Format; + } + + if (value.second) { + writer.OnKeyedItem("Truncated"); + writer.OnBooleanScalar(true); + } + + writer.OnEndMap(); + if (!discard) { + res.Data = out.Str(); + } + res.SetSuccess(); + + return res; + } catch (...) { + return ResultFromCurrentException<TResOrPullResult>(pos); + } + }); + } + + TFuture<void> DoSort(TYtSort /*sort*/, const TExecContext<TRunOptions>::TPtr& execCtx) { + YQL_ENSURE(execCtx->OutTables_.size() == 1); + + return execCtx->Session_->Queue_->Async([execCtx]() { + return execCtx->LookupQueryCacheAsync().Apply([execCtx] (const auto& f) { + YQL_LOG_CTX_ROOT_SESSION_SCOPE(execCtx->LogCtx_); + auto entry = execCtx->GetEntry(); + bool cacheHit = f.GetValue(); + TVector<TRichYPath> outYPaths = PrepareDestinations(execCtx->OutTables_, execCtx, entry, !cacheHit); + if (cacheHit) { + execCtx->QueryCacheItem.Destroy(); + return MakeFuture(); + } + + bool hasNonStrict = false; + TSortOperationSpec sortOpSpec; + for (const auto& table: execCtx->InputTables_) { + if (!table.Strict) { + hasNonStrict = true; + } + sortOpSpec.AddInput(table.Path); + } + + sortOpSpec.Output(outYPaths.front()); + sortOpSpec.SortBy(execCtx->OutTables_.front().SortedBy); + sortOpSpec.SchemaInferenceMode(ESchemaInferenceMode::FromOutput); + + NYT::TNode spec = execCtx->Session_->CreateSpecWithDesc(execCtx->CodeSnippets_); + + FillSpec(spec, *execCtx, entry, 0., Nothing(), EYtOpProp::IntermediateData); + if (hasNonStrict) { + spec["schema_inference_mode"] = "from_output"; // YTADMINREQ-17692 + } + + return execCtx->RunOperation([entry, sortOpSpec = std::move(sortOpSpec), spec = std::move(spec)](){ + return entry->Tx->Sort(sortOpSpec, TOperationOptions().StartOperationMode(TOperationOptions::EStartOperationMode::AsyncPrepare).Spec(spec)); + }); + }); + }); + } + + TFuture<void> DoCopy(TYtCopy /*copy*/, const TExecContext<TRunOptions>::TPtr& execCtx) { + YQL_ENSURE(execCtx->InputTables_.size() == 1); + YQL_ENSURE(execCtx->InputTables_.front().Temp); + YQL_ENSURE(execCtx->OutTables_.size() == 1); + + return execCtx->Session_->Queue_->Async([execCtx]() { + YQL_LOG_CTX_ROOT_SESSION_SCOPE(execCtx->LogCtx_); + auto entry = execCtx->GetEntry(); + execCtx->QueryCacheItem.Destroy(); // Don't use cache for YtCopy + TOutputInfo& out = execCtx->OutTables_.front(); + + entry->DeleteAtFinalize(out.Path); + + entry->CreateDefaultTmpFolder(); + CreateParents({out.Path}, entry->CacheTx); + entry->Tx->Copy(execCtx->InputTables_.front().Name, out.Path, TCopyOptions().Force(true)); + + }); + } + + TFuture<void> DoMerge(TYtMerge merge, const TExecContext<TRunOptions>::TPtr& execCtx) { + YQL_ENSURE(execCtx->OutTables_.size() == 1); + bool forceTransform = NYql::HasAnySetting(merge.Settings().Ref(), EYtSettingType::ForceTransform | EYtSettingType::TransformColGroups); + bool combineChunks = NYql::HasSetting(merge.Settings().Ref(), EYtSettingType::CombineChunks); + TMaybe<ui64> limit = GetLimit(merge.Settings().Ref()); + + return execCtx->Session_->Queue_->Async([forceTransform, combineChunks, limit, execCtx]() { + return execCtx->LookupQueryCacheAsync().Apply([forceTransform, combineChunks, limit, execCtx] (const auto& f) { + YQL_LOG_CTX_ROOT_SESSION_SCOPE(execCtx->LogCtx_); + auto entry = execCtx->GetEntry(); + bool cacheHit = f.GetValue(); + TVector<TRichYPath> outYPaths = PrepareDestinations(execCtx->OutTables_, execCtx, entry, !cacheHit); + if (cacheHit) { + execCtx->QueryCacheItem.Destroy(); + return MakeFuture(); + } + + bool hasNonStrict = false; + TMergeOperationSpec mergeOpSpec; + for (const auto& table: execCtx->InputTables_) { + if (!table.Strict) { + hasNonStrict = true; + } + mergeOpSpec.AddInput(table.Path); + } + + if (execCtx->OutTables_.front().SortedBy.Parts_.empty()) { + mergeOpSpec.Mode(EMergeMode::MM_ORDERED); + if (limit) { + outYPaths.front().RowCountLimit(*limit); + } + } else { + mergeOpSpec.Mode(EMergeMode::MM_SORTED); + mergeOpSpec.MergeBy(execCtx->OutTables_.front().SortedBy); + } + + mergeOpSpec.Output(outYPaths.front()); + + mergeOpSpec.ForceTransform(forceTransform); + mergeOpSpec.CombineChunks(combineChunks); + mergeOpSpec.SchemaInferenceMode(ESchemaInferenceMode::FromOutput); + + NYT::TNode spec = execCtx->Session_->CreateSpecWithDesc(execCtx->CodeSnippets_); + EYtOpProps flags = EYtOpProp::AllowSampling; + if (combineChunks) { + flags |= EYtOpProp::TemporaryChunkCombine; + } + FillSpec(spec, *execCtx, entry, 0., Nothing(), flags); + if (hasNonStrict) { + spec["schema_inference_mode"] = "from_output"; // YTADMINREQ-17692 + } + + return execCtx->RunOperation([entry, mergeOpSpec = std::move(mergeOpSpec), spec = std::move(spec)](){ + return entry->Tx->Merge(mergeOpSpec, TOperationOptions().StartOperationMode(TOperationOptions::EStartOperationMode::AsyncPrepare).Spec(spec)); + }); + }); + }); + } + + static TFuture<void> ExecMap( + bool ordered, + bool blockInput, + const TMaybe<ui64>& jobCount, + const TMaybe<ui64>& limit, + const TVector<TString>& sortLimitBy, + TString mapLambda, + const TString& inputType, + const TExpressionResorceUsage& extraUsage, + const TExecContext<TRunOptions>::TPtr& execCtx + ) { + const bool testRun = execCtx->Config_->GetLocalChainTest(); + TFuture<bool> ret = testRun ? MakeFuture<bool>(false) : execCtx->LookupQueryCacheAsync(); + return ret.Apply([ordered, blockInput, jobCount, limit, sortLimitBy, mapLambda, + inputType, extraUsage, execCtx, testRun] (const auto& f) mutable + { + YQL_LOG_CTX_ROOT_SESSION_SCOPE(execCtx->LogCtx_); + TTransactionCache::TEntry::TPtr entry; + TVector<TRichYPath> outYPaths; + if (testRun) { + YQL_ENSURE(execCtx->OutTables_.size() == 1U, "Test mode support single output only."); + const auto& out = execCtx->OutTables_.front(); + outYPaths.emplace_back(TRichYPath(out.Path).Schema(RowSpecToYTSchema(TestTables[out.Path].first = out.Spec[YqlRowSpecAttribute], NTCF_NONE))); + } else { + entry = execCtx->GetEntry(); + bool cacheHit = f.GetValue(); + outYPaths = PrepareDestinations(execCtx->OutTables_, execCtx, entry, !cacheHit); + if (cacheHit) { + execCtx->QueryCacheItem.Destroy(); + return MakeFuture(); + } + } + + TRawMapOperationSpec mapOpSpec; + auto job = MakeIntrusive<TYqlUserJob>(); + + job->SetInputType(inputType); + + for (size_t i: xrange(execCtx->OutTables_.size())) { + if (!execCtx->OutTables_[i].SortedBy.Parts_.empty()) { + mapOpSpec.Ordered(true); + } + else if (limit && sortLimitBy.empty()) { + outYPaths[i].RowCountLimit(*limit); + } + mapOpSpec.AddOutput(outYPaths[i]); + } + + TVector<ui32> groups; + TVector<TString> tables; + TVector<ui64> rowOffsets; + ui64 currentRowOffset = 0; + TSet<TString> remapperAllFiles; + TRemapperMap remapperMap; + + bool useSkiff = execCtx->Options_.Config()->UseSkiff.Get(execCtx->Cluster_).GetOrElse(DEFAULT_USE_SKIFF); + bool hasTablesWithoutQB2Premapper = false; + + for (const TInputInfo& table: execCtx->InputTables_) { + auto tablePath = table.Path; + if (!table.QB2Premapper.IsUndefined()) { + bool tableUseSkiff = false; + + ProcessTableQB2Premapper(table.QB2Premapper, table.Name, tablePath, mapOpSpec.GetInputs().size(), + remapperMap, remapperAllFiles, tableUseSkiff); + + useSkiff = useSkiff && tableUseSkiff; + } + else { + hasTablesWithoutQB2Premapper = true; + } + + if (!groups.empty() && groups.back() != table.Group) { + currentRowOffset = 0; + } + + mapOpSpec.AddInput(tablePath); + groups.push_back(table.Group); + tables.push_back(table.Temp ? TString() : table.Name); + rowOffsets.push_back(currentRowOffset); + currentRowOffset += table.Records; + } + + bool forceYsonInputFormat = false; + + if (useSkiff && !remapperMap.empty()) { + // Disable skiff in case of mix of QB2 and normal tables + if (hasTablesWithoutQB2Premapper) { + useSkiff = false; + } else { + UpdateQB2PremapperUseSkiff(remapperMap, useSkiff); + forceYsonInputFormat = useSkiff; + } + } + + const auto nativeTypeCompat = execCtx->Options_.Config()->NativeYtTypeCompatibility.Get(execCtx->Cluster_).GetOrElse(NTCF_LEGACY); + job->SetInputSpec(execCtx->GetInputSpec(!useSkiff || forceYsonInputFormat, nativeTypeCompat, false)); + job->SetOutSpec(execCtx->GetOutSpec(!useSkiff, nativeTypeCompat)); + if (!groups.empty() && groups.back() != 0) { + job->SetInputGroups(groups); + } + job->SetTableNames(tables); + job->SetRowOffsets(rowOffsets); + + if (ordered) { + mapOpSpec.Ordered(true); + } + + job->SetYamrInput(execCtx->YamrInput); + job->SetUseSkiff(useSkiff, testRun ? TMkqlIOSpecs::ESystemField(0) : TMkqlIOSpecs::ESystemField::RowIndex); + job->SetUseBlockInput(blockInput); + + auto tmpFiles = std::make_shared<TTempFiles>(execCtx->FileStorage_->GetTemp()); + { + TUserJobSpec userJobSpec; + TScopedAlloc alloc(__LOCATION__, NKikimr::TAlignedPagePoolCounters(), + execCtx->FunctionRegistry_->SupportsSizedAllocators()); + alloc.SetLimit(execCtx->Options_.Config()->DefaultCalcMemoryLimit.Get().GetOrElse(0)); + TNativeYtLambdaBuilder builder(alloc, execCtx->FunctionRegistry_, *execCtx->Session_); + TProgramBuilder pgmBuilder(builder.GetTypeEnvironment(), *execCtx->FunctionRegistry_); + TGatewayTransformer transform(execCtx, entry, pgmBuilder, *tmpFiles); + size_t nodeCount = 0; + builder.UpdateLambdaCode(mapLambda, nodeCount, transform); + if (nodeCount > execCtx->Options_.Config()->LLVMNodeCountLimit.Get(execCtx->Cluster_).GetOrElse(DEFAULT_LLVM_NODE_COUNT_LIMIT)) { + execCtx->Options_.OptLLVM("OFF"); + } + job->SetLambdaCode(mapLambda); + job->SetOptLLVM(execCtx->Options_.OptLLVM()); + job->SetUdfValidateMode(execCtx->Options_.UdfValidateMode()); + transform.ApplyJobProps(*job); + transform.ApplyUserJobSpec(userJobSpec, testRun); + + ui64 fileMemUsage = transform.GetUsedMemory(); + for (auto& f: remapperAllFiles) { + fileMemUsage += GetUncompressedFileSize(entry->Tx, f).GetOrElse(i64(1) << 10); + userJobSpec.AddFile(TRichYPath(f).Executable(true)); + } + if (!remapperMap.empty()) { + fileMemUsage += 512_MB; + } + + FillUserJobSpec(userJobSpec, execCtx, extraUsage, fileMemUsage, execCtx->EstimateLLVMMem(nodeCount), testRun, + GetQB2PremapperPrefix(remapperMap, useSkiff)); + + mapOpSpec.MapperSpec(userJobSpec); + } + FillOperationSpec(mapOpSpec, execCtx); + auto formats = job->GetIOFormats(execCtx->FunctionRegistry_); + mapOpSpec.InputFormat(forceYsonInputFormat ? NYT::TFormat::YsonBinary() : formats.first); + mapOpSpec.OutputFormat(formats.second); + + if (testRun) { + YQL_ENSURE(execCtx->InputTables_.size() == 1U, "Support single input only."); + YQL_ENSURE(execCtx->OutTables_.size() == 1U, "Support single output only."); + + const auto itI = TestTables.find(execCtx->InputTables_.front().Path.Path_); + YQL_ENSURE(TestTables.cend() != itI); + const auto itO = TestTables.find(execCtx->OutTables_.front().Path); + YQL_ENSURE(TestTables.cend() != itO); + + TStringInput in(itI->second.second); + TStringOutput out(itO->second.second); + + LocalRawMapReduce(mapOpSpec, job.Get(), &in, &out); + DumpLocalTable(itO->second.second, execCtx->Config_->GetLocalChainFile()); + return MakeFuture(); + } + + NYT::TNode spec = execCtx->Session_->CreateSpecWithDesc(execCtx->CodeSnippets_); + FillSpec(spec, *execCtx, entry, extraUsage.Cpu, Nothing(), + EYtOpProp::TemporaryAutoMerge | EYtOpProp::WithMapper | EYtOpProp::WithUserJobs | EYtOpProp::AllowSampling); + + if (jobCount) { + spec["job_count"] = static_cast<i64>(*jobCount); + } + + TOperationOptions opOpts; + FillOperationOptions(opOpts, execCtx, entry); + opOpts.StartOperationMode(TOperationOptions::EStartOperationMode::AsyncPrepare).Spec(spec); + + return execCtx->RunOperation([entry, execCtx, job, mapOpSpec = std::move(mapOpSpec), opOpts = std::move(opOpts), tmpFiles]() { + execCtx->SetNodeExecProgress("Uploading artifacts"); + return entry->Tx->RawMap(mapOpSpec, job, opOpts); + }); + }); + } + + TFuture<void> DoMap(TYtMap map, const TExecContext<TRunOptions>::TPtr& execCtx, TExprContext& ctx) { + const bool ordered = NYql::HasSetting(map.Settings().Ref(), EYtSettingType::Ordered); + const bool blockInput = NYql::HasSetting(map.Settings().Ref(), EYtSettingType::BlockInputApplied); + + TMaybe<ui64> jobCount; + if (auto setting = NYql::GetSetting(map.Settings().Ref(), EYtSettingType::JobCount)) { + jobCount = FromString<ui64>(setting->Child(1)->Content()); + } + + TString mapLambda; + { + TScopedAlloc alloc(__LOCATION__, NKikimr::TAlignedPagePoolCounters(), + execCtx->FunctionRegistry_->SupportsSizedAllocators()); + alloc.SetLimit(execCtx->Options_.Config()->DefaultCalcMemoryLimit.Get().GetOrElse(0)); + TNativeYtLambdaBuilder builder(alloc, Services_, *execCtx->Session_); + mapLambda = builder.BuildLambdaWithIO(*MkqlCompiler_, map.Mapper(), ctx); + } + + TVector<TString> sortLimitBy = NYql::GetSettingAsColumnList(map.Settings().Ref(), EYtSettingType::SortLimitBy); + TMaybe<ui64> limit = GetLimit(map.Settings().Ref()); + if (limit && !sortLimitBy.empty() && *limit > execCtx->Options_.Config()->TopSortMaxLimit.Get().GetOrElse(DEFAULT_TOP_SORT_LIMIT)) { + limit.Clear(); + } + auto extraUsage = execCtx->ScanExtraResourceUsage(map.Mapper().Body().Ref(), true); + TString inputType = NCommon::WriteTypeToYson(GetSequenceItemType(map.Input().Size() == 1U ? TExprBase(map.Input().Item(0)) : TExprBase(map.Mapper().Args().Arg(0)), true)); + + return execCtx->Session_->Queue_->Async([ordered, blockInput, jobCount, limit, sortLimitBy, mapLambda, inputType, extraUsage, execCtx]() { + YQL_LOG_CTX_ROOT_SESSION_SCOPE(execCtx->LogCtx_); + execCtx->MakeUserFiles(); + return ExecMap(ordered, blockInput, jobCount, limit, sortLimitBy, mapLambda, inputType, extraUsage, execCtx); + }); + } + + static TFuture<void> ExecReduce(const TVector<std::pair<TString, bool>>& reduceBy, + const TVector<std::pair<TString, bool>>& sortBy, + bool joinReduce, + const TMaybe<ui64>& maxDataSizePerJob, + bool useFirstAsPrimary, + const TMaybe<ui64>& limit, + const TVector<TString>& sortLimitBy, + TString reduceLambda, + const TString& inputType, + const TExpressionResorceUsage& extraUsage, + const TExecContext<TRunOptions>::TPtr& execCtx + ) { + const bool testRun = execCtx->Config_->GetLocalChainTest(); + TFuture<bool> ret = testRun ? MakeFuture<bool>(false) : execCtx->LookupQueryCacheAsync(); + return ret.Apply([reduceBy, sortBy, joinReduce, maxDataSizePerJob, useFirstAsPrimary, limit, + sortLimitBy, reduceLambda, inputType, extraUsage, execCtx, testRun] + (const auto& f) mutable + { + YQL_LOG_CTX_ROOT_SESSION_SCOPE(execCtx->LogCtx_); + TTransactionCache::TEntry::TPtr entry; + TVector<TRichYPath> outYPaths; + if (testRun) { + YQL_ENSURE(execCtx->OutTables_.size() == 1U, "Test mode support single output only."); + const auto& out = execCtx->OutTables_.front(); + outYPaths.emplace_back(TRichYPath(out.Path).Schema(RowSpecToYTSchema(TestTables[out.Path].first = out.Spec[YqlRowSpecAttribute], NTCF_NONE))); + } else { + entry = execCtx->GetEntry(); + const bool cacheHit = f.GetValue(); + outYPaths = PrepareDestinations(execCtx->OutTables_, execCtx, entry, !cacheHit); + if (cacheHit) { + execCtx->QueryCacheItem.Destroy(); + return MakeFuture(); + } + } + + TRawReduceOperationSpec reduceOpSpec; + auto job = MakeIntrusive<TYqlUserJob>(); + + job->SetInputType(inputType); + + for (size_t i: xrange(execCtx->OutTables_.size())) { + if (limit && sortLimitBy.empty()) { + outYPaths[i].RowCountLimit(*limit); + } + reduceOpSpec.AddOutput(outYPaths[i]); + } + + TVector<ui32> groups; + TVector<TString> tables; + TVector<ui64> rowOffsets; + ui64 currentRowOffset = 0; + YQL_ENSURE(!execCtx->InputTables_.empty()); + const ui32 primaryGroup = useFirstAsPrimary ? execCtx->InputTables_.front().Group : execCtx->InputTables_.back().Group; + for (const auto& table : execCtx->InputTables_) { + if (joinReduce) { + auto yPath = table.Path; + if (table.Group == primaryGroup) { + yPath.Primary(true); + } else { + yPath.Foreign(true); + } + reduceOpSpec.AddInput(yPath); + } else { + reduceOpSpec.AddInput(table.Path); + } + if (!groups.empty() && groups.back() != table.Group) { + currentRowOffset = 0; + } + + groups.push_back(table.Group); + tables.push_back(table.Temp ? TString() : table.Name); + rowOffsets.push_back(currentRowOffset); + currentRowOffset += table.Records; + } + + const bool useSkiff = execCtx->Options_.Config()->UseSkiff.Get(execCtx->Cluster_).GetOrElse(DEFAULT_USE_SKIFF); + + const auto nativeTypeCompat = execCtx->Options_.Config()->NativeYtTypeCompatibility.Get(execCtx->Cluster_).GetOrElse(NTCF_LEGACY); + job->SetInputSpec(execCtx->GetInputSpec(!useSkiff, nativeTypeCompat, false)); + job->SetOutSpec(execCtx->GetOutSpec(!useSkiff, nativeTypeCompat)); + YQL_ENSURE(!groups.empty()); + if (groups.back() != 0) { + job->SetInputGroups(groups); + } + job->SetTableNames(tables); + job->SetRowOffsets(rowOffsets); + + if (joinReduce) { + reduceOpSpec.JoinBy(ToYTSortColumns(reduceBy)); + reduceOpSpec.EnableKeyGuarantee(false); + } else { + reduceOpSpec.ReduceBy(ToYTSortColumns(reduceBy)); + } + + if (!sortBy.empty()) { + reduceOpSpec.SortBy(ToYTSortColumns(sortBy)); + } else { + reduceOpSpec.SortBy(ToYTSortColumns(reduceBy)); + } + + THashSet<TString> auxColumns; + std::for_each(reduceBy.begin(), reduceBy.end(), [&auxColumns](const auto& it) { auxColumns.insert(it.first); }); + if (!sortBy.empty()) { + std::for_each(sortBy.begin(), sortBy.end(), [&auxColumns](const auto& it) { auxColumns.insert(it.first); }); + } + job->SetAuxColumns(auxColumns); + + job->SetUseSkiff(useSkiff, TMkqlIOSpecs::ESystemField::RowIndex | TMkqlIOSpecs::ESystemField::KeySwitch); + job->SetYamrInput(execCtx->YamrInput); + + auto tmpFiles = std::make_shared<TTempFiles>(execCtx->FileStorage_->GetTemp()); + { + TUserJobSpec userJobSpec; + TScopedAlloc alloc(__LOCATION__, NKikimr::TAlignedPagePoolCounters(), + execCtx->FunctionRegistry_->SupportsSizedAllocators()); + alloc.SetLimit(execCtx->Options_.Config()->DefaultCalcMemoryLimit.Get().GetOrElse(0)); + TNativeYtLambdaBuilder builder(alloc, execCtx->FunctionRegistry_, *execCtx->Session_); + TProgramBuilder pgmBuilder(builder.GetTypeEnvironment(), *execCtx->FunctionRegistry_); + TGatewayTransformer transform(execCtx, entry, pgmBuilder, *tmpFiles); + size_t nodeCount = 0; + builder.UpdateLambdaCode(reduceLambda, nodeCount, transform); + if (nodeCount > execCtx->Options_.Config()->LLVMNodeCountLimit.Get(execCtx->Cluster_).GetOrElse(DEFAULT_LLVM_NODE_COUNT_LIMIT)) { + execCtx->Options_.OptLLVM("OFF"); + } + job->SetLambdaCode(reduceLambda); + job->SetOptLLVM(execCtx->Options_.OptLLVM()); + job->SetUdfValidateMode(execCtx->Options_.UdfValidateMode()); + transform.ApplyJobProps(*job); + transform.ApplyUserJobSpec(userJobSpec, testRun); + FillUserJobSpec(userJobSpec, execCtx, extraUsage, transform.GetUsedMemory(), execCtx->EstimateLLVMMem(nodeCount), testRun); + reduceOpSpec.ReducerSpec(userJobSpec); + } + FillOperationSpec(reduceOpSpec, execCtx); + auto formats = job->GetIOFormats(execCtx->FunctionRegistry_); + reduceOpSpec.InputFormat(formats.first); + reduceOpSpec.OutputFormat(formats.second); + + if (testRun) { + YQL_ENSURE(execCtx->InputTables_.size() == 1U, "Support single input only."); + YQL_ENSURE(execCtx->OutTables_.size() == 1U, "Support single output only."); + + const auto itI = TestTables.find(execCtx->InputTables_.front().Path.Path_); + YQL_ENSURE(TestTables.cend() != itI); + const auto itO = TestTables.find(execCtx->OutTables_.front().Path); + YQL_ENSURE(TestTables.cend() != itO); + + TStringInput in(itI->second.second); + TStringOutput out(itO->second.second); + + LocalRawMapReduce(reduceOpSpec, job.Get(), &in, &out); + DumpLocalTable(itO->second.second, execCtx->Config_->GetLocalChainFile()); + return MakeFuture(); + } + + NYT::TNode spec = execCtx->Session_->CreateSpecWithDesc(execCtx->CodeSnippets_); + FillSpec(spec, *execCtx, entry, extraUsage.Cpu, Nothing(), + EYtOpProp::TemporaryAutoMerge | EYtOpProp::WithReducer | EYtOpProp::WithUserJobs | EYtOpProp::AllowSampling); + + if (maxDataSizePerJob) { + spec["max_data_size_per_job"] = static_cast<i64>(*maxDataSizePerJob); + } + + TOperationOptions opOpts; + FillOperationOptions(opOpts, execCtx, entry); + opOpts.StartOperationMode(TOperationOptions::EStartOperationMode::AsyncPrepare).Spec(spec); + + return execCtx->RunOperation([entry, execCtx, job, reduceOpSpec = std::move(reduceOpSpec), opOpts = std::move(opOpts), tmpFiles]() { + execCtx->SetNodeExecProgress("Uploading artifacts"); + return entry->Tx->RawReduce(reduceOpSpec, job, opOpts); + }); + }); + } + + TFuture<void> DoReduce(TYtReduce reduce, const TExecContext<TRunOptions>::TPtr &execCtx, TExprContext& ctx) { + auto reduceBy = NYql::GetSettingAsColumnPairList(reduce.Settings().Ref(), EYtSettingType::ReduceBy); + auto sortBy = NYql::GetSettingAsColumnPairList(reduce.Settings().Ref(), EYtSettingType::SortBy); + bool joinReduce = NYql::HasSetting(reduce.Settings().Ref(), EYtSettingType::JoinReduce); + auto maxDataSizePerJob = NYql::GetMaxJobSizeForFirstAsPrimary(reduce.Settings().Ref()); + bool useFirstAsPrimary = NYql::HasSetting(reduce.Settings().Ref(), EYtSettingType::FirstAsPrimary); + + TString reduceLambda; + { + TScopedAlloc alloc(__LOCATION__, NKikimr::TAlignedPagePoolCounters(), + execCtx->FunctionRegistry_->SupportsSizedAllocators()); + alloc.SetLimit(execCtx->Options_.Config()->DefaultCalcMemoryLimit.Get().GetOrElse(0)); + TNativeYtLambdaBuilder builder(alloc, Services_, *execCtx->Session_); + reduceLambda = builder.BuildLambdaWithIO(*MkqlCompiler_, reduce.Reducer(), ctx); + } + + TVector<TString> sortLimitBy = NYql::GetSettingAsColumnList(reduce.Settings().Ref(), EYtSettingType::SortLimitBy); + TMaybe<ui64> limit = GetLimit(reduce.Settings().Ref()); + if (limit && !sortLimitBy.empty() && *limit > execCtx->Options_.Config()->TopSortMaxLimit.Get().GetOrElse(DEFAULT_TOP_SORT_LIMIT)) { + limit.Clear(); + } + auto extraUsage = execCtx->ScanExtraResourceUsage(reduce.Reducer().Body().Ref(), true); + const auto inputTypeSet = NYql::GetSetting(reduce.Settings().Ref(), EYtSettingType::ReduceInputType); + TString inputType = NCommon::WriteTypeToYson(inputTypeSet + ? inputTypeSet->Tail().GetTypeAnn()->Cast<TTypeExprType>()->GetType() + : GetSequenceItemType(reduce.Reducer().Args().Arg(0), true) + ); + + return execCtx->Session_->Queue_->Async([reduceBy, sortBy, joinReduce, maxDataSizePerJob, useFirstAsPrimary, limit, sortLimitBy, reduceLambda, inputType, extraUsage, execCtx]() { + YQL_LOG_CTX_ROOT_SESSION_SCOPE(execCtx->LogCtx_); + execCtx->MakeUserFiles(); + return ExecReduce(reduceBy, sortBy, joinReduce, maxDataSizePerJob, useFirstAsPrimary, limit, + sortLimitBy, reduceLambda, inputType, extraUsage, execCtx); + }); + } + + static TFuture<void> ExecMapReduce( + const TVector<std::pair<TString, bool>>& reduceBy, + const TVector<std::pair<TString, bool>>& sortBy, + const TMaybe<ui64>& limit, + const TVector<TString>& sortLimitBy, + TString mapLambda, + const TString& mapInputType, + size_t mapDirectOutputs, + const TExpressionResorceUsage& mapExtraUsage, + TString reduceLambda, + const TString& reduceInputType, + const TExpressionResorceUsage& reduceExtraUsage, + NYT::TNode intermediateMeta, + const NYT::TNode& intermediateSchema, + const NYT::TNode& intermediateStreams, + const TExecContext<TRunOptions>::TPtr& execCtx + ) { + const bool testRun = execCtx->Config_->GetLocalChainTest(); + TFuture<bool> ret = testRun ? MakeFuture<bool>(false) : execCtx->LookupQueryCacheAsync(); + return ret.Apply([reduceBy, sortBy, limit, sortLimitBy, mapLambda, mapInputType, mapDirectOutputs, + mapExtraUsage, reduceLambda, reduceInputType, reduceExtraUsage, + intermediateMeta, intermediateSchema, intermediateStreams, execCtx, testRun] + (const auto& f) mutable + { + YQL_LOG_CTX_ROOT_SESSION_SCOPE(execCtx->LogCtx_); + TTransactionCache::TEntry::TPtr entry; + TVector<TRichYPath> outYPaths; + + if (testRun) { + YQL_ENSURE(execCtx->OutTables_.size() == 1U, "Test mode support single output only."); + const auto& out = execCtx->OutTables_.front(); + outYPaths.emplace_back(TRichYPath(out.Path).Schema(RowSpecToYTSchema(TestTables[out.Path].first = out.Spec[YqlRowSpecAttribute], NTCF_NONE))); + } else { + entry = execCtx->GetEntry(); + const bool cacheHit = f.GetValue(); + outYPaths = PrepareDestinations(execCtx->OutTables_, execCtx, entry, !cacheHit); + if (cacheHit) { + execCtx->QueryCacheItem.Destroy(); + return MakeFuture(); + } + } + + TRawMapReduceOperationSpec mapReduceOpSpec; + YQL_ENSURE(mapDirectOutputs < execCtx->OutTables_.size()); + + for (size_t i: xrange(execCtx->OutTables_.size())) { + if (limit && sortLimitBy.empty()) { + outYPaths[i].RowCountLimit(*limit); + } + if (i < mapDirectOutputs) { + mapReduceOpSpec.AddMapOutput(outYPaths[i]); + } else { + mapReduceOpSpec.AddOutput(outYPaths[i]); + } + } + + bool useSkiff = execCtx->Options_.Config()->UseSkiff.Get(execCtx->Cluster_).GetOrElse(DEFAULT_USE_SKIFF); + const bool reduceUseSkiff = useSkiff; + bool hasTablesWithoutQB2Premapper = false; + + TVector<ui32> groups; + TVector<TString> tables; + TVector<ui64> rowOffsets; + ui64 currentRowOffset = 0; + TSet<TString> remapperAllFiles; + TRemapperMap remapperMap; + for (auto& table: execCtx->InputTables_) { + auto tablePath = table.Path; + if (!table.QB2Premapper.IsUndefined()) { + bool tableUseSkiff = false; + + ProcessTableQB2Premapper(table.QB2Premapper, table.Name, tablePath, mapReduceOpSpec.GetInputs().size(), + remapperMap, remapperAllFiles, tableUseSkiff); + + useSkiff = useSkiff && tableUseSkiff; + } + else { + hasTablesWithoutQB2Premapper = true; + } + if (!groups.empty() && groups.back() != table.Group) { + currentRowOffset = 0; + } + + mapReduceOpSpec.AddInput(tablePath); + groups.push_back(table.Group); + tables.push_back(table.Temp ? TString() : table.Name); + rowOffsets.push_back(currentRowOffset); + currentRowOffset += table.Records; + } + + bool forceYsonInputFormat = false; + + if (useSkiff && !remapperMap.empty()) { + // Disable skiff in case of mix of QB2 and normal tables + if (hasTablesWithoutQB2Premapper) { + useSkiff = false; + } else { + UpdateQB2PremapperUseSkiff(remapperMap, useSkiff); + forceYsonInputFormat = useSkiff; + } + } + + NYT::TNode mapSpec = intermediateMeta; + mapSpec.AsMap().erase(YqlSysColumnPrefix); + + const auto nativeTypeCompat = execCtx->Options_.Config()->NativeYtTypeCompatibility.Get(execCtx->Cluster_).GetOrElse(NTCF_LEGACY); + + NYT::TNode mapOutSpec = NYT::TNode::CreateMap(); + mapOutSpec[YqlIOSpecTables] = NYT::TNode::CreateList(); + mapOutSpec[YqlIOSpecTables].Add(mapSpec); + TString mapOutSpecStr; + if (mapDirectOutputs) { + mapOutSpecStr = execCtx->GetOutSpec(0, mapDirectOutputs, mapOutSpec, !reduceUseSkiff, nativeTypeCompat); + } else { + mapOutSpecStr = NYT::NodeToYsonString(mapOutSpec); + } + + auto mapJob = MakeIntrusive<TYqlUserJob>(); + mapJob->SetInputType(mapInputType); + mapJob->SetInputSpec(execCtx->GetInputSpec(!useSkiff || forceYsonInputFormat, nativeTypeCompat, false)); + mapJob->SetOutSpec(mapOutSpecStr); + if (!groups.empty() && groups.back() != 0) { + mapJob->SetInputGroups(groups); + } + mapJob->SetTableNames(tables); + mapJob->SetRowOffsets(rowOffsets); + mapJob->SetUseSkiff(useSkiff, TMkqlIOSpecs::ESystemField::RowIndex); + mapJob->SetYamrInput(execCtx->YamrInput); + + auto reduceJob = MakeIntrusive<TYqlUserJob>(); + reduceJob->SetInputType(reduceInputType); + reduceJob->SetInputSpec(NYT::NodeToYsonString(NYT::TNode::CreateMap()(TString{YqlIOSpecTables}, NYT::TNode::CreateList().Add(intermediateMeta)))); + reduceJob->SetOutSpec(execCtx->GetOutSpec(mapDirectOutputs, execCtx->OutTables_.size(), {}, !reduceUseSkiff, nativeTypeCompat)); + + mapReduceOpSpec.ReduceBy(ToYTSortColumns(reduceBy)); + if (!sortBy.empty()) { + mapReduceOpSpec.SortBy(ToYTSortColumns(sortBy)); + } else { + mapReduceOpSpec.SortBy(ToYTSortColumns(reduceBy)); + } + + THashSet<TString> auxColumns; + std::for_each(reduceBy.begin(), reduceBy.end(), [&auxColumns](const auto& it) { auxColumns.insert(it.first); }); + if (!sortBy.empty()) { + std::for_each(sortBy.begin(), sortBy.end(), [&auxColumns](const auto& it) { auxColumns.insert(it.first); }); + } + reduceJob->SetAuxColumns(auxColumns); + + reduceJob->SetUseSkiff(reduceUseSkiff, TMkqlIOSpecs::ESystemField::KeySwitch); + + auto tmpFiles = std::make_shared<TTempFiles>(execCtx->FileStorage_->GetTemp()); + { + TUserJobSpec mapUserJobSpec; + TScopedAlloc alloc(__LOCATION__, NKikimr::TAlignedPagePoolCounters(), + execCtx->FunctionRegistry_->SupportsSizedAllocators()); + alloc.SetLimit(execCtx->Options_.Config()->DefaultCalcMemoryLimit.Get().GetOrElse(0)); + TNativeYtLambdaBuilder builder(alloc, execCtx->FunctionRegistry_, *execCtx->Session_); + TProgramBuilder pgmBuilder(builder.GetTypeEnvironment(), *execCtx->FunctionRegistry_); + TGatewayTransformer transform(execCtx, entry, pgmBuilder, *tmpFiles); + size_t nodeCount = 0; + builder.UpdateLambdaCode(mapLambda, nodeCount, transform); + if (nodeCount > execCtx->Options_.Config()->LLVMNodeCountLimit.Get(execCtx->Cluster_).GetOrElse(DEFAULT_LLVM_NODE_COUNT_LIMIT)) { + execCtx->Options_.OptLLVM("OFF"); + } + mapJob->SetLambdaCode(mapLambda); + mapJob->SetOptLLVM(execCtx->Options_.OptLLVM()); + mapJob->SetUdfValidateMode(execCtx->Options_.UdfValidateMode()); + transform.ApplyJobProps(*mapJob); + transform.ApplyUserJobSpec(mapUserJobSpec, testRun); + + for (auto& f: remapperAllFiles) { + mapUserJobSpec.AddFile(TRichYPath(f).Executable(true)); + } + + FillUserJobSpec(mapUserJobSpec, execCtx, mapExtraUsage, transform.GetUsedMemory(), execCtx->EstimateLLVMMem(nodeCount), testRun, + GetQB2PremapperPrefix(remapperMap, useSkiff)); + + mapReduceOpSpec.MapperSpec(mapUserJobSpec); + } + + { + TUserJobSpec reduceUserJobSpec; + TScopedAlloc alloc(__LOCATION__, NKikimr::TAlignedPagePoolCounters(), + execCtx->FunctionRegistry_->SupportsSizedAllocators()); + alloc.SetLimit(execCtx->Options_.Config()->DefaultCalcMemoryLimit.Get().GetOrElse(0)); + TNativeYtLambdaBuilder builder(alloc, execCtx->FunctionRegistry_, *execCtx->Session_); + TProgramBuilder pgmBuilder(builder.GetTypeEnvironment(), *execCtx->FunctionRegistry_); + TGatewayTransformer transform(execCtx, entry, pgmBuilder, *tmpFiles); + size_t nodeCount = 0; + builder.UpdateLambdaCode(reduceLambda, nodeCount, transform); + if (nodeCount > execCtx->Options_.Config()->LLVMNodeCountLimit.Get(execCtx->Cluster_).GetOrElse(DEFAULT_LLVM_NODE_COUNT_LIMIT)) { + execCtx->Options_.OptLLVM("OFF"); + } + reduceJob->SetLambdaCode(reduceLambda); + reduceJob->SetOptLLVM(execCtx->Options_.OptLLVM()); + reduceJob->SetUdfValidateMode(execCtx->Options_.UdfValidateMode()); + transform.ApplyJobProps(*reduceJob); + transform.ApplyUserJobSpec(reduceUserJobSpec, testRun); + FillUserJobSpec(reduceUserJobSpec, execCtx, reduceExtraUsage, transform.GetUsedMemory(), execCtx->EstimateLLVMMem(nodeCount), testRun); + mapReduceOpSpec.ReducerSpec(reduceUserJobSpec); + } + FillOperationSpec(mapReduceOpSpec, execCtx); + auto formats = mapJob->GetIOFormats(execCtx->FunctionRegistry_); + if (!intermediateSchema.IsUndefined() && formats.second.Config.AsString() == "skiff") { + formats.second.Config.Attributes()["override_intermediate_table_schema"] = intermediateSchema; + } + mapReduceOpSpec.MapperInputFormat(forceYsonInputFormat ? NYT::TFormat::YsonBinary() : formats.first); + mapReduceOpSpec.MapperOutputFormat(formats.second); + formats = reduceJob->GetIOFormats(execCtx->FunctionRegistry_); + if (!intermediateSchema.IsUndefined() && formats.first.Config.AsString() == "skiff") { + formats.first.Config.Attributes()["override_intermediate_table_schema"] = intermediateSchema; + } + mapReduceOpSpec.ReducerInputFormat(formats.first); + mapReduceOpSpec.ReducerOutputFormat(formats.second); + + if (testRun) { + YQL_ENSURE(execCtx->InputTables_.size() == 1U, "Support single input only."); + YQL_ENSURE(execCtx->OutTables_.size() == 1U, "Support single output only."); + + const auto itI = TestTables.find(execCtx->InputTables_.front().Path.Path_); + YQL_ENSURE(TestTables.cend() != itI); + const auto itO = TestTables.find(execCtx->OutTables_.front().Path); + YQL_ENSURE(TestTables.cend() != itO); + + TStringInput in(itI->second.second); + TStringOutput out(itO->second.second); + + LocalRawMapReduce(mapReduceOpSpec, reduceJob.Get(), &in, &out); + DumpLocalTable(itO->second.second, execCtx->Config_->GetLocalChainFile()); + return MakeFuture(); + } + + NYT::TNode spec = execCtx->Session_->CreateSpecWithDesc(execCtx->CodeSnippets_); + FillSpec(spec, *execCtx, entry, mapExtraUsage.Cpu, reduceExtraUsage.Cpu, + EYtOpProp::IntermediateData | EYtOpProp::WithMapper | EYtOpProp::WithReducer | EYtOpProp::WithUserJobs | EYtOpProp::AllowSampling); + if (!intermediateStreams.IsUndefined()) { + spec["mapper"]["output_streams"] = intermediateStreams; + } + + TOperationOptions opOpts; + FillOperationOptions(opOpts, execCtx, entry); + opOpts.StartOperationMode(TOperationOptions::EStartOperationMode::AsyncPrepare).Spec(spec); + + return execCtx->RunOperation([entry, execCtx, mapJob, reduceJob, mapReduceOpSpec = std::move(mapReduceOpSpec), opOpts = std::move(opOpts), tmpFiles]() { + execCtx->SetNodeExecProgress("Uploading artifacts"); + return entry->Tx->RawMapReduce(mapReduceOpSpec, mapJob, {}, reduceJob, opOpts); + }); + }); + } + + static TFuture<void> ExecMapReduce( + const TVector<std::pair<TString, bool>>& reduceBy, + const TVector<std::pair<TString, bool>>& sortBy, + const TMaybe<ui64>& limit, + const TVector<TString>& sortLimitBy, + TString reduceLambda, + const TString& reduceInputType, + const TExpressionResorceUsage& reduceExtraUsage, + const NYT::TNode& intermediateSchema, + bool useIntermediateStreams, + const TExecContext<TRunOptions>::TPtr& execCtx + ) { + const bool testRun = execCtx->Config_->GetLocalChainTest(); + TFuture<bool> ret = testRun ? MakeFuture<bool>(false) : execCtx->LookupQueryCacheAsync(); + return ret.Apply([reduceBy, sortBy, limit, sortLimitBy, reduceLambda, reduceInputType, + reduceExtraUsage, intermediateSchema, useIntermediateStreams, execCtx, testRun] + (const auto& f) mutable + { + YQL_LOG_CTX_ROOT_SESSION_SCOPE(execCtx->LogCtx_); + TTransactionCache::TEntry::TPtr entry; + TVector<TRichYPath> outYPaths; + if (testRun) { + YQL_ENSURE(execCtx->OutTables_.size() == 1U, "Test mode support single output only."); + const auto& out = execCtx->OutTables_.front(); + outYPaths.emplace_back(TRichYPath(out.Path).Schema(RowSpecToYTSchema(TestTables[out.Path].first = out.Spec[YqlRowSpecAttribute], NTCF_NONE))); + } else { + entry = execCtx->GetEntry(); + const bool cacheHit = f.GetValue(); + outYPaths = PrepareDestinations(execCtx->OutTables_, execCtx, entry, !cacheHit); + if (cacheHit) { + execCtx->QueryCacheItem.Destroy(); + return MakeFuture(); + } + } + + TRawMapReduceOperationSpec mapReduceOpSpec; + + for (size_t i: xrange(execCtx->OutTables_.size())) { + if (limit && sortLimitBy.empty()) { + outYPaths[i].RowCountLimit(*limit); + } + mapReduceOpSpec.AddOutput(outYPaths[i]); + } + + TVector<ui32> groups; + for (auto& table: execCtx->InputTables_) { + mapReduceOpSpec.AddInput(table.Path); + groups.push_back(table.Group); + } + + auto reduceJob = MakeIntrusive<TYqlUserJob>(); + reduceJob->SetInputType(reduceInputType); + if (!groups.empty() && groups.back() != 0) { + reduceJob->SetInputGroups(groups); + } + + const bool useSkiff = execCtx->Options_.Config()->UseSkiff.Get(execCtx->Cluster_).GetOrElse(DEFAULT_USE_SKIFF); + + const auto nativeTypeCompat = execCtx->Options_.Config()->NativeYtTypeCompatibility.Get(execCtx->Cluster_).GetOrElse(NTCF_LEGACY); + reduceJob->SetInputSpec(execCtx->GetInputSpec(!useSkiff, nativeTypeCompat, !useIntermediateStreams)); + reduceJob->SetOutSpec(execCtx->GetOutSpec(!useSkiff, nativeTypeCompat)); + + mapReduceOpSpec.ReduceBy(ToYTSortColumns(reduceBy)); + if (!sortBy.empty()) { + mapReduceOpSpec.SortBy(ToYTSortColumns(sortBy)); + } else { + mapReduceOpSpec.SortBy(ToYTSortColumns(reduceBy)); + } + + THashSet<TString> auxColumns; + std::for_each(reduceBy.begin(), reduceBy.end(), [&auxColumns](const auto& it) { auxColumns.insert(it.first); }); + if (!sortBy.empty()) { + std::for_each(sortBy.begin(), sortBy.end(), [&auxColumns](const auto& it) { auxColumns.insert(it.first); }); + } + reduceJob->SetAuxColumns(auxColumns); + + reduceJob->SetUseSkiff(useSkiff, TMkqlIOSpecs::ESystemField::KeySwitch); + reduceJob->SetYamrInput(execCtx->YamrInput); + + auto tmpFiles = std::make_shared<TTempFiles>(execCtx->FileStorage_->GetTemp()); + { + TUserJobSpec reduceUserJobSpec; + TScopedAlloc alloc(__LOCATION__, NKikimr::TAlignedPagePoolCounters(), + execCtx->FunctionRegistry_->SupportsSizedAllocators()); + alloc.SetLimit(execCtx->Options_.Config()->DefaultCalcMemoryLimit.Get().GetOrElse(0)); + TNativeYtLambdaBuilder builder(alloc, execCtx->FunctionRegistry_, *execCtx->Session_); + TProgramBuilder pgmBuilder(builder.GetTypeEnvironment(), *execCtx->FunctionRegistry_); + TGatewayTransformer transform(execCtx, entry, pgmBuilder, *tmpFiles); + size_t nodeCount = 0; + builder.UpdateLambdaCode(reduceLambda, nodeCount, transform); + if (nodeCount > execCtx->Options_.Config()->LLVMNodeCountLimit.Get(execCtx->Cluster_).GetOrElse(DEFAULT_LLVM_NODE_COUNT_LIMIT)) { + execCtx->Options_.OptLLVM("OFF"); + } + reduceJob->SetLambdaCode(reduceLambda); + reduceJob->SetOptLLVM(execCtx->Options_.OptLLVM()); + reduceJob->SetUdfValidateMode(execCtx->Options_.UdfValidateMode()); + transform.ApplyJobProps(*reduceJob); + transform.ApplyUserJobSpec(reduceUserJobSpec, testRun); + FillUserJobSpec(reduceUserJobSpec, execCtx, reduceExtraUsage, transform.GetUsedMemory(), execCtx->EstimateLLVMMem(nodeCount), testRun); + mapReduceOpSpec.ReducerSpec(reduceUserJobSpec); + } + FillOperationSpec(mapReduceOpSpec, execCtx); + auto formats = reduceJob->GetIOFormats(execCtx->FunctionRegistry_); + if (!intermediateSchema.IsUndefined() && formats.first.Config.AsString() == "skiff") { + formats.first.Config.Attributes()["override_intermediate_table_schema"] = intermediateSchema; + } + mapReduceOpSpec.ReducerInputFormat(formats.first); + mapReduceOpSpec.ReducerOutputFormat(formats.second); + + if (testRun) { + YQL_ENSURE(execCtx->InputTables_.size() == 1U, "Support single input only."); + YQL_ENSURE(execCtx->OutTables_.size() == 1U, "Support single output only."); + + const auto itI = TestTables.find(execCtx->InputTables_.front().Path.Path_); + YQL_ENSURE(TestTables.cend() != itI); + const auto itO = TestTables.find(execCtx->OutTables_.front().Path); + YQL_ENSURE(TestTables.cend() != itO); + + TStringInput in(itI->second.second); + TStringOutput out(itO->second.second); + + LocalRawMapReduce(mapReduceOpSpec, reduceJob.Get(), &in, &out); + DumpLocalTable(itO->second.second, execCtx->Config_->GetLocalChainFile()); + return MakeFuture(); + } + + NYT::TNode spec = execCtx->Session_->CreateSpecWithDesc(execCtx->CodeSnippets_); + FillSpec(spec, *execCtx, entry, 0., reduceExtraUsage.Cpu, + EYtOpProp::IntermediateData | EYtOpProp::WithReducer | EYtOpProp::WithUserJobs | EYtOpProp::AllowSampling); + if (useIntermediateStreams) { + spec["reducer"]["enable_input_table_index"] = true; + } + + TOperationOptions opOpts; + FillOperationOptions(opOpts, execCtx, entry); + opOpts.StartOperationMode(TOperationOptions::EStartOperationMode::AsyncPrepare).Spec(spec); + + return execCtx->RunOperation([entry, execCtx, reduceJob, mapReduceOpSpec = std::move(mapReduceOpSpec), opOpts = std::move(opOpts), tmpFiles]() { + execCtx->SetNodeExecProgress("Uploading artifacts"); + return entry->Tx->RawMapReduce(mapReduceOpSpec, {}, {}, reduceJob, opOpts); + }); + }); + } + + TFuture<void> DoMapReduce(TYtMapReduce mapReduce, const TExecContext<TRunOptions>::TPtr& execCtx, TExprContext& ctx) { + auto reduceBy = NYql::GetSettingAsColumnPairList(mapReduce.Settings().Ref(), EYtSettingType::ReduceBy); + auto sortBy = NYql::GetSettingAsColumnPairList(mapReduce.Settings().Ref(), EYtSettingType::SortBy); + + const bool useNativeTypes = execCtx->Options_.Config()->UseNativeYtTypes.Get().GetOrElse(DEFAULT_USE_NATIVE_YT_TYPES); + const auto nativeTypeCompat = execCtx->Options_.Config()->NativeYtTypeCompatibility.Get(execCtx->Cluster_).GetOrElse(NTCF_LEGACY); + const auto useIntermediateStreams = execCtx->Options_.Config()->UseIntermediateStreams.Get().GetOrElse(DEFAULT_USE_INTERMEDIATE_STREAMS); + + NYT::TNode intermediateMeta; + NYT::TNode intermediateSchema; + NYT::TNode intermediateStreams; + TString mapLambda; + TExpressionResorceUsage mapExtraUsage; + TString mapInputType; + size_t mapDirectOutputs = 0; + if (!mapReduce.Mapper().Maybe<TCoVoid>()) { + auto createRowSpec = [](const TTypeAnnotationNode* itemType, bool useNativeTypes, ui64 nativeTypeCompat) -> NYT::TNode { + auto spec = NYT::TNode::CreateMap(); + spec[RowSpecAttrType] = NCommon::TypeToYsonNode(itemType); + spec[RowSpecAttrNativeYtTypeFlags] = useNativeTypes + ? (GetNativeYtTypeFlags(*itemType->Cast<TStructExprType>()) & nativeTypeCompat) + : 0ul; + return spec; + }; + + const auto mapTypeSet = NYql::GetSetting(mapReduce.Settings().Ref(), EYtSettingType::MapOutputType); + auto mapResultItem = mapTypeSet ? + mapTypeSet->Tail().GetTypeAnn()->Cast<TTypeExprType>()->GetType(): + GetSequenceItemType(mapReduce.Mapper(), true); + + if (mapResultItem->GetKind() == ETypeAnnotationKind::Variant) { + auto items = mapResultItem->Cast<TVariantExprType>()->GetUnderlyingType()->Cast<TTupleExprType>()->GetItems(); + YQL_ENSURE(!items.empty()); + mapDirectOutputs = items.size() - 1; + mapResultItem = items.front(); + + if (useIntermediateStreams) { + intermediateStreams = NYT::TNode::CreateList(); + bool front = true; + for (auto itemType: items) { + auto ytSchema = RowSpecToYTSchema(createRowSpec(itemType, useNativeTypes, nativeTypeCompat), nativeTypeCompat); + if (front) { + ytSchema.SortBy(ToYTSortColumns(sortBy.empty() ? reduceBy : sortBy)); + front = false; + } + intermediateStreams.Add( + NYT::TNode::CreateMap()("schema", ytSchema.ToNode()) + ); + } + } + } else { + if (useIntermediateStreams) { + intermediateStreams = NYT::TNode::CreateList(); + auto ytSchema = RowSpecToYTSchema(createRowSpec(mapResultItem, useNativeTypes, nativeTypeCompat), nativeTypeCompat); + ytSchema.SortBy(ToYTSortColumns(sortBy.empty() ? reduceBy : sortBy)); + intermediateStreams.Add( + NYT::TNode::CreateMap()("schema", ytSchema.ToNode()) + ); + } + } + + intermediateMeta = NYT::TNode::CreateMap(); + intermediateMeta[YqlRowSpecAttribute] = createRowSpec(mapResultItem, useNativeTypes, nativeTypeCompat); + if (useNativeTypes && !useIntermediateStreams) { + intermediateSchema = RowSpecToYTSchema(intermediateMeta[YqlRowSpecAttribute], nativeTypeCompat).ToNode(); + } + if (NYql::HasSetting(mapReduce.Settings().Ref(), EYtSettingType::KeySwitch)) { + intermediateMeta[YqlSysColumnPrefix].Add("keyswitch"); + } + mapExtraUsage = execCtx->ScanExtraResourceUsage(mapReduce.Mapper().Cast<TCoLambda>().Body().Ref(), true); + + TScopedAlloc alloc(__LOCATION__, NKikimr::TAlignedPagePoolCounters(), + execCtx->FunctionRegistry_->SupportsSizedAllocators()); + alloc.SetLimit(execCtx->Options_.Config()->DefaultCalcMemoryLimit.Get().GetOrElse(0)); + TNativeYtLambdaBuilder builder(alloc, Services_, *execCtx->Session_); + mapLambda = builder.BuildLambdaWithIO(*MkqlCompiler_, mapReduce.Mapper().Cast<TCoLambda>(), ctx); + mapInputType = NCommon::WriteTypeToYson(GetSequenceItemType(mapReduce.Input().Size() == 1U ? + TExprBase(mapReduce.Input().Item(0)) : TExprBase(mapReduce.Mapper().Cast<TCoLambda>().Args().Arg(0)), true)); + } else if (useNativeTypes && !useIntermediateStreams) { + YQL_ENSURE(mapReduce.Input().Size() == 1); + const TTypeAnnotationNode* itemType = GetSequenceItemType(mapReduce.Input().Item(0), false); + if (auto flags = GetNativeYtTypeFlags(*itemType->Cast<TStructExprType>())) { + auto rowSpec = NYT::TNode::CreateMap(); + rowSpec[RowSpecAttrType] = NCommon::TypeToYsonNode(itemType); + rowSpec[RowSpecAttrNativeYtTypeFlags] = (flags & nativeTypeCompat); + intermediateSchema = RowSpecToYTSchema(rowSpec, nativeTypeCompat).ToNode(); + } + } + TString reduceLambda; + { + TScopedAlloc alloc(__LOCATION__, NKikimr::TAlignedPagePoolCounters(), + execCtx->FunctionRegistry_->SupportsSizedAllocators()); + alloc.SetLimit(execCtx->Options_.Config()->DefaultCalcMemoryLimit.Get().GetOrElse(0)); + TNativeYtLambdaBuilder builder(alloc, Services_, *execCtx->Session_); + reduceLambda = builder.BuildLambdaWithIO(*MkqlCompiler_, mapReduce.Reducer(), ctx); + } + TExpressionResorceUsage reduceExtraUsage = execCtx->ScanExtraResourceUsage(mapReduce.Reducer().Body().Ref(), false); + + const auto inputTypeSet = NYql::GetSetting(mapReduce.Settings().Ref(), EYtSettingType::ReduceInputType); + TString reduceInputType = NCommon::WriteTypeToYson(inputTypeSet ? + inputTypeSet->Tail().GetTypeAnn()->Cast<TTypeExprType>()->GetType(): + GetSequenceItemType(mapReduce.Reducer().Args().Arg(0), false) + ); + + TVector<TString> sortLimitBy = NYql::GetSettingAsColumnList(mapReduce.Settings().Ref(), EYtSettingType::SortLimitBy); + TMaybe<ui64> limit = GetLimit(mapReduce.Settings().Ref()); + if (limit && !sortLimitBy.empty() && *limit > execCtx->Options_.Config()->TopSortMaxLimit.Get().GetOrElse(DEFAULT_TOP_SORT_LIMIT)) { + limit.Clear(); + } + + return execCtx->Session_->Queue_->Async([reduceBy, sortBy, limit, sortLimitBy, mapLambda, mapInputType, mapDirectOutputs, mapExtraUsage, + reduceLambda, reduceInputType, reduceExtraUsage, intermediateMeta, intermediateSchema, intermediateStreams, useIntermediateStreams, execCtx]() + { + YQL_LOG_CTX_ROOT_SESSION_SCOPE(execCtx->LogCtx_); + execCtx->MakeUserFiles(); + if (mapLambda) { + return ExecMapReduce(reduceBy, sortBy, limit, sortLimitBy, mapLambda, mapInputType, mapDirectOutputs, mapExtraUsage, + reduceLambda, reduceInputType, reduceExtraUsage, intermediateMeta, intermediateSchema, intermediateStreams, execCtx); + } else { + return ExecMapReduce(reduceBy, sortBy, limit, sortLimitBy, reduceLambda, reduceInputType, reduceExtraUsage, intermediateSchema, + useIntermediateStreams, execCtx); + } + }); + } + + static void ExecSafeFill(const TVector<TRichYPath>& outYPaths, + TRuntimeNode root, + const TString& outSpec, + const TExecContext<TRunOptions>::TPtr& execCtx, + const TTransactionCache::TEntry::TPtr& entry, + const TNativeYtLambdaBuilder& builder, + TScopedAlloc& alloc, + TString filePrefix + ) { + NYT::TTableWriterOptions writerOptions; + auto maxRowWeight = execCtx->Options_.Config()->MaxRowWeight.Get(execCtx->Cluster_); + auto maxKeyWeight = execCtx->Options_.Config()->MaxKeyWeight.Get(execCtx->Cluster_); + bool hasSecureParams = !execCtx->Options_.SecureParams().empty(); + + if (maxRowWeight || maxKeyWeight || hasSecureParams) { + NYT::TNode config; + if (maxRowWeight) { + config["max_row_weight"] = static_cast<i64>(*maxRowWeight); + } + if (maxKeyWeight) { + config["max_key_weight"] = static_cast<i64>(*maxKeyWeight); + } + if (hasSecureParams) { + FillSecureVault(config, execCtx->Options_.SecureParams()); + } + writerOptions.Config(config); + } + + NCommon::TCodecContext codecCtx(builder.GetTypeEnvironment(), *execCtx->FunctionRegistry_); + TMkqlIOSpecs specs; + if (execCtx->Options_.Config()->UseSkiff.Get(execCtx->Cluster_).GetOrElse(DEFAULT_USE_SKIFF)) { + specs.SetUseSkiff(execCtx->Options_.OptLLVM()); + } + specs.Init(codecCtx, outSpec); + + TVector<TRawTableWriterPtr> writers; + for (size_t i: xrange(outYPaths.size())) { + auto writer = entry->Tx->CreateRawWriter(outYPaths[i], specs.MakeOutputFormat(i), writerOptions); + writers.push_back(writer); + } + + TMkqlWriterImpl mkqlWriter(writers, 4_MB); + mkqlWriter.SetSpecs(specs); + mkqlWriter.SetWriteLimit(alloc.GetLimit()); + + TExploringNodeVisitor explorer; + auto localGraph = builder.BuildLocalGraph(GetGatewayNodeFactory(&codecCtx, &mkqlWriter, execCtx->UserFiles_, filePrefix), + execCtx->Options_.UdfValidateMode(), + NUdf::EValidatePolicy::Exception, "OFF" /* don't use LLVM locally */, EGraphPerProcess::Multi, explorer, root); + auto& graph = std::get<0>(localGraph); + const TBindTerminator bind(graph->GetTerminator()); + graph->Prepare(); + auto value = graph->GetValue(); + + if (root.GetStaticType()->IsStream()) { + NUdf::TUnboxedValue item; + const auto status = value.Fetch(item); + YQL_ENSURE(NUdf::EFetchStatus::Finish == status); + } else { + YQL_ENSURE(value.IsFinish()); + } + + mkqlWriter.Finish(); + for (auto& writer: writers) { + writer->Finish(); + } + } + + static TFuture<void> ExecFill(TString lambda, + const TExpressionResorceUsage& extraUsage, + const TString& tmpTable, + const TExecContext<TRunOptions>::TPtr& execCtx) + { + const bool testRun = execCtx->Config_->GetLocalChainTest(); + TFuture<bool> ret = testRun ? MakeFuture<bool>(false) : execCtx->LookupQueryCacheAsync(); + return ret.Apply([lambda, extraUsage, tmpTable, execCtx, testRun] (const auto& f) mutable { + YQL_LOG_CTX_ROOT_SESSION_SCOPE(execCtx->LogCtx_); + TTransactionCache::TEntry::TPtr entry; + TVector<TRichYPath> outYPaths; + if (testRun) { + YQL_ENSURE(execCtx->OutTables_.size() == 1U, "Test mode support single output only."); + const auto& out = execCtx->OutTables_.front(); + outYPaths.emplace_back(TRichYPath(out.Path).Schema(RowSpecToYTSchema(TestTables[out.Path].first = out.Spec[YqlRowSpecAttribute], NTCF_NONE))); + } else { + entry = execCtx->GetEntry(); + bool cacheHit = f.GetValue(); + outYPaths = PrepareDestinations(execCtx->OutTables_, execCtx, entry, !cacheHit); + if (cacheHit) { + execCtx->QueryCacheItem.Destroy(); + return MakeFuture(); + } + } + + const bool useSkiff = execCtx->Options_.Config()->UseSkiff.Get(execCtx->Cluster_).GetOrElse(DEFAULT_USE_SKIFF); + + TIntrusivePtr<TYqlUserJob> job; + TRawMapOperationSpec mapOpSpec; + + auto tmpFiles = std::make_shared<TTempFiles>(execCtx->FileStorage_->GetTemp()); + + bool localRun = !testRun && + (execCtx->Config_->HasExecuteUdfLocallyIfPossible() + ? execCtx->Config_->GetExecuteUdfLocallyIfPossible() : false); + { + TUserJobSpec userJobSpec; + TScopedAlloc alloc(__LOCATION__, NKikimr::TAlignedPagePoolCounters(), + execCtx->FunctionRegistry_->SupportsSizedAllocators()); + alloc.SetLimit(execCtx->Options_.Config()->DefaultCalcMemoryLimit.Get().GetOrElse(0)); + TNativeYtLambdaBuilder builder(alloc, execCtx->FunctionRegistry_, *execCtx->Session_); + TProgramBuilder pgmBuilder(builder.GetTypeEnvironment(), *execCtx->FunctionRegistry_); + TGatewayTransformer transform(execCtx, entry, pgmBuilder, *tmpFiles); + transform.SetTwoPhaseTransform(); + + TRuntimeNode root = builder.Deserialize(lambda); + root = builder.TransformAndOptimizeProgram(root, transform); + if (transform.HasSecondPhase()) { + root = builder.TransformAndOptimizeProgram(root, transform); + } + size_t nodeCount = 0; + std::tie(lambda, nodeCount) = builder.Serialize(root); + if (nodeCount > execCtx->Options_.Config()->LLVMNodeCountLimit.Get(execCtx->Cluster_).GetOrElse(DEFAULT_LLVM_NODE_COUNT_LIMIT)) { + execCtx->Options_.OptLLVM("OFF"); + } + + if (transform.CanExecuteInternally() && !testRun) { + const auto nativeTypeCompat = execCtx->Options_.Config()->NativeYtTypeCompatibility.Get(execCtx->Cluster_).GetOrElse(NTCF_LEGACY); + execCtx->SetNodeExecProgress("Waiting for concurrency limit"); + execCtx->Session_->InitLocalCalcSemaphore(execCtx->Options_.Config()); + TGuard<TFastSemaphore> guard(*execCtx->Session_->LocalCalcSemaphore_); + execCtx->SetNodeExecProgress("Local run"); + ExecSafeFill(outYPaths, root, execCtx->GetOutSpec(!useSkiff, nativeTypeCompat), execCtx, entry, builder, alloc, tmpFiles->TmpDir.GetPath() + '/'); + return MakeFuture(); + } + + localRun = localRun && transform.CanExecuteLocally(); + + job = MakeIntrusive<TYqlUserJob>(); + transform.ApplyJobProps(*job); + transform.ApplyUserJobSpec(userJobSpec, localRun || testRun); + + FillUserJobSpec(userJobSpec, execCtx, extraUsage, transform.GetUsedMemory(), + execCtx->EstimateLLVMMem(nodeCount), localRun || testRun); + mapOpSpec.MapperSpec(userJobSpec); + } + + job->SetLambdaCode(lambda); + job->SetOptLLVM(execCtx->Options_.OptLLVM()); + job->SetUdfValidateMode(execCtx->Options_.UdfValidateMode()); + const auto nativeTypeCompat = execCtx->Options_.Config()->NativeYtTypeCompatibility.Get(execCtx->Cluster_).GetOrElse(NTCF_LEGACY); + job->SetOutSpec(execCtx->GetOutSpec(!useSkiff, nativeTypeCompat)); + job->SetUseSkiff(useSkiff, 0); + + mapOpSpec.AddInput(tmpTable); + + for (size_t i: xrange(execCtx->OutTables_.size())) { + mapOpSpec.AddOutput(outYPaths[i]); + } + + FillOperationSpec(mapOpSpec, execCtx); + const auto formats = job->GetIOFormats(execCtx->FunctionRegistry_); + mapOpSpec.InputFormat(formats.first); + mapOpSpec.OutputFormat(formats.second); + + if (localRun && mapOpSpec.MapperSpec_.Files_.empty() && execCtx->OutTables_.size() == 1U) { + return LocalFillJob(mapOpSpec, job.Get(), entry); + } else if (testRun) { + YQL_ENSURE(execCtx->OutTables_.size() == 1U, "Support single output only."); + + const TString dummy(NYT::NodeListToYsonString({NYT::TNode()("input", "dummy")})); + TStringInput in(dummy); + + const auto itO = TestTables.find(execCtx->OutTables_.front().Path); + YQL_ENSURE(TestTables.cend() != itO); + + TStringOutput out(itO->second.second); + + LocalRawMapReduce(mapOpSpec, job.Get(), &in, &out); + DumpLocalTable(itO->second.second, execCtx->Config_->GetLocalChainFile()); + return MakeFuture(); + } else { + PrepareTempDestination(tmpTable, execCtx, entry, entry->Tx); + auto writer = entry->Tx->CreateTableWriter<NYT::TNode>(tmpTable); + writer->AddRow(NYT::TNode()("input", "dummy")); + writer->Finish(); + } + + NYT::TNode spec = execCtx->Session_->CreateSpecWithDesc(execCtx->CodeSnippets_); + FillSpec(spec, *execCtx, entry, extraUsage.Cpu, Nothing(), + EYtOpProp::TemporaryAutoMerge | EYtOpProp::WithMapper | EYtOpProp::WithUserJobs); + + TOperationOptions opOpts; + FillOperationOptions(opOpts, execCtx, entry); + opOpts.StartOperationMode(TOperationOptions::EStartOperationMode::AsyncPrepare).Spec(spec); + + return execCtx->RunOperation([entry, execCtx, job, mapOpSpec = std::move(mapOpSpec), opOpts = std::move(opOpts), tmpFiles]() { + execCtx->SetNodeExecProgress("Uploading artifacts"); + return entry->Tx->RawMap(mapOpSpec, job, opOpts); + }) + .Apply([tmpTable, entry](const TFuture<void>& f){ + f.GetValue(); + entry->RemoveInternal(tmpTable); + }); + }); + } + + TFuture<void> DoFill(TYtFill fill, const TExecContext<TRunOptions>::TPtr& execCtx, TExprContext& ctx) { + TString lambda; + { + TScopedAlloc alloc(__LOCATION__, NKikimr::TAlignedPagePoolCounters(), + Services_.FunctionRegistry->SupportsSizedAllocators()); + alloc.SetLimit(execCtx->Options_.Config()->DefaultCalcMemoryLimit.Get().GetOrElse(0)); + TNativeYtLambdaBuilder builder(alloc, Services_, *execCtx->Session_); + lambda = builder.BuildLambdaWithIO(*MkqlCompiler_, fill.Content(), ctx); + } + auto extraUsage = execCtx->ScanExtraResourceUsage(fill.Content().Ref(), false); + + TString tmpFolder = GetTablesTmpFolder(*execCtx->Options_.Config()); + TString tmpTablePath = NYql::TransformPath(tmpFolder, + TStringBuilder() << "tmp/" << GetGuidAsString(execCtx->Session_->RandomProvider_->GenGuid()), true, execCtx->Session_->UserName_); + + return execCtx->Session_->Queue_->Async([lambda, tmpTablePath, extraUsage, execCtx]() { + YQL_LOG_CTX_ROOT_SESSION_SCOPE(execCtx->LogCtx_); + execCtx->MakeUserFiles(); + return ExecFill(lambda, extraUsage, tmpTablePath, execCtx); + }); + } + + TFuture<void> DoTouch(TYtOutputOpBase touch, const TExecContext<TRunOptions>::TPtr& execCtx) { + Y_UNUSED(touch); + return execCtx->Session_->Queue_->Async([execCtx]() { + YQL_LOG_CTX_ROOT_SESSION_SCOPE(execCtx->LogCtx_); + auto entry = execCtx->GetEntry(); + PrepareDestinations(execCtx->OutTables_, execCtx, entry, true); + }); + } + + TFuture<bool> DoPrepare(TYtOutputOpBase write, const TExecContext<TPrepareOptions>::TPtr& execCtx) const { + Y_UNUSED(write); + return execCtx->Session_->Queue_->Async([execCtx]() { + return execCtx->LookupQueryCacheAsync().Apply([execCtx](const auto& f) { + YQL_LOG_CTX_ROOT_SESSION_SCOPE(execCtx->LogCtx_); + auto entry = execCtx->GetEntry(); + bool cacheHit = f.GetValue(); + PrepareDestinations(execCtx->OutTables_, execCtx, entry, !cacheHit, execCtx->Options_.SecurityTags()); + execCtx->QueryCacheItem.Destroy(); + return cacheHit; + }); + }); + } + + TFuture<void> DoDrop(TYtDropTable drop, const TExecContext<TRunOptions>::TPtr& execCtx) { + TString tmpFolder = GetTablesTmpFolder(*execCtx->Options_.Config()); + auto table = drop.Table(); + bool isAnonymous = NYql::HasSetting(table.Settings().Ref(), EYtSettingType::Anonymous); + TString path = NYql::TransformPath(tmpFolder, table.Name().Value(), isAnonymous, execCtx->Session_->UserName_); + YQL_CLOG(INFO, ProviderYt) << "Dropping: " << execCtx->Cluster_ << '.' << path; + + return execCtx->Session_->Queue_->Async([path, execCtx]() { + YQL_LOG_CTX_ROOT_SESSION_SCOPE(execCtx->LogCtx_); + auto entry = execCtx->GetEntry(); + entry->Tx->Remove(path, TRemoveOptions().Force(true)); + }); + } + + TFuture<void> DoStatOut(TYtStatOut statOut, const TExecContext<TRunOptions>::TPtr& execCtx) { + auto input = statOut.Input(); + + TString tmpFolder = GetTablesTmpFolder(*execCtx->Options_.Config()); + + TString ytTable = TString{GetOutTable(input).Cast<TYtOutTable>().Name().Value()}; + ytTable = NYql::TransformPath(tmpFolder, ytTable, true, execCtx->Session_->UserName_); + ytTable = NYT::AddPathPrefix(ytTable, NYT::TConfig::Get()->Prefix); + + TVector<TString> replaceMask; + for (const auto& item: statOut.ReplaceMask().Ptr()->Children()) { + replaceMask.push_back(TString{TCoAtom(item).Value()}); + } + + auto statUploadOptions = TStatUploadOptions { + TString{YtProviderName}, + execCtx->Options_.SessionId(), + *execCtx->Options_.PublicId(), + TString{statOut.Table().Cluster().Value()}, + TString{statOut.Table().Name().Value()}, + TString{statOut.Table().Scale().Value()}, + std::move(replaceMask), + execCtx->YtServer_, + std::move(ytTable), + GetGuidAsString(execCtx->GetEntry()->Tx->GetId()), + execCtx->GetAuth(), + NNative::GetPool(*execCtx, execCtx->Options_.Config()) + }; + execCtx->SetNodeExecProgress("Running"); + return StatUploader_->Upload(std::move(statUploadOptions)); + } + + static ui64 CalcDataSize(const NYT::TRichYPath& ytPath, const NYT::TNode& attrs) { + ui64 res = attrs["uncompressed_data_size"].IntCast<ui64>(); + const auto records = attrs["chunk_row_count"].IntCast<ui64>(); + if (auto usedRows = GetUsedRows(ytPath, records)) { + res *= double(*usedRows) / double(records); + } + return res; + } + + static bool AllPathColumnsAreInSchema(const NYT::TRichYPath& ytPath, const NYT::TNode& attrs) { + YQL_ENSURE(ytPath.Columns_.Defined()); + + if (!attrs.HasKey("schema")) { + YQL_CLOG(INFO, ProviderYt) << "Missing YT schema for " << ytPath.Path_; + return false; + } + + TSet<TString> columns(ytPath.Columns_->Parts_.begin(), ytPath.Columns_->Parts_.end()); + + for (const auto& schemaColumn : attrs["schema"].AsList()) { + auto it = columns.find(schemaColumn["name"].AsString()); + if (it != columns.end()) { + columns.erase(it); + } + if (columns.empty()) { + break; + } + } + + if (!columns.empty()) { + YQL_CLOG(INFO, ProviderYt) << "Columns {" << JoinSeq(", ", columns) << "} are missing in YT schema for table " + << ytPath.Path_ << ", assuming uncompressed data size"; + } + + return columns.empty(); + } + + static TPathStatResult ExecPathStat(const TExecContext<TPathStatOptions>::TPtr& execCtx, bool onlyCached) { + try { + TPathStatResult res; + res.DataSize.resize(execCtx->Options_.Paths().size(), 0); + res.Extended.resize(execCtx->Options_.Paths().size()); + + auto entry = execCtx->GetOrCreateEntry(); + auto tx = entry->Tx; + const TString tmpFolder = GetTablesTmpFolder(*execCtx->Options_.Config()); + const NYT::EOptimizeForAttr tmpOptimizeFor = execCtx->Options_.Config()->OptimizeFor.Get(execCtx->Cluster_).GetOrElse(NYT::EOptimizeForAttr::OF_LOOKUP_ATTR); + TVector<NYT::TRichYPath> ytPaths(Reserve(execCtx->Options_.Paths().size())); + TVector<size_t> pathMap; + bool extended = execCtx->Options_.Extended(); + + auto extractSysColumns = [] (NYT::TRichYPath& ytPath) -> TVector<TString> { + TVector<TString> res; + if (ytPath.Columns_) { + auto it = std::remove_if( + ytPath.Columns_->Parts_.begin(), + ytPath.Columns_->Parts_.end(), + [] (const TString& col) { return col.StartsWith(YqlSysColumnPrefix); } + ); + res.assign(it, ytPath.Columns_->Parts_.end()); + ytPath.Columns_->Parts_.erase(it, ytPath.Columns_->Parts_.end()); + } + return res; + }; + + for (size_t i: xrange(execCtx->Options_.Paths().size())) { + auto& req = execCtx->Options_.Paths()[i]; + NYT::TRichYPath ytPath = req.Path(); + auto tablePath = NYql::TransformPath(tmpFolder, ytPath.Path_, req.IsTemp(), execCtx->Session_->UserName_); + if (req.IsTemp() && !req.IsAnonymous()) { + ytPath.Path_ = NYT::AddPathPrefix(tablePath, NYT::TConfig::Get()->Prefix); + NYT::TNode attrs; + if (auto sysColumns = extractSysColumns(ytPath)) { + attrs = tx->Get(ytPath.Path_ + "/@", NYT::TGetOptions().AttributeFilter( + NYT::TAttributeFilter() + .AddAttribute(TString("uncompressed_data_size")) + .AddAttribute(TString("chunk_row_count")) + )); + auto records = attrs["chunk_row_count"].IntCast<ui64>(); + records = GetUsedRows(ytPath, records).GetOrElse(records); + for (auto col: sysColumns) { + auto size = 0; + if (col == YqlSysColumnNum || col == YqlSysColumnRecord) { + size = sizeof(ui64); + } else if (col == YqlSysColumnIndex) { + size = sizeof(ui32); + } + // zero size for YqlSysColumnPath for temp tables + size *= records; + res.DataSize[i] += size; + YQL_CLOG(INFO, ProviderYt) << "Adding stat for " << col << ": " << size << " (virtual)"; + } + } + TMaybe<ui64> cachedStat; + TMaybe<NYT::TTableColumnarStatistics> cachedExtendedStat; + if (!extended && (cachedStat = entry->GetColumnarStat(ytPath))) { + res.DataSize[i] += *cachedStat; + YQL_CLOG(INFO, ProviderYt) << "Stat for " << DebugPath(req.Path()) << ": " << res.DataSize[i] << " (from cache, extended: false)"; + } else if (extended && (cachedExtendedStat = entry->GetExtendedColumnarStat(ytPath))) { + PopulatePathStatResult(res, i, *cachedExtendedStat); + YQL_CLOG(INFO, ProviderYt) << "Stat for " << DebugPath(req.Path()) << " (from cache, extended: true)"; + } else if (onlyCached) { + YQL_CLOG(INFO, ProviderYt) << "Stat for " << DebugPath(req.Path()) << " is missing in cache - sync path stat failed (extended: " << extended << ")"; + return res; + } else if (NYT::EOptimizeForAttr::OF_SCAN_ATTR != tmpOptimizeFor && !extended) { + + // Use entire table size for lookup tables (YQL-7257) + if (attrs.IsUndefined()) { + attrs = tx->Get(ytPath.Path_ + "/@", NYT::TGetOptions().AttributeFilter( + NYT::TAttributeFilter() + .AddAttribute(TString("uncompressed_data_size")) + .AddAttribute(TString("chunk_row_count")) + )); + } + auto size = CalcDataSize(ytPath, attrs); + res.DataSize[i] += size; + entry->UpdateColumnarStat(ytPath, size); + YQL_CLOG(INFO, ProviderYt) << "Stat for " << DebugPath(req.Path()) << ": " << res.DataSize[i] << " (uncompressed_data_size for lookup, extended: false)"; + } else { + ytPaths.push_back(ytPath); + pathMap.push_back(i); + } + } else { + auto p = entry->Snapshots.FindPtr(std::make_pair(tablePath, req.Epoch())); + YQL_ENSURE(p, "Table " << tablePath << " (epoch=" << req.Epoch() << ") has no snapshot"); + ytPath.Path(std::get<0>(*p)).TransactionId(std::get<1>(*p)); + NYT::TNode attrs; + if (auto sysColumns = extractSysColumns(ytPath)) { + attrs = entry->Client->AttachTransaction(std::get<1>(*p))->Get(std::get<0>(*p) + "/@", NYT::TGetOptions().AttributeFilter( + NYT::TAttributeFilter() + .AddAttribute(TString("uncompressed_data_size")) + .AddAttribute(TString("optimize_for")) + .AddAttribute(TString("chunk_row_count")) + .AddAttribute(TString("schema")) + )); + auto records = attrs["chunk_row_count"].IntCast<ui64>(); + records = GetUsedRows(ytPath, records).GetOrElse(records); + for (auto col: sysColumns) { + auto size = 0; + if (col == YqlSysColumnNum || col == YqlSysColumnRecord) { + size = sizeof(ui64); + } else if (col == YqlSysColumnIndex) { + size = sizeof(ui32); + } else if (col == YqlSysColumnPath && !req.IsTemp()) { + size = tablePath.size(); + } + size *= records; + res.DataSize[i] += size; + YQL_CLOG(INFO, ProviderYt) << "Adding stat for " << col << ": " << size << " (virtual)"; + } + } + TMaybe<ui64> cachedStat; + TMaybe<NYT::TTableColumnarStatistics> cachedExtendedStat; + if (!extended && (cachedStat = entry->GetColumnarStat(ytPath))) { + res.DataSize[i] += *cachedStat; + YQL_CLOG(INFO, ProviderYt) << "Stat for " << DebugPath(req.Path()) << " (epoch=" << req.Epoch() << "): " << res.DataSize[i] << " (from cache, extended: false)"; + } else if (extended && (cachedExtendedStat = entry->GetExtendedColumnarStat(ytPath))) { + PopulatePathStatResult(res, i, *cachedExtendedStat); + YQL_CLOG(INFO, ProviderYt) << "Stat for " << DebugPath(req.Path()) << " (from cache, extended: true)"; + } else if (onlyCached) { + YQL_CLOG(INFO, ProviderYt) + << "Stat for " << DebugPath(req.Path()) + << " (epoch=" << req.Epoch() << ", extended: " << extended + << ") is missing in cache - sync path stat failed"; + return res; + } else { + if (attrs.IsUndefined()) { + attrs = entry->Client->AttachTransaction(std::get<1>(*p))->Get(std::get<0>(*p) + "/@", NYT::TGetOptions().AttributeFilter( + NYT::TAttributeFilter() + .AddAttribute(TString("uncompressed_data_size")) + .AddAttribute(TString("optimize_for")) + .AddAttribute(TString("chunk_row_count")) + .AddAttribute(TString("schema")) + )); + } + if (extended || + (attrs.HasKey("optimize_for") && attrs["optimize_for"] == "scan" && + AllPathColumnsAreInSchema(req.Path(), attrs))) + { + pathMap.push_back(i); + ytPaths.push_back(ytPath); + YQL_CLOG(INFO, ProviderYt) << "Stat for " << DebugPath(req.Path()) << " (epoch=" << req.Epoch() << ") add for request with path " << ytPath.Path_ << " (extended: " << extended << ")"; + } else { + // Use entire table size for lookup tables (YQL-7257) + auto size = CalcDataSize(ytPath, attrs); + res.DataSize[i] += size; + entry->UpdateColumnarStat(ytPath, size); + YQL_CLOG(INFO, ProviderYt) << "Stat for " << DebugPath(req.Path()) << " (epoch=" << req.Epoch() << "): " << res.DataSize[i] << " (uncompressed_data_size for lookup)"; + } + } + } + } + + if (ytPaths) { + YQL_ENSURE(!onlyCached); + auto fetchMode = extended ? + NYT::EColumnarStatisticsFetcherMode::FromNodes : + execCtx->Options_.Config()->JoinColumnarStatisticsFetcherMode.Get().GetOrElse(NYT::EColumnarStatisticsFetcherMode::Fallback); + auto columnStats = tx->GetTableColumnarStatistics(ytPaths, NYT::TGetTableColumnarStatisticsOptions().FetcherMode(fetchMode)); + YQL_ENSURE(pathMap.size() == columnStats.size()); + for (size_t i: xrange(columnStats.size())) { + auto& columnStat = columnStats[i]; + const ui64 weight = columnStat.LegacyChunksDataWeight + + Accumulate(columnStat.ColumnDataWeight.begin(), columnStat.ColumnDataWeight.end(), 0ull, + [](ui64 sum, decltype(*columnStat.ColumnDataWeight.begin())& v) { return sum + v.second; }); + + if (extended) { + PopulatePathStatResult(res, pathMap[i], columnStat); + } + + res.DataSize[pathMap[i]] += weight; + entry->UpdateColumnarStat(ytPaths[i], columnStat, extended); + YQL_CLOG(INFO, ProviderYt) << "Stat for " << execCtx->Options_.Paths()[pathMap[i]].Path().Path_ << ": " << weight << " (fetched)"; + } + } + + res.SetSuccess(); + return res; + } catch (...) { + return ResultFromCurrentException<TPathStatResult>(); + } + } + + static TRunResult MakeRunResult(const TVector<TOutputInfo>& outTables, const TTransactionCache::TEntry::TPtr& entry) { + TRunResult res; + res.SetSuccess(); + + if (outTables.empty()) { + return res; + } + + auto batchGet = entry->Tx->CreateBatchRequest(); + TVector<TFuture<TYtTableStatInfo::TPtr>> batchRes(Reserve(outTables.size())); + for (auto& out: outTables) { + batchRes.push_back( + batchGet->Get(out.Path + "/@", TGetOptions() + .AttributeFilter(TAttributeFilter() + .AddAttribute(TString("id")) + .AddAttribute(TString("dynamic")) + .AddAttribute(TString("row_count")) + .AddAttribute(TString("chunk_row_count")) + .AddAttribute(TString("uncompressed_data_size")) + .AddAttribute(TString("data_weight")) + .AddAttribute(TString("chunk_count")) + .AddAttribute(TString("modification_time")) + .AddAttribute(TString("schema")) + .AddAttribute(TString("revision")) + .AddAttribute(TString("content_revision")) + ) + ).Apply([out](const TFuture<NYT::TNode>& f) { + + auto attrs = f.GetValue(); + + TString expectedSortedBy = ToColumnList(out.SortedBy.Parts_); + TString realSortedBy = TString("[]"); + if (attrs.HasKey("schema")) { + auto keyColumns = KeyColumnsFromSchema(attrs["schema"]); + realSortedBy = ToColumnList(keyColumns.Keys); + } + YQL_ENSURE(expectedSortedBy == realSortedBy, "Output table " << out.Path + << " has unexpected \"sorted_by\" value. Expected: " << expectedSortedBy + << ", actual: " << realSortedBy); + + auto statInfo = MakeIntrusive<TYtTableStatInfo>(); + statInfo->Id = attrs["id"].AsString(); + statInfo->RecordsCount = GetTableRowCount(attrs); + statInfo->DataSize = GetDataWeight(attrs).GetOrElse(0); + statInfo->ChunkCount = attrs["chunk_count"].AsInt64(); + TString strModifyTime = attrs["modification_time"].AsString(); + statInfo->ModifyTime = TInstant::ParseIso8601(strModifyTime).Seconds(); + statInfo->TableRevision = attrs["revision"].IntCast<ui64>(); + statInfo->Revision = GetContentRevision(attrs); + return statInfo; + }) + ); + } + + batchGet->ExecuteBatch(); + + for (size_t i: xrange(outTables.size())) { + res.OutTableStats.emplace_back(outTables[i].Name, batchRes[i].GetValue()); + } + + return res; + } + + template <class TSpec> + static void LocalRawMapReduce(const TSpec& spec, IRawJob* mapper, IInputStream* in, IOutputStream* out) + { + YQL_ENSURE(GetRemoteFiles(spec).empty(), "Unexpected remote files in spec"); + const TTempDir tmp; + for (const auto& f : GetLocalFiles(spec)) { + TFsPath src(std::get<0U>(f)); + if (src.IsRelative()) { + src = (TFsPath::Cwd() / src).Fix(); + } + const TFsPath dst(tmp.Path().Child(src.GetName())); + YQL_ENSURE(NFs::SymLink(src, dst), "Can't make symlink " << dst << " on " << src); + } + + struct TJobBinaryPathVisitor { + TFsPath operator()(const TJobBinaryDefault&) const { + return GetPersistentExecPath(); + } + TFsPath operator()(const TJobBinaryLocalPath& item) const { + return item.Path; + } + TFsPath operator()(const TJobBinaryCypressPath&) const { + ythrow yexception() << "LocalRawMap: unexpected TJobBinaryCypressPath"; + } + }; + TFsPath src = std::visit(TJobBinaryPathVisitor(), GetJobBinary(spec)); + + if (src.IsRelative()) { + src = (TFsPath::Cwd() / src).Fix(); + } + const TFsPath dst(tmp.Path().Child(src.GetName())); + YQL_ENSURE(NFs::SymLink(src, dst), "Can't make symlink " << dst << " on " << src); + + TString jobstate; + TStringOutput job(jobstate); + mapper->Save(job); + job.Finish(); + + if (!jobstate.empty()) { + TFile(tmp.Path().Child("jobstate"), CreateNew | WrOnly).Write(jobstate.data(), jobstate.size()); + } + + TShellCommandOptions opts; + opts.SetUseShell(false).SetDetachSession(false).SetInputStream(in).SetOutputStream(out); + + opts.Environment.emplace("YQL_SUPPRESS_JOB_STATISTIC", '1'); + opts.Environment.emplace("YT_JOB_ID", '0'); + opts.Environment.emplace("YT_USE_CLIENT_PROTOBUF", TConfig::Get()->UseClientProtobuf ? '1' : '0'); + + TList<TString> args; + args.emplace_back("--yt-map"); + args.emplace_back(TJobFactory::Get()->GetJobName(mapper)); + args.emplace_back(ToString(spec.GetOutputs().size())); + args.emplace_back(jobstate.empty() ? '0' : '1'); + + TShellCommand command(dst.GetPath(), args, opts, tmp.Path()); + switch (const auto status = command.Run().GetStatus()) { + case TShellCommand::SHELL_FINISHED: break; + case TShellCommand::SHELL_ERROR: YQL_LOG_CTX_THROW yexception() << command.GetError(); + case TShellCommand::SHELL_INTERNAL_ERROR: YQL_LOG_CTX_THROW yexception() << command.GetInternalError(); + default: YQL_LOG_CTX_THROW yexception() << "Unexpected run status: " << int(status); + } + out->Finish(); + + YQL_CLOG(INFO, ProviderYt) << command.GetError(); + + YQL_ENSURE(command.GetExitCode() == 0, "Job returns: " << command.GetExitCode()); + } + + template <class TResultFactory> + static TFuture<typename TResultFactory::TResult> + LocalCalcJob(const TRawMapOperationSpec& spec, IRawJob* mapper, const TString& lambda, TResultFactory&& factory) + { + const auto& yson = NYT::NodeListToYsonString({NYT::TNode()("input", lambda)}); + TStringInput in(yson); + TStringStream out; + + LocalRawMapReduce(spec, mapper, &in, &out); + + const auto& builder = factory.Create(); + for (const auto& reader = NYT::CreateTableReader<NYT::TNode>(&out, NYT::TTableReaderOptions()); reader->IsValid(); reader->Next()) { + auto& row = reader->GetRow(); + if (!builder->WriteNext(row["output"])) { + break; + } + } + return MakeFuture(builder->Make()); + } + + static TFuture<void> + LocalFillJob(const TRawMapOperationSpec& spec, IRawJob* mapper, TTransactionCache::TEntry::TPtr entry) + { + const auto& yson = NYT::NodeListToYsonString({NYT::TNode()("input", "dummy")}); + TStringInput in(yson); + + const auto writer = entry->Tx->CreateRawWriter(spec.GetOutputs().front(), *spec.OutputFormat_); + + LocalRawMapReduce(spec, mapper, &in, writer.Get()); + return MakeFuture(); + } + + template <class TExecParamsPtr, class TResultFactory> + static TFuture<typename TResultFactory::TResult> ExecCalc( + TString lambda, + const TExpressionResorceUsage& extraUsage, + const TString& tmpTable, + const TExecParamsPtr& execCtx, + TTransactionCache::TEntry::TPtr entry, + TResultFactory&& factory, + const TVector<TString>* columns = nullptr, + IDataProvider::EResultFormat format = IDataProvider::EResultFormat::Yson + ) + { + TRawMapOperationSpec mapOpSpec; + mapOpSpec.Format(TFormat::YsonBinary()); + TIntrusivePtr<TYqlCalcJob> job; + auto tmpFiles = std::make_shared<TTempFiles>(execCtx->FileStorage_->GetTemp()); + + bool localRun = execCtx->Config_->HasExecuteUdfLocallyIfPossible() ? execCtx->Config_->GetExecuteUdfLocallyIfPossible() : false; + { + TUserJobSpec userJobSpec; + TScopedAlloc alloc(__LOCATION__, NKikimr::TAlignedPagePoolCounters(), + execCtx->FunctionRegistry_->SupportsSizedAllocators()); + alloc.SetLimit(execCtx->Options_.Config()->DefaultCalcMemoryLimit.Get().GetOrElse(0)); + auto secureParamsProvider = MakeSimpleSecureParamsProvider(execCtx->Options_.SecureParams()); + TNativeYtLambdaBuilder builder(alloc, execCtx->FunctionRegistry_, *execCtx->Session_, secureParamsProvider.get()); + THolder<TCodecContext> codecCtx; + TString pathPrefix; + TProgramBuilder pgmBuilder(builder.GetTypeEnvironment(), *execCtx->FunctionRegistry_); + TGatewayTransformer transform(execCtx, entry, pgmBuilder, *tmpFiles); + transform.SetTwoPhaseTransform(); + TRuntimeNode root = builder.Deserialize(lambda); + root = builder.TransformAndOptimizeProgram(root, transform); + if (transform.HasSecondPhase()) { + root = builder.TransformAndOptimizeProgram(root, transform); + codecCtx.Reset(new TCodecContext(builder.GetTypeEnvironment(), *execCtx->FunctionRegistry_)); + pathPrefix = tmpFiles->TmpDir.GetPath() + '/'; + } + size_t nodeCount = 0; + std::tie(lambda, nodeCount) = builder.Serialize(root); + if (nodeCount > execCtx->Options_.Config()->LLVMNodeCountLimit.Get(execCtx->Cluster_).GetOrElse(DEFAULT_LLVM_NODE_COUNT_LIMIT)) { + execCtx->Options_.OptLLVM("OFF"); + } + + if (transform.CanExecuteInternally()) { + execCtx->SetNodeExecProgress("Waiting for concurrency limit"); + execCtx->Session_->InitLocalCalcSemaphore(execCtx->Options_.Config()); + TGuard<TFastSemaphore> guard(*execCtx->Session_->LocalCalcSemaphore_); + execCtx->SetNodeExecProgress("Local run"); + TExploringNodeVisitor explorer; + auto localGraph = builder.BuildLocalGraph(GetGatewayNodeFactory(codecCtx.Get(), nullptr, execCtx->UserFiles_, pathPrefix), + execCtx->Options_.UdfValidateMode(), NUdf::EValidatePolicy::Exception, + "OFF" /* don't use LLVM locally */, EGraphPerProcess::Multi, explorer, root); + auto& graph = std::get<0>(localGraph); + const TBindTerminator bind(graph->GetTerminator()); + graph->Prepare(); + auto value = graph->GetValue(); + + switch (format) { + case IDataProvider::EResultFormat::Skiff: { + TMemoryUsageInfo memInfo("Calc"); + THolderFactory holderFactory(alloc.Ref(), memInfo, execCtx->FunctionRegistry_); + TCodecContext codecCtx(builder.GetTypeEnvironment(), *execCtx->FunctionRegistry_, &holderFactory); + + auto skiffBuilder = factory.Create(codecCtx, holderFactory); + skiffBuilder->WriteValue(value, root.GetStaticType()); + return MakeFuture(skiffBuilder->Make()); + } + case IDataProvider::EResultFormat::Yson: { + auto ysonBuilder = factory.Create(); + ysonBuilder->WriteValue(value, root.GetStaticType()); + return MakeFuture(ysonBuilder->Make()); + } + default: + YQL_LOG_CTX_THROW yexception() << "Invalid result type: " << format; + } + } + localRun = localRun && transform.CanExecuteLocally(); + { + TStringInput in(lambda); + TStringStream out; + TBrotliCompress compressor(&out, 8); + TransferData(&in, &compressor); + compressor.Finish(); + lambda = out.Str(); + } + job = MakeIntrusive<TYqlCalcJob>(); + transform.ApplyJobProps(*job); + transform.ApplyUserJobSpec(userJobSpec, localRun); + FillUserJobSpec(userJobSpec, execCtx, extraUsage, transform.GetUsedMemory(), execCtx->EstimateLLVMMem(nodeCount), localRun); + mapOpSpec.MapperSpec(userJobSpec); + } + + if (columns) { + job->SetColumns(*columns); + } + job->SetUseResultYson(factory.UseResultYson()); + job->SetOptLLVM(execCtx->Options_.OptLLVM()); + job->SetUdfValidateMode(execCtx->Options_.UdfValidateMode()); + + mapOpSpec.AddInput(tmpTable); + mapOpSpec.AddOutput(tmpTable); + FillOperationSpec(mapOpSpec, execCtx); + + if (localRun && mapOpSpec.MapperSpec_.Files_.empty()) { + return LocalCalcJob(mapOpSpec, job.Get(), lambda, std::move(factory)); + } + + if (!entry) { + entry = execCtx->GetOrCreateEntry(); + } + + const TString tmpFolder = GetTablesTmpFolder(*execCtx->Options_.Config()); + execCtx->SetCacheItem({tmpTable}, {NYT::TNode::CreateMap()}, tmpFolder); + + TFuture<bool> future = execCtx->Config_->GetLocalChainTest() + ? MakeFuture<bool>(false) + : execCtx->LookupQueryCacheAsync(); + return future + .Apply([execCtx, entry, mapOpSpec = std::move(mapOpSpec), job, tmpTable, lambda, extraUsage, tmpFiles] (const TFuture<bool>& f) { + if (f.GetValue()) { + execCtx->QueryCacheItem.Destroy(); + return MakeFuture(); + } + NYT::TNode spec = execCtx->Session_->CreateSpecWithDesc(execCtx->CodeSnippets_); + FillSpec(spec, *execCtx, entry, extraUsage.Cpu, Nothing(), EYtOpProp::WithMapper); + + PrepareTempDestination(tmpTable, execCtx, entry, entry->Tx); + + { + auto writer = entry->Tx->CreateTableWriter<NYT::TNode>(tmpTable, NYT::TTableWriterOptions().Config(NYT::TNode()("max_row_weight", 128_MB))); + writer->AddRow(NYT::TNode()("input", lambda)); + writer->Finish(); + } + + TOperationOptions opOpts; + FillOperationOptions(opOpts, execCtx, entry); + opOpts.StartOperationMode(TOperationOptions::EStartOperationMode::AsyncPrepare).Spec(spec); + + return execCtx->RunOperation([entry, execCtx, job, mapOpSpec = std::move(mapOpSpec), opOpts = std::move(opOpts), tmpFiles]() { + execCtx->SetNodeExecProgress("Uploading artifacts"); + return entry->Tx->RawMap(mapOpSpec, job, opOpts); + }); + }) + .Apply([execCtx, tmpTable, entry, factory = std::move(factory), format](const auto& f) { + f.GetValue(); + + auto reader = entry->Tx->CreateTableReader<NYT::TNode>(tmpTable); + + TScopedAlloc alloc(__LOCATION__, NKikimr::TAlignedPagePoolCounters(), + execCtx->FunctionRegistry_->SupportsSizedAllocators()); + alloc.SetLimit(execCtx->Options_.Config()->DefaultCalcMemoryLimit.Get().GetOrElse(0)); + TMemoryUsageInfo memInfo("Calc"); + TTypeEnvironment env(alloc); + THolderFactory holderFactory(alloc.Ref(), memInfo, execCtx->FunctionRegistry_); + + switch (format) { + case IDataProvider::EResultFormat::Skiff: { + TCodecContext codecCtx(env, *execCtx->FunctionRegistry_, &holderFactory); + + auto skiffBuilder = factory.Create(codecCtx, holderFactory); + for (; reader->IsValid(); reader->Next()) { + auto& row = reader->GetRow(); + if (!skiffBuilder->WriteNext(row["output"])) { + break; + } + } + return skiffBuilder->Make(); + } + case IDataProvider::EResultFormat::Yson: { + auto ysonBuilder = factory.Create(); + for (; reader->IsValid(); reader->Next()) { + auto& row = reader->GetRow(); + if (!ysonBuilder->WriteNext(row["output"])) { + break; + } + } + return ysonBuilder->Make(); + } + default: + YQL_LOG_CTX_THROW yexception() << "Unexpected result type: " << format; + } + }) + .Apply([tmpTable, execCtx, entry](const TFuture<typename TResultFactory::TResult>& f) { + YQL_LOG_CTX_ROOT_SESSION_SCOPE(execCtx->LogCtx_); + auto res = f.GetValue(); // rethrow error if any + execCtx->StoreQueryCache(); + entry->RemoveInternal(tmpTable); + return res; + }); + } + + template <class TExecParamsPtr> + static void PrepareCommonAttributes( + NYT::TNode& attrs, + const TExecParamsPtr& execCtx, + const TString& cluster, + bool createTable) + { + if (auto compressionCodec = execCtx->Options_.Config()->TemporaryCompressionCodec.Get(cluster)) { + attrs["compression_codec"] = *compressionCodec; + } + if (auto erasureCodec = execCtx->Options_.Config()->TemporaryErasureCodec.Get(cluster)) { + attrs["erasure_codec"] = ToString(*erasureCodec); + } + if (auto optimizeFor = execCtx->Options_.Config()->OptimizeFor.Get(cluster)) { + attrs["optimize_for"] = ToString(*optimizeFor); + } + if (auto ttl = execCtx->Options_.Config()->TempTablesTtl.Get().GetOrElse(TDuration::Zero())) { + attrs["expiration_timeout"] = ttl.MilliSeconds(); + } + + if (createTable) { + if (auto replicationFactor = execCtx->Options_.Config()->TemporaryReplicationFactor.Get(cluster)) { + attrs["replication_factor"] = static_cast<i64>(*replicationFactor); + } + if (auto media = execCtx->Options_.Config()->TemporaryMedia.Get(cluster)) { + attrs["media"] = *media; + } + if (auto primaryMedium = execCtx->Options_.Config()->TemporaryPrimaryMedium.Get(cluster)) { + attrs["primary_medium"] = *primaryMedium; + } + } + } + + template <class TExecParamsPtr> + static void PrepareAttributes( + NYT::TNode& attrs, + const TOutputInfo& out, + const TExecParamsPtr& execCtx, + const TString& cluster, + bool createTable, + const TSet<TString>& securityTags = {}) + { + PrepareCommonAttributes<TExecParamsPtr>(attrs, execCtx, cluster, createTable); + + NYT::MergeNodes(attrs, out.AttrSpec); + + if (createTable) { + const auto nativeTypeCompat = execCtx->Options_.Config()->NativeYtTypeCompatibility.Get(cluster).GetOrElse(NTCF_LEGACY); + attrs["schema"] = RowSpecToYTSchema(out.Spec[YqlRowSpecAttribute], nativeTypeCompat, out.ColumnGroups).ToNode(); + } + + if (!securityTags.empty()) { + auto tagsAttrNode = NYT::TNode::CreateList(); + for (const auto& tag : securityTags) { + tagsAttrNode.Add(tag); + } + attrs[SecurityTagsName] = std::move(tagsAttrNode); + } + } + + template <class TExecParamsPtr> + static TVector<TRichYPath> PrepareDestinations( + const TVector<TOutputInfo>& outTables, + const TExecParamsPtr& execCtx, + const TTransactionCache::TEntry::TPtr& entry, + bool createTables, + const TSet<TString>& securityTags = {}) + { + auto cluster = execCtx->Cluster_; + + TVector<TRichYPath> res; + for (auto& out: outTables) { + res.push_back(TRichYPath(out.Path)); + entry->DeleteAtFinalize(out.Path); + } + + if (createTables) { + TVector<TString> outPaths; + auto batchCreate = entry->Tx->CreateBatchRequest(); + TVector<TFuture<TLockId>> batchCreateRes; + + for (auto& out: outTables) { + NYT::TNode attrs = NYT::TNode::CreateMap(); + + PrepareAttributes(attrs, out, execCtx, cluster, true, securityTags); + + YQL_CLOG(INFO, ProviderYt) << "Create tmp table " << out.Path << ", attrs: " << NYT::NodeToYsonString(attrs); + + // Force table recreation, because some tables may exist after query cache lookup + batchCreateRes.push_back(batchCreate->Create(out.Path, NT_TABLE, TCreateOptions().Force(true).Attributes(attrs))); + outPaths.push_back(out.Path); + } + entry->CreateDefaultTmpFolder(); + CreateParents(outPaths, entry->CacheTx); + + batchCreate->ExecuteBatch(); + WaitExceptionOrAll(batchCreateRes).GetValue(); + } + else { + // set attributes in transactions + const auto multiSet = execCtx->Options_.Config()->_UseMultisetAttributes.Get().GetOrElse(DEFAULT_USE_MULTISET_ATTRS); + if (multiSet) { + for (auto& out: outTables) { + NYT::TNode attrs = NYT::TNode::CreateMap(); + PrepareAttributes(attrs, out, execCtx, cluster, false); + YQL_CLOG(INFO, ProviderYt) << "Update tmp table " << out.Path << ", attrs: " << NYT::NodeToYsonString(attrs); + entry->Tx->MultisetAttributes(out.Path + "/@", attrs.AsMap(), NYT::TMultisetAttributesOptions()); + } + } else { + auto batchSet = entry->Tx->CreateBatchRequest(); + TVector<TFuture<void>> batchSetRes; + + for (auto& out: outTables) { + NYT::TNode attrs = NYT::TNode::CreateMap(); + + PrepareAttributes(attrs, out, execCtx, cluster, false); + YQL_CLOG(INFO, ProviderYt) << "Update tmp table " << out.Path << ", attrs: " << NYT::NodeToYsonString(attrs); + for (auto& attr: attrs.AsMap()) { + batchSetRes.push_back(batchSet->Set(TStringBuilder() << out.Path << "/@" << attr.first, attr.second)); + } + } + + batchSet->ExecuteBatch(); + WaitExceptionOrAll(batchSetRes).GetValue(); + } + } + + return res; + } + + template <class TExecParamsPtr> + static void PrepareTempDestination( + const TString& tmpTable, + const TExecParamsPtr& execCtx, + const TTransactionCache::TEntry::TPtr& entry, + const NYT::ITransactionPtr tx) + { + auto cluster = execCtx->Cluster_; + + NYT::TNode attrs = NYT::TNode::CreateMap(); + PrepareCommonAttributes(attrs, execCtx, cluster, true); + + YQL_CLOG(INFO, ProviderYt) << "Table " << tmpTable << ", attrs: " << NYT::NodeToYsonString(attrs); + + entry->CreateDefaultTmpFolder(); + CreateParents(TVector<TString>{tmpTable}, entry->CacheTx); + + tx->Create(tmpTable, NT_TABLE, TCreateOptions().Force(true).Attributes(attrs)); + entry->DeleteAtFinalizeInternal(tmpTable); + } + + TSession::TPtr GetSession(const TString& sessionId, bool failIfNotExists = true) const { + auto guard = Guard(Mutex_); + if (auto p = Sessions_.FindPtr(sessionId)) { + return *p; + } + if (failIfNotExists) { + YQL_LOG_CTX_THROW yexception() << "Session doesn't exist: " << sessionId; + } + return {}; + } + + template <class TOptions> + typename TExecContext<TOptions>::TPtr MakeExecCtx( + TOptions&& options, + const TSession::TPtr& session, + const TString& cluster, + const TExprNode* root, + TExprContext* exprCtx) const + { + auto ctx = MakeIntrusive<TExecContext<TOptions>>(Services_, Clusters_, MkqlCompiler_, std::move(options), session, cluster, UrlMapper_, Services_.Metrics); + if (root) { + YQL_ENSURE(exprCtx); + if (TYtTransientOpBase::Match(root)) { + ctx->CodeSnippets_.emplace_back("settings", + ConvertToAst(*root->Child(TYtTransientOpBase::idx_Settings), *exprCtx, 0, true) + .Root->ToString(TAstPrintFlags::ShortQuote | TAstPrintFlags::PerLine | TAstPrintFlags::AdaptArbitraryContent)); + if (TYtMap::Match(root)) { + ctx->CodeSnippets_.emplace_back("mapper", + ConvertToAst(*root->Child(TYtMap::idx_Mapper), *exprCtx, 0, true) + .Root->ToString(TAstPrintFlags::ShortQuote | TAstPrintFlags::PerLine | TAstPrintFlags::AdaptArbitraryContent)); + } else if (TYtReduce::Match(root)) { + ctx->CodeSnippets_.emplace_back("reducer", + ConvertToAst(*root->Child(TYtReduce::idx_Reducer), *exprCtx, 0, true) + .Root->ToString(TAstPrintFlags::ShortQuote | TAstPrintFlags::PerLine | TAstPrintFlags::AdaptArbitraryContent)); + } else if (TYtMapReduce::Match(root)) { + ctx->CodeSnippets_.emplace_back("mapper", + ConvertToAst(*root->Child(TYtMapReduce::idx_Mapper), *exprCtx, 0, true) + .Root->ToString(TAstPrintFlags::ShortQuote | TAstPrintFlags::PerLine | TAstPrintFlags::AdaptArbitraryContent)); + ctx->CodeSnippets_.emplace_back("reducer", + ConvertToAst(*root->Child(TYtMapReduce::idx_Reducer), *exprCtx, 0, true) + .Root->ToString(TAstPrintFlags::ShortQuote | TAstPrintFlags::PerLine | TAstPrintFlags::AdaptArbitraryContent)); + } + } else if (TYtFill::Match(root)) { + ctx->CodeSnippets_.emplace_back("lambda", + ConvertToAst(*root->Child(TYtFill::idx_Content), *exprCtx, 0, true) + .Root->ToString(TAstPrintFlags::ShortQuote | TAstPrintFlags::PerLine | TAstPrintFlags::AdaptArbitraryContent)); + } else if (TYtPublish::Match(root)) { + ctx->CodeSnippets_.emplace_back("settings", + ConvertToAst(*root->Child(TYtPublish::idx_Settings), *exprCtx, 0, true) + .Root->ToString(TAstPrintFlags::ShortQuote | TAstPrintFlags::PerLine | TAstPrintFlags::AdaptArbitraryContent)); + } else { + ctx->CodeSnippets_.emplace_back("code", + ConvertToAst(*root, *exprCtx, 0, true).Root->ToString(TAstPrintFlags::ShortQuote | TAstPrintFlags::PerLine | TAstPrintFlags::AdaptArbitraryContent)); + } + } + return ctx; + } + +private: + const TYtNativeServices Services_; + const TConfigClusters::TPtr Clusters_; + TIntrusivePtr<NCommon::TMkqlCommonCallableCompiler> MkqlCompiler_; + TMutex Mutex_; + THashMap<TString, TSession::TPtr> Sessions_; + const TYtUrlMapper UrlMapper_; + IStatUploader::TPtr StatUploader_; +}; + +} // NNative + +IYtGateway::TPtr CreateYtNativeGateway(const TYtNativeServices& services) { + return MakeIntrusive<NNative::TYtNativeGateway>(services); +} + +} // NYql diff --git a/yt/yql/providers/yt/gateway/native/yql_yt_native.h b/yt/yql/providers/yt/gateway/native/yql_yt_native.h new file mode 100644 index 0000000000..4ae23ac55d --- /dev/null +++ b/yt/yql/providers/yt/gateway/native/yql_yt_native.h @@ -0,0 +1,28 @@ +#pragma once + +#include <yt/yql/providers/yt/provider/yql_yt_gateway.h> + +#include <yql/essentials/core/file_storage/file_storage.h> +#include <yql/essentials/minikql/mkql_function_registry.h> +#include <yql/essentials/providers/common/metrics/metrics_registry.h> + +#include <util/generic/ptr.h> + +namespace NYql { + +class TYtGatewayConfig; +using TYtGatewayConfigPtr = std::shared_ptr<TYtGatewayConfig>; + +struct TYtNativeServices { + const NKikimr::NMiniKQL::IFunctionRegistry* FunctionRegistry = nullptr; + + TFileStoragePtr FileStorage; + TYtGatewayConfigPtr Config; + // allow anonymous access for tests + bool DisableAnonymousClusterAccess = false; + IMetricsRegistryPtr Metrics; +}; + +IYtGateway::TPtr CreateYtNativeGateway(const TYtNativeServices& services); + +} diff --git a/yt/yql/providers/yt/gateway/native/yql_yt_native_folders.cpp b/yt/yql/providers/yt/gateway/native/yql_yt_native_folders.cpp new file mode 100644 index 0000000000..86f599a106 --- /dev/null +++ b/yt/yql/providers/yt/gateway/native/yql_yt_native_folders.cpp @@ -0,0 +1,254 @@ +#include "yql_yt_native_folders.h" + +#include <yt/cpp/mapreduce/interface/error_codes.h> +#include <yql/essentials/utils/log/log.h> +#include <yql/essentials/providers/common/proto/gateways_config.pb.h> +#include <yt/yql/providers/yt/gateway/lib/yt_helpers.h> + +namespace NYql::NNative { + +using namespace NYT; +using namespace NCommon; +using namespace NKikimr; +using namespace NKikimr::NMiniKQL; +using namespace NNodes; +using namespace NThreading; + +TString GetTypeFromAttributes(const NYT::TNode& attributes, bool getDocumentType) { + if (!attributes.HasKey("type")) { + return "unknown"; + } + const auto type = attributes["type"].AsString(); + if (getDocumentType && "document" == type) { + if (!attributes.HasKey("_yql_type")) { + return "unknown"; + } + return attributes["_yql_type"].AsString(); + } else { + return type; + } +} + +TString GetTypeFromNode(const NYT::TNode& node, bool getDocumentType) { + if (!node.HasAttributes()) { + return "unknown"; + } + return GetTypeFromAttributes(node.GetAttributes(), getDocumentType); +} + +TMaybe<TVector<IYtGateway::TBatchFolderResult::TFolderItem>> MaybeGetFolderFromCache(TTransactionCache::TEntry::TPtr entry, TStringBuf cacheKey) { + TVector<IYtGateway::TBatchFolderResult::TFolderItem> items; + with_lock(entry->Lock_) { + const auto listPtr = entry->FolderCache.FindPtr(cacheKey); + if (listPtr) { + YQL_CLOG(INFO, ProviderYt) << "Found folder in cache with key ('" << cacheKey << "') with " << listPtr->size() << " items"; + for (auto& el : *listPtr) { + IYtGateway::TBatchFolderResult::TFolderItem item; + std::tie(item.Type, item.Path, item.Attributes) = el; + items.emplace_back(std::move(item)); + } + return items; + } + } + return {}; +} + +TMaybe<TFileLinkPtr> MaybeGetFilePtrFromCache(TTransactionCache::TEntry::TPtr entry, const IYtGateway::TBatchFolderOptions::TFolderPrefixAttrs& folder) { + const auto cacheKey = std::accumulate(folder.AttrKeys.begin(), folder.AttrKeys.end(), folder.Prefix, + [] (TString&& str, const TString& arg) { + return str + "&" + arg; + }); + with_lock(entry->Lock_) { + const auto filePtr = entry->FolderFilePtrCache.FindPtr(cacheKey); + if (filePtr) { + return filePtr->Get(); + } + } + return {}; +} + +TAttributeFilter MakeAttrFilter(const TSet<TString>& attributes, bool isResolvingLink) { + NYT::TAttributeFilter filter; + for (const auto& attr : attributes) { + filter.AddAttribute(attr); + } + if (!isResolvingLink) { + filter.AddAttribute("target_path"); + filter.AddAttribute("broken"); + } + filter.AddAttribute("type"); + return filter; +} + +IYtGateway::TBatchFolderResult::TFolderItem MakeFolderItem(const NYT::TNode& node, const TString& path) { + IYtGateway::TBatchFolderResult::TFolderItem item; + item.Attributes = NYT::TNode::CreateMap(); + for (const auto& attr: node.GetAttributes().AsMap()) { + if (attr.first == "type") { + continue; + } + item.Attributes[attr.first] = attr.second; + } + item.Type = GetTypeFromNode(node, false); + item.Path = path.StartsWith(NYT::TConfig::Get()->Prefix) + ? path.substr(NYT::TConfig::Get()->Prefix.size()) + : path; + return item; +} + +const TTransactionCache::TEntry::TFolderCache::value_type& StoreResInCache(const TTransactionCache::TEntry::TPtr& entry, TVector<IYtGateway::TBatchFolderResult::TFolderItem>&& items, const TString& cacheKey) { + std::vector<std::tuple<TString, TString, NYT::TNode>> cache; + for (const auto& item : items) { + cache.emplace_back(std::move(item.Type), std::move(item.Path), std::move(item.Attributes)); + } + with_lock(entry->Lock_) { + const auto [it, _] = entry->FolderCache.insert_or_assign(cacheKey, std::move(cache)); + return *it; + } +} + +TFileLinkPtr SaveItemsToTempFile(const TExecContext<IYtGateway::TBatchFolderOptions>::TPtr& execCtx, const TVector<IYtGateway::TFolderResult::TFolderItem>& folderItems) { + const TString file = execCtx->FileStorage_->GetTemp() / GetGuidAsString(execCtx->Session_->RandomProvider_->GenGuid()); + YQL_CLOG(INFO, ProviderYt) << "Folder limit exceeded. Writing items to file " << file; + + auto out = MakeHolder<TOFStream>(file); + for (auto& item: folderItems) { + ::SaveMany(out.Get(), item.Type, item.Path, item.Attributes); + } + ::SaveSize(out.Get(), 0); + out.Destroy(); + return CreateFakeFileLink(file, "", true); +} + +IYtGateway::TBatchFolderResult ExecResolveLinks(const TExecContext<IYtGateway::TResolveOptions>::TPtr& execCtx) { + try { + auto batchGet = execCtx->GetEntry()->Tx->CreateBatchRequest(); + TVector<TFuture<IYtGateway::TBatchFolderResult::TFolderItem>> batchRes; + + for (const auto& [item, reqAttributes]: execCtx->Options_.Items()) { + if (item.Type != "link") { + batchRes.push_back(MakeFuture<IYtGateway::TBatchFolderResult::TFolderItem>(std::move(item))); + continue; + } + if (item.Attributes["broken"].AsBool()) { + continue; + } + const auto& targetPath = item.Attributes["target_path"].AsString(); + const auto& path = item.Path; + const auto attrFilter = MakeAttrFilter(reqAttributes, /* isResolvingLink */ true); + + batchRes.push_back( + batchGet->Get(targetPath, TGetOptions().AttributeFilter(attrFilter)) + .Apply([path, pos = execCtx->Options_.Pos()] (const auto& f) { + try { + const auto linkNode = f.GetValue(); + return MakeFolderItem(linkNode, path); + } catch (const NYT::TErrorResponse& e) { + return MakeFolderItem(NYT::TNode::CreateMap(), path); + } + }) + ); + } + IYtGateway::TBatchFolderResult res; + res.SetSuccess(); + if (batchRes.empty()) { + YQL_CLOG(INFO, ProviderYt) << "Batch get result is empty"; + return res; + } + + res.Items.reserve(batchRes.size()); + + batchGet->ExecuteBatch(); + WaitAll(batchRes).Wait(); + for (auto& f : batchRes) { + res.Items.push_back(f.ExtractValue()); + } + return res; + } + catch (...) { + return ResultFromCurrentException<IYtGateway::TBatchFolderResult>(execCtx->Options_.Pos()); + } +} + +IYtGateway::TBatchFolderResult ExecGetFolder(const TExecContext<IYtGateway::TBatchFolderOptions>::TPtr& execCtx) { + const auto entry = execCtx->GetOrCreateEntry(); + auto batchList = entry->Tx->CreateBatchRequest(); + TVector<TFuture<TVector<IYtGateway::TBatchFolderResult::TFolderItem>>> batchRes; + + IYtGateway::TBatchFolderResult folderResult; + folderResult.SetSuccess(); + + for (const auto& folder : execCtx->Options_.Folders()) { + const auto cacheKey = std::accumulate(folder.AttrKeys.begin(), folder.AttrKeys.end(), folder.Prefix, + [] (TString&& str, const TString& arg) { + return str + "&" + arg; + }); + YQL_CLOG(INFO, ProviderYt) << "Executing list command with prefix: " << folder.Prefix << " , cacheKey = " << cacheKey; + + auto maybeCached = MaybeGetFolderFromCache(entry, cacheKey); + if (maybeCached) { + batchRes.push_back(MakeFuture<TVector<IYtGateway::TBatchFolderResult::TFolderItem>>(std::move(*maybeCached))); + continue; + } + + const auto attrFilter = MakeAttrFilter(folder.AttrKeys, /* isResolvingLink */ false); + batchRes.push_back( + batchList->List(folder.Prefix, TListOptions().AttributeFilter(attrFilter)) + .Apply([&folder, cacheKey = std::move(cacheKey), &entry] (const TFuture<NYT::TNode::TListType>& f) + -> TFuture<TVector<IYtGateway::TBatchFolderResult::TFolderItem>> { + TVector<IYtGateway::TBatchFolderResult::TFolderItem> folderItems; + try { + auto nodeList = f.GetValue(); + folderItems.reserve(nodeList.size()); + for (const auto& node : nodeList) { + TStringBuilder path; + if (!folder.Prefix.empty()) { + path << folder.Prefix << "/"; + } + path << node.AsString(); + folderItems.push_back(MakeFolderItem(node, path)); + } + StoreResInCache(entry, std::move(folderItems), cacheKey); + return MakeFuture(std::move(folderItems)); + } + catch (const NYT::TErrorResponse& e) { + if (e.GetError().ContainsErrorCode(NYT::NClusterErrorCodes::NYTree::ResolveError)) { + // Return empty list on missing path + YQL_CLOG(INFO, ProviderYt) << "Storing empty folder in cache with key ('" << cacheKey << "')"; + StoreResInCache(entry, {}, cacheKey); + return MakeFuture<TVector<IYtGateway::TBatchFolderResult::TFolderItem>>({}); + } + return MakeErrorFuture<TVector<IYtGateway::TBatchFolderResult::TFolderItem>>(std::current_exception()); + } + catch (...) { + return MakeErrorFuture<TVector<IYtGateway::TBatchFolderResult::TFolderItem>>(std::current_exception()); + } + }) + ); + } + + TExecuteBatchOptions batchOptions; + if (batchRes.size() > 1) { + const size_t concurrency = execCtx->Options_.Config()->BatchListFolderConcurrency + .Get().GetOrElse(DEFAULT_BATCH_LIST_FOLDER_CONCURRENCY); + batchOptions.Concurrency(concurrency); + } + + try { + batchList->ExecuteBatch(batchOptions); + WaitExceptionOrAll(batchRes).Wait(); + for (auto& res : batchRes) { + const auto items = res.ExtractValue(); + folderResult.Items.reserve(folderResult.Items.size() + items.size()); + for (const auto& item : items) { + folderResult.Items.push_back(std::move(item)); + } + } + } + catch (...) { + return ResultFromCurrentException<IYtGateway::TBatchFolderResult>(execCtx->Options_.Pos()); + } + return folderResult; +} + +} // NYql::NNative diff --git a/yt/yql/providers/yt/gateway/native/yql_yt_native_folders.h b/yt/yql/providers/yt/gateway/native/yql_yt_native_folders.h new file mode 100644 index 0000000000..0c205ac11e --- /dev/null +++ b/yt/yql/providers/yt/gateway/native/yql_yt_native_folders.h @@ -0,0 +1,27 @@ +#include <yt/yql/providers/yt/gateway/native/yql_yt_exec_ctx.h> +#include <yt/yql/providers/yt/gateway/lib/transaction_cache.h> +#include <yt/yql/providers/yt/provider/yql_yt_gateway.h> + +namespace NYql::NNative { + +TString GetTypeFromAttributes(const NYT::TNode& attr, bool getDocumentType); + +TString GetTypeFromNode(const NYT::TNode& node, bool getDocumentType); + +TMaybe<TVector<IYtGateway::TBatchFolderResult::TFolderItem>> MaybeGetFolderFromCache(TTransactionCache::TEntry::TPtr entry, TStringBuf cacheKey); + +TMaybe<TFileLinkPtr> MaybeGetFilePtrFromCache(TTransactionCache::TEntry::TPtr entry, const IYtGateway::TBatchFolderOptions::TFolderPrefixAttrs& folder); + +NYT::TAttributeFilter MakeAttrFilter(const TSet<TString>& attributes, bool isResolvingLink); + +IYtGateway::TBatchFolderResult::TFolderItem MakeFolderItem(const NYT::TNode& node, const TString& prefix, const TString& name, const TVector<TString>& reqAttrKeys); + +const TTransactionCache::TEntry::TFolderCache::value_type& StoreResInCache(const TTransactionCache::TEntry::TPtr& entry, TVector<IYtGateway::TBatchFolderResult::TFolderItem>&& items, const TString& cacheKey); + +TFileLinkPtr SaveItemsToTempFile(const TExecContext<IYtGateway::TBatchFolderOptions>::TPtr& execCtx, const TVector<IYtGateway::TFolderResult::TFolderItem>& folderItems); + +IYtGateway::TBatchFolderResult ExecResolveLinks(const TExecContext<IYtGateway::TResolveOptions>::TPtr& execCtx); + +IYtGateway::TBatchFolderResult ExecGetFolder(const TExecContext<IYtGateway::TBatchFolderOptions>::TPtr& execCtx); + +} // NYql::NNative diff --git a/yt/yql/providers/yt/gateway/native/yql_yt_op_tracker.cpp b/yt/yql/providers/yt/gateway/native/yql_yt_op_tracker.cpp new file mode 100644 index 0000000000..c21df167bb --- /dev/null +++ b/yt/yql/providers/yt/gateway/native/yql_yt_op_tracker.cpp @@ -0,0 +1,226 @@ +#include "yql_yt_op_tracker.h" + +#include <yql/essentials/providers/common/provider/yql_provider_names.h> + +#include <yt/cpp/mapreduce/interface/operation.h> +#include <yt/cpp/mapreduce/interface/job_statistics.h> + +#include <util/datetime/base.h> +#include <util/system/guard.h> +#include <util/system/yassert.h> + +namespace NYql { + +namespace NNative { + +using namespace NThreading; + +// see https://yt.yandex-team.ru/docs/problems/jobstatistics.html for full list +const static TStringBuf YT_STATISTICS[] = { + "job_proxy/cpu/system", + "job_proxy/cpu/user", + "data/input/chunk_count", + "data/input/row_count", + "data/input/data_weight", + "time/exec", + "time/total", + "time/prepare", + "time/artifact_download", + "user_job/cpu/user", + "user_job/cpu/system", + "user_job/max_memory", + "user_job/woodpecker", +}; + +const static TStringBuf CUSTOM_STATISTICS[] = { + "CodeGen_CompileTime", + "CodeGen_GenerateTime", + "CodeGen_FullTime", + "Combine_FlushesCount", + "Combine_MaxRowsCount", + "Job_ElapsedTime", + "Job_InputBytes", + "Job_InputDecodeTime", + "Job_InputReadTime", + "Job_OutputBytes", + "Job_OutputEncodeTime", + "Job_OutputWriteTime", + "Job_SystemTime", + "Job_UserTime", + "Job_InitTime", + "Job_CalcTime", + "Join_Spill_Count", + "Join_Spill_MaxFileSize", + "Join_Spill_MaxRowsCount", + "PagePool_AllocCount", + "PagePool_PageAllocCount", + "PagePool_PageHitCount", + "PagePool_PageMissCount", + "PagePool_PeakAllocated", + "PagePool_PeakUsed", + "Switch_FlushesCount", + "Switch_MaxRowsCount", + "Udf_AppliesCount", +}; + +TOperationTracker::TOperationTracker() + : Thread_(TThread::TParams{Tracker, (void*)this}.SetName("yt_op_tracker")) +{ + Running_ = true; + Thread_.Start(); +} + +TOperationTracker::~TOperationTracker() { + Y_ABORT_UNLESS(!Thread_.Running()); +} + +void TOperationTracker::Stop() { + if (Running_) { + Running_ = false; + Thread_.Join(); + } +} + +TFuture<void> TOperationTracker::MakeOperationWaiter(const NYT::IOperationPtr& operation, TMaybe<ui32> publicId, + const TString& ytServer, const TString& ytClusterName, const TOperationProgressWriter& progressWriter, const TStatWriter& statWriter) +{ + auto future = operation->GetStartedFuture().Apply([operation](const auto& f) { + f.GetValue(); + return operation->Watch(); + }); + if (!publicId) { + return future; + } + + TOperationProgress progress(TString(YtProviderName), *publicId, + TOperationProgress::EState::InProgress); + + auto filter = NYT::TOperationAttributeFilter(); + filter.Add(NYT::EOperationAttribute::State); + + auto checker = [future, operation, ytServer, progress, progressWriter, filter, ytClusterName] () mutable { + bool done = future.Wait(TDuration::Zero()); + + if (!done) { + TString stage; + bool writeProgress = true; + if (operation->IsStarted()) { + if (!progress.RemoteId) { + progress.RemoteId = ytServer + "/" + GetGuidAsString(operation->GetId()); + } + progress.RemoteData["cluster_name"] = ytClusterName; + if (auto briefProgress = operation->GetBriefProgress()) { + progress.Counters.ConstructInPlace(); + progress.Counters->Completed = briefProgress->Completed; + progress.Counters->Running = briefProgress->Running; + progress.Counters->Total = briefProgress->Total; + progress.Counters->Aborted = briefProgress->Aborted; + progress.Counters->Failed = briefProgress->Failed; + progress.Counters->Lost = briefProgress->Lost; + progress.Counters->Pending = briefProgress->Pending; + stage = "Running"; + } else { + auto state = operation->GetAttributes(NYT::TGetOperationOptions().AttributeFilter(filter)).State; + if (state) { + stage = *state; + stage.to_upper(0, 1); + } + } + } else { + // Not started yet + writeProgress = false; + stage = operation->GetStatus(); + } + if (!stage.empty() && stage != progress.Stage.first) { + progress.Stage = TOperationProgress::TStage(stage, TInstant::Now()); + writeProgress = true; + } + if (writeProgress) { + progressWriter(progress); + } + } + return !done; + }; + + with_lock(Mutex_) { + // TODO: limit number of running operations + RunningOperations_.push_back(checker); + } + + // Make a final progress write + return future.Apply([operation, progress, progressWriter, statWriter, ytServer, ytClusterName] (const TFuture<void>& f) mutable { + f.GetValue(); + if (auto briefProgress = operation->GetBriefProgress()) { + progress.Counters.ConstructInPlace(); + progress.Counters->Completed = briefProgress->Completed; + progress.Counters->Running = briefProgress->Running; + progress.Counters->Total = briefProgress->Total; + progress.Counters->Aborted = briefProgress->Aborted; + progress.Counters->Failed = briefProgress->Failed; + progress.Counters->Lost = briefProgress->Lost; + progress.Counters->Pending = briefProgress->Pending; + } + auto operationStatistic = operation->GetJobStatistics(); + + TVector<TOperationStatistics::TEntry> statEntries; + for (auto statName : YT_STATISTICS) { + if (operationStatistic.HasStatistics(statName)) { + auto st = operationStatistic.GetStatistics(statName); + statEntries.emplace_back(TString{statName}, st.Sum(), st.Max(), st.Min(), st.Avg(), st.Count()); + } + } + + for (auto statName : CUSTOM_STATISTICS) { + if (operationStatistic.HasCustomStatistics(statName)) { + auto st = operationStatistic.GetCustomStatistics(statName); + statEntries.emplace_back(TString{statName}, st.Sum(), st.Max(), st.Min(), st.Avg(), st.Count()); + } + } + + statEntries.emplace_back("_cluster", ytServer); + statEntries.emplace_back("_cluster_name", ytClusterName); + statEntries.emplace_back("_id", GetGuidAsString(operation->GetId())); + statWriter(progress.Id, statEntries); + progressWriter(progress); + }); +} + +void* TOperationTracker::Tracker(void* param) { + static_cast<TOperationTracker*>(param)->Tracker(); + return nullptr; + +} + +void TOperationTracker::Tracker() { + while (Running_) { + decltype(RunningOperations_) ops; + with_lock(Mutex_) { + ops.reserve(RunningOperations_.size()); + ops.swap(RunningOperations_); + } + decltype(RunningOperations_) activeOps(Reserve(ops.size())); + for (auto& op: ops) { + try { + if (op()) { + activeOps.push_back(op); + } + } catch (...) { + } + if (!Running_) { + break; + } + } + if (!Running_) { + break; + } + with_lock(Mutex_) { + RunningOperations_.insert(RunningOperations_.end(), activeOps.begin(), activeOps.end()); + } + + Sleep(TDuration::Seconds(1)); + } +} + +} // NNative + +} // NYql diff --git a/yt/yql/providers/yt/gateway/native/yql_yt_op_tracker.h b/yt/yql/providers/yt/gateway/native/yql_yt_op_tracker.h new file mode 100644 index 0000000000..a142b304be --- /dev/null +++ b/yt/yql/providers/yt/gateway/native/yql_yt_op_tracker.h @@ -0,0 +1,49 @@ +#pragma once + +#include <yql/essentials/core/yql_execution.h> + +#include <yt/cpp/mapreduce/interface/fwd.h> + +#include <library/cpp/threading/future/future.h> + +#include <util/generic/maybe.h> +#include <util/generic/ptr.h> +#include <util/generic/string.h> +#include <util/generic/vector.h> +#include <util/system/mutex.h> +#include <util/system/thread.h> + +#include <atomic> +#include <functional> + +namespace NYql { + +namespace NNative { + +class TOperationTracker: public TThrRefBase { +public: + using TPtr = ::TIntrusivePtr<TOperationTracker>; + + TOperationTracker(); + ~TOperationTracker(); + + void Stop(); + + NThreading::TFuture<void> MakeOperationWaiter(const NYT::IOperationPtr& operation, TMaybe<ui32> publicId, + const TString& ytServer, const TString& ytClusterName, const TOperationProgressWriter& writer, const TStatWriter& statWriter); + +private: + static void* Tracker(void* param); + + void Tracker(); + +private: + TMutex Mutex_; + TVector<std::function<bool()>> RunningOperations_; + std::atomic<bool> Running_{false}; + TThread Thread_; +}; + +} // NNative + +} // NYql diff --git a/yt/yql/providers/yt/gateway/native/yql_yt_qb2.cpp b/yt/yql/providers/yt/gateway/native/yql_yt_qb2.cpp new file mode 100644 index 0000000000..45535215b1 --- /dev/null +++ b/yt/yql/providers/yt/gateway/native/yql_yt_qb2.cpp @@ -0,0 +1,152 @@ +#include "yql_yt_qb2.h" + +#include <yql/essentials/utils/yql_panic.h> + +#include <util/string/join.h> +#include <util/generic/hash.h> +#include <util/generic/hash_set.h> +#include <util/generic/vector.h> + +namespace NYql { + +namespace NNative { + +void ProcessTableQB2Premapper(const NYT::TNode& remapper, + const TString& tableName, + NYT::TRichYPath& tablePath, + size_t tableNum, + TRemapperMap& remapperMap, + TSet<TString>& remapperAllFiles, + bool& useSkiff) +{ + THashMap<TString, TVector<TString>> requiredColumnsForField; + THashMap<TString, TVector<TString>> requiredFilesForField; + THashSet<TString> passtroughFields; + bool allPasstrough = true; + for (const auto& field: remapper["fields"].AsMap()) { + auto& requiredColumns = requiredColumnsForField[field.first]; + for (const auto& column: field.second["required_columns"].AsList()) { + requiredColumns.push_back(column.AsString()); + } + + for (const auto& file: field.second["required_dict_paths"].AsList()) { + requiredFilesForField[field.first].push_back(file.AsString()); + } + + bool passthrough = false; + if (field.second.HasKey("passthrough")) { + passthrough = field.second["passthrough"].AsBool(); + // check real field name + if (passthrough && (requiredColumns.size() != 1 || requiredColumns.front() != field.first)) { + passthrough = false; + } + } + + if (passthrough) { + passtroughFields.insert(field.first); + } + } + + TVector<TString> remappedFields; + if (tablePath.Columns_) { + TSet<TString> requiredColumns; + for (const auto& field : tablePath.Columns_->Parts_) { + allPasstrough = allPasstrough && passtroughFields.contains(field); + remappedFields.push_back(field); + auto columns = requiredColumnsForField.FindPtr(field); + YQL_ENSURE(columns, "Unknown column name in remapper specification: " << field << ", table: " << tableName); + requiredColumns.insert(columns->begin(), columns->end()); + } + + if (!allPasstrough) { + for (const auto& field : tablePath.Columns_->Parts_) { + auto files = requiredFilesForField.FindPtr(field); + if (files) { + remapperAllFiles.insert(files->begin(), files->end()); + } + } + tablePath.Columns(TVector<TString>(requiredColumns.begin(), requiredColumns.end())); + } + } + else { + // add all fields + for (const auto& field: remapper["fields"].AsMap()) { + allPasstrough = allPasstrough && passtroughFields.contains(field.first); + remappedFields.push_back(field.first); + } + + if (!allPasstrough) { + for (const auto& x : requiredFilesForField) { + remapperAllFiles.insert(x.second.begin(), x.second.end()); + } + } + } + + if (!allPasstrough) { + remapperAllFiles.insert(remapper["binary_path"].AsString()); + + TRemapperKey key = std::make_tuple(remapper["command_prefix"].AsString(), remapper["log_name"].AsString(), JoinSeq(",", remappedFields)); + remapperMap[key].push_back(tableNum); + + if (remapper.HasKey("skiff")) { + const auto& skiffMap = remapper["skiff"]; + useSkiff = skiffMap.HasKey("output") && skiffMap["output"].AsBool(); + } else { + useSkiff = false; + } + } + +} + +TString GetQB2PremapperPrefix(const TRemapperMap& remapperMap, bool useSkiff) { + TString remapperPrefix; + TString prevPremapperBinary; + for (const auto& x : remapperMap) { + if (remapperPrefix.empty()) { + remapperPrefix.append("set -o pipefail; "); + } + + const auto& premapperBinary = std::get<0>(x.first); + const auto& logName = std::get<1>(x.first); + const auto& fields = std::get<2>(x.first); + if (premapperBinary != prevPremapperBinary) { + if (!prevPremapperBinary.empty()) { + remapperPrefix.append(" | "); + } + + prevPremapperBinary = premapperBinary; + remapperPrefix.append(premapperBinary); + } + + if (useSkiff) { + remapperPrefix.append(" --output-format skiff"); + } + + remapperPrefix.append(" -l ").append(logName); + remapperPrefix.append(" -f ").append(fields); + remapperPrefix.append(" -t ").append(JoinSeq(",", x.second)); + } + + if (!remapperPrefix.empty()) { + remapperPrefix.append(" | "); + } + return remapperPrefix; +} + +void UpdateQB2PremapperUseSkiff(const TRemapperMap& remapperMap, bool& useSkiff) { + THashSet<TString> remapperBinaries; + + for (const auto& pair : remapperMap) { + remapperBinaries.insert(std::get<0>(pair.first)); + } + + // explicitly disable qb2 premapper's skiff mode + // as it won't work with pipeline from GetQB2PremapperPrefix + if (remapperBinaries.size() > 1) { + useSkiff = false; + } +} + +} // NNative + +} // NYql diff --git a/yt/yql/providers/yt/gateway/native/yql_yt_qb2.h b/yt/yql/providers/yt/gateway/native/yql_yt_qb2.h new file mode 100644 index 0000000000..0a80eaa8a0 --- /dev/null +++ b/yt/yql/providers/yt/gateway/native/yql_yt_qb2.h @@ -0,0 +1,34 @@ +#pragma once + +#include <yt/cpp/mapreduce/interface/common.h> + +#include <util/generic/set.h> +#include <util/generic/string.h> + +#include <map> +#include <tuple> +#include <vector> + +namespace NYql { + +namespace NNative { + +using TRemapperKey = std::tuple<TString, TString, TString>; // binary path (must be first), log name, all fields +using TRemapperPayload = std::vector<ui32>; // table indicies +using TRemapperMap = std::map<TRemapperKey, TRemapperPayload>; + +void ProcessTableQB2Premapper(const NYT::TNode& remapper, + const TString& tableName, + NYT::TRichYPath& tablePath, + size_t tableNum, + TRemapperMap& remapperMap, + TSet<TString>& remapperAllFiles, + bool& useSkiff); + +TString GetQB2PremapperPrefix(const TRemapperMap& remapperMap, bool useSkiff); + +void UpdateQB2PremapperUseSkiff(const TRemapperMap& remapperMap, bool& useSkiff); + +} // NNative + +} // NYql diff --git a/yt/yql/providers/yt/gateway/native/yql_yt_session.cpp b/yt/yql/providers/yt/gateway/native/yql_yt_session.cpp new file mode 100644 index 0000000000..f0ee328903 --- /dev/null +++ b/yt/yql/providers/yt/gateway/native/yql_yt_session.cpp @@ -0,0 +1,86 @@ +#include "yql_yt_session.h" + +#include <yt/yql/providers/yt/gateway/lib/yt_helpers.h> +#include <yt/yql/providers/yt/lib/init_yt_api/init.h> + +#include <yql/essentials/utils/log/log.h> + +#include <util/system/env.h> + +namespace NYql { + +namespace NNative { + +TSession::TSession(IYtGateway::TOpenSessionOptions&& options, size_t numThreads) + : UserName_(std::move(options.UserName())) + , ProgressWriter_(std::move(options.ProgressWriter())) + , StatWriter_(std::move(options.StatWriter())) + , OperationOptions_(std::move(options.OperationOptions())) + , RandomProvider_(std::move(options.RandomProvider())) + , TimeProvider_(std::move(options.TimeProvider())) + , DeterministicMode_(GetEnv("YQL_DETERMINISTIC_MODE")) + , OperationSemaphore(nullptr) + , LocalCalcSemaphore_(nullptr) + , TxCache_(UserName_) + , SessionId_(options.SessionId_) +{ + InitYtApiOnce(OperationOptions_.AttrsYson); + + Queue_ = TAsyncQueue::Make(numThreads, "YtGateway"); + if (options.CreateOperationTracker()) { + OpTracker_ = MakeIntrusive<TOperationTracker>(); + } +} + +void TSession::StopQueueAndTracker() { + if (OpTracker_) { + OpTracker_->Stop(); + } + Queue_->Stop(); +} + +void TSession::Close() { + if (OperationSemaphore) { + OperationSemaphore->Cancel(); + } + + try { + TxCache_.AbortAll(); + } catch (...) { + YQL_CLOG(ERROR, ProviderYt) << CurrentExceptionMessage(); + StopQueueAndTracker(); + throw; + } + + StopQueueAndTracker(); +} + +NYT::TNode TSession::CreateSpecWithDesc(const TVector<std::pair<TString, TString>>& code) const { + return YqlOpOptionsToSpec(OperationOptions_, UserName_, code); +} + +NYT::TNode TSession::CreateTableAttrs() const { + return YqlOpOptionsToAttrs(OperationOptions_); +} + +void TSession::EnsureInitializedSemaphore(const TYtSettings::TConstPtr& settings) { + with_lock(Mutex_) { + if (!OperationSemaphore) { + const size_t parallelOperationsLimit = settings->ParallelOperationsLimit.Get().GetOrElse(1U << 20); + OperationSemaphore = NThreading::TAsyncSemaphore::Make(parallelOperationsLimit); + } + } +} + +void TSession::InitLocalCalcSemaphore(const TYtSettings::TConstPtr& settings) { + with_lock(Mutex_) { + if(!LocalCalcSemaphore_) { + const size_t localCalcLimit = settings->LocalCalcLimit.Get().GetOrElse(1U); + LocalCalcSemaphore_ = MakeHolder<TFastSemaphore>(localCalcLimit); + } + } +} + +} // NNative + +} // NYql diff --git a/yt/yql/providers/yt/gateway/native/yql_yt_session.h b/yt/yql/providers/yt/gateway/native/yql_yt_session.h new file mode 100644 index 0000000000..2bdafefb24 --- /dev/null +++ b/yt/yql/providers/yt/gateway/native/yql_yt_session.h @@ -0,0 +1,65 @@ +#pragma once + +#include "yql_yt_op_tracker.h" + +#include <yt/yql/providers/yt/gateway/lib/transaction_cache.h> +#include <yt/yql/providers/yt/provider/yql_yt_gateway.h> + +#include <yql/essentials/core/yql_execution.h> +#include <yql/essentials/core/yql_type_annotation.h> +#include <yql/essentials/utils/threading/async_queue.h> + +#include <library/cpp/random_provider/random_provider.h> +#include <library/cpp/threading/future/async_semaphore.h> +#include <library/cpp/time_provider/time_provider.h> + +#include <util/generic/ptr.h> +#include <util/generic/string.h> +#include <util/generic/vector.h> +#include <util/system/mutex.h> +#include <util/system/sem.h> +#include <util/system/tempfile.h> +#include <util/thread/pool.h> + +#include <utility> + +namespace NYql { + +namespace NNative { + +struct TSession: public TThrRefBase { + using TPtr = TIntrusivePtr<TSession>; + + TSession(IYtGateway::TOpenSessionOptions&& options, size_t numThreads); + ~TSession() = default; + + void Close(); + NYT::TNode CreateSpecWithDesc(const TVector<std::pair<TString, TString>>& code = {}) const; + NYT::TNode CreateTableAttrs() const; + + void EnsureInitializedSemaphore(const TYtSettings::TConstPtr& settings); + void InitLocalCalcSemaphore(const TYtSettings::TConstPtr& settings); + + const TString UserName_; + const TOperationProgressWriter ProgressWriter_; + const TStatWriter StatWriter_; + const TYqlOperationOptions OperationOptions_; + const TIntrusivePtr<IRandomProvider> RandomProvider_; + const TIntrusivePtr<ITimeProvider> TimeProvider_; + const bool DeterministicMode_; + TAsyncQueue::TPtr Queue_; + TOperationTracker::TPtr OpTracker_; + NThreading::TAsyncSemaphore::TPtr OperationSemaphore; + TMutex Mutex_; + THolder<TFastSemaphore> LocalCalcSemaphore_; + + TTransactionCache TxCache_; + TString SessionId_; + +private: + void StopQueueAndTracker(); +}; + +} // NNative + +} // NYql diff --git a/yt/yql/providers/yt/gateway/native/yql_yt_spec.cpp b/yt/yql/providers/yt/gateway/native/yql_yt_spec.cpp new file mode 100644 index 0000000000..5044596539 --- /dev/null +++ b/yt/yql/providers/yt/gateway/native/yql_yt_spec.cpp @@ -0,0 +1,658 @@ +#include "yql_yt_spec.h" + +#include <yt/yql/providers/yt/common/yql_configuration.h> +#include <yql/essentials/providers/common/proto/gateways_config.pb.h> +#include <yql/essentials/utils/log/log.h> + +#include <yt/cpp/mapreduce/interface/config.h> +#include <yt/cpp/mapreduce/common/helpers.h> + +#include <library/cpp/yson/writer.h> +#include <library/cpp/digest/md5/md5.h> + +#include <util/stream/str.h> +#include <util/system/env.h> +#include <util/system/execpath.h> +#include <util/generic/size_literals.h> + + +namespace NYql { + +namespace NNative { + +namespace { + +ui64 GetCombiningDataSizePerJob(ui64 dataSizePerJob, TMaybe<ui64> minChunkSize) { + static const ui64 DefaultCombineChunkSize = 1_GB; + ui64 result = dataSizePerJob; + if (!minChunkSize.Defined() || *minChunkSize == 0) { + result = Max(result, DefaultCombineChunkSize); + } else { + result = Max(result, *minChunkSize); + } + return result; +} + +const TString& GetPersistentExecPathMd5() +{ + static TString md5 = MD5::File(GetPersistentExecPath()); + return md5; +} + +} + +TMaybe<TString> GetPool( + const TExecContextBase& execCtx, + const TYtSettings::TConstPtr& settings) +{ + TMaybe<TString> pool; + + if (auto val = settings->Pool.Get(execCtx.Cluster_)) { + pool = *val; + } + else if (auto val = settings->StaticPool.Get()) { + pool = *val; + } + else if (settings->Auth.Get().GetOrElse(TString()).empty()) { + pool = execCtx.Session_->UserName_; + } + + return pool; +} + +void FillSpec(NYT::TNode& spec, + const TExecContextBase& execCtx, + const TYtSettings::TConstPtr& settings, + const TTransactionCache::TEntry::TPtr& entry, + double extraCpu, + const TMaybe<double>& secondExtraCpu, + EYtOpProps opProps, + const TSet<TString>& addSecTags) +{ + auto& cluster = execCtx.Cluster_; + + if (auto val = settings->OperationSpec.Get(cluster)) { + NYT::TNode tmpSpec = *val; + NYT::MergeNodes(tmpSpec, spec); + spec = std::move(tmpSpec); + } + + auto& sampling = execCtx.Sampling; + auto maxRowWeight = settings->MaxRowWeight.Get(cluster); + auto maxKeyWeight = settings->MaxKeyWeight.Get(cluster); + auto bufferRowCount = settings->BufferRowCount.Get(cluster); + + if (maxRowWeight || maxKeyWeight || bufferRowCount || (sampling && opProps.HasFlags(EYtOpProp::AllowSampling))) { + NYT::TNode jobIO; + if (maxRowWeight) { + jobIO["table_writer"]["max_row_weight"] = static_cast<i64>(*maxRowWeight); + } + if (maxKeyWeight) { + jobIO["table_writer"]["max_key_weight"] = static_cast<i64>(*maxKeyWeight); + } + if (bufferRowCount) { + jobIO["buffer_row_count"] = static_cast<i64>(*bufferRowCount); + } + if (!jobIO.IsUndefined() && opProps.HasFlags(EYtOpProp::IntermediateData)) { + // Both Sort and MapReduce + spec["sort_job_io"] = jobIO; + if (opProps.HasFlags(EYtOpProp::WithUserJobs)) { + // MapReduce + spec["reduce_job_io"] = jobIO; + } + else { + // Sort + spec["partition_job_io"] = jobIO; + spec["merge_job_io"] = jobIO; + } + } + + // Set sampling only for input jobs + if (sampling && opProps.HasFlags(EYtOpProp::AllowSampling)) { + if (sampling->Mode == EYtSampleMode::System) { + NYT::TNode systemSamplingParams = NYT::TNode::CreateMap(); + systemSamplingParams["sampling_rate"] = sampling->Percentage / 100.; + if (auto blockSize = settings->SamplingIoBlockSize.Get(cluster)) { + systemSamplingParams["io_block_size"] = static_cast<i64>(*blockSize); + } + spec["sampling"] = systemSamplingParams; + } else if (sampling->Mode == EYtSampleMode::Bernoulli) { + jobIO["table_reader"]["sampling_rate"] = sampling->Percentage / 100.; + if (sampling->Repeat) { + jobIO["table_reader"]["sampling_seed"] = static_cast<i64>(sampling->Repeat); + } + } + } + if (!jobIO.IsUndefined()) { + if (!opProps.HasFlags(EYtOpProp::IntermediateData)) { + // Merge, Map, Reduce + spec["job_io"] = jobIO; + } else if (opProps.HasFlags(EYtOpProp::WithUserJobs)) { + // MapReduce + spec["map_job_io"] = jobIO; + } + } + } + + if (opProps.HasFlags(EYtOpProp::IntermediateData)) { + const auto intermediateMedium = settings->IntermediateDataMedium.Get(cluster); + if (intermediateMedium) { + spec["intermediate_data_medium"] = *intermediateMedium; + } + } + + ui64 dataSizePerJob = settings->DataSizePerJob.Get(cluster).GetOrElse(0); + if (opProps.HasFlags(EYtOpProp::PublishedChunkCombine)) { + spec["enable_job_splitting"] = false; + dataSizePerJob = GetCombiningDataSizePerJob(dataSizePerJob, settings->MinPublishedAvgChunkSize.Get()); + } else if (opProps.HasFlags(EYtOpProp::TemporaryChunkCombine)) { + spec["enable_job_splitting"] = false; + dataSizePerJob = GetCombiningDataSizePerJob(dataSizePerJob, settings->MinTempAvgChunkSize.Get()); + } + + if (opProps.HasFlags(EYtOpProp::IntermediateData) && opProps.HasFlags(EYtOpProp::WithUserJobs)) { // MapReduce + ui64 dataSizePerMapJob = dataSizePerJob; + ui64 dataSizePerPartition = dataSizePerJob; + if (auto val = settings->DataSizePerMapJob.Get(cluster).GetOrElse(0)) { + dataSizePerMapJob = val; + } + + if (auto val = settings->DataSizePerPartition.Get(cluster).GetOrElse(0)) { + dataSizePerPartition = val; + } + + if (dataSizePerMapJob) { + if (extraCpu != 0.) { + dataSizePerMapJob /= extraCpu; + } + spec["data_size_per_map_job"] = static_cast<i64>(Max<ui64>(dataSizePerMapJob, 1)); + } + if (dataSizePerPartition) { + auto secondExtraCpuVal = secondExtraCpu.GetOrElse(extraCpu); + if (secondExtraCpuVal != 0) { + dataSizePerPartition /= secondExtraCpuVal; + } + spec["partition_data_size"] = static_cast<i64>(Max<ui64>(dataSizePerPartition, 1)); + } + + if (auto val = settings->DataSizePerSortJob.Get(cluster)) { + spec["data_size_per_sort_job"] = static_cast<i64>(*val); + } + + } else if (!opProps.HasFlags(EYtOpProp::IntermediateData)) { // Exclude Sort + if (dataSizePerJob) { + if (extraCpu != 0.) { + dataSizePerJob /= extraCpu; + } + spec["data_size_per_job"] = static_cast<i64>(Max<ui64>(dataSizePerJob, 1)); + } + } + + NYT::TNode annotations; + if (auto val = settings->Annotations.Get(cluster)) { + annotations = NYT::TNode::CreateMap(val.Get()->AsMap()); + } else { + annotations = NYT::TNode::CreateMap(); + } + + // merge annotations from attributes + if (auto attrs = execCtx.Session_->OperationOptions_.AttrsYson.GetOrElse(TString())) { + NYT::TNode node = NYT::NodeFromYsonString(attrs); + if (auto attrAnnotations = node.AsMap().FindPtr("yt_annotations")) { + if (!attrAnnotations->IsMap()) { + throw yexception() << "Operation attribute \"yt_annotations\" should be a map"; + } + for (const auto& [k, v] : attrAnnotations->AsMap()) { + auto it = annotations.AsMap().find(k); + if (it == annotations.AsMap().end()) { + annotations[k] = v; + } + } + } + } + + if (!annotations.Empty()) { + spec["annotations"] = std::move(annotations); + } + + if (auto val = settings->StartedBy.Get(cluster)) { + spec["started_by"] = *val; + } + + if (auto val = settings->Description.Get(cluster)) { + spec["description"] = *val; + } + + if (!opProps.HasFlags(EYtOpProp::IntermediateData)) { + if (auto val = settings->MaxJobCount.Get(cluster)) { + spec["max_job_count"] = static_cast<i64>(*val); + } + } + + if (auto val = settings->UserSlots.Get(cluster)) { + spec["resource_limits"]["user_slots"] = static_cast<i64>(*val); + } + + if (auto pool = GetPool(execCtx, settings)) { + spec["pool"] = *pool; + } + + if (auto val = settings->SchedulingTag.Get(cluster)) { + spec["scheduling_tag"] = *val; + } + + if (auto val = settings->SchedulingTagFilter.Get(cluster)) { + spec["scheduling_tag_filter"] = *val; + } + + if (auto val = settings->PoolTrees.Get(cluster)) { + NYT::TNode trees = NYT::TNode::CreateList(); + for (auto& tree : *val) { + trees.AsList().push_back(tree); + } + spec["pool_trees"] = trees; + } + + if (auto val = settings->TentativePoolTrees.Get(cluster)) { + NYT::TNode trees = NYT::TNode::CreateList(); + NYT::TNode tree_eligibility = NYT::TNode::CreateMap(); + + for (auto& tree : *val) { + trees.AsList().push_back(tree); + } + + if (auto v = settings->TentativeTreeEligibilitySampleJobCount.Get(cluster)) { + tree_eligibility["sample_job_count"] = *v; + } + + if (auto v = settings->TentativeTreeEligibilityMaxJobDurationRatio.Get(cluster)) { + tree_eligibility["max_tentative_job_duration_ratio"] = *v; + } + + if (auto v = settings->TentativeTreeEligibilityMinJobDuration.Get(cluster)) { + tree_eligibility["min_job_duration"] = *v; + } + + spec["tentative_pool_trees"] = trees; + spec["tentative_tree_eligibility"] = tree_eligibility; + } + + if (auto val = settings->UseDefaultTentativePoolTrees.Get(cluster)) { + spec["use_default_tentative_pool_trees"] = *val; + } + + if (auto val = settings->DefaultOperationWeight.Get(cluster)) { + spec["weight"] = *val; + } + + if (auto val = settings->DefaultMapSelectivityFactor.Get(cluster)) { + spec["map_selectivity_factor"] = *val; + } + + NYT::TNode aclList; + TSet<TString> ownersSet = settings->Owners.Get(cluster).GetOrElse(TSet<TString>()); + if (!ownersSet.empty()) { + NYT::TNode owners = NYT::TNode::CreateList(); + for (auto& o : ownersSet) { + owners.Add(o); + } + + NYT::TNode acl = NYT::TNode::CreateMap(); + acl["subjects"] = owners; + acl["action"] = "allow"; + acl["permissions"] = NYT::TNode::CreateList().Add("read").Add("manage"); + + aclList.Add(std::move(acl)); + } + if (auto val = settings->OperationReaders.Get(cluster)) { + NYT::TNode readers; + for (auto& o : *val) { + if (!ownersSet.contains(o)) { + readers.Add(o); + } + } + if (!readers.IsUndefined()) { + NYT::TNode acl = NYT::TNode::CreateMap(); + acl["subjects"] = readers; + acl["action"] = "allow"; + acl["permissions"] = NYT::TNode::CreateList().Add("read"); + + aclList.Add(std::move(acl)); + } + } + if (!aclList.IsUndefined()) { + spec["acl"] = std::move(aclList); + } + + if (opProps.HasFlags(EYtOpProp::IntermediateData)) { + if (auto val = settings->IntermediateAccount.Get(cluster)) { + spec["intermediate_data_account"] = *val; + } + else if (auto tmpFolder = GetTablesTmpFolder(*settings)) { + auto attrs = entry->Tx->Get(tmpFolder + "/@", NYT::TGetOptions().AttributeFilter(NYT::TAttributeFilter().AddAttribute(TString("account")))); + if (attrs.HasKey("account")) { + spec["intermediate_data_account"] = attrs["account"]; + } + } + + // YT merges this ACL with operation ACL + // By passing empty list, we allow only user+owners accessing the intermediate data + // (note: missing "intermediate_data_acl" actually implies "everyone=read") + spec["intermediate_data_acl"] = NYT::TNode::CreateList(); + + if (auto val = settings->IntermediateReplicationFactor.Get(cluster)) { + spec["intermediate_data_replication_factor"] = static_cast<i64>(*val); + } + + } + + if (opProps.HasFlags(EYtOpProp::TemporaryAutoMerge)) { + if (auto val = settings->TemporaryAutoMerge.Get(cluster)) { + spec["auto_merge"]["mode"] = *val; + } + } + + if (opProps.HasFlags(EYtOpProp::PublishedAutoMerge)) { + if (auto val = settings->PublishedAutoMerge.Get(cluster)) { + spec["auto_merge"]["mode"] = *val; + } + } + + if (settings->UseTmpfs.Get(cluster).GetOrElse(false)) { + if (opProps.HasFlags(EYtOpProp::WithMapper)) { + spec["mapper"]["tmpfs_path"] = TString("_yql_tmpfs"); + } + if (opProps.HasFlags(EYtOpProp::WithReducer)) { + spec["reducer"]["tmpfs_path"] = TString("_yql_tmpfs"); + } + } + if (GetEnv(TString("YQL_DETERMINISTIC_MODE"))) { + if (opProps.HasFlags(EYtOpProp::WithMapper)) { + spec["mapper"]["environment"]["YQL_DETERMINISTIC_MODE"] = TString("1"); + } + if (opProps.HasFlags(EYtOpProp::WithReducer)) { + spec["reducer"]["environment"]["YQL_DETERMINISTIC_MODE"] = TString("1"); + } + } + if (auto envMap = settings->JobEnv.Get(cluster)) { + if (opProps.HasFlags(EYtOpProp::WithMapper)) { + for (auto& p: envMap->AsMap()) { + spec["mapper"]["environment"][p.first] = p.second; + } + } + if (opProps.HasFlags(EYtOpProp::WithReducer)) { + for (auto& p: envMap->AsMap()) { + spec["reducer"]["environment"][p.first] = p.second; + } + } + } + + if (settings->EnforceJobUtc.Get(cluster).GetOrElse(false)) { + if (opProps.HasFlags(EYtOpProp::WithMapper)) { + spec["mapper"]["environment"]["TZ"] = TString("UTC0"); + } + + if (opProps.HasFlags(EYtOpProp::WithReducer)) { + spec["reducer"]["environment"]["TZ"] = TString("UTC0"); + } + } + + if (settings->SuspendIfAccountLimitExceeded.Get(cluster).GetOrElse(false)) { + spec["suspend_operation_if_account_limit_exceeded"] = true; + } + + if (settings->DisableJobSplitting.Get(cluster).GetOrElse(false)) { + spec["enable_job_splitting"] = false; + } + + if (auto val = settings->DefaultMemoryReserveFactor.Get(cluster)) { + if (opProps.HasFlags(EYtOpProp::WithMapper)) { + spec["mapper"]["memory_reserve_factor"] = *val; + } + if (opProps.HasFlags(EYtOpProp::WithReducer)) { + spec["reducer"]["memory_reserve_factor"] = *val; + } + } + + if (auto val = settings->DefaultMemoryDigestLowerBound.Get(cluster)) { + if (opProps.HasFlags(EYtOpProp::WithMapper)) { + spec["mapper"]["user_job_memory_digest_lower_bound"] = *val; + } + if (opProps.HasFlags(EYtOpProp::WithReducer)) { + spec["reducer"]["user_job_memory_digest_lower_bound"] = *val; + } + } + + if (auto val = settings->DefaultLocalityTimeout.Get(cluster)) { + spec["locality_timeout"] = static_cast<i64>((*val).Seconds()); + } + + if (auto val = settings->MapLocalityTimeout.Get(cluster)) { + spec["map_locality_timeout"] = static_cast<i64>((*val).Seconds()); + } + + if (auto val = settings->ReduceLocalityTimeout.Get(cluster)) { + spec["reduce_locality_timeout"] = static_cast<i64>((*val).Seconds()); + } + + if (auto val = settings->SortLocalityTimeout.Get(cluster)) { + spec["sort_locality_timeout"] = static_cast<i64>((*val).Seconds()); + } + + if (auto val = settings->MinLocalityInputDataWeight.Get(cluster)) { + spec["min_locality_input_data_weight"] = static_cast<i64>(*val); + } + + if (auto val = settings->UseColumnarStatistics.Get(cluster)) { + bool flag = true; + switch (*val) { + case EUseColumnarStatisticsMode::Force: + break; + case EUseColumnarStatisticsMode::Disable: + flag = false; + break; + case EUseColumnarStatisticsMode::Auto: + if (AnyOf(execCtx.InputTables_, [](const auto& input) { return input.Lookup; })) { + flag = false; + } + break; + } + spec["input_table_columnar_statistics"]["enabled"] = flag; + } + + if (opProps.HasFlags(EYtOpProp::WithUserJobs)) { + spec["user_file_columnar_statistics"]["enabled"] = settings->TableContentColumnarStatistics.Get(cluster).GetOrElse(true); + } + + if (auto val = settings->LayerPaths.Get(cluster)) { + if (opProps.HasFlags(EYtOpProp::WithMapper)) { + NYT::TNode& layersNode = spec["mapper"]["layer_paths"]; + for (auto& path: *val) { + layersNode.Add(NYT::AddPathPrefix(path, NYT::TConfig::Get()->Prefix)); + } + } + if (opProps.HasFlags(EYtOpProp::WithReducer)) { + NYT::TNode& layersNode = spec["reducer"]["layer_paths"]; + for (auto& path: *val) { + layersNode.Add(NYT::AddPathPrefix(path, NYT::TConfig::Get()->Prefix)); + } + } + } + + if (auto val = settings->DockerImage.Get(cluster)) { + if (opProps.HasFlags(EYtOpProp::WithMapper)) { + spec["mapper"]["docker_image"] = *val; + } + if (opProps.HasFlags(EYtOpProp::WithReducer)) { + spec["reducer"]["docker_image"] = *val; + } + } + + if (auto val = settings->MaxSpeculativeJobCountPerTask.Get(cluster)) { + spec["max_speculative_job_count_per_task"] = i64(*val); + } + + if (auto val = settings->NetworkProject.Get(cluster)) { + if (opProps.HasFlags(EYtOpProp::WithMapper)) { + spec["mapper"]["network_project"] = *val; + } + if (opProps.HasFlags(EYtOpProp::WithReducer)) { + spec["reducer"]["network_project"] = *val; + } + } + if (!opProps.HasFlags(EYtOpProp::IntermediateData)) { + if (auto val = settings->ForceJobSizeAdjuster.Get(cluster)) { + spec["force_job_size_adjuster"] = *val; + } + } + + if (opProps.HasFlags(EYtOpProp::WithMapper)) { + spec["mapper"]["environment"]["TMPDIR"] = "."; + } + + if (opProps.HasFlags(EYtOpProp::WithReducer)) { + spec["reducer"]["environment"]["TMPDIR"] = "."; + } + + if (!addSecTags.empty()) { + auto secTagsNode = NYT::TNode::CreateList(); + for (const auto& tag : addSecTags) { + secTagsNode.Add(tag); + } + spec["additional_security_tags"] = std::move(secTagsNode); + } +} + +void FillSecureVault(NYT::TNode& spec, const IYtGateway::TSecureParams& secureParams) { + if (secureParams.empty()) { + return; + } + TStringStream out; + NYson::TYsonWriter writer(&out, NYson::EYsonFormat::Text); + writer.OnBeginMap(); + for (const auto& it : secureParams) { + writer.OnKeyedItem(it.first); + writer.OnStringScalar(it.second); + } + writer.OnEndMap(); + spec["secure_vault"]["secure_params"] = out.Str(); +} + +void FillUserJobSpecImpl(NYT::TUserJobSpec& spec, + const TExecContextBase& execCtx, + const TYtSettings::TConstPtr& settings, + const TExpressionResorceUsage& extraUsage, + ui64 fileMemUsage, + ui64 llvmMemUsage, + bool localRun, + const TString& cmdPrefix) +{ + auto cluster = execCtx.Cluster_; + auto mrJobBin = execCtx.Config_->GetMrJobBin(); + TMaybe<TString> mrJobBinMd5; + if (!mrJobBin.empty()) { + if (execCtx.Config_->HasMrJobBinMd5()) { + mrJobBinMd5 = execCtx.Config_->GetMrJobBinMd5(); + } else { + YQL_CLOG(WARN, ProviderYt) << "MrJobBin without MD5"; + } + } + + if (!localRun) { + if (mrJobBin.empty()) { + mrJobBinMd5 = GetPersistentExecPathMd5(); + } else if (!mrJobBinMd5) { + if (GetEnv("YQL_LOCAL") == "1") { + // do not calculate heavy md5 in local mode (YQL-15353) + mrJobBinMd5 = MD5::Calc(mrJobBin); + } else { + mrJobBinMd5 = MD5::File(mrJobBin); + } + } + } + + const TString binTmpFolder = settings->BinaryTmpFolder.Get().GetOrElse(TString()); + const TString binCacheFolder = settings->_BinaryCacheFolder.Get(cluster).GetOrElse(TString()); + if (!localRun && (binTmpFolder || binCacheFolder)) { + TString bin = mrJobBin.empty() ? GetPersistentExecPath() : mrJobBin; + const auto binSize = TFileStat(bin).Size; + YQL_ENSURE(binSize != 0); + fileMemUsage += binSize; + TTransactionCache::TEntry::TPtr entry = execCtx.GetOrCreateEntry(settings); + bool useBinCache = false; + if (binCacheFolder) { + if (auto snapshot = entry->GetBinarySnapshotFromCache(binCacheFolder, *mrJobBinMd5, "mrjob")) { + spec.JobBinaryCypressPath(snapshot->first, snapshot->second); + useBinCache = true; + } + } + if (!useBinCache) { + if (binTmpFolder) { + const TDuration binExpiration = settings->BinaryExpirationInterval.Get().GetOrElse(TDuration()); + auto mrJobSnapshot = entry->GetBinarySnapshot(binTmpFolder, *mrJobBinMd5, bin, binExpiration); + spec.JobBinaryCypressPath(mrJobSnapshot.first, mrJobSnapshot.second); + } else if (!mrJobBin.empty()) { + spec.JobBinaryLocalPath(mrJobBin, mrJobBinMd5); + } + + } + } + else if (!mrJobBin.empty()) { + const auto binSize = TFileStat(mrJobBin).Size; + YQL_ENSURE(binSize != 0); + spec.JobBinaryLocalPath(mrJobBin, mrJobBinMd5); + fileMemUsage += binSize; + } + auto defaultMemoryLimit = settings->DefaultMemoryLimit.Get(cluster).GetOrElse(0); + ui64 tmpFsSize = settings->UseTmpfs.Get(cluster).GetOrElse(false) + ? (ui64)settings->ExtraTmpfsSize.Get(cluster).GetOrElse(8_MB) + : ui64(0); + + if (defaultMemoryLimit || fileMemUsage || llvmMemUsage || extraUsage.Memory || tmpFsSize) { + const ui64 memIoBuffers = YQL_JOB_CODEC_MEM * (static_cast<size_t>(!execCtx.InputTables_.empty()) + execCtx.OutTables_.size()); + const ui64 finalMemLimit = Max<ui64>( + defaultMemoryLimit, + 128_MB + fileMemUsage + extraUsage.Memory + tmpFsSize + memIoBuffers, + llvmMemUsage + memIoBuffers // LLVM consumes memory only once on job start, but after IO initialization + ); + YQL_CLOG(DEBUG, ProviderYt) << "Job memory limit: " << finalMemLimit + << " (from options: " << defaultMemoryLimit + << ", files: " << fileMemUsage + << ", llvm: " << llvmMemUsage + << ", extra: " << extraUsage.Memory + << ", extra tmpfs: " << tmpFsSize + << ", I/O buffers: " << memIoBuffers + << ")"; + spec.MemoryLimit(static_cast<i64>(finalMemLimit)); + } + + if (cmdPrefix) { + spec.JobCommandPrefix(cmdPrefix); + } +} + +void FillOperationOptionsImpl(NYT::TOperationOptions& opOpts, + const TYtSettings::TConstPtr& settings, + const TTransactionCache::TEntry::TPtr& entry) +{ + opOpts.UseTableFormats(true); + opOpts.CreateOutputTables(false); + if (TString tmpFolder = settings->TmpFolder.Get().GetOrElse(TString())) { + opOpts.FileStorage(tmpFolder); + + if (!entry->CacheTxId.IsEmpty()) { + opOpts.FileStorageTransactionId(entry->CacheTxId); + + // We need to switch to random-path-upload cache mode because of + // specified 'FileStorageTransactionId' (see https://st.yandex-team.ru/YT-8462). + opOpts.FileCacheMode(NYT::TOperationOptions::EFileCacheMode::CachelessRandomPathUpload); + } + } + if (auto ttl = settings->FileCacheTtl.Get().GetOrElse(TDuration::Days(7))) { + opOpts.FileExpirationTimeout(ttl); + } +} + +} // NNative + +} // NYql diff --git a/yt/yql/providers/yt/gateway/native/yql_yt_spec.h b/yt/yql/providers/yt/gateway/native/yql_yt_spec.h new file mode 100644 index 0000000000..c40caf9a39 --- /dev/null +++ b/yt/yql/providers/yt/gateway/native/yql_yt_spec.h @@ -0,0 +1,125 @@ +#pragma once + +#include "yql_yt_exec_ctx.h" + +#include <yt/yql/providers/yt/gateway/lib/transaction_cache.h> + +#include <yt/yql/providers/yt/lib/expr_traits/yql_expr_traits.h> +#include <yt/yql/providers/yt/provider/yql_yt_gateway.h> + +#include <yt/cpp/mapreduce/interface/operation.h> +#include <library/cpp/yson/node/node.h> + +#include <util/generic/flags.h> +#include <util/generic/maybe.h> +#include <util/generic/string.h> +#include <util/generic/typetraits.h> + +namespace NYql { + +namespace NNative { + +enum class EYtOpProp: ui32 { + IntermediateData = 1 << 0, + TemporaryAutoMerge = 1 << 1, + PublishedAutoMerge = 1 << 2, + WithMapper = 1 << 3, + WithReducer = 1 << 4, + WithUserJobs = 1 << 5, + AllowSampling = 1 << 6, + TemporaryChunkCombine = 1 << 7, + PublishedChunkCombine = 1 << 8, +}; + +Y_DECLARE_FLAGS(EYtOpProps, EYtOpProp); +Y_DECLARE_OPERATORS_FOR_FLAGS(EYtOpProps); + +/////////////////////////////////////////////////////////////////////////////////////////////////////// + +TMaybe<TString> GetPool( + const TExecContextBase& execCtx, + const TYtSettings::TConstPtr& settings); + +void FillSpec(NYT::TNode& spec, + const TExecContextBase& execCtx, + const TYtSettings::TConstPtr& settings, + const TTransactionCache::TEntry::TPtr& entry, + double extraCpu, + const TMaybe<double>& secondExtraCpu, + EYtOpProps opProps = 0, + const TSet<TString>& addSecTags = {}); + +void FillSecureVault(NYT::TNode& spec, const IYtGateway::TSecureParams& secureParams); + +void FillUserJobSpecImpl(NYT::TUserJobSpec& spec, + const TExecContextBase& execCtx, + const TYtSettings::TConstPtr& settings, + const TExpressionResorceUsage& extraUsage, + ui64 fileMemUsage, + ui64 llvmMemUsage, + bool localRun, + const TString& cmdPrefix); + +void FillOperationOptionsImpl(NYT::TOperationOptions& opOpts, + const TYtSettings::TConstPtr& settings, + const TTransactionCache::TEntry::TPtr& entry); + +/////////////////////////////////////////////////////////////////////////////////////////////////////// + +namespace NPrivate { + Y_HAS_MEMBER(SecureParams); + Y_HAS_MEMBER(AdditionalSecurityTags); +} + +template <class TOptions> +inline void FillSpec(NYT::TNode& spec, + const TExecContext<TOptions>& execCtx, + const TTransactionCache::TEntry::TPtr& entry, + double extraCpu, + const TMaybe<double>& secondExtraCpu, + EYtOpProps opProps = 0) +{ + TSet<TString> addSecTags = {}; + if constexpr (NPrivate::THasAdditionalSecurityTags<TOptions>::value) { + addSecTags = execCtx.Options_.AdditionalSecurityTags(); + } + FillSpec(spec, execCtx, execCtx.Options_.Config(), entry, extraCpu, secondExtraCpu, opProps, addSecTags); + if constexpr (NPrivate::THasSecureParams<TOptions>::value) { + FillSecureVault(spec, execCtx.Options_.SecureParams()); + } +} + +template <class TDerived, class TExecParamsPtr> +inline void FillOperationSpec(NYT::TUserOperationSpecBase<TDerived>& spec, const TExecParamsPtr& execCtx) { + if (auto val = execCtx->Options_.Config()->DefaultMaxJobFails.Get()) { + spec.MaxFailedJobCount(*val); + } + if (auto val = execCtx->Options_.Config()->CoreDumpPath.Get()) { + spec.CoreTablePath(*val); + } + +} + +template <class TExecParamsPtr> +inline void FillUserJobSpec(NYT::TUserJobSpec& spec, + const TExecParamsPtr& execCtx, + const TExpressionResorceUsage& extraUsage, + ui64 fileMemUsage, + ui64 llvmMemUsage, + bool localRun, + const TString& cmdPrefix = {}) +{ + FillUserJobSpecImpl(spec, *execCtx, execCtx->Options_.Config(), extraUsage, fileMemUsage, llvmMemUsage, localRun, cmdPrefix); +} + +template <class TExecParamsPtr> +inline void FillOperationOptions(NYT::TOperationOptions& opOpts, + const TExecParamsPtr& execCtx, + const TTransactionCache::TEntry::TPtr& entry) +{ + FillOperationOptionsImpl(opOpts, execCtx->Options_.Config(), entry); +} + +} // NNative + +} // NYql diff --git a/yt/yql/providers/yt/gateway/native/yql_yt_transform.cpp b/yt/yql/providers/yt/gateway/native/yql_yt_transform.cpp new file mode 100644 index 0000000000..d66943f868 --- /dev/null +++ b/yt/yql/providers/yt/gateway/native/yql_yt_transform.cpp @@ -0,0 +1,584 @@ +#include "yql_yt_transform.h" + +#include <yt/yql/providers/yt/lib/skiff/yql_skiff_schema.h> +#include <yt/yql/providers/yt/common/yql_names.h> +#include <yt/yql/providers/yt/common/yql_configuration.h> +#include <yql/essentials/providers/common/provider/yql_provider.h> +#include <yt/yql/providers/yt/codec/yt_codec.h> +#include <yt/yql/providers/yt/gateway/lib/yt_helpers.h> +#include <yt/yql/providers/yt/expr_nodes/yql_yt_expr_nodes.h> +#include <yql/essentials/providers/common/codec/yql_codec_type_flags.h> + +#include <yql/essentials/utils/log/log.h> +#include <yql/essentials/minikql/mkql_node_cast.h> +#include <yql/essentials/utils/yql_panic.h> + +#include <yt/cpp/mapreduce/interface/common.h> +#include <yt/cpp/mapreduce/interface/serialize.h> +#include <yt/cpp/mapreduce/common/helpers.h> +#include <yt/cpp/mapreduce/common/wait_proxy.h> +#include <yt/cpp/mapreduce/interface/config.h> + +#include <library/cpp/yson/node/node_io.h> +#include <library/cpp/streams/brotli/brotli.h> + +#include <util/system/env.h> +#include <util/folder/path.h> +#include <util/generic/maybe.h> +#include <util/generic/size_literals.h> + +namespace NYql { + +namespace NNative { + +using namespace NKikimr; +using namespace NKikimr::NMiniKQL; +using namespace NYT; +using namespace NNodes; + +TGatewayTransformer::TGatewayTransformer(const TExecContextBase& execCtx, TYtSettings::TConstPtr settings, const TString& optLLVM, + TUdfModulesTable udfModules, IUdfResolver::TPtr udfResolver, TTransactionCache::TEntry::TPtr entry, + TProgramBuilder& builder, TTempFiles& tmpFiles, TMaybe<ui32> publicId) + : ExecCtx_(execCtx) + , Settings_(std::move(settings)) + , UdfModules_(std::move(udfModules)) + , UdfResolver_(std::move(udfResolver)) + , Entry_(std::move(entry)) + , PgmBuilder_(builder) + , TmpFiles_(tmpFiles) + , PublicId_(publicId) + , ForceLocalTableContent_(false) + , TableContentFlag_(std::make_shared<bool>(false)) + , RemoteExecutionFlag_(std::make_shared<bool>(false)) + , UntrustedUdfFlag_(std::make_shared<bool>(false)) + , UsedMem_(std::make_shared<ui64>(ui64(0))) + , JobFileAliases_(std::make_shared<THashMap<TString, TString>>()) + , JobUdfs_(std::make_shared<THashMap<TString, TString>>()) + , UniqFiles_(std::make_shared<THashMap<TString, TString>>()) + , RemoteFiles_(std::make_shared<TVector<NYT::TRichYPath>>()) + , LocalFiles_(std::make_shared<TVector<std::pair<TString, TLocalFileInfo>>>()) + , DeferredUdfFiles_(std::make_shared<TVector<std::pair<TString, TLocalFileInfo>>>()) + +{ + if (optLLVM != "OFF") { + *UsedMem_ = 128_MB; + } + + for (const auto& f: ExecCtx_.UserFiles_->GetFiles()) { + if (f.second.IsPgExt || f.second.IsPgCatalog) { + AddFile(f.second.IsPgCatalog ? TString(NCommon::PgCatalogFileName) : "", f.second); + } + } +} + +TCallableVisitFunc TGatewayTransformer::operator()(TInternName internName) { + auto name = internName.Str(); + const bool small = name.SkipPrefix("Small"); + if (name == TYtTableContent::CallableName()) { + + *TableContentFlag_ = true; + *RemoteExecutionFlag_ = *RemoteExecutionFlag_ || !small; + + if (EPhase::Content == Phase_ || EPhase::All == Phase_) { + return [&](NMiniKQL::TCallable& callable, const TTypeEnvironment& env) { + YQL_ENSURE(callable.GetInputsCount() == 4, "Expected 4 args"); + + const TString cluster(AS_VALUE(TDataLiteral, callable.GetInput(0))->AsValue().AsStringRef()); + const TString& server = ExecCtx_.Clusters_->GetServer(cluster); + const TString tmpFolder = GetTablesTmpFolder(*Settings_); + TTransactionCache::TEntry::TPtr entry = ExecCtx_.Session_->TxCache_.GetEntry(server); + auto tx = entry->Tx; + + auto deliveryMode = ForceLocalTableContent_ ? ETableContentDeliveryMode::File : Settings_->TableContentDeliveryMode.Get(cluster).GetOrElse(ETableContentDeliveryMode::Native); + bool useSkiff = Settings_->TableContentUseSkiff.Get(cluster).GetOrElse(DEFAULT_USE_SKIFF); + const bool ensureOldTypesOnly = !useSkiff; + const ui64 maxChunksForNativeDelivery = Settings_->TableContentMaxChunksForNativeDelivery.Get().GetOrElse(1000ul); + TString contentTmpFolder = ForceLocalTableContent_ ? TString() : Settings_->TableContentTmpFolder.Get(cluster).GetOrElse(TString()); + if (contentTmpFolder.StartsWith("//")) { + contentTmpFolder = contentTmpFolder.substr(2); + } + if (contentTmpFolder.EndsWith('/')) { + contentTmpFolder.remove(contentTmpFolder.length() - 1); + } + + TString uniqueId = GetGuidAsString(ExecCtx_.Session_->RandomProvider_->GenGuid()); + + TListLiteral* groupList = AS_VALUE(TListLiteral, callable.GetInput(1)); + YQL_ENSURE(groupList->GetItemsCount() == 1); + TListLiteral* tableList = AS_VALUE(TListLiteral, groupList->GetItems()[0]); + + NYT::TNode specNode = NYT::TNode::CreateMap(); + NYT::TNode& tablesNode = specNode[YqlIOSpecTables]; + NYT::TNode& registryNode = specNode[YqlIOSpecRegistry]; + THashMap<TString, TString> uniqSpecs; + TVector<NYT::TRichYPath> richPaths; + TVector<NYT::TNode> formats; + + THashMap<TString, ui32> structColumns; + if (useSkiff) { + auto itemType = AS_TYPE(TListType, callable.GetType()->GetReturnType())->GetItemType(); + TStructType* itemTypeStruct = AS_TYPE(TStructType, itemType); + if (itemTypeStruct->GetMembersCount() == 0) { + useSkiff = false; // TODO: YT-12235 + } else { + for (ui32 index = 0; index < itemTypeStruct->GetMembersCount(); ++index) { + structColumns.emplace(itemTypeStruct->GetMemberName(index), index); + } + } + } + + for (ui32 i = 0; i < tableList->GetItemsCount(); ++i) { + TTupleLiteral* tuple = AS_VALUE(TTupleLiteral, tableList->GetItems()[i]); + YQL_ENSURE(tuple->GetValuesCount() == 7, "Expect 7 elements in the Tuple item"); + + TString refName = TStringBuilder() << "$table" << uniqSpecs.size(); + TString specStr = TString(AS_VALUE(TDataLiteral, tuple->GetValue(2))->AsValue().AsStringRef()); + const auto specNode = NYT::NodeFromYsonString(specStr); + + NYT::TRichYPath richYPath; + NYT::Deserialize(richYPath, NYT::NodeFromYsonString(TString(AS_VALUE(TDataLiteral, tuple->GetValue(0))->AsValue().AsStringRef()))); + const bool isTemporary = AS_VALUE(TDataLiteral, tuple->GetValue(1))->AsValue().Get<bool>(); + const bool isAnonymous = AS_VALUE(TDataLiteral, tuple->GetValue(5))->AsValue().Get<bool>(); + const ui32 epoch = AS_VALUE(TDataLiteral, tuple->GetValue(6))->AsValue().Get<ui32>(); + + auto tablePath = TransformPath(tmpFolder, richYPath.Path_, isTemporary, ExecCtx_.Session_->UserName_); + + auto res = uniqSpecs.emplace(specStr, refName); + if (res.second) { + registryNode[refName] = specNode; + } + else { + refName = res.first->second; + } + tablesNode.Add(refName); + if (useSkiff) { + formats.push_back(SingleTableSpecToInputSkiff(specNode, structColumns, false, false, false)); + } else { + if (ensureOldTypesOnly && specNode.HasKey(YqlRowSpecAttribute)) { + EnsureSpecDoesntUseNativeYtTypes(specNode, tablePath, true); + } + NYT::TNode formatNode("yson"); + formatNode.Attributes()["format"] = "binary"; + formats.push_back(formatNode); + } + + if (isTemporary && !isAnonymous) { + richYPath.Path_ = NYT::AddPathPrefix(tablePath, NYT::TConfig::Get()->Prefix); + } else { + auto p = entry->Snapshots.FindPtr(std::make_pair(tablePath, epoch)); + YQL_ENSURE(p, "Table " << tablePath << " has no snapshot"); + richYPath.Path(std::get<0>(*p)).TransactionId(std::get<1>(*p)); + } + richPaths.push_back(richYPath); + + const ui64 chunkCount = AS_VALUE(TDataLiteral, tuple->GetValue(3))->AsValue().Get<ui64>(); + if (chunkCount > maxChunksForNativeDelivery) { + deliveryMode = ETableContentDeliveryMode::File; + YQL_CLOG(DEBUG, ProviderYt) << "Switching to file delivery mode, because table " + << tablePath.Quote() << " has too many chunks: " << chunkCount; + } + } + + for (size_t i = 0; i < richPaths.size(); ++i) { + NYT::TRichYPath richYPath = richPaths[i]; + + TString richYPathDesc = NYT::NodeToYsonString(NYT::PathToNode(richYPath)); + TString fileName = TStringBuilder() << uniqueId << '_' << i; + + if (ETableContentDeliveryMode::Native == deliveryMode) { + richYPath.Format(formats[i]); + richYPath.FileName(fileName); + RemoteFiles_->push_back(richYPath); + + YQL_CLOG(DEBUG, ProviderYt) << "Passing table " << richYPathDesc << " as remote file " + << fileName.Quote(); + } + else { + NYT::TTableReaderOptions readerOptions; + readerOptions.CreateTransaction(false); + + auto readerTx = tx; + if (richYPath.TransactionId_) { + readerTx = entry->GetSnapshotTx(*richYPath.TransactionId_); + richYPath.TransactionId_.Clear(); + } + + TTupleLiteral* samplingTuple = AS_VALUE(TTupleLiteral, callable.GetInput(2)); + if (samplingTuple->GetValuesCount() != 0) { + YQL_ENSURE(samplingTuple->GetValuesCount() == 3); + double samplingPercent = AS_VALUE(TDataLiteral, samplingTuple->GetValue(0))->AsValue().Get<double>(); + ui64 samplingSeed = AS_VALUE(TDataLiteral, samplingTuple->GetValue(1))->AsValue().Get<ui64>(); + bool isSystemSampling = AS_VALUE(TDataLiteral, samplingTuple->GetValue(2))->AsValue().Get<bool>(); + if (!isSystemSampling) { + NYT::TNode spec = NYT::TNode::CreateMap(); + spec["sampling_rate"] = samplingPercent / 100.; + if (samplingSeed) { + spec["sampling_seed"] = static_cast<i64>(samplingSeed); + } + readerOptions.Config(spec); + } + } + + TRawTableReaderPtr reader; + const int lastAttempt = NYT::TConfig::Get()->ReadRetryCount - 1; + for (int attempt = 0; attempt <= lastAttempt; ++attempt) { + try { + reader = readerTx->CreateRawReader(richYPath, NYT::TFormat(formats[i]), readerOptions); + break; + } catch (const NYT::TErrorResponse& e) { + YQL_CLOG(ERROR, ProviderYt) << "Error creating reader for " << richYPathDesc << ": " << e.what(); + // Already retried inside CreateRawReader + throw; + } catch (const yexception& e) { + YQL_CLOG(ERROR, ProviderYt) << "Error creating reader for " << richYPathDesc << ": " << e.what(); + if (attempt == lastAttempt) { + throw; + } + NYT::NDetail::TWaitProxy::Get()->Sleep(NYT::TConfig::Get()->RetryInterval); + } + } + + if (contentTmpFolder) { + entry->GetRoot()->Create(contentTmpFolder, NT_MAP, + TCreateOptions().Recursive(true).IgnoreExisting(true)); + + auto remotePath = TString(contentTmpFolder).append('/').append(fileName); + + while (true) { + try { + auto out = tx->CreateFileWriter(TRichYPath(remotePath)); + TBrotliCompress compressor(out.Get(), Settings_->TableContentCompressLevel.Get(cluster).GetOrElse(8)); + TransferData(reader.Get(), &compressor); + compressor.Finish(); + out->Finish(); + } catch (const yexception& e) { + YQL_CLOG(ERROR, ProviderYt) << "Error transferring " << richYPathDesc << " to " << remotePath << ": " << e.what(); + if (reader->Retry(Nothing(), Nothing(), std::make_exception_ptr(e))) { + continue; + } + throw; + } + break; + } + entry->DeleteAtFinalize(remotePath); + YQL_CLOG(DEBUG, ProviderYt) << "Passing table " << richYPathDesc << " as remote file " << remotePath.Quote(); + + RemoteFiles_->push_back(NYT::TRichYPath(NYT::AddPathPrefix(remotePath, NYT::TConfig::Get()->Prefix)).FileName(fileName)); + + } else { + TString outPath = TmpFiles_.AddFile(fileName); + + if (PublicId_) { + auto progress = TOperationProgress(TString(YtProviderName), *PublicId_, + TOperationProgress::EState::InProgress, "Preparing table content"); + ExecCtx_.Session_->ProgressWriter_(progress); + } + while (true) { + try { + TOFStream out(outPath); + out.SetFinishPropagateMode(false); + out.SetFlushPropagateMode(false); + TBrotliCompress compressor(&out, Settings_->TableContentCompressLevel.Get(cluster).GetOrElse(8)); + TransferData(reader.Get(), &compressor); + compressor.Finish(); + out.Finish(); + } catch (const TIoException& e) { + YQL_CLOG(ERROR, ProviderYt) << "Error reading " << richYPathDesc << ": " << e.what(); + // Don't retry IO errors + throw; + } catch (const yexception& e) { + YQL_CLOG(ERROR, ProviderYt) << "Error reading " << richYPathDesc << ": " << e.what(); + if (reader->Retry(Nothing(), Nothing(), std::make_exception_ptr(e))) { + continue; + } + throw; + } + break; + } + YQL_CLOG(DEBUG, ProviderYt) << "Passing table " << richYPathDesc << " as file " + << fileName.Quote() << " (size=" << TFileStat(outPath).Size << ')'; + + LocalFiles_->emplace_back(outPath, TLocalFileInfo{TString(), false}); + } + } + } + + TCallableBuilder call(env, + TStringBuilder() << TYtTableContent::CallableName() << TStringBuf("Job"), + callable.GetType()->GetReturnType()); + + call.Add(PgmBuilder_.NewDataLiteral<NUdf::EDataSlot::String>(uniqueId)); + call.Add(PgmBuilder_.NewDataLiteral(tableList->GetItemsCount())); + call.Add(PgmBuilder_.NewDataLiteral<NUdf::EDataSlot::String>(NYT::NodeToYsonString(specNode))); + call.Add(PgmBuilder_.NewDataLiteral(useSkiff)); + call.Add(PgmBuilder_.NewDataLiteral(ETableContentDeliveryMode::File == deliveryMode)); // use compression + call.Add(callable.GetInput(3)); // length + return TRuntimeNode(call.Build(), false); + }; + } + } + + if (EPhase::Other == Phase_ || EPhase::All == Phase_) { + if (name == TStringBuf("Udf") || name == TStringBuf("ScriptUdf")) { + return [&](NMiniKQL::TCallable& callable, const TTypeEnvironment& /*env*/) { + YQL_ENSURE(callable.GetInputsCount() > 0, "Expected at least one argument"); + const TString udfName(AS_VALUE(TDataLiteral, callable.GetInput(0))->AsValue().AsStringRef()); + const auto moduleName = ModuleName(udfName); + + *UntrustedUdfFlag_ = *UntrustedUdfFlag_ || + callable.GetType()->GetName() == TStringBuf("ScriptUdf") || + !ExecCtx_.FunctionRegistry_->IsLoadedUdfModule(moduleName) || + moduleName == TStringBuf("Geo"); + + if (moduleName.StartsWith("SystemPython")) { + *RemoteExecutionFlag_ = true; + } + + const auto udfPath = FindUdfPath(moduleName); + if (!udfPath.StartsWith(NMiniKQL::StaticModulePrefix)) { + const auto fileInfo = ExecCtx_.UserFiles_->GetFile(udfPath); + YQL_ENSURE(fileInfo, "Unknown udf path " << udfPath); + AddFile(udfPath, *fileInfo, FindUdfPrefix(moduleName)); + } + + if (moduleName == TStringBuf("Geo")) { + if (const auto fileInfo = ExecCtx_.UserFiles_->GetFile("/home/geodata6.bin")) { + AddFile("./geodata6.bin", *fileInfo); + } + } + + return TRuntimeNode(&callable, false); + }; + } + + if (name == TStringBuf("FilePath") || name == TStringBuf("FileContent") || name == TStringBuf("FolderPath")) { + if (name == TStringBuf("FolderPath")) { + *RemoteExecutionFlag_ = true; + } + return [&](NMiniKQL::TCallable& callable, const TTypeEnvironment& env) { + YQL_ENSURE(callable.GetInputsCount() == 1, "Expected 1 arguments"); + const TString fileName(AS_VALUE(TDataLiteral, callable.GetInput(0))->AsValue().AsStringRef()); + const TString fullFileName = TUserDataStorage::MakeFullName(fileName); + + auto callableName = callable.GetType()->GetName(); + if (callableName == TStringBuf("FolderPath")) { + TVector<TString> files; + if (!ExecCtx_.UserFiles_->FindFolder(fullFileName, files)) { + ythrow yexception() << "Folder not found: " << fullFileName; + } + for (const auto& x : files) { + auto fileInfo = ExecCtx_.UserFiles_->GetFile(x); + YQL_ENSURE(fileInfo, "File not found: " << x); + AddFile(x, *fileInfo); + } + } else { + auto fileInfo = ExecCtx_.UserFiles_->GetFile(fullFileName); + YQL_ENSURE(fileInfo, "File not found: " << fullFileName); + AddFile(fileName, *fileInfo); + } + + TStringBuilder jobCallable; + if (callableName == TStringBuf("FolderPath")) { + jobCallable << "FilePath"; + } else { + jobCallable << callableName; + } + + TCallableBuilder builder(env, jobCallable << TStringBuf("Job"), + callable.GetType()->GetReturnType(), false); + builder.Add(PgmBuilder_.NewDataLiteral<NUdf::EDataSlot::String>(fullFileName)); + return TRuntimeNode(builder.Build(), false); + }; + } + + if (name == TYtTablePath::CallableName() + || name == TYtTableIndex::CallableName() + || name == TYtTableRecord::CallableName() + || name == TYtIsKeySwitch::CallableName() + || name == TYtRowNumber::CallableName()) + { + *RemoteExecutionFlag_ = true; + } + } + + return TCallableVisitFunc(); +} + +bool TGatewayTransformer::HasSecondPhase() { + YQL_ENSURE(EPhase::Other == Phase_); + if (!*TableContentFlag_) { + return false; + } + ForceLocalTableContent_ = CanExecuteInternally(); + Phase_ = EPhase::Content; + return true; +} + +void TGatewayTransformer::ApplyJobProps(TYqlJobBase& job) { + for (auto& x: *JobFileAliases_) { + job.AddFileAlias(x.first, x.second); + } + JobFileAliases_->clear(); + + for (auto& x: *JobUdfs_) { + job.AddUdfModule(x.first, x.second); + } + JobUdfs_->clear(); + UniqFiles_->clear(); +} + +void TGatewayTransformer::ApplyUserJobSpec(NYT::TUserJobSpec& spec, bool localRun) { + if (!RemoteFiles_->empty()) { + YQL_ENSURE(!localRun, "Unexpected remote files"); + for (auto& file: *RemoteFiles_) { + spec.AddFile(file); + } + } + bool fakeChecksum = (GetEnv("YQL_LOCAL") == "1"); // YQL-15353 + for (auto& file: *LocalFiles_) { + TAddLocalFileOptions opts; + if (!fakeChecksum && file.second.Hash) { + opts.MD5CheckSum(file.second.Hash); + } + opts.BypassArtifactCache(file.second.BypassArtifactCache); + spec.AddLocalFile(file.first, opts); + } + const TString binTmpFolder = Settings_->BinaryTmpFolder.Get().GetOrElse(TString()); + const TString binCacheFolder = Settings_->_BinaryCacheFolder.Get(ExecCtx_.Cluster_).GetOrElse(TString()); + if (!localRun && binCacheFolder) { + auto udfFiles = std::move(*DeferredUdfFiles_); + TTransactionCache::TEntry::TPtr entry = GetEntry(); + for (auto& file: udfFiles) { + YQL_ENSURE(!file.second.Hash.empty()); + if (auto snapshot = entry->GetBinarySnapshotFromCache(binCacheFolder, file.second.Hash, file.first)) { + spec.AddFile(TRichYPath(snapshot->first).TransactionId(snapshot->second) + .FileName(TFsPath(file.first) + .GetName()) + .Executable(true) + .BypassArtifactCache(file.second.BypassArtifactCache)); + } else { + DeferredUdfFiles_->push_back(file); + } + } + } + if (!DeferredUdfFiles_->empty()) { + if (localRun || !binTmpFolder) { + for (auto& file: *DeferredUdfFiles_) { + TAddLocalFileOptions opts; + if (!fakeChecksum && file.second.Hash) { + opts.MD5CheckSum(file.second.Hash); + } + YQL_ENSURE(TFileStat(file.first).Size != 0); + opts.BypassArtifactCache(file.second.BypassArtifactCache); + spec.AddLocalFile(file.first, opts); + } + } else { + const TDuration binExpiration = Settings_->BinaryExpirationInterval.Get().GetOrElse(TDuration()); + auto entry = GetEntry(); + for (auto& file: *DeferredUdfFiles_) { + YQL_ENSURE(TFileStat(file.first).Size != 0); + YQL_ENSURE(!file.second.Hash.empty()); + auto snapshot = entry->GetBinarySnapshot(binTmpFolder, file.second.Hash, file.first, binExpiration); + spec.AddFile(TRichYPath(snapshot.first).TransactionId(snapshot.second).FileName(TFsPath(file.first).GetName()).Executable(true).BypassArtifactCache(file.second.BypassArtifactCache)); + } + } + } + RemoteFiles_->clear(); + LocalFiles_->clear(); + DeferredUdfFiles_->clear(); +} + +NYT::ITransactionPtr TGatewayTransformer::GetTx() { + return GetEntry()->Tx; +} + +TTransactionCache::TEntry::TPtr TGatewayTransformer::GetEntry() { + if (!Entry_) { + Entry_ = ExecCtx_.GetOrCreateEntry(Settings_); + } + return Entry_; +} + +void TGatewayTransformer::AddFile(TString alias, + const TUserFiles::TFileInfo& fileInfo, const TString& udfPrefix) { + if (alias.StartsWith('/')) { + alias = alias.substr(1); + } + if (alias.StartsWith(TStringBuf("home/"))) { + alias = alias.substr(TStringBuf("home/").length()); + } + + TString basename; + if (fileInfo.Path) { + // Pass only unique files to YT + auto insertRes = UniqFiles_->insert({fileInfo.Path->GetMd5(), fileInfo.Path->GetPath()}); + TString filePath; + if (insertRes.second) { + filePath = fileInfo.Path->GetPath(); + *UsedMem_ += fileInfo.InMemorySize; + if (fileInfo.IsUdf) { + DeferredUdfFiles_->emplace_back(filePath, TLocalFileInfo{fileInfo.Path->GetMd5(), fileInfo.BypassArtifactCache}); + } else { + LocalFiles_->emplace_back(filePath, TLocalFileInfo{fileInfo.Path->GetMd5(), fileInfo.BypassArtifactCache}); + } + } else { + filePath = insertRes.first->second; + } + + basename = TFsPath(filePath).GetName(); + if (alias && alias != basename) { + JobFileAliases_->insert({alias, basename}); + } + + } else { + *RemoteExecutionFlag_ = true; + TString filePath = NYT::AddPathPrefix(fileInfo.RemotePath, NYT::TConfig::Get()->Prefix); + TRichYPath remoteFile(filePath); + if (alias) { + remoteFile.FileName(alias); + filePath = alias; + } else { + alias = TFsPath(filePath).GetName(); + } + auto insertRes = UniqFiles_->insert({alias, remoteFile.Path_}); + if (insertRes.second) { + RemoteFiles_->push_back(remoteFile.Executable(true).BypassArtifactCache(fileInfo.BypassArtifactCache)); + if (fileInfo.RemoteMemoryFactor > 0.) { + *UsedMem_ += fileInfo.RemoteMemoryFactor * GetUncompressedFileSize(GetTx(), remoteFile.Path_).GetOrElse(ui64(1) << 10); + } + } else { + YQL_ENSURE(remoteFile.Path_ == insertRes.first->second, "Duplicate file alias " << alias.Quote() + << " for different files " << remoteFile.Path_.Quote() << " and " << insertRes.first->second.Quote()); + } + basename = TFsPath(filePath).GetName(); + } + + if (fileInfo.IsUdf) { + if (alias) { + JobUdfs_->insert({"./" + alias, udfPrefix}); + } else { + JobUdfs_->insert({"./" + basename, udfPrefix}); + } + } +} + +TString TGatewayTransformer::FindUdfPath(const TStringBuf moduleName) const { + if (auto udfInfo = UdfModules_.FindPtr(moduleName)) { + return TUserDataStorage::MakeFullName(udfInfo->FileAlias); + } + + TMaybe<TFilePathWithMd5> udfPathWithMd5 = UdfResolver_->GetSystemModulePath(moduleName); + YQL_ENSURE(udfPathWithMd5.Defined()); + return TFsPath(udfPathWithMd5->Path).GetName(); +} + +TString TGatewayTransformer::FindUdfPrefix(const TStringBuf moduleName) const { + if (auto udfInfo = UdfModules_.FindPtr(moduleName)) { + return udfInfo->Prefix; + } + return TString(); +} + +} // NNative + +} // NYql diff --git a/yt/yql/providers/yt/gateway/native/yql_yt_transform.h b/yt/yql/providers/yt/gateway/native/yql_yt_transform.h new file mode 100644 index 0000000000..b6dd25f2e9 --- /dev/null +++ b/yt/yql/providers/yt/gateway/native/yql_yt_transform.h @@ -0,0 +1,114 @@ +#pragma once + +#include "yql_yt_exec_ctx.h" + +#include <yt/yql/providers/yt/gateway/lib/temp_files.h> +#include <yt/yql/providers/yt/gateway/lib/transaction_cache.h> +#include <yt/yql/providers/yt/gateway/lib/user_files.h> + +#include <yt/yql/providers/yt/common/yql_yt_settings.h> +#include <yt/yql/providers/yt/job/yql_job_base.h> + +#include <yql/essentials/core/yql_type_annotation.h> +#include <yql/essentials/core/yql_udf_resolver.h> +#include <yql/essentials/minikql/mkql_node.h> +#include <yql/essentials/minikql/mkql_node_visitor.h> +#include <yql/essentials/minikql/mkql_program_builder.h> + +#include <yt/cpp/mapreduce/interface/operation.h> + +#include <util/generic/hash.h> +#include <util/generic/hash_set.h> +#include <util/generic/ptr.h> +#include <util/generic/strbuf.h> +#include <util/generic/string.h> + +namespace NYql { + +namespace NNative { + +class TGatewayTransformer { +public: + TGatewayTransformer(const TExecContextBase& execCtx, TYtSettings::TConstPtr settings, const TString& optLLVM, + TUdfModulesTable udfModules, IUdfResolver::TPtr udfResolver, TTransactionCache::TEntry::TPtr entry, + NKikimr::NMiniKQL::TProgramBuilder& builder, TTempFiles& tmpFiles, TMaybe<ui32> publicId); + + template <class TExecContextPtr> + TGatewayTransformer(const TExecContextPtr& execCtx, TTransactionCache::TEntry::TPtr entry, + NKikimr::NMiniKQL::TProgramBuilder& builder, TTempFiles& tmpFiles) + : TGatewayTransformer(*execCtx, execCtx->Options_.Config(), execCtx->Options_.OptLLVM(), + execCtx->Options_.UdfModules(), execCtx->Options_.UdfResolver(), std::move(entry), + builder, tmpFiles, execCtx->Options_.PublicId()) + { + } + + enum class EPhase { + Content, + Other, + All + }; + + struct TLocalFileInfo { + TString Hash; + bool BypassArtifactCache; + }; + + NKikimr::NMiniKQL::TCallableVisitFunc operator()(NKikimr::NMiniKQL::TInternName name); + + void SetTwoPhaseTransform() { + Phase_ = EPhase::Other; + } + + bool HasSecondPhase(); + + inline bool CanExecuteInternally() const { + return !*RemoteExecutionFlag_ && !*UntrustedUdfFlag_; + } + + inline bool CanExecuteLocally() const { + return !*RemoteExecutionFlag_; + } + + inline ui64 GetUsedMemory() const { + return *UsedMem_; + } + + void ApplyJobProps(TYqlJobBase& job); + void ApplyUserJobSpec(NYT::TUserJobSpec& spec, bool localRun); + +private: + NYT::ITransactionPtr GetTx(); + TTransactionCache::TEntry::TPtr GetEntry(); + void AddFile(TString alias, const TUserFiles::TFileInfo& fileInfo, const TString& udfPrefix = {}); + TString FindUdfPath(const TStringBuf moduleName) const; + TString FindUdfPrefix(const TStringBuf moduleName) const; + +private: + const TExecContextBase& ExecCtx_; + TYtSettings::TConstPtr Settings_; + TUdfModulesTable UdfModules_; + IUdfResolver::TPtr UdfResolver_; + + TTransactionCache::TEntry::TPtr Entry_; + NKikimr::NMiniKQL::TProgramBuilder& PgmBuilder_; + TTempFiles& TmpFiles_; + TMaybe<ui32> PublicId_; + EPhase Phase_ = EPhase::All; + bool ForceLocalTableContent_; + + // Wrap to shared ptr because TGatewayTransformer is passed by value + std::shared_ptr<bool> TableContentFlag_; + std::shared_ptr<bool> RemoteExecutionFlag_; + std::shared_ptr<bool> UntrustedUdfFlag_; + std::shared_ptr<ui64> UsedMem_; + std::shared_ptr<THashMap<TString, TString>> JobFileAliases_; + std::shared_ptr<THashMap<TString, TString>> JobUdfs_; + std::shared_ptr<THashMap<TString, TString>> UniqFiles_; + std::shared_ptr<TVector<NYT::TRichYPath>> RemoteFiles_; + std::shared_ptr<TVector<std::pair<TString, TLocalFileInfo>>> LocalFiles_; + std::shared_ptr<TVector<std::pair<TString, TLocalFileInfo>>> DeferredUdfFiles_; +}; + +} // NNative + +} // NYql diff --git a/yt/yql/providers/yt/gateway/qplayer/ya.make b/yt/yql/providers/yt/gateway/qplayer/ya.make new file mode 100644 index 0000000000..3494947329 --- /dev/null +++ b/yt/yql/providers/yt/gateway/qplayer/ya.make @@ -0,0 +1,22 @@ +LIBRARY() + +SRCS( + yql_yt_qplayer_gateway.cpp +) + +PEERDIR( + yql/essentials/core/qplayer/storage/interface + yql/essentials/providers/common/schema/expr + yql/essentials/core + yql/essentials/core/file_storage + library/cpp/yson/node + library/cpp/random_provider + yt/cpp/mapreduce/interface + contrib/libs/openssl +) + +YQL_LAST_ABI_VERSION() + +END() + + diff --git a/yt/yql/providers/yt/gateway/qplayer/yql_yt_qplayer_gateway.cpp b/yt/yql/providers/yt/gateway/qplayer/yql_yt_qplayer_gateway.cpp new file mode 100644 index 0000000000..bb3bd3aa88 --- /dev/null +++ b/yt/yql/providers/yt/gateway/qplayer/yql_yt_qplayer_gateway.cpp @@ -0,0 +1,965 @@ +#include "yql_yt_qplayer_gateway.h" + +#include <yql/essentials/core/file_storage/storage.h> +#include <library/cpp/yson/node/node_io.h> +#include <library/cpp/yson/node/node_builder.h> +#include <yt/cpp/mapreduce/interface/serialize.h> + +#include <util/stream/file.h> + +#include <openssl/sha.h> + +namespace NYT { + //TODO: use from header + void Deserialize(TReadRange& readRange, const TNode& node); +} + +namespace NYql { + +namespace { + +const TString YtGateway_CanonizePaths = "YtGateway_CanonizePaths"; +const TString YtGateway_GetTableInfo = "YtGateway_GetTableInfo"; +const TString YtGateway_GetTableRange = "YtGateway_GetTableRange"; +const TString YtGateway_GetFolder = "YtGateway_GetFolder"; +const TString YtGateway_GetFolders = "YtGateway_GetFolders"; +const TString YtGateway_ResolveLinks = "YtGateway_ResolveLinks"; +const TString YtGateway_PathStat = "YtGateway_PathStat"; +const TString YtGateway_PathStatMissing = "YtGateway_PathStatMissing"; + +TString MakeHash(const TString& str) { + SHA256_CTX sha; + SHA256_Init(&sha); + SHA256_Update(&sha, str.data(), str.size()); + unsigned char hash[SHA256_DIGEST_LENGTH]; + SHA256_Final(hash, &sha); + return TString((const char*)hash, sizeof(hash)); +} + +class TGateway : public IYtGateway { +public: + TGateway(IYtGateway::TPtr inner, const TQContext& qContext, + const TIntrusivePtr<IRandomProvider>& randomProvider, + const TFileStoragePtr& fileStorage) + : Inner_(inner) + , QContext_(qContext) + , RandomProvider_(randomProvider) + , FileStorage_(fileStorage) + {} + + void OpenSession(TOpenSessionOptions&& options) final { + SessionGenerations_[options.SessionId()] = 0; + return Inner_->OpenSession(std::move(options)); + } + + NThreading::TFuture<void> CloseSession(TCloseSessionOptions&& options) final { + return Inner_->CloseSession(std::move(options)); + } + + NThreading::TFuture<void> CleanupSession(TCleanupSessionOptions&& options) final { + ++SessionGenerations_[options.SessionId()]; + return Inner_->CleanupSession(std::move(options)); + } + + NThreading::TFuture<TFinalizeResult> Finalize(TFinalizeOptions&& options) final { + if (QContext_.CanRead()) { + TFinalizeResult res; + res.SetSuccess(); + return NThreading::MakeFuture<TFinalizeResult>(res); + } + + return Inner_->Finalize(std::move(options)); + } + + static TString MakeCanonizePathKey(const TCanonizeReq& req) { + auto node = NYT::TNode() + ("Cluster", req.Cluster()) + ("Path", req.Path()); + + return MakeHash(NYT::NodeToCanonicalYsonString(node)); + } + + NThreading::TFuture<TCanonizePathsResult> CanonizePaths(TCanonizePathsOptions&& options) final { + if (QContext_.CanRead()) { + TCanonizePathsResult res; + res.SetSuccess(); + for (const auto& req : options.Paths()) { + auto key = MakeCanonizePathKey(req); + auto item = QContext_.GetReader()->Get({YtGateway_CanonizePaths, key}).GetValueSync(); + if (!item) { + throw yexception() << "Missing replay data"; + } + + auto valueNode = NYT::NodeFromYsonString(item->Value); + TCanonizedPath p; + p.Path = valueNode["Path"].AsString(); + if (valueNode.HasKey("Columns")) { + p.Columns.ConstructInPlace(); + for (const auto& c : valueNode["Columns"].AsList()) { + p.Columns->push_back(c.AsString()); + } + } + + if (valueNode.HasKey("Ranges")) { + p.Ranges.ConstructInPlace(); + for (const auto& r : valueNode["Ranges"].AsList()) { + NYT::TReadRange range; + NYT::Deserialize(range, r); + p.Ranges->push_back(range); + } + } + + if (valueNode.HasKey("AdditionalAttributes")) { + p.AdditionalAttributes = valueNode["AdditionalAttributes"].AsString(); + } + + res.Data.push_back(p); + } + + return NThreading::MakeFuture<TCanonizePathsResult>(res); + } + + auto optionsDup = options; + return Inner_->CanonizePaths(std::move(options)) + .Subscribe([qContext = QContext_, optionsDup](const NThreading::TFuture<TCanonizePathsResult>& future) { + if (!qContext.CanWrite() || future.HasException()) { + return; + } + + const auto& res = future.GetValueSync(); + if (!res.Success()) { + return; + } + + Y_ENSURE(res.Data.size() == optionsDup.Paths().size()); + for (size_t i = 0; i < res.Data.size(); ++i) { + auto key = MakeCanonizePathKey(optionsDup.Paths()[i]); + auto valueNode = NYT::TNode(); + const auto& canon = res.Data[i]; + valueNode("Path", canon.Path); + if (canon.Columns) { + NYT::TNode columnsNode = NYT::TNode::CreateList(); + for (const auto& c : *canon.Columns) { + columnsNode.Add(NYT::TNode(c)); + } + + valueNode("Columns", columnsNode); + } + + if (canon.Ranges) { + NYT::TNode rangesNode = NYT::TNode::CreateList(); + for (const auto& r : *canon.Ranges) { + NYT::TNode rangeNode; + NYT::TNodeBuilder builder(&rangeNode); + NYT::Serialize(r, &builder); + rangesNode.Add(rangeNode); + } + + valueNode("Ranges", rangesNode); + } + + if (canon.AdditionalAttributes) { + valueNode("AdditionalAttributes", NYT::TNode(*canon.AdditionalAttributes)); + } + + auto value = NYT::NodeToYsonString(valueNode, NYT::NYson::EYsonFormat::Binary); + qContext.GetWriter()->Put({YtGateway_CanonizePaths, key}, value).GetValueSync(); + } + }); + } + + static TString MakeGetTableInfoKey(const TTableReq& req, ui32 epoch, ui64 generation) { + auto tableNode = NYT::TNode() + ("Generation", generation) + ("Cluster", req.Cluster()) + ("Table", req.Table()); + + if (req.InferSchemaRows() != 0) { + tableNode("InferSchemaRows", req.InferSchemaRows()); + } + + if (req.ForceInferSchema()) { + tableNode("ForceInferSchema", req.ForceInferSchema()); + } + + if (req.Anonymous()) { + tableNode("Anonymous", req.Anonymous()); + } + + if (req.IgnoreYamrDsv()) { + tableNode("IgnoreYamrDsv", req.IgnoreYamrDsv()); + } + + if (req.IgnoreWeakSchema()) { + tableNode("IgnoreWeakSchema", req.IgnoreWeakSchema()); + } + + auto node = NYT::TNode() + ("Table", tableNode) + ("Epoch", epoch); + + return MakeHash(NYT::NodeToCanonicalYsonString(node)); + } + + NThreading::TFuture<TTableInfoResult> GetTableInfo(TGetTableInfoOptions&& options) final { + ui64 generation = SessionGenerations_[options.SessionId()]; + if (QContext_.CanRead()) { + TTableInfoResult res; + res.SetSuccess(); + for (const auto& req : options.Tables()) { + TTableInfoResult::TTableData data; + auto key = MakeGetTableInfoKey(req, options.Epoch(), generation); + auto item = QContext_.GetReader()->Get({YtGateway_GetTableInfo, key}).GetValueSync(); + if (!item) { + throw yexception() << "Missing replay data"; + } + + auto valueNode = NYT::NodeFromYsonString(item->Value); + if (valueNode.HasKey("Meta")) { + data.Meta = MakeIntrusive<TYtTableMetaInfo>(); + auto metaNode = valueNode["Meta"]; + + data.Meta->CanWrite = metaNode["CanWrite"].AsBool(); + data.Meta->DoesExist = metaNode["DoesExist"].AsBool(); + data.Meta->YqlCompatibleScheme = metaNode["YqlCompatibleScheme"].AsBool(); + data.Meta->InferredScheme = metaNode["InferredScheme"].AsBool(); + data.Meta->IsDynamic = metaNode["IsDynamic"].AsBool(); + data.Meta->SqlView = metaNode["SqlView"].AsString(); + data.Meta->SqlViewSyntaxVersion = metaNode["SqlViewSyntaxVersion"].AsUint64(); + for (const auto& x : metaNode["Attrs"].AsMap()) { + data.Meta->Attrs[x.first] = x.second.AsString(); + } + } + if (valueNode.HasKey("Stat")) { + data.Stat = MakeIntrusive<TYtTableStatInfo>(); + auto statNode = valueNode["Stat"]; + data.Stat->Id = statNode["Id"].AsString(); + data.Stat->RecordsCount = statNode["RecordsCount"].AsUint64(); + data.Stat->DataSize = statNode["DataSize"].AsUint64(); + data.Stat->ChunkCount = statNode["ChunkCount"].AsUint64(); + data.Stat->ModifyTime = statNode["ModifyTime"].AsUint64(); + data.Stat->Revision = statNode["Revision"].AsUint64(); + data.Stat->TableRevision = statNode["TableRevision"].AsUint64(); + } + data.WriteLock = options.ReadOnly() ? false : valueNode["WriteLock"].AsBool(); + res.Data.push_back(data); + } + + return NThreading::MakeFuture<TTableInfoResult>(res); + } + + auto optionsDup = options; + return Inner_->GetTableInfo(std::move(options)) + .Subscribe([optionsDup, qContext = QContext_, generation](const NThreading::TFuture<TTableInfoResult>& future) { + if (!qContext.CanWrite() || future.HasException()) { + return; + } + + const auto& res = future.GetValueSync(); + if (!res.Success()) { + return; + } + + Y_ENSURE(res.Data.size() == optionsDup.Tables().size()); + for (size_t i = 0; i < res.Data.size(); ++i) { + const auto& req = optionsDup.Tables()[i]; + const auto& data = res.Data[i]; + auto key = MakeGetTableInfoKey(req, optionsDup.Epoch(), generation); + + auto attrsNode = NYT::TNode::CreateMap(); + if (data.Meta) { + for (const auto& a : data.Meta->Attrs) { + attrsNode(a.first, a.second); + } + } + + auto metaNode = data.Meta ? NYT::TNode() + ("CanWrite",data.Meta->CanWrite) + ("DoesExist",data.Meta->DoesExist) + ("YqlCompatibleScheme",data.Meta->YqlCompatibleScheme) + ("InferredScheme",data.Meta->InferredScheme) + ("IsDynamic",data.Meta->IsDynamic) + ("SqlView",data.Meta->SqlView) + ("SqlViewSyntaxVersion",ui64(data.Meta->SqlViewSyntaxVersion)) + ("Attrs",attrsNode) : NYT::TNode(); + + auto statNode = data.Stat ? NYT::TNode() + ("Id",data.Stat->Id) + ("RecordsCount",data.Stat->RecordsCount) + ("DataSize",data.Stat->DataSize) + ("ChunkCount",data.Stat->ChunkCount) + ("ModifyTime",data.Stat->ModifyTime) + ("Revision",data.Stat->Revision) + ("TableRevision",data.Stat->TableRevision) : NYT::TNode(); + + auto valueNode = NYT::TNode::CreateMap(); + if (data.Meta) { + valueNode("Meta", metaNode); + } + if (data.Stat) { + valueNode("Stat", statNode); + } + valueNode("WriteLock", data.WriteLock); + + auto value = NYT::NodeToYsonString(valueNode, NYT::NYson::EYsonFormat::Binary); + qContext.GetWriter()->Put({YtGateway_GetTableInfo, key},value).GetValueSync(); + } + }); + } + + static TString MakeGetTableRangeKey(const TTableRangeOptions& options, ui64 generation) { + auto keyNode = NYT::TNode() + ("Generation", generation) + ("Cluster", options.Cluster()) + ("Prefix", options.Prefix()) + ("Suffix", options.Suffix()); + + if (options.Filter()) { + keyNode("Filter", MakeCacheKey(*options.Filter())); + } + + return MakeHash(NYT::NodeToCanonicalYsonString(keyNode, NYT::NYson::EYsonFormat::Binary)); + } + + NThreading::TFuture<TTableRangeResult> GetTableRange(TTableRangeOptions&& options) final { + ui64 generation = SessionGenerations_[options.SessionId()]; + TString key; + if (QContext_) { + key = MakeGetTableRangeKey(options, generation); + } + + if (QContext_.CanRead()) { + TTableRangeResult res; + res.SetSuccess(); + auto item = QContext_.GetReader()->Get({YtGateway_GetTableRange, key}).GetValueSync(); + if (!item) { + throw yexception() << "Missing replay data"; + } + + auto listNode = NYT::NodeFromYsonString(item->Value); + for (const auto& valueNode : listNode.AsList()) { + TCanonizedPath p; + p.Path = valueNode["Path"].AsString(); + if (valueNode.HasKey("Columns")) { + p.Columns.ConstructInPlace(); + for (const auto& c : valueNode["Columns"].AsList()) { + p.Columns->push_back(c.AsString()); + } + } + + if (valueNode.HasKey("Ranges")) { + p.Ranges.ConstructInPlace(); + for (const auto& r : valueNode["Ranges"].AsString()) { + NYT::TReadRange range; + NYT::Deserialize(range, r); + p.Ranges->push_back(range); + } + } + + if (valueNode.HasKey("AdditionalAttributes")) { + p.AdditionalAttributes = valueNode["AdditionalAttributes"].AsString(); + } + + res.Tables.push_back(p); + } + + return NThreading::MakeFuture<TTableRangeResult>(res); + } + + return Inner_->GetTableRange(std::move(options)) + .Subscribe([key, qContext = QContext_](const NThreading::TFuture<TTableRangeResult>& future) { + if (!qContext.CanWrite() || future.HasException()) { + return; + } + + const auto& res = future.GetValueSync(); + if (!res.Success()) { + return; + } + + auto listNode = NYT::TNode::CreateList(); + for (const auto& t : res.Tables) { + listNode.Add(); + auto& valueNode = listNode.AsList().back(); + valueNode("Path", t.Path); + if (t.Columns) { + NYT::TNode columnsNode = NYT::TNode::CreateList(); + for (const auto& c : *t.Columns) { + columnsNode.Add(NYT::TNode(c)); + } + + valueNode("Columns", columnsNode); + } + + if (t.Ranges) { + NYT::TNode rangesNode = NYT::TNode::CreateList(); + for (const auto& r : *t.Ranges) { + NYT::TNode rangeNode; + NYT::TNodeBuilder builder(&rangeNode); + NYT::Serialize(r, &builder); + rangesNode.Add(rangeNode); + } + + valueNode("Ranges", rangesNode); + } + + if (t.AdditionalAttributes) { + valueNode("AdditionalAttributes", NYT::TNode(*t.AdditionalAttributes)); + } + } + + auto value = NYT::NodeToYsonString(listNode, NYT::NYson::EYsonFormat::Binary); + qContext.GetWriter()->Put({YtGateway_GetTableRange, key}, value).GetValueSync(); + }); + } + + static TString MakeGetFolderKey(const TFolderOptions& options, ui64 generation) { + auto attrNode = NYT::TNode::CreateList(); + for (const auto& attr : options.Attributes()) { + attrNode.Add(NYT::TNode(attr)); + } + + auto keyNode = NYT::TNode() + ("Generation", generation) + ("Cluster", options.Cluster()) + ("Prefix", options.Prefix()) + ("Attributes", attrNode); + + return MakeHash(NYT::NodeToCanonicalYsonString(keyNode, NYT::NYson::EYsonFormat::Binary)); + } + + static TString MakeResolveLinksKey(const TResolveOptions& options, ui64 generation) { + auto itemsNode = NYT::TNode::CreateList(); + for (const auto& item : options.Items()) { + auto attrNode = NYT::TNode::CreateList(); + for (const auto& attr : item.AttrKeys) { + attrNode.Add(NYT::TNode(attr)); + } + + itemsNode.Add(NYT::TNode() + ("AttrKeys", attrNode) + ("FolderItem", NYT::TNode() + ("Path", item.Item.Path) + ("Type", item.Item.Type) + ("Attributes", item.Item.Attributes))); + } + + auto keyNode = NYT::TNode() + ("Generation", generation) + ("Cluster", options.Cluster()) + ("Items", itemsNode); + + return MakeHash(NYT::NodeToCanonicalYsonString(keyNode, NYT::NYson::EYsonFormat::Binary)); + } + + static TString MakeGetFoldersKey(const TBatchFolderOptions& options, ui64 generation) { + auto itemsNode = NYT::TNode(); + TMap<TString, size_t> order; + for (size_t i = 0; i < options.Folders().size(); ++i) { + order[options.Folders()[i].Prefix] = i; + } + + for (const auto& o : order) { + const auto& folder = options.Folders()[o.second]; + auto attrNode = NYT::TNode::CreateList(); + for (const auto& attr : folder.AttrKeys) { + attrNode.Add(NYT::TNode(attr)); + } + + itemsNode.Add(NYT::TNode() + ("Prefix", folder.Prefix) + ("Attrs", attrNode)); + } + + auto keyNode = NYT::TNode() + ("Generation", generation) + ("Cluster", options.Cluster()) + ("Items", itemsNode); + + return MakeHash(NYT::NodeToCanonicalYsonString(keyNode, NYT::NYson::EYsonFormat::Binary)); + } + + template <typename T> + static NYT::TNode SerializeFolderItem(const T& item) { + return NYT::TNode() + ("Path", item.Path) + ("Type", item.Type) + ("Attributes", item.Attributes); + } + + template <typename T> + static void DeserializeFolderItem(T& item, const NYT::TNode& node) { + item.Path = node["Path"].AsString(); + item.Type = node["Type"].AsString(); + if constexpr (std::is_same_v<decltype(item.Attributes), TString>) { + item.Attributes = node["Attributes"].AsString(); + } else { + item.Attributes = node["Attributes"]; + } + } + + NThreading::TFuture<TFolderResult> GetFolder(TFolderOptions&& options) final { + ui64 generation = SessionGenerations_[options.SessionId()]; + if (QContext_.CanRead()) { + const auto& key = MakeGetFolderKey(options, generation); + auto item = QContext_.GetReader()->Get({YtGateway_GetFolder, key}).GetValueSync(); + if (!item) { + throw yexception() << "Missing replay data"; + } + + TFolderResult res; + res.SetSuccess(); + auto valueNode = NYT::NodeFromYsonString(TStringBuf(item->Value)); + if (valueNode.IsString()) { + const TString file = FileStorage_->GetTemp() / GetGuidAsString(RandomProvider_->GenGuid()); + auto out = MakeHolder<TOFStream>(file); + out->Write(valueNode.AsString()); + res.ItemsOrFileLink = CreateFakeFileLink(file, "", true); + } else { + TVector<TFolderResult::TFolderItem> items; + for (const auto& child : valueNode.AsList()) { + TFolderResult::TFolderItem item; + DeserializeFolderItem(item, child); + items.push_back(item); + } + + res.ItemsOrFileLink = items; + } + return NThreading::MakeFuture<TFolderResult>(res); + } + + auto optionsDup = options; + return Inner_->GetFolder(std::move(options)) + .Subscribe([optionsDup, qContext = QContext_, generation](const NThreading::TFuture<TFolderResult>& future) { + if (!qContext.CanWrite() || future.HasException()) { + return; + } + + const auto& res = future.GetValueSync(); + if (!res.Success()) { + return; + } + + const auto& key = MakeGetFolderKey(optionsDup, generation); + auto valueNode = NYT::TNode(); + + if (std::holds_alternative<TFileLinkPtr>(res.ItemsOrFileLink)) { + const auto& file = std::get<TFileLinkPtr>(res.ItemsOrFileLink); + valueNode = NYT::TNode(TFileInput(file->GetPath()).ReadAll()); + } else { + valueNode = NYT::TNode::CreateList(); + const auto& items = std::get<TVector<TFolderResult::TFolderItem>>(res.ItemsOrFileLink); + for (const auto& item: items) { + valueNode.Add(SerializeFolderItem(item)); + } + } + + auto value = NYT::NodeToYsonString(valueNode, NYT::NYson::EYsonFormat::Binary); + qContext.GetWriter()->Put({YtGateway_GetFolder, key}, value).GetValueSync(); + }); + } + + NThreading::TFuture<TBatchFolderResult> ResolveLinks(TResolveOptions&& options) final { + ui64 generation = SessionGenerations_[options.SessionId()]; + if (QContext_.CanRead()) { + TBatchFolderResult res; + res.SetSuccess(); + const auto& key = MakeResolveLinksKey(options, generation); + auto item = QContext_.GetReader()->Get({YtGateway_ResolveLinks, key}).GetValueSync(); + if (!item) { + throw yexception() << "Missing replay data"; + } + + auto valueNode = NYT::NodeFromYsonString(TStringBuf(item->Value)); + for (const auto& child : valueNode.AsList()) { + TBatchFolderResult::TFolderItem folderItem; + DeserializeFolderItem(folderItem, child); + res.Items.push_back(folderItem); + } + + return NThreading::MakeFuture<TBatchFolderResult>(res); + } + + auto optionsDup = options; + return Inner_->ResolveLinks(std::move(options)) + .Subscribe([optionsDup, qContext = QContext_, generation](const NThreading::TFuture<TBatchFolderResult>& future) { + if (!qContext.CanWrite() || future.HasException()) { + return; + } + + const auto& res = future.GetValueSync(); + if (!res.Success()) { + return; + } + + const auto& key = MakeResolveLinksKey(optionsDup, generation); + NYT::TNode valueNode = NYT::TNode::CreateList(); + for (const auto& item : res.Items) { + valueNode.Add(SerializeFolderItem(item)); + } + + auto value = NYT::NodeToYsonString(valueNode, NYT::NYson::EYsonFormat::Binary); + qContext.GetWriter()->Put({YtGateway_ResolveLinks, key}, value).GetValueSync(); + }); + } + + NThreading::TFuture<TBatchFolderResult> GetFolders(TBatchFolderOptions&& options) final { + ui64 generation = SessionGenerations_[options.SessionId()]; + if (QContext_.CanRead()) { + TBatchFolderResult res; + res.SetSuccess(); + const auto& key = MakeGetFoldersKey(options, generation); + auto item = QContext_.GetReader()->Get({YtGateway_GetFolders, key}).GetValueSync(); + if (!item) { + throw yexception() << "Missing replay data"; + } + + auto valueNode = NYT::NodeFromYsonString(TStringBuf(item->Value)); + for (const auto& child : valueNode.AsList()) { + TBatchFolderResult::TFolderItem folderItem; + DeserializeFolderItem(folderItem, child); + res.Items.push_back(folderItem); + } + + return NThreading::MakeFuture<TBatchFolderResult>(res); + } + + auto optionsDup = options; + return Inner_->GetFolders(std::move(options)) + .Subscribe([optionsDup, qContext = QContext_, generation](const NThreading::TFuture<TBatchFolderResult>& future) { + if (!qContext.CanWrite() || future.HasException()) { + return; + } + + const auto& res = future.GetValueSync(); + if (!res.Success()) { + return; + } + + const auto& key = MakeGetFoldersKey(optionsDup, generation); + NYT::TNode valueNode = NYT::TNode::CreateList(); + for (const auto& item : res.Items) { + valueNode.Add(SerializeFolderItem(item)); + } + + auto value = NYT::NodeToYsonString(valueNode, NYT::NYson::EYsonFormat::Binary); + qContext.GetWriter()->Put({YtGateway_GetFolders, key}, value).GetValueSync(); + }); + } + + NThreading::TFuture<TResOrPullResult> ResOrPull(const TExprNode::TPtr& node, TExprContext& ctx, TResOrPullOptions&& options) final { + if (QContext_.CanRead()) { + throw yexception() << "Can't replay ResOrPull"; + } + + return Inner_->ResOrPull(node, ctx, std::move(options)); + } + + NThreading::TFuture<TRunResult> Run(const TExprNode::TPtr& node, TExprContext& ctx, TRunOptions&& options) final { + if (QContext_.CanRead()) { + throw yexception() << "Can't replay Run"; + } + + return Inner_->Run(node, ctx, std::move(options)); + } + + NThreading::TFuture<TRunResult> Prepare(const TExprNode::TPtr& node, TExprContext& ctx, TPrepareOptions&& options) const final { + if (QContext_.CanRead()) { + throw yexception() << "Can't replay Prepare"; + } + + return Inner_->Prepare(node, ctx, std::move(options)); + } + + NThreading::TFuture<TRunResult> GetTableStat(const TExprNode::TPtr& node, TExprContext& ctx, TPrepareOptions&& options) final { + if (QContext_.CanRead()) { + throw yexception() << "Can't replay GetTableStat"; + } + + return Inner_->GetTableStat(node, ctx, std::move(options)); + } + + NThreading::TFuture<TCalcResult> Calc(const TExprNode::TListType& nodes, TExprContext& ctx, TCalcOptions&& options) final { + if (QContext_.CanRead()) { + throw yexception() << "Can't replay Calc"; + } + + return Inner_->Calc(nodes, ctx, std::move(options)); + } + + NThreading::TFuture<TPublishResult> Publish(const TExprNode::TPtr& node, TExprContext& ctx, TPublishOptions&& options) final { + if (QContext_.CanRead()) { + throw yexception() << "Can't replay Publish"; + } + + return Inner_->Publish(node, ctx, std::move(options)); + } + + NThreading::TFuture<TCommitResult> Commit(TCommitOptions&& options) final { + if (QContext_.CanRead()) { + throw yexception() << "Can't replay Commit"; + } + + return Inner_->Commit(std::move(options)); + } + + NThreading::TFuture<TDropTrackablesResult> DropTrackables(TDropTrackablesOptions&& options) final { + if (QContext_.CanRead()) { + throw yexception() << "Can't replay DropTrackables"; + } + + return Inner_->DropTrackables(std::move(options)); + } + + static TString MakePathStatKey(const TString& cluster, bool extended, const TPathStatReq& req, ui64 generation) { + auto node = NYT::TNode() + ("Generation", generation) + ("Cluster", cluster) + ("Extended", extended); + + NYT::TNode pathNode; + NYT::TNodeBuilder builder(&pathNode); + NYT::Serialize(req.Path(), &builder); + auto path = NYT::TNode() + ("Path", pathNode) + ("IsTemp", req.IsTemp()) + ("IsAnonymous", req.IsAnonymous()) + ("Epoch", req.Epoch()); + + node("Path", path); + return MakeHash(NYT::NodeToCanonicalYsonString(node)); + } + + static TString SerializePathStat(const TPathStatResult& stat, ui32 index) { + Y_ENSURE(index < stat.DataSize.size()); + Y_ENSURE(index < stat.Extended.size()); + auto xNode = NYT::TNode(); + if (!stat.Extended[index].Defined()) { + xNode = NYT::TNode::CreateEntity(); + } else { + auto dataWeightMap = NYT::TNode::CreateMap(); + for (const auto& d : stat.Extended[index]->DataWeight) { + dataWeightMap(d.first, d.second); + } + + auto uniqCountsMap = NYT::TNode::CreateMap(); + for (const auto& e : stat.Extended[index]->EstimatedUniqueCounts) { + uniqCountsMap(e.first, e.second); + } + + xNode = NYT::TNode() + ("DataWeight", dataWeightMap) + ("EstimatedUniqueCounts", uniqCountsMap); + } + + auto node = NYT::TNode() + ("DataSize", stat.DataSize[index]) + ("Extended", xNode); + + return NYT::NodeToYsonString(node, NYT::NYson::EYsonFormat::Binary); + } + + static void DeserializePathStat(const NYT::TNode& node, TPathStatResult& stat, ui32 index) { + Y_ENSURE(index < stat.DataSize.size()); + Y_ENSURE(index < stat.Extended.size()); + stat.DataSize[index] = node["DataSize"].AsUint64(); + stat.Extended[index] = Nothing(); + const auto& x = node["Extended"]; + if (!x.IsEntity()) { + auto& xValue = stat.Extended[index]; + xValue.ConstructInPlace(); + for (const auto& d : x["DataWeight"].AsMap()) { + xValue->DataWeight[d.first] = d.second.AsInt64(); + } + + for (const auto& e : x["EstimatedUniqueCounts"].AsMap()) { + xValue->EstimatedUniqueCounts[e.first] = e.second.AsUint64(); + } + } + } + + NThreading::TFuture<TPathStatResult> PathStat(TPathStatOptions&& options) final { + ui64 generation = SessionGenerations_[options.SessionId()]; + if (QContext_.CanRead()) { + TPathStatResult res; + res.DataSize.resize(options.Paths().size(), 0); + res.Extended.resize(options.Paths().size()); + + for (ui32 index = 0; index < options.Paths().size(); ++index) { + const auto& key = MakePathStatKey(options.Cluster(), options.Extended(), options.Paths()[index], generation); + auto item = QContext_.GetReader()->Get({YtGateway_PathStat, key}).GetValueSync(); + if (!item) { + throw yexception() << "Missing replay data"; + } + + PathStatKeys_.emplace(key); + auto valueNode = NYT::NodeFromYsonString(TStringBuf(item->Value)); + DeserializePathStat(valueNode, res, index); + } + + res.SetSuccess(); + return NThreading::MakeFuture<TPathStatResult>(res); + } + + auto optionsDup = options; + return Inner_->PathStat(std::move(options)) + .Subscribe([optionsDup, qContext = QContext_, generation](const NThreading::TFuture<TPathStatResult>& future) { + if (!qContext.CanWrite() || future.HasException()) { + return; + } + + const auto& res = future.GetValueSync(); + if (!res.Success()) { + return; + } + + for (ui32 index = 0; index < optionsDup.Paths().size(); ++index) { + const auto& key = MakePathStatKey(optionsDup.Cluster(), optionsDup.Extended(), optionsDup.Paths()[index], generation); + auto value = SerializePathStat(res, index); + qContext.GetWriter()->Put({YtGateway_PathStat, key}, value).GetValueSync(); + } + }); + } + + TPathStatResult TryPathStat(TPathStatOptions&& options) final { + ui64 generation = SessionGenerations_[options.SessionId()]; + if (QContext_.CanRead()) { + TPathStatResult res; + res.DataSize.resize(options.Paths().size(), 0); + res.Extended.resize(options.Paths().size()); + + for (ui32 index = 0; index < options.Paths().size(); ++index) { + const auto& key = MakePathStatKey(options.Cluster(), options.Extended(), options.Paths()[index], generation); + bool allow = false; + if (PathStatKeys_.contains(key)) { + allow = true; + } else { + auto missingItem = QContext_.GetReader()->Get({YtGateway_PathStatMissing, key}).GetValueSync(); + if (!missingItem) { + allow = true; + PathStatKeys_.emplace(key); + } + } + + if (!allow) { + return res; + } + + auto item = QContext_.GetReader()->Get({YtGateway_PathStat, key}).GetValueSync(); + if (!item) { + return res; + } + + auto valueNode = NYT::NodeFromYsonString(TStringBuf(item->Value)); + DeserializePathStat(valueNode, res, index); + } + + res.SetSuccess(); + return res; + } + + auto optionsDup = options; + auto res = Inner_->TryPathStat(std::move(options)); + if (!QContext_.CanWrite()) { + return res; + } + + if (!res.Success()) { + for (ui32 index = 0; index < optionsDup.Paths().size(); ++index) { + const auto& key = MakePathStatKey(optionsDup.Cluster(), optionsDup.Extended(), optionsDup.Paths()[index], generation); + QContext_.GetWriter()->Put({YtGateway_PathStatMissing, key}, "1").GetValueSync(); + } + + return res; + } + + for (ui32 index = 0; index < optionsDup.Paths().size(); ++index) { + const auto& key = MakePathStatKey(optionsDup.Cluster(), optionsDup.Extended(), optionsDup.Paths()[index], generation); + auto value = SerializePathStat(res, index); + QContext_.GetWriter()->Put({YtGateway_PathStat, key}, value).GetValueSync(); + } + + return res; + } + + bool TryParseYtUrl(const TString& url, TString* cluster, TString* path) const final { + return Inner_->TryParseYtUrl(url, cluster, path); + } + + TString GetDefaultClusterName() const final { + return Inner_->GetDefaultClusterName(); + } + + TString GetClusterServer(const TString& cluster) const final { + return Inner_->GetClusterServer(cluster); + } + + NYT::TRichYPath GetRealTable(const TString& sessionId, const TString& cluster, const TString& table, ui32 epoch, const TString& tmpFolder) const final { + return Inner_->GetRealTable(sessionId, cluster, table, epoch, tmpFolder); + } + + NYT::TRichYPath GetWriteTable(const TString& sessionId, const TString& cluster, const TString& table, const TString& tmpFolder) const final { + return Inner_->GetWriteTable(sessionId, cluster, table, tmpFolder); + } + + NThreading::TFuture<TDownloadTablesResult> DownloadTables(TDownloadTablesOptions&& options) final { + if (QContext_.CanRead()) { + TDownloadTablesResult res; + res.SetSuccess(); + return NThreading::MakeFuture(std::move(res)); + } + return Inner_->DownloadTables(std::move(options)); + } + + NThreading::TFuture<TUploadTableResult> UploadTable(TUploadTableOptions&& options) final { + if (QContext_.CanRead()) { + throw yexception() << "Can't replay UploadTable"; + } + return Inner_->UploadTable(std::move(options)); + } + + TFullResultTableResult PrepareFullResultTable(TFullResultTableOptions&& options) final { + if (QContext_.CanRead()) { + throw yexception() << "Can't replay PrepareFullResultTable"; + } + + return Inner_->PrepareFullResultTable(std::move(options)); + } + + void SetStatUploader(IStatUploader::TPtr statUploader) final { + return Inner_->SetStatUploader(statUploader); + } + + void RegisterMkqlCompiler(NCommon::TMkqlCallableCompilerBase& compiler) final { + return Inner_->RegisterMkqlCompiler(compiler); + } + + TGetTablePartitionsResult GetTablePartitions(TGetTablePartitionsOptions&& options) final { + if (QContext_.CanRead()) { + throw yexception() << "Can't replay GetTablePartitions"; + } + + return Inner_->GetTablePartitions(std::move(options)); + } + + void AddCluster(const TYtClusterConfig& cluster) final { + return Inner_->AddCluster(cluster); + } + +private: + const IYtGateway::TPtr Inner_; + const TQContext QContext_; + const TIntrusivePtr<IRandomProvider> RandomProvider_; + const TFileStoragePtr FileStorage_; + THashSet<TString> PathStatKeys_; + THashMap<TString, ui64> SessionGenerations_; +}; + +} + +IYtGateway::TPtr WrapYtGatewayWithQContext(IYtGateway::TPtr gateway, const TQContext& qContext, + const TIntrusivePtr<IRandomProvider>& randomProvider, const TFileStoragePtr& fileStorage) { + return MakeIntrusive<TGateway>(gateway, qContext, randomProvider, fileStorage); +} + +} diff --git a/yt/yql/providers/yt/gateway/qplayer/yql_yt_qplayer_gateway.h b/yt/yql/providers/yt/gateway/qplayer/yql_yt_qplayer_gateway.h new file mode 100644 index 0000000000..4c327fcb0d --- /dev/null +++ b/yt/yql/providers/yt/gateway/qplayer/yql_yt_qplayer_gateway.h @@ -0,0 +1,13 @@ +#pragma once +#include <yt/yql/providers/yt/provider/yql_yt_gateway.h> +#include <yql/essentials/core/qplayer/storage/interface/yql_qstorage.h> +#include <yql/essentials/core/file_storage/file_storage.h> + +#include <library/cpp/random_provider/random_provider.h> + +namespace NYql { + +IYtGateway::TPtr WrapYtGatewayWithQContext(IYtGateway::TPtr gateway, const TQContext& qContext, + const TIntrusivePtr<IRandomProvider>& randomProvider, const TFileStoragePtr& fileStorage); + +}
\ No newline at end of file diff --git a/yt/yql/providers/yt/job/ya.make b/yt/yql/providers/yt/job/ya.make new file mode 100644 index 0000000000..fe892d4f71 --- /dev/null +++ b/yt/yql/providers/yt/job/ya.make @@ -0,0 +1,42 @@ +LIBRARY() + +SRCS( + yql_job_base.cpp + yql_job_calc.cpp + yql_job_factory.cpp + yql_job_infer_schema.cpp + yql_job_registry.h + yql_job_stats_writer.cpp + yql_job_user.cpp +) + +PEERDIR( + library/cpp/random_provider + library/cpp/streams/brotli + library/cpp/time_provider + library/cpp/yson/node + yt/cpp/mapreduce/interface + yt/cpp/mapreduce/library/user_job_statistics + yql/essentials/minikql/comp_nodes + yql/essentials/public/udf + yql/essentials/utils + yql/essentials/utils/backtrace + yql/essentials/parser/pg_catalog + yql/essentials/parser/pg_wrapper/interface + yql/essentials/providers/common/provider + yql/essentials/providers/common/codec + yql/essentials/providers/common/comp_nodes + yql/essentials/providers/common/mkql + yql/essentials/providers/common/schema/mkql + yql/essentials/providers/common/schema/parser + yt/yql/providers/yt/codec + yt/yql/providers/yt/common + yt/yql/providers/yt/comp_nodes + yt/yql/providers/yt/lib/infer_schema + yt/yql/providers/yt/lib/lambda_builder + yt/yql/providers/yt/lib/mkql_helpers +) + +YQL_LAST_ABI_VERSION() + +END() diff --git a/yt/yql/providers/yt/job/yql_job_base.cpp b/yt/yql/providers/yt/job/yql_job_base.cpp new file mode 100644 index 0000000000..03a663d8c3 --- /dev/null +++ b/yt/yql/providers/yt/job/yql_job_base.cpp @@ -0,0 +1,309 @@ +#include "yql_job_base.h" +#include "yql_job_stats_writer.h" +#include "yql_job_factory.h" + +#include <yql/essentials/providers/common/provider/yql_provider.h> +#include <yql/essentials/parser/pg_wrapper/interface/context.h> +#include <yql/essentials/parser/pg_wrapper/interface/parser.h> +#include <yql/essentials/parser/pg_catalog/catalog.h> +#include <yql/essentials/minikql/invoke_builtins/mkql_builtins.h> +#include <yql/essentials/minikql/mkql_function_registry.h> +#include <yql/essentials/minikql/mkql_stats_registry.h> +#include <yql/essentials/minikql/mkql_node.h> +#include <yql/essentials/minikql/mkql_node_cast.h> +#include <yql/essentials/utils/backtrace/backtrace.h> +#include <yql/essentials/utils/yql_panic.h> +#include <yql/essentials/utils/debug_info.h> + +#include <library/cpp/yson/node/node_io.h> + +#include <library/cpp/random_provider/random_provider.h> +#include <library/cpp/time_provider/time_provider.h> + +#include <util/generic/yexception.h> +#include <util/folder/path.h> +#include <util/system/rusage.h> +#include <util/system/env.h> +#include <util/system/fs.h> +#include <util/system/error.h> +#include <util/system/datetime.h> +#include <util/datetime/cputimer.h> +#include <util/ysaveload.h> + +namespace NYql { + +using namespace NKikimr; +using namespace NKikimr::NMiniKQL; + +TStatKey Job_ThreadsCount("Job_ThreadsCount", false); +TStatKey Job_ElapsedTime("Job_ElapsedTime", false); +TStatKey Job_UserTime("Job_UserTime", false); +TStatKey Job_SystemTime("Job_SystemTime", false); +TStatKey Job_MajorPageFaults("Job_MajorPageFaults", false); + +class TEnvSecureParamsProvider : public NUdf::ISecureParamsProvider { +public: + TEnvSecureParamsProvider(const TString& envName) + { + const TString& yson = GetEnv(envName + "_secure_params"); + // Absent variable is not an error + if (!yson) + return; + auto attrs = NYT::NodeFromYsonString(yson); + YQL_ENSURE(attrs.IsMap()); + + SecureMap = attrs.AsMap(); + } + + bool GetSecureParam(NUdf::TStringRef key, NUdf::TStringRef& value) const override { + auto p = SecureMap.FindPtr(TString(key.Data(), key.Size())); + if (!p) + return false; + if (!p->IsString()) + return false; + + value = p->AsString(); + return true; + } + +private: + NYT::TNode::TMapType SecureMap; +}; + +NKikimr::NUdf::TCounter TJobCountersProvider::GetCounter(const NKikimr::NUdf::TStringRef& module, + const NKikimr::NUdf::TStringRef& name, bool deriv) { + Y_UNUSED(deriv); + auto fullName = std::make_pair(TString(TStringBuf(module)), TString(TStringBuf(name))); + return NKikimr::NUdf::TCounter(&Counters_[fullName]); +} + +NKikimr::NUdf::TScopedProbe TJobCountersProvider::GetScopedProbe(const NKikimr::NUdf::TStringRef& module, + const NKikimr::NUdf::TStringRef& name) { + auto fullName = std::make_pair(TString(TStringBuf(module)), TString(TStringBuf(name))); + return NKikimr::NUdf::TScopedProbe(this, &Probes_[fullName]); +} + +void TJobCountersProvider::Acquire(void* cookie) { + auto state = (TProbeState*)cookie; + state->LastAcquire = GetCycleCount(); +} + +void TJobCountersProvider::Release(void* cookie) { + auto state = (TProbeState*)cookie; + state->TotalCycles += GetCycleCount() - state->LastAcquire; +} + +TString MakeLocalPath(TString fileName) { + TString localPath = fileName; + if (localPath.StartsWith(TStringBuf("/home/"))) { + localPath = localPath.substr(TStringBuf("/home").length()); // Keep leading slash + } + if (!localPath.StartsWith('/')) { + localPath.prepend('/'); + } + localPath.prepend('.'); + return localPath; +} + +class TJobTransformProvider { +public: + TJobTransformProvider(THashMap<TString, TRuntimeNode>* extraArgs) + : ExtraArgs(extraArgs) + { + } + + TCallableVisitFunc operator()(TInternName name) { + if (name == "FilePathJob") { + return [](TCallable& callable, const TTypeEnvironment& env) { + YQL_ENSURE(callable.GetInputsCount() == 1, "Expected 1 argument"); + const TString path(AS_VALUE(TDataLiteral, callable.GetInput(0))->AsValue().AsStringRef()); + return TRuntimeNode(BuildDataLiteral(MakeLocalPath(path), NUdf::TDataType<char*>::Id, env), true); + }; + } + + if (name == "FileContentJob") { + return [](TCallable& callable, const TTypeEnvironment& env) { + YQL_ENSURE(callable.GetInputsCount() == 1, "Expected 1 argument"); + const TString path(AS_VALUE(TDataLiteral, callable.GetInput(0))->AsValue().AsStringRef()); + auto content = TFileInput(MakeLocalPath(path)).ReadAll(); + return TRuntimeNode(BuildDataLiteral(content, NUdf::TDataType<char*>::Id, env), true); + }; + } + + auto cutName = name.Str(); + if (cutName.SkipPrefix("Yt")) { + if (cutName == "TableIndex" || cutName == "TablePath" || cutName == "TableRecord" || cutName == "IsKeySwitch" || cutName == "RowNumber") { + return [this](NMiniKQL::TCallable& callable, const TTypeEnvironment& env) { + return GetExtraArg(TString{callable.GetType()->GetName()}, + *AS_TYPE(TDataType, callable.GetType()->GetReturnType())->GetDataSlot(), env); + }; + } + if (cutName == "Input") { + // Rename and add additional args + return [this](NMiniKQL::TCallable& callable, const TTypeEnvironment& env) { + TCallableBuilder callableBuilder(env, + TStringBuilder() << callable.GetType()->GetName() << "Job", callable.GetType()->GetReturnType(), false); + + if (const auto args = callable.GetInputsCount()) { + callableBuilder.Add(GetExtraArg("YtTableIndex", NUdf::EDataSlot::Uint32, env)); + callableBuilder.Add(GetExtraArg("YtTablePath", NUdf::EDataSlot::String, env)); + callableBuilder.Add(GetExtraArg("YtTableRecord", NUdf::EDataSlot::Uint64, env)); + callableBuilder.Add(GetExtraArg("YtIsKeySwitch", NUdf::EDataSlot::Bool, env)); + callableBuilder.Add(GetExtraArg("YtRowNumber", NUdf::EDataSlot::Uint64, env)); + for (ui32 i: xrange(args)) { + callableBuilder.Add(callable.GetInput(i)); + } + } + + return TRuntimeNode(callableBuilder.Build(), false); + }; + } + if (cutName == "Output") { + // Rename + return [](NMiniKQL::TCallable& callable, const TTypeEnvironment& env) { + TCallableBuilder callableBuilder(env, + TStringBuilder() << callable.GetType()->GetName() << "Job", callable.GetType()->GetReturnType(), false); + for (ui32 i: xrange(callable.GetInputsCount())) { + callableBuilder.Add(callable.GetInput(i)); + } + return TRuntimeNode(callableBuilder.Build(), false); + }; + } + } + + return TCallableVisitFunc(); + } + +private: + TRuntimeNode GetExtraArg(const TString& name, NUdf::EDataSlot slot, const TTypeEnvironment& env) { + YQL_ENSURE(ExtraArgs, "Unexpected " << name << " usage"); + TRuntimeNode& node = (*ExtraArgs)[name]; + if (!node) { + TCallableBuilder builder(env, "Arg", TDataType::Create(NUdf::GetDataTypeInfo(slot).TypeId, env), true); + node = TRuntimeNode(builder.Build(), false); + } + return node; + } + +private: + THashMap<TString, TRuntimeNode>* ExtraArgs; +}; + +TYqlJobBase::~TYqlJobBase() { + try { + if (JobStats) { + JobStats->SetStat(Job_ElapsedTime, (GetCycleCount() - StartCycles) / GetCyclesPerMillisecond()); + TRusage ru; + ru.Fill(); + JobStats->SetStat(Job_UserTime, ru.Utime.MilliSeconds()); + JobStats->SetStat(Job_SystemTime, ru.Stime.MilliSeconds()); + JobStats->SetStat(Job_MajorPageFaults, ru.MajorPageFaults); + + WriteJobStats(JobStats.Get(), JobCountersProvider); + } + } catch (...) { + /* do not throw exceptions in destructor */ + } +} + +void TYqlJobBase::Init() { + StartCycles = GetCycleCount(); + StartTime = ThreadCPUTime(); + + auto funcRegistry = CreateFunctionRegistry(CreateBuiltinRegistry())->Clone(); + funcRegistry->SetBackTraceCallback(&NYql::NBacktrace::KikimrBackTrace); + if (GetEnv(TString("YQL_DETERMINISTIC_MODE"))) { + RandomProvider = CreateDeterministicRandomProvider(1); + TimeProvider = CreateDeterministicTimeProvider(10000000); + } + else { + RandomProvider = CreateDefaultRandomProvider(); + TimeProvider = CreateDefaultTimeProvider(); + } + + const bool hasTmpfs = TFsPath("_yql_tmpfs").IsDirectory(); + Y_UNUSED(hasTmpfs); // _win_ compiler + + for (const auto& a: FileAliases) { + TFsPath lnk = TFsPath(".") / a.first; + TFsPath parent = lnk.Parent(); + if (!parent.Exists()) { + parent.MkDirs(); + } else { + // Local mode support. Overwrite existing link if any + if (lnk.IsSymlink()) { + lnk.DeleteIfExists(); + } + } +#ifndef _win_ + TFsPath lnkTarget = TFsPath(".").RealPath() / a.second; + if (hasTmpfs && !a.first.EndsWith(TStringBuf(".debug"))) { + NFs::Copy(lnkTarget, TFsPath("_yql_tmpfs") / a.second); + lnkTarget = TFsPath("_yql_tmpfs") / a.second; + } + YQL_ENSURE(NFs::SymLink(lnkTarget, lnk), "Failed to create file alias from " + << lnkTarget.GetPath().Quote() << " to " << lnk.GetPath().Quote() + << ": " << LastSystemErrorText()); +#endif + } + + NPg::LoadSystemFunctions(*NSQLTranslationPG::CreateSystemFunctionsParser()); + if (TFsPath(NCommon::PgCatalogFileName).Exists()) { + TFileInput file(TString{NCommon::PgCatalogFileName}); + NPg::ImportExtensions(file.ReadAll(), false, + NKikimr::NMiniKQL::CreateExtensionLoader().get()); + } + + FillStaticModules(*funcRegistry); + for (const auto& mod: UdfModules) { + auto path = mod.first; +#ifdef _win_ + path += ".dll"; +#endif + funcRegistry->LoadUdfs(path, {}, 0, mod.second); + } + + FunctionRegistry.Reset(funcRegistry.Release()); + + Alloc.Reset(new TScopedAlloc(__LOCATION__, NKikimr::TAlignedPagePoolCounters(), + FunctionRegistry->SupportsSizedAllocators())); + Env.Reset(new TTypeEnvironment(*Alloc)); + CodecCtx.Reset(new NCommon::TCodecContext(*Env, *FunctionRegistry)); + if (!GetEnv(TString("YQL_SUPPRESS_JOB_STATISTIC"))) { + JobStats = CreateDefaultStatsRegistry(); + } + SecureParamsProvider.Reset(new TEnvSecureParamsProvider("YT_SECURE_VAULT")); +} + +void TYqlJobBase::Save(IOutputStream& s) const { + ::SaveMany(&s, + UdfModules, + FileAliases, + UdfValidateMode, + OptLLVM, + TableNames + ); +} + +void TYqlJobBase::Load(IInputStream& s) { + ::LoadMany(&s, + UdfModules, + FileAliases, + UdfValidateMode, + OptLLVM, + TableNames + ); +} + +void TYqlJobBase::Do(const NYT::TRawJobContext& jobContext) { + DoImpl(jobContext.GetInputFile(), jobContext.GetOutputFileList()); + if (JobStats) { + JobStats->SetStat(Job_ThreadsCount, GetRunnigThreadsCount()); + } +} + +TCallableVisitFuncProvider TYqlJobBase::MakeTransformProvider(THashMap<TString, TRuntimeNode>* extraArgs) const { + return TJobTransformProvider(extraArgs); +} + +} // NYql diff --git a/yt/yql/providers/yt/job/yql_job_base.h b/yt/yql/providers/yt/job/yql_job_base.h new file mode 100644 index 0000000000..eceb1716ce --- /dev/null +++ b/yt/yql/providers/yt/job/yql_job_base.h @@ -0,0 +1,106 @@ +#pragma once + +#include <yql/essentials/providers/common/codec/yql_codec.h> +#include <yql/essentials/public/udf/udf_validate.h> +#include <yql/essentials/public/udf/udf_counter.h> +#include <yql/essentials/minikql/mkql_node_visitor.h> + +#include <yt/cpp/mapreduce/interface/io.h> +#include <yt/cpp/mapreduce/interface/operation.h> +#include <yt/cpp/mapreduce/io/job_reader.h> +#include <yt/cpp/mapreduce/io/job_writer.h> + +#include <util/generic/string.h> +#include <util/generic/vector.h> +#include <util/generic/ptr.h> +#include <util/generic/hash_set.h> +#include <util/generic/hash.h> + + +namespace NKikimr { + namespace NMiniKQL { + class IFunctionRegistry; + } +} + +class IRandomProvider; +class ITimeProvider; + +namespace NYql { + +struct TJobCountersProvider : public NKikimr::NUdf::ICountersProvider, public NKikimr::NUdf::IScopedProbeHost { + NKikimr::NUdf::TCounter GetCounter(const NKikimr::NUdf::TStringRef& module, const NKikimr::NUdf::TStringRef& name, bool deriv) override; + NKikimr::NUdf::TScopedProbe GetScopedProbe(const NKikimr::NUdf::TStringRef& module, const NKikimr::NUdf::TStringRef& name) override; + void Acquire(void* cookie) override; + void Release(void* cookie) override; + + struct TProbeState { + i64 TotalCycles = 0; + i64 LastAcquire; + }; + + THashMap<std::pair<TString, TString>, i64> Counters_; + THashMap<std::pair<TString, TString>, TProbeState> Probes_; +}; + +/////////////////////////////////////////////////////////////////////////////////////////////////////// + +class TYqlJobBase: public NYT::IRawJob { +protected: + TYqlJobBase() = default; + virtual ~TYqlJobBase(); + +public: + void AddUdfModule(const TString& udfModule, const TString& udfPrefix) { + UdfModules.insert({udfModule, udfPrefix}); + } + void AddFileAlias(const TString& alias, const TString& filePath) { + FileAliases[alias] = filePath; + } + + void SetUdfValidateMode(NKikimr::NUdf::EValidateMode mode) { + UdfValidateMode = mode; + } + + void SetOptLLVM(const TString& optLLVM) { + OptLLVM = optLLVM; + } + + void SetTableNames(const TVector<TString>& tableNames) { + TableNames = tableNames; + } + + void Do(const NYT::TRawJobContext& jobContext) override; + void Save(IOutputStream& stream) const override; + void Load(IInputStream& stream) override; + +protected: + NKikimr::NMiniKQL::TCallableVisitFuncProvider MakeTransformProvider(THashMap<TString, NKikimr::NMiniKQL::TRuntimeNode>* extraArgs = nullptr) const; + + void Init(); + + virtual void DoImpl(const TFile& inHandle, const TVector<TFile>& outHandles) = 0; + +protected: + // Serializable part (don't forget to add new members to Save/Load) + THashMap<TString, TString> UdfModules; // udf module path -> udf module prefix + THashMap<TString, TString> FileAliases; + NKikimr::NUdf::EValidateMode UdfValidateMode = NKikimr::NUdf::EValidateMode::None; + TString OptLLVM; + TVector<TString> TableNames; + // End serializable part + + ui64 StartCycles = 0; + ui64 StartTime = 0; + TIntrusivePtr<NKikimr::NMiniKQL::IFunctionRegistry> FunctionRegistry; + TIntrusivePtr<IRandomProvider> RandomProvider; + TIntrusivePtr<ITimeProvider> TimeProvider; + THolder<NKikimr::NMiniKQL::TScopedAlloc> Alloc; + THolder<NKikimr::NMiniKQL::TTypeEnvironment> Env; + NKikimr::NMiniKQL::IStatsRegistryPtr JobStats; + TJobCountersProvider JobCountersProvider; + THolder<NKikimr::NUdf::ISecureParamsProvider> SecureParamsProvider; + THolder<NCommon::TCodecContext> CodecCtx; +}; + +} // NYql diff --git a/yt/yql/providers/yt/job/yql_job_calc.cpp b/yt/yql/providers/yt/job/yql_job_calc.cpp new file mode 100644 index 0000000000..b35a312dba --- /dev/null +++ b/yt/yql/providers/yt/job/yql_job_calc.cpp @@ -0,0 +1,105 @@ +#include "yql_job_calc.h" +#include "yql_job_factory.h" + +#include <yt/yql/providers/yt/lib/mkql_helpers/mkql_helpers.h> +#include <yt/yql/providers/yt/lib/lambda_builder/lambda_builder.h> +#include <yql/essentials/providers/common/codec/yql_codec.h> + +#include <yql/essentials/minikql/mkql_node.h> +#include <yql/essentials/minikql/mkql_node_cast.h> +#include <yql/essentials/minikql/mkql_node_serialization.h> + +#include <library/cpp/yson/node/node_builder.h> +#include <yt/cpp/mapreduce/io/node_table_reader.h> +#include <yt/cpp/mapreduce/io/node_table_writer.h> + +#include <library/cpp/streams/brotli/brotli.h> + +#include <util/generic/xrange.h> +#include <util/generic/yexception.h> +#include <util/stream/str.h> +#include <util/stream/input.h> +#include <util/ysaveload.h> + +namespace NYql { + +using namespace NKikimr; +using namespace NKikimr::NMiniKQL; + +void TYqlCalcJob::Save(IOutputStream& stream) const { + TYqlJobBase::Save(stream); + ::Save(&stream, Columns_); + ::Save(&stream, UseResultYson_); +} + +void TYqlCalcJob::Load(IInputStream& stream) { + TYqlJobBase::Load(stream); + ::Load(&stream, Columns_); + ::Load(&stream, UseResultYson_); +} + +void TYqlCalcJob::DoImpl(const TFile& inHandle, const TVector<TFile>& outHandles) { + NYT::TTableReader<NYT::TNode> reader(MakeIntrusive<NYT::TNodeTableReader>(MakeIntrusive<NYT::TJobReader>(inHandle))); + NYT::TTableWriter<NYT::TNode> writer(MakeIntrusive<NYT::TNodeTableWriter>(MakeHolder<NYT::TJobWriter>(outHandles))); + + Init(); + + TLambdaBuilder builder(FunctionRegistry.Get(), *Alloc,Env.Get(), + RandomProvider.Get(), TimeProvider.Get(), JobStats.Get(), nullptr, SecureParamsProvider.Get()); + + std::function<void(const NUdf::TUnboxedValuePod&, TType*, TVector<ui32>*)> flush; + if (UseResultYson_) { + flush = [&writer] (const NUdf::TUnboxedValuePod& v, TType* type, TVector<ui32>* structPositions) { + NYT::TNode row = NYT::TNode::CreateMap(); + NYT::TNodeBuilder nodeBuilder(&row["output"]); + NCommon::WriteYsonValue(nodeBuilder, v, type, structPositions); + writer.AddRow(row); + }; + } + else { + flush = [&writer] (const NUdf::TUnboxedValuePod& v, TType* type, TVector<ui32>* /*structPositions*/) { + writer.AddRow(NYT::TNode()("output", NCommon::ValueToNode(v, type))); + }; + } + + auto factory = GetJobFactory(*CodecCtx, OptLLVM, nullptr, nullptr, nullptr); + for (; reader.IsValid(); reader.Next()) { + const auto& row = reader.GetRow(); + TStringStream lambda; + { + TStringInput in(row["input"].AsString()); + TBrotliDecompress decompress(&in); + TransferData(&decompress, &lambda); + } + TRuntimeNode rootNode = DeserializeRuntimeNode(lambda.Str(), *Env); + rootNode = builder.TransformAndOptimizeProgram(rootNode, MakeTransformProvider()); + TType* outType = rootNode.GetStaticType(); + TExploringNodeVisitor explorer; + auto graph = builder.BuildGraph( + factory, + UdfValidateMode, + NUdf::EValidatePolicy::Fail, OptLLVM, + EGraphPerProcess::Single, + explorer, rootNode); + const TBindTerminator bind(graph->GetTerminator()); + graph->Prepare(); + auto value = graph->GetValue(); + if (outType->IsTuple()) { + auto tupleType = AS_TYPE(NMiniKQL::TTupleType, outType); + for (ui32 i: xrange(tupleType->GetElementsCount())) { + flush(value.GetElement(i), tupleType->GetElementType(i), nullptr); + } + } else if (outType->IsList()) { + auto itemType = AS_TYPE(NMiniKQL::TListType, outType)->GetItemType(); + TMaybe<TVector<ui32>> structPositions = NCommon::CreateStructPositions(itemType, Columns_.Get()); + const auto it = value.GetListIterator(); + for (NUdf::TUnboxedValue item; it.Next(item);) { + flush(item, itemType, structPositions.Get()); + } + } else { + flush(value, outType, nullptr); + } + } +} + +} // NYql diff --git a/yt/yql/providers/yt/job/yql_job_calc.h b/yt/yql/providers/yt/job/yql_job_calc.h new file mode 100644 index 0000000000..4e5f2398a2 --- /dev/null +++ b/yt/yql/providers/yt/job/yql_job_calc.h @@ -0,0 +1,39 @@ +#pragma once + +#include "yql_job_base.h" + +#include <yt/cpp/mapreduce/interface/operation.h> +#include <yt/cpp/mapreduce/interface/io.h> + +#include <util/stream/input.h> +#include <util/stream/output.h> +#include <util/generic/vector.h> +#include <util/generic/maybe.h> + +namespace NYql { + +class TYqlCalcJob : public TYqlJobBase { +public: + TYqlCalcJob() = default; + ~TYqlCalcJob() = default; + + void SetColumns(const TVector<TString>& columns) { + Columns_ = columns; + } + + void SetUseResultYson(bool flag) { + UseResultYson_ = flag; + } + + void Save(IOutputStream& stream) const override; + void Load(IInputStream& stream) override; + +protected: + void DoImpl(const TFile& inHandle, const TVector<TFile>& outHandles) override; + +private: + TMaybe<TVector<TString>> Columns_; + bool UseResultYson_ = false; +}; + +} // NYql diff --git a/yt/yql/providers/yt/job/yql_job_factory.cpp b/yt/yql/providers/yt/job/yql_job_factory.cpp new file mode 100644 index 0000000000..6ff4a02606 --- /dev/null +++ b/yt/yql/providers/yt/job/yql_job_factory.cpp @@ -0,0 +1,57 @@ +#include "yql_job_factory.h" + +#include <yt/yql/providers/yt/comp_nodes/yql_mkql_input.h> +#include <yt/yql/providers/yt/comp_nodes/yql_mkql_output.h> +#include <yt/yql/providers/yt/comp_nodes/yql_mkql_table_content.h> +#include <yql/essentials/providers/common/comp_nodes/yql_factory.h> +#include <yql/essentials/minikql/comp_nodes/mkql_factories.h> +#include <yql/essentials/parser/pg_wrapper/interface/comp_factory.h> + +#include <util/generic/strbuf.h> + +namespace NYql { + +using namespace NKikimr; +using namespace NKikimr::NMiniKQL; + +TComputationNodeFactory GetJobFactory(NYql::NCommon::TCodecContext& codecCtx, const TString& optLLVM, + const TMkqlIOSpecs* specs, NYT::IReaderImplBase* reader, TJobMkqlWriterImpl* writer) +{ + TMaybe<ui32> exprContextObject; + return [&codecCtx, optLLVM, specs, reader, writer, exprContextObject](NMiniKQL::TCallable& callable, const TComputationNodeFactoryContext& ctx) mutable -> IComputationNode* { + TStringBuf name = callable.GetType()->GetName(); + if (name.SkipPrefix("Yt") && name.ChopSuffix("Job")) { + if (name == "TableContent") { + return WrapYtTableContent(codecCtx, ctx.Mutables, callable, optLLVM, {} /*empty pathPrefix inside job*/); + } + if (name == "Input") { + YQL_ENSURE(reader); + YQL_ENSURE(specs); + return WrapYtInput(callable, ctx, *specs, reader); + } + if (name == "Output") { + YQL_ENSURE(writer); + return WrapYtOutput(callable, ctx, *writer); + } + } + + if (!exprContextObject) { + exprContextObject = ctx.Mutables.CurValueIndex++; + } + + auto yql = GetYqlFactory(*exprContextObject)(callable, ctx); + if (yql) { + return yql; + } + + auto pg = GetPgFactory()(callable, ctx); + if (pg) { + return pg; + } + + return GetBuiltinFactory()(callable, ctx); + }; +} + + +} // NYql diff --git a/yt/yql/providers/yt/job/yql_job_factory.h b/yt/yql/providers/yt/job/yql_job_factory.h new file mode 100644 index 0000000000..aa5edfa060 --- /dev/null +++ b/yt/yql/providers/yt/job/yql_job_factory.h @@ -0,0 +1,17 @@ +#pragma once + +#include <yt/yql/providers/yt/codec/yt_codec_job.h> +#include <yql/essentials/providers/common/codec/yql_codec.h> +#include <yql/essentials/minikql/computation/mkql_computation_node.h> + +#include <yt/cpp/mapreduce/interface/io.h> + +#include <util/generic/string.h> +#include <util/generic/vector.h> + +namespace NYql { + +NKikimr::NMiniKQL::TComputationNodeFactory GetJobFactory(NYql::NCommon::TCodecContext& codecCtx, + const TString& optLLVM, const TMkqlIOSpecs* specs, NYT::IReaderImplBase* reader, TJobMkqlWriterImpl* writer); + +} // NYql diff --git a/yt/yql/providers/yt/job/yql_job_infer_schema.cpp b/yt/yql/providers/yt/job/yql_job_infer_schema.cpp new file mode 100644 index 0000000000..3e7c3eb117 --- /dev/null +++ b/yt/yql/providers/yt/job/yql_job_infer_schema.cpp @@ -0,0 +1,49 @@ +#include "yql_job_infer_schema.h" + +#include <yt/yql/providers/yt/lib/infer_schema/infer_schema.h> + +#include <yt/cpp/mapreduce/io/node_table_reader.h> +#include <yt/cpp/mapreduce/io/node_table_writer.h> +#include <library/cpp/yson/node/node_io.h> + +namespace NYql { + +void TYqlInferSchemaJob::DoImpl(const TFile& inHandle, const TVector<TFile>& outHandles) { + NYT::TTableReader<NYT::TNode> reader(MakeIntrusive<NYT::TNodeTableReader>(MakeIntrusive<NYT::TJobReader>(inHandle))); + NYT::TTableWriter<NYT::TNode> writer(MakeIntrusive<NYT::TNodeTableWriter>(MakeHolder<NYT::TJobWriter>(outHandles))); + + Init(); + + THashMap<ui32, TStreamSchemaInferer> infererByTableIndex; + + for (; reader.IsValid(); reader.Next()) { + ui32 idx = reader.GetTableIndex(); + + YQL_ENSURE(idx < TableNames.size()); + TString tableName = TableNames[idx]; + + auto it = infererByTableIndex.find(idx); + + if (it == infererByTableIndex.end()) { + it = infererByTableIndex.insert({ idx, TStreamSchemaInferer(tableName) }).first; + } + + it->second.AddRow(reader.GetRow()); + } + + for (const auto &i : infererByTableIndex) { + ui32 idx = i.first; + + NYT::TNode schema; + try { + schema = i.second.GetSchema(); + } catch (const std::exception& e) { + schema = NYT::TNode(e.what()); + } + + NYT::TNode output = NYT::TNode()("index", idx)("schema", NYT::NodeToYsonString(schema)); + writer.AddRow(output); + } +} + +} // NYql diff --git a/yt/yql/providers/yt/job/yql_job_infer_schema.h b/yt/yql/providers/yt/job/yql_job_infer_schema.h new file mode 100644 index 0000000000..8124eb7f34 --- /dev/null +++ b/yt/yql/providers/yt/job/yql_job_infer_schema.h @@ -0,0 +1,20 @@ +#pragma once + +#include "yql_job_base.h" +#include "yql_job_factory.h" + +#include <yt/cpp/mapreduce/interface/operation.h> +#include <yt/cpp/mapreduce/interface/io.h> + +namespace NYql { + +class TYqlInferSchemaJob : public TYqlJobBase { +public: + TYqlInferSchemaJob() = default; + ~TYqlInferSchemaJob() = default; + +protected: + void DoImpl(const TFile& inHandle, const TVector<TFile>& outHandles) override; +}; + +} // NYql diff --git a/yt/yql/providers/yt/job/yql_job_registry.h b/yt/yql/providers/yt/job/yql_job_registry.h new file mode 100644 index 0000000000..e5a9d715bd --- /dev/null +++ b/yt/yql/providers/yt/job/yql_job_registry.h @@ -0,0 +1,15 @@ +#pragma once + +#include "yql_job_calc.h" +#include "yql_job_infer_schema.h" +#include "yql_job_user.h" + +#include <yt/cpp/mapreduce/interface/operation.h> + +namespace NYql { + +REGISTER_NAMED_RAW_JOB("TYqlCalcJob", TYqlCalcJob); +REGISTER_NAMED_RAW_JOB("TYqlInferSchemaJob", TYqlInferSchemaJob); +REGISTER_NAMED_RAW_JOB("TYqlUserJob", TYqlUserJob); + +} diff --git a/yt/yql/providers/yt/job/yql_job_stats_writer.cpp b/yt/yql/providers/yt/job/yql_job_stats_writer.cpp new file mode 100644 index 0000000000..275f410f87 --- /dev/null +++ b/yt/yql/providers/yt/job/yql_job_stats_writer.cpp @@ -0,0 +1,41 @@ +#include "yql_job_stats_writer.h" +#include "yql_job_base.h" + +#include <yt/cpp/mapreduce/library/user_job_statistics/user_job_statistics.h> +#include <yt/cpp/mapreduce/common/helpers.h> + +using NKikimr::NMiniKQL::IStatsRegistry; +using NKikimr::NMiniKQL::TStatKey; + +namespace NYql { + +void WriteJobStats(const IStatsRegistry* stats, const TJobCountersProvider& countersProvider, IOutputStream* out) { + NYtTools::TUserJobStatsProxy statsWriter; + statsWriter.Init(out); + auto& stream = *statsWriter.GetStream(); + + stats->ForEachStat([&stream](const TStatKey& key, i64 value) { + auto node = NYT::TNode{}(TString(key.GetName()), value); + stream << NYT::NodeToYsonString(node, NYson::EYsonFormat::Text) << ";\n"; + }); + + for (const auto& x : countersProvider.Counters_) { + auto moduleMap = NYT::TNode{}(x.first.second, x.second); + auto counterMap = NYT::TNode{}(x.first.first, moduleMap); + auto udfMap = NYT::TNode{}("Counter", counterMap); + auto node = NYT::TNode{}("Udf", udfMap); + stream << NYT::NodeToYsonString(node, NYson::EYsonFormat::Text) << ";\n"; + } + + for (const auto& x : countersProvider.Probes_) { + auto moduleMap = NYT::TNode{}(x.first.second, i64(1000.0 * x.second.TotalCycles / GetCyclesPerMillisecond())); + auto timeMap = NYT::TNode{}(x.first.first, moduleMap); + auto udfMap = NYT::TNode{}("TimeUsec", timeMap); + auto node = NYT::TNode{}("Udf", udfMap); + stream << NYT::NodeToYsonString(node, NYson::EYsonFormat::Text) << ";\n"; + } + + stream.Flush(); +} + +} // namspace NYql diff --git a/yt/yql/providers/yt/job/yql_job_stats_writer.h b/yt/yql/providers/yt/job/yql_job_stats_writer.h new file mode 100644 index 0000000000..a707d52600 --- /dev/null +++ b/yt/yql/providers/yt/job/yql_job_stats_writer.h @@ -0,0 +1,20 @@ +#pragma once + +#include <yql/essentials/minikql/mkql_stats_registry.h> + +#include <util/stream/output.h> + +namespace NYql { + +struct TJobCountersProvider; + +/** + * @brief Writes stats to out stream (if defined) or to special YT-job file + * descriptor (see https://wiki.yandex-team.ru/yt/userdoc/jobs/#sborstatistikivdzhobax). + */ +void WriteJobStats( + const NKikimr::NMiniKQL::IStatsRegistry* stats, + const TJobCountersProvider& countersProvider, + IOutputStream* out = nullptr); + +} // namspace NYql diff --git a/yt/yql/providers/yt/job/yql_job_user.cpp b/yt/yql/providers/yt/job/yql_job_user.cpp new file mode 100644 index 0000000000..88e1290d26 --- /dev/null +++ b/yt/yql/providers/yt/job/yql_job_user.cpp @@ -0,0 +1,246 @@ +#include "yql_job_user.h" +#include "yql_job_factory.h" + +#include <yt/yql/providers/yt/lib/lambda_builder/lambda_builder.h> +#include <yt/yql/providers/yt/lib/mkql_helpers/mkql_helpers.h> +#include <yt/yql/providers/yt/common/yql_names.h> +#include <yql/essentials/providers/common/codec/yql_codec.h> +#include <yql/essentials/providers/common/schema/parser/yql_type_parser.h> +#include <yql/essentials/providers/common/schema/mkql/yql_mkql_schema.h> +#include <yql/essentials/minikql/mkql_node_serialization.h> +#include <yql/essentials/minikql/mkql_node_cast.h> +#include <yql/essentials/minikql/mkql_stats_registry.h> +#include <yql/essentials/utils/yql_panic.h> + +#include <yt/cpp/mapreduce/client/structured_table_formats.h> +#include <yt/cpp/mapreduce/io/yamr_table_reader.h> +#include <library/cpp/yson/node/node_io.h> + +#include <util/generic/maybe.h> +#include <util/generic/xrange.h> +#include <util/generic/yexception.h> +#include <util/stream/str.h> +#include <util/system/rusage.h> +#include <util/system/datetime.h> +#include <util/ysaveload.h> + + +namespace NYql { + +using namespace NKikimr; +using namespace NKikimr::NMiniKQL; + +namespace { + + const static TStatKey Mkql_TotalRuntimeNodes("Mkql_TotalRuntimeNodes", false); + const static TStatKey Mkql_BuildGraphRssDelta("Mkql_BuildGraphRssDelta", false); + const static TStatKey Job_InitTime("Job_InitTime", false); + const static TStatKey Job_CalcTime("Job_CalcTime", false); + + NYT::TFormat MakeTableYaMRFormat(const TString& inputSpec) { + NYT::TNode inAttrs; + TStringStream err; + if (!NCommon::ParseYson(inAttrs, inputSpec, err)) { + ythrow yexception() << "Invalid input attrs: " << err.Str(); + } + YQL_ENSURE(inAttrs.IsMap(), "Expect Map type of output meta attrs, but got type " << inAttrs.GetType()); + YQL_ENSURE(inAttrs.HasKey(YqlIOSpecTables), "Expect " << TString{YqlIOSpecTables}.Quote() << " key"); + + auto& inputSpecs = inAttrs[YqlIOSpecTables].AsList(); + YQL_ENSURE(!inputSpecs.empty(), "Expect list with at least one element in input attrs: " << inputSpec); + + TVector<TMaybe<NYT::TNode>> formats; + THashMap<TString, NYT::TNode> specRegistry; + for (auto& attrs: inputSpecs) { + NYT::TNode spec; + if (attrs.IsString()) { + auto refName = attrs.AsString(); + if (auto p = specRegistry.FindPtr(refName)) { + spec = *p; + } else { + YQL_ENSURE(inAttrs.HasKey(YqlIOSpecRegistry) && inAttrs[YqlIOSpecRegistry].HasKey(refName), "Bad input registry reference: " << refName); + NYT::TNode& r = specRegistry[refName]; + r = inAttrs[YqlIOSpecRegistry][refName]; + spec = r; + } + } else { + spec = attrs; + } + formats.push_back(spec.HasKey(FORMAT_ATTR_NAME) ? MakeMaybe(spec[FORMAT_ATTR_NAME]) : Nothing()); + } + + NYT::TNode format = NYT::GetCommonTableFormat(formats).GetOrElse(NYT::TNode("yamred_dsv")); + format.Attributes()["lenval"] = true; + format.Attributes()["has_subkey"] = true; + format.Attributes()["enable_table_index"] = true; + return NYT::TFormat(format); + } +} + + +std::pair<NYT::TFormat, NYT::TFormat> TYqlUserJob::GetIOFormats(const NKikimr::NMiniKQL::IFunctionRegistry* functionRegistry) const { + TScopedAlloc alloc(__LOCATION__); + TMkqlIOSpecs specs; + if (UseBlockInput) { + specs.SetUseBlockInput(); + } + + if (!UseSkiff) { + return std::make_pair(YamrInput ? MakeTableYaMRFormat(InputSpec) : specs.MakeInputFormat(AuxColumns), specs.MakeOutputFormat()); + } + + TTypeEnvironment env(alloc); + NCommon::TCodecContext codecCtx(env, *functionRegistry); + + TType* itemType = nullptr; + if (InputType) { + TStringStream err; + TProgramBuilder pgmBuilder(env, *functionRegistry); + itemType = NCommon::ParseTypeFromYson(TStringBuf{InputType}, pgmBuilder, err); + YQL_ENSURE(itemType, << err.Str()); + } + + specs.SetUseSkiff(OptLLVM, SkiffSysFields); + specs.Init(codecCtx, InputSpec, InputGroups, TableNames, itemType, AuxColumns, OutSpec); + + return std::make_pair(YamrInput ? MakeTableYaMRFormat(InputSpec) : specs.MakeInputFormat(AuxColumns), specs.MakeOutputFormat()); +} + +void TYqlUserJob::Save(IOutputStream& s) const { + TYqlJobBase::Save(s); + ::SaveMany(&s, + UseSkiff, + UseBlockInput, + SkiffSysFields, + YamrInput, + LambdaCode, + InputSpec, + OutSpec, + InputGroups, + AuxColumns, + InputType, + RowOffsets + ); +} + +void TYqlUserJob::Load(IInputStream& s) { + TYqlJobBase::Load(s); + ::LoadMany(&s, + UseSkiff, + UseBlockInput, + SkiffSysFields, + YamrInput, + LambdaCode, + InputSpec, + OutSpec, + InputGroups, + AuxColumns, + InputType, + RowOffsets + ); +} + +void TYqlUserJob::DoImpl(const TFile& inHandle, const TVector<TFile>& outHandles) { + TYqlJobBase::Init(); + + TLambdaBuilder builder(FunctionRegistry.Get(), *Alloc, + Env.Get(), RandomProvider.Get(), TimeProvider.Get(), JobStats.Get(), &JobCountersProvider, SecureParamsProvider.Get()); + + TType* itemType = nullptr; + if (InputType) { + TStringStream err; + TProgramBuilder pgmBuilder(*Env, *FunctionRegistry); + itemType = NCommon::ParseTypeFromYson(TStringBuf{InputType}, pgmBuilder, err); + YQL_ENSURE(itemType, << err.Str()); + } + + YQL_ENSURE(LambdaCode); + TRuntimeNode rootNode = DeserializeRuntimeNode(LambdaCode, *Env); + THashMap<TString, TRuntimeNode> extraArgs; + rootNode = builder.TransformAndOptimizeProgram(rootNode, MakeTransformProvider(&extraArgs)); + + MkqlIOSpecs.Reset(new TMkqlIOSpecs()); + if (UseSkiff) { + MkqlIOSpecs->SetUseSkiff(OptLLVM, SkiffSysFields); + } + if (UseBlockInput) { + MkqlIOSpecs->SetUseBlockInput(); + } + MkqlIOSpecs->Init(*CodecCtx, InputSpec, InputGroups, TableNames, itemType, AuxColumns, OutSpec, JobStats.Get()); + if (!RowOffsets.empty()) { + MkqlIOSpecs->SetTableOffsets(RowOffsets); + } + + TIntrusivePtr<TJobMkqlWriterImpl> mkqlWriter = MakeIntrusive<TJobMkqlWriterImpl>(*MkqlIOSpecs, outHandles); + TIntrusivePtr<NYT::IReaderImplBase> reader; + + if (itemType) { + if (YamrInput) { + reader = MakeIntrusive<NYT::TYaMRTableReader>(MakeIntrusive<NYT::TJobReader>(inHandle)); + } + else { + reader = MakeIntrusive<TJobMkqlReaderImpl>(inHandle); + } + } + + std::vector<NKikimr::NMiniKQL::TNode*> entryPoints(1, rootNode.GetNode()); + for (auto& item: extraArgs) { + entryPoints.push_back(item.second.GetNode()); + } + auto maxRss = TRusage::Get().MaxRss; + CompGraph = builder.BuildGraph( + GetJobFactory(*CodecCtx, OptLLVM, MkqlIOSpecs.Get(), reader.Get(), mkqlWriter.Get()), + UdfValidateMode, + NUdf::EValidatePolicy::Fail, OptLLVM, + EGraphPerProcess::Single, + Explorer, + rootNode, + std::move(entryPoints) + ); + + MKQL_SET_STAT(JobStats, Mkql_BuildGraphRssDelta, TRusage::Get().MaxRss - maxRss); + MKQL_SET_STAT(JobStats, Mkql_TotalRuntimeNodes, Explorer.GetNodes().size()); + MKQL_SET_STAT(JobStats, Job_InitTime, (ThreadCPUTime() - StartTime) / 1000); + + auto beginCalcTime = ThreadCPUTime(); + + if (CompGraph) { + for (size_t i: xrange(extraArgs.size())) { + if (auto entry = CompGraph->GetEntryPoint(i + 1, false)) { + entry->SetValue(CompGraph->GetContext(), NUdf::TUnboxedValue::Zero()); + } + } + + CodecCtx->HolderFactory = &CompGraph->GetHolderFactory(); + CompGraph->Prepare(); + BindTerminator.Reset(new TBindTerminator(CompGraph->GetTerminator())); + + if (auto mkqlReader = dynamic_cast<TJobMkqlReaderImpl*>(reader.Get())) { + mkqlReader->SetSpecs(*MkqlIOSpecs, CompGraph->GetHolderFactory()); + mkqlReader->Next(); // Prefetch first record to unify behavior with TYaMRTableReader + } + } + + NUdf::TUnboxedValue value = CompGraph->GetValue(); + if (rootNode.GetStaticType()->IsStream()) { + NUdf::TUnboxedValue item; + const auto status = value.Fetch(item); + YQL_ENSURE(status == NUdf::EFetchStatus::Finish); + } else { + YQL_ENSURE(value.IsFinish()); + } + + MKQL_SET_STAT(JobStats, Job_CalcTime, (ThreadCPUTime() - beginCalcTime) / 1000); + + if (auto mkqlReader = dynamic_cast<TJobMkqlReaderImpl*>(reader.Get())) { + mkqlReader->Finish(); + } + reader.Drop(); + mkqlWriter->Finish(); + mkqlWriter.Drop(); + + MkqlIOSpecs->Clear(); + MkqlIOSpecs.Destroy(); +} + +} // NYql diff --git a/yt/yql/providers/yt/job/yql_job_user.h b/yt/yql/providers/yt/job/yql_job_user.h new file mode 100644 index 0000000000..7d5ece627b --- /dev/null +++ b/yt/yql/providers/yt/job/yql_job_user.h @@ -0,0 +1,112 @@ +#pragma once + +#include "yql_job_base.h" + +#include <yt/yql/providers/yt/codec/yt_codec_job.h> +#include <yql/essentials/providers/common/codec/yql_codec_type_flags.h> +#include <yql/essentials/minikql/computation/mkql_computation_node.h> +#include <yql/essentials/minikql/computation/mkql_value_builder.h> +#include <yql/essentials/minikql/mkql_node.h> +#include <yql/essentials/minikql/mkql_node_visitor.h> +#include <yql/essentials/minikql/mkql_alloc.h> +#include <yql/essentials/minikql/mkql_terminator.h> + +#include <yt/cpp/mapreduce/interface/format.h> +#include <yt/cpp/mapreduce/io/job_reader.h> +#include <yt/cpp/mapreduce/io/job_writer.h> + +#include <util/generic/string.h> +#include <util/generic/vector.h> +#include <util/generic/ptr.h> +#include <util/generic/hash_set.h> + +#include <utility> + +namespace NKikimr { + namespace NMiniKQL { + class IFunctionRegistry; + } +} + +namespace NYql { + +class TYqlUserJob: public TYqlJobBase { +public: + TYqlUserJob() + : TYqlJobBase() + { + } + virtual ~TYqlUserJob() = default; + + void SetUseSkiff(bool useSkiff, TMkqlIOSpecs::TSystemFields sysFields) { + UseSkiff = useSkiff; + SkiffSysFields = sysFields; + } + + void SetUseBlockInput(bool useBlockInput) { + UseBlockInput = useBlockInput; + } + + void SetYamrInput(bool yamrInput) { + YamrInput = yamrInput; + } + + void SetLambdaCode(const TString& code) { + LambdaCode = code; + } + + void SetInputSpec(const TString& spec) { + InputSpec = spec; + } + + void SetInputGroups(const TVector<ui32>& inputGroups) { + InputGroups = inputGroups; + } + + void SetOutSpec(const TString& spec) { + OutSpec = spec; + } + + void SetAuxColumns(const THashSet<TString>& auxColumns) { + AuxColumns = auxColumns; + } + + void SetInputType(const TString& type) { + InputType = type; + } + + void SetRowOffsets(const TVector<ui64>& rowOffsets) { + RowOffsets = rowOffsets; + } + + std::pair<NYT::TFormat, NYT::TFormat> GetIOFormats(const NKikimr::NMiniKQL::IFunctionRegistry* functionRegistry) const; + + void Save(IOutputStream& s) const override; + void Load(IInputStream& s) override; + +protected: + void DoImpl(const TFile& inHandle, const TVector<TFile>& outHandles) final; + +protected: + // Serializable part (don't forget to add new members to Save/Load) + bool UseSkiff = false; + bool UseBlockInput = false; + TMkqlIOSpecs::TSystemFields SkiffSysFields; + bool YamrInput = false; + TString LambdaCode; + TString InputSpec; + TString OutSpec; + TVector<ui32> InputGroups; + THashSet<TString> AuxColumns; + TString InputType; + TVector<ui64> RowOffsets; + // End of serializable part + + NKikimr::NMiniKQL::TExploringNodeVisitor Explorer; + THolder<NKikimr::NMiniKQL::IComputationGraph> CompGraph; + THolder<NKikimr::NMiniKQL::TBindTerminator> BindTerminator; + + THolder<TMkqlIOSpecs> MkqlIOSpecs; +}; + +} // NYql diff --git a/yt/yql/providers/yt/lib/config_clusters/config_clusters.cpp b/yt/yql/providers/yt/lib/config_clusters/config_clusters.cpp new file mode 100644 index 0000000000..e3a0c569c2 --- /dev/null +++ b/yt/yql/providers/yt/lib/config_clusters/config_clusters.cpp @@ -0,0 +1,114 @@ +#include "config_clusters.h" + +#include <yql/essentials/providers/common/proto/gateways_config.pb.h> + +#include <util/generic/vector.h> +#include <util/generic/yexception.h> + +namespace NYql { + +TConfigClusters::TConfigClusters(const TYtGatewayConfig& config) { + for (auto& cluster: config.GetClusterMapping()) { + AddCluster(cluster, true); + } +} + +void TConfigClusters::AddCluster(const TYtClusterConfig& cluster, bool checkDuplicate) { + if (!cluster.GetName()) { + ythrow yexception() << "TYtGatewayConfig: Cluster name must be specified"; + } + if (checkDuplicate && Clusters_.contains(cluster.GetName())) { + ythrow yexception() << "TYtGatewayConfig: Duplicate cluster name: " << cluster.GetName(); + } + + TClusterInfo& info = Clusters_[cluster.GetName()]; + if (cluster.GetCluster()) { + info.RealName = cluster.GetCluster(); + info.YtName = cluster.HasYTName() ? cluster.GetYTName() : cluster.GetName(); + TString ytName = info.YtName; + ytName.to_lower(); + YtName2Name_.emplace(ytName, cluster.GetName()); + } else { + ythrow yexception() << "TYtGatewayConfig: Cluster address must be specified"; + } + + if (cluster.HasYTToken()) { + info.Token = cluster.GetYTToken(); + } + + if (cluster.GetDefault()) { + if (DefaultClusterName_) { + ythrow yexception() << "TYtGatewayConfig: More than one default cluster (current: " + << cluster.GetName() << ", previous: " << DefaultClusterName_ << ")"; + } + DefaultClusterName_ = cluster.GetName(); + } +} + +const TString& TConfigClusters::GetServer(const TString& name) const { + if (const TClusterInfo* info = Clusters_.FindPtr(name)) { + return info->RealName; + } else { + ythrow yexception() << "Unknown cluster name: " << name; + } +} + +TString TConfigClusters::TryGetServer(const TString& name) const { + if (const TClusterInfo* info = Clusters_.FindPtr(name)) { + return info->RealName; + } else { + return {}; + } +} + +const TString& TConfigClusters::GetYtName(const TString& name) const { + if (const TClusterInfo* info = Clusters_.FindPtr(name)) { + return info->YtName; + } else { + ythrow yexception() << "Unknown cluster name: " << name; + } +} + +TString TConfigClusters::GetNameByYtName(const TString& ytName) const { + TString ytNameCopy = ytName; + ytNameCopy.to_lower(); + + if (const TString* name = YtName2Name_.FindPtr(ytNameCopy)) { + return *name; + } + + // no exception + return ytName; +} + +TMaybe<TString> TConfigClusters::GetAuth(const TString& name) const { + if (const TClusterInfo* info = Clusters_.FindPtr(name)) { + return info->Token; + } + return Nothing(); +} + +void TConfigClusters::GetAllClusters(TVector<TString>& names) const { + names.clear(); + for (const auto& c: Clusters_) { + names.push_back(c.first); + } +} + +const TString& TConfigClusters::GetDefaultClusterName() const { + if (!DefaultClusterName_) { + ythrow yexception() << "TYtGatewayConfig: No default cluster"; + } + return DefaultClusterName_; +} + +TString TConfigClusters::GetDefaultYtServer(const TYtGatewayConfig& config) { + for (auto& cluster: config.GetClusterMapping()) { + if (cluster.GetDefault()) { + return cluster.GetCluster(); + } + } + return {}; +} + +} // NYql diff --git a/yt/yql/providers/yt/lib/config_clusters/config_clusters.h b/yt/yql/providers/yt/lib/config_clusters/config_clusters.h new file mode 100644 index 0000000000..9ac135883f --- /dev/null +++ b/yt/yql/providers/yt/lib/config_clusters/config_clusters.h @@ -0,0 +1,44 @@ +#pragma once + +#include <util/generic/maybe.h> +#include <util/generic/hash.h> +#include <util/generic/string.h> +#include <util/generic/ptr.h> + +namespace NYql { + +class TYtClusterConfig; +class TYtGatewayConfig; + +class TConfigClusters: public TThrRefBase { +private: + struct TClusterInfo { + TString RealName; + TString YtName; + TMaybe<TString> Token; + }; +public: + using TPtr = TIntrusivePtr<TConfigClusters>; + + explicit TConfigClusters(const TYtGatewayConfig& config); + + void AddCluster(const TYtClusterConfig& cluster, bool checkDuplicate); + + const TString& GetServer(const TString& name) const; + TString TryGetServer(const TString& name) const; + const TString& GetYtName(const TString& name) const; + TString GetNameByYtName(const TString& ytName) const; + TMaybe<TString> GetAuth(const TString& name) const; + void GetAllClusters(TVector<TString>& names) const; + const TString& GetDefaultClusterName() const; + + static TString GetDefaultYtServer(const TYtGatewayConfig& config); + +private: + THashMap<TString, TClusterInfo> Clusters_; + // ytName.to_lower() -> Name + THashMap<TString, TString> YtName2Name_; + TString DefaultClusterName_; +}; + +} // NYql diff --git a/yt/yql/providers/yt/lib/config_clusters/ya.make b/yt/yql/providers/yt/lib/config_clusters/ya.make new file mode 100644 index 0000000000..c2b4177920 --- /dev/null +++ b/yt/yql/providers/yt/lib/config_clusters/ya.make @@ -0,0 +1,12 @@ +LIBRARY() + +SRCS( + config_clusters.cpp + config_clusters.h +) + +PEERDIR( + yql/essentials/providers/common/proto +) + +END() diff --git a/yt/yql/providers/yt/lib/expr_traits/ya.make b/yt/yql/providers/yt/lib/expr_traits/ya.make new file mode 100644 index 0000000000..13111a73c0 --- /dev/null +++ b/yt/yql/providers/yt/lib/expr_traits/ya.make @@ -0,0 +1,20 @@ +LIBRARY() + +SRCS( + yql_expr_traits.cpp +) + +PEERDIR( + yql/essentials/ast + yql/essentials/minikql/computation + yql/essentials/utils/log + yql/essentials/core + yql/essentials/core/expr_nodes + yql/essentials/providers/common/provider + yt/yql/providers/yt/common + yt/yql/providers/yt/expr_nodes +) + +YQL_LAST_ABI_VERSION() + +END() diff --git a/yt/yql/providers/yt/lib/expr_traits/yql_expr_traits.cpp b/yt/yql/providers/yt/lib/expr_traits/yql_expr_traits.cpp new file mode 100644 index 0000000000..0e2e6de57f --- /dev/null +++ b/yt/yql/providers/yt/lib/expr_traits/yql_expr_traits.cpp @@ -0,0 +1,443 @@ +#include "yql_expr_traits.h" + +#include <yql/essentials/providers/common/provider/yql_provider.h> +#include <yt/yql/providers/yt/expr_nodes/yql_yt_expr_nodes.h> +#include <yt/yql/providers/yt/common/yql_yt_settings.h> +#include <yql/essentials/core/expr_nodes/yql_expr_nodes.h> +#include <yql/essentials/core/yql_opt_utils.h> +#include <yql/essentials/core/yql_expr_optimize.h> +#include <yql/essentials/core/yql_type_annotation.h> +#include <yql/essentials/minikql/computation/mkql_computation_node_holders.h> +#include <yql/essentials/utils/log/log.h> + +#include <util/generic/hash_set.h> +#include <util/generic/vector.h> +#include <util/generic/algorithm.h> +#include <util/generic/size_literals.h> +#include <util/string/cast.h> + +namespace NYql { + namespace { + // Mapping of comparison op to its normalized form + const THashSet<TStringBuf> RANGE_COMPARISON_OPS = { + NNodes::TCoCmpLess::CallableName(), + NNodes::TCoCmpLessOrEqual::CallableName(), + NNodes::TCoCmpEqual::CallableName(), + NNodes::TCoCmpGreater::CallableName(), + NNodes::TCoCmpGreaterOrEqual::CallableName(), + NNodes::TCoCmpStartsWith::CallableName(), + }; + + } + + bool IsRangeComparison(const TStringBuf& operation) { + return RANGE_COMPARISON_OPS.contains(operation); + } + + void ScanResourceUsage(const TExprNode& input, const TYtSettings& config, const TTypeAnnotationContext* types, + TMap<TStringBuf, ui64>* memoryUsage, TMap<TStringBuf, double>* cpuUsage, size_t* files) + { + VisitExpr(input, [&](const TExprNode& node) { + if (NNodes::TYtOutput::Match(&node)) { + // Stop traversing dependent operations + return false; + } + + if (memoryUsage) { + if (node.IsCallable("CommonJoinCore")) { + if (auto memLimitSetting = GetSetting(*node.Child(5), "memLimit")) { + (*memoryUsage)["CommonJoinCore"] += FromString<ui64>(memLimitSetting->Child(1)->Content()); + } + } else if (node.IsCallable("WideCombiner")) { + (*memoryUsage)["WideCombiner"] += FromString<ui64>(node.Child(1U)->Content()); + } else if (NNodes::TCoCombineCore::Match(&node)) { + (*memoryUsage)["CombineCore"] += FromString<ui64>(node.Child(NNodes::TCoCombineCore::idx_MemLimit)->Content()); + } + } + + if (node.IsCallable("Udf")) { // TODO: use YQL-369 when it's ready + if (files) { + ++*files; + } + if (node.Child(0)->Content().StartsWith("Geo.")) { + if (files) { + ++*files; // geobase + } + if (memoryUsage) { + (*memoryUsage)["Geo module"] = 2_GB; // Take into account only once + } + } + if (memoryUsage && node.Child(0)->Content().StartsWith("UserSessions.")) { + (*memoryUsage)["UserSessions module"] = 512_MB; // Take into account only once + } + } + + if (NNodes::TYtTableContent::Match(&node)) { + if (files) { + auto content = NNodes::TYtTableContent(&node); + if (auto read = content.Input().Maybe<NNodes::TYtReadTable>()) { + for (auto section: read.Cast().Input()) { + *files += section.Paths().Size(); + } + } else { + // YtOutput + ++*files; + } + } + if (memoryUsage) { + if (auto setting = NYql::GetSetting(*node.Child(NNodes::TYtTableContent::idx_Settings), "memUsage")) { + (*memoryUsage)["YtTableContent"] += FromString<ui64>(setting->Child(1)->Content()); + } + } + // Increase CPU only for CROSS JOIN. Check "rowFactor" as CROSS JOIN flag + if (cpuUsage && HasSetting(*node.Child(NNodes::TYtTableContent::idx_Settings), "rowFactor")) { + if (auto setting = NYql::GetSetting(*node.Child(NNodes::TYtTableContent::idx_Settings), "itemsCount")) { + (*cpuUsage)["YtTableContent"] = double(FromString<ui64>(setting->Child(1)->Content())); + } + } + } + + if (NNodes::TCoSwitch::Match(&node)) { + if (memoryUsage) { + (*memoryUsage)["Switch"] += FromString<ui64>(node.Child(1)->Content()); + } + if (cpuUsage) { + const auto& inputItemType = GetSeqItemType(*node.Head().GetTypeAnn()); + ui32 inputStreamsCount = 1; + if (inputItemType.GetKind() == ETypeAnnotationKind::Variant) { + auto underlyingType = inputItemType.Cast<TVariantExprType>()->GetUnderlyingType(); + inputStreamsCount = underlyingType->Cast<TTupleExprType>()->GetSize(); + } + + TVector<ui32> streamUsage(inputStreamsCount, 0); + for (ui32 i = 2; i < node.ChildrenSize(); i += 2) { + for (auto& child : node.Child(i)->Children()) { + ++streamUsage[FromString<ui32>(child->Content())]; + } + } + auto maxStreamUsage = *MaxElement(streamUsage.begin(), streamUsage.end()); + if (maxStreamUsage > 1) { + double usage = maxStreamUsage; + if (auto prev = cpuUsage->FindPtr("Switch")) { + usage *= *prev; + } + (*cpuUsage)["Switch"] = usage; + } + } + } + + if (node.IsCallable(TStringBuf("ScriptUdf"))) { + if (files) { + ++*files; + } + if (cpuUsage) { + if (auto cpu = config.ScriptCpu.Get(NCommon::ALL_CLUSTERS)) { + (*cpuUsage)["ScriptUdf"] = *cpu; // Take into account only once + } + if (node.Child(0)->Content().EndsWith("Javascript")) { + if (auto cpu = config.JavascriptCpu.Get(NCommon::ALL_CLUSTERS)) { + (*cpuUsage)["Javascript module"] = *cpu; // Take into account only once + } + } + } + if (node.Child(0)->Content().Contains("Python")) { + if (memoryUsage) { + (*memoryUsage)["Python module"] = 512_MB; // Take into account only once + } + if (cpuUsage) { + if (auto cpu = config.PythonCpu.Get(NCommon::ALL_CLUSTERS)) { + (*cpuUsage)["Python module"] = *cpu; // Take into account only once + } + } + } + if (node.ChildrenSize() >= 5) { + if (cpuUsage) { + if (auto cpuSetting = GetSetting(*node.Child(4), "cpu")) { + double usage = FromString<double>(cpuSetting->Child(1)->Content()); + if (auto prev = cpuUsage->FindPtr("ScriptUdf from settings arg")) { + usage *= *prev; + } + (*cpuUsage)["ScriptUdf from settings arg"] = usage; + } + } + if (memoryUsage) { + if (auto extraMemSetting = GetSetting(*node.Child(4), "extraMem")) { + (*memoryUsage)["ScriptUdf from settings arg"] += FromString<ui64>(extraMemSetting->Child(1)->Content()); + } + } + } + } + if (files) { + if (node.IsCallable("FilePath") || node.IsCallable("FileContent")) { + ++*files; + } + else if (node.IsCallable("FolderPath")) { + YQL_ENSURE(types); + const auto& name = node.Head().Content(); + if (auto blocks = types->UserDataStorage->FindUserDataFolder(name)) { + *files += blocks->size(); + } + } + } + return true; + }); + } + + TExpressionResorceUsage ScanExtraResourceUsage(const TExprNode& input, const TYtSettings& config) { + TExpressionResorceUsage ret; + TMap<TStringBuf, ui64> memory; + TMap<TStringBuf, double> cpu; + ScanResourceUsage(input, config, nullptr, &memory, &cpu, nullptr); + + for (auto& m: memory) { + ret.Memory += m.second; + YQL_CLOG(DEBUG, ProviderYt) << "Increased extraMemUsage for " << m.first << " to " << ret.Memory << " (by " << m.second << ")"; + } + for (auto& c: cpu) { + ret.Cpu *= c.second; + YQL_CLOG(DEBUG, ProviderYt) << "Increased cpu for " << c.first << " to " << ret.Cpu; + } + return ret; + } + + namespace { + bool IsPrimitiveType(const TTypeAnnotationNode* type) { + if (ETypeAnnotationKind::Data != type->GetKind()) { + return false; + } + using namespace NKikimr::NUdf; + switch (type->Cast<TDataExprType>()->GetSlot()) { + case EDataSlot::Bool: + case EDataSlot::Uint8: + case EDataSlot::Int8: + case EDataSlot::Uint16: + case EDataSlot::Int16: + case EDataSlot::Uint32: + case EDataSlot::Int32: + case EDataSlot::Uint64: + case EDataSlot::Int64: + case EDataSlot::Float: + case EDataSlot::Double: + case EDataSlot::Date: + case EDataSlot::Datetime: + case EDataSlot::Timestamp: + case EDataSlot::Interval: + case EDataSlot::Date32: + case EDataSlot::Datetime64: + case EDataSlot::Timestamp64: + case EDataSlot::Interval64: + return true; + case EDataSlot::String: + case EDataSlot::Utf8: + case EDataSlot::Yson: + case EDataSlot::Json: + case EDataSlot::Decimal: + case EDataSlot::Uuid: + case EDataSlot::TzDate: + case EDataSlot::TzDatetime: + case EDataSlot::TzTimestamp: + case EDataSlot::TzDate32: + case EDataSlot::TzDatetime64: + case EDataSlot::TzTimestamp64: + case EDataSlot::DyNumber: + case EDataSlot::JsonDocument: + break; + } + return false; + } + + bool IsSmallType(const TTypeAnnotationNode* type) { + if (!IsPrimitiveType(type)) { + return false; + } + using namespace NKikimr::NUdf; + switch (type->Cast<TDataExprType>()->GetSlot()) { + case EDataSlot::Uint64: // Doesn't fit to 7 bytes in common case + case EDataSlot::Int64: // Doesn't fit to 7 bytes in common case + case EDataSlot::Double: // Doesn't fit to 7 bytes + case EDataSlot::Timestamp: + case EDataSlot::Interval: + return false; + default: + break; + } + return true; + } + + } + + void CalcToDictFactors( + const TTypeAnnotationNode* keyType, + const TTypeAnnotationNode* payloadType, + EDictType type, bool many, bool compact, + double& sizeFactor, ui64& rowFactor) { + sizeFactor = 1.; + rowFactor = 0ULL; + type = SelectDictType(type, keyType); + + // See https://st.yandex-team.ru/YQL-848#1473441162000 for measures + if (compact) { + if (!many && ETypeAnnotationKind::Void == payloadType->GetKind()) { + if (IsPrimitiveType(keyType)) { + // THashedDictSetAccumulator<THashedSingleFixedCompactSetTraits> + sizeFactor = 0.; + rowFactor = 16; + } else { + // THashedDictSetAccumulator<THashedCompactSetTraits> + sizeFactor = 3.0; + rowFactor = 26; + } + } else { + if (IsPrimitiveType(keyType)) { + // THashedDictMapAccumulator<THashedSingleFixedCompactMapTraits> + sizeFactor = (!IsSmallType(payloadType)) * 3.3; + rowFactor = 28; + } else { + // THashedDictMapAccumulator<THashedCompactMapTraits> + sizeFactor = (!IsSmallType(keyType) + !IsSmallType(payloadType)) * 3.3; + rowFactor = 28; + } + } + } else if (type == EDictType::Hashed) { + // Not tuned. Should not be used with large dicts + sizeFactor = 1.5; + rowFactor = 46; + } else { + sizeFactor = 1.5; + rowFactor = sizeof(NKikimr::NMiniKQL::TKeyPayloadPair); + } + } + + TMaybe<TIssue> CalcToDictFactors(const TExprNode& toDictNode, TExprContext& ctx, double& sizeFactor, ui64& rowFactor) { + YQL_ENSURE(toDictNode.IsCallable({"ToDict", "SqueezeToDict", "SqlIn"})); + + TMaybe<bool> isMany; + TMaybe<EDictType> type; + bool isCompact = false; + TMaybe<ui64> itemsCount; + if (toDictNode.IsCallable("SqlIn")) { + if (auto typeAnn = toDictNode.Head().GetTypeAnn(); typeAnn->GetKind() == ETypeAnnotationKind::List + && typeAnn->Cast<TListExprType>()->GetItemType()->GetKind() == ETypeAnnotationKind::Struct) { + isMany = false; + type = EDictType::Auto; + isCompact = true; + + auto structType = typeAnn->Cast<TListExprType>()->GetItemType()->Cast<TStructExprType>(); + YQL_ENSURE(structType->GetSize() == 1); + auto dictKeyType = structType->GetItems()[0]->GetItemType(); + + CalcToDictFactors( + dictKeyType, + ctx.MakeType<TVoidExprType>(), + *type, false/*isMany*/, true/*isCompact*/, + sizeFactor, rowFactor); + } + return {}; + } else if (auto err = ParseToDictSettings(toDictNode, ctx, type, isMany, itemsCount, isCompact)) { + return err; + } + + CalcToDictFactors( + toDictNode.Child(1)->GetTypeAnn(), + toDictNode.Child(2)->GetTypeAnn(), + *type, *isMany, isCompact, + sizeFactor, rowFactor); + + return {}; + } + + namespace { + + THashSet<TStringBuf> TABLE_CONTENT_CONSUMER = { + TStringBuf("Fold"), + TStringBuf("Fold1"), + TStringBuf("Apply"), + TStringBuf("ToOptional"), + TStringBuf("Head"), + TStringBuf("Last"), + TStringBuf("ToDict"), + TStringBuf("SqueezeToDict"), + TStringBuf("Iterator"), // Why? + TStringBuf("Collect"), + TStringBuf("Length"), + TStringBuf("HasItems"), + TStringBuf("SqlIn"), + }; + + bool HasExternalArgsImpl(const TExprNode& root, TNodeSet& visited, TNodeSet& activeArgs) { + if (!visited.emplace(&root).second) { + return false; + } + + if (root.Type() == TExprNode::Argument) { + if (activeArgs.find(&root) == activeArgs.cend()) { + return true; + } + } + + if (root.Type() == TExprNode::Lambda) { + root.Child(0)->ForEachChild([&](const TExprNode& arg) { + activeArgs.emplace(&arg); + }); + } + + bool res = false; + root.ForEachChild([&](const TExprNode& child) { + res = res || HasExternalArgsImpl(child, visited, activeArgs); + }); + return res; + } + + bool HasExternalArgs(const TExprNode& node) { + bool res = false; + node.ForEachChild([&](const TExprNode& child) { + if (!res && child.Type() == TExprNode::Lambda) { + TNodeSet visited; + TNodeSet activeArgs; + res = res || HasExternalArgsImpl(child, visited, activeArgs); + } + }); + return res; + } + } + + bool GetTableContentConsumerNodes(const TExprNode& node, const TExprNode& rootNode, + const TParentsMap& parentsMap, TNodeSet& consumers) + { + const auto parents = parentsMap.find(&node); + if (parents == parentsMap.cend()) { + return true; + } + + for (const auto& parent : parents->second) { + if (parent == &rootNode) { + continue; + } + else if (parent->Type() == TExprNode::Arguments) { + continue; + } + else if (parent->IsCallable("DependsOn")) { + continue; + } + else if (parent->IsCallable(TABLE_CONTENT_CONSUMER)) { + if (HasExternalArgs(*parent)) { + return false; + } + consumers.insert(parent); + } + else if (const auto kind = parent->GetTypeAnn()->GetKind(); ETypeAnnotationKind::Flow == kind || ETypeAnnotationKind::List == kind || ETypeAnnotationKind::Stream == kind) { + if (HasExternalArgs(*parent)) { + return false; + } + if (!GetTableContentConsumerNodes(*parent, rootNode, parentsMap, consumers)) { + return false; + } + } + else { + return false; + } + } + + return true; + } +} diff --git a/yt/yql/providers/yt/lib/expr_traits/yql_expr_traits.h b/yt/yql/providers/yt/lib/expr_traits/yql_expr_traits.h new file mode 100644 index 0000000000..7523b4c776 --- /dev/null +++ b/yt/yql/providers/yt/lib/expr_traits/yql_expr_traits.h @@ -0,0 +1,32 @@ +#pragma once + +#include <yql/essentials/ast/yql_expr.h> +#include <yql/essentials/core/yql_opt_utils.h> + +#include <util/generic/strbuf.h> +#include <util/generic/map.h> + +namespace NYql { + +struct TYtSettings; +struct TTypeAnnotationContext; + +bool IsRangeComparison(const TStringBuf& operation); + +struct TExpressionResorceUsage { + ui64 Memory = 0; + double Cpu = 1.0; +}; + +void ScanResourceUsage(const TExprNode& input, const TYtSettings& config, const TTypeAnnotationContext* types, + TMap<TStringBuf, ui64>* memoryUsage, TMap<TStringBuf, double>* cpuUsage, size_t* files); +TExpressionResorceUsage ScanExtraResourceUsage(const TExprNode& input, const TYtSettings& config); + +void CalcToDictFactors(const TTypeAnnotationNode* keyType, const TTypeAnnotationNode* payloadType, + EDictType type, bool many, bool compact, double& sizeFactor, ui64& rowFactor); +TMaybe<TIssue> CalcToDictFactors(const TExprNode& toDictNode, TExprContext& ctx, double& sizeFactor, ui64& rowFactor); + +bool GetTableContentConsumerNodes(const TExprNode& node, const TExprNode& rootNode, + const TParentsMap& parentsMap, TNodeSet& consumers); + +} // NYql diff --git a/yt/yql/providers/yt/lib/graph_reorder/ya.make b/yt/yql/providers/yt/lib/graph_reorder/ya.make new file mode 100644 index 0000000000..299e4be66f --- /dev/null +++ b/yt/yql/providers/yt/lib/graph_reorder/ya.make @@ -0,0 +1,17 @@ +LIBRARY() + +SRCS( + yql_graph_reorder.cpp +) + +PEERDIR( + yql/essentials/ast + yql/essentials/utils/log + yql/essentials/core + yql/essentials/core/expr_nodes + yql/essentials/providers/common/provider +) + +YQL_LAST_ABI_VERSION() + +END() diff --git a/yt/yql/providers/yt/lib/graph_reorder/yql_graph_reorder.cpp b/yt/yql/providers/yt/lib/graph_reorder/yql_graph_reorder.cpp new file mode 100644 index 0000000000..5df20d7082 --- /dev/null +++ b/yt/yql/providers/yt/lib/graph_reorder/yql_graph_reorder.cpp @@ -0,0 +1,357 @@ +#include "yql_graph_reorder.h" + +#include <yql/essentials/providers/common/provider/yql_provider.h> +#include <yql/essentials/providers/common/provider/yql_provider_names.h> +#include <yql/essentials/core/expr_nodes/yql_expr_nodes.h> +#include <yql/essentials/core/yql_expr_optimize.h> +#include <yql/essentials/utils/log/log.h> + +#include <tuple> + +namespace NYql { + +using namespace NNodes; + +TDependencyUpdater::TDependencyUpdater(TStringBuf provider, TStringBuf newConfigureName) + : Provider(provider) + , NewConfigureName(newConfigureName) +{ +} + +void TDependencyUpdater::ScanConfigureDeps(const TExprNode::TPtr& node) { + VisitedNodes.insert(node.Get()); + + TMaybe<TExprNode*> prevConfigure; + TMaybe<TVector<TExprNode*>> prevReads; + bool popRead = false; + // Don't use TCoConfigure().DataSource() - result provider uses DataSink here + const bool isConfigure = TCoConfigure::Match(node.Get()) && node->Child(TCoConfigure::idx_DataSource)->Child(0)->Content() == Provider; + const bool isNewConfigure = node->IsCallable(NewConfigureName); + if (isConfigure || isNewConfigure) { + if (LastConfigure) { + auto& map = NodeToConfigureDeps[LastConfigure]; + map.emplace(node, map.size()); + } + if (isConfigure) { + Configures.push_back(node); + // All Configure! nodes must be present in NodeToConfigureDeps + NodeToConfigureDeps.emplace(node.Get(), TSyncMap()); + } + prevConfigure = LastConfigure; + LastConfigure = node.Get(); + + for (auto read: LastReads) { + auto& map = NodeToConfigureDeps[read]; + map.emplace(node, map.size()); + } + prevReads.ConstructInPlace(); + prevReads->swap(LastReads); + } + else if (TCoRead::Match(node.Get()) && TCoRead(node).DataSource().Category().Value() == Provider) { + LastReads.push_back(node.Get()); + popRead = true; + } + + if (!isNewConfigure) { // Assume all nodes under provider specific Configure are already processed + for (const auto& child : node->Children()) { + if (VisitedNodes.cend() == VisitedNodes.find(child.Get()) || (child->Content() != SyncName && child->Content().EndsWith('!'))) { + ScanConfigureDeps(child); + } + } + } + + if (prevConfigure) { + LastConfigure = *prevConfigure; + } + if (prevReads) { + LastReads.swap(*prevReads); + } + if (popRead) { + LastReads.pop_back(); + } +}; + +void TDependencyUpdater::ScanNewReadDeps(const TExprNode::TPtr& input, TExprContext& ctx) { + THashMap<ui32, TExprNode::TPtr> commits; + VisitExprByFirst(input, [&](const TExprNode::TPtr& node) { + if (TCoCommit::Match(node.Get())) { + auto commit = TCoCommit(node); + if (commit.DataSink().Cast<TCoDataSink>().Category().Value() == Provider) { + auto settings = NCommon::ParseCommitSettings(commit, ctx); + if (!settings.Epoch) { + return false; + } + + ui32 commitEpoch = FromString<ui32>(settings.Epoch.Cast().Value()); + commits.emplace(commitEpoch, node); + } + } + return true; + }); + + TNodeMap<ui32> maxEpochs; + VisitExpr(input, [&](const TExprNode::TPtr& node) { + if (TCoRead::Match(node.Get()) && TCoRead(node).DataSource().Category().Value() == Provider) { + if (TMaybe<ui32> maxEpoch = GetReadEpoch(node)) { + auto world = node->Child(0); + if (*maxEpoch == 0) { + if (world->Type() != TExprNode::World + && !TCoSync::Match(world) + && !world->IsCallable(NewConfigureName)) + { + NewReadToCommitDeps[node.Get()] = TExprNode::TPtr(); + } + } + else if (*maxEpoch > 0) { + auto commit = commits.find(*maxEpoch); + YQL_ENSURE(commits.cend() != commit); + if (world != commit->second.Get() + && !TCoSync::Match(world) + && !world->IsCallable(NewConfigureName)) + { + NewReadToCommitDeps[node.Get()] = commit->second; + } + } + } + } + return true; + }); +} + +void TDependencyUpdater::ScanNewWriteDeps(const TExprNode::TPtr& input) { + TExprNode::TPtr lastWorld; + TExprNode::TPtr lastConfigure; + THashMap<TString, std::tuple<TExprNode::TPtr, size_t, TExprNode::TPtr>> writesByTarget; // target table -> (last Write! node, order between different writes, last Configure! dependency) + VisitExprByFirst(input, + [&](const TExprNode::TPtr& node) { + // Initially Sync! and NewConfigureName cannot appear in graph. + // So if we catch them then this part of graph is already processed. + if (node->IsCallable(NewConfigureName)) { + lastConfigure = node; + return false; + } + if (TCoSync::Match(node.Get()) || node->IsCallable(NewConfigureName)) { + lastWorld = node; + return false; + } + return true; + }, + [&](const TExprNode::TPtr& node) { + if (TCoConfigure::Match(node.Get()) && node->Child(TCoConfigure::idx_DataSource)->Child(0)->Content() == Provider) { + lastConfigure = node; + return true; + } + + if (TCoWrite::Match(node.Get())) { + auto category = TCoWrite(node).DataSink().Category().Value(); + TString target; + if (Provider == category) { + target = GetWriteTarget(node); + } else if (ResultProviderName == category) { + target = category; + } + if (target) { + auto& prevWrite = writesByTarget[target]; + if (std::get<0>(prevWrite)) { + if (node->HeadPtr() != std::get<0>(prevWrite)) { + NewWriteDeps[node.Get()].push_back(std::get<0>(prevWrite)); + } + if (lastConfigure && lastConfigure != std::get<2>(prevWrite)) { + auto& map = NodeToConfigureDeps[node.Get()]; + map.emplace(lastConfigure, map.size()); + std::get<2>(prevWrite) = lastConfigure; + } + } else { + std::get<1>(prevWrite) = writesByTarget.size(); + if (lastWorld && node->HeadPtr() != lastWorld) { + NewWriteDeps[node.Get()].push_back(lastWorld); + } + if (lastConfigure) { + auto& map = NodeToConfigureDeps[node.Get()]; + map.emplace(lastConfigure, map.size()); + std::get<2>(prevWrite) = lastConfigure; + } + } + std::get<0>(prevWrite) = node; + return true; + } + } + if (!writesByTarget.empty()) { + if (writesByTarget.size() > 1) { + auto& deps = NewWriteDeps[node.Get()]; + std::vector<std::pair<TExprNode::TPtr, size_t>> values; + std::transform(writesByTarget.begin(), writesByTarget.end(), std::back_inserter(values), + [](const auto& v) { return std::make_pair(std::get<0>(v.second), std::get<1>(v.second)); }); + std::sort(values.begin(), values.end(), [](const auto& l, const auto& r) { return l.second < r.second; }); + std::transform(values.begin(), values.end(), std::back_inserter(deps), [](const auto& v) { return v.first; }); + } else { + if (node->HeadPtr() != std::get<0>(writesByTarget.begin()->second)) { + NewWriteDeps[node.Get()].push_back(std::get<0>(writesByTarget.begin()->second)); + } + } + } + + lastWorld = node; + writesByTarget.clear(); + return true; + } + ); +} + +TExprNode::TPtr TDependencyUpdater::MakeSync(const TSyncMap& nodes, TExprNode::TChildrenType extra, TPositionHandle pos, TExprContext& ctx) { + using TPair = std::pair<TExprNode::TPtr, ui64>; + TVector<TPair> sortedList(nodes.cbegin(), nodes.cend()); + Sort(sortedList, [](const TPair& x, const TPair& y) { return x.second < y.second; }); + TExprNode::TListType syncChildren; + syncChildren.reserve(sortedList.size() + extra.size()); + std::transform(sortedList.begin(), sortedList.end(), std::back_inserter(syncChildren), [](const TPair& x) { return x.first; }); + if (extra) { + syncChildren.insert(syncChildren.end(), extra.begin(), extra.end()); + } + syncChildren.erase(std::remove_if(syncChildren.begin(), syncChildren.end(), [](const auto& n) { return n->IsWorld(); } ), syncChildren.end()); + return syncChildren.empty() + ? ctx.NewWorld(pos) + : syncChildren.size() == 1 + ? syncChildren.front() + : ctx.NewCallable(pos, TCoSync::CallableName(), std::move(syncChildren)); +} + +IGraphTransformer::TStatus TDependencyUpdater::ReorderGraph(const TExprNode::TPtr& input, TExprNode::TPtr& output, + TExprContext& ctx) +{ + ScanConfigureDeps(input); + ScanNewReadDeps(input, ctx); + ScanNewWriteDeps(input); + + TOptimizeExprSettings settings(nullptr); + settings.VisitChanges = true; + IGraphTransformer::TStatus status = IGraphTransformer::TStatus::Ok; + + output = input; + if (!NewReadToCommitDeps.empty() || !NewWriteDeps.empty()) { + if (!NodeToConfigureDeps.empty()) { + // Exclude all our Configure! from the graph + status = status.Combine(OptimizeExpr(output, output, [&](const TExprNode::TPtr& node, TExprContext& /*ctx*/) -> TExprNode::TPtr { + if (TCoConfigure::Match(node.Get()) && NodeToConfigureDeps.contains(node.Get())) { + // Skip Configure! itself + return node; + } + if (node->ChildrenSize() > 0) { + TExprNode::TPtr next = node->ChildPtr(0); + while (TCoConfigure::Match(next.Get()) && NodeToConfigureDeps.contains(next.Get())) { + next = next->ChildPtr(0); + } + if (next != node->ChildPtr(0)) { + node->ChildRef(0) = next; + } + } + return node; + }, ctx, settings)); + + if (status.Level == IGraphTransformer::TStatus::Error) { + return status; + } + + // Update root + while (TCoConfigure::Match(output.Get()) && NodeToConfigureDeps.contains(output.Get())) { + output = output->ChildPtr(0); + } + + // Make a separate chain of Configure! nodes + for (auto& item: NodeToConfigureDeps) { + if (!TCoConfigure::Match(item.first)) { + continue; + } + if (item.second.empty()) { + if (item.first->Child(0)->Type() != TExprNode::World) { + item.first->ChildRef(0) = ctx.NewWorld(item.first->Pos()); + } + } + else { + item.first->ChildRef(0) = MakeSync(item.second, {}, item.first->Pos(), ctx); + } + } + } + + // Reorder graph + TNodeOnNodeOwnedMap oldReadDeps; + status = status.Combine(OptimizeExpr(output, output, [&](const TExprNode::TPtr& node, TExprContext& ctx) -> TExprNode::TPtr { + if (node->ChildrenSize() > 0 && TCoLeft::Match(node->Child(0))) { + auto read = node->Child(0)->Child(0); + auto found = oldReadDeps.find(read); + if (found != oldReadDeps.cend()) { + node->ChildRef(0) = found->second; + } + else if (read->ChildrenSize() > 1 && read->Child(1)->Child(0)->Content() == Provider) { + node->ChildRef(0) = read->ChildPtr(0); + } + } + + if (TCoRight::Match(node.Get())) { + auto read = node->ChildPtr(0); + auto confSet = NodeToConfigureDeps.FindPtr(read.Get()); + auto found = NewReadToCommitDeps.find(read.Get()); + if (NewReadToCommitDeps.cend() != found) { + TExprNode::TPtr newWorld; + if (!found->second) { + if (confSet) { + YQL_ENSURE(!confSet->empty()); + newWorld = MakeSync(*confSet, {}, read->Pos(), ctx); + } + else if (read->Child(0)->Type() != TExprNode::World) { + newWorld = ctx.NewWorld(read->Pos()); + } + } + else { + if (confSet) { + newWorld = MakeSync(*confSet, {found->second}, read->Pos(), ctx); + } + else if (read->Child(0) != found->second.Get()) { + newWorld = found->second; + } + } + + if (newWorld && oldReadDeps.emplace(read.Get(), read->ChildPtr(0)).second) { + read->ChildRef(0) = newWorld; + } + } + else if (confSet) { + read->ChildRef(0) = MakeSync(*confSet, {read->HeadPtr()}, read->Pos(), ctx); + } + } + else if (!TCoRead::Match(node.Get())) { + auto deps = NewWriteDeps.Value(node.Get(), TExprNode::TListType()); + if (auto confSet = NodeToConfigureDeps.FindPtr(node.Get())) { + if (deps.empty()) { + deps.push_back(node->HeadPtr()); + } + node->ChildRef(0) = MakeSync(*confSet, deps, node->Pos(), ctx); + } else if (!deps.empty()) { + node->ChildRef(0) = MakeSync({}, deps, node->Pos(), ctx); + } + } + return node; + }, ctx, settings)); + + if (status.Level == IGraphTransformer::TStatus::Error) { + return status; + } + + YQL_CLOG(INFO, ProviderYt) << "DependencyUpdater-ReorderGraph"; + } + + if (!NodeToConfigureDeps.empty()) { + // Rename Configure! nodes + status = status.Combine(OptimizeExpr(output, output, [&](const TExprNode::TPtr& node, TExprContext& ctx) -> TExprNode::TPtr { + // Don't use TCoConfigure().DataSource() - result provider uses DataSink here + if (TCoConfigure::Match(node.Get()) && node->Child(TCoConfigure::idx_DataSource)->Child(0)->Content() == Provider) { + return ctx.RenameNode(*node, NewConfigureName); + } + return node; + }, ctx, settings)); + } + + return status; +} + +} // namespace NYql diff --git a/yt/yql/providers/yt/lib/graph_reorder/yql_graph_reorder.h b/yt/yql/providers/yt/lib/graph_reorder/yql_graph_reorder.h new file mode 100644 index 0000000000..70ca4adfc6 --- /dev/null +++ b/yt/yql/providers/yt/lib/graph_reorder/yql_graph_reorder.h @@ -0,0 +1,45 @@ +#pragma once + +#include <yql/essentials/core/yql_graph_transformer.h> +#include <yql/essentials/ast/yql_expr.h> + +#include <util/digest/numeric.h> +#include <util/generic/hash.h> +#include <util/generic/hash_set.h> +#include <util/generic/vector.h> +#include <util/generic/strbuf.h> +#include <util/generic/maybe.h> + +namespace NYql { + +// Scans (Configure!/Read!/Write!/Commit!)<-Configure! dependencies +class TDependencyUpdater { +public: + TDependencyUpdater(TStringBuf provider, TStringBuf newConfigureName); + virtual ~TDependencyUpdater() = default; + + IGraphTransformer::TStatus ReorderGraph(const TExprNode::TPtr& input, TExprNode::TPtr& output, TExprContext& ctx); + +protected: + virtual TMaybe<ui32> GetReadEpoch(const TExprNode::TPtr& readNode) const = 0; + virtual TString GetWriteTarget(const TExprNode::TPtr& writeNode) const = 0; + + void ScanConfigureDeps(const TExprNode::TPtr& node); + void ScanNewReadDeps(const TExprNode::TPtr& input, TExprContext& ctx); + void ScanNewWriteDeps(const TExprNode::TPtr& input); + static TExprNode::TPtr MakeSync(const TSyncMap& nodes, TExprNode::TChildrenType extra, TPositionHandle pos, TExprContext& ctx); + +private: + + const TStringBuf Provider; + const TStringBuf NewConfigureName; + THashMap<TExprNode*, TSyncMap> NodeToConfigureDeps; + TExprNode::TListType Configures; // To protect Configure! from die + TNodeOnNodeOwnedMap NewReadToCommitDeps; + TNodeSet VisitedNodes; + TExprNode* LastConfigure = nullptr; + TVector<TExprNode*> LastReads; + THashMap<TExprNode*, TExprNode::TListType> NewWriteDeps; +}; + +} // namespace NYql diff --git a/yt/yql/providers/yt/lib/hash/ya.make b/yt/yql/providers/yt/lib/hash/ya.make new file mode 100644 index 0000000000..508131391f --- /dev/null +++ b/yt/yql/providers/yt/lib/hash/ya.make @@ -0,0 +1,17 @@ +LIBRARY() + +SRCS( + yql_hash_builder.cpp + yql_op_hash.cpp +) + +PEERDIR( + contrib/libs/openssl + yql/essentials/ast + yql/essentials/utils + yql/essentials/utils/log + yql/essentials/core + yql/essentials/core/expr_nodes +) + +END() diff --git a/yt/yql/providers/yt/lib/hash/yql_hash_builder.cpp b/yt/yql/providers/yt/lib/hash/yql_hash_builder.cpp new file mode 100644 index 0000000000..8a2a843f3f --- /dev/null +++ b/yt/yql/providers/yt/lib/hash/yql_hash_builder.cpp @@ -0,0 +1,27 @@ +#include "yql_hash_builder.h" + +namespace NYql { + +THashBuilder::THashBuilder() { + SHA256_Init(&Sha256); +} + +void THashBuilder::Update(const void* data, size_t len) { + SHA256_Update(&Sha256, data, len); +} + +void THashBuilder::Update(TStringBuf data) { + SHA256_Update(&Sha256, data.data(), data.size()); +} + +void THashBuilder::Update(const TString& data) { + SHA256_Update(&Sha256, data.data(), data.size()); +} + +TString THashBuilder::Finish() { + unsigned char hash[SHA256_DIGEST_LENGTH]; + SHA256_Final(hash, &Sha256); + return TString((const char*)hash, (const char*)hash + SHA256_DIGEST_LENGTH); +} + +} // NYql diff --git a/yt/yql/providers/yt/lib/hash/yql_hash_builder.h b/yt/yql/providers/yt/lib/hash/yql_hash_builder.h new file mode 100644 index 0000000000..4194364770 --- /dev/null +++ b/yt/yql/providers/yt/lib/hash/yql_hash_builder.h @@ -0,0 +1,45 @@ +#pragma once + +#include <util/generic/strbuf.h> +#include <util/generic/string.h> + +#include <openssl/sha.h> + +#include <utility> +#include <type_traits> + +namespace NYql { + +class THashBuilder { +public: + THashBuilder(); + + template <typename T, std::enable_if_t<std::is_pod<std::remove_cv_t<T>>::value>* = nullptr> + void Update(T data) { + Update(&data, sizeof(data)); + } + void Update(TStringBuf data); + void Update(const TString& data); + + TString Finish(); + +private: + void Update(const void* data, size_t len); + +private: + SHA256_CTX Sha256; +}; + +template <class T> +static inline THashBuilder& operator<<(THashBuilder& builder, const T& t) { + builder.Update(t); + return builder; +} + +template <class T> +static inline THashBuilder&& operator<<(THashBuilder&& builder, const T& t) { + builder.Update(t); + return std::move(builder); +} + +} diff --git a/yt/yql/providers/yt/lib/hash/yql_op_hash.cpp b/yt/yql/providers/yt/lib/hash/yql_op_hash.cpp new file mode 100644 index 0000000000..0fe82bd4e2 --- /dev/null +++ b/yt/yql/providers/yt/lib/hash/yql_op_hash.cpp @@ -0,0 +1,140 @@ +#include "yql_op_hash.h" +#include "yql_hash_builder.h" + +#include <yql/essentials/core/yql_type_annotation.h> +#include <yql/essentials/core/expr_nodes/yql_expr_nodes.h> +#include <yql/essentials/utils/log/log.h> +#include <yql/essentials/utils/yql_panic.h> + +namespace NYql { + +using namespace NNodes; + +void TNodeHashCalculator::UpdateFileHash(THashBuilder& builder, TStringBuf alias) const { + auto block = Types.UserDataStorage->FindUserDataBlock(alias); + YQL_ENSURE(block, "File " << alias << " not found"); + if (block->FrozenFile || block->Type != EUserDataType::URL) { + YQL_ENSURE(block->FrozenFile, "File " << alias << " is not frozen"); + YQL_ENSURE(block->FrozenFile->GetMd5(), "MD5 for file " << alias << " is empty"); + + builder << alias << (ui32)block->Type << block->FrozenFile->GetMd5(); + return; + } + + // temporary approach: for YT remote files we support URL hashing rather than file content hashing + // todo: rework it and use file metadata + builder << alias << (ui32)block->Type << block->Data; +} + +bool TNodeHashCalculator::UpdateChildrenHash(THashBuilder& builder, const TExprNode& node, TArgIndex& argIndex, ui32 frameLevel, size_t fromIndex) const { + for (size_t i = fromIndex; i < node.ChildrenSize(); ++i) { + auto childHash = GetHashImpl(*node.Child(i), argIndex, frameLevel); + if (childHash.empty()) { + return false; + } + + builder << childHash; + } + return true; +} + +TString TNodeHashCalculator::GetHashImpl(const TExprNode& node, TArgIndex& argIndex, ui32 frameLevel) const { + auto it = NodeHash.find(node.UniqueId()); + if (it != NodeHash.end()) { + return it->second; + } + + bool isHashable = true; + TString myHash; + ui32 typeNum = node.Type(); + THashBuilder builder; + builder << Salt << typeNum; + switch (node.Type()) { + case TExprNode::List: { + if (!UpdateChildrenHash(builder, node, argIndex, frameLevel)) { + isHashable = false; + } + break; + } + case TExprNode::Atom: { + builder << node.Content(); + break; + } + case TExprNode::Callable: { + if (auto p = Hashers.FindPtr(node.Content())) { + auto callableHash = (*p)(node, argIndex, frameLevel); + if (callableHash.empty()) { + isHashable = false; + } + else { + builder << callableHash; + } + } + else { + builder << node.Content(); + if (node.ChildrenSize() > 0 && node.Child(0)->GetTypeAnn()->GetKind() == ETypeAnnotationKind::World) { + YQL_CLOG(ERROR, ProviderYt) << "Cannot calculate hash for " << node.Content(); + isHashable = false; + } + else { + if (!UpdateChildrenHash(builder, node, argIndex, frameLevel)) { + isHashable = false; + } + else { + if (TCoUdf::Match(&node) && node.ChildrenSize() > TCoUdf::idx_FileAlias && !node.Child(TCoUdf::idx_FileAlias)->Content().empty()) { + // an udf from imported file, use hash of file + auto alias = node.Child(TCoUdf::idx_FileAlias)->Content(); + UpdateFileHash(builder, alias); + } else if (node.Content() == "FilePath" || node.Content() == "FileContent") { + auto alias = node.Child(0)->Content(); + UpdateFileHash(builder, alias); + } else if (node.Content() == "FolderPath") { + auto alias = node.Child(0)->Content(); + auto blocks = Types.UserDataStorage->FindUserDataFolder(alias); + YQL_ENSURE(blocks, "Folder" << alias << " not found"); + // keys for blocks must be ordered (not a hashmap) + for (const auto& x : *blocks) { + UpdateFileHash(builder, x.first.Alias()); + } + } + } + } + } + break; + } + case TExprNode::Lambda: { + ui32 pos = 0; + for (const auto& arg : node.Child(0)->Children()) { + // argument is described by it's frame level (starting from 1) and position + YQL_ENSURE(argIndex.insert(std::make_pair(arg.Get(), std::make_pair(frameLevel + 1, pos++ ))).second); + } + + if (!UpdateChildrenHash(builder, node, argIndex, frameLevel + 1, 1)) { + isHashable = false; + } + break; + } + + case TExprNode::Argument: { + auto it = argIndex.find(&node); + YQL_ENSURE(it != argIndex.end()); + builder << (frameLevel - it->second.first) << it->second.second; + break; + } + + case TExprNode::World: + isHashable = false; + break; + default: + YQL_ENSURE(false, "unexpected"); + } + + if (isHashable) { + myHash = builder.Finish(); + } + + NodeHash.emplace(node.UniqueId(), myHash); + return myHash; +} + +} // NYql diff --git a/yt/yql/providers/yt/lib/hash/yql_op_hash.h b/yt/yql/providers/yt/lib/hash/yql_op_hash.h new file mode 100644 index 0000000000..cf2314ac3e --- /dev/null +++ b/yt/yql/providers/yt/lib/hash/yql_op_hash.h @@ -0,0 +1,47 @@ +#pragma once + +#include <yql/essentials/ast/yql_expr.h> + +#include <util/generic/strbuf.h> +#include <util/generic/string.h> +#include <util/generic/hash.h> + +#include <utility> +#include <functional> + +namespace NYql { + +struct TTypeAnnotationContext; +class THashBuilder; + +class TNodeHashCalculator { +public: + using TArgIndex = THashMap<const TExprNode*, std::pair<ui32, ui32>>; + using THasher = std::function<TString(const TExprNode& node, TArgIndex& argIndex, ui32 frameLevel)>; + using THasherMap = THashMap<TStringBuf, THasher>; + + TNodeHashCalculator(const TTypeAnnotationContext& types, std::unordered_map<ui64, TString>& nodeHash, const TString& salt) + : Types(types) + , NodeHash(nodeHash) + , Salt(salt) + { + } + + TString GetHash(const TExprNode& node) const { + TArgIndex argIndex; + return GetHashImpl(node, argIndex, 0); + } + +protected: + void UpdateFileHash(THashBuilder& builder, TStringBuf alias) const; + bool UpdateChildrenHash(THashBuilder& builder, const TExprNode& node, TArgIndex& argIndex, ui32 frameLevel, size_t fromIndex = 0) const; + TString GetHashImpl(const TExprNode& node, TArgIndex& argIndex, ui32 frameLevel) const; + +protected: + const TTypeAnnotationContext& Types; + std::unordered_map<ui64, TString>& NodeHash; + THasherMap Hashers; + TString Salt; +}; + +} diff --git a/yt/yql/providers/yt/lib/infer_schema/infer_schema.cpp b/yt/yql/providers/yt/lib/infer_schema/infer_schema.cpp new file mode 100644 index 0000000000..d80e23883c --- /dev/null +++ b/yt/yql/providers/yt/lib/infer_schema/infer_schema.cpp @@ -0,0 +1,112 @@ +#include "infer_schema.h" + +#include <yql/essentials/core/issue/yql_issue.h> +#include <yql/essentials/public/issue/yql_issue.h> +#include <yql/essentials/utils/log/log.h> + +#include <util/generic/yexception.h> + + +namespace NYql { + +TStreamSchemaInferer::TStreamSchemaInferer(const TString& tableName) + : TableName(tableName) + , UsedRows(0) +{ +} + +void TStreamSchemaInferer::AddRow(const NYT::TNode& row) +{ + for (const auto& column : row.AsMap()) { + if (column.first.StartsWith("$")) { + // Skip system columns + continue; + } + TString columnType; + switch (column.second.GetType()) { + case NYT::TNode::String: + columnType = "string"; + break; + case NYT::TNode::Int64: + columnType = "int64"; + break; + case NYT::TNode::Uint64: + columnType = "uint64"; + break; + case NYT::TNode::Double: + columnType = "double"; + break; + case NYT::TNode::Bool: + columnType = "boolean"; + break; + case NYT::TNode::List: + columnType = "any"; + break; + case NYT::TNode::Map: + columnType = "any"; + break; + case NYT::TNode::Null: + continue; + default: + YQL_LOG_CTX_THROW yexception() << + "Cannot infer schema for table " << TableName << + ", undefined NYT::TNode"; + } + + auto& type = ColumnTypes[column.first]; + if (type.empty()) { + type = columnType; + } else if (type != columnType) { + type = "any"; + } + } + ++UsedRows; +} + +NYT::TNode TStreamSchemaInferer::GetSchema() const +{ + NYT::TNode schema = NYT::TNode::CreateList(); + for (auto& x : ColumnTypes) { + auto& columnNode = schema.Add(); + columnNode["name"] = x.first; + columnNode["type"] = x.second; + } + + if (schema.Empty()) { + YQL_LOG_CTX_THROW TErrorException(TIssuesIds::YT_INFER_SCHEMA) << + "Cannot infer schema for table " << TableName << + ", first " << UsedRows << " row(s) has no columns"; + } + schema.Attributes()["strict"] = false; + return schema; +} + +NYT::TNode InferSchemaFromSample(const NYT::TNode& sample, const TString& tableName, ui32 rows) +{ + TStreamSchemaInferer inferer(tableName); + const size_t useRows = Min<size_t>(rows, sample.AsList().size()); + + for (size_t i = 0; i < useRows; ++i) { + inferer.AddRow(sample.AsList()[i]); + } + + return inferer.GetSchema(); +} + +TMaybe<NYT::TNode> InferSchemaFromTableContents(const NYT::ITransactionPtr& tx, const TString& tableId, const TString& tableName, ui32 rows) +{ + auto reader = tx->CreateTableReader<NYT::TNode>(NYT::TRichYPath(tableId).AddRange(NYT::TReadRange::FromRowIndices(0, rows))); + if (!reader->IsValid()) { + return Nothing(); + } + + TStreamSchemaInferer inferer(tableName); + for (ui32 i = 0; reader->IsValid() && i < rows; ++i) { + inferer.AddRow(reader->GetRow()); + reader->Next(); + } + + return inferer.GetSchema(); +} + +} // NYql diff --git a/yt/yql/providers/yt/lib/infer_schema/infer_schema.h b/yt/yql/providers/yt/lib/infer_schema/infer_schema.h new file mode 100644 index 0000000000..3e467cee20 --- /dev/null +++ b/yt/yql/providers/yt/lib/infer_schema/infer_schema.h @@ -0,0 +1,33 @@ +#pragma once + +#include <yt/cpp/mapreduce/interface/client.h> +#include <yt/cpp/mapreduce/interface/common.h> + +#include <library/cpp/yson/node/node.h> + +#include <util/generic/maybe.h> +#include <util/generic/string.h> +#include <util/generic/map.h> + +#include <functional> +#include <utility> + +namespace NYql { + +class TStreamSchemaInferer +{ +public: + explicit TStreamSchemaInferer(const TString& tableName); + void AddRow(const NYT::TNode& row); + NYT::TNode GetSchema() const; + +private: + const TString TableName; + TMap<TString, TString> ColumnTypes; + size_t UsedRows; +}; + +NYT::TNode InferSchemaFromSample(const NYT::TNode& sample, const TString& tableName, ui32 rows); +TMaybe<NYT::TNode> InferSchemaFromTableContents(const NYT::ITransactionPtr& tx, const TString& tableId, const TString& tableName, ui32 rows); + +} // NYql diff --git a/yt/yql/providers/yt/lib/infer_schema/infer_schema_rpc.cpp b/yt/yql/providers/yt/lib/infer_schema/infer_schema_rpc.cpp new file mode 100644 index 0000000000..1fef2482e7 --- /dev/null +++ b/yt/yql/providers/yt/lib/infer_schema/infer_schema_rpc.cpp @@ -0,0 +1,142 @@ +#include "infer_schema_rpc.h" + +#include "yt/cpp/mapreduce/common/helpers.h" + +#include <yt/yt/library/auth/auth.h> + +#include <yt/yt/client/api/client.h> + +#include <yt/yt/client/api/rpc_proxy/client_impl.h> +#include <yt/yt/client/api/rpc_proxy/config.h> +#include <yt/yt/client/api/rpc_proxy/connection.h> +#include <yt/yt/client/api/rpc_proxy/row_stream.h> + +#include <yql/essentials/utils/yql_panic.h> +#include <yql/essentials/utils/log/log.h> + +#include <yt/yql/providers/yt/lib/yt_rpc_helpers/yt_convert_helpers.h> + +#include <library/cpp/yson/parser.h> + +namespace NYql { +void OnPayload(const NYT::TSharedRef& block, size_t i, std::vector<TStreamSchemaInferer>& inferers, std::vector<NYT::TPromise<void>>& promises) { + NYT::NApi::NRpcProxy::NProto::TRowsetDescriptor descriptor; + NYT::NApi::NRpcProxy::NProto::TRowsetStatistics statistics; + auto currentPayload = std::move(NYT::NApi::NRpcProxy::DeserializeRowStreamBlockEnvelope(block, &descriptor, &statistics)); + if (descriptor.rowset_format() != NYT::NApi::NRpcProxy::NProto::RF_FORMAT) { + return; + } + if (currentPayload.empty()) { + return; + } + try { + NYT::TNode res; + NYT::TNodeBuilder builder(&res); + TMemoryInput mem(currentPayload.begin(), currentPayload.Size()); + NYson::TYsonListParser parser(&builder, &mem); + while (parser.Parse()) { + auto& lst = res.AsList(); + for (size_t j = 0; j < lst.size(); ++j) { + inferers[i].AddRow(lst[j]); + } + res.Clear(); + } + } catch (std::exception& e) { + promises[i].Set(e); + } +} + +TVector<TMaybe<NYT::TNode>> InferSchemaFromTablesContents(const TString& cluster, const TString& token, const NYT::TTransactionId& tx, const std::vector<TTableInferSchemaRequest>& requests, TAsyncQueue::TPtr queue) { + const ui32 Timeout = 300'000; + auto connectionConfig = NYT::New<NYT::NApi::NRpcProxy::TConnectionConfig>(); + connectionConfig->ClusterUrl = cluster; + connectionConfig->DefaultTotalStreamingTimeout = TDuration::MilliSeconds(Timeout); + connectionConfig->EnableRetries = true; + connectionConfig->DefaultPingPeriod = TDuration::MilliSeconds(5000); + + auto connection = CreateConnection(connectionConfig); + auto clientOptions = NYT::NApi::TClientOptions(); + + if (!token.empty()) { + clientOptions.Token = token; + } + + auto client = DynamicPointerCast<NYT::NApi::NRpcProxy::TClient>(connection->CreateClient(clientOptions)); + Y_ABORT_UNLESS(client); + auto apiServiceProxy = client->CreateApiServiceProxy(); + + TVector<NYT::NConcurrency::IAsyncZeroCopyInputStreamPtr> inputs(requests.size()); + size_t i = 0; + std::vector<NYT::TFuture<void>> futures; + std::vector<NYT::TPromise<void>> promises; + std::vector<TStreamSchemaInferer> inferers; + inferers.reserve(requests.size()); + + std::function<void(size_t)> runRead = [&](size_t i) { + YT_UNUSED_FUTURE(inputs[i]->Read().ApplyUnique(BIND([queue, &inferers, &promises, &runRead, i = i](NYT::TErrorOr<NYT::TSharedRef>&& res){ + if (res.IsOK() && !res.Value()) { + // EOS + promises[i].Set(); + return; + } + if (!res.IsOK()) { + promises[i].Set(res); + return; + } + Y_UNUSED(queue->Async([&inferers, &promises, &runRead, i = i, block = std::move(res.Value())] { + OnPayload(block, i, inferers, promises); + runRead(i); + })); + }))); + }; + + futures.reserve(requests.size()); + promises.reserve(requests.size()); + + YQL_CLOG(TRACE, ProviderYt) << "Infering started"; + for (auto& req: requests) { + inferers.emplace_back(req.TableName); + + auto request = apiServiceProxy.ReadTable(); + client->InitStreamingRequest(*request); + request->ClientAttachmentsStreamingParameters().ReadTimeout = TDuration::MilliSeconds(Timeout); + request->ClientAttachmentsStreamingParameters().WriteTimeout = TDuration::MilliSeconds(Timeout); + + TString ppath; + NYT::NYPath::TRichYPath tableYPath(req.TableId); + NYT::NChunkClient::TReadRange range; + + range.LowerLimit().SetRowIndex(0); + range.UpperLimit().SetRowIndex(req.Rows); + tableYPath.SetRanges({range}); + + NYT::NYPath::ToProto(&ppath, tableYPath); + request->set_path(ppath); + request->set_desired_rowset_format(NYT::NApi::NRpcProxy::NProto::ERowsetFormat::RF_FORMAT); + + request->set_unordered(true); + NDqs::ConfigureTransaction(request, tx.dw); + + // Get skiff format yson string + request->set_format("<format=binary>yson"); + promises.push_back(NYT::NewPromise<void>()); + futures.push_back(promises.back().ToFuture()); + YT_UNUSED_FUTURE(CreateRpcClientInputStream(std::move(request)).ApplyUnique(BIND([&runRead, &inputs, i](NYT::NConcurrency::IAsyncZeroCopyInputStreamPtr&& stream) { + // first packet contains meta, skip it + return stream->Read().ApplyUnique(BIND([&runRead, stream = std::move(stream), i, &inputs](NYT::TSharedRef&&) { + inputs[i] = std::move(stream); + runRead(i); + })); + }))); + ++i; + } + YQL_CLOG(TRACE, ProviderYt) << "Futures prepared"; + auto success = NYT::NConcurrency::WaitFor(AllSucceeded(futures)); + YQL_CLOG(TRACE, ProviderYt) << "Infered"; + success.ThrowOnError(); + TVector<TMaybe<NYT::TNode>> result; + result.reserve(requests.size()); + std::transform(inferers.begin(), inferers.end(), std::back_inserter(result), [](auto& x) { return x.GetSchema();}); + return result; +} +} diff --git a/yt/yql/providers/yt/lib/infer_schema/infer_schema_rpc.h b/yt/yql/providers/yt/lib/infer_schema/infer_schema_rpc.h new file mode 100644 index 0000000000..f99c4ca977 --- /dev/null +++ b/yt/yql/providers/yt/lib/infer_schema/infer_schema_rpc.h @@ -0,0 +1,13 @@ +#pragma once +#include "infer_schema.h" + +#include <yql/essentials/utils/threading/async_queue.h> + +namespace NYql { +struct TTableInferSchemaRequest { + const TString& TableId; + const TString& TableName; + ui32 Rows; +}; +TVector<TMaybe<NYT::TNode>> InferSchemaFromTablesContents(const TString& cluster, const TString& token, const NYT::TTransactionId& tx, const std::vector<TTableInferSchemaRequest>& requests, TAsyncQueue::TPtr queue); +}
\ No newline at end of file diff --git a/yt/yql/providers/yt/lib/infer_schema/infer_schema_rpc_impl.cpp b/yt/yql/providers/yt/lib/infer_schema/infer_schema_rpc_impl.cpp new file mode 100644 index 0000000000..2ff33dad8b --- /dev/null +++ b/yt/yql/providers/yt/lib/infer_schema/infer_schema_rpc_impl.cpp @@ -0,0 +1,9 @@ +#include "infer_schema_rpc.h" + +#include <yql/essentials/utils/yql_panic.h> + +namespace NYql { +TVector<TMaybe<NYT::TNode>> InferSchemaFromTablesContents(const TString&, const TString&, const NYT::TTransactionId&, const std::vector<TTableInferSchemaRequest>&) { + YQL_ENSURE(false, "Not implemented"); +} +} diff --git a/yt/yql/providers/yt/lib/infer_schema/ya.make b/yt/yql/providers/yt/lib/infer_schema/ya.make new file mode 100644 index 0000000000..3cb5c0f520 --- /dev/null +++ b/yt/yql/providers/yt/lib/infer_schema/ya.make @@ -0,0 +1,30 @@ +LIBRARY() + +SRCS( + infer_schema.cpp +) + +PEERDIR( + library/cpp/yson/node + yt/cpp/mapreduce/interface + yql/essentials/public/issue + yql/essentials/utils/log + yql/essentials/core/issue +) + +IF(LINUX) + PEERDIR( + yt/yt/client + yt/yql/providers/yt/lib/yt_rpc_helpers + ) + + SRCS( + infer_schema_rpc.cpp + ) +ELSE() + SRCS( + infer_schema_rpc_impl.cpp + ) +ENDIF() + +END() diff --git a/yt/yql/providers/yt/lib/init_yt_api/init.cpp b/yt/yql/providers/yt/lib/init_yt_api/init.cpp new file mode 100644 index 0000000000..c6bb9df4c0 --- /dev/null +++ b/yt/yql/providers/yt/lib/init_yt_api/init.cpp @@ -0,0 +1,97 @@ +#include "init.h" + +#include <yql/essentials/utils/log/log.h> + +#include <yt/cpp/mapreduce/interface/config.h> +#include <yt/cpp/mapreduce/client/init.h> + +#include <library/cpp/yson/node/node.h> +#include <library/cpp/yson/node/node_io.h> + +#include <util/generic/singleton.h> +#include <util/system/compiler.h> +#include <util/system/env.h> + +namespace NYql { + +namespace { + +class TInitYtApi { +public: + TInitYtApi() + : DeterministicMode_(GetEnv("YQL_DETERMINISTIC_MODE")) + { + YQL_CLOG(DEBUG, ProviderYt) << "Entering YT API init"; + if (NYT::NDetail::GetInitStatus() == NYT::NDetail::EInitStatus::NotInitialized) { + YQL_CLOG(DEBUG, ProviderYt) << "Init jobless YT API"; + NYT::JoblessInitialize(); + } + NYT::TConfig::Get()->RetryCount = 1 << 6; + NYT::TConfig::Get()->StartOperationRetryCount = 1 << 10; + if (GetEnv("YT_FORCE_IPV6").empty()) { + NYT::TConfig::Get()->ForceIpV6 = true; + } + if (NYT::TConfig::Get()->Prefix.empty()) { + NYT::TConfig::Get()->Prefix = "//"; + } + NYT::TConfig::Get()->AsyncHttpClientThreads = 2; + NYT::TConfig::Get()->AsyncTxPingerPoolThreads = 2; + YQL_CLOG(DEBUG, ProviderYt) << "Using YT global prefix: " << NYT::TConfig::Get()->Prefix; + } + + void UpdateProps(const TMaybe<TString>& attrs) { + if (!attrs) { + return; + } + NYT::TNode node; + try { + node = NYT::NodeFromYsonString(*attrs); + node.AsMap(); + } catch (...) { + YQL_CLOG(ERROR, ProviderYt) << "Cannot parse yql operation attrs: " << CurrentExceptionMessage(); + return; + } + if (DeterministicMode_) { + TString prefix; + if (auto p = node.AsMap().FindPtr("test_prefix")) { + prefix = p->AsString(); + } + if (prefix) { + if (!prefix.StartsWith("//")) { + prefix.prepend("//"); + } + if (!prefix.EndsWith('/')) { + prefix.append('/'); + } + if (NYT::TConfig::Get()->Prefix != prefix) { + NYT::TConfig::Get()->Prefix = prefix; + YQL_CLOG(DEBUG, ProviderYt) << "Using YT global prefix: " << prefix; + } + } + } + if (auto p = node.AsMap().FindPtr("poller_interval")) { + try { + if (auto d = TDuration::Parse(p->AsString()); d >= TDuration::MilliSeconds(200) && d <= TDuration::Seconds(10)) { + NYT::TConfig::Get()->WaitLockPollInterval = d; + YQL_CLOG(DEBUG, ProviderYt) << "Set poller_interval to " << d; + } else { + YQL_CLOG(ERROR, ProviderYt) << "Invalid poller_interval value " << p->AsString(); + throw yexception() << "Invalid poller_interval value " << p->AsString(); + } + } catch (...) { + YQL_CLOG(ERROR, ProviderYt) << "Cannot parse poller_interval: " << CurrentExceptionMessage(); + throw; + } + } + } +private: + const bool DeterministicMode_; +}; + +} + +void InitYtApiOnce(const TMaybe<TString>& attrs) { + Singleton<TInitYtApi>()->UpdateProps(attrs); +} + +} // NYql diff --git a/yt/yql/providers/yt/lib/init_yt_api/init.h b/yt/yql/providers/yt/lib/init_yt_api/init.h new file mode 100644 index 0000000000..1925b314ba --- /dev/null +++ b/yt/yql/providers/yt/lib/init_yt_api/init.h @@ -0,0 +1,10 @@ +#pragma once + +#include <util/generic/string.h> +#include <util/generic/maybe.h> + +namespace NYql { + +void InitYtApiOnce(const TMaybe<TString>& attrs = Nothing()); + +} // NYql diff --git a/yt/yql/providers/yt/lib/init_yt_api/ya.make b/yt/yql/providers/yt/lib/init_yt_api/ya.make new file mode 100644 index 0000000000..8cb935deee --- /dev/null +++ b/yt/yql/providers/yt/lib/init_yt_api/ya.make @@ -0,0 +1,14 @@ +LIBRARY() + +SRCS( + init.cpp +) + +PEERDIR( + yql/essentials/utils/log + yt/cpp/mapreduce/common + yt/cpp/mapreduce/client + library/cpp/yson/node +) + +END() diff --git a/yt/yql/providers/yt/lib/key_filter/ya.make b/yt/yql/providers/yt/lib/key_filter/ya.make new file mode 100644 index 0000000000..515f67e269 --- /dev/null +++ b/yt/yql/providers/yt/lib/key_filter/ya.make @@ -0,0 +1,16 @@ +LIBRARY() + +SRCS( + yql_key_filter.cpp +) + +PEERDIR( + yql/essentials/core/expr_nodes + yql/essentials/core + yql/essentials/utils + yql/essentials/ast +) + +YQL_LAST_ABI_VERSION() + +END() diff --git a/yt/yql/providers/yt/lib/key_filter/yql_key_filter.cpp b/yt/yql/providers/yt/lib/key_filter/yql_key_filter.cpp new file mode 100644 index 0000000000..25924596c0 --- /dev/null +++ b/yt/yql/providers/yt/lib/key_filter/yql_key_filter.cpp @@ -0,0 +1,177 @@ +#include "yql_key_filter.h" + +#include <yql/essentials/core/expr_nodes/yql_expr_nodes.h> +#include <yql/essentials/core/yql_expr_type_annotation.h> +#include <yql/essentials/utils/yql_panic.h> + +#include <util/string/builder.h> + +namespace NYql { + +using namespace NNodes; + +void TSortMembersCollection::AddTableInfo(size_t tableIndex, const TString& tableName, + const TVector<TString>& sortMembers, + const TTypeAnnotationNode::TListType& sortedByTypes, + const TVector<bool>& sortDirections) +{ + THashMap<TString, TMemberDescrPtr>* nextMembers = &Members; + for (size_t i = 0; i < sortMembers.size(); ++i) { + if (!sortDirections.empty() && !sortDirections.at(i)) { + break; + } + + TMemberDescrPtr member; + if (auto p = nextMembers->FindPtr(sortMembers[i])) { + member = *p; + } + else { + member = MakeIntrusive<TMemberDescr>(); + YQL_ENSURE(IsDataOrOptionalOfData(sortedByTypes[i], member->IsColumnOptional, member->ColumnType), + "Table " << tableName.Quote() << ", field " << sortMembers[i].Quote() + << " has incompatible type " << *sortedByTypes[i]); + nextMembers->emplace(sortMembers[i], member); + } + + member->Tables.insert(tableIndex); + nextMembers = &member->NextMembers; + } +} + +bool TSortMembersCollection::ApplyRanges(const TKeyFilterPredicates& ranges, TExprContext& ctx) { + if (!ApplyRangesImpl(0, ranges, ctx)) { + return false; + } + + DropMembersWithoutRanges(Members); + + // Keep table indexes only in leafs + ApplyRecurs(Members, [&] (const TString& /*name*/, TMemberDescr& member) -> bool { + if (!member.NextMembers.empty()) { + for (auto& item: member.NextMembers) { + for (auto t: item.second->Tables) { + member.Tables.erase(t); + } + } + return true; + } + return false; + }); + + return true; +} + +bool TSortMembersCollection::ApplyRanges(const TVector<TKeyFilterPredicates>& ranges, TExprContext& ctx) { + for (size_t i = 0; i < ranges.size(); ++i) { + if (!ApplyRangesImpl(i, ranges[i], ctx)) { + return false; + } + } + + // Ensure that all OR clauses have at least first key + for (auto& item: Members) { + for (size_t i = 0; i < ranges.size(); ++i) { + if (!item.second->Ranges.contains(i)) { + item.second->Ranges.clear(); + break; + } + } + } + + DropMembersWithoutRanges(Members); + + // Keep table indexes only in leafs + ApplyRecurs(Members, [&] (const TString& /*name*/, TMemberDescr& member) -> bool { + if (!member.NextMembers.empty()) { + for (auto& item: member.NextMembers) { + for (auto t: item.second->Tables) { + member.Tables.erase(t); + } + } + return true; + } + return false; + }); + + return true; +} + +bool TSortMembersCollection::ApplyRangesImpl(size_t groupIndex, const TKeyFilterPredicates& ranges, TExprContext& ctx) { + bool hasErrors = false; + ApplyRecurs(Members, [&] (const TString& name, TMemberDescr& member) -> bool { + if (hasErrors) { + return false; + } + + auto memberRanges = ranges.equal_range(name); + if (memberRanges.first == memberRanges.second) { + return false; + } + + // ensure that we can compare right part of every range with current column + bool foundEqualComparisons = false; + for (auto it = memberRanges.first; it != memberRanges.second; ++it) { + TString cmpOp; + TExprNode::TPtr value; + std::tie(cmpOp, value) = it->second; + const TDataExprType* dataType = nullptr; + + if (value) { + bool isOptional = false; + if (!EnsureDataOrOptionalOfData(*value, isOptional, dataType, ctx)) { + hasErrors = true; + return false; + } + + if (!GetSuperType(dataType->GetSlot(), member.ColumnType->GetSlot())) { + ctx.AddError(TIssue(ctx.GetPosition(value->Pos()), + TStringBuilder() << "Column " << name.Quote() + << " of " << *static_cast<const TTypeAnnotationNode*>(member.ColumnType) + << " type cannot be compared with " << *value->GetTypeAnn())); + hasErrors = true; + return false; + } + } + else { + // Not(Exists(member)) case + if (!member.IsColumnOptional) { + return false; + } + } + + member.Ranges.emplace(groupIndex, std::make_tuple(cmpOp, value, dataType)); + if (cmpOp == TCoCmpEqual::CallableName()) { + foundEqualComparisons = true; + } + } + // ensure that ranges contain at least one '==' operation to continue with next member of the compound key + return foundEqualComparisons; + }); + + return !hasErrors; +} + +void TSortMembersCollection::DropMembersWithoutRanges(TMemberDescrMap& members) { + TVector<TString> toDrop; + for (auto& item: members) { + if (item.second->Ranges.empty()) { + toDrop.push_back(item.first); + } + else { + DropMembersWithoutRanges(item.second->NextMembers); + } + } + for (auto& name: toDrop) { + members.erase(name); + } +} + +void TSortMembersCollection::ApplyRecurs(TMemberDescrMap& members, const std::function<bool(const TString&, TMemberDescr&)>& func) { + for (auto& item: members) { + if (func(item.first, *item.second)) { + ApplyRecurs(item.second->NextMembers, func); + } + } +} + +} // NYql diff --git a/yt/yql/providers/yt/lib/key_filter/yql_key_filter.h b/yt/yql/providers/yt/lib/key_filter/yql_key_filter.h new file mode 100644 index 0000000000..1d605c6714 --- /dev/null +++ b/yt/yql/providers/yt/lib/key_filter/yql_key_filter.h @@ -0,0 +1,61 @@ +#pragma once + +#include <yql/essentials/ast/yql_expr.h> + +#include <util/generic/string.h> +#include <util/generic/vector.h> +#include <util/generic/hash.h> +#include <util/generic/hash_multi_map.h> +#include <util/generic/ptr.h> +#include <util/generic/set.h> +#include <util/generic/map.h> +#include <util/generic/strbuf.h> + +#include <tuple> +#include <utility> +#include <functional> + +namespace NYql { + +using TKeyFilterPredicates = THashMultiMap<TString, std::pair<TString, TExprNode::TPtr>>; + +class TSortMembersCollection { +public: + struct TMemberDescr; + using TMemberDescrPtr = TIntrusivePtr<TMemberDescr>; + using TMemberDescrMap = THashMap<TString, TMemberDescrPtr>; + + struct TMemberDescr: public TThrRefBase { + const TDataExprType* ColumnType = nullptr; + bool IsColumnOptional = false; + TMultiMap<size_t, std::tuple<TString, TExprNode::TPtr, const TDataExprType*>> Ranges; + TSet<size_t> Tables; + TMemberDescrMap NextMembers; + }; + + TSortMembersCollection() = default; + ~TSortMembersCollection() = default; + + bool Empty() const { + return Members.empty(); + } + + void AddTableInfo(size_t tableIndex, const TString& tableName, + const TVector<TString>& sortMembers, + const TTypeAnnotationNode::TListType& sortedByTypes, + const TVector<bool>& sortDirections); + + bool ApplyRanges(const TKeyFilterPredicates& ranges, TExprContext& ctx); + bool ApplyRanges(const TVector<TKeyFilterPredicates>& ranges, TExprContext& ctx); + +protected: + bool ApplyRangesImpl(size_t groupIndex, const TKeyFilterPredicates& ranges, TExprContext& ctx); + static void DropMembersWithoutRanges(TMemberDescrMap& members); + static void ApplyRecurs(TMemberDescrMap& members, + const std::function<bool(const TString&, TMemberDescr&)>& func); + +protected: + TMemberDescrMap Members; +}; + +} // NYql diff --git a/yt/yql/providers/yt/lib/lambda_builder/lambda_builder.cpp b/yt/yql/providers/yt/lib/lambda_builder/lambda_builder.cpp new file mode 100644 index 0000000000..dd6a8bbb20 --- /dev/null +++ b/yt/yql/providers/yt/lib/lambda_builder/lambda_builder.cpp @@ -0,0 +1,230 @@ +#include "lambda_builder.h" + +#include <yql/essentials/utils/yql_panic.h> +#include <yql/essentials/minikql/mkql_opt_literal.h> +#include <yql/essentials/minikql/mkql_program_builder.h> +#include <yql/essentials/minikql/mkql_node_serialization.h> +#include <yql/essentials/minikql/comp_nodes/mkql_factories.h> + +#include <util/generic/strbuf.h> +#include <util/system/env.h> + +namespace NYql { + +using namespace NCommon; +using namespace NKikimr; +using namespace NKikimr::NMiniKQL; + +TLambdaBuilder::TLambdaBuilder(const NKikimr::NMiniKQL::IFunctionRegistry* functionRegistry, + NKikimr::NMiniKQL::TScopedAlloc& alloc, + const NKikimr::NMiniKQL::TTypeEnvironment* env, + const TIntrusivePtr<IRandomProvider>& randomProvider, + const TIntrusivePtr<ITimeProvider>& timeProvider, + NKikimr::NMiniKQL::IStatsRegistry* jobStats, + NKikimr::NUdf::ICountersProvider* counters, + const NKikimr::NUdf::ISecureParamsProvider* secureParamsProvider) + : FunctionRegistry(functionRegistry) + , Alloc(alloc) + , RandomProvider(randomProvider) + , TimeProvider(timeProvider) + , JobStats(jobStats) + , Counters(counters) + , SecureParamsProvider(secureParamsProvider) + , Env(env) +{ +} + +TLambdaBuilder::~TLambdaBuilder() { +} + +void TLambdaBuilder::SetExternalEnv(const NKikimr::NMiniKQL::TTypeEnvironment* env) { + Env = env; +} + +const NKikimr::NMiniKQL::TTypeEnvironment* TLambdaBuilder::CreateTypeEnv() const { + YQL_ENSURE(!EnvPtr); + EnvPtr = std::make_shared<NKikimr::NMiniKQL::TTypeEnvironment>(Alloc); + return EnvPtr.get(); +} + +TRuntimeNode TLambdaBuilder::BuildLambda(const IMkqlCallableCompiler& compiler, const TExprNode::TPtr& lambdaNode, + TExprContext& exprCtx, TArgumentsMap&& arguments) const +{ + TProgramBuilder pgmBuilder(GetTypeEnvironment(), *FunctionRegistry); + TMkqlBuildContext ctx(compiler, pgmBuilder, exprCtx, lambdaNode->UniqueId(), std::move(arguments)); + return MkqlBuildExpr(*lambdaNode, ctx); +} + +TRuntimeNode TLambdaBuilder::TransformAndOptimizeProgram(NKikimr::NMiniKQL::TRuntimeNode root, + TCallableVisitFuncProvider funcProvider) { + TExploringNodeVisitor explorer; + explorer.Walk(root.GetNode(), GetTypeEnvironment()); + bool wereChanges = false; + TRuntimeNode program = SinglePassVisitCallables(root, explorer, funcProvider, GetTypeEnvironment(), true, wereChanges); + program = LiteralPropagationOptimization(program, GetTypeEnvironment(), true); + return program; +} + +THolder<IComputationGraph> TLambdaBuilder::BuildGraph( + const NKikimr::NMiniKQL::TComputationNodeFactory& factory, + NUdf::EValidateMode validateMode, + NUdf::EValidatePolicy validatePolicy, + const TString& optLLVM, + NKikimr::NMiniKQL::EGraphPerProcess graphPerProcess, + TExploringNodeVisitor& explorer, + TRuntimeNode root) const +{ + return BuildGraph(factory, validateMode, validatePolicy, optLLVM, graphPerProcess, explorer, root, {root.GetNode()}); +} + +std::tuple< + THolder<NKikimr::NMiniKQL::IComputationGraph>, + TIntrusivePtr<IRandomProvider>, + TIntrusivePtr<ITimeProvider> +> +TLambdaBuilder::BuildLocalGraph( + const NKikimr::NMiniKQL::TComputationNodeFactory& factory, + NUdf::EValidateMode validateMode, + NUdf::EValidatePolicy validatePolicy, + const TString& optLLVM, + NKikimr::NMiniKQL::EGraphPerProcess graphPerProcess, + TExploringNodeVisitor& explorer, + TRuntimeNode root) const +{ + auto randomProvider = RandomProvider; + auto timeProvider = TimeProvider; + if (GetEnv(TString("YQL_DETERMINISTIC_MODE"))) { + randomProvider = CreateDeterministicRandomProvider(1); + timeProvider = CreateDeterministicTimeProvider(10000000); + } + + return std::make_tuple(BuildGraph(factory, validateMode, validatePolicy, optLLVM, graphPerProcess, explorer, root, {root.GetNode()}, + randomProvider, timeProvider), randomProvider, timeProvider); +} + +class TComputationGraphProxy: public IComputationGraph { +public: + TComputationGraphProxy(IComputationPattern::TPtr&& pattern, THolder<IComputationGraph>&& graph) + : Pattern(std::move(pattern)) + , Graph(std::move(graph)) + {} + + void Prepare() final { + Graph->Prepare(); + } + NUdf::TUnboxedValue GetValue() final { + return Graph->GetValue(); + } + TComputationContext& GetContext() final { + return Graph->GetContext(); + } + IComputationExternalNode* GetEntryPoint(size_t index, bool require) override { + return Graph->GetEntryPoint(index, require); + } + const TArrowKernelsTopology* GetKernelsTopology() override { + return Graph->GetKernelsTopology(); + } + const TComputationNodePtrDeque& GetNodes() const final { + return Graph->GetNodes(); + } + void Invalidate() final { + return Graph->Invalidate(); + } + TMemoryUsageInfo& GetMemInfo() const final { + return Graph->GetMemInfo(); + } + const THolderFactory& GetHolderFactory() const final { + return Graph->GetHolderFactory(); + } + ITerminator* GetTerminator() const final { + return Graph->GetTerminator(); + } + bool SetExecuteLLVM(bool value) final { + return Graph->SetExecuteLLVM(value); + } + TString SaveGraphState() final { + return Graph->SaveGraphState(); + } + void LoadGraphState(TStringBuf state) final { + Graph->LoadGraphState(state); + } +private: + IComputationPattern::TPtr Pattern; + THolder<IComputationGraph> Graph; +}; + + +THolder<IComputationGraph> TLambdaBuilder::BuildGraph( + const NKikimr::NMiniKQL::TComputationNodeFactory& factory, + NUdf::EValidateMode validateMode, + NUdf::EValidatePolicy validatePolicy, + const TString& optLLVM, + NKikimr::NMiniKQL::EGraphPerProcess graphPerProcess, + TExploringNodeVisitor& explorer, + TRuntimeNode& root, + std::vector<NKikimr::NMiniKQL::TNode*>&& entryPoints, + TIntrusivePtr<IRandomProvider> randomProvider, + TIntrusivePtr<ITimeProvider> timeProvider) const +{ + if (!randomProvider) { + randomProvider = RandomProvider; + } + + if (!timeProvider) { + timeProvider = TimeProvider; + } + + TString serialized; + + TComputationPatternOpts patternOpts(Alloc.Ref(), GetTypeEnvironment()); + patternOpts.SetOptions(factory, FunctionRegistry, validateMode, validatePolicy, optLLVM, graphPerProcess, JobStats, Counters, SecureParamsProvider); + auto preparePatternFunc = [&]() { + if (serialized) { + auto tupleRunTimeNodes = DeserializeRuntimeNode(serialized, GetTypeEnvironment()); + auto tupleNodes = static_cast<const TTupleLiteral*>(tupleRunTimeNodes.GetNode()); + root = tupleNodes->GetValue(0); + for (size_t index = 0; index < entryPoints.size(); ++index) { + entryPoints[index] = tupleNodes->GetValue(1 + index).GetNode(); + } + } + explorer.Walk(root.GetNode(), GetTypeEnvironment()); + auto pattern = MakeComputationPattern(explorer, root, entryPoints, patternOpts); + for (const auto& node : explorer.GetNodes()) { + node->SetCookie(0); + } + return pattern; + }; + + auto pattern = preparePatternFunc(); + YQL_ENSURE(pattern); + + const TComputationOptsFull computeOpts(JobStats, Alloc.Ref(), GetTypeEnvironment(), *randomProvider, *timeProvider, validatePolicy, SecureParamsProvider, Counters); + auto graph = pattern->Clone(computeOpts); + return MakeHolder<TComputationGraphProxy>(std::move(pattern), std::move(graph)); +} + +TRuntimeNode TLambdaBuilder::MakeTuple(const TVector<TRuntimeNode>& items) const { + TProgramBuilder pgmBuilder(GetTypeEnvironment(), *FunctionRegistry); + return pgmBuilder.NewTuple(items); +} + +TRuntimeNode TLambdaBuilder::Deserialize(const TString& code) { + return DeserializeRuntimeNode(code, GetTypeEnvironment()); +} + +std::pair<TString, size_t> TLambdaBuilder::Serialize(TRuntimeNode rootNode) { + TExploringNodeVisitor explorer; + explorer.Walk(rootNode.GetNode(), GetTypeEnvironment()); + TString code = SerializeRuntimeNode(explorer, rootNode, GetTypeEnvironment()); + size_t nodes = explorer.GetNodes().size(); + return std::make_pair(code, nodes); +} + +TRuntimeNode TLambdaBuilder::UpdateLambdaCode(TString& code, size_t& nodes, TCallableVisitFuncProvider funcProvider) { + TRuntimeNode rootNode = Deserialize(code); + rootNode = TransformAndOptimizeProgram(rootNode, funcProvider); + std::tie(code, nodes) = Serialize(rootNode); + return rootNode; +} + +} diff --git a/yt/yql/providers/yt/lib/lambda_builder/lambda_builder.h b/yt/yql/providers/yt/lib/lambda_builder/lambda_builder.h new file mode 100644 index 0000000000..ff0e19eea1 --- /dev/null +++ b/yt/yql/providers/yt/lib/lambda_builder/lambda_builder.h @@ -0,0 +1,119 @@ +#pragma once + +#include <yql/essentials/providers/common/mkql/yql_provider_mkql.h> + +#include <yql/essentials/ast/yql_expr.h> +#include <yql/essentials/minikql/mkql_node_visitor.h> +#include <yql/essentials/minikql/computation/mkql_computation_node.h> +#include <yql/essentials/minikql/mkql_function_registry.h> +#include <yql/essentials/minikql/mkql_stats_registry.h> +#include <yql/essentials/minikql/mkql_node.h> +#include <yql/essentials/minikql/mkql_alloc.h> +#include <yql/essentials/public/udf/udf_validate.h> + +#include <library/cpp/random_provider/random_provider.h> +#include <library/cpp/time_provider/time_provider.h> + +#include <util/generic/ptr.h> +#include <util/generic/vector.h> + +#include <tuple> + +namespace NYql { + +class TPatternCache; + +class TLambdaBuilder { +public: + using TArgumentsMap = NCommon::TMkqlBuildContext::TArgumentsMap; + + TLambdaBuilder(const NKikimr::NMiniKQL::IFunctionRegistry* functionRegistry, + NKikimr::NMiniKQL::TScopedAlloc& alloc, + const NKikimr::NMiniKQL::TTypeEnvironment* env = nullptr, + const TIntrusivePtr<IRandomProvider>& randomProvider = {}, + const TIntrusivePtr<ITimeProvider>& timeProvider = {}, + NKikimr::NMiniKQL::IStatsRegistry* jobStats = nullptr, + NKikimr::NUdf::ICountersProvider* counters = nullptr, + const NKikimr::NUdf::ISecureParamsProvider *secureParamsProvider = nullptr); + + ~TLambdaBuilder(); + + const NKikimr::NMiniKQL::TTypeEnvironment& GetTypeEnvironment() const { + if (!Env) { + Env = CreateTypeEnv(); + } + return *Env; + } + + const NKikimr::NMiniKQL::IFunctionRegistry& GetFunctionRegistry() const { + return *FunctionRegistry; + } + + NKikimr::NMiniKQL::TRuntimeNode BuildLambda( + const NCommon::IMkqlCallableCompiler& compiler, + const TExprNode::TPtr& lambdaNode, + TExprContext& exprCtx, + TArgumentsMap&& arguments = {} + ) const; + + NKikimr::NMiniKQL::TRuntimeNode TransformAndOptimizeProgram(NKikimr::NMiniKQL::TRuntimeNode root, + NKikimr::NMiniKQL::TCallableVisitFuncProvider funcProvider); + + THolder<NKikimr::NMiniKQL::IComputationGraph> BuildGraph( + const NKikimr::NMiniKQL::TComputationNodeFactory& factory, + NKikimr::NUdf::EValidateMode validateMode, + NKikimr::NUdf::EValidatePolicy validatePolicy, + const TString& optLLVM, + NKikimr::NMiniKQL::EGraphPerProcess graphPerProcess, + NKikimr::NMiniKQL::TExploringNodeVisitor& explorer, + NKikimr::NMiniKQL::TRuntimeNode root) const; + THolder<NKikimr::NMiniKQL::IComputationGraph> BuildGraph( + const NKikimr::NMiniKQL::TComputationNodeFactory& factory, + NKikimr::NUdf::EValidateMode validateMode, + NKikimr::NUdf::EValidatePolicy validatePolicy, + const TString& optLLVM, + NKikimr::NMiniKQL::EGraphPerProcess graphPerProcess, + NKikimr::NMiniKQL::TExploringNodeVisitor& explorer, + NKikimr::NMiniKQL::TRuntimeNode& root, + std::vector<NKikimr::NMiniKQL::TNode*>&& entryPoints, + TIntrusivePtr<IRandomProvider> randomProvider = {}, + TIntrusivePtr<ITimeProvider> timeProvider = {}) const; + std::tuple< + THolder<NKikimr::NMiniKQL::IComputationGraph>, + TIntrusivePtr<IRandomProvider>, + TIntrusivePtr<ITimeProvider> + > + BuildLocalGraph( + const NKikimr::NMiniKQL::TComputationNodeFactory& factory, + NKikimr::NUdf::EValidateMode validateMode, + NKikimr::NUdf::EValidatePolicy validatePolicy, + const TString& optLLVM, + NKikimr::NMiniKQL::EGraphPerProcess graphPerProcess, + NKikimr::NMiniKQL::TExploringNodeVisitor& explorer, + NKikimr::NMiniKQL::TRuntimeNode root) const; + + NKikimr::NMiniKQL::TRuntimeNode MakeTuple(const TVector<NKikimr::NMiniKQL::TRuntimeNode>& items) const; + + NKikimr::NMiniKQL::TRuntimeNode UpdateLambdaCode(TString& code, size_t& nodes, NKikimr::NMiniKQL::TCallableVisitFuncProvider funcProvider); + NKikimr::NMiniKQL::TRuntimeNode Deserialize(const TString& code); + std::pair<TString, size_t> Serialize(NKikimr::NMiniKQL::TRuntimeNode rootNode); + +protected: + const NKikimr::NMiniKQL::IFunctionRegistry* FunctionRegistry; + NKikimr::NMiniKQL::TScopedAlloc& Alloc; + const TIntrusivePtr<IRandomProvider> RandomProvider; + const TIntrusivePtr<ITimeProvider> TimeProvider; + NKikimr::NMiniKQL::IStatsRegistry* const JobStats; + NKikimr::NUdf::ICountersProvider* const Counters; + const NKikimr::NUdf::ISecureParamsProvider* SecureParamsProvider; + + /// TODO: remove? + void SetExternalEnv(const NKikimr::NMiniKQL::TTypeEnvironment* env); +private: + const NKikimr::NMiniKQL::TTypeEnvironment* CreateTypeEnv() const; + mutable std::shared_ptr<NKikimr::NMiniKQL::TTypeEnvironment> InjectedEnvPtr; + mutable std::shared_ptr<NKikimr::NMiniKQL::TTypeEnvironment> EnvPtr; + mutable const NKikimr::NMiniKQL::TTypeEnvironment* Env; +}; + +} diff --git a/yt/yql/providers/yt/lib/lambda_builder/ya.make b/yt/yql/providers/yt/lib/lambda_builder/ya.make new file mode 100644 index 0000000000..306049a0ee --- /dev/null +++ b/yt/yql/providers/yt/lib/lambda_builder/ya.make @@ -0,0 +1,19 @@ +LIBRARY() + +SRCS( + lambda_builder.cpp +) + +PEERDIR( + library/cpp/random_provider + library/cpp/time_provider + yql/essentials/ast + yql/essentials/minikql/computation + yql/essentials/public/udf + yql/essentials/utils + yql/essentials/providers/common/mkql +) + +YQL_LAST_ABI_VERSION() + +END() diff --git a/yt/yql/providers/yt/lib/log/ya.make b/yt/yql/providers/yt/lib/log/ya.make new file mode 100644 index 0000000000..0efd961ad5 --- /dev/null +++ b/yt/yql/providers/yt/lib/log/ya.make @@ -0,0 +1,15 @@ +LIBRARY() + +SRCS( + yt_logger.cpp + yt_logger.h +) + +PEERDIR( + yt/yql/providers/yt/lib/init_yt_api + yt/cpp/mapreduce/interface/logging + yql/essentials/utils/log + yql/essentials/utils/backtrace +) + +END() diff --git a/yt/yql/providers/yt/lib/log/yt_logger.cpp b/yt/yql/providers/yt/lib/log/yt_logger.cpp new file mode 100644 index 0000000000..14f12fe090 --- /dev/null +++ b/yt/yql/providers/yt/lib/log/yt_logger.cpp @@ -0,0 +1,156 @@ +#include "yt_logger.h" + +#include <yt/yql/providers/yt/lib/init_yt_api/init.h> + +#include <yql/essentials/utils/log/log.h> +#include <yql/essentials/utils/backtrace/backtrace.h> + +#include <yt/cpp/mapreduce/interface/logging/logger.h> + +#include <util/stream/printf.h> +#include <util/stream/str.h> +#include <util/stream/file.h> +#include <util/system/spinlock.h> +#include <util/system/guard.h> +#include <util/system/file.h> +#include <util/generic/mem_copy.h> + + +namespace NYql { + +class TGlobalLoggerImpl: public NYT::ILogger +{ +public: + TGlobalLoggerImpl(int cutLevel, size_t debugLogBufferSize, const TString& debugLogFile, bool debugLogAlwaysWrite) + : CutLevel_(cutLevel) + , BufferSize_(debugLogBufferSize) + , DebugLogFile_(debugLogFile) + , DebugLogAlwaysWrite_(debugLogAlwaysWrite) + { + if (BufferSize_ && DebugLogFile_) { + Buffer_ = TArrayHolder<char>(new char[BufferSize_]); + } + } + + ~TGlobalLoggerImpl() { + FlushYtDebugLog(); + } + + void FlushYtDebugLog() { + THolder<char, TDeleteArray> buffer; + with_lock(BufferLock_) { + buffer.Swap(Buffer_); + } + if (buffer) { + try { + TUnbufferedFileOutput out(TFile(DebugLogFile_, OpenAlways | WrOnly | Seq | ForAppend | NoReuse)); + if (BufferFull_ && BufferWritePos_ < BufferSize_) { + out.Write(buffer.Get() + BufferWritePos_, BufferSize_ - BufferWritePos_); + } + if (BufferWritePos_ > 0) { + out.Write(buffer.Get(), BufferWritePos_); + } + } catch (...) { + YQL_CLOG(ERROR, ProviderYt) << CurrentExceptionMessage(); + } + } + } + + void DropYtDebugLog() { + if (DebugLogAlwaysWrite_) { + FlushYtDebugLog(); + return; + } + with_lock(BufferLock_) { + Buffer_.Destroy(); + } + } + + void Log(ELevel level, const TSourceLocation& sl, const char* format, va_list args) override { + NLog::ELevel yqlLevel = NLog::ELevel::TRACE; + switch (level) { + case FATAL: + yqlLevel = NLog::ELevel::FATAL; + break; + case ERROR: + yqlLevel = NLog::ELevel::ERROR; + break; + case INFO: + yqlLevel = NLog::ELevel::INFO; + break; + case DEBUG: + yqlLevel = NLog::ELevel::DEBUG; + break; + } + const bool needLog = int(level) <= CutLevel_ && NLog::YqlLogger().NeedToLog(NLog::EComponent::ProviderYt, yqlLevel); + with_lock(BufferLock_) { + if (!needLog && !Buffer_) { + return; + } + } + + TStringStream stream; + NLog::YqlLogger().WriteLogPrefix(&stream, NLog::EComponent::ProviderYt, yqlLevel, sl.File, sl.Line); + NLog::OutputLogCtx(&stream, true); + Printf(stream, format, args); + stream << Endl; + + if (needLog) { + NLog::YqlLogger().Write(NLog::ELevelHelpers::ToLogPriority(yqlLevel), stream.Str().data(), stream.Str().length()); + } + with_lock(BufferLock_) { + if (Buffer_) { + const char* ptr = stream.Str().data(); + size_t remaining = stream.Str().length(); + while (remaining) { + const size_t write = Min(remaining, BufferSize_ - BufferWritePos_); + MemCopy(Buffer_.Get() + BufferWritePos_, ptr, write); + ptr += write; + BufferWritePos_ += write; + remaining -= write; + if (BufferWritePos_ >= BufferSize_) { + BufferWritePos_ = 0; + BufferFull_ = true; + } + } + } + } + } + +private: + int CutLevel_; + THolder<char, TDeleteArray> Buffer_; + size_t BufferSize_ = 0; + size_t BufferWritePos_ = 0; + bool BufferFull_ = false; + TSpinLock BufferLock_; + TString DebugLogFile_; + const bool DebugLogAlwaysWrite_; +}; + +void SetYtLoggerGlobalBackend(int level, size_t debugLogBufferSize, const TString& debugLogFile, bool debugLogAlwaysWrite) { + // Important to initialize YT API before setting logger. Otherwise YT API initialization will rest it to default + InitYtApiOnce(); + if (level >= 0 || (debugLogBufferSize && debugLogFile)) { + NYT::SetLogger(new TGlobalLoggerImpl(level, debugLogBufferSize, debugLogFile, debugLogAlwaysWrite)); + NYql::NBacktrace::AddAfterFatalCallback([](int ){ NYql::FlushYtDebugLog(); }); + } else { + NYT::SetLogger(NYT::ILoggerPtr()); + } +} + +void FlushYtDebugLog() { + auto logger = NYT::GetLogger(); + if (auto yqlLogger = dynamic_cast<TGlobalLoggerImpl*>(logger.Get())) { + yqlLogger->FlushYtDebugLog(); + } +} + +void DropYtDebugLog() { + auto logger = NYT::GetLogger(); + if (auto yqlLogger = dynamic_cast<TGlobalLoggerImpl*>(logger.Get())) { + yqlLogger->DropYtDebugLog(); + } +} + +} // NYql diff --git a/yt/yql/providers/yt/lib/log/yt_logger.h b/yt/yql/providers/yt/lib/log/yt_logger.h new file mode 100644 index 0000000000..6f541d28f4 --- /dev/null +++ b/yt/yql/providers/yt/lib/log/yt_logger.h @@ -0,0 +1,12 @@ +#pragma once + +#include <util/generic/string.h> + +namespace NYql { + +// Use NYT::ILogger::ELevel for level +void SetYtLoggerGlobalBackend(int level, size_t debugLogBufferSize = 0, const TString& debugLogFile = TString(), bool debugLogAlwaysWrite = false); +void FlushYtDebugLog(); +void DropYtDebugLog(); + +} // NYql diff --git a/yt/yql/providers/yt/lib/mkql_helpers/mkql_helpers.cpp b/yt/yql/providers/yt/lib/mkql_helpers/mkql_helpers.cpp new file mode 100644 index 0000000000..bc307c3178 --- /dev/null +++ b/yt/yql/providers/yt/lib/mkql_helpers/mkql_helpers.cpp @@ -0,0 +1,97 @@ +#include "mkql_helpers.h" + +#include <yql/essentials/core/yql_expr_type_annotation.h> +#include <yql/essentials/ast/yql_expr.h> +#include <yql/essentials/utils/yql_panic.h> + +#include <yql/essentials/minikql/mkql_node_cast.h> + +#include <util/generic/strbuf.h> +#include <util/generic/utility.h> +#include <util/generic/ylimits.h> +#include <util/generic/hash.h> + +namespace NYql { + +using namespace NKikimr; +using namespace NKikimr::NMiniKQL; + +namespace { + +bool UpdateRecordsRange(TRecordsRange& range, TStringBuf settingName, ui64 data) +{ + if (settingName == TStringBuf("take")) { + range.Limit = Min(data, range.Limit.GetOrElse(Max<ui64>())); + } else if (settingName == TStringBuf("skip")) { + if (range.Limit.Defined()) { + if (data >= range.Limit.GetRef()) { + range.Limit = 0; + range.Offset = 0; + return false; + } + + range.Offset = data; + range.Limit = range.Limit.GetRef() - data; + } else { + ui64 prevOffset = range.Offset.GetOrElse(0); + if (data > Max<ui64>() - prevOffset) { + range.Limit = 0; + range.Offset = 0; + return false; + } + + range.Offset = data + prevOffset; + } + } + return true; +} + +const THashMap<TStringBuf, NUdf::TDataTypeId> SYS_COLUMN_TYPE_IDS = { + {"path", NUdf::TDataType<char*>::Id}, + {"record", NUdf::TDataType<ui64>::Id}, + {"index", NUdf::TDataType<ui32>::Id}, + {"num", NUdf::TDataType<ui64>::Id}, + {"keyswitch", NUdf::TDataType<bool>::Id}, +}; + +} + +void TRecordsRange::Fill(const TExprNode& settingsNode) { + Offset.Clear(); + Limit.Clear(); + + for (auto& setting: settingsNode.Children()) { + if (setting->ChildrenSize() == 0) { + continue; + } + + auto settingName = setting->Child(0)->Content(); + if (settingName != TStringBuf("take") && settingName != TStringBuf("skip")) { + continue; + } + YQL_ENSURE(setting->Child(1)->IsCallable("Uint64")); + if (!UpdateRecordsRange(*this, settingName, NYql::FromString<ui64>(*setting->Child(1)->Child(0), NUdf::EDataSlot::Uint64))) { + break; + } + } +} + +TType* GetRecordType(TType* type) { + if (type->GetKind() == TType::EKind::List) { + return AS_TYPE(TListType, type)->GetItemType(); + } else if (type->GetKind() == TType::EKind::Optional) { + return AS_TYPE(TOptionalType, type)->GetItemType(); + } else if (type->GetKind() == TType::EKind::Stream) { + return AS_TYPE(TStreamType, type)->GetItemType(); + } + + return type; +} + +NUdf::TDataTypeId GetSysColumnTypeId(TStringBuf column) { + auto p = SYS_COLUMN_TYPE_IDS.FindPtr(column); + YQL_ENSURE(p, "Unsupported system column:" << column); + return *p; +} + +} // NYql diff --git a/yt/yql/providers/yt/lib/mkql_helpers/mkql_helpers.h b/yt/yql/providers/yt/lib/mkql_helpers/mkql_helpers.h new file mode 100644 index 0000000000..1231e375dd --- /dev/null +++ b/yt/yql/providers/yt/lib/mkql_helpers/mkql_helpers.h @@ -0,0 +1,27 @@ +#pragma once + +#include <yql/essentials/minikql/mkql_node.h> + +#include <util/generic/maybe.h> +#include <util/generic/strbuf.h> + +namespace NYql { + +NKikimr::NMiniKQL::TType* GetRecordType(NKikimr::NMiniKQL::TType* type); + +class TExprNode; + +struct TRecordsRange { + TMaybe<ui64> Offset; + TMaybe<ui64> Limit; + + explicit operator bool() const { + return Offset.Defined() || Limit.Defined(); + } + + void Fill(const TExprNode& settingsNode); +}; + +NUdf::TDataTypeId GetSysColumnTypeId(TStringBuf column); + +} diff --git a/yt/yql/providers/yt/lib/mkql_helpers/ya.make b/yt/yql/providers/yt/lib/mkql_helpers/ya.make new file mode 100644 index 0000000000..5cf2554e11 --- /dev/null +++ b/yt/yql/providers/yt/lib/mkql_helpers/ya.make @@ -0,0 +1,16 @@ +LIBRARY() + +SRCS( + mkql_helpers.cpp +) + +PEERDIR( + yql/essentials/minikql + yql/essentials/core + yql/essentials/ast + yql/essentials/utils +) + +YQL_LAST_ABI_VERSION() + +END() diff --git a/yt/yql/providers/yt/lib/res_pull/res_or_pull.cpp b/yt/yql/providers/yt/lib/res_pull/res_or_pull.cpp new file mode 100644 index 0000000000..d7800dc2d7 --- /dev/null +++ b/yt/yql/providers/yt/lib/res_pull/res_or_pull.cpp @@ -0,0 +1,288 @@ +#include "res_or_pull.h" + +#include <yql/essentials/providers/common/codec/yql_codec.h> +#include <yql/essentials/utils/yql_panic.h> + +#include <yql/essentials/minikql/mkql_node_cast.h> + +#include <util/stream/holder.h> +#include <util/stream/str.h> + +namespace NYql { + +using namespace NKikimr; +using namespace NKikimr::NMiniKQL; + +////////////////////////////////////////////////////////////////////////////////////////////////////////// + +TYsonExecuteResOrPull::TYsonExecuteResOrPull(TMaybe<ui64> rowLimit, TMaybe<ui64> byteLimit, const TMaybe<TVector<TString>>& columns) + : IExecuteResOrPull(rowLimit, byteLimit, columns) + , Writer(new NYson::TYsonWriter(Out.Get(), NYson::EYsonFormat::Binary, ::NYson::EYsonType::Node, true)) +{ +} + +TString TYsonExecuteResOrPull::Finish() { + if (IsList) { + Writer->OnEndList(); + } + Writer.Destroy(); + Out.Destroy(); + return Result; +} + +bool TYsonExecuteResOrPull::WriteNext(const NYT::TNode& item) { + if (IsList) { + if (!HasCapacity()) { + Truncated = true; + return false; + } + Writer->OnListItem(); + } + + Writer->OnRaw(NYT::NodeToYsonString(item, NYT::NYson::EYsonFormat::Binary), ::NYson::EYsonType::Node); + ++Row; + + return IsList; +} + +void TYsonExecuteResOrPull::WriteValue(const NUdf::TUnboxedValue& value, TType* type) { + if (type->IsList()) { + auto inputType = AS_TYPE(TListType, type)->GetItemType(); + TMaybe<TVector<ui32>> structPositions = NCommon::CreateStructPositions(inputType, Columns.Defined() ? Columns.Get() : nullptr); + SetListResult(); + const auto it = value.GetListIterator(); + for (NUdf::TUnboxedValue item; it.Next(item); ++Row) { + if (!HasCapacity()) { + Truncated = true; + break; + } + + Writer->OnListItem(); + NCommon::WriteYsonValue(*Writer, item, inputType, structPositions.Get()); + } + } else { + NCommon::WriteYsonValue(*Writer, value, type, nullptr); + } +} + +bool TYsonExecuteResOrPull::WriteNext(TMkqlIOCache& specsCache, const NYT::TNode& rec, ui32 tableIndex) { + if (!HasCapacity()) { + Truncated = true; + return false; + } + + NYql::DecodeToYson(specsCache, tableIndex, rec, *Out); + ++Row; + return true; +} + +bool TYsonExecuteResOrPull::WriteNext(TMkqlIOCache& specsCache, const NYT::TYaMRRow& rec, ui32 tableIndex) { + if (!HasCapacity()) { + Truncated = true; + return false; + } + + NYql::DecodeToYson(specsCache, tableIndex, rec, *Out); + ++Row; + return true; +} + +bool TYsonExecuteResOrPull::WriteNext(TMkqlIOCache& specsCache, const NUdf::TUnboxedValue& rec, ui32 tableIndex) { + if (!HasCapacity()) { + Truncated = true; + return false; + } + + NYql::DecodeToYson(specsCache, tableIndex, rec, *Out); + ++Row; + return true; +} + +void TYsonExecuteResOrPull::SetListResult() { + if (!IsList) { + IsList = true; + Writer->OnBeginList(); + } +} + +////////////////////////////////////////////////////////////////////////////////////////////////////////// + +TSkiffExecuteResOrPull::TSkiffExecuteResOrPull(TMaybe<ui64> rowLimit, TMaybe<ui64> byteLimit, NCommon::TCodecContext& codecCtx, const NKikimr::NMiniKQL::THolderFactory& holderFactory, const NYT::TNode& attrs, const TString& optLLVM, const TVector<TString>& columns) + : IExecuteResOrPull(rowLimit, byteLimit, columns) + , HolderFactory(holderFactory) + , SkiffWriter(*Out.Get(), 0, 4_MB) +{ + Specs.SetUseSkiff(optLLVM); + Specs.Init(codecCtx, attrs); + YQL_ENSURE(Specs.Outputs.size() == 1); + + SkiffWriter.SetSpecs(Specs); + + AlphabeticPermutation = CreateAlphabeticPositions(Specs.Outputs[0].RowType, columns); +} + +TString TSkiffExecuteResOrPull::Finish() { + SkiffWriter.Finish(); + Out.Destroy(); + Specs.Clear(); + return Result; +} + +bool TSkiffExecuteResOrPull::WriteNext(const NYT::TNode& item) { + if (IsList) { + if (!HasCapacity()) { + Truncated = true; + return false; + } + } + + // For now this method is used only for the nodes that are lists of columns + // Feel free to extend it for other types (maps, for example) but make sure + // that the order of columns is correct. + YQL_ENSURE(item.GetType() == NYT::TNode::EType::List, "Expected list node"); + const auto& listNode = item.UncheckedAsList(); + + const auto& permutation = *AlphabeticPermutation; + YQL_ENSURE(permutation.size() == listNode.size(), "Expected the same number of columns and values"); + + // TODO: Node is being copied here. This can be avoided by doing in-place swaps + // but it requires changing the signature of function to pass mutable node here. + // Note, that it can be implemented without actual change of the node by + // applying inverse permutation after the shuffle. + auto alphabeticItem = NYT::TNode::CreateList(); + auto& alphabeticList = alphabeticItem.UncheckedAsList(); + alphabeticList.reserve(listNode.size()); + for (size_t index = 0; index < listNode.size(); ++index) { + alphabeticList.push_back(listNode[permutation[index]]); + } + + TStringStream err; + auto value = NCommon::ParseYsonNodeInResultFormat(HolderFactory, alphabeticItem, Specs.Outputs[0].RowType, &err); + if (!value) { + throw yexception() << "Could not parse yson node with error: " << err.Str(); + } + + // Call above produces rows in alphabetic order. + SkiffWriter.AddRow(*value); + + return IsList; +} + +void TSkiffExecuteResOrPull::WriteValue(const NUdf::TUnboxedValue& value, TType* type) { + if (type->IsList()) { + SetListResult(); + const auto it = value.GetListIterator(); + for (NUdf::TUnboxedValue item; it.Next(item); ++Row) { + if (!HasCapacity()) { + Truncated = true; + break; + } + + SkiffWriter.AddRow(item); + } + } else { + SkiffWriter.AddRow(value); + } +} + +bool TSkiffExecuteResOrPull::WriteNext(TMkqlIOCache& specsCache, const NYT::TNode& rec, ui32 /*tableIndex*/) { + if (!HasCapacity()) { + Truncated = true; + return false; + } + + // For now this method is used only for the nodes that are maps from column name + // to value. Feel free to extend it for other types (lists, for example) but + // make sure that the order of columns is correct. + YQL_ENSURE(rec.GetType() == NYT::TNode::EType::Map, "Expected map node"); + + TStringStream err; + auto value = NCommon::ParseYsonNode(specsCache.GetHolderFactory(), rec, Specs.Outputs[0].RowType, Specs.Outputs[0].NativeYtTypeFlags, &err); + if (!value) { + throw yexception() << "Could not parse yson node with error: " << err.Str(); + } + SkiffWriter.AddRow(*value); + + ++Row; + return true; +} + +bool TSkiffExecuteResOrPull::WriteNext(TMkqlIOCache& specsCache, const NYT::TYaMRRow& rec, ui32 tableIndex) { + if (!HasCapacity()) { + Truncated = true; + return false; + } + + NUdf::TUnboxedValue node; + node = DecodeYamr(specsCache, tableIndex, rec); + SkiffWriter.AddRow(node); + + ++Row; + return true; +} + +bool TSkiffExecuteResOrPull::WriteNext(TMkqlIOCache& specsCache, const NUdf::TUnboxedValue& rec, ui32 tableIndex) { + Y_UNUSED(specsCache); + Y_UNUSED(tableIndex); + + if (!HasCapacity()) { + Truncated = true; + return false; + } + + YQL_ENSURE(rec.GetListLength() == AlphabeticPermutation->size()); + TUnboxedValueVector alphabeticValues(rec.GetListLength()); + for (size_t index = 0; index < rec.GetListLength(); ++index) { + alphabeticValues[index] = rec.GetElement((*AlphabeticPermutation)[index]); + } + + NUdf::TUnboxedValue alphabeticRecord = HolderFactory.RangeAsArray(alphabeticValues.begin(), alphabeticValues.end()); + SkiffWriter.AddRow(alphabeticRecord); + + ++Row; + return true; +} + +void TSkiffExecuteResOrPull::SetListResult() { + IsList = true; +} + +TMaybe<TVector<ui32>> TSkiffExecuteResOrPull::CreateAlphabeticPositions(NKikimr::NMiniKQL::TType* inputType, const TVector<TString>& columns) +{ + if (inputType->GetKind() != TType::EKind::Struct) { + return Nothing(); + } + auto inputStruct = AS_TYPE(TStructType, inputType); + + YQL_ENSURE(columns.empty() || columns.size() == inputStruct->GetMembersCount()); + + if (columns.empty()) { + TVector<ui32> positions(inputStruct->GetMembersCount()); + for (size_t index = 0; index < positions.size(); ++index) { + positions[index] = index; + } + return positions; + } + + TMap<TStringBuf, ui32> orders; + for (size_t index = 0; index < columns.size(); ++index) { + orders.emplace(columns[index], -1); + } + { + ui32 index = 0; + for (auto& [column, order] : orders) { + order = index++; + } + } + + TVector<ui32> positions(columns.size()); + for (size_t index = 0; index < columns.size(); ++index) { + positions[orders[columns[index]]] = index; + } + + return positions; +} + +////////////////////////////////////////////////////////////////////////////////////////////////////////// + +} // NYql diff --git a/yt/yql/providers/yt/lib/res_pull/res_or_pull.h b/yt/yql/providers/yt/lib/res_pull/res_or_pull.h new file mode 100644 index 0000000000..0facb948f1 --- /dev/null +++ b/yt/yql/providers/yt/lib/res_pull/res_or_pull.h @@ -0,0 +1,151 @@ +#pragma once + +#include <yt/yql/providers/yt/codec/yt_codec_io.h> +#include <yql/essentials/public/udf/udf_value.h> + +#include <yql/essentials/minikql/mkql_node.h> + +#include <library/cpp/yson/node/node_io.h> + +#include <library/cpp/yson/writer.h> + +#include <util/stream/length.h> +#include <util/generic/maybe.h> +#include <util/generic/vector.h> +#include <util/generic/string.h> +#include <util/generic/strbuf.h> +#include <util/generic/ptr.h> +#include <util/generic/noncopyable.h> + +namespace NYql { + +/////////////////////////////////////////////////////////////////////////////// +// IExecuteResOrPull +/////////////////////////////////////////////////////////////////////////////// +class IExecuteResOrPull : public TNonCopyable { +public: + IExecuteResOrPull(TMaybe<ui64> rowLimit, TMaybe<ui64> byteLimit, const TMaybe<TVector<TString>>& columns) + : Rows(rowLimit) + , Bytes(byteLimit) + , Columns(columns) + , Out(new THoldingStream<TCountingOutput>(THolder(new TStringOutput(Result)))) + , IsList(false) + , Truncated(false) + , Row(0) + { + } + virtual ~IExecuteResOrPull() = default; + + bool HasCapacity() const { + return (!Rows || Row < *Rows) && (!Bytes || Out->Counter() < *Bytes); + } + + bool IsTruncated() const { + return Truncated; + } + + ui64 GetWrittenSize() const { + YQL_ENSURE(Out, "GetWritten() must be callled before Finish()"); + return Out->Counter(); + } + + ui64 GetWrittenRows() const { + return Row; + } + + TMaybe<ui64> GetRowsLimit() const { + return Rows; + } + + const TMaybe<TVector<TString>>& GetColumns() const { + return Columns; + } + + std::pair<TString, bool> Make() { + return {Finish(), IsTruncated()}; + } + + virtual TString Finish() = 0; + + virtual void SetListResult() = 0; + + virtual bool WriteNext(const NYT::TNode& item) = 0; + + virtual bool WriteNext(TMkqlIOCache& specsCache, const NUdf::TUnboxedValue& rec, ui32 tableIndex) = 0; + virtual bool WriteNext(TMkqlIOCache& specsCache, const NYT::TYaMRRow& rec, ui32 tableIndex) = 0; + virtual bool WriteNext(TMkqlIOCache& specsCache, const NYT::TNode& rec, ui32 tableIndex) = 0; + + virtual void WriteValue(const NKikimr::NUdf::TUnboxedValue& value, NKikimr::NMiniKQL::TType* type) = 0; + +protected: + const TMaybe<ui64> Rows; + const TMaybe<ui64> Bytes; + const TMaybe<TVector<TString>> Columns; + TString Result; + THolder<TCountingOutput> Out; + bool IsList; + bool Truncated; + ui64 Row; +}; + +/////////////////////////////////////////////////////////////////////////////// +// TYsonExecuteResOrPull +/////////////////////////////////////////////////////////////////////////////// +class TYsonExecuteResOrPull : public IExecuteResOrPull { +public: + TYsonExecuteResOrPull(TMaybe<ui64> rowLimit, TMaybe<ui64> byteLimit, const TMaybe<TVector<TString>>& columns); + ~TYsonExecuteResOrPull() = default; + + TString Finish() override; + + void SetListResult() override; + + bool WriteNext(const NYT::TNode& item) override; + + bool WriteNext(TMkqlIOCache& specsCache, const NUdf::TUnboxedValue& rec, ui32 tableIndex) override; + bool WriteNext(TMkqlIOCache& specsCache, const NYT::TYaMRRow& rec, ui32 tableIndex) override; + bool WriteNext(TMkqlIOCache& specsCache, const NYT::TNode& rec, ui32 tableIndex) override; + + void WriteValue(const NKikimr::NUdf::TUnboxedValue& value, NKikimr::NMiniKQL::TType* type) override; +protected: + THolder<NYson::TYsonWriter> Writer; +}; + +/////////////////////////////////////////////////////////////////////////////// +// TSkiffExecuteResOrPull +/////////////////////////////////////////////////////////////////////////////// +class TSkiffExecuteResOrPull : public IExecuteResOrPull { +public: + TSkiffExecuteResOrPull(TMaybe<ui64> rowLimit, TMaybe<ui64> byteLimit, NCommon::TCodecContext& codecCtx, const NKikimr::NMiniKQL::THolderFactory& holderFactory, const NYT::TNode& attrs, const TString& optLLVM, const TVector<TString>& columns = {}); + ~TSkiffExecuteResOrPull() = default; + + TString Finish() override; + + void SetListResult() override; + + bool WriteNext(const NYT::TNode& item) override; + + bool WriteNext(TMkqlIOCache& specsCache, const NUdf::TUnboxedValue& rec, ui32 tableIndex) override; + bool WriteNext(TMkqlIOCache& specsCache, const NYT::TYaMRRow& rec, ui32 tableIndex) override; + bool WriteNext(TMkqlIOCache& specsCache, const NYT::TNode& rec, ui32 tableIndex) override; + + void WriteValue(const NKikimr::NUdf::TUnboxedValue& value, NKikimr::NMiniKQL::TType* type) override; +protected: + const NKikimr::NMiniKQL::THolderFactory& HolderFactory; + + TMkqlIOSpecs Specs; + TMkqlWriterImpl SkiffWriter; + + // This vector contains the permutation of the columns that should be applied to the rows of a (single) + // output table to transform it from "shuffled" to "alphabetic" order. + // Absense of the vector means that columns were not provided and thus transforming rows + // is not possible (and thus is not required). + // i-th element of the permutation means that i-th value of the "shuffled" row is p[i]-th value of the "alphabetic" row. + TMaybe<TVector<ui32>> AlphabeticPermutation; + + // Returns a permutation such that i-th column in alphabetic order is at positions[i]. + // If columns is empty, returns an identity permutation. + static TMaybe<TVector<ui32>> CreateAlphabeticPositions(NKikimr::NMiniKQL::TType* inputType, const TVector<TString>& columns); +}; + +} diff --git a/yt/yql/providers/yt/lib/res_pull/table_limiter.cpp b/yt/yql/providers/yt/lib/res_pull/table_limiter.cpp new file mode 100644 index 0000000000..98731da58f --- /dev/null +++ b/yt/yql/providers/yt/lib/res_pull/table_limiter.cpp @@ -0,0 +1,60 @@ +#include "table_limiter.h" + +#include <yt/yql/providers/yt/lib/mkql_helpers/mkql_helpers.h> + +#include <util/generic/ylimits.h> + +namespace NYql { + +TTableLimiter::TTableLimiter(const TRecordsRange& range) + : Start(range.Offset.GetOrElse(0ULL)) + , End(range.Limit.Defined() ? Start + *range.Limit : Max()) + , Current(0ULL) + , TableStart(0ULL) + , TableEnd(Max()) +{ +} + +bool TTableLimiter::NextTable(ui64 recordCount) { + TableStart = 0ULL; + TableEnd = Max(); + if (!recordCount) { // Skip empty tables + return false; + } + if (Start && Current + recordCount <= Start) { + Current += recordCount; + return false; + } + if (Start && Current < Start && Current + recordCount > Start) { + TableStart = Start - Current; + } + + if (Current < End && Current + recordCount > End) { + TableEnd = End - Current; + } + + Current += recordCount; + return true; +} + +void TTableLimiter::NextDynamicTable() { + TableStart = 0ULL; + TableEnd = Max(); + if (Start && Current < Start) { + TableStart = Start - Current; + } + + if (Current < End) { + TableEnd = End - Current; + } +} + +void TTableLimiter::Skip(ui64 recordCount) { + Current += recordCount; +} + +ui64 TTableLimiter::GetTableZEnd() const { + return Max<ui64>() == TableEnd ? 0ULL : TableEnd; +} + +} // NYql diff --git a/yt/yql/providers/yt/lib/res_pull/table_limiter.h b/yt/yql/providers/yt/lib/res_pull/table_limiter.h new file mode 100644 index 0000000000..e5aaea7bda --- /dev/null +++ b/yt/yql/providers/yt/lib/res_pull/table_limiter.h @@ -0,0 +1,37 @@ +#pragma once + +#include <util/system/defaults.h> + +namespace NYql { + +struct TRecordsRange; + +class TTableLimiter { +public: + TTableLimiter(const TRecordsRange& range); + + bool NextTable(ui64 recordCount); + void NextDynamicTable(); + inline ui64 GetTableStart() const { + return TableStart; + } + inline ui64 GetTableEnd() const { + return TableEnd; + } + ui64 GetTableZEnd() const; + inline bool Exceed() const { + return Current >= End; + } + inline explicit operator bool() const { + return !Exceed(); + } + void Skip(ui64 recordCount); +private: + ui64 Start; + ui64 End; + ui64 Current; + ui64 TableStart; + ui64 TableEnd; +}; + +} // NYql diff --git a/yt/yql/providers/yt/lib/res_pull/ya.make b/yt/yql/providers/yt/lib/res_pull/ya.make new file mode 100644 index 0000000000..04d6591da1 --- /dev/null +++ b/yt/yql/providers/yt/lib/res_pull/ya.make @@ -0,0 +1,20 @@ +LIBRARY() + +SRCS( + res_or_pull.cpp + table_limiter.cpp +) + +PEERDIR( + library/cpp/yson + yql/essentials/minikql + yql/essentials/public/udf + yql/essentials/utils + yql/essentials/providers/common/codec + yt/yql/providers/yt/codec + yt/yql/providers/yt/lib/mkql_helpers +) + +YQL_LAST_ABI_VERSION() + +END() diff --git a/yt/yql/providers/yt/lib/row_spec/ya.make b/yt/yql/providers/yt/lib/row_spec/ya.make new file mode 100644 index 0000000000..1b1d5bd2d5 --- /dev/null +++ b/yt/yql/providers/yt/lib/row_spec/ya.make @@ -0,0 +1,24 @@ +LIBRARY() + +SRCS( + yql_row_spec.cpp +) + +PEERDIR( + library/cpp/yson/node + yql/essentials/ast + yql/essentials/core/expr_nodes_gen + yql/essentials/core + yql/essentials/core/expr_nodes + yql/essentials/core/issue + yql/essentials/providers/common/codec + yql/essentials/providers/common/provider + yql/essentials/providers/common/schema + yql/essentials/providers/common/schema/expr + yt/yql/providers/yt/common + yt/yql/providers/yt/expr_nodes +) + +YQL_LAST_ABI_VERSION() + +END() diff --git a/yt/yql/providers/yt/lib/row_spec/yql_row_spec.cpp b/yt/yql/providers/yt/lib/row_spec/yql_row_spec.cpp new file mode 100644 index 0000000000..10ee1b54ad --- /dev/null +++ b/yt/yql/providers/yt/lib/row_spec/yql_row_spec.cpp @@ -0,0 +1,1548 @@ +#include "yql_row_spec.h" + +#include <yt/yql/providers/yt/expr_nodes/yql_yt_expr_nodes.h> +#include <yt/yql/providers/yt/lib/schema/schema.h> +#include <yt/yql/providers/yt/common/yql_names.h> +#include <yt/yql/providers/yt/common/yql_yt_settings.h> +#include <yql/essentials/providers/common/provider/yql_provider.h> +#include <yql/essentials/providers/common/codec/yql_codec_type_flags.h> +#include <yql/essentials/providers/common/schema/expr/yql_expr_schema.h> +#include <yql/essentials/providers/common/schema/yql_schema_utils.h> +#include <yql/essentials/core/expr_nodes/yql_expr_nodes.h> +#include <yql/essentials/core/issue/yql_issue.h> +#include <yql/essentials/core/yql_expr_type_annotation.h> +#include <yql/essentials/utils/log/log.h> + +#include <library/cpp/yson/node/node_io.h> +#include <library/cpp/yson/node/node_builder.h> + +#include <util/generic/cast.h> +#include <util/generic/xrange.h> +#include <util/string/builder.h> + +#include <algorithm> + +namespace NYql { + +namespace { + +ui64 GetNativeYtTypeFlagsImpl(const TTypeAnnotationNode* itemType) { + switch (itemType->GetKind()) { + case ETypeAnnotationKind::Pg: { + auto name = itemType->Cast<TPgExprType>()->GetName(); + if (name == "float4") { + return NTCF_FLOAT | NTCF_NO_YT_SUPPORT; + } + + return NTCF_NO_YT_SUPPORT; + } + case ETypeAnnotationKind::Data: + switch (itemType->Cast<TDataExprType>()->GetSlot()) { + case EDataSlot::Date: + case EDataSlot::Datetime: + case EDataSlot::Timestamp: + case EDataSlot::Interval: + return NTCF_DATE; + case EDataSlot::Date32: + case EDataSlot::Datetime64: + case EDataSlot::Timestamp64: + case EDataSlot::Interval64: + return NTCF_BIGDATE; + case EDataSlot::Json: + return NTCF_JSON; + case EDataSlot::Float: + return NTCF_FLOAT; + case EDataSlot::Decimal: + return NTCF_DECIMAL; + case EDataSlot::Uuid: + return NTCF_UUID; + case EDataSlot::TzDate: + case EDataSlot::TzDatetime: + case EDataSlot::TzTimestamp: + case EDataSlot::TzDate32: + case EDataSlot::TzDatetime64: + case EDataSlot::TzTimestamp64: + case EDataSlot::DyNumber: + case EDataSlot::JsonDocument: + return NTCF_NO_YT_SUPPORT; + default: + return NTCF_NONE; + } + case ETypeAnnotationKind::Null: + return NTCF_NULL; + case ETypeAnnotationKind::Void: + return NTCF_VOID; + case ETypeAnnotationKind::Optional: + return NTCF_COMPLEX | GetNativeYtTypeFlagsImpl(itemType->Cast<TOptionalExprType>()->GetItemType()); + case ETypeAnnotationKind::List: + return NTCF_COMPLEX | GetNativeYtTypeFlagsImpl(itemType->Cast<TListExprType>()->GetItemType()); + case ETypeAnnotationKind::Dict: { + auto dictType = itemType->Cast<TDictExprType>(); + return NTCF_COMPLEX | GetNativeYtTypeFlagsImpl(dictType->GetKeyType()) | GetNativeYtTypeFlagsImpl(dictType->GetPayloadType()); + } + case ETypeAnnotationKind::Variant: + return NTCF_COMPLEX | GetNativeYtTypeFlagsImpl(itemType->Cast<TVariantExprType>()->GetUnderlyingType()); + case ETypeAnnotationKind::Struct: { + ui64 flags = NTCF_COMPLEX; + for (auto item: itemType->Cast<TStructExprType>()->GetItems()) { + flags |= GetNativeYtTypeFlagsImpl(item->GetItemType()); + } + return flags; + } + case ETypeAnnotationKind::Tuple: { + ui64 flags = NTCF_COMPLEX; + for (auto item: itemType->Cast<TTupleExprType>()->GetItems()) { + flags |= GetNativeYtTypeFlagsImpl(item); + } + return flags; + } + case ETypeAnnotationKind::Tagged: + return NTCF_COMPLEX | GetNativeYtTypeFlagsImpl(itemType->Cast<TTaggedExprType>()->GetBaseType()); + case ETypeAnnotationKind::EmptyDict: + case ETypeAnnotationKind::EmptyList: + return NTCF_COMPLEX; + case ETypeAnnotationKind::World: + case ETypeAnnotationKind::Unit: + case ETypeAnnotationKind::Item: + case ETypeAnnotationKind::Callable: + case ETypeAnnotationKind::Generic: + case ETypeAnnotationKind::Error: + case ETypeAnnotationKind::Resource: + case ETypeAnnotationKind::Stream: + case ETypeAnnotationKind::Flow: + case ETypeAnnotationKind::Multi: + case ETypeAnnotationKind::Type: + case ETypeAnnotationKind::Block: + case ETypeAnnotationKind::Scalar: + case ETypeAnnotationKind::LastType: + break; + } + return NTCF_NONE; +} + +} + +ui64 GetNativeYtTypeFlags(const TStructExprType& type, const NCommon::TStructMemberMapper& mapper) { + ui64 flags = 0; + for (auto item: type.GetItems()) { + if (!mapper || mapper(item->GetName())) { + const TTypeAnnotationNode* itemType = item->GetItemType(); + bool wasOptional = false; + if (itemType->GetKind() == ETypeAnnotationKind::Optional) { + wasOptional = true; + itemType = itemType->Cast<TOptionalExprType>()->GetItemType(); + } + + if (wasOptional && itemType->GetKind() == ETypeAnnotationKind::Pg) { + flags |= NTCF_COMPLEX; + } + + flags |= GetNativeYtTypeFlagsImpl(itemType); + } + } + flags &= ~NTCF_NO_YT_SUPPORT; + return flags; +} + +using namespace NNodes; + +bool TYqlRowSpecInfo::Parse(const TString& rowSpecYson, TExprContext& ctx, const TPositionHandle& pos) { + try { + return Parse(NYT::NodeFromYsonString(rowSpecYson), ctx, pos); + } catch (const std::exception& e) { + ctx.AddError(TIssue(ctx.GetPosition(pos), TStringBuilder() << "Failed to parse row spec: " << e.what())); + return false; + } +} + +bool TYqlRowSpecInfo::Parse(const NYT::TNode& rowSpecAttr, TExprContext& ctx, const TPositionHandle& pos) { + *this = {}; + try { + if (!ParseType(rowSpecAttr, ctx, pos) || !ParseSort(rowSpecAttr, ctx, pos)) { + return false; + } + ParseFlags(rowSpecAttr); + ParseDefValues(rowSpecAttr); + ParseConstraints(rowSpecAttr); + ParseConstraintsNode(ctx); + } catch (const std::exception& e) { + ctx.AddError(TIssue(ctx.GetPosition(pos), TStringBuilder() << "Failed to parse row spec: " << e.what())); + return false; + } + + return Validate(ctx, pos); +} + +bool TYqlRowSpecInfo::ParsePatched(const NYT::TNode& rowSpecAttr, const THashMap<TString, TString>& attrs, TExprContext& ctx, const TPositionHandle& pos) { + auto schemaAttr = attrs.FindPtr(SCHEMA_ATTR_NAME); + if (!schemaAttr) { + YQL_LOG_CTX_THROW yexception() << YqlRowSpecAttribute << " with " << RowSpecAttrTypePatch << " attribute requires YT schema to be present"; + } + auto schema = NYT::NodeFromYsonString(*schemaAttr); + auto strict = schema.GetAttributes()["strict"]; + if (!strict.IsUndefined() && !NYT::GetBool(strict)) { + YQL_LOG_CTX_THROW yexception() << YqlRowSpecAttribute << " with " << RowSpecAttrTypePatch << " attribute can only be used with 'strict' schema"; + } + auto mode = schema.GetAttributes()[SCHEMA_MODE_ATTR_NAME]; + if (!mode.IsUndefined() && mode.AsString() == "weak") { + YQL_LOG_CTX_THROW yexception() << YqlRowSpecAttribute << " with " << RowSpecAttrTypePatch << " attribute can only be used with 'strong' schema"; + } + + auto schemaAsRowSpec = YTSchemaToRowSpec(schema); + if (!ParseType(schemaAsRowSpec, ctx, pos) || !ParseSort(schemaAsRowSpec, ctx, pos)) { + return false; + } + ParseFlags(schemaAsRowSpec); + + auto typePatch = NCommon::ParseTypeFromYson(rowSpecAttr[RowSpecAttrTypePatch], ctx, ctx.GetPosition(pos)); + if (!typePatch) { + return false; + } + + if (typePatch->GetKind() != ETypeAnnotationKind::Struct) { + YQL_LOG_CTX_THROW yexception() << "Row spec TypePatch has a non struct type: " << *typePatch; + } + + if (!ParseSort(rowSpecAttr, ctx, pos)) { + return false; + } + + TSet<TString> auxFields; + for (size_t i = 0; i < SortedBy.size(); ++i) { + if ((i >= SortMembers.size() || SortedBy[i] != SortMembers[i]) && IsSystemMember(SortedBy[i])) { + auxFields.insert(SortedBy[i]); + } + } + auto typePatchStruct = typePatch->Cast<TStructExprType>(); + if (typePatchStruct->GetSize() || !auxFields.empty()) { + // Patch Type + auto updatedItems = Type->GetItems(); + for (auto& patchItem: typePatchStruct->GetItems()) { + auto name = patchItem->GetName(); + YQL_ENSURE(!auxFields.contains(name)); + auto itemPos = Type->FindItem(name); + if (!itemPos) { + throw yexception() << "Row spec TypePatch refers to unknown field: " << name; + } + updatedItems[*itemPos] = patchItem; + } + for (auto it = auxFields.rbegin(); it != auxFields.rend(); ++it) { + auto itemPos = Type->FindItem(*it); + if (!itemPos) { + throw yexception() << "Row spec SortedBy refers to unknown field: " << *it; + } + YQL_ENSURE(*itemPos < updatedItems.size(), "Something wrong!"); + updatedItems.erase(updatedItems.begin() + *itemPos); + } + Type = ctx.MakeType<TStructExprType>(updatedItems); + + // Patch TypeNode + THashMap<TString, const NYT::TNode*> patchNodes; + for (auto& item: rowSpecAttr[RowSpecAttrTypePatch][1].AsList()) { + patchNodes.emplace(item[0].AsString(), &item); + } + if (auxFields.empty()) { + for (auto& item: TypeNode[1].AsList()) { + if (auto p = patchNodes.FindPtr(item[0].AsString())) { + item = **p; + } + } + } else { + auto& membersList = TypeNode[1].AsList(); + NYT::TNode::TListType newMembers; + newMembers.reserve(membersList.size()); + for (auto& item: membersList) { + if (!auxFields.contains(item[0].AsString())) { + if (auto p = patchNodes.FindPtr(item[0].AsString())) { + newMembers.push_back(**p); + } else { + newMembers.push_back(item); + } + } + } + membersList = std::move(newMembers); + + // Patch Columns + TColumnOrder newColumns; + for (auto& [col, gen_col]: *Columns) { + if (!auxFields.contains(col)) { + newColumns.AddColumn(col); + } + } + Columns = std::move(newColumns); + } + YQL_ENSURE(Type->GetSize() == TypeNode[1].AsList().size()); + } + + TYTSortInfo sortInfo = KeyColumnsFromSchema(schema); + if (!ValidateSort(sortInfo, ctx, pos)) { + return false; + } + + ParseFlags(rowSpecAttr); + ParseDefValues(rowSpecAttr); + ParseConstraints(rowSpecAttr); + ParseConstraintsNode(ctx); + return true; +} + +bool TYqlRowSpecInfo::ParseFull(const NYT::TNode& rowSpecAttr, const THashMap<TString, TString>& attrs, TExprContext& ctx, const TPositionHandle& pos) { + if (!ParseType(rowSpecAttr, ctx, pos) || !ParseSort(rowSpecAttr, ctx, pos)) { + return false; + } + ParseFlags(rowSpecAttr); + ParseDefValues(rowSpecAttr); + ParseConstraints(rowSpecAttr); + ParseConstraintsNode(ctx); + + if (auto schemaAttr = attrs.FindPtr(SCHEMA_ATTR_NAME)) { + auto schema = NYT::NodeFromYsonString(*schemaAttr); + auto modeAttr = schema.GetAttributes()[SCHEMA_MODE_ATTR_NAME]; + const bool weak = !modeAttr.IsUndefined() && modeAttr.AsString() == "weak"; + // Validate type for non weak schema only + if (!weak) { + auto schemaAsRowSpec = YTSchemaToRowSpec(schema); + auto type = NCommon::ParseTypeFromYson(schemaAsRowSpec[RowSpecAttrType], ctx, ctx.GetPosition(pos)); + if (!type) { + return false; + } + if (type->GetKind() != ETypeAnnotationKind::Struct) { + YQL_LOG_CTX_THROW yexception() << "YT schema type has a non struct type: " << *type; + } + THashSet<TStringBuf> auxFields(SortedBy.cbegin(), SortedBy.cend()); + TStringBuilder hiddenFields; + for (auto item: type->Cast<TStructExprType>()->GetItems()) { + if (const auto name = item->GetName(); !Type->FindItem(name) && !auxFields.contains(name)) { + if (hiddenFields.size() > 100) { + hiddenFields << ", ..."; + break; + } + if (!hiddenFields.empty()) { + hiddenFields << ", "; + } + hiddenFields << item->GetName(); + } + } + if (!hiddenFields.empty()) { + hiddenFields.prepend("Table attribute '_yql_row_spec' hides fields: "); + if (!ctx.AddWarning(YqlIssue(ctx.GetPosition(pos), EYqlIssueCode::TIssuesIds_EIssueCode_YT_ROWSPEC_HIDES_FIELDS, hiddenFields))) { + return false; + } + } + } + TYTSortInfo sortInfo = KeyColumnsFromSchema(schema); + if (!ValidateSort(sortInfo, ctx, pos)) { + return false; + } + } + return true; +} + +// Priority: +// 1. YqlRowSpec(TypePatch) + Schema +// 2. YqlRowSpec(Type) +// 3. _infer_schema + schema(SortBy) +// 4. _read_schema + schema(SortBy) +// 5. schema +bool TYqlRowSpecInfo::Parse(const THashMap<TString, TString>& attrs, TExprContext& ctx, const TPositionHandle& pos) { + *this = {}; + try { + if (auto rowSpecAttr = attrs.FindPtr(YqlRowSpecAttribute)) { + auto rowSpec = NYT::NodeFromYsonString(*rowSpecAttr); + if (rowSpec.HasKey(RowSpecAttrTypePatch)) { + if (!ParsePatched(rowSpec, attrs, ctx, pos)) { + return false; + } + } else { + if (!ParseFull(rowSpec, attrs, ctx, pos)) { + return false; + } + } + } else if (auto inferSchemaAttr = attrs.FindPtr(INFER_SCHEMA_ATTR_NAME)) { + auto inferSchema = NYT::NodeFromYsonString(*inferSchemaAttr); + + TYTSortInfo sortInfo; + auto schemaAttr = attrs.FindPtr(SCHEMA_ATTR_NAME); + if (schemaAttr) { + auto schema = NYT::NodeFromYsonString(*schemaAttr); + sortInfo = KeyColumnsFromSchema(schema); + MergeInferredSchemeWithSort(inferSchema, sortInfo); + } + auto schemaAsRowSpec = YTSchemaToRowSpec(inferSchema, schemaAttr ? &sortInfo : nullptr); + if (!ParseType(schemaAsRowSpec, ctx, pos) || !ParseSort(schemaAsRowSpec, ctx, pos)) { + return false; + } + ParseFlags(schemaAsRowSpec); + } else if (auto readSchema = attrs.FindPtr(READ_SCHEMA_ATTR_NAME)) { + TYTSortInfo sortInfo; + if (auto schemaAttr = attrs.FindPtr(SCHEMA_ATTR_NAME)) { + sortInfo = KeyColumnsFromSchema(NYT::NodeFromYsonString(*schemaAttr)); + } + auto schemaAsRowSpec = YTSchemaToRowSpec(NYT::NodeFromYsonString(*readSchema), &sortInfo); + if (!ParseType(schemaAsRowSpec, ctx, pos) || !ParseSort(schemaAsRowSpec, ctx, pos)) { + return false; + } + ParseFlags(schemaAsRowSpec); + } else if (auto schema = attrs.FindPtr(SCHEMA_ATTR_NAME)) { + auto schemaAsRowSpec = YTSchemaToRowSpec(NYT::NodeFromYsonString(*schema)); + if (!ParseType(schemaAsRowSpec, ctx, pos) || !ParseSort(schemaAsRowSpec, ctx, pos)) { + return false; + } + ParseFlags(schemaAsRowSpec); + } else { + YQL_LOG_CTX_THROW yexception() << "Table has no supported schema attributes"; + } + } catch (const std::exception& e) { + ctx.AddError(TIssue(ctx.GetPosition(pos), TStringBuilder() << "Failed to parse row spec: " << e.what())); + return false; + } + + return Validate(ctx, pos); +} + +bool TYqlRowSpecInfo::ParseType(const NYT::TNode& rowSpecAttr, TExprContext& ctx, const TPositionHandle& pos) { + if (!rowSpecAttr.HasKey(RowSpecAttrType)) { + YQL_LOG_CTX_THROW yexception() << "Row spec doesn't have mandatory Type attribute"; + } + TColumnOrder columns; + auto type = NCommon::ParseOrderAwareTypeFromYson(rowSpecAttr[RowSpecAttrType], columns, ctx, ctx.GetPosition(pos)); + if (!type) { + return false; + } + if (type->GetKind() != ETypeAnnotationKind::Struct) { + YQL_LOG_CTX_THROW yexception() << "Row spec defines not a struct type"; + } + + Type = type->Cast<TStructExprType>(); + TypeNode = rowSpecAttr[RowSpecAttrType]; + Columns = std::move(columns); + + if (rowSpecAttr.HasKey(RowSpecAttrStrictSchema)) { + // Backward compatible parse. Old code saves 'StrictSchema' as Int64 + StrictSchema = rowSpecAttr[RowSpecAttrStrictSchema].IsInt64() + ? rowSpecAttr[RowSpecAttrStrictSchema].AsInt64() != 0 + : NYT::GetBool(rowSpecAttr[RowSpecAttrStrictSchema]); + if (!StrictSchema) { + auto items = Type->GetItems(); + auto dictType = ctx.MakeType<TDictExprType>( + ctx.MakeType<TDataExprType>(EDataSlot::String), + ctx.MakeType<TDataExprType>(EDataSlot::String)); + items.push_back(ctx.MakeType<TItemExprType>(YqlOthersColumnName, dictType)); + Type = ctx.MakeType<TStructExprType>(items); + Columns->AddColumn(TString(YqlOthersColumnName)); + } + } + + return true; +} + +bool TYqlRowSpecInfo::ParseSort(const NYT::TNode& rowSpecAttr, TExprContext& ctx, const TPositionHandle& pos) { + if (rowSpecAttr.HasKey(RowSpecAttrSortMembers) || rowSpecAttr.HasKey(RowSpecAttrSortedBy) || rowSpecAttr.HasKey(RowSpecAttrSortDirections)) { + ClearSortness(ctx); + } + if (rowSpecAttr.HasKey(RowSpecAttrSortDirections)) { + for (auto& item: rowSpecAttr[RowSpecAttrSortDirections].AsList()) { + SortDirections.push_back(item.AsInt64() != 0); + } + } + + auto loadColumnList = [&] (TStringBuf name, TVector<TString>& columns) { + if (rowSpecAttr.HasKey(name)) { + auto& list = rowSpecAttr[name].AsList(); + for (const auto& item : list) { + columns.push_back(item.AsString()); + } + } + }; + + loadColumnList(RowSpecAttrSortMembers, SortMembers); + loadColumnList(RowSpecAttrSortedBy, SortedBy); + + if (rowSpecAttr.HasKey(RowSpecAttrSortedByTypes)) { + auto& list = rowSpecAttr[RowSpecAttrSortedByTypes].AsList(); + for (auto& type : list) { + if (auto sortType = NCommon::ParseTypeFromYson(type, ctx, ctx.GetPosition(pos))) { + SortedByTypes.push_back(sortType); + } else { + return false; + } + } + } + + if (rowSpecAttr.HasKey(RowSpecAttrUniqueKeys)) { + UniqueKeys = NYT::GetBool(rowSpecAttr[RowSpecAttrUniqueKeys]); + } + return true; +} + +void TYqlRowSpecInfo::ParseFlags(const NYT::TNode& rowSpecAttr) { + if (rowSpecAttr.HasKey(RowSpecAttrNativeYtTypeFlags)) { + NativeYtTypeFlags = rowSpecAttr[RowSpecAttrNativeYtTypeFlags].AsUint64(); + } else { + if (rowSpecAttr.HasKey(RowSpecAttrUseNativeYtTypes)) { + NativeYtTypeFlags = NYT::GetBool(rowSpecAttr[RowSpecAttrUseNativeYtTypes]) ? NTCF_LEGACY : NTCF_NONE; + } else if (rowSpecAttr.HasKey(RowSpecAttrUseTypeV2)) { + NativeYtTypeFlags = NYT::GetBool(rowSpecAttr[RowSpecAttrUseTypeV2]) ? NTCF_LEGACY : NTCF_NONE; + } + } + if (NativeYtTypeFlags) { + NativeYtTypeFlags &= NYql::GetNativeYtTypeFlags(*Type); + } +} + +void TYqlRowSpecInfo::ParseDefValues(const NYT::TNode& rowSpecAttr) { + if (rowSpecAttr.HasKey(RowSpecAttrDefaultValues)) { + for (auto& value : rowSpecAttr[RowSpecAttrDefaultValues].AsMap()) { + DefaultValues[value.first] = NYT::NodeFromYsonString(value.second.AsString()).AsString(); + } + } +} + +bool TYqlRowSpecInfo::HasNonTrivialSort() const { + return Sorted && (HasAuxColumns() || std::any_of(Sorted->GetContent().cbegin(), Sorted->GetContent().cend(), + [](const TSortedConstraintNode::TContainerType::value_type& item) {return 1U != item.first.size() || 1U != item.first.front().size(); })); +} + +NYT::TNode TYqlRowSpecInfo::GetConstraintsNode() const { + if (ConstraintsNode.HasValue()) + return ConstraintsNode; + + TConstraintSet set; + if (HasNonTrivialSort()) { + set.AddConstraint(Sorted); + } + set.AddConstraint(Unique); + set.AddConstraint(Distinct); + return set.ToYson(); +} + +void TYqlRowSpecInfo::FillConstraints(NYT::TNode& attrs) const { + if (HasNonTrivialSort() || Unique || Distinct) + attrs[RowSpecAttrConstraints] = GetConstraintsNode(); +} + +void TYqlRowSpecInfo::ParseConstraintsNode(TExprContext& ctx) { + if (!ConstraintsNode.HasValue()) + return; + + try { + SetConstraints(ctx.MakeConstraintSet(ConstraintsNode)); + } catch (const yexception& error) { + Sorted = nullptr; + Unique = nullptr; + Distinct = nullptr; + YQL_CLOG(WARN, ProviderYt) << " Error '" << error << "' on parse constraints node: " << ConstraintsNode.AsString(); + } +} + +TConstraintSet TYqlRowSpecInfo::GetSomeConstraints(ui64 mask, TExprContext& ctx) { + TConstraintSet set; + if (mask) { + ParseConstraintsNode(ctx); + if (Sorted && (ui64(EStoredConstraint::Sorted) & mask)) + set.AddConstraint(Sorted); + if (Unique && (ui64(EStoredConstraint::Unique) & mask)) + set.AddConstraint(Unique); + if (Distinct && (ui64(EStoredConstraint::Distinct) & mask)) + set.AddConstraint(Distinct); + } + return set; +} + +TConstraintSet TYqlRowSpecInfo::GetAllConstraints(TExprContext& ctx) { + return GetSomeConstraints(ui64(EStoredConstraint::Sorted) | ui64(EStoredConstraint::Unique) | ui64(EStoredConstraint::Distinct), ctx); +} + +void TYqlRowSpecInfo::ParseConstraints(const NYT::TNode& rowSpecAttr) { + if (rowSpecAttr.HasKey(RowSpecAttrConstraints)) + ConstraintsNode = rowSpecAttr[RowSpecAttrConstraints]; +} + +bool TYqlRowSpecInfo::ValidateSort(const TYTSortInfo& sortInfo, TExprContext& ctx, const TPositionHandle& pos) { + if (sortInfo.Keys.empty() && IsSorted()) { + ClearSortness(ctx); + if (!ctx.AddWarning(YqlIssue(ctx.GetPosition(pos), EYqlIssueCode::TIssuesIds_EIssueCode_YT_ROWSPEC_DIFF_SORT, + "Table attribute '_yql_row_spec' defines sorting, but the table is not really sorted. The sorting will be ignored."))) { + return false; + } + } + else if (!sortInfo.Keys.empty() && !IsSorted()) { + if (!ctx.AddWarning(YqlIssue(ctx.GetPosition(pos), EYqlIssueCode::TIssuesIds_EIssueCode_YT_ROWSPEC_DIFF_SORT, + "Table attribute '_yql_row_spec' hides the table sorting. The sorting will not be used in query optimization."))) { + return false; + } + } else if (IsSorted()) { + bool diff = false; + if (SortedBy.size() > sortInfo.Keys.size()) { + ClearSortness(ctx, sortInfo.Keys.size()); + diff = true; + } + auto backendSort = GetForeignSort(); + for (size_t i = 0; i < backendSort.size(); ++i) { + if (backendSort[i].first != sortInfo.Keys[i].first || backendSort[i].second != (bool)sortInfo.Keys[i].second) { + ClearSortness(ctx, i); + diff = true; + break; + } + } + if (diff) { + TStringBuilder warning; + warning << "Table attribute '_yql_row_spec' defines sorting, which differs from the actual one. The table will be assumed "; + if (IsSorted()) { + warning << "ordered by "; + for (size_t i: xrange(SortMembers.size())) { + if (i != 0) { + warning << ','; + } + warning << SortMembers[i] << '(' + << (SortDirections[i] ? "asc" : "desc") << ")"; + } + } else { + warning << "unordered"; + } + if (!ctx.AddWarning(YqlIssue(ctx.GetPosition(pos), EYqlIssueCode::TIssuesIds_EIssueCode_YT_ROWSPEC_DIFF_SORT, warning))) { + return false; + } + } + } + return true; +} + +bool TYqlRowSpecInfo::Validate(const TExprNode& node, TExprContext& ctx, const TStructExprType*& type, TMaybe<TColumnOrder>& columnOrder) { + type = nullptr; + columnOrder = {}; + if (!EnsureCallable(node, ctx)) { + return false; + } + if (!node.IsCallable(TYqlRowSpec::CallableName())) { + ctx.AddError(TIssue(ctx.GetPosition(node.Pos()), TStringBuilder() << "Expected " << TYqlRowSpec::CallableName() + << " callable, but got " << node.Content())); + return false; + } + + TVector<TString> sortedBy; + TVector<TString> sortMembers; + size_t sortDirectionsCount = 0; + TTypeAnnotationNode::TListType sortedByTypes; + THashSet<TStringBuf> defaultNames; + TVector<TString> explicitYson; + bool extendNonStrict = false; + bool strict = true; + for (auto child: node.Children()) { + if (!EnsureTupleSize(*child, 2, ctx)) { + return false; + } + const TExprNode* name = child->Child(0); + TExprNode* value = child->Child(1); + if (!EnsureAtom(*name, ctx)) { + return false; + } + bool flagValue = false; + if (name->Content() == RowSpecAttrStrictSchema) { + if (!EnsureAtom(*value, ctx)) { + return false; + } + if (!TryFromString(value->Content(), strict)) { + ctx.AddError(TIssue(ctx.GetPosition(value->Pos()), TStringBuilder() << "Bad value of " + << TString{RowSpecAttrStrictSchema}.Quote() << " attribute: " << value->Content())); + return false; + } + } else if (name->Content() == RowSpecAttrUseTypeV2 || name->Content() == RowSpecAttrUseNativeYtTypes) { + if (!EnsureAtom(*value, ctx)) { + return false; + } + if (!TryFromString(value->Content(), flagValue)) { + ctx.AddError(TIssue(ctx.GetPosition(value->Pos()), TStringBuilder() << "Bad value of " + << TString{name->Content()}.Quote() << " attribute: " << value->Content())); + return false; + } + } else if (name->Content() == RowSpecAttrNativeYtTypeFlags) { + if (!EnsureAtom(*value, ctx)) { + return false; + } + ui64 flags = 0; + if (!TryFromString(value->Content(), flags)) { + ctx.AddError(TIssue(ctx.GetPosition(value->Pos()), TStringBuilder() << "Bad value of " + << TString{name->Content()}.Quote() << " attribute: " << value->Content())); + return false; + } + } else if (name->Content() == RowSpecAttrUniqueKeys) { + if (!EnsureAtom(*value, ctx)) { + return false; + } + if (!TryFromString(value->Content(), flagValue)) { + ctx.AddError(TIssue(ctx.GetPosition(value->Pos()), TStringBuilder() << "Bad value of " + << TString{RowSpecAttrUniqueKeys}.Quote() << " attribute: " << value->Content())); + return false; + } + } else if (name->Content() == RowSpecAttrType) { + const TTypeAnnotationNode* rawType = nullptr; + if (value->Type() == TExprNode::Atom) { + columnOrder.ConstructInPlace(); + rawType = NCommon::ParseOrderAwareTypeFromYson(value->Content(), *columnOrder, ctx, ctx.GetPosition(value->Pos())); + if (!rawType) { + return false; + } + extendNonStrict = true; + } else { + if (!EnsureType(*value, ctx)) { + return false; + } + rawType = value->GetTypeAnn()->Cast<TTypeExprType>()->GetType(); + } + if (!EnsureStructType(value->Pos(), *rawType, ctx)) { + return false; + } + type = rawType->Cast<TStructExprType>(); + } else if (name->Content() == RowSpecAttrSortedBy) { + if (!EnsureTuple(*value, ctx)) { + return false; + } + for (const TExprNode::TPtr& item: value->Children()) { + if (!EnsureAtom(*item, ctx)) { + return false; + } + sortedBy.push_back(TString{item->Content()}); + } + } else if (name->Content() == RowSpecAttrSortMembers) { + if (!EnsureTuple(*value, ctx)) { + return false; + } + for (const TExprNode::TPtr& item: value->Children()) { + if (!EnsureAtom(*item, ctx)) { + return false; + } + sortMembers.push_back(TString{item->Content()}); + } + } else if (name->Content() == RowSpecAttrSortDirections) { + if (!EnsureTuple(*value, ctx)) { + return false; + } + for (const TExprNode::TPtr& item: value->Children()) { + if (!EnsureCallable(*item, ctx)) { + return false; + } + if (!item->IsCallable(TCoBool::CallableName())) { + ctx.AddError(TIssue(ctx.GetPosition(item->Pos()), TStringBuilder() << "Expected " << TCoBool::CallableName() + << ", but got " << item->Content())); + return false; + } + } + sortDirectionsCount = value->Children().size(); + } else if (name->Content() == RowSpecAttrSortedByTypes) { + if (!EnsureTuple(*value, ctx)) { + return false; + } + for (const TExprNode::TPtr& item: value->Children()) { + if (!EnsureType(*item, ctx)) { + return false; + } + sortedByTypes.push_back(item->GetTypeAnn()->Cast<TTypeExprType>()->GetType()); + } + } else if (name->Content() == RowSpecAttrDefaultValues) { + if (!EnsureTupleMinSize(*value, 1, ctx)) { + return false; + } + for (const TExprNode::TPtr& item: value->Children()) { + if (!EnsureTupleSize(*item, 2, ctx)) { + return false; + } + for (const TExprNode::TPtr& atom: item->Children()) { + if (!EnsureAtom(*atom, ctx)) { + return false; + } + } + if (!defaultNames.insert(item->Child(0)->Content()).second) { + ctx.AddError(TIssue(ctx.GetPosition(item->Child(0)->Pos()), TStringBuilder() << "Duplicate " + << TString{RowSpecAttrDefaultValues}.Quote() << " key: " << item->Child(0)->Content())); + return false; + } + } + } else if (name->Content() == RowSpecAttrExplicitYson) { + if (!EnsureTuple(*value, ctx)) { + return false; + } + for (const TExprNode::TPtr& item: value->Children()) { + if (!EnsureAtom(*item, ctx)) { + return false; + } + explicitYson.emplace_back(item->Content()); + } + } else if (name->Content() == RowSpecAttrConstraints) { + if (!EnsureAtom(*value, ctx)) { + return false; + } + } else { + ctx.AddError(TIssue(ctx.GetPosition(child->Pos()), TStringBuilder() << "Unsupported " + << TYqlRowSpec::CallableName() << " option: " << name->Content())); + return false; + } + } + if (!type) { + ctx.AddError(TIssue(ctx.GetPosition(node.Pos()), TStringBuilder() << TString{RowSpecAttrType}.Quote() + << " option is mandatory for " << TYqlRowSpec::CallableName())); + return false; + } + if (sortedBy.size() != sortDirectionsCount) { + ctx.AddError(TIssue(ctx.GetPosition(node.Pos()), TStringBuilder() << TString{RowSpecAttrSortDirections}.Quote() + << " should have the same size as " << TString{RowSpecAttrSortedBy}.Quote())); + return false; + } + if (sortedBy.size() != sortedByTypes.size()) { + ctx.AddError(TIssue(ctx.GetPosition(node.Pos()), TStringBuilder() << TString{RowSpecAttrSortedByTypes}.Quote() + << " should have the same size as " << TString{RowSpecAttrSortedBy}.Quote())); + return false; + } + if (sortMembers.size() > sortedBy.size()) { + ctx.AddError(TIssue(ctx.GetPosition(node.Pos()), TStringBuilder() << TString{RowSpecAttrSortMembers}.Quote() + << " should have the size not greater than " << TString{RowSpecAttrSortedBy}.Quote() << " size")); + return false; + } + for (auto& field: sortMembers) { + if (!type->FindItem(field)) { + ctx.AddError(TIssue(ctx.GetPosition(node.Pos()), TStringBuilder() << TString{RowSpecAttrSortMembers}.Quote() + << " uses unknown field " << field.Quote())); + return false; + } + } + for (size_t i: xrange(sortedBy.size())) { + if (auto ndx = type->FindItem(sortedBy[i])) { + if (!IsSameAnnotation(*type->GetItems()[*ndx]->GetItemType(), *sortedByTypes[i])) { + ctx.AddError(TIssue(ctx.GetPosition(node.Pos()), TStringBuilder() << TString{RowSpecAttrSortedByTypes}.Quote() + << " for " << sortedBy[i].Quote() << " field uses unequal type")); + return false; + } + } + } + for (auto& field: defaultNames) { + if (!type->FindItem(field)) { + ctx.AddError(TIssue(ctx.GetPosition(node.Pos()), TStringBuilder() << TString{RowSpecAttrDefaultValues}.Quote() + << " uses unknown field " << TString{field}.Quote())); + return false; + } + } + for (auto& field: explicitYson) { + if (!type->FindItem(field)) { + ctx.AddError(TIssue(ctx.GetPosition(node.Pos()), TStringBuilder() << TString{RowSpecAttrExplicitYson}.Quote() + << " uses unknown field " << field.Quote())); + return false; + } + } + if (!strict && extendNonStrict) { + auto items = type->GetItems(); + auto dictType = ctx.MakeType<TDictExprType>( + ctx.MakeType<TDataExprType>(EDataSlot::String), + ctx.MakeType<TDataExprType>(EDataSlot::String)); + items.push_back(ctx.MakeType<TItemExprType>(YqlOthersColumnName, dictType)); + if (columnOrder) { + columnOrder->AddColumn(TString(YqlOthersColumnName)); + } + type = ctx.MakeType<TStructExprType>(items); + } + + return true; +} + +bool TYqlRowSpecInfo::Validate(TExprContext& ctx, TPositionHandle positionHandle) { + auto pos = ctx.GetPosition(positionHandle); + if (SortedBy.size() != SortDirections.size()) { + ctx.AddError(TIssue(pos, TStringBuilder() << TString{RowSpecAttrSortDirections}.Quote() + << " should have the same size as " << TString{RowSpecAttrSortedBy}.Quote())); + return false; + } + if (SortedBy.size() != SortedByTypes.size()) { + ctx.AddError(TIssue(pos, TStringBuilder() << TString{RowSpecAttrSortedByTypes}.Quote() + << " should have the same size as " << TString{RowSpecAttrSortedBy}.Quote())); + return false; + } + if (SortMembers.size() > SortedBy.size()) { + ctx.AddError(TIssue(pos, TStringBuilder() << TString{RowSpecAttrSortMembers}.Quote() + << " should have the size not greater than " << TString{RowSpecAttrSortedBy}.Quote() << " size")); + return false; + } + for (auto& field: SortMembers) { + if (!Type->FindItem(field)) { + ctx.AddError(TIssue(pos, TStringBuilder() << TString{RowSpecAttrSortMembers}.Quote() + << " uses unknown field " << field.Quote())); + return false; + } + } + for (size_t i: xrange(SortedBy.size())) { + if (auto ndx = Type->FindItem(SortedBy[i])) { + if (!IsSameAnnotation(*Type->GetItems()[*ndx]->GetItemType(), *SortedByTypes[i])) { + ctx.AddError(TIssue(pos, TStringBuilder() << TString{RowSpecAttrSortedByTypes}.Quote() + << " for " << SortedBy[i].Quote() << " field uses unequal type")); + return false; + } + } + } + for (auto& field: DefaultValues) { + if (!Type->FindItem(field.first)) { + ctx.AddError(TIssue(pos, TStringBuilder() << TString{RowSpecAttrDefaultValues}.Quote() + << " uses unknown field " << field.first.Quote())); + return false; + } + } + return true; +} + +void TYqlRowSpecInfo::Parse(NNodes::TExprBase node, bool withTypes) { + *this = {}; + FromNode = node; + for (auto child: node.Cast<TYqlRowSpec>()) { + auto setting = child.Cast<TCoNameValueTuple>(); + + if (setting.Name().Value() == RowSpecAttrNativeYtTypeFlags) { + NativeYtTypeFlags = FromString<ui64>(setting.Value().Cast<TCoAtom>().Value()); + } else if (setting.Name().Value() == RowSpecAttrUseNativeYtTypes) { + NativeYtTypeFlags = FromString<bool>(setting.Value().Cast<TCoAtom>().Value()) ? NTCF_LEGACY : NTCF_NONE; + } else if (setting.Name().Value() == RowSpecAttrUseTypeV2) { + NativeYtTypeFlags = FromString<bool>(setting.Value().Cast<TCoAtom>().Value()) ? NTCF_LEGACY : NTCF_NONE; + } else if (setting.Name().Value() == RowSpecAttrStrictSchema) { + StrictSchema = FromString<bool>(setting.Value().Cast<TCoAtom>().Value()); + } else if (setting.Name().Value() == RowSpecAttrUniqueKeys) { + UniqueKeys = FromString<bool>(setting.Value().Cast<TCoAtom>().Value()); + } else if (setting.Name().Value() == RowSpecAttrType) { + auto& val = setting.Value().Cast().Ref(); + if (withTypes) { + if (val.Type() == TExprNode::Atom) { + TypeNode = NYT::NodeFromYsonString(val.Content()); + Columns = TColumnOrder(NCommon::ExtractColumnOrderFromYsonStructType(TypeNode)); + } + Type = node.Ref().GetTypeAnn()->Cast<TStructExprType>(); + } + } else if (setting.Name().Value() == RowSpecAttrConstraints) { + ConstraintsNode = NYT::NodeFromYsonString(setting.Value().Cast().Ref().Content()); + } else if (setting.Name().Value() == RowSpecAttrSortedBy) { + for (auto item: setting.Value().Cast<TCoAtomList>()) { + SortedBy.push_back(TString{item.Value()}); + } + } else if (setting.Name().Value() == RowSpecAttrSortMembers) { + for (auto item: setting.Value().Cast<TCoAtomList>()) { + SortMembers.push_back(TString{item.Value()}); + } + } else if (setting.Name().Value() == RowSpecAttrSortDirections) { + for (auto item: setting.Value().Cast<TExprList>()) { + SortDirections.push_back(TStringBuf("true") == item.Cast<TCoBool>().Literal().Value()); + } + } else if (setting.Name().Value() == RowSpecAttrSortedByTypes) { + for (auto item: setting.Value().Cast<TExprList>()) { + SortedByTypes.push_back(withTypes ? item.Ref().GetTypeAnn()->Cast<TTypeExprType>()->GetType() : nullptr); + } + } else if (setting.Name().Value() == RowSpecAttrDefaultValues) { + for (auto item: setting.Value().Cast<TExprList>()) { + auto atomList = item.Cast<TCoAtomList>(); + DefaultValues[TString{atomList.Item(0).Value()}] = TString{atomList.Item(1).Value()}; + } + } else if (setting.Name().Value() == RowSpecAttrExplicitYson) { + for (auto item: setting.Value().Cast<TCoAtomList>()) { + ExplicitYson.emplace_back(item.Value()); + } + } else { + YQL_ENSURE(false, "Unexpected option " << setting.Name().Value()); + } + } + if (Columns && !StrictSchema) { + Columns->AddColumn(TString(YqlOthersColumnName)); + } +} + +ui64 TYqlRowSpecInfo::GetNativeYtTypeFlags(const NCommon::TStructMemberMapper& mapper) const { + return mapper ? (NativeYtTypeFlags & NYql::GetNativeYtTypeFlags(*Type, mapper)) : NativeYtTypeFlags; +} + +NYT::TNode TYqlRowSpecInfo::GetTypeNode(const NCommon::TStructMemberMapper& mapper) const { + if (!TypeNode.IsUndefined()) { + if (!mapper) { + return TypeNode; + } + YQL_ENSURE(TypeNode.IsList() && TypeNode.Size() == 2 && TypeNode[0].IsString() && TypeNode[0].AsString() == "StructType" && TypeNode[1].IsList()); + + NYT::TNode members = NYT::TNode::CreateList(); + for (auto& member : TypeNode[1].AsList()) { + YQL_ENSURE(member.IsList() && member.Size() == 2 && member[0].IsString()); + + if (auto name = mapper(member[0].AsString())) { + members.Add(NYT::TNode::CreateList().Add(*name).Add(member[1])); + } + } + return NYT::TNode::CreateList().Add("StructType").Add(members); + } + NYT::TNode typeNode; + NYT::TNodeBuilder nodeBuilder(&typeNode); + NCommon::SaveStructTypeToYson(nodeBuilder, Type, Columns, mapper); + return typeNode; +} + +void TYqlRowSpecInfo::SetType(const TStructExprType* type, TMaybe<ui64> nativeYtTypeFlags) { + Type = type; + Columns = {}; + TypeNode = {}; + if (nativeYtTypeFlags) { + NativeYtTypeFlags = *nativeYtTypeFlags; + } + NativeYtTypeFlags &= NYql::GetNativeYtTypeFlags(*Type); +} + +void TYqlRowSpecInfo::SetColumnOrder(const TMaybe<TColumnOrder>& columns) { + TypeNode = {}; + Columns = columns; +} + +TString TYqlRowSpecInfo::ToYsonString() const { + NYT::TNode attrs = NYT::TNode::CreateMap(); + FillCodecNode(attrs[YqlRowSpecAttribute]); + return NYT::NodeToCanonicalYsonString(attrs); +} + +void TYqlRowSpecInfo::CopyTypeOrders(const NYT::TNode& typeNode) { + YQL_ENSURE(Type); + if (!TypeNode.IsUndefined() || 0 == NativeYtTypeFlags) { + return; + } + + YQL_ENSURE(typeNode.IsList() && typeNode.Size() == 2 && typeNode[0].IsString() && typeNode[0].AsString() == "StructType" && typeNode[1].IsList()); + + THashMap<TString, NYT::TNode> fromMembers; + for (auto& member : typeNode[1].AsList()) { + YQL_ENSURE(member.IsList() && member.Size() == 2 && member[0].IsString()); + + fromMembers.emplace(member[0].AsString(), member[1]); + } + + NYT::TNode members = NYT::TNode::CreateList(); + TColumnOrder columns; + if (Columns.Defined() && Columns->Size() == Type->GetSize()) { + columns = *Columns; + } else { + for (auto& item : Type->GetItems()) { + columns.AddColumn(TString(item->GetName())); + } + } + for (auto& [name, gen_name]: columns) { + if (!StrictSchema && name == YqlOthersColumnName) { + continue; + } + auto origType = Type->FindItemType(name); + YQL_ENSURE(origType); + auto origTypeNode = NCommon::TypeToYsonNode(origType); + auto it = fromMembers.find(name); + if (it == fromMembers.end() || !NCommon::EqualsYsonTypesIgnoreStructOrder(origTypeNode, it->second)) { + members.Add(NYT::TNode::CreateList().Add(name).Add(origTypeNode)); + } else { + members.Add(NYT::TNode::CreateList().Add(name).Add(it->second)); + } + } + + TypeNode = NYT::TNode::CreateList().Add("StructType").Add(members); +} + +void TYqlRowSpecInfo::FillTypeTransform(NYT::TNode& attrs, TStringBuf typeNameAttr, const NCommon::TStructMemberMapper& mapper) const { + if (!TypeNode.IsUndefined()) { + YQL_ENSURE(TypeNode.IsList() && TypeNode.Size() == 2 && TypeNode[0].IsString() && TypeNode[0].AsString() == "StructType" && TypeNode[1].IsList()); + + NYT::TNode members = NYT::TNode::CreateList(); + for (auto& member : TypeNode[1].AsList()) { + YQL_ENSURE(member.IsList() && member.Size() == 2 && member[0].IsString()); + + if (auto name = mapper(member[0].AsString())) { + members.Add(NYT::TNode::CreateList().Add(*name).Add(member[1])); + } + } + attrs[typeNameAttr] = NYT::TNode::CreateList().Add("StructType").Add(members); + + } else { + NYT::TNodeBuilder specAttrBuilder(&attrs[typeNameAttr]); + NCommon::SaveStructTypeToYson(specAttrBuilder, Type, Columns, mapper); + } +} + +void TYqlRowSpecInfo::FillSort(NYT::TNode& attrs, const NCommon::TStructMemberMapper& mapper) const { + TVector<bool> sortDirections; + TVector<TString> sortedBy; + TVector<TString> sortMembers; + TTypeAnnotationNode::TListType sortedByTypes; + + bool curUniqueKeys = UniqueKeys; + const TVector<bool>* curSortDirections = &SortDirections; + const TVector<TString>* curSortedBy = &SortedBy; + const TVector<TString>* curSortMembers = &SortMembers; + const TTypeAnnotationNode::TListType* curSortedByTypes = &SortedByTypes; + if (mapper) { + sortDirections = SortDirections; + sortedBy = SortedBy; + sortMembers = SortMembers; + sortedByTypes = SortedByTypes; + + curSortDirections = &sortDirections; + curSortedBy = &sortedBy; + curSortMembers = &sortMembers; + curSortedByTypes = &sortedByTypes; + for (size_t i = 0; i < sortedBy.size(); ++i) { + if (Type->FindItem(sortedBy[i])) { + if (auto name = mapper(sortedBy[i])) { + sortedBy[i] = TString{*name}; + if (i < sortMembers.size()) { + sortMembers[i] = sortedBy[i]; + } + } else { + if (i < sortMembers.size()) { + sortMembers.erase(sortMembers.begin() + i, sortMembers.end()); + } + sortedBy.erase(sortedBy.begin() + i, sortedBy.end()); + sortedByTypes.erase(sortedByTypes.begin() + i, sortedByTypes.end()); + sortDirections.erase(sortDirections.begin() + i, sortDirections.end()); + curUniqueKeys = false; + break; + } + } + } + } + if (!curSortedBy->empty()) { + attrs[RowSpecAttrUniqueKeys] = curUniqueKeys; + } + + if (!curSortDirections->empty()) { + auto list = NYT::TNode::CreateList(); + for (bool dir: *curSortDirections) { + list.Add(dir ? 1 : 0); + } + attrs[RowSpecAttrSortDirections] = list; + } + + auto saveColumnList = [&attrs] (TStringBuf name, const TVector<TString>& columns) { + if (!columns.empty()) { + auto list = NYT::TNode::CreateList(); + for (const auto& item : columns) { + list.Add(item); + } + attrs[name] = list; + } + }; + + saveColumnList(RowSpecAttrSortMembers, *curSortMembers); + saveColumnList(RowSpecAttrSortedBy, *curSortedBy); + + if (!curSortedByTypes->empty()) { + auto list = NYT::TNode::CreateList(); + for (auto type: *curSortedByTypes) { + list.Add(NCommon::TypeToYsonNode(type)); + } + attrs[RowSpecAttrSortedByTypes] = list; + } +} + +void TYqlRowSpecInfo::FillDefValues(NYT::TNode& attrs, const NCommon::TStructMemberMapper& mapper) const { + if (!DefaultValues.empty()) { + auto map = NYT::TNode::CreateMap(); + if (mapper) { + for (const auto& val: DefaultValues) { + if (auto name = mapper(val.first)) { + map[*name] = NYT::NodeToYsonString(NYT::TNode(val.second)); + } + } + } else { + for (const auto& val: DefaultValues) { + map[val.first] = NYT::NodeToYsonString(NYT::TNode(val.second)); + } + } + if (!map.AsMap().empty()) { + attrs[RowSpecAttrDefaultValues] = map; + } + } +} + +void TYqlRowSpecInfo::FillFlags(NYT::TNode& attrs) const { + attrs[RowSpecAttrStrictSchema] = StrictSchema; + attrs[RowSpecAttrNativeYtTypeFlags] = NativeYtTypeFlags; + // Backward compatibility. TODO: remove after releasing compatibility flags + if (NativeYtTypeFlags != 0) { + attrs[RowSpecAttrUseNativeYtTypes] = true; + } +} + +void TYqlRowSpecInfo::FillExplicitYson(NYT::TNode& attrs, const NCommon::TStructMemberMapper& mapper) const { + TVector<TString> localExplicitYson; + const TVector<TString>* curExplicitYson = &ExplicitYson; + if (mapper) { + for (size_t i = 0; i < ExplicitYson.size(); ++i) { + if (Type->FindItem(ExplicitYson[i])) { + if (auto name = mapper(ExplicitYson[i])) { + localExplicitYson.emplace_back(TString{*name}); + } + } + } + curExplicitYson = &localExplicitYson; + } + if (!curExplicitYson->empty()) { + auto list = NYT::TNode::CreateList(); + for (const auto& item : *curExplicitYson) { + list.Add(item); + } + attrs[RowSpecAttrExplicitYson] = list; + } +} + +void TYqlRowSpecInfo::FillCodecNode(NYT::TNode& attrs, const NCommon::TStructMemberMapper& mapper) const { + attrs = NYT::TNode::CreateMap(); + + attrs[RowSpecAttrType] = GetTypeNode(mapper); + FillSort(attrs, mapper); + FillDefValues(attrs, mapper); + FillFlags(attrs); + FillExplicitYson(attrs, mapper); +} + +void TYqlRowSpecInfo::FillAttrNode(NYT::TNode& attrs, ui64 nativeTypeCompatibility, bool useCompactForm) const { + attrs = NYT::TNode::CreateMap(); + + if (!useCompactForm) { + auto otherFilter = [strict = StrictSchema](TStringBuf name) -> TMaybe<TStringBuf> { + if (!strict && name == YqlOthersColumnName) { + return Nothing(); + } + return MakeMaybe(name); + }; + attrs[RowSpecAttrType] = GetTypeNode(otherFilter); + } + + THashSet<TStringBuf> patchedFields; + for (auto item: Type->GetItems()) { + const TTypeAnnotationNode* itemType = item->GetItemType(); + // Top-level strict Yson is converted to Yson? in YT schema + if (itemType->GetKind() == ETypeAnnotationKind::Data && itemType->Cast<TDataExprType>()->GetSlot() == EDataSlot::Yson) { + patchedFields.insert(item->GetName()); + } else { + if (itemType->GetKind() == ETypeAnnotationKind::Optional) { + itemType = itemType->Cast<TOptionalExprType>()->GetItemType(); + } + auto flags = GetNativeYtTypeFlagsImpl(itemType); + if (flags != (flags & NativeYtTypeFlags & nativeTypeCompatibility)) { + patchedFields.insert(item->GetName()); + } + } + } + + attrs[RowSpecAttrTypePatch] = GetTypeNode([&patchedFields](TStringBuf name) -> TMaybe<TStringBuf> { + return patchedFields.contains(name) ? MakeMaybe(name) : Nothing(); + }); + + if (!useCompactForm || HasAuxColumns() || AnyOf(SortedBy, [&patchedFields](const auto& name) { return patchedFields.contains(name); } )) { + FillSort(attrs); + } + FillDefValues(attrs); + FillFlags(attrs); + FillConstraints(attrs); +} + +NNodes::TExprBase TYqlRowSpecInfo::ToExprNode(TExprContext& ctx, const TPositionHandle& pos) const { + auto rowSpecBuilder = Build<TYqlRowSpec>(ctx, pos); + + auto otherFilter = [strict = StrictSchema](TStringBuf name) -> TMaybe<TStringBuf> { + if (!strict && name == YqlOthersColumnName) { + return Nothing(); + } + return MakeMaybe(name); + }; + rowSpecBuilder + .Add() + .Name() + .Value(RowSpecAttrNativeYtTypeFlags, TNodeFlags::Default) + .Build() + .Value<TCoAtom>() + .Value(ToString(NativeYtTypeFlags), TNodeFlags::Default) + .Build() + .Build() + .Add() + .Name() + .Value(RowSpecAttrStrictSchema, TNodeFlags::Default) + .Build() + .Value<TCoAtom>() + .Value(StrictSchema ? TStringBuf("1") : TStringBuf("0"), TNodeFlags::Default) + .Build() + .Build() + .Add() + .Name() + .Value(RowSpecAttrUniqueKeys, TNodeFlags::Default) + .Build() + .Value<TCoAtom>() + .Value(UniqueKeys ? TStringBuf("1") : TStringBuf("0"), TNodeFlags::Default) + .Build() + .Build() + .Add() + .Name() + .Value(RowSpecAttrType, TNodeFlags::Default) + .Build() + .Value<TCoAtom>() + .Value(NYT::NodeToYsonString(GetTypeNode(otherFilter), NYson::EYsonFormat::Text), TNodeFlags::MultilineContent) + .Build() + .Build(); + + if (ConstraintsNode.HasValue() || HasNonTrivialSort() || Unique || Distinct) { + rowSpecBuilder.Add() + .Name() + .Value(RowSpecAttrConstraints, TNodeFlags::Default) + .Build() + .Value<TCoAtom>() + .Value(NYT::NodeToYsonString(GetConstraintsNode(), NYson::EYsonFormat::Text), TNodeFlags::MultilineContent) + .Build() + .Build(); + } + + if (!SortDirections.empty()) { + auto listBuilder = Build<TExprList>(ctx, pos); + for (bool dir: SortDirections) { + listBuilder.Add<TCoBool>() + .Literal<TCoAtom>() + .Value(dir ? TStringBuf("true") : TStringBuf("false"), TNodeFlags::Default) + .Build() + .Build(); + } + rowSpecBuilder + .Add() + .Name() + .Value(RowSpecAttrSortDirections, TNodeFlags::Default) + .Build() + .Value(listBuilder.Done()) + .Build(); + } + + auto saveColumnList = [&] (TStringBuf name, const TVector<TString>& columns) { + if (!columns.empty()) { + auto listBuilder = Build<TExprList>(ctx, pos); + for (auto& column: columns) { + listBuilder.Add<TCoAtom>() + .Value(column) + .Build(); + } + rowSpecBuilder + .Add() + .Name() + .Value(name, TNodeFlags::Default) + .Build() + .Value(listBuilder.Done()) + .Build(); + } + }; + + saveColumnList(RowSpecAttrSortMembers, SortMembers); + saveColumnList(RowSpecAttrSortedBy, SortedBy); + + if (!SortedByTypes.empty()) { + auto listBuilder = Build<TExprList>(ctx, pos); + for (auto type: SortedByTypes) { + listBuilder.Add(TExprBase(NCommon::BuildTypeExpr(pos, *type, ctx))); + } + rowSpecBuilder + .Add() + .Name() + .Value(RowSpecAttrSortedByTypes, TNodeFlags::Default) + .Build() + .Value(listBuilder.Done()) + .Build(); + } + + if (!DefaultValues.empty()) { + auto listBuilder = Build<TExprList>(ctx, pos); + for (const auto& val: DefaultValues) { + listBuilder.Add<TCoAtomList>() + .Add().Value(val.first).Build() + .Add().Value(val.second).Build() + .Build(); + } + rowSpecBuilder + .Add() + .Name() + .Value(RowSpecAttrDefaultValues, TNodeFlags::Default) + .Build() + .Value(listBuilder.Done()) + .Build(); + } + saveColumnList(RowSpecAttrExplicitYson, ExplicitYson); + + return rowSpecBuilder.Done(); +} + +bool TYqlRowSpecInfo::HasAuxColumns() const { + for (auto& x: SortedBy) { + if (!Type->FindItem(x)) { + return true; + } + } + return false; +} + +TVector<std::pair<TString, const TTypeAnnotationNode*>> TYqlRowSpecInfo::GetAuxColumns() const { + TVector<std::pair<TString, const TTypeAnnotationNode*>> res; + for (size_t i: xrange(SortedBy.size())) { + if (!Type->FindItem(SortedBy[i])) { + res.emplace_back(SortedBy[i], SortedByTypes[i]); + } + } + return res; +} + +const TStructExprType* TYqlRowSpecInfo::GetExtendedType(TExprContext& ctx) const { + if (!IsSorted()) { + return Type; + } + bool extended = false; + TVector<const TItemExprType*> items = Type->GetItems(); + for (size_t i: xrange(SortedBy.size())) { + if (!Type->FindItem(SortedBy[i])) { + items.push_back(ctx.MakeType<TItemExprType>(SortedBy[i], SortedByTypes[i])); + extended = true; + } + } + return extended ? ctx.MakeType<TStructExprType>(items) : Type; +} + +bool TYqlRowSpecInfo::CopySortness(TExprContext& ctx, const TYqlRowSpecInfo& from, ECopySort mode) { + SortDirections = from.SortDirections; + SortMembers = from.SortMembers; + SortedBy = from.SortedBy; + SortedByTypes = from.SortedByTypes; + UniqueKeys = from.UniqueKeys; + bool sortIsChanged = false; + if (ECopySort::Exact != mode) { + YQL_ENSURE(SortMembers.size() <= SortedBy.size()); + for (size_t i = 0; i < SortMembers.size(); ++i) { + const auto itemNdx = Type->FindItem(SortMembers[i]); + if (!itemNdx || (SortedBy[i] == SortMembers[i] && Type->GetItems()[*itemNdx]->GetItemType() != SortedByTypes[i])) { + sortIsChanged = ClearSortness(ctx, i); + break; + } else if (ECopySort::Pure == mode && SortedBy[i] != SortMembers[i]) { + sortIsChanged = ClearSortness(ctx, i); + break; + } + } + if (ECopySort::WithCalc != mode) { + if (SortMembers.size() < SortedBy.size()) { + sortIsChanged = ClearSortness(ctx, SortMembers.size()) || sortIsChanged; + } + } + } + return sortIsChanged; +} + +void TYqlRowSpecInfo::CopyConstraints(const TYqlRowSpecInfo& from) { + ConstraintsNode = from.ConstraintsNode; + Sorted = from.Sorted; + Unique = from.Unique; + Distinct = from.Distinct; +} + +bool TYqlRowSpecInfo::KeepPureSortOnly(TExprContext& ctx) { + bool sortIsChanged = false; + for (size_t i = 0; i < SortMembers.size(); ++i) { + if (!Type->FindItem(SortMembers[i])) { + sortIsChanged = ClearSortness(ctx, i); + break; + } else if (SortedBy[i] != SortMembers[i]) { + sortIsChanged = ClearSortness(ctx, i); + break; + } + } + if (SortMembers.size() < SortedBy.size()) { + sortIsChanged = ClearSortness(ctx, SortMembers.size()) || sortIsChanged; + } + return sortIsChanged; +} + +bool TYqlRowSpecInfo::ClearNativeDescendingSort(TExprContext& ctx) { + for (size_t i = 0; i < SortDirections.size(); ++i) { + if (!SortDirections[i] && Type->FindItem(SortedBy[i])) { + return ClearSortness(ctx, i); + } + } + return false; +} + +bool TYqlRowSpecInfo::MakeCommonSortness(TExprContext& ctx, const TYqlRowSpecInfo& from) { + bool sortIsChanged = false; + UniqueKeys = false; // Merge of two and more tables cannot have unique keys + const size_t resultSize = Min<size_t>(SortMembers.size(), from.SortMembers.size()); // Truncate all calculated columns + if (SortedBy.size() > resultSize) { + sortIsChanged = ClearSortness(ctx, resultSize); + } + for (size_t i = 0; i < resultSize; ++i) { + if (SortMembers[i] != from.SortMembers[i] || SortedBy[i] != from.SortedBy[i] || SortedByTypes[i] != from.SortedByTypes[i] || SortDirections[i] != from.SortDirections[i]) { + sortIsChanged = ClearSortness(ctx, i) || sortIsChanged; + break; + } + } + return sortIsChanged; +} + +bool TYqlRowSpecInfo::CompareSortness(const TYqlRowSpecInfo& with, bool checkUniqueFlag) const { + return SortDirections == with.SortDirections + && SortMembers == with.SortMembers + && SortedBy == with.SortedBy + && SortedByTypes.size() == with.SortedByTypes.size() + && std::equal(SortedByTypes.cbegin(), SortedByTypes.cend(), with.SortedByTypes.cbegin(), TTypeAnnotationNode::TEqual()) + && (!checkUniqueFlag || UniqueKeys == with.UniqueKeys); +} + +bool TYqlRowSpecInfo::ClearSortness(TExprContext& ctx, size_t fromMember) { + if (fromMember <= SortMembers.size()) { + SortMembers.erase(SortMembers.begin() + fromMember, SortMembers.end()); + SortedBy.erase(SortedBy.begin() + fromMember, SortedBy.end()); + SortedByTypes.erase(SortedByTypes.begin() + fromMember, SortedByTypes.end()); + SortDirections.erase(SortDirections.begin() + fromMember, SortDirections.end()); + UniqueKeys = false; + ParseConstraintsNode(ctx); + ConstraintsNode.Clear(); + Sorted = MakeSortConstraint(ctx); + return true; + } + return false; +} + +const TSortedConstraintNode* TYqlRowSpecInfo::MakeSortConstraint(TExprContext& ctx) const { + if (!SortMembers.empty()) { + TSortedConstraintNode::TContainerType sorted; + for (auto i = 0U; i < SortMembers.size(); ++i) { + const auto column = ctx.AppendString(SortMembers[i]); + sorted.emplace_back(TPartOfConstraintBase::TSetType{TPartOfConstraintBase::TPathType{column}}, i >= SortDirections.size() || SortDirections[i]); + } + return ctx.MakeConstraint<TSortedConstraintNode>(std::move(sorted)); + } + return nullptr; +} + +const TDistinctConstraintNode* TYqlRowSpecInfo::MakeDistinctConstraint(TExprContext& ctx) const { + if (UniqueKeys && !SortMembers.empty() && SortedBy.size() == SortMembers.size()) { + std::vector<std::string_view> uniqColumns(SortMembers.size()); + std::transform(SortMembers.cbegin(), SortMembers.cend(), uniqColumns.begin(), std::bind(&TExprContext::AppendString, std::ref(ctx), std::placeholders::_1)); + return ctx.MakeConstraint<TDistinctConstraintNode>(uniqColumns); + } + return nullptr; +} + +TVector<std::pair<TString, bool>> TYqlRowSpecInfo::GetForeignSort() const { + TVector<std::pair<TString, bool>> res; + for (size_t i = 0; i < SortedBy.size(); ++i) { + res.emplace_back(SortedBy[i], Type->FindItem(SortedBy[i]) ? SortDirections.at(i) : true); + } + return res; +} + +void TYqlRowSpecInfo::SetConstraints(const TConstraintSet& constraints) { + ConstraintsNode.Clear(); + Sorted = constraints.GetConstraint<TSortedConstraintNode>(); + Unique = constraints.GetConstraint<TUniqueConstraintNode>(); + Distinct = constraints.GetConstraint<TDistinctConstraintNode>(); +} + +TConstraintSet TYqlRowSpecInfo::GetConstraints() const { + TConstraintSet set; + if (Sorted) + set.AddConstraint(Sorted); + if (Unique) + set.AddConstraint(Unique); + if (Distinct) + set.AddConstraint(Distinct); + return set; +} + +} diff --git a/yt/yql/providers/yt/lib/row_spec/yql_row_spec.h b/yt/yql/providers/yt/lib/row_spec/yql_row_spec.h new file mode 100644 index 0000000000..049b67d2a6 --- /dev/null +++ b/yt/yql/providers/yt/lib/row_spec/yql_row_spec.h @@ -0,0 +1,155 @@ +#pragma once + +#include <yql/essentials/providers/common/schema/expr/yql_expr_schema.h> +#include <yql/essentials/core/expr_nodes_gen/yql_expr_nodes_gen.h> +#include <yql/essentials/ast/yql_expr.h> +#include <yql/essentials/ast/yql_constraint.h> +#include <yql/essentials/core/yql_type_annotation.h> + +#include <library/cpp/yson/node/node.h> + +#include <util/generic/string.h> +#include <util/generic/strbuf.h> +#include <util/generic/vector.h> +#include <util/generic/ptr.h> +#include <util/generic/map.h> +#include <util/generic/hash_set.h> +#include <util/generic/maybe.h> + +#include <functional> + +namespace NYql { + +class TStructExprType; +struct TYTSortInfo; + +struct TYqlRowSpecInfo: public TThrRefBase { + using TPtr = TIntrusivePtr<TYqlRowSpecInfo>; + + enum class ECopySort { + Pure, + WithDesc, + WithCalc, + Exact, + }; + + TYqlRowSpecInfo() { + } + + TYqlRowSpecInfo(NNodes::TExprBase node, bool withTypes = true) { + Parse(node, withTypes); + } + + static bool Validate(const TExprNode& node, TExprContext& ctx, const TStructExprType*& type, TMaybe<TColumnOrder>& columnOrder); + bool Parse(const TString& rowSpecYson, TExprContext& ctx, const TPositionHandle& pos = {}); + bool Parse(const THashMap<TString, TString>& attrs, TExprContext& ctx, const TPositionHandle& pos = {}); + bool Parse(const NYT::TNode& rowSpecAttr, TExprContext& ctx, const TPositionHandle& pos = {}); + void Parse(NNodes::TExprBase node, bool withTypes = true); + bool Validate(TExprContext& ctx, TPositionHandle pos); + + TString ToYsonString() const; + void FillCodecNode(NYT::TNode& attrs, const NCommon::TStructMemberMapper& mapper = {}) const; + void FillAttrNode(NYT::TNode& attrs, ui64 nativeTypeCompatibility, bool useCompactForm) const; + NNodes::TExprBase ToExprNode(TExprContext& ctx, const TPositionHandle& pos) const; + + bool IsSorted() const { + return !SortedBy.empty(); + } + bool HasAuxColumns() const; + TVector<std::pair<TString, const TTypeAnnotationNode*>> GetAuxColumns() const; + // Includes aux columns + const TStructExprType* GetExtendedType(TExprContext& ctx) const; + // Returns true if sortness is changed + bool CopySortness(TExprContext& ctx, const TYqlRowSpecInfo& from, ECopySort mode = ECopySort::Pure); + // Returns true if sortness is changed + bool MakeCommonSortness(TExprContext& ctx, const TYqlRowSpecInfo& from); + bool CompareSortness(const TYqlRowSpecInfo& with, bool checkUniqueFlag = true) const; + // Returns true if sortness is changed + bool ClearSortness(TExprContext& ctx, size_t fromMember = 0); + // Returns true if sortness is changed + bool KeepPureSortOnly(TExprContext& ctx); + bool ClearNativeDescendingSort(TExprContext& ctx); + const TSortedConstraintNode* MakeSortConstraint(TExprContext& ctx) const; + const TDistinctConstraintNode* MakeDistinctConstraint(TExprContext& ctx) const; + void CopyConstraints(const TYqlRowSpecInfo& from); + + const TStructExprType* GetType() const { + return Type; + } + + NYT::TNode GetTypeNode(const NCommon::TStructMemberMapper& mapper = {}) const; + + const TMaybe<TColumnOrder>& GetColumnOrder() const { + return Columns; + } + + void SetType(const TStructExprType* type, TMaybe<ui64> nativeYtTypeFlags = Nothing()); + void SetColumnOrder(const TMaybe<TColumnOrder>& columnOrder); + void SetConstraints(const TConstraintSet& constraints); + TConstraintSet GetConstraints() const; + TConstraintSet GetSomeConstraints(ui64 mask, TExprContext& ctx); + TConstraintSet GetAllConstraints(TExprContext& ctx); + + void CopyType(const TYqlRowSpecInfo& from) { + Type = from.Type; + Columns = from.Columns; + TypeNode = from.TypeNode; + NativeYtTypeFlags = from.NativeYtTypeFlags; + } + + void CopyTypeOrders(const NYT::TNode& typeNode); + + ui64 GetNativeYtTypeFlags(const NCommon::TStructMemberMapper& mapper = {}) const; + + TMaybe<NYT::TNode> GetNativeYtType(const NCommon::TStructMemberMapper& mapper = {}) const { + return 0 != GetNativeYtTypeFlags(mapper) ? MakeMaybe(GetTypeNode(mapper)) : Nothing(); + } + + TVector<std::pair<TString, bool>> GetForeignSort() const; + + NNodes::TMaybeNode<NNodes::TExprBase> FromNode; + + bool StrictSchema = true; + bool UniqueKeys = false; + + TVector<bool> SortDirections; + TVector<TString> SortedBy; + TVector<TString> SortMembers; + TTypeAnnotationNode::TListType SortedByTypes; + TMap<TString, TString> DefaultValues; + TVector<TString> ExplicitYson; + +private: + void FillTypeTransform(NYT::TNode& attrs, TStringBuf typeNameAttr, const NCommon::TStructMemberMapper& mapper) const; + void FillSort(NYT::TNode& attrs, const NCommon::TStructMemberMapper& mapper = {}) const; + void FillDefValues(NYT::TNode& attrs, const NCommon::TStructMemberMapper& mapper = {}) const; + void FillFlags(NYT::TNode& attrs) const; + void FillConstraints(NYT::TNode& attrs) const; + void FillExplicitYson(NYT::TNode& attrs, const NCommon::TStructMemberMapper& mapper) const; + bool ParsePatched(const NYT::TNode& rowSpecAttr, const THashMap<TString, TString>& attrs, TExprContext& ctx, const TPositionHandle& pos); + bool ParseFull(const NYT::TNode& rowSpecAttr, const THashMap<TString, TString>& attrs, TExprContext& ctx, const TPositionHandle& pos); + bool ParseType(const NYT::TNode& rowSpecAttr, TExprContext& ctx, const TPositionHandle& pos); + bool ParseSort(const NYT::TNode& rowSpecAttr, TExprContext& ctx, const TPositionHandle& pos); + void ParseFlags(const NYT::TNode& rowSpecAttr); + void ParseConstraints(const NYT::TNode& rowSpecAttr); + void ParseConstraintsNode(TExprContext& ctx); + void ParseDefValues(const NYT::TNode& rowSpecAttr); + bool ValidateSort(const TYTSortInfo& sortInfo, TExprContext& ctx, const TPositionHandle& pos); + + NYT::TNode GetConstraintsNode() const; + bool HasNonTrivialSort() const; +private: + const TStructExprType* Type = nullptr; + TMaybe<TColumnOrder> Columns; + NYT::TNode TypeNode; + ui64 NativeYtTypeFlags = 0ul; + + NYT::TNode ConstraintsNode; + const TSortedConstraintNode* Sorted = nullptr; + const TUniqueConstraintNode* Unique = nullptr; + const TDistinctConstraintNode* Distinct = nullptr; +}; + +ui64 GetNativeYtTypeFlags(const TStructExprType& type, const NCommon::TStructMemberMapper& mapper = {}); + +} diff --git a/yt/yql/providers/yt/lib/schema/schema.cpp b/yt/yql/providers/yt/lib/schema/schema.cpp new file mode 100644 index 0000000000..e3ff3bd24c --- /dev/null +++ b/yt/yql/providers/yt/lib/schema/schema.cpp @@ -0,0 +1,1092 @@ +#include "schema.h" + +#include <yt/yql/providers/yt/common/yql_names.h> +#include <yql/essentials/providers/common/codec/yql_codec_type_flags.h> +#include <yql/essentials/providers/common/schema/expr/yql_expr_schema.h> + +#include <yql/essentials/utils/log/log.h> +#include <yql/essentials/utils/yql_panic.h> + +#include <library/cpp/yson/node/node_io.h> +#include <library/cpp/yson/node/node_builder.h> + +#include <util/string/cast.h> +#include <util/generic/yexception.h> +#include <util/generic/hash_set.h> +#include <util/generic/set.h> +#include <util/generic/hash.h> + + +namespace NYql { + +static TString ConvertYtDataType(const TString& ytType, ui64& nativeYtTypeFlags) { + TString yqlType; + if (ytType == "string") { + yqlType = "String"; + } + else if (ytType == "uuid") { + nativeYtTypeFlags |= NTCF_UUID; + yqlType = "Uuid"; + } + else if (ytType == "utf8") { + yqlType = "Utf8"; + } + else if (ytType == "int64") { + yqlType = "Int64"; + } + else if (ytType == "uint64") { + yqlType = "Uint64"; + } + else if (ytType == "int32") { + yqlType = "Int32"; + } + else if (ytType == "uint32") { + yqlType = "Uint32"; + } + else if (ytType == "int16") { + yqlType = "Int16"; + } + else if (ytType == "uint16") { + yqlType = "Uint16"; + } + else if (ytType == "int8") { + yqlType = "Int8"; + } + else if (ytType == "uint8") { + yqlType = "Uint8"; + } + else if (ytType == "double") { + yqlType = "Double"; + } + else if (ytType == "float") { + nativeYtTypeFlags |= NTCF_FLOAT; + yqlType = "Float"; + } + else if (ytType == "boolean") { // V2 + yqlType = "Bool"; + } + else if (ytType == "bool") { // V3 + yqlType = "Bool"; + } + else if (ytType == "date") { + nativeYtTypeFlags |= NTCF_DATE; + yqlType = "Date"; + } + else if (ytType == "datetime") { + nativeYtTypeFlags |= NTCF_DATE; + yqlType = "Datetime"; + } + else if (ytType == "timestamp") { + nativeYtTypeFlags |= NTCF_DATE; + yqlType = "Timestamp"; + } + else if (ytType == "interval") { + nativeYtTypeFlags |= NTCF_DATE; + yqlType = "Interval"; + } + else if (ytType == "date32") { + nativeYtTypeFlags |= NTCF_BIGDATE; + yqlType = "Date32"; + } + else if (ytType == "datetime64") { + nativeYtTypeFlags |= NTCF_BIGDATE; + yqlType = "Datetime64"; + } + else if (ytType == "timestamp64") { + nativeYtTypeFlags |= NTCF_BIGDATE; + yqlType = "Timestamp64"; + } + else if (ytType == "interval64") { + nativeYtTypeFlags |= NTCF_BIGDATE; + yqlType = "Interval64"; + } + else if (ytType == "yson") { // V3 + yqlType = "Yson"; + } + else if (ytType == "json") { + nativeYtTypeFlags |= NTCF_JSON; + yqlType = "Json"; + } + else if (ytType == "any") { + yqlType = "Yson"; + } else { + YQL_LOG_CTX_THROW yexception() << "Unknown type " << ytType.Quote() << " in yson schema"; + } + + return yqlType; +} + +static NYT::TNode ConvertNativeYtType(const NYT::TNode& raw, bool root, bool& hasYson, ui64& nativeYtTypeFlags) { + if (raw.IsString()) { + if (raw.AsString() == "null") { + nativeYtTypeFlags |= NTCF_NULL; + return NYT::TNode().Add("NullType"); + } + + if (raw.AsString() == "void") { + nativeYtTypeFlags |= NTCF_VOID; + return NYT::TNode().Add("VoidType"); + } + + hasYson = hasYson || "yson" == raw.AsString(); + return NYT::TNode() + .Add("DataType") + .Add(ConvertYtDataType(raw.AsString(), nativeYtTypeFlags)); + } + + const auto& typeName = raw["type_name"].AsString(); + if (typeName == "decimal") { + nativeYtTypeFlags |= NTCF_DECIMAL; + return NYT::TNode() + .Add("DataType") + .Add("Decimal") + .Add(ToString(raw["precision"].AsInt64())) + .Add(ToString(raw["scale"].AsInt64())); + } else if (typeName == "list") { + nativeYtTypeFlags |= NTCF_COMPLEX; + return NYT::TNode() + .Add("ListType") + .Add(ConvertNativeYtType(raw["item"], false, hasYson, nativeYtTypeFlags)); + } else if (typeName == "optional") { + if (!root) { + nativeYtTypeFlags |= NTCF_COMPLEX; + } + return NYT::TNode() + .Add("OptionalType") + .Add(ConvertNativeYtType(raw["item"], false, hasYson, nativeYtTypeFlags)); + } else if (typeName == "tuple") { + nativeYtTypeFlags |= NTCF_COMPLEX; + auto list = NYT::TNode::CreateList(); + for (const auto& x : raw["elements"].AsList()) { + list.Add(ConvertNativeYtType(x["type"], false, hasYson, nativeYtTypeFlags)); + } + return NYT::TNode() + .Add("TupleType") + .Add(list); + } else if (typeName == "struct") { + nativeYtTypeFlags |= NTCF_COMPLEX; + auto list = NYT::TNode::CreateList(); + for (const auto& x : raw["members"].AsList()) { + list.Add(NYT::TNode() + .Add(x["name"].AsString()) + .Add(ConvertNativeYtType(x["type"], false, hasYson, nativeYtTypeFlags))); + } + return NYT::TNode() + .Add("StructType") + .Add(list); + } else if (typeName == "variant") { + nativeYtTypeFlags |= NTCF_COMPLEX; + auto list = NYT::TNode::CreateList(); + if (raw.HasKey("elements")) { + for (const auto& x : raw["elements"].AsList()) { + list.Add(ConvertNativeYtType(x["type"], false, hasYson, nativeYtTypeFlags)); + } + + return NYT::TNode() + .Add("VariantType") + .Add(NYT::TNode() + .Add("TupleType") + .Add(list)); + } else { + for (const auto& x : raw["members"].AsList()) { + list.Add(NYT::TNode() + .Add(x["name"].AsString()) + .Add(ConvertNativeYtType(x["type"], false, hasYson, nativeYtTypeFlags))); + } + + return NYT::TNode() + .Add("VariantType") + .Add(NYT::TNode() + .Add("StructType") + .Add(list)); + } + } else if (typeName == "tagged") { + nativeYtTypeFlags |= NTCF_COMPLEX; + auto tag = raw["tag"].AsString(); + if (tag == "_EmptyList") { + return NYT::TNode().Add("EmptyListType"); + } + + if (tag == "_EmptyDict") { + return NYT::TNode().Add("EmptyDictType"); + } + + if (tag == "_Void") { + return NYT::TNode().Add("VoidType"); + } + + if (tag == "_Null") { + return NYT::TNode().Add("NullType"); + } + + return NYT::TNode() + .Add("TaggedType") + .Add(tag) + .Add(ConvertNativeYtType(raw["item"], false, hasYson, nativeYtTypeFlags)); + } else if (typeName == "dict") { + nativeYtTypeFlags |= NTCF_COMPLEX; + return NYT::TNode() + .Add("DictType") + .Add(ConvertNativeYtType(raw["key"], false, hasYson, nativeYtTypeFlags)) + .Add(ConvertNativeYtType(raw["value"], false, hasYson, nativeYtTypeFlags)); + } else { + YQL_LOG_CTX_THROW yexception() << "Unknown metatype " << typeName.Quote() << " in yson schema"; + } +} + +static std::pair<TString, NYT::TNode> ExtractYtType(const NYT::TNode& entry, bool strictSchema, + THashSet<TString>& anyColumns, ui64& nativeYtTypeFlags) { + TMaybe<NYT::TNode> nativeTypeRaw; + TMaybe<TString> fieldName; + TMaybe<TString> fieldType; + bool nullable = true; + bool hasYson = false; + + for (const auto& it : entry.AsMap()) { + if (it.first == "name") { + fieldName = it.second.AsString(); + if (!strictSchema && *fieldName == YqlOthersColumnName) { + YQL_LOG_CTX_THROW yexception() << "Non-strict schema contains '_other' column, which conflicts with YQL virtual column"; + } + } + else if (it.first == "type") { + fieldType = it.second.AsString(); + hasYson = hasYson || ("any" == fieldType); + } + else if (it.first == "nullable") { + nullable = it.second.AsBool(); + } + else if (it.first == "required") { + nullable = !it.second.AsBool(); + } + else if (it.first == "type_v3") { + nativeTypeRaw = it.second; + } + } + + if (!fieldName) { + YQL_LOG_CTX_THROW yexception() << "No 'name' in schema tuple"; + } + if (!fieldType && !nativeTypeRaw) { + YQL_LOG_CTX_THROW yexception() << "No 'type' or 'type_v3' in schema tuple"; + } + + if (nativeTypeRaw) { + fieldType.Clear(); + } + + if (nativeTypeRaw && nativeTypeRaw->IsString()) { + fieldType = nativeTypeRaw->AsString(); + } + + NYT::TNode typeNode; + if (fieldType && fieldType != "null" && fieldType != "void" && fieldType != "decimal") { + TString yqlType = ConvertYtDataType(*fieldType, nativeYtTypeFlags); + auto dataTypeNode = NYT::TNode() + .Add("DataType") + .Add(yqlType); + + typeNode = nullable ? NYT::TNode() + .Add("OptionalType") + .Add(dataTypeNode) : dataTypeNode; + + } else { + typeNode = ConvertNativeYtType(nativeTypeRaw ? *nativeTypeRaw : NYT::TNode(fieldType.GetOrElse({})), true, hasYson, nativeYtTypeFlags); + } + if (hasYson) { + anyColumns.insert(*fieldName); + } + return { *fieldName, typeNode }; +} + +NYT::TNode YTSchemaToRowSpec(const NYT::TNode& schema, const TYTSortInfo* sortInfo) { + NYT::TNode rowSpec; + auto& rowType = rowSpec[RowSpecAttrType]; + rowType.Add("StructType"); + auto& resultTypes = rowType.Add(); + resultTypes = NYT::TNode::CreateList(); + + const auto* strictAttr = schema.GetAttributes().AsMap().FindPtr("strict"); + const bool strictSchema = !strictAttr || NYT::GetBool(*strictAttr); + rowSpec[RowSpecAttrStrictSchema] = static_cast<int>(strictSchema); + + THashSet<TString> anyColumns; + NYT::TNode sortedBy = NYT::TNode::CreateList(); + NYT::TNode sortedByTypes; + NYT::TNode sortDirections; + ui64 nativeYtTypeFlags = 0; + + THashMap<TString, NYT::TNode> types; + for (const auto& entry : schema.AsList()) { + if (!entry.IsMap()) { + YQL_LOG_CTX_THROW yexception() << "Invalid table schema: list element is not a map node"; + } + + auto ft = ExtractYtType(entry, strictSchema, anyColumns, nativeYtTypeFlags); + types[ft.first] = ft.second; + } + rowSpec[RowSpecAttrNativeYtTypeFlags] = nativeYtTypeFlags; + + TYTSortInfo localSortInfo; + if (!sortInfo) { + localSortInfo = KeyColumnsFromSchema(schema); + sortInfo = &localSortInfo; + } + bool uniqueKeys = sortInfo->Unique; + for (const auto& field : sortInfo->Keys) { + const auto& fieldName = field.first; + auto* type = types.FindPtr(fieldName); + if (!type) { + uniqueKeys = false; + break; + } + sortedBy.Add(fieldName); + sortedByTypes.Add(*type); + sortDirections.Add(field.second); + } + + for (const auto& entry : schema.AsList()) { + for (const auto& it : entry.AsMap()) { + if (it.first != "name") { + continue; + } + const auto& fieldName = it.second.AsString(); + auto* type = types.FindPtr(fieldName); + if (!type) { + continue; + } + if (YqlFakeColumnName != fieldName || schema.AsList().size() > 1) { + resultTypes.Add(NYT::TNode() + .Add(fieldName) + .Add(*type)); + } + } + } + + if (!sortedBy.Empty()) { + bool hasAnyInKey = false; + for (const auto& key : sortedBy.AsList()) { + if (anyColumns.contains(key.AsString())) { + hasAnyInKey = true; + break; + } + } + + if (!hasAnyInKey) { + rowSpec[RowSpecAttrSortedBy] = sortedBy; + rowSpec[RowSpecAttrSortedByTypes] = sortedByTypes; + rowSpec[RowSpecAttrSortDirections] = sortDirections; + rowSpec[RowSpecAttrSortMembers] = sortedBy; + if (uniqueKeys) { + rowSpec[RowSpecAttrUniqueKeys] = uniqueKeys; + } + } + } + + return rowSpec; +} + +NYT::TNode QB2PremapperToRowSpec(const NYT::TNode& qb2, const NYT::TNode& originalScheme) { + NYT::TNode rowSpec; + auto& rowType = rowSpec[RowSpecAttrType]; + rowType.Add("StructType"); + auto& resultTypes = rowType.Add(); + + rowSpec[RowSpecAttrStrictSchema] = 1; + + THashSet<TString> anyColumns; + THashMap<TString, NYT::TNode> types; + THashSet<TString> passthroughFields; + for (const auto& field: qb2["fields"].AsMap()) { + auto ytType = field.second["type"].AsString(); + const auto& reqColumns = field.second["required_columns"].AsList(); + + bool isNullable = true; + if (field.second.HasKey("nullable")) { + isNullable = field.second["nullable"].AsBool(); + } + + bool passthrough = false; + if (field.second.HasKey("passthrough")) { + passthrough = field.second["passthrough"].AsBool(); + } + + // check real field name + if (passthrough && (reqColumns.size() != 1 || reqColumns[0].AsString() != field.first)) { + passthrough = false; + } + + if (passthrough) { + passthroughFields.insert(field.first); + } + + if ("yson" == ytType || "any" == ytType) { + anyColumns.insert(field.first); + } + + ui64 nativeYtTypeFlags = 0; + TString yqlType = ConvertYtDataType(ytType, nativeYtTypeFlags); + YQL_ENSURE(0 == nativeYtTypeFlags, "QB2 premapper with native YT types is not supported"); + auto dataTypeNode = NYT::TNode() + .Add("DataType") + .Add(yqlType); + + auto typeNode = isNullable ? NYT::TNode() + .Add("OptionalType") + .Add(dataTypeNode) : dataTypeNode; + + types[field.first] = typeNode; + } + + auto sortInfo = KeyColumnsFromSchema(originalScheme); + + // switch off sort if some key isn't marked as passtrough or has 'any' type + for (auto& x: sortInfo.Keys) { + if (!passthroughFields.contains(x.first) || anyColumns.contains(x.first)) { + sortInfo.Keys.clear(); + break; + } + } + + NYT::TNode sortedBy = NYT::TNode::CreateList(); + NYT::TNode sortedByTypes; + NYT::TNode sortDirections; + + for (const auto& field : sortInfo.Keys) { + const auto& fieldName = field.first; + auto* type = types.FindPtr(fieldName); + if (!type) { + sortInfo.Unique = false; + break; + } + resultTypes.Add(NYT::TNode() + .Add(fieldName) + .Add(*type)); + + sortedBy.Add(fieldName); + sortedByTypes.Add(*type); + sortDirections.Add(field.second); + + types.erase(fieldName); + } + + for (auto entry: types) { + resultTypes.Add(NYT::TNode() + .Add(entry.first) + .Add(entry.second)); + } + + if (!sortedBy.Empty()) { + rowSpec[RowSpecAttrSortedBy] = sortedBy; + rowSpec[RowSpecAttrSortedByTypes] = sortedByTypes; + rowSpec[RowSpecAttrSortDirections] = sortDirections; + rowSpec[RowSpecAttrSortMembers] = sortedBy; + if (sortInfo.Unique) { + rowSpec[RowSpecAttrUniqueKeys] = sortInfo.Unique; + } + } + + return rowSpec; +} + +NYT::TNode GetSchemaFromAttributes(const NYT::TNode& attributes, bool onlySystem, bool ignoreWeakSchema) { + NYT::TNode result; + auto trySchema = [&] (const TStringBuf& attrName) -> TMaybe<NYT::TNode> { + if (attributes.HasKey(attrName)) { + const auto& attr = attributes[attrName]; + if (attr.IsList() && !attr.Empty()) { + return attr; + } + } + return Nothing(); + }; + + auto readSchema = !onlySystem ? trySchema(READ_SCHEMA_ATTR_NAME) : Nothing(); + auto schema = trySchema(SCHEMA_ATTR_NAME); + + if (readSchema) { + result[READ_SCHEMA_ATTR_NAME] = *readSchema; + } + + if (schema) { + if (!ignoreWeakSchema || !attributes.HasKey(SCHEMA_MODE_ATTR_NAME) || + attributes[SCHEMA_MODE_ATTR_NAME].AsString() != TStringBuf("weak")) { + result[SCHEMA_ATTR_NAME] = *schema; + } + } + return result; +} + +TYTSortInfo KeyColumnsFromSchema(const NYT::TNode& schema) { + TYTSortInfo result; + for (const auto& entry : schema.AsList()) { + if (!entry.IsMap()) { + YQL_LOG_CTX_THROW yexception() << "Invalid schema: list element is not a map node"; + } + + TMaybe<TString> fieldName; + TMaybe<int> sortDirection; + + for (const auto& it : entry.AsMap()) { + if (it.first == "name") { + fieldName = it.second.AsString(); + } else if (it.first == "sort_order") { + if (!it.second.IsEntity()) { + if (!it.second.IsString()) { + YQL_LOG_CTX_THROW yexception() << "Invalid schema sort order: " << NYT::NodeToYsonString(entry); + } + if (it.second == "ascending") { + sortDirection = 1; + } else if (it.second == "descending") { + sortDirection = 0; + } else { + YQL_LOG_CTX_THROW yexception() << "Invalid schema sort order: " << NYT::NodeToYsonString(entry); + } + } + } + } + + if (fieldName && sortDirection) { + result.Keys.emplace_back(*fieldName, *sortDirection); + } + } + result.Unique = schema.GetAttributes().HasKey("unique_keys") && NYT::GetBool(schema.GetAttributes()["unique_keys"]); + return result; +} + +bool ValidateTableSchema( + const TString& tableName, const NYT::TNode& attributes, + bool ignoreYamrDsv, bool ignoreWeakSchema +) { + if (attributes.HasKey(YqlRowSpecAttribute)) { + return true; + } + + auto hasYTSchema = [&] (const TStringBuf& attrName) { + if (attributes.HasKey(attrName)) { + const auto& schema = attributes[attrName]; + if (!schema.IsList()) { + YQL_LOG_CTX_THROW yexception() << "Invalid schema, " << + tableName << "/@" << attrName << " is not a list node"; + } + return !schema.Empty(); + } + return false; + }; + + if (hasYTSchema(READ_SCHEMA_ATTR_NAME)) { + return true; + } + + if (hasYTSchema(SCHEMA_ATTR_NAME) && !( + ignoreWeakSchema && + attributes.HasKey(SCHEMA_MODE_ATTR_NAME) && + attributes[SCHEMA_MODE_ATTR_NAME].AsString() == TStringBuf("weak"))) + { + return true; + } + + if (!ignoreYamrDsv && attributes.HasKey(FORMAT_ATTR_NAME) && + attributes[FORMAT_ATTR_NAME].AsString() == TStringBuf("yamred_dsv")) + { + return true; + } + + return false; +} + +void MergeInferredSchemeWithSort(NYT::TNode& schema, TYTSortInfo& sortInfo) { + if (sortInfo.Keys.empty()) { + return; + } + + TSet<TString> keys; + for (const auto& x : sortInfo.Keys) { + keys.insert(x.first); + } + + TSet<TString> unusedKeys = keys; + for (const auto& column : schema.AsList()) { + const auto& name = column["name"].AsString(); + unusedKeys.erase(name); + } + + // add all unused keys with yson type, update sortness + for (const auto& key : unusedKeys) { + auto map = NYT::TNode::CreateMap(); + map["name"] = key; + map["type"] = "any"; + schema.Add(map); + } + + for (ui32 i = 0; i < sortInfo.Keys.size(); ++i) { + if (unusedKeys.contains(sortInfo.Keys[i].first)) { + sortInfo.Keys.erase(sortInfo.Keys.begin() + i, sortInfo.Keys.end()); + sortInfo.Unique = false; + break; + } + } +} + +std::pair<NYT::EValueType, bool> RowSpecYqlTypeToYtType(const NYT::TNode& rowSpecType, ui64 nativeYtTypeFlags) { + const auto* type = &rowSpecType; + + while ((*type)[0] == TStringBuf("TaggedType")) { + type = &(*type)[2]; + } + + if ((*type)[0] == TStringBuf("PgType")) { + const auto& name = (*type)[1].AsString(); + NYT::EValueType ytType; + if (name == "bool") { + ytType = NYT::VT_BOOLEAN; + } else if (name == "int2") { + ytType = NYT::VT_INT16; + } else if (name == "int4") { + ytType = NYT::VT_INT32; + } else if (name == "int8") { + ytType = NYT::VT_INT64; + } else if (name == "float8") { + ytType = NYT::VT_DOUBLE; + } else if (name == "float4") { + ytType = (nativeYtTypeFlags & NTCF_FLOAT) ? NYT::VT_FLOAT : NYT::VT_DOUBLE; + } else if (name == "text" || name == "varchar" || name == "cstring") { + ytType = NYT::VT_UTF8; + } else { + ytType = NYT::VT_STRING; + } + + return { ytType, false }; + } + + bool required = true; + if ((*type)[0] == TStringBuf("OptionalType")) { + type = &(*type)[1]; + required = false; + } + + if ((*type)[0] == TStringBuf("NullType")) { + return {NYT::VT_NULL, false}; + } + + if ((*type)[0] == TStringBuf("VoidType")) { + return {NYT::VT_VOID, false}; + } + + if ((*type)[0] != TStringBuf("DataType")) { + return {NYT::VT_ANY, false}; + } + + const auto& yqlType = (*type)[1].AsString(); + NYT::EValueType ytType; + if (yqlType == TStringBuf("String") || yqlType == TStringBuf("Longint") || yqlType == TStringBuf("JsonDocument") || yqlType == TStringBuf("DyNumber")) { + ytType = NYT::VT_STRING; + } else if (yqlType == TStringBuf("Uuid")) { + ytType = (nativeYtTypeFlags & NTCF_UUID) ? NYT::VT_UUID : NYT::VT_STRING; + } else if (yqlType == TStringBuf("Json")) { + ytType = (nativeYtTypeFlags & NTCF_JSON) ? NYT::VT_JSON : NYT::VT_STRING; + } else if (yqlType == TStringBuf("Decimal")) { + ytType = NYT::VT_STRING; + } else if (yqlType == TStringBuf("Utf8")) { + ytType = NYT::VT_UTF8; + } else if (yqlType == TStringBuf("Int64")) { + ytType = NYT::VT_INT64; + } else if (yqlType == TStringBuf("Interval")) { + ytType = (nativeYtTypeFlags & NTCF_DATE) ? NYT::VT_INTERVAL : NYT::VT_INT64; + } else if (yqlType == TStringBuf("Int32")) { + ytType = NYT::VT_INT32; + } else if (yqlType == TStringBuf("Int16")) { + ytType = NYT::VT_INT16; + } else if (yqlType == TStringBuf("Int8")) { + ytType = NYT::VT_INT8; + } else if (yqlType == TStringBuf("Uint64")) { + ytType = NYT::VT_UINT64; + } else if (yqlType == TStringBuf("Timestamp")) { + ytType = (nativeYtTypeFlags & NTCF_DATE) ? NYT::VT_TIMESTAMP : NYT::VT_UINT64; + } else if (yqlType == TStringBuf("Uint32")) { + ytType = NYT::VT_UINT32; + } else if (yqlType == TStringBuf("Datetime")) { + ytType = (nativeYtTypeFlags & NTCF_DATE) ? NYT::VT_DATETIME : NYT::VT_UINT32; + } else if (yqlType == TStringBuf("Uint16")) { + ytType = NYT::VT_UINT16; + } else if (yqlType == TStringBuf("Date")) { + ytType = (nativeYtTypeFlags & NTCF_DATE) ? NYT::VT_DATE : NYT::VT_UINT16; + } else if (yqlType == TStringBuf("Date32")) { + ytType = (nativeYtTypeFlags & NTCF_BIGDATE) ? NYT::VT_DATE32 : NYT::VT_INT32; + } else if (yqlType == TStringBuf("Datetime64")) { + ytType = (nativeYtTypeFlags & NTCF_BIGDATE) ? NYT::VT_DATETIME64 : NYT::VT_INT64; + } else if (yqlType == TStringBuf("Timestamp64")) { + ytType = (nativeYtTypeFlags & NTCF_BIGDATE) ? NYT::VT_TIMESTAMP64 : NYT::VT_INT64; + } else if (yqlType == TStringBuf("Interval64")) { + ytType = (nativeYtTypeFlags & NTCF_BIGDATE) ? NYT::VT_INTERVAL64 : NYT::VT_INT64; + } else if (yqlType == TStringBuf("Uint8")) { + ytType = NYT::VT_UINT8; + } else if (yqlType == TStringBuf("Float")) { + ytType = (nativeYtTypeFlags & NTCF_FLOAT) ? NYT::VT_FLOAT : NYT::VT_DOUBLE; + } else if (yqlType == TStringBuf("Double")) { + ytType = NYT::VT_DOUBLE; + } else if (yqlType == TStringBuf("Bool")) { + ytType = NYT::VT_BOOLEAN; + } else if (yqlType == TStringBuf("Yson")) { + ytType = NYT::VT_ANY; + required = false; + } else if (yqlType == TStringBuf("TzDate") || yqlType == TStringBuf("TzDatetime") || yqlType == TStringBuf("TzTimestamp") || + yqlType == TStringBuf("TzDate32") || yqlType == TStringBuf("TzDatetime64") || yqlType == TStringBuf("TzTimestamp64")) { + ytType = NYT::VT_STRING; + } else { + YQL_LOG_CTX_THROW yexception() << "Unknown type " << yqlType.Quote() << " in row spec"; + } + + return { ytType, required }; +} + +NYT::TNode RowSpecYqlTypeToYtNativeType(const NYT::TNode& rowSpecType, ui64 nativeYtTypeFlags) { + const auto* type = &rowSpecType; + + if ((*type)[0] == TStringBuf("DataType")) { + const auto& yqlType = (*type)[1].AsString(); + TString ytType; + if (yqlType == TStringBuf("String") || yqlType == TStringBuf("Longint") || yqlType == TStringBuf("JsonDocument") || yqlType == TStringBuf("DyNumber")) { + ytType = "string"; + } else if (yqlType == TStringBuf("Uuid")) { + ytType = "uuid"; + } else if (yqlType == TStringBuf("Json")) { + ytType = (nativeYtTypeFlags & NTCF_JSON) ? "json" : "string"; + } else if (yqlType == TStringBuf("Utf8")) { + ytType = "utf8"; + } else if (yqlType == TStringBuf("Int64")) { + ytType = "int64"; + } else if (yqlType == TStringBuf("Int32")) { + ytType = "int32"; + } else if (yqlType == TStringBuf("Int16")) { + ytType = "int16"; + } else if (yqlType == TStringBuf("Int8")) { + ytType = "int8"; + } else if (yqlType == TStringBuf("Uint64")) { + ytType = "uint64"; + } else if (yqlType == TStringBuf("Uint32")) { + ytType = "uint32"; + } else if (yqlType == TStringBuf("Uint16")) { + ytType = "uint16"; + } else if (yqlType == TStringBuf("Uint8")) { + ytType = "uint8"; + } else if (yqlType == TStringBuf("Double")) { + ytType = "double"; + } else if (yqlType == TStringBuf("Float")) { + ytType = (nativeYtTypeFlags & NTCF_FLOAT) ? "float" : "double"; + } else if (yqlType == TStringBuf("Bool")) { + ytType = "bool"; + } else if (yqlType == TStringBuf("Yson")) { + ytType = "yson"; + } else if (yqlType == TStringBuf("TzDate") || yqlType == TStringBuf("TzDatetime") || yqlType == TStringBuf("TzTimestamp") || + yqlType == TStringBuf("TzDate32") || yqlType == TStringBuf("TzDatetime64") || yqlType == TStringBuf("TzTimestamp64")) { + ytType = "string"; + } else if (yqlType == TStringBuf("Date")) { + ytType = (nativeYtTypeFlags & NTCF_DATE) ? "date" : "uint16"; + } else if (yqlType == TStringBuf("Datetime")) { + ytType = (nativeYtTypeFlags & NTCF_DATE) ? "datetime" : "uint32"; + } else if (yqlType == TStringBuf("Timestamp")) { + ytType = (nativeYtTypeFlags & NTCF_DATE) ? "timestamp" : "uint64"; + } else if (yqlType == TStringBuf("Interval")) { + ytType = (nativeYtTypeFlags & NTCF_DATE) ? "interval" : "int64"; + } else if (yqlType == TStringBuf("Date32")) { + ytType = (nativeYtTypeFlags & NTCF_BIGDATE) ? "date32" : "int32"; + } else if (yqlType == TStringBuf("Datetime64")) { + ytType = (nativeYtTypeFlags & NTCF_BIGDATE) ? "datetime64" : "int64"; + } else if (yqlType == TStringBuf("Timestamp64")) { + ytType = (nativeYtTypeFlags & NTCF_BIGDATE) ? "timestamp64" : "int64"; + } else if (yqlType == TStringBuf("Interval64")) { + ytType = (nativeYtTypeFlags & NTCF_BIGDATE) ? "interval64" : "int64"; + } else if (yqlType == TStringBuf("Decimal")) { + if (nativeYtTypeFlags & NTCF_DECIMAL) { + try { + return NYT::TNode::CreateMap() + ("type_name", "decimal") + ("precision", FromString<int>((*type)[2].AsString())) + ("scale", FromString<int>((*type)[3].AsString())) + ; + } catch (...) { + YQL_LOG_CTX_THROW yexception() << "Invalid Decimal type in row spec: " << CurrentExceptionMessage(); + } + } else { + ytType = "string"; + } + } else { + YQL_LOG_CTX_THROW yexception() << "Not supported data type: " << yqlType; + } + + return NYT::TNode(ytType); + } else if ((*type)[0] == TStringBuf("OptionalType")) { + return NYT::TNode::CreateMap() + ("type_name", "optional") + ("item", RowSpecYqlTypeToYtNativeType((*type)[1], nativeYtTypeFlags)) + ; + } else if ((*type)[0] == TStringBuf("ListType")) { + return NYT::TNode::CreateMap() + ("type_name", "list") + ("item", RowSpecYqlTypeToYtNativeType((*type)[1], nativeYtTypeFlags)) + ; + } else if ((*type)[0] == TStringBuf("TupleType")) { + auto elements = NYT::TNode::CreateList(); + for (const auto& x : (*type)[1].AsList()) { + elements.Add(NYT::TNode::CreateMap()("type", RowSpecYqlTypeToYtNativeType(x, nativeYtTypeFlags))); + } + + return NYT::TNode::CreateMap() + ("type_name", "tuple") + ("elements", elements) + ; + } else if ((*type)[0] == TStringBuf("StructType")) { + auto members = NYT::TNode::CreateList(); + for (const auto& x : (*type)[1].AsList()) { + members.Add(NYT::TNode::CreateMap() + ("name", x[0].AsString()) + ("type", RowSpecYqlTypeToYtNativeType(x[1], nativeYtTypeFlags)) + ); + } + + return NYT::TNode::CreateMap() + ("type_name", "struct") + ("members", members) + ; + } else if ((*type)[0] == TStringBuf("VariantType")) { + auto base = (*type)[1]; + if (base[0] == TStringBuf("TupleType")) { + auto elements = NYT::TNode::CreateList(); + for (const auto& x : base[1].AsList()) { + elements.Add(NYT::TNode::CreateMap()("type", RowSpecYqlTypeToYtNativeType(x, nativeYtTypeFlags))); + } + + return NYT::TNode::CreateMap() + ("type_name", "variant") + ("elements", elements) + ; + + } else if (base[0] == TStringBuf("StructType")) { + auto members = NYT::TNode::CreateList(); + for (const auto& x : base[1].AsList()) { + members.Add(NYT::TNode::CreateMap() + ("name", x[0].AsString()) + ("type", RowSpecYqlTypeToYtNativeType(x[1], nativeYtTypeFlags)) + ); + } + + return NYT::TNode::CreateMap() + ("type_name", "variant") + ("members", members) + ; + } else { + YQL_ENSURE(false, "Not supported variant base type: " << base[0].AsString()); + } + } else if ((*type)[0] == TStringBuf("VoidType")) { + if (nativeYtTypeFlags & NTCF_VOID) { + return NYT::TNode("void"); + } + return NYT::TNode::CreateMap({ + {"type_name", "tagged"}, + {"tag", "_Void"}, + {"item", "null"} + }); + + } else if ((*type)[0] == TStringBuf("EmptyListType")) { + return NYT::TNode::CreateMap({ + {"type_name", "tagged"}, + {"tag", "_EmptyList"}, + {"item", "null"} + }); + } else if ((*type)[0] == TStringBuf("EmptyDictType")) { + return NYT::TNode::CreateMap({ + {"type_name", "tagged"}, + {"tag", "_EmptyDict"}, + {"item", "null"} + }); + } else if ((*type)[0] == TStringBuf("NullType")) { + if (nativeYtTypeFlags & NTCF_NULL) { + return NYT::TNode("null"); + } + return NYT::TNode::CreateMap({ + {"type_name", "tagged"}, + {"tag", "_Null"}, + {"item", "null"} + }); + } else if ((*type)[0] == TStringBuf("TaggedType")) { + return NYT::TNode::CreateMap() + ("type_name", "tagged") + ("tag", (*type)[1]) + ("item", RowSpecYqlTypeToYtNativeType((*type)[2], nativeYtTypeFlags)) + ; + } else if ((*type)[0] == TStringBuf("DictType")) { + return NYT::TNode::CreateMap() + ("type_name", "dict") + ("key", RowSpecYqlTypeToYtNativeType((*type)[1], nativeYtTypeFlags)) + ("value", RowSpecYqlTypeToYtNativeType((*type)[2], nativeYtTypeFlags)) + ; + } else if ((*type)[0] == TStringBuf("PgType")) { + const auto& name = (*type)[1].AsString(); + TString ytType; + if (name == "bool") { + ytType = "bool"; + } else if (name == "int2") { + ytType = "int16"; + } else if (name == "int4") { + ytType = "int32"; + } else if (name == "int8") { + ytType = "int64"; + } else if (name == "float8") { + ytType = "double";; + } else if (name == "float4") { + ytType = (nativeYtTypeFlags & NTCF_FLOAT) ? "float" : "double"; + } else if (name == "text" || name == "varchar" || name == "cstring") { + ytType = "utf8"; + } else { + ytType = "string"; + } + + return NYT::TNode::CreateMap() + ("type_name", "optional") + ("item", NYT::TNode(ytType)); + } + + YQL_ENSURE(false, "Not supported type: " << (*type)[0].AsString()); +} + +NYT::TTableSchema RowSpecToYTSchema(const NYT::TNode& rowSpec, ui64 nativeTypeCompatibility, const NYT::TNode& columnGroupsSpec) { + + TString defaultGroup; + THashMap<TString, TString> columnGroups; + if (!columnGroupsSpec.IsUndefined()) { + for (const auto& grp: columnGroupsSpec.AsMap()) { + if (grp.second.IsEntity()) { + defaultGroup = grp.first; + } else { + for (const auto& col: grp.second.AsList()) { + columnGroups[col.AsString()] = grp.first; + } + } + } + } + + NYT::TTableSchema schema; + const auto& rowSpecMap = rowSpec.AsMap(); + + const auto* rootType = rowSpecMap.FindPtr(RowSpecAttrType); + if (!rootType || !rootType->IsList()) { + YQL_LOG_CTX_THROW yexception() << "Invalid Type in row spec"; + } + + ui64 nativeYtTypeFlags = 0; + if (rowSpec.HasKey(RowSpecAttrNativeYtTypeFlags)) { + nativeYtTypeFlags = rowSpec[RowSpecAttrNativeYtTypeFlags].AsUint64(); + } else { + if (rowSpec.HasKey(RowSpecAttrUseNativeYtTypes)) { + nativeYtTypeFlags = NYT::GetBool(rowSpec[RowSpecAttrUseNativeYtTypes]) ? NTCF_LEGACY : NTCF_NONE; + } else if (rowSpec.HasKey(RowSpecAttrUseTypeV2)) { + nativeYtTypeFlags = NYT::GetBool(rowSpec[RowSpecAttrUseTypeV2]) ? NTCF_LEGACY : NTCF_NONE; + } + } + nativeYtTypeFlags &= nativeTypeCompatibility; + bool useNativeTypes = nativeYtTypeFlags & (NTCF_COMPLEX | NTCF_DECIMAL); + + THashMap<TString, std::pair<NYT::EValueType, bool>> fieldTypes; + THashMap<TString, NYT::TNode> fieldNativeTypes; + TVector<TString> columns; + + for (const auto& entry : rootType->AsList()[1].AsList()) { + TString column = entry[0].AsString(); + if (useNativeTypes) { + fieldNativeTypes[column] = RowSpecYqlTypeToYtNativeType(entry[1], nativeYtTypeFlags); + } else { + fieldTypes[column] = RowSpecYqlTypeToYtType(entry[1], nativeYtTypeFlags); + } + columns.push_back(column); + } + + const auto* sortedBy = rowSpecMap.FindPtr(RowSpecAttrSortedBy); + const auto* sortedByTypes = rowSpecMap.FindPtr(RowSpecAttrSortedByTypes); + const auto* sortDirections = rowSpecMap.FindPtr(RowSpecAttrSortDirections); + + THashSet<TString> keyColumns; + if (sortedBy && sortedByTypes) { + auto sortedByType = sortedByTypes->AsList().begin(); + auto sortDir = sortDirections->AsList().begin(); + for (const auto& column : sortedBy->AsList()) { + const auto& columnString = column.AsString(); + + keyColumns.insert(columnString); + + auto columnNode = NYT::TColumnSchema() + .Name(columnString); + + if (auto group = columnGroups.Value(columnString, defaultGroup)) { + columnNode.Group(std::move(group)); + } + + bool auxField = false; + if (useNativeTypes) { + auto ytType = RowSpecYqlTypeToYtNativeType(*sortedByType, nativeYtTypeFlags); + columnNode.RawTypeV3(ytType); + auxField = 0 == fieldNativeTypes.erase(columnString); + } else { + auto ytType = RowSpecYqlTypeToYtType(*sortedByType, nativeYtTypeFlags); + columnNode.Type(ytType.first, /*required*/ ytType.second); + auxField = 0 == fieldTypes.erase(columnString); + } + + columnNode.SortOrder(sortDir->AsInt64() || auxField ? NYT::SO_ASCENDING : NYT::SO_DESCENDING); + schema.AddColumn(columnNode); + ++sortedByType; + ++sortDir; + } + } + + for (const auto& column : columns) { + if (keyColumns.contains(column)) { + continue; + } + auto columnNode = NYT::TColumnSchema().Name(column); + if (auto group = columnGroups.Value(column, defaultGroup)) { + columnNode.Group(std::move(group)); + } + if (useNativeTypes) { + auto field = fieldNativeTypes.find(column); + YQL_ENSURE(field != fieldNativeTypes.end()); + columnNode.RawTypeV3(field->second); + + } else { + auto field = fieldTypes.find(column); + YQL_ENSURE(field != fieldTypes.end()); + columnNode.Type(field->second.first, /*required*/ field->second.second); + } + schema.AddColumn(std::move(columnNode)); + } + + // add fake column to avoid slow 0-columns YT schema + if (schema.Columns().empty()) { + schema.AddColumn(NYT::TColumnSchema() + .Name(TString{YqlFakeColumnName}) + .Type(NYT::EValueType::VT_BOOLEAN, /*required*/ false)); + } + + if (rowSpec.HasKey(RowSpecAttrUniqueKeys)) { + schema.UniqueKeys(NYT::GetBool(rowSpec[RowSpecAttrUniqueKeys])); + } + + return schema; +} + +NYT::TSortColumns ToYTSortColumns(const TVector<std::pair<TString, bool>>& sortColumns) { + NYT::TSortColumns res; + for (auto& item: sortColumns) { + res.Add(NYT::TSortColumn().Name(item.first).SortOrder(item.second ? NYT::ESortOrder::SO_ASCENDING : NYT::ESortOrder::SO_DESCENDING)); + } + return res; +} + +TString GetTypeV3String(const TTypeAnnotationNode& type, ui64 nativeTypeCompatibility) { + NYT::TNode typeNode; + NYT::TNodeBuilder nodeBuilder(&typeNode); + NCommon::WriteTypeToYson(nodeBuilder, &type); + return NYT::NodeToCanonicalYsonString(RowSpecYqlTypeToYtNativeType(typeNode, nativeTypeCompatibility)); +} + +} // NYql diff --git a/yt/yql/providers/yt/lib/schema/schema.h b/yt/yql/providers/yt/lib/schema/schema.h new file mode 100644 index 0000000000..88130939a7 --- /dev/null +++ b/yt/yql/providers/yt/lib/schema/schema.h @@ -0,0 +1,35 @@ +#pragma once + +#include <yql/essentials/providers/common/codec/yql_codec_type_flags.h> + +#include <yt/cpp/mapreduce/interface/common.h> + +#include <library/cpp/yson/node/node.h> + +#include <util/generic/maybe.h> +#include <util/generic/vector.h> +#include <util/generic/string.h> + +#include <utility> + +namespace NYql { + +class TTypeAnnotationNode; + +struct TYTSortInfo { + TVector<std::pair<TString, int>> Keys; + bool Unique = false; +}; + + +NYT::TNode YTSchemaToRowSpec(const NYT::TNode& schema, const TYTSortInfo* sortInfo = nullptr); +NYT::TNode QB2PremapperToRowSpec(const NYT::TNode& qb2, const NYT::TNode& originalScheme); +NYT::TNode GetSchemaFromAttributes(const NYT::TNode& attributes, bool onlySystem = false, bool ignoreWeakSchema = false); +TYTSortInfo KeyColumnsFromSchema(const NYT::TNode& schema); +bool ValidateTableSchema(const TString& tableName, const NYT::TNode& attributes, bool ignoreYamrDsv, bool ignoreWeakSchema = false); +void MergeInferredSchemeWithSort(NYT::TNode& schema, TYTSortInfo& sortInfo); +NYT::TTableSchema RowSpecToYTSchema(const NYT::TNode& rowSpec, ui64 nativeTypeCompatibility, const NYT::TNode& columnGroupsSpec = {}); +NYT::TSortColumns ToYTSortColumns(const TVector<std::pair<TString, bool>>& sortColumns); +TString GetTypeV3String(const TTypeAnnotationNode& type, ui64 nativeTypeCompatibility = NTCF_ALL); + +} // NYql diff --git a/yt/yql/providers/yt/lib/schema/ya.make b/yt/yql/providers/yt/lib/schema/ya.make new file mode 100644 index 0000000000..daf4a97494 --- /dev/null +++ b/yt/yql/providers/yt/lib/schema/ya.make @@ -0,0 +1,17 @@ +LIBRARY() + +SRCS( + schema.cpp +) + +PEERDIR( + library/cpp/yson/node + yt/cpp/mapreduce/interface + yql/essentials/utils + yql/essentials/utils/log + yql/essentials/providers/common/codec + yql/essentials/providers/common/schema/expr + yt/yql/providers/yt/common +) + +END() diff --git a/yt/yql/providers/yt/lib/skiff/ya.make b/yt/yql/providers/yt/lib/skiff/ya.make new file mode 100644 index 0000000000..6e9cdea099 --- /dev/null +++ b/yt/yql/providers/yt/lib/skiff/ya.make @@ -0,0 +1,15 @@ +LIBRARY() + +SRCS( + yql_skiff_schema.cpp +) + +PEERDIR( + library/cpp/yson + yt/yql/providers/yt/common + yql/essentials/providers/common/codec + yql/essentials/providers/common/schema/skiff + yql/essentials/utils +) + +END() diff --git a/yt/yql/providers/yt/lib/skiff/yql_skiff_schema.cpp b/yt/yql/providers/yt/lib/skiff/yql_skiff_schema.cpp new file mode 100644 index 0000000000..c17502f3d2 --- /dev/null +++ b/yt/yql/providers/yt/lib/skiff/yql_skiff_schema.cpp @@ -0,0 +1,384 @@ +#include "yql_skiff_schema.h" + +#include <yt/yql/providers/yt/common/yql_names.h> +#include <yql/essentials/providers/common/codec/yql_codec_type_flags.h> +#include <yql/essentials/providers/common/schema/skiff/yql_skiff_schema.h> +#include <yql/essentials/utils/yql_panic.h> + +#include <library/cpp/yson/node/node.h> + +#include <util/generic/hash_set.h> +#include <util/generic/xrange.h> +#include <util/generic/map.h> + + +namespace NYql { + +namespace { + +NYT::TNode RowSpecToInputSkiff(const NYT::TNode& attrs, const THashMap<TString, ui32>& structColumns, + const THashSet<TString>& auxColumns, bool rowIndex, bool rangeIndex, bool keySwitch) +{ + + YQL_ENSURE(attrs.HasKey(YqlRowSpecAttribute), "Missing mandatory " + << TString{YqlRowSpecAttribute}.Quote() << " attribute"); + + const auto& rowSpec = attrs[YqlRowSpecAttribute]; + + ui64 nativeYtTypeFlags = 0; + if (rowSpec.HasKey(RowSpecAttrNativeYtTypeFlags)) { + nativeYtTypeFlags = rowSpec[RowSpecAttrNativeYtTypeFlags].AsUint64(); + } else { + if (rowSpec.HasKey(RowSpecAttrUseNativeYtTypes)) { + nativeYtTypeFlags = NYT::GetBool(rowSpec[RowSpecAttrUseNativeYtTypes]) ? NTCF_LEGACY : NTCF_NONE; + } else if (rowSpec.HasKey(RowSpecAttrUseTypeV2)) { + nativeYtTypeFlags = NYT::GetBool(rowSpec[RowSpecAttrUseTypeV2]) ? NTCF_LEGACY : NTCF_NONE; + } + } + + THashSet<TString> fieldsWithDefValue; + if (rowSpec.HasKey(RowSpecAttrDefaultValues)) { + for (auto& value : rowSpec[RowSpecAttrDefaultValues].AsMap()) { + fieldsWithDefValue.insert(value.first); + } + } + + THashSet<TString> fieldsWithExplicitYson; + if (rowSpec.HasKey(RowSpecAttrExplicitYson)) { + for (auto& value : rowSpec[RowSpecAttrExplicitYson].AsList()) { + fieldsWithExplicitYson.emplace(value.AsString()); + } + } + + auto typeNode = NCommon::ParseSkiffTypeFromYson(rowSpec[RowSpecAttrType], nativeYtTypeFlags); + TMap<TString, NYT::TNode> typeColumns; // Must be ordered + const auto optType = NYT::TNode("variant8"); + for (auto item: typeNode["children"].AsList()) { + auto name = item["name"].AsString(); + if (fieldsWithExplicitYson.contains(name)) { + item = NYT::TNode() + ("name", name) + ("wire_type", "variant8") + ("children", NYT::TNode() + .Add(NYT::TNode()("wire_type", "nothing")) + .Add(NYT::TNode()("wire_type", "yson32")) + ); + + } else if (fieldsWithDefValue.contains(name) && item["wire_type"] != optType) { + item = NYT::TNode() + ("name", name) + ("wire_type", "variant8") + ("children", NYT::TNode() + .Add(NYT::TNode()("wire_type", "nothing")) + .Add(std::move(item)) + ); + } + YQL_ENSURE(typeColumns.emplace(name, std::move(item)).second, "Duplicate struct column: " << name); + } + + THashMap<TString, NYT::TNode> sortAuxColumns; + if (rowSpec.HasKey(RowSpecAttrSortedBy)) { + auto& sortedBy = rowSpec[RowSpecAttrSortedBy].AsList(); + auto& sortedByType = rowSpec[RowSpecAttrSortedByTypes].AsList(); + auto& sortedDirections = rowSpec[RowSpecAttrSortDirections].AsList(); + for (size_t i: xrange(sortedBy.size())) { + auto name = sortedBy[i].AsString(); + if (!typeColumns.contains(name)) { + NYT::TNode fieldType; + if (!sortedDirections.empty() && !sortedDirections[i].AsInt64()) { + fieldType = NYT::TNode()("wire_type", "string32"); + } else { + fieldType = NCommon::ParseSkiffTypeFromYson(sortedByType[i], nativeYtTypeFlags); + } + fieldType["name"] = name; + sortAuxColumns.emplace(name, std::move(fieldType)); + } + } + } + + bool strictSchema = true; + if (rowSpec.HasKey(RowSpecAttrStrictSchema)) { + strictSchema = rowSpec[RowSpecAttrStrictSchema].IsInt64() + ? rowSpec[RowSpecAttrStrictSchema].AsInt64() != 0 + : NYT::GetBool(rowSpec[RowSpecAttrStrictSchema]); + } + + if (!strictSchema) { + if (rowSpec.HasKey(RowSpecAttrWeakFields)) { + for (auto& field: rowSpec[RowSpecAttrWeakFields].AsList()) { + if (!typeColumns.contains(field.AsString())) { + auto column = NYT::TNode() + ("name", field) + ("wire_type", "variant8") + ("children", NYT::TNode() + .Add(NYT::TNode()("wire_type", "nothing")) + .Add(NYT::TNode()("wire_type", "yson32")) + ); + + typeColumns.emplace(field.AsString(), std::move(column)); + } + } + } + } + + NYT::TNode tableSchema; + tableSchema["wire_type"] = "tuple"; + auto& tableColumns = tableSchema["children"]; + tableColumns = NYT::TNode::CreateList(); + + if (rangeIndex) { + tableColumns.Add(NYT::TNode() + ("name", "$range_index") + ("wire_type", "variant8") + ("children", NYT::TNode() + .Add(NYT::TNode()("wire_type", "nothing")) + .Add(NYT::TNode()("wire_type", "int64")) + ) + ); + } + + const bool dynamic = attrs.HasKey(YqlDynamicAttribute) && attrs[YqlDynamicAttribute].AsBool(); + + if (!dynamic && rowIndex) { + tableColumns.Add(NYT::TNode() + ("name", "$row_index") + ("wire_type", "variant8") + ("children", NYT::TNode() + .Add(NYT::TNode()("wire_type", "nothing")) + .Add(NYT::TNode()("wire_type", "int64")) + ) + ); + } + + if (keySwitch) { + tableColumns.Add(NYT::TNode() + ("name", "$key_switch") + ("wire_type", "boolean") + ); + } + + TMap<ui32, NYT::TNode> columns; + const bool hasOthers = !strictSchema && structColumns.contains(YqlOthersColumnName); + for (auto& item: typeColumns) { + if (auto p = structColumns.FindPtr(item.first)) { + ui32 pos = *p; + YQL_ENSURE(columns.emplace(pos, item.second).second, "Reused column position"); + } + } + for (auto& item: sortAuxColumns) { + if (auto p = structColumns.FindPtr(item.first)) { + ui32 pos = *p; + YQL_ENSURE(columns.emplace(pos, item.second).second, "Reused column position"); + } + } + for (auto& item: columns) { + tableColumns.Add(item.second); + } + + if (hasOthers || !auxColumns.empty()) { + for (auto& item: typeColumns) { + if (!structColumns.contains(item.first)) { + if (hasOthers || auxColumns.contains(item.first)) { + tableColumns.Add(item.second); + } + } + } + if (hasOthers) { + tableColumns.Add(NYT::TNode() + ("name", "$other_columns") + ("wire_type", "yson32") + ); + } + } + return tableSchema; +} + +NYT::TNode RowSpecToOutputSkiff(const NYT::TNode& attrs) { + YQL_ENSURE(attrs.HasKey(YqlRowSpecAttribute), "Missing mandatory " + << TString{YqlRowSpecAttribute}.Quote() << " attribute"); + + const auto& rowSpec = attrs[YqlRowSpecAttribute]; + ui64 nativeYtTypeFlags = 0; + if (rowSpec.HasKey(RowSpecAttrNativeYtTypeFlags)) { + nativeYtTypeFlags = rowSpec[RowSpecAttrNativeYtTypeFlags].AsUint64(); + } else { + if (rowSpec.HasKey(RowSpecAttrUseNativeYtTypes)) { + nativeYtTypeFlags = NYT::GetBool(rowSpec[RowSpecAttrUseNativeYtTypes]) ? NTCF_LEGACY : NTCF_NONE; + } else if (rowSpec.HasKey(RowSpecAttrUseTypeV2)) { + nativeYtTypeFlags = NYT::GetBool(rowSpec[RowSpecAttrUseTypeV2]) ? NTCF_LEGACY : NTCF_NONE; + } + } + + auto typeNode = NCommon::ParseSkiffTypeFromYson(rowSpec[RowSpecAttrType], nativeYtTypeFlags); + TMap<TString, NYT::TNode> typeColumns; // Must be ordered + for (auto item: typeNode["children"].AsList()) { + auto name = item["name"].AsString(); + YQL_ENSURE(typeColumns.emplace(name, std::move(item)).second, "Duplicate struct column: " << name); + } + + if (rowSpec.HasKey(RowSpecAttrSortedBy)) { + auto& sortedBy = rowSpec[RowSpecAttrSortedBy].AsList(); + auto& sortedByType = rowSpec[RowSpecAttrSortedByTypes].AsList(); + auto& sortedDirections = rowSpec[RowSpecAttrSortDirections].AsList(); + for (size_t i: xrange(sortedBy.size())) { + auto name = sortedBy[i].AsString(); + if (!typeColumns.contains(name)) { + NYT::TNode fieldType; + if (!sortedDirections.empty() && !sortedDirections[i].AsInt64()) { + fieldType = NYT::TNode()("wire_type", "string32"); + } else { + fieldType = NCommon::ParseSkiffTypeFromYson(sortedByType[i], nativeYtTypeFlags); + } + fieldType["name"] = name; + YQL_ENSURE(typeColumns.emplace(name, std::move(fieldType)).second, "Duplicate struct aux column: " << name); + } + } + } + + if (typeColumns.empty()) { + typeColumns.emplace(YqlFakeColumnName, NYT::TNode() + ("name", YqlFakeColumnName) + ("wire_type", "variant8") + ("children", NYT::TNode() + .Add(NYT::TNode()("wire_type", "nothing")) + .Add(NYT::TNode()("wire_type", "boolean")) + ) + ); + } + + NYT::TNode tableSchema; + tableSchema["wire_type"] = "tuple"; + auto& tableColumns = tableSchema["children"]; + tableColumns = NYT::TNode::CreateList(); + for (auto& item: typeColumns) { + tableColumns.Add(item.second); + } + + return tableSchema; +} + +} + +NYT::TNode SingleTableSpecToInputSkiff(const NYT::TNode& spec, const THashMap<TString, ui32>& structColumns, bool rowIndex, bool rangeIndex, bool keySwitch) { + NYT::TNode formatConfig = NYT::TNode("skiff"); + auto& skiffConfig = formatConfig.Attributes(); + auto& skiffSchemas = skiffConfig["table_skiff_schemas"]; + skiffSchemas.Add(RowSpecToInputSkiff(spec, structColumns, {}, rowIndex, rangeIndex, keySwitch)); + + return formatConfig; +} + +NYT::TNode SingleTableSpecToInputSkiffSchema(const NYT::TNode& spec, size_t tableIndex, const THashMap<TString, ui32>& structColumns, const THashSet<TString>& auxColumns, bool rowIndex, bool rangeIndex, bool keySwitch) { + YQL_ENSURE(spec.IsMap(), "Expect Map type of input meta attrs, but got type " << spec.GetType()); + YQL_ENSURE(spec.HasKey(YqlIOSpecTables), "Expect " << TString{YqlIOSpecTables}.Quote() << " key"); + + const auto& inputSpecs = spec[YqlIOSpecTables].AsList(); + YQL_ENSURE(tableIndex < inputSpecs.size()); + + if (inputSpecs[tableIndex].IsString()) { + auto refName = inputSpecs[tableIndex].AsString(); + YQL_ENSURE(spec.HasKey(YqlIOSpecRegistry) && spec[YqlIOSpecRegistry].HasKey(refName), "Bad input registry reference: " << refName); + return RowSpecToInputSkiff(spec[YqlIOSpecRegistry][refName], structColumns, auxColumns, rowIndex, rangeIndex, keySwitch); + } else { + return RowSpecToInputSkiff(inputSpecs[tableIndex], structColumns, auxColumns, rowIndex, rangeIndex, keySwitch); + } +} + +NYT::TNode TablesSpecToInputSkiff(const NYT::TNode& spec, const THashMap<TString, ui32>& structColumns, bool rowIndex, bool rangeIndex, bool keySwitch) { + YQL_ENSURE(spec.IsMap(), "Expect Map type of input meta attrs, but got type " << spec.GetType()); + YQL_ENSURE(spec.HasKey(YqlIOSpecTables), "Expect " << TString{YqlIOSpecTables}.Quote() << " key"); + + const auto& inputSpecs = spec[YqlIOSpecTables].AsList(); + + NYT::TNode formatConfig = NYT::TNode("skiff"); + auto& skiffConfig = formatConfig.Attributes(); + auto skiffSchemas = NYT::TNode::CreateList(); + THashMap<TString, size_t> uniqSchemas; + for (size_t inputIndex = 0; inputIndex < inputSpecs.size(); ++inputIndex) { + if (inputSpecs[inputIndex].IsString()) { + auto refName = inputSpecs[inputIndex].AsString(); + + size_t schemaId = uniqSchemas.size(); + auto p = uniqSchemas.emplace(refName, schemaId); + if (p.second) { + YQL_ENSURE(spec.HasKey(YqlIOSpecRegistry) && spec[YqlIOSpecRegistry].HasKey(refName), "Bad input registry reference: " << refName); + NYT::TNode tableSchema = RowSpecToInputSkiff(spec[YqlIOSpecRegistry][refName], structColumns, {}, rowIndex, rangeIndex, keySwitch); + skiffConfig["skiff_schema_registry"][TStringBuilder() << "table" << schemaId] = std::move(tableSchema); + } + else { + schemaId = p.first->second; + } + + skiffSchemas.Add(NYT::TNode(TStringBuilder() << "$table" << schemaId)); + } else { + NYT::TNode tableSchema = RowSpecToInputSkiff(inputSpecs[inputIndex], structColumns, {}, rowIndex, rangeIndex, keySwitch); + skiffSchemas.Add(std::move(tableSchema)); + } + } + + skiffConfig["table_skiff_schemas"] = std::move(skiffSchemas); + return formatConfig; +} + +NYT::TNode TablesSpecToOutputSkiff(const NYT::TNode& spec) { + YQL_ENSURE(spec.IsMap(), "Expect Map type of output meta attrs, but got type " << spec.GetType()); + YQL_ENSURE(spec.HasKey(YqlIOSpecTables), "Expect " << TString{YqlIOSpecTables}.Quote() << " key"); + + const auto& outSpecs = spec[YqlIOSpecTables].AsList(); + + NYT::TNode formatConfig = NYT::TNode("skiff"); + auto& skiffConfig = formatConfig.Attributes(); + auto skiffSchemas = NYT::TNode::CreateList(); + THashMap<TString, size_t> uniqSchemas; + for (size_t outIndex = 0; outIndex < outSpecs.size(); ++outIndex) { + if (outSpecs[outIndex].IsString()) { + auto refName = outSpecs[outIndex].AsString(); + + size_t schemaId = uniqSchemas.size(); + auto p = uniqSchemas.emplace(refName, schemaId); + if (p.second) { + YQL_ENSURE(spec.HasKey(YqlIOSpecRegistry) && spec[YqlIOSpecRegistry].HasKey(refName), "Bad output registry reference: " << refName); + NYT::TNode tableSchema = RowSpecToOutputSkiff(spec[YqlIOSpecRegistry][refName]); + skiffConfig["skiff_schema_registry"][TStringBuilder() << "table" << schemaId] = std::move(tableSchema); + } + else { + schemaId = p.first->second; + } + + skiffSchemas.Add(NYT::TNode(TStringBuilder() << "$table" << schemaId)); + } else { + NYT::TNode tableSchema = RowSpecToOutputSkiff(outSpecs[outIndex]); + skiffSchemas.Add(std::move(tableSchema)); + } + } + + skiffConfig["table_skiff_schemas"] = std::move(skiffSchemas); + return formatConfig; +} + +NYT::TNode SingleTableSpecToOutputSkiff(const NYT::TNode& spec, size_t tableIndex) { + YQL_ENSURE(spec.IsMap(), "Expect Map type of output meta attrs, but got type " << spec.GetType()); + YQL_ENSURE(spec.HasKey(YqlIOSpecTables), "Expect " << TString{YqlIOSpecTables}.Quote() << " key"); + + const auto& outSpecs = spec[YqlIOSpecTables].AsList(); + YQL_ENSURE(tableIndex < outSpecs.size()); + + NYT::TNode formatConfig = NYT::TNode("skiff"); + auto& skiffConfig = formatConfig.Attributes(); + auto skiffSchemas = NYT::TNode::CreateList(); + THashMap<TString, size_t> uniqSchemas; + if (outSpecs[tableIndex].IsString()) { + auto refName = outSpecs[tableIndex].AsString(); + YQL_ENSURE(spec.HasKey(YqlIOSpecRegistry) && spec[YqlIOSpecRegistry].HasKey(refName), "Bad output registry reference: " << refName); + NYT::TNode tableSchema = RowSpecToOutputSkiff(spec[YqlIOSpecRegistry][refName]); + skiffSchemas.Add(std::move(tableSchema)); + } else { + NYT::TNode tableSchema = RowSpecToOutputSkiff(outSpecs[tableIndex]); + skiffSchemas.Add(std::move(tableSchema)); + } + + skiffConfig["table_skiff_schemas"] = std::move(skiffSchemas); + return formatConfig; +} + +} // NYql diff --git a/yt/yql/providers/yt/lib/skiff/yql_skiff_schema.h b/yt/yql/providers/yt/lib/skiff/yql_skiff_schema.h new file mode 100644 index 0000000000..a92a508c85 --- /dev/null +++ b/yt/yql/providers/yt/lib/skiff/yql_skiff_schema.h @@ -0,0 +1,18 @@ +#pragma once + +#include <library/cpp/yson/node/node.h> + +#include <util/generic/string.h> +#include <util/generic/hash.h> + + +namespace NYql { + +NYT::TNode TablesSpecToInputSkiff(const NYT::TNode& spec, const THashMap<TString, ui32>& structColumns, bool rowIndex, bool rangeIndex, bool keySwitch); +NYT::TNode SingleTableSpecToInputSkiff(const NYT::TNode& spec, const THashMap<TString, ui32>& structColumns, bool rowIndex, bool rangeIndex, bool keySwitch); +NYT::TNode SingleTableSpecToInputSkiffSchema(const NYT::TNode& spec, size_t tableIndex, const THashMap<TString, ui32>& structColumns, const THashSet<TString>& auxColumns, bool rowIndex, bool rangeIndex, bool keySwitch); + +NYT::TNode TablesSpecToOutputSkiff(const NYT::TNode& spec); +NYT::TNode SingleTableSpecToOutputSkiff(const NYT::TNode& spec, size_t tableIndex); + +} // NYql diff --git a/yt/yql/providers/yt/lib/url_mapper/ya.make b/yt/yql/providers/yt/lib/url_mapper/ya.make new file mode 100644 index 0000000000..22b7cfd023 --- /dev/null +++ b/yt/yql/providers/yt/lib/url_mapper/ya.make @@ -0,0 +1,14 @@ +LIBRARY() + +SRCS( + yql_yt_url_mapper.cpp +) + +PEERDIR( + yql/essentials/providers/common/proto + library/cpp/regex/pcre + library/cpp/uri + library/cpp/cgiparam +) + +END() diff --git a/yt/yql/providers/yt/lib/url_mapper/yql_yt_url_mapper.cpp b/yt/yql/providers/yt/lib/url_mapper/yql_yt_url_mapper.cpp new file mode 100644 index 0000000000..56e3023937 --- /dev/null +++ b/yt/yql/providers/yt/lib/url_mapper/yql_yt_url_mapper.cpp @@ -0,0 +1,65 @@ +#include "yql_yt_url_mapper.h" + +#include <yql/essentials/providers/common/proto/gateways_config.pb.h> + +#include <library/cpp/uri/http_url.h> +#include <library/cpp/cgiparam/cgiparam.h> + +#include <util/generic/algorithm.h> +#include <util/generic/yexception.h> + +namespace NYql { + +TYtUrlMapper::TYtUrlMapper(const TYtGatewayConfig& config) + : RemoteFilePatterns_(BuildRemoteFilePatterns(config)) +{ +} + +bool TYtUrlMapper::MapYtUrl(const TString& url, TString* cluster, TString* path) const { + THttpURL parsedUrl; + if (THttpURL::ParsedOK != parsedUrl.Parse(url, THttpURL::FeaturesAll | NUri::TFeature::FeatureConvertHostIDN | NUri::TFeature::FeatureNoRelPath, {}, 65536)) { + return false; + } + + const auto rawScheme = parsedUrl.GetField(NUri::TField::FieldScheme); + auto host = parsedUrl.GetHost(); + if (NUri::EqualNoCase(rawScheme, "yt") || host.StartsWith("yt.yandex") || host.EndsWith(".yt.yandex-team.ru") || host.EndsWith(".yt.yandex.net")) { + if (cluster) { + if (host.StartsWith("yt.")) { + host = parsedUrl.GetField(NUri::TField::FieldPath); + host = host.Skip(1).Before('/'); + } else { + host.ChopSuffix(".yt.yandex-team.ru"); + host.ChopSuffix(".yt.yandex.net"); + } + *cluster = host; + } + + if (path) { + if (!FindMatchingRemoteFilePattern(url)) { + return false; + } + TCgiParameters params(parsedUrl.GetField(NUri::TField::FieldQuery)); + *path = params.Has("path") ? params.Get("path") : TString{TStringBuf(parsedUrl.GetField(NUri::TField::FieldPath)).Skip(1)}; + } + + return true; + } + return false; +} + +TVector<TRegExMatch> TYtUrlMapper::BuildRemoteFilePatterns(const TYtGatewayConfig& config) { + TVector<TRegExMatch> res; + for (auto& fp : config.GetRemoteFilePatterns()) { + res.emplace_back(fp.GetPattern()); + } + return res; +} + +bool TYtUrlMapper::FindMatchingRemoteFilePattern(const TString& url) const { + return AnyOf(RemoteFilePatterns_, [&](auto& p) { + return p.Match(url.data()); + }); +} + +} diff --git a/yt/yql/providers/yt/lib/url_mapper/yql_yt_url_mapper.h b/yt/yql/providers/yt/lib/url_mapper/yql_yt_url_mapper.h new file mode 100644 index 0000000000..d0dfd7ecdf --- /dev/null +++ b/yt/yql/providers/yt/lib/url_mapper/yql_yt_url_mapper.h @@ -0,0 +1,24 @@ +#pragma once + +#include <library/cpp/regex/pcre/regexp.h> + +#include <util/generic/maybe.h> +#include <util/generic/vector.h> + +namespace NYql { + +class TYtGatewayConfig; + +class TYtUrlMapper { +public: + explicit TYtUrlMapper(const TYtGatewayConfig& config); + bool MapYtUrl(const TString& url, TString* cluster, TString* path) const; + +private: + static TVector<TRegExMatch> BuildRemoteFilePatterns(const TYtGatewayConfig& config); + bool FindMatchingRemoteFilePattern(const TString& url) const; + +private: + const TVector<TRegExMatch> RemoteFilePatterns_; +}; +} diff --git a/yt/yql/providers/yt/lib/yson_helpers/ya.make b/yt/yql/providers/yt/lib/yson_helpers/ya.make new file mode 100644 index 0000000000..f5f7592cf0 --- /dev/null +++ b/yt/yql/providers/yt/lib/yson_helpers/ya.make @@ -0,0 +1,15 @@ +LIBRARY() + +SRCS( + yson_helpers.cpp +) + +PEERDIR( + library/cpp/yson + library/cpp/yson/node + yql/essentials/utils + yql/essentials/utils/log + yt/yql/providers/yt/common +) + +END() diff --git a/yt/yql/providers/yt/lib/yson_helpers/yson_helpers.cpp b/yt/yql/providers/yt/lib/yson_helpers/yson_helpers.cpp new file mode 100644 index 0000000000..45cd7a89ff --- /dev/null +++ b/yt/yql/providers/yt/lib/yson_helpers/yson_helpers.cpp @@ -0,0 +1,310 @@ +#include "yson_helpers.h" + +#include <yt/yql/providers/yt/common/yql_names.h> +#include <yql/essentials/utils/yql_panic.h> +#include <yql/essentials/utils/log/context.h> + +#include <library/cpp/yson/node/node_io.h> +#include <library/cpp/yson/detail.h> + +#include <util/generic/yexception.h> + +namespace NYql { + +TBinaryYsonWriter::TBinaryYsonWriter(IOutputStream* stream, NYson::EYsonType type) + : NYson::TYsonWriter(stream, NYson::EYsonFormat::Binary, type, false) +{ + if (type != ::NYson::EYsonType::Node) { + Stack_.push_back(false); + } +} + +void TBinaryYsonWriter::OnBeginList() { + Stack_.push_back(false); + NYson::TYsonWriter::OnBeginList(); +} + +void TBinaryYsonWriter::OnListItem() { + Stack_.back() = true; + NYson::TYsonWriter::OnListItem(); +} + +void TBinaryYsonWriter::OnEndList() { + if (Stack_.back()) { + Stream->Write(NYson::NDetail::ListItemSeparatorSymbol); + } + Stack_.pop_back(); + NYson::TYsonWriter::OnEndList(); +} + +void TBinaryYsonWriter::OnBeginMap() { + Stack_.push_back(false); + NYson::TYsonWriter::OnBeginMap(); +} + +void TBinaryYsonWriter::OnKeyedItem(TStringBuf key) { + Stack_.back() = true; + NYson::TYsonWriter::OnKeyedItem(key); +} + +void TBinaryYsonWriter::OnEndMap() { + if (Stack_.back()) { + Stream->Write(NYson::NDetail::KeyedItemSeparatorSymbol); + } + Stack_.pop_back(); + NYson::TYsonWriter::OnEndMap(); +} + +void TBinaryYsonWriter::OnBeginAttributes() { + Stack_.push_back(false); + NYson::TYsonWriter::OnBeginAttributes(); +} + +void TBinaryYsonWriter::OnEndAttributes() { + if (Stack_.back()) { + Stream->Write(NYson::NDetail::KeyedItemSeparatorSymbol); + } + Stack_.pop_back(); + NYson::TYsonWriter::OnEndAttributes(); +} + + +TColumnFilteringConsumer::TColumnFilteringConsumer(NYT::NYson::IYsonConsumer* parent, const TMaybe<TSet<TStringBuf>>& columns, + const TMaybe<THashMap<TStringBuf, TStringBuf>>& renameColumns) + : Parent_(parent) + , Columns_(columns) + , RenameColumns_(renameColumns) +{ +} + +void TColumnFilteringConsumer::OnStringScalar(TStringBuf value) { + if (Enable_) { + Parent_->OnStringScalar(value); + } +} + +void TColumnFilteringConsumer::OnInt64Scalar(i64 value) { + if (Enable_) { + Parent_->OnInt64Scalar(value); + } +} + +void TColumnFilteringConsumer::OnUint64Scalar(ui64 value) { + if (Enable_) { + Parent_->OnUint64Scalar(value); + } +} + +void TColumnFilteringConsumer::OnDoubleScalar(double value) { + if (Enable_) { + Parent_->OnDoubleScalar(value); + } +} + +void TColumnFilteringConsumer::OnBooleanScalar(bool value) { + if (Enable_) { + Parent_->OnBooleanScalar(value); + } +} + +void TColumnFilteringConsumer::OnEntity() { + if (Enable_) { + Parent_->OnEntity(); + } +} + +void TColumnFilteringConsumer::OnBeginList() { + Stack_.push_back(Enable_); + if (Enable_) { + Parent_->OnBeginList(); + } +} + +void TColumnFilteringConsumer::OnListItem() { + if (Enable_) { + Parent_->OnListItem(); + } +} + +void TColumnFilteringConsumer::OnEndList() { + Enable_ = Stack_.back(); + Stack_.pop_back(); + if (Enable_) { + Parent_->OnEndList(); + } +} + +void TColumnFilteringConsumer::OnBeginMap() { + Stack_.push_back(Enable_); + if (Enable_) { + Parent_->OnBeginMap(); + } +} + +void TColumnFilteringConsumer::OnKeyedItem(TStringBuf key) { + TStringBuf outputKey = key; + if (Stack_.size() == 1) { + if (RenameColumns_) { + auto r = RenameColumns_->find(key); + if (r != RenameColumns_->end()) { + outputKey = r->second; + } + } + Enable_ = !Columns_ || Columns_->find(outputKey) != Columns_->end(); + } + if (Enable_) { + Parent_->OnKeyedItem(outputKey); + } +} + +void TColumnFilteringConsumer::OnEndMap() { + Enable_ = Stack_.back(); + Stack_.pop_back(); + if (Enable_) { + Parent_->OnEndMap(); + } +} + +void TColumnFilteringConsumer::OnBeginAttributes() { + if (Stack_.back()) { + Parent_->OnBeginAttributes(); + } +} + +void TColumnFilteringConsumer::OnEndAttributes() { + if (Enable_) { + Parent_->OnEndAttributes(); + } +} + +void TColumnFilteringConsumer::OnRaw(TStringBuf yson, NYson::EYsonType type) { + if (Enable_) { + Parent_->OnRaw(yson, type); + } +} + +TDoubleHighPrecisionYsonWriter::TDoubleHighPrecisionYsonWriter(IOutputStream* stream, NYson::EYsonType type, bool enableRaw) + : TYsonWriter(stream, NYson::EYsonFormat::Text, type, enableRaw) +{ +} + +void TDoubleHighPrecisionYsonWriter::OnDoubleScalar(double value) { + TString str; + if (std::isfinite(value)) { + str = ::FloatToString(value); + } else { + if (std::isnan(value)) { + str = TStringBuf("%nan"); + } else if (value > 0) { + str = TStringBuf("%inf"); + } else { + str = TStringBuf("%-inf"); + } + } + + Stream->Write(str); + if (str.find('.') == TString::npos && str.find('e') == TString::npos && std::isfinite(value)) { + Stream->Write("."); + } + EndNode(); +} + +void WriteTableReference(NYson::TYsonWriter& writer, TStringBuf provider, TStringBuf cluster, TStringBuf table, + bool remove, const TVector<TString>& columns) +{ + writer.OnBeginMap(); + writer.OnKeyedItem("Reference"); + writer.OnBeginList(); + writer.OnListItem(); + writer.OnStringScalar(provider); + writer.OnListItem(); + writer.OnStringScalar(cluster); + writer.OnListItem(); + writer.OnStringScalar(table); + writer.OnEndList(); + if (!columns.empty()) { + writer.OnKeyedItem("Columns"); + writer.OnBeginList(); + for (auto& column: columns) { + writer.OnListItem(); + writer.OnStringScalar(column); + } + writer.OnEndList(); + } + writer.OnKeyedItem("Remove"); + writer.OnBooleanScalar(remove); + writer.OnEndMap(); +} + +bool HasYqlRowSpec(const NYT::TNode& inAttrs) +{ + bool first = true; + bool hasScheme = false; + for (auto& attrs: inAttrs.AsList()) { + bool tableHasScheme = attrs.HasKey(YqlRowSpecAttribute); + if (first) { + hasScheme = tableHasScheme; + first = false; + } else { + YQL_ENSURE(hasScheme == tableHasScheme, "Different schemas of input tables"); + } + } + return hasScheme; +} + +bool HasYqlRowSpec(const TString& inputAttrs) +{ + NYT::TNode inAttrs; + try { + inAttrs = NYT::NodeFromYsonString(inputAttrs); + } catch (const yexception& e) { + YQL_LOG_CTX_THROW yexception() << "Invalid input meta attrs: " << e.what(); + } + YQL_ENSURE(inAttrs.IsList(), "Expect List type of input meta attrs, but got type " << inAttrs.GetType()); + return HasYqlRowSpec(inAttrs); +} + +bool HasStrictSchema(const NYT::TNode& attrs) +{ + if (attrs.HasKey(YqlRowSpecAttribute) && attrs[YqlRowSpecAttribute].HasKey(RowSpecAttrStrictSchema)) { + auto& strictSchemaAttr = attrs[YqlRowSpecAttribute][RowSpecAttrStrictSchema]; + return strictSchemaAttr.IsInt64() + ? strictSchemaAttr.AsInt64() != 0 + : NYT::GetBool(strictSchemaAttr); + } + return true; +} + +TMaybe<ui64> GetDataWeight(const NYT::TNode& tableAttrs) { + ui64 ret = 0; + bool hasAttr = false; + if (tableAttrs.AsMap().contains("data_weight")) { + ret = (ui64)tableAttrs["data_weight"].AsInt64(); + hasAttr = true; + } + + if (!ret && tableAttrs.AsMap().contains("uncompressed_data_size")) { + ret = (ui64)tableAttrs["uncompressed_data_size"].AsInt64(); + hasAttr = true; + } + + if (hasAttr) { + return ret; + } + + return Nothing(); +} + +ui64 GetTableRowCount(const NYT::TNode& tableAttrs) { + return NYT::GetBool(tableAttrs[TStringBuf("dynamic")]) + ? tableAttrs[TStringBuf("chunk_row_count")].AsInt64() + : tableAttrs[TStringBuf("row_count")].AsInt64(); +} + +ui64 GetContentRevision(const NYT::TNode& tableAttrs) { + return tableAttrs.HasKey("content_revision") + ? tableAttrs["content_revision"].IntCast<ui64>() + : tableAttrs["revision"].IntCast<ui64>(); +} + +} // NYql diff --git a/yt/yql/providers/yt/lib/yson_helpers/yson_helpers.h b/yt/yql/providers/yt/lib/yson_helpers/yson_helpers.h new file mode 100644 index 0000000000..aabc8f5aa2 --- /dev/null +++ b/yt/yql/providers/yt/lib/yson_helpers/yson_helpers.h @@ -0,0 +1,81 @@ +#pragma once + +#include <library/cpp/yson/node/node.h> +#include <library/cpp/yson/writer.h> + +#include <util/generic/fwd.h> +#include <util/generic/strbuf.h> +#include <util/generic/set.h> +#include <util/generic/vector.h> +#include <util/generic/maybe.h> + +namespace NYql { + +////////////////////////////////////////////////////////////////////////////////////////////////////////// + +class TBinaryYsonWriter : public NYson::TYsonWriter { +public: + TBinaryYsonWriter(IOutputStream* stream, NYson::EYsonType type = ::NYson::EYsonType::Node); + + void OnBeginList() override; + void OnListItem() override; + void OnEndList() override; + + void OnBeginMap() override; + void OnKeyedItem(TStringBuf key) override; + void OnEndMap() override; + + void OnBeginAttributes() override; + void OnEndAttributes() override; + +private: + TVector<bool> Stack_; +}; + +class TColumnFilteringConsumer: public NYT::NYson::IYsonConsumer { +public: + TColumnFilteringConsumer(NYT::NYson::IYsonConsumer* parent, const TMaybe<TSet<TStringBuf>>& columns, + const TMaybe<THashMap<TStringBuf, TStringBuf>>& renameColumns); + + virtual void OnStringScalar(TStringBuf value) final; + virtual void OnInt64Scalar(i64 value) final; + virtual void OnUint64Scalar(ui64 value) final; + virtual void OnDoubleScalar(double value) final; + virtual void OnBooleanScalar(bool value) final; + virtual void OnEntity() final; + virtual void OnBeginList() final; + virtual void OnListItem() final; + virtual void OnEndList() final; + virtual void OnBeginMap() final; + virtual void OnKeyedItem(TStringBuf key) final; + virtual void OnEndMap() final; + virtual void OnBeginAttributes() final; + virtual void OnEndAttributes() final; + virtual void OnRaw(TStringBuf yson, NYson::EYsonType type) final; +private: + NYT::NYson::IYsonConsumer* Parent_; + bool Enable_ = true; + TVector<bool> Stack_; + TMaybe<TSet<TStringBuf>> Columns_; + TMaybe<THashMap<TStringBuf, TStringBuf>> RenameColumns_; +}; + +class TDoubleHighPrecisionYsonWriter: public NYson::TYsonWriter { +public: + TDoubleHighPrecisionYsonWriter(IOutputStream* stream, NYson::EYsonType type = ::NYson::EYsonType::Node, bool enableRaw = false); + void OnDoubleScalar(double value); +}; + +void WriteTableReference(NYson::TYsonWriter& writer, TStringBuf provider, TStringBuf cluster, TStringBuf table, + bool remove, const TVector<TString>& columns); + +bool HasYqlRowSpec(const NYT::TNode& inAttrs); +bool HasYqlRowSpec(const TString& inputAttrs); + +bool HasStrictSchema(const NYT::TNode& attrs); + +TMaybe<ui64> GetDataWeight(const NYT::TNode& inAttrs); +ui64 GetTableRowCount(const NYT::TNode& tableAttrs); +ui64 GetContentRevision(const NYT::TNode& tableAttrs); + +} // NYql diff --git a/yt/yql/providers/yt/lib/yt_download/ya.make b/yt/yql/providers/yt/lib/yt_download/ya.make new file mode 100644 index 0000000000..323f5f7150 --- /dev/null +++ b/yt/yql/providers/yt/lib/yt_download/ya.make @@ -0,0 +1,16 @@ +LIBRARY() + +SRCS( + yt_download.cpp +) + +PEERDIR( + yt/yql/providers/yt/lib/init_yt_api + yql/essentials/core/file_storage + yql/essentials/utils/log + yql/essentials/utils + library/cpp/cgiparam + library/cpp/digest/md5 +) + +END() diff --git a/yt/yql/providers/yt/lib/yt_download/yt_download.cpp b/yt/yql/providers/yt/lib/yt_download/yt_download.cpp new file mode 100644 index 0000000000..8c27c88dfc --- /dev/null +++ b/yt/yql/providers/yt/lib/yt_download/yt_download.cpp @@ -0,0 +1,111 @@ +#include "yt_download.h" + +#include <yt/yql/providers/yt/lib/yson_helpers/yson_helpers.h> +#include <yt/yql/providers/yt/lib/init_yt_api/init.h> +#include <yql/essentials/core/file_storage/defs/provider.h> + +#include <yql/essentials/utils/md5_stream.h> +#include <yql/essentials/utils/yql_panic.h> +#include <yql/essentials/utils/log/log.h> + +#include <yt/cpp/mapreduce/interface/client.h> + +#include <library/cpp/cgiparam/cgiparam.h> + +#include <util/string/cast.h> +#include <util/stream/input.h> +#include <util/stream/file.h> +#include <util/system/fstat.h> + +namespace NYql { + +class TYtDownloader: public NYql::NFS::IDownloader { +public: + TYtDownloader(const TFileStorageConfig& /*config*/, const TString& defaultServer) + : DefaultServer_(defaultServer) + { + } + ~TYtDownloader() = default; + + bool Accept(const THttpURL& url) final { + const auto rawScheme = url.GetField(NUri::TField::FieldScheme); + return NUri::EqualNoCase(rawScheme, "yt"); + } + + std::tuple<NYql::NFS::TDataProvider, TString, TString> Download(const THttpURL& url, const TString& oauthToken, const TString& oldEtag, const TString& /*oldLastModified*/) final { + InitYtApiOnce(); + + TCgiParameters params(url.GetField(NUri::TField::FieldQuery)); + NYT::TCreateClientOptions createOpts; + if (oauthToken) { + createOpts.Token(oauthToken); + } + auto host = url.PrintS(NUri::TField::FlagHostPort); + if (host == "current") { + if (!DefaultServer_) { + throw yexception() << "Cannot download url: " << url.PrintS() << ", default cluster is not defined"; + } + host = DefaultServer_; + } + auto path = params.Has("path") ? params.Get("path") : TString{TStringBuf(url.GetField(NUri::TField::FieldPath)).Skip(1)}; + + auto client = NYT::CreateClient(host, createOpts); + NYT::IClientBasePtr tx = client; + TString txId = params.Get("transaction_id"); + if (!txId) { + txId = params.Get("t"); + } + YQL_LOG(INFO) << "YtDownload: host=" << host << ", path='" << path << "', tx=" << txId; + + if (txId) { + TGUID g; + if (!GetGuid(txId, g)) { + throw yexception() << "Bad transaction ID: " << txId; + } + if (g) { + tx = client->AttachTransaction(g); + } + } + + const NYT::TNode attrs = tx->Get(path + "/@", NYT::TGetOptions().AttributeFilter( + NYT::TAttributeFilter() + .AddAttribute(TString("revision")) + .AddAttribute(TString("content_revision")) + )); + auto rev = ToString(GetContentRevision(attrs)); + if (oldEtag == rev) { + return std::make_tuple(NYql::NFS::TDataProvider{}, TString{}, TString{}); + } + + auto puller = [tx = std::move(tx), path = std::move(path)](const TFsPath& dstFile) -> std::pair<ui64, TString> { + auto reader = tx->CreateFileReader(path); + + TFile outFile(dstFile, CreateAlways | ARW | AX); + TUnbufferedFileOutput out(outFile); + TMd5OutputStream md5Out(out); + + const ui64 size = TransferData(reader.Get(), &md5Out); + auto result = std::make_pair(size, md5Out.Finalize()); + out.Finish(); + outFile.Close(); + + i64 dstFileLen = GetFileLength(dstFile.c_str()); + if (dstFileLen == -1) { + ythrow TSystemError() << "cannot get file length: " << dstFile; + } + + YQL_ENSURE(static_cast<ui64>(dstFileLen) == size); + return result; + }; + return std::make_tuple(puller, rev, TString{}); + } + +private: + const TString DefaultServer_; +}; + +NYql::NFS::IDownloaderPtr MakeYtDownloader(const TFileStorageConfig& config, const TString& defaultServer) { + return MakeIntrusive<TYtDownloader>(config, defaultServer); +} + +} // NYql diff --git a/yt/yql/providers/yt/lib/yt_download/yt_download.h b/yt/yql/providers/yt/lib/yt_download/yt_download.h new file mode 100644 index 0000000000..30d39df0cc --- /dev/null +++ b/yt/yql/providers/yt/lib/yt_download/yt_download.h @@ -0,0 +1,11 @@ +#pragma once + +#include <yql/essentials/core/file_storage/defs/downloader.h> + +namespace NYql { + +class TFileStorageConfig; + +NYql::NFS::IDownloaderPtr MakeYtDownloader(const TFileStorageConfig& config, const TString& defaultServer = {}); + +} // NYql diff --git a/yt/yql/providers/yt/lib/yt_rpc_helpers/ya.make b/yt/yql/providers/yt/lib/yt_rpc_helpers/ya.make new file mode 100644 index 0000000000..1594b77ab4 --- /dev/null +++ b/yt/yql/providers/yt/lib/yt_rpc_helpers/ya.make @@ -0,0 +1,7 @@ +LIBRARY() + +PEERDIR( + yt/cpp/mapreduce/interface +) + +END() diff --git a/yt/yql/providers/yt/lib/yt_rpc_helpers/yt_convert_helpers.h b/yt/yql/providers/yt/lib/yt_rpc_helpers/yt_convert_helpers.h new file mode 100644 index 0000000000..9da478c14f --- /dev/null +++ b/yt/yql/providers/yt/lib/yt_rpc_helpers/yt_convert_helpers.h @@ -0,0 +1,18 @@ +#include <yt/cpp/mapreduce/interface/common.h> + +namespace NYql::NDqs { + +template<typename T> +void ConfigureTransaction(T& request, const ui32* dw) { + // P. S. No proper way to convert it + request->mutable_transactional_options()->mutable_transaction_id()->set_first((ui64)dw[3] | (ui64(dw[2]) << 32)); + request->mutable_transactional_options()->mutable_transaction_id()->set_second((ui64)dw[1] | (ui64(dw[0]) << 32)); +} +template<typename T> +void ConfigureTransaction(T& request, NYT::TRichYPath& richYPath) { + + if (richYPath.TransactionId_) { + ConfigureTransaction(request, richYPath.TransactionId_->dw); + } +} +};
\ No newline at end of file diff --git a/yt/yql/providers/yt/lib/yt_url_lister/ya.make b/yt/yql/providers/yt/lib/yt_url_lister/ya.make new file mode 100644 index 0000000000..361daa526c --- /dev/null +++ b/yt/yql/providers/yt/lib/yt_url_lister/ya.make @@ -0,0 +1,16 @@ +LIBRARY() + +SRCS( + yt_url_lister.cpp +) + +PEERDIR( + library/cpp/cgiparam + yql/essentials/core/url_lister/interface + yt/yql/providers/yt/lib/init_yt_api + yql/essentials/utils/fetch + yql/essentials/utils/log + yt/cpp/mapreduce/interface +) + +END() diff --git a/yt/yql/providers/yt/lib/yt_url_lister/yt_url_lister.cpp b/yt/yql/providers/yt/lib/yt_url_lister/yt_url_lister.cpp new file mode 100644 index 0000000000..a472a72071 --- /dev/null +++ b/yt/yql/providers/yt/lib/yt_url_lister/yt_url_lister.cpp @@ -0,0 +1,103 @@ +#include "yt_url_lister.h" + +#include <library/cpp/cgiparam/cgiparam.h> + +#include <yt/yql/providers/yt/lib/init_yt_api/init.h> +#include <yql/essentials/utils/fetch/fetch.h> +#include <yql/essentials/utils/log/log.h> + +#include <yt/cpp/mapreduce/interface/client.h> + +#include <util/generic/guid.h> +#include <util/generic/strbuf.h> +#include <util/string/builder.h> + + +namespace NYql::NPrivate { + +const char Sep = '/'; +const TStringBuf Scheme = "yt"; + +class TYtUrlLister: public IUrlLister { +public: + TYtUrlLister() = default; + +public: + bool Accept(const TString& url) const override { + auto httpUrl = ParseURL(url); + auto rawScheme = httpUrl.GetField(NUri::TField::FieldScheme); + return NUri::EqualNoCase(rawScheme, Scheme); + } + + TVector<TUrlListEntry> ListUrl(const TString& url, const TString& token) const override { + InitYtApiOnce(); + + auto httpUrl = ParseURL(url); + + TCgiParameters params(httpUrl.GetField(NUri::TField::FieldQuery)); + + NYT::TCreateClientOptions createOpts; + if (token) { + createOpts.Token(token); + } + + auto host = httpUrl.PrintS(NUri::TField::FlagHostPort); + + auto path = params.Has("path") + ? params.Get("path") + : TString(TStringBuf(httpUrl.GetField(NUri::TField::FieldPath)).Skip(1)); + + auto client = NYT::CreateClient(host, createOpts); + NYT::IClientBasePtr tx = client; + TString txId = params.Get("transaction_id"); + if (!txId) { + txId = params.Get("t"); + } + YQL_LOG(INFO) << "YtUrlLister: host=" << host << ", path='" << path << "', tx=" << txId; + if (txId) { + TGUID guid; + if (!GetGuid(txId, guid)) { + ythrow yexception() << "Bad transaction ID: " << txId; + } + tx = client->AttachTransaction(guid); + } + auto composeUrl = [&](auto name) { + THttpURL url; + url.Set(NUri::TField::FieldScheme, Scheme); + url.Set(NUri::TField::FieldHost, host); + url.Set(NUri::TField::FieldPath, TStringBuilder() << Sep << path << Sep << name); + if (txId) { + url.Set(NUri::TField::FieldQuery, TStringBuilder() << "transaction_id=" << txId); + } + + return url.PrintS(); + }; + + NYT::TListOptions listOpts; + listOpts.AttributeFilter( + NYT::TAttributeFilter().Attributes({"type"}) + ); + TVector<TUrlListEntry> entries; + for (const auto& item: tx->List(path, listOpts)) { + auto& entry = entries.emplace_back(); + const auto& itemName = item.AsString(); + const auto& itemType = item.GetAttributes()["type"].AsString(); + entry.Name = itemName; + entry.Url = composeUrl(itemName); + entry.Type = itemType == "map_node" + ? EUrlListEntryType::DIRECTORY + : EUrlListEntryType::FILE; + } + return entries; + } +}; + +} + +namespace NYql { + +IUrlListerPtr MakeYtUrlLister() { + return MakeIntrusive<NPrivate::TYtUrlLister>(); +} + +} diff --git a/yt/yql/providers/yt/lib/yt_url_lister/yt_url_lister.h b/yt/yql/providers/yt/lib/yt_url_lister/yt_url_lister.h new file mode 100644 index 0000000000..0c900a599a --- /dev/null +++ b/yt/yql/providers/yt/lib/yt_url_lister/yt_url_lister.h @@ -0,0 +1,10 @@ +#pragma once + +#include <yql/essentials/core/url_lister/interface/url_lister.h> + + +namespace NYql { + +IUrlListerPtr MakeYtUrlLister(); + +} diff --git a/yt/yql/providers/yt/mkql_dq/ya.make b/yt/yql/providers/yt/mkql_dq/ya.make new file mode 100644 index 0000000000..a578b41be2 --- /dev/null +++ b/yt/yql/providers/yt/mkql_dq/ya.make @@ -0,0 +1,18 @@ +LIBRARY() + +SRCS( + yql_yt_dq_transform.cpp +) + +PEERDIR( + library/cpp/yson/node + yt/cpp/mapreduce/common + yt/cpp/mapreduce/interface + yql/essentials/minikql + yql/essentials/public/udf + yql/essentials/utils +) + +YQL_LAST_ABI_VERSION() + +END() diff --git a/yt/yql/providers/yt/mkql_dq/yql_yt_dq_transform.cpp b/yt/yql/providers/yt/mkql_dq/yql_yt_dq_transform.cpp new file mode 100644 index 0000000000..c7fd013fc0 --- /dev/null +++ b/yt/yql/providers/yt/mkql_dq/yql_yt_dq_transform.cpp @@ -0,0 +1,132 @@ +#include "yql_yt_dq_transform.h" + +#include <yql/essentials/minikql/mkql_program_builder.h> +#include <yql/essentials/minikql/mkql_node_cast.h> +#include <yql/essentials/minikql/mkql_function_registry.h> +#include <yql/essentials/minikql/mkql_node_printer.h> +#include <yql/essentials/minikql/defs.h> +#include <yql/essentials/utils/yql_panic.h> +#include <yql/essentials/utils/log/log.h> + +#include <yt/cpp/mapreduce/interface/serialize.h> +#include <yt/cpp/mapreduce/common/helpers.h> + +#include <library/cpp/yson/node/serialize.h> +#include <library/cpp/yson/node/node_io.h> + +#include <util/generic/vector.h> +#include <util/generic/guid.h> + + +namespace NYql { + +using namespace NKikimr; + +class TYtDqTaskTransform { +public: + TYtDqTaskTransform(THashMap<TString, TString> taskParams, const NMiniKQL::IFunctionRegistry& functionRegistry) + : TaskParams(std::move(taskParams)) + , FunctionRegistry(functionRegistry) + { + } + + NMiniKQL::TCallableVisitFunc operator()(NMiniKQL::TInternName name) { + if (TaskParams.contains("yt") && (name == "DqYtRead" || name == "DqYtBlockRead")) { + return [this](NMiniKQL::TCallable& callable, const NMiniKQL::TTypeEnvironment& env) { + using namespace NMiniKQL; + + TProgramBuilder pgmBuilder(env, FunctionRegistry); + + YQL_ENSURE(callable.GetInputsCount() == 8 || callable.GetInputsCount() == 9, "Expected 8 or 9 arguments."); + + TCallableBuilder callableBuilder(env, callable.GetType()->GetName(), callable.GetType()->GetReturnType(), false); + callableBuilder.Add(callable.GetInput(0)); + callableBuilder.Add(callable.GetInput(1)); + callableBuilder.Add(callable.GetInput(2)); + callableBuilder.Add(callable.GetInput(3)); + + if (callable.GetInputsCount() == 8U) + callableBuilder.Add(callable.GetInput(4)); + else { + auto params = NYT::NodeFromYsonString(TaskParams.Value("yt", TString())).AsMap(); + + TVector<TRuntimeNode> newGrpList; + TListLiteral* groupList = AS_VALUE(TListLiteral, callable.GetInput(4)); + for (ui32 grp = 0; grp < groupList->GetItemsCount(); ++grp) { + TListLiteral* tableList = AS_VALUE(TListLiteral, groupList->GetItems()[grp]); + TVector<TRuntimeNode> newTableList; + for (ui32 i = 0; i < tableList->GetItemsCount(); ++i) { + TString paramsKey = TStringBuilder() << grp << "/" << i; + + TTupleLiteral* tableTuple = AS_VALUE(TTupleLiteral, tableList->GetItems()[i]); + YQL_ENSURE(tableTuple->GetValuesCount() == 4); + + NYT::TRichYPath richYPath; + NYT::Deserialize(richYPath, NYT::NodeFromYsonString(TString(AS_VALUE(TDataLiteral, tableTuple->GetValue(1))->AsValue().AsStringRef()))); + + if (params.contains(paramsKey)) { + NYT::TRichYPath ranges; + NYT::Deserialize(ranges, params[paramsKey]); + richYPath.MutableRanges() = ranges.GetRanges(); + } else { + richYPath.MutableRanges().ConstructInPlace(); + } + + newTableList.push_back(pgmBuilder.NewTuple({ + tableTuple->GetValue(0), + pgmBuilder.NewDataLiteral<NUdf::EDataSlot::String>(NYT::NodeToYsonString(NYT::PathToNode(richYPath))), + tableTuple->GetValue(2), + tableTuple->GetValue(3) + })); + } + newGrpList.push_back(pgmBuilder.NewList(newTableList.front().GetStaticType(), newTableList)); + } + callableBuilder.Add(pgmBuilder.NewList(newGrpList.front().GetStaticType(), newGrpList)); + } + callableBuilder.Add(callable.GetInput(5)); + callableBuilder.Add(callable.GetInput(6)); + callableBuilder.Add(callable.GetInput(7)); + return TRuntimeNode(callableBuilder.Build(), false); + }; + } + if (name == "YtDqRowsWideWrite") { + YQL_ENSURE(TaskParams.contains("yt.write.tx"), "Expected nested transaction"); + return [this](NMiniKQL::TCallable& callable, const NMiniKQL::TTypeEnvironment& env) -> NMiniKQL::TRuntimeNode { + using namespace NMiniKQL; + + TProgramBuilder pgmBuilder(env, FunctionRegistry); + + YQL_ENSURE(callable.GetInputsCount() == 6, "Expected six arguments."); + + TCallableBuilder callableBuilder(env, callable.GetType()->GetName(), callable.GetType()->GetReturnType(), false); + callableBuilder.Add(callable.GetInput(0)); + callableBuilder.Add(callable.GetInput(1)); + callableBuilder.Add(callable.GetInput(2)); + + NYT::TRichYPath richYPath; + NYT::Deserialize(richYPath, NYT::NodeFromYsonString(TString(AS_VALUE(TDataLiteral, callable.GetInput(3))->AsValue().AsStringRef()))); + richYPath.TransactionId(GetGuid(TaskParams.Value("yt.write.tx", TString()))); + callableBuilder.Add(pgmBuilder.NewDataLiteral<NUdf::EDataSlot::String>(NYT::NodeToYsonString(NYT::PathToNode(richYPath)))); + + callableBuilder.Add(callable.GetInput(4)); + callableBuilder.Add(callable.GetInput(5)); + + return TRuntimeNode(callableBuilder.Build(), false); + }; + } + + return NMiniKQL::TCallableVisitFunc(); + } + +private: + THashMap<TString, TString> TaskParams; + const NMiniKQL::IFunctionRegistry& FunctionRegistry; +}; + +TTaskTransformFactory CreateYtDqTaskTransformFactory() { + return [] (const THashMap<TString, TString>& taskParams, const NKikimr::NMiniKQL::IFunctionRegistry* funcRegistry) -> NKikimr::NMiniKQL::TCallableVisitFuncProvider { + return TYtDqTaskTransform(taskParams, *funcRegistry); + }; +} + +} // NYql diff --git a/yt/yql/providers/yt/mkql_dq/yql_yt_dq_transform.h b/yt/yql/providers/yt/mkql_dq/yql_yt_dq_transform.h new file mode 100644 index 0000000000..c95d1befd5 --- /dev/null +++ b/yt/yql/providers/yt/mkql_dq/yql_yt_dq_transform.h @@ -0,0 +1,9 @@ +#pragma once + +#include <yql/essentials/core/dq_integration/transform/yql_dq_task_transform.h> + +namespace NYql { + +TTaskTransformFactory CreateYtDqTaskTransformFactory(); + +} diff --git a/yt/yql/providers/yt/opt/ya.make b/yt/yql/providers/yt/opt/ya.make new file mode 100644 index 0000000000..f78676efda --- /dev/null +++ b/yt/yql/providers/yt/opt/ya.make @@ -0,0 +1,19 @@ +LIBRARY() + +SRCS( + yql_yt_join.cpp + yql_yt_key_selector.cpp +) + +PEERDIR( + yt/yql/providers/yt/lib/row_spec + yql/essentials/core/expr_nodes + yql/essentials/core + yql/essentials/ast +) + + + YQL_LAST_ABI_VERSION() + + +END() diff --git a/yt/yql/providers/yt/opt/yql_yt_join.cpp b/yt/yql/providers/yt/opt/yql_yt_join.cpp new file mode 100644 index 0000000000..8dd4f10aa5 --- /dev/null +++ b/yt/yql/providers/yt/opt/yql_yt_join.cpp @@ -0,0 +1,676 @@ +#include "yql_yt_join.h" + +#include <yql/essentials/core/yql_expr_type_annotation.h> +#include <yql/essentials/core/yql_opt_utils.h> +#include <yql/essentials/utils/yql_panic.h> + +namespace NYql { + +TChoice Invert(TChoice choice) { + switch (choice) { + case TChoice::None: + case TChoice::Both: + return choice; + case TChoice::Left: + return TChoice::Right; + case TChoice::Right: + return TChoice::Left; + } +} + +bool IsLeftOrRight(TChoice choice) { + return choice == TChoice::Left || choice == TChoice::Right; +} + +TChoice Merge(TChoice a, TChoice b) { + return TChoice(ui8(a) | ui8(b)); +} + +TMaybe<ui64> TMapJoinSettings::CalculatePartSize(ui64 rows) const { + if (MapJoinShardCount > 1 && rows > MapJoinShardMinRows) { + ui64 partSize = (rows + MapJoinShardCount - 1) / MapJoinShardCount; + if (partSize < MapJoinShardMinRows) { + // make less parts + partSize = MapJoinShardMinRows; + ui64 count = (rows + partSize - 1) / partSize; + if (count > 1) { + return partSize; + } + + return {}; + } + else { + return partSize; + } + } + + return {}; +} + +THashSet<TString> BuildJoinKeys(const TJoinLabel& label, const TExprNode& keys) { + THashSet<TString> result; + for (ui32 i = 0; i < keys.ChildrenSize(); i += 2) { + auto tableName = keys.Child(i)->Content(); + auto column = keys.Child(i + 1)->Content(); + result.insert(label.MemberName(tableName, column)); + } + + return result; +} + +TVector<TString> BuildJoinKeyList(const TJoinLabel& label, const TExprNode& keys) { + TVector<TString> result; + for (ui32 i = 0; i < keys.ChildrenSize(); i += 2) { + auto tableName = keys.Child(i)->Content(); + auto column = keys.Child(i + 1)->Content(); + result.push_back(label.MemberName(tableName, column)); + } + + return result; +} + +TVector<const TTypeAnnotationNode*> BuildJoinKeyType(const TJoinLabel& label, const TExprNode& keys) { + TVector<const TTypeAnnotationNode*> ret; + for (ui32 i = 0; i < keys.ChildrenSize(); i += 2) { + auto tableName = keys.Child(i)->Content(); + auto column = keys.Child(i + 1)->Content(); + auto type = label.FindColumn(tableName, column); + ret.push_back(*type); + } + + return ret; +} + +TMap<TString, const TTypeAnnotationNode*> BuildJoinKeyTypeMap(const TJoinLabel& label, const TExprNode& keys) { + auto names = BuildJoinKeyList(label, keys); + auto types = BuildJoinKeyType(label, keys); + + YQL_ENSURE(names.size() == types.size()); + + TMap<TString, const TTypeAnnotationNode*> ret; + for (size_t i = 0; i < names.size(); ++i) { + ret[names[i]] = types[i]; + } + return ret; +} + +TVector<const TTypeAnnotationNode*> RemoveNullsFromJoinKeyType(const TVector<const TTypeAnnotationNode*>& inputKeyType) { + TVector<const TTypeAnnotationNode*> ret; + for (const auto& x : inputKeyType) { + ret.push_back(RemoveOptionalType(x)); + } + + return ret; +} + +const TTypeAnnotationNode* AsDictKeyType(const TVector<const TTypeAnnotationNode*>& inputKeyType, TExprContext& ctx) { + YQL_ENSURE(inputKeyType.size() > 0); + if (inputKeyType.size() == 1) { + return inputKeyType.front(); + } else { + return ctx.MakeType<TTupleExprType>(inputKeyType); + } +} + +void SwapJoinType(TPositionHandle pos, TExprNode::TPtr& joinType, TExprContext& ctx) { + if (joinType->Content() == "RightSemi") { + joinType = ctx.NewAtom(pos, "LeftSemi"); + } + else if (joinType->Content() == "RightOnly") { + joinType = ctx.NewAtom(pos, "LeftOnly"); + } + else if (joinType->Content() == "Right") { + joinType = ctx.NewAtom(pos, "Left"); + } + else if (joinType->Content() == "LeftSemi") { + joinType = ctx.NewAtom(pos, "RightSemi"); + } + else if (joinType->Content() == "LeftOnly") { + joinType = ctx.NewAtom(pos, "RightOnly"); + } + else if (joinType->Content() == "Left") { + joinType = ctx.NewAtom(pos, "Right"); + } +} + +const TStructExprType* MakeOutputJoinColumns(const THashMap<TString, const TTypeAnnotationNode*>& columnTypes, + const TJoinLabel& label, TExprContext& ctx) { + TVector<const TItemExprType*> resultFields; + for (auto& x : label.InputType->GetItems()) { + TString fullName = label.FullName(x->GetName()); + if (auto columnType = columnTypes.FindPtr(fullName)) { + resultFields.push_back(ctx.MakeType<TItemExprType>(fullName, *columnType)); + } + } + + return ctx.MakeType<TStructExprType>(resultFields); +} + +const TTypeAnnotationNode* UnifyJoinKeyType(TPositionHandle pos, const TVector<const TTypeAnnotationNode*>& types, TExprContext& ctx) { + TTypeAnnotationNode::TListType t = types; + const TTypeAnnotationNode* commonType = CommonType(pos, t, ctx); + if (commonType && !commonType->IsOptionalOrNull()) { + NUdf::TCastResultOptions options = 0; + for (auto type : types) { + options |= CastResult<true>(type, commonType); + } + + YQL_ENSURE(!(options & NKikimr::NUdf::ECastOptions::Impossible)); + if (options & NKikimr::NUdf::ECastOptions::MayFail) { + commonType = ctx.MakeType<TOptionalExprType>(commonType); + } + } + return commonType; +} + +TVector<const TTypeAnnotationNode*> UnifyJoinKeyType(TPositionHandle pos, const TVector<const TTypeAnnotationNode*>& left, + const TVector<const TTypeAnnotationNode*>& right, TExprContext& ctx) +{ + YQL_ENSURE(left.size() == right.size()); + TVector<const TTypeAnnotationNode*> ret; + ret.reserve(left.size()); + for (size_t i = 0; i < left.size(); ++i) { + ret.push_back(UnifyJoinKeyType(pos, { left[i], right[i] }, ctx)); + } + + return ret; +} + +namespace { +bool NeedSkipNulls(const TTypeAnnotationNode& keyType, const TTypeAnnotationNode& unifiedKeyType) { + if (keyType.HasOptionalOrNull()) { + return true; + } + + auto options = CastResult<true>(&keyType, &unifiedKeyType); + YQL_ENSURE(!(options & NKikimr::NUdf::ECastOptions::Impossible)); + return options & NKikimr::NUdf::ECastOptions::MayFail; +} + +} + +TExprNode::TPtr RemapNonConvertibleItems(const TExprNode::TPtr& input, const TJoinLabel& label, + const TExprNode& keys, const TVector<const TTypeAnnotationNode*>& unifiedKeyTypes, + TExprNode::TListType& columnNodes, TExprNode::TListType& columnNodesForSkipNull, TExprContext& ctx) { + + YQL_ENSURE(keys.ChildrenSize() % 2 == 0); + YQL_ENSURE(keys.ChildrenSize() > 0); + + auto result = input; + auto keysCount = keys.ChildrenSize() / 2; + + size_t unifiedKeysCount = unifiedKeyTypes.size(); + YQL_ENSURE(keysCount == unifiedKeysCount); + + columnNodes.clear(); + columnNodesForSkipNull.clear(); + columnNodes.reserve(keysCount); + + for (ui32 i = 0; i < keysCount; ++i) { + const TTypeAnnotationNode* unifiedType = unifiedKeyTypes[i]; + + auto tableName = keys.Child(2 * i)->Content(); + auto column = keys.Child(2 * i + 1)->Content(); + auto memberName = label.MemberName(tableName, column); + const TTypeAnnotationNode* inputType = *label.FindColumn(tableName, column); + + auto arg = ctx.NewArgument(input->Pos(), "arg"); + auto columnValue = ctx.Builder(input->Pos()) + .Callable("Member") + .Add(0, arg) + .Atom(1, memberName) + .Seal() + .Build(); + + auto remapped = RemapNonConvertibleMemberForJoin(input->Pos(), columnValue, *inputType, *unifiedType, ctx); + if (remapped != columnValue) { + auto newColumn = TStringBuilder() << "_yql_ej_convert_column_" << i; + TString newMemberName = label.MemberName(tableName, newColumn); + + auto lambdaBody = ctx.Builder(input->Pos()) + .Callable("AddMember") + .Add(0, arg) + .Atom(1, newMemberName) + .Add(2, std::move(remapped)) + .Seal() + .Build(); + + auto lambda = ctx.NewLambda(input->Pos(), ctx.NewArguments(input->Pos(), {std::move(arg)}), std::move(lambdaBody)); + + result = ctx.Builder(input->Pos()) + .Callable("Map") + .Add(0, result) + .Add(1, lambda) + .Seal() + .Build(); + + columnNodes.push_back(ctx.NewAtom(input->Pos(), newMemberName)); + if (NeedSkipNulls(*inputType, *unifiedType)) { + columnNodesForSkipNull.push_back(columnNodes.back()); + } + } else { + columnNodes.push_back(ctx.NewAtom(input->Pos(), memberName)); + } + } + + return result; +} + +// generate Map/Reduce transformations which filter out null keys for each optional and non-convertible key +// TODO: merge common code with MakeCommonJoinCoreReduceLambda +TCommonJoinCoreLambdas MakeCommonJoinCoreLambdas(TPositionHandle pos, TExprContext& ctx, const TJoinLabel& label, + const TJoinLabel& otherLabel, const TVector<const TTypeAnnotationNode*>& outputKeyType, + const TExprNode& keyColumnsNode, TStringBuf joinType, + const TStructExprType* myOutputSchemeType, const TStructExprType* otherOutputSchemeType, + ui32 tableIndex, bool useSortedReduce, ui32 sortIndex, + const TMap<TStringBuf, TVector<TStringBuf>>& renameMap, + bool myData, bool otherData, const TVector<TString>& ytReduceByColumns) +{ + TCommonJoinCoreLambdas result; + auto arg = ctx.NewArgument(pos, "item"); + + TVector<const TItemExprType*> items; + TExprNode::TListType outStructItems; + TVector<const TTypeAnnotationNode*> inputKeyType; + + if (joinType != "Cross") { + YQL_ENSURE(ytReduceByColumns.size() == outputKeyType.size()); + for (ui32 i = 0; i < outputKeyType.size(); ++i) { + const auto inputTable = keyColumnsNode.Child(2 * i)->Content(); + const auto columnName = keyColumnsNode.Child(2 * i + 1)->Content(); + + const TString memberName = label.MemberName(inputTable, columnName); + + auto keyNode = ctx.NewCallable(pos, "Member", { arg, ctx.NewAtom(pos, memberName) }); + + const auto fullName = FullColumnName(inputTable, columnName); + const auto originalType = *label.FindColumn(inputTable, columnName); + inputKeyType.push_back(originalType); + + auto targetType = outputKeyType[i]; + keyNode = RemapNonConvertibleMemberForJoin(pos, keyNode, *originalType, *targetType, ctx); + + auto keyItem = ctx.Builder(pos) + .List() + .Atom(0, ytReduceByColumns[i]) + .Add(1, keyNode) + .Seal().Build(); + outStructItems.emplace_back(std::move(keyItem)); + items.emplace_back(ctx.MakeType<TItemExprType>(ytReduceByColumns[i], targetType)); + } + } else { + YQL_ENSURE(ytReduceByColumns.size() == 1); + auto keyNode = ctx.Builder(pos) + .List() + .Atom(0, ytReduceByColumns.front()) + .Callable(1, "Uint32") + .Atom(0, "0", TNodeFlags::Default) + .Seal() + .Seal().Build(); + outStructItems.emplace_back(std::move(keyNode)); + items.emplace_back(ctx.MakeType<TItemExprType>(ytReduceByColumns.front(), ctx.MakeType<TDataExprType>(EDataSlot::Uint32))); + } + + if (useSortedReduce) { + outStructItems.emplace_back( + ctx.Builder(pos) + .List() + .Atom(0, "_yql_sort") + .Callable(1, "Uint32") + .Atom(0, ToString(sortIndex), TNodeFlags::Default) + .Seal() + .Seal() + .Build()); + items.emplace_back(ctx.MakeType<TItemExprType>("_yql_sort", ctx.MakeType<TDataExprType>(EDataSlot::Uint32))); + } + + // add payload + { + YQL_ENSURE(tableIndex <= 1); + TTypeAnnotationNode::TListType variantItems(2); + + variantItems[tableIndex] = (const TTypeAnnotationNode*)label.InputType; + variantItems[1 - tableIndex] = otherLabel.InputType; + + auto variantType = ctx.MakeType<TVariantExprType>(ctx.MakeType<TTupleExprType>(std::move(variantItems))); + + outStructItems.emplace_back( + ctx.Builder(pos) + .List() + .Atom(0, "_yql_join_payload") + .Callable(1, "Variant") + .Add(0, arg) + .Atom(1, ToString(tableIndex), TNodeFlags::Default) + .Add(2, ExpandType(pos, *variantType, ctx)) + .Seal() + .Seal() + .Build()); + } + + TExprNodeList filterKeyColumns; + if (joinType == "Inner" || (joinType == (tableIndex == 0 ? "Right" : "Left")) || joinType == "LeftSemi" + || (joinType == (tableIndex == 0 ? "RightOnly" : "LeftOnly")) || joinType == "RightSemi") + { + YQL_ENSURE(ytReduceByColumns.size() == outputKeyType.size()); + YQL_ENSURE(inputKeyType.size() == outputKeyType.size()); + for (size_t i = 0; i < ytReduceByColumns.size(); ++i) { + if (NeedSkipNulls(*inputKeyType[i], *outputKeyType[i])) { + filterKeyColumns.push_back(ctx.NewAtom(pos, ytReduceByColumns[i])); + } + } + } + + auto body = ctx.NewCallable(pos, "Just", { ctx.NewCallable(pos, "AsStruct", std::move(outStructItems)) }); + if (!filterKeyColumns.empty()) { + body = ctx.NewCallable(pos, "SkipNullMembers", { body, ctx.NewList(pos, std::move(filterKeyColumns)) }); + } + + result.MapLambda = ctx.NewLambda(pos, ctx.NewArguments(pos, { std::move(arg) }), std::move(body)); + + arg = ctx.NewArgument(pos, "item"); + + // fill myRow using arg + TExprNode::TListType outValueItems; + auto& myRowItems = myOutputSchemeType->GetItems(); + if (myData) { + for (auto item : myRowItems) { + if (auto renamed = renameMap.FindPtr(item->GetName())) { + if (renamed->empty()) { + continue; + } + } + + auto inputTable = label.TableName(item->GetName()); + auto columnName = label.ColumnName(item->GetName()); + TString memberName = label.AddLabel ? label.MemberName(inputTable, columnName) : TString(item->GetName()); + + auto pair = ctx.Builder(pos) + .List() + .Atom(0, item->GetName()) + .Callable(1, "Member") + .Callable(0, "Just") + .Add(0, arg) + .Seal() + .Atom(1, memberName) + .Seal() + .Seal() + .Build(); + outValueItems.push_back(pair); + items.emplace_back(ctx.MakeType<TItemExprType>(item->GetName(), item->GetItemType()->IsOptionalOrNull() ? item->GetItemType() : ctx.MakeType<TOptionalExprType>(item->GetItemType()))); + } + } + + if (otherData) { + auto& otherRowItems = otherOutputSchemeType->GetItems(); + for (auto item : otherRowItems) { + if (auto renamed = renameMap.FindPtr(item->GetName())) { + if (renamed->empty()) { + continue; + } + } + + auto columnType = item->GetItemType(); + if (!columnType->IsOptionalOrNull()) { + columnType = ctx.MakeType<TOptionalExprType>(columnType); + } + + auto pair = ctx.Builder(pos) + .List() + .Atom(0, item->GetName()) + .Callable(1, "Nothing") + .Add(0, ExpandType(pos, *columnType, ctx)) + .Seal() + .Seal().Build(); + outValueItems.emplace_back(std::move(pair)); + items.emplace_back(ctx.MakeType<TItemExprType>(item->GetName(), item->GetItemType()->IsOptionalOrNull() ? item->GetItemType() : ctx.MakeType<TOptionalExprType>(item->GetItemType()))); + } + } + + auto tableIndexNode = ctx.Builder(pos) + .List() + .Atom(0, "_yql_table_index") + .Callable(1, "Uint32") + .Atom(0, ToString(tableIndex), TNodeFlags::Default) + .Seal() + .Seal().Build(); + outValueItems.emplace_back(std::move(tableIndexNode)); + items.emplace_back(ctx.MakeType<TItemExprType>("_yql_table_index", ctx.MakeType<TDataExprType>(EDataSlot::Uint32))); + + result.ReduceLambda = ctx.NewLambda(pos, ctx.NewArguments(pos, { std::move(arg) }), + ctx.NewCallable(pos, "AsStruct", std::move(outValueItems))); + result.CommonJoinCoreInputType = ctx.MakeType<TStructExprType>(items); + return result; +} + +TExprNode::TPtr PrepareForCommonJoinCore(TPositionHandle pos, TExprContext& ctx, const TExprNode::TPtr& input, + const TExprNode::TPtr& reduceLambdaZero, const TExprNode::TPtr& reduceLambdaOne) +{ + return ctx.Builder(pos) + .Callable("OrderedMap") + .Add(0, input) + .Lambda(1) + .Param("item") + .Callable("FlattenMembers") + .List(0) + .Atom(0, "") + .Callable(1, "RemoveMember") + .Arg(0, "item") + .Atom(1, "_yql_join_payload") + .Seal() + .Seal() + .List(1) + .Atom(0, "") + .Callable(1, "Visit") + .Callable(0, "Member") + .Arg(0, "item") + .Atom(1, "_yql_join_payload") + .Seal() + .Atom(1, "0", TNodeFlags::Default) + .Add(2, reduceLambdaZero) + .Atom(3, "1", TNodeFlags::Default) + .Add(4, reduceLambdaOne) + .Seal() + .Seal() + .Seal() + .Seal() + .Seal() + .Build(); +} + +// generate Reduce-only transformations which filter out null keys for each optional and non-convertible key +TCommonJoinCoreLambdas MakeCommonJoinCoreReduceLambda(TPositionHandle pos, TExprContext& ctx, const TJoinLabel& label, + const TVector<const TTypeAnnotationNode*>& outputKeyType, const TExprNode& keyColumnsNode, + TStringBuf joinType, const TStructExprType* myOutputSchemeType, const TStructExprType* otherOutputSchemeType, + ui32 tableIndex, bool useSortedReduce, ui32 sortIndex, + const TMap<TStringBuf, TVector<TStringBuf>>& renameMap, + bool myData, bool otherData, const TVector<TString>& ytReduceByColumns) +{ + + TExprNode::TListType keyNodes; + TVector<std::pair<TStringBuf, TStringBuf>> inputKeyColumns; + TVector<const TItemExprType*> items; + TVector<const TTypeAnnotationNode*> keyTypes; + + TTypeAnnotationNode::TListType outputKeyColumnsTypes; + if (joinType != "Cross") { + for (ui32 i = 0; i < outputKeyType.size(); ++i) { + inputKeyColumns.push_back({ keyColumnsNode.Child(2 * i)->Content(), + keyColumnsNode.Child(2 * i + 1)->Content() }); + + outputKeyColumnsTypes.push_back(outputKeyType[i]); + } + } + + auto arg = ctx.NewArgument(pos, "item"); + for (ui32 i = 0; i < inputKeyColumns.size(); ++i) { + const auto& inputTable = inputKeyColumns[i].first; + const auto& columnName = inputKeyColumns[i].second; + const TString memberName = label.MemberName(inputTable, columnName); + + keyNodes.emplace_back(ctx.NewCallable(pos, "Member", { arg, ctx.NewAtom(pos, memberName) })); + + auto& keyNode = keyNodes.back(); + const auto fullName = FullColumnName(inputTable, columnName); + const auto originalType = *label.FindColumn(inputTable, columnName); + + auto targetType = outputKeyColumnsTypes[i]; + if (!targetType->IsOptionalOrNull()) { + targetType = ctx.MakeType<TOptionalExprType>(targetType); + } + keyNode = RemapNonConvertibleMemberForJoin(pos, keyNode, *originalType, *targetType, ctx); + keyTypes.emplace_back(targetType); + } + + // fill myRow using arg + TExprNode::TListType outValueItems; + auto& myRowItems = myOutputSchemeType->GetItems(); + if (myData) { + for (auto item : myRowItems) { + if (auto renamed = renameMap.FindPtr(item->GetName())) { + if (renamed->empty()) { + continue; + } + } + + auto inputTable = label.TableName(item->GetName()); + auto columnName = label.ColumnName(item->GetName()); + TString memberName = label.AddLabel ? label.MemberName(inputTable, columnName) : TString(item->GetName()); + auto columnValue = ctx.Builder(pos) + .Callable("Member") + .Add(0, arg) + .Atom(1, memberName) + .Seal() + .Build(); + + if (!item->GetItemType()->IsOptionalOrNull()) { + columnValue = ctx.NewCallable(pos, "Just", { columnValue }); + } + + auto pair = ctx.Builder(pos) + .List() + .Atom(0, item->GetName()) + .Add(1, columnValue) + .Seal() + .Build(); + outValueItems.push_back(pair); + items.emplace_back(ctx.MakeType<TItemExprType>(item->GetName(), item->GetItemType()->IsOptionalOrNull() ? item->GetItemType() : ctx.MakeType<TOptionalExprType>(item->GetItemType()))); + } + } + + if (otherData) { + auto& otherRowItems = otherOutputSchemeType->GetItems(); + for (auto item : otherRowItems) { + if (auto renamed = renameMap.FindPtr(item->GetName())) { + if (renamed->empty()) { + continue; + } + } + + auto columnType = item->GetItemType(); + if (!columnType->IsOptionalOrNull()) { + columnType = ctx.MakeType<TOptionalExprType>(columnType); + } + + auto pair = ctx.Builder(pos) + .List() + .Atom(0, item->GetName()) + .Callable(1, "Nothing") + .Add(0, ExpandType(pos, *columnType, ctx)) + .Seal() + .Seal().Build(); + outValueItems.emplace_back(std::move(pair)); + items.emplace_back(ctx.MakeType<TItemExprType>(item->GetName(), item->GetItemType()->IsOptionalOrNull() ? item->GetItemType() : ctx.MakeType<TOptionalExprType>(item->GetItemType()))); + } + } + + auto tableIndexNode = ctx.Builder(pos) + .List() + .Atom(0, "_yql_table_index") + .Callable(1, "Uint32") + .Atom(0, ToString(tableIndex), TNodeFlags::Default) + .Seal() + .Seal().Build(); + outValueItems.emplace_back(std::move(tableIndexNode)); + items.emplace_back(ctx.MakeType<TItemExprType>("_yql_table_index", ctx.MakeType<TDataExprType>(EDataSlot::Uint32))); + + TExprNode::TListType outStructItems; + outStructItems = outValueItems; + + if (useSortedReduce) { + outStructItems.emplace_back(ctx.Builder(pos) + .List() + .Atom(0, "_yql_sort") + .Callable(1, "Uint32") + .Atom(0, ToString(sortIndex), TNodeFlags::Default) + .Seal() + .Seal().Build()); + items.emplace_back(ctx.MakeType<TItemExprType>("_yql_sort", ctx.MakeType<TDataExprType>(EDataSlot::Uint32))); + } + + // fill keys in out struct + if (joinType == "Cross") { + auto keyNode = ctx.Builder(pos) + .List() + .Atom(0, ytReduceByColumns.front()) + .Callable(1, "Uint32") + .Atom(0, "0", TNodeFlags::Default) + .Seal() + .Seal().Build(); + outStructItems.emplace_back(std::move(keyNode)); + items.emplace_back(ctx.MakeType<TItemExprType>(ytReduceByColumns.front(), ctx.MakeType<TDataExprType>(EDataSlot::Uint32))); + } else { + for (ui32 i = 0; i < keyNodes.size(); ++i) { + auto keyNode = ctx.Builder(pos) + .List() + .Atom(0, ytReduceByColumns[i]) + .Add(1, keyNodes[i]) + .Seal().Build(); + outStructItems.emplace_back(std::move(keyNode)); + items.emplace_back(ctx.MakeType<TItemExprType>(ytReduceByColumns[i], keyTypes[i])); + } + } + + TExprNodeList filterKeyColumns; + + if (joinType == "Inner" || (joinType == (tableIndex == 0 ? "Right" : "Left")) || joinType == "LeftSemi" + || (joinType == (tableIndex == 0 ? "RightOnly" : "LeftOnly")) || joinType == "RightSemi") + { + for (const auto &keyCol : ytReduceByColumns) { + filterKeyColumns.push_back(ctx.NewAtom(pos, keyCol)); + } + } + + auto body = ctx.Builder(pos) + .Callable("SkipNullMembers") + .Callable(0, "Just") + .Add(0, ctx.NewCallable(pos, "AsStruct", std::move(outStructItems))) + .Seal() + .Add(1, ctx.NewList(pos, std::move(filterKeyColumns))) + .Seal() + .Build(); + + TCommonJoinCoreLambdas result; + result.ReduceLambda = ctx.NewLambda(pos, ctx.NewArguments(pos, { std::move(arg) }), std::move(body)); + result.CommonJoinCoreInputType = ctx.MakeType<TStructExprType>(items); + return result; +} + +void AddJoinRemappedColumn(TPositionHandle pos, const TExprNode::TPtr& pairArg, TExprNode::TListType& joinedBodyChildren, + TStringBuf name, TStringBuf newName, TExprContext& ctx) +{ + auto member = ctx.Builder(pos) + .Callable("Member") + .Add(0, pairArg) + .Atom(1, name) + .Seal() + .Build(); + + joinedBodyChildren.emplace_back(ctx.Builder(pos) + .List() + .Atom(0, newName) + .Add(1, std::move(member)) + .Seal() + .Build()); +} + +} diff --git a/yt/yql/providers/yt/opt/yql_yt_join.h b/yt/yql/providers/yt/opt/yql_yt_join.h new file mode 100644 index 0000000000..3ccc860f7f --- /dev/null +++ b/yt/yql/providers/yt/opt/yql_yt_join.h @@ -0,0 +1,91 @@ +#pragma once + +#include <yql/essentials/core/yql_join.h> +#include <yql/essentials/ast/yql_expr.h> +#include <yql/essentials/ast/yql_expr_builder.h> + +#include <util/generic/maybe.h> +#include <util/generic/hash_set.h> +#include <util/generic/hash.h> +#include <util/generic/string.h> +#include <util/generic/strbuf.h> +#include <util/generic/vector.h> +#include <util/generic/set.h> + +namespace NYql { + +enum class TChoice : ui8 { + None = 0, + Left, + Right, + Both, +}; + +TChoice Invert(TChoice choice); +bool IsLeftOrRight(TChoice choice); +TChoice Merge(TChoice a, TChoice b); + +struct TMapJoinSettings { + ui64 MapJoinLimit = 0; + ui64 MapJoinShardMinRows = 1; + ui64 MapJoinShardCount = 1; + bool SwapTables = false; + ui64 LeftRows = 0; + ui64 RightRows = 0; + ui64 LeftSize = 0; + ui64 RightSize = 0; + ui64 LeftMemSize = 0; + ui64 RightMemSize = 0; + bool LeftUnique = false; + bool RightUnique = false; + ui64 LeftCount = 0; + ui64 RightCount = 0; + + TMaybe<ui64> CalculatePartSize(ui64 rows) const; +}; + +THashSet<TString> BuildJoinKeys(const TJoinLabel& label, const TExprNode& keys); +TVector<TString> BuildJoinKeyList(const TJoinLabel& label, const TExprNode& keys); +TVector<const TTypeAnnotationNode*> BuildJoinKeyType(const TJoinLabel& label, const TExprNode& keys); +TMap<TString, const TTypeAnnotationNode*> BuildJoinKeyTypeMap(const TJoinLabel& label, const TExprNode& keys); +TVector<const TTypeAnnotationNode*> RemoveNullsFromJoinKeyType(const TVector<const TTypeAnnotationNode*>& inputKeyType); +const TTypeAnnotationNode* AsDictKeyType(const TVector<const TTypeAnnotationNode*>& inputKeyType, TExprContext& ctx); +void SwapJoinType(TPositionHandle pos, TExprNode::TPtr& joinType, TExprContext& ctx); +const TStructExprType* MakeOutputJoinColumns(const THashMap<TString, const TTypeAnnotationNode*>& columnTypes, + const TJoinLabel& label, TExprContext& ctx); +const TTypeAnnotationNode* UnifyJoinKeyType(TPositionHandle pos, const TVector<const TTypeAnnotationNode*>& types, TExprContext& ctx); +TVector<const TTypeAnnotationNode*> UnifyJoinKeyType(TPositionHandle pos, const TVector<const TTypeAnnotationNode*>& left, + const TVector<const TTypeAnnotationNode*>& right, TExprContext& ctx); +TExprNode::TPtr RemapNonConvertibleItems(const TExprNode::TPtr& input, const TJoinLabel& label, + const TExprNode& keys, const TVector<const TTypeAnnotationNode*>& unifiedKeyTypes, + TExprNode::TListType& columnNodes, TExprNode::TListType& columnNodesForSkipNull, TExprContext& ctx); + +struct TCommonJoinCoreLambdas { + TExprNode::TPtr MapLambda; + TExprNode::TPtr ReduceLambda; + const TTypeAnnotationNode* CommonJoinCoreInputType = nullptr; +}; + +TCommonJoinCoreLambdas MakeCommonJoinCoreLambdas(TPositionHandle pos, TExprContext& ctx, const TJoinLabel& label, + const TJoinLabel& otherLabel, const TVector<const TTypeAnnotationNode*>& outputKeyType, + const TExprNode& keyColumnsNode, TStringBuf joinType, + const TStructExprType* myOutputSchemeType, const TStructExprType* otherOutputSchemeType, + ui32 tableIndex, bool useSortedReduce, ui32 sortIndex, + const TMap<TStringBuf, TVector<TStringBuf>>& renameMap, + bool myData, bool otherData, const TVector<TString>& ytReduceByColumns); +TExprNode::TPtr PrepareForCommonJoinCore(TPositionHandle pos, TExprContext& ctx, const TExprNode::TPtr& input, + const TExprNode::TPtr& reduceLambdaZero, const TExprNode::TPtr& reduceLambdaOne); + + +TCommonJoinCoreLambdas MakeCommonJoinCoreReduceLambda(TPositionHandle pos, TExprContext& ctx, const TJoinLabel& label, + const TVector<const TTypeAnnotationNode*>& outputKeyType, + const TExprNode& keyColumnsNode, TStringBuf joinType, + const TStructExprType* myOutputSchemeType, const TStructExprType* otherOutputSchemeType, + ui32 tableIndex, bool useSortedReduce, ui32 sortIndex, + const TMap<TStringBuf, TVector<TStringBuf>>& renameMap, + bool myData, bool otherData, const TVector<TString>& ytReduceByColumns); + +void AddJoinRemappedColumn(TPositionHandle pos, const TExprNode::TPtr& pairArg, TExprNode::TListType& joinedBodyChildren, + TStringBuf name, TStringBuf newName, TExprContext& ctx); + +} diff --git a/yt/yql/providers/yt/opt/yql_yt_key_selector.cpp b/yt/yql/providers/yt/opt/yql_yt_key_selector.cpp new file mode 100644 index 0000000000..9ba1d72ce0 --- /dev/null +++ b/yt/yql/providers/yt/opt/yql_yt_key_selector.cpp @@ -0,0 +1,347 @@ +#include "yql_yt_key_selector.h" + +#include <yql/essentials/core/expr_nodes/yql_expr_nodes.h> +#include <yql/essentials/core/yql_expr_type_annotation.h> +#include <yql/essentials/core/yql_opt_utils.h> + +#include <util/generic/strbuf.h> +#include <util/string/type.h> + +namespace NYql { + +using namespace NNodes; + +static TMaybeNode<TCoLambda> GetEqualVisitKeyExtractorLambda(const TCoLambda& lambda) { + const auto members = GetCommonKeysFromVariantSelector(lambda); + if (!members.empty()) { + auto maybeVisit = lambda.Body().Maybe<TCoVisit>(); + return TCoLambda(maybeVisit.Raw()->Child(2)); + } + return {}; +} + +TKeySelectorBuilder::TKeySelectorBuilder(TPositionHandle pos, TExprContext& ctx, bool useNativeDescSort, + const TTypeAnnotationNode* itemType) + : Pos_(pos) + , Ctx_(ctx) + , Arg_(Build<TCoArgument>(ctx, pos).Name("item").Done().Ptr()) + , LambdaBody_(Arg_) + , NonStructInput(itemType && itemType->GetKind() != ETypeAnnotationKind::Struct) + , UseNativeDescSort(useNativeDescSort) +{ + if (itemType) { + if (itemType->GetKind() == ETypeAnnotationKind::Struct) { + StructType = itemType->Cast<TStructExprType>(); + FieldTypes_ = StructType->GetItems(); + } + else { + FieldTypes_.push_back(ctx.MakeType<TItemExprType>("_yql_original_row", itemType)); + LambdaBody_ = Build<TCoAsStruct>(Ctx_, Pos_) + .Add<TExprList>() + .Add<TCoAtom>() + .Value("_yql_original_row") + .Build() + .Add(Arg_) + .Build() + .Done() + .Ptr(); + } + } +} + +void TKeySelectorBuilder::ProcessKeySelector(const TExprNode::TPtr& keySelectorLambda, const TExprNode::TPtr& sortDirections, bool unordered) { + YQL_ENSURE(!unordered || !sortDirections); + auto lambda = TCoLambda(keySelectorLambda); + if (auto maybeLambda = GetEqualVisitKeyExtractorLambda(lambda)) { + lambda = maybeLambda.Cast(); + } + + auto keySelectorBody = lambda.Body(); + auto keySelectorArg = lambda.Args().Arg(0).Ptr(); + + const bool allAscending = !sortDirections; + const bool multiKey = sortDirections + ? (sortDirections->IsList() && sortDirections->ChildrenSize() != 1) + : keySelectorBody.Maybe<TExprList>() || keySelectorBody.Ref().GetTypeAnn()->GetKind() == ETypeAnnotationKind::Tuple; + + if (multiKey) { + if (keySelectorBody.Maybe<TExprList>()) { + auto columnList = keySelectorBody.Cast<TExprList>(); + for (size_t i = 0; i < columnList.Size(); ++i) { + AddColumn<false, false>(keySelectorLambda, + columnList.Item(i).Ptr(), + allAscending || IsTrue(TExprList(sortDirections).Item(i).Cast<TCoBool>().Literal().Value()), + i, + keySelectorArg, + unordered); + } + } else { + size_t tupleSize = keySelectorBody.Ref().GetTypeAnn()->Cast<TTupleExprType>()->GetItems().size(); + for (size_t i = 0; i < tupleSize; ++i) { + AddColumn<true, false>(keySelectorLambda, + keySelectorBody.Ptr(), + allAscending || IsTrue(TExprList(sortDirections).Item(i).Cast<TCoBool>().Literal().Value()), + i, + keySelectorArg, + unordered); + } + } + } else { + const bool ascending = allAscending || ( + sortDirections->IsList() + ? IsTrue(TExprList(sortDirections).Item(0).Cast<TCoBool>().Literal().Value()) + : IsTrue(TCoBool(sortDirections).Literal().Value()) + ); + AddColumn<false, true>(keySelectorLambda, + keySelectorBody.Ptr(), + ascending, + 0, + keySelectorArg, + unordered); + } +} + +void TKeySelectorBuilder::ProcessConstraint(const TSortedConstraintNode& sortConstraint) { + YQL_ENSURE(StructType); + for (const auto& item : sortConstraint.GetContent()) { + bool good = false; + for (const auto& path : item.first) { + if (path.size() == 1U) { + const auto& column = path.front(); + const auto pos = StructType->FindItem(column); + YQL_ENSURE(pos, "Column " << column << " is missing in struct type"); + AddColumn(column, StructType->GetItems()[*pos]->GetItemType(), item.second, false); + good = true; + break; + } + } + if (!good) + break; + } +} + +void TKeySelectorBuilder::ProcessRowSpec(const TYqlRowSpecInfo& rowSpec) { + auto& columns = rowSpec.SortMembers; + auto& dirs = rowSpec.SortDirections; + YQL_ENSURE(columns.size() <= dirs.size()); + YQL_ENSURE(StructType); + for (size_t i = 0; i < columns.size(); ++i) { + auto pos = StructType->FindItem(columns[i]); + YQL_ENSURE(pos, "Column " << columns[i] << " is missing in struct type"); + AddColumn(columns[i], StructType->GetItems()[*pos]->GetItemType(), dirs[i], false); + } +} + +TExprNode::TPtr TKeySelectorBuilder::MakeRemapLambda(bool ordered) const { + return Build<TCoLambda>(Ctx_, Pos_) + .Args({TStringBuf("stream")}) + .Body<TCoFlatMapBase>() + .CallableName(ordered ? TCoOrderedFlatMap::CallableName() : TCoFlatMap::CallableName()) + .Input(TStringBuf("stream")) + .Lambda() + .Args(TCoArgument(Arg_)) + .Body<TCoJust>() + .Input(LambdaBody_) + .Build() + .Build() + .Build() + .Done() + .Ptr(); +} + +template <bool ComputedTuple, bool SingleColumn> +void TKeySelectorBuilder::AddColumn(const TExprNode::TPtr& rootLambda, const TExprNode::TPtr& keyNode, bool ascending, size_t columnIndex, + const TExprNode::TPtr& structArg, bool unordered) { + const TTypeAnnotationNode* columnType = nullptr; + if (ComputedTuple) { + columnType = keyNode->GetTypeAnn()->Cast<TTupleExprType>()->GetItems()[columnIndex]; + } else { + columnType = keyNode->GetTypeAnn(); + } + + auto presortColumnType = columnType; + bool needPresort = false; + if (ascending) { + needPresort = RemoveOptionalType(columnType)->GetKind() != ETypeAnnotationKind::Data; + } else { + needPresort = true; + } + + if (needPresort) { + presortColumnType = Ctx_.MakeType<TDataExprType>(EDataSlot::String); + } + + auto maybeMember = TMaybeNode<TCoMember>(keyNode); + if (!ComputedTuple && maybeMember + && (maybeMember.Cast().Struct().Raw() == structArg.Get() + || maybeMember.Cast().Struct().Maybe<TCoVariantItem>().Variant().Raw() == structArg.Get())) + { + auto memberName = TString{maybeMember.Cast().Name().Value()}; + if (!HasComputedColumn_) { + Members_.push_back(memberName); + } + if (!UniqMemberColumns_.insert(memberName).second) { + return; + } + // Reset descending presort only for non-computed fields + if (!ascending && UseNativeDescSort && RemoveOptionalType(columnType)->GetKind() == ETypeAnnotationKind::Data) { + needPresort = false; + } + + if (!needPresort) { + Columns_.push_back(memberName); + ColumnTypes_.push_back(columnType); + SortDirections_.push_back(ascending); + ForeignSortDirections_.push_back(ascending); + if (NonStructInput) { + FieldTypes_.push_back(Ctx_.MakeType<TItemExprType>(memberName, columnType)); + auto key = TExprBase(Ctx_.ReplaceNode(rootLambda->TailPtr(), rootLambda->Head().Head(), Arg_)); + if (!SingleColumn && rootLambda->GetTypeAnn()->GetKind() == ETypeAnnotationKind::Tuple) { + key = Build<TCoNth>(Ctx_, Pos_) + .Tuple(key) + .Index().Value(ToString(columnIndex)).Build() + .Done(); + } + LambdaBody_ = Build<TCoAddMember>(Ctx_, Pos_) + .Struct(LambdaBody_) + .Name().Value(memberName).Build() + .Item(key) + .Done() + .Ptr(); + } + return; + } + } else { + HasComputedColumn_ = true; + } + + NeedMap_ = true; + + auto key = TExprBase(Ctx_.ReplaceNode(rootLambda->TailPtr(), rootLambda->Head().Head(), Arg_)); + if (ComputedTuple || (!SingleColumn && rootLambda->GetTypeAnn()->GetKind() == ETypeAnnotationKind::Tuple)) { + key = Build<TCoNth>(Ctx_, Pos_) + .Tuple(key) + .Index().Value(ToString(columnIndex)).Build() + .Done(); + } + if (needPresort) { + if (unordered) { + key = Build<TCoStablePickle>(Ctx_, Pos_) + .Value(key) + .Done(); + } else { + if (ascending) { + key = Build<TCoAscending>(Ctx_, Pos_) + .Input(key) + .Done(); + } else { + key = Build<TCoDescending>(Ctx_, Pos_) + .Input(key) + .Done(); + } + } + columnType = presortColumnType; + } + + TString column = TStringBuilder() << "_yql_column_" << Index_++; + Columns_.push_back(column); + ColumnTypes_.push_back(columnType); + SortDirections_.push_back(ascending); + ForeignSortDirections_.push_back(true); + FieldTypes_.push_back(Ctx_.MakeType<TItemExprType>(column, presortColumnType)); + + LambdaBody_ = Build<TCoAddMember>(Ctx_, Pos_) + .Struct(LambdaBody_) + .Name().Value(column).Build() + .Item(key) + .Done() + .Ptr(); +} + +void TKeySelectorBuilder::AddColumn(const TStringBuf memberName, const TTypeAnnotationNode* columnType, + bool ascending, bool unordered) { + auto presortColumnType = columnType; + bool needPresort = false; + if (ascending) { + needPresort = RemoveOptionalType(columnType)->GetKind() != ETypeAnnotationKind::Data; + } else { + needPresort = !UseNativeDescSort || RemoveOptionalType(columnType)->GetKind() != ETypeAnnotationKind::Data; + } + + if (needPresort) { + presortColumnType = Ctx_.MakeType<TDataExprType>(EDataSlot::String); + } + + Members_.emplace_back(memberName); + if (!UniqMemberColumns_.emplace(memberName).second) { + return; + } + if (!needPresort) { + Columns_.emplace_back(memberName); + ColumnTypes_.push_back(columnType); + SortDirections_.push_back(ascending); + ForeignSortDirections_.push_back(ascending); + return; + } + + NeedMap_ = true; + + TExprBase key = Build<TCoMember>(Ctx_, Pos_) + .Struct(Arg_) + .Name() + .Value(memberName) + .Build() + .Done(); + + if (needPresort) { + if (unordered) { + key = Build<TCoStablePickle>(Ctx_, Pos_) + .Value(key) + .Done(); + } else { + if (ascending) { + key = Build<TCoAscending>(Ctx_, Pos_) + .Input(key) + .Done(); + } else { + key = Build<TCoDescending>(Ctx_, Pos_) + .Input(key) + .Done(); + } + } + + columnType = presortColumnType; + } + + TString column = TStringBuilder() << "_yql_column_" << Index_++; + Columns_.push_back(column); + ColumnTypes_.push_back(columnType); + SortDirections_.push_back(ascending); + ForeignSortDirections_.push_back(true); + FieldTypes_.push_back(Ctx_.MakeType<TItemExprType>(column, presortColumnType)); + + LambdaBody_ = Build<TCoAddMember>(Ctx_, Pos_) + .Struct(LambdaBody_) + .Name().Value(column).Build() + .Item(key) + .Done() + .Ptr(); +} + +TVector<std::pair<TString, bool>> TKeySelectorBuilder::ForeignSortColumns() const { + TVector<std::pair<TString, bool>> res; + YQL_ENSURE(ForeignSortDirections_.size() == Columns_.size()); + for (size_t i = 0; i < Columns_.size(); ++i) { + res.emplace_back(Columns_[i], ForeignSortDirections_[i]); + } + return res; +} + +void TKeySelectorBuilder::FillRowSpecSort(TYqlRowSpecInfo& rowSpec) { + rowSpec.SortMembers = Members_; + rowSpec.SortedBy = Columns_; + rowSpec.SortedByTypes = ColumnTypes_; + rowSpec.SortDirections = SortDirections_; +} + +} diff --git a/yt/yql/providers/yt/opt/yql_yt_key_selector.h b/yt/yql/providers/yt/opt/yql_yt_key_selector.h new file mode 100644 index 0000000000..04401117d0 --- /dev/null +++ b/yt/yql/providers/yt/opt/yql_yt_key_selector.h @@ -0,0 +1,76 @@ +#pragma once + +#include <yt/yql/providers/yt/lib/row_spec/yql_row_spec.h> + +#include <yql/essentials/ast/yql_expr.h> +#include <yql/essentials/ast/yql_constraint.h> + +#include <util/generic/vector.h> +#include <util/generic/string.h> +#include <util/generic/hash_set.h> + +namespace NYql { + +class TKeySelectorBuilder { +public: + TKeySelectorBuilder(TPositionHandle pos, TExprContext& ctx, bool useNativeDescSort, + const TTypeAnnotationNode* itemType = nullptr); + + void ProcessKeySelector(const TExprNode::TPtr& keySelectorLambda, + const TExprNode::TPtr& sortDirections = {}, + bool unordered = false); + void ProcessConstraint(const TSortedConstraintNode& sortConstraint); + void ProcessRowSpec(const TYqlRowSpecInfo& rowSpec); + + bool NeedMap() const { + return NeedMap_; + } + + TExprNode::TPtr MakeRemapLambda(bool ordered = false) const; + + const TStructExprType* MakeRemapType() const { + return Ctx_.MakeType<TStructExprType>(FieldTypes_); + } + + const TVector<bool>& SortDirections() const { + return SortDirections_; + } + + const TVector<TString>& Columns() const { + return Columns_; + } + + const TVector<TString>& Members() const { + return Members_; + } + + TVector<std::pair<TString, bool>> ForeignSortColumns() const; + + void FillRowSpecSort(TYqlRowSpecInfo& rowSpec); + +private: + template <bool ComputedTuple, bool SingleColumn> + void AddColumn(const TExprNode::TPtr& rootLambda, const TExprNode::TPtr& keyNode, bool ascending, size_t columnIndex, const TExprNode::TPtr& structArg, bool unordered); + void AddColumn(const TStringBuf memberName, const TTypeAnnotationNode* columnType, bool ascending, bool unordered); + +private: + TPositionHandle Pos_; + TExprContext& Ctx_; + const TStructExprType* StructType = nullptr; + TVector<const TItemExprType*> FieldTypes_; + bool NeedMap_ = false; + TExprNode::TPtr Arg_; + TExprNode::TPtr LambdaBody_; + TVector<bool> SortDirections_; + TVector<bool> ForeignSortDirections_; + TVector<TString> Columns_; + TVector<TString> Members_; + TTypeAnnotationNode::TListType ColumnTypes_; + bool HasComputedColumn_ = false; + ui32 Index_ = 0; + THashSet<TString> UniqMemberColumns_; + const bool NonStructInput; + const bool UseNativeDescSort; +}; + +} diff --git a/yt/yql/providers/yt/proto/source.proto b/yt/yql/providers/yt/proto/source.proto new file mode 100644 index 0000000000..dedd5df130 --- /dev/null +++ b/yt/yql/providers/yt/proto/source.proto @@ -0,0 +1,9 @@ +syntax = "proto3"; + +package NYql.NYt.NSource; + +message TLookupSource { + string Cluster = 1; + string Table = 2; + string RowSpec = 3; +}
\ No newline at end of file diff --git a/yt/yql/providers/yt/proto/ya.make b/yt/yql/providers/yt/proto/ya.make new file mode 100644 index 0000000000..6ef74a07fc --- /dev/null +++ b/yt/yql/providers/yt/proto/ya.make @@ -0,0 +1,11 @@ +PROTO_LIBRARY() + +ONLY_TAGS(CPP_PROTO) + + +SRCS( + source.proto +) + +END() + diff --git a/yt/yql/providers/yt/provider/phy_opt/yql_yt_phy_opt.cpp b/yt/yql/providers/yt/provider/phy_opt/yql_yt_phy_opt.cpp new file mode 100644 index 0000000000..51c9d75136 --- /dev/null +++ b/yt/yql/providers/yt/provider/phy_opt/yql_yt_phy_opt.cpp @@ -0,0 +1,92 @@ + +#include "yql_yt_phy_opt.h" + +#include <yql/essentials/providers/result/expr_nodes/yql_res_expr_nodes.h> +#include <yql/providers/stat/expr_nodes/yql_stat_expr_nodes.h> + +#include <yql/essentials/utils/log/log.h> + + +namespace NYql { + +using namespace NNodes; +using namespace NPrivate; + +TYtPhysicalOptProposalTransformer::TYtPhysicalOptProposalTransformer(TYtState::TPtr state) + : TOptimizeTransformerBase(state->Types, NLog::EComponent::ProviderYt, state->Configuration->DisableOptimizers.Get().GetOrElse(TSet<TString>())) + , State_(state) +{ +#define HNDL(name) "PhysicalOptimizer-"#name, Hndl(&TYtPhysicalOptProposalTransformer::name) + AddHandler(0, &TCoMux::Match, HNDL(Mux)); + AddHandler(0, &TYtWriteTable::Match, HNDL(Write)); + if (!State_->Configuration->_EnableYtDqProcessWriteConstraints.Get().GetOrElse(DEFAULT_ENABLE_DQ_WRITE_CONSTRAINTS)) { + AddHandler(0, &TYtWriteTable::Match, HNDL(DqWrite)); + } + AddHandler(0, Names({TCoLength::CallableName(), TCoHasItems::CallableName()}), HNDL(Length)); + AddHandler(0, &TCoSort::Match, HNDL(Sort<false>)); + AddHandler(0, &TCoTopSort::Match, HNDL(Sort<true>)); + AddHandler(0, &TCoTop::Match, HNDL(Sort<true>)); + AddHandler(0, &TYtSort::Match, HNDL(YtSortOverAlreadySorted)); + AddHandler(0, &TCoPartitionByKeyBase::Match, HNDL(PartitionByKey)); + AddHandler(0, &TCoFlatMapBase::Match, HNDL(FlatMap)); + AddHandler(0, &TCoCombineByKey::Match, HNDL(CombineByKey)); + AddHandler(0, &TCoLMap::Match, HNDL(LMap<TCoLMap>)); + AddHandler(0, &TCoOrderedLMap::Match, HNDL(LMap<TCoOrderedLMap>)); + AddHandler(0, &TCoEquiJoin::Match, HNDL(EquiJoin)); + AddHandler(0, &TCoCountBase::Match, HNDL(TakeOrSkip)); + if (State_->Configuration->_EnableYtDqProcessWriteConstraints.Get().GetOrElse(DEFAULT_ENABLE_DQ_WRITE_CONSTRAINTS)) { + AddHandler(0, &TYtMaterialize::Match, HNDL(DqMaterialize)); + AddHandler(0, &TYtMaterialize::Match, HNDL(Materialize)); + AddHandler(0, &TYtWriteTable::Match, HNDL(FillToMaterialize)); + } else { + AddHandler(0, &TYtWriteTable::Match, HNDL(Fill)); + } + AddHandler(0, &TResPull::Match, HNDL(ResPull)); + if (State_->Configuration->UseNewPredicateExtraction.Get().GetOrElse(DEFAULT_USE_NEW_PREDICATE_EXTRACTION)) { + AddHandler(0, Names({TYtMap::CallableName(), TYtMapReduce::CallableName()}), HNDL(ExtractKeyRange)); + AddHandler(0, &TCoFlatMapBase::Match, HNDL(ExtractKeyRangeDqReadWrap)); + } else { + AddHandler(0, Names({TYtMap::CallableName(), TYtMapReduce::CallableName()}), HNDL(ExtractKeyRangeLegacy)); + } + AddHandler(0, &TCoExtendBase::Match, HNDL(Extend)); + AddHandler(0, &TCoAssumeSorted::Match, HNDL(AssumeConstraints)); + AddHandler(0, &TCoAssumeConstraints::Match, HNDL(AssumeConstraints)); + AddHandler(0, &TCoAssumeUnique::Match, HNDL(AssumeConstraints)); + AddHandler(0, &TCoAssumeDistinct::Match, HNDL(AssumeConstraints)); + AddHandler(0, &TYtDqWrite::Match, HNDL(YtDqWrite)); + AddHandler(0, &TYtDqProcessWrite::Match, HNDL(YtDqProcessWrite)); + AddHandler(0, &TYtEquiJoin::Match, HNDL(EarlyMergeJoin)); + AddHandler(0, &TYtOutputOpBase::Match, HNDL(TableContentWithSettings)); + AddHandler(0, &TYtOutputOpBase::Match, HNDL(NonOptimalTableContent)); + + if (!State_->Configuration->DisableFuseOperations.Get().GetOrElse(DEFAULT_DISABLE_FUSE_OPERATIONS)) { + AddHandler(1, &TYtMap::Match, HNDL(FuseInnerMap)); + AddHandler(1, &TYtMap::Match, HNDL(FuseOuterMap)); + } + AddHandler(1, Names({TYtMap::CallableName(), TYtMapReduce::CallableName()}), HNDL(MapFieldsSubset)); + AddHandler(1, Names({TYtMapReduce::CallableName(), TYtReduce::CallableName()}), HNDL(ReduceFieldsSubset)); + AddHandler(1, Names({TYtMap::CallableName(), TYtMapReduce::CallableName()}), HNDL(MultiMapFieldsSubset)); + AddHandler(1, Names({TYtMapReduce::CallableName(), TYtReduce::CallableName()}), HNDL(MultiReduceFieldsSubset)); + AddHandler(1, Names({TYtMap::CallableName(), TYtMapReduce::CallableName()}), HNDL(WeakFields)); + AddHandler(1, &TYtTransientOpBase::Match, HNDL(BypassMerge)); + AddHandler(1, &TYtPublish::Match, HNDL(BypassMergeBeforePublish)); + AddHandler(1, &TCoRight::Match, HNDL(ReadWithSettings)); + AddHandler(1, &TYtTransientOpBase::Match, HNDL(PushDownKeyExtract)); + AddHandler(1, &TYtTransientOpBase::Match, HNDL(TransientOpWithSettings)); + AddHandler(1, &TYtSort::Match, HNDL(TopSort)); + AddHandler(1, &TYtWithUserJobsOpBase::Match, HNDL(EmbedLimit)); + AddHandler(1, &TYtMerge::Match, HNDL(PushMergeLimitToInput)); + if (!State_->Configuration->DisableFuseOperations.Get().GetOrElse(DEFAULT_DISABLE_FUSE_OPERATIONS)) { + AddHandler(1, &TYtReduce::Match, HNDL(FuseReduce)); + } + + AddHandler(2, &TYtEquiJoin::Match, HNDL(RuntimeEquiJoin)); + AddHandler(2, &TStatWriteTable::Match, HNDL(ReplaceStatWriteTable)); + AddHandler(2, &TYtMap::Match, HNDL(MapToMerge)); + AddHandler(2, &TYtMap::Match, HNDL(PushDownYtMapOverSortedMerge)); + AddHandler(2, &TYtMerge::Match, HNDL(ForceTransform)); + AddHandler(2, &TYtMerge::Match, HNDL(MergeToCopy)); +#undef HNDL +} + +} // namespace NYql diff --git a/yt/yql/providers/yt/provider/phy_opt/yql_yt_phy_opt.h b/yt/yql/providers/yt/provider/phy_opt/yql_yt_phy_opt.h new file mode 100644 index 0000000000..95f9435922 --- /dev/null +++ b/yt/yql/providers/yt/provider/phy_opt/yql_yt_phy_opt.h @@ -0,0 +1,158 @@ +#pragma once + +#include <yt/yql/providers/yt/provider/yql_yt_provider_impl.h> +#include <yt/yql/providers/yt/lib/key_filter/yql_key_filter.h> +#include <yt/yql/providers/yt/expr_nodes/yql_yt_expr_nodes.h> +#include <yql/essentials/providers/common/transform/yql_optimize.h> +#include <yql/essentials/core/extract_predicate/extract_predicate.h> +#include <yql/essentials/core/expr_nodes/yql_expr_nodes.h> + +#include <util/generic/vector.h> +#include <util/generic/set.h> +#include <util/generic/string.h> + +namespace NYql { + +class TYtPhysicalOptProposalTransformer : public TOptimizeTransformerBase { +public: + TYtPhysicalOptProposalTransformer(TYtState::TPtr state); + +private: + static bool CanBePulledIntoParentEquiJoin(const NNodes::TCoFlatMapBase& flatMap, const TGetParents& getParents); + + NNodes::TCoLambda MakeJobLambdaNoArg(NNodes::TExprBase content, TExprContext& ctx) const; + + NNodes::TMaybeNode<NNodes::TExprBase> Mux(NNodes::TExprBase node, TExprContext& ctx) const; + + NNodes::TMaybeNode<NNodes::TExprBase> YtSortOverAlreadySorted(NNodes::TExprBase node, TExprContext& ctx) const; + + NNodes::TMaybeNode<NNodes::TExprBase> PartitionByKey(NNodes::TExprBase node, TExprContext& ctx) const; + + NNodes::TMaybeNode<NNodes::TExprBase> FlatMap(NNodes::TExprBase node, TExprContext& ctx, const TGetParents& getParents) const; + + NNodes::TMaybeNode<NNodes::TExprBase> CombineByKey(NNodes::TExprBase node, TExprContext& ctx) const; + + NNodes::TMaybeNode<NNodes::TExprBase> DqWrite(NNodes::TExprBase node, TExprContext& ctx, IOptimizationContext& optCtx, const TGetParents& getParents) const; + + NNodes::TMaybeNode<NNodes::TExprBase> Materialize(NNodes::TExprBase node, TExprContext& ctx) const; + + NNodes::TMaybeNode<NNodes::TExprBase> DqMaterialize(NNodes::TExprBase node, TExprContext& ctx, IOptimizationContext& optCtx, const TGetParents& getParents) const; + + NNodes::TMaybeNode<NNodes::TExprBase> YtDqProcessWrite(NNodes::TExprBase node, TExprContext& ctx) const; + + NNodes::TMaybeNode<NNodes::TExprBase> Write(NNodes::TExprBase node, TExprContext& ctx) const; + + NNodes::TMaybeNode<NNodes::TExprBase> Fill(NNodes::TExprBase node, TExprContext& ctx) const; + + NNodes::TMaybeNode<NNodes::TExprBase> FillToMaterialize(NNodes::TExprBase node, TExprContext& ctx) const; + + NNodes::TMaybeNode<NNodes::TExprBase> TakeOrSkip(NNodes::TExprBase node, TExprContext& ctx, const TGetParents& getParents) const; + + NNodes::TMaybeNode<NNodes::TExprBase> Extend(NNodes::TExprBase node, TExprContext& ctx) const; + + NNodes::TMaybeNode<NNodes::TExprBase> Length(NNodes::TExprBase node, TExprContext& ctx) const; + + NNodes::TMaybeNode<NNodes::TExprBase> ResPull(NNodes::TExprBase node, TExprContext& ctx) const; + + struct TRangeBuildResult { + TVector<TString> Keys; + TSet<size_t> TableIndexes; + IPredicateRangeExtractor::TBuildResult BuildResult; + }; + + TMaybe<TVector<TRangeBuildResult>> ExtractKeyRangeFromLambda(NNodes::TCoLambda lambda, NNodes::TYtSection section, TExprContext& ctx) const; + + TExprNode::TPtr UpdateSectionWithKeyRanges(TPositionHandle pos, NNodes::TYtSection section, const TVector<TRangeBuildResult>& results, TExprContext& ctx) const; + + NNodes::TMaybeNode<NNodes::TExprBase> ExtractKeyRangeDqReadWrap(NNodes::TExprBase node, TExprContext& ctx) const; + + NNodes::TMaybeNode<NNodes::TExprBase> ExtractKeyRange(NNodes::TExprBase node, TExprContext& ctx) const; + + // All keyFilter settings are combined by OR. + // keyFilter value := '(<memberItem>+) <optional tableIndex> + // <memberItem> := '(<memberName> '(<cmpItem>+)) + // <cmpItem> := '(<cmpOp> <value>) + NNodes::TMaybeNode<NNodes::TExprBase> ExtractKeyRangeLegacy(NNodes::TExprBase node, TExprContext& ctx) const; + + NNodes::TMaybeNode<NNodes::TExprBase> FuseReduce(NNodes::TExprBase node, TExprContext& ctx, const TGetParents& getParents) const; + + NNodes::TMaybeNode<NNodes::TExprBase> FuseInnerMap(NNodes::TExprBase node, TExprContext& ctx, const TGetParents& getParents) const; + + NNodes::TMaybeNode<NNodes::TExprBase> FuseOuterMap(NNodes::TExprBase node, TExprContext& ctx, const TGetParents& getParents) const; + + NNodes::TMaybeNode<NNodes::TExprBase> AssumeConstraints(NNodes::TExprBase node, TExprContext& ctx, const TGetParents& getParents) const; + + NNodes::TMaybeNode<NNodes::TExprBase> LambdaFieldsSubset(NNodes::TYtWithUserJobsOpBase op, size_t lambdaIdx, TExprContext& ctx, const TGetParents& getParents) const; + + NNodes::TMaybeNode<NNodes::TExprBase> MapFieldsSubset(NNodes::TExprBase node, TExprContext& ctx, const TGetParents& getParents) const; + + NNodes::TMaybeNode<NNodes::TExprBase> ReduceFieldsSubset(NNodes::TExprBase node, TExprContext& ctx, const TGetParents& getParents) const; + + NNodes::TMaybeNode<NNodes::TExprBase> LambdaVisitFieldsSubset(NNodes::TYtWithUserJobsOpBase op, size_t lambdaIdx, TExprContext& ctx, const TGetParents& getParents) const; + + NNodes::TMaybeNode<NNodes::TExprBase> MultiMapFieldsSubset(NNodes::TExprBase node, TExprContext& ctx, const TGetParents& getParents) const; + + NNodes::TMaybeNode<NNodes::TExprBase> MultiReduceFieldsSubset(NNodes::TExprBase node, TExprContext& ctx, const TGetParents& getParents) const; + + NNodes::TMaybeNode<NNodes::TExprBase> WeakFields(NNodes::TExprBase node, TExprContext& ctx, const TGetParents& getParents) const; + + NNodes::TMaybeNode<NNodes::TExprBase> EquiJoin(NNodes::TExprBase node, TExprContext& ctx, const TGetParents& getParents) const; + + NNodes::TMaybeNode<NNodes::TExprBase> EarlyMergeJoin(NNodes::TExprBase node, TExprContext& ctx) const; + + NNodes::TMaybeNode<NNodes::TExprBase> RuntimeEquiJoin(NNodes::TExprBase node, TExprContext& ctx) const; + + NNodes::TMaybeNode<NNodes::TExprBase> TableContentWithSettings(NNodes::TExprBase node, TExprContext& ctx) const; + + NNodes::TMaybeNode<NNodes::TExprBase> NonOptimalTableContent(NNodes::TExprBase node, TExprContext& ctx) const; + + NNodes::TMaybeNode<NNodes::TExprBase> ReadWithSettings(NNodes::TExprBase node, TExprContext& ctx) const; + + NNodes::TMaybeNode<NNodes::TExprBase> TransientOpWithSettings(NNodes::TExprBase node, TExprContext& ctx) const; + + NNodes::TMaybeNode<NNodes::TExprBase> ReplaceStatWriteTable(NNodes::TExprBase node, TExprContext& ctx) const; + + template<bool IsTop> + NNodes::TMaybeNode<NNodes::TExprBase> Sort(NNodes::TExprBase node, TExprContext& ctx) const; + + NNodes::TMaybeNode<NNodes::TExprBase> TopSort(NNodes::TExprBase node, TExprContext& ctx) const; + + NNodes::TMaybeNode<NNodes::TExprBase> EmbedLimit(NNodes::TExprBase node, TExprContext& ctx) const; + + NNodes::TMaybeNode<NNodes::TExprBase> PushMergeLimitToInput(NNodes::TExprBase node, TExprContext& ctx) const; + + NNodes::TMaybeNode<NNodes::TExprBase> PushDownKeyExtract(NNodes::TExprBase node, TExprContext& ctx) const; + + NNodes::TMaybeNode<NNodes::TExprBase> BypassMerge(NNodes::TExprBase node, TExprContext& ctx) const; + + NNodes::TMaybeNode<NNodes::TExprBase> BypassMergeBeforePublish(NNodes::TExprBase node, TExprContext& ctx) const; + + NNodes::TMaybeNode<NNodes::TExprBase> MapToMerge(NNodes::TExprBase node, TExprContext& ctx) const; + + NNodes::TMaybeNode<NNodes::TExprBase> YtDqWrite(NNodes::TExprBase node, TExprContext& ctx) const; + + NNodes::TMaybeNode<NNodes::TExprBase> PushDownYtMapOverSortedMerge(NNodes::TExprBase node, TExprContext& ctx, const TGetParents& getParents) const; + + NNodes::TMaybeNode<NNodes::TExprBase> MergeToCopy(NNodes::TExprBase node, TExprContext& ctx) const; + + NNodes::TMaybeNode<NNodes::TExprBase> ForceTransform(NNodes::TExprBase node, TExprContext& ctx) const; + + template <typename TLMapType> + NNodes::TMaybeNode<NNodes::TExprBase> LMap(NNodes::TExprBase node, TExprContext& ctx) const; + + template<bool WithList> + NNodes::TCoLambda MakeJobLambda(NNodes::TCoLambda lambda, bool useFlow, TExprContext& ctx) const; + + template <class TExpr> + NNodes::TMaybeNode<TExpr> CleanupWorld(TExpr node, TExprContext& ctx) const { + return NNodes::TMaybeNode<TExpr>(YtCleanupWorld(node.Ptr(), ctx, State_)); + } + + TMaybe<bool> CanFuseLambdas(const NNodes::TCoLambda& innerLambda, const NNodes::TCoLambda& outerLambda, TExprContext& ctx) const; + + +private: + const TYtState::TPtr State_; +}; + +} // namespace NYql diff --git a/yt/yql/providers/yt/provider/phy_opt/yql_yt_phy_opt_content.cpp b/yt/yql/providers/yt/provider/phy_opt/yql_yt_phy_opt_content.cpp new file mode 100644 index 0000000000..0afdbe789a --- /dev/null +++ b/yt/yql/providers/yt/provider/phy_opt/yql_yt_phy_opt_content.cpp @@ -0,0 +1,285 @@ +#include "yql_yt_phy_opt.h" + +#include <yt/yql/providers/yt/provider/yql_yt_helpers.h> +#include <yt/yql/providers/yt/provider/yql_yt_optimize.h> + +namespace NYql { + +using namespace NNodes; + +TMaybeNode<TExprBase> TYtPhysicalOptProposalTransformer::TableContentWithSettings(TExprBase node, TExprContext& ctx) const { + auto op = node.Cast<TYtOutputOpBase>(); + + TExprNode::TPtr res = op.Ptr(); + + TNodeSet nodesToOptimize; + TProcessedNodesSet processedNodes; + processedNodes.insert(res->Head().UniqueId()); + VisitExpr(res, [&nodesToOptimize, &processedNodes](const TExprNode::TPtr& input) { + if (processedNodes.contains(input->UniqueId())) { + return false; + } + + if (auto read = TMaybeNode<TYtLength>(input).Input().Maybe<TYtReadTable>()) { + nodesToOptimize.insert(read.Cast().Raw()); + return false; + } + + if (auto read = TMaybeNode<TYtTableContent>(input).Input().Maybe<TYtReadTable>()) { + nodesToOptimize.insert(read.Cast().Raw()); + return false; + } + if (TYtOutput::Match(input.Get())) { + processedNodes.insert(input->UniqueId()); + return false; + } + return true; + }); + + if (nodesToOptimize.empty()) { + return node; + } + + TSyncMap syncList; + TOptimizeExprSettings settings(State_->Types); + settings.ProcessedNodes = &processedNodes; // Prevent optimizer to go deeper than current operation + auto status = OptimizeExpr(res, res, [&syncList, &nodesToOptimize, state = State_](const TExprNode::TPtr& input, TExprContext& ctx) -> TExprNode::TPtr { + if (nodesToOptimize.find(input.Get()) != nodesToOptimize.end()) { + return OptimizeReadWithSettings(input, false, true, syncList, state, ctx); + } + return input; + }, ctx, settings); + + if (status.Level == IGraphTransformer::TStatus::Error) { + return {}; + } + + if (status.Level == IGraphTransformer::TStatus::Ok) { + return node; + } + + if (!syncList.empty()) { + using TPair = std::pair<TExprNode::TPtr, ui64>; + TVector<TPair> sortedList(syncList.cbegin(), syncList.cend()); + TExprNode::TListType syncChildren; + syncChildren.push_back(res->ChildPtr(TYtOutputOpBase::idx_World)); + ::Sort(sortedList, [](const TPair& x, const TPair& y) { return x.second < y.second; }); + for (auto& x: sortedList) { + auto world = ctx.NewCallable(node.Pos(), TCoLeft::CallableName(), { x.first }); + syncChildren.push_back(world); + } + + res = ctx.ChangeChild(*res, TYtOutputOpBase::idx_World, + ctx.NewCallable(node.Pos(), TCoSync::CallableName(), std::move(syncChildren))); + } + + return TExprBase(res); +} + +TMaybeNode<TExprBase> TYtPhysicalOptProposalTransformer::NonOptimalTableContent(TExprBase node, TExprContext& ctx) const { + auto op = node.Cast<TYtOutputOpBase>(); + + TExprNode::TPtr res = op.Ptr(); + + TNodeSet nodesToOptimize; + TProcessedNodesSet processedNodes; + processedNodes.insert(res->Head().UniqueId()); + VisitExpr(res, [&nodesToOptimize, &processedNodes](const TExprNode::TPtr& input) { + if (processedNodes.contains(input->UniqueId())) { + return false; + } + + if (TYtTableContent::Match(input.Get())) { + nodesToOptimize.insert(input.Get()); + return false; + } + if (TYtOutput::Match(input.Get())) { + processedNodes.insert(input->UniqueId()); + return false; + } + return true; + }); + + if (nodesToOptimize.empty()) { + return node; + } + + TSyncMap syncList; + const auto maxTables = State_->Configuration->TableContentMaxInputTables.Get().GetOrElse(1000); + const auto minChunkSize = State_->Configuration->TableContentMinAvgChunkSize.Get().GetOrElse(1_GB); + const auto maxChunks = State_->Configuration->TableContentMaxChunksForNativeDelivery.Get().GetOrElse(1000ul); + auto state = State_; + auto world = res->ChildPtr(TYtOutputOpBase::idx_World); + TOptimizeExprSettings settings(State_->Types); + settings.ProcessedNodes = &processedNodes; // Prevent optimizer to go deeper than current operation + auto status = OptimizeExpr(res, res, [&syncList, &nodesToOptimize, maxTables, minChunkSize, maxChunks, state, world](const TExprNode::TPtr& input, TExprContext& ctx) -> TExprNode::TPtr { + if (nodesToOptimize.find(input.Get()) != nodesToOptimize.end()) { + if (auto read = TYtTableContent(input).Input().Maybe<TYtReadTable>()) { + bool materialize = false; + const bool singleSection = 1 == read.Cast().Input().Size(); + TVector<TYtSection> newSections; + for (auto section: read.Cast().Input()) { + if (NYql::HasAnySetting(section.Settings().Ref(), EYtSettingType::Sample | EYtSettingType::SysColumns)) { + materialize = true; + } + else if (section.Paths().Size() > maxTables) { + materialize = true; + } + else { + TMaybeNode<TYtMerge> oldOp; + if (section.Paths().Size() == 1) { + oldOp = section.Paths().Item(0).Table().Maybe<TYtOutput>().Operation().Maybe<TYtMerge>(); + } + if (!oldOp.IsValid() || !NYql::HasSetting(oldOp.Cast().Settings().Ref(), EYtSettingType::CombineChunks)) { + for (auto path: section.Paths()) { + TYtTableBaseInfo::TPtr tableInfo = TYtTableBaseInfo::Parse(path.Table()); + if (auto tableStat = tableInfo->Stat) { + if (tableStat->ChunkCount > maxChunks || (tableStat->ChunkCount > 1 && tableStat->DataSize / tableStat->ChunkCount < minChunkSize)) { + materialize = true; + break; + } + } + if (!tableInfo->IsTemp && tableInfo->Meta) { + auto p = tableInfo->Meta->Attrs.FindPtr("erasure_codec"); + if (p && *p != "none") { + materialize = true; + break; + } + else if (tableInfo->Meta->IsDynamic) { + materialize = true; + break; + } + } + } + } + } + if (materialize) { + auto path = CopyOrTrivialMap(section.Pos(), + TExprBase(world), + TYtDSink(ctx.RenameNode(read.DataSource().Ref(), "DataSink")), + *section.Ref().GetTypeAnn()->Cast<TListExprType>()->GetItemType(), + Build<TYtSection>(ctx, section.Pos()) + .Paths(section.Paths()) + .Settings(NYql::RemoveSettings(section.Settings().Ref(), EYtSettingType::Unordered | EYtSettingType::NonUnique, ctx)) + .Done(), + {}, ctx, state, + TCopyOrTrivialMapOpts() + .SetTryKeepSortness(!NYql::HasSetting(section.Settings().Ref(), EYtSettingType::Unordered)) + .SetSectionUniq(section.Ref().GetConstraint<TDistinctConstraintNode>()) + .SetConstraints(section.Ref().GetConstraintSet()) + .SetCombineChunks(true) + ); + + syncList[path.Table().Cast<TYtOutput>().Operation().Ptr()] = syncList.size(); + + if (singleSection) { + return ctx.ChangeChild(*input, TYtTableContent::idx_Input, path.Table().Ptr()); + } else { + newSections.push_back(Build<TYtSection>(ctx, section.Pos()) + .Paths() + .Add(path) + .Build() + .Settings().Build() + .Done()); + } + + } else { + newSections.push_back(section); + } + + } + if (materialize) { + auto newRead = Build<TYtReadTable>(ctx, read.Cast().Pos()) + .InitFrom(read.Cast()) + .Input() + .Add(newSections) + .Build() + .Done(); + + return ctx.ChangeChild(*input, TYtTableContent::idx_Input, newRead.Ptr()); + } + } + else if (auto out = TYtTableContent(input).Input().Maybe<TYtOutput>()) { + auto oldOp = GetOutputOp(out.Cast()); + if (!oldOp.Maybe<TYtMerge>() || !NYql::HasSetting(oldOp.Cast<TYtMerge>().Settings().Ref(), EYtSettingType::CombineChunks)) { + auto outTable = GetOutTable(out.Cast()); + TYtOutTableInfo tableInfo(outTable); + if (auto tableStat = tableInfo.Stat) { + if (tableStat->ChunkCount > maxChunks || (tableStat->ChunkCount > 1 && tableStat->DataSize / tableStat->ChunkCount < minChunkSize)) { + auto newOp = Build<TYtMerge>(ctx, input->Pos()) + .World(world) + .DataSink(oldOp.DataSink()) + .Output() + .Add() + .InitFrom(outTable.Cast<TYtOutTable>()) + .Name().Value("").Build() + .Stat<TCoVoid>().Build() + .Build() + .Build() + .Input() + .Add() + .Paths() + .Add() + .Table(out.Cast()) + .Columns<TCoVoid>().Build() + .Ranges<TCoVoid>().Build() + .Stat<TCoVoid>().Build() + .Build() + .Build() + .Settings<TCoNameValueTupleList>() + .Build() + .Build() + .Build() + .Settings() + .Add() + .Name().Value(ToString(EYtSettingType::CombineChunks)).Build() + .Build() + .Add() + .Name().Value(ToString(EYtSettingType::ForceTransform)).Build() + .Build() + .Build() + .Done(); + + syncList[newOp.Ptr()] = syncList.size(); + + auto newOutput = Build<TYtOutput>(ctx, input->Pos()) + .Operation(newOp) + .OutIndex().Value(0U).Build() + .Done().Ptr(); + + return ctx.ChangeChild(*input, TYtTableContent::idx_Input, std::move(newOutput)); + } + } + } + } + } + return input; + }, ctx, settings); + + if (status.Level == IGraphTransformer::TStatus::Error) { + return {}; + } + + if (status.Level == IGraphTransformer::TStatus::Ok) { + return node; + } + + if (!syncList.empty()) { + using TPair = std::pair<TExprNode::TPtr, ui64>; + TVector<TPair> sortedList(syncList.cbegin(), syncList.cend()); + TExprNode::TListType syncChildren; + syncChildren.push_back(res->ChildPtr(TYtOutputOpBase::idx_World)); + ::Sort(sortedList, [](const TPair& x, const TPair& y) { return x.second < y.second; }); + for (auto& x: sortedList) { + auto world = ctx.NewCallable(node.Pos(), TCoLeft::CallableName(), { x.first }); + syncChildren.push_back(world); + } + + res = ctx.ChangeChild(*res, TYtOutputOpBase::idx_World, + ctx.NewCallable(node.Pos(), TCoSync::CallableName(), std::move(syncChildren))); + } + + return TExprBase(res); +} + +} // namespace NYql diff --git a/yt/yql/providers/yt/provider/phy_opt/yql_yt_phy_opt_field_subset.cpp b/yt/yql/providers/yt/provider/phy_opt/yql_yt_phy_opt_field_subset.cpp new file mode 100644 index 0000000000..e70d54fffa --- /dev/null +++ b/yt/yql/providers/yt/provider/phy_opt/yql_yt_phy_opt_field_subset.cpp @@ -0,0 +1,183 @@ +#include "yql_yt_phy_opt.h" + +#include <yt/yql/providers/yt/provider/yql_yt_helpers.h> + +#include <yql/essentials/core/yql_opt_utils.h> + +namespace NYql { + +using namespace NNodes; + +TMaybeNode<TExprBase> TYtPhysicalOptProposalTransformer::LambdaFieldsSubset(TYtWithUserJobsOpBase op, size_t lambdaIdx, TExprContext& ctx, const TGetParents& getParents) const { + auto lambda = TCoLambda(op.Ref().ChildPtr(lambdaIdx)); + + bool hasUpdates = false; + TYtSection section = op.Input().Item(0); + + const TParentsMap* parentsMap = getParents(); + auto parents = parentsMap->find(lambda.Args().Arg(0).Raw()); + if (parents == parentsMap->cend()) { + // Argument is not used in lambda body + return op; + } + if (parents->second.size() == 1 && TCoExtractMembers::Match(*parents->second.begin())) { + auto members = TCoExtractMembers(*parents->second.begin()).Members(); + TSet<TStringBuf> memberSet; + std::for_each(members.begin(), members.end(), [&memberSet](const auto& m) { memberSet.insert(m.Value()); }); + auto reduceBy = NYql::GetSettingAsColumnList(op.Settings().Ref(), EYtSettingType::ReduceBy); + memberSet.insert(reduceBy.cbegin(), reduceBy.cend()); + auto sortBy = NYql::GetSettingAsColumnList(op.Settings().Ref(), EYtSettingType::SortBy); + memberSet.insert(sortBy.cbegin(), sortBy.cend()); + + auto itemType = GetSeqItemType(lambda.Args().Arg(0).Ref().GetTypeAnn())->Cast<TStructExprType>(); + if (memberSet.size() < itemType->GetSize()) { + section = UpdateInputFields(section, std::move(memberSet), ctx, NYql::HasSetting(op.Settings().Ref(), EYtSettingType::WeakFields)); + hasUpdates = true; + } + } + + if (!hasUpdates) { + return op; + } + + auto res = ctx.ChangeChild(op.Ref(), TYtTransientOpBase::idx_Input, + Build<TYtSectionList>(ctx, op.Input().Pos()) + .Add(section) + .Done().Ptr()); + + res = ctx.ChangeChild(*res, lambdaIdx, ctx.DeepCopyLambda(lambda.Ref())); + + return TExprBase(res); + +} + +TMaybeNode<TExprBase> TYtPhysicalOptProposalTransformer::LambdaVisitFieldsSubset(TYtWithUserJobsOpBase op, size_t lambdaIdx, TExprContext& ctx, const TGetParents& getParents) const { + auto opLambda = TCoLambda(op.Ref().ChildPtr(lambdaIdx)); + + const TParentsMap* parentsMap = getParents(); + auto maybeLambda = GetFlatMapOverInputStream(opLambda, *parentsMap).Lambda(); + if (!maybeLambda) { + return op; + } + + TCoLambda lambda = maybeLambda.Cast(); + auto arg = lambda.Args().Arg(0); + + // Check arg is used only in Visit + auto it = parentsMap->find(arg.Raw()); + if (it == parentsMap->cend() || it->second.size() != 1 || !TCoVisit::Match(*it->second.begin())) { + return op; + } + + const TExprNode* visit = *it->second.begin(); + TVector<std::pair<size_t, TSet<TStringBuf>>> sectionFields; + for (ui32 index = 1; index < visit->ChildrenSize(); ++index) { + if (visit->Child(index)->IsAtom()) { + size_t inputNum = FromString<size_t>(visit->Child(index)->Content()); + YQL_ENSURE(inputNum < op.Input().Size()); + + ++index; + auto visitLambda = visit->ChildPtr(index); + + TSet<TStringBuf> memberSet; + if (HaveFieldsSubset(visitLambda->TailPtr(), visitLambda->Head().Head(), memberSet, *parentsMap)) { + auto itemType = visitLambda->Head().Head().GetTypeAnn()->Cast<TStructExprType>(); + auto reduceBy = NYql::GetSettingAsColumnList(op.Settings().Ref(), EYtSettingType::ReduceBy); + for (auto& col: reduceBy) { + if (auto type = itemType->FindItemType(col)) { + memberSet.insert(type->Cast<TItemExprType>()->GetName()); + } + } + auto sortBy = NYql::GetSettingAsColumnList(op.Settings().Ref(), EYtSettingType::SortBy); + for (auto& col: sortBy) { + if (auto type = itemType->FindItemType(col)) { + memberSet.insert(type->Cast<TItemExprType>()->GetName()); + } + } + + if (memberSet.size() < itemType->GetSize()) { + sectionFields.emplace_back(inputNum, std::move(memberSet)); + } + } + } + } + + if (sectionFields.empty()) { + return op; + } + + auto res = ctx.ChangeChild(op.Ref(), lambdaIdx, ctx.DeepCopyLambda(opLambda.Ref())); + + TVector<TYtSection> updatedSections(op.Input().begin(), op.Input().end()); + const bool hasWeak = NYql::HasSetting(op.Settings().Ref(), EYtSettingType::WeakFields); + for (auto& pair: sectionFields) { + auto& section = updatedSections[pair.first]; + auto& memberSet = pair.second; + section = UpdateInputFields(section, std::move(memberSet), ctx, hasWeak); + } + + res = ctx.ChangeChild(*res, TYtTransientOpBase::idx_Input, + Build<TYtSectionList>(ctx, op.Input().Pos()) + .Add(updatedSections) + .Done().Ptr()); + + return TExprBase(res); +} + +TMaybeNode<TExprBase> TYtPhysicalOptProposalTransformer::MapFieldsSubset(TExprBase node, TExprContext& ctx, const TGetParents& getParents) const { + auto op = node.Cast<TYtWithUserJobsOpBase>(); + if (op.Input().Size() != 1) { + return node; + } + if (auto map = op.Maybe<TYtMap>()) { + return LambdaFieldsSubset(op, TYtMap::idx_Mapper, ctx, getParents); + } else if (op.Maybe<TYtMapReduce>().Mapper().Maybe<TCoLambda>()) { + return LambdaFieldsSubset(op, TYtMapReduce::idx_Mapper, ctx, getParents); + } + + return node; +} + +TMaybeNode<TExprBase> TYtPhysicalOptProposalTransformer::ReduceFieldsSubset(TExprBase node, TExprContext& ctx, const TGetParents& getParents) const { + auto op = node.Cast<TYtWithUserJobsOpBase>(); + if (op.Input().Size() != 1) { + return node; + } + if (auto reduce = op.Maybe<TYtReduce>()) { + return LambdaFieldsSubset(op, TYtReduce::idx_Reducer, ctx, getParents); + } else if (!op.Maybe<TYtMapReduce>().Mapper().Maybe<TCoLambda>()) { + return LambdaFieldsSubset(op, TYtMapReduce::idx_Reducer, ctx, getParents); + } + + return node; +} + +TMaybeNode<TExprBase> TYtPhysicalOptProposalTransformer::MultiMapFieldsSubset(TExprBase node, TExprContext& ctx, const TGetParents& getParents) const { + auto op = node.Cast<TYtWithUserJobsOpBase>(); + if (op.Input().Size() < 2) { + return node; + } + if (auto map = op.Maybe<TYtMap>()) { + return LambdaVisitFieldsSubset(op, TYtMap::idx_Mapper, ctx, getParents); + } else if (op.Maybe<TYtMapReduce>().Mapper().Maybe<TCoLambda>()) { + return LambdaVisitFieldsSubset(op, TYtMapReduce::idx_Mapper, ctx, getParents); + } + + return node; +} + +TMaybeNode<TExprBase> TYtPhysicalOptProposalTransformer::MultiReduceFieldsSubset(TExprBase node, TExprContext& ctx, const TGetParents& getParents) const { + auto op = node.Cast<TYtWithUserJobsOpBase>(); + if (op.Input().Size() < 2) { + return node; + } + if (auto reduce = op.Maybe<TYtReduce>()) { + return LambdaVisitFieldsSubset(op, TYtReduce::idx_Reducer, ctx, getParents); + } else if (!op.Maybe<TYtMapReduce>().Mapper().Maybe<TCoLambda>()) { + return LambdaVisitFieldsSubset(op, TYtMapReduce::idx_Reducer, ctx, getParents); + } + + return node; +} + +} // namespace NYql diff --git a/yt/yql/providers/yt/provider/phy_opt/yql_yt_phy_opt_fuse.cpp b/yt/yql/providers/yt/provider/phy_opt/yql_yt_phy_opt_fuse.cpp new file mode 100644 index 0000000000..96fe6dab33 --- /dev/null +++ b/yt/yql/providers/yt/provider/phy_opt/yql_yt_phy_opt_fuse.cpp @@ -0,0 +1,548 @@ +#include "yql_yt_phy_opt.h" +#include "yql_yt_phy_opt_helper.h" + +#include <yt/yql/providers/yt/provider/yql_yt_optimize.h> +#include <yt/yql/providers/yt/provider/yql_yt_helpers.h> +#include <yt/yql/providers/yt/lib/expr_traits/yql_expr_traits.h> +#include <yql/essentials/providers/common/provider/yql_provider.h> + +#include <yql/essentials/core/yql_opt_utils.h> +#include <yql/essentials/core/yql_type_helpers.h> + +#include <yql/essentials/utils/log/log.h> + +namespace NYql { + +using namespace NNodes; +using namespace NPrivate; + +TMaybe<bool> TYtPhysicalOptProposalTransformer::CanFuseLambdas(const TCoLambda& innerLambda, const TCoLambda& outerLambda, TExprContext& ctx) const { + auto maxJobMemoryLimit = State_->Configuration->MaxExtraJobMemoryToFuseOperations.Get(); + auto maxOperationFiles = State_->Configuration->MaxOperationFiles.Get().GetOrElse(DEFAULT_MAX_OPERATION_FILES); + TMap<TStringBuf, ui64> memUsage; + + TExprNode::TPtr updatedBody = innerLambda.Body().Ptr(); + if (maxJobMemoryLimit) { + auto status = UpdateTableContentMemoryUsage(innerLambda.Body().Ptr(), updatedBody, State_, ctx, false); + if (status.Level != TStatus::Ok) { + return {}; + } + } + size_t innerFiles = 1; // jobstate. Take into account only once + ScanResourceUsage(*updatedBody, *State_->Configuration, State_->Types, maxJobMemoryLimit ? &memUsage : nullptr, nullptr, &innerFiles); + + auto prevMemory = Accumulate(memUsage.begin(), memUsage.end(), 0ul, + [](ui64 sum, const std::pair<const TStringBuf, ui64>& val) { return sum + val.second; }); + + updatedBody = outerLambda.Body().Ptr(); + if (maxJobMemoryLimit) { + auto status = UpdateTableContentMemoryUsage(outerLambda.Body().Ptr(), updatedBody, State_, ctx, false); + if (status.Level != TStatus::Ok) { + return {}; + } + } + size_t outerFiles = 0; + ScanResourceUsage(*updatedBody, *State_->Configuration, State_->Types, maxJobMemoryLimit ? &memUsage : nullptr, nullptr, &outerFiles); + + auto currMemory = Accumulate(memUsage.begin(), memUsage.end(), 0ul, + [](ui64 sum, const std::pair<const TStringBuf, ui64>& val) { return sum + val.second; }); + + if (maxJobMemoryLimit && currMemory != prevMemory && currMemory > *maxJobMemoryLimit) { + YQL_CLOG(DEBUG, ProviderYt) << "Memory usage: innerLambda=" << prevMemory + << ", joinedLambda=" << currMemory << ", MaxJobMemoryLimit=" << *maxJobMemoryLimit; + return false; + } + if (innerFiles + outerFiles > maxOperationFiles) { + YQL_CLOG(DEBUG, ProviderYt) << "Files usage: innerLambda=" << innerFiles + << ", outerLambda=" << outerFiles << ", MaxOperationFiles=" << maxOperationFiles; + return false; + } + + if (auto maxReplcationFactor = State_->Configuration->MaxReplicationFactorToFuseOperations.Get()) { + double replicationFactor1 = NCommon::GetDataReplicationFactor(innerLambda.Ref(), ctx); + double replicationFactor2 = NCommon::GetDataReplicationFactor(outerLambda.Ref(), ctx); + YQL_CLOG(DEBUG, ProviderYt) << "Replication factors: innerLambda=" << replicationFactor1 + << ", outerLambda=" << replicationFactor2 << ", MaxReplicationFactorToFuseOperations=" << *maxReplcationFactor; + + if (replicationFactor1 > 1.0 && replicationFactor2 > 1.0 && replicationFactor1 * replicationFactor2 > *maxReplcationFactor) { + return false; + } + } + return true; +} + +TMaybeNode<TExprBase> TYtPhysicalOptProposalTransformer::FuseReduce(TExprBase node, TExprContext& ctx, const TGetParents& getParents) const { + auto outerReduce = node.Cast<TYtReduce>(); + + if (outerReduce.Input().Size() != 1 || outerReduce.Input().Item(0).Paths().Size() != 1) { + return node; + } + if (outerReduce.Input().Item(0).Settings().Size() != 0) { + return node; + } + TYtPath path = outerReduce.Input().Item(0).Paths().Item(0); + if (!path.Ranges().Maybe<TCoVoid>()) { + return node; + } + auto maybeInnerReduce = path.Table().Maybe<TYtOutput>().Operation().Maybe<TYtReduce>(); + if (!maybeInnerReduce) { + return node; + } + TYtReduce innerReduce = maybeInnerReduce.Cast(); + + if (innerReduce.Ref().StartsExecution() || innerReduce.Ref().HasResult()) { + return node; + } + if (innerReduce.Output().Size() > 1) { + return node; + } + + if (outerReduce.DataSink().Cluster().Value() != innerReduce.DataSink().Cluster().Value()) { + return node; + } + + const TParentsMap* parentsReduce = getParents(); + if (IsOutputUsedMultipleTimes(innerReduce.Ref(), *parentsReduce)) { + // Inner reduce output is used more than once + return node; + } + // Check world dependencies + auto parentsIt = parentsReduce->find(innerReduce.Raw()); + YQL_ENSURE(parentsIt != parentsReduce->cend()); + for (auto dep: parentsIt->second) { + if (!TYtOutput::Match(dep)) { + return node; + } + } + + if (!NYql::HasSetting(innerReduce.Settings().Ref(), EYtSettingType::KeySwitch) || + !NYql::HasSetting(innerReduce.Settings().Ref(), EYtSettingType::Flow) || + !NYql::HasSetting(innerReduce.Settings().Ref(), EYtSettingType::ReduceBy)) { + return node; + } + if (NYql::HasSetting(outerReduce.Settings().Ref(), EYtSettingType::SortBy)) { + auto innerSortBy = NYql::GetSettingAsColumnList(innerReduce.Settings().Ref(), EYtSettingType::SortBy); + auto outerSortBy = NYql::GetSettingAsColumnList(outerReduce.Settings().Ref(), EYtSettingType::SortBy); + if (outerSortBy.size() > innerSortBy.size()) { + return node; + } + if (!std::equal(outerSortBy.cbegin(), outerSortBy.cend(), innerSortBy.cbegin())) { + return node; + } + } + + if (NYql::HasSettingsExcept(innerReduce.Settings().Ref(), EYtSettingType::ReduceBy | + EYtSettingType::KeySwitch | + EYtSettingType::Flow | + EYtSettingType::FirstAsPrimary | + EYtSettingType::SortBy | + EYtSettingType::KeepSorted | + EYtSettingType::NoDq)) { + return node; + } + + if (!EqualSettingsExcept(innerReduce.Settings().Ref(), outerReduce.Settings().Ref(), + EYtSettingType::ReduceBy | + EYtSettingType::FirstAsPrimary | + EYtSettingType::NoDq | + EYtSettingType::SortBy)) { + return node; + } + + auto innerLambda = innerReduce.Reducer(); + auto outerLambda = outerReduce.Reducer(); + auto fuseRes = CanFuseLambdas(innerLambda, outerLambda, ctx); + if (!fuseRes) { + // Some error + return {}; + } + if (!*fuseRes) { + // Cannot fuse + return node; + } + + auto [placeHolder, lambdaWithPlaceholder] = ReplaceDependsOn(outerLambda.Ptr(), ctx, State_->Types); + if (!placeHolder) { + return {}; + } + + + if (lambdaWithPlaceholder != outerLambda.Ptr()) { + outerLambda = TCoLambda(lambdaWithPlaceholder); + } + + innerLambda = FallbackLambdaOutput(innerLambda, ctx); + outerLambda = FallbackLambdaInput(outerLambda, ctx); + + + const auto outerReduceBy = NYql::GetSettingAsColumnList(outerReduce.Settings().Ref(), EYtSettingType::ReduceBy); + auto reduceByList = [&](TExprNodeBuilder& parent) -> TExprNodeBuilder& { + size_t index = 0; + for (const auto& reduceByName: outerReduceBy) { + parent.Callable(index++, "Member") + .Arg(0, "item") + .Atom(1, reduceByName) + .Seal(); + } + return parent; + }; + + // adds _yql_sys_tablekeyswitch column which is required for outer lambda + // _yql_sys_tableswitch equals "true" when reduce key is changed + TExprNode::TPtr keySwitchLambda = ctx.Builder(node.Pos()) + .Lambda() + .Param("stream") + .Callable(0, "Fold1Map") + .Arg(0, "stream") + .Lambda(1) + .Param("item") + .List(0) + .Callable(0, "AddMember") + .Arg(0, "item") + .Atom(1, "_yql_sys_tablekeyswitch") + .Callable(2, "Bool").Atom(0, "true").Seal() + .Seal() + .List(1).Do(reduceByList).Seal() + .Seal() + .Seal() + .Lambda(2) + .Param("item") + .Param("state") + .List(0) + .Callable(0, "AddMember") + .Arg(0, "item") + .Atom(1, "_yql_sys_tablekeyswitch") + .Callable(2, "If") + .Callable(0, "AggrEquals") + .List(0).Do(reduceByList).Seal() + .Arg(1, "state") + .Seal() + .Callable(1, "Bool").Atom(0, "false").Seal() + .Callable(2, "Bool").Atom(0, "true").Seal() + .Seal() + .Seal() + .List(1).Do(reduceByList).Seal() + .Seal() + .Seal() + .Seal() + .Seal() + .Build(); + + auto newSettings = innerReduce.Settings().Ptr(); + if (NYql::HasSetting(outerReduce.Settings().Ref(), EYtSettingType::NoDq) && + !NYql::HasSetting(innerReduce.Settings().Ref(), EYtSettingType::NoDq)) { + newSettings = NYql::AddSetting(*newSettings, EYtSettingType::NoDq, {}, ctx); + } + + if (NYql::HasSetting(outerReduce.Settings().Ref(), EYtSettingType::KeepSorted) && + !NYql::HasSetting(innerReduce.Settings().Ref(), EYtSettingType::KeepSorted)) { + newSettings = NYql::AddSetting(*newSettings, EYtSettingType::KeepSorted, {}, ctx); + } + + return Build<TYtReduce>(ctx, node.Pos()) + .InitFrom(outerReduce) + .World<TCoSync>() + .Add(innerReduce.World()) + .Add(outerReduce.World()) + .Build() + .Input(innerReduce.Input()) + .Reducer() + .Args({"stream"}) + .Body<TExprApplier>() + .Apply(outerLambda) + .With<TExprApplier>(0) + .Apply(TCoLambda(keySwitchLambda)) + .With<TExprApplier>(0) + .Apply(innerLambda) + .With(0, "stream") + .Build() + .Build() + .With(TExprBase(placeHolder), "stream") + .Build() + .Build() + .Settings(newSettings) + .Done(); +} + +TMaybeNode<TExprBase> TYtPhysicalOptProposalTransformer::FuseInnerMap(TExprBase node, TExprContext& ctx, const TGetParents& getParents) const { + auto outerMap = node.Cast<TYtMap>(); + if (outerMap.Input().Size() != 1 || outerMap.Input().Item(0).Paths().Size() != 1) { + return node; + } + + TYtPath path = outerMap.Input().Item(0).Paths().Item(0); + auto maybeInnerMap = path.Table().Maybe<TYtOutput>().Operation().Maybe<TYtMap>(); + if (!maybeInnerMap) { + return node; + } + TYtMap innerMap = maybeInnerMap.Cast(); + + if (innerMap.Ref().StartsExecution() || innerMap.Ref().HasResult()) { + return node; + } + if (innerMap.Output().Size() > 1) { + return node; + } + if (outerMap.DataSink().Cluster().Value() != innerMap.DataSink().Cluster().Value()) { + return node; + } + if (NYql::HasAnySetting(innerMap.Settings().Ref(), EYtSettingType::Limit | EYtSettingType::SortLimitBy | EYtSettingType::JobCount)) { + return node; + } + if (NYql::HasAnySetting(outerMap.Input().Item(0).Settings().Ref(), + EYtSettingType::Take | EYtSettingType::Skip | EYtSettingType::DirectRead | EYtSettingType::Sample | EYtSettingType::SysColumns | EYtSettingType::BlockInputApplied)) + { + return node; + } + if (NYql::HasSetting(innerMap.Settings().Ref(), EYtSettingType::Flow) != NYql::HasSetting(outerMap.Settings().Ref(), EYtSettingType::Flow)) { + return node; + } + if (NYql::HasAnySetting(outerMap.Settings().Ref(), EYtSettingType::JobCount)) { + return node; + } + if (!path.Ranges().Maybe<TCoVoid>()) { + return node; + } + + if (NYql::HasNonEmptyKeyFilter(outerMap.Input().Item(0))) { + return node; + } + + const TParentsMap* parentsMap = getParents(); + if (IsOutputUsedMultipleTimes(innerMap.Ref(), *parentsMap)) { + // Inner map output is used more than once + return node; + } + // Check world dependencies + auto parentsIt = parentsMap->find(innerMap.Raw()); + YQL_ENSURE(parentsIt != parentsMap->cend()); + for (auto dep: parentsIt->second) { + if (!TYtOutput::Match(dep)) { + return node; + } + } + + auto innerLambda = innerMap.Mapper(); + auto outerLambda = outerMap.Mapper(); + if (HasYtRowNumber(outerLambda.Body().Ref())) { + return node; + } + + auto fuseRes = CanFuseLambdas(innerLambda, outerLambda, ctx); + if (!fuseRes) { + // Some error + return {}; + } + if (!*fuseRes) { + // Cannot fuse + return node; + } + + const bool unorderedOut = IsUnorderedOutput(path.Table().Cast<TYtOutput>()); + + auto [placeHolder, lambdaWithPlaceholder] = ReplaceDependsOn(outerLambda.Ptr(), ctx, State_->Types); + if (!placeHolder) { + return {}; + } + + if (lambdaWithPlaceholder != outerLambda.Ptr()) { + outerLambda = TCoLambda(lambdaWithPlaceholder); + } + + innerLambda = FallbackLambdaOutput(innerLambda, ctx); + if (unorderedOut) { + innerLambda = Build<TCoLambda>(ctx, innerLambda.Pos()) + .Args({"stream"}) + .Body<TCoUnordered>() + .Input<TExprApplier>() + .Apply(innerLambda) + .With(0, "stream") + .Build() + .Build() + .Done(); + } + outerLambda = FallbackLambdaInput(outerLambda, ctx); + + if (!path.Columns().Maybe<TCoVoid>()) { + const bool ordered = !unorderedOut && NYql::HasSetting(innerMap.Settings().Ref(), EYtSettingType::Ordered) + && NYql::HasSetting(outerMap.Settings().Ref(), EYtSettingType::Ordered); + outerLambda = MapEmbedInputFieldsFilter(outerLambda, ordered, path.Columns().Cast<TCoAtomList>(), ctx); + } else if (TYqlRowSpecInfo(innerMap.Output().Item(0).RowSpec()).HasAuxColumns()) { + auto itemType = GetSequenceItemType(path, false, ctx); + if (!itemType) { + return {}; + } + TSet<TStringBuf> fields; + for (auto item: itemType->Cast<TStructExprType>()->GetItems()) { + fields.insert(item->GetName()); + } + const bool ordered = !unorderedOut && NYql::HasSetting(innerMap.Settings().Ref(), EYtSettingType::Ordered) + && NYql::HasSetting(outerMap.Settings().Ref(), EYtSettingType::Ordered); + outerLambda = MapEmbedInputFieldsFilter(outerLambda, ordered, TCoAtomList(ToAtomList(fields, node.Pos(), ctx)), ctx); + } + + const auto mergedSettings = MergeSettings( + *NYql::RemoveSettings(outerMap.Settings().Ref(), EYtSettingType::Flow | EYtSettingType::BlockInputReady, ctx), + *NYql::RemoveSettings(innerMap.Settings().Ref(), EYtSettingType::Ordered | EYtSettingType::KeepSorted, ctx), ctx); + + return Build<TYtMap>(ctx, node.Pos()) + .InitFrom(outerMap) + .World<TCoSync>() + .Add(innerMap.World()) + .Add(outerMap.World()) + .Build() + .Input(innerMap.Input()) + .Mapper() + .Args({"stream"}) + .Body<TExprApplier>() + .Apply(outerLambda) + .With<TExprApplier>(0) + .Apply(innerLambda) + .With(0, "stream") + .Build() + .With(TExprBase(placeHolder), "stream") + .Build() + .Build() + .Settings(mergedSettings) + .Done(); +} + +TMaybeNode<TExprBase> TYtPhysicalOptProposalTransformer::FuseOuterMap(TExprBase node, TExprContext& ctx, const TGetParents& getParents) const { + auto outerMap = node.Cast<TYtMap>(); + if (outerMap.Input().Size() != 1 || outerMap.Input().Item(0).Paths().Size() != 1) { + return node; + } + + TYtPath path = outerMap.Input().Item(0).Paths().Item(0); + auto maybeInner = path.Table().Maybe<TYtOutput>().Operation().Maybe<TYtWithUserJobsOpBase>(); + if (!maybeInner) { + return node; + } + if (!maybeInner.Maybe<TYtReduce>() && !maybeInner.Maybe<TYtMapReduce>()) { + return node; + } + auto inner = maybeInner.Cast(); + + if (inner.Ref().StartsExecution() || inner.Ref().HasResult()) { + return node; + } + if (inner.Output().Size() > 1) { + return node; + } + if (outerMap.DataSink().Cluster().Value() != inner.DataSink().Cluster().Value()) { + return node; + } + if (NYql::HasAnySetting(inner.Settings().Ref(), EYtSettingType::Limit | EYtSettingType::SortLimitBy | EYtSettingType::JobCount)) { + return node; + } + if (NYql::HasAnySetting(outerMap.Settings().Ref(), EYtSettingType::JobCount | EYtSettingType::BlockInputApplied)) { + return node; + } + if (outerMap.Input().Item(0).Settings().Size() != 0) { + return node; + } + if (NYql::HasSetting(inner.Settings().Ref(), EYtSettingType::Flow) != NYql::HasSetting(outerMap.Settings().Ref(), EYtSettingType::Flow)) { + return node; + } + if (!path.Ranges().Maybe<TCoVoid>()) { + return node; + } + if (inner.Maybe<TYtMapReduce>()) { + for (auto out: outerMap.Output()) { + if (TYqlRowSpecInfo(out.RowSpec()).IsSorted()) { + return node; + } + } + } + + const TParentsMap* parentsMap = getParents(); + if (IsOutputUsedMultipleTimes(inner.Ref(), *parentsMap)) { + // Inner output is used more than once + return node; + } + // Check world dependencies + auto parentsIt = parentsMap->find(inner.Raw()); + YQL_ENSURE(parentsIt != parentsMap->cend()); + for (auto dep: parentsIt->second) { + if (!TYtOutput::Match(dep)) { + return node; + } + } + + auto outerLambda = outerMap.Mapper(); + if (HasYtRowNumber(outerLambda.Body().Ref())) { + return node; + } + + auto lambda = inner.Maybe<TYtMapReduce>() ? inner.Cast<TYtMapReduce>().Reducer() : inner.Cast<TYtReduce>().Reducer(); + + auto fuseRes = CanFuseLambdas(lambda, outerLambda, ctx); + if (!fuseRes) { + // Some error + return {}; + } + if (!*fuseRes) { + // Cannot fuse + return node; + } + + auto [placeHolder, lambdaWithPlaceholder] = ReplaceDependsOn(outerLambda.Ptr(), ctx, State_->Types); + if (!placeHolder) { + return {}; + } + + if (lambdaWithPlaceholder != outerLambda.Ptr()) { + outerLambda = TCoLambda(lambdaWithPlaceholder); + } + + lambda = FallbackLambdaOutput(lambda, ctx); + outerLambda = FallbackLambdaInput(outerLambda, ctx); + + if (!path.Columns().Maybe<TCoVoid>()) { + const bool ordered = inner.Maybe<TYtReduce>() && TYqlRowSpecInfo(inner.Output().Item(0).RowSpec()).IsSorted() + && NYql::HasSetting(outerMap.Settings().Ref(), EYtSettingType::Ordered); + + outerLambda = MapEmbedInputFieldsFilter(outerLambda, ordered, path.Columns().Cast<TCoAtomList>(), ctx); + } else if (inner.Maybe<TYtReduce>() && TYqlRowSpecInfo(inner.Output().Item(0).RowSpec()).HasAuxColumns()) { + auto itemType = GetSequenceItemType(path, false, ctx); + if (!itemType) { + return {}; + } + TSet<TStringBuf> fields; + for (auto item: itemType->Cast<TStructExprType>()->GetItems()) { + fields.insert(item->GetName()); + } + const bool ordered = NYql::HasSetting(outerMap.Settings().Ref(), EYtSettingType::Ordered); + outerLambda = MapEmbedInputFieldsFilter(outerLambda, ordered, TCoAtomList(ToAtomList(fields, node.Pos(), ctx)), ctx); + } + + lambda = Build<TCoLambda>(ctx, lambda.Pos()) + .Args({"stream"}) + .Body<TExprApplier>() + .Apply(outerLambda) + .With<TExprApplier>(0) + .Apply(lambda) + .With(0, "stream") + .Build() + .With(TExprBase(placeHolder), "stream") + .Build() + .Done(); + + auto res = ctx.ChangeChild(inner.Ref(), + inner.Maybe<TYtMapReduce>() ? TYtMapReduce::idx_Reducer : TYtReduce::idx_Reducer, + lambda.Ptr()); + res = ctx.ChangeChild(*res, TYtWithUserJobsOpBase::idx_Output, outerMap.Output().Ptr()); + + auto mergedSettings = NYql::RemoveSettings(outerMap.Settings().Ref(), EYtSettingType::Ordered | EYtSettingType::Sharded | EYtSettingType::Flow | EYtSettingType::BlockInputReady, ctx); + mergedSettings = MergeSettings(inner.Settings().Ref(), *mergedSettings, ctx); + res = ctx.ChangeChild(*res, TYtWithUserJobsOpBase::idx_Settings, std::move(mergedSettings)); + res = ctx.ChangeChild(*res, TYtWithUserJobsOpBase::idx_World, + Build<TCoSync>(ctx, inner.Pos()) + .Add(inner.World()) + .Add(outerMap.World()) + .Done().Ptr()); + + return TExprBase(res); +} + +} // namespace NYql diff --git a/yt/yql/providers/yt/provider/phy_opt/yql_yt_phy_opt_helper.cpp b/yt/yql/providers/yt/provider/phy_opt/yql_yt_phy_opt_helper.cpp new file mode 100644 index 0000000000..608724b4e3 --- /dev/null +++ b/yt/yql/providers/yt/provider/phy_opt/yql_yt_phy_opt_helper.cpp @@ -0,0 +1,1071 @@ +#include "yql_yt_phy_opt_helper.h" + +#include <yt/yql/providers/yt/provider/yql_yt_helpers.h> +#include <yt/yql/providers/yt/lib/expr_traits/yql_expr_traits.h> +#include <yt/yql/providers/yt/opt/yql_yt_key_selector.h> +#include <yql/essentials/providers/common/codec/yql_codec_type_flags.h> + +#include <yql/essentials/core/yql_opt_utils.h> +#include <yql/essentials/core/yql_type_helpers.h> +#include <yql/essentials/core/yql_expr_optimize.h> +#include <yql/essentials/core/yql_join.h> + +#include <yql/essentials/utils/log/log.h> + +#include <util/string/type.h> + +namespace NYql::NPrivate { + +using namespace NNodes; + +TYtSectionList ConvertInputTable(TExprBase input, TExprContext& ctx, const TConvertInputOpts& opts) +{ + TVector<TYtSection> sections; + TExprBase columns = opts.CustomFields_ ? TExprBase(opts.CustomFields_.Cast()) : TExprBase(Build<TCoVoid>(ctx, input.Pos()).Done()); + if (auto out = input.Maybe<TYtOutput>()) { + auto settings = opts.Settings_; + if (!settings) { + settings = Build<TCoNameValueTupleList>(ctx, input.Pos()).Done(); + } + TMaybeNode<TCoAtom> mode = out.Mode(); + if (opts.MakeUnordered_) { + mode = Build<TCoAtom>(ctx, out.Cast().Pos()).Value(ToString(EYtSettingType::Unordered)).Done(); + } else if (opts.ClearUnordered_) { + mode = {}; + } + sections.push_back(Build<TYtSection>(ctx, input.Pos()) + .Paths() + .Add() + .Table<TYtOutput>() + .InitFrom(out.Cast()) + .Mode(mode) + .Build() + .Columns(columns) + .Ranges<TCoVoid>().Build() + .Stat<TCoVoid>().Build() + .Build() + .Build() + .Settings(settings.Cast()) + .Done()); + } + else { + auto read = input.Maybe<TCoRight>().Input().Maybe<TYtReadTable>(); + YQL_ENSURE(read, "Unknown operation input"); + + for (auto section: read.Cast().Input()) { + bool makeUnordered = opts.MakeUnordered_; + + auto mergedSettings = section.Settings().Ptr(); + if (NYql::HasSetting(*mergedSettings, EYtSettingType::Unordered)) { + mergedSettings = NYql::RemoveSetting(*mergedSettings, EYtSettingType::Unordered, ctx); + makeUnordered = false; + } + if (!opts.KeepDirecRead_) { + mergedSettings = NYql::RemoveSetting(*mergedSettings, EYtSettingType::DirectRead, ctx); + } + if (opts.Settings_) { + mergedSettings = MergeSettings(*mergedSettings, opts.Settings_.Cast().Ref(), ctx); + } + + section = Build<TYtSection>(ctx, section.Pos()) + .InitFrom(section) + .Settings(mergedSettings) + .Done(); + + if (makeUnordered) { + section = MakeUnorderedSection(section, ctx); + } else if (opts.ClearUnordered_) { + section = ClearUnorderedSection(section, ctx); + } + + if (opts.CustomFields_) { + section = UpdateInputFields(section, opts.CustomFields_.Cast(), ctx); + } + + sections.push_back(section); + } + } + + return Build<TYtSectionList>(ctx, input.Pos()) + .Add(sections) + .Done(); +} + +bool CollectMemberPaths(TExprBase row, const TExprNode::TPtr& lookupItem, + TMap<TStringBuf, TVector<ui32>>& memberPaths, TVector<ui32>& currPath) +{ + if (lookupItem->IsCallable("Member") && lookupItem->Child(0) == row.Raw()) { + TStringBuf memberName = lookupItem->Child(1)->Content(); + memberPaths.insert({ memberName, currPath }); + return true; + } + + if (lookupItem->IsList()) { + for (ui32 i = 0; i < lookupItem->ChildrenSize(); ++i) { + currPath.push_back(i); + auto res = CollectMemberPaths(row, lookupItem->ChildPtr(i), memberPaths, currPath); + currPath.pop_back(); + if (!res) { + return false; + } + } + return true; + } + + return !IsDepended(*lookupItem, row.Ref()); +} + +bool CollectMemberPaths(TExprBase row, const TExprNode::TPtr& lookupItem, + TMap<TStringBuf, TVector<ui32>>& memberPaths) +{ + TVector<ui32> currPath; + return CollectMemberPaths(row, lookupItem, memberPaths, currPath); +} + +bool CollectKeyPredicatesFromLookup(TExprBase row, TCoLookupBase lookup, TVector<TKeyFilterPredicates>& ranges, + size_t maxTables) +{ + TExprNode::TPtr collection; + if (lookup.Collection().Ref().IsList()) { + collection = lookup.Collection().Ptr(); + } else if (auto maybeAsList = lookup.Collection().Maybe<TCoAsList>()) { + collection = maybeAsList.Cast().Ptr(); + } else if (auto maybeDictFromKeys = lookup.Collection().Maybe<TCoDictFromKeys>()) { + collection = maybeDictFromKeys.Cast().Keys().Ptr(); + } else { + YQL_CLOG(DEBUG, ProviderYt) << __FUNCTION__ << ": unsupported collection " << lookup.Collection().Ref().Content(); + return false; + } + + auto size = collection->ChildrenSize(); + if (!size) { + YQL_CLOG(DEBUG, ProviderYt) << __FUNCTION__ << ": empty keys collection"; + return false; + } + + if (size + ranges.size() > maxTables) { + YQL_CLOG(DEBUG, ProviderYt) << __FUNCTION__ << ": too many dict keys - " << size; + return false; + } + + if (IsDepended(*collection, row.Ref())) { + YQL_CLOG(DEBUG, ProviderYt) << __FUNCTION__ << ": depends on lambda arg"; + return false; + } + + TExprNode::TPtr lookupItem = lookup.Lookup().Ptr(); + TMap<TStringBuf, TVector<ui32>> memberPaths; + if (!CollectMemberPaths(row, lookupItem, memberPaths)) { + YQL_CLOG(DEBUG, ProviderYt) << __FUNCTION__ << ": unsupported lookup item"; + return false; + } + + if (memberPaths.empty()) { + YQL_CLOG(DEBUG, ProviderYt) << __FUNCTION__ << ": no key predicates in lookup item"; + return false; + } + + for (auto& collectionItem : collection->Children()) { + ranges.emplace_back(); + for (auto& memberAndPath : memberPaths) { + auto member = memberAndPath.first; + auto& path = memberAndPath.second; + auto value = collectionItem; + for (auto idx : path) { + if (!value->IsList() || idx >= value->ChildrenSize()) { + YQL_CLOG(DEBUG, ProviderYt) << __FUNCTION__ << ": unexpected literal structure"; + return false; + } + value = value->ChildPtr(idx); + } + ranges.back().emplace(member, std::make_pair(TString{TCoCmpEqual::CallableName()}, value)); + } + } + + return true; +} + +bool CollectKeyPredicatesAnd(TExprBase row, const std::vector<TExprBase>& predicates, TVector<TKeyFilterPredicates>& ranges, size_t maxTables) +{ + TVector<TKeyFilterPredicates> leftRanges; + for (const auto& predicate : predicates) { + TVector<TKeyFilterPredicates> rightRanges; + if (CollectKeyPredicates(row, predicate, rightRanges, maxTables)) { + if (leftRanges.empty()) { + leftRanges = std::move(rightRanges); + } else { + const auto total = leftRanges.size() * rightRanges.size(); + if (total + ranges.size() > maxTables) { + YQL_CLOG(DEBUG, ProviderYt) << __func__ << ": too many tables - " << (total + ranges.size()); + return false; + } + + if (1U == total) { + leftRanges.front().insert(rightRanges.front().cbegin(), rightRanges.front().cend()); + } else { + TVector<TKeyFilterPredicates> temp; + temp.reserve(total); + for (const auto& left : leftRanges) { + for (const auto& right : rightRanges) { + temp.emplace_back(left); + temp.back().insert(right.cbegin(), right.cend()); + } + } + leftRanges = std::move(temp); + } + } + } + } + + if (leftRanges.empty()) { + return false; + } + + std::move(leftRanges.begin(), leftRanges.end(), std::back_inserter(ranges)); + return true; +} + +bool CollectKeyPredicatesOr(TExprBase row, const std::vector<TExprBase>& predicates, TVector<TKeyFilterPredicates>& ranges, size_t maxTables) +{ + for (const auto& predicate : predicates) { + if (!CollectKeyPredicates(row, predicate, ranges, maxTables)) { + return false; + } + } + return true; +} + +bool CollectKeyPredicates(TExprBase row, TExprBase predicate, TVector<TKeyFilterPredicates>& ranges, size_t maxTables) +{ + if (const auto maybeAnd = predicate.Maybe<TCoAnd>()) { + const auto size = maybeAnd.Cast().Args().size(); + std::vector<TExprBase> predicates; + predicates.reserve(size); + for (auto i = 0U; i < size; ++i) { + predicates.emplace_back(maybeAnd.Cast().Arg(i)); + } + return CollectKeyPredicatesAnd(row, predicates, ranges, maxTables); + } + + if (const auto maybeOr = predicate.Maybe<TCoOr>()) { + const auto size = maybeOr.Cast().Args().size(); + std::vector<TExprBase> predicates; + predicates.reserve(size); + for (auto i = 0U; i < size; ++i) { + predicates.emplace_back(maybeOr.Cast().Arg(i)); + } + return CollectKeyPredicatesOr(row, predicates, ranges, maxTables); + } + + TMaybeNode<TCoCompare> maybeCompare = predicate.Maybe<TCoCompare>(); + if (auto maybeLiteral = predicate.Maybe<TCoCoalesce>().Value().Maybe<TCoBool>().Literal()) { + if (maybeLiteral.Cast().Value() == "false") { + maybeCompare = predicate.Cast<TCoCoalesce>().Predicate().Maybe<TCoCompare>(); + } + } + + auto getRowMember = [row] (TExprBase expr) { + if (auto maybeMember = expr.Maybe<TCoMember>()) { + if (maybeMember.Cast().Struct().Raw() == row.Raw()) { + return maybeMember; + } + } + + return TMaybeNode<TCoMember>(); + }; + + if (maybeCompare) { + auto left = maybeCompare.Cast().Left(); + auto right = maybeCompare.Cast().Right(); + + TMaybeNode<TCoMember> maybeMember = getRowMember(left); + TMaybeNode<TExprBase> maybeValue = right; + bool invert = false; + if (!maybeMember) { + maybeMember = getRowMember(right); + maybeValue = left; + invert = true; + } + + if (!maybeMember) { + YQL_CLOG(DEBUG, ProviderYt) << __FUNCTION__ << ": comparison with non-member"; + return false; + } + + const auto value = maybeValue.Cast(); + if (IsDepended(value.Ref(), row.Ref())) { + YQL_CLOG(DEBUG, ProviderYt) << __FUNCTION__ << ": depends on lambda arg"; + return false; + } + + auto column = maybeMember.Cast().Name().Value(); + TString cmpOp = TString{maybeCompare.Cast().Ref().Content()}; + if (!IsRangeComparison(cmpOp)) { + YQL_CLOG(DEBUG, ProviderYt) << __FUNCTION__ << ": non-range comparison " << cmpOp; + return false; + } + + if (invert) { + if (TCoCmpStartsWith::CallableName() == cmpOp) + return false; + + switch (cmpOp.front()) { + case '<': cmpOp.replace(0, 1, 1, '>'); break; + case '>': cmpOp.replace(0, 1, 1, '<'); break; + default: break; + } + } + + ranges.emplace_back(); + ranges.back().emplace(column, std::make_pair(cmpOp, value.Ptr())); + return true; + } + + if (auto maybeLookup = predicate.Maybe<TCoLookupBase>()) { + return CollectKeyPredicatesFromLookup(row, maybeLookup.Cast(), ranges, maxTables); + } + + if (auto maybeLiteral = predicate.Maybe<TCoCoalesce>().Value().Maybe<TCoBool>().Literal()) { + if (maybeLiteral.Cast().Value() == "false") { + if (auto maybeLookup = predicate.Maybe<TCoCoalesce>().Predicate().Maybe<TCoLookupBase>()) { + return CollectKeyPredicatesFromLookup(row, maybeLookup.Cast(), ranges, maxTables); + } + } + } + + if (auto maybeNotExists = predicate.Maybe<TCoNot>().Value().Maybe<TCoExists>().Optional()) { + TMaybeNode<TCoMember> maybeMember = getRowMember(maybeNotExists.Cast()); + if (!maybeMember) { + YQL_CLOG(DEBUG, ProviderYt) << __FUNCTION__ << ": Not Exists for non-member"; + return false; + } + auto column = maybeMember.Cast().Name().Value(); + ranges.emplace_back(); + ranges.back().emplace(column, std::make_pair(TString{TCoCmpEqual::CallableName()}, TExprNode::TPtr())); + return true; + } + + YQL_CLOG(DEBUG, ProviderYt) << __FUNCTION__ << ": unsupported predicate " << predicate.Ref().Content(); + return false; +} + +TMaybeNode<TCoLambda> GetLambdaWithPredicate(TCoLambda lambda) { + if (auto innerFlatMap = lambda.Body().Maybe<TCoFlatMapBase>()) { + if (auto arg = innerFlatMap.Input().Maybe<TCoFilterNullMembersBase>().Input().Maybe<TCoJust>().Input()) { + if (arg.Cast().Raw() == lambda.Args().Arg(0).Raw()) { + lambda = innerFlatMap.Lambda().Cast(); + } + } + } + if (!lambda.Body().Maybe<TCoConditionalValueBase>()) { + return {}; + } + return lambda; +} + +bool IsAscending(const TExprNode& node) { + TMaybe<bool> ascending; + if (node.IsCallable("Bool")) { + ascending = IsTrue(node.Child(0)->Content()); + } + return ascending.Defined() && *ascending; +} + +bool CollectSortSet(const TExprNode& sortNode, TSet<TVector<TStringBuf>>& sortSets) { + if (sortNode.IsCallable("Sort")) { + auto directions = sortNode.ChildPtr(1); + + auto lambdaArg = sortNode.Child(2)->Child(0)->Child(0); + auto lambdaBody = sortNode.Child(2)->ChildPtr(1); + + TExprNode::TListType directionItems; + if (directions->GetTypeAnn()->GetKind() == ETypeAnnotationKind::Tuple) { + directionItems = directions->ChildrenList(); + } else { + directionItems.push_back(directions); + } + + if (AnyOf(directionItems, [](const TExprNode::TPtr& direction) { return !IsAscending(*direction); })) { + return false; + } + + TExprNode::TListType lambdaBodyItems; + if (directions->GetTypeAnn()->GetKind() == ETypeAnnotationKind::Tuple) { + lambdaBodyItems = lambdaBody->ChildrenList(); + } else { + lambdaBodyItems.push_back(lambdaBody); + } + + TVector<TStringBuf> sortBy; + for (auto& item : lambdaBodyItems) { + if (!item->IsCallable("Member") || item->Child(0) != lambdaArg) { + return false; + } + YQL_ENSURE(item->Child(1)->IsAtom()); + sortBy.push_back(item->Child(1)->Content()); + } + + return sortSets.insert(sortBy).second; + } else if (sortNode.IsCallable("Aggregate")) { + if (!HasSetting(TCoAggregate(&sortNode).Settings().Ref(), "compact")) { + return false; + } + auto keys = sortNode.Child(1); + const auto keyNum = keys->ChildrenSize(); + if (keyNum == 0) { + return false; + } + + TVector<TStringBuf> keyList; + keyList.reserve(keys->ChildrenSize()); + + for (const auto& key : keys->ChildrenList()) { + keyList.push_back(key->Content()); + } + + do { + TVector<TStringBuf> sortBy; + sortBy.reserve(keyNum); + copy(keyList.begin(), keyList.end(), std::back_inserter(sortBy)); + sortSets.insert(sortBy); + if (sortSets.size() > 20) { + YQL_CLOG(WARN, ProviderYt) << __FUNCTION__ << ": join's preferred_sort can't have more than 20 key combinations"; + return true; + } + } while(next_permutation(keyList.begin(), keyList.end())); + sortSets.insert(keyList); + + return true; + } else { + return false; + } +} + +TExprNode::TPtr CollectPreferredSortsForEquiJoinOutput(TExprBase join, const TExprNode::TPtr& options, + TExprContext& ctx, const TParentsMap& parentsMap) +{ + auto parentsIt = parentsMap.find(join.Raw()); + if (parentsIt == parentsMap.end()) { + return options; + } + + TSet<TVector<TStringBuf>> sortSets = LoadJoinSortSets(*options); + size_t collected = 0; + for (auto& parent : parentsIt->second) { + if (CollectSortSet(*parent, sortSets)) { + ++collected; + } + } + + if (!collected) { + return options; + } + + YQL_CLOG(INFO, ProviderYt) << __FUNCTION__ << ": Collected " << collected << " new sorts"; + + auto removedOptions = RemoveSetting(*options, "preferred_sort", ctx); + TExprNode::TListType optionsNodes = removedOptions->ChildrenList(); + AppendEquiJoinSortSets(options->Pos(), sortSets, optionsNodes, ctx); + + return ctx.NewList(options->Pos(), std::move(optionsNodes)); +} + +bool CanExtraColumnBePulledIntoEquiJoin(const TTypeAnnotationNode* type) { + if (type->GetKind() == ETypeAnnotationKind::Optional) { + type = type->Cast<TOptionalExprType>()->GetItemType(); + } + + switch (type->GetKind()) { + case ETypeAnnotationKind::Data: + return IsFixedSizeData(type); + case ETypeAnnotationKind::Null: + case ETypeAnnotationKind::Void: + return true; + default: + return false; + } +} + +bool IsYtOrPlainTablePropsDependent(NNodes::TExprBase input) { + bool found = false; + VisitExpr(input.Ref(), [&found](const TExprNode& n) { + found = found || TYtTablePropBase::Match(&n) || TCoTablePropBase::Match(&n); + return !found; + }); + return found; +} + +bool IsLambdaSuitableForPullingIntoEquiJoin(const TCoFlatMapBase& flatMap, const TExprNode& label, + const THashMap<TStringBuf, THashSet<TStringBuf>>& tableKeysMap, + const TExprNode* extractedMembers) +{ + if (!label.IsAtom()) { + return false; + } + + auto inputSeqItem = SilentGetSequenceItemType(flatMap.Input().Ref(), false); + if (!inputSeqItem || !inputSeqItem->IsPersistable()) { + return false; + } + + auto outputSeqItem = SilentGetSequenceItemType(flatMap.Ref(), false); + if (!outputSeqItem || !outputSeqItem->IsPersistable()) { + return false; + } + + if (IsYtOrPlainTablePropsDependent(flatMap.Lambda().Body())) { + return false; + } + + // allow only projective FlatMaps + if (!IsJustOrSingleAsList(flatMap.Lambda().Body().Ref())) { + return false; + } + + // all input column should be either renamed, removed or passed as is + // all join keys should be passed as-is + // only fixed-size data type can be added + auto arg = flatMap.Lambda().Args().Arg(0).Raw(); + auto outItem = flatMap.Lambda().Body().Ref().Child(0); + + if (outItem->IsCallable("AsStruct")) { + size_t joinKeysPassed = 0; + auto it = tableKeysMap.find(label.Content()); + const size_t joinKeysCount = (it == tableKeysMap.end()) ? 0 : it->second.size(); + + TMaybe<THashSet<TStringBuf>> filteredMemberSet; + if (extractedMembers) { + filteredMemberSet.ConstructInPlace(); + for (auto member : extractedMembers->ChildrenList()) { + YQL_ENSURE(member->IsAtom()); + filteredMemberSet->insert(member->Content()); + } + } + for (auto& item : outItem->Children()) { + TStringBuf outMemberName = item->Child(0)->Content(); + if (filteredMemberSet && !filteredMemberSet->contains(outMemberName)) { + // member will be filtered out by parent ExtractMembers + continue; + } + if (item->Child(1)->IsCallable("Member") && item->Child(1)->Child(0) == arg) { + TStringBuf inMemberName = item->Child(1)->Child(1)->Content(); + bool isJoinKey = joinKeysCount && it->second.contains(outMemberName); + if (isJoinKey && inMemberName != outMemberName) { + return false; + } + joinKeysPassed += isJoinKey; + } else if (!CanExtraColumnBePulledIntoEquiJoin(item->Child(1)->GetTypeAnn())) { + return false; + } + } + + YQL_ENSURE(joinKeysPassed <= joinKeysCount); + return joinKeysPassed == joinKeysCount; + } else { + return outItem == arg; + } +} + +TExprNode::TPtr BuildYtEquiJoinPremap(TExprBase list, TMaybeNode<TCoLambda> premapLambda, TExprContext& ctx) { + if (auto type = GetSequenceItemType(list, false, ctx)) { + if (!EnsurePersistableType(list.Pos(), *type, ctx)) { + return {}; + } + if (premapLambda) { + return premapLambda.Cast().Ptr(); + } + return Build<TCoVoid>(ctx, list.Pos()).Done().Ptr(); + } + return {}; +} + +// label -> pair(<asc sort keys>, <inputs matched by keys>) +THashMap<TStringBuf, std::pair<TVector<TStringBuf>, ui32>> CollectTableSortKeysUsage(const TYtState::TPtr& state, const TCoEquiJoin& equiJoin) { + THashMap<TStringBuf, std::pair<TVector<TStringBuf>, ui32>> tableSortKeys; + for (size_t i = 0; i + 2 < equiJoin.ArgCount(); ++i) { + auto joinInput = equiJoin.Arg(i).Cast<TCoEquiJoinInput>(); + auto list = joinInput.List(); + if (joinInput.Scope().Ref().IsAtom()) { + TVector<TStringBuf> sortKeys; + if (const auto sorted = list.Ref().GetConstraint<TSortedConstraintNode>()) { + for (const auto& key : sorted->GetContent()) { + bool appropriate = false; + if (key.second && !key.first.empty()) { + for (auto& alt: key.first) { + if (alt.size() == 1U) { + sortKeys.emplace_back(alt.front()); + appropriate = true; + break; + } + } + } + if (!appropriate) { + break; + } + } + } + tableSortKeys[joinInput.Scope().Ref().Content()] = std::make_pair(std::move(sortKeys), 0); + } + } + + // Only Lookup, Merge, and Star strategies use a table sort + if (!state->Configuration->JoinMergeTablesLimit.Get().GetOrElse(0) + && !(state->Configuration->LookupJoinLimit.Get().GetOrElse(0) && state->Configuration->LookupJoinMaxRows.Get().GetOrElse(0)) + && !(state->Configuration->JoinEnableStarJoin.Get().GetOrElse(false) && state->Configuration->JoinAllowColumnRenames.Get().GetOrElse(true)) + ) { + return tableSortKeys; + } + + TVector<const TExprNode*> joinTreeNodes; + joinTreeNodes.push_back(equiJoin.Arg(equiJoin.ArgCount() - 2).Raw()); + while (!joinTreeNodes.empty()) { + const TExprNode* joinTree = joinTreeNodes.back(); + joinTreeNodes.pop_back(); + + if (!joinTree->Child(1)->IsAtom()) { + joinTreeNodes.push_back(joinTree->Child(1)); + } + + if (!joinTree->Child(2)->IsAtom()) { + joinTreeNodes.push_back(joinTree->Child(2)); + } + + if (joinTree->Child(0)->Content() != "Cross") { + THashMap<TStringBuf, THashSet<TStringBuf>> tableJoinKeys; + for (auto keys: {joinTree->Child(3), joinTree->Child(4)}) { + for (ui32 i = 0; i < keys->ChildrenSize(); i += 2) { + auto tableName = keys->Child(i)->Content(); + auto column = keys->Child(i + 1)->Content(); + tableJoinKeys[tableName].insert(column); + } + } + + for (auto& [label, joinKeys]: tableJoinKeys) { + if (auto sortKeys = tableSortKeys.FindPtr(label)) { + if (joinKeys.size() <= sortKeys->first.size()) { + bool matched = true; + for (size_t i = 0; i < joinKeys.size(); ++i) { + if (!joinKeys.contains(sortKeys->first[i])) { + matched = false; + break; + } + } + if (matched) { + ++sortKeys->second; + } + } + } + } + } + } + + return tableSortKeys; +} + +TCoLambda FallbackLambdaInput(TCoLambda lambda, TExprContext& ctx) { + if (const auto narrow = FindNode(lambda.Ref().TailPtr(), [&](const TExprNode::TPtr& node) { return node->IsCallable("NarrowMap") && &lambda.Args().Arg(0).Ref() == &node->Head(); })) { + return TCoLambda(ctx.DeepCopyLambda(lambda.Ref(), ctx.ReplaceNodes(lambda.Ref().TailPtr(), {{narrow.Get(), narrow->HeadPtr()}}))); + } + + return lambda; +} + +TCoLambda FallbackLambdaOutput(TCoLambda lambda, TExprContext& ctx) { + if (auto body = lambda.Ref().TailPtr(); body->IsCallable("ExpandMap")) { + return TCoLambda(ctx.DeepCopyLambda(lambda.Ref(), body->HeadPtr())); + } + + return lambda; +} + +TYtDSink GetDataSink(TExprBase input, TExprContext& ctx) { + if (auto read = input.Maybe<TCoRight>().Input().Maybe<TYtReadTable>()) { + return TYtDSink(ctx.RenameNode(read.Cast().DataSource().Ref(), "DataSink")); + } else if (auto out = input.Maybe<TYtOutput>()) { + return GetOutputOp(out.Cast()).DataSink(); + } else { + YQL_ENSURE(false, "Unknown operation input"); + } +} + +TExprBase GetWorld(TExprBase input, TMaybeNode<TExprBase> main, TExprContext& ctx) { + if (!main) { + main = ctx.NewWorld(input.Pos()); + } + TSyncMap syncList; + if (auto maybeRead = input.Maybe<TCoRight>().Input().Maybe<TYtReadTable>()) { + auto read = maybeRead.Cast(); + if (read.World().Ref().Type() != TExprNode::World) { + syncList.emplace(read.World().Ptr(), syncList.size()); + } + } else { + YQL_ENSURE(input.Maybe<TYtOutput>(), "Unknown operation input: " << input.Ref().Content()); + } + return TExprBase(ApplySyncListToWorld(main.Cast().Ptr(), syncList, ctx)); +} + +TConvertInputOpts::TConvertInputOpts() + : KeepDirecRead_(false) + , MakeUnordered_(false) + , ClearUnordered_(false) +{ +} + +TConvertInputOpts& +TConvertInputOpts::Settings(const TMaybeNode<TCoNameValueTupleList>& settings) { + Settings_ = settings; + return *this; +} + +TConvertInputOpts& +TConvertInputOpts::CustomFields(const TMaybeNode<TCoAtomList>& customFields) { + CustomFields_ = customFields; + return *this; +} + +TConvertInputOpts& +TConvertInputOpts::ExplicitFields(const TYqlRowSpecInfo& rowSpec, TPositionHandle pos, TExprContext& ctx) { + TVector<TString> columns; + for (auto item: rowSpec.GetType()->GetItems()) { + columns.emplace_back(item->GetName()); + } + for (auto item: rowSpec.GetAuxColumns()) { + columns.emplace_back(item.first); + } + CustomFields_ = ToAtomList(columns, pos, ctx); + return *this; +} + +TConvertInputOpts& +TConvertInputOpts::ExplicitFields(const TStructExprType& type, TPositionHandle pos, TExprContext& ctx) { + TVector<TString> columns; + for (auto item: type.GetItems()) { + columns.emplace_back(item->GetName()); + } + CustomFields_ = ToAtomList(columns, pos, ctx); + return *this; +} + +TConvertInputOpts& +TConvertInputOpts::KeepDirecRead(bool keepDirecRead) { + KeepDirecRead_ = keepDirecRead; + return *this; +} + +TConvertInputOpts& +TConvertInputOpts::MakeUnordered(bool makeUnordered) { + YQL_ENSURE(!makeUnordered || !ClearUnordered_); + MakeUnordered_ = makeUnordered; + return *this; +} + +TConvertInputOpts& +TConvertInputOpts::ClearUnordered() { + YQL_ENSURE(!MakeUnordered_); + ClearUnordered_ = true; + return *this; +} + +void TYtSortMembersCollection::BuildKeyFilters(TPositionHandle pos, size_t tableCount, size_t orGroupCount, TExprNode::TListType& result, TExprContext& ctx) { + TExprNode::TListType prefix; + for (size_t orGroup = 0; orGroup < orGroupCount; ++orGroup) { + BuildKeyFiltersImpl(pos, tableCount, orGroup, prefix, Members, result, ctx); + } + } + +void TYtSortMembersCollection::BuildKeyFiltersImpl(TPositionHandle pos, size_t tableCount, size_t orGroup, TExprNode::TListType& prefix, + const TMemberDescrMap& members, TExprNode::TListType& result, TExprContext& ctx) +{ + for (auto& item: members) { + size_t prefixLen = prefix.size(); + + auto keyPredicateBuilder = Build<TCoNameValueTupleList>(ctx, pos); + auto iterRange = item.second->Ranges.equal_range(orGroup); + if (iterRange.first != iterRange.second) { + for (auto it = iterRange.first; it != iterRange.second; ++it) { + TString cmpOp; + TExprNode::TPtr value; + const TDataExprType* dataType = nullptr; + std::tie(cmpOp, value, dataType) = it->second; + + if (!value) { + keyPredicateBuilder + .Add() + .Name() + .Value(cmpOp) + .Build() + .Value<TCoNull>() + .Build() + .Build(); + } else { + keyPredicateBuilder + .Add() + .Name() + .Value(cmpOp) + .Build() + .Value(value) + .Build(); + } + } + + prefix.push_back( + Build<TCoNameValueTuple>(ctx, pos) + .Name() + .Value(item.first) + .Build() + .Value(keyPredicateBuilder.Done()) + .Done().Ptr() + ); + } + + if (!item.second->Tables.empty()) { + YQL_ENSURE(!prefix.empty()); + if (item.second->Tables.size() == tableCount) { + result.push_back(Build<TCoNameValueTuple>(ctx, pos) + .Name() + .Value(ToString(EYtSettingType::KeyFilter)) + .Build() + .Value<TExprList>() + .Add<TExprList>() + .Add(prefix) + .Build() + .Build() + .Done().Ptr() + ); + } + else { + for (auto tableNdx: item.second->Tables) { + result.push_back(Build<TCoNameValueTuple>(ctx, pos) + .Name() + .Value(ToString(EYtSettingType::KeyFilter)) + .Build() + .Value<TExprList>() + .Add<TExprList>() + .Add(prefix) + .Build() + .Add<TCoAtom>() + .Value(ToString(tableNdx)) + .Build() + .Build() + .Done().Ptr() + ); + } + } + } + YQL_ENSURE(item.second->Tables.size() != tableCount || item.second->NextMembers.empty()); + BuildKeyFiltersImpl(pos, tableCount, orGroup, prefix, item.second->NextMembers, result, ctx); + + prefix.erase(prefix.begin() + prefixLen, prefix.end()); + } +} + +TExprBase WrapOp(TYtOutputOpBase op, TExprContext& ctx) { + if (op.Output().Size() > 1) { + return Build<TCoRight>(ctx, op.Pos()) + .Input(op) + .Done(); + } + + return Build<TYtOutput>(ctx, op.Pos()) + .Operation(op) + .OutIndex().Value("0").Build() + .Done(); +} + +TCoLambda MapEmbedInputFieldsFilter(TCoLambda lambda, bool ordered, TCoAtomList fields, TExprContext& ctx) { + auto filter = [&](TExprNodeBuilder& parent) -> TExprNodeBuilder& { + ui32 index = 0; + for (const auto& x : fields) { + parent + .List(index++) + .Add(0, x.Ptr()) + .Callable(1, "Member") + .Arg(0, "row") + .Add(1, x.Ptr()) + .Seal() + .Seal(); + } + + return parent; + }; + + return TCoLambda(ctx.Builder(lambda.Pos()) + .Lambda() + .Param("stream") + .Apply(lambda.Ptr()).With(0) + .Callable(ordered ? "OrderedFlatMap" : "FlatMap") + .Arg(0, "stream") + .Lambda(1) + .Param("row") + .Callable("Just") + .Callable(0, "AsStruct") + .Do(filter) + .Seal() + .Seal() + .Seal() + .Seal() + .Done() + .Seal() + .Seal() + .Build()); +} + +TVector<TYtOutTable> ConvertMultiOutTables(TPositionHandle pos, const TTypeAnnotationNode* outItemType, TExprContext& ctx, + const TYtState::TPtr& state, const TMultiConstraintNode* multi) { + TVector<TYtOutTable> outTables; + YQL_ENSURE(outItemType->GetKind() == ETypeAnnotationKind::Variant); + const TTupleExprType* tupleType = outItemType->Cast<TVariantExprType>()->GetUnderlyingType()->Cast<TTupleExprType>(); + size_t ndx = 0; + const ui64 nativeTypeFlags = state->Configuration->UseNativeYtTypes.Get().GetOrElse(DEFAULT_USE_NATIVE_YT_TYPES) ? NTCF_ALL : NTCF_NONE; + for (auto tupleItemType: tupleType->GetItems()) { + TYtOutTableInfo outTableInfo(tupleItemType->Cast<TStructExprType>(), nativeTypeFlags); + const TConstraintSet* constraints = multi ? multi->GetItem(ndx) : nullptr; + if (constraints) + outTableInfo.RowSpec->SetConstraints(*constraints); + outTables.push_back(outTableInfo.SetUnique(constraints ? constraints->GetConstraint<TDistinctConstraintNode>() : nullptr, pos, ctx).ToExprNode(ctx, pos).Cast<TYtOutTable>()); + ++ndx; + } + return outTables; +} + +TVector<TYtOutTable> ConvertOutTables(TPositionHandle pos, const TTypeAnnotationNode* outItemType, TExprContext& ctx, + const TYtState::TPtr& state, const TConstraintSet* constraint) { + if (outItemType->GetKind() == ETypeAnnotationKind::Variant) { + return ConvertMultiOutTables(pos, outItemType, ctx, state, constraint ? constraint->GetConstraint<TMultiConstraintNode>() : nullptr); + } + + TYtOutTableInfo outTableInfo(outItemType->Cast<TStructExprType>(), state->Configuration->UseNativeYtTypes.Get().GetOrElse(DEFAULT_USE_NATIVE_YT_TYPES) ? NTCF_ALL : NTCF_NONE); + if (constraint) + outTableInfo.RowSpec->SetConstraints(*constraint); + return TVector<TYtOutTable>{outTableInfo.SetUnique(constraint ? constraint->GetConstraint<TDistinctConstraintNode>() : nullptr, pos, ctx).ToExprNode(ctx, pos).Cast<TYtOutTable>()}; +} + +TVector<TYtOutTable> ConvertMultiOutTablesWithSortAware(TExprNode::TPtr& lambda, bool& ordered, TPositionHandle pos, + const TTypeAnnotationNode* outItemType, TExprContext& ctx, const TYtState::TPtr& state, const TConstraintSet& constraints) { + + YQL_ENSURE(outItemType->GetKind() == ETypeAnnotationKind::Variant); + + const ui64 nativeTypeFlags = state->Configuration->UseNativeYtTypes.Get().GetOrElse(DEFAULT_USE_NATIVE_YT_TYPES) ? NTCF_ALL : NTCF_NONE; + const bool useNativeDescSort = state->Configuration->UseNativeDescSort.Get().GetOrElse(DEFAULT_USE_NATIVE_DESC_SORT); + const auto multi = constraints.GetConstraint<TMultiConstraintNode>(); + const TTupleExprType* tupleType = outItemType->Cast<TVariantExprType>()->GetUnderlyingType()->Cast<TTupleExprType>(); + + ordered = false; + TVector<TYtOutTable> outTables; + size_t ndx = 0; + TVector<TExprBase> switchArgs; + for (auto tupleItemType: tupleType->GetItems()) { + const TConstraintSet* itemConstraints = multi ? multi->GetItem(ndx) : nullptr; + TYtOutTableInfo outTable(tupleItemType->Cast<TStructExprType>(), nativeTypeFlags); + TExprNode::TPtr remapper; + if (auto sorted = itemConstraints ? itemConstraints->GetConstraint<TSortedConstraintNode>() : nullptr) { + TKeySelectorBuilder builder(pos, ctx, useNativeDescSort, tupleItemType->Cast<TStructExprType>()); + builder.ProcessConstraint(*sorted); + builder.FillRowSpecSort(*outTable.RowSpec); + if (builder.NeedMap()) { + remapper = builder.MakeRemapLambda(true); + } + ordered = true; + } + if (remapper) { + if (ndx > 0 && switchArgs.empty()) { + for (size_t i = 0; i < ndx; ++i) { + switchArgs.push_back( + Build<TCoAtomList>(ctx, pos) + .Add() + .Value(i) + .Build() + .Done()); + switchArgs.push_back(Build<TCoLambda>(ctx, pos).Args({"stream"}).Body("stream").Done()); + } + } + switchArgs.push_back( + Build<TCoAtomList>(ctx, pos) + .Add() + .Value(ndx) + .Build() + .Done()); + switchArgs.push_back(TExprBase(remapper)); + } else if (!switchArgs.empty()) { + switchArgs.push_back( + Build<TCoAtomList>(ctx, pos) + .Add() + .Value(ndx) + .Build() + .Done()); + switchArgs.push_back(Build<TCoLambda>(ctx, pos).Args({"stream"}).Body("stream").Done()); + } + if (itemConstraints) + outTable.RowSpec->SetConstraints(*itemConstraints); + outTables.push_back(outTable + .SetUnique(itemConstraints ? itemConstraints->GetConstraint<TDistinctConstraintNode>() : nullptr, pos, ctx) + .ToExprNode(ctx, pos).Cast<TYtOutTable>() + ); + ++ndx; + } + if (!switchArgs.empty()) { + lambda = Build<TCoLambda>(ctx, pos) + .Args({"stream"}) + .Body<TCoSwitch>() + .Input<TExprApplier>() + .Apply(TCoLambda(lambda)) + .With(0, "stream") + .Build() + .BufferBytes() + .Value(state->Configuration->SwitchLimit.Get().GetOrElse(DEFAULT_SWITCH_MEMORY_LIMIT)) + .Build() + .FreeArgs() + .Add(switchArgs) + .Build() + .Build() + .Done().Ptr(); + } + return outTables; +} + +TYtOutTable ConvertSingleOutTableWithSortAware(TExprNode::TPtr& lambda, bool& ordered, TPositionHandle pos, + const TTypeAnnotationNode* outItemType, TExprContext& ctx, const TYtState::TPtr& state, const TConstraintSet& constraints) { + + const ui64 nativeTypeFlags = state->Configuration->UseNativeYtTypes.Get().GetOrElse(DEFAULT_USE_NATIVE_YT_TYPES) ? NTCF_ALL : NTCF_NONE; + const bool useNativeDescSort = state->Configuration->UseNativeDescSort.Get().GetOrElse(DEFAULT_USE_NATIVE_DESC_SORT); + const auto outStructType = outItemType->Cast<TStructExprType>(); + + ordered = false; + TYtOutTableInfo outTable(outStructType, nativeTypeFlags); + if (auto sorted = constraints.GetConstraint<TSortedConstraintNode>()) { + TKeySelectorBuilder builder(pos, ctx, useNativeDescSort, outStructType); + builder.ProcessConstraint(*sorted); + builder.FillRowSpecSort(*outTable.RowSpec); + + if (builder.NeedMap()) { + lambda = ctx.Builder(pos) + .Lambda() + .Param("stream") + .Apply(builder.MakeRemapLambda(true)) + .With(0) + .Apply(*lambda) + .With(0, "stream") + .Seal() + .Done() + .Seal() + .Seal() + .Build(); + } + ordered = true; + } + outTable.RowSpec->SetConstraints(constraints); + outTable.SetUnique(constraints.GetConstraint<TDistinctConstraintNode>(), pos, ctx); + return outTable.ToExprNode(ctx, pos).Cast<TYtOutTable>(); +} + +TVector<TYtOutTable> ConvertOutTablesWithSortAware(TExprNode::TPtr& lambda, bool& ordered, TPositionHandle pos, + const TTypeAnnotationNode* outItemType, TExprContext& ctx, const TYtState::TPtr& state, const TConstraintSet& constraints) { + TVector<TYtOutTable> outTables; + if (outItemType->GetKind() == ETypeAnnotationKind::Variant) { + return ConvertMultiOutTablesWithSortAware(lambda, ordered, pos, outItemType, ctx, state, constraints); + } + + return TVector<TYtOutTable>{ConvertSingleOutTableWithSortAware(lambda, ordered, pos, outItemType, ctx, state, constraints)}; +} + +} // namespace NYql diff --git a/yt/yql/providers/yt/provider/phy_opt/yql_yt_phy_opt_helper.h b/yt/yql/providers/yt/provider/phy_opt/yql_yt_phy_opt_helper.h new file mode 100644 index 0000000000..0598225ae2 --- /dev/null +++ b/yt/yql/providers/yt/provider/phy_opt/yql_yt_phy_opt_helper.h @@ -0,0 +1,116 @@ +#pragma once + +#include <yt/yql/providers/yt/provider/yql_yt_provider_impl.h> +#include <yt/yql/providers/yt/lib/key_filter/yql_key_filter.h> +#include <yt/yql/providers/yt/expr_nodes/yql_yt_expr_nodes.h> +#include <yql/essentials/core/expr_nodes/yql_expr_nodes.h> + +#include <util/generic/strbuf.h> +#include <util/generic/set.h> +#include <util/generic/map.h> +#include <util/generic/vector.h> +#include <util/generic/hash.h> +#include <util/generic/hash_set.h> + +namespace NYql::NPrivate { + +class TYtSortMembersCollection: public TSortMembersCollection { +public: + void BuildKeyFilters(TPositionHandle pos, size_t tableCount, size_t orGroupCount, TExprNode::TListType& result, TExprContext& ctx); + +private: + void BuildKeyFiltersImpl(TPositionHandle pos, size_t tableCount, size_t orGroup, TExprNode::TListType& prefix, + const TMemberDescrMap& members, TExprNode::TListType& result, TExprContext& ctx); +}; + +NNodes::TMaybeNode<NNodes::TCoLambda> GetLambdaWithPredicate(NNodes::TCoLambda lambda); + +bool IsAscending(const TExprNode& node); + +bool CollectSortSet(const TExprNode& sortNode, TSet<TVector<TStringBuf>>& sortSets); + +TExprNode::TPtr CollectPreferredSortsForEquiJoinOutput(NNodes::TExprBase join, const TExprNode::TPtr& options, + TExprContext& ctx, const TParentsMap& parentsMap); + +bool CanExtraColumnBePulledIntoEquiJoin(const TTypeAnnotationNode* type); + +bool IsYtOrPlainTablePropsDependent(NNodes::TExprBase input); + +bool IsLambdaSuitableForPullingIntoEquiJoin(const NNodes::TCoFlatMapBase& flatMap, const TExprNode& label, + const THashMap<TStringBuf, THashSet<TStringBuf>>& tableKeysMap, + const TExprNode* extractedMembers); + +TExprNode::TPtr BuildYtEquiJoinPremap(NNodes::TExprBase list, NNodes::TMaybeNode<NNodes::TCoLambda> premapLambda, TExprContext& ctx); + +// label -> pair(<asc sort keys>, <inputs matched by keys>) +THashMap<TStringBuf, std::pair<TVector<TStringBuf>, ui32>> CollectTableSortKeysUsage(const TYtState::TPtr& state, const NNodes::TCoEquiJoin& equiJoin); + +NNodes::TCoLambda FallbackLambdaInput(NNodes::TCoLambda lambda, TExprContext& ctx); + +NNodes::TCoLambda FallbackLambdaOutput(NNodes::TCoLambda lambda, TExprContext& ctx); + +NNodes::TYtDSink GetDataSink(NNodes::TExprBase input, TExprContext& ctx); + +NNodes::TExprBase GetWorld(NNodes::TExprBase input, NNodes::TMaybeNode<NNodes::TExprBase> main, TExprContext& ctx); + +struct TConvertInputOpts { + NNodes::TMaybeNode<NNodes::TCoNameValueTupleList> Settings_; + NNodes::TMaybeNode<NNodes::TCoAtomList> CustomFields_; + bool KeepDirecRead_; + bool MakeUnordered_; + bool ClearUnordered_; + + TConvertInputOpts(); + + TConvertInputOpts& Settings(const NNodes::TMaybeNode<NNodes::TCoNameValueTupleList>& settings); + + TConvertInputOpts& CustomFields(const NNodes::TMaybeNode<NNodes::TCoAtomList>& customFields); + + TConvertInputOpts& ExplicitFields(const TYqlRowSpecInfo& rowSpec, TPositionHandle pos, TExprContext& ctx); + + TConvertInputOpts& ExplicitFields(const TStructExprType& type, TPositionHandle pos, TExprContext& ctx); + + TConvertInputOpts& KeepDirecRead(bool keepDirecRead = true); + + TConvertInputOpts& MakeUnordered(bool makeUnordered = true); + + TConvertInputOpts& ClearUnordered(); +}; + +NNodes::TYtSectionList ConvertInputTable(NNodes::TExprBase input, TExprContext& ctx, const TConvertInputOpts& opts = {}); + +bool CollectMemberPaths(NNodes::TExprBase row, const TExprNode::TPtr& lookupItem, + TMap<TStringBuf, TVector<ui32>>& memberPaths, TVector<ui32>& currPath); + +bool CollectMemberPaths(NNodes::TExprBase row, const TExprNode::TPtr& lookupItem, + TMap<TStringBuf, TVector<ui32>>& memberPaths); + +bool CollectKeyPredicatesFromLookup(NNodes::TExprBase row, NNodes::TCoLookupBase lookup, TVector<TKeyFilterPredicates>& ranges, + size_t maxTables); + +bool CollectKeyPredicatesAnd(NNodes::TExprBase row, const std::vector<NNodes::TExprBase>& predicates, TVector<TKeyFilterPredicates>& ranges, size_t maxTables); + +bool CollectKeyPredicatesOr(NNodes::TExprBase row, const std::vector<NNodes::TExprBase>& predicates, TVector<TKeyFilterPredicates>& ranges, size_t maxTables); + +bool CollectKeyPredicates(NNodes::TExprBase row, NNodes::TExprBase predicate, TVector<TKeyFilterPredicates>& ranges, size_t maxTables); + +TVector<NNodes::TYtOutTable> ConvertMultiOutTables(TPositionHandle pos, const TTypeAnnotationNode* outItemType, TExprContext& ctx, + const TYtState::TPtr& state, const TMultiConstraintNode* multi = nullptr); + +TVector<NNodes::TYtOutTable> ConvertOutTables(TPositionHandle pos, const TTypeAnnotationNode* outItemType, TExprContext& ctx, + const TYtState::TPtr& state, const TConstraintSet* constraint = nullptr); + +TVector<NNodes::TYtOutTable> ConvertMultiOutTablesWithSortAware(TExprNode::TPtr& lambda, bool& ordered, TPositionHandle pos, + const TTypeAnnotationNode* outItemType, TExprContext& ctx, const TYtState::TPtr& state, const TConstraintSet& constraints); + +NNodes::TYtOutTable ConvertSingleOutTableWithSortAware(TExprNode::TPtr& lambda, bool& ordered, TPositionHandle pos, + const TTypeAnnotationNode* outItemType, TExprContext& ctx, const TYtState::TPtr& state, const TConstraintSet& constraints); + +TVector<NNodes::TYtOutTable> ConvertOutTablesWithSortAware(TExprNode::TPtr& lambda, bool& ordered, TPositionHandle pos, + const TTypeAnnotationNode* outItemType, TExprContext& ctx, const TYtState::TPtr& state, const TConstraintSet& constraints); + +NNodes::TExprBase WrapOp(NNodes::TYtOutputOpBase op, TExprContext& ctx); + +NNodes::TCoLambda MapEmbedInputFieldsFilter(NNodes::TCoLambda lambda, bool ordered, NNodes::TCoAtomList fields, TExprContext& ctx); + +} // namespace NYql::NPrivate diff --git a/yt/yql/providers/yt/provider/phy_opt/yql_yt_phy_opt_join.cpp b/yt/yql/providers/yt/provider/phy_opt/yql_yt_phy_opt_join.cpp new file mode 100644 index 0000000000..6623cd7b4f --- /dev/null +++ b/yt/yql/providers/yt/provider/phy_opt/yql_yt_phy_opt_join.cpp @@ -0,0 +1,377 @@ +#include "yql_yt_phy_opt.h" +#include "yql_yt_phy_opt_helper.h" + +#include <yt/yql/providers/yt/provider/yql_yt_helpers.h> +#include <yt/yql/providers/yt/provider/yql_yt_join_impl.h> +#include <yql/essentials/providers/common/codec/yql_codec_type_flags.h> + +#include <yql/essentials/core/yql_type_helpers.h> +#include <yql/essentials/core/yql_opt_utils.h> +#include <yql/essentials/providers/common/provider/yql_provider.h> + +#include <yql/essentials/utils/log/log.h> + +namespace NYql { + +using namespace NNodes; +using namespace NPrivate; + +TMaybeNode<TExprBase> TYtPhysicalOptProposalTransformer::EquiJoin(TExprBase node, TExprContext& ctx, const TGetParents& getParents) const { + if (State_->Types->EvaluationInProgress || State_->PassiveExecution) { + return node; + } + + auto equiJoin = node.Cast<TCoEquiJoin>(); + + TMaybeNode<TYtDSink> dataSink; + TString usedCluster; + for (size_t i = 0; i + 2 < equiJoin.ArgCount(); ++i) { + auto list = equiJoin.Arg(i).Cast<TCoEquiJoinInput>().List(); + if (auto maybeExtractMembers = list.Maybe<TCoExtractMembers>()) { + list = maybeExtractMembers.Cast().Input(); + } + if (auto maybeFlatMap = list.Maybe<TCoFlatMapBase>()) { + TSyncMap syncList; + if (!IsYtCompleteIsolatedLambda(maybeFlatMap.Cast().Lambda().Ref(), syncList, usedCluster, false)) { + return node; + } + list = maybeFlatMap.Cast().Input(); + } + if (!IsYtProviderInput(list)) { + TSyncMap syncList; + if (!IsYtCompleteIsolatedLambda(list.Ref(), syncList, usedCluster, false)) { + return node; + } + continue; + } + + if (!dataSink) { + dataSink = GetDataSink(list, ctx); + } + auto cluster = ToString(GetClusterName(list)); + if (!UpdateUsedCluster(usedCluster, cluster)) { + return node; + } + } + + if (!dataSink) { + return node; + } + + THashMap<TStringBuf, std::pair<TVector<TStringBuf>, ui32>> tableSortKeysUsage = + CollectTableSortKeysUsage(State_, equiJoin); + + // label -> join keys + THashMap<TStringBuf, THashSet<TStringBuf>> tableKeysMap = + CollectEquiJoinKeyColumnsByLabel(equiJoin.Arg(equiJoin.ArgCount() - 2).Ref()); + + TNodeOnNodeOwnedMap updatedInputs; + TExprNode::TListType sections; + TExprNode::TListType premaps; + TSyncMap worldList; + for (size_t i = 0; i + 2 < equiJoin.ArgCount(); ++i) { + auto joinInput = equiJoin.Arg(i).Cast<TCoEquiJoinInput>(); + auto list = joinInput.List(); + + TMaybeNode<TCoLambda> premapLambda; + TExprNode::TPtr extractedMembers; + + auto listStepForward = list; + if (auto maybeExtractMembers = listStepForward.Maybe<TCoExtractMembers>()) { + extractedMembers = maybeExtractMembers.Cast().Members().Ptr(); + listStepForward = maybeExtractMembers.Cast().Input(); + } + + if (auto maybeFlatMap = listStepForward.Maybe<TCoFlatMapBase>()) { + auto flatMap = maybeFlatMap.Cast(); + if (IsLambdaSuitableForPullingIntoEquiJoin(flatMap, joinInput.Scope().Ref(), tableKeysMap, extractedMembers.Get())) { + if (!IsYtCompleteIsolatedLambda(flatMap.Lambda().Ref(), worldList, usedCluster, false)) { + return node; + } + + auto maybeLambda = CleanupWorld(flatMap.Lambda(), ctx); + if (!maybeLambda) { + return {}; + } + + auto lambda = ctx.DeepCopyLambda(maybeLambda.Cast().Ref()); + if (!extractedMembers) { + premapLambda = lambda; + YQL_CLOG(INFO, ProviderYt) << __FUNCTION__ << ": Collected input #" << i << " as premap"; + } else { + premapLambda = ctx.Builder(lambda->Pos()) + .Lambda() + .Param("item") + .Callable("ExtractMembers") + .Apply(0, lambda) + .With(0, "item") + .Seal() + .Add(1, extractedMembers) + .Seal() + .Seal() + .Build(); + YQL_CLOG(INFO, ProviderYt) << __FUNCTION__ << ": Collected input #" << i << " as premap with extract members"; + } + + list = flatMap.Input(); + } + } + + TExprNode::TPtr section; + if (!IsYtProviderInput(list)) { + auto& newSection = updatedInputs[list.Raw()]; + if (newSection) { + section = Build<TYtSection>(ctx, list.Pos()) + .InitFrom(TYtSection(newSection)) + .Settings() + .Add() + .Name() + .Value(ToString(EYtSettingType::JoinLabel)) + .Build() + .Value(joinInput.Scope()) + .Build() + .Build() + .Done().Ptr(); + } + else { + TSyncMap syncList; + if (!IsYtCompleteIsolatedLambda(list.Ref(), syncList, usedCluster, false)) { + return node; + } + + const TStructExprType* outItemType = nullptr; + if (auto type = GetSequenceItemType(list, false, ctx)) { + if (!EnsurePersistableType(list.Pos(), *type, ctx)) { + return {}; + } + outItemType = type->Cast<TStructExprType>(); + } else { + return {}; + } + + TYtOutTableInfo outTable(outItemType, State_->Configuration->UseNativeYtTypes.Get().GetOrElse(DEFAULT_USE_NATIVE_YT_TYPES) ? NTCF_ALL : NTCF_NONE); + outTable.RowSpec->SetConstraints(list.Ref().GetConstraintSet()); + outTable.SetUnique(list.Ref().GetConstraint<TDistinctConstraintNode>(), list.Pos(), ctx); + + auto cleanup = CleanupWorld(list, ctx); + if (!cleanup) { + return {}; + } + + section = newSection = Build<TYtSection>(ctx, list.Pos()) + .Paths() + .Add() + .Table<TYtOutput>() + .Operation<TYtFill>() + .World(ApplySyncListToWorld(ctx.NewWorld(list.Pos()), syncList, ctx)) + .DataSink(dataSink.Cast()) + .Content(MakeJobLambdaNoArg(cleanup.Cast(), ctx)) + .Output() + .Add(outTable.ToExprNode(ctx, list.Pos()).Cast<TYtOutTable>()) + .Build() + .Settings(GetFlowSettings(list.Pos(), *State_, ctx)) + .Build() + .OutIndex().Value(0U).Build() + .Build() + .Columns<TCoVoid>().Build() + .Ranges<TCoVoid>().Build() + .Stat<TCoVoid>().Build() + .Build() + .Build() + .Settings() + .Add() + .Name() + .Value(ToString(EYtSettingType::JoinLabel)) + .Build() + .Value(joinInput.Scope()) + .Build() + .Build() + .Done().Ptr(); + } + } + else { + auto settings = Build<TCoNameValueTupleList>(ctx, list.Pos()) + .Add() + .Name() + .Value(ToString(EYtSettingType::JoinLabel)) + .Build() + .Value(joinInput.Scope()) + .Build() + .Done(); + + TExprNode::TPtr world; + if (auto maybeRead = list.Maybe<TCoRight>().Input().Maybe<TYtReadTable>()) { + auto read = maybeRead.Cast(); + if (read.World().Ref().Type() != TExprNode::World) { + world = read.World().Ptr(); + } + } + + bool makeUnordered = false; + if (ctx.IsConstraintEnabled<TSortedConstraintNode>() && joinInput.Scope().Ref().IsAtom()) { + makeUnordered = (0 == tableSortKeysUsage[joinInput.Scope().Ref().Content()].second); + } + + auto sectionList = ConvertInputTable(list, ctx, TConvertInputOpts().Settings(settings).MakeUnordered(makeUnordered)); + YQL_ENSURE(sectionList.Size() == 1, "EquiJoin input should contain exactly one section, but has: " << sectionList.Size()); + bool clearWorld = false; + auto sectionNode = sectionList.Item(0); + + if (NYql::HasSetting(sectionNode.Settings().Ref(), EYtSettingType::Sample)) { + auto scheme = list.Ref().GetTypeAnn()->Cast<TListExprType>()->GetItemType(); + + auto path = CopyOrTrivialMap(sectionNode.Pos(), + TExprBase(world ? world : ctx.NewWorld(sectionNode.Pos())), + dataSink.Cast(), + *scheme, + Build<TYtSection>(ctx, sectionNode.Pos()) + .InitFrom(sectionNode) + .Settings(NYql::RemoveSettings(sectionNode.Settings().Ref(), EYtSettingType::StatColumns + | EYtSettingType::JoinLabel | EYtSettingType::Unordered, ctx)) + .Done(), + {}, ctx, State_, + TCopyOrTrivialMapOpts().SetTryKeepSortness(!makeUnordered).SetSectionUniq(list.Ref().GetConstraint<TDistinctConstraintNode>())); + + clearWorld = true; + + sectionNode = Build<TYtSection>(ctx, sectionNode.Pos()) + .Paths() + .Add(path) + .Build() + .Settings(NYql::RemoveSettings(sectionNode.Settings().Ref(), EYtSettingType::Take | EYtSettingType::Skip | + EYtSettingType::Sample, ctx)) + .Done(); + } + + if (!clearWorld && world) { + worldList.emplace(world, worldList.size()); + } + + section = sectionNode.Ptr(); + } + + YQL_ENSURE(section); + sections.push_back(section); + auto premap = BuildYtEquiJoinPremap(list, premapLambda, ctx); + if (!premap) { + return {}; + } + premaps.push_back(premap); + } + + const TStructExprType* outItemType = nullptr; + if (auto type = GetSequenceItemType(node, false, ctx)) { + if (!EnsurePersistableType(node.Pos(), *type, ctx)) { + return {}; + } + outItemType = type->Cast<TStructExprType>(); + } else { + return {}; + } + + auto parentsMap = getParents(); + YQL_ENSURE(parentsMap); + auto joinOptions = CollectPreferredSortsForEquiJoinOutput(node, equiJoin.Arg(equiJoin.ArgCount() - 1).Ptr(), ctx, *parentsMap); + + const auto join = Build<TYtEquiJoin>(ctx, node.Pos()) + .World(ApplySyncListToWorld(ctx.NewWorld(node.Pos()), worldList, ctx)) + .DataSink(dataSink.Cast()) + .Input() + .Add(sections) + .Build() + .Output() + .Add(TYtOutTableInfo(outItemType, State_->Configuration->UseNativeYtTypes.Get().GetOrElse(DEFAULT_USE_NATIVE_YT_TYPES) ? NTCF_ALL : NTCF_NONE) + .ToExprNode(ctx, node.Pos()).Cast<TYtOutTable>()) + .Build() + .Settings() + .Build() + .Joins(equiJoin.Arg(equiJoin.ArgCount() - 2)) + .JoinOptions(joinOptions) + .Done(); + + auto children = join.Ref().ChildrenList(); + children.reserve(children.size() + premaps.size()); + std::move(premaps.begin(), premaps.end(), std::back_inserter(children)); + return Build<TYtOutput>(ctx, node.Pos()) + .Operation(ctx.ChangeChildren(join.Ref(), std::move(children))) + .OutIndex().Value(0U).Build() + .Done(); +} + +TMaybeNode<TExprBase> TYtPhysicalOptProposalTransformer::EarlyMergeJoin(TExprBase node, TExprContext& ctx) const { + if (State_->Configuration->JoinMergeTablesLimit.Get()) { + auto equiJoin = node.Cast<TYtEquiJoin>(); + const auto tree = ImportYtEquiJoin(equiJoin, ctx); + if (State_->Configuration->JoinMergeForce.Get() || tree->LinkSettings.ForceSortedMerge) { + const auto rewriteStatus = RewriteYtEquiJoinLeaves(equiJoin, *tree, State_, ctx); + switch (rewriteStatus.Level) { + case TStatus::Repeat: + return node; + case TStatus::Error: + return {}; + case TStatus::Ok: + break; + default: + YQL_ENSURE(false, "Unexpected rewrite status"); + } + return ExportYtEquiJoin(equiJoin, *tree, ctx, State_); + } + } + return node; +} + +TMaybeNode<TExprBase> TYtPhysicalOptProposalTransformer::RuntimeEquiJoin(TExprBase node, TExprContext& ctx) const { + auto equiJoin = node.Cast<TYtEquiJoin>(); + auto cluster = equiJoin.DataSink().Cluster().StringValue(); + + const bool tryReorder = State_->Types->CostBasedOptimizer != ECostBasedOptimizerType::Disable + && equiJoin.Input().Size() > 2 + && !HasSetting(equiJoin.JoinOptions().Ref(), "cbo_passed"); + + const bool waitAllInputs = State_->Configuration->JoinWaitAllInputs.Get().GetOrElse(false) || tryReorder; + + if (waitAllInputs) { + for (auto section: equiJoin.Input()) { + for (auto path: section.Paths()) { + TYtPathInfo pathInfo(path); + if (!pathInfo.Table->Stat) { + return node; + } + } + } + } + const auto tree = ImportYtEquiJoin(equiJoin, ctx); + + const TMaybe<ui64> maxChunkCountExtendedStats = State_->Configuration->ExtendedStatsMaxChunkCount.Get(); + + if (tryReorder && waitAllInputs && maxChunkCountExtendedStats) { + YQL_CLOG(INFO, ProviderYt) << "Collecting cbo stats for equiJoin"; + auto collectStatus = CollectCboStats(cluster, *tree, State_, ctx); + if (collectStatus == TStatus::Repeat) { + return ExportYtEquiJoin(equiJoin, *tree, ctx, State_); + } + } + if (tryReorder) { + const auto optimizedTree = OrderJoins(tree, State_, cluster, ctx); + if (optimizedTree != tree) { + return ExportYtEquiJoin(equiJoin, *optimizedTree, ctx, State_); + } + } + const auto rewriteStatus = RewriteYtEquiJoin(equiJoin, *tree, State_, ctx); + + switch (rewriteStatus.Level) { + case TStatus::Repeat: + YQL_ENSURE(!waitAllInputs); + return node; + case TStatus::Error: + return {}; + case TStatus::Ok: + break; + default: + YQL_ENSURE(false, "Unexpected rewrite status"); + } + + return ExportYtEquiJoin(equiJoin, *tree, ctx, State_); +} + +} // namespace NYql diff --git a/yt/yql/providers/yt/provider/phy_opt/yql_yt_phy_opt_key_range.cpp b/yt/yql/providers/yt/provider/phy_opt/yql_yt_phy_opt_key_range.cpp new file mode 100644 index 0000000000..b1ac01de04 --- /dev/null +++ b/yt/yql/providers/yt/provider/phy_opt/yql_yt_phy_opt_key_range.cpp @@ -0,0 +1,354 @@ +#include "yql_yt_phy_opt.h" +#include "yql_yt_phy_opt_helper.h" + +#include <yt/yql/providers/yt/provider/yql_yt_helpers.h> + +#include <yql/essentials/core/dq_expr_nodes/dq_expr_nodes.h> +#include <yql/essentials/utils/log/log.h> + +namespace NYql { + +using namespace NNodes; +using namespace NNodes::NDq; +using namespace NPrivate; + +// All keyFilter settings are combined by OR. +// keyFilter value := '(<memberItem>+) <optional tableIndex> +// <memberItem> := '(<memberName> '(<cmpItem>+)) +// <cmpItem> := '(<cmpOp> <value>) +TMaybeNode<TExprBase> TYtPhysicalOptProposalTransformer::ExtractKeyRangeLegacy(TExprBase node, TExprContext& ctx) const { + auto op = node.Cast<TYtTransientOpBase>(); + if (op.Input().Size() > 1) { + // Extract key ranges before horizontal joins + return node; + } + if (op.Maybe<TYtMapReduce>().Mapper().Maybe<TCoVoid>()) { + return node; + } + + auto section = op.Input().Item(0); + if (NYql::HasAnySetting(section.Settings().Ref(), EYtSettingType::KeyFilter | EYtSettingType::KeyFilter2 | EYtSettingType::Take | EYtSettingType::Skip)) { + return node; + } + + TYtSortMembersCollection sortMembers; + for (size_t tableIndex = 0; tableIndex < section.Paths().Size(); ++tableIndex) { + TYtPathInfo pathInfo(section.Paths().Item(tableIndex)); + if (pathInfo.Ranges) { + return node; + } + TYtTableBaseInfo::TPtr tableInfo = pathInfo.Table; + if (tableInfo->RowSpec && tableInfo->RowSpec->IsSorted() && !tableInfo->RowSpec->SortMembers.empty()) { + sortMembers.AddTableInfo(tableIndex, tableInfo->Name, + tableInfo->RowSpec->SortMembers, + tableInfo->RowSpec->SortedByTypes, + tableInfo->RowSpec->SortDirections); + } + } + if (sortMembers.Empty()) { + return node; + } + + TCoLambda mapper = op.Maybe<TYtMap>() ? op.Cast<TYtMap>().Mapper() : op.Cast<TYtMapReduce>().Mapper().Cast<TCoLambda>(); + auto maybeLambda = GetFlatMapOverInputStream(mapper).Lambda(); + if (!maybeLambda) { + return node; + } + TCoLambda lambda = maybeLambda.Cast(); + if (auto innerFlatMap = lambda.Body().Maybe<TCoFlatMapBase>()) { + if (auto arg = innerFlatMap.Input().Maybe<TCoFilterNullMembersBase>().Input().Maybe<TCoJust>().Input()) { + if (arg.Cast().Raw() == lambda.Args().Arg(0).Raw()) { + lambda = innerFlatMap.Lambda().Cast(); + } + } + } + if (!lambda.Body().Maybe<TCoConditionalValueBase>()) { + return node; + } + auto predicate = lambda.Body().Cast<TCoConditionalValueBase>().Predicate(); + if (predicate.Ref().Type() != TExprNode::Callable) { + return node; + } + + const size_t maxTables = State_->Configuration->MaxInputTables.Get().GetOrElse(DEFAULT_MAX_INPUT_TABLES); + TVector<TKeyFilterPredicates> ranges; + if (!CollectKeyPredicates(lambda.Args().Arg(0), predicate, ranges, maxTables)) { + return node; + } + + if (ranges.size() > maxTables) { + YQL_CLOG(DEBUG, ProviderYt) << __FUNCTION__ << ": too many tables - " << ranges.size(); + return node; + } + + if (!sortMembers.ApplyRanges(ranges, ctx)) { + return {}; + } + + if (sortMembers.Empty()) { + return node; + } + + auto newSettingsChildren = section.Settings().Ref().ChildrenList(); + sortMembers.BuildKeyFilters(predicate.Pos(), section.Paths().Size(), ranges.size(), newSettingsChildren, ctx); + + auto newSection = ctx.ChangeChild(section.Ref(), TYtSection::idx_Settings, + ctx.NewList(section.Settings().Pos(), std::move(newSettingsChildren))); + return ctx.ChangeChild(op.Ref(), TYtTransientOpBase::idx_Input, + Build<TYtSectionList>(ctx, op.Input().Pos()) + .Add(newSection) + .Done().Ptr() + ); +} + +TMaybe<TVector<TYtPhysicalOptProposalTransformer::TRangeBuildResult>> +TYtPhysicalOptProposalTransformer::ExtractKeyRangeFromLambda(TCoLambda lambda, TYtSection section, TExprContext& ctx) const { + YQL_ENSURE(lambda.Body().Maybe<TCoConditionalValueBase>().IsValid()); + + TMap<TVector<TString>, TSet<size_t>> tableIndexesBySortKey; + TMap<size_t, TString> tableNamesByIndex; + for (size_t tableIndex = 0; tableIndex < section.Paths().Size(); ++tableIndex) { + TYtPathInfo pathInfo(section.Paths().Item(tableIndex)); + if (pathInfo.Ranges) { + return TVector<TRangeBuildResult>{}; + } + TYtTableBaseInfo::TPtr tableInfo = pathInfo.Table; + if (tableInfo->RowSpec) { + auto rowSpec = tableInfo->RowSpec; + if (rowSpec->IsSorted()) { + TVector<TString> keyPrefix; + for (size_t i = 0; i < rowSpec->SortedBy.size(); ++i) { + if (!rowSpec->GetType()->FindItem(rowSpec->SortedBy[i])) { + break; + } + YQL_ENSURE(i < rowSpec->SortDirections.size()); + + if (!rowSpec->SortDirections[i]) { + // TODO: allow native descending YT sort if UseYtKeyBounds is enabled + break; + } + keyPrefix.push_back(rowSpec->SortedBy[i]); + } + if (!keyPrefix.empty()) { + tableIndexesBySortKey[keyPrefix].insert(tableIndex); + } + tableNamesByIndex[tableIndex] = tableInfo->Name; + } + } + } + if (tableIndexesBySortKey.empty()) { + return TVector<TRangeBuildResult>{}; + } + + TPredicateExtractorSettings rangeSettings; + rangeSettings.MergeAdjacentPointRanges = State_->Configuration->MergeAdjacentPointRanges.Get().GetOrElse(DEFAULT_MERGE_ADJACENT_POINT_RANGES); + rangeSettings.HaveNextValueCallable = State_->Configuration->KeyFilterForStartsWith.Get().GetOrElse(DEFAULT_KEY_FILTER_FOR_STARTS_WITH); + rangeSettings.MaxRanges = State_->Configuration->MaxKeyRangeCount.Get().GetOrElse(DEFAULT_MAX_KEY_RANGE_COUNT); + + THashSet<TString> possibleIndexKeys; + auto rowType = section.Ref().GetTypeAnn()->Cast<TListExprType>()->GetItemType(); + auto extractor = MakePredicateRangeExtractor(rangeSettings); + if (!extractor->Prepare(lambda.Ptr(), *rowType, possibleIndexKeys, ctx, *State_->Types)) { + return Nothing(); + } + + TVector<TRangeBuildResult> results; + for (auto& [keys, tableIndexes] : tableIndexesBySortKey) { + if (AllOf(keys, [&possibleIndexKeys](const TString& key) { return !possibleIndexKeys.contains(key); })) { + continue; + } + + TRangeBuildResult result; + result.Keys = keys; + result.TableIndexes = tableIndexes; + result.BuildResult = extractor->BuildComputeNode(keys, ctx, *State_->Types); + auto& compute = result.BuildResult.ComputeNode; + if (compute) { + compute = ctx.NewCallable(compute->Pos(), "EvaluateExprIfPure", { compute }); + results.push_back(result); + + TVector<TString> tableNames; + for (const auto& idx : tableIndexes) { + YQL_ENSURE(tableNamesByIndex.contains(idx)); + tableNames.push_back(tableNamesByIndex[idx]); + } + + YQL_CLOG(INFO, ProviderYt) << __FUNCTION__ + << ": Will use key filter for tables [" << JoinSeq(",", tableNames) << "] with key columns [" + << JoinSeq(",", keys) << "]"; + } + } + + return results; +} + +TExprNode::TPtr TYtPhysicalOptProposalTransformer::UpdateSectionWithKeyRanges(TPositionHandle pos, TYtSection section, const TVector<TRangeBuildResult>& results, TExprContext& ctx) const { + const bool sameSort = results.size() == 1 && results.front().TableIndexes.size() == section.Paths().Size(); + + auto newSettingsChildren = section.Settings().Ref().ChildrenList(); + + TExprNode::TListType updatedPaths = section.Paths().Ref().ChildrenList(); + bool hasPathUpdates = false; + for (auto& result : results) { + TExprNodeList items = { result.BuildResult.ComputeNode }; + + TExprNodeList usedKeys; + YQL_ENSURE(result.BuildResult.UsedPrefixLen <= result.Keys.size()); + for (size_t i = 0; i < result.BuildResult.UsedPrefixLen; ++i) { + usedKeys.push_back(ctx.NewAtom(pos, result.Keys[i])); + } + auto usedKeysNode = ctx.NewList(pos, std::move(usedKeys)); + auto usedKeysSetting = ctx.NewList(pos, { ctx.NewAtom(pos, "usedKeys"), usedKeysNode }); + items.push_back(ctx.NewList(pos, { usedKeysSetting })); + + for (const auto& idx : result.TableIndexes) { + if (auto out = TYtPath(updatedPaths[idx]).Table().Maybe<TYtOutput>(); out && IsUnorderedOutput(out.Cast())) { + updatedPaths[idx] = Build<TYtPath>(ctx, updatedPaths[idx]->Pos()) + .InitFrom(TYtPath(updatedPaths[idx])) + .Table<TYtOutput>() + .InitFrom(out.Cast()) + .Mode(TMaybeNode<TCoAtom>()) + .Build() + .Done().Ptr(); + hasPathUpdates = true; + } + } + + if (!sameSort) { + TExprNodeList idxs; + for (const auto& idx : result.TableIndexes) { + idxs.push_back(ctx.NewAtom(pos, idx)); + } + items.push_back(ctx.NewList(pos, std::move(idxs))); + } + newSettingsChildren.push_back( + Build<TCoNameValueTuple>(ctx, pos) + .Name() + .Value(ToString(EYtSettingType::KeyFilter2)) + .Build() + .Value(ctx.NewList(pos, std::move(items))) + .Done().Ptr()); + } + + auto newSection = ctx.ChangeChild(section.Ref(), TYtSection::idx_Settings, + ctx.NewList(section.Settings().Pos(), std::move(newSettingsChildren))); + if (hasPathUpdates) { + newSection = ctx.ChangeChild(*newSection, TYtSection::idx_Paths, + ctx.NewList(section.Paths().Pos(), std::move(updatedPaths))); + } + return newSection; +} + +TMaybeNode<TExprBase> TYtPhysicalOptProposalTransformer::ExtractKeyRangeDqReadWrap(TExprBase node, TExprContext& ctx) const { + auto flatMap = node.Cast<TCoFlatMapBase>(); + auto maybeYtRead = flatMap.Input().Maybe<TDqReadWrapBase>().Input().Maybe<TYtReadTable>(); + if (!maybeYtRead) { + return node; + } + auto ytRead = maybeYtRead.Cast(); + if (ytRead.Input().Size() > 1) { + return node; + } + + TYtDSource dataSource = GetDataSource(ytRead, ctx); + if (!State_->Configuration->_EnableYtPartitioning.Get(dataSource.Cluster().StringValue()).GetOrElse(false)) { + return node; + } + + auto section = ytRead.Input().Item(0); + if (NYql::HasAnySetting(section.Settings().Ref(), EYtSettingType::KeyFilter | EYtSettingType::KeyFilter2 | EYtSettingType::Take | EYtSettingType::Skip)) { + return node; + } + + auto maybeLambda = GetLambdaWithPredicate(flatMap.Lambda()); + if (!maybeLambda) { + return node; + } + auto lambda = maybeLambda.Cast(); + auto maybeResult = ExtractKeyRangeFromLambda(lambda, section, ctx); + if (!maybeResult) { + return {}; + } + auto results = *maybeResult; + if (results.empty()) { + return node; + } + auto predPos = lambda.Body().Cast<TCoConditionalValueBase>().Predicate().Pos(); + auto newSection = UpdateSectionWithKeyRanges(predPos, section, results, ctx); + + auto newYtRead = Build<TYtReadTable>(ctx, ytRead.Pos()) + .InitFrom(ytRead) + .Input() + .Add(newSection) + .Build() + .Done().Ptr(); + + auto newFlatMap = ctx.ChangeChild(flatMap.Ref(), TCoFlatMapBase::idx_Input, + ctx.ChangeChild(flatMap.Input().Ref(), TDqReadWrapBase::idx_Input, std::move(newYtRead)) + ); + + const bool sameSort = results.size() == 1 && results.front().TableIndexes.size() == section.Paths().Size(); + const bool pruneLambda = State_->Configuration->DqPruneKeyFilterLambda.Get().GetOrElse(DEFAULT_DQ_PRUNE_KEY_FILTER_LAMBDA); + if (sameSort && pruneLambda) { + YQL_CLOG(INFO, ProviderYt) << __FUNCTION__ << ": Will prune key filter lambda"; + newFlatMap = ctx.ReplaceNodes(std::move(newFlatMap), {{ lambda.Raw(), results.front().BuildResult.PrunedLambda }}); + } + + return newFlatMap; +} + +TMaybeNode<TExprBase> TYtPhysicalOptProposalTransformer::ExtractKeyRange(TExprBase node, TExprContext& ctx) const { + auto op = node.Cast<TYtTransientOpBase>(); + if (op.Input().Size() > 1) { + // Extract key ranges before horizontal joins + return node; + } + if (op.Maybe<TYtMapReduce>().Mapper().Maybe<TCoVoid>()) { + return node; + } + + auto section = op.Input().Item(0); + if (NYql::HasAnySetting(section.Settings().Ref(), EYtSettingType::KeyFilter | EYtSettingType::KeyFilter2 | EYtSettingType::Take | EYtSettingType::Skip)) { + return node; + } + + TCoLambda mapper = op.Maybe<TYtMap>() ? op.Cast<TYtMap>().Mapper() : op.Cast<TYtMapReduce>().Mapper().Cast<TCoLambda>(); + auto maybeLambda = GetFlatMapOverInputStream(mapper).Lambda(); + if (!maybeLambda) { + return node; + } + maybeLambda = GetLambdaWithPredicate(maybeLambda.Cast()); + if (!maybeLambda) { + return node; + } + auto lambda = maybeLambda.Cast(); + auto maybeResult = ExtractKeyRangeFromLambda(lambda, section, ctx); + if (!maybeResult) { + return {}; + } + auto results = *maybeResult; + if (results.empty()) { + return node; + } + + auto predPos = lambda.Body().Cast<TCoConditionalValueBase>().Predicate().Pos(); + auto newSection = UpdateSectionWithKeyRanges(predPos, section, results, ctx); + + auto newOp = ctx.ChangeChild(op.Ref(), + TYtTransientOpBase::idx_Input, + Build<TYtSectionList>(ctx, op.Input().Pos()) + .Add(newSection) + .Done().Ptr() + ); + + const bool sameSort = results.size() == 1 && results.front().TableIndexes.size() == section.Paths().Size(); + const bool pruneLambda = State_->Configuration->PruneKeyFilterLambda.Get().GetOrElse(DEFAULT_PRUNE_KEY_FILTER_LAMBDA); + if (sameSort && pruneLambda) { + YQL_CLOG(INFO, ProviderYt) << __FUNCTION__ << ": Will prune key filter lambda"; + newOp = ctx.ReplaceNodes(std::move(newOp), {{ lambda.Raw(), results.front().BuildResult.PrunedLambda }}); + } + + return newOp; +} + +} // namespace NYql diff --git a/yt/yql/providers/yt/provider/phy_opt/yql_yt_phy_opt_lambda.cpp b/yt/yql/providers/yt/provider/phy_opt/yql_yt_phy_opt_lambda.cpp new file mode 100644 index 0000000000..b3d67d7005 --- /dev/null +++ b/yt/yql/providers/yt/provider/phy_opt/yql_yt_phy_opt_lambda.cpp @@ -0,0 +1,95 @@ +#include "yql_yt_phy_opt.h" + +namespace NYql { + +using namespace NNodes; + +TCoLambda TYtPhysicalOptProposalTransformer::MakeJobLambdaNoArg(TExprBase content, TExprContext& ctx) const { + if (State_->Configuration->UseFlow.Get().GetOrElse(DEFAULT_USE_FLOW)) { + content = Build<TCoToFlow>(ctx, content.Pos()) + .Input(content) + .FreeArgs() + .Add<TCoDependsOn>() + .Input<TCoString>() + .Literal().Build(State_->FlowDependsOnId++) + .Build() + .Build() + .Build() + .Done(); + } else { + content = Build<TCoToStream>(ctx, content.Pos()).Input(content).Done(); + } + + return Build<TCoLambda>(ctx, content.Pos()).Args({}).Body(content).Done(); +} + + +template<> +TCoLambda TYtPhysicalOptProposalTransformer::MakeJobLambda<false>(TCoLambda lambda, bool useFlow, TExprContext& ctx) const +{ + if (useFlow) { + return Build<TCoLambda>(ctx, lambda.Pos()) + .Args({"flow"}) + .Body<TCoToFlow>() + .Input<TExprApplier>() + .Apply(lambda) + .With<TCoFromFlow>(0) + .Input("flow") + .Build() + .Build() + .FreeArgs() + .Add<TCoDependsOn>() + .Input("flow") + .Build() + .Build() + .Build() + .Done(); + } else { + return Build<TCoLambda>(ctx, lambda.Pos()) + .Args({"stream"}) + .Body<TCoToStream>() + .Input<TExprApplier>() + .Apply(lambda) + .With(0, "stream") + .Build() + .Build() + .Done(); + } +} + +template<> +TCoLambda TYtPhysicalOptProposalTransformer::MakeJobLambda<true>(TCoLambda lambda, bool useFlow, TExprContext& ctx) const +{ + if (useFlow) { + return Build<TCoLambda>(ctx, lambda.Pos()) + .Args({"flow"}) + .Body<TCoToFlow>() + .Input<TExprApplier>() + .Apply(lambda) + .With<TCoForwardList>(0) + .Stream("flow") + .Build() + .Build() + .FreeArgs() + .Add<TCoDependsOn>() + .Input("flow") + .Build() + .Build() + .Build() + .Done(); + } else { + return Build<TCoLambda>(ctx, lambda.Pos()) + .Args({"stream"}) + .Body<TCoToStream>() + .Input<TExprApplier>() + .Apply(lambda) + .With<TCoForwardList>(0) + .Stream("stream") + .Build() + .Build() + .Build() + .Done(); + } +} + +} // namespace NYql diff --git a/yt/yql/providers/yt/provider/phy_opt/yql_yt_phy_opt_map.cpp b/yt/yql/providers/yt/provider/phy_opt/yql_yt_phy_opt_map.cpp new file mode 100644 index 0000000000..5032e5a0bc --- /dev/null +++ b/yt/yql/providers/yt/provider/phy_opt/yql_yt_phy_opt_map.cpp @@ -0,0 +1,394 @@ +#include "yql_yt_phy_opt.h" +#include "yql_yt_phy_opt_helper.h" + +#include <yt/yql/providers/yt/provider/yql_yt_helpers.h> +#include <yql/essentials/providers/common/codec/yql_codec_type_flags.h> + +#include <yql/essentials/core/yql_type_helpers.h> +#include <yql/essentials/core/yql_opt_utils.h> +#include <yql/essentials/core/yql_join.h> + +#include <yql/essentials/utils/log/log.h> + +namespace NYql { + +using namespace NNodes; +using namespace NPrivate; + +bool TYtPhysicalOptProposalTransformer::CanBePulledIntoParentEquiJoin(const TCoFlatMapBase& flatMap, const TGetParents& getParents) { + const TParentsMap* parents = getParents(); + YQL_ENSURE(parents); + + auto equiJoinParents = CollectEquiJoinOnlyParents(flatMap, *parents); + if (equiJoinParents.empty()) { + return false; + } + + bool suitable = true; + for (auto it = equiJoinParents.begin(); it != equiJoinParents.end() && suitable; ++it) { + TCoEquiJoin equiJoin(it->Node); + auto inputIndex = it->Index; + + auto equiJoinTree = equiJoin.Arg(equiJoin.ArgCount() - 2); + THashMap<TStringBuf, THashSet<TStringBuf>> tableKeysMap = + CollectEquiJoinKeyColumnsByLabel(equiJoinTree.Ref()); + + auto input = equiJoin.Arg(inputIndex).Cast<TCoEquiJoinInput>(); + + suitable = suitable && IsLambdaSuitableForPullingIntoEquiJoin(flatMap, input.Scope().Ref(), tableKeysMap, + it->ExtractedMembers); + } + + return suitable; +} + +TMaybeNode<TExprBase> TYtPhysicalOptProposalTransformer::FlatMap(TExprBase node, TExprContext& ctx, const TGetParents& getParents) const { + if (State_->Types->EvaluationInProgress || State_->PassiveExecution) { + return node; + } + + auto flatMap = node.Cast<TCoFlatMapBase>(); + + const auto disableOptimizers = State_->Configuration->DisableOptimizers.Get().GetOrElse(TSet<TString>()); + if (!disableOptimizers.contains("EquiJoinPremap") && CanBePulledIntoParentEquiJoin(flatMap, getParents)) { + YQL_CLOG(INFO, ProviderYt) << __FUNCTION__ << ": " << flatMap.Ref().Content() << " can be pulled into parent EquiJoin"; + return node; + } + + auto input = flatMap.Input(); + if (!IsYtProviderInput(input, true)) { + return node; + } + + auto cluster = TString{GetClusterName(input)}; + TSyncMap syncList; + if (!IsYtCompleteIsolatedLambda(flatMap.Lambda().Ref(), syncList, cluster, false)) { + return node; + } + + auto outItemType = SilentGetSequenceItemType(flatMap.Lambda().Body().Ref(), true); + if (!outItemType || !outItemType->IsPersistable()) { + return node; + } + + auto cleanup = CleanupWorld(flatMap.Lambda(), ctx); + if (!cleanup) { + return {}; + } + + auto mapper = ctx.Builder(node.Pos()) + .Lambda() + .Param("stream") + .Callable(flatMap.Ref().Content()) + .Arg(0, "stream") + .Lambda(1) + .Param("item") + .Apply(cleanup.Cast().Ptr()) + .With(0, "item") + .Seal() + .Seal() + .Seal() + .Seal() + .Build(); + + bool sortedOutput = false; + TVector<TYtOutTable> outTables = ConvertOutTablesWithSortAware(mapper, sortedOutput, flatMap.Pos(), + outItemType, ctx, State_, flatMap.Ref().GetConstraintSet()); + + auto settingsBuilder = Build<TCoNameValueTupleList>(ctx, flatMap.Pos()); + if (TCoOrderedFlatMap::Match(flatMap.Raw()) || sortedOutput) { + settingsBuilder + .Add() + .Name() + .Value(ToString(EYtSettingType::Ordered)) + .Build() + .Build(); + } + if (State_->Configuration->UseFlow.Get().GetOrElse(DEFAULT_USE_FLOW)) { + settingsBuilder + .Add() + .Name() + .Value(ToString(EYtSettingType::Flow)) + .Build() + .Build(); + } + + auto ytMap = Build<TYtMap>(ctx, node.Pos()) + .World(ApplySyncListToWorld(GetWorld(input, {}, ctx).Ptr(), syncList, ctx)) + .DataSink(GetDataSink(input, ctx)) + .Input(ConvertInputTable(input, ctx)) + .Output() + .Add(outTables) + .Build() + .Settings(settingsBuilder.Done()) + .Mapper(std::move(mapper)) + .Done(); + + return WrapOp(ytMap, ctx); +} + +template <typename TLMapType> +TMaybeNode<TExprBase> TYtPhysicalOptProposalTransformer::LMap(TExprBase node, TExprContext& ctx) const { + if (State_->Types->EvaluationInProgress || State_->PassiveExecution) { + return node; + } + + auto lmap = node.Cast<TLMapType>(); + + if (!IsYtProviderInput(lmap.Input(), true)) { + return node; + } + + const auto inItemType = GetSequenceItemType(lmap.Input(), true, ctx); + if (!inItemType) { + return {}; + } + const auto outItemType = SilentGetSequenceItemType(lmap.Lambda().Body().Ref(), true); + if (!outItemType || !outItemType->IsPersistable()) { + return node; + } + + auto cluster = TString{GetClusterName(lmap.Input())}; + TSyncMap syncList; + if (!IsYtCompleteIsolatedLambda(lmap.Lambda().Ref(), syncList, cluster, false)) { + return node; + } + + auto cleanup = CleanupWorld(lmap.Lambda(), ctx); + if (!cleanup) { + return {}; + } + + auto mapper = cleanup.Cast().Ptr(); + bool sortedOutput = false; + TVector<TYtOutTable> outTables = NPrivate::ConvertOutTablesWithSortAware(mapper, sortedOutput, lmap.Pos(), + outItemType, ctx, State_, lmap.Ref().GetConstraintSet()); + + const bool useFlow = State_->Configuration->UseFlow.Get().GetOrElse(DEFAULT_USE_FLOW); + + auto settingsBuilder = Build<TCoNameValueTupleList>(ctx, lmap.Pos()); + if (std::is_same<TLMapType, TCoOrderedLMap>::value) { + settingsBuilder + .Add() + .Name() + .Value(ToString(EYtSettingType::Ordered)) + .Build() + .Build(); + } + + if (useFlow) { + settingsBuilder + .Add() + .Name() + .Value(ToString(EYtSettingType::Flow)) + .Build() + .Build(); + } + + auto map = Build<TYtMap>(ctx, lmap.Pos()) + .World(ApplySyncListToWorld(NPrivate::GetWorld(lmap.Input(), {}, ctx).Ptr(), syncList, ctx)) + .DataSink(NPrivate::GetDataSink(lmap.Input(), ctx)) + .Input(NPrivate::ConvertInputTable(lmap.Input(), ctx)) + .Output() + .Add(outTables) + .Build() + .Settings(settingsBuilder.Done()) + .Mapper(MakeJobLambda<false>(TCoLambda(mapper), useFlow, ctx)) + .Done(); + + return NPrivate::WrapOp(map, ctx); +} + +template TMaybeNode<TExprBase> TYtPhysicalOptProposalTransformer::LMap<TCoLMap>(TExprBase node, TExprContext& ctx) const; +template TMaybeNode<TExprBase> TYtPhysicalOptProposalTransformer::LMap<TCoOrderedLMap>(TExprBase node, TExprContext& ctx) const; + + +TMaybeNode<TExprBase> TYtPhysicalOptProposalTransformer::CombineByKey(TExprBase node, TExprContext& ctx) const { + if (State_->Types->EvaluationInProgress || State_->PassiveExecution) { + return node; + } + + auto combineByKey = node.Cast<TCoCombineByKey>(); + + auto input = combineByKey.Input(); + if (!IsYtProviderInput(input)) { + return node; + } + + if (!GetSequenceItemType(input, false, ctx)) { + return {}; + } + + const TStructExprType* outItemType = nullptr; + if (auto type = SilentGetSequenceItemType(combineByKey.FinishHandlerLambda().Body().Ref(), false); type && type->IsPersistable()) { + outItemType = type->Cast<TStructExprType>(); + } else { + return node; + } + + auto cluster = TString{GetClusterName(input)}; + TSyncMap syncList; + if (!IsYtCompleteIsolatedLambda(combineByKey.PreMapLambda().Ref(), syncList, cluster, false)) { + return node; + } + if (!IsYtCompleteIsolatedLambda(combineByKey.KeySelectorLambda().Ref(), syncList, cluster, false)) { + return node; + } + if (!IsYtCompleteIsolatedLambda(combineByKey.InitHandlerLambda().Ref(), syncList, cluster, false)) { + return node; + } + if (!IsYtCompleteIsolatedLambda(combineByKey.UpdateHandlerLambda().Ref(), syncList, cluster, false)) { + return node; + } + if (!IsYtCompleteIsolatedLambda(combineByKey.FinishHandlerLambda().Ref(), syncList, cluster, false)) { + return node; + } + + auto preMapLambda = CleanupWorld(combineByKey.PreMapLambda(), ctx); + auto keySelectorLambda = CleanupWorld(combineByKey.KeySelectorLambda(), ctx); + auto initHandlerLambda = CleanupWorld(combineByKey.InitHandlerLambda(), ctx); + auto updateHandlerLambda = CleanupWorld(combineByKey.UpdateHandlerLambda(), ctx); + auto finishHandlerLambda = CleanupWorld(combineByKey.FinishHandlerLambda(), ctx); + if (!preMapLambda || !keySelectorLambda || !initHandlerLambda || !updateHandlerLambda || !finishHandlerLambda) { + return {}; + } + + auto lambdaBuilder = Build<TCoLambda>(ctx, combineByKey.Pos()); + TMaybe<TCoLambda> lambdaRet; + if (!IsDepended(keySelectorLambda.Cast().Body().Ref(), keySelectorLambda.Cast().Args().Arg(0).Ref())) { + lambdaBuilder + .Args({TStringBuf("stream")}) + .Body<TCoFlatMap>() + .Input<TCoCondense1>() + .Input<TCoFlatMap>() + .Input(TStringBuf("stream")) + .Lambda() + .Args({TStringBuf("item")}) + .Body<TExprApplier>() + .Apply(preMapLambda.Cast()) + .With(0, TStringBuf("item")) + .Build() + .Build() + .Build() + .InitHandler() + .Args({TStringBuf("item")}) + .Body<TExprApplier>() + .Apply(initHandlerLambda.Cast()) + .With(0, keySelectorLambda.Cast().Body()) + .With(1, TStringBuf("item")) + .Build() + .Build() + .SwitchHandler() + .Args({TStringBuf("item"), TStringBuf("state")}) + .Body<TCoBool>() + .Literal().Build("false") + .Build() + .Build() + .UpdateHandler() + .Args({TStringBuf("item"), TStringBuf("state")}) + .Body<TExprApplier>() + .Apply(updateHandlerLambda.Cast()) + .With(0, keySelectorLambda.Cast().Body()) + .With(1, TStringBuf("item")) + .With(2, TStringBuf("state")) + .Build() + .Build() + .Build() + .Lambda() + .Args({TStringBuf("state")}) + .Body<TExprApplier>() + .Apply(finishHandlerLambda.Cast()) + .With(0, keySelectorLambda.Cast().Body()) + .With(1, TStringBuf("state")) + .Build() + .Build() + .Build(); + + lambdaRet = lambdaBuilder.Done(); + } else { + lambdaBuilder + .Args({TStringBuf("stream")}) + .Body<TCoCombineCore>() + .Input<TCoFlatMap>() + .Input(TStringBuf("stream")) + .Lambda() + .Args({TStringBuf("item")}) + .Body<TExprApplier>() + .Apply(preMapLambda.Cast()) + .With(0, TStringBuf("item")) + .Build() + .Build() + .Build() + .KeyExtractor() + .Args({TStringBuf("item")}) + .Body<TExprApplier>() + .Apply(keySelectorLambda.Cast()) + .With(0, TStringBuf("item")) + .Build() + .Build() + .InitHandler() + .Args({TStringBuf("key"), TStringBuf("item")}) + .Body<TExprApplier>() + .Apply(initHandlerLambda.Cast()) + .With(0, TStringBuf("key")) + .With(1, TStringBuf("item")) + .Build() + .Build() + .UpdateHandler() + .Args({TStringBuf("key"), TStringBuf("item"), TStringBuf("state")}) + .Body<TExprApplier>() + .Apply(updateHandlerLambda.Cast()) + .With(0, TStringBuf("key")) + .With(1, TStringBuf("item")) + .With(2, TStringBuf("state")) + .Build() + .Build() + .FinishHandler() + .Args({TStringBuf("key"), TStringBuf("state")}) + .Body<TExprApplier>() + .Apply(finishHandlerLambda.Cast()) + .With(0, TStringBuf("key")) + .With(1, TStringBuf("state")) + .Build() + .Build() + .MemLimit() + .Value(ToString(State_->Configuration->CombineCoreLimit.Get().GetOrElse(0))) + .Build() + .Build(); + + lambdaRet = lambdaBuilder.Done(); + } + + if (HasContextFuncs(*lambdaRet->Ptr())) { + lambdaRet = Build<TCoLambda>(ctx, combineByKey.Pos()) + .Args({ TStringBuf("stream") }) + .Body<TCoWithContext>() + .Name() + .Value("Agg") + .Build() + .Input<TExprApplier>() + .Apply(*lambdaRet) + .With(0, TStringBuf("stream")) + .Build() + .Build() + .Done(); + } + + TYtOutTableInfo combineOut(outItemType, State_->Configuration->UseNativeYtTypes.Get().GetOrElse(DEFAULT_USE_NATIVE_YT_TYPES) ? NTCF_ALL : NTCF_NONE); + + return Build<TYtOutput>(ctx, combineByKey.Pos()) + .Operation<TYtMap>() + .World(ApplySyncListToWorld(GetWorld(input, {}, ctx).Ptr(), syncList, ctx)) + .DataSink(GetDataSink(input, ctx)) + .Input(ConvertInputTable(input, ctx)) + .Output() + .Add(combineOut.ToExprNode(ctx, combineByKey.Pos()).Cast<TYtOutTable>()) + .Build() + .Settings(GetFlowSettings(combineByKey.Pos(), *State_, ctx)) + .Mapper(*lambdaRet) + .Build() + .OutIndex().Value(0U).Build() + .Done(); +} + +} // namespace NYql diff --git a/yt/yql/providers/yt/provider/phy_opt/yql_yt_phy_opt_merge.cpp b/yt/yql/providers/yt/provider/phy_opt/yql_yt_phy_opt_merge.cpp new file mode 100644 index 0000000000..b42173cf62 --- /dev/null +++ b/yt/yql/providers/yt/provider/phy_opt/yql_yt_phy_opt_merge.cpp @@ -0,0 +1,527 @@ +#include "yql_yt_phy_opt.h" + +#include <yt/yql/providers/yt/provider/yql_yt_helpers.h> + +namespace NYql { + +using namespace NNodes; + +TMaybeNode<TExprBase> TYtPhysicalOptProposalTransformer::BypassMerge(TExprBase node, TExprContext& ctx) const { + if (node.Ref().HasResult()) { + return node; + } + + auto op = node.Cast<TYtTransientOpBase>(); + if (op.Maybe<TYtCopy>()) { + return node; + } + + if (op.Maybe<TYtWithUserJobsOpBase>()) { + size_t lambdaIdx = op.Maybe<TYtMapReduce>() + ? TYtMapReduce::idx_Mapper + : op.Maybe<TYtReduce>() + ? TYtReduce::idx_Reducer + : TYtMap::idx_Mapper; + + bool usesTableIndex = false; + VisitExpr(op.Ref().ChildPtr(lambdaIdx), [&usesTableIndex](const TExprNode::TPtr& n) { + if (TYtTableIndex::Match(n.Get())) { + usesTableIndex = true; + } else if (TYtOutput::Match(n.Get())) { + return false; + } + return !usesTableIndex; + }); + if (usesTableIndex) { + return node; + } + } + + auto maxTables = State_->Configuration->MaxInputTables.Get(); + auto maxSortedTables = State_->Configuration->MaxInputTablesForSortedMerge.Get(); + const bool opOrdered = NYql::HasSetting(op.Settings().Ref(), EYtSettingType::Ordered); + bool hasUpdates = false; + TVector<TExprBase> updatedSections; + TSyncMap syncList; + for (auto section: op.Input()) { + const EYtSettingType kfType = NYql::HasSetting(section.Settings().Ref(), EYtSettingType::KeyFilter2) ? + EYtSettingType::KeyFilter2 : EYtSettingType::KeyFilter; + const auto keyFiltersValues = NYql::GetAllSettingValues(section.Settings().Ref(), kfType); + const bool hasTableKeyFilters = AnyOf(keyFiltersValues, + [kfType](const TExprNode::TPtr& keyFilter) { + return keyFilter->ChildrenSize() >= GetMinChildrenForIndexedKeyFilter(kfType); + }); + const bool hasTakeSkip = NYql::HasAnySetting(section.Settings().Ref(), EYtSettingType::Take | EYtSettingType::Skip); + + bool hasPathUpdates = false; + TVector<TYtPath> updatedPaths; + size_t inputCount = section.Paths().Size(); + if (!hasTableKeyFilters) { + for (auto path: section.Paths()) { + updatedPaths.push_back(path); + + bool hasRanges = false; + if (!path.Ranges().Maybe<TCoVoid>()) { + bool pathLimits = false; + for (auto range: path.Ranges().Cast<TExprList>()) { + if (range.Maybe<TYtRow>() || range.Maybe<TYtRowRange>()) { + pathLimits = true; + break; + } + if (range.Maybe<TYtRangeItemBase>()) { + hasRanges = true; + } + } + if (pathLimits) { + continue; + } + } + auto maybeInnerMerge = path.Table().Maybe<TYtOutput>().Operation().Maybe<TYtMerge>(); + if (!maybeInnerMerge) { + continue; + } + auto innerMerge = maybeInnerMerge.Cast(); + + if (innerMerge.Ref().StartsExecution() || innerMerge.Ref().HasResult()) { + continue; + } + + if (NYql::HasSettingsExcept(innerMerge.Settings().Ref(), EYtSettingType::KeepSorted | EYtSettingType::Limit)) { + continue; + } + + if (auto limitSetting = NYql::GetSetting(innerMerge.Settings().Ref(), EYtSettingType::Limit)) { + if (limitSetting->ChildPtr(1)->ChildrenSize()) { + continue; + } + } + + auto innerMergeSection = innerMerge.Input().Item(0); + + bool hasIncompatibleSettings = false; + TExprNode::TListType innerMergeKeyFiltersValues; + for (auto& setting : innerMergeSection.Settings().Ref().Children()) { + const auto type = FromString<EYtSettingType>(setting->Child(0)->Content()); + if (setting->ChildrenSize() == 2 && (type == EYtSettingType::KeyFilter || type == EYtSettingType::KeyFilter2)) { + innerMergeKeyFiltersValues.push_back(setting->ChildPtr(1)); + } else { + hasIncompatibleSettings = true; + break; + } + } + + if (hasIncompatibleSettings) { + continue; + } + if (AnyOf(innerMergeKeyFiltersValues, [](const TExprNode::TPtr& keyFilter) { return keyFilter->ChildrenSize() > 0; })) { + continue; + } + + auto mergeOutRowSpec = TYqlRowSpecInfo(innerMerge.Output().Item(0).RowSpec()); + const bool sortedMerge = mergeOutRowSpec.IsSorted(); + if (hasTakeSkip && sortedMerge && NYql::HasSetting(innerMerge.Settings().Ref(), EYtSettingType::KeepSorted)) { + continue; + } + if ((hasTakeSkip || hasRanges) && AnyOf(innerMergeSection.Paths(), [](const auto& path) { return !path.Ranges().template Maybe<TCoVoid>(); })) { + continue; + } + + const bool unordered = IsUnorderedOutput(path.Table().Cast<TYtOutput>()); + if (innerMergeSection.Paths().Size() > 1) { + if (hasTakeSkip && sortedMerge) { + continue; + } + // Only YtMap can change semantic if substitute single sorted input by multiple sorted ones. + // Other operations (YtMerge, YtReduce, YtMapReduce, YtEquiJoin, YtSort) can be safely optimized. + // YtCopy cannot, but it is ignored early + if (op.Maybe<TYtMap>() && opOrdered && !unordered && sortedMerge) { + continue; + } + auto limit = maxTables; + if (maxSortedTables && (op.Maybe<TYtReduce>() || (op.Maybe<TYtMerge>() && TYqlRowSpecInfo(op.Output().Item(0).RowSpec()).IsSorted()))) { + limit = maxSortedTables; + } + if (limit && (inputCount - 1 + innerMergeSection.Paths().Size()) > *limit) { + continue; + } + + if (mergeOutRowSpec.GetAllConstraints(ctx).GetConstraint<TDistinctConstraintNode>() || mergeOutRowSpec.GetAllConstraints(ctx).GetConstraint<TUniqueConstraintNode>()) { + continue; + } + } + + hasPathUpdates = true; + updatedPaths.pop_back(); + TMaybeNode<TExprBase> columns; + if (!path.Columns().Maybe<TCoVoid>()) { + columns = path.Columns(); + } else if ((op.Maybe<TYtWithUserJobsOpBase>() || op.Maybe<TYtEquiJoin>()) && mergeOutRowSpec.HasAuxColumns()) { + TVector<TStringBuf> items; + for (auto item: mergeOutRowSpec.GetType()->GetItems()) { + items.push_back(item->GetName()); + } + columns = ToAtomList(items, op.Pos(), ctx); + } + + if (!columns.IsValid() && path.Ranges().Maybe<TCoVoid>() && !unordered) { + for (auto mergePath: innerMergeSection.Paths()) { + updatedPaths.push_back(mergePath); + } + } else { + for (auto mergePath: innerMergeSection.Paths()) { + auto builder = Build<TYtPath>(ctx, mergePath.Pos()).InitFrom(mergePath); + + if (columns) { + builder.Columns(columns.Cast()); + } + if (!path.Ranges().Maybe<TCoVoid>()) { + builder.Ranges(path.Ranges()); + } + + auto updatedPath = builder.Done(); + if (unordered) { + updatedPath = MakeUnorderedPath(updatedPath, false, ctx); + } + + updatedPaths.push_back(updatedPath); + } + } + + if (innerMerge.World().Ref().Type() != TExprNode::World) { + syncList.emplace(innerMerge.World().Ptr(), syncList.size()); + } + inputCount += innerMergeSection.Paths().Size() - 1; + } + } + if (hasPathUpdates) { + hasUpdates = true; + updatedSections.push_back( + Build<TYtSection>(ctx, section.Pos()) + .InitFrom(section) + .Paths() + .Add(updatedPaths) + .Build() + .Done()); + } else { + updatedSections.push_back(section); + } + } + if (!hasUpdates) { + return node; + } + + auto sectionList = Build<TYtSectionList>(ctx, op.Input().Pos()) + .Add(updatedSections) + .Done(); + + auto res = ctx.ChangeChild(node.Ref(), TYtTransientOpBase::idx_Input, sectionList.Ptr()); + if (!syncList.empty()) { + res = ctx.ChangeChild(*res, TYtTransientOpBase::idx_World, ApplySyncListToWorld(op.World().Ptr(), syncList, ctx)); + } + return TExprBase(res); +} + +TMaybeNode<TExprBase> TYtPhysicalOptProposalTransformer::BypassMergeBeforePublish(TExprBase node, TExprContext& ctx) const { + if (node.Ref().HasResult()) { + return node; + } + + auto publish = node.Cast<TYtPublish>(); + + auto cluster = publish.DataSink().Cluster().StringValue(); + auto path = publish.Publish().Name().StringValue(); + auto commitEpoch = TEpochInfo::Parse(publish.Publish().CommitEpoch().Ref()).GetOrElse(0); + + auto dstRowSpec = State_->TablesData->GetTable(cluster, path, commitEpoch).RowSpec; + + auto maxTables = dstRowSpec->IsSorted() ? State_->Configuration->MaxInputTablesForSortedMerge.Get() : State_->Configuration->MaxInputTables.Get(); + bool hasUpdates = false; + TVector<TYtOutput> updateInputs; + size_t inputCount = publish.Input().Size(); + for (auto out: publish.Input()) { + updateInputs.push_back(out); + if (auto maybeMerge = out.Operation().Maybe<TYtMerge>()) { + auto merge = maybeMerge.Cast(); + + if (!merge.World().Ref().IsWorld()) { + continue; + } + + if (merge.Ref().StartsExecution() || merge.Ref().HasResult()) { + continue; + } + + if (merge.Settings().Size() != 0) { + continue; + } + + auto mergeSection = merge.Input().Item(0); + if (NYql::HasSettingsExcept(mergeSection.Settings().Ref(), EYtSettingType::KeyFilter | EYtSettingType::KeyFilter2)) { + continue; + } + if (HasNonEmptyKeyFilter(mergeSection)) { + continue; + } + + if (maxTables && inputCount + mergeSection.Paths().Size() - 1 > *maxTables) { + continue; + } + + if (mergeSection.Paths().Size() < 2) { + continue; + } + + if (!AllOf(mergeSection.Paths(), [](TYtPath path) { + return path.Table().Maybe<TYtOutput>() + && path.Columns().Maybe<TCoVoid>() + && path.Ranges().Maybe<TCoVoid>() + && !TYtTableBaseInfo::GetMeta(path.Table())->IsDynamic; + })) { + continue; + } + + if (dstRowSpec->GetColumnOrder().Defined() && AnyOf(mergeSection.Paths(), [colOrder = *dstRowSpec->GetColumnOrder()](auto path) { + auto rowSpec = TYtTableBaseInfo::GetRowSpec(path.Table()); + return rowSpec->GetColumnOrder().Defined() && *rowSpec->GetColumnOrder() != colOrder; + })) { + continue; + } + + hasUpdates = true; + inputCount += mergeSection.Paths().Size() - 1; + updateInputs.pop_back(); + if (IsUnorderedOutput(out)) { + std::transform(mergeSection.Paths().begin(), mergeSection.Paths().end(), std::back_inserter(updateInputs), + [mode = out.Mode(), &ctx](TYtPath path) { + auto origOut = path.Table().Cast<TYtOutput>(); + return Build<TYtOutput>(ctx, origOut.Pos()) + .InitFrom(origOut) + .Mode(mode) + .Done(); + } + ); + } else { + std::transform(mergeSection.Paths().begin(), mergeSection.Paths().end(), std::back_inserter(updateInputs), + [](TYtPath path) { + return path.Table().Cast<TYtOutput>(); + } + ); + } + } + } + if (hasUpdates) { + return Build<TYtPublish>(ctx, publish.Pos()) + .InitFrom(publish) + .Input() + .Add(updateInputs) + .Build() + .Done().Ptr(); + } + return node; +} + +TMaybeNode<TExprBase> TYtPhysicalOptProposalTransformer::MapToMerge(TExprBase node, TExprContext& ctx) const { + auto map = node.Cast<TYtMap>(); + + auto mapper = map.Mapper(); + if (mapper.Body().Raw() != mapper.Args().Arg(0).Raw()) { + // Only trivial lambda + return node; + } + + if (map.Ref().HasResult()) { + return node; + } + + if (map.Input().Size() > 1 || map.Output().Size() > 1) { + return node; + } + + if (NYql::HasAnySetting(map.Settings().Ref(), EYtSettingType::JobCount | EYtSettingType::WeakFields | EYtSettingType::Sharded | EYtSettingType::SortLimitBy)) { + return node; + } + + auto section = map.Input().Item(0); + if (NYql::HasSetting(section.Settings().Ref(), EYtSettingType::SysColumns)) { + return node; + } + bool useExplicitColumns = false; + const auto outRowSpec = TYqlRowSpecInfo(map.Output().Item(0).RowSpec()); + const auto nativeType = outRowSpec.GetNativeYtType(); + const auto nativeTypeFlags = outRowSpec.GetNativeYtTypeFlags(); + + for (auto path: section.Paths()) { + TYtPathInfo pathInfo(path); + if (pathInfo.RequiresRemap()) { + return node; + } + if (nativeType != pathInfo.GetNativeYtType() + || nativeTypeFlags != pathInfo.GetNativeYtTypeFlags()) { + return node; + } + if (!pathInfo.HasColumns() && (!pathInfo.Table->IsTemp || (pathInfo.Table->RowSpec && pathInfo.Table->RowSpec->HasAuxColumns()))) { + useExplicitColumns = true; + } + } + + if (auto outSorted = map.Output().Item(0).Ref().GetConstraint<TSortedConstraintNode>()) { + auto inputSorted = map.Input().Item(0).Ref().GetConstraint<TSortedConstraintNode>(); + if (!inputSorted || !outSorted->IsPrefixOf(*inputSorted)) { + // Don't convert YtMap, which produces sorted output from unsorted input + return node; + } + if (auto maxTablesForSortedMerge = State_->Configuration->MaxInputTablesForSortedMerge.Get()) { + if (map.Input().Item(0).Paths().Size() > *maxTablesForSortedMerge) { + return node; + } + } + } + + if (useExplicitColumns) { + auto inputStructType = section.Ref().GetTypeAnn()->Cast<TListExprType>()->GetItemType()->Cast<TStructExprType>(); + TSet<TStringBuf> columns; + for (auto item: inputStructType->GetItems()) { + columns.insert(item->GetName()); + } + + section = UpdateInputFields(section, std::move(columns), ctx, false); + } + + return Build<TYtMerge>(ctx, node.Pos()) + .World(map.World()) + .DataSink(map.DataSink()) + .Output(map.Output()) + .Input() + .Add(section) + .Build() + .Settings(NYql::KeepOnlySettings(map.Settings().Ref(), EYtSettingType::Limit | EYtSettingType::KeepSorted, ctx)) + .Done(); +} + +TMaybeNode<TExprBase> TYtPhysicalOptProposalTransformer::MergeToCopy(TExprBase node, TExprContext& ctx) const { + auto merge = node.Cast<TYtMerge>(); + + if (merge.Ref().HasResult()) { + return node; + } + + if (merge.Input().Item(0).Paths().Size() > 1) { + return node; + } + + if (NYql::HasAnySetting(merge.Settings().Ref(), EYtSettingType::ForceTransform | EYtSettingType::TransformColGroups | EYtSettingType::CombineChunks)) { + return node; + } + + auto limitNode = NYql::GetSetting(merge.Settings().Ref(), EYtSettingType::Limit); + if (limitNode && limitNode->ChildrenSize() > 0) { + return node; + } + + TYtSection section = merge.Input().Item(0); + TYtPath path = section.Paths().Item(0); + if (!path.Ranges().Maybe<TCoVoid>() || !path.Ref().GetTypeAnn()->Equals(*path.Table().Ref().GetTypeAnn())) { + return node; + } + if (path.Table().Maybe<TYtOutput>().Operation().Maybe<TYtEquiJoin>()) { + // YtEquiJoin may change output sort after rewrite + return node; + } + auto tableInfo = TYtTableBaseInfo::Parse(path.Table()); + if (path.Table().Maybe<TYtTable>() || tableInfo->Meta->IsDynamic || !tableInfo->RowSpec || !tableInfo->RowSpec->StrictSchema) { + return node; + } + if (tableInfo->IsUnordered && tableInfo->RowSpec->IsSorted()) { + return node; + } + if (NYql::HasAnySetting(section.Settings().Ref(), EYtSettingType::Take | EYtSettingType::Skip | EYtSettingType::Sample)) { + return node; + } + if (NYql::HasNonEmptyKeyFilter(section)) { + return node; + } + if (NYql::HasSetting(merge.Settings().Ref(), EYtSettingType::KeepSorted)) { + auto op = path.Table().Maybe<TYtOutput>().Operation().Cast(); + if (!(op.Ref().HasResult() && op.Ref().GetResult().Type() == TExprNode::World || op.Maybe<TYtTouch>())) { + return node; + } + } + TYtOutTableInfo outTableInfo(merge.Output().Item(0)); + if (!tableInfo->RowSpec->CompareSortness(*outTableInfo.RowSpec)) { + return node; + } + + return Build<TYtCopy>(ctx, node.Pos()) + .World(merge.World()) + .DataSink(merge.DataSink()) + .Output(merge.Output()) + .Input() + .Add() + .Paths() + .Add() + .InitFrom(path) + .Columns<TCoVoid>().Build() + .Build() + .Build() + .Settings() + .Build() + .Build() + .Build() + .Settings() + .Build() + .Done(); +} + +TMaybeNode<TExprBase> TYtPhysicalOptProposalTransformer::ForceTransform(TExprBase node, TExprContext& ctx) const { + auto merge = node.Cast<TYtMerge>(); + + if (merge.Ref().HasResult()) { + return node; + } + + if (!NYql::HasSetting(merge.Settings().Ref(), EYtSettingType::ForceTransform) + && NYql::HasSetting(merge.Input().Item(0).Settings().Ref(), EYtSettingType::Sample)) { + return TExprBase(ctx.ChangeChild(merge.Ref(), TYtMerge::idx_Settings, NYql::AddSetting(merge.Settings().Ref(), EYtSettingType::ForceTransform, {}, ctx))); + } + + const auto cluster = merge.DataSink().Cluster().StringValue(); + if (State_->Configuration->OptimizeFor.Get(cluster).GetOrElse(NYT::OF_LOOKUP_ATTR) == NYT::OF_LOOKUP_ATTR) { + return node; + } + + TString outGroup; + if (auto setting = NYql::GetSetting(merge.Output().Item(0).Settings().Ref(), EYtSettingType::ColumnGroups)) { + outGroup = setting->Tail().Content(); + } + + std::vector<TString> inputColGroupSpecs; + for (const auto& path: merge.Input().Item(0).Paths()) { + inputColGroupSpecs.emplace_back(); + if (auto table = path.Table().Maybe<TYtTable>()) { + if (auto tableDesc = State_->TablesData->FindTable(cluster, TString{TYtTableInfo::GetTableLabel(table.Cast())}, TEpochInfo::Parse(table.Cast().Epoch().Ref()))) { + inputColGroupSpecs.back() = tableDesc->ColumnGroupSpec; + } + } else if (auto out = path.Table().Maybe<TYtOutput>()) { + if (auto setting = NYql::GetSetting(GetOutputOp(out.Cast()).Output().Item(FromString<ui32>(out.Cast().OutIndex().Value())).Settings().Ref(), EYtSettingType::ColumnGroups)) { + inputColGroupSpecs.back() = setting->Tail().Content(); + } + } + } + + bool needTransformColGroups = false; + if (!outGroup.empty() && AnyOf(inputColGroupSpecs, [&outGroup](const auto& g) { return outGroup != g; })) { + needTransformColGroups = true; + } + if (outGroup.empty() && AnyOf(inputColGroupSpecs, [](const auto& g) { return !g.empty(); })) { + needTransformColGroups = true; + } + + if (needTransformColGroups && !NYql::HasSetting(merge.Settings().Ref(), EYtSettingType::TransformColGroups)) { + return TExprBase(ctx.ChangeChild(merge.Ref(), TYtMerge::idx_Settings, NYql::AddSetting(merge.Settings().Ref(), EYtSettingType::TransformColGroups, {}, ctx))); + } else if (!needTransformColGroups && NYql::HasSetting(merge.Settings().Ref(), EYtSettingType::TransformColGroups)) { + return TExprBase(ctx.ChangeChild(merge.Ref(), TYtMerge::idx_Settings, NYql::RemoveSetting(merge.Settings().Ref(), EYtSettingType::TransformColGroups, ctx))); + } + return node; +} + +} // namespace NYql diff --git a/yt/yql/providers/yt/provider/phy_opt/yql_yt_phy_opt_misc.cpp b/yt/yql/providers/yt/provider/phy_opt/yql_yt_phy_opt_misc.cpp new file mode 100644 index 0000000000..e31ad70e39 --- /dev/null +++ b/yt/yql/providers/yt/provider/phy_opt/yql_yt_phy_opt_misc.cpp @@ -0,0 +1,899 @@ +#include "yql_yt_phy_opt.h" +#include "yql_yt_phy_opt_helper.h" + +#include <yt/yql/providers/yt/provider/yql_yt_helpers.h> +#include <yt/yql/providers/yt/provider/yql_yt_optimize.h> +#include <yt/yql/providers/yt/opt/yql_yt_key_selector.h> +#include <yql/essentials/providers/common/codec/yql_codec_type_flags.h> +#include <yql/essentials/providers/result/expr_nodes/yql_res_expr_nodes.h> + +#include <yql/essentials/core/yql_opt_utils.h> +#include <yql/essentials/core/yql_type_helpers.h> + +#include <util/generic/xrange.h> +namespace NYql { + +using namespace NNodes; +using namespace NPrivate; + +TMaybeNode<TExprBase> TYtPhysicalOptProposalTransformer::EmbedLimit(TExprBase node, TExprContext& ctx) const { + auto op = node.Cast<TYtWithUserJobsOpBase>(); + if (op.Output().Size() != 1) { + return node; + } + auto settings = op.Settings(); + auto limitSetting = NYql::GetSetting(settings.Ref(), EYtSettingType::Limit); + if (!limitSetting) { + return node; + } + if (HasNodesToCalculate(node.Ptr())) { + return node; + } + + TMaybe<ui64> limit = GetLimit(settings.Ref()); + if (!limit) { + return node; + } + + auto sortLimitBy = NYql::GetSettingAsColumnPairList(settings.Ref(), EYtSettingType::SortLimitBy); + if (!sortLimitBy.empty() && *limit > State_->Configuration->TopSortMaxLimit.Get().GetOrElse(DEFAULT_TOP_SORT_LIMIT)) { + return node; + } + + size_t lambdaIdx = op.Maybe<TYtMapReduce>() + ? TYtMapReduce::idx_Reducer + : op.Maybe<TYtReduce>() + ? TYtReduce::idx_Reducer + : TYtMap::idx_Mapper; + + auto lambda = TCoLambda(op.Ref().ChildPtr(lambdaIdx)); + if (IsEmptyContainer(lambda.Body().Ref()) || IsEmpty(lambda.Body().Ref(), *State_->Types)) { + return node; + } + + if (sortLimitBy.empty()) { + if (lambda.Body().Maybe<TCoTake>()) { + return node; + } + + lambda = Build<TCoLambda>(ctx, lambda.Pos()) + .Args({"stream"}) + .Body<TCoTake>() + .Input<TExprApplier>() + .Apply(lambda) + .With(0, "stream") + .Build() + .Count<TCoUint64>() + .Literal() + .Value(ToString(*limit)) + .Build() + .Build() + .Build() + .Done(); + } else { + if (lambda.Body().Maybe<TCoTopBase>()) { + return node; + } + + if (const auto& body = lambda.Body().Ref(); body.IsCallable("ExpandMap") && body.Head().IsCallable({"Top", "TopSort"})) { + return node; + } + + lambda = Build<TCoLambda>(ctx, lambda.Pos()) + .Args({"stream"}) + .Body<TCoTop>() + .Input<TExprApplier>() + .Apply(lambda) + .With(0, "stream") + .Build() + .Count<TCoUint64>() + .Literal() + .Value(ToString(*limit)) + .Build() + .Build() + .SortDirections([&sortLimitBy] (TExprNodeBuilder& builder) { + auto listBuilder = builder.List(); + for (size_t i: xrange(sortLimitBy.size())) { + listBuilder.Callable(i, TCoBool::CallableName()) + .Atom(0, sortLimitBy[i].second ? "True" : "False") + .Seal(); + } + listBuilder.Seal(); + }) + .KeySelectorLambda() + .Args({"item"}) + .Body([&sortLimitBy] (TExprNodeBuilder& builder) { + auto listBuilder = builder.List(); + for (size_t i: xrange(sortLimitBy.size())) { + listBuilder.Callable(i, TCoMember::CallableName()) + .Arg(0, "item") + .Atom(1, sortLimitBy[i].first) + .Seal(); + } + listBuilder.Seal(); + }) + .Build() + .Build().Done(); + + if (auto& l = lambda.Ref(); l.Tail().Head().IsCallable("ExpandMap")) { + lambda = TCoLambda(ctx.ChangeChild(l, 1U, ctx.SwapWithHead(l.Tail()))); + } + } + + return TExprBase(ctx.ChangeChild(op.Ref(), lambdaIdx, lambda.Ptr())); +} + +TMaybeNode<TExprBase> TYtPhysicalOptProposalTransformer::Mux(TExprBase node, TExprContext& ctx) const { + auto mux = node.Cast<TCoMux>(); + const TTypeAnnotationNode* muxItemTypeNode = GetSeqItemType(mux.Ref().GetTypeAnn()); + if (!muxItemTypeNode) { + return node; + } + auto muxItemType = muxItemTypeNode->Cast<TVariantExprType>(); + if (muxItemType->GetUnderlyingType()->GetKind() != ETypeAnnotationKind::Tuple) { + return node; + } + + const bool useNativeDescSort = State_->Configuration->UseNativeDescSort.Get().GetOrElse(DEFAULT_USE_NATIVE_DESC_SORT); + bool allAreTables = true; + bool hasTables = false; + bool allAreTableContents = true; + bool hasContents = false; + TString resultCluster; + TMaybeNode<TYtDSource> dataSource; + for (auto child: mux.Input().Cast<TExprList>()) { + bool isTable = IsYtProviderInput(child); + bool isContent = child.Maybe<TYtTableContent>().IsValid(); + if (!isTable && !isContent) { + // Don't match foreign provider input + if (child.Maybe<TCoRight>()) { + return node; + } + } else { + if (!dataSource) { + dataSource = GetDataSource(child, ctx); + } + + if (!resultCluster) { + resultCluster = TString{dataSource.Cast().Cluster().Value()}; + } + else if (resultCluster != dataSource.Cast().Cluster().Value()) { + ctx.AddError(TIssue(ctx.GetPosition(node.Pos()), TStringBuilder() + << "Different source clusters in Mux: " << resultCluster + << " and " << dataSource.Cast().Cluster().Value())); + return {}; + } + } + allAreTables = allAreTables && isTable; + hasTables = hasTables || isTable; + allAreTableContents = allAreTableContents && isContent; + hasContents = hasContents || isContent; + } + + if (!hasTables && !hasContents) { + return node; + } + + auto dataSink = TYtDSink(ctx.RenameNode(dataSource.Ref(), "DataSink")); + if (allAreTables || allAreTableContents) { + TVector<TExprBase> worlds; + TVector<TYtSection> sections; + for (auto child: mux.Input().Cast<TExprList>()) { + auto read = child.Maybe<TCoRight>().Input().Maybe<TYtReadTable>(); + if (!read) { + read = child.Maybe<TYtTableContent>().Input().Maybe<TYtReadTable>(); + } + if (read) { + YQL_ENSURE(read.Cast().Input().Size() == 1); + auto section = read.Cast().Input().Item(0); + sections.push_back(section); + if (allAreTables) { + worlds.push_back(GetWorld(child, {}, ctx)); + } + } else { + YQL_ENSURE(child.Maybe<TYtOutput>(), "Unknown Mux element: " << child.Ref().Content()); + sections.push_back( + Build<TYtSection>(ctx, child.Pos()) + .Paths() + .Add() + .Table(child) // child is TYtOutput + .Columns<TCoVoid>().Build() + .Ranges<TCoVoid>().Build() + .Stat<TCoVoid>().Build() + .Build() + .Build() + .Settings() + .Build() + .Done() + ); + } + } + + auto world = worlds.empty() + ? TExprBase(ctx.NewWorld(mux.Pos())) + : worlds.size() == 1 + ? worlds.front() + : Build<TCoSync>(ctx, mux.Pos()).Add(worlds).Done(); + + auto resRead = Build<TYtReadTable>(ctx, mux.Pos()) + .World(world) + .DataSource(dataSource.Cast()) + .Input() + .Add(sections) + .Build() + .Done(); + + return allAreTables + ? Build<TCoRight>(ctx, mux.Pos()) + .Input(resRead) + .Done().Cast<TExprBase>() + : Build<TYtTableContent>(ctx, mux.Pos()) + .Input(resRead) + .Settings().Build() + .Done().Cast<TExprBase>(); + } + + if (!hasTables) { + return node; + } + + TVector<TExprBase> newMuxParts; + for (auto child: mux.Input().Cast<TExprList>()) { + if (!IsYtProviderInput(child)) { + if (State_->Types->EvaluationInProgress) { + return node; + } + TSyncMap syncList; + if (!IsYtCompleteIsolatedLambda(child.Ref(), syncList, resultCluster, false)) { + return node; + } + + const TStructExprType* outItemType = nullptr; + if (auto type = GetSequenceItemType(child, false, ctx)) { + if (!EnsurePersistableType(child.Pos(), *type, ctx)) { + return {}; + } + outItemType = type->Cast<TStructExprType>(); + } else { + return {}; + } + + TYtOutTableInfo outTable(outItemType, State_->Configuration->UseNativeYtTypes.Get().GetOrElse(DEFAULT_USE_NATIVE_YT_TYPES) ? NTCF_ALL : NTCF_NONE); + auto content = child; + if (auto sorted = child.Ref().GetConstraint<TSortedConstraintNode>()) { + TKeySelectorBuilder builder(child.Pos(), ctx, useNativeDescSort, outItemType); + builder.ProcessConstraint(*sorted); + builder.FillRowSpecSort(*outTable.RowSpec); + + if (builder.NeedMap()) { + content = Build<TExprApplier>(ctx, child.Pos()) + .Apply(TCoLambda(builder.MakeRemapLambda(true))) + .With(0, content) + .Done(); + } + + } else if (auto unordered = content.Maybe<TCoUnorderedBase>()) { + content = unordered.Cast().Input(); + } + outTable.RowSpec->SetConstraints(child.Ref().GetConstraintSet()); + outTable.SetUnique(child.Ref().GetConstraint<TDistinctConstraintNode>(), child.Pos(), ctx); + + auto cleanup = CleanupWorld(content, ctx); + if (!cleanup) { + return {}; + } + + newMuxParts.push_back( + Build<TYtOutput>(ctx, child.Pos()) + .Operation<TYtFill>() + .World(ApplySyncListToWorld(ctx.NewWorld(child.Pos()), syncList, ctx)) + .DataSink(dataSink) + .Content(MakeJobLambdaNoArg(cleanup.Cast(), ctx)) + .Output() + .Add(outTable.ToExprNode(ctx, child.Pos()).Cast<TYtOutTable>()) + .Build() + .Settings(GetFlowSettings(child.Pos(), *State_, ctx)) + .Build() + .OutIndex().Value(0U).Build() + .Done() + ); + } + else { + newMuxParts.push_back(child); + } + } + + return Build<TCoMux>(ctx, mux.Pos()) + .Input<TExprList>() + .Add(newMuxParts) + .Build() + .Done(); +} + +TMaybeNode<TExprBase> TYtPhysicalOptProposalTransformer::TakeOrSkip(TExprBase node, TExprContext& ctx, const TGetParents& getParents) const { + auto countBase = node.Cast<TCoCountBase>(); + auto input = countBase.Input(); + if (!IsYtProviderInput(input)) { + return node; + } + + auto cluster = TString{GetClusterName(input)}; + TSyncMap syncList; + if (!IsYtCompleteIsolatedLambda(countBase.Count().Ref(), syncList, cluster, false)) { + return node; + } + + auto count = State_->PassiveExecution ? countBase.Count() : CleanupWorld(countBase.Count(), ctx); + if (!count) { + return {}; + } + + EYtSettingType settingType = node.Maybe<TCoSkip>() ? EYtSettingType::Skip : EYtSettingType::Take; + + auto settings = Build<TCoNameValueTupleList>(ctx, countBase.Pos()) + .Add() + .Name() + .Value(ToString(settingType)) + .Build() + .Value(count.Cast()) + .Build() + .Done(); + + if (!ctx.IsConstraintEnabled<TSortedConstraintNode>()) { + if (auto maybeMap = input.Maybe<TYtOutput>().Operation().Maybe<TYtMap>()) { + TYtMap map = maybeMap.Cast(); + if (!IsOutputUsedMultipleTimes(map.Ref(), *getParents())) { + TYtOutTableInfo mapOut(map.Output().Item(0)); + if (mapOut.RowSpec->IsSorted()) { + mapOut.RowSpec->ClearSortness(ctx); + input = Build<TYtOutput>(ctx, input.Pos()) + .InitFrom(input.Cast<TYtOutput>()) + .Operation<TYtMap>() + .InitFrom(map) + .Output() + .Add(mapOut.ToExprNode(ctx, map.Output().Item(0).Pos()).Cast<TYtOutTable>()) + .Build() + .Build() + .Done(); + } + } + } + } + + auto res = Build<TCoRight>(ctx, countBase.Pos()) + .Input<TYtReadTable>() + .World(ApplySyncListToWorld(GetWorld(input, {}, ctx).Ptr(), syncList, ctx)) + .DataSource(GetDataSource(input, ctx)) + .Input(ConvertInputTable(input, ctx, TConvertInputOpts().KeepDirecRead(true).Settings(settings))) + .Build() + .Done(); + return KeepColumnOrder(res.Ptr(), node.Ref(), ctx, *State_->Types); +} + +TMaybeNode<TExprBase> TYtPhysicalOptProposalTransformer::Extend(TExprBase node, TExprContext& ctx) const { + if (State_->PassiveExecution) { + return node; + } + + auto extend = node.Cast<TCoExtendBase>(); + + bool allAreTables = true; + bool hasTables = false; + bool allAreTableContents = true; + bool hasContents = false; + bool keepSort = !ctx.IsConstraintEnabled<TSortedConstraintNode>() || (bool)extend.Ref().GetConstraint<TSortedConstraintNode>(); + TString resultCluster; + TMaybeNode<TYtDSource> dataSource; + + for (auto child: extend) { + bool isTable = IsYtProviderInput(child); + bool isContent = child.Maybe<TYtTableContent>().IsValid(); + if (!isTable && !isContent) { + // Don't match foreign provider input + if (child.Maybe<TCoRight>()) { + return node; + } + } else { + auto currentDataSource = GetDataSource(child, ctx); + auto currentCluster = TString{currentDataSource.Cluster().Value()}; + if (!dataSource) { + dataSource = currentDataSource; + resultCluster = currentCluster; + } else if (resultCluster != currentCluster) { + ctx.AddError(TIssue(ctx.GetPosition(node.Pos()), TStringBuilder() + << "Different source clusters in " << extend.Ref().Content() << ": " << resultCluster + << " and " << currentCluster)); + return {}; + } + } + allAreTables = allAreTables && isTable; + hasTables = hasTables || isTable; + allAreTableContents = allAreTableContents && isContent; + hasContents = hasContents || isContent; + } + + if (!hasTables && !hasContents) { + return node; + } + + auto dataSink = TYtDSink(ctx.RenameNode(dataSource.Ref(), "DataSink")); + if (allAreTables || allAreTableContents) { + TVector<TExprBase> worlds; + TVector<TYtPath> paths; + TExprNode::TListType newExtendParts; + bool updateChildren = false; + bool unordered = false; + bool nonUniq = false; + for (auto child: extend) { + newExtendParts.push_back(child.Ptr()); + + auto read = child.Maybe<TCoRight>().Input().Maybe<TYtReadTable>(); + if (!read) { + read = child.Maybe<TYtTableContent>().Input().Maybe<TYtReadTable>(); + } + if (read) { + YQL_ENSURE(read.Cast().Input().Size() == 1); + auto section = read.Cast().Input().Item(0); + unordered = unordered || NYql::HasSetting(section.Settings().Ref(), EYtSettingType::Unordered); + nonUniq = nonUniq || NYql::HasSetting(section.Settings().Ref(), EYtSettingType::NonUnique); + TExprNode::TPtr settings = NYql::RemoveSettings(section.Settings().Ref(), EYtSettingType::DirectRead | EYtSettingType::Unordered | EYtSettingType::NonUnique, ctx); + if (settings->ChildrenSize() != 0) { + if (State_->Types->EvaluationInProgress || allAreTableContents) { + return node; + } + auto scheme = section.Ref().GetTypeAnn()->Cast<TListExprType>()->GetItemType(); + auto path = CopyOrTrivialMap(section.Pos(), + read.Cast().World(), dataSink, + *scheme, + TYtSection(ctx.ChangeChild(section.Ref(), TYtSection::idx_Settings, std::move(settings))), + {}, ctx, State_, + TCopyOrTrivialMapOpts().SetTryKeepSortness(keepSort).SetRangesResetSort(!keepSort).SetSectionUniq(section.Ref().GetConstraint<TDistinctConstraintNode>()).SetConstraints(extend.Ref().GetConstraintSet())); + updateChildren = true; + newExtendParts.back() = allAreTableContents + ? ctx.ChangeChild(child.Ref(), TYtTableContent::idx_Input, path.Table().Ptr()) + : path.Table().Ptr(); + } else { + paths.insert(paths.end(), section.Paths().begin(), section.Paths().end()); + if (allAreTables) { + worlds.push_back(GetWorld(child, {}, ctx)); + } + } + } else { + YQL_ENSURE(child.Maybe<TYtOutput>(), "Unknown extend element: " << child.Ref().Content()); + paths.push_back( + Build<TYtPath>(ctx, child.Pos()) + .Table(child) // child is TYtOutput + .Columns<TCoVoid>().Build() + .Ranges<TCoVoid>().Build() + .Stat<TCoVoid>().Build() + .Done() + ); + } + } + + if (updateChildren) { + return TExprBase(ctx.ChangeChildren(extend.Ref(), std::move(newExtendParts))); + } + + newExtendParts.clear(); + + auto world = worlds.empty() + ? TExprBase(ctx.NewWorld(extend.Pos())) + : worlds.size() == 1 + ? worlds.front() + : Build<TCoSync>(ctx, extend.Pos()).Add(worlds).Done(); + + auto scheme = extend.Ref().GetTypeAnn()->Cast<TListExprType>()->GetItemType(); + + if (keepSort && extend.Maybe<TCoMerge>() && paths.size() > 1) { + if (State_->Types->EvaluationInProgress) { + return node; + } + auto path = CopyOrTrivialMap(extend.Pos(), + world, dataSink, + *scheme, + Build<TYtSection>(ctx, extend.Pos()) + .Paths() + .Add(paths) + .Build() + .Settings() + .Build() + .Done(), + {}, ctx, State_, + TCopyOrTrivialMapOpts().SetTryKeepSortness(keepSort).SetRangesResetSort(!keepSort).SetSectionUniq(extend.Ref().GetConstraint<TDistinctConstraintNode>()).SetConstraints(extend.Ref().GetConstraintSet())); + world = TExprBase(ctx.NewWorld(extend.Pos())); + paths.assign(1, path); + } + + if (paths.size() == 1 && paths.front().Columns().Maybe<TCoVoid>() && paths.front().Ranges().Maybe<TCoVoid>()) { + return allAreTables + ? paths.front().Table() + : Build<TYtTableContent>(ctx, extend.Pos()) + .Input(paths.front().Table()) + .Settings().Build() + .Done().Cast<TExprBase>(); + } + + auto newSettings = ctx.NewList(extend.Pos(), {}); + if (nonUniq) { + newSettings = NYql::AddSetting(*newSettings, EYtSettingType::NonUnique, {}, ctx); + } + auto newSection = Build<TYtSection>(ctx, extend.Pos()) + .Paths() + .Add(paths) + .Build() + .Settings(newSettings) + .Done(); + if (unordered) { + newSection = MakeUnorderedSection<true>(newSection, ctx); + } + + auto resRead = Build<TYtReadTable>(ctx, extend.Pos()) + .World(world) + .DataSource(dataSource.Cast()) + .Input() + .Add(newSection) + .Build() + .Done(); + + return allAreTables + ? Build<TCoRight>(ctx, extend.Pos()) + .Input(resRead) + .Done().Cast<TExprBase>() + : Build<TYtTableContent>(ctx, extend.Pos()) + .Input(resRead) + .Settings().Build() + .Done().Cast<TExprBase>(); + } + + if (!hasTables) { + return node; + } + + const bool useNativeDescSort = State_->Configuration->UseNativeDescSort.Get().GetOrElse(DEFAULT_USE_NATIVE_DESC_SORT); + TExprNode::TListType newExtendParts; + for (auto child: extend) { + if (!IsYtProviderInput(child)) { + if (State_->Types->EvaluationInProgress) { + return node; + } + TSyncMap syncList; + if (!IsYtCompleteIsolatedLambda(child.Ref(), syncList, resultCluster, false)) { + return node; + } + + const TStructExprType* outItemType = nullptr; + if (auto type = GetSequenceItemType(child, false, ctx)) { + if (!EnsurePersistableType(child.Pos(), *type, ctx)) { + return {}; + } + outItemType = type->Cast<TStructExprType>(); + } else { + return {}; + } + + TYtOutTableInfo outTable(outItemType, State_->Configuration->UseNativeYtTypes.Get().GetOrElse(DEFAULT_USE_NATIVE_YT_TYPES) ? NTCF_ALL : NTCF_NONE); + auto content = child; + auto sorted = child.Ref().GetConstraint<TSortedConstraintNode>(); + if (keepSort && sorted) { + TKeySelectorBuilder builder(child.Pos(), ctx, useNativeDescSort, outItemType); + builder.ProcessConstraint(*sorted); + builder.FillRowSpecSort(*outTable.RowSpec); + + if (builder.NeedMap()) { + content = Build<TExprApplier>(ctx, child.Pos()) + .Apply(TCoLambda(builder.MakeRemapLambda(true))) + .With(0, content) + .Done(); + outItemType = builder.MakeRemapType(); + } + + } else if (auto unordered = content.Maybe<TCoUnorderedBase>()) { + content = unordered.Cast().Input(); + } + outTable.RowSpec->SetConstraints(child.Ref().GetConstraintSet()); + outTable.SetUnique(child.Ref().GetConstraint<TDistinctConstraintNode>(), child.Pos(), ctx); + + auto cleanup = CleanupWorld(content, ctx); + if (!cleanup) { + return {}; + } + + newExtendParts.push_back( + Build<TYtOutput>(ctx, child.Pos()) + .Operation<TYtFill>() + .World(ApplySyncListToWorld(ctx.NewWorld(child.Pos()), syncList, ctx)) + .DataSink(dataSink) + .Content(MakeJobLambdaNoArg(cleanup.Cast(), ctx)) + .Output() + .Add(outTable.ToExprNode(ctx, child.Pos()).Cast<TYtOutTable>()) + .Build() + .Settings(GetFlowSettings(child.Pos(), *State_, ctx)) + .Build() + .OutIndex().Value(0U).Build() + .Done().Ptr() + ); + } + else { + newExtendParts.push_back(child.Ptr()); + } + } + + return TExprBase(ctx.ChangeChildren(extend.Ref(), std::move(newExtendParts))); +} + +TMaybeNode<TExprBase> TYtPhysicalOptProposalTransformer::Length(TExprBase node, TExprContext& ctx) const { + TExprBase list = node.Maybe<TCoLength>() + ? node.Cast<TCoLength>().List() + : node.Cast<TCoHasItems>().List(); + + TExprBase ytLengthInput = list; + if (auto content = list.Maybe<TYtTableContent>()) { + ytLengthInput = content.Cast().Input(); + } else if (!IsYtProviderInput(list)) { + return node; + } + + if (auto right = ytLengthInput.Maybe<TCoRight>()) { + ytLengthInput = right.Cast().Input(); + } + // Now ytLengthInput is either YtReadTable or YtOutput + + TVector<TCoNameValueTuple> takeSkip; + if (auto maybeRead = ytLengthInput.Maybe<TYtReadTable>()) { + auto read = maybeRead.Cast(); + YQL_ENSURE(read.Input().Size() == 1); + TYtSection section = read.Input().Item(0); + bool needMaterialize = NYql::HasSetting(section.Settings().Ref(), EYtSettingType::Sample) + || AnyOf(section.Paths(), [](const TYtPath& path) { return !path.Ranges().Maybe<TCoVoid>() || TYtTableBaseInfo::GetMeta(path.Table())->IsDynamic; }); + for (auto s: section.Settings()) { + switch (FromString<EYtSettingType>(s.Name().Value())) { + case EYtSettingType::Take: + case EYtSettingType::Skip: + takeSkip.push_back(s); + break; + default: + // Skip other settings + break; + } + } + + if (needMaterialize) { + if (State_->Types->EvaluationInProgress) { + return node; + } + + auto scheme = section.Ref().GetTypeAnn()->Cast<TListExprType>()->GetItemType(); + ytLengthInput = CopyOrTrivialMap(section.Pos(), + TExprBase(ctx.NewWorld(section.Pos())), + TYtDSink(ctx.RenameNode(read.DataSource().Ref(), "DataSink")), + *scheme, + Build<TYtSection>(ctx, section.Pos()) + .Paths(section.Paths()) + .Settings(NYql::RemoveSettings(section.Settings().Ref(), EYtSettingType::Take | EYtSettingType::Skip + | EYtSettingType::DirectRead | EYtSettingType::Unordered | EYtSettingType::NonUnique, ctx)) + .Done(), + {}, ctx, State_, TCopyOrTrivialMapOpts()).Table(); + } + else { + auto settings = section.Settings().Ptr(); + if (!takeSkip.empty()) { + settings = NYql::RemoveSettings(*settings, EYtSettingType::Take | EYtSettingType::Skip + | EYtSettingType::DirectRead | EYtSettingType::Unordered | EYtSettingType::NonUnique, ctx); + } + + if (read.World().Ref().Type() == TExprNode::World && settings->ChildrenSize() == 0 && section.Paths().Size() == 1 && section.Paths().Item(0).Table().Maybe<TYtOutput>()) { + // Simplify + ytLengthInput = Build<TYtOutput>(ctx, section.Paths().Item(0).Table().Pos()) + .InitFrom(section.Paths().Item(0).Table().Cast<TYtOutput>()) + .Mode() + .Value(ToString(EYtSettingType::Unordered)) + .Build() + .Done(); + + } else { + ytLengthInput = Build<TYtReadTable>(ctx, read.Pos()) + .InitFrom(read) + .Input() + .Add() + .InitFrom(MakeUnorderedSection(section, ctx)) + .Settings(settings) + .Build() + .Build() + .Done(); + } + } + } + else { + ytLengthInput = Build<TYtOutput>(ctx, ytLengthInput.Pos()) + .InitFrom(ytLengthInput.Cast<TYtOutput>()) + .Mode() + .Value(ToString(EYtSettingType::Unordered)) + .Build() + .Done(); + } + + TExprBase res = Build<TYtLength>(ctx, node.Pos()) + .Input(ytLengthInput) + .Done(); + + for (TCoNameValueTuple s: takeSkip) { + switch (FromString<EYtSettingType>(s.Name().Value())) { + case EYtSettingType::Take: + res = Build<TCoMin>(ctx, node.Pos()) + .Add(res) + .Add(s.Value().Cast()) + .Done(); + break; + case EYtSettingType::Skip: + res = Build<TCoSub>(ctx, node.Pos()) + .Left<TCoMax>() + .Add(res) + .Add(s.Value().Cast()) + .Build() + .Right(s.Value().Cast()) + .Done(); + break; + default: + break; + } + } + + if (node.Maybe<TCoHasItems>()) { + res = Build<TCoAggrNotEqual>(ctx, node.Pos()) + .Left(res) + .Right<TCoUint64>() + .Literal() + .Value(0U) + .Build() + .Build() + .Done(); + } + + return res; +} + +TMaybeNode<TExprBase> TYtPhysicalOptProposalTransformer::ResPull(TExprBase node, TExprContext& ctx) const { + auto resPull = node.Cast<TResPull>(); + + auto maybeRead = resPull.Data().Maybe<TCoRight>().Input().Maybe<TYtReadTable>(); + if (!maybeRead) { + // Nothing to optimize in case of ResPull! over YtOutput! + return node; + } + + auto read = maybeRead.Cast(); + if (read.Input().Size() != 1) { + return node; + } + auto section = read.Input().Item(0); + auto scheme = section.Ref().GetTypeAnn()->Cast<TListExprType>()->GetItemType(); + bool directRead = NYql::HasSetting(section.Settings().Ref(), EYtSettingType::DirectRead); + const bool hasSettings = NYql::HasAnySetting(section.Settings().Ref(), + EYtSettingType::Take | EYtSettingType::Skip | EYtSettingType::Sample | EYtSettingType::SysColumns); + + const ui64 nativeTypeFlags = State_->Configuration->UseNativeYtTypes.Get().GetOrElse(DEFAULT_USE_NATIVE_YT_TYPES) + ? GetNativeYtTypeFlags(*scheme->Cast<TStructExprType>()) + : 0ul; + + bool requiresMapOrMerge = false; + bool hasRanges = false; + bool hasNonTemp = false; + bool hasDynamic = false; + bool first = true; + TMaybe<NYT::TNode> firstNativeType; + for (auto path: section.Paths()) { + TYtPathInfo pathInfo(path); + if (first) { + first = false; + firstNativeType = pathInfo.GetNativeYtType(); + } + requiresMapOrMerge = requiresMapOrMerge || pathInfo.Table->RequiresRemap() + || !IsSameAnnotation(*scheme, *pathInfo.Table->RowSpec->GetType()) + || nativeTypeFlags != pathInfo.GetNativeYtTypeFlags() + || firstNativeType != pathInfo.GetNativeYtType(); + hasRanges = hasRanges || pathInfo.Ranges; + hasNonTemp = hasNonTemp || !pathInfo.Table->IsTemp; + hasDynamic = hasDynamic || pathInfo.Table->Meta->IsDynamic; + } + + if (!requiresMapOrMerge && !hasRanges && !hasSettings) + return node; + + // Ignore DirectRead pragma for temporary tables and dynamic tables with sampling or ranges + if (!hasNonTemp || (hasDynamic && (NYql::HasSetting(section.Settings().Ref(), EYtSettingType::Sample) || hasRanges))) { + directRead = false; + } + + if (directRead) { + return node; + } + + bool keepSorted = ctx.IsConstraintEnabled<TSortedConstraintNode>() + ? (!NYql::HasSetting(section.Settings().Ref(), EYtSettingType::Unordered) && !hasNonTemp && section.Paths().Size() == 1) // single sorted input from operation + : (!hasDynamic || !NYql::HasAnySetting(section.Settings().Ref(), EYtSettingType::Take | EYtSettingType::Skip)); // compatibility - all except dynamic with limit + auto path = CopyOrTrivialMap(read.Pos(), + read.World(), + TYtDSink(ctx.RenameNode(read.DataSource().Ref(), "DataSink")), + *scheme, + Build<TYtSection>(ctx, section.Pos()) + .Paths(section.Paths()) + .Settings(NYql::RemoveSettings(section.Settings().Ref(), EYtSettingType::DirectRead | EYtSettingType::Unordered | EYtSettingType::NonUnique, ctx)) + .Done(), + {}, ctx, State_, + TCopyOrTrivialMapOpts().SetTryKeepSortness(keepSorted).SetSectionUniq(section.Ref().GetConstraint<TDistinctConstraintNode>()).SetConstraints(read.Ref().GetConstraintSet())); + + auto newData = path.Columns().Maybe<TCoVoid>() && path.Ranges().Maybe<TCoVoid>() + ? path.Table() + : Build<TCoRight>(ctx, resPull.Pos()) + .Input<TYtReadTable>() + .World(ctx.NewWorld(resPull.Pos())) + .DataSource(read.DataSource()) + .Input() + .Add() + .Paths() + .Add(path) + .Build() + .Settings() + .Build() + .Build() + .Build() + .Build() + .Done(); + + return ctx.ChangeChild(resPull.Ref(), TResPull::idx_Data, newData.Ptr()); +} + +TMaybeNode<TExprBase> TYtPhysicalOptProposalTransformer::TransientOpWithSettings(TExprBase node, TExprContext& ctx) const { + auto op = node.Cast<TYtTransientOpBase>(); + + if (node.Ref().HasResult() && node.Ref().GetResult().Type() == TExprNode::World) { + return node; + } + + TYqlRowSpecInfo::TPtr outRowSpec; + bool keepSortness = false; + if (op.Maybe<TYtReduce>()) { + keepSortness = true; + } else if (op.Maybe<TYtCopy>() || op.Maybe<TYtMerge>()) { + outRowSpec = MakeIntrusive<TYqlRowSpecInfo>(op.Output().Item(0).RowSpec()); + keepSortness = outRowSpec->IsSorted(); + } else if (op.Maybe<TYtMap>()) { + keepSortness = AnyOf(op.Output(), [] (const TYtOutTable& out) { + return TYqlRowSpecInfo(out.RowSpec()).IsSorted(); + }); + } + + bool hasUpdates = false; + TVector<TExprBase> updatedSections; + TSyncMap syncList; + for (auto section: op.Input()) { + updatedSections.push_back(section); + + if (auto updatedSection = UpdateSectionWithSettings(op.World(), section, op.DataSink(), outRowSpec, keepSortness, true, true, syncList, State_, ctx)) { + updatedSections.back() = updatedSection.Cast(); + hasUpdates = true; + } + } + if (!hasUpdates) { + return node; + } + + auto res = ctx.ChangeChild(op.Ref(), TYtTransientOpBase::idx_Input, + Build<TYtSectionList>(ctx, op.Input().Pos()) + .Add(updatedSections) + .Done().Ptr()); + if (!syncList.empty()) { + res = ctx.ChangeChild(*res, TYtTransientOpBase::idx_World, + ApplySyncListToWorld(res->ChildPtr(TYtTransientOpBase::idx_World), syncList, ctx)); + } + // Transform YtCopy to YtMerge in case of ranges + if (op.Maybe<TYtCopy>()) { + if (AnyOf(updatedSections.front().Cast<TYtSection>().Paths(), [](TYtPath path) { return !path.Ranges().Maybe<TCoVoid>(); })) { + res = ctx.RenameNode(*res, TYtMerge::CallableName()); + } + } + return TExprBase(res); +} + +} // namespace NYql diff --git a/yt/yql/providers/yt/provider/phy_opt/yql_yt_phy_opt_partition.cpp b/yt/yql/providers/yt/provider/phy_opt/yql_yt_phy_opt_partition.cpp new file mode 100644 index 0000000000..a62a1be62a --- /dev/null +++ b/yt/yql/providers/yt/provider/phy_opt/yql_yt_phy_opt_partition.cpp @@ -0,0 +1,887 @@ +#include "yql_yt_phy_opt.h" +#include "yql_yt_phy_opt_helper.h" + +#include <yt/yql/providers/yt/provider/yql_yt_helpers.h> +#include <yt/yql/providers/yt/opt/yql_yt_key_selector.h> +#include <yt/yql/providers/yt/common/yql_names.h> + +#include <yql/essentials/core/yql_type_helpers.h> +#include <yql/essentials/core/yql_opt_utils.h> + +namespace NYql { + +using namespace NNodes; +using namespace NPrivate; + +TMaybeNode<TExprBase> TYtPhysicalOptProposalTransformer::PartitionByKey(TExprBase node, TExprContext& ctx) const { + if (State_->Types->EvaluationInProgress || State_->PassiveExecution) { + return node; + } + + auto partByKey = node.Cast<TCoPartitionByKeyBase>(); + + TExprBase input = partByKey.Input(); + TCoLambda keySelectorLambda = partByKey.KeySelectorLambda(); + TCoLambda handlerLambda = partByKey.ListHandlerLambda(); + + if (!IsYtProviderInput(input, true)) { + return node; + } + + auto outItemType = SilentGetSequenceItemType(handlerLambda.Body().Ref(), true); + if (!outItemType || !outItemType->IsPersistable()) { + return node; + } + + auto cluster = TString{GetClusterName(input)}; + TSyncMap syncList; + if (!IsYtCompleteIsolatedLambda(keySelectorLambda.Ref(), syncList, cluster, false) + || !IsYtCompleteIsolatedLambda(handlerLambda.Ref(), syncList, cluster, false)) { + return node; + } + + const auto inputItemType = GetSequenceItemType(input, true, ctx); + if (!inputItemType) { + return {}; + } + const bool multiInput = (inputItemType->GetKind() == ETypeAnnotationKind::Variant); + bool useSystemColumns = State_->Configuration->UseSystemColumns.Get().GetOrElse(DEFAULT_USE_SYS_COLUMNS); + const bool useNativeDescSort = State_->Configuration->UseNativeDescSort.Get().GetOrElse(DEFAULT_USE_NATIVE_DESC_SORT); + + TVector<TYtPathInfo::TPtr> inputPaths = GetInputPaths(input); + bool needMap = AnyOf(inputPaths, [] (const TYtPathInfo::TPtr& path) { + return path->RequiresRemap(); + }); + + bool forceMapper = false; + if (auto maybeRead = input.Maybe<TCoRight>().Input().Maybe<TYtReadTable>()) { + forceMapper = AnyOf(maybeRead.Cast().Input(), [] (const TYtSection& section) { + return NYql::HasSetting(section.Settings().Ref(), EYtSettingType::SysColumns); + }); + } + + if (!multiInput) { + const ui64 nativeTypeFlags = State_->Configuration->UseNativeYtTypes.Get().GetOrElse(DEFAULT_USE_NATIVE_YT_TYPES) + ? GetNativeYtTypeFlags(*inputItemType->Cast<TStructExprType>()) + : 0ul; + + TMaybe<NYT::TNode> firstNativeType; + if (!inputPaths.empty()) { + firstNativeType = inputPaths.front()->GetNativeYtType(); + } + + forceMapper = forceMapper || AnyOf(inputPaths, [nativeTypeFlags, firstNativeType] (const TYtPathInfo::TPtr& path) { + return nativeTypeFlags != path->GetNativeYtTypeFlags() + || firstNativeType != path->GetNativeYtType(); + }); + } + + bool useExplicitColumns = AnyOf(inputPaths, [] (const TYtPathInfo::TPtr& path) { + return !path->Table->IsTemp + || (path->Table->RowSpec && path->Table->RowSpec->HasAuxColumns()); + }); + + TKeySelectorBuilder builder(node.Pos(), ctx, useNativeDescSort, inputItemType); + builder.ProcessKeySelector(keySelectorLambda.Ptr(), {}, true); + + TVector<std::pair<TString, bool>> reduceByColumns = builder.ForeignSortColumns(); + TVector<std::pair<TString, bool>> sortByColumns; + + if (!partByKey.SortDirections().Maybe<TCoVoid>()) { + TExprBase sortDirections = partByKey.SortDirections(); + if (!IsConstExpSortDirections(sortDirections)) { + return node; + } + + TCoLambda sortKeySelectorLambda = partByKey.SortKeySelectorLambda().Cast<TCoLambda>(); + if (!IsYtCompleteIsolatedLambda(sortKeySelectorLambda.Ref(), syncList, cluster, false)) { + return node; + } + + builder.ProcessKeySelector(sortKeySelectorLambda.Ptr(), sortDirections.Ptr()); + sortByColumns = builder.ForeignSortColumns(); + } + + TExprBase mapper = Build<TCoVoid>(ctx, node.Pos()).Done(); + needMap = needMap || builder.NeedMap(); + + bool hasInputSampling = false; + // Read sampling settings from the first section only, because all sections should have the same sampling settings + if (auto maybeReadSettings = input.Maybe<TCoRight>().Input().Maybe<TYtReadTable>().Input().Item(0).Settings()) { + hasInputSampling = NYql::HasSetting(maybeReadSettings.Ref(), EYtSettingType::Sample); + } + + bool canUseReduce = !needMap; + if (canUseReduce) { + TVector<std::pair<TString, bool>> sortPrefix; + for (auto& pathInfo: inputPaths) { + if (pathInfo->Table->IsUnordered + || !pathInfo->Table->RowSpec + || !pathInfo->Table->RowSpec->IsSorted() + || pathInfo->Table->RowSpec->SortedBy.size() < builder.Columns().size()) + { + canUseReduce = false; + break; + } + if (sortPrefix.empty()) { + // reduceBy columns can be in any order, with any ascending + THashMap<TString, bool> partColumnSet(reduceByColumns.begin(), reduceByColumns.end()); + auto sortedBy = pathInfo->Table->RowSpec->GetForeignSort(); + const bool equalReduceByPrefix = AllOf(sortedBy.begin(), sortedBy.begin() + reduceByColumns.size(), + [&partColumnSet](const std::pair<TString, bool>& c) { + return partColumnSet.contains(c.first); + }); + + // sortBy suffix should exactly match + const bool equalSortBySuffix = equalReduceByPrefix && (sortByColumns.empty() + || std::equal(sortByColumns.begin() + reduceByColumns.size(), sortByColumns.end(), sortedBy.begin() + reduceByColumns.size())); + + if (equalSortBySuffix) { + sortPrefix.assign(sortedBy.begin(), sortedBy.begin() + builder.Columns().size()); + // All other tables should have the same sort order as the first one + } else { + canUseReduce = false; + break; + } + } else { + auto sortedBy = pathInfo->Table->RowSpec->GetForeignSort(); + if (!std::equal(sortPrefix.begin(), sortPrefix.end(), sortedBy.begin())) { + canUseReduce = false; + break; + } + } + } + if (canUseReduce) { + const auto reduceBySize = reduceByColumns.size(); + reduceByColumns.assign(sortPrefix.begin(), sortPrefix.begin() + reduceBySize); + if (!sortByColumns.empty()) { + sortByColumns = std::move(sortPrefix); + } + } + } + + const bool canUseMapInsteadOfReduce = keySelectorLambda.Body().Ref().IsComplete() && + partByKey.SortDirections().Maybe<TCoVoid>() && + State_->Configuration->PartitionByConstantKeysViaMap.Get().GetOrElse(DEFAULT_PARTITION_BY_CONSTANT_KEYS_VIA_MAP); + + if (canUseMapInsteadOfReduce) { + YQL_ENSURE(!canUseReduce); + YQL_ENSURE(sortByColumns.empty()); + useSystemColumns = false; + } + + auto settingsBuilder = Build<TCoNameValueTupleList>(ctx, node.Pos()); + if (!canUseMapInsteadOfReduce) { + settingsBuilder + .Add() + .Name().Value(ToString(EYtSettingType::ReduceBy)).Build() + .Value(TExprBase(ToColumnPairList(reduceByColumns, node.Pos(), ctx))) + .Build(); + } + + if (!sortByColumns.empty()) { + settingsBuilder + .Add() + .Name().Value(ToString(EYtSettingType::SortBy)).Build() + .Value(TExprBase(ToColumnPairList(sortByColumns, node.Pos(), ctx))) + .Build(); + } + if (!canUseReduce && multiInput) { + needMap = true; // YtMapReduce with empty mapper doesn't support table indices + } + + bool useReduceFlow = State_->Configuration->UseFlow.Get().GetOrElse(DEFAULT_USE_FLOW); + bool useMapFlow = useReduceFlow; + + const bool newPartsByKeys = bool(partByKey.Maybe<TCoPartitionsByKeys>()); + + // Convert reduce output to stream + if (newPartsByKeys) { + if (useSystemColumns) { + TNodeSet nodesToOptimize; + TProcessedNodesSet processedNodes; + auto arg = handlerLambda.Args().Arg(0).Ptr(); + VisitExpr(handlerLambda.Body().Ptr(), [&nodesToOptimize, &processedNodes, arg](const TExprNode::TPtr& node) { + if (TMaybeNode<TCoChopper>(node).GroupSwitch().Body().Maybe<TCoIsKeySwitch>()) { + if (IsDepended(node->Head(), *arg)) { + nodesToOptimize.insert(node.Get()); + } + } + else if (TMaybeNode<TCoCondense1>(node).SwitchHandler().Body().Maybe<TCoIsKeySwitch>()) { + if (IsDepended(node->Head(), *arg)) { + nodesToOptimize.insert(node.Get()); + } + } + else if (TMaybeNode<TCoCondense>(node).SwitchHandler().Body().Maybe<TCoIsKeySwitch>()) { + if (IsDepended(node->Head(), *arg)) { + nodesToOptimize.insert(node.Get()); + } + } + else if (TYtOutput::Match(node.Get())) { + // Stop traversing dependent operations + processedNodes.insert(node->UniqueId()); + return false; + } + return true; + }); + + if (!nodesToOptimize.empty()) { + TOptimizeExprSettings settings(State_->Types); + settings.ProcessedNodes = &processedNodes; // Prevent optimizer to go deeper than current operation + TExprNode::TPtr newBody = handlerLambda.Body().Ptr(); + auto status = OptimizeExpr(newBody, newBody, [&](const TExprNode::TPtr& node, TExprContext& ctx) -> TExprNode::TPtr { + if (nodesToOptimize.find(node.Get()) == nodesToOptimize.end()) { + return node; + } + + if (auto maybeChopper = TMaybeNode<TCoChopper>(node)) { + auto chopper = maybeChopper.Cast(); + + auto chopperSwitch = ctx.Builder(chopper.GroupSwitch().Pos()) + .Lambda() + .Param("key") + .Param("item") + .Callable("SqlExtractKey") + .Arg(0, "item") + .Lambda(1) + .Param("row") + .Callable("Member") + .Arg(0, "row") + .Atom(1, YqlSysColumnKeySwitch, TNodeFlags::Default) + .Seal() + .Seal() + .Seal() + .Seal() + .Build(); + + TExprNode::TPtr chopperHandler; + TExprNode::TPtr chopperKeyExtract; + if (!canUseReduce && multiInput) { + chopperKeyExtract = ctx.Builder(chopper.Handler().Pos()) + .Lambda() + .Param("item") + .Apply(chopper.KeyExtractor().Ptr()) + .With(0) + .Callable("Member") + .Arg(0, "item") + .Atom(1, "_yql_original_row", TNodeFlags::Default) + .Seal() + .Done() + .Seal() + .Seal() + .Build(); + + chopperHandler = ctx.Builder(chopper.Handler().Pos()) + .Lambda() + .Param("key") + .Param("group") + .Apply(chopper.Handler().Ptr()) + .With(0, "key") + .With(1) + .Callable("Map") + .Arg(0, "group") + .Lambda(1) + .Param("row") + .Callable("Member") + .Arg(0, "row") + .Atom(1, "_yql_original_row", TNodeFlags::Default) + .Seal() + .Seal() + .Seal() + .Done() + .Seal() + .Seal() + .Build(); + } else { + chopperKeyExtract = ctx.Builder(chopper.Handler().Pos()) + .Lambda() + .Param("item") + .Apply(chopper.KeyExtractor().Ptr()) + .With(0) + .Callable("RemovePrefixMembers") + .Arg(0, "item") + .List(1) + .Atom(0, YqlSysColumnKeySwitch, TNodeFlags::Default) + .Seal() + .Seal() + .Done() + .Seal() + .Seal() + .Build(); + + chopperHandler = ctx.Builder(chopper.Handler().Pos()) + .Lambda() + .Param("key") + .Param("group") + .Apply(chopper.Handler().Ptr()) + .With(0, "key") + .With(1) + .Callable("RemovePrefixMembers") + .Arg(0, "group") + .List(1) + .Atom(0, YqlSysColumnKeySwitch, TNodeFlags::Default) + .Seal() + .Seal() + .Done() + .Seal() + .Seal() + .Build(); + } + + TNodeOnNodeOwnedMap deepClones; + return Build<TCoChopper>(ctx, chopper.Pos()) + .Input(chopper.Input()) + .KeyExtractor(chopperKeyExtract) + .GroupSwitch(chopperSwitch) + .Handler(chopperHandler) + .Done().Ptr(); + } + else if (auto maybeCondense = TMaybeNode<TCoCondense1>(node)) { + auto condense = maybeCondense.Cast(); + auto switchHandler = ctx.Builder(condense.SwitchHandler().Pos()) + .Lambda() + .Param("item") + .Param("state") + .Callable("SqlExtractKey") + .Arg(0, "item") + .Lambda(1) + .Param("row") + .Callable("Member") + .Arg(0, "row") + .Atom(1, YqlSysColumnKeySwitch, TNodeFlags::Default) + .Seal() + .Seal() + .Seal() + .Seal() + .Build(); + + TExprNode::TPtr initHandler; + TExprNode::TPtr updateHandler; + + if (!canUseReduce && multiInput) { + initHandler = ctx.Builder(condense.InitHandler().Pos()) + .Lambda() + .Param("item") + .Apply(condense.InitHandler().Ptr()) + .With(0) + .Callable("Member") + .Arg(0, "item") + .Atom(1, "_yql_original_row", TNodeFlags::Default) + .Seal() + .Done() + .Seal() + .Seal() + .Build(); + + updateHandler = ctx.Builder(condense.UpdateHandler().Pos()) + .Lambda() + .Param("item") + .Param("state") + .Apply(condense.UpdateHandler().Ptr()) + .With(0) + .Callable("Member") + .Arg(0, "item") + .Atom(1, "_yql_original_row", TNodeFlags::Default) + .Seal() + .Done() + .With(1, "state") + .Seal() + .Seal() + .Build(); + } else { + initHandler = ctx.Builder(condense.InitHandler().Pos()) + .Lambda() + .Param("item") + .Apply(condense.InitHandler().Ptr()) + .With(0) + .Callable("RemovePrefixMembers") + .Arg(0, "item") + .List(1) + .Atom(0, YqlSysColumnKeySwitch, TNodeFlags::Default) + .Seal() + .Seal() + .Done() + .Seal() + .Seal() + .Build(); + + updateHandler = ctx.Builder(condense.UpdateHandler().Pos()) + .Lambda() + .Param("item") + .Param("state") + .Apply(condense.UpdateHandler().Ptr()) + .With(0) + .Callable("RemovePrefixMembers") + .Arg(0, "item") + .List(1) + .Atom(0, YqlSysColumnKeySwitch, TNodeFlags::Default) + .Seal() + .Seal() + .Done() + .With(1, "state") + .Seal() + .Seal() + .Build(); + } + + return Build<TCoCondense1>(ctx, condense.Pos()) + .Input(condense.Input()) + .InitHandler(initHandler) + .SwitchHandler(switchHandler) + .UpdateHandler(updateHandler) + .Done().Ptr(); + } + else if (auto maybeCondense = TMaybeNode<TCoCondense>(node)) { + auto condense = maybeCondense.Cast(); + + auto switchHandler = ctx.Builder(condense.SwitchHandler().Pos()) + .Lambda() + .Param("item") + .Param("state") + .Callable("SqlExtractKey") + .Arg(0, "item") + .Lambda(1) + .Param("row") + .Callable("Member") + .Arg(0, "row") + .Atom(1, YqlSysColumnKeySwitch, TNodeFlags::Default) + .Seal() + .Seal() + .Seal() + .Seal() + .Build(); + + TExprNode::TPtr updateHandler; + if (!canUseReduce && multiInput) { + updateHandler = ctx.Builder(condense.UpdateHandler().Pos()) + .Lambda() + .Param("item") + .Param("state") + .Apply(condense.UpdateHandler().Ptr()) + .With(0) + .Callable("Member") + .Arg(0, "item") + .Atom(1, "_yql_original_row", TNodeFlags::Default) + .Seal() + .Done() + .With(1, "state") + .Seal() + .Seal() + .Build(); + } else { + updateHandler = ctx.Builder(condense.UpdateHandler().Pos()) + .Lambda() + .Param("item") + .Param("state") + .Apply(condense.UpdateHandler().Ptr()) + .With(0) + .Callable("RemovePrefixMembers") + .Arg(0, "item") + .List(1) + .Atom(0, YqlSysColumnKeySwitch, TNodeFlags::Default) + .Seal() + .Seal() + .Done() + .With(1, "state") + .Seal() + .Seal() + .Build(); + } + + return Build<TCoCondense>(ctx, condense.Pos()) + .Input(condense.Input()) + .State(condense.State()) + .SwitchHandler(switchHandler) + .UpdateHandler(updateHandler) + .Done().Ptr(); + } + + return node; + }, ctx, settings); + + if (status.Level == TStatus::Error) { + return {}; + } + + if (status.Level == TStatus::Ok) { + useSystemColumns = false; + } + else { + handlerLambda = Build<TCoLambda>(ctx, handlerLambda.Pos()) + .Args({TStringBuf("stream")}) + .Body<TExprApplier>() + .Apply(TExprBase(newBody)) + .With(handlerLambda.Args().Arg(0), TStringBuf("stream")) + .Build() + .Done(); + } + } + else { + useSystemColumns = false; + } + } + + if (!useSystemColumns) { + auto preReduceLambda = Build<TCoLambda>(ctx, handlerLambda.Pos()) + .Args({"stream"}) + .Body("stream") + .Done(); + + if (!canUseReduce && multiInput) { + preReduceLambda = Build<TCoLambda>(ctx, handlerLambda.Pos()) + .Args({"stream"}) + .Body<TCoFlatMap>() + .Input("stream") + .Lambda() + .Args({"item"}) + .Body<TCoJust>() + .Input<TCoMember>() + .Struct("item") + .Name() + .Value("_yql_original_row") + .Build() + .Build() + .Build() + .Build() + .Build() + .Done(); + } + + handlerLambda = Build<TCoLambda>(ctx, handlerLambda.Pos()) + .Args({"stream"}) + .Body<TExprApplier>() + .Apply(handlerLambda) + .With<TExprApplier>(0) + .Apply(preReduceLambda) + .With(0, "stream") + .Build() + .Build() + .Done(); + } + } + else { + TExprNode::TPtr groupSwitch; + TExprNode::TPtr keyExtractor; + TExprNode::TPtr handler; + + if (canUseMapInsteadOfReduce) { + groupSwitch = Build<TCoLambda>(ctx, handlerLambda.Pos()) + .Args({"key", "item"}) + .Body<TCoBool>() + .Literal().Build("false") + .Build() + .Done().Ptr(); + } else if (useSystemColumns) { + groupSwitch = Build<TCoLambda>(ctx, handlerLambda.Pos()) + .Args({"key", "item"}) + .Body<TCoSqlExtractKey>() + .Item("item") + .Extractor() + .Args({"row"}) + .Body<TCoMember>() + .Struct("row") + .Name() + .Value(YqlSysColumnKeySwitch, TNodeFlags::Default) + .Build() + .Build() + .Build() + .Build() + .Done().Ptr(); + } else { + groupSwitch = Build<TCoLambda>(ctx, handlerLambda.Pos()) + .Args({"key", "item"}) + .Body<TYtIsKeySwitch>() + .DependsOn() + .Input("item") + .Build() + .Build() + .Done().Ptr(); + } + + if (!canUseReduce && multiInput) { + keyExtractor = Build<TCoLambda>(ctx, handlerLambda.Pos()) + .Args({"item"}) + .Body<TExprApplier>() + .Apply(keySelectorLambda) + .With<TCoMember>(0) + .Struct("item") + .Name() + .Value("_yql_original_row") + .Build() + .Build() + .Build() + .Done().Ptr(); + + handler = Build<TCoLambda>(ctx, handlerLambda.Pos()) + .Args({"item"}) + .Body<TCoMember>() + .Struct("item") + .Name() + .Value("_yql_original_row") + .Build() + .Build() + .Done().Ptr(); + } else { + keyExtractor = Build<TCoLambda>(ctx, handlerLambda.Pos()) + .Args({"item"}) + .Body<TExprApplier>() + .Apply(keySelectorLambda) + .With(0, "item") + .Build() + .Done().Ptr(); + + if (canUseMapInsteadOfReduce) { + handler = MakeIdentityLambda(handlerLambda.Pos(), ctx); + } else { + handler = Build<TCoLambda>(ctx, handlerLambda.Pos()) + .Args({"item"}) + .Body<TCoRemovePrefixMembers>() + .Input("item") + .Prefixes() + .Add() + .Value(YqlSysColumnKeySwitch) + .Build() + .Build() + .Build() + .Done().Ptr(); + } + } + + handlerLambda = Build<TCoLambda>(ctx, handlerLambda.Pos()) + .Args({"stream"}) + .Body<TExprApplier>() + .Apply(handlerLambda) + .With<TCoGroupingCore>(0) + .Input("stream") + .GroupSwitch(groupSwitch) + .KeyExtractor(keyExtractor) + .ConvertHandler(handler) + .Build() + .Build() + .Done(); + } + + auto handlerLambdaCleanup = CleanupWorld(handlerLambda, ctx); + if (!handlerLambdaCleanup) { + return {}; + } + + const auto mapOutputType = needMap ? builder.MakeRemapType() : nullptr; + auto reduceInputType = mapOutputType ? mapOutputType : inputItemType; + if (useSystemColumns) { + settingsBuilder + .Add() + .Name().Value(ToString(EYtSettingType::KeySwitch)).Build() + .Build(); + + if (ETypeAnnotationKind::Struct == reduceInputType->GetKind()) { + auto items = reduceInputType->Cast<TStructExprType>()->GetItems(); + items.emplace_back(ctx.MakeType<TItemExprType>(YqlSysColumnKeySwitch, ctx.MakeType<TDataExprType>(EDataSlot::Bool))); + reduceInputType = ctx.MakeType<TStructExprType>(items); + } + } + + TVector<TString> filterColumns; + if (needMap) { + if (auto maybeMapper = CleanupWorld(TCoLambda(builder.MakeRemapLambda()), ctx)) { + mapper = maybeMapper.Cast(); + } else { + return {}; + } + + if (builder.NeedMap() || multiInput) { + if (multiInput) { + filterColumns.emplace_back("_yql_original_row"); + } + else { + for (auto& item: inputItemType->Cast<TStructExprType>()->GetItems()) { + filterColumns.emplace_back(item->GetName()); + } + } + + if (ETypeAnnotationKind::Struct == reduceInputType->GetKind()) { + const std::unordered_set<std::string_view> set(filterColumns.cbegin(), filterColumns.cend()); + TVector<const TItemExprType*> items; + items.reserve(set.size()); + for (const auto& item : reduceInputType->Cast<TStructExprType>()->GetItems()) + if (const auto& name = item->GetName(); YqlSysColumnKeySwitch == name || set.cend() != set.find(name)) + items.emplace_back(item); + reduceInputType = ctx.MakeType<TStructExprType>(items); + } + } + } + + auto reducer = newPartsByKeys ? + MakeJobLambda<true>(handlerLambdaCleanup.Cast(), useReduceFlow, ctx): + MakeJobLambda<false>(handlerLambdaCleanup.Cast(), useReduceFlow, ctx); + + if (useReduceFlow) { + settingsBuilder + .Add() + .Name() + .Value(ToString(EYtSettingType::Flow)) + .Build() + .Build(); + } + + if (canUseReduce) { + auto reduce = Build<TYtReduce>(ctx, node.Pos()) + .World(ApplySyncListToWorld(GetWorld(input, {}, ctx).Ptr(), syncList, ctx)) + .DataSink(GetDataSink(input, ctx)) + .Input(ConvertInputTable(input, ctx)) + .Output() + .Add(ConvertOutTables(node.Pos(), outItemType, ctx, State_, &partByKey.Ref().GetConstraintSet())) + .Build() + .Settings(settingsBuilder.Done()) + .Reducer(reducer) + .Done(); + return WrapOp(reduce, ctx); + } + + if (needMap && (builder.NeedMap() || multiInput) && !canUseMapInsteadOfReduce) { + settingsBuilder + .Add() + .Name().Value(ToString(EYtSettingType::ReduceFilterBy)).Build() + .Value(TExprBase(ToAtomList(filterColumns, node.Pos(), ctx))) + .Build(); + } + + if (canUseMapInsteadOfReduce && !filterColumns.empty()) { + reducer = Build<TCoLambda>(ctx, reducer.Pos()) + .Args({"input"}) + .Body<TExprApplier>() + .Apply(reducer) + .With<TCoMap>(0) + .Input("input") + .Lambda() + .Args({"item"}) + .Body<TCoSelectMembers>() + .Input("item") + .Members(ToAtomList(filterColumns, node.Pos(), ctx)) + .Build() + .Build() + .Build() + .Build() + .Done(); + } + + bool unordered = ctx.IsConstraintEnabled<TSortedConstraintNode>(); + TExprBase world = GetWorld(input, {}, ctx); + if (hasInputSampling) { + + if (forceMapper && !needMap) { + mapper = Build<TCoLambda>(ctx, node.Pos()).Args({"stream"}).Body("stream").Done(); + needMap = true; + } + + if (needMap) { + input = Build<TYtOutput>(ctx, node.Pos()) + .Operation<TYtMap>() + .World(world) + .DataSink(GetDataSink(input, ctx)) + .Input(ConvertInputTable(input, ctx, TConvertInputOpts().MakeUnordered(unordered))) + .Output() + .Add(ConvertOutTables(node.Pos(), mapOutputType ? mapOutputType : inputItemType, ctx, State_)) + .Build() + .Settings(GetFlowSettings(node.Pos(), *State_, ctx)) + .Mapper(MakeJobLambda<false>(mapper.Cast<TCoLambda>(), useMapFlow, ctx)) + .Build() + .OutIndex().Value(0U).Build() + .Done(); + + mapper = Build<TCoVoid>(ctx, node.Pos()).Done(); + needMap = false; + forceMapper = false; + } + else { + TConvertInputOpts opts; + if (useExplicitColumns) { + opts.ExplicitFields(*inputItemType->Cast<TStructExprType>(), node.Pos(), ctx); + } + + input = Build<TYtOutput>(ctx, node.Pos()) + .Operation<TYtMerge>() + .World(world) + .DataSink(GetDataSink(input, ctx)) + .Input(ConvertInputTable(input, ctx, opts.MakeUnordered(unordered))) + .Output() + .Add(ConvertOutTables(node.Pos(), inputItemType, ctx, State_)) + .Build() + .Settings() + .Add() + .Name() + .Value(ToString(EYtSettingType::ForceTransform)) + .Build() + .Build() + .Build() + .Build() + .OutIndex().Value(0U).Build() + .Done(); + } + world = TExprBase(ctx.NewWorld(node.Pos())); + unordered = false; + } + + if (needMap) { + if (multiInput) { + input = Build<TYtOutput>(ctx, node.Pos()) + .Operation<TYtMap>() + .World(world) + .DataSink(GetDataSink(input, ctx)) + .Input(ConvertInputTable(input, ctx, TConvertInputOpts().MakeUnordered(unordered))) + .Output() + .Add(ConvertOutTables(node.Pos(), mapOutputType, ctx, State_)) + .Build() + .Settings(GetFlowSettings(node.Pos(), *State_, ctx)) + .Mapper(MakeJobLambda<false>(mapper.Cast<TCoLambda>(), useMapFlow, ctx)) + .Build() + .OutIndex().Value(0U).Build() + .Done(); + + mapper = Build<TCoVoid>(ctx, node.Pos()).Done(); + world = TExprBase(ctx.NewWorld(node.Pos())); + unordered = false; + } else { + useMapFlow = useReduceFlow; + mapper = MakeJobLambda<false>(mapper.Cast<TCoLambda>(), useMapFlow, ctx); + } + } + + if (canUseMapInsteadOfReduce) { + settingsBuilder + .Add() + .Name().Value(ToString(EYtSettingType::JobCount)).Build() + .Value(TExprBase(ctx.NewAtom(node.Pos(), 1u))) + .Build(); + + auto result = Build<TYtMap>(ctx, node.Pos()) + .World(ApplySyncListToWorld(world.Ptr(), syncList, ctx)) + .DataSink(GetDataSink(input, ctx)) + .Input(ConvertInputTable(input, ctx, TConvertInputOpts().MakeUnordered(unordered))) + .Output() + .Add(ConvertOutTables(node.Pos(), outItemType, ctx, State_, &partByKey.Ref().GetConstraintSet())) + .Build() + .Settings(settingsBuilder.Done()) + .Mapper(reducer) + .Done(); + return WrapOp(result, ctx); + } + + if (forceMapper && mapper.Maybe<TCoVoid>()) { + mapper = MakeJobLambda<false>(Build<TCoLambda>(ctx, node.Pos()).Args({"stream"}).Body("stream").Done(), useMapFlow, ctx); + } + auto mapReduce = Build<TYtMapReduce>(ctx, node.Pos()) + .World(ApplySyncListToWorld(world.Ptr(), syncList, ctx)) + .DataSink(GetDataSink(input, ctx)) + .Input(ConvertInputTable(input, ctx, TConvertInputOpts().MakeUnordered(unordered))) + .Output() + .Add(ConvertOutTables(node.Pos(), outItemType, ctx, State_, &partByKey.Ref().GetConstraintSet())) + .Build() + .Settings(settingsBuilder.Done()) + .Mapper(mapper) + .Reducer(reducer) + .Done(); + return WrapOp(mapReduce, ctx); +} + +} // namespace NYql diff --git a/yt/yql/providers/yt/provider/phy_opt/yql_yt_phy_opt_push.cpp b/yt/yql/providers/yt/provider/phy_opt/yql_yt_phy_opt_push.cpp new file mode 100644 index 0000000000..bf30c27f30 --- /dev/null +++ b/yt/yql/providers/yt/provider/phy_opt/yql_yt_phy_opt_push.cpp @@ -0,0 +1,383 @@ +#include "yql_yt_phy_opt.h" + +#include <yt/yql/providers/yt/provider/yql_yt_helpers.h> +#include <yql/essentials/providers/common/provider/yql_provider.h> + +#include <yql/essentials/core/yql_opt_utils.h> + +namespace NYql { + +using namespace NNodes; + +TMaybeNode<TExprBase> TYtPhysicalOptProposalTransformer::PushMergeLimitToInput(TExprBase node, TExprContext& ctx) const { + if (node.Ref().HasResult() && node.Ref().GetResult().Type() != TExprNode::World) { + return node; + } + + auto op = node.Cast<TYtMerge>(); + + auto settings = op.Settings(); + auto limitSetting = NYql::GetSetting(settings.Ref(), EYtSettingType::Limit); + if (!limitSetting) { + return node; + } + + auto section = op.Input().Item(0); + if (NYql::HasAnySetting(section.Settings().Ref(), EYtSettingType::Skip | EYtSettingType::Sample)) { + return node; + } + if (NYql::HasNonEmptyKeyFilter(section)) { + return node; + } + + if (AnyOf(section.Paths(), [](const TYtPath& path) { return !path.Ranges().Maybe<TCoVoid>().IsValid(); })) { + return node; + } + + for (auto path: section.Paths()) { + TYtPathInfo pathInfo(path); + // Dynamic tables don't support range selectors + if (pathInfo.Table->Meta->IsDynamic) { + return node; + } + } + + TExprNode::TPtr effectiveLimit = GetLimitExpr(limitSetting, ctx); + if (!effectiveLimit) { + return node; + } + + auto sectionSettings = section.Settings().Ptr(); + auto sectionLimitSetting = NYql::GetSetting(*sectionSettings, EYtSettingType::Take); + if (sectionLimitSetting) { + effectiveLimit = ctx.NewCallable(node.Pos(), "Min", { effectiveLimit, sectionLimitSetting->ChildPtr(1) }); + sectionSettings = NYql::RemoveSetting(*sectionSettings, EYtSettingType::Take, ctx); + } + + sectionSettings = NYql::AddSetting(*sectionSettings, EYtSettingType::Take, effectiveLimit, ctx); + + // Keep empty "limit" setting to prevent repeated Limits optimization + auto updatedSettings = NYql::RemoveSetting(settings.Ref(), EYtSettingType::Limit, ctx); + updatedSettings = NYql::AddSetting(*updatedSettings, EYtSettingType::Limit, ctx.NewList(node.Pos(), {}), ctx); + + return Build<TYtMerge>(ctx, op.Pos()) + .InitFrom(op) + .Input() + .Add() + .InitFrom(section) + .Settings(sectionSettings) + .Build() + .Build() + .Settings(updatedSettings) + .Done(); +} + +TMaybeNode<TExprBase> TYtPhysicalOptProposalTransformer::PushDownKeyExtract(TExprBase node, TExprContext& ctx) const { + if (node.Ref().HasResult() && node.Ref().GetResult().Type() != TExprNode::World) { + return node; + } + + auto op = node.Cast<TYtTransientOpBase>(); + + auto getInnerOpForUpdate = [] (const TYtPath& path, const TVector<TStringBuf>& usedKeyFilterColumns) -> TMaybeNode<TYtTransientOpBase> { + auto maybeOp = path.Table().Maybe<TYtOutput>().Operation().Maybe<TYtTransientOpBase>(); + if (!maybeOp) { + return {}; + } + auto innerOp = maybeOp.Cast(); + if (innerOp.Ref().StartsExecution() || innerOp.Ref().HasResult()) { + return {}; + } + + if (!innerOp.Maybe<TYtMerge>() && !innerOp.Maybe<TYtMap>()) { + return {}; + } + + if (innerOp.Input().Size() != 1 || innerOp.Output().Size() != 1) { + return {}; + } + + if (NYql::HasSetting(innerOp.Settings().Ref(), EYtSettingType::Limit)) { + return {}; + } + const auto outSorted = innerOp.Output().Item(0).Ref().GetConstraint<TSortedConstraintNode>(); + if (!outSorted) { + return {}; + } + for (auto path: innerOp.Input().Item(0).Paths()) { + const auto inputSorted = path.Ref().GetConstraint<TSortedConstraintNode>(); + if (!inputSorted || !inputSorted->Includes(*outSorted)) { + return {}; + } + } + + auto innerSection = innerOp.Input().Item(0); + if (NYql::HasSettingsExcept(innerSection.Settings().Ref(), EYtSettingType::SysColumns)) { + return {}; + } + + if (auto maybeMap = innerOp.Maybe<TYtMap>()) { + // lambda must be passthrough for columns used in key filter + // TODO: use passthrough constraints here + TCoLambda lambda = maybeMap.Cast().Mapper(); + TMaybe<THashSet<TStringBuf>> passthroughColumns; + bool analyzeJustMember = true; + if (&lambda.Args().Arg(0).Ref() != &lambda.Body().Ref()) { + auto maybeInnerFlatMap = GetFlatMapOverInputStream(lambda); + if (!maybeInnerFlatMap) { + return {}; + } + + if (!IsPassthroughFlatMap(maybeInnerFlatMap.Cast(), &passthroughColumns, analyzeJustMember)) { + return {}; + } + } + + if (passthroughColumns && + !AllOf(usedKeyFilterColumns, [&](const TStringBuf& col) { return passthroughColumns->contains(col); })) + { + return {}; + } + } + + return maybeOp; + }; + + bool hasUpdates = false; + TVector<TExprBase> updatedSections; + for (auto section: op.Input()) { + bool hasPathUpdates = false; + TVector<TYtPath> updatedPaths; + auto settings = section.Settings().Ptr(); + const EYtSettingType kfType = NYql::HasSetting(*settings, EYtSettingType::KeyFilter2) ? + EYtSettingType::KeyFilter2 : EYtSettingType::KeyFilter; + const auto keyFilters = NYql::GetAllSettingValues(*settings, kfType); + // Non empty filters and without table index + const bool haveNonEmptyKeyFiltersWithoutIndex = + AnyOf(keyFilters, [](const TExprNode::TPtr& f) { return f->ChildrenSize() > 0; }) && + AllOf(keyFilters, [&](const TExprNode::TPtr& f) { return f->ChildrenSize() < GetMinChildrenForIndexedKeyFilter(kfType); }); + + bool allPathUpdated = true; + if (haveNonEmptyKeyFiltersWithoutIndex) { + + TSyncMap syncList; + for (auto filter: keyFilters) { + if (!IsYtCompleteIsolatedLambda(*filter, syncList, false)) { + return node; + } + } + + // TODO: should actually be true for both kf1/kf2 - enforce in ValidateSettings() + YQL_ENSURE(kfType == EYtSettingType::KeyFilter || keyFilters.size() == 1); + const auto kfColumns = GetKeyFilterColumns(section, kfType); + YQL_ENSURE(!kfColumns.empty()); + for (auto path: section.Paths()) { + if (auto maybeOp = getInnerOpForUpdate(path, kfColumns)) { + auto innerOp = maybeOp.Cast(); + if (kfType == EYtSettingType::KeyFilter2) { + // check input/output keyFilter columns are of same type + const TStructExprType* inputType = + innerOp.Input().Item(0).Ref().GetTypeAnn()->Cast<TListExprType>()->GetItemType()->Cast<TStructExprType>(); + const TStructExprType* outputType = + innerOp.Output().Item(0).Ref().GetTypeAnn()->Cast<TListExprType>()->GetItemType()->Cast<TStructExprType>(); + bool sameTypes = true; + for (auto& keyColumn : kfColumns) { + auto inPos = inputType->FindItem(keyColumn); + auto outPos = outputType->FindItem(keyColumn); + YQL_ENSURE(inPos); + YQL_ENSURE(outPos); + const TTypeAnnotationNode* inColumnType = inputType->GetItems()[*inPos]->GetItemType(); + const TTypeAnnotationNode* outColumnType = outputType->GetItems()[*outPos]->GetItemType(); + if (!IsSameAnnotation(*inColumnType, *outColumnType)) { + sameTypes = false; + break; + } + } + + if (!sameTypes) { + // TODO: improve + updatedPaths.push_back(path); + allPathUpdated = false; + continue; + } + } + + auto innerOpSection = innerOp.Input().Item(0); + auto updatedSection = Build<TYtSection>(ctx, innerOpSection.Pos()) + .InitFrom(innerOpSection) + .Settings(NYql::MergeSettings(innerOpSection.Settings().Ref(), *NYql::KeepOnlySettings(section.Settings().Ref(), EYtSettingType::KeyFilter | EYtSettingType::KeyFilter2, ctx), ctx)) + .Done(); + + auto updatedSectionList = Build<TYtSectionList>(ctx, innerOp.Input().Pos()).Add(updatedSection).Done(); + auto updatedInnerOp = ctx.ChangeChild(innerOp.Ref(), TYtTransientOpBase::idx_Input, updatedSectionList.Ptr()); + if (!syncList.empty()) { + updatedInnerOp = ctx.ChangeChild(*updatedInnerOp, TYtTransientOpBase::idx_World, ApplySyncListToWorld(innerOp.World().Ptr(), syncList, ctx)); + } + + updatedPaths.push_back( + Build<TYtPath>(ctx, path.Pos()) + .InitFrom(path) + .Table<TYtOutput>() + .InitFrom(path.Table().Cast<TYtOutput>()) + .Operation(updatedInnerOp) + .Build() + .Done()); + + hasPathUpdates = true; + } else { + updatedPaths.push_back(path); + allPathUpdated = false; + } + } + } + if (hasPathUpdates) { + hasUpdates = true; + if (allPathUpdated) { + settings = NYql::RemoveSettings(*settings, EYtSettingType::KeyFilter | EYtSettingType::KeyFilter2, ctx); + settings = NYql::AddSetting(*settings, kfType, ctx.NewList(section.Pos(), {}), ctx); + } + updatedSections.push_back( + Build<TYtSection>(ctx, section.Pos()) + .InitFrom(section) + .Paths() + .Add(updatedPaths) + .Build() + .Settings(settings) + .Done()); + } else { + updatedSections.push_back(section); + } + } + + if (!hasUpdates) { + return node; + } + + auto sectionList = Build<TYtSectionList>(ctx, op.Input().Pos()) + .Add(updatedSections) + .Done(); + + return TExprBase(ctx.ChangeChild(node.Ref(), TYtTransientOpBase::idx_Input, sectionList.Ptr())); +} + +TMaybeNode<TExprBase> TYtPhysicalOptProposalTransformer::PushDownYtMapOverSortedMerge(TExprBase node, TExprContext& ctx, const TGetParents& getParents) const { + auto map = node.Cast<TYtMap>(); + + if (map.Ref().HasResult()) { + return node; + } + + if (map.Input().Size() > 1 || map.Output().Size() > 1) { + return node; + } + + if (NYql::HasAnySetting(map.Settings().Ref(), EYtSettingType::Sharded | EYtSettingType::JobCount)) { + return node; + } + + if (!NYql::HasSetting(map.Settings().Ref(), EYtSettingType::Ordered)) { + return node; + } + + auto section = map.Input().Item(0); + if (section.Paths().Size() > 1) { + return node; + } + if (NYql::HasSettingsExcept(section.Settings().Ref(), EYtSettingType::KeyFilter | EYtSettingType::KeyFilter2)) { + return node; + } + if (NYql::HasNonEmptyKeyFilter(section)) { + return node; + } + auto path = section.Paths().Item(0); + if (!path.Columns().Maybe<TCoVoid>() || !path.Ranges().Maybe<TCoVoid>()) { + return node; + } + auto maybeMerge = path.Table().Maybe<TYtOutput>().Operation().Maybe<TYtMerge>(); + if (!maybeMerge) { + return node; + } + auto merge = maybeMerge.Cast(); + if (merge.Ref().StartsExecution() || merge.Ref().HasResult()) { + return node; + } + const auto rowSpec = TYqlRowSpecInfo(merge.Output().Item(0).RowSpec()); + if (!rowSpec.IsSorted()) { + return node; + } + TMaybeNode<TExprBase> columns; + if (rowSpec.HasAuxColumns()) { + TSet<TStringBuf> members; + for (auto item: rowSpec.GetType()->GetItems()) { + members.insert(item->GetName()); + } + columns = TExprBase(ToAtomList(members, merge.Pos(), ctx)); + } + + auto mergeSection = merge.Input().Item(0); + if (NYql::HasSettingsExcept(mergeSection.Settings().Ref(), EYtSettingType::KeyFilter | EYtSettingType::KeyFilter2)) { + return node; + } + if (NYql::HasNonEmptyKeyFilter(mergeSection)) { + return node; + } + if (merge.Settings().Size() > 0) { + return node; + } + + const TParentsMap* parentsMap = getParents(); + if (IsOutputUsedMultipleTimes(merge.Ref(), *parentsMap)) { + // Merge output is used more than once + return node; + } + + auto world = map.World().Ptr(); + if (!merge.World().Ref().IsWorld()) { + world = Build<TCoSync>(ctx, map.Pos()).Add(world).Add(merge.World()).Done().Ptr(); + } + TVector<TYtPath> paths; + for (auto path: mergeSection.Paths()) { + auto newPath = Build<TYtPath>(ctx, map.Pos()) + .Table<TYtOutput>() + .Operation<TYtMap>() + .InitFrom(map) + .World(world) + .Input() + .Add() + .Paths() + .Add<TYtPath>() + .InitFrom(path) + .Columns(columns.IsValid() ? columns.Cast() : path.Columns()) + .Build() + .Build() + .Settings(section.Settings()) + .Build() + .Build() + .Build() + .OutIndex().Value("0").Build() + .Build() + .Columns<TCoVoid>().Build() + .Ranges<TCoVoid>().Build() + .Stat<TCoVoid>().Build() + .Done(); + paths.push_back(std::move(newPath)); + } + + return Build<TYtMerge>(ctx, node.Pos()) + .World<TCoWorld>().Build() + .DataSink(merge.DataSink()) + .Output(map.Output()) // Rewrite output type from YtMap + .Input() + .Add() + .Paths() + .Add(paths) + .Build() + .Settings(mergeSection.Settings()) + .Build() + .Build() + .Settings() + .Build() + .Done(); +} + +} // namespace NYql diff --git a/yt/yql/providers/yt/provider/phy_opt/yql_yt_phy_opt_sort.cpp b/yt/yql/providers/yt/provider/phy_opt/yql_yt_phy_opt_sort.cpp new file mode 100644 index 0000000000..9300a67d13 --- /dev/null +++ b/yt/yql/providers/yt/provider/phy_opt/yql_yt_phy_opt_sort.cpp @@ -0,0 +1,694 @@ +#include "yql_yt_phy_opt.h" +#include "yql_yt_phy_opt_helper.h" + +#include <yt/yql/providers/yt/provider/yql_yt_helpers.h> +#include <yt/yql/providers/yt/opt/yql_yt_key_selector.h> +#include <yql/essentials/providers/common/codec/yql_codec_type_flags.h> + +#include <yql/essentials/core/yql_type_helpers.h> +#include <yql/essentials/utils/log/log.h> + +#include <util/generic/xrange.h> + +namespace NYql { + +using namespace NNodes; +using namespace NPrivate; + +TMaybeNode<TExprBase> TYtPhysicalOptProposalTransformer::YtSortOverAlreadySorted(TExprBase node, TExprContext& ctx) const { + auto sort = node.Cast<TYtSort>(); + + if (auto maxTablesForSortedMerge = State_->Configuration->MaxInputTablesForSortedMerge.Get()) { + if (sort.Input().Item(0).Paths().Size() > *maxTablesForSortedMerge) { + return node; + } + } + + const TYqlRowSpecInfo outRowSpec(sort.Output().Item(0).RowSpec()); + + TYqlRowSpecInfo commonSorted = outRowSpec; + auto section = sort.Input().Item(0); + for (auto path: section.Paths()) { + commonSorted.MakeCommonSortness(ctx, *TYtTableBaseInfo::GetRowSpec(path.Table())); + } + + if (outRowSpec.CompareSortness(commonSorted)) { + // input is sorted at least as strictly as output + auto res = ctx.RenameNode(sort.Ref(), TYtMerge::CallableName()); + res = ctx.ChangeChild(*res, TYtMerge::idx_Settings, + Build<TCoNameValueTupleList>(ctx, sort.Pos()) + .Add() + .Name() + .Value(ToString(EYtSettingType::KeepSorted)) + .Build() + .Build() + .Done().Ptr() + ); + section = ClearUnorderedSection(section, ctx); + if (section.Ptr() != sort.Input().Item(0).Ptr()) { + res = ctx.ChangeChild(*res, TYtMerge::idx_Input, Build<TYtSectionList>(ctx, sort.Input().Pos()).Add(section).Done().Ptr()); + } + return TExprBase(res); + } + + return node; +} + +template<bool IsTop> +TMaybeNode<TExprBase> TYtPhysicalOptProposalTransformer::Sort(TExprBase node, TExprContext& ctx) const { + if (State_->Types->EvaluationInProgress || State_->PassiveExecution) { + return node; + } + + const auto sort = node.Cast<std::conditional_t<IsTop, TCoTopBase, TCoSort>>(); + if (!IsYtProviderInput(sort.Input())) { + return node; + } + + auto sortDirections = sort.SortDirections(); + if (!IsConstExpSortDirections(sortDirections)) { + return node; + } + + auto keySelectorLambda = sort.KeySelectorLambda(); + auto cluster = TString{GetClusterName(sort.Input())}; + TSyncMap syncList; + if (!IsYtCompleteIsolatedLambda(keySelectorLambda.Ref(), syncList, cluster, false)) { + return node; + } + + const TStructExprType* outType = nullptr; + if (auto type = GetSequenceItemType(node, false, ctx)) { + outType = type->Cast<TStructExprType>(); + } else { + return {}; + } + + TVector<TYtPathInfo::TPtr> inputInfos = GetInputPaths(sort.Input()); + + TMaybe<NYT::TNode> firstNativeType; + if (!inputInfos.empty()) { + firstNativeType = inputInfos.front()->GetNativeYtType(); + } + auto maybeReadSettings = sort.Input().template Maybe<TCoRight>().Input().template Maybe<TYtReadTable>().Input().Item(0).Settings(); + const ui64 nativeTypeFlags = State_->Configuration->UseNativeYtTypes.Get().GetOrElse(DEFAULT_USE_NATIVE_YT_TYPES) + ? GetNativeYtTypeFlags(*outType) + : 0ul; + const bool needMap = (maybeReadSettings && NYql::HasSetting(maybeReadSettings.Ref(), EYtSettingType::SysColumns)) + || AnyOf(inputInfos, [nativeTypeFlags, firstNativeType] (const TYtPathInfo::TPtr& path) { + return path->RequiresRemap() + || nativeTypeFlags != path->GetNativeYtTypeFlags() + || firstNativeType != path->GetNativeYtType(); + }); + + bool useExplicitColumns = AnyOf(inputInfos, [] (const TYtPathInfo::TPtr& path) { + return !path->Table->IsTemp || (path->Table->RowSpec && path->Table->RowSpec->HasAuxColumns()); + }); + + const bool needMerge = maybeReadSettings && NYql::HasSetting(maybeReadSettings.Ref(), EYtSettingType::Sample); + + const bool useNativeDescSort = State_->Configuration->UseNativeDescSort.Get().GetOrElse(DEFAULT_USE_NATIVE_DESC_SORT); + + TKeySelectorBuilder builder(node.Pos(), ctx, useNativeDescSort, outType); + builder.ProcessKeySelector(keySelectorLambda.Ptr(), sortDirections.Ptr()); + + TYtOutTableInfo sortOut(outType, nativeTypeFlags); + builder.FillRowSpecSort(*sortOut.RowSpec); + sortOut.SetUnique(sort.Ref().template GetConstraint<TDistinctConstraintNode>(), node.Pos(), ctx); + + TExprBase sortInput = sort.Input(); + TExprBase world = TExprBase(ApplySyncListToWorld(NPrivate::GetWorld(sortInput, {}, ctx).Ptr(), syncList, ctx)); + bool unordered = ctx.IsConstraintEnabled<TSortedConstraintNode>(); + if (needMap || builder.NeedMap()) { + auto mapper = builder.MakeRemapLambda(); + + auto mapperClean = CleanupWorld(TCoLambda(mapper), ctx); + if (!mapperClean) { + return {}; + } + + TYtOutTableInfo mapOut(builder.MakeRemapType(), nativeTypeFlags); + mapOut.SetUnique(sort.Ref().template GetConstraint<TDistinctConstraintNode>(), node.Pos(), ctx); + + sortInput = Build<TYtOutput>(ctx, node.Pos()) + .Operation<TYtMap>() + .World(world) + .DataSink(NPrivate::GetDataSink(sort.Input(), ctx)) + .Input(NPrivate::ConvertInputTable(sort.Input(), ctx, NPrivate::TConvertInputOpts().MakeUnordered(unordered))) + .Output() + .Add(mapOut.ToExprNode(ctx, node.Pos()).Cast<TYtOutTable>()) + .Build() + .Settings(GetFlowSettings(node.Pos(), *State_, ctx)) + .Mapper(mapperClean.Cast()) + .Build() + .OutIndex() + .Value(0U) + .Build() + .Done(); + world = TExprBase(ctx.NewWorld(node.Pos())); + unordered = false; + } + else if (needMerge) { + TYtOutTableInfo mergeOut(outType, nativeTypeFlags); + mergeOut.SetUnique(sort.Ref().template GetConstraint<TDistinctConstraintNode>(), node.Pos(), ctx); + if (firstNativeType) { + mergeOut.RowSpec->CopyTypeOrders(*firstNativeType); + sortOut.RowSpec->CopyTypeOrders(*firstNativeType); + } + + NPrivate::TConvertInputOpts opts; + if (useExplicitColumns) { + opts.ExplicitFields(*mergeOut.RowSpec, node.Pos(), ctx); + useExplicitColumns = false; + } + + sortInput = Build<TYtOutput>(ctx, node.Pos()) + .Operation<TYtMerge>() + .World(world) + .DataSink(NPrivate::GetDataSink(sort.Input(), ctx)) + .Input(NPrivate::ConvertInputTable(sort.Input(), ctx, opts.MakeUnordered(unordered))) + .Output() + .Add(mergeOut.ToExprNode(ctx, node.Pos()).Cast<TYtOutTable>()) + .Build() + .Settings() + .Add() + .Name() + .Value(ToString(EYtSettingType::ForceTransform), TNodeFlags::Default) + .Build() + .Build() + .Build() + .Build() + .OutIndex() + .Value(0U) + .Build() + .Done(); + world = TExprBase(ctx.NewWorld(node.Pos())); + unordered = false; + } else if (firstNativeType) { + sortOut.RowSpec->CopyTypeOrders(*firstNativeType); + } + + bool canUseMerge = !needMap && !needMerge; + if (auto maxTablesForSortedMerge = State_->Configuration->MaxInputTablesForSortedMerge.Get()) { + if (inputInfos.size() > *maxTablesForSortedMerge) { + canUseMerge = false; + } + } + + if (canUseMerge) { + TYqlRowSpecInfo commonSorted = *sortOut.RowSpec; + for (auto& pathInfo: inputInfos) { + commonSorted.MakeCommonSortness(ctx, *pathInfo->Table->RowSpec); + } + // input is sorted at least as strictly as output + if (!sortOut.RowSpec->CompareSortness(commonSorted)) { + canUseMerge = false; + } + } + + sortOut.RowSpec->SetConstraints(sort.Ref().GetConstraintSet()); + + NPrivate::TConvertInputOpts opts; + if (useExplicitColumns) { + opts.ExplicitFields(*sortOut.RowSpec, node.Pos(), ctx); + } + + auto res = canUseMerge ? + TExprBase(Build<TYtMerge>(ctx, node.Pos()) + .World(world) + .DataSink(NPrivate::GetDataSink(sortInput, ctx)) + .Input(NPrivate::ConvertInputTable(sortInput, ctx, opts.ClearUnordered())) + .Output() + .Add(sortOut.ToExprNode(ctx, node.Pos()).Cast<TYtOutTable>()) + .Build() + .Settings() + .Add() + .Name() + .Value(ToString(EYtSettingType::KeepSorted), TNodeFlags::Default) + .Build() + .Build() + .Build() + .Done()): + TExprBase(Build<TYtSort>(ctx, node.Pos()) + .World(world) + .DataSink(NPrivate::GetDataSink(sortInput, ctx)) + .Input(NPrivate::ConvertInputTable(sortInput, ctx, opts.MakeUnordered(unordered))) + .Output() + .Add(sortOut.ToExprNode(ctx, node.Pos()).Cast<TYtOutTable>()) + .Build() + .Settings().Build() + .Done()); + + res = Build<TYtOutput>(ctx, node.Pos()) + .Operation(res) + .OutIndex().Value(0U).Build() + .Done(); + + + if constexpr (IsTop) { + res = Build<TCoTake>(ctx, node.Pos()) + .Input(res) + .Count(sort.Count()) + .Done(); + } + + return res; +} + +template TMaybeNode<TExprBase> TYtPhysicalOptProposalTransformer::Sort<true>(TExprBase node, TExprContext& ctx) const; +template TMaybeNode<TExprBase> TYtPhysicalOptProposalTransformer::Sort<false>(TExprBase node, TExprContext& ctx) const; + + +TMaybeNode<TExprBase> TYtPhysicalOptProposalTransformer::TopSort(TExprBase node, TExprContext& ctx) const { + auto sort = node.Cast<TYtSort>(); + + auto settings = sort.Settings(); + auto limitSetting = NYql::GetSetting(settings.Ref(), EYtSettingType::Limit); + if (!limitSetting) { + return node; + } + if (HasNodesToCalculate(node.Ptr())) { + return node; + } + + if (NYql::HasAnySetting(sort.Input().Item(0).Settings().Ref(), EYtSettingType::Take | EYtSettingType::Skip)) { + return node; + } + + if (NYql::HasNonEmptyKeyFilter(sort.Input().Item(0))) { + return node; + } + + const ui64 maxLimit = State_->Configuration->TopSortMaxLimit.Get().GetOrElse(DEFAULT_TOP_SORT_LIMIT); + TMaybe<ui64> limit = GetLimit(settings.Ref()); + if (!limit || *limit == 0 || *limit > maxLimit) { + YQL_CLOG(INFO, ProviderYt) << __FUNCTION__ << " !!! TopSort - zero limit"; + // Keep empty "limit" setting to prevent repeated Limits optimization + if (limitSetting->ChildPtr(1)->ChildrenSize() != 0) { + auto updatedSettings = NYql::RemoveSetting(settings.Ref(), EYtSettingType::Limit, ctx); + updatedSettings = NYql::AddSetting(*updatedSettings, EYtSettingType::Limit, ctx.NewList(node.Pos(), {}), ctx); + return TExprBase(ctx.ChangeChild(node.Ref(), TYtSort::idx_Settings, std::move(updatedSettings))); + } + return node; + } + + ui64 size = 0; + ui64 rows = 0; + for (auto path: sort.Input().Item(0).Paths()) { + auto tableInfo = TYtTableBaseInfo::Parse(path.Table()); + if (!tableInfo->Stat) { + return node; + } + ui64 tableSize = tableInfo->Stat->DataSize; + ui64 tableRows = tableInfo->Stat->RecordsCount; + + if (!path.Ranges().Maybe<TCoVoid>()) { + if (TMaybe<ui64> usedRows = TYtRangesInfo(path.Ranges()).GetUsedRows(tableRows)) { + // Make it proportional to used rows + tableSize = tableSize * usedRows.GetRef() / tableRows; + tableRows = usedRows.GetRef(); + } else { + // non-row ranges are present + return node; + } + } + size += tableSize; + rows += tableRows; + } + + if (rows <= *limit) { + // Just do YtSort + // Keep empty "limit" setting to prevent repeated Limits optimization + auto updatedSettings = NYql::RemoveSetting(settings.Ref(), EYtSettingType::Limit, ctx); + updatedSettings = NYql::AddSetting(*updatedSettings, EYtSettingType::Limit, ctx.NewList(node.Pos(), {}), ctx); + return TExprBase(ctx.ChangeChild(node.Ref(), TYtSort::idx_Settings, std::move(updatedSettings))); + } + + const ui64 sizePerJob = State_->Configuration->TopSortSizePerJob.Get().GetOrElse(128_MB); + const ui64 rowMultiplierPerJob = State_->Configuration->TopSortRowMultiplierPerJob.Get().GetOrElse(10u); + ui64 partsBySize = size / sizePerJob; + ui64 partsByRecords = rows / (rowMultiplierPerJob * limit.GetRef()); + ui64 jobCount = Max<ui64>(Min<ui64>(partsBySize, partsByRecords), 1); + if (partsBySize <= 1 || partsByRecords <= 10) { + jobCount = 1; + } + + auto sortedBy = TYqlRowSpecInfo(sort.Output().Item(0).RowSpec()).GetForeignSort(); + auto updatedSettings = NYql::AddSettingAsColumnPairList(sort.Settings().Ref(), EYtSettingType::SortLimitBy, sortedBy, ctx); + if (jobCount <= 5000) { + updatedSettings = NYql::AddSetting(*updatedSettings, EYtSettingType::JobCount, ctx.NewAtom(sort.Pos(), ToString(jobCount)), ctx); + } + + auto inputItemType = GetSequenceItemType(sort.Input().Item(0), false, ctx); + if (!inputItemType) { + return {}; + } + + if (jobCount == 1) { + updatedSettings = NYql::AddSetting(*updatedSettings, EYtSettingType::KeepSorted, {}, ctx); + + auto mapper = Build<TCoLambda>(ctx, sort.Pos()) + .Args({"stream"}) + .Body<TCoTopSort>() + .Input("stream") + .Count<TCoUint64>() + .Literal() + .Value(ToString(*limit)) + .Build() + .Build() + .SortDirections([&sortedBy] (TExprNodeBuilder& builder) { + auto listBuilder = builder.List(); + for (size_t i: xrange(sortedBy.size())) { + listBuilder.Callable(i, TCoBool::CallableName()) + .Atom(0, sortedBy[i].second ? "True" : "False") + .Seal(); + } + listBuilder.Seal(); + }) + .KeySelectorLambda() + .Args({"item"}) + .Body([&sortedBy] (TExprNodeBuilder& builder) { + auto listBuilder = builder.List(); + for (size_t i: xrange(sortedBy.size())) { + listBuilder.Callable(i, TCoMember::CallableName()) + .Arg(0, "item") + .Atom(1, sortedBy[i].first) + .Seal(); + } + listBuilder.Seal(); + }) + .Build() + .Build().Done(); + + // Final map + return Build<TYtMap>(ctx, sort.Pos()) + .World(sort.World()) + .DataSink(sort.DataSink()) + .Input(sort.Input()) + .Output(sort.Output()) + .Settings(GetFlowSettings(sort.Pos(), *State_, ctx, updatedSettings)) + .Mapper(mapper) + .Done(); + } + + auto mapper = Build<TCoLambda>(ctx, sort.Pos()) + .Args({"stream"}) + .Body<TCoTop>() + .Input("stream") + .Count<TCoUint64>() + .Literal() + .Value(ToString(*limit)) + .Build() + .Build() + .SortDirections([&sortedBy] (TExprNodeBuilder& builder) { + auto listBuilder = builder.List(); + for (size_t i: xrange(sortedBy.size())) { + listBuilder.Callable(i, TCoBool::CallableName()) + .Atom(0, sortedBy[i].second ? "True" : "False") + .Seal(); + } + listBuilder.Seal(); + }) + .KeySelectorLambda() + .Args({"item"}) + .Body([&sortedBy] (TExprNodeBuilder& builder) { + auto listBuilder = builder.List(); + for (size_t i: xrange(sortedBy.size())) { + listBuilder.Callable(i, TCoMember::CallableName()) + .Arg(0, "item") + .Atom(1, sortedBy[i].first) + .Seal(); + } + listBuilder.Seal(); + }) + .Build() + .Build().Done(); + + TYtOutTableInfo outTable(inputItemType->Cast<TStructExprType>(), State_->Configuration->UseNativeYtTypes.Get().GetOrElse(DEFAULT_USE_NATIVE_YT_TYPES) ? NTCF_ALL : NTCF_NONE); + outTable.RowSpec->SetConstraints(sort.Ref().GetConstraintSet()); + + return Build<TYtSort>(ctx, sort.Pos()) + .InitFrom(sort) + .World<TCoWorld>().Build() + .Input() + .Add() + .Paths() + .Add() + .Table<TYtOutput>() + .Operation<TYtMap>() + .World(sort.World()) + .DataSink(sort.DataSink()) + .Input(sort.Input()) + .Output() + .Add(outTable.SetUnique(sort.Ref().GetConstraint<TDistinctConstraintNode>(), sort.Pos(), ctx).ToExprNode(ctx, sort.Pos()).Cast<TYtOutTable>()) + .Build() + .Settings(GetFlowSettings(sort.Pos(), *State_, ctx, updatedSettings)) + .Mapper(mapper) + .Build() + .OutIndex() + .Value(0U) + .Build() + .Build() + .Columns<TCoVoid>().Build() + .Ranges<TCoVoid>().Build() + .Stat<TCoVoid>().Build() + .Build() + .Build() + .Settings() + .Build() + .Build() + .Build() + .Done(); +} + + +TMaybeNode<TExprBase> TYtPhysicalOptProposalTransformer::AssumeConstraints(TExprBase assume, TExprContext& ctx, const TGetParents& getParents) const { + if (State_->Types->EvaluationInProgress || State_->PassiveExecution) { + return assume; + } + + auto input = TExprBase(assume.Ref().HeadPtr()); + if (!IsYtProviderInput(input)) { + return assume; + } + + auto sorted = assume.Ref().GetConstraint<TSortedConstraintNode>(); + + auto maybeOp = input.Maybe<TYtOutput>().Operation(); + bool needSeparateOp = !maybeOp + || maybeOp.Raw()->StartsExecution() + || (maybeOp.Raw()->HasResult() && maybeOp.Raw()->GetResult().Type() == TExprNode::World) + || IsOutputUsedMultipleTimes(maybeOp.Ref(), *getParents()) + || (sorted && maybeOp.Maybe<TYtMapReduce>()) + || (sorted && maybeOp.Maybe<TYtEquiJoin>()); + + bool canMerge = !sorted; + bool equalSort = false; + if (sorted) { + if (auto inputSort = input.Ref().GetConstraint<TSortedConstraintNode>()) { + if (sorted->IsPrefixOf(*inputSort)) { + canMerge = true; + equalSort = sorted->Equals(*inputSort); + } + } + } + if (equalSort && maybeOp.Maybe<TYtSort>() && assume.Ref().GetAllConstraints().size() == 1 /* only sort constraint */) { + return input; + } + + const TStructExprType* outItemType = nullptr; + if (auto type = GetSequenceItemType(assume, false, ctx)) { + outItemType = type->Cast<TStructExprType>(); + } else { + return {}; + } + + const bool useNativeDescSort = State_->Configuration->UseNativeDescSort.Get().GetOrElse(DEFAULT_USE_NATIVE_DESC_SORT); + + THolder<TKeySelectorBuilder> builder; + if (sorted) { + builder = MakeHolder<TKeySelectorBuilder>(assume.Pos(), ctx, useNativeDescSort, outItemType); + builder->ProcessConstraint(*sorted); + needSeparateOp = needSeparateOp || (builder->NeedMap() && !equalSort && !maybeOp.Maybe<TYtDqProcessWrite>()); + } + + if (needSeparateOp) { + TYtOutTableInfo outTable(outItemType, State_->Configuration->UseNativeYtTypes.Get().GetOrElse(DEFAULT_USE_NATIVE_YT_TYPES) ? NTCF_ALL : NTCF_NONE); + outTable.RowSpec->SetConstraints(assume.Ref().GetConstraintSet()); + + if (auto maybeReadSettings = input.Maybe<TCoRight>().Input().Maybe<TYtReadTable>().Input().Item(0).Settings()) { + if (NYql::HasSetting(maybeReadSettings.Ref(), EYtSettingType::SysColumns)) { + canMerge = false; + } + } + auto inputPaths = GetInputPaths(input); + TMaybe<NYT::TNode> firstNativeType; + if (!inputPaths.empty()) { + firstNativeType = inputPaths.front()->GetNativeYtType(); + } + + canMerge = canMerge && AllOf(inputPaths, [&outTable, firstNativeType] (const TYtPathInfo::TPtr& path) { + return !path->RequiresRemap() + && path->GetNativeYtTypeFlags() == outTable.RowSpec->GetNativeYtTypeFlags() + && firstNativeType == path->GetNativeYtType(); + }); + if (canMerge) { + if (sorted) { + outTable.RowSpec->CopySortness(ctx, *inputPaths.front()->Table->RowSpec, TYqlRowSpecInfo::ECopySort::WithDesc); + outTable.RowSpec->ClearSortness(ctx, sorted->GetContent().size()); + } + outTable.SetUnique(assume.Ref().GetConstraint<TDistinctConstraintNode>(), assume.Pos(), ctx); + + if (firstNativeType) { + outTable.RowSpec->CopyTypeOrders(*firstNativeType); + } + + YQL_ENSURE(!sorted || sorted->GetContent().size() == outTable.RowSpec->SortMembers.size()); + const bool useExplicitColumns = AnyOf(inputPaths, [] (const TYtPathInfo::TPtr& path) { + return !path->Table->IsTemp || (path->Table->RowSpec && path->Table->RowSpec->HasAuxColumns()); + }); + + TConvertInputOpts opts; + if (useExplicitColumns) { + opts.ExplicitFields(*outTable.RowSpec, assume.Pos(), ctx); + } + + return Build<TYtOutput>(ctx, assume.Pos()) + .Operation<TYtMerge>() + .World(GetWorld(input, {}, ctx)) + .DataSink(GetDataSink(input, ctx)) + .Input(ConvertInputTable(input, ctx, opts)) + .Output() + .Add(outTable.ToExprNode(ctx, assume.Pos()).Cast<TYtOutTable>()) + .Build() + .Settings() + .Add() + .Name() + .Value(ToString(EYtSettingType::KeepSorted)) + .Build() + .Build() + .Build() + .Build() + .OutIndex().Value(0U).Build() + .Done(); + } + else { + if (builder) { + builder->FillRowSpecSort(*outTable.RowSpec); + } + outTable.SetUnique(assume.Ref().GetConstraint<TDistinctConstraintNode>(), assume.Pos(), ctx); + + TCoLambda mapper = builder && builder->NeedMap() + ? Build<TCoLambda>(ctx, assume.Pos()) + .Args({"stream"}) + .Body<TExprApplier>() + .Apply(TCoLambda(builder->MakeRemapLambda(true))) + .With(0, "stream") + .Build() + .Done() + : Build<TCoLambda>(ctx, assume.Pos()) + .Args({"stream"}) + .Body("stream") + .Done(); + + auto settingsBuilder = Build<TCoNameValueTupleList>(ctx, assume.Pos()); + if (sorted) { + settingsBuilder + .Add() + .Name() + .Value(ToString(EYtSettingType::KeepSorted)) + .Build() + .Build() + .Add() + .Name() + .Value(ToString(EYtSettingType::Ordered)) + .Build() + .Build(); + } + if (State_->Configuration->UseFlow.Get().GetOrElse(DEFAULT_USE_FLOW)) { + settingsBuilder + .Add() + .Name() + .Value(ToString(EYtSettingType::Flow)) + .Build() + .Build(); + } + + return Build<TYtOutput>(ctx, assume.Pos()) + .Operation<TYtMap>() + .World(GetWorld(input, {}, ctx)) + .DataSink(GetDataSink(input, ctx)) + .Input(ConvertInputTable(input, ctx)) + .Output() + .Add(outTable.ToExprNode(ctx, assume.Pos()).Cast<TYtOutTable>()) + .Build() + .Settings(settingsBuilder.Done()) + .Mapper(mapper) + .Build() + .OutIndex().Value(0U).Build() + .Done(); + } + } + + auto op = GetOutputOp(input.Cast<TYtOutput>()); + TExprNode::TPtr newOp = op.Ptr(); + + if (builder && builder->NeedMap() && maybeOp.Maybe<TYtDqProcessWrite>()) { + TNodeOnNodeOwnedMap remaps; + VisitExpr(maybeOp.Cast<TYtDqProcessWrite>().Input().Ptr(), [&builder, &remaps, &ctx](const TExprNode::TPtr& n) { + if (TYtOutput::Match(n.Get())) { + // Stop traversing dependent operations + return false; + } + if (TYtDqWrite::Match(n.Get())) { + auto newInput = Build<TExprApplier>(ctx, n->Pos()) + .Apply(TCoLambda(builder->MakeRemapLambda(true))) + .With(0, TExprBase(n->ChildPtr(TYtDqWrite::idx_Input))) + .Done(); + remaps[n.Get()] = ctx.ChangeChild(*n, TYtDqWrite::idx_Input, newInput.Ptr()); + } + return true; + }); + newOp = ctx.ChangeChild(*newOp, TYtDqProcessWrite::idx_Input, ctx.ReplaceNodes(newOp->ChildPtr(TYtDqProcessWrite::idx_Input), remaps)); + } + + if (!op.Maybe<TYtSort>() && sorted) { + if (auto settings = op.Maybe<TYtTransientOpBase>().Settings()) { + if (!NYql::HasSetting(settings.Ref(), EYtSettingType::KeepSorted)) { + newOp = ctx.ChangeChild(*newOp, TYtTransientOpBase::idx_Settings, NYql::AddSetting(settings.Ref(), EYtSettingType::KeepSorted, {}, ctx)); + } + } else if (auto settings = op.Maybe<TYtFill>().Settings()) { + if (!NYql::HasSetting(settings.Ref(), EYtSettingType::KeepSorted)) { + newOp = ctx.ChangeChild(*newOp, TYtFill::idx_Settings, NYql::AddSetting(settings.Ref(), EYtSettingType::KeepSorted, {}, ctx)); + } + } + } + if (!equalSort) { + const size_t index = FromString(input.Cast<TYtOutput>().OutIndex().Value()); + TYtOutTableInfo outTable(op.Output().Item(index)); + if (builder) { + builder->FillRowSpecSort(*outTable.RowSpec); + } + outTable.RowSpec->SetConstraints(assume.Ref().GetConstraintSet()); + outTable.SetUnique(assume.Ref().GetConstraint<TDistinctConstraintNode>(), assume.Pos(), ctx); + + TVector<TYtOutTable> outputs; + for (size_t i = 0; i < op.Output().Size(); ++i) { + if (index == i) { + outputs.push_back(outTable.ToExprNode(ctx, op.Pos()).Cast<TYtOutTable>()); + } else { + outputs.push_back(op.Output().Item(i)); + } + } + + newOp = ctx.ChangeChild(*newOp, TYtOutputOpBase::idx_Output, Build<TYtOutSection>(ctx, op.Pos()).Add(outputs).Done().Ptr()); + } + + return Build<TYtOutput>(ctx, assume.Pos()) + .Operation(newOp) + .OutIndex(input.Cast<TYtOutput>().OutIndex()) + .Mode(sorted ? TMaybeNode<TCoAtom>() : input.Cast<TYtOutput>().Mode()) + .Done(); +} + +} // namespace NYql diff --git a/yt/yql/providers/yt/provider/phy_opt/yql_yt_phy_opt_weak_fields.cpp b/yt/yql/providers/yt/provider/phy_opt/yql_yt_phy_opt_weak_fields.cpp new file mode 100644 index 0000000000..2b6049df3c --- /dev/null +++ b/yt/yql/providers/yt/provider/phy_opt/yql_yt_phy_opt_weak_fields.cpp @@ -0,0 +1,300 @@ +#include "yql_yt_phy_opt.h" + +#include <yt/yql/providers/yt/provider/yql_yt_helpers.h> +#include <yt/yql/providers/yt/common/yql_names.h> + +#include <yql/essentials/core/yql_opt_utils.h> + +#include <util/generic/xrange.h> + +namespace NYql { + +using namespace NNodes; + +TMaybeNode<TExprBase> TYtPhysicalOptProposalTransformer::WeakFields(TExprBase node, TExprContext& ctx, const TGetParents& getParents) const { + auto op = node.Cast<TYtWithUserJobsOpBase>(); + + if (op.Input().Size() > 1) { + return node; + } + + if (NYql::HasSetting(op.Settings().Ref(), EYtSettingType::WeakFields)) { + return node; + } + + auto section = op.Input().Item(0); + auto inputType = section.Ref().GetTypeAnn()->Cast<TListExprType>()->GetItemType()->Cast<TStructExprType>(); + if (!inputType->FindItem(YqlOthersColumnName)) { + return node; + } + + for (auto path: section.Paths()) { + TYtTableBaseInfo::TPtr info = TYtTableBaseInfo::Parse(path.Table()); + if (!info->RowSpec || info->RowSpec->StrictSchema || info->Meta->Attrs.contains(QB2Premapper)) { + return node; + } + } + + TMaybeNode<TCoLambda> maybeMapper; + if (auto map = op.Maybe<TYtMap>()) { + maybeMapper = map.Mapper(); + } else { + maybeMapper = op.Maybe<TYtMapReduce>().Mapper().Maybe<TCoLambda>(); + } + if (!maybeMapper) { + return node; + } + auto mapper = maybeMapper.Cast(); + auto mapperIdx = op.Maybe<TYtMap>() ? TYtMap::idx_Mapper : TYtMapReduce::idx_Mapper; + + auto lambdaBody = mapper.Body(); + if (!lambdaBody.Maybe<TCoFlatMapBase>() && !lambdaBody.Maybe<TCoCombineCore>()) { + return node; + } + + TVector<TExprBase> stack{lambdaBody}; + auto input = lambdaBody.Cast<TCoInputBase>().Input(); + while (input.Raw() != mapper.Args().Arg(0).Raw()) { + TMaybe<THashSet<TStringBuf>> passthroughFields; + if (!input.Maybe<TCoFlatMapBase>() + || !IsPassthroughFlatMap(input.Cast<TCoFlatMapBase>(), &passthroughFields) + || (passthroughFields && !passthroughFields->contains(YqlOthersColumnName))) + { + return node; + } + stack.push_back(input); + input = input.Cast<TCoFlatMapBase>().Input(); + } + + auto getMemberColumn = [] (const TExprNode* node, const TExprNode* arg) { + if (auto maybeMember = TMaybeNode<TCoMember>(node)) { + if (maybeMember.Cast().Struct().Raw() == arg) { + return maybeMember.Cast().Name().Value(); + } + } + return TStringBuf(); + }; + + THashMap<const TExprNode*, TExprNode::TPtr> weaks; // map TryWeakMemberFromDict -> row argument + THashSet<const TExprNode*> otherMembers; + TSet<TStringBuf> finalWeakColumns; + const TParentsMap* parentsMap = getParents(); + for (size_t pass: xrange(stack.size())) { + auto consumer = stack[pass]; + + TExprNode::TListType rowArgs; + if (auto maybeCombineCore = consumer.Maybe<TCoCombineCore>()) { + auto combineCore = maybeCombineCore.Cast(); + rowArgs.push_back(combineCore.KeyExtractor().Args().Arg(0).Ptr()); + rowArgs.push_back(combineCore.InitHandler().Args().Arg(1).Ptr()); + rowArgs.push_back(combineCore.UpdateHandler().Args().Arg(1).Ptr()); + } + else { + rowArgs.push_back(consumer.Cast<TCoFlatMapBase>().Lambda().Args().Arg(0).Ptr()); + } + + for (const auto& rowArg : rowArgs) { + auto rowArgParentsIt = parentsMap->find(rowArg.Get()); + YQL_ENSURE(rowArgParentsIt != parentsMap->end()); + for (const auto& memberNode: rowArgParentsIt->second) { + if (auto column = getMemberColumn(memberNode, rowArg.Get())) { + if (column != YqlOthersColumnName) { + continue; + } + } else { + return node; + } + + if (pass > 0) { + otherMembers.insert(memberNode); + } + + auto justArgIt = parentsMap->find(memberNode); + YQL_ENSURE(justArgIt != parentsMap->end()); + for (const auto& justNode: justArgIt->second) { + if (!justNode->IsCallable("Just") || justNode->Child(0) != memberNode) { + if (pass > 0) { + continue; + } + return node; + } + + auto weakIt = parentsMap->find(justNode); + YQL_ENSURE(weakIt != parentsMap->end()); + for (const auto& weakNode : weakIt->second) { + if (!weakNode->IsCallable("TryWeakMemberFromDict") || weakNode->Child(0) != justNode) { + if (pass > 0) { + continue; + } + return node; + } + + weaks.insert(std::make_pair(weakNode, rowArg)); + if (pass == 0) { + finalWeakColumns.insert(weakNode->Child(3)->Content()); + } + } + } + } + } + } + + TSet<TStringBuf> filteredFields; + for (auto item: inputType->GetItems()) { + if (item->GetName() != YqlOthersColumnName) { + filteredFields.insert(item->GetName()); + } + } + + TSet<TStringBuf> weakFields; + TExprNode::TPtr newLambda; + TOptimizeExprSettings settings(State_->Types); + settings.VisitChanges = true; + auto status = OptimizeExpr(mapper.Ptr(), newLambda, [&](const TExprNode::TPtr& input, TExprContext& ctx) { + if (auto maybeTryWeak = TMaybeNode<TCoTryWeakMemberFromDict>(input)) { + auto it = weaks.find(input.Get()); + if (it == weaks.end()) { + return input; + } + auto tryWeak = maybeTryWeak.Cast(); + auto weakName = tryWeak.Name().Value(); + if (!filteredFields.contains(weakName)) { + weakFields.insert(weakName); + } + + TExprBase member = Build<TCoMember>(ctx, input->Pos()) + .Struct(it->second) + .Name(tryWeak.Name()) + .Done(); + + const TStructExprType* structType = it->second->GetTypeAnn()->Cast<TStructExprType>(); + auto structMemberPos = structType->FindItem(weakName); + bool notYsonMember = false; + if (structMemberPos) { + auto structMemberType = structType->GetItems()[*structMemberPos]->GetItemType(); + if (structMemberType->GetKind() == ETypeAnnotationKind::Optional) { + structMemberType = structMemberType->Cast<TOptionalExprType>()->GetItemType(); + } + if (structMemberType->GetKind() != ETypeAnnotationKind::Data) { + notYsonMember = true; + } else { + auto structMemberSlot = structMemberType->Cast<TDataExprType>()->GetSlot(); + if (structMemberSlot != EDataSlot::Yson && structMemberSlot != EDataSlot::String) { + notYsonMember = true; + } + } + } + + TExprBase fromYson = (notYsonMember || tryWeak.Type().Value() == "Yson") + ? member + : Build<TCoFromYsonSimpleType>(ctx, input->Pos()) + .Value(member) + .Type(tryWeak.Type()) + .Done(); + + if (tryWeak.RestDict().Maybe<TCoNothing>()) { + return fromYson.Ptr(); + } + + return Build<TCoCoalesce>(ctx, input->Pos()) + .Predicate(fromYson) + .Value<TCoTryWeakMemberFromDict>() + .InitFrom(tryWeak) + .OtherDict<TCoNull>() + .Build() + .Build() + .Done().Ptr(); + } + + if (stack.size() > 1) { + if (auto maybeStruct = TMaybeNode<TCoAsStruct>(input)) { + auto asStruct = maybeStruct.Cast(); + for (size_t i: xrange(asStruct.ArgCount())) { + auto list = asStruct.Arg(i); + if (list.Item(0).Cast<TCoAtom>().Value() == YqlOthersColumnName && otherMembers.contains(list.Item(1).Raw())) { + // rebuild AsStruct without other fields + auto row = list.Item(1).Cast<TCoMember>().Struct(); + auto newChildren = input->ChildrenList(); + newChildren.erase(newChildren.begin() + i); + // and with weak fields for combiner core + for (ui32 j = 0; j < newChildren.size(); ++j) { + finalWeakColumns.erase(newChildren[j]->Child(0)->Content()); + } + + for (auto column : finalWeakColumns) { + newChildren.push_back(Build<TExprList>(ctx, input->Pos()) + .Add<TCoAtom>() + .Value(column) + .Build() + .Add<TCoMember>() + .Struct(row) + .Name() + .Value(column) + .Build() + .Build() + .Done().Ptr()); + } + + return ctx.ChangeChildren(*input, std::move(newChildren)); + } + } + + return input; + } + } + + return input; + }, ctx, settings); + + if (status.Level == IGraphTransformer::TStatus::Error) { + return nullptr; + } + + // refresh CombineCore lambdas + auto child = newLambda->Child(TCoLambda::idx_Body); + if (TCoCombineCore::Match(child)) { + child->ChildRef(TCoCombineCore::idx_KeyExtractor) = ctx.DeepCopyLambda(*child->Child(TCoCombineCore::idx_KeyExtractor)); + child->ChildRef(TCoCombineCore::idx_InitHandler) = ctx.DeepCopyLambda(*child->Child(TCoCombineCore::idx_InitHandler)); + child->ChildRef(TCoCombineCore::idx_UpdateHandler) = ctx.DeepCopyLambda(*child->Child(TCoCombineCore::idx_UpdateHandler)); + } + + // refresh flatmaps too + if (stack.size() > 1) { + child = child->Child(TCoInputBase::idx_Input); + for (size_t i = 1; i < stack.size(); ++i) { + child->ChildRef(TCoFlatMapBase::idx_Lambda) = ctx.DeepCopyLambda(*child->Child(TCoFlatMapBase::idx_Lambda)); + child = child->Child(TCoInputBase::idx_Input); + } + } + + auto columnsBuilder = Build<TExprList>(ctx, op.Pos()); + for (auto& field: filteredFields) { + columnsBuilder + .Add<TCoAtom>() + .Value(field) + .Build(); + } + for (auto& field: weakFields) { + columnsBuilder + .Add<TCoAtomList>() + .Add() + .Value(field) + .Build() + .Add() + .Value("weak") + .Build() + .Build(); + } + + auto res = ctx.ChangeChild(op.Ref(), TYtWithUserJobsOpBase::idx_Input, + Build<TYtSectionList>(ctx, op.Input().Pos()) + .Add(UpdateInputFields(section, columnsBuilder.Done(), ctx)) + .Done().Ptr()); + + res = ctx.ChangeChild(*res, TYtWithUserJobsOpBase::idx_Settings, NYql::AddSetting(op.Settings().Ref(), EYtSettingType::WeakFields, {}, ctx)); + res = ctx.ChangeChild(*res, mapperIdx, std::move(newLambda)); + + return TExprBase(res); +} + +} // namespace NYql diff --git a/yt/yql/providers/yt/provider/phy_opt/yql_yt_phy_opt_write.cpp b/yt/yql/providers/yt/provider/phy_opt/yql_yt_phy_opt_write.cpp new file mode 100644 index 0000000000..77e89116a0 --- /dev/null +++ b/yt/yql/providers/yt/provider/phy_opt/yql_yt_phy_opt_write.cpp @@ -0,0 +1,919 @@ +#include "yql_yt_phy_opt.h" +#include "yql_yt_phy_opt_helper.h" + +#include <yt/yql/providers/yt/provider/yql_yt_helpers.h> +#include <yt/yql/providers/yt/provider/yql_yt_optimize.h> +#include <yt/yql/providers/yt/opt/yql_yt_key_selector.h> +#include <yql/providers/stat/expr_nodes/yql_stat_expr_nodes.h> + +#include <yql/essentials/core/dq_expr_nodes/dq_expr_nodes.h> +#include <yql/essentials/core/dqs_expr_nodes/dqs_expr_nodes.h> +#include <yql/essentials/core/yql_opt_utils.h> +#include <yql/essentials/core/yql_type_helpers.h> +#include <yql/essentials/providers/common/codec/yql_codec_type_flags.h> + +namespace NYql { + +using namespace NNodes; +using namespace NNodes::NDq; +using namespace NPrivate; + +TMaybeNode<TExprBase> TYtPhysicalOptProposalTransformer::DqWrite(TExprBase node, TExprContext& ctx, IOptimizationContext& optCtx, const TGetParents& getParents) const { + if (State_->PassiveExecution || !State_->DqHelper) { + return node; + } + + auto write = node.Cast<TYtWriteTable>(); + if (!TDqCnUnionAll::Match(write.Content().Raw())) { + return node; + } + + const TStructExprType* outItemType; + if (auto type = GetSequenceItemType(write.Content(), false, ctx)) { + outItemType = type->Cast<TStructExprType>(); + } else { + return node; + } + + if (!State_->DqHelper->IsSingleConsumerConnection(write.Content().Ptr(), *getParents())) { + return node; + } + + TSyncMap syncList; + if (!IsYtCompleteIsolatedLambda(write.Content().Ref(), syncList, true)) { + return node; + } + + const ui64 nativeTypeFlags = State_->Configuration->UseNativeYtTypes.Get().GetOrElse(DEFAULT_USE_NATIVE_YT_TYPES) ? NTCF_ALL : NTCF_NONE; + TYtOutTableInfo outTable(outItemType, nativeTypeFlags); + + const auto dqUnion = write.Content().Cast<TDqCnUnionAll>(); + + TMaybeNode<TCoSort> sort; + TMaybeNode<TCoTopSort> topSort; + TMaybeNode<TDqCnMerge> mergeConnection; + auto topLambdaBody = dqUnion.Output().Stage().Program().Body(); + + // Look for the sort-only stage or DcCnMerge connection. + bool sortOnlyLambda = true; + auto& topNode = SkipCallables(topLambdaBody.Ref(), {"ToFlow", "FromFlow", "ToStream"}); + if (auto maybeSort = TMaybeNode<TCoSort>(&topNode)) { + sort = maybeSort; + } else if (auto maybeTopSort = TMaybeNode<TCoTopSort>(&topNode)) { + topSort = maybeTopSort; + } else { + sortOnlyLambda = false; + if (auto inputs = dqUnion.Output().Stage().Inputs(); inputs.Size() == 1 && inputs.Item(0).Maybe<TDqCnMerge>().IsValid()) { + if (SkipCallables(topNode, {"Skip", "Take"}).IsArgument()) { + mergeConnection = inputs.Item(0).Maybe<TDqCnMerge>(); + } else if (topNode.IsCallable(TDqReplicate::CallableName()) && topNode.Head().IsArgument()) { + auto ndx = FromString<size_t>(dqUnion.Output().Index().Value()); + YQL_ENSURE(ndx + 1 < topNode.ChildrenSize()); + if (&topNode.Child(ndx + 1)->Head().Head() == &topNode.Child(ndx + 1)->Tail()) { // trivial lambda + mergeConnection = inputs.Item(0).Maybe<TDqCnMerge>(); + } + } + } + } + + if (sortOnlyLambda) { + auto& bottomNode = SkipCallables(topNode.Head(), {"ToFlow", "FromFlow", "ToStream"}); + sortOnlyLambda = bottomNode.IsArgument(); + } + + TCoLambda writeLambda = Build<TCoLambda>(ctx, write.Pos()) + .Args({"stream"}) + .Body<TDqWrite>() + .Input("stream") + .Provider().Value(YtProviderName).Build() + .Settings().Build() + .Build() + .Done(); + + if (sortOnlyLambda && (sort || topSort)) { + // Add Unordered callable to kill sort in a stage. Sorting will be handled by YtSort. + writeLambda = Build<TCoLambda>(ctx, write.Pos()) + .Args({"stream"}) + .Body<TExprApplier>() + .Apply(writeLambda) + .With<TCoUnordered>(0) + .Input("stream") + .Build() + .Build() + .Done(); + } + + auto result = State_->DqHelper->PushLambdaAndCreateCnResult(dqUnion.Ptr(), writeLambda.Ptr(), write.Pos(), ctx, optCtx); + if (!result) { + return {}; + } + + result = YtCleanupWorld(result, ctx, State_); + + auto writeOp = Build<TYtDqProcessWrite>(ctx, write.Pos()) + .World(ApplySyncListToWorld(ctx.NewWorld(write.Pos()), syncList, ctx)) + .DataSink(write.DataSink().Ptr()) + .Output() + .Add(outTable.ToExprNode(ctx, write.Pos()).Cast<TYtOutTable>()) + .Build() + .Input(result) + .Done().Ptr(); + + auto writeOutput = Build<TYtOutput>(ctx, write.Pos()) + .Operation(writeOp) + .OutIndex().Value(0U).Build() + .Done().Ptr(); + + if (sort) { + writeOutput = Build<TCoSort>(ctx, write.Pos()) + .Input(writeOutput) + .SortDirections(sort.SortDirections().Cast()) + .KeySelectorLambda(sort.KeySelectorLambda().Cast()) + .Done().Ptr(); + } else if (topSort) { + writeOutput = Build<TCoTopSort>(ctx, write.Pos()) + .Input(writeOutput) + .Count(topSort.Count().Cast()) + .SortDirections(topSort.SortDirections().Cast()) + .KeySelectorLambda(topSort.KeySelectorLambda().Cast()) + .Done().Ptr(); + } else if (mergeConnection) { + writeOutput = Build<TCoSort>(ctx, write.Pos()) + .Input(writeOutput) + .SortDirections( + ctx.Builder(write.Pos()) + .List() + .Do([&](TExprNodeBuilder& parent) -> TExprNodeBuilder& { + size_t index = 0; + for (auto col: mergeConnection.Cast().SortColumns()) { + parent.Callable(index++, "Bool") + .Atom(0, col.SortDirection().Value() == "Asc" ? "1" : "0", TNodeFlags::Default) + .Seal(); + } + return parent; + }) + .Seal() + .Build() + ) + .KeySelectorLambda( + ctx.Builder(write.Pos()) + .Lambda() + .Param("item") + .List() + .Do([&](TExprNodeBuilder& parent) -> TExprNodeBuilder& { + size_t index = 0; + for (auto col: mergeConnection.Cast().SortColumns()) { + parent.Callable(index++, "Member") + .Arg(0, "item") + .Atom(1, col.Column().Value()) + .Seal(); + } + return parent; + }) + .Seal() + .Seal() + .Build() + ) + .Done().Ptr(); + } + + return ctx.ChangeChild(write.Ref(), TYtWriteTable::idx_Content, std::move(writeOutput)); +} + +TMaybeNode<TExprBase> TYtPhysicalOptProposalTransformer::DqMaterialize(TExprBase node, TExprContext& ctx, IOptimizationContext& optCtx, const TGetParents& getParents) const { + if (State_->PassiveExecution || !State_->DqHelper) { + return node; + } + + auto materialize = node.Cast<TYtMaterialize>(); + + if (!TDqCnUnionAll::Match(materialize.Input().Raw())) { + return node; + } + + const TStructExprType* outItemType; + if (auto type = GetSequenceItemType(materialize.Input(), false, ctx)) { + outItemType = type->Cast<TStructExprType>(); + } else { + return node; + } + + if (!State_->DqHelper->IsSingleConsumerConnection(materialize.Input().Ptr(), *getParents())) { + return node; + } + + TSyncMap syncList; + if (!IsYtCompleteIsolatedLambda(materialize.Input().Ref(), syncList, true)) { + return node; + } + + TCoLambda writeLambda = Build<TCoLambda>(ctx, materialize.Pos()) + .Args({"stream"}) + .Body<TDqWrite>() + .Input("stream") + .Provider().Value(YtProviderName).Build() + .Settings().Build() + .Build() + .Done(); + + const ui64 nativeTypeFlags = State_->Configuration->UseNativeYtTypes.Get().GetOrElse(DEFAULT_USE_NATIVE_YT_TYPES) ? NTCF_ALL : NTCF_NONE; + TYtOutTableInfo outTable(outItemType, nativeTypeFlags); + + if (auto sorted = materialize.Input().Ref().GetConstraint<TSortedConstraintNode>()) { + const bool useNativeDescSort = State_->Configuration->UseNativeDescSort.Get().GetOrElse(DEFAULT_USE_NATIVE_DESC_SORT); + TKeySelectorBuilder builder(materialize.Pos(), ctx, useNativeDescSort, outItemType); + builder.ProcessConstraint(*sorted); + builder.FillRowSpecSort(*outTable.RowSpec); + + if (builder.NeedMap()) { + writeLambda = Build<TCoLambda>(ctx, materialize.Pos()) + .Args({"stream"}) + .Body<TExprApplier>() + .Apply(writeLambda) + .With<TExprApplier>(0) + .Apply(TCoLambda(builder.MakeRemapLambda(true))) + .With(0, "stream") + .Build() + .Build() + .Done(); + } + } + + outTable.RowSpec->SetConstraints(materialize.Input().Ref().GetConstraintSet()); + outTable.SetUnique(materialize.Input().Ref().GetConstraint<TDistinctConstraintNode>(), node.Pos(), ctx); + + const auto dqUnion = materialize.Input().Cast<TDqCnUnionAll>(); + + auto result = State_->DqHelper->PushLambdaAndCreateCnResult(dqUnion.Ptr(), writeLambda.Ptr(), materialize.Pos(), ctx, optCtx); + if (!result) { + return {}; + } + + result = YtCleanupWorld(result, ctx, State_); + + auto writeOp = Build<TYtDqProcessWrite>(ctx, materialize.Pos()) + .World(ApplySyncListToWorld(materialize.World().Ptr(), syncList, ctx)) + .DataSink(materialize.DataSink().Ptr()) + .Output() + .Add(outTable.ToExprNode(ctx, materialize.Pos()).Cast<TYtOutTable>()) + .Build() + .Input(result) + .Done().Ptr(); + + return Build<TYtOutput>(ctx, materialize.Pos()) + .Operation(writeOp) + .OutIndex().Value(0U).Build() + .Done(); +} + +TMaybeNode<TExprBase> TYtPhysicalOptProposalTransformer::YtDqProcessWrite(TExprBase node, TExprContext& ctx) const { + const auto& write = node.Cast<TYtDqProcessWrite>(); + if (const auto& contents = FindNodes(write.Input().Ptr(), + [] (const TExprNode::TPtr& node) { return !TYtOutputOpBase::Match(node.Get()); }, + [] (const TExprNode::TPtr& node) { return node->IsCallable({TCoToFlow::CallableName(), TCoIterator::CallableName()}) && node->Head().IsCallable(TYtTableContent::CallableName()); }); + !contents.empty()) { + TNodeOnNodeOwnedMap replaces(contents.size()); + const bool addToken = !State_->Configuration->Auth.Get().GetOrElse(TString()).empty(); + + for (const auto& cont : contents) { + const TYtTableContent content(cont->HeadPtr()); + auto input = content.Input(); + const auto output = input.Maybe<TYtOutput>(); + const auto structType = GetSeqItemType(output ? output.Cast().Ref().GetTypeAnn() : input.Ref().GetTypeAnn()->Cast<TTupleExprType>()->GetItems().back())->Cast<TStructExprType>(); + if (output) { + input = ConvertContentInputToRead(output.Cast(), {}, ctx); + } + TMaybeNode<TCoSecureParam> secParams; + if (addToken) { + const auto cluster = input.Cast<TYtReadTable>().DataSource().Cluster(); + secParams = Build<TCoSecureParam>(ctx, node.Pos()).Name().Build(TString("cluster:default_").append(cluster)).Done(); + } + + TExprNode::TListType flags; + if (!NYql::HasSetting(content.Settings().Ref(), EYtSettingType::Split)) + flags.emplace_back(ctx.NewAtom(cont->Pos(), "Solid", TNodeFlags::Default)); + + const auto read = Build<TDqReadWideWrap>(ctx, cont->Pos()) + .Input(input) + .Flags().Add(std::move(flags)).Build() + .Token(secParams) + .Done(); + + auto narrow = ctx.Builder(cont->Pos()) + .Callable("NarrowMap") + .Add(0, read.Ptr()) + .Lambda(1) + .Params("fields", structType->GetSize()) + .Callable(TCoAsStruct::CallableName()) + .Do([&](TExprNodeBuilder& parent) -> TExprNodeBuilder& { + ui32 i = 0U; + for (const auto& item : structType->GetItems()) { + parent.List(i) + .Atom(0, item->GetName()) + .Arg(1, "fields", i) + .Seal(); + ++i; + } + return parent; + }) + .Seal() + .Seal() + .Seal().Build(); + + narrow = ctx.WrapByCallableIf(cont->IsCallable(TCoIterator::CallableName()), TCoFromFlow::CallableName(), std::move(narrow)); + replaces.emplace(cont.Get(), std::move(narrow)); + } + + return Build<TYtDqProcessWrite>(ctx, write.Pos()) + .InitFrom(write) + .Input(ctx.ReplaceNodes(write.Input().Ptr(), replaces)) + .Done(); + } + + return node; +} + +TMaybeNode<TExprBase> TYtPhysicalOptProposalTransformer::Write(TExprBase node, TExprContext& ctx) const { + auto write = node.Cast<TYtWriteTable>(); + if (!IsYtProviderInput(write.Content())) { + return node; + } + + auto cluster = TString{write.DataSink().Cluster().Value()}; + auto srcCluster = GetClusterName(write.Content()); + if (cluster != srcCluster) { + ctx.AddError(TIssue(ctx.GetPosition(node.Pos()), TStringBuilder() + << "Result from cluster " << TString{srcCluster}.Quote() + << " cannot be written to a different destination cluster " << cluster.Quote())); + return {}; + } + + TVector<TYtPathInfo::TPtr> inputPaths = GetInputPaths(write.Content()); + TYtTableInfo::TPtr outTableInfo = MakeIntrusive<TYtTableInfo>(write.Table()); + + const auto mode = NYql::GetSetting(write.Settings().Ref(), EYtSettingType::Mode); + const bool renew = !mode || FromString<EYtWriteMode>(mode->Child(1)->Content()) == EYtWriteMode::Renew; + const bool flush = mode && FromString<EYtWriteMode>(mode->Child(1)->Content()) == EYtWriteMode::Flush; + const bool transactionalOverrideTarget = NYql::GetSetting(write.Settings().Ref(), EYtSettingType::Initial) + && !flush && (renew || !outTableInfo->Meta->DoesExist); + + const TStructExprType* outItemType = nullptr; + if (auto type = GetSequenceItemType(write.Content(), false, ctx)) { + outItemType = type->Cast<TStructExprType>(); + } else { + return {}; + } + + auto maybeReadSettings = write.Content().Maybe<TCoRight>().Input().Maybe<TYtReadTable>().Input().Item(0).Settings(); + + const TYtTableDescription& nextDescription = State_->TablesData->GetTable(cluster, outTableInfo->Name, outTableInfo->CommitEpoch); + const ui64 nativeTypeFlags = nextDescription.RowSpec->GetNativeYtTypeFlags(); + + TMaybe<NYT::TNode> firstNativeType; + ui64 firstNativeTypeFlags = 0; + if (!inputPaths.empty()) { + firstNativeType = inputPaths.front()->GetNativeYtType(); + firstNativeTypeFlags = inputPaths.front()->GetNativeYtTypeFlags(); + } + + bool requiresMap = (maybeReadSettings && NYql::HasSetting(maybeReadSettings.Ref(), EYtSettingType::SysColumns)) + || AnyOf(inputPaths, [firstNativeType] (const TYtPathInfo::TPtr& path) { + return path->RequiresRemap() || firstNativeType != path->GetNativeYtType(); + }); + + const bool requiresMerge = !requiresMap && ( + AnyOf(inputPaths, [] (const TYtPathInfo::TPtr& path) { + return path->Ranges || path->HasColumns() || path->Table->Meta->IsDynamic || path->Table->FromNode.Maybe<TYtTable>(); + }) + || (maybeReadSettings && NYql::HasAnySetting(maybeReadSettings.Ref(), + EYtSettingType::Take | EYtSettingType::Skip | EYtSettingType::KeyFilter | EYtSettingType::KeyFilter2 | EYtSettingType::Sample)) + || (nextDescription.RowSpec->GetColumnOrder().Defined() && AnyOf(inputPaths, [colOrder = *nextDescription.RowSpec->GetColumnOrder()] (const TYtPathInfo::TPtr& path) { + return path->Table->RowSpec->GetColumnOrder().Defined() && *path->Table->RowSpec->GetColumnOrder() != colOrder; + })) + ); + + TMaybeNode<TCoAtom> outMode; + if (ctx.IsConstraintEnabled<TSortedConstraintNode>() && maybeReadSettings && NYql::HasSetting(maybeReadSettings.Ref(), EYtSettingType::Unordered)) { + outMode = Build<TCoAtom>(ctx, write.Pos()).Value(ToString(EYtSettingType::Unordered)).Done(); + } + + TVector<TYtOutput> publishInput; + if (requiresMap || requiresMerge) { + TExprNode::TPtr mapper; + if (requiresMap) { + mapper = Build<TCoLambda>(ctx, write.Pos()) + .Args({"stream"}) + .Body("stream") + .Done().Ptr(); + } + + // For YtMerge passthrough native flags as is. AlignPublishTypes optimizer will add additional remapping + TYtOutTableInfo outTable(outItemType, requiresMerge ? firstNativeTypeFlags : nativeTypeFlags); + if (firstNativeType) { + outTable.RowSpec->CopyTypeOrders(*firstNativeType); + } + auto settingsBuilder = Build<TCoNameValueTupleList>(ctx, write.Pos()); + bool useExplicitColumns = requiresMerge && AnyOf(inputPaths, [] (const TYtPathInfo::TPtr& path) { + return !path->Table->IsTemp; + }); + if (requiresMap) { + if (ctx.IsConstraintEnabled<TSortedConstraintNode>()) { + if (auto sorted = write.Content().Ref().GetConstraint<TSortedConstraintNode>()) { + const bool useNativeDescSort = State_->Configuration->UseNativeDescSort.Get().GetOrElse(DEFAULT_USE_NATIVE_DESC_SORT); + TKeySelectorBuilder builder(write.Pos(), ctx, useNativeDescSort, outItemType); + builder.ProcessConstraint(*sorted); + builder.FillRowSpecSort(*outTable.RowSpec); + + if (builder.NeedMap()) { + mapper = ctx.Builder(write.Pos()) + .Lambda() + .Param("stream") + .Apply(builder.MakeRemapLambda(true)) + .With(0) + .Apply(mapper) + .With(0, "stream") + .Seal() + .Done() + .Seal() + .Seal() + .Build(); + } + } + } else { + if (inputPaths.size() == 1 && inputPaths.front()->Table->RowSpec && inputPaths.front()->Table->RowSpec->IsSorted()) { + outTable.RowSpec->CopySortness(ctx, *inputPaths.front()->Table->RowSpec); + } + } + } + else { // requiresMerge + // TODO: should we keep sort if multiple inputs? + if (outMode || AnyOf(inputPaths, [] (const TYtPathInfo::TPtr& path) { return path->Table->IsUnordered; })) { + useExplicitColumns = useExplicitColumns || AnyOf(inputPaths, [] (const TYtPathInfo::TPtr& path) { return path->Table->RowSpec->HasAuxColumns(); }); + } + else { + const bool exactCopySort = inputPaths.size() == 1 && !inputPaths.front()->HasColumns(); + bool hasAux = inputPaths.front()->Table->RowSpec->HasAuxColumns(); + bool sortIsChanged = inputPaths.front()->Table->IsUnordered + ? inputPaths.front()->Table->RowSpec->IsSorted() + : outTable.RowSpec->CopySortness(ctx, *inputPaths.front()->Table->RowSpec, + exactCopySort ? TYqlRowSpecInfo::ECopySort::Exact : TYqlRowSpecInfo::ECopySort::WithDesc); + useExplicitColumns = useExplicitColumns || (inputPaths.front()->HasColumns() && hasAux); + + for (size_t i = 1; i < inputPaths.size(); ++i) { + sortIsChanged = outTable.RowSpec->MakeCommonSortness(ctx, *inputPaths[i]->Table->RowSpec) || sortIsChanged; + const bool tableHasAux = inputPaths[i]->Table->RowSpec->HasAuxColumns(); + hasAux = hasAux || tableHasAux; + if (inputPaths[i]->HasColumns() && tableHasAux) { + useExplicitColumns = true; + } + } + useExplicitColumns = useExplicitColumns || (sortIsChanged && hasAux); + } + + if (maybeReadSettings && NYql::HasSetting(maybeReadSettings.Ref(), EYtSettingType::Sample)) { + settingsBuilder.Add() + .Name().Value(ToString(EYtSettingType::ForceTransform)).Build() + .Build(); + } + } + outTable.SetUnique(write.Content().Ref().GetConstraint<TDistinctConstraintNode>(), write.Pos(), ctx); + outTable.RowSpec->SetConstraints(write.Content().Ref().GetConstraintSet()); + + TMaybeNode<TYtTransientOpBase> op; + if (requiresMap) { + if (outTable.RowSpec->IsSorted()) { + settingsBuilder + .Add() + .Name() + .Value(ToString(EYtSettingType::Ordered)) + .Build() + .Build(); + } + if (State_->Configuration->UseFlow.Get().GetOrElse(DEFAULT_USE_FLOW)) { + settingsBuilder + .Add() + .Name() + .Value(ToString(EYtSettingType::Flow)) + .Build() + .Build(); + } + + op = Build<TYtMap>(ctx, write.Pos()) + .World(GetWorld(write.Content(), {}, ctx)) + .DataSink(write.DataSink()) + .Input(ConvertInputTable(write.Content(), ctx)) + .Output() + .Add(outTable.ToExprNode(ctx, write.Pos()).Cast<TYtOutTable>()) + .Build() + .Settings(settingsBuilder.Done()) + .Mapper(mapper) + .Done(); + } + else { + TConvertInputOpts opts; + if (useExplicitColumns) { + opts.ExplicitFields(*outTable.RowSpec, write.Pos(), ctx); + } + op = Build<TYtMerge>(ctx, write.Pos()) + .World(GetWorld(write.Content(), {}, ctx)) + .DataSink(write.DataSink()) + .Input(ConvertInputTable(write.Content(), ctx, opts)) + .Output() + .Add(outTable.ToExprNode(ctx, write.Pos()).Cast<TYtOutTable>()) + .Build() + .Settings(settingsBuilder.Done()) + .Done(); + } + + publishInput.push_back(Build<TYtOutput>(ctx, write.Pos()) + .Operation(op.Cast()) + .OutIndex().Value(0U).Build() + .Mode(outMode) + .Done()); + } + else { + if (auto out = write.Content().Maybe<TYtOutput>()) { + publishInput.push_back(out.Cast()); + } else { + for (auto path: write.Content().Cast<TCoRight>().Input().Cast<TYtReadTable>().Input().Item(0).Paths()) { + publishInput.push_back(Build<TYtOutput>(ctx, path.Table().Pos()) + .InitFrom(path.Table().Cast<TYtOutput>()) + .Mode(outMode) + .Done()); + } + } + } + + auto publishSettings = write.Settings(); + if (transactionalOverrideTarget) { + publishSettings = TCoNameValueTupleList(NYql::RemoveSetting(publishSettings.Ref(), EYtSettingType::Mode, ctx)); + } + + return Build<TYtPublish>(ctx, write.Pos()) + .World(write.World()) + .DataSink(write.DataSink()) + .Input() + .Add(publishInput) + .Build() + .Publish(write.Table().Cast<TYtTable>()) + .Settings(publishSettings) + .Done(); +} +TMaybeNode<TExprBase> TYtPhysicalOptProposalTransformer::ReadWithSettings(TExprBase node, TExprContext& ctx) const { + if (State_->PassiveExecution) { + return node; + } + + auto maybeRead = node.Cast<TCoRight>().Input().Maybe<TYtReadTable>(); + if (!maybeRead) { + return node; + } + + auto read = maybeRead.Cast().Ptr(); + TSyncMap syncList; + auto ret = OptimizeReadWithSettings(read, true, true, syncList, State_, ctx); + if (ret != read) { + if (ret) { + if (!syncList.empty()) { + ret = ctx.ChangeChild(*ret, TYtReadTable::idx_World, + ApplySyncListToWorld(ret->ChildPtr(TYtReadTable::idx_World), syncList, ctx)); + } + return Build<TCoRight>(ctx, node.Pos()) + .Input(ret) + .Done(); + } + return {}; + } + return node; +} +TMaybeNode<TExprBase> TYtPhysicalOptProposalTransformer::ReplaceStatWriteTable(TExprBase node, TExprContext& ctx) const { + if (State_->PassiveExecution) { + return node; + } + + auto write = node.Cast<TStatWriteTable>(); + auto input = write.Input(); + + TMaybeNode<TYtOutput> newInput; + + if (!IsYtProviderInput(input, false)) { + if (!EnsurePersistable(input.Ref(), ctx)) { + return {}; + } + + TString cluster; + TSyncMap syncList; + if (!IsYtCompleteIsolatedLambda(input.Ref(), syncList, cluster, false)) { + return node; + } + + const TTypeAnnotationNode* outItemType; + if (!GetSequenceItemType(input.Ref(), outItemType, ctx)) { + return {}; + } + if (!EnsurePersistableType(input.Pos(), *outItemType, ctx)) { + return {}; + } + + auto cleanup = CleanupWorld(input, ctx); + if (!cleanup) { + return {}; + } + + TYtOutTableInfo outTable {outItemType->Cast<TStructExprType>(), + State_->Configuration->UseNativeYtTypes.Get().GetOrElse(DEFAULT_USE_NATIVE_YT_TYPES) ? NTCF_ALL : NTCF_NONE}; + outTable.RowSpec->SetConstraints(input.Ref().GetConstraintSet()); + outTable.SetUnique(input.Ref().GetConstraint<TDistinctConstraintNode>(), input.Pos(), ctx); + + if (!cluster) { + cluster = State_->Configuration->DefaultCluster + .Get().GetOrElse(State_->Gateway->GetDefaultClusterName()); + } + + input = Build<TYtOutput>(ctx, write.Pos()) + .Operation<TYtFill>() + .World(ApplySyncListToWorld(ctx.NewWorld(write.Pos()), syncList, ctx)) + .DataSink<TYtDSink>() + .Category() + .Value(YtProviderName) + .Build() + .Cluster() + .Value(cluster) + .Build() + .Build() + .Output() + .Add(outTable.ToExprNode(ctx, write.Pos()).Cast<TYtOutTable>()) + .Build() + .Content(MakeJobLambdaNoArg(cleanup.Cast(), ctx)) + .Settings(GetFlowSettings(write.Pos(), *State_, ctx)) + .Build() + .OutIndex() + .Value("0") + .Build() + .Done(); + } + + if (auto maybeRead = input.Maybe<TCoRight>().Input().Maybe<TYtReadTable>()) { + auto read = maybeRead.Cast(); + if (read.Input().Size() != 1) { + ctx.AddError(TIssue(ctx.GetPosition(read.Input().Pos()), TStringBuilder() << + "Unexpected read with several sections on " << node.Ptr()->Content() + )); + return {}; + } + + auto section = read.Input().Item(0); + auto scheme = section.Ptr()->GetTypeAnn()->Cast<TListExprType>()->GetItemType(); + + auto path = CopyOrTrivialMap(section.Pos(), + GetWorld(input, {}, ctx), + GetDataSink(input, ctx), + *scheme, + Build<TYtSection>(ctx, section.Pos()) + .InitFrom(section) + .Settings(NYql::RemoveSettings(section.Settings().Ref(), EYtSettingType::Unordered | EYtSettingType::Unordered, ctx)) + .Done(), + {}, ctx, State_, + TCopyOrTrivialMapOpts().SetTryKeepSortness(true).SetSectionUniq(section.Ref().GetConstraint<TDistinctConstraintNode>())); + + YQL_ENSURE( + path.Ranges().Maybe<TCoVoid>(), + "Unexpected slices: " << path.Ranges().Ref().Content() + ); + + YQL_ENSURE( + path.Table().Maybe<TYtOutput>().Operation(), + "Unexpected node: " << path.Table().Ref().Content() + ); + + newInput = path.Table().Cast<TYtOutput>(); + } else if (auto op = input.Maybe<TYtOutput>().Operation()) { + newInput = input.Cast<TYtOutput>(); + } else { + YQL_ENSURE(false, "Unexpected operation input: " << input.Ptr()->Content()); + } + + auto table = ctx.RenameNode(write.Table().Ref(), TYtStatOutTable::CallableName()); + + return Build<TYtStatOut>(ctx, write.Pos()) + .World(GetWorld(input, {}, ctx)) + .DataSink(GetDataSink(input, ctx)) + .Input(newInput.Cast()) + .Table(table) + .ReplaceMask(write.ReplaceMask()) + .Settings(write.Settings()) + .Done(); +} + +TMaybeNode<TExprBase> TYtPhysicalOptProposalTransformer::YtDqWrite(TExprBase node, TExprContext& ctx) const { + const auto write = node.Cast<TYtDqWrite>(); + + if (ETypeAnnotationKind::Stream == write.Ref().GetTypeAnn()->GetKind()) { + return Build<TCoFromFlow>(ctx, write.Pos()) + .Input<TYtDqWrite>() + .Settings(write.Settings()) + .Input<TCoToFlow>() + .Input(write.Input()) + .Build() + .Build().Done(); + } + + return node; +} + +TMaybeNode<TExprBase> TYtPhysicalOptProposalTransformer::Fill(TExprBase node, TExprContext& ctx) const { + if (State_->PassiveExecution) { + return node; + } + + auto write = node.Cast<TYtWriteTable>(); + + auto mode = NYql::GetSetting(write.Settings().Ref(), EYtSettingType::Mode); + + if (mode && FromString<EYtWriteMode>(mode->Child(1)->Content()) == EYtWriteMode::Drop) { + return node; + } + + auto cluster = TString{write.DataSink().Cluster().Value()}; + TSyncMap syncList; + if (!IsYtCompleteIsolatedLambda(write.Content().Ref(), syncList, cluster, false)) { + return node; + } + + if (FindNode(write.Content().Ptr(), + [] (const TExprNode::TPtr& node) { return !TMaybeNode<TYtOutputOpBase>(node).IsValid(); }, + [] (const TExprNode::TPtr& node) { return TMaybeNode<TDqConnection>(node).IsValid(); })) { + return node; + } + + const TStructExprType* outItemType = nullptr; + if (auto type = GetSequenceItemType(write.Content(), false, ctx)) { + if (!EnsurePersistableType(write.Content().Pos(), *type, ctx)) { + return {}; + } + outItemType = type->Cast<TStructExprType>(); + } else { + return {}; + } + TYtOutTableInfo outTable(outItemType, State_->Configuration->UseNativeYtTypes.Get().GetOrElse(DEFAULT_USE_NATIVE_YT_TYPES) ? NTCF_ALL : NTCF_NONE); + + { + auto path = write.Table().Name().StringValue(); + auto commitEpoch = TEpochInfo::Parse(write.Table().CommitEpoch().Ref()).GetOrElse(0); + auto dstRowSpec = State_->TablesData->GetTable(cluster, path, commitEpoch).RowSpec; + outTable.RowSpec->SetColumnOrder(dstRowSpec->GetColumnOrder()); + } + auto content = write.Content(); + if (auto sorted = content.Ref().GetConstraint<TSortedConstraintNode>()) { + const bool useNativeDescSort = State_->Configuration->UseNativeDescSort.Get().GetOrElse(DEFAULT_USE_NATIVE_DESC_SORT); + TKeySelectorBuilder builder(node.Pos(), ctx, useNativeDescSort, outItemType); + builder.ProcessConstraint(*sorted); + builder.FillRowSpecSort(*outTable.RowSpec); + + if (builder.NeedMap()) { + content = Build<TExprApplier>(ctx, content.Pos()) + .Apply(TCoLambda(builder.MakeRemapLambda(true))) + .With(0, content) + .Done(); + outItemType = builder.MakeRemapType(); + } + + } else if (auto unordered = content.Maybe<TCoUnorderedBase>()) { + content = unordered.Cast().Input(); + } + outTable.RowSpec->SetConstraints(write.Content().Ref().GetConstraintSet()); + outTable.SetUnique(write.Content().Ref().GetConstraint<TDistinctConstraintNode>(), node.Pos(), ctx); + + TYtTableInfo::TPtr pubTableInfo = MakeIntrusive<TYtTableInfo>(write.Table()); + const bool renew = !mode || FromString<EYtWriteMode>(mode->Child(1)->Content()) == EYtWriteMode::Renew; + const bool flush = mode && FromString<EYtWriteMode>(mode->Child(1)->Content()) == EYtWriteMode::Flush; + const bool transactionalOverrideTarget = NYql::GetSetting(write.Settings().Ref(), EYtSettingType::Initial) + && !flush && (renew || !pubTableInfo->Meta->DoesExist); + + auto publishSettings = write.Settings(); + if (transactionalOverrideTarget) { + publishSettings = TCoNameValueTupleList(NYql::RemoveSetting(publishSettings.Ref(), EYtSettingType::Mode, ctx)); + } + + auto cleanup = CleanupWorld(content, ctx); + if (!cleanup) { + return {}; + } + + return Build<TYtPublish>(ctx, write.Pos()) + .World(write.World()) + .DataSink(write.DataSink()) + .Input() + .Add() + .Operation<TYtFill>() + .World(ApplySyncListToWorld(ctx.NewWorld(write.Pos()), syncList, ctx)) + .DataSink(write.DataSink()) + .Content(MakeJobLambdaNoArg(cleanup.Cast(), ctx)) + .Output() + .Add(outTable.ToExprNode(ctx, write.Pos()).Cast<TYtOutTable>()) + .Build() + .Settings(GetFlowSettings(write.Pos(), *State_, ctx)) + .Build() + .OutIndex().Value(0U).Build() + .Build() + .Build() + .Publish(write.Table()) + .Settings(publishSettings) + .Done(); +} + +TMaybeNode<TExprBase> TYtPhysicalOptProposalTransformer::FillToMaterialize(TExprBase node, TExprContext& ctx) const { + if (State_->PassiveExecution) { + return node; + } + + auto write = node.Cast<TYtWriteTable>(); + + auto mode = NYql::GetSetting(write.Settings().Ref(), EYtSettingType::Mode); + + if (mode && FromString<EYtWriteMode>(mode->Child(1)->Content()) == EYtWriteMode::Drop) { + return node; + } + + auto content = write.Content(); + + TSyncMap syncList; + if (!IsYtCompleteIsolatedLambda(content.Ref(), syncList, /* no dq expected */false)) { + return node; + } + + content = Build<TYtMaterialize>(ctx, content.Pos()) + .World(ctx.NewWorld(write.Pos())/*TODO: write.World()*/) + .DataSink(write.DataSink()) + .Input(content) + .Settings().Build() + .Done(); + + return TExprBase(ctx.ChangeChild(node.Ref(), TYtWriteTable::idx_Content, content.Ptr())); +} + +TMaybeNode<TExprBase> TYtPhysicalOptProposalTransformer::Materialize(TExprBase node, TExprContext& ctx) const { + if (State_->PassiveExecution) { + return node; + } + + auto materialize = node.Cast<TYtMaterialize>(); + auto content = materialize.Input(); + if (IsYtProviderInput(content)) { + return content; + } + + auto cluster = materialize.DataSink().Cluster().StringValue(); + TSyncMap syncList; + if (!IsYtCompleteIsolatedLambda(content.Ref(), syncList, cluster, false)) { + return node; + } + + const TStructExprType* outItemType = nullptr; + if (auto type = GetSequenceItemType(content, false, ctx)) { + outItemType = type->Cast<TStructExprType>(); + } else { + return {}; + } + TYtOutTableInfo outTable(outItemType, State_->Configuration->UseNativeYtTypes.Get().GetOrElse(DEFAULT_USE_NATIVE_YT_TYPES) ? NTCF_ALL : NTCF_NONE); + + if (auto sorted = content.Ref().GetConstraint<TSortedConstraintNode>()) { + const bool useNativeDescSort = State_->Configuration->UseNativeDescSort.Get().GetOrElse(DEFAULT_USE_NATIVE_DESC_SORT); + TKeySelectorBuilder builder(node.Pos(), ctx, useNativeDescSort, outItemType); + builder.ProcessConstraint(*sorted); + builder.FillRowSpecSort(*outTable.RowSpec); + + if (builder.NeedMap()) { + content = Build<TExprApplier>(ctx, content.Pos()) + .Apply(TCoLambda(builder.MakeRemapLambda(true))) + .With(0, content) + .Done(); + outItemType = builder.MakeRemapType(); + } + + } else if (auto unordered = content.Maybe<TCoUnorderedBase>()) { + content = unordered.Cast().Input(); + } + outTable.RowSpec->SetConstraints(materialize.Input().Ref().GetConstraintSet()); + outTable.SetUnique(materialize.Input().Ref().GetConstraint<TDistinctConstraintNode>(), node.Pos(), ctx); + + auto cleanup = CleanupWorld(content, ctx); + if (!cleanup) { + return {}; + } + + return Build<TYtOutput>(ctx, materialize.Pos()) + .Operation<TYtFill>() + .World(ApplySyncListToWorld(materialize.World().Ptr(), syncList, ctx)) + .DataSink(materialize.DataSink()) + .Content(MakeJobLambdaNoArg(cleanup.Cast(), ctx)) + .Output() + .Add(outTable.ToExprNode(ctx, materialize.Pos()).Cast<TYtOutTable>()) + .Build() + .Settings(GetFlowSettings(materialize.Pos(), *State_, ctx)) + .Build() + .OutIndex().Value(0U).Build() + .Done(); +} + +} // namespace NYql diff --git a/yt/yql/providers/yt/provider/ut/ya.make b/yt/yql/providers/yt/provider/ut/ya.make new file mode 100644 index 0000000000..3b29f30999 --- /dev/null +++ b/yt/yql/providers/yt/provider/ut/ya.make @@ -0,0 +1,40 @@ +IF (NOT OPENSOURCE) + +UNITTEST_FOR(yt/yql/providers/yt/provider) + +SIZE(SMALL) + +SRCS( + yql_yt_dq_integration_ut.cpp + yql_yt_epoch_ut.cpp + yql_yt_cbo_ut.cpp +) + +PEERDIR( + contrib/ydb/library/yql/dq/opt + yt/yql/providers/yt/lib/schema + yt/yql/providers/yt/provider + yt/yql/providers/yt/gateway/file + yt/yql/providers/yt/codec/codegen + yt/yql/providers/yt/comp_nodes/llvm14 + yql/essentials/core/ut_common + yql/essentials/ast + yql/essentials/public/udf/service/terminate_policy + yql/essentials/core/services + yql/essentials/core + yql/essentials/providers/common/gateway + yql/essentials/providers/common/provider + yql/essentials/providers/common/config + yql/essentials/providers/config + yql/essentials/providers/result/provider + yql/essentials/sql + yql/essentials/minikql/invoke_builtins/llvm14 + yql/essentials/minikql/comp_nodes/llvm14 + yql/essentials/sql/pg +) + +YQL_LAST_ABI_VERSION() + +END() + +ENDIF() diff --git a/yt/yql/providers/yt/provider/ya.make b/yt/yql/providers/yt/provider/ya.make new file mode 100644 index 0000000000..9540fd7f06 --- /dev/null +++ b/yt/yql/providers/yt/provider/ya.make @@ -0,0 +1,128 @@ +LIBRARY() + +SRCS( + yql_yt_block_input_filter.cpp + yql_yt_block_input.cpp + yql_yt_datasink_constraints.cpp + yql_yt_datasink_exec.cpp + yql_yt_datasink_finalize.cpp + yql_yt_datasink_trackable.cpp + yql_yt_datasink_type_ann.cpp + yql_yt_datasink.cpp + yql_yt_datasource_constraints.cpp + yql_yt_datasource_exec.cpp + yql_yt_datasource_type_ann.cpp + yql_yt_datasource.cpp + yql_yt_epoch.cpp + yql_yt_gateway.cpp + yql_yt_horizontal_join.cpp + yql_yt_helpers.cpp + yql_yt_intent_determination.cpp + yql_yt_io_discovery.cpp + yql_yt_io_discovery_walk_folders.cpp + yql_yt_join_impl.cpp + yql_yt_join_reorder.cpp + yql_yt_key.cpp + yql_yt_load_table_meta.cpp + yql_yt_load_columnar_stats.cpp + yql_yt_logical_optimize.cpp + yql_yt_mkql_compiler.cpp + yql_yt_op_hash.cpp + yql_yt_op_settings.cpp + yql_yt_optimize.cpp + yql_yt_peephole.cpp + yql_yt_physical_finalizing.cpp + yql_yt_physical_optimize.cpp + yql_yt_provider_context.cpp + yql_yt_provider_impl.cpp + yql_yt_provider.cpp + yql_yt_provider.h + yql_yt_provider_impl.h + yql_yt_table_desc.cpp + yql_yt_table.cpp + yql_yt_dq_integration.cpp + yql_yt_dq_optimize.cpp + yql_yt_dq_hybrid.cpp + yql_yt_wide_flow.cpp + + phy_opt/yql_yt_phy_opt.cpp + phy_opt/yql_yt_phy_opt_content.cpp + phy_opt/yql_yt_phy_opt_fuse.cpp + phy_opt/yql_yt_phy_opt_helper.h + phy_opt/yql_yt_phy_opt_lambda.cpp + phy_opt/yql_yt_phy_opt_misc.cpp + phy_opt/yql_yt_phy_opt_sort.cpp + phy_opt/yql_yt_phy_opt_join.cpp + phy_opt/yql_yt_phy_opt_map.cpp + phy_opt/yql_yt_phy_opt_partition.cpp + phy_opt/yql_yt_phy_opt_weak_fields.cpp + phy_opt/yql_yt_phy_opt_field_subset.cpp + phy_opt/yql_yt_phy_opt_helper.cpp + phy_opt/yql_yt_phy_opt_key_range.cpp + phy_opt/yql_yt_phy_opt_merge.cpp + phy_opt/yql_yt_phy_opt_push.cpp + phy_opt/yql_yt_phy_opt_write.cpp +) + +PEERDIR( + library/cpp/yson/node + library/cpp/disjoint_sets + yt/cpp/mapreduce/common + yt/cpp/mapreduce/interface + yql/essentials/ast + yql/essentials/core/extract_predicate + yql/essentials/public/udf + yql/essentials/public/udf/tz + yql/essentials/sql + yql/essentials/utils + yql/essentials/utils/log + yql/essentials/core + yql/essentials/core/dq_expr_nodes + yql/essentials/core/dqs_expr_nodes + yql/essentials/core/expr_nodes + yql/essentials/core/issue + yql/essentials/core/issue/protos + yql/essentials/core/peephole_opt + yql/essentials/core/type_ann + yql/essentials/core/file_storage + yql/essentials/core/url_lister/interface + yql/essentials/core/dq_integration + yql/essentials/minikql + yql/essentials/providers/common/codec + yql/essentials/providers/common/config + yql/essentials/providers/common/dq + yql/essentials/providers/common/mkql + yql/essentials/providers/common/proto + yql/essentials/providers/common/activation + yql/essentials/providers/common/provider + yql/essentials/providers/common/schema/expr + yql/essentials/providers/common/structured_token + yql/essentials/providers/common/transform + yql/essentials/providers/result/expr_nodes + yql/providers/stat/expr_nodes + yt/yql/providers/yt/common + yt/yql/providers/yt/expr_nodes + yt/yql/providers/yt/lib/expr_traits + yt/yql/providers/yt/lib/graph_reorder + yt/yql/providers/yt/lib/hash + yt/yql/providers/yt/lib/key_filter + yt/yql/providers/yt/lib/mkql_helpers + yt/yql/providers/yt/lib/res_pull + yt/yql/providers/yt/lib/row_spec + yt/yql/providers/yt/lib/schema + yt/yql/providers/yt/lib/skiff + yt/yql/providers/yt/lib/yson_helpers + yt/yql/providers/yt/opt + yt/yql/providers/yt/gateway/qplayer + yt/yql/providers/yt/proto +) + +YQL_LAST_ABI_VERSION() + +GENERATE_ENUM_SERIALIZATION(yql_yt_op_settings.h) + +END() + +RECURSE_FOR_TESTS( + ut +) diff --git a/yt/yql/providers/yt/provider/yql_yt_block_input.cpp b/yt/yql/providers/yt/provider/yql_yt_block_input.cpp new file mode 100644 index 0000000000..03b19b0f8b --- /dev/null +++ b/yt/yql/providers/yt/provider/yql_yt_block_input.cpp @@ -0,0 +1,81 @@ +#include "yql_yt_provider_impl.h" + +#include <yql/essentials/core/yql_opt_utils.h> +#include <yql/essentials/providers/common/transform/yql_optimize.h> +#include <yt/yql/providers/yt/common/yql_names.h> +#include <yql/essentials/utils/log/log.h> + +namespace NYql { + +namespace { + +using namespace NNodes; + +class TYtBlockInputTransformer : public TOptimizeTransformerBase { +public: + TYtBlockInputTransformer(TYtState::TPtr state) + : TOptimizeTransformerBase( + state ? state->Types : nullptr, + NLog::EComponent::ProviderYt, + state ? state->Configuration->DisableOptimizers.Get().GetOrElse(TSet<TString>()) : TSet<TString>() + ) + , State_(std::move(state)) + { +#define HNDL(name) "YtBlockInput-"#name, Hndl(&TYtBlockInputTransformer::name) + AddHandler(0, &TYtMap::Match, HNDL(TryTransformMap)); +#undef HNDL + } + +private: + TMaybeNode<TExprBase> TryTransformMap(TExprBase node, TExprContext& ctx) const { + auto map = node.Cast<TYtMap>(); + + if ( + NYql::HasSetting(map.Settings().Ref(), EYtSettingType::BlockInputApplied) + || !NYql::HasSetting(map.Settings().Ref(), EYtSettingType::BlockInputReady) + || !CanRewriteMap(map, ctx) + ) { + return map; + } + + YQL_CLOG(INFO, ProviderYt) << "Rewrite YtMap with block input"; + + auto settings = RemoveSetting(map.Settings().Ref(), EYtSettingType::BlockInputReady, ctx); + settings = AddSetting(*settings, EYtSettingType::BlockInputApplied, TExprNode::TPtr(), ctx); + auto mapperLambda = Build<TCoLambda>(ctx, map.Mapper().Pos()) + .Args({"flow"}) + .Body<TExprApplier>() + .Apply(map.Mapper()) + .With<TCoWideFromBlocks>(0) + .Input("flow") + .Build() + .Build() + .Done() + .Ptr(); + + return Build<TYtMap>(ctx, node.Pos()) + .InitFrom(map) + .Settings(settings) + .Mapper(mapperLambda) + .Done(); + } + + bool CanRewriteMap(const TYtMap& map, TExprContext& ctx) const { + if (auto flowSetting = NYql::GetSetting(map.Settings().Ref(), EYtSettingType::Flow); !flowSetting || flowSetting->ChildrenSize() < 2) { + return false; + } + + return EnsureWideFlowType(map.Mapper().Args().Arg(0).Ref(), ctx); + } + +private: + const TYtState::TPtr State_; +}; + +} // namespace + +THolder<IGraphTransformer> CreateYtBlockInputTransformer(TYtState::TPtr state) { + return THolder<IGraphTransformer>(new TYtBlockInputTransformer(std::move(state))); +} + +} // namespace NYql diff --git a/yt/yql/providers/yt/provider/yql_yt_block_input_filter.cpp b/yt/yql/providers/yt/provider/yql_yt_block_input_filter.cpp new file mode 100644 index 0000000000..df3853282a --- /dev/null +++ b/yt/yql/providers/yt/provider/yql_yt_block_input_filter.cpp @@ -0,0 +1,126 @@ +#include "yql_yt_provider_impl.h" +#include "yql_yt_dq_integration.h" // TODO remove + +#include <yql/essentials/core/yql_opt_utils.h> +#include <yql/essentials/providers/common/transform/yql_optimize.h> +#include <yt/yql/providers/yt/common/yql_names.h> +#include <yt/yql/providers/yt/provider/yql_yt_helpers.h> +#include <yql/essentials/utils/log/log.h> + +namespace NYql { + +namespace { + +using namespace NNodes; +class YtBlockInputFilterTransformer : public TOptimizeTransformerBase { +public: + YtBlockInputFilterTransformer(TYtState::TPtr state, THolder<IGraphTransformer>&& finalizer) + : TOptimizeTransformerBase(state->Types, NLog::EComponent::ProviderYt, state->Configuration->DisableOptimizers.Get().GetOrElse(TSet<TString>())) + , State_(std::move(state)) + , Finalizer_(std::move(finalizer)) + { +#define HNDL(name) "YtBlockInputFilter-"#name, Hndl(&YtBlockInputFilterTransformer::name) + AddHandler(0, &TYtMap::Match, HNDL(HandleMap)); +#undef HNDL + } + +private: + TStatus DoTransform(TExprNode::TPtr input, TExprNode::TPtr& output, TExprContext& ctx) final { + if (const auto status = Finalizer_->Transform(input, output, ctx); status.Level != TStatus::Ok) + return status; + + return TOptimizeTransformerBase::DoTransform(input, output, ctx); + } + + void Rewind() final { + Finalizer_->Rewind(); + TOptimizeTransformerBase::Rewind(); + } + + TMaybeNode<TExprBase> HandleMap(TExprBase node, TExprContext& ctx) const { + auto map = node.Cast<TYtMap>(); + if (!State_->Configuration->JobBlockInput.Get().GetOrElse(Types->UseBlocks)) { + return map; + } + + if (NYql::HasSetting(map.Settings().Ref(), EYtSettingType::BlockInputApplied)) { + return map; + } + + auto settings = map.Settings().Ptr(); + bool canUseBlockInput = CanUseBlockInputForMap(map); + bool hasSetting = HasSetting(*settings, EYtSettingType::BlockInputReady); + if (canUseBlockInput && !hasSetting) { + settings = AddSetting(*settings, EYtSettingType::BlockInputReady, TExprNode::TPtr(), ctx); + } else if (!canUseBlockInput && hasSetting) { + settings = RemoveSetting(*settings, EYtSettingType::BlockInputReady, ctx); + } else { + return map; + } + return Build<TYtMap>(ctx, node.Pos()) + .InitFrom(map) + .Settings(settings) + .Done(); + } + + bool CanUseBlockInputForMap(const TYtMap& map) const { + if (!NYql::HasSetting(map.Settings().Ref(), EYtSettingType::Flow)) { + return false; + } + + if (map.Input().Size() > 1) { + return false; + } + + for (auto path : map.Input().Item(0).Paths()) { + if (!IsYtTableSuitableForArrowInput(path.Table(), [](const TString&) {})) { + return false; + } + } + + auto supportedTypes = State_->Configuration->JobBlockInputSupportedTypes.Get(map.DataSink().Cluster().StringValue()).GetOrElse(DEFAULT_BLOCK_INPUT_SUPPORTED_TYPES); + auto supportedDataTypes = State_->Configuration->JobBlockInputSupportedDataTypes.Get(map.DataSink().Cluster().StringValue()).GetOrElse(DEFAULT_BLOCK_INPUT_SUPPORTED_DATA_TYPES); + + auto lambdaInputType = map.Mapper().Args().Arg(0).Ref().GetTypeAnn()->Cast<TFlowExprType>()->GetItemType(); + if (lambdaInputType->GetKind() == ETypeAnnotationKind::Multi) { + auto& items = lambdaInputType->Cast<TMultiExprType>()->GetItems(); + if (items.empty()) { + return false; + } + + if (!CheckSupportedTypesOld(items, supportedTypes, supportedDataTypes, [](const TString&) {})) { + return false; + } + } else if (lambdaInputType->GetKind() == ETypeAnnotationKind::Struct) { + auto& items = lambdaInputType->Cast<TStructExprType>()->GetItems(); + if (items.empty()) { + return false; + } + + TTypeAnnotationNode::TListType itemTypes; + for (auto item: items) { + itemTypes.push_back(item->GetItemType()); + } + + if (!CheckSupportedTypesOld(itemTypes, supportedTypes, supportedDataTypes, [](const TString&) {})) { + return false; + } + } else { + return false; + } + + return true; + } + +private: + const TYtState::TPtr State_; + const THolder<IGraphTransformer> Finalizer_; +}; + +} // namespace + +THolder<IGraphTransformer> CreateYtBlockInputFilterTransformer(TYtState::TPtr state, THolder<IGraphTransformer>&& finalizer) { + return THolder<IGraphTransformer>(new YtBlockInputFilterTransformer(std::move(state), std::move(finalizer))); +} + +} // namespace NYql diff --git a/yt/yql/providers/yt/provider/yql_yt_datasink.cpp b/yt/yql/providers/yt/provider/yql_yt_datasink.cpp new file mode 100644 index 0000000000..f0843bce5b --- /dev/null +++ b/yt/yql/providers/yt/provider/yql_yt_datasink.cpp @@ -0,0 +1,655 @@ +#include "yql_yt_provider_impl.h" +#include "yql_yt_op_settings.h" +#include "yql_yt_helpers.h" +#include "yql_yt_dq_integration.h" + +#include <yt/yql/providers/yt/lib/yson_helpers/yson_helpers.h> +#include <yt/yql/providers/yt/expr_nodes/yql_yt_expr_nodes.h> +#include <yt/yql/providers/yt/common/yql_names.h> +#include <yt/yql/providers/yt/common/yql_configuration.h> +#include <yt/yql/providers/yt/lib/schema/schema.h> +#include <yql/essentials/providers/common/codec/yql_codec_type_flags.h> +#include <yql/essentials/providers/common/provider/yql_provider_names.h> +#include <yql/essentials/providers/common/provider/yql_provider.h> +#include <yql/essentials/providers/common/provider/yql_data_provider_impl.h> +#include <yql/essentials/providers/common/transform/yql_lazy_init.h> +#include <yql/essentials/providers/common/schema/expr/yql_expr_schema.h> +#include <yql/essentials/core/expr_nodes/yql_expr_nodes.h> +#include <yql/essentials/core/yql_expr_type_annotation.h> +#include <yql/essentials/core/yql_opt_utils.h> +#include <yql/essentials/core/yql_type_helpers.h> +#include <yql/essentials/utils/log/log.h> + +#include <library/cpp/yson/node/node_io.h> + +#include <util/string/cast.h> +#include <util/string/builder.h> +#include <util/generic/set.h> +#include <util/generic/utility.h> +#include <util/generic/ylimits.h> + +#include <algorithm> +#include <iterator> + +namespace NYql { + +using namespace NNodes; + +class TYtDataSinkTrackableNodeProcessor : public TTrackableNodeProcessorBase { +public: + TYtDataSinkTrackableNodeProcessor(const TYtState::TPtr& state, bool collectNodes) + : CollectNodes(collectNodes) + , CleanupTransformer(collectNodes ? CreateYtDataSinkTrackableNodesCleanupTransformer(state) : nullptr) + { + } + + void GetUsedNodes(const TExprNode& input, TVector<TString>& usedNodeIds) override { + usedNodeIds.clear(); + if (!CollectNodes) { + return; + } + + if (TMaybeNode<TYtOutputOpBase>(&input)) { + for (size_t i = TYtOutputOpBase::idx_Output + 1; i < input.ChildrenSize(); ++i) { + ScanForUsedOutputTables(*input.Child(i), usedNodeIds); + } + } else if (TMaybeNode<TYtPublish>(&input)) { + ScanForUsedOutputTables(*input.Child(TYtPublish::idx_Input), usedNodeIds); + } else if (TMaybeNode<TYtStatOut>(&input)) { + ScanForUsedOutputTables(*input.Child(TYtStatOut::idx_Input), usedNodeIds); + } + } + + void GetCreatedNodes(const TExprNode& node, TVector<TExprNodeAndId>& created, TExprContext& ctx) override { + created.clear(); + if (!CollectNodes) { + return; + } + + if (auto maybeOp = TMaybeNode<TYtOutputOpBase>(&node)) { + TString clusterName = TString{maybeOp.Cast().DataSink().Cast<TYtDSink>().Cluster().Value()}; + auto clusterPtr = maybeOp.Cast().DataSink().Ptr(); + for (auto table: maybeOp.Cast().Output()) { + TString tableName = TString{table.Name().Value()}; + auto tablePtr = table.Ptr(); + + TExprNodeAndId nodeAndId; + nodeAndId.Id = MakeUsedNodeId(clusterName, tableName); + nodeAndId.Node = ctx.NewList(tablePtr->Pos(), {clusterPtr, tablePtr}); + + created.push_back(std::move(nodeAndId)); + } + } + } + + IGraphTransformer& GetCleanupTransformer() override { + return CollectNodes ? *CleanupTransformer : NullTransformer_; + } + +private: + const bool CollectNodes; + THolder<IGraphTransformer> CleanupTransformer; +}; + +class TYtDataSink : public TDataProviderBase { +public: + TYtDataSink(TYtState::TPtr state) + : State_(state) + , IntentDeterminationTransformer_([this]() { return CreateYtIntentDeterminationTransformer(State_); }) + , TypeAnnotationTransformer_([this]() { return CreateYtDataSinkTypeAnnotationTransformer(State_); }) + , ConstraintTransformer_([this]() { return CreateYtDataSinkConstraintTransformer(State_, false); }) + , SubConstraintTransformer_([this]() { return CreateYtDataSinkConstraintTransformer(State_, true); }) + , ExecTransformer_([this]() { return CreateYtDataSinkExecTransformer(State_); }) + , LogicalOptProposalTransformer_([this]() { return CreateYtLogicalOptProposalTransformer(State_); }) + , PhysicalOptProposalTransformer_([this]() { return CreateYtPhysicalOptProposalTransformer(State_); }) + , PhysicalFinalizingTransformer_([this]() { + auto transformer = CreateYtPhysicalFinalizingTransformer(State_); + transformer = CreateYtBlockInputFilterTransformer(State_, std::move(transformer)); + if (State_->IsHybridEnabled()) + transformer = CreateYtDqHybridTransformer(State_, std::move(transformer)); + return transformer; + }) + , FinalizingTransformer_([this]() { return CreateYtDataSinkFinalizingTransformer(State_); }) + , TrackableNodeProcessor_([this]() { + auto mode = GetReleaseTempDataMode(*State_->Configuration); + bool collectNodes = mode == EReleaseTempDataMode::Immediate; + return MakeHolder<TYtDataSinkTrackableNodeProcessor>(State_, collectNodes); + }) + { + } + + TStringBuf GetName() const override { + return YtProviderName; + } + + IGraphTransformer& GetIntentDeterminationTransformer() override { + return *IntentDeterminationTransformer_; + } + + IGraphTransformer& GetTypeAnnotationTransformer(bool instantOnly) override { + Y_UNUSED(instantOnly); + return *TypeAnnotationTransformer_; + } + + IGraphTransformer& GetConstraintTransformer(bool instantOnly, bool subGraph) override { + Y_UNUSED(instantOnly); + return subGraph ? *SubConstraintTransformer_ : *ConstraintTransformer_; + } + + IGraphTransformer& GetLogicalOptProposalTransformer() override { + return *LogicalOptProposalTransformer_; + } + + IGraphTransformer& GetPhysicalOptProposalTransformer() override { + return *PhysicalOptProposalTransformer_; + } + + IGraphTransformer& GetPhysicalFinalizingTransformer() override { + return *PhysicalFinalizingTransformer_; + } + + IGraphTransformer& GetCallableExecutionTransformer() override { + return *ExecTransformer_; + } + + IGraphTransformer& GetFinalizingTransformer() override { + return *FinalizingTransformer_; + } + + bool CollectStatistics(NYson::TYsonWriter& writer, bool totalOnly) override { + if (State_->Statistics.empty()) { + return false; + } + + writer.OnBeginMap(); + NCommon::WriteStatistics(writer, totalOnly, State_->Statistics, true, false); + writer.OnKeyedItem("HybridTotal"); + writer.OnBeginMap(); + for (const auto& [subFolder, stats] : State_->HybridStatistics) { + if (subFolder.empty()) { + NCommon::WriteStatistics(writer, totalOnly, {{0, stats}}, false, false); + } else { + writer.OnKeyedItem(subFolder); + NCommon::WriteStatistics(writer, totalOnly, {{0, stats}}, false); + } + } + writer.OnEndMap(); + writer.OnKeyedItem("Hybrid"); + writer.OnBeginMap(); + for (const auto& [opName, hybridStats] : State_->HybridOpStatistics) { + writer.OnKeyedItem(opName); + writer.OnBeginMap(); + for (const auto& [subFolder, stats] : hybridStats) { + if (subFolder.empty()) { + NCommon::WriteStatistics(writer, totalOnly, {{0, stats}}, false, false); + } else { + writer.OnKeyedItem(subFolder); + NCommon::WriteStatistics(writer, totalOnly, {{0, stats}}, false); + } + } + writer.OnEndMap(); + } + writer.OnEndMap(); + writer.OnEndMap(); + + return true; + } + + bool ValidateParameters(TExprNode& node, TExprContext& ctx, TMaybe<TString>& cluster) override { + if (node.IsCallable(TCoDataSink::CallableName())) { + if (!EnsureArgsCount(node, 2, ctx)) { + return false; + } + + if (node.Child(0)->Content() == YtProviderName) { + if (!node.Child(1)->IsCallable("EvaluateAtom")) { + if (!EnsureAtom(*node.Child(1), ctx)) { + return false; + } + + if (node.Child(1)->Content().empty()) { + ctx.AddError(TIssue(ctx.GetPosition(node.Child(1)->Pos()), "Empty cluster name")); + return false; + } + + cluster = TString(node.Child(1)->Content()); + } + + return true; + } + } + + ctx.AddError(TIssue(ctx.GetPosition(node.Pos()), "Invalid Yt DataSink parameters")); + return false; + } + + bool CanParse(const TExprNode& node) override { + if (node.IsCallable(TCoWrite::CallableName())) { + return TYtDSink::Match(node.Child(1)); + } + + return TypeAnnotationTransformer_->CanParse(node); + } + + void FillModifyCallables(THashSet<TStringBuf>& callables) override { + callables.insert(TYtWriteTable::CallableName()); + callables.insert(TYtDropTable::CallableName()); + callables.insert(TYtConfigure::CallableName()); + } + + bool IsWrite(const TExprNode& node) override { + return TYtWriteTable::Match(&node); + } + + TExprNode::TPtr RewriteIO(const TExprNode::TPtr& node, TExprContext& ctx) override { + YQL_ENSURE(TMaybeNode<TYtWrite>(node).DataSink()); + auto mode = NYql::GetSetting(*node->Child(4), EYtSettingType::Mode); + if (mode && FromString<EYtWriteMode>(mode->Child(1)->Content()) == EYtWriteMode::Drop) { + if (!node->Child(3)->IsCallable("Void")) { + ctx.AddError(TIssue(ctx.GetPosition(node->Child(3)->Pos()), TStringBuilder() + << "Expected Void, but got: " << node->Child(3)->Content())); + return {}; + } + + TExprNode::TListType children = node->ChildrenList(); + children.resize(3); + return ctx.NewCallable(node->Pos(), TYtDropTable::CallableName(), std::move(children)); + } else { + auto res = ctx.RenameNode(*node, TYtWriteTable::CallableName()); + if ((!mode || FromString<EYtWriteMode>(mode->Child(1)->Content()) == EYtWriteMode::Renew) && NYql::HasSetting(*node->Child(4), EYtSettingType::KeepMeta)) { + auto settings = NYql::AddSetting( + *NYql::RemoveSettings(*node->Child(4), EYtSettingType::Mode | EYtSettingType::KeepMeta, ctx), + EYtSettingType::Mode, + ctx.NewAtom(node->Child(4)->Pos(), ToString(EYtWriteMode::RenewKeepMeta), TNodeFlags::ArbitraryContent), + ctx); + res = ctx.ChangeChild(*res, TYtWriteTable::idx_Settings, std::move(settings)); + } + if (auto columnGroup = NYql::GetSetting(*res->Child(TYtWriteTable::idx_Settings), EYtSettingType::ColumnGroups)) { + const TString normalized = NormalizeColumnGroupSpec(columnGroup->Tail().Content()); + res = ctx.ChangeChild(*res, TYtWriteTable::idx_Settings, + NYql::UpdateSettingValue(*res->Child(TYtWriteTable::idx_Settings), + EYtSettingType::ColumnGroups, + ctx.NewAtom(res->Child(TYtWriteTable::idx_Settings)->Pos(), normalized, TNodeFlags::MultilineContent), + ctx) + ); + } else if (NYql::HasSetting(*res->Child(TYtWriteTable::idx_Table)->Child(TYtTable::idx_Settings), EYtSettingType::Anonymous)) { + if (const auto mode = State_->Configuration->ColumnGroupMode.Get().GetOrElse(EColumnGroupMode::Disable); mode != EColumnGroupMode::Disable) { + res = ctx.ChangeChild(*res, TYtWriteTable::idx_Settings, + NYql::AddSetting(*res->Child(TYtWriteTable::idx_Settings), + EYtSettingType::ColumnGroups, + ctx.NewAtom(res->Child(TYtWriteTable::idx_Settings)->Pos(), NYql::GetSingleColumnGroupSpec(), TNodeFlags::MultilineContent), + ctx) + ); + } + } + auto mutationId = ++NextMutationId_; + res = ctx.ChangeChild(*res, TYtWriteTable::idx_Settings, + NYql::AddSetting(*res->Child(TYtWriteTable::idx_Settings), + EYtSettingType::MutationId, + ctx.NewAtom(res->Child(TYtWriteTable::idx_Settings)->Pos(), mutationId), + ctx) + ); + if (State_->Configuration->UseSystemColumns.Get().GetOrElse(DEFAULT_USE_SYS_COLUMNS)) { + res = ctx.ChangeChild(*res, TYtWriteTable::idx_Content, + ctx.Builder(node->Pos()) + .Callable("RemovePrefixMembers") + .Add(0, node->ChildPtr(TYtWriteTable::idx_Content)) + .List(1) + .Atom(0, YqlSysColumnPrefix) + .Seal() + .Seal() + .Build() + ); + } + return res; + } + } + + void PostRewriteIO() final { + if (!State_->Types->EvaluationInProgress) { + State_->TablesData->CleanupCompiledSQL(); + } + } + + void Reset() final { + TDataProviderBase::Reset(); + State_->Reset(); + NextMutationId_ = 0; + } + + bool CanExecute(const TExprNode& node) override { + return ExecTransformer_->CanExec(node); + } + + bool ValidateExecution(const TExprNode& node, TExprContext& ctx) override { + if (TYtDqProcessWrite::Match(&node)) { + auto dqProvider = State_->Types->DataSourceMap.FindPtr(DqProviderName); + YQL_ENSURE(dqProvider); + return (*dqProvider)->ValidateExecution(TYtDqProcessWrite(&node).Input().Ref(), ctx); + } + return true; + } + + void GetRequiredChildren(const TExprNode& node, TExprNode::TListType& children) override { + if (CanExecute(node)) { + children.push_back(node.ChildPtr(0)); + if (TYtTransientOpBase::Match(&node)) { + children.push_back(node.ChildPtr(TYtTransientOpBase::idx_Input)); + } else if (TYtPublish::Match(&node)) { + children.push_back(node.ChildPtr(TYtPublish::idx_Input)); + } else if (TYtStatOut::Match(&node)) { + children.push_back(node.ChildPtr(TYtStatOut::idx_Input)); + } + } + } + + bool GetDependencies(const TExprNode& node, TExprNode::TListType& children, bool compact) override { + Y_UNUSED(compact); + if (CanExecute(node)) { + children.emplace_back(node.HeadPtr()); + + if (TMaybeNode<TYtOutputOpBase>(&node)) { + for (size_t i = TYtOutputOpBase::idx_Output + 1; i < node.ChildrenSize(); ++i) { + ScanPlanDependencies(node.ChildPtr(i), children); + } + } else if (TMaybeNode<TYtPublish>(&node)) { + ScanPlanDependencies(node.ChildPtr(TYtPublish::idx_Input), children); + } else if (TMaybeNode<TYtStatOut>(&node)) { + ScanPlanDependencies(node.ChildPtr(TYtStatOut::idx_Input), children); + } + + return !TYtDqProcessWrite::Match(&node); + } + + return false; + } + + void WritePlanDetails(const TExprNode& node, NYson::TYsonWriter& writer, bool withLimits) override { + if (auto maybeOp = TMaybeNode<TYtTransientOpBase>(&node)) { + writer.OnKeyedItem("InputColumns"); + auto op = maybeOp.Cast(); + if (op.Input().Size() > 1) { + writer.OnBeginList(); + for (auto section: op.Input()) { + writer.OnListItem(); + WriteColumns(writer, section.Paths().Item(0).Columns()); + } + writer.OnEndList(); + } + else { + WriteColumns(writer, op.Input().Item(0).Paths().Item(0).Columns()); + } + + if (op.Input().Size() > 1) { + writer.OnKeyedItem("InputSections"); + auto op = maybeOp.Cast(); + writer.OnBeginList(); + ui64 ndx = 0; + for (auto section: op.Input()) { + writer.OnListItem(); + writer.OnBeginList(); + for (ui32 i = 0; i < Min((ui32)section.Paths().Size(), (withLimits && State_->PlanLimits) ? State_->PlanLimits : Max<ui32>()); ++i) { + writer.OnListItem(); + writer.OnUint64Scalar(ndx++); + } + writer.OnEndList(); + } + writer.OnEndList(); + } + + if (op.Maybe<TYtMap>() || op.Maybe<TYtMapReduce>() || op.Maybe<TYtMerge>() || + op.Maybe<TYtReduce>() || op.Maybe<TYtSort>() || op.Maybe<TYtEquiJoin>()) + { + TSet<TString> keyFilterColumns; + for (auto section: op.Input()) { + for (auto col : GetKeyFilterColumns(section, EYtSettingType::KeyFilter | EYtSettingType::KeyFilter2)) { + keyFilterColumns.insert(TString(col)); + } + for (auto path: section.Paths()) { + size_t keyLength = 0; + if (!path.Ranges().Maybe<TCoVoid>()) { + for (auto item: path.Ranges().Cast<TExprList>()) { + if (auto keyExact = item.Maybe<TYtKeyExact>()) { + keyLength = Max(keyLength, keyExact.Cast().Key().Size()); + } else if (auto keyRange = item.Maybe<TYtKeyRange>()) { + keyLength = Max(keyLength, keyRange.Cast().Lower().Size(), keyRange.Cast().Upper().Size()); + } + } + } + if (keyLength) { + auto rowSpec = TYtTableBaseInfo::GetRowSpec(path.Table()); + YQL_ENSURE(rowSpec); + YQL_ENSURE(keyLength <= rowSpec->SortedBy.size()); + keyFilterColumns.insert(rowSpec->SortedBy.begin(), rowSpec->SortedBy.begin() + keyLength); + } + } + } + + if (!keyFilterColumns.empty()) { + writer.OnKeyedItem("InputKeyFilterColumns"); + writer.OnBeginList(); + for (auto column : keyFilterColumns) { + writer.OnListItem(); + writer.OnStringScalar(column); + } + writer.OnEndList(); + } + } + + static const EYtSettingType specialSettings[] = {EYtSettingType::FirstAsPrimary, EYtSettingType::JoinReduce}; + if (AnyOf(specialSettings, [&op](const auto& setting) { return NYql::HasSetting(op.Settings().Ref(), setting); })) { + writer.OnKeyedItem("Settings"); + writer.OnBeginMap(); + + for (auto setting: specialSettings) { + if (NYql::HasSetting(op.Settings().Ref(), setting)) { + writer.OnKeyedItem(ToString(setting)); + writer.OnStringScalar("true"); + } + } + + if (NYql::UseJoinReduceForSecondAsPrimary(op.Settings().Ref())) { + writer.OnKeyedItem(NYql::JoinReduceForSecondAsPrimaryName); + writer.OnStringScalar("true"); + } + + writer.OnEndMap(); + } + } + + if (auto maybeOp = TMaybeNode<TYtMap>(&node)) { + writer.OnKeyedItem("Streams"); + writer.OnBeginMap(); + NCommon::WriteStreams(writer, "Mapper", maybeOp.Cast().Mapper()); + writer.OnEndMap(); + } else if (auto maybeOp = TMaybeNode<TYtReduce>(&node)) { + writer.OnKeyedItem("Streams"); + writer.OnBeginMap(); + NCommon::WriteStreams(writer, "Reducer", maybeOp.Cast().Reducer()); + writer.OnEndMap(); + } else if (auto maybeOp = TMaybeNode<TYtMapReduce>(&node)) { + writer.OnKeyedItem("Streams"); + writer.OnBeginMap(); + if (auto maybeLambda = maybeOp.Cast().Mapper().Maybe<TCoLambda>()) { + NCommon::WriteStreams(writer, "Mapper", maybeLambda.Cast()); + } + + NCommon::WriteStreams(writer, "Reducer", maybeOp.Cast().Reducer()); + writer.OnEndMap(); + } + } + + TString GetProviderPath(const TExprNode& node) override { + return TStringBuilder() << YtProviderName << '.' << node.Child(1)->Content(); + } + + void WriteDetails(const TExprNode& node, NYson::TYsonWriter& writer) override { + writer.OnKeyedItem("Cluster"); + writer.OnStringScalar(node.Child(1)->Content()); + } + + ui32 GetInputs(const TExprNode& node, TVector<TPinInfo>& inputs, bool withLimits) override { + ui32 count = 0; + if (auto maybeOp = TMaybeNode<TYtTransientOpBase>(&node)) { + auto op = maybeOp.Cast(); + for (auto section: op.Input()) { + ui32 i = 0; + for (auto path: section.Paths()) { + if (auto maybeTable = path.Table().Maybe<TYtTable>()) { + inputs.push_back(TPinInfo(nullptr, op.DataSink().Raw(), path.Raw(), MakeTableDisplayName(maybeTable.Cast(), false), false)); + } + else { + auto tmpTable = GetOutTable(path.Table()); + inputs.push_back(TPinInfo(nullptr, op.DataSink().Raw(), tmpTable.Raw(), MakeTableDisplayName(tmpTable, false), true)); + } + if (withLimits && State_->PlanLimits && ++i >= State_->PlanLimits) { + break; + } + } + count += section.Paths().Size(); + } + } + else if (auto maybePublish = TMaybeNode<TYtPublish>(&node)) { + auto publish = maybePublish.Cast(); + ui32 i = 0; + for (auto out: publish.Input()) { + auto tmpTable = GetOutTable(out); + inputs.push_back(TPinInfo(nullptr, publish.DataSink().Raw(), tmpTable.Raw(), MakeTableDisplayName(tmpTable, false), true)); + if (withLimits && State_->PlanLimits && ++i >= State_->PlanLimits) { + break; + } + } + count = publish.Input().Size(); + } else if (auto maybeStatOut = TMaybeNode<TYtStatOut>(&node)) { + auto statOut = maybeStatOut.Cast(); + auto table = GetOutTable(statOut.Input()); + inputs.push_back(TPinInfo(nullptr, statOut.DataSink().Raw(), table.Raw(), MakeTableDisplayName(table, false), true)); + count = 1; + } + return count; + } + + ui32 GetOutputs(const TExprNode& node, TVector<TPinInfo>& outputs, bool withLimits) override { + Y_UNUSED(withLimits); + ui32 count = 0; + if (auto maybeOp = TMaybeNode<TYtOutputOpBase>(&node)) { + auto op = maybeOp.Cast(); + for (auto table: op.Output()) { + outputs.push_back(TPinInfo(nullptr, op.DataSink().Raw(), table.Raw(), MakeTableDisplayName(table, true), true)); + } + count = op.Output().Size(); + } + else if (auto maybePublish = TMaybeNode<TYtPublish>(&node)) { + auto publish = maybePublish.Cast(); + outputs.push_back(TPinInfo(nullptr, publish.DataSink().Raw(), publish.Publish().Raw(), MakeTableDisplayName(publish.Publish(), true), false)); + count = 1; + } else if (auto maybeStatOut = TMaybeNode<TYtStatOut>(&node)) { + auto statOut = maybeStatOut.Cast(); + auto statTable = statOut.Table(); + outputs.push_back(TPinInfo(nullptr, statOut.DataSink().Raw(), statTable.Raw(), "(tmp)", true)); + count = 1; + } else if (auto maybeWrite = TMaybeNode<TYtWriteTable>(&node)) { + auto write = maybeWrite.Cast(); + outputs.push_back(TPinInfo(nullptr, write.DataSink().Raw(), write.Table().Raw(), MakeTableDisplayName(write.Table(), true), true)); + count = 1; + } + return count; + } + + void WritePinDetails(const TExprNode& node, NYson::TYsonWriter& writer) override { + writer.OnKeyedItem("Table"); + if (auto path = TMaybeNode<TYtPath>(&node)) { + const auto& table = path.Cast().Table().Cast<TYtTable>(); + writer.OnStringScalar(table.Name().Value()); + writer.OnKeyedItem("Type"); + auto rowType = path.Ref().GetTypeAnn()->Cast<TListExprType>()->GetItemType(); + NCommon::WriteTypeToYson(writer, rowType); + } else if (auto table = TMaybeNode<TYtTable>(&node)) { + writer.OnStringScalar(table.Cast().Name().Value()); + const auto tableInfo = TYtTableInfo(table.Cast()); + auto& desc = State_->TablesData->GetTable(tableInfo.Cluster, tableInfo.Name, tableInfo.CommitEpoch); + if (desc.RowSpec) { + writer.OnKeyedItem("Type"); + auto rowType = desc.RowSpec->GetType(); + NCommon::WriteTypeToYson(writer, rowType); + } + } else { + writer.OnStringScalar("(tmp)"); + } + } + + TString GetOperationDisplayName(const TExprNode& node) override { + if (auto maybeCommit = TMaybeNode<TCoCommit>(&node)) { + auto commit = maybeCommit.Cast(); + + TStringBuilder res; + res << node.Content() << " on " << commit.DataSink().Cast<TYtDSink>().Cluster().Value(); + if (commit.Settings()) { + if (auto epochNode = NYql::GetSetting(commit.Settings().Cast().Ref(), "epoch")) { + res << " #" << epochNode->Child(1)->Content(); + } + } + return res; + } + + return TString{node.Content()}; + } + + bool WriteSchemaHeader(NYson::TYsonWriter& writer) override { + writer.OnKeyedItem("YtSchema"); + return true; + } + + void WriteTypeDetails(NYson::TYsonWriter& writer, const TTypeAnnotationNode& type) override { + writer.OnStringScalar(GetTypeV3String(type)); + } + + ITrackableNodeProcessor& GetTrackableNodeProcessor() override { + return *TrackableNodeProcessor_; + } + + IDqIntegration* GetDqIntegration() override { + return State_->DqIntegration_.Get(); + } + +private: + static void WriteColumns(NYson::TYsonWriter& writer, TExprBase columns) { + if (auto maybeList = columns.Maybe<TExprList>()) { + writer.OnBeginList(); + for (const auto& column : maybeList.Cast()) { + writer.OnListItem(); + if (auto atom = column.Maybe<TCoAtom>()) { + writer.OnStringScalar(atom.Cast().Value()); + } + else { + writer.OnStringScalar(column.Cast<TExprList>().Item(0).Cast<TCoAtom>().Value()); + } + } + writer.OnEndList(); + } else if (columns.Maybe<TCoVoid>()) { + writer.OnStringScalar("*"); + } else { + writer.OnStringScalar("?"); + } + } + +private: + ui32 NextMutationId_ = 0; + TYtState::TPtr State_; + TLazyInitHolder<IGraphTransformer> IntentDeterminationTransformer_; + TLazyInitHolder<TVisitorTransformerBase> TypeAnnotationTransformer_; + TLazyInitHolder<IGraphTransformer> ConstraintTransformer_; + TLazyInitHolder<IGraphTransformer> SubConstraintTransformer_; + TLazyInitHolder<TExecTransformerBase> ExecTransformer_; + TLazyInitHolder<IGraphTransformer> LogicalOptProposalTransformer_; + TLazyInitHolder<IGraphTransformer> PhysicalOptProposalTransformer_; + TLazyInitHolder<IGraphTransformer> PhysicalFinalizingTransformer_; + TLazyInitHolder<IGraphTransformer> FinalizingTransformer_; + TLazyInitHolder<ITrackableNodeProcessor> TrackableNodeProcessor_; +}; + +TIntrusivePtr<IDataProvider> CreateYtDataSink(TYtState::TPtr state) { + return new TYtDataSink(state); +} + +} diff --git a/yt/yql/providers/yt/provider/yql_yt_datasink_constraints.cpp b/yt/yql/providers/yt/provider/yql_yt_datasink_constraints.cpp new file mode 100644 index 0000000000..c9978f95aa --- /dev/null +++ b/yt/yql/providers/yt/provider/yql_yt_datasink_constraints.cpp @@ -0,0 +1,468 @@ +#include "yql_yt_provider_impl.h" +#include "yql_yt_helpers.h" +#include "yql_yt_join_impl.h" + +#include <yt/yql/providers/yt/expr_nodes/yql_yt_expr_nodes.h> +#include <yql/essentials/providers/common/transform/yql_visit.h> +#include <yql/essentials/providers/common/provider/yql_provider.h> +#include <yql/essentials/core/yql_expr_constraint.h> +#include <yql/essentials/ast/yql_constraint.h> + +#include <util/generic/xrange.h> + +namespace NYql { + +using namespace NNodes; + +namespace { + +class TYtDataSinkConstraintTransformer : public TVisitorTransformerBase { +public: + TYtDataSinkConstraintTransformer(TYtState::TPtr state, bool subGraph) + : TVisitorTransformerBase(false) + , State_(state) + , SubGraph(subGraph) + { + AddHandler({TYtOutTable::CallableName()}, Hndl(&TYtDataSinkConstraintTransformer::HandleOutTable)); + AddHandler({TYtOutput::CallableName()}, Hndl(&TYtDataSinkConstraintTransformer::HandleOutput)); + AddHandler({TYtSort::CallableName()}, Hndl(&TYtDataSinkConstraintTransformer::HandleTransientOp)); + AddHandler({TYtCopy::CallableName()}, Hndl(&TYtDataSinkConstraintTransformer::HandleTransientOp)); + AddHandler({TYtMerge::CallableName()}, Hndl(&TYtDataSinkConstraintTransformer::HandleTransientOp)); + AddHandler({TYtMap::CallableName()}, Hndl(&TYtDataSinkConstraintTransformer::HandleUserJobOp<TYtMap::idx_Mapper, TYtMap::idx_Mapper>)); + AddHandler({TYtReduce::CallableName()}, Hndl(&TYtDataSinkConstraintTransformer::HandleUserJobOp<TYtReduce::idx_Reducer, TYtReduce::idx_Reducer>)); + AddHandler({TYtMapReduce::CallableName()}, Hndl(&TYtDataSinkConstraintTransformer::HandleUserJobOp<TYtMapReduce::idx_Mapper, TYtMapReduce::idx_Reducer>)); + AddHandler({TYtWriteTable::CallableName()}, Hndl(&TYtDataSinkConstraintTransformer::HandleWriteTable)); + AddHandler({TYtFill::CallableName()}, Hndl(&TYtDataSinkConstraintTransformer::HandleFill)); + AddHandler({TYtTouch::CallableName()}, Hndl(&TYtDataSinkConstraintTransformer::HandleTouch)); + AddHandler({TYtDropTable::CallableName()}, Hndl(&TYtDataSinkConstraintTransformer::HandleDefault)); + AddHandler({TCoCommit::CallableName()}, Hndl(&TYtDataSinkConstraintTransformer::HandleCommit)); + AddHandler({TYtPublish::CallableName()}, Hndl(&TYtDataSinkConstraintTransformer::HandlePublish)); + AddHandler({TYtEquiJoin::CallableName()}, Hndl(&TYtDataSinkConstraintTransformer::HandleEquiJoin)); + AddHandler({TYtStatOutTable::CallableName()}, Hndl(&TYtDataSinkConstraintTransformer::HandleDefault)); + AddHandler({TYtStatOut::CallableName()}, Hndl(&TYtDataSinkConstraintTransformer::HandleDefault)); + AddHandler({TYtDqProcessWrite ::CallableName()}, Hndl(&TYtDataSinkConstraintTransformer::HandleDqProcessWrite)); + AddHandler({TYtTryFirst ::CallableName()}, Hndl(&TYtDataSinkConstraintTransformer::HandleTryFirst)); + AddHandler({TYtMaterialize ::CallableName()}, Hndl(&TYtDataSinkConstraintTransformer::HandleMaterialize)); + } +private: + static void CopyExcept(TExprNode* dst, const TExprNode& from, const TStringBuf& except) { + for (const auto c: from.GetAllConstraints()) { + if (c->GetName() != except) { + dst->AddConstraint(c); + } + } + } + + TStatus HandleOutTable(TExprBase input, TExprContext& ctx) { + const auto table = input.Cast<TYtOutTable>(); + TConstraintSet set; + if (!table.RowSpec().Maybe<TCoVoid>()) { + TYqlRowSpecInfo rowSpec(table.RowSpec(), false); + set = rowSpec.GetAllConstraints(ctx); + + if (!set.GetConstraint<TSortedConstraintNode>()) { + if (const auto sorted = rowSpec.MakeSortConstraint(ctx)) + set.AddConstraint(sorted); + } + } + + if (!(set.GetConstraint<TDistinctConstraintNode>() || set.GetConstraint<TUniqueConstraintNode>())) { + if (const auto& uniqueBy = NYql::GetSetting(table.Settings().Ref(), EYtSettingType::UniqueBy)) { + std::vector<std::string_view> columns; + columns.reserve(uniqueBy->Tail().ChildrenSize()); + uniqueBy->Tail().ForEachChild([&columns](const TExprNode& column) { columns.emplace_back(column.Content()); }); + set.AddConstraint(ctx.MakeConstraint<TUniqueConstraintNode>(columns)); + set.AddConstraint(ctx.MakeConstraint<TDistinctConstraintNode>(columns)); + } + } + + if (!table.Stat().Maybe<TCoVoid>()) { + if (TYtTableStatInfo(table.Stat()).IsEmpty()) { + set.AddConstraint(ctx.MakeConstraint<TEmptyConstraintNode>()); + } + } + input.Ptr()->SetConstraints(set); + return TStatus::Ok; + } + + TStatus HandleOutput(TExprBase input, TExprContext& ctx) { + auto out = input.Cast<TYtOutput>(); + auto op = GetOutputOp(out); + const bool skipSort = IsUnorderedOutput(out); + if (op.Output().Size() > 1) { + if (auto multi = op.Ref().GetConstraint<TMultiConstraintNode>()) { + if (auto constraints = multi->GetItem(FromString<ui32>(out.OutIndex().Value()))) { + for (auto c: constraints->GetAllConstraints()) { + if (!skipSort || c->GetName() != TSortedConstraintNode::Name()) { + input.Ptr()->AddConstraint(c); + } + } + } else { + input.Ptr()->AddConstraint(ctx.MakeConstraint<TEmptyConstraintNode>()); + } + } + if (auto empty = op.Ref().GetConstraint<TEmptyConstraintNode>()) { + input.Ptr()->AddConstraint(empty); + } + } else { + for (auto c: op.Ref().GetAllConstraints()) { + if (!skipSort || c->GetName() != TSortedConstraintNode::Name()) { + input.Ptr()->AddConstraint(c); + } + } + } + + return TStatus::Ok; + } + + template <size_t InLambdaNdx, size_t OutLambdaNdx> + TStatus HandleUserJobOp(const TExprNode::TPtr& input, TExprNode::TPtr& output, TExprContext& ctx) { + auto op = TYtWithUserJobsOpBase(input); + TExprNode::TPtr lambda = input->ChildPtr(InLambdaNdx); + size_t lambdaNdx = InLambdaNdx; + + bool singleLambda = InLambdaNdx == OutLambdaNdx; + if (!singleLambda && TCoVoid::Match(lambda.Get())) { + singleLambda = true; + lambda = input->ChildPtr(OutLambdaNdx); + lambdaNdx = OutLambdaNdx; + } + + bool emptyInput = false; + const auto filter = NYql::HasSetting(op.Settings().Ref(), EYtSettingType::Ordered) ? + [](const std::string_view& name) { return TEmptyConstraintNode::Name() == name || TUniqueConstraintNode::Name() == name || TDistinctConstraintNode::Name() == name || TSortedConstraintNode::Name() == name; }: + [](const std::string_view& name) { return TEmptyConstraintNode::Name() == name || TUniqueConstraintNode::Name() == name || TDistinctConstraintNode::Name() == name; }; + TConstraintNode::TListType argConstraints; + if (op.Input().Size() > 1) { + TMultiConstraintNode::TMapType multiItems; + emptyInput = true; + for (ui32 index = 0; index < op.Input().Size(); ++index) { + auto section = op.Input().Item(index); + if (!section.Ref().GetConstraint<TEmptyConstraintNode>()) { + multiItems.push_back(std::make_pair(index, section.Ref().GetConstraintSet())); + multiItems.back().second.FilterConstraints(filter); + emptyInput = false; + } + } + if (!multiItems.empty()) { + argConstraints.push_back(ctx.MakeConstraint<TMultiConstraintNode>(std::move(multiItems))); + } else if (emptyInput) { + argConstraints.push_back(ctx.MakeConstraint<TEmptyConstraintNode>()); + } + } else { + auto set = op.Input().Item(0).Ref().GetConstraintSet(); + emptyInput = nullptr != set.GetConstraint<TEmptyConstraintNode>(); + set.FilterConstraints(filter); + argConstraints = set.GetAllConstraints(); + if (singleLambda) { + if (const auto& reduceBy = NYql::GetSettingAsColumnList(op.Settings().Ref(), EYtSettingType::ReduceBy); !reduceBy.empty()) { + TPartOfConstraintBase::TSetOfSetsType sets; + sets.reserve(reduceBy.size()); + std::transform(reduceBy.cbegin(), reduceBy.cend(), std::back_inserter(sets), [&ctx](const TString& column) { return TPartOfConstraintBase::TSetType{TPartOfConstraintBase::TPathType(1U, ctx.AppendString(column))}; }); + argConstraints.push_back(ctx.MakeConstraint<TChoppedConstraintNode>(std::move(sets))); + } + } + } + + auto status = UpdateLambdaConstraints(lambda, ctx, {argConstraints}); + if (lambda != input->ChildPtr(lambdaNdx)) { + output = ctx.ChangeChild(*input, lambdaNdx, std::move(lambda)); + status = status.Combine(TStatus::Repeat); + return status; + } + if (status != TStatus::Ok) { + return status; + } + + if (!singleLambda) { + TConstraintNode::TListType argConstraints; + if (auto empty = lambda->GetConstraint<TEmptyConstraintNode>()) { + argConstraints.push_back(empty); + } + + if (const auto& reduceBy = NYql::GetSettingAsColumnList(op.Settings().Ref(), EYtSettingType::ReduceBy); !reduceBy.empty()) { + TPartOfConstraintBase::TSetOfSetsType sets; + sets.reserve(reduceBy.size()); + std::transform(reduceBy.cbegin(), reduceBy.cend(), std::back_inserter(sets), [&ctx](const TString& column) { return TPartOfConstraintBase::TSetType{TPartOfConstraintBase::TPathType(1U, ctx.AppendString(column))}; }); + argConstraints.push_back(ctx.MakeConstraint<TChoppedConstraintNode>(std::move(sets))); + } + + TExprNode::TPtr outLambda = input->ChildPtr(OutLambdaNdx); + auto status = UpdateLambdaConstraints(outLambda, ctx, {argConstraints}); + if (outLambda != input->ChildPtr(OutLambdaNdx)) { + output = ctx.ChangeChild(*input, OutLambdaNdx, std::move(outLambda)); + status = status.Combine(TStatus::Repeat); + } + if (status != TStatus::Ok) { + return status; + } + } + + SetResultConstraint(input, *input->Child(OutLambdaNdx), op.Output(), ctx); + if (emptyInput) { + input->AddConstraint(ctx.MakeConstraint<TEmptyConstraintNode>()); + } + + return TStatus::Ok; + } + + TStatus HandleFill(TExprBase input, TExprContext& ctx) { + auto fill = input.Cast<TYtFill>(); + auto status = UpdateLambdaConstraints(fill.Content().Ref()); + if (status != TStatus::Ok) { + return status; + } + + SetResultConstraint(input.Ptr(), fill.Content().Ref(), fill.Output(), ctx); + return TStatus::Ok; + } + + static TConstraintNode::TListType GetConstraintsForInputArgument(const TConstraintSet& set, TExprContext& ctx) { + TConstraintNode::TListType argsConstraints; + if (auto mapping = TPartOfUniqueConstraintNode::GetCommonMapping(set.GetConstraint<TUniqueConstraintNode>()); !mapping.empty()) { + argsConstraints.emplace_back(ctx.MakeConstraint<TPartOfUniqueConstraintNode>(std::move(mapping))); + } + if (auto mapping = TPartOfDistinctConstraintNode::GetCommonMapping(set.GetConstraint<TDistinctConstraintNode>()); !mapping.empty()) { + argsConstraints.emplace_back(ctx.MakeConstraint<TPartOfDistinctConstraintNode>(std::move(mapping))); + } + return argsConstraints; + } + + TStatus HandleEquiJoin(TExprBase input, TExprContext& ctx) { + const auto equiJoin = input.Cast<TYtEquiJoin>(); + TStatus status = TStatus::Ok; + for (const auto i : xrange(equiJoin.Input().Size())) { + const auto premapIndex = i + 7U; + if (equiJoin.Ref().Child(premapIndex)->IsLambda()) { + status = status.Combine(UpdateLambdaConstraints(equiJoin.Ptr()->ChildRef(premapIndex), ctx, {GetConstraintsForInputArgument(equiJoin.Input().Item(i).Ref().GetConstraintSet(), ctx)})); + if (status == TStatus::Error) + return status; + } + } + if (status != TStatus::Ok) + return status; + + input.Ptr()->SetConstraints(ImportYtEquiJoin(equiJoin, ctx)->Constraints); + return TStatus::Ok; + } + + TStatus HandleTransientOp(TExprBase input, TExprContext& ctx) { + auto status = HandleDefault(input, ctx); + if (status != TStatus::Ok) { + return status; + } + auto op = input.Cast<TYtTransientOpBase>(); + YQL_ENSURE(op.Input().Size() == 1); + YQL_ENSURE(op.Output().Size() == 1); + + input.Ptr()->CopyConstraints(op.Output().Item(0).Ref()); + if (auto empty = op.Input().Item(0).Ref().GetConstraint<TEmptyConstraintNode>()) { + input.Ptr()->AddConstraint(empty); + } + + return TStatus::Ok; + } + + TStatus HandleTouch(TExprBase input, TExprContext& ctx) { + auto status = HandleDefault(input, ctx); + if (status != TStatus::Ok) { + return status; + } + input.Ptr()->CopyConstraints(input.Cast<TYtTouch>().Output().Item(0).Ref()); + input.Ptr()->AddConstraint(ctx.MakeConstraint<TEmptyConstraintNode>()); + return TStatus::Ok; + } + + TStatus HandleDefault(TExprBase input, TExprContext& /*ctx*/) { + return UpdateAllChildLambdasConstraints(input.Ref()); + } + + TStatus HandleWriteTable(TExprBase input, TExprContext& ctx) { + if (SubGraph) { + return TStatus::Ok; + } + + auto writeTable = input.Cast<TYtWriteTable>(); + + const bool initialWrite = NYql::HasSetting(writeTable.Settings().Ref(), EYtSettingType::Initial); + const auto outTableInfo = TYtTableInfo(writeTable.Table()); + + if (auto commitEpoch = outTableInfo.CommitEpoch.GetOrElse(0)) { + const auto cluster = TString{writeTable.DataSink().Cluster().Value()}; + TYtTableDescription& nextDescription = State_->TablesData->GetModifTable(cluster, outTableInfo.Name, commitEpoch); + + if (initialWrite) { + nextDescription.ConstraintsReady = false; + nextDescription.Constraints.Clear(); + if (nextDescription.IsReplaced) { + nextDescription.Constraints = writeTable.Content().Ref().GetConstraintSet(); + } else { + const TYtTableDescription& description = State_->TablesData->GetTable(cluster, outTableInfo.Name, outTableInfo.Epoch); + YQL_ENSURE(description.ConstraintsReady); + nextDescription.Constraints = description.Constraints; + } + } + + if (!initialWrite || !nextDescription.IsReplaced) { + if (const auto tableSort = nextDescription.Constraints.RemoveConstraint<TSortedConstraintNode>()) { + if (const auto contentSort = writeTable.Content().Ref().GetConstraint<TSortedConstraintNode>()) { + if (const auto commonSort = tableSort->MakeCommon(contentSort, ctx)) { + nextDescription.Constraints.AddConstraint(commonSort); + } + } + } + if (!writeTable.Content().Ref().GetConstraint<TEmptyConstraintNode>()) { + nextDescription.Constraints.RemoveConstraint<TEmptyConstraintNode>(); + } + nextDescription.Constraints.RemoveConstraint<TUniqueConstraintNode>(); + nextDescription.Constraints.RemoveConstraint<TDistinctConstraintNode>(); + } + } + + return TStatus::Ok; + } + + TStatus HandlePublish(TExprBase input, TExprContext& ctx) { + if (SubGraph) { + return TStatus::Ok; + } + + auto publish = input.Cast<TYtPublish>(); + + const bool initialWrite = NYql::HasSetting(publish.Settings().Ref(), EYtSettingType::Initial); + const auto outTableInfo = TYtTableInfo(publish.Publish()); + + if (auto commitEpoch = outTableInfo.CommitEpoch.GetOrElse(0)) { + + const auto cluster = TString{publish.DataSink().Cluster().Value()}; + const auto tableName = TString{TYtTableInfo::GetTableLabel(publish.Publish())}; + TYtTableDescription& nextDescription = State_->TablesData->GetModifTable(cluster, tableName, commitEpoch); + + if (initialWrite) { + nextDescription.ConstraintsReady = false; + nextDescription.Constraints.Clear(); + if (nextDescription.IsReplaced) { + nextDescription.Constraints = publish.Input().Item(0).Ref().GetConstraintSet(); + } else { + const TYtTableDescription& description = State_->TablesData->GetTable(cluster, tableName, outTableInfo.Epoch); + YQL_ENSURE(description.ConstraintsReady); + nextDescription.Constraints = description.Constraints; + } + } + + const size_t from = nextDescription.IsReplaced ? 1 : 0; + auto tableSort = nextDescription.Constraints.RemoveConstraint<TSortedConstraintNode>(); + for (size_t i = from; i < publish.Input().Size() && tableSort; ++i) { + tableSort = tableSort->MakeCommon(publish.Input().Item(i).Ref().GetConstraint<TSortedConstraintNode>(), ctx); + } + if (tableSort) { + nextDescription.Constraints.AddConstraint(tableSort); + } + if (AnyOf(publish.Input(), [](const TYtOutput& out) { return !out.Ref().GetConstraint<TEmptyConstraintNode>(); })) { + nextDescription.Constraints.RemoveConstraint<TEmptyConstraintNode>(); + } + if (publish.Input().Size() > (nextDescription.IsReplaced ? 1 : 0)) { + nextDescription.Constraints.RemoveConstraint<TUniqueConstraintNode>(); + nextDescription.Constraints.RemoveConstraint<TDistinctConstraintNode>(); + } + } + + return TStatus::Ok; + } + + TStatus HandleCommit(TExprBase input, TExprContext& ctx) { + if (SubGraph) { + return TStatus::Ok; + } + + const auto commit = input.Cast<TCoCommit>(); + const auto settings = NCommon::ParseCommitSettings(commit, ctx); + + if (settings.Epoch) { + const ui32 epoch = FromString(settings.Epoch.Cast().Value()); + for (const auto& clusterAndTable : State_->TablesData->GetAllEpochTables(epoch)) { + State_->TablesData->GetModifTable(clusterAndTable.first, clusterAndTable.second, epoch).SetConstraintsReady(); + } + } + return TStatus::Ok; + } + +private: + void SetResultConstraint(const TExprNode::TPtr& input, const TExprNode& source, const TYtOutSection& outputs, TExprContext& ctx) { + if (outputs.Size() == 1) { + auto out = outputs.Item(0); + input->CopyConstraints(out.Ref()); + if (auto empty = source.GetConstraint<TEmptyConstraintNode>()) { + input->AddConstraint(empty); + if (!out.Stat().Maybe<TCoVoid>() && State_->Types->IsConstraintCheckEnabled<TEmptyConstraintNode>()) { + YQL_ENSURE(out.Ref().GetConstraint<TEmptyConstraintNode>(), "Invalid Empty constraint"); + } + } + } else { + TMultiConstraintNode::TMapType multiItems; + auto multi = source.GetConstraint<TMultiConstraintNode>(); + auto empty = source.GetConstraint<TEmptyConstraintNode>(); + if (multi) { + multiItems = multi->GetItems(); + } + for (ui32 i = 0; i < outputs.Size(); ++i) { + auto out = outputs.Item(i); + bool addEmpty = false; + if ((multi && !multiItems.has(i)) || empty) { + if (!out.Stat().Maybe<TCoVoid>() && State_->Types->IsConstraintCheckEnabled<TEmptyConstraintNode>()) { + YQL_ENSURE(out.Ref().GetConstraint<TEmptyConstraintNode>(), "Invalid Empty constraint"); + } + addEmpty = true; + } + multiItems[i] = out.Ref().GetConstraintSet(); + if (addEmpty) { + multiItems[i].AddConstraint(ctx.MakeConstraint<TEmptyConstraintNode>()); + } + } + if (!multiItems.empty()) { + input->AddConstraint(ctx.MakeConstraint<TMultiConstraintNode>(std::move(multiItems))); + } + if (empty) { + input->AddConstraint(empty); + } + } + } + + TStatus HandleDqProcessWrite(TExprBase input, TExprContext& ctx) { + if (const auto status = HandleDefault(input, ctx); status != TStatus::Ok) { + return status; + } + bool complete = input.Ref().GetState() >= TExprNode::EState::ExecutionComplete; + if (!complete && input.Ref().HasResult()) { + const auto& result = input.Ref().GetResult(); + complete = result.IsWorld(); + } + if (complete) + input.Ptr()->CopyConstraints(input.Cast<TYtDqProcessWrite>().Output().Item(0).Ref()); + else + CopyExcept(input.Ptr().Get(), input.Cast<TYtDqProcessWrite>().Output().Item(0).Ref(), TEmptyConstraintNode::Name()); + return TStatus::Ok; + } + + TStatus HandleTryFirst(TExprBase input, TExprContext&) { + input.Ptr()->CopyConstraints(input.Ref().Tail()); + return TStatus::Ok; + } + + TStatus HandleMaterialize(TExprBase input, TExprContext&) { + input.Ptr()->CopyConstraints(*input.Ref().Child(TYtMaterialize::idx_Input)); + return TStatus::Ok; + } + +private: + const TYtState::TPtr State_; + const bool SubGraph; +}; + +} + +THolder<IGraphTransformer> CreateYtDataSinkConstraintTransformer(TYtState::TPtr state, bool subGraph) { + return THolder(new TYtDataSinkConstraintTransformer(state, subGraph)); +} + +} diff --git a/yt/yql/providers/yt/provider/yql_yt_datasink_exec.cpp b/yt/yql/providers/yt/provider/yql_yt_datasink_exec.cpp new file mode 100644 index 0000000000..39a42b240d --- /dev/null +++ b/yt/yql/providers/yt/provider/yql_yt_datasink_exec.cpp @@ -0,0 +1,1005 @@ +#include "yql_yt_provider_impl.h" +#include "yql_yt_op_settings.h" +#include "yql_yt_op_hash.h" +#include "yql_yt_helpers.h" +#include "yql_yt_optimize.h" + +#include <yt/yql/providers/yt/gateway/lib/yt_helpers.h> +#include <yt/yql/providers/yt/expr_nodes/yql_yt_expr_nodes.h> +#include <yt/yql/providers/yt/common/yql_configuration.h> +#include <yt/yql/providers/yt/lib/expr_traits/yql_expr_traits.h> +#include <yt/yql/providers/yt/lib/hash/yql_hash_builder.h> +#include <yt/yql/providers/yt/provider/yql_yt_helpers.h> +#include <yql/essentials/providers/common/provider/yql_provider.h> +#include <yql/essentials/providers/common/transform/yql_exec.h> +#include <yql/essentials/providers/common/schema/expr/yql_expr_schema.h> +#include <yql/essentials/core/type_ann/type_ann_expr.h> +#include <yql/essentials/core/yql_execution.h> +#include <yql/essentials/core/yql_graph_transformer.h> +#include <yql/essentials/utils/log/log.h> +#include <yql/essentials/ast/yql_ast.h> + +#include <yql/essentials/providers/result/expr_nodes/yql_res_expr_nodes.h> + +#include <yt/cpp/mapreduce/common/helpers.h> + +#include <library/cpp/yson/node/node_io.h> +#include <library/cpp/yson/writer.h> + +#include <util/generic/overloaded.h> +#include <util/generic/xrange.h> +#include <util/generic/ylimits.h> +#include <util/generic/guid.h> +#include <util/generic/maybe.h> +#include <util/generic/scope.h> +#include <util/string/cast.h> +#include <util/string/hex.h> + +#include <algorithm> +#include <memory> + +namespace NYql { + +namespace { + +using namespace NNodes; +using namespace NThreading; + +bool NeedFallback(const TIssues& issues) { + for (const auto& issue : issues) + if (TIssuesIds::DQ_GATEWAY_NEED_FALLBACK_ERROR == issue.GetCode()) + return true; + + return false; +} + +TIssue WrapIssuesOnHybridFallback(TPosition pos, const TIssues& issues, TString fallbackOpName) { + TIssue result(pos, "Hybrid execution fallback on YT: " + fallbackOpName); + result.SetCode(TIssuesIds::DQ_GATEWAY_NEED_FALLBACK_ERROR, TSeverityIds::S_INFO); + + const std::function<void(TIssue& issue)> toInfo = [&](TIssue& issue) { + if (issue.Severity == TSeverityIds::S_ERROR + || issue.Severity == TSeverityIds::S_FATAL + || issue.Severity == TSeverityIds::S_WARNING) { + issue.Severity = TSeverityIds::S_INFO; + } + for (const auto& subissue : issue.GetSubIssues()) { + toInfo(*subissue); + } + }; + + for (const auto& issue : issues) { + TIssuePtr info(new TIssue(issue)); + toInfo(*info); + result.AddSubIssue(std::move(info)); + } + + return result; +} + +class TYtDataSinkExecTransformer : public TExecTransformerBase { +public: + TYtDataSinkExecTransformer(TYtState::TPtr state) + : State_(state) + , Delegated_(new TNodeMap<TDelegatedInfo>()) + { + AddHandler( + { + TYtSort::CallableName(), + TYtMap::CallableName(), + TYtCopy::CallableName(), + TYtMerge::CallableName(), + TYtMapReduce::CallableName(), + }, + RequireForTransientOp(), + Hndl(&TYtDataSinkExecTransformer::HandleOutputOp<true>) + ); + AddHandler( + { + TYtFill::CallableName(), + TYtTouch::CallableName(), + }, + RequireFirst(), + Hndl(&TYtDataSinkExecTransformer::HandleOutputOp<true>) + ); + AddHandler({TYtReduce::CallableName()}, RequireForTransientOp(), Hndl(&TYtDataSinkExecTransformer::HandleReduce)); + AddHandler({TYtOutput::CallableName()}, RequireFirst(), Pass()); + AddHandler({TYtPublish::CallableName()}, RequireAllOf({TYtPublish::idx_World, TYtPublish::idx_Input}), Hndl(&TYtDataSinkExecTransformer::HandlePublish)); + AddHandler({TYtDropTable::CallableName()}, RequireFirst(), Hndl(&TYtDataSinkExecTransformer::HandleDrop)); + AddHandler({TCoCommit::CallableName()}, RequireFirst(), Hndl(&TYtDataSinkExecTransformer::HandleCommit)); + AddHandler({TYtEquiJoin::CallableName()}, RequireSequenceOf({TYtEquiJoin::idx_World, TYtEquiJoin::idx_Input}), + Hndl(&TYtDataSinkExecTransformer::HandleEquiJoin)); + AddHandler({TYtStatOut::CallableName()}, RequireAllOf({TYtStatOut::idx_World, TYtStatOut::idx_Input}), + Hndl(&TYtDataSinkExecTransformer::HandleStatOut)); + AddHandler({TYtDqProcessWrite::CallableName()}, RequireFirst(), + Hndl(&TYtDataSinkExecTransformer::HandleYtDqProcessWrite)); + AddHandler({TYtTryFirst::CallableName()}, RequireFirst(), Hndl(&TYtDataSinkExecTransformer::HandleTryFirst)); + } + + void Rewind() override { + Delegated_->clear(); + TExecTransformerBase::Rewind(); + } + + static TExecTransformerBase::TPrerequisite RequireForTransientOp() { + return [] (const TExprNode::TPtr& input) { + auto status = RequireChild(*input, TYtTransientOpBase::idx_World); + // We have to run input only if it has no settings to calculate. + // Otherwise, we first of all wait world completion. + // Then begins node execution, which run settings calculation. + // And after that, starts input execution + // See YQL-19303 + if (!HasNodesToCalculate(input->ChildPtr(TYtTransientOpBase::idx_Input))) { + status = status.Combine(RequireChild(*input, TYtTransientOpBase::idx_Input)); + } + return status; + }; + } + +private: + static void PushHybridStats(const TYtState::TPtr& state, TStringBuf statName, TStringBuf opName, const TStringBuf& folderName = "") { + with_lock(state->StatisticsMutex) { + state->HybridStatistics[folderName].Entries.emplace_back(TString{statName}, 0, 0, 0, 0, 1); + state->HybridOpStatistics[opName][folderName].Entries.emplace_back(TString{statName}, 0, 0, 0, 0, 1); + } + } + + static TExprNode::TPtr FinalizeOutputOp(const TYtState::TPtr& state, const TString& operationHash, + const IYtGateway::TRunResult& res, const TExprNode::TPtr& input, TExprNode::TPtr& output, TExprContext& ctx, bool markFinished) + { + if (markFinished && !TYtDqProcessWrite::Match(input.Get())) { + PushHybridStats(state, "YtExecution", input->Content()); + } + auto outSection = TYtOutputOpBase(input).Output(); + YQL_ENSURE(outSection.Size() == res.OutTableStats.size(), "Invalid output table count in IYtGateway::TRunResult"); + TExprNode::TListType newOutTables; + for (size_t i: xrange(outSection.Size())) { + TYtOutTable outTable = outSection.Item(i); + TExprNode::TListType children = outTable.Raw()->ChildrenList(); + if (auto& name = children[TYtOutTable::idx_Name]; name->IsAtom("") && !res.OutTableStats[i].first.empty()) + name = ctx.NewAtom(name->Pos(), res.OutTableStats[i].first); + if (const auto stat = res.OutTableStats[i].second) + children[TYtOutTable::idx_Stat] = stat->ToExprNode(ctx, outTable.Pos()).Ptr(); + + newOutTables.push_back(ctx.ChangeChildren(outTable.Ref(), std::move(children))); + } + output = ctx.ChangeChild(*input, TYtOutputOpBase::idx_Output, ctx.NewList(outSection.Pos(), std::move(newOutTables))); + state->NodeHash.emplace(output->UniqueId(), operationHash); + return markFinished ? ctx.NewWorld(input->Pos()) : ctx.NewAtom(input->Pos(), ""); + } + + using TLaunchOpResult = std::variant<TFuture<IYtGateway::TRunResult>, TStatusCallbackPair>; + TLaunchOpResult LaunchOutputOp(TString& operationHash, const TExprNode::TPtr& input, TExprContext& ctx) { + TYtOutputOpBase op(input); + + if (auto opInput = op.Maybe<TYtTransientOpBase>().Input()) { + bool error = false; + for (auto section: opInput.Cast()) { + for (auto path: section.Paths()) { + if (auto table = path.Table().Maybe<TYtTable>()) { + auto tableInfo = TYtTableInfo(table.Cast()); + if (!tableInfo.Meta) { + ctx.AddError(TIssue(ctx.GetPosition(input->Pos()), TStringBuilder() << "Table " << tableInfo.Name.Quote() << " has no metadata")); + error = true; + } + if (!tableInfo.Stat) { + ctx.AddError(TIssue(ctx.GetPosition(input->Pos()), TStringBuilder() << "Table " << tableInfo.Name.Quote() << " has no stat")); + error = true; + } + } + } + } + if (error) { + return SyncError(); + } + } + + auto cluster = TString{op.DataSink().Cluster().Value()}; + // Scan entire node because inner lambda may contain YtTableContent with YtPath + TExprNode::TListType needCalc = GetNodesToCalculate(input); + if (!needCalc.empty()) { + YQL_CLOG(DEBUG, ProviderYt) << "Calculating nodes for " << input->Content() << " (UniqueId=" << input->UniqueId() << ")"; + return CalculateNodes(State_, input, cluster, needCalc, ctx); + } + + if (auto opInput = op.Maybe<TYtTransientOpBase>().Input()) { + YQL_ENSURE(opInput.Ref().GetState() == TExprNode::EState::ExecutionComplete); + } + + auto outSection = op.Output(); + + size_t outWithoutName = 0; + for (auto out: outSection) { + if (out.Name().Value().empty()) { + ++outWithoutName; + } + } + if (outWithoutName != outSection.Size()) { + ctx.AddError(TIssue(ctx.GetPosition(outSection.Pos()), TStringBuilder() << "Incomplete execution of " + << input->Content() << ", #" << input->UniqueId())); + return SyncError(); + } + + input->SetState(TExprNode::EState::ExecutionInProgress); + + auto newWorld = ctx.NewWorld(input->Child(0)->Pos()); + newWorld->SetTypeAnn(input->Child(0)->GetTypeAnn()); + newWorld->SetState(TExprNode::EState::ConstrComplete); + + TExprNode::TPtr clonedNode = ctx.ChangeChild(*input, 0, std::move(newWorld)); + clonedNode->SetTypeAnn(input->GetTypeAnn()); + clonedNode->CopyConstraints(*input); + + TExprNode::TPtr optimizedNode = clonedNode; + if (const auto status = SubstTables(optimizedNode, State_, false, ctx); status.Level == TStatus::Error) { + return SyncStatus(status); + } + + const auto settings = State_->Configuration->GetSettingsForNode(*input); + TUserDataTable crutches = State_->Types->UserDataStorageCrutches; + if (const auto& defaultGeobase = settings->GeobaseDownloadUrl.Get(cluster)) { + auto& userDataBlock = (crutches[TUserDataKey::File(TStringBuf("/home/geodata6.bin"))] = TUserDataBlock{EUserDataType::URL, {}, *defaultGeobase, {}, {}}); + userDataBlock.Usage.Set(EUserDataBlockUsage::Path); + } + + bool hasNonDeterministicFunctions = false; + if (const auto status = PeepHoleOptimizeBeforeExec(optimizedNode, optimizedNode, State_, hasNonDeterministicFunctions, ctx, false); status.Level != TStatus::Ok) { + return SyncStatus(status); + } + + TUserDataTable files; + auto filesRes = NCommon::FreezeUsedFiles(*optimizedNode, files, *State_->Types, ctx, MakeUserFilesDownloadFilter(*State_->Gateway, TString(cluster)), crutches); + if (filesRes.first.Level != TStatus::Ok) { + if (filesRes.first.Level != TStatus::Error) { + YQL_CLOG(DEBUG, ProviderYt) << "Freezing files for " << input->Content() << " (UniqueId=" << input->UniqueId() << ")"; + } + return filesRes; + } + + THashMap<TString, TString> secureParams; + NCommon::FillSecureParams(optimizedNode, *State_->Types, secureParams); + + auto config = State_->Configuration->GetSettingsForNode(*input); + const auto queryCacheMode = config->QueryCacheMode.Get().GetOrElse(EQueryCacheMode::Disable); + if (queryCacheMode != EQueryCacheMode::Disable) { + if (!hasNonDeterministicFunctions) { + operationHash = TYtNodeHashCalculator(State_, cluster, config).GetHash(*optimizedNode); + } + YQL_CLOG(DEBUG, ProviderYt) << "Operation hash: " << HexEncode(operationHash).Quote() + << ", cache mode: " << queryCacheMode; + } + + TSet<TString> addSecTags; + if (settings->TableContentDeliveryMode.Get(cluster) == ETableContentDeliveryMode::File || TYtFill::Match(input.Get())) { + for (size_t pos = 0; pos < optimizedNode->ChildrenSize(); pos++) { + auto childPtr = optimizedNode->ChildPtr(pos); + if (childPtr->Type() == TExprNode::Lambda) { + VisitExpr(childPtr->TailPtr(), [&addSecTags](const TExprNode::TPtr& node) -> bool { + if (TYtTableContent::Match(node.Get())) { + auto tableContent = TYtTableContent(node.Get()); + if (auto readTable = tableContent.Input().Maybe<TYtReadTable>()) { + for (auto section : readTable.Cast().Input()) { + for (auto path : section.Paths()) { + if (auto tableBase = path.Table().Maybe<TYtTableBase>()) { + if (auto stat = TYtTableBaseInfo::GetStat(tableBase.Cast())) { + for (const auto& tag : stat->SecurityTags) { + addSecTags.insert(tag); + } + } + } + } + } + } + return false; + } + return true; + }); + } + } + } + + YQL_CLOG(DEBUG, ProviderYt) << "Executing " << input->Content() << " (UniqueId=" << input->UniqueId() << ")"; + + return State_->Gateway->Run(optimizedNode, ctx, + IYtGateway::TRunOptions(State_->SessionId) + .UserDataBlocks(files) + .UdfModules(State_->Types->UdfModules) + .UdfResolver(State_->Types->UdfResolver) + .UdfValidateMode(State_->Types->ValidateMode) + .PublicId(State_->Types->TranslateOperationId(input->UniqueId())) + .Config(std::move(config)) + .OptLLVM(State_->Types->OptLLVM.GetOrElse(TString())) + .OperationHash(operationHash) + .SecureParams(secureParams) + .AdditionalSecurityTags(addSecTags) + ); + } + + template <bool MarkFinished> + TStatusCallbackPair HandleOutputOp(const TExprNode::TPtr& input, TExprContext& ctx) { + if (input->HasResult() && input->GetResult().Type() == TExprNode::World) { + return SyncOk(); + } + + TString operationHash; + TLaunchOpResult callbackPairOrFuture = LaunchOutputOp(operationHash, input, ctx); + return std::visit(TOverloaded{ + [&](const TStatusCallbackPair& pair) { + return pair; + }, + [&](TFuture<IYtGateway::TRunResult>& future) { + return WrapModifyFuture(future, [operationHash, state = State_](const IYtGateway::TRunResult& res, + const TExprNode::TPtr& input, + TExprNode::TPtr& output, + TExprContext& ctx) + { + return FinalizeOutputOp(state, operationHash, res, input, output, ctx, MarkFinished); + }); + } + }, callbackPairOrFuture); + } + + TStatusCallbackPair HandleTryFirst(const TExprNode::TPtr& input, TExprNode::TPtr& output, TExprContext&) { + + switch (input->Head().GetState()) { + case TExprNode::EState::ExecutionComplete: + PushHybridStats(State_, "Execution", input->TailPtr()->Content()); + output = input->HeadPtr(); + break; + case TExprNode::EState::Error: { + PushHybridStats(State_, "Fallback", input->TailPtr()->Content()); + if (State_->Configuration->HybridDqExecutionFallback.Get().GetOrElse(true)) { + output = input->TailPtr(); + } else { + input->SetState(TExprNode::EState::Error); + return SyncError(); + } + break; + } + default: + Y_UNREACHABLE(); + } + return SyncStatus(TStatus(TStatus::Repeat, true)); + } + + TStatusCallbackPair HandleReduce(const TExprNode::TPtr& input, TExprContext& ctx) { + TYtReduce reduce(input); + + if (!NYql::HasSetting(reduce.Settings().Ref(), EYtSettingType::FirstAsPrimary)) { + return HandleOutputOp<true>(input, ctx); + } + + if (input->HasResult() && input->GetResult().Type() == TExprNode::World) { + return SyncOk(); + } + + TString operationHash; + TLaunchOpResult callbackPairOrFuture = LaunchOutputOp(operationHash, input, ctx); + if (auto* pair = std::get_if<TStatusCallbackPair>(&callbackPairOrFuture)) { + return *pair; + } + + auto future = std::get<TFuture<IYtGateway::TRunResult>>(callbackPairOrFuture); + return std::make_pair(IGraphTransformer::TStatus::Async, future.Apply( + [operationHash, state = State_](const TFuture<IYtGateway::TRunResult>& completedFuture) { + return TAsyncTransformCallback( + [completedFuture, operationHash, state](const TExprNode::TPtr& input, TExprNode::TPtr& output, TExprContext& ctx) { + output = input; + const auto& res = completedFuture.GetValue(); + if (!res.Success()) { + YQL_ENSURE(!res.Issues().Empty()); + if (res.Issues().begin()->IssueCode == TIssuesIds::YT_MAX_DATAWEIGHT_PER_JOB_EXCEEDED) { + YQL_CLOG(INFO, ProviderYt) << "Execution of node: " << input->Content() + << " with FirstAsPrimary exceeds max dataweight per job, rebuilding node"; + TYtReduce reduce(input); + const bool joinReduceForSecond = NYql::UseJoinReduceForSecondAsPrimary( + reduce.Settings().Ref()); + auto settings = NYql::RemoveSettings(reduce.Settings().Ref(), + EYtSettingType::FirstAsPrimary | EYtSettingType::JoinReduce, ctx); + if (joinReduceForSecond) { + settings = NYql::AddSetting(*settings, EYtSettingType::JoinReduce, nullptr, ctx); + } + + output = ctx.ChangeChild(*input, TYtReduce::idx_Settings, std::move(settings)); + output->SetState(TExprNode::EState::Initial); + return IGraphTransformer::TStatus(IGraphTransformer::TStatus::Repeat, true); + } + } + + TIssueScopeGuard issueScope(ctx.IssueManager, [&]() { + return MakeIntrusive<TIssue>( + ctx.GetPosition(input->Pos()), + TStringBuilder() << "Execution of node: " << input->Content() << " with TryFirstAsPrimary"); + }); + res.ReportIssues(ctx.IssueManager); + + if (!res.Success()) { + input->SetState(TExprNode::EState::Error); + return IGraphTransformer::TStatus(IGraphTransformer::TStatus::Error); + } else { + TExprNode::TPtr resultNode = FinalizeOutputOp(state, operationHash, res, input, output, ctx, true); + input->SetState(TExprNode::EState::ExecutionComplete); + output->SetResult(std::move(resultNode)); + if (input != output) { + return IGraphTransformer::TStatus(IGraphTransformer::TStatus::Repeat, true); + } + return IGraphTransformer::TStatus(IGraphTransformer::TStatus::Ok); + } + }); + })); + } + + TStatusCallbackPair HandleDrop(const TExprNode::TPtr& input, TExprContext& ctx) { + input->SetState(TExprNode::EState::ExecutionInProgress); + + auto drop = TYtDropTable(input); + + auto newWorld = ctx.ShallowCopy(*input->Child(0)); + newWorld->SetTypeAnn(input->Child(0)->GetTypeAnn()); + newWorld->SetState(TExprNode::EState::ExecutionComplete); + + TExprNode::TPtr clonedNode = ctx.ChangeChild(*input, 0, std::move(newWorld)); + clonedNode->SetTypeAnn(input->GetTypeAnn()); + clonedNode->CopyConstraints(*input); + + auto status = SubstTables(clonedNode, State_, true, ctx); + if (status.Level == TStatus::Error) { + return SyncStatus(status); + } + + auto future = State_->Gateway->Run(clonedNode, ctx, + IYtGateway::TRunOptions(State_->SessionId) + .PublicId(State_->Types->TranslateOperationId(input->UniqueId())) + .Config(State_->Configuration->GetSettingsForNode(*input)) + ); + + return WrapFuture(future, [](const IYtGateway::TRunResult& /*res*/, const TExprNode::TPtr& input, TExprContext& ctx) { + return ctx.NewWorld(input->Pos()); + }); + } + + TStatusCallbackPair HandlePublish(const TExprNode::TPtr& input, TExprContext& ctx) { + auto publish = TYtPublish(input); + auto cluster = TString{publish.DataSink().Cluster().Value()}; + auto path = TString{ publish.Publish().Name().Value() }; + + auto commitEpoch = TEpochInfo::Parse(publish.Publish().CommitEpoch().Ref()).GetOrElse(0); + TYtTableDescription& nextDescription = State_->TablesData->GetModifTable(cluster, path, commitEpoch); + + auto config = State_->Configuration->GetSettingsForNode(*input); + + const auto mode = NYql::GetSetting(publish.Settings().Ref(), EYtSettingType::Mode); + const bool initial = NYql::HasSetting(publish.Settings().Ref(), EYtSettingType::Initial); + + auto dataHash = TYtNodeHashCalculator(State_, cluster, config).GetHash(publish.Input().Ref()); + YQL_CLOG(INFO, ProviderYt) << "Publish data hash \"" << HexEncode(dataHash) << "\" for table " << cluster << "." << path << "#" << commitEpoch; + TString nextHash; + if (nextDescription.IsReplaced && initial) { + nextHash = dataHash; + } else { + auto epoch = TEpochInfo::Parse(publish.Publish().Epoch().Ref()).GetOrElse(0); + const TYtTableDescription& readDescription = State_->TablesData->GetTable(cluster, path, epoch); + TString prevHash; + if (!initial) { + prevHash = nextDescription.Hash.GetOrElse({}); + } else { + if (readDescription.Hash) { + prevHash = *readDescription.Hash; + } else { + prevHash = TYtNodeHashCalculator(State_, cluster, config).GetHash(publish.Publish().Ref()); + } + } + YQL_CLOG(INFO, ProviderYt) << "Publish prev content hash \"" << HexEncode(prevHash) << "\" for table " << cluster << "." << path << "#" << commitEpoch; + if (!prevHash.empty() && !dataHash.empty()) { + THashBuilder builder; + builder << TYtNodeHashCalculator::MakeSalt(config, cluster) << prevHash << dataHash; + nextHash = builder.Finish(); + } + } + nextDescription.Hash = nextHash; + if (!nextDescription.Hash->empty()) { + YQL_CLOG(INFO, ProviderYt) << "Using publish hash \"" << HexEncode(*nextDescription.Hash) << "\" for table " << cluster << "." << path << "#" << commitEpoch; + } + + input->SetState(TExprNode::EState::ExecutionInProgress); + + auto newWorld = ctx.NewWorld(input->Child(0)->Pos()); + newWorld->SetTypeAnn(input->Child(0)->GetTypeAnn()); + newWorld->SetState(TExprNode::EState::ExecutionComplete); + + TExprNode::TPtr clonedNode = ctx.ChangeChild(*input, TYtPublish::idx_World, std::move(newWorld)); + clonedNode->SetTypeAnn(publish.Ref().GetTypeAnn()); + clonedNode->SetState(TExprNode::EState::ConstrComplete); + + auto status = SubstTables(clonedNode, State_, true, ctx); + if (status.Level == TStatus::Error) { + return SyncStatus(status); + } + + auto future = State_->Gateway->Publish(clonedNode, ctx, + IYtGateway::TPublishOptions(State_->SessionId) + .PublicId(State_->Types->TranslateOperationId(input->UniqueId())) + .DestinationRowSpec(nextDescription.RowSpec) + .Config(std::move(config)) + .OptLLVM(State_->Types->OptLLVM.GetOrElse(TString())) + .OperationHash(nextDescription.Hash.GetOrElse("")) + ); + + return WrapFuture(future, [](const IYtGateway::TPublishResult& /*res*/, const TExprNode::TPtr& input, TExprContext& ctx) { + return ctx.NewWorld(input->Pos()); + }); + } + + TStatusCallbackPair HandleCommit(const TExprNode::TPtr& input, TExprContext& ctx) { + auto commit = TCoCommit(input); + auto settings = NCommon::ParseCommitSettings(commit, ctx); + YQL_ENSURE(settings.Epoch); + + ui32 epoch = FromString(settings.Epoch.Cast().Value()); + auto cluster = commit.DataSink().Cast<TYtDSink>().Cluster().Value(); + + auto settingsVer = State_->Configuration->FindNodeVer(*input); + auto future = State_->Gateway->Commit( + IYtGateway::TCommitOptions(State_->SessionId) + .Cluster(TString{cluster}) + ); + + if (State_->EpochDependencies.contains(epoch)) { + auto state = State_; + return WrapModifyFuture(future, [state, epoch, settingsVer](const IYtGateway::TCommitResult& /*res*/, const TExprNode::TPtr& input, TExprNode::TPtr& output, TExprContext& ctx) { + input->SetState(TExprNode::EState::ExecutionComplete); // Don't execute this node again + output = ctx.ExactShallowCopy(*input); + state->LoadEpochMetadata.ConstructInPlace(epoch, settingsVer); + ctx.Step.Repeat(TExprStep::LoadTablesMetadata); + return ctx.NewWorld(input->Pos()); + }); + } + + return WrapFuture(future, [](const IYtGateway::TCommitResult& /*res*/, const TExprNode::TPtr& input, TExprContext& ctx) { + return ctx.NewWorld(input->Pos()); + }); + } + + TStatusCallbackPair HandleEquiJoin(const TExprNode::TPtr& input, TExprContext& ctx) { + ctx.AddError(TIssue(ctx.GetPosition(input->Pos()), TStringBuilder() << "Execution of " + << input->Content() << ", #" << input->UniqueId() << " should not be reached")); + return SyncError(); + } + + TStatusCallbackPair HandleStatOut(const TExprNode::TPtr& input, TExprContext& ctx) { + Y_UNUSED(ctx); + + auto config = State_->Configuration->GetSettingsForNode(*input); + auto future = State_->Gateway->Run(input, ctx, + IYtGateway::TRunOptions(State_->SessionId) + .Config(std::move(config)) + .PublicId(State_->Types->TranslateOperationId(input->UniqueId())) + ); + + return WrapFuture( + future, + [](const NCommon::TOperationResult& result, const TExprNode::TPtr& input, TExprContext& ctx) { + Y_UNUSED(result); + return ctx.NewWorld(input->Pos()); + } + ); + } + + TStatusCallbackPair HandleYtDqProcessWrite(const TExprNode::TPtr& input, TExprContext& ctx) { + const TYtDqProcessWrite op(input); + const auto section = op.Output().Cast<TYtOutSection>(); + Y_ENSURE(section.Size() == 1, "TYtDqProcessWrite expects 1 output table but got " << section.Size()); + const TYtOutTable tmpTable = section.Item(0); + + if (!input->HasResult()) { + if (!tmpTable.Name().Value().empty()) { + ctx.AddError(TIssue(ctx.GetPosition(section.Pos()), TStringBuilder() << "Incomplete execution of " + << input->Content() << ", #" << input->UniqueId())); + return SyncError(); + } + + input->SetState(TExprNode::EState::ExecutionInProgress); + + return MakeTableForDqWrite(input, ctx); + } + else if (const auto& result = input->GetResult(); result.IsAtom()) { + const auto publicId = State_->Types->TranslateOperationId(input->UniqueId()); + if (result.IsAtom("")) { + // Second iteration: do the actual write. + if (publicId) { + if (State_->HybridInFlightOprations.empty()) + State_->HybridStartTime = NMonotonic::TMonotonic::Now(); + State_->HybridInFlightOprations.emplace(*publicId); + } + return RunDqWrite(input, ctx, tmpTable); + } else { + if (publicId) { + YQL_ENSURE(State_->HybridInFlightOprations.erase(*publicId), "Operation " << *publicId << " not found."); + if (State_->HybridInFlightOprations.empty()) { + const auto interval = NMonotonic::TMonotonic::Now() - State_->HybridStartTime; + State_->TimeSpentInHybrid += interval; + with_lock(State_->StatisticsMutex) { + State_->Statistics[Max<ui32>()].Entries.emplace_back("HybridTimeSpent", 0, 0, 0, 0, interval.MilliSeconds()); + } + } + } + + if (result.IsAtom("FallbackOnError")) + return SyncOk(); + + YQL_ENSURE(result.IsAtom("DQ_completed"), "Unexpected result atom: " << result.Content()); + // Third iteration: collect temporary table statistics. + return CollectDqWrittenTableStats(input, ctx); + } + } + else { + // Fourth iteration: everything is done, return ok status. + Y_ENSURE(input->GetResult().Type() == TExprNode::World, "Unexpected result type: " << input->GetResult().Type()); + return SyncOk(); + } + } + +private: + TStatusCallbackPair MakeTableForDqWrite(const TExprNode::TPtr& input, TExprContext& ctx) { + if (input->HasResult() && input->GetResult().Type() == TExprNode::World) { + return SyncOk(); + } + + auto newWorld = ctx.NewWorld(input->Head().Pos()); + newWorld->SetTypeAnn(input->Head().GetTypeAnn()); + newWorld->SetState(TExprNode::EState::ConstrComplete); + + auto optimizedNode = ctx.ChangeChild(*input, 0, std::move(newWorld)); + optimizedNode->SetTypeAnn(input->GetTypeAnn()); + optimizedNode->CopyConstraints(*input); + + if (const auto status = SubstTables(optimizedNode, State_, false, ctx); status.Level == TStatus::Error) { + return SyncStatus(status); + } + + const TYtDqProcessWrite op(input); + const auto cluster = op.DataSink().Cluster().StringValue(); + const auto config = State_->Configuration->GetSettingsForNode(*input); + + TUserDataTable crutches = State_->Types->UserDataStorageCrutches; + if (const auto& defaultGeobase = config->GeobaseDownloadUrl.Get(cluster)) { + auto& userDataBlock = (crutches[TUserDataKey::File(TStringBuf("/home/geodata6.bin"))] = TUserDataBlock{EUserDataType::URL, {}, *defaultGeobase, {}, {}}); + userDataBlock.Usage.Set(EUserDataBlockUsage::Path); + } + + bool hasNonDeterministicFunctions = false; + if (const auto status = PeepHoleOptimizeBeforeExec(optimizedNode, optimizedNode, State_, hasNonDeterministicFunctions, ctx, false); status.Level != TStatus::Ok) { + return SyncStatus(status); + } + + TUserDataTable files; + if (const auto filesRes = NCommon::FreezeUsedFiles(*optimizedNode, files, *State_->Types, ctx, MakeUserFilesDownloadFilter(*State_->Gateway, TString(cluster)), crutches); + filesRes.first.Level != TStatus::Ok) { + if (filesRes.first.Level != TStatus::Error) { + YQL_CLOG(DEBUG, ProviderYt) << "Freezing files for " << input->Content() << " (UniqueId=" << input->UniqueId() << ")"; + } + return filesRes; + } + + TString operationHash; + if (const auto queryCacheMode = config->QueryCacheMode.Get().GetOrElse(EQueryCacheMode::Disable); queryCacheMode != EQueryCacheMode::Disable) { + if (!hasNonDeterministicFunctions) { + operationHash = TYtNodeHashCalculator(State_, cluster, config).GetHash(*input); + } + YQL_CLOG(DEBUG, ProviderYt) << "Operation hash: " << HexEncode(operationHash).Quote() << ", cache mode: " << queryCacheMode; + } + + TSet<TString> securityTags; + VisitExpr(optimizedNode->ChildPtr(TYtDqProcessWrite::idx_Input), [&securityTags](const TExprNode::TPtr& node) -> bool { + if (TYtTableBase::Match(node.Get())) { + if (auto stat = TYtTableBaseInfo::GetStat(TExprBase(node))) { + for (const auto& tag : stat->SecurityTags) { + securityTags.insert(tag); + } + } + return false; + } + return true; + }); + + YQL_CLOG(DEBUG, ProviderYt) << "Preparing " << input->Content() << " (UniqueId=" << input->UniqueId() << ")"; + + auto future = State_->Gateway->Prepare(input, ctx, + IYtGateway::TPrepareOptions(State_->SessionId) + .PublicId(State_->Types->TranslateOperationId(input->UniqueId())) + .Config(std::move(config)) + .OperationHash(operationHash) + .SecurityTags(securityTags) + ); + + return WrapModifyFuture(future, [operationHash, state = State_](const IYtGateway::TRunResult& res, + const TExprNode::TPtr& input, + TExprNode::TPtr& output, + TExprContext& ctx) + { + return FinalizeOutputOp(state, operationHash, res, input, output, ctx, bool(res.OutTableStats.front().second)); + }); + } + + TStatusCallbackPair RunDqWrite(const TExprNode::TPtr& input, TExprContext& ctx, const TYtOutTable& tmpTable) { + const TYtDqProcessWrite op(input); + + IDataProvider* dqProvider = nullptr; + TExprNode::TPtr delegatedNode; + + if (auto it = Delegated_->find(input.Get()); it != Delegated_->end()) { + dqProvider = it->second.DelegatedProvider; + delegatedNode = it->second.DelegatedNode; + } + + if (!delegatedNode) { + const auto cluster = op.DataSink().Cluster(); + const auto config = State_->Configuration->GetSettingsForNode(*input); + const auto tmpFolder = GetTablesTmpFolder(*config); + auto clusterStr = TString{cluster.Value()}; + + delegatedNode = input->ChildPtr(TYtDqProcessWrite::idx_Input); + + auto server = State_->Gateway->GetClusterServer(clusterStr); + YQL_ENSURE(server, "Invalid YT cluster: " << clusterStr); + + NYT::TRichYPath realTable = State_->Gateway->GetWriteTable(State_->SessionId, clusterStr, tmpTable.Name().StringValue(), tmpFolder); + realTable.Append(true); + YQL_ENSURE(realTable.TransactionId_.Defined(), "Expected TransactionId"); + + NYT::TNode writerOptions = NYT::TNode::CreateMap(); + if (auto maxRowWeight = config->MaxRowWeight.Get(clusterStr)) { + writerOptions["max_row_weight"] = static_cast<i64>(maxRowWeight->GetValue()); + } + + NYT::TNode outSpec; + NYT::TNode type; + { + auto rowSpec = TYqlRowSpecInfo(tmpTable.RowSpec()); + NYT::TNode spec; + rowSpec.FillCodecNode(spec[YqlRowSpecAttribute]); + outSpec = NYT::TNode::CreateMap()(TString{YqlIOSpecTables}, NYT::TNode::CreateList().Add(spec)); + type = NCommon::TypeToYsonNode(rowSpec.GetExtendedType(ctx)); + } + + // These settings will be passed to YT peephole callback from DQ + auto settings = Build<TCoNameValueTupleList>(ctx, delegatedNode->Pos()) + .Add() + .Name().Value("yt_cluster", TNodeFlags::Default).Build() + .Value<TCoAtom>().Value(clusterStr).Build() + .Build() + .Add() + .Name().Value("yt_server", TNodeFlags::Default).Build() + .Value<TCoAtom>().Value(server).Build() + .Build() + .Add() + .Name().Value("yt_table", TNodeFlags::Default).Build() + .Value<TCoAtom>().Value(NYT::NodeToYsonString(NYT::PathToNode(realTable))).Build() + .Build() + .Add() + .Name().Value("yt_tableName", TNodeFlags::Default).Build() + .Value<TCoAtom>().Value(tmpTable.Name().Value()).Build() + .Build() + .Add() + .Name().Value("yt_tableType", TNodeFlags::Default).Build() + .Value<TCoAtom>().Value(NYT::NodeToYsonString(type)).Build() + .Build() + .Add() + .Name().Value("yt_writeOptions", TNodeFlags::Default).Build() + .Value<TCoAtom>().Value(NYT::NodeToYsonString(writerOptions)).Build() + .Build() + .Add() + .Name().Value("yt_outSpec", TNodeFlags::Default).Build() + .Value<TCoAtom>().Value(NYT::NodeToYsonString(outSpec)).Build() + .Build() + .Add() + .Name().Value("yt_tx", TNodeFlags::Default).Build() + .Value<TCoAtom>().Value(GetGuidAsString(*realTable.TransactionId_), TNodeFlags::Default).Build() + .Build() + .Done().Ptr(); + + auto atomType = ctx.MakeType<TUnitExprType>(); + + for (auto child: settings->Children()) { + child->Child(0)->SetTypeAnn(atomType); + child->Child(0)->SetState(TExprNode::EState::ConstrComplete); + child->Child(1)->SetTypeAnn(atomType); + child->Child(1)->SetState(TExprNode::EState::ConstrComplete); + } + + delegatedNode = Build<TPull>(ctx, delegatedNode->Pos()) + .Input(std::move(delegatedNode)) + .BytesLimit() + .Value(TString()) + .Build() + .RowsLimit() + .Value(0U) + .Build() + .FormatDetails() + .Value(ui32(NYson::EYsonFormat::Binary)) + .Build() + .Settings(settings) + .Format() + .Value(0U) + .Build() + .PublicId() + .Value(ToString(State_->Types->TranslateOperationId(input->UniqueId()))) + .Build() + .Discard() + .Value(ToString(true), TNodeFlags::Default) + .Build() + .Origin(input) + .Done().Ptr(); + + for (auto idx: {TResOrPullBase::idx_BytesLimit, TResOrPullBase::idx_RowsLimit, TResOrPullBase::idx_FormatDetails, + TResOrPullBase::idx_Settings, TResOrPullBase::idx_Format, TResOrPullBase::idx_PublicId, TResOrPullBase::idx_Discard }) { + delegatedNode->Child(idx)->SetTypeAnn(atomType); + delegatedNode->Child(idx)->SetState(TExprNode::EState::ConstrComplete); + } + + delegatedNode->SetTypeAnn(input->GetTypeAnn()); + delegatedNode->SetState(TExprNode::EState::ConstrComplete); + } + + if (!dqProvider) { + if (auto p = State_->Types->DataSourceMap.FindPtr(DqProviderName)) { + dqProvider = p->Get(); + } + } + YQL_ENSURE(dqProvider); + + input->SetState(TExprNode::EState::ExecutionInProgress); + TExprNode::TPtr delegatedNodeOutput; + + if (auto status = dqProvider->GetCallableExecutionTransformer().Transform(delegatedNode, delegatedNodeOutput, ctx); status.Level != TStatus::Async) { + YQL_ENSURE(status.Level != TStatus::Ok, "Asynchronous execution is expected in a happy path."); + if (const auto flags = op.Flags()) { + TString fallbackOpName; + for (const auto& atom : flags.Cast()) { + TStringBuf flagName = atom.Value(); + if (flagName.SkipPrefix("FallbackOp")) { + fallbackOpName = flagName; + break; + } + } + for (const auto& atom : flags.Cast()) { + if (atom.Value() == "FallbackOnError") { + input->SetResult(atom.Ptr()); + input->SetState(TExprNode::EState::Error); + if (const auto issies = ctx.AssociativeIssues.extract(delegatedNode.Get())) { + if (NeedFallback(issies.mapped())) { + ctx.IssueManager.RaiseIssue(WrapIssuesOnHybridFallback(ctx.GetPosition(input->Pos()), issies.mapped(), fallbackOpName)); + return SyncStatus(IGraphTransformer::TStatus(IGraphTransformer::TStatus::Repeat, true)); + } else { + ctx.IssueManager.RaiseIssues(issies.mapped()); + return SyncStatus(status); + } + } + } + } + } + + if (const auto issies = ctx.AssociativeIssues.extract(delegatedNode.Get())) { + ctx.IssueManager.RaiseIssues(issies.mapped()); + } + return SyncStatus(status); + } + + (*Delegated_)[input.Get()] = TDelegatedInfo{dqProvider, delegatedNode}; + + auto dqFuture = dqProvider->GetCallableExecutionTransformer().GetAsyncFuture(*delegatedNode); + + TAsyncTransformCallbackFuture callbackFuture = dqFuture.Apply( + [delegated = std::weak_ptr<TNodeMap<TDelegatedInfo>>(Delegated_), state = State_](const TFuture<void>& completedFuture) { + return TAsyncTransformCallback( + [completedFuture, delegated, state](const TExprNode::TPtr& input, TExprNode::TPtr& output, TExprContext& ctx) { + completedFuture.GetValue(); + TExprNode::TPtr delegatedNode; + IDataProvider* dqProvider = nullptr; + auto lock = delegated.lock(); + if (lock) { + if (auto it = lock->find(input.Get()); it != lock->end()) { + delegatedNode = it->second.DelegatedNode; + dqProvider = it->second.DelegatedProvider; + } + } + YQL_ENSURE(delegatedNode && dqProvider); + TExprNode::TPtr delegatedNodeOutput; + const auto dqWriteStatus = dqProvider->GetCallableExecutionTransformer() + .ApplyAsyncChanges(delegatedNode, delegatedNodeOutput, ctx); + + YQL_ENSURE(dqWriteStatus != TStatus::Async, "ApplyAsyncChanges should not return Async."); + + if (dqWriteStatus == TStatus::Repeat) { + output = input; + input->SetState(TExprNode::EState::ExecutionRequired); + return dqWriteStatus; + } + lock->erase(input.Get()); + + if (dqWriteStatus != TStatus::Ok) { + output = input; + if (const auto flags = TYtDqProcessWrite(input).Flags()) { + TString fallbackOpName; + for (const auto& atom : flags.Cast()) { + TStringBuf flagName = atom.Value(); + if (flagName.SkipPrefix("FallbackOp")) { + fallbackOpName = flagName; + break; + } + } + for (const auto& atom : flags.Cast()) { + if (atom.Value() == "FallbackOnError") { + output->SetResult(atom.Ptr()); + output->SetState(TExprNode::EState::Error); + if (const auto issies = ctx.AssociativeIssues.extract(delegatedNode.Get())) { + if (NeedFallback(issies.mapped())) { + ctx.IssueManager.RaiseIssue(WrapIssuesOnHybridFallback(ctx.GetPosition(input->Pos()), issies.mapped(), fallbackOpName)); + return IGraphTransformer::TStatus(IGraphTransformer::TStatus::Repeat, true); + } else { + ctx.IssueManager.RaiseIssues(issies.mapped()); + return dqWriteStatus; + } + } + } + } + } + if (const auto issies = ctx.AssociativeIssues.extract(delegatedNode.Get())) { + ctx.IssueManager.RaiseIssues(issies.mapped()); + } + return dqWriteStatus; + } + + input->SetState(TExprNode::EState::ExecutionComplete); + output = ctx.ShallowCopy(*input); + output->SetResult(ctx.NewAtom(input->Pos(), "DQ_completed")); + if (const auto it = state->NodeHash.find(input->UniqueId()); state->NodeHash.cend() != it) { + auto hash = state->NodeHash.extract(it); + hash.key() = output->UniqueId(); + state->NodeHash.insert(std::move(hash)); + } + return IGraphTransformer::TStatus(IGraphTransformer::TStatus::Repeat, true); + }); + }); + + return std::make_pair(IGraphTransformer::TStatus::Async, callbackFuture); + } + + TStatusCallbackPair CollectDqWrittenTableStats(const TExprNode::TPtr& input, TExprContext& ctx) { + auto statsFuture = State_->Gateway->GetTableStat(input, ctx, + IYtGateway::TPrepareOptions(State_->SessionId) + .PublicId(State_->Types->TranslateOperationId(input->UniqueId())) + .Config(State_->Configuration->GetSettingsForNode(*input)) + .OperationHash(State_->NodeHash[input->UniqueId()]) + ); + + return WrapFutureCallback( + statsFuture, + [state = State_](const IYtGateway::TRunResult& res, const TExprNode::TPtr& in, TExprNode::TPtr& out, TExprContext& ctx) { + auto result = FinalizeOutputOp(state, state->NodeHash[in->UniqueId()], res, in, out, ctx, true); + out->SetResult(std::move(result)); + in->SetState(TExprNode::EState::ExecutionComplete); + if (in != out) { + return IGraphTransformer::TStatus(IGraphTransformer::TStatus::Repeat, true); + } + return IGraphTransformer::TStatus(IGraphTransformer::TStatus::Ok); + } + ); + } + + const TYtState::TPtr State_; + + struct TDelegatedInfo { + IDataProvider* DelegatedProvider; + TExprNode::TPtr DelegatedNode; + }; + std::shared_ptr<TNodeMap<TDelegatedInfo>> Delegated_; +}; + +} + +THolder<TExecTransformerBase> CreateYtDataSinkExecTransformer(TYtState::TPtr state) { + return THolder(new TYtDataSinkExecTransformer(state)); +} + +} diff --git a/yt/yql/providers/yt/provider/yql_yt_datasink_finalize.cpp b/yt/yql/providers/yt/provider/yql_yt_datasink_finalize.cpp new file mode 100644 index 0000000000..6a0a4aa5d4 --- /dev/null +++ b/yt/yql/providers/yt/provider/yql_yt_datasink_finalize.cpp @@ -0,0 +1,39 @@ +#include "yql_yt_provider_impl.h" + +#include <yql/essentials/core/yql_graph_transformer.h> + +namespace NYql { + +using namespace NNodes; + +class TYtDataSinkFinalizingTransformer: public TAsyncCallbackTransformer<TYtDataSinkFinalizingTransformer> { +public: + TYtDataSinkFinalizingTransformer(TYtState::TPtr state) + : State_(state) + { + } + + std::pair<TStatus, TAsyncTransformCallbackFuture> CallbackTransform(const TExprNode::TPtr& input, TExprNode::TPtr& output, TExprContext& ctx) { + Y_UNUSED(ctx); + output = input; + auto future = State_->Gateway->Finalize( + IYtGateway::TFinalizeOptions(State_->SessionId) + .Config(State_->Configuration->Snapshot()) + .Abort(State_->Types->HiddenMode == EHiddenMode::Force) + .DetachSnapshotTxs(State_->PassiveExecution) + ); + return WrapFuture(future, [](const IYtGateway::TFinalizeResult& res, const TExprNode::TPtr& input, TExprContext& ctx) { + Y_UNUSED(res); + return ctx.NewWorld(input->Pos()); + }); + } + +private: + TYtState::TPtr State_; +}; + +THolder<IGraphTransformer> CreateYtDataSinkFinalizingTransformer(TYtState::TPtr state) { + return THolder(new TYtDataSinkFinalizingTransformer(state)); +} + +} // NYql diff --git a/yt/yql/providers/yt/provider/yql_yt_datasink_trackable.cpp b/yt/yql/providers/yt/provider/yql_yt_datasink_trackable.cpp new file mode 100644 index 0000000000..2735b46659 --- /dev/null +++ b/yt/yql/providers/yt/provider/yql_yt_datasink_trackable.cpp @@ -0,0 +1,62 @@ +#include "yql_yt_provider_impl.h" + +#include <yt/yql/providers/yt/expr_nodes/yql_yt_expr_nodes.h> + +#include <yql/essentials/core/yql_graph_transformer.h> + +namespace NYql { + +using namespace NNodes; + +namespace { + +class TYtDataSinkTrackableCleanupTransformer : public TAsyncCallbackTransformer<TYtDataSinkTrackableCleanupTransformer> { +public: + TYtDataSinkTrackableCleanupTransformer(TYtState::TPtr state) + : State_(state) { + } + + std::pair<TStatus, TAsyncTransformCallbackFuture> + CallbackTransform(const TExprNode::TPtr& input, TExprNode::TPtr& output, TExprContext& ctx) { + Y_UNUSED(ctx); + output = input; + auto options = IYtGateway::TDropTrackablesOptions(State_->SessionId) + .Config(State_->Configuration->GetSettingsForNode(*input)); + + + YQL_ENSURE(input->IsList()); + input->ForEachChild([&options](const TExprNode& node) + { + YQL_ENSURE(node.IsList()); + YQL_ENSURE(node.ChildrenSize() == 2); + auto ytDataSink = TMaybeNode<TYtDSink>(node.Child(0)); + YQL_ENSURE(ytDataSink); + auto ytOutTable = TMaybeNode<TYtOutTable>(node.Child(1)); + YQL_ENSURE(ytOutTable); + + IYtGateway::TDropTrackablesOptions::TClusterAndPath clusterAndPath; + clusterAndPath.Path = TString{ytOutTable.Cast().Name().Value()}; + clusterAndPath.Cluster = TString{ytDataSink.Cast().Cluster().Value()}; + + options.Pathes().push_back(clusterAndPath); + }); + + auto future = State_->Gateway->DropTrackables(std::move(options)); + + return WrapFuture(future, + [](const IYtGateway::TDropTrackablesResult& res, const TExprNode::TPtr& input, TExprContext& ctx) { + Y_UNUSED(res); + return ctx.NewWorld(input->Pos()); + }); + } +private: + TYtState::TPtr State_; +}; + +} + +THolder<IGraphTransformer> CreateYtDataSinkTrackableNodesCleanupTransformer(TYtState::TPtr state) { + return THolder(new TYtDataSinkTrackableCleanupTransformer(state)); +} + +} // NYql diff --git a/yt/yql/providers/yt/provider/yql_yt_datasink_type_ann.cpp b/yt/yql/providers/yt/provider/yql_yt_datasink_type_ann.cpp new file mode 100644 index 0000000000..428ae977ec --- /dev/null +++ b/yt/yql/providers/yt/provider/yql_yt_datasink_type_ann.cpp @@ -0,0 +1,2127 @@ +#include "yql_yt_provider_impl.h" +#include "yql_yt_op_settings.h" +#include "yql_yt_helpers.h" + +#include <yt/yql/providers/yt/expr_nodes/yql_yt_expr_nodes.h> +#include <yt/yql/providers/yt/provider/yql_yt_table.h> +#include <yt/yql/providers/yt/common/yql_names.h> +#include <yt/yql/providers/yt/common/yql_configuration.h> +#include <yql/essentials/providers/common/provider/yql_provider_names.h> +#include <yql/essentials/providers/common/provider/yql_provider.h> +#include <yql/essentials/providers/common/transform/yql_visit.h> +#include <yql/essentials/providers/common/codec/yql_codec_type_flags.h> +#include <yql/essentials/core/yql_expr_type_annotation.h> +#include <yql/essentials/core/yql_join.h> +#include <yql/essentials/core/yql_opt_utils.h> +#include <yql/essentials/core/yql_type_helpers.h> +#include <yql/essentials/utils/yql_panic.h> +#include <yql/essentials/utils/log/log.h> + +#include <util/generic/xrange.h> + +namespace NYql { + +namespace { + +bool IsWideRepresentation(const TTypeAnnotationNode* leftType, const TTypeAnnotationNode* rightType) { + const auto structType = dynamic_cast<const TStructExprType*>(leftType); + const auto multiType = dynamic_cast<const TMultiExprType*>(rightType); + if (!structType || !multiType || structType->GetSize() != multiType->GetSize()) + return false; + + const auto& structItems = structType->GetItems(); + const auto& multiItems = multiType->GetItems(); + + for (auto i = 0U; i < multiItems.size(); ++i) + if (!IsSameAnnotation(*multiItems[i], *structItems[i]->GetItemType())) + return false; + + return true; +} + +const TTypeAnnotationNode* MakeInputType(const TTypeAnnotationNode* itemType, const TExprNode::TPtr& useFlowSetting, const TExprNode::TPtr& blockInputAppliedSetting, TExprContext& ctx) { + if (!useFlowSetting) { + return ctx.MakeType<TStreamExprType>(itemType); + } + + if (const auto structType = dynamic_cast<const TStructExprType*>(itemType)) { + if (ui32 limit; structType && 2U == useFlowSetting->ChildrenSize() && TryFromString<ui32>(useFlowSetting->Tail().Content(), limit) && structType->GetSize() < limit && structType->GetSize() > 0U) { + TTypeAnnotationNode::TListType types; + const auto& items = structType->GetItems(); + types.reserve(items.size()); + + std::transform(items.cbegin(), items.cend(), std::back_inserter(types), std::bind(&TItemExprType::GetItemType, std::placeholders::_1)); + if (blockInputAppliedSetting) { + std::transform(types.begin(), types.end(), types.begin(), [&](auto type) { + return ctx.MakeType<TBlockExprType>(type); + }); + types.push_back(ctx.MakeType<TScalarExprType>(ctx.MakeType<TDataExprType>(EDataSlot::Uint64))); + } + + return ctx.MakeType<TFlowExprType>(ctx.MakeType<TMultiExprType>(types)); + } + } + + return ctx.MakeType<TFlowExprType>(itemType); +} + +using namespace NNodes; + +class TYtDataSinkTypeAnnotationTransformer : public TVisitorTransformerBase { +public: + TYtDataSinkTypeAnnotationTransformer(TYtState::TPtr state) + : TVisitorTransformerBase(true) + , State_(state) + { + AddHandler({TYtOutTable::CallableName()}, Hndl(&TYtDataSinkTypeAnnotationTransformer::HandleOutTable)); + AddHandler({TYtOutput::CallableName()}, Hndl(&TYtDataSinkTypeAnnotationTransformer::HandleOutput)); + AddHandler({TYtSort::CallableName()}, Hndl(&TYtDataSinkTypeAnnotationTransformer::HandleSort)); + AddHandler({TYtCopy::CallableName()}, Hndl(&TYtDataSinkTypeAnnotationTransformer::HandleCopy)); + AddHandler({TYtMerge::CallableName()}, Hndl(&TYtDataSinkTypeAnnotationTransformer::HandleMerge)); + AddHandler({TYtMap::CallableName()}, Hndl(&TYtDataSinkTypeAnnotationTransformer::HandleMap)); + AddHandler({TYtReduce::CallableName()}, Hndl(&TYtDataSinkTypeAnnotationTransformer::HandleReduce)); + AddHandler({TYtMapReduce::CallableName()}, Hndl(&TYtDataSinkTypeAnnotationTransformer::HandleMapReduce)); + AddHandler({TYtWriteTable::CallableName()}, Hndl(&TYtDataSinkTypeAnnotationTransformer::HandleWriteTable)); + AddHandler({TYtFill::CallableName()}, Hndl(&TYtDataSinkTypeAnnotationTransformer::HandleFill)); + AddHandler({TYtTouch::CallableName()}, Hndl(&TYtDataSinkTypeAnnotationTransformer::HandleTouch)); + AddHandler({TYtDropTable::CallableName()}, Hndl(&TYtDataSinkTypeAnnotationTransformer::HandleDropTable)); + AddHandler({TCoCommit::CallableName()}, Hndl(&TYtDataSinkTypeAnnotationTransformer::HandleCommit)); + AddHandler({TYtPublish::CallableName()}, Hndl(&TYtDataSinkTypeAnnotationTransformer::HandlePublish)); + AddHandler({TYtEquiJoin::CallableName()}, Hndl(&TYtDataSinkTypeAnnotationTransformer::HandleEquiJoin)); + AddHandler({TYtStatOutTable::CallableName()}, Hndl(&TYtDataSinkTypeAnnotationTransformer::HandleStatOutTable)); + AddHandler({TYtStatOut::CallableName()}, Hndl(&TYtDataSinkTypeAnnotationTransformer::HandleStatOut)); + AddHandler({TYtDqProcessWrite ::CallableName()}, Hndl(&TYtDataSinkTypeAnnotationTransformer::HandleYtDqProcessWrite)); + AddHandler({TYtDqWrite::CallableName()}, Hndl(&TYtDataSinkTypeAnnotationTransformer::HandleDqWrite<false>)); + AddHandler({TYtDqWideWrite::CallableName()}, Hndl(&TYtDataSinkTypeAnnotationTransformer::HandleDqWrite<true>)); + AddHandler({TYtTryFirst::CallableName()}, Hndl(&TYtDataSinkTypeAnnotationTransformer::HandleTryFirst)); + AddHandler({TYtMaterialize::CallableName()}, Hndl(&TYtDataSinkTypeAnnotationTransformer::HandleMaterialize)); + } + +private: + static bool ValidateOpBase(const TExprNode::TPtr& input, TExprContext& ctx) { + if (!EnsureWorldType(*input->Child(TYtOpBase::idx_World), ctx)) { + return false; + } + + if (!EnsureSpecificDataSink(*input->Child(TYtOpBase::idx_DataSink), YtProviderName, ctx)) { + return false; + } + return true; + } + + static bool ValidateOutputOpBase(const TExprNode::TPtr& input, TExprContext& ctx, bool multiIO) { + if (!ValidateOpBase(input, ctx)) { + return false; + } + + // Output + if (multiIO) { + if (!EnsureTupleMinSize(*input->Child(TYtOutputOpBase::idx_Output), 1, ctx)) { + return false; + } + } else { + if (!EnsureTupleSize(*input->Child(TYtOutputOpBase::idx_Output), 1, ctx)) { + return false; + } + } + + return true; + } + + TStatus ValidateAndUpdateTransientOpBase(const TExprNode::TPtr& input, TExprNode::TPtr& output, + TExprContext& ctx, bool multiIO, EYtSettingTypes allowedSectionSettings) const { + if (!ValidateOutputOpBase(input, ctx, multiIO)) { + return TStatus::Error; + } + + // Input + if (multiIO) { + if (!EnsureTupleMinSize(*input->Child(TYtTransientOpBase::idx_Input), 1, ctx)) { + return TStatus::Error; + } + } else { + if (!EnsureTupleSize(*input->Child(TYtTransientOpBase::idx_Input), 1, ctx)) { + return TStatus::Error; + } + } + + for (auto& section: input->Child(TYtTransientOpBase::idx_Input)->Children()) { + if (!section->IsCallable(TYtSection::CallableName())) { + ctx.AddError(TIssue(ctx.GetPosition(section->Pos()), TStringBuilder() << "Expected " << TYtSection::CallableName())); + return TStatus::Error; + } + } + + auto clusterName = TString{TYtDSink(input->ChildPtr(TYtTransientOpBase::idx_DataSink)).Cluster().Value()}; + TMaybe<bool> yamrFormat; + TMaybe<TSampleParams> sampling; + for (size_t i = 0; i < input->Child(TYtTransientOpBase::idx_Input)->ChildrenSize(); ++i) { + auto section = TYtSection(input->Child(TYtTransientOpBase::idx_Input)->ChildPtr(i)); + for (auto setting: section.Settings()) { + if (!allowedSectionSettings.HasFlags(FromString<EYtSettingType>(setting.Name().Value()))) { + ctx.AddError(TIssue(ctx.GetPosition(section.Pos()), TStringBuilder() + << "Settings " << TString{setting.Name().Value()}.Quote() << " is not allowed in " + << input->Content() << " section")); + return TStatus::Error; + } + } + + if (0 == i) { + sampling = NYql::GetSampleParams(section.Settings().Ref()); + } else if (NYql::GetSampleParams(section.Settings().Ref()) != sampling) { + ctx.AddError(TIssue(ctx.GetPosition(section.Pos()), "Sections have different sample values")); + return TStatus::Error; + } + + for (auto path: section.Paths()) { + if (auto maybeTable = path.Table().Maybe<TYtTable>()) { + auto table = maybeTable.Cast(); + auto tableName = table.Name().Value(); + if (!NYql::HasSetting(table.Settings().Ref(), EYtSettingType::UserSchema)) { + // Don't validate already substituted anonymous tables + if (!TYtTableInfo::HasSubstAnonymousLabel(table)) { + const TYtTableDescription& tableDesc = State_->TablesData->GetTable(clusterName, + TString{tableName}, + TEpochInfo::Parse(table.Epoch().Ref())); + + if (!tableDesc.Validate(ctx.GetPosition(table.Pos()), clusterName, tableName, + NYql::HasSetting(table.Settings().Ref(), EYtSettingType::WithQB), State_->AnonymousLabels, ctx)) { + return TStatus::Error; + } + } + } + } + if (yamrFormat) { + if (*yamrFormat != path.Table().Maybe<TYtTable>().RowSpec().Maybe<TCoVoid>().IsValid()) { + ctx.AddError(TIssue(ctx.GetPosition(input->Pos()), TStringBuilder() << "Mixed Yamr/Yson input table formats")); + return TStatus::Error; + } + } else { + yamrFormat = path.Table().Maybe<TYtTable>().RowSpec().Maybe<TCoVoid>().IsValid(); + } + } + } + + // Basic Settings validation + if (!EnsureTuple(*input->Child(TYtTransientOpBase::idx_Settings), ctx)) { + return TStatus::Error; + } + + for (auto& setting: input->Child(TYtTransientOpBase::idx_Settings)->Children()) { + if (!EnsureTupleMinSize(*setting, 1, ctx)) { + return TStatus::Error; + } + + if (!EnsureAtom(*setting->Child(0), ctx)) { + return TStatus::Error; + } + } + + auto opInput = input->ChildPtr(TYtTransientOpBase::idx_Input); + auto newInput = ValidateAndUpdateTablesMeta(opInput, clusterName, State_->TablesData, State_->Types->UseTableMetaFromGraph, ctx); + if (!newInput) { + return TStatus::Error; + } + else if (newInput != opInput) { + output = ctx.ChangeChild(*input, TYtTransientOpBase::idx_Input, std::move(newInput)); + return TStatus::Repeat; + } + + return TStatus::Ok; + } + + static bool ValidateOutputType(const TTypeAnnotationNode* itemType, TPositionHandle positionHandle, TYtOutSection outTables, + size_t beginIdx, size_t endIdx, bool useExtendedType, TExprContext& ctx) + { + YQL_ENSURE(beginIdx <= endIdx); + YQL_ENSURE(endIdx <= outTables.Ref().ChildrenSize()); + YQL_ENSURE(itemType); + const size_t outTablesSize = endIdx - beginIdx; + TPosition pos = ctx.GetPosition(positionHandle); + + if (!EnsurePersistableType(positionHandle, *itemType, ctx)) { + return false; + } + + if (itemType->GetKind() == ETypeAnnotationKind::Variant) { + if (itemType->Cast<TVariantExprType>()->GetUnderlyingType()->GetKind() == ETypeAnnotationKind::Tuple) { + auto tupleType = itemType->Cast<TVariantExprType>()->GetUnderlyingType()->Cast<TTupleExprType>(); + if (tupleType->GetSize() != outTablesSize) { + ctx.AddError(TIssue(pos, TStringBuilder() << "Expected variant of " + << outTablesSize << " items, but got: " << tupleType->GetSize())); + return false; + } + for (size_t i = beginIdx; i < endIdx; ++i) { + auto rowSpec = TYqlRowSpecInfo(outTables.Item(i).RowSpec()); + const TTypeAnnotationNode* tableItemType = nullptr; + if (useExtendedType) { + tableItemType = rowSpec.GetExtendedType(ctx); + } else if (!GetSequenceItemType(outTables.Item(i).Ref(), tableItemType, ctx)) { + return false; + } + if (tableItemType->HasBareYson() && 0 != rowSpec.GetNativeYtTypeFlags()) { + ctx.AddError(TIssue(pos, TStringBuilder() << "Strict Yson type is not allowed to write, please use Optional<Yson>, item type: " + << *tableItemType)); + return false; + } + + if (!IsSameAnnotation(*tupleType->GetItems()[i - beginIdx], *tableItemType)) { + ctx.AddError(TIssue(pos, TStringBuilder() + << "Output table " << i << " row type differs from the write row type: " + << GetTypeDiff(*tableItemType, *tupleType->GetItems()[i - beginIdx]))); + return false; + } + } + } else { + auto structType = itemType->Cast<TVariantExprType>()->GetUnderlyingType()->Cast<TStructExprType>(); + if (structType->GetSize() != outTablesSize) { + ctx.AddError(TIssue(pos, TStringBuilder() << "Expected variant of " + << outTablesSize << " items, but got: " << structType->GetSize())); + return false; + } + for (size_t i = beginIdx; i < endIdx; ++i) { + auto rowSpec = TYqlRowSpecInfo(outTables.Item(i).RowSpec()); + const TTypeAnnotationNode* tableItemType = nullptr; + if (useExtendedType) { + tableItemType = rowSpec.GetExtendedType(ctx); + } else if (!GetSequenceItemType(outTables.Item(i).Ref(), tableItemType, ctx)) { + return false; + } + + if (tableItemType->HasBareYson() && 0 != rowSpec.GetNativeYtTypeFlags()) { + ctx.AddError(TIssue(pos, TStringBuilder() << "Strict Yson type is not allowed to write, please use Optional<Yson>, item type: " + << *tableItemType)); + return false; + } + + if (!IsSameAnnotation(*structType->GetItems()[i - beginIdx]->GetItemType(), *tableItemType)) { + ctx.AddError(TIssue(pos, TStringBuilder() + << "Output table " << i << " row type differs from the write row type: " + << GetTypeDiff(*tableItemType, *structType->GetItems()[i - beginIdx]->GetItemType()))); + return false; + } + } + } + } else { + if (outTablesSize != 1) { + ctx.AddError(TIssue(pos, TStringBuilder() << "Expected variant of " + << outTablesSize << " items, but got single item: " << *itemType)); + return false; + } + auto rowSpec = TYqlRowSpecInfo(outTables.Item(0).RowSpec()); + const TTypeAnnotationNode* tableItemType = nullptr; + if (useExtendedType) { + tableItemType = rowSpec.GetExtendedType(ctx); + } else if (!GetSequenceItemType(outTables.Item(0).Ref(), tableItemType, ctx)) { + return false; + } + + if (tableItemType->HasBareYson() && 0 != rowSpec.GetNativeYtTypeFlags()) { + ctx.AddError(TIssue(pos, TStringBuilder() << "Strict Yson type is not allowed to write, please use Optional<Yson>, item type: " + << *tableItemType)); + return false; + } + + if (!(IsSameAnnotation(*itemType, *tableItemType) || IsWideRepresentation(tableItemType, itemType))) { + ctx.AddError(TIssue(pos, TStringBuilder() + << "Output table row type differs from the write row type: " + << GetTypeDiff(*tableItemType, *itemType))); + return false; + } + } + return true; + } + + static bool ValidateOutputType(const TTypeAnnotationNode* itemType, TPositionHandle positionHandle, TYtOutSection outTables, + TExprContext& ctx, bool useExtendedType = false) + { + return ValidateOutputType(itemType, positionHandle, outTables, 0, outTables.Ref().ChildrenSize(), useExtendedType, ctx); + } + + static bool ValidateOutputType(const TExprNode& list, TYtOutSection outTables, TExprContext& ctx, bool useExtendedType = false) { + const TTypeAnnotationNode* itemType = GetSequenceItemType(list.Pos(), list.GetTypeAnn(), true, ctx); + if (nullptr == itemType) { + return false; + } + return ValidateOutputType(itemType, list.Pos(), outTables, ctx, useExtendedType); + } + + static bool ValidateColumnListSetting(TPosition pos, TExprContext& ctx, const TTypeAnnotationNode* itemType, + const TVector<TString>& columns, EYtSettingType settingType, bool isInput) + { + TStringBuf inOut = isInput ? " input" : " output"; + if (itemType->GetKind() == ETypeAnnotationKind::Variant) { + TTypeAnnotationNode::TListType columnTypes(columns.size(), nullptr); + auto tupleType = itemType->Cast<TVariantExprType>()->GetUnderlyingType()->Cast<TTupleExprType>(); + for (size_t t: xrange(tupleType->GetSize())) { + auto structType = tupleType->GetItems()[t]->Cast<TStructExprType>(); + for (size_t i: xrange(columns.size())) { + auto id = structType->FindItem(columns[i]); + if (!id.Defined()) { + ctx.AddError(TIssue(pos, TStringBuilder() + << settingType << " field " << columns[i] << " is missing in " + << t << inOut << " section type")); + return false; + } + auto columnType = structType->GetItems()[*id]->GetItemType(); + // Clear optionality before comparing + if (columnType->GetKind() == ETypeAnnotationKind::Optional) { + columnType = columnType->Cast<TOptionalExprType>()->GetItemType(); + } + if (0 == t) { + columnTypes[i] = columnType; + } else if (!IsSameAnnotation(*columnTypes[i], *columnType)) { + ctx.AddError(TIssue(pos, TStringBuilder() + << settingType << " field " << columns[i] << " has different type in " + << t << inOut << " section type: " + << GetTypeDiff(*columnTypes[i], *columnType))); + return false; + } + } + } + } else if (ETypeAnnotationKind::Struct == itemType->GetKind()) { + auto structType = itemType->Cast<TStructExprType>(); + for (size_t i: xrange(columns.size())) { + auto id = structType->FindItem(columns[i]); + if (!id.Defined()) { + ctx.AddError(TIssue(pos, TStringBuilder() + << settingType << " field " << columns[i] << " is missing in" << inOut << " type")); + return false; + } + } + } + return true; + } + + TStatus ValidateTableWrite(const TPosition& pos, const TExprNode::TPtr& table, TExprNode::TPtr& content, const TTypeAnnotationNode* itemType, + const TVector<TYqlRowSpecInfo::TPtr>& contentRowSpecs, const TString& cluster, const TExprNode& settings, TExprContext& ctx) const + { + YQL_ENSURE(itemType); + if (content && !EnsurePersistableType(content->Pos(), *itemType, ctx)) { + return TStatus::Error; + } + + EYtWriteMode mode = EYtWriteMode::Renew; + if (auto modeSetting = NYql::GetSetting(settings, EYtSettingType::Mode)) { + mode = FromString<EYtWriteMode>(modeSetting->Child(1)->Content()); + } + const bool initialWrite = NYql::HasSetting(settings, EYtSettingType::Initial); + const bool monotonicKeys = NYql::HasSetting(settings, EYtSettingType::MonotonicKeys); + TString columnGroups; + if (auto setting = NYql::GetSetting(settings, EYtSettingType::ColumnGroups)) { + if (!ValidateColumnGroups(*setting, *itemType->Cast<TStructExprType>(), ctx)) { + return TStatus::Error; + } + columnGroups.assign(setting->Tail().Content()); + } + + if (!initialWrite && mode != EYtWriteMode::Append) { + ctx.AddError(TIssue(pos, TStringBuilder() << + "Replacing " << TString{table->Child(TYtTable::idx_Name)->Content()}.Quote() << " table content after another table modifications in the same transaction")); + return TStatus::Error; + } + + auto outTableInfo = TYtTableInfo(table); + TYtTableDescription& description = State_->TablesData->GetModifTable(cluster, outTableInfo.Name, outTableInfo.Epoch); + + auto meta = description.Meta; + YQL_ENSURE(meta); + + if (meta->SqlView) { + ctx.AddError(TIssue(pos, TStringBuilder() + << "Modification of " << outTableInfo.Name.Quote() << " view is not supported")); + return TStatus::Error; + } + + if (meta->IsDynamic) { + ctx.AddError(TIssue(pos, TStringBuilder() << + "Modification of dynamic table " << outTableInfo.Name.Quote() << " is not supported")); + return TStatus::Error; + } + + bool replaceMeta = !meta->DoesExist || (mode != EYtWriteMode::Append && mode != EYtWriteMode::RenewKeepMeta); + bool checkLayout = meta->DoesExist && (mode == EYtWriteMode::Append || mode == EYtWriteMode::RenewKeepMeta || description.IsReplaced); + + if (monotonicKeys && initialWrite && replaceMeta) { + ctx.AddError(TIssue(pos, TStringBuilder() + << "Insert with " + << ToString(EYtSettingType::MonotonicKeys).Quote() + << " setting cannot be used with a non-existent table")); + return TStatus::Error; + } + + if (initialWrite && !replaceMeta && columnGroups != description.ColumnGroupSpec) { + ctx.AddError(TIssue(pos, TStringBuilder() + << "Insert with " + << (outTableInfo.Epoch.GetOrElse(0) ? "different " : "") + << ToString(EYtSettingType::ColumnGroups).Quote() + << " to existing table is not allowed")); + return TStatus::Error; + } + + if (auto commitEpoch = outTableInfo.CommitEpoch.GetOrElse(0)) { + // Check type compatibility with previous epoch + if (auto nextDescription = State_->TablesData->FindTable(cluster, outTableInfo.Name, commitEpoch)) { + if (nextDescription->Meta) { + if (!nextDescription->Meta->DoesExist) { + ctx.AddError(TIssue(pos, TStringBuilder() << + "Table " << outTableInfo.Name << " is modified and dropped in the same transaction")); + return TStatus::Error; + } + checkLayout = !nextDescription->IsReplaced; + } else { + checkLayout = !replaceMeta; + } + } else { + checkLayout = !replaceMeta; + } + } + + TMaybe<TColumnOrder> contentColumnOrder; + if (content) { + contentColumnOrder = State_->Types->LookupColumnOrder(*content); + if (content->IsCallable("AssumeColumnOrder")) { + YQL_ENSURE(contentColumnOrder); + YQL_CLOG(INFO, ProviderYt) << "Dropping top level " << content->Content() << " from WriteTable input"; + content = content->HeadPtr(); + } + } + + if (content && TCoPgSelect::Match(content.Get())) { + auto pgSelect = TCoPgSelect(content); + if (NCommon::NeedToRenamePgSelectColumns(pgSelect)) { + TExprNode::TPtr output; + + const auto& columnOrder = (outTableInfo.RowSpec) + ? outTableInfo.RowSpec->GetColumnOrder() + : contentColumnOrder; + + bool result = NCommon::RenamePgSelectColumns(pgSelect, output, columnOrder, ctx, *State_->Types); + if (!result) { + return TStatus::Error; + } + if (output != content) { + content = output; + return TStatus::Repeat; + } + } + } + + if (checkLayout) { + auto rowSpec = description.RowSpec; + TString modeStr = EYtWriteMode::RenewKeepMeta == mode ? "truncate with keep meta" : ToString(mode); + if (!rowSpec) { + ctx.AddError(TIssue(pos, TStringBuilder() + << "Table " << outTableInfo.Name.Quote() + << " does not have any scheme attribute supported by YQL, " << modeStr << " is not allowed")); + return TStatus::Error; + } + if (description.IgnoreTypeV3) { + ctx.AddError(TIssue(pos, TStringBuilder() + << "Table " << outTableInfo.Name.Quote() << " has IgnoreTypeV3 remapper, " << modeStr << " is not allowed")); + return TStatus::Error; + } + if (description.HasUdfApply) { + ctx.AddError(TIssue(pos, TStringBuilder() + << "Table " << outTableInfo.Name.Quote() << " has udf remappers, " << modeStr << " is not allowed")); + return TStatus::Error; + } + if (meta->InferredScheme) { + ctx.AddError(TIssue(pos, TStringBuilder() + << "Table " << outTableInfo.Name.Quote() << " has inferred schema, " << modeStr << " is not allowed")); + return TStatus::Error; + } + if (!rowSpec->StrictSchema) { + ctx.AddError(TIssue(pos, TStringBuilder() + << "Table " << outTableInfo.Name.Quote() << " has non-strict schema, " << modeStr << " is not allowed")); + return TStatus::Error; + } + if (meta->Attrs.contains(QB2Premapper)) { + ctx.AddError(TIssue(pos, TStringBuilder() + << "Table " << outTableInfo.Name.Quote() << " has qb2 remapper, " << modeStr << " is not allowed")); + return TStatus::Error; + } + + if (!IsSameAnnotation(*description.RowType, *itemType)) { + if (content) { + auto expectedType = ctx.MakeType<TListExprType>(description.RowType); + auto status = TryConvertTo(content, *expectedType, ctx); + if (status.Level != TStatus::Error) { + return status; + } + } + + ctx.AddError(TIssue(pos, TStringBuilder() + << "Table " << outTableInfo.Name.Quote() << " row type differs from the written row type: " + << GetTypeDiff(*description.RowType, *itemType))); + return TStatus::Error; + } + } + + if (auto commitEpoch = outTableInfo.CommitEpoch.GetOrElse(0)) { + TYtTableDescription& nextDescription = State_->TablesData->GetOrAddTable(cluster, outTableInfo.Name, commitEpoch); + + if (!nextDescription.Meta) { + nextDescription.RowType = itemType; + nextDescription.RawRowType = itemType; + nextDescription.IsReplaced = replaceMeta; + + TYtTableMetaInfo::TPtr nextMetadata = (nextDescription.Meta = MakeIntrusive<TYtTableMetaInfo>()); + nextMetadata->DoesExist = true; + nextMetadata->YqlCompatibleScheme = true; + + TYqlRowSpecInfo::TPtr nextRowSpec = (nextDescription.RowSpec = MakeIntrusive<TYqlRowSpecInfo>()); + if (replaceMeta) { + nextRowSpec->SetType(itemType->Cast<TStructExprType>(), State_->Configuration->UseNativeYtTypes.Get().GetOrElse(DEFAULT_USE_NATIVE_YT_TYPES) ? NTCF_ALL : NTCF_NONE); + YQL_CLOG(INFO, ProviderYt) << "Saving column order: " << FormatColumnOrder(contentColumnOrder, 10); + nextRowSpec->SetColumnOrder(contentColumnOrder); + } else { + nextRowSpec->CopyType(*description.RowSpec); + } + + if (!replaceMeta) { + nextRowSpec->StrictSchema = !description.RowSpec || description.RowSpec->StrictSchema; + nextMetadata->Attrs = meta->Attrs; + } + } + else { + if (!IsSameAnnotation(*nextDescription.RowType, *itemType)) { + if (content) { + auto expectedType = ctx.MakeType<TListExprType>(nextDescription.RowType); + auto status = TryConvertTo(content, *expectedType, ctx); + if (status.Level != TStatus::Error) { + return status; + } + } + + ctx.AddError(TIssue(pos, TStringBuilder() + << "Table " << outTableInfo.Name.Quote() << " row type differs from the appended row type: " + << GetTypeDiff(*nextDescription.RowType, *itemType))); + return TStatus::Error; + } + } + + if (initialWrite) { + nextDescription.ColumnGroupSpec = columnGroups; + } else if (columnGroups != nextDescription.ColumnGroupSpec) { + ctx.AddError(TIssue(pos, TStringBuilder() + << "All appends within the same commit should have the equal " + << ToString(EYtSettingType::ColumnGroups).Quote() + << " value")); + return TStatus::Error; + } + + YQL_ENSURE(nextDescription.RowSpec); + if (contentRowSpecs) { + size_t from = 0; + if (initialWrite) { + nextDescription.RowSpecSortReady = true; + if (nextDescription.IsReplaced) { + nextDescription.RowSpec->CopySortness(ctx, *contentRowSpecs.front(), TYqlRowSpecInfo::ECopySort::Exact); + if (auto contentNativeType = contentRowSpecs.front()->GetNativeYtType()) { + nextDescription.RowSpec->CopyTypeOrders(*contentNativeType); + } + from = 1; + } else { + nextDescription.MonotonicKeys = monotonicKeys; + if (description.RowSpec) { + nextDescription.RowSpec->CopySortness(ctx, *description.RowSpec, TYqlRowSpecInfo::ECopySort::Exact); + const auto currNativeType = description.RowSpec->GetNativeYtType(); + if (currNativeType && nextDescription.RowSpec->GetNativeYtType() != currNativeType) { + nextDescription.RowSpec->CopyTypeOrders(*currNativeType); + } + } + } + if (monotonicKeys && !nextDescription.RowSpec->IsSorted()) { + ctx.AddError(TIssue(pos, TStringBuilder() + << "Insert with " + << ToString(EYtSettingType::MonotonicKeys).Quote() + << " setting cannot be used with a unsorted table")); + return TStatus::Error; + } + } else if (nextDescription.RowSpecSortReady) { + if (!nextDescription.MonotonicKeys) { + nextDescription.MonotonicKeys = monotonicKeys; + } else if (*nextDescription.MonotonicKeys != monotonicKeys) { + ctx.AddError(TIssue(pos, TStringBuilder() + << "All appends within the same commit should have the same " + << ToString(EYtSettingType::MonotonicKeys).Quote() + << " flag")); + return TStatus::Error; + } + } + + if (nextDescription.RowSpecSortReady) { + const bool uniqueKeys = nextDescription.RowSpec->UniqueKeys; + for (size_t s = from; s < contentRowSpecs.size(); ++s) { + const bool hasSortChanges = nextDescription.RowSpec->MakeCommonSortness(ctx, *contentRowSpecs[s]); + const bool breaksSorting = hasSortChanges || !nextDescription.RowSpec->CompareSortness(*contentRowSpecs[s], false); + if (monotonicKeys) { + if (breaksSorting) { + ctx.AddError(TIssue(pos, TStringBuilder() + << "Inserts with " + << ToString(EYtSettingType::MonotonicKeys).Quote() + << " setting must not change output table sorting")); + return TStatus::Error; + } + nextDescription.RowSpec->UniqueKeys = uniqueKeys; + } + ui32 mutationId = 0; + if (auto setting = NYql::GetSetting(settings, EYtSettingType::MutationId)) { + mutationId = FromString<ui32>(setting->Child(1)->Content()); + } + + if (++nextDescription.WriteValidateCount[mutationId] < 2) { + TStringBuilder warning; + if (breaksSorting) { + warning << "Sort order of written data differs from the order of " + << outTableInfo.Name.Quote() << " table content. Result table content will be "; + if (nextDescription.RowSpec->IsSorted()) { + warning << "ordered by "; + for (size_t i: xrange(nextDescription.RowSpec->SortMembers.size())) { + if (i != 0) { + warning << ','; + } + warning << nextDescription.RowSpec->SortMembers[i] << '(' + << (nextDescription.RowSpec->SortDirections[i] ? "asc" : "desc") << ")"; + } + } else { + warning << "unordered"; + } + } else if (uniqueKeys && !nextDescription.RowSpec->UniqueKeys) { + warning << "Result table content will have non unique keys"; + } + + if (warning && !ctx.AddWarning(YqlIssue(pos, EYqlIssueCode::TIssuesIds_EIssueCode_YT_SORT_ORDER_CHANGE, warning))) { + return TStatus::Error; + } + } + } + } + } + } + else if (replaceMeta) { + description.RowType = itemType; + description.RawRowType = itemType; + description.IsReplaced = true; + + description.Meta->DoesExist = true; + description.Meta->YqlCompatibleScheme = true; + if (!description.RowSpec) { + description.RowSpec = MakeIntrusive<TYqlRowSpecInfo>(); + } + description.RowSpec->SetType(itemType->Cast<TStructExprType>()); + YQL_CLOG(INFO, ProviderYt) << "Saving column order: " << FormatColumnOrder(contentColumnOrder, 10); + description.RowSpec->SetColumnOrder(contentColumnOrder); + } + + return TStatus::Ok; + } + + static const TTypeAnnotationNode* GetInputItemType(TYtSectionList input, TExprContext& ctx) { + TTypeAnnotationNode::TListType items; + for (auto section: input.Cast<TYtSectionList>()) { + items.push_back(section.Ref().GetTypeAnn()->Cast<TListExprType>()->GetItemType()); + } + return items.size() == 1 + ? items.front() + : ctx.MakeType<TVariantExprType>(ctx.MakeType<TTupleExprType>(items)); + } + + static const TTypeAnnotationNode* MakeOutputOperationType(TYtOutputOpBase op, TExprContext& ctx) { + TTypeAnnotationNode::TListType items; + for (auto out: op.Output()) { + items.push_back(out.Ref().GetTypeAnn()->Cast<TListExprType>()->GetItemType()); + } + + const TTypeAnnotationNode* itemType = items.size() == 1 + ? items.front() + : ctx.MakeType<TVariantExprType>(ctx.MakeType<TTupleExprType>(items)); + + return ctx.MakeType<TTupleExprType>(TTypeAnnotationNode::TListType{ + op.World().Ref().GetTypeAnn(), + ctx.MakeType<TListExprType>(itemType) + }); + } + +private: + TStatus HandleOutTable(TExprBase input, TExprContext& ctx) { + if (!TYtOutTableInfo::Validate(input.Ref(), ctx)) { + return TStatus::Error; + } + input.Ptr()->SetTypeAnn(ctx.MakeType<TListExprType>(input.Cast<TYtOutTable>().RowSpec().Ref().GetTypeAnn())); + return TStatus::Ok; + } + + TStatus HandleOutput(TExprBase input, TExprContext& ctx) { + if (!EnsureMinMaxArgsCount(input.Ref(), 2, 3, ctx)) { + return TStatus::Error; + } + + const auto op = input.Ref().Child(TYtOutput::idx_Operation); + if (!(TYtOutputOpBase::Match(op) || TYtTryFirst::Match(op))) { + ctx.AddError(TIssue(ctx.GetPosition(op->Pos()), TStringBuilder() << "Expect YT operation, but got: " + << op->Content())); + return TStatus::Error; + } + const auto opOut = TYtTryFirst::Match(op) ? + op->Tail().Child(TYtOutputOpBase::idx_Output): + op->Child(TYtOutputOpBase::idx_Output); + + if (!EnsureAtom(*input.Ptr()->Child(TYtOutput::idx_OutIndex), ctx)) { + return TStatus::Error; + } + + size_t ndx = 0; + if (!TryFromString<size_t>(input.Ptr()->Child(TYtOutput::idx_OutIndex)->Content(), ndx) || ndx >= opOut->ChildrenSize()) { + ctx.AddError(TIssue(ctx.GetPosition(input.Ptr()->Child(TYtOutput::idx_Operation)->Pos()), TStringBuilder() + << "Bad " << TYtOutput::CallableName() << " output index value: " << input.Ptr()->Child(TYtOutput::idx_OutIndex)->Content())); + return TStatus::Error; + } + + if (input.Ref().ChildrenSize() == 3) { + if (!EnsureAtom(*input.Ref().Child(TYtOutput::idx_Mode), ctx)) { + return TStatus::Error; + } + if (input.Ref().Child(TYtOutput::idx_Mode)->Content() != ToString(EYtSettingType::Unordered)) { + ctx.AddError(TIssue(ctx.GetPosition(input.Ref().Child(TYtOutput::idx_Mode)->Pos()), TStringBuilder() + << "Bad " << TYtOutput::CallableName() << " mode: " << input.Ref().Child(TYtOutput::idx_Mode)->Content())); + return TStatus::Error; + } + } + + input.Ptr()->SetTypeAnn(opOut->Child(ndx)->GetTypeAnn()); + return TStatus::Ok; + } + + TStatus HandleSort(const TExprNode::TPtr& input, TExprNode::TPtr& output, TExprContext& ctx) { + if (!EnsureArgsCount(*input, 5, ctx)) { + return TStatus::Error; + } + + auto status = ValidateAndUpdateTransientOpBase(input, output, ctx, false, + EYtSettingType::KeyFilter | EYtSettingType::KeyFilter2 | EYtSettingType::Take | EYtSettingType::Skip); + if (status.Level != TStatus::Ok) { + return status; + } + + auto sort = TYtSort(input); + + TYtOutTableInfo outTableInfo(sort.Output().Item(0)); + if (!outTableInfo.RowSpec) { + ctx.AddError(TIssue(ctx.GetPosition(sort.Output().Item(0).Pos()), + TStringBuilder() << TString{YqlRowSpecAttribute}.Quote() << " of " << TYtSort::CallableName() << " output table should be filled")); + return TStatus::Error; + } + + if (outTableInfo.RowSpec->SortedBy.empty()) { + ctx.AddError(TIssue(ctx.GetPosition(sort.Output().Item(0).Pos()), + TStringBuilder() << "SortedBy attribute of " << TString{YqlRowSpecAttribute}.Quote() + << " should be filled in output table")); + return TStatus::Error; + } + auto inputType = sort.Input().Item(0).Ref().GetTypeAnn()->Cast<TListExprType>()->GetItemType()->Cast<TStructExprType>(); + for (auto& field: outTableInfo.RowSpec->SortedBy) { + if (!inputType->FindItem(field)) { + ctx.AddError(TIssue(ctx.GetPosition(sort.Output().Item(0).Pos()), + TStringBuilder() << "SortedBy attribute of " << TString{YqlRowSpecAttribute}.Quote() + << " refers to unknown field " << field.Quote())); + return TStatus::Error; + } + } + + if (!ValidateSettings(sort.Settings().Ref(), EYtSettingType::Limit | EYtSettingType::NoDq, ctx)) { + return TStatus::Error; + } + + for (TYtSection section: sort.Input()) { + for (TYtPath path: section.Paths()) { + TYtPathInfo pathInfo(path); + if (pathInfo.RequiresRemap()) { + ctx.AddError(TIssue(ctx.GetPosition(path.Pos()), TStringBuilder() << TYtSort::CallableName() + << " cannot be applied to tables with QB2 premapper, inferred, yamred_dsv, or non-strict schemas")); + return TStatus::Error; + } + if (pathInfo.Table->RowSpec && pathInfo.GetNativeYtTypeFlags() != outTableInfo.RowSpec->GetNativeYtTypeFlags()) { + ctx.AddError(TIssue(ctx.GetPosition(path.Pos()), TStringBuilder() << TYtSort::CallableName() + << " has different input/output native YT types")); + return TStatus::Error; + } + } + } + + auto outTypeItems = outTableInfo.RowSpec->GetType()->GetItems(); + const auto& directions = outTableInfo.RowSpec->SortDirections; + const auto& sortedBy = outTableInfo.RowSpec->SortedBy; + for (auto& aux: outTableInfo.RowSpec->GetAuxColumns()) { + bool adjust = false; + for (ui32 i = 0; i < directions.size(); ++i) { + if (!directions[i] && sortedBy[i] == aux.first) { + adjust = true; + break; + } + } + + auto type = aux.second; + if (adjust) { + type = ctx.MakeType<TDataExprType>(EDataSlot::String); + } + + outTypeItems.push_back(ctx.MakeType<TItemExprType>(aux.first, type)); + } + auto outType = ctx.MakeType<TStructExprType>(outTypeItems); + + if (!IsSameAnnotation(*inputType, *outType)) { + ctx.AddError(TIssue(ctx.GetPosition(sort.Output().Item(0).Pos()), TStringBuilder() + << "Sort's output row type differs from input row type: " + << GetTypeDiff(*inputType, *outType))); + return TStatus::Error; + } + + input->SetTypeAnn(MakeOutputOperationType(sort, ctx)); + return TStatus::Ok; + } + + TStatus HandleCopy(const TExprNode::TPtr& input, TExprNode::TPtr& output, TExprContext& ctx) { + if (!EnsureArgsCount(*input, 5, ctx)) { + return TStatus::Error; + } + + auto status = ValidateAndUpdateTransientOpBase(input, output, ctx, false, {}); + if (status.Level != TStatus::Ok) { + return status; + } + + auto copy = TYtCopy(input); + + // YtCopy! has exactly one input table + if (!EnsureArgsCount(copy.Input().Item(0).Paths().Ref(), 1, ctx)) { + return TStatus::Error; + } + + TYqlRowSpecInfo outRowSpec(copy.Output().Item(0).RowSpec()); + for (TYtSection section: copy.Input()) { + for (TYtPath path: section.Paths()) { + if (!path.Ranges().Maybe<TCoVoid>()) { + ctx.AddError(TIssue(ctx.GetPosition(path.Pos()), TStringBuilder() << TYtCopy::CallableName() << " cannot be used with range selection")); + return TStatus::Error; + } + auto tableInfo = TYtTableBaseInfo::Parse(path.Table()); + if (!tableInfo->IsTemp) { + ctx.AddError(TIssue(ctx.GetPosition(path.Pos()), TStringBuilder() << TYtCopy::CallableName() << " cannot be used with non-temporary tables")); + return TStatus::Error; + } + if (tableInfo->Meta->IsDynamic) { + ctx.AddError(TIssue(ctx.GetPosition(path.Pos()), TStringBuilder() << TYtCopy::CallableName() << " cannot be used with dynamic tables")); + return TStatus::Error; + } + auto tableColumnsSize = tableInfo->RowSpec ? tableInfo->RowSpec->GetType()->GetSize() : YAMR_FIELDS.size(); + if (!path.Columns().Maybe<TCoVoid>() && path.Columns().Cast<TExprList>().Size() != tableColumnsSize) { + ctx.AddError(TIssue(ctx.GetPosition(path.Pos()), TStringBuilder() << TYtCopy::CallableName() << " cannot be used with column selection")); + return TStatus::Error; + } + if (tableInfo->RequiresRemap()) { + ctx.AddError(TIssue(ctx.GetPosition(path.Pos()), TStringBuilder() << TYtCopy::CallableName() + << " cannot be applied to tables with QB2 premapper, inferred, yamred_dsv, or non-strict schemas")); + return TStatus::Error; + } + if (tableInfo->RowSpec && tableInfo->RowSpec->GetNativeYtTypeFlags() != outRowSpec.GetNativeYtTypeFlags()) { + ctx.AddError(TIssue(ctx.GetPosition(path.Pos()), TStringBuilder() << TYtCopy::CallableName() + << " has different input/output native YT types")); + return TStatus::Error; + } + } + } + + auto inputRowSpec = TYtTableBaseInfo::GetRowSpec(copy.Input().Item(0).Paths().Item(0).Table()); + if ((inputRowSpec && !inputRowSpec->CompareSortness(outRowSpec)) || (!inputRowSpec && outRowSpec.IsSorted())) { + ctx.AddError(TIssue(ctx.GetPosition(copy.Output().Item(0).Pos()), TStringBuilder() + << "Input/output tables have different sort order")); + return TStatus::Error; + } + + // YtCopy! has no settings + if (!EnsureTupleSize(copy.Settings().MutableRef(), 0, ctx)) { + return TStatus::Error; + } + + if (!ValidateOutputType(copy.Input().Item(0).Paths().Item(0).Ref(), copy.Output(), ctx)) { + return TStatus::Error; + } + + TStringBuf outGroup; + if (auto setting = NYql::GetSetting(copy.Output().Item(0).Settings().Ref(), EYtSettingType::ColumnGroups)) { + outGroup = setting->Tail().Content(); + } + + TStringBuf inputColGroupSpec; + const auto& path = copy.Input().Item(0).Paths().Item(0); + if (auto table = path.Table().Maybe<TYtTable>()) { + if (auto tableDesc = State_->TablesData->FindTable(copy.DataSink().Cluster().StringValue(), TString{TYtTableInfo::GetTableLabel(table.Cast())}, TEpochInfo::Parse(table.Cast().Epoch().Ref()))) { + inputColGroupSpec = tableDesc->ColumnGroupSpec; + } + } else if (auto out = path.Table().Maybe<TYtOutput>()) { + if (auto setting = NYql::GetSetting(GetOutputOp(out.Cast()).Output().Item(FromString<ui32>(out.Cast().OutIndex().Value())).Settings().Ref(), EYtSettingType::ColumnGroups)) { + inputColGroupSpec = setting->Tail().Content(); + } + } else if (auto outTable = path.Table().Maybe<TYtOutTable>()) { + if (auto setting = NYql::GetSetting(outTable.Cast().Settings().Ref(), EYtSettingType::ColumnGroups)) { + inputColGroupSpec = setting->Tail().Content(); + } + } + + if (outGroup != inputColGroupSpec) { + ctx.AddError(TIssue(ctx.GetPosition(copy.Output().Item(0).Settings().Pos()), TStringBuilder() << TYtCopy::CallableName() + << "has input/output tables with different " << EYtSettingType::ColumnGroups << " values")); + return TStatus::Error; + } + + input->SetTypeAnn(MakeOutputOperationType(copy, ctx)); + return TStatus::Ok; + } + + TStatus HandleMerge(const TExprNode::TPtr& input, TExprNode::TPtr& output, TExprContext& ctx) { + if (!EnsureArgsCount(*input, 5, ctx)) { + return TStatus::Error; + } + + auto status = ValidateAndUpdateTransientOpBase(input, output, ctx, false, + EYtSettingType::KeyFilter | EYtSettingType::KeyFilter2 | EYtSettingType::Take | EYtSettingType::Skip | EYtSettingType::Sample); + if (status.Level != TStatus::Ok) { + return status; + } + + auto merge = TYtMerge(input); + + if (!ValidateSettings(merge.Settings().Ref(), EYtSettingType::ForceTransform | EYtSettingType::TransformColGroups | EYtSettingType::CombineChunks | EYtSettingType::Limit | EYtSettingType::KeepSorted | EYtSettingType::NoDq, ctx)) { + return TStatus::Error; + } + + TYqlRowSpecInfo outRowSpec(merge.Output().Item(0).RowSpec()); + TVector<TString> outSortedBy = outRowSpec.SortedBy; + for (auto section: merge.Input()) { + for (auto path: section.Paths()) { + if (!IsSameAnnotation(*path.Ref().GetTypeAnn(), *merge.Output().Item(0).Ref().GetTypeAnn())) { + ctx.AddError(TIssue(ctx.GetPosition(path.Pos()), TStringBuilder() + << "Input/output tables should have the same scheme. Found diff: " + << GetTypeDiff(*path.Ref().GetTypeAnn(), *merge.Output().Item(0).Ref().GetTypeAnn()))); + return TStatus::Error; + } + TYtPathInfo pathInfo(path); + if (pathInfo.RequiresRemap()) { + ctx.AddError(TIssue(ctx.GetPosition(path.Pos()), TStringBuilder() << TYtMerge::CallableName() + << " cannot be applied to tables with QB2 premapper, inferred, yamred_dsv, or non-strict schemas")); + return TStatus::Error; + } + if (pathInfo.GetNativeYtTypeFlags() != outRowSpec.GetNativeYtTypeFlags()) { + ctx.AddError(TIssue(ctx.GetPosition(path.Pos()), TStringBuilder() << TYtMerge::CallableName() + << " has different input/output native YT types")); + return TStatus::Error; + } + auto inputRowSpec = pathInfo.Table->RowSpec; + TVector<TString> inSortedBy; + TVector<bool> inSortDir; + if (inputRowSpec) { + inSortedBy = inputRowSpec->SortedBy; + inSortDir = inputRowSpec->SortDirections; + } + if (!outSortedBy.empty()) { + if (outSortedBy.size() > inSortedBy.size() || + !std::equal(outSortedBy.begin(), outSortedBy.end(), inSortedBy.begin())) { + + ctx.AddError(TIssue(ctx.GetPosition(path.Pos()), TStringBuilder() + << "Output table has sortedBy columns " << JoinSeq(",", outSortedBy).Quote() + << ", which is not a subset of " << TString{TYtTableBaseInfo::GetTableName(path.Table())}.Quote() + << " input table sortedBy columns " << JoinSeq(",", inSortedBy).Quote())); + return TStatus::Error; + } + if (!std::equal(outRowSpec.SortDirections.begin(), outRowSpec.SortDirections.end(), inSortDir.begin())) { + ctx.AddError(TIssue(ctx.GetPosition(path.Pos()), TStringBuilder() + << "Input/output tables have different sort directions")); + return TStatus::Error; + } + } + } + } + + if (!ValidateOutputType(merge.Input().Item(0).Ref(), merge.Output(), ctx)) { + return TStatus::Error; + } + + input->SetTypeAnn(MakeOutputOperationType(merge, ctx)); + return TStatus::Ok; + } + + TStatus HandleMap(const TExprNode::TPtr& input, TExprNode::TPtr& output, TExprContext& ctx) { + if (!EnsureArgsCount(*input, 6, ctx)) { + return TStatus::Error; + } + + auto status = ValidateAndUpdateTransientOpBase(input, output, ctx, true, + EYtSettingType::KeyFilter | EYtSettingType::KeyFilter2 | EYtSettingType::Take | EYtSettingType::Skip | EYtSettingType::Sample | EYtSettingType::SysColumns); + if (status.Level != TStatus::Ok) { + return status; + } + + auto map = TYtMap(input); + + const EYtSettingTypes accpeted = EYtSettingType::Ordered + | EYtSettingType::Limit + | EYtSettingType::SortLimitBy + | EYtSettingType::WeakFields + | EYtSettingType::Sharded + | EYtSettingType::JobCount + | EYtSettingType::Flow + | EYtSettingType::KeepSorted + | EYtSettingType::NoDq + | EYtSettingType::BlockInputReady + | EYtSettingType::BlockInputApplied; + if (!ValidateSettings(map.Settings().Ref(), accpeted, ctx)) { + return TStatus::Error; + } + if (map.Output().Size() != 1 && NYql::HasSetting(map.Settings().Ref(), EYtSettingType::Limit)) { + ctx.AddError(TIssue(ctx.GetPosition(input->Pos()), TStringBuilder() + << EYtSettingType::Limit << " setting is not allowed for operation with multiple outputs")); + return TStatus::Error; + } + TVector<TString> sortLimitBy = NYql::GetSettingAsColumnList(map.Settings().Ref(), EYtSettingType::SortLimitBy); + if (!sortLimitBy.empty()) { + auto outItemType = TYqlRowSpecInfo(map.Output().Item(0).RowSpec()).GetExtendedType(ctx); + if (!ValidateColumnListSetting(ctx.GetPosition(input->Pos()), ctx, outItemType, sortLimitBy, EYtSettingType::SortLimitBy, false)) { + return TStatus::Error; + } + } + + status = ConvertToLambda(input->ChildRef(TYtMap::idx_Mapper), ctx, 1); + if (status.Level != TStatus::Ok) { + return status; + } + + const auto inputItemType = GetInputItemType(map.Input(), ctx); + const auto useFlow = NYql::GetSetting(map.Settings().Ref(), EYtSettingType::Flow); + const auto blockInputApplied = NYql::GetSetting(map.Settings().Ref(), EYtSettingType::BlockInputApplied); + const auto lambdaInputType = MakeInputType(inputItemType, useFlow, blockInputApplied, ctx); + + auto& lambda = input->ChildRef(TYtMap::idx_Mapper); + + if (!UpdateLambdaAllArgumentsTypes(lambda, {lambdaInputType}, ctx)) { + return TStatus::Error; + } + + if (!lambda->GetTypeAnn()) { + return TStatus::Repeat; + } + + if (!(useFlow ? EnsureFlowType(*lambda, ctx) : EnsureStreamType(*lambda, ctx))) { + return TStatus::Error; + } + + if (!ValidateOutputType(*lambda, map.Output(), ctx, true)) { + lambda->SetTypeAnn(nullptr); + lambda->SetState(TExprNode::EState::Initial); + return TStatus::Error; + } + + input->SetTypeAnn(MakeOutputOperationType(map, ctx)); + return TStatus::Ok; + } + + TStatus HandleReduce(const TExprNode::TPtr& input, TExprNode::TPtr& output, TExprContext& ctx) { + if (!EnsureArgsCount(*input, 6, ctx)) { + return TStatus::Error; + } + + auto status = ValidateAndUpdateTransientOpBase(input, output, ctx, true, + EYtSettingType::KeyFilter | EYtSettingType::KeyFilter2 | EYtSettingType::Take | EYtSettingType::Skip | EYtSettingType::Sample | EYtSettingType::SysColumns); + if (status.Level != TStatus::Ok) { + return status; + } + + auto reduce = TYtReduce(input); + + const EYtSettingTypes accepted = EYtSettingType::ReduceBy + | EYtSettingType::Limit + | EYtSettingType::SortLimitBy + | EYtSettingType::SortBy + | EYtSettingType::JoinReduce + | EYtSettingType::FirstAsPrimary + | EYtSettingType::Flow + | EYtSettingType::KeepSorted + | EYtSettingType::KeySwitch + | EYtSettingType::ReduceInputType + | EYtSettingType::NoDq; + + if (!ValidateSettings(reduce.Settings().Ref(), accepted, ctx)) { + return TStatus::Error; + } + + if (reduce.Output().Size() != 1 && NYql::HasSetting(reduce.Settings().Ref(), EYtSettingType::Limit)) { + ctx.AddError(TIssue(ctx.GetPosition(input->Pos()), TStringBuilder() + << EYtSettingType::Limit << " setting is not allowed in operation with multiple outputs")); + return TStatus::Error; + } + TVector<TString> sortLimitBy = NYql::GetSettingAsColumnList(reduce.Settings().Ref(), EYtSettingType::SortLimitBy); + if (!sortLimitBy.empty()) { + auto outItemType = reduce.Output().Item(0).Ref().GetTypeAnn()->Cast<TListExprType>()->GetItemType(); + if (!ValidateColumnListSetting(ctx.GetPosition(input->Pos()), ctx, outItemType, sortLimitBy, EYtSettingType::SortLimitBy, false)) { + return TStatus::Error; + } + } + + status = ConvertToLambda(input->ChildRef(TYtReduce::idx_Reducer), ctx, 1); + if (status.Level != TStatus::Ok) { + return status; + } + + TVector<TString> reduceBy = NYql::GetSettingAsColumnList(reduce.Settings().Ref(), EYtSettingType::ReduceBy); + if (reduceBy.empty()) { + ctx.AddError(TIssue(ctx.GetPosition(input->Pos()), TStringBuilder() << "Empty reduceBy option is not allowed in " << input->Content())); + return TStatus::Error; + } + + auto inputItemType = GetInputItemType(reduce.Input(), ctx); + if (!ValidateColumnListSetting(ctx.GetPosition(input->Pos()), ctx, inputItemType, reduceBy, EYtSettingType::ReduceBy, true)) { + return TStatus::Error; + } + + if (NYql::HasSetting(reduce.Settings().Ref(), EYtSettingType::SortBy)) { + TVector<TString> sortBy = NYql::GetSettingAsColumnList(reduce.Settings().Ref(), EYtSettingType::SortBy); + if (!ValidateColumnListSetting(ctx.GetPosition(input->Pos()), ctx, inputItemType, sortBy, EYtSettingType::SortBy, true)) { + return TStatus::Error; + } + } + + if (NYql::HasSetting(reduce.Settings().Ref(), EYtSettingType::KeySwitch)) { + if (inputItemType->GetKind() == ETypeAnnotationKind::Variant) { + auto underTupleType = inputItemType->Cast<TVariantExprType>()->GetUnderlyingType()->Cast<TTupleExprType>(); + TTypeAnnotationNode::TListType newTupleItems; + for (size_t i = 0; i < underTupleType->GetSize(); ++i) { + auto tupleItemType = underTupleType->GetItems()[i]; + auto items = tupleItemType->Cast<TStructExprType>()->GetItems(); + items.push_back(ctx.MakeType<TItemExprType>(YqlSysColumnKeySwitch, ctx.MakeType<TDataExprType>(EDataSlot::Bool))); + newTupleItems.push_back(ctx.MakeType<TStructExprType>(items)); + } + inputItemType = ctx.MakeType<TVariantExprType>(ctx.MakeType<TTupleExprType>(newTupleItems)); + } else { + auto items = inputItemType->Cast<TStructExprType>()->GetItems(); + items.push_back(ctx.MakeType<TItemExprType>(YqlSysColumnKeySwitch, ctx.MakeType<TDataExprType>(EDataSlot::Bool))); + inputItemType = ctx.MakeType<TStructExprType>(items); + } + } + + if (NYql::HasSetting(reduce.Settings().Ref(), EYtSettingType::JoinReduce)) { + // Require at least two sections + if (!EnsureTupleMinSize(reduce.Input().MutableRef(), 2, ctx)) { + return TStatus::Error; + } + } + + const auto useFlow = NYql::GetSetting(reduce.Settings().Ref(), EYtSettingType::Flow); + const auto lambdaInputType = MakeInputType(inputItemType, useFlow, TExprNode::TPtr(), ctx); + + auto& lambda = input->ChildRef(TYtReduce::idx_Reducer); + if (!UpdateLambdaAllArgumentsTypes(lambda, {lambdaInputType}, ctx)) { + return TStatus::Error; + } + + if (!lambda->GetTypeAnn()) { + return TStatus::Repeat; + } + + if (!(useFlow ? EnsureFlowType(*lambda, ctx) : EnsureStreamType(*lambda, ctx))) { + return TStatus::Error; + } + + if (!ValidateOutputType(*lambda, reduce.Output(), ctx)) { + lambda->SetTypeAnn(nullptr); + lambda->SetState(TExprNode::EState::Initial); + return TStatus::Error; + } + + input->SetTypeAnn(MakeOutputOperationType(reduce, ctx)); + return TStatus::Ok; + } + + TStatus HandleMapReduce(const TExprNode::TPtr& input, TExprNode::TPtr& output, TExprContext& ctx) { + if (!EnsureArgsCount(*input, 7, ctx)) { + return TStatus::Error; + } + + const bool hasMapLambda = !TCoVoid::Match(input->Child(TYtMapReduce::idx_Mapper)); + EYtSettingTypes sectionSettings = EYtSettingType::KeyFilter | EYtSettingType::KeyFilter2 | EYtSettingType::Take | EYtSettingType::Skip | EYtSettingType::Sample; + if (hasMapLambda) { + sectionSettings |= EYtSettingType::SysColumns; + } + auto status = ValidateAndUpdateTransientOpBase(input, output, ctx, true, sectionSettings); + if (status.Level != TStatus::Ok) { + return status; + } + + auto mapReduce = TYtMapReduce(input); + + const auto acceptedSettings = EYtSettingType::ReduceBy + | EYtSettingType::ReduceFilterBy + | EYtSettingType::SortBy + | EYtSettingType::Limit + | EYtSettingType::SortLimitBy + | EYtSettingType::WeakFields + | EYtSettingType::Flow + | EYtSettingType::KeySwitch + | EYtSettingType::MapOutputType + | EYtSettingType::ReduceInputType + | EYtSettingType::NoDq; + if (!ValidateSettings(mapReduce.Settings().Ref(), acceptedSettings, ctx)) { + return TStatus::Error; + } + + if (mapReduce.Output().Size() != 1 && NYql::HasSetting(mapReduce.Settings().Ref(), EYtSettingType::Limit)) { + ctx.AddError(TIssue(ctx.GetPosition(input->Pos()), TStringBuilder() + << EYtSettingType::Limit << " setting is not allowed in operation with multiple outputs")); + return TStatus::Error; + } + TVector<TString> sortLimitBy = NYql::GetSettingAsColumnList(mapReduce.Settings().Ref(), EYtSettingType::SortLimitBy); + if (!sortLimitBy.empty()) { + auto outItemType = mapReduce.Output().Item(0).Ref().GetTypeAnn()->Cast<TListExprType>()->GetItemType(); + if (!ValidateColumnListSetting(ctx.GetPosition(input->Pos()), ctx, outItemType, sortLimitBy, EYtSettingType::SortLimitBy, false)) { + return TStatus::Error; + } + } + + // Ensure output table is not sorted + for (auto out: mapReduce.Output()) { + if (TYqlRowSpecInfo(out.RowSpec()).IsSorted()) { + ctx.AddError(TIssue(ctx.GetPosition(input->Pos()), TStringBuilder() + << TYtMapReduce::CallableName() << " cannot produce sorted output")); + return TStatus::Error; + } + } + + status = TStatus::Ok; + if (hasMapLambda) { + status = status.Combine(ConvertToLambda(input->ChildRef(TYtMapReduce::idx_Mapper), ctx, 1)); + } + status = status.Combine(ConvertToLambda(input->ChildRef(TYtMapReduce::idx_Reducer), ctx, 1)); + if (status.Level != TStatus::Ok) { + return status; + } + + auto itemType = GetInputItemType(mapReduce.Input(), ctx); + const auto useFlow = NYql::GetSetting(mapReduce.Settings().Ref(), EYtSettingType::Flow); + + auto& mapLambda = input->ChildRef(TYtMapReduce::idx_Mapper); + TTypeAnnotationNode::TListType mapDirectOutputTypes; + if (hasMapLambda) { + const auto mapLambdaInputType = MakeInputType(itemType, useFlow, TExprNode::TPtr(), ctx); + + if (!UpdateLambdaAllArgumentsTypes(mapLambda, {mapLambdaInputType}, ctx)) { + return TStatus::Error; + } + + if (!mapLambda->GetTypeAnn()) { + return TStatus::Repeat; + } + + if (!(useFlow ? EnsureFlowType(*mapLambda, ctx) : EnsureStreamType(*mapLambda, ctx))) { + return TStatus::Error; + } + + itemType = GetSequenceItemType(mapLambda->Pos(), mapLambda->GetTypeAnn(), true, ctx); + if (!itemType) { + return TStatus::Error; + } + + if (!EnsurePersistableType(mapLambda->Pos(), *itemType, ctx)) { + return TStatus::Error; + } + + if (itemType->GetKind() == ETypeAnnotationKind::Variant) { + auto tupleType = itemType->Cast<TVariantExprType>()->GetUnderlyingType(); + if (tupleType->GetKind() != ETypeAnnotationKind::Tuple) { + ctx.AddError(TIssue(ctx.GetPosition(input->Pos()), TStringBuilder() + << "Expected Variant over Tuple as map output type, but got: " << *itemType)); + return TStatus::Error; + } + + auto mapOutputs = tupleType->Cast<TTupleExprType>()->GetItems(); + YQL_ENSURE(!mapOutputs.empty(), "Got Variant over empty tuple"); + itemType = mapOutputs.front(); + for (size_t i = 1; i < mapOutputs.size(); ++i) { + if (mapOutputs[i]->GetKind() != ETypeAnnotationKind::Struct) { + ctx.AddError(TIssue(ctx.GetPosition(input->Pos()), TStringBuilder() + << "Expected Struct as map direct output #" << i << ", but got: " << *mapOutputs[i])); + return TStatus::Error; + } + } + mapDirectOutputTypes.assign(mapOutputs.begin() + 1, mapOutputs.end()); + } + + if (const auto kind = itemType->GetKind(); ETypeAnnotationKind::Multi != kind && ETypeAnnotationKind::Struct != kind) { + ctx.AddError(TIssue(ctx.GetPosition(input->Pos()), TStringBuilder() + << "Expected Struct or Multi as map output item type, but got: " << *itemType)); + return TStatus::Error; + } + } + + TVector<TString> reduceBy = NYql::GetSettingAsColumnList(mapReduce.Settings().Ref(), EYtSettingType::ReduceBy); + if (reduceBy.empty()) { + ctx.AddError(TIssue(ctx.GetPosition(input->Pos()), TStringBuilder() + << "Empty " << EYtSettingType::ReduceBy << " option is not allowed in " << input->Content())); + return TStatus::Error; + } + + if (!ValidateColumnListSetting(ctx.GetPosition(input->Pos()), ctx, itemType, reduceBy, EYtSettingType::ReduceBy, true)) { + return TStatus::Error; + } + + if (NYql::HasSetting(mapReduce.Settings().Ref(), EYtSettingType::SortBy)) { + TVector<TString> sortBy = NYql::GetSettingAsColumnList(mapReduce.Settings().Ref(), EYtSettingType::SortBy); + if (!ValidateColumnListSetting(ctx.GetPosition(input->Pos()), ctx, itemType, sortBy, EYtSettingType::SortBy, true)) { + return TStatus::Error; + } + } + + if (const auto reduceInputTypeSetting = NYql::GetSetting(mapReduce.Settings().Ref(), EYtSettingType::ReduceInputType)) { + itemType = reduceInputTypeSetting->Tail().GetTypeAnn()->Cast<TTypeExprType>()->GetType(); + } else { + if (NYql::HasSetting(mapReduce.Settings().Ref(), EYtSettingType::ReduceFilterBy)) { + TVector<TString> reduceFilterBy = NYql::GetSettingAsColumnList(mapReduce.Settings().Ref(), EYtSettingType::ReduceFilterBy); + if (!ValidateColumnListSetting(ctx.GetPosition(input->Pos()), ctx, itemType, reduceFilterBy, EYtSettingType::ReduceFilterBy, true)) { + return TStatus::Error; + } + + auto structType = itemType->Cast<TStructExprType>(); + TVector<const TItemExprType*> filteredItems; + for (auto& column: reduceFilterBy) { + filteredItems.push_back(structType->GetItems()[*structType->FindItem(column)]); + } + + itemType = ctx.MakeType<TStructExprType>(filteredItems); + } + + if (NYql::HasSetting(mapReduce.Settings().Ref(), EYtSettingType::KeySwitch)) { + if (itemType->GetKind() == ETypeAnnotationKind::Variant) { + auto underTupleType = itemType->Cast<TVariantExprType>()->GetUnderlyingType()->Cast<TTupleExprType>(); + TTypeAnnotationNode::TListType newTupleItems; + for (size_t i = 0; i < underTupleType->GetSize(); ++i) { + auto tupleItemType = underTupleType->GetItems()[i]; + auto items = tupleItemType->Cast<TStructExprType>()->GetItems(); + items.push_back(ctx.MakeType<TItemExprType>(YqlSysColumnKeySwitch, ctx.MakeType<TDataExprType>(EDataSlot::Bool))); + newTupleItems.push_back(ctx.MakeType<TStructExprType>(items)); + } + itemType = ctx.MakeType<TVariantExprType>(ctx.MakeType<TTupleExprType>(newTupleItems)); + } else { + auto items = itemType->Cast<TStructExprType>()->GetItems(); + items.push_back(ctx.MakeType<TItemExprType>(YqlSysColumnKeySwitch, ctx.MakeType<TDataExprType>(EDataSlot::Bool))); + itemType = ctx.MakeType<TStructExprType>(items); + } + } + } + + auto& reduceLambda = input->ChildRef(TYtMapReduce::idx_Reducer); + const auto reduceLambdaInputType = MakeInputType(itemType, useFlow, TExprNode::TPtr(), ctx); + + if (!UpdateLambdaAllArgumentsTypes(reduceLambda, {reduceLambdaInputType}, ctx)) { + return TStatus::Error; + } + + if (!reduceLambda->GetTypeAnn()) { + return TStatus::Repeat; + } + + if (!(useFlow ? EnsureFlowType(*reduceLambda, ctx) : EnsureStreamType(*reduceLambda, ctx))) { + return TStatus::Error; + } + + size_t mapDirects = mapDirectOutputTypes.size(); + if (mapDirects == 0) { + if (!ValidateOutputType(*reduceLambda, mapReduce.Output(), ctx)) { + return TStatus::Error; + } + } else { + const size_t outputsSize = mapReduce.Output().Ref().ChildrenSize(); + if (mapDirects >= outputsSize) { + ctx.AddError(TIssue(ctx.GetPosition(input->Pos()), TStringBuilder() + << "Too many map lambda direct outputs: " << mapDirects << ". Total outputs is " << outputsSize)); + return TStatus::Error; + } + + const TTypeAnnotationNode* mapOut = ctx.MakeType<TVariantExprType>(ctx.MakeType<TTupleExprType>(mapDirectOutputTypes)); + if (!ValidateOutputType(mapOut, mapLambda->Pos(), mapReduce.Output(), 0, mapDirects, false, ctx)) { + return TStatus::Error; + } + + const TTypeAnnotationNode* reduceOutType = GetSequenceItemType(reduceLambda->Pos(), reduceLambda->GetTypeAnn(), true, ctx); + if (!reduceOutType) { + return TStatus::Error; + } + + if (!ValidateOutputType(reduceOutType, reduceLambda->Pos(), mapReduce.Output(), mapDirects, outputsSize, false, ctx)) { + return TStatus::Error; + } + } + + input->SetTypeAnn(MakeOutputOperationType(mapReduce, ctx)); + return TStatus::Ok; + } + + TStatus HandleWriteTable(const TExprNode::TPtr& input, TExprNode::TPtr& output, TExprContext& ctx) { + if (!EnsureArgsCount(*input, 5, ctx)) { + return TStatus::Error; + } + + if (input->Child(TYtWriteTable::idx_Content)->IsList()) { + output = ctx.ChangeChild(*input, TYtWriteTable::idx_Content, + Build<TCoAsList>(ctx, input->Pos()) + .Add(input->Child(TYtWriteTable::idx_Content)->ChildrenList()) + .Done() + .Ptr() + ); + return TStatus::Repeat; + } + + if (input->Child(TYtWriteTable::idx_Content)->GetTypeAnn()->GetKind() == ETypeAnnotationKind::EmptyList) { + output = ctx.ChangeChild(*input, TYtWriteTable::idx_Content, + Build<TCoList>(ctx, input->Pos()) + .ListType<TCoListType>() + .ItemType<TCoStructType>() + .Build() + .Build() + .Done().Ptr() + ); + return TStatus::Repeat; + } + + if (!ValidateOpBase(input, ctx)) { + return TStatus::Error; + } + + auto table = input->ChildPtr(TYtWriteTable::idx_Table); + if (!EnsureCallable(*table, ctx)) { + return TStatus::Error; + } + if (!table->IsCallable(TYtTable::CallableName())) { + ctx.AddError(TIssue(ctx.GetPosition(table->Pos()), TStringBuilder() + << "Unexpected callable: " << table->Content())); + return TStatus::Error; + } + + auto settings = input->Child(TYtWriteTable::idx_Settings); + if (!EnsureTuple(*settings, ctx)) { + return TStatus::Error; + } + + if (!ValidateSettings(*settings, EYtSettingType::Mode + | EYtSettingType::Initial + | EYtSettingType::CompressionCodec + | EYtSettingType::ErasureCodec + | EYtSettingType::ReplicationFactor + | EYtSettingType::UserAttrs + | EYtSettingType::Media + | EYtSettingType::PrimaryMedium + | EYtSettingType::Expiration + | EYtSettingType::MonotonicKeys + | EYtSettingType::MutationId + | EYtSettingType::ColumnGroups + | EYtSettingType::SecurityTags + , ctx)) + { + return TStatus::Error; + } + + TExprNode::TPtr newTable; + auto status = UpdateTableMeta(table, newTable, State_->TablesData, false, State_->Types->UseTableMetaFromGraph, ctx); + if (TStatus::Ok != status.Level) { + if (TStatus::Error != status.Level && newTable != table) { + output = ctx.ChangeChild(*input, TYtWriteTable::idx_Table, std::move(newTable)); + } + return status.Combine(TStatus::Repeat); + } + + auto writeTable = TYtWriteTable(input); + auto cluster = writeTable.DataSink().Cluster().StringValue(); + + const TTypeAnnotationNode* itemType = nullptr; + if (!GetSequenceItemType(writeTable.Content().Ref(), itemType, ctx)) { + return TStatus::Error; + } + + auto content = writeTable.Content().Ptr(); + status = ValidateTableWrite(ctx.GetPosition(input->Pos()), table, content, itemType, {}, cluster, *settings, ctx); + if (TStatus::Error == status.Level) { + return status; + } + else if (content != writeTable.Content().Ptr()) { + output = ctx.ChangeChild(*input, TYtWriteTable::idx_Content, std::move(content)); + return status.Combine(TStatus::Repeat); + } + + input->SetTypeAnn(writeTable.World().Ref().GetTypeAnn()); + return TStatus::Ok; + } + + TStatus HandleFill(const TExprNode::TPtr& input, TExprNode::TPtr& /*output*/, TExprContext& ctx) { + if (!EnsureArgsCount(*input, 5, ctx)) { + return TStatus::Error; + } + + if (!ValidateOutputOpBase(input, ctx, true)) { + return TStatus::Error; + } + + auto status = ConvertToLambda(input->ChildRef(TYtFill::idx_Content), ctx, 0); + if (status.Level != TStatus::Ok) { + return status; + } + + // Basic Settings validation + if (!EnsureTuple(*input->Child(TYtFill::idx_Settings), ctx)) { + return TStatus::Error; + } + + for (auto& setting: input->Child(TYtFill::idx_Settings)->Children()) { + if (!EnsureTupleMinSize(*setting, 1, ctx)) { + return TStatus::Error; + } + + if (!EnsureAtom(*setting->Child(0), ctx)) { + return TStatus::Error; + } + } + + const EYtSettingTypes accepted = EYtSettingType::Flow | EYtSettingType::KeepSorted | EYtSettingType::NoDq; + if (!ValidateSettings(*input->Child(TYtFill::idx_Settings), accepted, ctx)) { + return TStatus::Error; + } + + auto fill = TYtFill(input); + auto lambda = fill.Content(); + if (!lambda.Args().Ref().GetTypeAnn()) { + if (!UpdateLambdaArgumentsType(lambda.Ref(), ctx)) + return TStatus::Error; + return TStatus::Repeat; + } + + if (!lambda.Ref().GetTypeAnn()) { + return TStatus::Repeat; + } + + const bool useFlow = NYql::HasSetting(fill.Settings().Ref(), EYtSettingType::Flow); + if (!(useFlow ? EnsureFlowType(lambda.Ref(), ctx) : EnsureStreamType(lambda.Ref(), ctx))) { + return TStatus::Error; + } + + if (!ValidateOutputType(lambda.Ref(), fill.Output(), ctx, true)) { + return TStatus::Error; + } + + input->SetTypeAnn(MakeOutputOperationType(fill, ctx)); + return TStatus::Ok; + } + + TStatus HandleTouch(TExprBase input, TExprContext& ctx) { + if (!EnsureArgsCount(input.Ref(), 3, ctx)) { + return TStatus::Error; + } + + if (!ValidateOutputOpBase(input.Ptr(), ctx, true)) { + return TStatus::Error; + } + + TYtTouch touch = input.Cast<TYtTouch>(); + + input.Ptr()->SetTypeAnn(MakeOutputOperationType(touch, ctx)); + return TStatus::Ok; + } + + TStatus HandleDropTable(const TExprNode::TPtr& input, TExprNode::TPtr& output, TExprContext& ctx) { + if (!EnsureArgsCount(*input, 3, ctx)) { + return TStatus::Error; + } + + if (!ValidateOpBase(input, ctx)) { + return TStatus::Error; + } + + auto table = input->ChildPtr(TYtDropTable::idx_Table); + if (!EnsureCallable(*table, ctx)) { + return TStatus::Error; + } + if (!table->IsCallable(TYtTable::CallableName())) { + ctx.AddError(TIssue(ctx.GetPosition(table->Pos()), TStringBuilder() << "Expected " << TYtTable::CallableName() + << " callable, but got " << table->Content())); + return TStatus::Error; + } + + auto dropTable = TYtDropTable(input); + if (!TYtTableInfo::HasSubstAnonymousLabel(dropTable.Table())) { + TExprNode::TPtr newTable; + auto status = UpdateTableMeta(table, newTable, State_->TablesData, false, State_->Types->UseTableMetaFromGraph, ctx); + if (TStatus::Ok != status.Level) { + if (TStatus::Error != status.Level && newTable != table) { + output = ctx.ChangeChild(*input, TYtWriteTable::idx_Table, std::move(newTable)); + } + return status.Combine(TStatus::Repeat); + } + auto tableInfo = TYtTableInfo(dropTable.Table()); + YQL_ENSURE(tableInfo.Meta); + if (tableInfo.Meta->SqlView) { + ctx.AddError(TIssue(ctx.GetPosition(dropTable.Table().Pos()), TStringBuilder() + << "Drop of " << tableInfo.Name.Quote() << " view is not supported")); + return TStatus::Error; + } + + if (tableInfo.Meta->IsDynamic) { + ctx.AddError(TIssue(ctx.GetPosition(dropTable.Table().Pos()), TStringBuilder() << + "Drop of dynamic table " << tableInfo.Name.Quote() << " is not supported")); + return TStatus::Error; + } + + if (auto commitEpoch = tableInfo.CommitEpoch) { + TYtTableDescription& nextDescription = State_->TablesData->GetOrAddTable( + TString{dropTable.DataSink().Cluster().Value()}, + tableInfo.Name, + commitEpoch + ); + + TYtTableMetaInfo::TPtr nextMetadata = nextDescription.Meta; + if (!nextMetadata) { + nextDescription.RowType = nullptr; + nextDescription.RawRowType = nullptr; + + nextMetadata = nextDescription.Meta = MakeIntrusive<TYtTableMetaInfo>(); + nextMetadata->DoesExist = false; + } + else if (nextMetadata->DoesExist) { + ctx.AddError(TIssue(ctx.GetPosition(dropTable.Table().Pos()), TStringBuilder() << + "Table " << tableInfo.Name << " is modified and dropped in the same transaction")); + return TStatus::Error; + } + } + } + + input->SetTypeAnn(dropTable.World().Ref().GetTypeAnn()); + return TStatus::Ok; + } + + TStatus HandleCommit(TExprBase input, TExprContext& ctx) { + auto commit = input.Cast<TCoCommit>(); + auto settings = NCommon::ParseCommitSettings(commit, ctx); + + if (settings.Epoch) { + ui32 epoch = 0; + if (!TryFromString(settings.Epoch.Cast().Value(), epoch)) { + ctx.AddError(TIssue(ctx.GetPosition(commit.Pos()), TStringBuilder() << "Bad commit epoch: " + << settings.Epoch.Cast().Value())); + return TStatus::Error; + } + } + + if (!settings.EnsureModeEmpty(ctx)) { + return TStatus::Error; + } + if (!settings.EnsureOtherEmpty(ctx)) { + return TStatus::Error; + } + + input.Ptr()->SetTypeAnn(commit.World().Ref().GetTypeAnn()); + return TStatus::Ok; + } + + TStatus HandlePublish(const TExprNode::TPtr& input, TExprNode::TPtr& output, TExprContext& ctx) { + if (!EnsureArgsCount(*input, 5, ctx)) { + return TStatus::Error; + } + + if (!ValidateOpBase(input, ctx)) { + return TStatus::Error; + } + + if (!EnsureTupleMinSize(*input->Child(TYtPublish::idx_Input), 1, ctx)) { + return TStatus::Error; + } + + const TTypeAnnotationNode* itemType = nullptr; + for (auto& child: input->Child(TYtPublish::idx_Input)->Children()) { + if (!EnsureCallable(*child, ctx)) { + return TStatus::Error; + } + + if (!child->IsCallable(TYtOutput::CallableName())) { + ctx.AddError(TIssue(ctx.GetPosition(child->Pos()), TStringBuilder() + << "Expected " << TYtOutput::CallableName() << " callable, but got " + << child->Content())); + return TStatus::Error; + } + + const TTypeAnnotationNode* childItemType = nullptr; + if (!GetSequenceItemType(*child, childItemType, ctx)) { + return TStatus::Error; + } + if (nullptr == itemType) { + itemType = childItemType; + } else if (!IsSameAnnotation(*itemType, *childItemType)) { + ctx.AddError(TIssue(ctx.GetPosition(child->Pos()), TStringBuilder() + << TYtPublish::CallableName() << " inputs have different item types: " + << GetTypeDiff(*itemType, *childItemType))); + return TStatus::Error; + } + } + + auto table = input->ChildPtr(TYtPublish::idx_Publish); + if (!EnsureCallable(*table, ctx)) { + return TStatus::Error; + } + + if (!table->IsCallable(TYtTable::CallableName())) { + ctx.AddError(TIssue(ctx.GetPosition(table->Pos()), TStringBuilder() << "Expected " << TYtTable::CallableName() + << " callable, but got " << table->Content())); + return TStatus::Error; + } + + auto settings = input->Child(TYtPublish::idx_Settings); + if (!EnsureTuple(*settings, ctx)) { + return TStatus::Error; + } + + if (!ValidateSettings(*settings, EYtSettingType::Mode + | EYtSettingType::Initial + | EYtSettingType::CompressionCodec + | EYtSettingType::ErasureCodec + | EYtSettingType::ReplicationFactor + | EYtSettingType::UserAttrs + | EYtSettingType::Media + | EYtSettingType::PrimaryMedium + | EYtSettingType::Expiration + | EYtSettingType::MonotonicKeys + | EYtSettingType::MutationId + | EYtSettingType::ColumnGroups + | EYtSettingType::SecurityTags + , ctx)) + { + return TStatus::Error; + } + + if (!NYql::HasSetting(*table->Child(TYtTable::idx_Settings), EYtSettingType::Anonymous) + || !table->Child(TYtTable::idx_Name)->Content().StartsWith("tmp/")) + { + auto publish = TYtPublish(input); + + TVector<TYqlRowSpecInfo::TPtr> contentRowSpecs; + for (auto out: publish.Input()) { + contentRowSpecs.push_back(MakeIntrusive<TYqlRowSpecInfo>(GetOutTable(out).Cast<TYtOutTable>().RowSpec())); + if (IsUnorderedOutput(out)) { + contentRowSpecs.back()->ClearSortness(ctx); + } + } + TExprNode::TPtr content; // Don't try to convert content + auto status = ValidateTableWrite(ctx.GetPosition(input->Pos()), table, content, itemType, contentRowSpecs, publish.DataSink().Cluster().StringValue(), *settings, ctx); + if (TStatus::Ok != status.Level) { + return status; + } + + TExprNode::TPtr newTable; + status = UpdateTableMeta(table, newTable, State_->TablesData, false, State_->Types->UseTableMetaFromGraph, ctx); + if (TStatus::Ok != status.Level) { + if (TStatus::Error != status.Level && newTable != table) { + output = ctx.ChangeChild(*input, TYtPublish::idx_Publish, std::move(newTable)); + } + return status.Combine(TStatus::Repeat); + } + } + + input->SetTypeAnn(input->Child(TYtPublish::idx_World)->GetTypeAnn()); + return TStatus::Ok; + } + + TStatus HandleEquiJoin(const TExprNode::TPtr& input, TExprNode::TPtr& output, TExprContext& ctx) { + if (!EnsureMinArgsCount(*input, 8, ctx)) { + return TStatus::Error; + } + + auto status = ValidateAndUpdateTransientOpBase(input, output, ctx, true, + EYtSettingType::KeyFilter | EYtSettingType::KeyFilter2 | EYtSettingType::Take | EYtSettingType::Skip | EYtSettingType::JoinLabel | EYtSettingType::StatColumns | EYtSettingType::SysColumns); + if (status.Level != TStatus::Ok) { + return status; + } + + // Only one output + if (!EnsureTupleSize(*input->Child(TYtEquiJoin::idx_Output), 1, ctx)) { + return TStatus::Error; + } + + if (!ValidateSettings(*input->Child(TYtEquiJoin::idx_Settings), EYtSettingType::Limit, ctx)) { + return TStatus::Error; + } + + const auto inputsCount = input->ChildPtr(TYtEquiJoin::idx_Input)->ChildrenSize(); + if (!EnsureArgsCount(*input, 7U + inputsCount, ctx)) { + return TStatus::Error; + } + + TJoinLabels labels; + for (ui32 i = 0; i < inputsCount; ++i) { + const TYtSection section(input->ChildPtr(TYtEquiJoin::idx_Input)->ChildPtr(i)); + if (auto label = NYql::GetSetting(section.Settings().Ref(), EYtSettingType::JoinLabel)) { + auto itemType = GetSequenceItemType(section, false, ctx); + if (!itemType || !EnsurePersistableType(section.Pos(), *itemType, ctx)) { + return TStatus::Error; + } + + if (auto& lambda = input->ChildRef(i + 7U); lambda->IsLambda()) { + if (!UpdateLambdaAllArgumentsTypes(lambda, {GetSequenceItemType(section, false, ctx)}, ctx)) + return TStatus::Error; + + if (!lambda->GetTypeAnn()) { + return TStatus::Repeat; + } + + itemType = GetSequenceItemType(TExprBase(lambda), false, ctx); + if (!itemType || !EnsurePersistableType(lambda->Pos(), *itemType, ctx)) { + return TStatus::Error; + } + } else if (!lambda->IsCallable("Void")) { + ctx.AddError(TIssue(ctx.GetPosition(lambda->Pos()), TStringBuilder() + << "Premap node should be either lambda or Void, got: " << lambda->Content())); + return TStatus::Error; + } + + if (auto err = labels.Add(ctx, *label->Child(1), itemType->Cast<TStructExprType>())) { + ctx.AddError(*err); + return TStatus::Error; + } + } + else { + ctx.AddError(TIssue(ctx.GetPosition(section.Pos()), TStringBuilder() + << "Setting \"" << EYtSettingType::JoinLabel + << "\" is required in " << input->Content() << " section")); + return TStatus::Error; + } + } + + TJoinOptions joinOptions; + status = ValidateEquiJoinOptions(input->Pos(), *input->Child(TYtEquiJoin::idx_JoinOptions), joinOptions, ctx); + if (status != TStatus::Ok) { + return status; + } + + const TStructExprType* resultType = nullptr; + status = EquiJoinAnnotation(input->Pos(), resultType, labels, + *input->Child(TYtEquiJoin::idx_Joins), joinOptions, ctx); + if (status != TStatus::Ok) { + return status; + } + + const TYtEquiJoin equiJoin(input); + if (!ValidateOutputType(resultType, equiJoin.Pos(), equiJoin.Output(), ctx)) { + return TStatus::Error; + } + + input->SetTypeAnn(MakeOutputOperationType(equiJoin, ctx)); + return TStatus::Ok; + } + + TStatus HandleStatOutTable(const TExprNode::TPtr& input, TExprNode::TPtr& output, TExprContext& ctx) { + Y_UNUSED(output); + + if (!EnsureMinArgsCount(*input, 3, ctx)) { + return TStatus::Error; + } + + if (!EnsureAtom(*input->Child(TYtStatOutTable::idx_Name), ctx)) { + return TStatus::Error; + } + + if (!EnsureAtom(*input->Child(TYtStatOutTable::idx_Scale), ctx)) { + return TStatus::Error; + } + + if (!EnsureAtom(*input->Child(TYtStatOutTable::idx_Cluster), ctx)) { + return TStatus::Error; + } + + input->SetTypeAnn(ctx.MakeType<TUnitExprType>()); + + return TStatus::Ok; + } + + TStatus HandleStatOut(const TExprNode::TPtr& input, TExprNode::TPtr& output, TExprContext& ctx) { + Y_UNUSED(output); + + if (!EnsureArgsCount(*input, 6, ctx)) { + return TStatus::Error; + } + + if (!ValidateOpBase(input, ctx)) { + return TStatus::Error; + } + + auto inputChild = input->Child(TYtStatOut::idx_Input); + if (!TMaybeNode<TYtOutput>(inputChild)) { + ctx.AddError(TIssue(ctx.GetPosition(inputChild->Pos()), TStringBuilder() + << "Unexpected input: " << inputChild->Content())); + return TStatus::Error; + } + + auto table = input->Child(TYtStatOut::idx_Table); + if (!table->IsCallable(TYtStatOutTable::CallableName())) { + ctx.AddError(TIssue(ctx.GetPosition(table->Pos()), TStringBuilder() << + "Unexpected callable: " << table->Content() + )); + return TStatus::Error; + } + + auto replaceMask = input->Child(TYtStatOut::idx_ReplaceMask); + if (!EnsureTuple(*replaceMask, ctx)) { + return TStatus::Error; + } + + for (const auto& child: replaceMask->Children()) { + if (!EnsureAtom(*child, ctx)) { + return TStatus::Error; + } + } + + auto settings = input->Child(TYtStatOut::idx_Settings); + if (!EnsureTuple(*settings, ctx)) { + return TStatus::Error; + } + + for (auto& child: settings->Children()) { + if (!EnsureTupleMinSize(*child, 1, ctx)) { + return TStatus::Error; + } + + if (!EnsureAtom(*child->Child(0), ctx)) { + return TStatus::Error; + } + } + + input->SetTypeAnn(input->Child(TYtStatOut::idx_World)->GetTypeAnn()); + + return TStatus::Ok; + } + + TStatus HandleYtDqProcessWrite(const TExprNode::TPtr& input, TExprContext& ctx) { + if (!ValidateOutputOpBase(input, ctx, false)) { + return TStatus::Error; + } + + if (!EnsureMinMaxArgsCount(*input, 4, 5, ctx)) { + return TStatus::Error; + } + + if (!EnsureCallable(*input->Child(3), ctx)) { + return TStatus::Error; + } + + if (input->ChildrenSize() > 4 && !EnsureTupleOfAtoms(input->Tail(), ctx)) { + return TStatus::Error; + } + + auto processWrite = TYtDqProcessWrite(input); + + input->SetTypeAnn(MakeOutputOperationType(processWrite, ctx)); + return TStatus::Ok; + } + + template <bool Wide> + TStatus HandleDqWrite(const TExprNode::TPtr& input, TExprContext& ctx) { + if (!EnsureArgsCount(*input, 2, ctx)) { + return TStatus::Error; + } + + if constexpr (Wide) { + if (!EnsureWideFlowType(input->Head(), ctx)) { + return TStatus::Error; + } + } else { + if (!EnsureNewSeqType<false, false, true>(input->Head(), ctx)) { + return TStatus::Error; + } + } + + if (!EnsureTuple(input->Tail(), ctx)) { + return TStatus::Error; + } + + for (auto& setting: input->Tail().Children()) { + if (!EnsureTupleMinSize(*setting, 1, ctx)) { + return TStatus::Error; + } + + if (!EnsureAtom(*setting->Child(0), ctx)) { + return TStatus::Error; + } + } + + input->SetTypeAnn(MakeSequenceType(input->Head().GetTypeAnn()->GetKind(), *ctx.MakeType<TVoidExprType>(), ctx)); + return TStatus::Ok; + } + + TStatus HandleTryFirst(TExprBase input, TExprContext& ctx) { + if (!EnsureArgsCount(input.Ref(), 2, ctx)) { + return TStatus::Error; + } + + if (!TYtOutputOpBase::Match(input.Ref().Child(TYtTryFirst::idx_First))) { + ctx.AddError(TIssue(ctx.GetPosition(input.Ref().Child(TYtTryFirst::idx_First)->Pos()), TStringBuilder() << "Expect YT operation, but got: " + << input.Ref().Child(TYtTryFirst::idx_First)->Content())); + return TStatus::Error; + } + + if (!TYtOutputOpBase::Match(input.Ref().Child(TYtTryFirst::idx_Second))) { + ctx.AddError(TIssue(ctx.GetPosition(input.Ref().Child(TYtTryFirst::idx_Second)->Pos()), TStringBuilder() << "Expect YT operation, but got: " + << input.Ref().Child(TYtTryFirst::idx_Second)->Content())); + return TStatus::Error; + } + + if (!IsSameAnnotation(*input.Ref().Child(TYtTryFirst::idx_First)->GetTypeAnn(), *input.Ref().Child(TYtTryFirst::idx_Second)->GetTypeAnn())) { + ctx.AddError(TIssue(ctx.GetPosition(input.Pos()), TStringBuilder() << "Both argumensts must be same type.")); + return TStatus::Error; + } + + input.Ptr()->SetTypeAnn(input.Ref().Head().GetTypeAnn()); + return TStatus::Ok; + } + + TStatus HandleMaterialize(TExprBase input, TExprContext& ctx) { + if (!EnsureArgsCount(input.Ref(), 4, ctx)) { + return TStatus::Error; + } + + if (!ValidateOpBase(input.Ptr(), ctx)) { + return TStatus::Error; + } + + if (!EnsureSeqOrOptionalType(*input.Ref().Child(TYtMaterialize::idx_Input), ctx)) { + return IGraphTransformer::TStatus::Error; + } + const auto& itemType = GetSeqItemType(*input.Ref().Child(TYtMaterialize::idx_Input)->GetTypeAnn()); + if (!EnsurePersistableType(input.Ref().Head().Pos(), itemType, ctx)) { + return IGraphTransformer::TStatus::Error; + } + + // Basic Settings validation + if (!EnsureTuple(*input.Ref().Child(TYtMaterialize::idx_Settings), ctx)) { + return TStatus::Error; + } + + if (!ValidateSettings(*input.Ref().Child(TYtMaterialize::idx_Settings), EYtSettingTypes{}, ctx)) { + return TStatus::Error; + } + + input.Ptr()->SetTypeAnn(ctx.MakeType<TListExprType>(&itemType)); + return TStatus::Ok; + } + +private: + const TYtState::TPtr State_; +}; + +} + +THolder<TVisitorTransformerBase> CreateYtDataSinkTypeAnnotationTransformer(TYtState::TPtr state) { + return THolder(new TYtDataSinkTypeAnnotationTransformer(state)); +} + +} diff --git a/yt/yql/providers/yt/provider/yql_yt_datasource.cpp b/yt/yql/providers/yt/provider/yql_yt_datasource.cpp new file mode 100644 index 0000000000..7e1ea43a21 --- /dev/null +++ b/yt/yql/providers/yt/provider/yql_yt_datasource.cpp @@ -0,0 +1,933 @@ +#include "yql_yt_provider.h" +#include "yql_yt_provider_impl.h" +#include "yql_yt_helpers.h" +#include "yql_yt_key.h" +#include "yql_yt_op_settings.h" +#include "yql_yt_dq_integration.h" +#include "yql_yt_dq_optimize.h" + +#include <yql/essentials/providers/common/structured_token/yql_token_builder.h> +#include <yt/yql/providers/yt/expr_nodes/yql_yt_expr_nodes.h> +#include <yql/essentials/providers/result/expr_nodes/yql_res_expr_nodes.h> +#include <yt/yql/providers/yt/common/yql_names.h> +#include <yt/yql/providers/yt/common/yql_configuration.h> +#include <yt/yql/providers/yt/lib/schema/schema.h> +#include <yql/essentials/providers/common/proto/gateways_config.pb.h> +#include <yql/essentials/providers/common/provider/yql_provider_names.h> +#include <yql/essentials/providers/common/provider/yql_provider.h> +#include <yql/essentials/providers/common/provider/yql_data_provider_impl.h> +#include <yql/essentials/providers/common/transform/yql_lazy_init.h> +#include <yql/essentials/providers/common/config/yql_configuration_transformer.h> +#include <yql/essentials/providers/common/config/yql_dispatch.h> +#include <yql/essentials/core/expr_nodes/yql_expr_nodes.h> +#include <yql/essentials/core/yql_expr_type_annotation.h> +#include <yql/essentials/core/yql_opt_utils.h> + +#include <yql/essentials/utils/log/log.h> + +#include <yt/cpp/mapreduce/common/helpers.h> + +#include <library/cpp/yson/node/node_io.h> + +namespace NYql { + +using namespace NNodes; + +class TYtDataSourceTrackableNodeProcessor : public TTrackableNodeProcessorBase { +public: + TYtDataSourceTrackableNodeProcessor(bool collectNodes) + : CollectNodes(collectNodes) + { + } + + void GetUsedNodes(const TExprNode& input, TVector<TString>& usedNodeIds) override { + usedNodeIds.clear(); + if (!CollectNodes) { + return; + } + + if (auto maybeResPull = TMaybeNode<TResPull>(&input)) { + ScanForUsedOutputTables(maybeResPull.Cast().Data().Ref(), usedNodeIds); + } else if (auto maybeResFill = TMaybeNode<TResFill>(&input)){ + ScanForUsedOutputTables(maybeResFill.Cast().Data().Ref(), usedNodeIds); + } else if (auto maybeResIf = TMaybeNode<TResIf>(&input)){ + ScanForUsedOutputTables(maybeResIf.Cast().Condition().Ref(), usedNodeIds); + } else if (TMaybeNode<TYtReadTable>(&input)) { + ScanForUsedOutputTables(*input.Child(TYtReadTable::idx_Input), usedNodeIds); + } else if (TMaybeNode<TYtReadTableScheme>(&input)) { + ScanForUsedOutputTables(*input.Child(TYtReadTableScheme::idx_Type), usedNodeIds); + } + } + +private: + const bool CollectNodes; +}; + +namespace { + +const THashSet<TStringBuf> CONFIGURE_CALLABLES = { + TCoConfigure::CallableName(), + TYtConfigure::CallableName(), +}; + +} + +class TYtDataSource : public TDataProviderBase { +public: + TYtDataSource(TYtState::TPtr state) + : State_(state) + , ConfigurationTransformer_([this]() { + return MakeHolder<NCommon::TProviderConfigurationTransformer>(State_->Configuration, *State_->Types, TString{YtProviderName}, CONFIGURE_CALLABLES); + }) + , IODiscoveryTransformer_([this]() { return CreateYtIODiscoveryTransformer(State_); }) + , EpochTransformer_([this]() { return CreateYtEpochTransformer(State_); }) + , IntentDeterminationTransformer_([this]() { return CreateYtIntentDeterminationTransformer(State_); }) + , LoadMetaDataTransformer_([this]() { return CreateYtLoadTableMetadataTransformer(State_); }) + , TypeAnnotationTransformer_([this]() { return CreateYtDataSourceTypeAnnotationTransformer(State_); }) + , ConstraintTransformer_([this]() { return CreateYtDataSourceConstraintTransformer(State_); }) + , ExecTransformer_([this]() { return CreateYtDataSourceExecTransformer(State_); }) + , TrackableNodeProcessor_([this]() { + auto mode = GetReleaseTempDataMode(*State_->Configuration); + bool collectNodes = mode == EReleaseTempDataMode::Immediate; + return MakeHolder<TYtDataSourceTrackableNodeProcessor>(collectNodes); + }) + , DqOptimizer_([this]() { return CreateYtDqOptimizers(State_); }) + { + } + + void AddCluster(const TString& name, const THashMap<TString, TString>& properties) override { + const TString& token = properties.Value("token", ""); + + State_->Configuration->AddValidCluster(name); + if (token) { + // Empty token is forbidden for yt reader + State_->Configuration->Tokens[name] = ComposeStructuredTokenJsonForTokenAuthWithSecret(properties.Value("tokenReference", ""), token); + } + + TYtClusterConfig cluster; + cluster.SetName(name); + cluster.SetCluster(properties.Value("location", "")); + cluster.SetYTToken(token); + State_->Gateway->AddCluster(cluster); + } + + TStringBuf GetName() const override { + return YtProviderName; + } + + IGraphTransformer& GetConfigurationTransformer() override { + return *ConfigurationTransformer_; + } + + IGraphTransformer& GetIODiscoveryTransformer() override { + return *IODiscoveryTransformer_; + } + + IGraphTransformer& GetEpochsTransformer() override { + return *EpochTransformer_; + } + + IGraphTransformer& GetIntentDeterminationTransformer() override { + return *IntentDeterminationTransformer_; + } + + IGraphTransformer& GetTypeAnnotationTransformer(bool instantOnly) override { + Y_UNUSED(instantOnly); + return *TypeAnnotationTransformer_; + } + + IGraphTransformer& GetConstraintTransformer(bool instantOnly, bool subGraph) override { + Y_UNUSED(instantOnly); + Y_UNUSED(subGraph); + return *ConstraintTransformer_; + } + + IGraphTransformer& GetLoadTableMetadataTransformer() override { + return *LoadMetaDataTransformer_; + } + + IGraphTransformer& GetCallableExecutionTransformer() override { + return *ExecTransformer_; + } + + bool Initialize(TExprContext& ctx) override { + auto category = YtProviderName; + auto cred = State_->Types->Credentials->FindCredential(TString("default_").append(category)); + if (cred) { + if (cred->Category != category) { + ctx.AddError(TIssue({}, TStringBuilder() + << "Mismatch default credential category, expected: " << category + << ", but found: " << cred->Category)); + return false; + } + State_->Configuration->Auth = cred->Content; + } + + return true; + } + + const THashMap<TString, TString>* GetClusterTokens() override { + return &State_->Configuration->Tokens; + } + + bool ValidateParameters(TExprNode& node, TExprContext& ctx, TMaybe<TString>& cluster) override { + if (node.IsCallable(TCoDataSource::CallableName())) { + if (!EnsureArgsCount(node, 2, ctx)) { + return false; + } + + if (node.Child(0)->Content() == YtProviderName) { + if (!node.Child(1)->IsCallable("EvaluateAtom")) { + if (!EnsureAtom(*node.Child(1), ctx)) { + return false; + } + + if (node.Child(1)->Content().empty()) { + ctx.AddError(TIssue(ctx.GetPosition(node.Child(1)->Pos()), "Empty cluster name")); + return false; + } + + cluster = TString(node.Child(1)->Content()); + } + + return true; + } + } + + ctx.AddError(TIssue(ctx.GetPosition(node.Pos()), "Invalid Yt DataSource parameters")); + return false; + } + + bool CanParse(const TExprNode& node) override { + if (node.IsCallable(TCoRead::CallableName())) { + return TYtDSource::Match(node.Child(1)); + } + + return TypeAnnotationTransformer_->CanParse(node); + } + + TExprNode::TPtr RewriteIO(const TExprNode::TPtr& node, TExprContext& ctx) override { + YQL_CLOG(INFO, ProviderYt) << "RewriteIO"; + + auto read = TCoInputBase(node).Input().Cast<TYtRead>(); + bool buildLeft = TCoLeft::Match(node.Get()); + bool buildReadTableScheme = NYql::HasSetting(read.Arg(4).Ref(), EYtSettingType::Scheme); + + if (buildLeft && (buildReadTableScheme || !State_->PassiveExecution)) { + return read.World().Ptr(); + } + + if (buildReadTableScheme) { + YQL_ENSURE(read.Arg(2).Maybe<TExprList>().Item(0).Maybe<TYtPath>()); + + auto newRead = InjectUdfRemapperOrView(read, ctx, true, false); + + return Build<TCoRight>(ctx, node->Pos()) + .Input<TYtReadTableScheme>() + .World(read.World()) + .DataSource(read.DataSource()) + .Table(read.Arg(2).Cast<TExprList>().Item(0).Cast<TYtPath>().Table().Cast<TYtTable>()) + .Type<TCoTypeOf>() + .Value(newRead) + .Build() + .Build() + .Done().Ptr(); + } + + YQL_ENSURE(read.Arg(2).Maybe<TExprList>().Item(0).Maybe<TYtPath>()); // At least one table + return InjectUdfRemapperOrView(read, ctx, false, buildLeft); + } + + void PostRewriteIO() final { + if (!State_->Types->EvaluationInProgress) { + State_->TablesData->CleanupCompiledSQL(); + } + } + + void Reset() final { + TDataProviderBase::Reset(); + State_->TablesData = MakeIntrusive<TYtTablesData>(); + State_->Configuration->ClearVersions(); + State_->NodeHash.clear(); + State_->Checkpoints.clear(); + } + + void EnterEvaluation(ui64 id) final { + State_->EnterEvaluation(id); + } + + void LeaveEvaluation(ui64 id) final { + State_->LeaveEvaluation(id); + } + + bool IsPersistent(const TExprNode& node) override { + return IsYtProviderInput(NNodes::TExprBase(&node)); + } + + bool IsRead(const TExprNode& node) override { + return TYtReadTable::Match(&node); + } + + bool CanBuildResult(const TExprNode& node, TSyncMap& syncList) override { + TString usedCluster; + return IsYtCompleteIsolatedLambda(node, syncList, usedCluster, false); + } + + bool GetExecWorld(const TExprNode::TPtr& node, TExprNode::TPtr& root) override { + auto read = TMaybeNode<TYtReadTable>(node); + if (!read) { + return false; + } + + root = read.Cast().World().Ptr(); + return true; + } + + bool CanEvaluate(const TExprNode& node) override { + return TYtConfigure::Match(&node); + } + + bool CanPullResult(const TExprNode& node, TSyncMap& syncList, bool& canRef) override { + Y_UNUSED(syncList); + canRef = true; + auto input = NNodes::TExprBase(&node); + return IsYtProviderInput(input) || input.Maybe<TCoRight>().Input().Maybe<TYtReadTableScheme>(); + } + + TExprNode::TPtr CleanupWorld(const TExprNode::TPtr& node, TExprContext& ctx) override { + return YtCleanupWorld(node, ctx, State_); + } + + TExprNode::TPtr OptimizePull(const TExprNode::TPtr& source, const TFillSettings& fillSettings, TExprContext& ctx, + IOptimizationContext& optCtx) override + { + Y_UNUSED(optCtx); + + auto maybeRight = TMaybeNode<TCoRight>(source); + if (!maybeRight || !fillSettings.RowsLimitPerWrite) { + return source; + } + + auto maybeRead = maybeRight.Input().Maybe<TYtReadTable>(); + if (!maybeRead) { + return source; + } + auto read = maybeRead.Cast(); + if (read.Input().Size() > 1) { + return source; + } + auto section = read.Input().Item(0); + ui64 totalCount = 0; + for (auto path: section.Paths()) { + TYtTableBaseInfo::TPtr tableInfo = TYtTableBaseInfo::Parse(path.Table()); + if (!tableInfo->Meta || !tableInfo->Stat || tableInfo->Meta->IsDynamic) { + totalCount = Max<ui64>(); + break; + } + totalCount += tableInfo->Stat->RecordsCount; + } + + if (totalCount <= *fillSettings.RowsLimitPerWrite) { + return source; + } + + auto newSettings = NYql::AddSetting( + section.Settings().Ref(), + EYtSettingType::Take, + Build<TCoUint64>(ctx, read.Pos()) + .Literal() + .Value(ToString(*fillSettings.RowsLimitPerWrite)) + .Build() + .Done().Ptr(), + ctx); + + return Build<TCoRight>(ctx, maybeRight.Cast().Pos()) + .Input<TYtReadTable>() + .World(read.World()) + .DataSource(read.DataSource()) + .Input() + .Add() + .Paths(section.Paths()) + .Settings(newSettings) + .Build() + .Build() + .Build() + .Done().Ptr(); + } + + bool CanExecute(const TExprNode& node) override { + return ExecTransformer_->CanExec(node); + } + + void GetRequiredChildren(const TExprNode& node, TExprNode::TListType& children) override { + if (CanExecute(node) && !TYtTable::Match(&node)) { + children.push_back(node.ChildPtr(0)); + if (TYtReadTable::Match(&node)) { + children.push_back(node.ChildPtr(TYtReadTable::idx_Input)); + } + } + } + + bool GetDependencies(const TExprNode& node, TExprNode::TListType& children, bool compact) override { + Y_UNUSED(compact); + if (CanExecute(node)) { + children.push_back(node.ChildPtr(0)); + if (node.Content() == TYtConfigure::CallableName()) { + return false; + } + + if (TMaybeNode<TYtReadTable>(&node)) { + ScanPlanDependencies(node.ChildPtr(TYtReadTable::idx_Input), children); + } + + if (TMaybeNode<TYtReadTableScheme>(&node)) { + ScanPlanDependencies(node.ChildPtr(TYtReadTableScheme::idx_Type), children); + } + + return true; + } + + return false; + } + + void GetResultDependencies(const TExprNode::TPtr& node, TExprNode::TListType& children, bool compact) override { + Y_UNUSED(compact); + ScanPlanDependencies(node, children); + } + + void WritePlanDetails(const TExprNode& node, NYson::TYsonWriter& writer, bool withLimits) override { + if (auto maybeRead = TMaybeNode<TYtReadTable>(&node)) { + writer.OnKeyedItem("InputColumns"); + auto read = maybeRead.Cast(); + if (read.Input().Size() > 1) { + writer.OnBeginList(); + for (auto section: read.Input()) { + writer.OnListItem(); + NCommon::WriteColumns(writer, section.Paths().Item(0).Columns()); + } + writer.OnEndList(); + } + else { + NCommon::WriteColumns(writer, read.Input().Item(0).Paths().Item(0).Columns()); + } + + if (read.Input().Size() > 1) { + writer.OnKeyedItem("InputSections"); + writer.OnBeginList(); + ui64 ndx = 0; + for (auto section: read.Input()) { + writer.OnListItem(); + writer.OnBeginList(); + for (ui32 i = 0; i < Min((ui32)section.Paths().Size(), withLimits && State_->PlanLimits ? State_->PlanLimits : Max<ui32>()); ++i) { + writer.OnListItem(); + writer.OnUint64Scalar(ndx++); + } + writer.OnEndList(); + } + writer.OnEndList(); + } + } + } + + void WritePullDetails(const TExprNode& node, NYson::TYsonWriter& writer) override { + writer.OnKeyedItem("PullOperation"); + writer.OnStringScalar(node.Child(0)->Content()); + } + + TString GetProviderPath(const TExprNode& node) override { + return TStringBuilder() << YtProviderName << '.' << node.Child(1)->Content(); + } + + void WriteDetails(const TExprNode& node, NYson::TYsonWriter& writer) override { + writer.OnKeyedItem("Cluster"); + writer.OnStringScalar(node.Child(1)->Content()); + } + + ui32 GetInputs(const TExprNode& node, TVector<TPinInfo>& inputs, bool withLimit) override { + ui32 count = 0; + if (auto maybeRead = TMaybeNode<TYtReadTable>(&node)) { + auto read = maybeRead.Cast(); + for (auto section: read.Input()) { + ui32 i = 0; + for (auto path: section.Paths()) { + if (auto maybeTable = path.Table().Maybe<TYtTable>()) { + inputs.push_back(TPinInfo(read.DataSource().Raw(), nullptr, maybeTable.Cast().Raw(), MakeTableDisplayName(maybeTable.Cast(), false), false)); + } + else { + auto tmpTable = GetOutTable(path.Table()); + inputs.push_back(TPinInfo(read.DataSource().Raw(), nullptr, tmpTable.Raw(), MakeTableDisplayName(tmpTable, false), true)); + } + if (withLimit && State_->PlanLimits && ++i >= State_->PlanLimits) { + break; + } + } + count += section.Paths().Size(); + } + } + else if (auto maybeReadScheme = TMaybeNode<TYtReadTableScheme>(&node)) { + auto readScheme = maybeReadScheme.Cast(); + inputs.push_back(TPinInfo(readScheme.DataSource().Raw(), nullptr, readScheme.Table().Raw(), MakeTableDisplayName(readScheme.Table(), false), false)); + count = 1; + } + return count; + } + + void WritePinDetails(const TExprNode& node, NYson::TYsonWriter& writer) override { + writer.OnKeyedItem("Table"); + if (auto table = TMaybeNode<TYtTable>(&node)) { + writer.OnStringScalar(table.Cast().Name().Value()); + } + else { + writer.OnStringScalar("(tmp)"); + } + } + + bool WriteSchemaHeader(NYson::TYsonWriter& writer) override { + writer.OnKeyedItem("YtSchema"); + return true; + } + + void WriteTypeDetails(NYson::TYsonWriter& writer, const TTypeAnnotationNode& type) override { + writer.OnStringScalar(GetTypeV3String(type)); + } + + ITrackableNodeProcessor& GetTrackableNodeProcessor() override { + return *TrackableNodeProcessor_; + } + + bool CollectDiscoveredData(NYson::TYsonWriter& writer) override { + THashMap<std::pair<TString, TString>, THashSet<TString>> tables; + State_->TablesData->ForEach([&tables](const TString& cluster, const TString& table, ui32 /*epoch*/, const TYtTableDescription& tableDesc) { + if (!tableDesc.IsAnonymous) { + TStringBuf intent; + if (tableDesc.Intents & TYtTableIntent::Drop) { + intent = "drop"; + } else if (tableDesc.Intents & (TYtTableIntent::Override | TYtTableIntent::Append)) { + intent = "modify"; + } else if (tableDesc.Intents & TYtTableIntent::Flush) { + intent = "flush"; + } else { + intent = "read"; + } + tables[std::make_pair(cluster, table)].emplace(intent); + } + }); + if (tables.empty()) { + return false; + } + writer.OnBeginList(); + for (auto& item: tables) { + writer.OnListItem(); + + writer.OnBeginList(); + writer.OnListItem(); + writer.OnStringScalar(item.first.first); + writer.OnListItem(); + writer.OnStringScalar(item.first.second); + writer.OnListItem(); + + writer.OnBeginList(); + for (auto& intent: item.second) { + writer.OnListItem(); + writer.OnStringScalar(intent); + } + writer.OnEndList(); + + writer.OnEndList(); + } + writer.OnEndList(); + return true; + } + + IDqIntegration* GetDqIntegration() override { + return State_->DqIntegration_.Get(); + } + + IDqOptimization* GetDqOptimization() override { + return DqOptimizer_.Get(); + } + +private: + TExprNode::TPtr InjectUdfRemapperOrView(TYtRead readNode, TExprContext& ctx, bool fromReadSchema, bool buildLeft) { + const bool weakConcat = NYql::HasSetting(readNode.Arg(4).Ref(), EYtSettingType::WeakConcat); + const bool ignoreNonExisting = NYql::HasSetting(readNode.Arg(4).Ref(), EYtSettingType::IgnoreNonExisting); + const bool warnNonExisting = NYql::HasSetting(readNode.Arg(4).Ref(), EYtSettingType::WarnNonExisting); + const bool inlineContent = NYql::HasSetting(readNode.Arg(4).Ref(), EYtSettingType::Inline); + const auto cleanReadSettings = NYql::RemoveSettings(readNode.Arg(4).Ref(), + EYtSettingType::WeakConcat | EYtSettingType::Inline | EYtSettingType::Scheme | EYtSettingType::IgnoreNonExisting | EYtSettingType::WarnNonExisting, ctx); + + const bool useSysColumns = !fromReadSchema && State_->Configuration->UseSystemColumns.Get().GetOrElse(DEFAULT_USE_SYS_COLUMNS); + bool hasSysColumns = false; + + auto cluster = TString{readNode.DataSource().Cluster().Value()}; + + bool hasNonExisting = false; + bool readAllFields = false; + TSet<TString> subsetFields; + TExprNode::TListType tableReads; + for (auto node: readNode.Arg(2).Cast<TExprList>()) { + TYtPath path = node.Cast<TYtPath>(); + const bool hasMultipleUserRanges = path.Ranges().Maybe<TExprList>() && path.Ranges().Cast<TExprList>().Size() > 1; + TYtTable table = path.Table().Cast<TYtTable>(); + if (!path.Columns().Maybe<TCoVoid>() && !readNode.Arg(3).Maybe<TCoVoid>()) { + ctx.AddError(TIssue(ctx.GetPosition(readNode.Pos()), TStringBuilder() + << "Column selection in both read and YPath for table " << table.Name().Value())); + return {}; + } + + auto epoch = TEpochInfo::Parse(table.Epoch().Ref()); + const TYtTableDescription& tableDesc = State_->TablesData->GetTable(cluster, TString{table.Name().Value()}, epoch); + + auto tableMeta = tableDesc.Meta; + auto tableStat = tableDesc.Stat; + + TExprBase origColumnList = path.Columns(); + if (!path.Columns().Maybe<TCoVoid>()) { + for (auto child: path.Columns().Ref().Children()) { + subsetFields.insert(TString{child->Content()}); + } + } + else if (!readNode.Arg(3).Maybe<TCoVoid>()) { + for (auto child: readNode.Arg(3).Ref().Children()) { + subsetFields.insert(TString{child->Content()}); + } + origColumnList = readNode.Arg(3); + YQL_ENSURE(origColumnList.Ref().IsList()); + } + else { + readAllFields = true; + } + + bool skipTable = false; + if (weakConcat + && !tableDesc.View // Don't skip view + && epoch.GetOrElse(0) == 0 + && tableStat && tableStat->IsEmpty() + && tableMeta && !tableMeta->YqlCompatibleScheme) { + // skip empty tables without YQL compatible scheme + skipTable = true; + } + + if (ignoreNonExisting + && epoch.GetOrElse(0) == 0 + && tableMeta && !tableMeta->DoesExist) { + // skip non-existing tables + hasNonExisting = true; + skipTable = true; + if (warnNonExisting) { + auto issue = TIssue(ctx.GetPosition(table.Pos()), TStringBuilder() + << "Table " << cluster << '.' << table.Name().Value() << " doesn't exist"); + SetIssueCode(EYqlIssueCode::TIssuesIds_EIssueCode_YT_WARN_TABLE_DOES_NOT_EXIST, issue); + if (!ctx.AddWarning(issue)) { + return {}; + } + } + } + + if (skipTable) { + auto userSchema = GetSetting(table.Settings().Ref(), EYtSettingType::UserSchema); + if (userSchema) { + tableReads.push_back(ctx.Builder(table.Pos()) + .Callable(buildLeft ? "Left!" : "Right!") + .Add(0, BuildEmptyTablesRead(table.Pos(), *userSchema, ctx)) + .Seal() + .Build()); + } + + continue; + } + + auto updatedSettings = NYql::RemoveSetting(table.Settings().Ref(), EYtSettingType::XLock, ctx); + + TString view; + if (auto viewNode = NYql::GetSetting(*updatedSettings, EYtSettingType::View)) { + view = TString{viewNode->Child(1)->Content()}; + // truncate view + updatedSettings = NYql::RemoveSetting(*updatedSettings, EYtSettingType::View, ctx); + } + + bool withQB = tableDesc.QB2RowSpec && view != "raw"; + if (withQB) { + if (inlineContent) { + ctx.AddError(TIssue(ctx.GetPosition(node.Pos()), TStringBuilder() + << "Table with QB2 premapper cannot be inlined: " << cluster << '.' << table.Name().Value())); + return {}; + } + if (tableDesc.RowSpec && 0 != tableDesc.RowSpec->GetNativeYtTypeFlags()) { + ctx.AddError(TIssue(ctx.GetPosition(node.Pos()), TStringBuilder() + << "Table with type_v3 schema cannot be used with QB2 premapper: " << cluster << '.' << table.Name().Value())); + return {}; + } + updatedSettings = NYql::AddSetting(*updatedSettings, EYtSettingType::WithQB, {}, ctx); + } + + table = Build<TYtTable>(ctx, table.Pos()) + .InitFrom(table) + .Settings(updatedSettings) + .Done(); + + const bool needRewrite = weakConcat + || tableDesc.UdfApplyLambda + || !view.empty() + || withQB; + + path = Build<TYtPath>(ctx, path.Pos()) + .InitFrom(path) + .Table(table) + .Columns(needRewrite ? Build<TCoVoid>(ctx, path.Columns().Pos()).Done() : origColumnList) + .Done(); + + bool tableSysColumns = useSysColumns && !tableDesc.View && (view.empty() || view == "raw"); + if (tableSysColumns) { + hasSysColumns = true; + } + + TExprNode::TPtr effectiveSettings = cleanReadSettings; + if (tableSysColumns) { + effectiveSettings = NYql::AddSettingAsColumnList(*effectiveSettings, EYtSettingType::SysColumns, {TString{"path"}, TString{"record"}}, ctx); + } + + if (hasMultipleUserRanges) { + effectiveSettings = NYql::AddSetting(*effectiveSettings, EYtSettingType::Unordered, nullptr, ctx); + effectiveSettings = NYql::AddSetting(*effectiveSettings, EYtSettingType::NonUnique, nullptr, ctx); + } + + auto origReadNode = Build<TYtReadTable>(ctx, readNode.Pos()) + .World(readNode.World()) + .DataSource(readNode.DataSource()) + .Input() + .Add() + .Paths() + .Add(path) + .Build() + .Settings(effectiveSettings) + .Build() + .Build() + .Done(); + + if (buildLeft) { + TExprNode::TPtr leftOverRead = Build<TCoLeft>(ctx, readNode.Pos()) + .Input(origReadNode) + .Done().Ptr(); + + tableReads.push_back(leftOverRead); + continue; + } + + TExprNode::TPtr rightOverRead = inlineContent + ? Build<TYtTableContent>(ctx, readNode.Pos()) + .Input(origReadNode) + .Settings().Build() + .Done().Ptr() + : Build<TCoRight>(ctx, readNode.Pos()) + .Input(origReadNode) + .Done().Ptr(); + + TExprNode::TPtr newReadNode = rightOverRead; + + if (tableDesc.View) { + auto root = tableDesc.View->CompiledSql; + YQL_ENSURE(root); + if (readNode.World().Ref().Type() != TExprNode::World) { + // Inject original Read! dependencies + auto status = OptimizeExpr(root, root, [&readNode](const TExprNode::TPtr& node, TExprContext& ctx) { + if (node->ChildrenSize() > 0 && node->Child(0)->Type() == TExprNode::World) { + return ctx.ChangeChild(*node, 0, readNode.World().Ptr()); + } + return node; + }, ctx, TOptimizeExprSettings(nullptr)); + + if (status.Level == IGraphTransformer::TStatus::Error) { + return {}; + } + } + + newReadNode = root; + ctx.Step + .Repeat(TExprStep::ExpandApplyForLambdas) + .Repeat(TExprStep::ExprEval) + .Repeat(TExprStep::DiscoveryIO) + .Repeat(TExprStep::Epochs) + .Repeat(TExprStep::Intents) + .Repeat(TExprStep::LoadTablesMetadata) + .Repeat(TExprStep::RewriteIO); + } + + if (tableDesc.UdfApplyLambda) { + if (tableSysColumns) { + newReadNode = ctx.Builder(readNode.Pos()) + .Callable(ctx.IsConstraintEnabled<TSortedConstraintNode>() ? TCoOrderedMap::CallableName() : TCoMap::CallableName()) + .Add(0, newReadNode) + .Lambda(1) + .Param("row") + .Callable("AddMember") + .Callable(0, "AddMember") + .Apply(0, tableDesc.UdfApplyLambda) + .With(0) + .Callable("RemovePrefixMembers") + .Arg(0, "row") + .List(1) + .Atom(0, YqlSysColumnPrefix, TNodeFlags::Default) + .Seal() + .Seal() + .Done() + .Seal() + .Atom(1, YqlSysColumnRecord, TNodeFlags::Default) + .Callable(2, "Member") + .Arg(0, "row") + .Atom(1, YqlSysColumnRecord, TNodeFlags::Default) + .Seal() + .Seal() + .Atom(1, YqlSysColumnPath, TNodeFlags::Default) + .Callable(2, "Member") + .Arg(0, "row") + .Atom(1, YqlSysColumnPath, TNodeFlags::Default) + .Seal() + .Seal() + .Seal() + .Seal() + .Build(); + } else { + newReadNode = ctx.Builder(readNode.Pos()) + .Callable(ctx.IsConstraintEnabled<TSortedConstraintNode>() ? TCoOrderedMap::CallableName() : TCoMap::CallableName()) + .Add(0, newReadNode) + .Add(1, tableDesc.UdfApplyLambda) + .Seal() + .Build(); + } + } + + if (!view.empty()) { + auto viewMeta = tableDesc.Views.FindPtr(view); + YQL_ENSURE(viewMeta, "Unknown view: " << view); + auto root = viewMeta->CompiledSql; + YQL_ENSURE(root, "View is not initialized: " << view); + TOptimizeExprSettings settings(nullptr); + settings.VisitChanges = true; + TExprNode::TListType innerWorlds; + const bool enableViewIsolation = State_->Configuration->ViewIsolation.Get().GetOrElse(false); + auto status = OptimizeExpr(root, root, [newReadNode, rightOverRead, &readNode, enableViewIsolation, &innerWorlds](const TExprNode::TPtr& node, TExprContext& ctx) { + if (auto world = TMaybeNode<TCoLeft>(node).Input().Maybe<TCoRead>().World()) { + return world.Cast().Ptr(); + } + + if (auto maybeRead = TMaybeNode<TCoRight>(node).Input().Maybe<TCoRead>()) { + TCoRead read = maybeRead.Cast(); + TYtInputKeys keys; + if (!keys.Parse(read.Arg(2).Ref(), ctx)) { + return TExprNode::TPtr(); + } + YQL_ENSURE(keys.GetKeys().size() == 1, "Expected single table name"); + auto tableName = keys.GetKeys().front().GetPath(); + TExprNode::TPtr selfRead; + if (tableName == TStringBuf("self")) { + selfRead = newReadNode; + } else if (tableName == TStringBuf("self_raw")) { + selfRead = rightOverRead; + } else { + YQL_ENSURE(false, "Unknown table name (should be self or self_raw): " << tableName); + } + + if (enableViewIsolation) { + if (read.World().Raw()->Type() != TExprNode::World) { + innerWorlds.push_back(read.World().Ptr()); + } + } + + return selfRead; + } + + // Inject original Read! dependencies + if (node->ChildrenSize() > 0 && node->Child(0)->Type() == TExprNode::World) { + return ctx.ChangeChild(*node, 0, readNode.World().Ptr()); + } + + return node; + }, ctx, settings); + + if (status.Level == IGraphTransformer::TStatus::Error) { + return {}; + } + + if (!innerWorlds.empty()) { + innerWorlds.push_back(origReadNode.World().Ptr()); + auto combined = ctx.NewCallable(origReadNode.World().Pos(), "Sync!", std::move(innerWorlds)); + root = ctx.ReplaceNode(std::move(root), *origReadNode.World().Raw(), combined); + } + + newReadNode = root; + ctx.Step + .Repeat(TExprStep::ExpandApplyForLambdas) + .Repeat(TExprStep::ExprEval) + .Repeat(TExprStep::RewriteIO); + } + + if (needRewrite && !weakConcat && !origColumnList.Maybe<TCoVoid>()) { + TSet<TString> names; + for (auto child: origColumnList.Ref().Children()) { + names.emplace(child->Content()); + } + if (tableSysColumns) { + names.insert(TString{YqlSysColumnPath}); + names.insert(TString{YqlSysColumnRecord}); + } + newReadNode = FilterByFields(readNode.Pos(), newReadNode, names, ctx, false); + } + + tableReads.push_back(newReadNode); + } + + if (buildLeft) { + return ctx.NewCallable(readNode.Pos(), TCoSync::CallableName(), std::move(tableReads)); + } + + if (tableReads.empty()) { + if (hasNonExisting) { + ctx.AddError(TIssue(ctx.GetPosition(readNode.Pos()), "The list of tables is empty")); + return {}; + } + + return Build<TCoList>(ctx, readNode.Pos()) + .ListType<TCoListType>() + .ItemType<TCoStructType>() + .Build() + .Build() + .Done().Ptr(); + } + + TStringBuf extendName = ctx.IsConstraintEnabled<TSortedConstraintNode>() + ? (weakConcat ? TCoUnionMerge::CallableName() : TCoMerge::CallableName()) + : (weakConcat ? TCoUnionAll::CallableName() : TCoExtend::CallableName()); + + auto ret = ctx.NewCallable(readNode.Pos(), extendName, std::move(tableReads)); + + if (weakConcat && !readAllFields) { + if (hasSysColumns) { + subsetFields.insert(TString{YqlSysColumnPath}); + subsetFields.insert(TString{YqlSysColumnRecord}); + } + ret = FilterByFields(readNode.Pos(), ret, subsetFields, ctx, false); + } + + return ret; + } + +private: + TYtState::TPtr State_; + TLazyInitHolder<IGraphTransformer> ConfigurationTransformer_; + TLazyInitHolder<IGraphTransformer> IODiscoveryTransformer_; + TLazyInitHolder<IGraphTransformer> EpochTransformer_; + TLazyInitHolder<IGraphTransformer> IntentDeterminationTransformer_; + TLazyInitHolder<IGraphTransformer> LoadMetaDataTransformer_; + TLazyInitHolder<TVisitorTransformerBase> TypeAnnotationTransformer_; + TLazyInitHolder<IGraphTransformer> ConstraintTransformer_; + TLazyInitHolder<TExecTransformerBase> ExecTransformer_; + TLazyInitHolder<TYtDataSourceTrackableNodeProcessor> TrackableNodeProcessor_; + TLazyInitHolder<IDqOptimization> DqOptimizer_; +}; + +TIntrusivePtr<IDataProvider> CreateYtDataSource(TYtState::TPtr state) { + return new TYtDataSource(state); +} + +} diff --git a/yt/yql/providers/yt/provider/yql_yt_datasource_constraints.cpp b/yt/yql/providers/yt/provider/yql_yt_datasource_constraints.cpp new file mode 100644 index 0000000000..3c5d2f4554 --- /dev/null +++ b/yt/yql/providers/yt/provider/yql_yt_datasource_constraints.cpp @@ -0,0 +1,205 @@ +#include "yql_yt_provider_impl.h" +#include "yql_yt_table.h" + +#include <yt/yql/providers/yt/expr_nodes/yql_yt_expr_nodes.h> +#include <yt/yql/providers/yt/lib/row_spec/yql_row_spec.h> +#include <yql/essentials/providers/common/transform/yql_visit.h> +#include <yql/essentials/core/yql_expr_constraint.h> +#include <yql/essentials/ast/yql_constraint.h> + +#include <util/generic/variant.h> + + +namespace NYql { + +using namespace NNodes; + +namespace { + +class TYtDataSourceConstraintTransformer : public TVisitorTransformerBase { +public: + TYtDataSourceConstraintTransformer(TYtState::TPtr state) + : TVisitorTransformerBase(true) + , State_(state) + { + AddHandler({TYtTable::CallableName()}, Hndl(&TYtDataSourceConstraintTransformer::HandleTable)); + AddHandler({TYtPath::CallableName()}, Hndl(&TYtDataSourceConstraintTransformer::HandlePath)); + AddHandler({TYtSection::CallableName()}, Hndl(&TYtDataSourceConstraintTransformer::HandleSection)); + AddHandler({TYtReadTable::CallableName()}, Hndl(&TYtDataSourceConstraintTransformer::HandleReadTable)); + AddHandler({TYtTableContent::CallableName()}, Hndl(&TYtDataSourceConstraintTransformer::HandleTableContent)); + + AddHandler({TYtIsKeySwitch::CallableName()}, Hndl(&TYtDataSourceConstraintTransformer::HandleDefault)); + AddHandler({TYqlRowSpec::CallableName()}, Hndl(&TYtDataSourceConstraintTransformer::HandleDefault)); + AddHandler({TEpoch::CallableName()}, Hndl(&TYtDataSourceConstraintTransformer::HandleDefault)); + AddHandler({TYtMeta::CallableName()}, Hndl(&TYtDataSourceConstraintTransformer::HandleDefault)); + AddHandler({TYtStat::CallableName()}, Hndl(&TYtDataSourceConstraintTransformer::HandleDefault)); + AddHandler({TYtRow::CallableName()}, Hndl(&TYtDataSourceConstraintTransformer::HandleDefault)); + AddHandler({TYtRowRange::CallableName()}, Hndl(&TYtDataSourceConstraintTransformer::HandleDefault)); + AddHandler({TYtKeyExact::CallableName()}, Hndl(&TYtDataSourceConstraintTransformer::HandleDefault)); + AddHandler({TYtKeyRange::CallableName()}, Hndl(&TYtDataSourceConstraintTransformer::HandleDefault)); + AddHandler({TYtReadTableScheme::CallableName()}, Hndl(&TYtDataSourceConstraintTransformer::HandleDefault)); + AddHandler({TYtLength::CallableName()}, Hndl(&TYtDataSourceConstraintTransformer::HandleDefault)); + AddHandler({TCoConfigure::CallableName()}, Hndl(&TYtDataSourceConstraintTransformer::HandleDefault)); + AddHandler({TYtConfigure::CallableName()}, Hndl(&TYtDataSourceConstraintTransformer::HandleDefault)); + AddHandler({TYtTablePath::CallableName()}, Hndl(&TYtDataSourceConstraintTransformer::HandleDefault)); + AddHandler({TYtTableRecord::CallableName()}, Hndl(&TYtDataSourceConstraintTransformer::HandleDefault)); + AddHandler({TYtTableIndex::CallableName()}, Hndl(&TYtDataSourceConstraintTransformer::HandleDefault)); + AddHandler({TYtRowNumber::CallableName()}, Hndl(&TYtDataSourceConstraintTransformer::HandleDefault)); + AddHandler({TYtTableName::CallableName()}, Hndl(&TYtDataSourceConstraintTransformer::HandleDefault)); + } + + TStatus HandleTable(TExprBase input, TExprContext& ctx) { + const auto table = input.Cast<TYtTable>(); + const auto epoch = TEpochInfo::Parse(table.Epoch().Ref()); + const auto tableName = TString{TYtTableInfo::GetTableLabel(table)}; + TYtTableDescription& tableDesc = State_->TablesData->GetModifTable(TString{table.Cluster().Value()}, tableName, epoch); + if (epoch) { + if (!tableDesc.ConstraintsReady) { + if (State_->Types->EvaluationInProgress) { + ctx.AddError(TIssue(ctx.GetPosition(input.Pos()), TStringBuilder() + << "Table " << tableName.Quote() << " is used before commit")); + return TStatus::Error; + } + return TStatus(TStatus::Repeat, true); + } + input.Ptr()->SetConstraints(tableDesc.Constraints); + } else if (!table.RowSpec().Maybe<TCoVoid>()) { + TYqlRowSpecInfo rowSpec(table.RowSpec(), false); + auto set = rowSpec.GetSomeConstraints(State_->Configuration->ApplyStoredConstraints.Get().GetOrElse(DEFAULT_APPLY_STORED_CONSTRAINTS), ctx); + + if (!set.GetConstraint<TSortedConstraintNode>()) { + if (const auto sorted = rowSpec.MakeSortConstraint(ctx)) + set.AddConstraint(sorted); + } + + if (!set.GetConstraint<TDistinctConstraintNode>()) { + if (const auto distinct = rowSpec.MakeDistinctConstraint(ctx)) { + set.AddConstraint(ctx.MakeConstraint<TUniqueConstraintNode>(TUniqueConstraintNode::TContentType(distinct->GetContent()))); + set.AddConstraint(distinct); + } + } + input.Ptr()->SetConstraints(set); + if (!tableDesc.ConstraintsReady) { + tableDesc.Constraints = set; + tableDesc.SetConstraintsReady(); + } + } + if (!table.Stat().Maybe<TCoVoid>()) { + if (TYtTableStatInfo(table.Stat()).IsEmpty() && !TYtTableMetaInfo(table.Meta()).IsDynamic) { + input.Ptr()->AddConstraint(ctx.MakeConstraint<TEmptyConstraintNode>()); + } + } + return TStatus::Ok; + } + + TStatus HandlePath(TExprBase input, TExprContext& ctx) { + auto path = input.Cast<TYtPath>(); + const auto outItemType = path.Ref().GetTypeAnn()->Cast<TListExprType>()->GetItemType()->Cast<TStructExprType>(); + const auto filter = [outItemType](const TPartOfConstraintBase::TPathType& path) { return !path.empty() && outItemType->FindItem(path.front()); }; + + if (const auto sort = path.Table().Ref().GetConstraint<TSortedConstraintNode>()) { + if (const auto filtered = sort->FilterFields(ctx, filter)) { + path.Ptr()->AddConstraint(filtered); + } + } + + if (const auto uniq = path.Table().Ref().GetConstraint<TUniqueConstraintNode>()) { + if (const auto filtered = uniq->FilterFields(ctx, filter)) { + path.Ptr()->AddConstraint(filtered); + } + } + + if (const auto dist = path.Table().Ref().GetConstraint<TDistinctConstraintNode>()) { + if (const auto filtered = dist->FilterFields(ctx, filter)) { + path.Ptr()->AddConstraint(filtered); + } + } + + if (auto empty = path.Table().Ref().GetConstraint<TEmptyConstraintNode>()) { + path.Ptr()->AddConstraint(empty); + } else if (path.Ranges().Maybe<TExprList>()) { + auto rangeInfo = TYtRangesInfo(path.Ranges()); + if (rangeInfo.IsEmpty()) { + path.Ptr()->AddConstraint(ctx.MakeConstraint<TEmptyConstraintNode>()); + } + } + + return TStatus::Ok; + } + + TStatus HandleSection(TExprBase input, TExprContext& ctx) { + auto section = input.Cast<TYtSection>(); + if (section.Paths().Size() == 1) { + auto path = section.Paths().Item(0); + if (!NYql::HasSetting(section.Settings().Ref(), EYtSettingType::Unordered)) { + if (auto sorted = path.Ref().GetConstraint<TSortedConstraintNode>()) { + input.Ptr()->AddConstraint(sorted); + } + } + if (!NYql::HasSetting(section.Settings().Ref(), EYtSettingType::NonUnique)) { + if (const auto unique = path.Ref().GetConstraint<TUniqueConstraintNode>()) { + input.Ptr()->AddConstraint(unique); + } + if (const auto distinct = path.Ref().GetConstraint<TDistinctConstraintNode>()) { + input.Ptr()->AddConstraint(distinct); + } + } + } + + TVector<const TConstraintSet*> allConstraints; + for (const auto& path : section.Paths()) { + allConstraints.push_back(&path.Ref().GetConstraintSet()); + } + + if (auto empty = TEmptyConstraintNode::MakeCommon(allConstraints, ctx)) { + input.Ptr()->AddConstraint(empty); + } + + return TStatus::Ok; + } + + TStatus HandleReadTable(TExprBase input, TExprContext& ctx) { + auto read = input.Cast<TYtReadTable>(); + if (read.Input().Size() == 1) { + auto section = read.Input().Item(0); + input.Ptr()->CopyConstraints(section.Ref()); + } else { + TMultiConstraintNode::TMapType multiItems; + bool allEmpty = true; + for (ui32 index = 0; index < read.Input().Size(); ++index) { + auto section = read.Input().Item(index); + if (!section.Ref().GetConstraint<TEmptyConstraintNode>()) { + multiItems.push_back(std::make_pair(index, section.Ref().GetConstraintSet())); + allEmpty = false; + } + } + if (!multiItems.empty()) { + input.Ptr()->AddConstraint(ctx.MakeConstraint<TMultiConstraintNode>(std::move(multiItems))); + } else if (allEmpty) { + input.Ptr()->AddConstraint(ctx.MakeConstraint<TEmptyConstraintNode>()); + } + } + return TStatus::Ok; + } + + TStatus HandleTableContent(TExprBase input, TExprContext& /*ctx*/) { + TYtTableContent tableContent = input.Cast<TYtTableContent>(); + input.Ptr()->CopyConstraints(tableContent.Input().Ref()); + return TStatus::Ok; + } + + TStatus HandleDefault(TExprBase input, TExprContext& /*ctx*/) { + return UpdateAllChildLambdasConstraints(input.Ref()); + } + +private: + const TYtState::TPtr State_; +}; + +} + +THolder<IGraphTransformer> CreateYtDataSourceConstraintTransformer(TYtState::TPtr state) { + return THolder(new TYtDataSourceConstraintTransformer(state)); +} + +} diff --git a/yt/yql/providers/yt/provider/yql_yt_datasource_exec.cpp b/yt/yql/providers/yt/provider/yql_yt_datasource_exec.cpp new file mode 100644 index 0000000000..ae79dfb091 --- /dev/null +++ b/yt/yql/providers/yt/provider/yql_yt_datasource_exec.cpp @@ -0,0 +1,245 @@ +#include "yql_yt_provider_impl.h" +#include "yql_yt_helpers.h" +#include "yql_yt_optimize.h" +#include "yql_yt_op_hash.h" + +#include <yt/yql/providers/yt/expr_nodes/yql_yt_expr_nodes.h> +#include <yt/yql/providers/yt/provider/yql_yt_helpers.h> +#include <yt/yql/providers/yt/lib/hash/yql_hash_builder.h> +#include <yql/essentials/providers/result/expr_nodes/yql_res_expr_nodes.h> +#include <yql/essentials/providers/common/provider/yql_provider.h> +#include <yql/essentials/providers/common/transform/yql_exec.h> +#include <yql/essentials/providers/common/schema/expr/yql_expr_schema.h> +#include <yql/essentials/core/expr_nodes/yql_expr_nodes.h> +#include <yql/essentials/core/type_ann/type_ann_expr.h> +#include <yql/essentials/core/yql_execution.h> +#include <yql/essentials/core/yql_graph_transformer.h> +#include <yql/essentials/core/yql_expr_optimize.h> +#include <yql/essentials/utils/log/log.h> + +#include <util/string/hex.h> + +namespace NYql { + +using namespace NNodes; + +namespace { + +class TYtDataSourceExecTransformer : public TExecTransformerBase { +public: + TYtDataSourceExecTransformer(TYtState::TPtr state) + : State_(state) + { + + AddHandler({TStringBuf("Result"), TStringBuf("Pull")}, RequireNone(), Hndl(&TYtDataSourceExecTransformer::HandleResOrPull)); + AddHandler( + { + TYtReadTableScheme::CallableName(), + TYtPath::CallableName(), + TYtSection::CallableName(), + }, + RequireFirst(), + Pass()); + AddHandler({TYtReadTable::CallableName()}, RequireSequenceOf({TYtReadTable::idx_World, TYtReadTable::idx_Input}), Pass()); + AddHandler({TYtTable::CallableName()}, RequireNone(), Pass()); + AddHandler({TYtConfigure::CallableName()}, RequireFirst(), Hndl(&TYtDataSourceExecTransformer::HandleConfigure)); + } + +protected: + TString WriteTableScheme(TYtReadTableScheme readScheme, NYson::EYsonFormat ysonFormat, bool withType) { + TStringStream out; + NYson::TYsonWriter writer(&out, ysonFormat, ::NYson::EYsonType::Node, true); + writer.OnBeginMap(); + + if (withType) { + writer.OnKeyedItem("Type"); + auto valueType = readScheme.Ref().GetTypeAnn()->Cast<TTupleExprType>()->GetItems()[1]; + NCommon::WriteTypeToYson(writer, valueType); + } + + auto cluster = TString{readScheme.DataSource().Cluster().Value()}; + auto tableName = TString{readScheme.Table().Name().Value()}; + auto view = NYql::GetSetting(readScheme.Table().Settings().Ref(), EYtSettingType::View); + TString viewName = view ? TString{view->Child(1)->Content()} : TString(); + + const TYtTableDescription& tableDesc = State_->TablesData->GetTable(cluster, tableName, + TEpochInfo::Parse(readScheme.Table().Epoch().Ref())); + + writer.OnKeyedItem("Data"); + tableDesc.ToYson(writer, cluster, tableName, viewName); + + writer.OnEndMap(); + return out.Str(); + } + +protected: + TStatusCallbackPair HandleResOrPull(const TExprNode::TPtr& input, TExprContext& ctx) { + YQL_CLOG(DEBUG, ProviderYt) << "Executing " << input->Content() << " (UniqueId=" << input->UniqueId() << ")"; + TResOrPullBase resOrPull(input); + + IDataProvider::TFillSettings fillSettings = NCommon::GetFillSettings(resOrPull.Ref()); + YQL_ENSURE(fillSettings.Format == IDataProvider::EResultFormat::Yson || fillSettings.Format == IDataProvider::EResultFormat::Skiff); + + auto data = resOrPull.Input(); + if (auto maybePull = resOrPull.Maybe<TPull>()) { + TExprNode::TListType needCalc = GetNodesToCalculate(data.Ptr()); + if (!needCalc.empty()) { + return CalculateNodes(State_, data.Ptr(), TString{GetClusterName(data)}, needCalc, ctx); + } + + if (auto maybeScheme = data.Maybe<TCoRight>().Input().Maybe<TYtReadTableScheme>()) { + TString result = WriteTableScheme(maybeScheme.Cast(), NCommon::GetYsonFormat(fillSettings), + NCommon::HasResOrPullOption(resOrPull.Ref(), "type")); + input->SetState(TExprNode::EState::ExecutionComplete); + input->SetResult(ctx.NewAtom(input->Pos(), result)); + return SyncOk(); + } + + if (auto read = data.Maybe<TCoRight>().Input().Maybe<TYtReadTable>()) { + auto newRead = ctx.ExactShallowCopy(read.Cast().Ref()); + newRead->ChildRef(0) = ctx.NewWorld(read.Cast().World().Pos()); + newRead->Child(0)->SetTypeAnn(ctx.MakeType<TWorldExprType>()); + newRead->Child(0)->SetState(TExprNode::EState::ExecutionComplete); + newRead->CopyConstraints(read.Cast().Ref()); + + auto newRight = Build<TCoRight>(ctx, data.Pos()) + .Input(newRead) + .Done(); + newRight.Ptr()->SetTypeAnn(data.Ref().GetTypeAnn()); + newRight.Ptr()->CopyConstraints(data.Ref()); + + data = newRight; + } + } + + TString usedCluster; + TSyncMap syncList; + if (!IsYtIsolatedLambda(data.Ref(), syncList, usedCluster, false)) { + ctx.AddError(TIssue(ctx.GetPosition(data.Pos()), TStringBuilder() << "Failed to execute node due to bad graph: " << input->Content())); + return SyncError(); + } + if (usedCluster.empty()) { + usedCluster = State_->Configuration->DefaultCluster.Get().GetOrElse(State_->Gateway->GetDefaultClusterName()); + } + + TExprNode::TPtr optimizedInput = data.Ptr(); + if (const auto status = SubstTables(optimizedInput, State_, false, ctx); status.Level == TStatus::Error) { + return SyncStatus(status); + } + + const auto settings = State_->Configuration->GetSettingsForNode(*input); + TUserDataTable crutches = State_->Types->UserDataStorageCrutches; + if (const auto& defaultGeobase = settings->GeobaseDownloadUrl.Get(usedCluster)) { + auto& userDataBlock = (crutches[TUserDataKey::File(TStringBuf("/home/geodata6.bin"))] = TUserDataBlock{EUserDataType::URL, {}, *defaultGeobase, {}, {}}); + userDataBlock.Usage.Set(EUserDataBlockUsage::Path); + } + + bool hasNonDeterministicFunctions = false; + if (const auto status = PeepHoleOptimizeBeforeExec(optimizedInput, optimizedInput, State_, hasNonDeterministicFunctions, ctx, true); status.Level != IGraphTransformer::TStatus::Ok) { + return SyncStatus(status); + } + + TUserDataTable files; + auto filesRes = NCommon::FreezeUsedFiles(*optimizedInput, files, *State_->Types, ctx, MakeUserFilesDownloadFilter(*State_->Gateway, usedCluster), crutches); + if (filesRes.first.Level != TStatus::Ok) { + return filesRes; + } + + THashMap<TString, TString> secureParams; + NCommon::FillSecureParams(optimizedInput, *State_->Types, secureParams); + + auto optimizeChildren = input->ChildrenList(); + optimizeChildren[0] = optimizedInput; + resOrPull = TResOrPullBase(ctx.ExactChangeChildren(resOrPull.Ref(), std::move(optimizeChildren))); + + TString operationHash; + if (resOrPull.Maybe<TResult>()) { + const auto queryCacheMode = settings->QueryCacheMode.Get().GetOrElse(EQueryCacheMode::Disable); + if (queryCacheMode != EQueryCacheMode::Disable) { + if (!hasNonDeterministicFunctions && settings->QueryCacheUseForCalc.Get().GetOrElse(true)) { + operationHash = TYtNodeHashCalculator(State_, usedCluster, settings).GetHash(*optimizedInput); + if (!operationHash.empty()) { + // Update hash with columns hint. See YQL-10405 + TVector<TString> columns = NCommon::GetResOrPullColumnHints(resOrPull.Ref()); + THashBuilder builder; + builder << TYtNodeHashCalculator::MakeSalt(settings, usedCluster) << operationHash << columns.size(); + for (auto& col: columns) { + builder << col; + } + operationHash = builder.Finish(); + } + } + YQL_CLOG(DEBUG, ProviderYt) << "Operation hash: " << HexEncode(operationHash).Quote() + << ", cache mode: " << queryCacheMode; + } + } + + auto publicId = resOrPull.PublicId().Value() + ? MakeMaybe(FromString<ui32>(resOrPull.PublicId().Value())) + : Nothing(); + + auto future = State_->Gateway->ResOrPull(resOrPull.Ptr(), ctx, + IYtGateway::TResOrPullOptions(State_->SessionId) + .FillSettings(fillSettings) + .UserDataBlocks(files) + .UdfModules(State_->Types->UdfModules) + .UdfResolver(State_->Types->UdfResolver) + .UdfValidateMode(State_->Types->ValidateMode) + .PublicId(publicId) + .Config(State_->Configuration->GetSettingsForNode(resOrPull.Origin().Ref())) + .UsedCluster(usedCluster) + .OptLLVM(State_->Types->OptLLVM.GetOrElse(TString())) + .OperationHash(operationHash) + .SecureParams(secureParams) + ); + + return WrapFuture(future, [](const IYtGateway::TResOrPullResult& res, const TExprNode::TPtr& input, TExprContext& ctx) { + auto ret = ctx.NewAtom(input->Pos(), res.Data); + return ret; + }); + } + + TStatusCallbackPair HandleConfigure(const TExprNode::TPtr& input, TExprContext& ctx) { + YQL_CLOG(DEBUG, ProviderYt) << "Executing " << input->Content() << " (UniqueId=" << input->UniqueId() << ")"; + auto configure = TYtConfigure(input); + auto clusterName = TString{configure.DataSource().Cluster().Value()}; + State_->Configuration->FreezeZeroVersion(); + if (configure.Arg(2).Cast<TCoAtom>().Value() == TStringBuf("Attr")) { + auto name = TString{configure.Arg(3).Cast<TCoAtom>().Value()}; + TMaybe<TString> value; + if (configure.Args().Count() == 5) { + value = TString{configure.Arg(4).Cast<TCoAtom>().Value()}; + } + if (State_->Configuration->IsRuntime(name)) { + if (!State_->Configuration->Dispatch(clusterName, name, value, NCommon::TSettingDispatcher::EStage::RUNTIME, NCommon::TSettingDispatcher::GetErrorCallback(input->Pos(), ctx))) { + return SyncError(); + } + State_->Configuration->PromoteVersion(*input); + YQL_CLOG(DEBUG, ProviderYt) << "Setting pragma " + << (NCommon::ALL_CLUSTERS == clusterName ? YtProviderName : clusterName) << '.' + << name << '=' << (value ? value->Quote() : "(default)") + << " in ver." << State_->Configuration->GetLastVersion(); + } + } + + input->SetState(TExprNode::EState::ExecutionComplete); + input->SetResult(ctx.NewWorld(input->Pos())); + return SyncOk(); + } + + void Rewind() final { + State_->Configuration->ClearVersions(); + TExecTransformerBase::Rewind(); + } + +private: + const TYtState::TPtr State_; +}; + +} + +THolder<TExecTransformerBase> CreateYtDataSourceExecTransformer(TYtState::TPtr state) { + return THolder(new TYtDataSourceExecTransformer(state)); +} + +} diff --git a/yt/yql/providers/yt/provider/yql_yt_datasource_type_ann.cpp b/yt/yql/providers/yt/provider/yql_yt_datasource_type_ann.cpp new file mode 100644 index 0000000000..4f3ffa5444 --- /dev/null +++ b/yt/yql/providers/yt/provider/yql_yt_datasource_type_ann.cpp @@ -0,0 +1,1028 @@ +#include "yql_yt_provider_impl.h" +#include "yql_yt_op_settings.h" +#include "yql_yt_helpers.h" + +#include <yt/yql/providers/yt/expr_nodes/yql_yt_expr_nodes.h> +#include <yt/yql/providers/yt/provider/yql_yt_table.h> +#include <yt/yql/providers/yt/common/yql_names.h> +#include <yt/yql/providers/yt/common/yql_configuration.h> +#include <yt/yql/providers/yt/lib/mkql_helpers/mkql_helpers.h> +#include <yql/essentials/providers/common/provider/yql_provider_names.h> +#include <yql/essentials/providers/common/provider/yql_provider.h> +#include <yql/essentials/providers/common/transform/yql_visit.h> +#include <yql/essentials/core/yql_expr_type_annotation.h> +#include <yql/essentials/core/yql_opt_utils.h> +#include <yql/essentials/core/yql_type_helpers.h> +#include <yql/essentials/utils/yql_panic.h> + +#include <util/string/builder.h> +#include <util/generic/xrange.h> +#include <util/generic/hash.h> +#include <util/generic/string.h> + +namespace NYql { + +using namespace NNodes; + +namespace { + +class TYtDataSourceTypeAnnotationTransformer : public TVisitorTransformerBase { +public: + TYtDataSourceTypeAnnotationTransformer(TYtState::TPtr state) + : TVisitorTransformerBase(true) + , State_(state) + { + AddHandler({TEpoch::CallableName()}, Hndl(&TYtDataSourceTypeAnnotationTransformer::HandleAux<TEpochInfo>)); + AddHandler({TYtMeta::CallableName()}, Hndl(&TYtDataSourceTypeAnnotationTransformer::HandleAux<TYtTableMetaInfo>)); + AddHandler({TYtStat::CallableName()}, Hndl(&TYtDataSourceTypeAnnotationTransformer::HandleAux<TYtTableStatInfo>)); + AddHandler({TYqlRowSpec::CallableName()}, Hndl(&TYtDataSourceTypeAnnotationTransformer::HandleRowSpec)); + AddHandler({TYtTable::CallableName()}, Hndl(&TYtDataSourceTypeAnnotationTransformer::HandleTable)); + AddHandler({TYtRow::CallableName()}, Hndl(&TYtDataSourceTypeAnnotationTransformer::HandleRow)); + AddHandler({TYtRowRange::CallableName()}, Hndl(&TYtDataSourceTypeAnnotationTransformer::HandleRowRange)); + AddHandler({TYtKeyExact::CallableName()}, Hndl(&TYtDataSourceTypeAnnotationTransformer::HandleKeyExact)); + AddHandler({TYtKeyRange::CallableName()}, Hndl(&TYtDataSourceTypeAnnotationTransformer::HandleKeyRange)); + AddHandler({TYtPath::CallableName()}, Hndl(&TYtDataSourceTypeAnnotationTransformer::HandlePath)); + AddHandler({TYtSection::CallableName()}, Hndl(&TYtDataSourceTypeAnnotationTransformer::HandleSection)); + AddHandler({TYtReadTable::CallableName()}, Hndl(&TYtDataSourceTypeAnnotationTransformer::HandleReadTable)); + AddHandler({TYtReadTableScheme::CallableName()}, Hndl(&TYtDataSourceTypeAnnotationTransformer::HandleReadTableScheme)); + AddHandler({TYtTableContent::CallableName()}, Hndl(&TYtDataSourceTypeAnnotationTransformer::HandleTableContent)); + AddHandler({TYtLength::CallableName()}, Hndl(&TYtDataSourceTypeAnnotationTransformer::HandleLength)); + AddHandler({TCoConfigure::CallableName()}, Hndl(&TYtDataSourceTypeAnnotationTransformer::HandleConfigure)); + AddHandler({TYtConfigure::CallableName()}, Hndl(&TYtDataSourceTypeAnnotationTransformer::HandleYtConfigure)); + AddHandler({TYtTablePath::CallableName()}, Hndl(&TYtDataSourceTypeAnnotationTransformer::HandleTablePath)); + AddHandler({TYtTableRecord::CallableName()}, Hndl(&TYtDataSourceTypeAnnotationTransformer::HandleTableRecord)); + AddHandler({TYtRowNumber::CallableName()}, Hndl(&TYtDataSourceTypeAnnotationTransformer::HandleTableRecord)); + AddHandler({TYtTableIndex::CallableName()}, Hndl(&TYtDataSourceTypeAnnotationTransformer::HandleTableIndex)); + AddHandler({TYtIsKeySwitch::CallableName()}, Hndl(&TYtDataSourceTypeAnnotationTransformer::HandleIsKeySwitch)); + AddHandler({TYtTableName::CallableName()}, Hndl(&TYtDataSourceTypeAnnotationTransformer::HandleTableName)); + } + + TStatus HandleUnit(TExprBase input, TExprContext& ctx) { + input.Ptr()->SetTypeAnn(ctx.MakeType<TUnitExprType>()); + return TStatus::Ok; + } + + template <class TValidator> + TStatus HandleAux(TExprBase input, TExprContext& ctx) { + if (!TValidator::Validate(input.Ref(), ctx)) { + return TStatus::Error; + } + input.Ptr()->SetTypeAnn(ctx.MakeType<TUnitExprType>()); + return TStatus::Ok; + } + + TStatus HandleRowSpec(TExprBase input, TExprContext& ctx) { + const TStructExprType* type = nullptr; + TMaybe<TColumnOrder> columnOrder; + if (!TYqlRowSpecInfo::Validate(input.Ref(), ctx, type, columnOrder)) { + return TStatus::Error; + } + input.Ptr()->SetTypeAnn(type); + if (columnOrder) { + return State_->Types->SetColumnOrder(input.Ref(), *columnOrder, ctx); + } + return TStatus::Ok; + } + + TStatus HandleTable(const TExprNode::TPtr& input, TExprNode::TPtr& output, TExprContext& ctx) { + EYtSettingTypes accepted = EYtSettingType::InferScheme | EYtSettingType::ForceInferScheme + | EYtSettingType::DoNotFailOnInvalidSchema | EYtSettingType::Anonymous + | EYtSettingType::WithQB | EYtSettingType::View | EYtSettingType::UserSchema + | EYtSettingType::UserColumns | EYtSettingType::IgnoreTypeV3; + + if (!TYtTableInfo::Validate(*input, accepted, ctx)) { + return TStatus::Error; + } + + if (!TYtTableInfo::HasSubstAnonymousLabel(TExprBase(input))) { + auto status = UpdateTableMeta(input, output, State_->TablesData, false, State_->Types->UseTableMetaFromGraph, ctx); + if (status.Level != TStatus::Ok) { + return status; + } + } + + auto table = TYtTable(input); + + const TTypeAnnotationNode* itemType = nullptr; + TMaybe<TColumnOrder> columnOrder; + if (table.RowSpec().Maybe<TCoVoid>()) { + columnOrder.ConstructInPlace(); + TVector<const TItemExprType*> items; + for (auto& name : YAMR_FIELDS) { + items.push_back(ctx.MakeType<TItemExprType>(name, ctx.MakeType<TDataExprType>(EDataSlot::String))); + columnOrder->AddColumn(TString(name)); + } + itemType = ctx.MakeType<TStructExprType>(items); + } else { + itemType = table.RowSpec().Ref().GetTypeAnn(); + columnOrder = State_->Types->LookupColumnOrder(table.RowSpec().Ref()); + } + + input->SetTypeAnn(ctx.MakeType<TListExprType>(itemType)); + if (columnOrder) { + return State_->Types->SetColumnOrder(*input, *columnOrder, ctx); + } + return TStatus::Ok; + } + + TStatus HandleRow(TExprBase input, TExprContext& ctx) { + if (!EnsureArgsCount(input.Ref(), 1, ctx)) { + return TStatus::Error; + } + + if (!input.Ref().Child(TYtRow::idx_Index)->IsCallable(TCoUint64::CallableName())) { + ctx.AddError(TIssue(ctx.GetPosition(input.Ref().Child(TYtRow::idx_Index)->Pos()), TStringBuilder() << "Expected " << TCoUint64::CallableName())); + return TStatus::Error; + } + + return HandleUnit(input, ctx); + } + + TStatus HandleRowRange(TExprBase input, TExprContext& ctx) { + if (!EnsureArgsCount(input.Ref(), 2, ctx)) { + return TStatus::Error; + } + + if (!input.Ref().Child(TYtRowRange::idx_Lower)->IsCallable(TCoVoid::CallableName()) + && !input.Ref().Child(TYtRowRange::idx_Lower)->IsCallable(TCoUint64::CallableName())) + { + ctx.AddError(TIssue(ctx.GetPosition(input.Ref().Child(TYtRowRange::idx_Lower)->Pos()), TStringBuilder() + << "Expected " << TCoUint64::CallableName() << " or " << TCoVoid::CallableName())); + return TStatus::Error; + } + + if (!input.Ref().Child(TYtRowRange::idx_Upper)->IsCallable(TCoVoid::CallableName()) + && !input.Ref().Child(TYtRowRange::idx_Upper)->IsCallable(TCoUint64::CallableName())) + { + ctx.AddError(TIssue(ctx.GetPosition(input.Ref().Child(TYtRowRange::idx_Upper)->Pos()), TStringBuilder() + << "Expected " << TCoUint64::CallableName() << " or " << TCoVoid::CallableName())); + return TStatus::Error; + } + + if (input.Ref().Child(TYtRowRange::idx_Lower)->IsCallable(TCoVoid::CallableName()) + && input.Ref().Child(TYtRowRange::idx_Upper)->IsCallable(TCoVoid::CallableName())) { + ctx.AddError(TIssue(ctx.GetPosition(input.Pos()), TStringBuilder() + << "Either Lower or Upper part of " << TYtRowRange::CallableName() << " should be specified")); + return TStatus::Error; + } + + return HandleUnit(input, ctx); + } + + static bool ValidateKeyLiteral(TExprBase node, TExprContext& ctx) { + if (node.Maybe<TCoNull>()) { + return true; + } + + bool isOptional; + const TDataExprType* dataType; + if (!EnsureDataOrOptionalOfData(node.Ref(), isOptional, dataType, ctx)) { + return false; + } + + if (isOptional) { + if (node.Maybe<TCoNothing>()) { + return true; + } + auto maybeJust = node.Maybe<TCoJust>(); + if (!maybeJust) { + ctx.AddError(TIssue(ctx.GetPosition(node.Pos()), TStringBuilder() << "Expected Optional literal - Nothing or Just")); + return false; + } + node = maybeJust.Cast().Input(); + } + + if (!node.Maybe<TCoDataCtor>()) { + ctx.AddError(TIssue(ctx.GetPosition(node.Pos()), TStringBuilder() << "Expected one of Data value")); + return false; + } + return true; + } + + TStatus HandleKeyExact(TExprBase input, TExprContext& ctx) { + if (!EnsureArgsCount(input.Ref(), 1, ctx)) { + return TStatus::Error; + } + + if (!EnsureTupleMinSize(*input.Ref().Child(TYtKeyExact::idx_Key), 1, ctx)) { + return TStatus::Error; + } + + TExprNodeList keyValues = input.Ref().Child(TYtKeyExact::idx_Key)->ChildrenList(); + if (AnyOf(keyValues, [&ctx](const auto& child) { return !ValidateKeyLiteral(TExprBase(child), ctx); })) { + return TStatus::Error; + } + + return HandleUnit(input, ctx); + } + + TStatus HandleKeyRange(TExprBase input, TExprContext& ctx) { + if (!EnsureMinArgsCount(input.Ref(), 2, ctx)) { + return TStatus::Error; + } + + if (!EnsureMaxArgsCount(input.Ref(), 3, ctx)) { + return TStatus::Error; + } + + if (!EnsureTuple(*input.Ref().Child(TYtKeyRange::idx_Lower), ctx)) { + return TStatus::Error; + } + + const TExprNodeList& lowerValues = input.Ref().Child(TYtKeyRange::idx_Lower)->ChildrenList(); + if (AnyOf(lowerValues, [&ctx](const auto& child) { return !ValidateKeyLiteral(TExprBase(child), ctx); })) { + return TStatus::Error; + } + + if (!EnsureTuple(*input.Ref().Child(TYtKeyRange::idx_Upper), ctx)) { + return TStatus::Error; + } + + const TExprNodeList& upperValues = input.Ref().Child(TYtKeyRange::idx_Upper)->ChildrenList(); + if (AnyOf(upperValues, [&ctx](const auto& child) { return !ValidateKeyLiteral(TExprBase(child), ctx); })) { + return TStatus::Error; + } + + if (lowerValues.empty() && upperValues.empty()) { + ctx.AddError(TIssue(ctx.GetPosition(input.Pos()), TStringBuilder() + << "Either Lower or Upper part of " << TYtKeyRange::CallableName() << " should be specified")); + return TStatus::Error; + } + + for (size_t i : xrange(std::min(lowerValues.size(), upperValues.size()))) { + const TTypeAnnotationNode* lowerType = lowerValues[i]->GetTypeAnn(); + const TTypeAnnotationNode* upperType = upperValues[i]->GetTypeAnn(); + if (lowerType->GetKind() == ETypeAnnotationKind::Null || upperType->GetKind() == ETypeAnnotationKind::Null) { + continue; + } + + if (!IsSameAnnotation(*lowerType, *upperType)) { + ctx.AddError(TIssue(ctx.GetPosition(input.Pos()), TStringBuilder() + << "Lower/Upper type mismatch at index " << i << ": lower: " << *lowerType << ", upper: " << *upperType)); + return TStatus::Error; + } + } + + if (input.Ref().ChildrenSize() == 3) { + if (!EnsureTupleMinSize(*input.Ref().Child(TYtKeyRange::idx_Flags), 1, ctx)) { + return TStatus::Error; + } + for (auto& atom: input.Ref().Child(TYtKeyRange::idx_Flags)->Children()) { + if (!EnsureAtom(*atom, ctx)) { + return TStatus::Error; + } + if (atom->Content() == TStringBuf("excludeLower")) { + if (lowerValues.empty()) { + ctx.AddError(TIssue(ctx.GetPosition(atom->Pos()), TStringBuilder() + << "Expected non-empty Lower part for 'excludeLower' setting")); + return TStatus::Error; + } + } else if (atom->Content() == TStringBuf("includeUpper")) { + if (upperValues.empty()) { + ctx.AddError(TIssue(ctx.GetPosition(atom->Pos()), TStringBuilder() + << "Expected non-empty Upper part for 'includeUpper' setting")); + return TStatus::Error; + } + } else if (atom->Content() != TStringBuf("useKeyBound")) { + ctx.AddError(TIssue(ctx.GetPosition(atom->Pos()), TStringBuilder() + << "Expected 'excludeLower', 'includeUpper' or 'useKeyBound', but got " << atom->Content())); + return TStatus::Error; + } + } + } + + return HandleUnit(input, ctx); + } + + TStatus HandlePath(TExprBase input, TExprContext& ctx) { + if (!TYtPathInfo::Validate(input.Ref(), ctx)) { + return TStatus::Error; + } + + TYtPathInfo pathInfo(input); + if (pathInfo.Table->Meta && !pathInfo.Table->Meta->DoesExist) { + if (NYql::HasSetting(pathInfo.Table->Settings.Ref(), EYtSettingType::Anonymous)) { + ctx.AddError(TIssue(ctx.GetPosition(input.Pos()), TStringBuilder() + << "Anonymous table " << pathInfo.Table->Name.Quote() << " must be materialized. Use COMMIT before reading from it.")); + } + else { + ctx.AddError(TIssue(ctx.GetPosition(input.Pos()), TStringBuilder() + << "Table " << pathInfo.Table->Name.Quote() << " does not exist").SetCode(TIssuesIds::YT_TABLE_NOT_FOUND, TSeverityIds::S_ERROR)); + } + return IGraphTransformer::TStatus::Error; + } + + const TTypeAnnotationNode* itemType = nullptr; + TExprNode::TPtr newFields; + auto status = pathInfo.GetType(itemType, newFields, ctx, input.Pos()); + if (newFields) { + input.Ptr()->ChildRef(TYtPath::idx_Columns) = newFields; + } + if (status.Level != TStatus::Ok) { + return status; + } + + input.Ptr()->SetTypeAnn(ctx.MakeType<TListExprType>(itemType)); + if (auto columnOrder = State_->Types->LookupColumnOrder(input.Ref().Head())) { + if (pathInfo.Columns) { + auto& renames = pathInfo.Columns->GetRenames(); + if (renames) { + TColumnOrder renamedOrder; + for (auto& [col, gen_col] : *columnOrder) { + if (auto renamed = renames->FindPtr(col)) { + renamedOrder.AddColumn(*renamed); + } else { + renamedOrder.AddColumn(col); + } + } + *columnOrder = renamedOrder; + } + } + + // sync with output type (add weak columns, etc.) + TSet<TStringBuf> allColumns = GetColumnsOfStructOrSequenceOfStruct(*itemType); + columnOrder->EraseIf([&](const TString& col) { return !allColumns.contains(col); }); + for (auto& [col, gen_col] : *columnOrder) { + allColumns.erase(allColumns.find(gen_col)); + } + for (auto& col : allColumns) { + columnOrder->AddColumn(TString(col)); + } + + return State_->Types->SetColumnOrder(input.Ref(), *columnOrder, ctx); + } + + return TStatus::Ok; + } + + TStatus HandleSection(TExprBase input, TExprContext& ctx) { + if (!EnsureArgsCount(input.Ref(), 2, ctx)) { + return TStatus::Error; + } + if (!EnsureTupleMinSize(*input.Ref().Child(TYtSection::idx_Paths), 1, ctx)) { + return TStatus::Error; + } + for (auto& child: input.Ref().Child(TYtSection::idx_Paths)->Children()) { + if (!child->IsCallable(TYtPath::CallableName())) { + ctx.AddError(TIssue(ctx.GetPosition(child->Pos()), TStringBuilder() << "Expected " << TYtPath::CallableName())); + return TStatus::Error; + } + } + if (!EnsureTuple(*input.Ref().Child(TYtSection::idx_Settings), ctx)) { + return TStatus::Error; + } + EYtSettingTypes acceptedSettings = EYtSettingType::KeyFilter + | EYtSettingType::KeyFilter2 + | EYtSettingType::Take + | EYtSettingType::Skip + | EYtSettingType::DirectRead + | EYtSettingType::Sample + | EYtSettingType::JoinLabel + | EYtSettingType::Unordered + | EYtSettingType::NonUnique + | EYtSettingType::UserSchema + | EYtSettingType::UserColumns + | EYtSettingType::StatColumns + | EYtSettingType::SysColumns; + if (!ValidateSettings(*input.Ref().Child(TYtSection::idx_Settings), acceptedSettings, ctx)) { + return TStatus::Error; + } + + const TTypeAnnotationNode* tableType = nullptr; + TMaybe<bool> yamrFormat; + for (auto child: input.Cast<TYtSection>().Paths()) { + if (tableType) { + if (!IsSameAnnotation(*tableType, *child.Ref().GetTypeAnn())) { + ctx.AddError(TIssue(ctx.GetPosition(input.Pos()), TStringBuilder() + << "All section tables must have the same scheme. Found diff: " + << GetTypeDiff(*tableType, *child.Ref().GetTypeAnn()))); + return TStatus::Error; + } + } else { + tableType = child.Ref().GetTypeAnn(); + } + if (yamrFormat) { + if (*yamrFormat != child.Table().Maybe<TYtTable>().RowSpec().Maybe<TCoVoid>().IsValid()) { + ctx.AddError(TIssue(ctx.GetPosition(input.Pos()), TStringBuilder() << "Mixed Yamr/Yson input table formats")); + return TStatus::Error; + } + } else { + yamrFormat = child.Table().Maybe<TYtTable>().RowSpec().Maybe<TCoVoid>().IsValid(); + } + } + + const auto sysColumns = NYql::GetSettingAsColumnList(*input.Ref().Child(TYtSection::idx_Settings), EYtSettingType::SysColumns); + if (!sysColumns.empty()) { + if (AnyOf(sysColumns, [](const TString& col) { return col == "path" || col == "record"; })) { + for (auto& child: input.Ref().Child(TYtSection::idx_Paths)->Children()) { + if (!TYtTable::Match(child->Child(TYtPath::idx_Table))) { + ctx.AddError(TIssue(ctx.GetPosition(child->Pos()), + TStringBuilder() << EYtSettingType::SysColumns << " setting cannot be used with operation results")); + return TStatus::Error; + } + } + } + + const auto structType = tableType->Cast<TListExprType>()->GetItemType()->Cast<TStructExprType>(); + auto columns = structType->GetItems(); + for (auto sys: sysColumns) { + auto sysColName = TString(YqlSysColumnPrefix).append(sys); + if (!structType->FindItem(sysColName)) { + try { + columns.push_back(ctx.MakeType<TItemExprType>(sysColName, ctx.MakeType<TDataExprType>(NUdf::GetDataSlot(GetSysColumnTypeId(sys))))); + } catch (const TYqlPanic&) { + ctx.AddError(TIssue(ctx.GetPosition(input.Ref().Child(TYtSection::idx_Settings)->Pos()), + TStringBuilder() << "Unsupported system column " << sys.Quote())); + return TStatus::Error; + } + } + } + tableType = ctx.MakeType<TListExprType>(ctx.MakeType<TStructExprType>(columns)); + } + + auto keyFilters = GetAllSettingValues(*input.Ref().Child(TYtSection::idx_Settings), EYtSettingType::KeyFilter); + if (keyFilters.size() > 0) { + TVector<std::pair<TString, TYqlRowSpecInfo::TPtr>> rowSpecs; + for (auto path: input.Cast<TYtSection>().Paths()) { + TYtPathInfo pathInfo(path); + rowSpecs.emplace_back(pathInfo.Table->Name, pathInfo.Table->RowSpec); + } + + TIssueScopeGuard issueScope(ctx.IssueManager, [pos = ctx.GetPosition(input.Pos())]() { + return MakeIntrusive<TIssue>(pos, TStringBuilder() << "Setting " << EYtSettingType::KeyFilter); + }); + + for (auto keyFilter: keyFilters) { + if (keyFilter->ChildrenSize() > 0) { + TMaybe<size_t> tableIndex; + if (keyFilter->ChildrenSize() > 1) { + tableIndex = FromString<size_t>(keyFilter->Child(1)->Content()); + if (tableIndex >= rowSpecs.size()) { + ctx.AddError(TIssue(ctx.GetPosition(keyFilter->Child(1)->Pos()), TStringBuilder() + << "Invalid table index value: " << *tableIndex)); + return TStatus::Error; + } + } + auto andGrp = TCoNameValueTupleList(keyFilter->ChildPtr(0)); + size_t memberIndex = 0; + for (auto keyPredicates: andGrp) { + auto validateMember = [&] (const TString& tableName, const TYqlRowSpecInfo::TPtr& rowSpec) { + if (!rowSpec || !rowSpec->IsSorted()) { + ctx.AddError(TIssue(ctx.GetPosition(input.Pos()), TStringBuilder() + << "Setting " << EYtSettingType::KeyFilter << " cannot be used with unsorted table " + << tableName)); + return false; + } + if (memberIndex >= rowSpec->SortMembers.size() + || rowSpec->SortMembers[memberIndex] != keyPredicates.Name().Value()) + { + ctx.AddError(TIssue(ctx.GetPosition(input.Pos()), TStringBuilder() + << "Predicate column " << TString{keyPredicates.Name().Value()}.Quote() << " doesn't match " + << tableName.Quote() << " table sort columns")); + return false; + } + if (!rowSpec->SortDirections[memberIndex]) { + ctx.AddError(TIssue(ctx.GetPosition(input.Pos()), TStringBuilder() + << "Predicate column " << TString{keyPredicates.Name().Value()}.Quote() + << " has descending sort order in " + << tableName.Quote() << " table")); + return false; + } + return true; + }; + + if (tableIndex) { + auto& rowSpec = rowSpecs[*tableIndex]; + if (!validateMember(rowSpec.first, rowSpec.second)) { + return TStatus::Error; + } + } + else { + for (auto& rowSpec: rowSpecs) { + if (!validateMember(rowSpec.first, rowSpec.second)) { + return TStatus::Error; + } + } + } + for (auto cmp: keyPredicates.Value().Cast<TCoNameValueTupleList>()) { + TExprBase value = cmp.Value().Cast(); + if (!IsNull(value.Ref())) { + bool isOptional = false; + const TDataExprType* valueType = nullptr; + if (!EnsureDataOrOptionalOfData(value.Ref(), isOptional, valueType, ctx)) { + return TStatus::Error; + } + + auto& rowSpec = rowSpecs[tableIndex.GetOrElse(0)]; + const TDataExprType* columnType = nullptr; + if (!EnsureDataOrOptionalOfData(rowSpec.second->FromNode.Cast().Pos(), rowSpec.second->SortedByTypes[memberIndex], isOptional, columnType, ctx)) { + return TStatus::Error; + } + if (valueType->GetSlot() != columnType->GetSlot() + && !GetSuperType(valueType->GetSlot(), columnType->GetSlot())) + { + ctx.AddError(TIssue(ctx.GetPosition(input.Pos()), TStringBuilder() + << "Predicate " << TString{keyPredicates.Name().Value()}.Quote() << " value type " + << *value.Ref().GetTypeAnn() << " is incompatible with " + << *rowSpec.second->SortedByTypes[memberIndex] << " column type of " + << rowSpec.first.Quote() << " table")); + return TStatus::Error; + } + } + } + ++memberIndex; + } + } + } + } + + keyFilters = GetAllSettingValues(*input.Ref().Child(TYtSection::idx_Settings), EYtSettingType::KeyFilter2); + if (keyFilters.size() > 0) { + TVector<std::pair<TString, TYqlRowSpecInfo::TPtr>> rowSpecs; + for (auto path: input.Cast<TYtSection>().Paths()) { + TYtPathInfo pathInfo(path); + rowSpecs.emplace_back(pathInfo.Table->Name, pathInfo.Table->RowSpec); + } + + TIssueScopeGuard issueScope(ctx.IssueManager, [pos = ctx.GetPosition(input.Pos())]() { + return MakeIntrusive<TIssue>(pos, TStringBuilder() << "Setting " << EYtSettingType::KeyFilter2); + }); + + TSet<size_t> processedIndexes; + for (auto keyFilter: keyFilters) { + if (keyFilter->ChildrenSize() == 0) { + continue; + } + + TMaybe<TVector<size_t>> indexes; + if (keyFilter->ChildrenSize() == 3) { + indexes.ConstructInPlace(); + for (auto& idxNode : keyFilter->Tail().ChildrenList()) { + YQL_ENSURE(idxNode->IsAtom()); + indexes->push_back(FromString<size_t>(idxNode->Content())); + } + } + + TVector<TStringBuf> usedKeys; + if (auto usedKeysSetting = GetSetting(*keyFilter->Child(1), "usedKeys")) { + for (auto& key : usedKeysSetting->Tail().ChildrenList()) { + YQL_ENSURE(key->IsAtom()); + usedKeys.push_back(key->Content()); + } + } + + auto& computeNode = keyFilter->Head(); + auto boundaryTypes = computeNode.GetTypeAnn()->Cast<TListExprType>()->GetItemType() + ->Cast<TTupleExprType>()->GetItems().front()->Cast<TTupleExprType>()->GetItems(); + + YQL_ENSURE(boundaryTypes.size() > 1); + // drop include/exclude flag + boundaryTypes.resize(boundaryTypes.size() - 1); + + for (size_t i = 0; i < (indexes ? indexes->size() : rowSpecs.size()); ++i) { + size_t idx = indexes ? (*indexes)[i] : i; + if (idx >= rowSpecs.size()) { + ctx.AddError(TIssue(ctx.GetPosition(keyFilter->Pos()), TStringBuilder() + << "Invalid table index " << idx << ": got only " << rowSpecs.size() << " input tables")); + return TStatus::Error; + } + if (!processedIndexes.insert(idx).second) { + ctx.AddError(TIssue(ctx.GetPosition(input.Pos()), TStringBuilder() + << "Duplicate table index " << idx)); + return TStatus::Error; + } + + const auto& nameAndSpec = rowSpecs[idx]; + if (!nameAndSpec.second || !nameAndSpec.second->IsSorted()) { + ctx.AddError(TIssue(ctx.GetPosition(input.Pos()), TStringBuilder() + << "Setting " << EYtSettingType::KeyFilter2 << " cannot be used with unsorted table " + << nameAndSpec.first.Quote())); + return TStatus::Error; + } + + if (nameAndSpec.second->SortedByTypes.size() < boundaryTypes.size()) { + ctx.AddError(TIssue(ctx.GetPosition(input.Pos()), TStringBuilder() + << "KeyFilter type size mismatch for index " << idx)); + return TStatus::Error; + } + + if (nameAndSpec.second->SortedBy.size() < usedKeys.size()) { + ctx.AddError(TIssue(ctx.GetPosition(input.Pos()), TStringBuilder() + << "KeyFilter used keys size mismatch for index " << idx)); + return TStatus::Error; + } + + YQL_ENSURE(usedKeys.size() <= boundaryTypes.size()); + + for (size_t j = 0; j < boundaryTypes.size(); ++j) { + auto specType = nameAndSpec.second->SortedByTypes[j]; + auto boundaryType = boundaryTypes[j]->Cast<TOptionalExprType>()->GetItemType(); + if (!IsSameAnnotation(*specType, *boundaryType)) { + ctx.AddError(TIssue(ctx.GetPosition(input.Pos()), TStringBuilder() + << "KeyFilter type mismatch for index " << idx << ", table name " + << nameAndSpec.first.Quote() << ", types: " << *specType + << " vs " << *boundaryType)); + return TStatus::Error; + } + + if (j < usedKeys.size()) { + auto specName = nameAndSpec.second->SortedBy[j]; + auto usedKey = usedKeys[j]; + if (specName != usedKey) { + ctx.AddError(TIssue(ctx.GetPosition(input.Pos()), TStringBuilder() + << "KeyFilter key column name mismatch for index " << idx << ", table name " + << nameAndSpec.first.Quote() << ", names: " << specName << " vs " << usedKey)); + return TStatus::Error; + } + } + } + } + } + } + + input.Ptr()->SetTypeAnn(tableType); + + auto paths = input.Ref().Child(TYtSection::idx_Paths)->ChildrenList(); + YQL_ENSURE(!paths.empty()); + auto common = State_->Types->LookupColumnOrder(*paths.front()); + if (!common) { + return TStatus::Ok; + } + + for (ui32 i = 1; i < paths.size(); ++i) { + auto current = State_->Types->LookupColumnOrder(*paths[i]); + if (!current || *common != *current) { + return TStatus::Ok; + } + } + + // add system columns + auto extraColumns = sysColumns; + for (auto& sys: extraColumns) { + sys = TString(YqlSysColumnPrefix).append(sys); + } + Sort(extraColumns); + for (auto &e: extraColumns) { + common->AddColumn(e); + } + return State_->Types->SetColumnOrder(input.Ref(), *common, ctx); + } + + TStatus HandleReadTable(const TExprNode::TPtr& input, TExprNode::TPtr& output, TExprContext& ctx) { + if (!EnsureArgsCount(*input, 3, ctx)) { + return TStatus::Error; + } + + if (!EnsureWorldType(*input->Child(TYtReadTable::idx_World), ctx)) { + return TStatus::Error; + } + + if (!EnsureSpecificDataSource(*input->Child(TYtReadTable::idx_DataSource), YtProviderName, ctx)) { + return TStatus::Error; + } + + if (!EnsureTupleMinSize(*input->Child(TYtReadTable::idx_Input), 1, ctx)) { + return TStatus::Error; + } + + for (auto& section: input->Child(TYtReadTable::idx_Input)->Children()) { + if (!section->IsCallable(TYtSection::CallableName())) { + ctx.AddError(TIssue(ctx.GetPosition(section->Pos()), TStringBuilder() << "Expected " << TYtSection::CallableName())); + return TStatus::Error; + } + } + + auto cluster = TString{TYtDSource(input->ChildPtr(TYtReadTable::idx_DataSource)).Cluster().Value()}; + TMaybe<bool> yamrFormat; + TMaybe<TSampleParams> sampling; + for (size_t i = 0; i < input->Child(TYtReadTable::idx_Input)->ChildrenSize(); ++i) { + auto section = TYtSection(input->Child(TYtReadTable::idx_Input)->ChildPtr(i)); + if (NYql::HasSetting(section.Settings().Ref(), EYtSettingType::JoinLabel)) { + ctx.AddError(TIssue(ctx.GetPosition(section.Pos()), TStringBuilder() + << "Setting \"" << EYtSettingType::JoinLabel << "\" is not allowed in " << TYtReadTable::CallableName())); + return TStatus::Error; + } + + if (0 == i) { + sampling = NYql::GetSampleParams(section.Settings().Ref()); + } else if (NYql::GetSampleParams(section.Settings().Ref()) != sampling) { + ctx.AddError(TIssue(ctx.GetPosition(section.Pos()), "Sections have different sample values")); + return TStatus::Error; + } + + for (auto path: section.Paths()) { + if (auto maybeTable = path.Table().Maybe<TYtTable>()) { + auto table = maybeTable.Cast(); + auto tableName = table.Name().Value(); + if (!NYql::HasSetting(table.Settings().Ref(), EYtSettingType::UserSchema)) { + // Don't validate already substituted anonymous tables + if (!NYql::HasSetting(table.Settings().Ref(), EYtSettingType::Anonymous) || !tableName.StartsWith("tmp/")) { + const TYtTableDescription& tableDesc = State_->TablesData->GetTable(cluster, + TString{tableName}, + TEpochInfo::Parse(table.Epoch().Ref())); + + if (!tableDesc.Validate(ctx.GetPosition(table.Pos()), cluster, tableName, + NYql::HasSetting(table.Settings().Ref(), EYtSettingType::WithQB), State_->AnonymousLabels, ctx)) { + return TStatus::Error; + } + } + } + } + if (yamrFormat) { + if (*yamrFormat != path.Table().Maybe<TYtTable>().RowSpec().Maybe<TCoVoid>().IsValid()) { + ctx.AddError(TIssue(ctx.GetPosition(input->Pos()), TStringBuilder() << "Mixed Yamr/Yson input table formats")); + return TStatus::Error; + } + } else { + yamrFormat = path.Table().Maybe<TYtTable>().RowSpec().Maybe<TCoVoid>().IsValid(); + } + } + } + + auto readInput = input->ChildPtr(TYtReadTable::idx_Input); + auto newInput = ValidateAndUpdateTablesMeta(readInput, cluster, State_->TablesData, State_->Types->UseTableMetaFromGraph, ctx); + if (!newInput) { + return TStatus::Error; + } + else if (newInput != readInput) { + output = ctx.ChangeChild(*input, TYtReadTable::idx_Input, std::move(newInput)); + return TStatus::Repeat; + } + + TTypeAnnotationNode::TListType items; + for (auto section: TYtSectionList(input->ChildPtr(TYtReadTable::idx_Input))) { + items.push_back(section.Ref().GetTypeAnn()->Cast<TListExprType>()->GetItemType()); + } + auto itemType = items.size() == 1 + ? items.front() + : ctx.MakeType<TVariantExprType>(ctx.MakeType<TTupleExprType>(items)); + + input->SetTypeAnn(ctx.MakeType<TTupleExprType>(TTypeAnnotationNode::TListType{ + input->Child(TYtReadTable::idx_World)->GetTypeAnn(), + ctx.MakeType<TListExprType>(itemType) + })); + + if (items.size() == 1) { + if (auto columnOrder = State_->Types->LookupColumnOrder(input->Child(TYtReadTable::idx_Input)->Head())) { + return State_->Types->SetColumnOrder(*input, *columnOrder, ctx); + } + } + return TStatus::Ok; + } + + TStatus HandleReadTableScheme(const TExprNode::TPtr& input, TExprNode::TPtr& /*output*/, TExprContext& ctx) { + if (!EnsureArgsCount(*input, 4, ctx)) { + return TStatus::Error; + } + + if (!EnsureWorldType(*input->Child(TYtReadTableScheme::idx_World), ctx)) { + return TStatus::Error; + } + + if (!EnsureSpecificDataSource(*input->Child(TYtReadTableScheme::idx_DataSource), YtProviderName, ctx)) { + return TStatus::Error; + } + + if (!input->Child(TYtReadTableScheme::idx_Table)->IsCallable(TYtTable::CallableName())) { + ctx.AddError(TIssue(ctx.GetPosition(input->Child(TYtReadTableScheme::idx_Table)->Pos()), TStringBuilder() + << "Expected " << TYtTable::CallableName())); + return TStatus::Error; + } + + if (!EnsureType(*input->Child(TYtReadTableScheme::idx_Type), ctx)) { + return TStatus::Error; + } + + auto tableType = input->Child(TYtReadTableScheme::idx_Type)->GetTypeAnn()->Cast<TTypeExprType>()->GetType(); + if (!EnsureListType(input->Child(TYtReadTableScheme::idx_Type)->Pos(), *tableType, ctx)) { + return TStatus::Error; + } + + auto rowType = tableType->Cast<TListExprType>()->GetItemType(); + if (!EnsureStructType(input->Child(TYtReadTableScheme::idx_Type)->Pos(), *rowType, ctx)) { + return TStatus::Error; + } + + auto readScheme = TYtReadTableScheme(input); + auto cluster = TString{readScheme.DataSource().Cluster().Value()}; + auto tableName = TString{readScheme.Table().Name().Value()}; + auto view = NYql::GetSetting(readScheme.Table().Settings().Ref(), EYtSettingType::View); + TString viewName = view ? TString{view->Child(1)->Content()} : TString(); + + TYtTableDescription& tableDesc = State_->TablesData->GetOrAddTable(cluster, tableName, + TEpochInfo::Parse(readScheme.Table().Epoch().Ref())); + + // update RowType in description + if (!viewName.empty()) { + tableDesc.Views[viewName].RowType = rowType; + } else if (tableDesc.View.Defined()) { + tableDesc.View->RowType = rowType; + } else { + tableDesc.RowType = rowType; + } + + input->SetTypeAnn(ctx.MakeType<TTupleExprType>(TTypeAnnotationNode::TListType{ + input->Child(TYtReadTableScheme::idx_World)->GetTypeAnn(), + ctx.MakeType<TDataExprType>(EDataSlot::Yson) + })); + return TStatus::Ok; + } + + TStatus HandleTableContent(TExprBase input, TExprContext& ctx) { + if (!EnsureArgsCount(input.Ref(), 2, ctx)) { + return TStatus::Error; + } + + const auto tableContent = input.Cast<TYtTableContent>(); + + if (!tableContent.Input().Ref().IsCallable(TYtReadTable::CallableName()) + && !tableContent.Input().Ref().IsCallable(TYtOutput::CallableName())) { + ctx.AddError(TIssue(ctx.GetPosition(tableContent.Input().Pos()), TStringBuilder() + << "Expected " << TYtReadTable::CallableName() << " or " << TYtOutput::CallableName())); + return TStatus::Error; + } + + if (!EnsureTuple(tableContent.Settings().MutableRef(), ctx)) { + return TStatus::Error; + } + + const EYtSettingTypes allowed = EYtSettingType::MemUsage | EYtSettingType::ItemsCount | EYtSettingType::RowFactor | EYtSettingType::Split | EYtSettingType::Small; + if (!ValidateSettings(tableContent.Settings().Ref(), allowed, ctx)) { + return TStatus::Error; + } + + input.Ptr()->SetTypeAnn(tableContent.Input().Maybe<TYtOutput>() + ? tableContent.Input().Ref().GetTypeAnn() + : tableContent.Input().Ref().GetTypeAnn()->Cast<TTupleExprType>()->GetItems().back()); + + if (auto columnOrder = State_->Types->LookupColumnOrder(tableContent.Input().Ref())) { + return State_->Types->SetColumnOrder(input.Ref(), *columnOrder, ctx); + } + return TStatus::Ok; + } + + TStatus HandleLength(const TExprNode::TPtr& input, TExprNode::TPtr& /*output*/, TExprContext& ctx) { + if (!EnsureArgsCount(*input, 1, ctx)) { + return TStatus::Error; + } + + auto lenInput = input->Child(TYtLength::idx_Input); + if (lenInput->IsCallable(TYtReadTable::CallableName())) { + if (TYtReadTable(lenInput).Input().Size() != 1) { + ctx.AddError(TIssue(ctx.GetPosition(lenInput->Pos()), TStringBuilder() + << "Unsupported " << TYtReadTable::CallableName() << " with multiple sections")); + return TStatus::Error; + } + } + else if (!lenInput->IsCallable(TYtOutput::CallableName())) { + ctx.AddError(TIssue(ctx.GetPosition(lenInput->Pos()), TStringBuilder() + << "Expected " << TYtReadTable::CallableName() << " or " << TYtOutput::CallableName())); + return TStatus::Error; + } + + input->SetTypeAnn(ctx.MakeType<TDataExprType>(EDataSlot::Uint64));; + return TStatus::Ok; + } + + TStatus HandleConfigure(TExprBase input, TExprContext& ctx) { + if (!EnsureMinArgsCount(input.Ref(), 2, ctx)) { + return TStatus::Error; + } + + if (!EnsureWorldType(*input.Ptr()->Child(TCoConfigure::idx_World), ctx)) { + return TStatus::Error; + } + + if (!EnsureSpecificDataSource(*input.Ptr()->Child(TCoConfigure::idx_DataSource), YtProviderName, ctx)) { + return TStatus::Error; + } + + input.Ptr()->SetTypeAnn(input.Ref().Child(TCoConfigure::idx_World)->GetTypeAnn()); + return TStatus::Ok; + } + + TStatus HandleYtConfigure(TExprBase input, TExprContext& ctx) { + if (!EnsureMinArgsCount(input.Ref(), 2, ctx)) { + return TStatus::Error; + } + + if (!EnsureWorldType(*input.Ptr()->Child(TYtConfigure::idx_World), ctx)) { + return TStatus::Error; + } + + if (!EnsureSpecificDataSource(*input.Ptr()->Child(TYtConfigure::idx_DataSource), YtProviderName, ctx)) { + return TStatus::Error; + } + + input.Ptr()->SetTypeAnn(input.Ref().Child(TYtConfigure::idx_World)->GetTypeAnn()); + return TStatus::Ok; + } + + TStatus HandleTablePath(TExprBase input, TExprContext& ctx) { + if (State_->Configuration->UseSystemColumns.Get().GetOrElse(DEFAULT_USE_SYS_COLUMNS)) { + ctx.AddError(TIssue(ctx.GetPosition(input.Pos()), + TStringBuilder() << "Unsupported callable " << input.Ref().Content())); + return TStatus::Error; + } + + if (!EnsureArgsCount(input.Ref(), 1, ctx)) { + return TStatus::Error; + } + + if (!EnsureDependsOn(*input.Ptr()->Child(TYtTablePath::idx_DependsOn), ctx)) { + return TStatus::Error; + } + + input.Ptr()->SetTypeAnn(ctx.MakeType<TDataExprType>(EDataSlot::String)); + return TStatus::Ok; + } + + TStatus HandleTableRecord(TExprBase input, TExprContext& ctx) { + if (State_->Configuration->UseSystemColumns.Get().GetOrElse(DEFAULT_USE_SYS_COLUMNS)) { + ctx.AddError(TIssue(ctx.GetPosition(input.Pos()), + TStringBuilder() << "Unsupported callable " << input.Ref().Content())); + return TStatus::Error; + } + + if (!EnsureArgsCount(input.Ref(), 1, ctx)) { + return TStatus::Error; + } + + if (!EnsureDependsOn(*input.Ptr()->Child(TYtTablePropBase::idx_DependsOn), ctx)) { + return TStatus::Error; + } + + input.Ptr()->SetTypeAnn(ctx.MakeType<TDataExprType>(EDataSlot::Uint64)); + return TStatus::Ok; + } + + TStatus HandleTableIndex(TExprBase input, TExprContext& ctx) { + if (State_->Configuration->UseSystemColumns.Get().GetOrElse(DEFAULT_USE_SYS_COLUMNS)) { + ctx.AddError(TIssue(ctx.GetPosition(input.Pos()), + TStringBuilder() << "Unsupported callable " << input.Ref().Content())); + return TStatus::Error; + } + + if (!EnsureArgsCount(input.Ref(), 1, ctx)) { + return TStatus::Error; + } + + if (!EnsureDependsOn(*input.Ptr()->Child(TYtTableIndex::idx_DependsOn), ctx)) { + return TStatus::Error; + } + + input.Ptr()->SetTypeAnn(ctx.MakeType<TDataExprType>(EDataSlot::Uint32)); + return TStatus::Ok; + } + + TStatus HandleIsKeySwitch(TExprBase input, TExprContext& ctx) { + if (State_->Configuration->UseSystemColumns.Get().GetOrElse(DEFAULT_USE_SYS_COLUMNS)) { + ctx.AddError(TIssue(ctx.GetPosition(input.Pos()), + TStringBuilder() << "Unsupported callable " << input.Ref().Content())); + return TStatus::Error; + } + + if (!EnsureArgsCount(input.Ref(), 1, ctx)) { + return TStatus::Error; + } + + if (!EnsureDependsOn(*input.Ptr()->Child(TYtIsKeySwitch::idx_DependsOn), ctx)) { + return TStatus::Error; + } + + input.Ptr()->SetTypeAnn(ctx.MakeType<TDataExprType>(EDataSlot::Bool)); + return TStatus::Ok; + } + + TStatus HandleTableName(const TExprNode::TPtr& input, TExprNode::TPtr& output, TExprContext& ctx) { + if (!EnsureArgsCount(*input, 1, ctx)) { + return TStatus::Error; + } + + output = ctx.Builder(input->Pos()) + .Callable("Substring") + .Add(0, input->ChildPtr(0)) + .Callable(1, "Inc") + .Callable(0, "RFind") + .Add(0, input->ChildPtr(0)) + .Callable(1, "String").Atom(0, "/").Seal() + .Callable(2, "Null").Seal() + .Seal() + .Seal() + .Callable(2, "Null").Seal() + .Seal() + .Build(); + + return TStatus::Repeat; + } +private: + const TYtState::TPtr State_; +}; + +} + +THolder<TVisitorTransformerBase> CreateYtDataSourceTypeAnnotationTransformer(TYtState::TPtr state) { + return THolder(new TYtDataSourceTypeAnnotationTransformer(state)); +} + +} diff --git a/yt/yql/providers/yt/provider/yql_yt_dq_hybrid.cpp b/yt/yql/providers/yt/provider/yql_yt_dq_hybrid.cpp new file mode 100644 index 0000000000..26ffebf27e --- /dev/null +++ b/yt/yql/providers/yt/provider/yql_yt_dq_hybrid.cpp @@ -0,0 +1,732 @@ +#include "yql_yt_provider_impl.h" +#include "yql_yt_helpers.h" +#include "yql_yt_op_settings.h" +#include "yql_yt_optimize.h" + +#include <yt/yql/providers/yt/expr_nodes/yql_yt_expr_nodes.h> +#include <yt/yql/providers/yt/common/yql_names.h> +#include <yt/yql/providers/yt/common/yql_configuration.h> +#include <yql/essentials/providers/common/provider/yql_provider_names.h> +#include <yql/essentials/providers/common/provider/yql_provider.h> +#include <yql/essentials/providers/common/transform/yql_optimize.h> +#include <yql/essentials/core/dq_expr_nodes/dq_expr_nodes.h> +#include <yql/essentials/core/dqs_expr_nodes/dqs_expr_nodes.h> +#include <yql/essentials/core/expr_nodes/yql_expr_nodes.h> +#include <yql/essentials/core/yql_expr_type_annotation.h> +#include <yql/essentials/core/yql_expr_optimize.h> +#include <yql/essentials/core/yql_opt_utils.h> +#include <yql/essentials/core/yql_type_helpers.h> +#include <yql/essentials/core/yql_data_provider.h> +#include <yql/essentials/utils/log/log.h> +#include <yql/essentials/utils/yql_panic.h> +#include <yql/essentials/minikql/mkql_program_builder.h> + +#include <util/generic/size_literals.h> + +#include <algorithm> +#include <type_traits> + +namespace NYql { + +namespace { + +using namespace NNodes; +using namespace NNodes::NDq; + +class TYtDqHybridTransformer : public TOptimizeTransformerBase { +public: + TYtDqHybridTransformer(TYtState::TPtr state, THolder<IGraphTransformer>&& finalizer) + : TOptimizeTransformerBase(state->Types, NLog::EComponent::ProviderYt, state->Configuration->DisableOptimizers.Get().GetOrElse(TSet<TString>())) + , State_(std::move(state)), Finalizer_(std::move(finalizer)) + { +#define HNDL(name) "YtDqHybrid-"#name, Hndl(&TYtDqHybridTransformer::name) + if (State_->DqHelper) { + AddHandler(0, &TYtFill::Match, HNDL(TryYtFillByDq)); + AddHandler(0, &TYtSort::Match, HNDL(TryYtSortByDq)); + AddHandler(0, &TYtMap::Match, HNDL(TryYtMapByDq)); + AddHandler(0, &TYtReduce::Match, HNDL(TryYtReduceByDq)); + AddHandler(0, &TYtMapReduce::Match, HNDL(TryYtMapReduceByDq)); + AddHandler(0, &TYtMerge::Match, HNDL(TryYtMergeByDq)); + } +#undef HNDL + } +private: + TStatus DoTransform(TExprNode::TPtr input, TExprNode::TPtr& output, TExprContext& ctx) final { + if (const auto status = Finalizer_->Transform(input, output, ctx); status.Level != TStatus::Ok) + return status; + + return TOptimizeTransformerBase::DoTransform(input, output, ctx); + } + + void Rewind() final { + Finalizer_->Rewind(); + TOptimizeTransformerBase::Rewind(); + } + + bool CanReplaceOnHybrid(const TYtOutputOpBase& operation) const { + const TStringBuf nodeName = operation.Raw()->Content(); + if (!State_->IsHybridEnabledForCluster(operation.DataSink().Cluster().Value())) { + PushSkipStat("DisabledCluster", nodeName); + return false; + } + + if (State_->HybridTakesTooLong()) { + PushSkipStat("TakesTooLong", nodeName); + return false; + } + + if (operation.Ref().StartsExecution() || operation.Ref().HasResult()) + return false; + + if (!State_->Configuration->UseSystemColumns.Get().GetOrElse(DEFAULT_USE_SYS_COLUMNS)) { + PushSkipStat("FalseSystemColumns", nodeName); + return false; + } + + if (operation.Output().Size() != 1U) { + PushSkipStat("MultipleOutputs", nodeName); + return false; + } + + if (const auto& trans = operation.Maybe<TYtTransientOpBase>(); trans && trans.Cast().Input().Size() != 1U) { + PushSkipStat("MultipleInputs", nodeName); + return false; + } + + const auto& settings = *operation.Ref().Child(4U); + if (HasSettingsExcept(settings, DqOpSupportedSettings)) { + if (!NYql::HasSetting(settings, EYtSettingType::NoDq)) { + PushSkipStat("UnsupportedDqOpSettings", nodeName); + PushSettingsToStat(settings, nodeName, "SkipDqOpSettings", DqOpSupportedSettings); + } + return false; + } + + return true; + } + + bool HasDescOrderOutput(const TYtOutputOpBase& operation) const { + TYqlRowSpecInfo outRowSpec(operation.Output().Item(0).RowSpec()); + if (outRowSpec.IsSorted() && outRowSpec.HasAuxColumns()) { + PushSkipStat("DescSort", operation.Raw()->Content()); + return true; + } + return false; + } + + bool CanReadHybrid(const TYtSection& section, const TStringBuf& nodeName, bool orderedInput) const { + const auto& settings = section.Settings().Ref(); + if (HasSettingsExcept(settings, DqReadSupportedSettings)) { + PushSkipStat("UnsupportedDqReadSettings", nodeName); + PushSettingsToStat(settings, nodeName, "SkipDqReadSettings", DqReadSupportedSettings); + return false; + } + if (HasNonEmptyKeyFilter(section)) { + PushSkipStat("NonEmptyKeyFilter", nodeName); + return false; + } + ui64 dataSize = 0ULL, dataChunks = 0ULL; + for (const auto& path : section.Paths()) { + const TYtPathInfo info(path); + const auto& tableInfo = info.Table; + if (!tableInfo || !tableInfo->Stat || !tableInfo->Meta || !tableInfo->RowSpec) { + return false; + } + const auto canUseYtPartitioningApi = State_->Configuration->_EnableYtPartitioning.Get(tableInfo->Cluster).GetOrElse(false); + if ((info.Ranges || tableInfo->Meta->IsDynamic) && !canUseYtPartitioningApi) { + return false; + } + if (NYql::HasSetting(tableInfo->Settings.Ref(), EYtSettingType::WithQB)) { + PushSkipStat("WithQB", nodeName); + return false; + } + auto tableSize = tableInfo->Stat->DataSize; + if (tableInfo->Meta->Attrs.Value("erasure_codec", "none") != "none") { + if (const auto codecCpu = State_->Configuration->ErasureCodecCpuForDq.Get(tableInfo->Cluster)) { + tableSize *=* codecCpu; + } + } + + dataSize += tableSize; + dataChunks += tableInfo->Stat->ChunkCount; + } + const auto chunksLimit = State_->Configuration->MaxChunksForDqRead.Get().GetOrElse(DEFAULT_MAX_CHUNKS_FOR_DQ_READ); + + const auto sizeLimit = orderedInput ? + State_->Configuration->HybridDqDataSizeLimitForOrdered.Get().GetOrElse(DefaultHybridDqDataSizeLimitForOrdered): + State_->Configuration->HybridDqDataSizeLimitForUnordered.Get().GetOrElse(DefaultHybridDqDataSizeLimitForUnordered); + + if (dataSize > sizeLimit || dataChunks > chunksLimit) { + PushSkipStat("OverLimits", nodeName); + return false; + } + + return true; + } + + bool CanExecuteInHybrid(const TExprNode::TPtr& handler, const TStringBuf& nodeName, bool orderedInput) const { + bool flow = false; + auto sources = FindNodes(handler, + [](const TExprNode::TPtr& node) { + return !TYtOutput::Match(node.Get()); + }, + [](const TExprNode::TPtr& node) { + return TYtTableContent::Match(node.Get()); + }); + TNodeSet flowSources; + std::for_each(sources.cbegin(), sources.cend(), [&flowSources](const TExprNode::TPtr& node) { flowSources.insert(node.Get()); }); + bool noNonTransparentNode = !FindNonYieldTransparentNode(handler, *State_->Types, flowSources); + if (!noNonTransparentNode) { + PushSkipStat("NonTransparentNode", nodeName); + } + return noNonTransparentNode && + !FindNode(handler, [&flow, this, &nodeName, orderedInput] (const TExprNode::TPtr& node) { + if (TCoScriptUdf::Match(node.Get()) && NKikimr::NMiniKQL::IsSystemPython(NKikimr::NMiniKQL::ScriptTypeFromStr(node->Head().Content()))) { + return true; + } + + if (const auto& tableContent = TMaybeNode<TYtTableContent>(node)) { + if (!flow) + return true; + if (const auto& maybeRead = tableContent.Cast().Input().Maybe<TYtReadTable>()) { + const auto& read = maybeRead.Cast(); + if (1U != read.Input().Size()) { + PushSkipStat("MultipleInputs", nodeName); + return true; + } + if(!CanReadHybrid(read.Input().Item(0), nodeName, orderedInput)) { + return true; + } + return false; + } + } + flow = node->IsCallable(TCoToFlow::CallableName()) && node->Head().IsCallable(TYtTableContent::CallableName()); + return false; + }); + } + + TMaybeNode<TExprBase> TryYtFillByDq(TExprBase node, TExprContext& ctx) const { + const TStringBuf nodeName = node.Raw()->Content(); + const auto fill = node.Cast<TYtFill>(); + if (CanReplaceOnHybrid(fill) && CanExecuteInHybrid(fill.Content().Ptr(), nodeName, true)) { + YQL_CLOG(INFO, ProviderYt) << "Rewrite " << nodeName << " node by hybrid"; + PushHybridStat("Try", nodeName); + return Build<TYtTryFirst>(ctx, fill.Pos()) + .First<TYtDqProcessWrite>() + .World(fill.World()) + .DataSink(fill.DataSink()) + .Output(fill.Output()) + .Input<TDqCnResult>() + .Output() + .Stage<TDqStage>() + .Inputs().Build() + .Program<TCoLambda>() + .Args({}) + .Body<TDqWrite>() + .Input(CloneCompleteFlow(fill.Content().Body().Ptr(), ctx)) + .Provider().Value(YtProviderName).Build() + .Settings<TCoNameValueTupleList>().Build() + .Build() + .Build() + .Settings(State_->DqHelper->CreateDqStageSettings(true, ctx, fill.Pos())) + .Build() + .Index().Build(ctx.GetIndexAsString(0), TNodeFlags::Default) + .Build() + .ColumnHints().Build() + .Build() + .Flags().Add(GetHybridFlags(fill, ctx)).Build() + .Build() + .Second<TYtFill>() + .InitFrom(fill) + .Settings(NYql::AddSetting(fill.Settings().Ref(), EYtSettingType::NoDq, {}, ctx)) + .Build() + .Done(); + } + return node; + } + + TExprBase MakeYtSortByDq(const TYtTransientOpBase& sort, TExprContext& ctx) const { + TSyncMap syncList; + const auto& paths = sort.Input().Item(0).Paths(); + for (auto i = 0U; i < paths.Size(); ++i) { + if (const auto mayOut = paths.Item(i).Table().Maybe<TYtOutput>()) { + syncList.emplace(GetOutputOp(mayOut.Cast()).Ptr(), syncList.size()); + } + } + auto newWorld = ApplySyncListToWorld(sort.World().Ptr(), syncList, ctx); + + const auto input = Build<TCoToFlow>(ctx, sort.Pos()) + .Input<TYtTableContent>() + .Input<TYtReadTable>() + .World<TCoWorld>().Build() + .DataSource<TYtDSource>() + .Category(sort.DataSink().Category()) + .Cluster(sort.DataSink().Cluster()) + .Build() + .Input(sort.Input()) + .Build() + .Settings().Build() + .Build() + .Done(); + + TExprNode::TPtr limit; + if (const auto& limitNode = NYql::GetSetting(sort.Settings().Ref(), EYtSettingType::Limit)) { + limit = GetLimitExpr(limitNode, ctx); + } + + auto [direct, selector] = GetOutputSortSettings(sort, ctx); + auto work = direct && selector ? + limit ? + Build<TCoTopSort>(ctx, sort.Pos()) + .Input(input) + .Count(std::move(limit)) + .SortDirections(std::move(direct)) + .KeySelectorLambda(std::move(selector)) + .Done().Ptr(): + Build<TCoSort>(ctx, sort.Pos()) + .Input(input) + .SortDirections(std::move(direct)) + .KeySelectorLambda(std::move(selector)) + .Done().Ptr(): + limit ? + Build<TCoTake>(ctx, sort.Pos()) + .Input(input) + .Count(std::move(limit)) + .Done().Ptr(): + input.Ptr(); + + auto settings = NYql::AddSetting(sort.Settings().Ref(), EYtSettingType::NoDq, {}, ctx); + auto operation = ctx.ChangeChild(sort.Ref(), TYtTransientOpBase::idx_Settings, std::move(settings)); + + return Build<TYtTryFirst>(ctx, sort.Pos()) + .First<TYtDqProcessWrite>() + .World(std::move(newWorld)) + .DataSink(sort.DataSink()) + .Output(sort.Output()) + .Input<TDqCnResult>() + .Output() + .Stage<TDqStage>() + .Inputs().Build() + .Program<TCoLambda>() + .Args({}) + .Body<TDqWrite>() + .Input(std::move(work)) + .Provider().Value(YtProviderName).Build() + .Settings<TCoNameValueTupleList>().Build() + .Build() + .Build() + .Settings(State_->DqHelper->CreateDqStageSettings(true, ctx, sort.Pos())) + .Build() + .Index().Build(ctx.GetIndexAsString(0), TNodeFlags::Default) + .Build() + .ColumnHints().Build() + .Build() + .Flags().Add(GetHybridFlags(sort, ctx)).Build() + .Build() + .Second(std::move(operation)) + .Done(); + } + + TMaybeNode<TExprBase> TryYtSortByDq(TExprBase node, TExprContext& ctx) const { + const auto sort = node.Cast<TYtSort>(); + const TStringBuf nodeName = node.Raw()->Content(); + if (CanReplaceOnHybrid(sort) && CanReadHybrid(sort.Input().Item(0), node.Raw()->Content(), true) && !HasDescOrderOutput(sort)) { + PushHybridStat("Try", nodeName); + YQL_CLOG(INFO, ProviderYt) << "Rewrite " << nodeName << " node by hybrid"; + return MakeYtSortByDq(sort, ctx); + } + return node; + } + + TMaybeNode<TExprBase> TryYtMergeByDq(TExprBase node, TExprContext& ctx) const { + const auto merge = node.Cast<TYtMerge>(); + const TStringBuf nodeName = node.Raw()->Content(); + if (CanReplaceOnHybrid(merge) && CanReadHybrid(merge.Input().Item(0), node.Raw()->Content(), true) && !HasDescOrderOutput(merge)) { + PushHybridStat("Try", nodeName); + YQL_CLOG(INFO, ProviderYt) << "Rewrite " << nodeName << " node by hybrid"; + return MakeYtSortByDq(merge, ctx); + } + return node; + } + + TMaybeNode<TExprBase> TryYtMapByDq(TExprBase node, TExprContext& ctx) const { + const TStringBuf& nodeName = node.Raw()->Content(); + const auto map = node.Cast<TYtMap>(); + bool ordered = NYql::HasSetting(map.Settings().Ref(), EYtSettingType::Ordered); + if (!ordered) { + auto setting = NYql::GetSetting(map.Settings().Ref(), EYtSettingType::JobCount); + if (setting && FromString<ui64>(setting->Child(1)->Content()) == 1) { + ordered = true; + } + } + if (CanReplaceOnHybrid(map) && CanReadHybrid(map.Input().Item(0), nodeName, ordered) && CanExecuteInHybrid(map.Mapper().Ptr(), nodeName, ordered)) { + YQL_CLOG(INFO, ProviderYt) << "Rewrite " << nodeName << " node by hybrid"; + PushHybridStat("Try", nodeName); + TSyncMap syncList; + const auto& paths = map.Input().Item(0).Paths(); + for (auto i = 0U; i < paths.Size(); ++i) { + if (const auto mayOut = paths.Item(i).Table().Maybe<TYtOutput>()) { + syncList.emplace(GetOutputOp(mayOut.Cast()).Ptr(), syncList.size()); + } + } + auto newWorld = ApplySyncListToWorld(map.World().Ptr(), syncList, ctx); + + auto settings = ctx.NewList(map.Input().Pos(), {}); + if (!ordered) { + settings = NYql::AddSetting(*settings, EYtSettingType::Split, nullptr, ctx); + } + + auto stage = Build<TDqStage>(ctx, map.Pos()) + .Inputs().Build() + .Program<TCoLambda>() + .Args({}) + .Body<TDqWrite>() + .Input<TExprApplier>() + .Apply(map.Mapper()) + .With<TCoToFlow>(0) + .Input<TYtTableContent>() + .Input<TYtReadTable>() + .World<TCoWorld>().Build() + .DataSource<TYtDSource>() + .Category(map.DataSink().Category()) + .Cluster(map.DataSink().Cluster()) + .Build() + .Input(map.Input()) + .Build() + .Settings(std::move(settings)) + .Build() + .Build() + .Build() + .Provider().Value(YtProviderName).Build() + .Settings<TCoNameValueTupleList>().Build() + .Build() + .Build() + .Settings(State_->DqHelper->CreateDqStageSettings(ordered, ctx, map.Pos())) + .Done(); + + if (!ordered) { + stage = Build<TDqStage>(ctx, map.Pos()) + .Inputs() + .Add<TDqCnUnionAll>() + .Output() + .Stage(std::move(stage)) + .Index().Build(ctx.GetIndexAsString(0), TNodeFlags::Default) + .Build() + .Build() + .Build() + .Program().Args({"pass"}).Body("pass").Build() + .Settings(State_->DqHelper->CreateDqStageSettings(false, ctx, map.Pos())) + .Done(); + } + + return Build<TYtTryFirst>(ctx, map.Pos()) + .First<TYtDqProcessWrite>() + .World(std::move(newWorld)) + .DataSink(map.DataSink()) + .Output(map.Output()) + .Input<TDqCnResult>() + .Output() + .Stage(std::move(stage)) + .Index().Build(ctx.GetIndexAsString(0), TNodeFlags::Default) + .Build() + .ColumnHints().Build() + .Build() + .Flags().Add(GetHybridFlags(map, ctx)).Build() + .Build() + .Second<TYtMap>() + .InitFrom(map) + .Settings(NYql::AddSetting(map.Settings().Ref(), EYtSettingType::NoDq, {}, ctx)) + .Build() + .Done(); + } + return node; + } + + template<class TYtOperation> + TExprBase MakeYtReduceByDq(const TYtOperation& reduce, TExprContext& ctx) const { + TSyncMap syncList; + const auto& paths = reduce.Input().Item(0).Paths(); + for (auto i = 0U; i < paths.Size(); ++i) { + if (const auto mayOut = paths.Item(i).Table().template Maybe<TYtOutput>()) { + syncList.emplace(GetOutputOp(mayOut.Cast()).Ptr(), syncList.size()); + } + } + auto newWorld = ApplySyncListToWorld(reduce.World().Ptr(), syncList, ctx); + + auto keys = NYql::GetSettingAsColumnAtomList(reduce.Settings().Ref(), EYtSettingType::ReduceBy); + + auto sortKeys = ctx.NewCallable(reduce.Pos(), "Void", {}); + auto sortDirs = sortKeys; + + const auto keysBuilder = [](TExprNode::TListType& keys, TExprNodeBuilder& parent) -> TExprNodeBuilder& { + if (1U == keys.size()) { + parent.Callable("Member") + .Arg(0, "row") + .Add(1, std::move(keys.front())) + .Seal(); + } else { + parent.List() + .Do([&](TExprNodeBuilder& parent) -> TExprNodeBuilder& { + for (auto i = 0U; i < keys.size(); ++i) { + parent.Callable(i, "Member") + .Arg(0, "row") + .Add(1, std::move(keys[i])) + .Seal(); + } + return parent; + }) + .Seal(); + } + return parent; + }; + + if (auto sortBy = NYql::GetSettingAsColumnAtomPairList(reduce.Settings().Ref(), EYtSettingType::SortBy); !sortBy.empty()) { + for (auto it = keys.cbegin(); !sortBy.empty() && keys.cend() != it && *it == sortBy.front().first; ++it) + sortBy.erase(sortBy.cbegin()); + + if (!sortBy.empty()) { + TExprNode::TListType sort, dirs; + sort.reserve(sortBy.size()); + dirs.reserve(sortBy.size()); + + for (auto& item : sortBy) { + sort.emplace_back(std::move(item.first)); + dirs.emplace_back(MakeBool(reduce.Pos(), item.second, ctx)); + } + + sortDirs = 1U == dirs.size() ? std::move(dirs.front()) : ctx.NewList(reduce.Pos(), std::move(dirs)); + sortKeys = ctx.Builder(reduce.Pos()) + .Lambda() + .Param("row") + .Do(std::bind(keysBuilder, std::ref(sort), std::placeholders::_1)) + .Seal().Build(); + } + } + + const auto extract = TCoLambda(ctx.Builder(reduce.Pos()) + .Lambda() + .Param("row") + .Do(std::bind(keysBuilder, std::ref(keys), std::placeholders::_1)) + .Seal().Build()); + + const bool hasGetSysKeySwitch = bool(FindNode(reduce.Reducer().Body().Ptr(), + [](const TExprNode::TPtr& node) { + return !TYtOutput::Match(node.Get()); + }, + [](const TExprNode::TPtr& node) { + return TCoMember::Match(node.Get()) && node->Head().IsArgument() && node->Tail().IsAtom(YqlSysColumnKeySwitch); + } + )); + + auto input = Build<TCoToFlow>(ctx, reduce.Pos()) + .template Input<TYtTableContent>() + .template Input<TYtReadTable>() + .template World<TCoWorld>().Build() + .template DataSource<TYtDSource>() + .Category(reduce.DataSink().Category()) + .Cluster(reduce.DataSink().Cluster()) + .Build() + .Input(reduce.Input()) + .Build() + .Settings().Build() + .Build() + .Done().Ptr(); + + if constexpr (std::is_same<TYtOperation, TYtMapReduce>::value) { + if (const auto maybeMapper = reduce.Mapper().template Maybe<TCoLambda>()) { + input = Build<TExprApplier>(ctx, reduce.Pos()).Apply(maybeMapper.Cast()).With(0, TExprBase(std::move(input))).Done().Ptr(); + } + } + + auto reducer = Build<TCoLambda>(ctx, reduce.Pos()) + .Args({"list"}) + .Body("list") + .Done(); + + if (hasGetSysKeySwitch) { + reducer = Build<TCoLambda>(ctx, reducer.Pos()) + .Args({"list"}) + .template Body<TCoChain1Map>() + .Input("list") + .InitHandler() + .Args({"first"}) + .template Body<TCoAddMember>() + .Struct("first") + .Name().Value(YqlSysColumnKeySwitch).Build() + .Item(MakeBool<false>(reduce.Pos(), ctx)) + .Build() + .Build() + .UpdateHandler() + .Args({"next", "prev"}) + .template Body<TCoAddMember>() + .Struct("next") + .Name().Value(YqlSysColumnKeySwitch).Build() + .template Item<TCoAggrNotEqual>() + .template Left<TExprApplier>() + .Apply(extract).With(0, "next") + .Build() + .template Right<TExprApplier>() + .Apply(extract).With(0, "prev") + .Build() + .Build() + .Build() + .Build() + .Build() + .Done(); + } + + const auto& items = GetSeqItemType(*reduce.Reducer().Args().Arg(0).Ref().GetTypeAnn()).template Cast<TStructExprType>()->GetItems(); + TExprNode::TListType fields(items.size()); + std::transform(items.cbegin(), items.cend(), fields.begin(), [&](const TItemExprType* item) { return ctx.NewAtom(reduce.Pos(), item->GetName()); }); + auto [placeHolder, lambdaWithPlaceholder] = ReplaceDependsOn(reduce.Reducer().Ptr(), ctx, State_->Types); + reducer = Build<TCoLambda>(ctx, reduce.Pos()) + .Args({"list"}) + .template Body<TExprApplier>() + .Apply(TCoLambda(lambdaWithPlaceholder)) + .template With<TCoExtractMembers>(0) + .template Input<TExprApplier>() + .Apply(reducer) + .With(0, "list") + .Build() + .Members().Add(std::move(fields)).Build() + .Build() + .With(TExprBase(placeHolder), "list") + .Build() + .Done(); + + auto partitionsByKeys = Build<TCoPartitionsByKeys>(ctx, reduce.Pos()) + .Input(std::move(input)) + .KeySelectorLambda(extract) + .SortDirections(std::move(sortDirs)) + .SortKeySelectorLambda(std::move(sortKeys)) + .ListHandlerLambda(std::move(reducer)) + .Done().Ptr(); + + auto [direct, selector] = GetOutputSortSettings(reduce, ctx); + if (direct && selector) { + partitionsByKeys = Build<TCoSort>(ctx, reduce.Pos()) + .Input(std::move(partitionsByKeys)) + .SortDirections(std::move(direct)) + .KeySelectorLambda(std::move(selector)) + .Done().Ptr(); + } + + return Build<TYtTryFirst>(ctx, reduce.Pos()) + .template First<TYtDqProcessWrite>() + .World(std::move(newWorld)) + .DataSink(reduce.DataSink()) + .Output(reduce.Output()) + .template Input<TDqCnResult>() + .Output() + .template Stage<TDqStage>() + .Inputs().Build() + .template Program<TCoLambda>() + .Args({}) + .template Body<TDqWrite>() + .Input(std::move(partitionsByKeys)) + .Provider().Value(YtProviderName).Build() + .template Settings<TCoNameValueTupleList>().Build() + .Build() + .Build() + .Settings(State_->DqHelper->CreateDqStageSettings(true, ctx, reduce.Pos())) + .Build() + .Index().Build(ctx.GetIndexAsString(0), TNodeFlags::Default) + .Build() + .ColumnHints().Build() + .Build() + .Flags().Add(GetHybridFlags(reduce, ctx)).Build() + .Build() + .template Second<TYtOperation>() + .InitFrom(reduce) + .Settings(NYql::AddSetting(reduce.Settings().Ref(), EYtSettingType::NoDq, {}, ctx)) + .Build() + .Done(); + } + + TMaybeNode<TExprBase> TryYtReduceByDq(TExprBase node, TExprContext& ctx) const { + const TStringBuf& nodeName = node.Raw()->Content(); + const auto reduce = node.Cast<TYtReduce>(); + if (CanReplaceOnHybrid(reduce) && CanReadHybrid(reduce.Input().Item(0), nodeName, true) && CanExecuteInHybrid(reduce.Reducer().Ptr(), nodeName, true)) { + if (ETypeAnnotationKind::Struct != GetSeqItemType(*reduce.Reducer().Args().Arg(0).Ref().GetTypeAnn()).GetKind()) { + PushSkipStat("NotStructReducerType", nodeName); + return node; + } + YQL_CLOG(INFO, ProviderYt) << "Rewrite " << nodeName << " node by hybrid"; + PushHybridStat("Try", nodeName); + return MakeYtReduceByDq(reduce, ctx); + } + return node; + } + + TMaybeNode<TExprBase> TryYtMapReduceByDq(TExprBase node, TExprContext& ctx) const { + const TStringBuf& nodeName = node.Raw()->Content(); + const auto mapReduce = node.Cast<TYtMapReduce>(); + if (CanReplaceOnHybrid(mapReduce) && CanReadHybrid(mapReduce.Input().Item(0), nodeName, true) && + CanExecuteInHybrid(mapReduce.Reducer().Ptr(), nodeName, true) && CanExecuteInHybrid(mapReduce.Mapper().Ptr(), nodeName, true)) { + if (ETypeAnnotationKind::Struct != GetSeqItemType(*mapReduce.Reducer().Args().Arg(0).Ref().GetTypeAnn()).GetKind()) { + PushSkipStat("NotStructReducerType", nodeName); + return node; + } + YQL_CLOG(INFO, ProviderYt) << "Rewrite " << nodeName << " node by hybrid"; + PushHybridStat("Try", nodeName); + return MakeYtReduceByDq(mapReduce, ctx); + } + return node; + } + + std::pair<TExprNode::TPtr, TExprNode::TPtr> GetOutputSortSettings(const TYtOutputOpBase& op, TExprContext& ctx) const { + TExprNode::TPtr direct, selector; + if (const auto& sorted = TYtOutTableInfo(op.Output().Item(0)).RowSpec->GetForeignSort(); !sorted.empty()) { + TExprNode::TListType nodes(sorted.size()); + std::transform(sorted.cbegin(), sorted.cend(), nodes.begin(), + [&](const std::pair<TString, bool>& item) { return MakeBool(op.Pos(), item.second, ctx); }); + direct = nodes.size() > 1U ? ctx.NewList(op.Pos(), std::move(nodes)) : std::move(nodes.front()); + nodes.resize(sorted.size()); + auto arg = ctx.NewArgument(op.Pos(), "row"); + std::transform(sorted.cbegin(), sorted.cend(), nodes.begin(), [&](const std::pair<TString, bool>& item) { + return ctx.NewCallable(op.Pos(), "Member", {arg, ctx.NewAtom(op.Pos(), item.first)}); + }); + selector = ctx.NewLambda(op.Pos(), ctx.NewArguments(op.Pos(), {std::move(arg)}), + nodes.size() > 1U ? ctx.NewList(op.Pos(), std::move(nodes)) : std::move(nodes.front())); + } + return std::make_pair(direct, selector); + } + + void PushSkipStat(const TStringBuf& statName, const TStringBuf& nodeName) const { + PushHybridStat(statName, nodeName, "SkipReasons"); + PushHybridStat("Skip", nodeName); + } + + void PushHybridStat(const TStringBuf& statName, const TStringBuf& nodeName, const TStringBuf& folderName = "") const { + with_lock(State_->StatisticsMutex) { + State_->HybridStatistics[folderName].Entries.emplace_back(TString{statName}, 0, 0, 0, 0, 1); + State_->HybridOpStatistics[nodeName][folderName].Entries.emplace_back(TString{statName}, 0, 0, 0, 0, 1); + } + }; + + void PushSettingsToStat(const TExprNode & settings, const TStringBuf& nodeName, const TStringBuf& folderName, EYtSettingTypes types) const { + for (auto& setting: settings.Children()) { + if (setting->ChildrenSize() != 0 && !types.HasFlags(FromString<EYtSettingType>(setting->Child(0)->Content()))) { + PushHybridStat(setting->Child(0)->Content(), nodeName, folderName); + } + } + } + + TExprNode::TListType GetHybridFlags(TExprBase node, TExprContext& ctx) const { + TExprNode::TListType flags; + flags.emplace_back(ctx.NewAtom(node.Pos(), "FallbackOnError", TNodeFlags::Default)); + flags.emplace_back(ctx.NewAtom(node.Pos(), "FallbackOp" + TString(node.Raw()->Content()), TNodeFlags::Default)); + return flags; + } + + const TYtState::TPtr State_; + const THolder<IGraphTransformer> Finalizer_; +}; + +} // namespce + +THolder<IGraphTransformer> CreateYtDqHybridTransformer(TYtState::TPtr state, THolder<IGraphTransformer>&& finalizer) { + return THolder<IGraphTransformer>(new TYtDqHybridTransformer(std::move(state), std::move(finalizer))); +} + +} // namespace NYql diff --git a/yt/yql/providers/yt/provider/yql_yt_dq_integration.cpp b/yt/yql/providers/yt/provider/yql_yt_dq_integration.cpp new file mode 100644 index 0000000000..d45ff380a7 --- /dev/null +++ b/yt/yql/providers/yt/provider/yql_yt_dq_integration.cpp @@ -0,0 +1,942 @@ +#include "yql_yt_dq_integration.h" +#include "yql_yt_table.h" +#include "yql_yt_mkql_compiler.h" +#include "yql_yt_helpers.h" +#include "yql_yt_op_settings.h" +#include "yql_yt_provider_impl.h" + +#include <yt/yql/providers/yt/expr_nodes/yql_yt_expr_nodes.h> +#include <yt/yql/providers/yt/common/yql_configuration.h> +#include <yt/yql/providers/yt/lib/yson_helpers/yson_helpers.h> +#include <yt/yql/providers/yt/proto/source.pb.h> + +#include <yql/essentials/providers/common/dq/yql_dq_integration_impl.h> +#include <yql/essentials/providers/common/codec/yql_codec_type_flags.h> +#include <yql/essentials/providers/common/config/yql_dispatch.h> +#include <yql/essentials/providers/result/expr_nodes/yql_res_expr_nodes.h> +#include <yql/essentials/ast/yql_expr.h> +#include <yql/essentials/core/dq_expr_nodes/dq_expr_nodes.h> +#include <yql/essentials/core/dqs_expr_nodes/dqs_expr_nodes.h> +#include <yql/essentials/core/yql_type_helpers.h> +#include <yql/essentials/core/yql_expr_optimize.h> +#include <yql/essentials/core/yql_opt_utils.h> +#include <yql/essentials/core/services/yql_transform_pipeline.h> +#include <yql/essentials/utils/log/log.h> + +#include <yt/cpp/mapreduce/common/helpers.h> + +#include <library/cpp/iterator/enumerate.h> +#include <library/cpp/yson/node/node_io.h> +#include <library/cpp/yson/node/node.h> + +#include <util/generic/size_literals.h> +#include <util/generic/utility.h> + + +namespace NYql { + +static const THashSet<TStringBuf> UNSUPPORTED_YT_PRAGMAS = {"maxrowweight", "layerpaths", "dockerimage", "operationspec"}; +static const THashSet<TStringBuf> POOL_TREES_WHITELIST = {"physical", "cloud", "cloud_default"}; + +using namespace NNodes; +using namespace NNodes::NDq; + +bool CheckSupportedTypesOld(const TTypeAnnotationNode::TListType& typesToCheck, const TSet<TString>& supportedTypes, const TSet<NUdf::EDataSlot>& supportedDataTypes, std::function<void(const TString&)> unsupportedTypeHandler) { + TSet<ETypeAnnotationKind> supported; + for (const auto &e: supportedTypes) { + if (e == "pg") { + supported.insert(ETypeAnnotationKind::Pg); + } else if (e == "tuple") { + supported.emplace(ETypeAnnotationKind::Tuple); + } else if (e == "struct") { + supported.emplace(ETypeAnnotationKind::Struct); + } else if (e == "dict") { + supported.emplace(ETypeAnnotationKind::Dict); + } else if (e == "list") { + supported.emplace(ETypeAnnotationKind::List); + } else if (e == "variant") { + supported.emplace(ETypeAnnotationKind::Variant); + } else { + // Unknown type + unsupportedTypeHandler(TStringBuilder() << "unknown type: " << e); + return false; + } + } + if (supportedDataTypes.size()) { + supported.emplace(ETypeAnnotationKind::Data); + } + auto checkType = [&] (const TTypeAnnotationNode* type) { + if (type->GetKind() == ETypeAnnotationKind::Data) { + if (!supported.contains(ETypeAnnotationKind::Data)) { + unsupportedTypeHandler(TStringBuilder() << "unsupported data types"); + return false; + } + if (!supportedDataTypes.contains(type->Cast<TDataExprType>()->GetSlot())) { + unsupportedTypeHandler(TStringBuilder() << "unsupported data type: " << type->Cast<TDataExprType>()->GetSlot()); + return false; + } + } else if (type->GetKind() == ETypeAnnotationKind::Pg) { + if (!supported.contains(ETypeAnnotationKind::Pg)) { + unsupportedTypeHandler(TStringBuilder() << "unsupported pg"); + return false; + } + auto name = type->Cast<TPgExprType>()->GetName(); + if (name == "float4" && !supportedDataTypes.contains(NUdf::EDataSlot::Float)) { + unsupportedTypeHandler(TStringBuilder() << "PgFloat4 unsupported yet since float is no supported"); + return false; + } + } else { + unsupportedTypeHandler(TStringBuilder() << "unsupported annotation kind: " << type->GetKind()); + return false; + } + return true; + }; + + TVector<const TTypeAnnotationNode*> stack(typesToCheck.begin(), typesToCheck.end()); + while (!stack.empty()) { + auto el = stack.back(); + stack.pop_back(); + if (el->GetKind() == ETypeAnnotationKind::Optional) { + stack.push_back(el->Cast<TOptionalExprType>()->GetItemType()); + continue; + } + if (!supported.contains(el->GetKind())) { + unsupportedTypeHandler(TStringBuilder() << "unsupported " << el->GetKind()); + return false; + } + if (el->GetKind() == ETypeAnnotationKind::Tuple) { + for (auto e: el->Cast<TTupleExprType>()->GetItems()) { + stack.push_back(e); + } + continue; + } else if (el->GetKind() == ETypeAnnotationKind::Struct) { + for (auto e: el->Cast<TStructExprType>()->GetItems()) { + stack.push_back(e->GetItemType()); + } + continue; + } else if (el->GetKind() == ETypeAnnotationKind::List) { + stack.push_back(el->Cast<TListExprType>()->GetItemType()); + continue; + } else if (el->GetKind() == ETypeAnnotationKind::Dict) { + stack.push_back(el->Cast<TDictExprType>()->GetKeyType()); + stack.push_back(el->Cast<TDictExprType>()->GetPayloadType()); + continue; + } else if (el->GetKind() == ETypeAnnotationKind::Variant) { + stack.push_back(el->Cast<TVariantExprType>()->GetUnderlyingType()); + continue; + } + if (!checkType(el)) { + return false; + } + } + return true; +} + +namespace { + void BlockReaderAddInfo(TExprContext& ctx, const TPosition& pos, const TString& msg) { + ctx.IssueManager.RaiseIssue(YqlIssue(pos, EYqlIssueCode::TIssuesIds_EIssueCode_INFO, "Can't use block reader: " + msg)); + } + + bool CheckBlockReaderSupportedTypes(const TSet<TString>& supportedTypes, const TSet<NUdf::EDataSlot>& supportedDataTypes, const TStructExprType* types, TExprContext& ctx, const TPosition& pos) { + TTypeAnnotationNode::TListType typesToCheck; + for (auto sub: types->GetItems()) { + auto subT = sub->GetItemType(); + typesToCheck.push_back(subT); + } + + return CheckSupportedTypesOld(typesToCheck, supportedTypes, supportedDataTypes, [&ctx, &pos](const TString& msg) { + BlockReaderAddInfo(ctx, pos, msg); + }); + } +}; + +class TYtDqIntegration: public TDqIntegrationBase { +public: + TYtDqIntegration(TYtState* state) + : State_(state) + { + } + + TVector<TVector<ui64>> EstimateColumnStats(TExprContext& ctx, const TString& cluster, const TVector<TVector<TYtPathInfo::TPtr>>& groupIdPathInfos, ui64& sumAllTableSizes) { + TVector<TVector<ui64>> groupIdColumnarStats; + groupIdColumnarStats.reserve(groupIdPathInfos.size()); + TVector<bool> lookupsInfo; + TVector<TYtPathInfo::TPtr> flattenPaths; + for (const auto& pathInfos: groupIdPathInfos) { + for (const auto& pathInfo: pathInfos) { + auto hasLookup = pathInfo->Table->Meta && pathInfo->Table->Meta->Attrs.Value("optimize_for", "scan") == "lookup"; + lookupsInfo.push_back(hasLookup); + if (!pathInfo->Table->Stat) { + continue; + } + if (hasLookup) { + continue; + } + flattenPaths.push_back(pathInfo); + } + } + auto result = EstimateDataSize(cluster, flattenPaths, Nothing(), *State_, ctx); + size_t statIdx = 0; + size_t pathIdx = 0; + for (const auto& [idx, pathInfos]: Enumerate(groupIdPathInfos)) { + TVector<ui64> columnarStatInner; + columnarStatInner.reserve(pathInfos.size()); + for (auto& path: pathInfos) { + const auto& tableInfo = *path->Table; + if (lookupsInfo[pathIdx++] || !tableInfo.Stat) { + columnarStatInner.push_back(tableInfo.Stat ? tableInfo.Stat->DataSize : 0); + sumAllTableSizes += columnarStatInner.back(); + continue; + } + columnarStatInner.push_back(result ? result->at(statIdx) : tableInfo.Stat->DataSize); + sumAllTableSizes += columnarStatInner.back(); + ++statIdx; + } + groupIdColumnarStats.emplace_back(std::move(columnarStatInner)); + } + return groupIdColumnarStats; + } + + ui64 Partition(const TExprNode& node, TVector<TString>& serializedPartitions, TString* clusterName, TExprContext& ctx, const TPartitionSettings& settings) override + { + YQL_ENSURE(settings.DataSizePerJob.Defined()); + ui64 dataSizePerJob = *settings.DataSizePerJob; + if (!TMaybeNode<TYtReadTable>(&node).IsValid()) { + return 0; + } + + const auto ytRead = TYtReadTable(&node); + const auto cluster = ytRead.DataSource().Cluster().StringValue(); + + if (clusterName) { + *clusterName = cluster; + } + + TVector<TVector<TYtPathInfo::TPtr>> groupIdPathInfos; + bool hasErasure = false; + ui64 chunksCount = 0; + for (const auto& input : ytRead.Input()) { + TVector<TYtPathInfo::TPtr> pathInfos; + for (const auto& path : input.Paths()) { + TYtPathInfo::TPtr pathInfo = new TYtPathInfo(path); + pathInfos.push_back(pathInfo); + if (pathInfo->Table->Meta && pathInfo->Table->Meta->Attrs.Value("erasure_codec", "none") != "none") { + hasErasure = true; + } + chunksCount += pathInfo->Table->Stat->ChunkCount; + } + groupIdPathInfos.push_back(pathInfos); + } + + if (auto maxChunks = State_->Configuration->MaxChunksForDqRead.Get().GetOrElse(DEFAULT_MAX_CHUNKS_FOR_DQ_READ); settings.CanFallback && chunksCount > maxChunks) { + throw TFallbackError() << DqFallbackErrorMessageWrap( TStringBuilder() << "table with too many chunks: " << chunksCount << " > " << maxChunks); + } + + if (hasErasure) { + if (auto codecCpu = State_->Configuration->ErasureCodecCpuForDq.Get(cluster)) { + dataSizePerJob = Max(ui64(dataSizePerJob / *codecCpu), 10_KB); + } else { + hasErasure = false; + } + } + + auto maxTasks = settings.MaxPartitions; + ui64 maxDataSizePerJob = 0; + if (State_->Configuration->_EnableYtPartitioning.Get(cluster).GetOrElse(false)) { + TVector<TYtPathInfo::TPtr> paths; + TVector<TString> keys; + TMaybe<double> sample; + for (const auto& [groupId, pathInfos] : Enumerate(groupIdPathInfos)) { + if (auto sampleSetting = GetSetting(ytRead.Input().Item(groupId).Settings().Ref(), EYtSettingType::Sample)) { + sample = FromString<double>(sampleSetting->Child(1)->Child(1)->Content()); + } + for (const auto& [pathId, pathInfo] : Enumerate(pathInfos)) { + auto tableName = pathInfo->Table->Name; + if (pathInfo->Table->IsAnonymous && !TYtTableInfo::HasSubstAnonymousLabel(pathInfo->Table->FromNode.Cast())) { + tableName = State_->AnonymousLabels.Value(std::make_pair(cluster, tableName), TString()); + YQL_ENSURE(tableName, "Unaccounted anonymous table: " << pathInfo->Table->Name); + pathInfo->Table->Name = tableName; + } + + paths.push_back(pathInfo); + keys.emplace_back(TStringBuilder() << groupId << "/" << pathId); + } + } + if (sample && *sample > 0) { + dataSizePerJob /= *sample; + } + + auto res = State_->Gateway->GetTablePartitions(NYql::IYtGateway::TGetTablePartitionsOptions(State_->SessionId) + .Cluster(cluster) + .MaxPartitions(maxTasks) + .DataSizePerJob(dataSizePerJob) + .AdjustDataWeightPerPartition(!settings.CanFallback) + .Config(State_->Configuration->Snapshot()) + .Paths(std::move(paths))); + if (!res.Success()) { + const auto message = DqFallbackErrorMessageWrap("failed to partition table"); + YQL_CLOG(ERROR, ProviderDq) << message; + auto issue = YqlIssue(TPosition(), TIssuesIds::DQ_GATEWAY_NEED_FALLBACK_ERROR, message); + for (auto& subIssue: res.Issues()) { + issue.AddSubIssue(MakeIntrusive<TIssue>(subIssue)); + } + + if (settings.CanFallback) { + throw TFallbackError(MakeIntrusive<TIssue>(std::move(issue))) << message; + } else { + ctx.IssueManager.RaiseIssue(issue); + throw TErrorException(TIssuesIds::DQ_GATEWAY_NEED_FALLBACK_ERROR) << message; + } + } + + serializedPartitions.reserve(res.Partitions.Partitions.size()); + for (const auto& partition : res.Partitions.Partitions) { + NYT::TNode part = NYT::TNode::CreateMap(); + for (const auto& [pathId, path]: Enumerate(partition.TableRanges)) { + // n.b. we're expecting YT API to return ranges in the same order as they were passed + part[keys[pathId]] = NYT::PathToNode(path); + } + serializedPartitions.push_back(NYT::NodeToYsonString(part)); + YQL_CLOG(TRACE, ProviderDq) << "Partition: " << NYT::NodeToYsonString(part, ::NYson::EYsonFormat::Pretty); + } + } else { + TVector<TVector<std::tuple<ui64, ui64, NYT::TRichYPath>>> partitionTuplesArr; + ui64 sumAllTableSizes = 0; + TVector<TVector<ui64>> groupIdColumnarStats = EstimateColumnStats(ctx, cluster, {groupIdPathInfos}, sumAllTableSizes); + ui64 parts = (sumAllTableSizes + dataSizePerJob - 1) / dataSizePerJob; + if (settings.CanFallback && hasErasure && parts > maxTasks) { + auto message = DqFallbackErrorMessageWrap("too big table with erasure codec"); + YQL_CLOG(INFO, ProviderDq) << message; + throw TFallbackError() << message; + } + parts = Min<ui64>(parts, maxTasks); + parts = Max<ui64>(parts, 1); + partitionTuplesArr.resize(parts); + serializedPartitions.resize(parts); + + for (const auto& [groupId, input] : Enumerate(ytRead.Input())) { + TMaybe<double> sample; + auto sampleSetting = GetSetting(input.Settings().Ref(), EYtSettingType::Sample); + if (sampleSetting) { + sample = FromString<double>(sampleSetting->Child(1)->Child(1)->Content()); + } + auto& groupStats = groupIdColumnarStats[groupId]; + for (const auto& [pathId, path] : Enumerate(groupIdPathInfos[groupId])) { + const auto& tableInfo = *path->Table; + YQL_ENSURE(tableInfo.Stat, "Table has no stat."); + ui64 dataSize = groupStats[pathId]; + if (sample) { + dataSize *=* sample; + } + maxDataSizePerJob = Max(maxDataSizePerJob, (dataSize + parts - 1) / parts); + ui64 rowsPerPart = (tableInfo.Stat->RecordsCount + parts - 1) / parts; + for (ui64 from = 0, i = 0; from < tableInfo.Stat->RecordsCount; from += rowsPerPart, i++) { + ui64 to = Min(from + rowsPerPart, tableInfo.Stat->RecordsCount); + NYT::TRichYPath path; + path.AddRange(NYT::TReadRange::FromRowIndices(from, to)); + partitionTuplesArr[i].push_back({groupId, pathId, path}); + } + } + } + int i = 0; + for (const auto& partitionTuples: partitionTuplesArr) { + TStringStream out; + NYson::TYsonWriter writer((IOutputStream*)&out); + writer.OnBeginMap(); + for (const auto& partitionTuple : partitionTuples) { + writer.OnKeyedItem(TStringBuilder() << std::get<0>(partitionTuple) << "/" << std::get<1>(partitionTuple)); + writer.OnRaw(NYT::NodeToYsonString(NYT::PathToNode(std::get<2>(partitionTuple)))); + } + writer.OnEndMap(); + serializedPartitions[i++] = out.Str(); + } + } + return maxDataSizePerJob; + } + + void AddMessage(TExprContext& ctx, const TString& message, bool skipIssues, bool riseError) { + if (skipIssues && !riseError) { + return; + } + + TIssue issue(DqFallbackErrorMessageWrap(message)); + if (riseError) { + YQL_CLOG(ERROR, ProviderDq) << message; + issue.Severity = TSeverityIds::S_ERROR; + } else { + YQL_CLOG(INFO, ProviderDq) << message; + issue.Severity = TSeverityIds::S_INFO; + } + ctx.IssueManager.RaiseIssue(issue); + } + + void AddInfo(TExprContext& ctx, const TString& message, bool skipIssues) { + AddMessage(ctx, message, skipIssues, false); + } + + bool CheckPragmas(const TExprNode& node, TExprContext& ctx, bool skipIssues) override { + if (TYtConfigure::Match(&node)) { + if (node.ChildrenSize() >= 5) { + if (node.Child(2)->Content() == "Attr" && node.Child(3)->Content() == "maxrowweight") { + if (FromString<NSize::TSize>(node.Child(4)->Content()).GetValue()>NSize::FromMegaBytes(128)) { + State_->OnlyNativeExecution = true; + return false; + } else { + return true; + } + } + } + + if (node.ChildrenSize() >= 4 && node.Child(2)->Content() == "Attr") { + auto pragma = node.Child(3)->Content(); + if (UNSUPPORTED_YT_PRAGMAS.contains(pragma)) { + AddInfo(ctx, TStringBuilder() << "unsupported yt pragma: " << pragma, skipIssues); + State_->OnlyNativeExecution = true; + return false; + } + + if (pragma == "pooltrees") { + auto pools = NPrivate::GetDefaultParser<TVector<TString>>()(TString{node.Child(4)->Content()}); + for (const auto& pool : pools) { + if (!POOL_TREES_WHITELIST.contains(pool)) { + AddInfo(ctx, TStringBuilder() << "unsupported pool tree: " << pool, skipIssues); + State_->OnlyNativeExecution = true; + return false; + } + } + } + } + } + return true; + } + + bool CanRead(const TExprNode& node, TExprContext& ctx, bool skipIssues) override { + if (TYtConfigure::Match(&node)) { + return CheckPragmas(node, ctx, skipIssues); + } else if (auto maybeRead = TMaybeNode<TYtReadTable>(&node)) { + auto cluster = maybeRead.Cast().DataSource().Cluster().StringValue(); + if (!State_->Configuration->_EnableDq.Get(cluster).GetOrElse(true)) { + AddMessage(ctx, TStringBuilder() << "disabled for cluster " << cluster, skipIssues, State_->PassiveExecution); + return false; + } + const auto canUseYtPartitioningApi = State_->Configuration->_EnableYtPartitioning.Get(cluster).GetOrElse(false); + ui64 chunksCount = 0ull; + for (auto section: maybeRead.Cast().Input()) { + if (HasSettingsExcept(maybeRead.Cast().Input().Item(0).Settings().Ref(), DqReadSupportedSettings) || HasNonEmptyKeyFilter(maybeRead.Cast().Input().Item(0))) { + TStringBuilder info; + info << "unsupported path settings: "; + if (maybeRead.Cast().Input().Item(0).Settings().Size() > 0) { + for (auto& setting : maybeRead.Cast().Input().Item(0).Settings().Ref().Children()) { + if (setting->ChildrenSize() != 0) { + info << setting->Child(0)->Content() << ","; + } + } + } + AddMessage(ctx, info, skipIssues, State_->PassiveExecution); + return false; + } + for (auto path: section.Paths()) { + if (!path.Table().Maybe<TYtTable>()) { + AddMessage(ctx, "non-table path", skipIssues, State_->PassiveExecution); + return false; + } else { + auto pathInfo = TYtPathInfo(path); + auto tableInfo = pathInfo.Table; + auto epoch = TEpochInfo::Parse(path.Table().Maybe<TYtTable>().CommitEpoch().Ref()); + if (!tableInfo->Stat) { + AddMessage(ctx, "table without statistics", skipIssues, State_->PassiveExecution); + return false; + } else if (!tableInfo->RowSpec) { + AddMessage(ctx, "table without row spec", skipIssues, State_->PassiveExecution); + return false; + } else if (!tableInfo->Meta) { + AddMessage(ctx, "table without meta", skipIssues, State_->PassiveExecution); + return false; + } else if (tableInfo->IsAnonymous) { + AddMessage(ctx, "anonymous table", skipIssues, State_->PassiveExecution); + return false; + } else if ((!epoch.Empty() && *epoch.Get() > 0)) { + AddMessage(ctx, "table with non-empty epoch", skipIssues, State_->PassiveExecution); + return false; + } else if (NYql::HasSetting(tableInfo->Settings.Ref(), EYtSettingType::WithQB)) { + AddMessage(ctx, "table with QB2 premapper", skipIssues, State_->PassiveExecution); + return false; + } else if (pathInfo.Ranges && !canUseYtPartitioningApi) { + AddMessage(ctx, "table with ranges", skipIssues, State_->PassiveExecution); + return false; + } else if (tableInfo->Meta->IsDynamic && !canUseYtPartitioningApi) { + AddMessage(ctx, "dynamic table", skipIssues, State_->PassiveExecution); + return false; + } + + chunksCount += tableInfo->Stat->ChunkCount; + } + } + } + if (auto maxChunks = State_->Configuration->MaxChunksForDqRead.Get().GetOrElse(DEFAULT_MAX_CHUNKS_FOR_DQ_READ); chunksCount > maxChunks) { + AddMessage(ctx, TStringBuilder() << "table with too many chunks: " << chunksCount << " > " << maxChunks, skipIssues, State_->PassiveExecution); + return false; + } + return true; + } + AddInfo(ctx, TStringBuilder() << "unsupported callable: " << node.Content(), skipIssues); + return false; + } + + bool CanBlockRead(const NNodes::TExprBase& node, TExprContext& ctx, TTypeAnnotationContext&) override { + auto wrap = node.Cast<TDqReadWideWrap>(); + auto maybeRead = wrap.Input().Maybe<TYtReadTable>(); + if (!maybeRead) { + return false; + } + + if (!State_->Configuration->UseRPCReaderInDQ.Get(maybeRead.Cast().DataSource().Cluster().StringValue()).GetOrElse(DEFAULT_USE_RPC_READER_IN_DQ)) { + return false; + } + + auto supportedTypes = State_->Configuration->BlockReaderSupportedTypes.Get(maybeRead.Cast().DataSource().Cluster().StringValue()).GetOrElse(DEFAULT_BLOCK_READER_SUPPORTED_TYPES); + auto supportedDataTypes = State_->Configuration->BlockReaderSupportedDataTypes.Get(maybeRead.Cast().DataSource().Cluster().StringValue()).GetOrElse(DEFAULT_BLOCK_READER_SUPPORTED_DATA_TYPES); + const auto structType = GetSeqItemType(maybeRead.Raw()->GetTypeAnn()->Cast<TTupleExprType>()->GetItems().back())->Cast<TStructExprType>(); + if (!CheckBlockReaderSupportedTypes(supportedTypes, supportedDataTypes, structType, ctx, ctx.GetPosition(node.Pos()))) { + return false; + } + + TVector<const TTypeAnnotationNode*> subTypeAnn(Reserve(structType->GetItems().size())); + for (const auto& type: structType->GetItems()) { + subTypeAnn.emplace_back(type->GetItemType()); + } + + if (!State_->Types->ArrowResolver) { + BlockReaderAddInfo(ctx, ctx.GetPosition(node.Pos()), "no arrow resolver provided"); + return false; + } + + if (State_->Types->ArrowResolver->AreTypesSupported(ctx.GetPosition(node.Pos()), subTypeAnn, ctx) != IArrowResolver::EStatus::OK) { + BlockReaderAddInfo(ctx, ctx.GetPosition(node.Pos()), "arrow resolver don't support these types"); + return false; + } + + const TYtSectionList& sectionList = wrap.Input().Cast<TYtReadTable>().Input(); + for (size_t i = 0; i < sectionList.Size(); ++i) { + auto section = sectionList.Item(i); + auto paths = section.Paths(); + for (const auto& path : section.Paths()) { + if (!IsYtTableSuitableForArrowInput(path.Table(), [&ctx, &node](const TString& msg) { + BlockReaderAddInfo(ctx, ctx.GetPosition(node.Pos()), msg); + })) { + return false; + } + } + if (!NYql::GetSettingAsColumnList(section.Settings().Ref(), EYtSettingType::SysColumns).empty()) { + BlockReaderAddInfo(ctx, ctx.GetPosition(node.Pos()), "system column"); + return false; + } + } + return true; + } + + TMaybe<TOptimizerStatistics> ReadStatistics(const TExprNode::TPtr& read, TExprContext& ctx) override { + Y_UNUSED(ctx); + TOptimizerStatistics stat; + if (auto maybeRead = TMaybeNode<TYtReadTable>(read)) { + auto input = maybeRead.Cast().Input(); + for (auto section: input) { + for (const auto& path: section.Paths()) { + auto pathInfo = MakeIntrusive<TYtPathInfo>(path); + auto tableInfo = pathInfo->Table; + YQL_ENSURE(tableInfo); + + if (tableInfo->Stat) { + stat.Nrows += tableInfo->Stat->RecordsCount; + } + if (pathInfo->Columns && pathInfo->Columns->GetColumns()) { + stat.Ncols += pathInfo->Columns->GetColumns()->size(); + } + } + } + } + stat.Cost = stat.Nrows * std::max(stat.Ncols, 1); + return stat; + } + + TMaybe<ui64> EstimateReadSize(ui64 dataSizePerJob, ui32 maxTasksPerStage, const TVector<const TExprNode*>& nodes, TExprContext& ctx) override { + TVector<bool> hasErasurePerNode; + hasErasurePerNode.reserve(nodes.size()); + TVector<ui64> dataSizes(nodes.size()); + THashMap<TString, TVector<std::pair<const TExprNode*, bool>>> clusterToNodesAndErasure; + THashMap<TString, TVector<TVector<TYtPathInfo::TPtr>>> clusterToGroups; + const auto maxChunks = State_->Configuration->MaxChunksForDqRead.Get().GetOrElse(DEFAULT_MAX_CHUNKS_FOR_DQ_READ); + ui64 chunksCount = 0u; + + for (const auto &node_: nodes) { + if (auto maybeRead = TMaybeNode<TYtReadTable>(node_)) { + + bool hasErasure = false; + auto cluster = maybeRead.Cast().DataSource().Cluster().StringValue(); + auto& groupIdPathInfo = clusterToGroups[cluster]; + + const auto canUseYtPartitioningApi = State_->Configuration->_EnableYtPartitioning.Get(cluster).GetOrElse(false); + + auto input = maybeRead.Cast().Input(); + for (auto section: input) { + groupIdPathInfo.emplace_back(); + for (const auto& path: section.Paths()) { + auto pathInfo = MakeIntrusive<TYtPathInfo>(path); + auto tableInfo = pathInfo->Table; + + YQL_ENSURE(tableInfo); + + if (pathInfo->Ranges && !canUseYtPartitioningApi) { + AddErrorWrap(ctx, node_->Pos(), "table with ranges"); + return Nothing(); + } else if (tableInfo->Meta->IsDynamic && !canUseYtPartitioningApi) { + AddErrorWrap(ctx, node_->Pos(), "dynamic table"); + return Nothing(); + } else { // + if (tableInfo->Meta->Attrs.Value("erasure_codec", "none") != "none") { + hasErasure = true; + } + if (tableInfo->Stat) { + chunksCount += tableInfo->Stat->ChunkCount; + if (chunksCount > maxChunks) { + AddErrorWrap(ctx, node_->Pos(), TStringBuilder() << "table with too many chunks: " << chunksCount << " > " << maxChunks); + return Nothing(); + } + } + } + groupIdPathInfo.back().emplace_back(pathInfo); + } + } + clusterToNodesAndErasure[cluster].push_back({node_, hasErasure}); + } else { + AddErrorWrap(ctx, node_->Pos(), TStringBuilder() << "unsupported callable: " << node_->Content()); + return Nothing(); + } + } + ui64 dataSize = 0; + for (auto& [cluster, info]: clusterToNodesAndErasure) { + auto res = EstimateColumnStats(ctx, cluster, clusterToGroups[cluster], dataSize); + auto codecCpu = State_->Configuration->ErasureCodecCpuForDq.Get(cluster); + if (!codecCpu) { + continue; + } + size_t idx = 0; + for (auto& [node, hasErasure]: info) { + if (!hasErasure) { + ++idx; + continue; + } + ui64 readSize = std::accumulate(res[idx].begin(), res[idx].end(), 0); + ++idx; + dataSizePerJob = Max(ui64(dataSizePerJob / *codecCpu), 10_KB); + const ui64 parts = (readSize + dataSizePerJob - 1) / dataSizePerJob; + if (parts > maxTasksPerStage) { + AddErrorWrap(ctx, node->Pos(), "too big table with erasure codec"); + return Nothing(); + } + } + } + return dataSize; + } + + void AddErrorWrap(TExprContext& ctx, const NYql::TPositionHandle& where, const TString& cause) { + ctx.AddError(YqlIssue(ctx.GetPosition(where), TIssuesIds::DQ_OPTIMIZE_ERROR, DqFallbackErrorMessageWrap(cause))); + } + + TExprNode::TPtr WrapRead(const TExprNode::TPtr& read, TExprContext& ctx, const TWrapReadSettings&) override { + if (auto maybeYtReadTable = TMaybeNode<TYtReadTable>(read)) { + TMaybeNode<TCoSecureParam> secParams; + const auto cluster = maybeYtReadTable.Cast().DataSource().Cluster(); + if (State_->Configuration->Auth.Get().GetOrElse(TString()) || State_->Configuration->Tokens.Value(cluster, "")) { + secParams = Build<TCoSecureParam>(ctx, read->Pos()).Name().Build(TString("cluster:default_").append(cluster)).Done(); + } + return Build<TDqReadWrap>(ctx, read->Pos()) + .Input(maybeYtReadTable.Cast()) + .Flags().Build() + .Token(secParams) + .Done().Ptr(); + } + return read; + } + + TExprNode::TPtr RecaptureWrite(const TExprNode::TPtr& write, TExprContext& ctx) override { + if (auto maybeWrite = TMaybeNode<TYtWriteTable>(write)) { + if (State_->Configuration->_EnableYtDqProcessWriteConstraints.Get().GetOrElse(DEFAULT_ENABLE_DQ_WRITE_CONSTRAINTS)) { + const auto& content = maybeWrite.Cast().Content(); + if (TYtMaterialize::Match(&SkipCallables(content.Ref(), {TCoSort::CallableName(), TCoTopSort::CallableName(), TCoAssumeSorted::CallableName(), TCoAssumeConstraints::CallableName()}))) { + return write; + } + TExprNode::TPtr newContent; + const auto materializeWorld = ctx.NewWorld(write->Pos()); // TODO: maybeWrite.Cast().World() + if (content.Maybe<TCoAssumeSorted>()) { + // Duplicate AssumeSorted before YtMaterialize, because DQ cannot keep sort and so optimizes AssumeSorted as complete Sort + // Before: YtWrite -> AssumeSorted -> ... + // After: YtWrite -> AssumeConstraints -> YtMaterialize -> AssumeSorted -> ... + newContent = Build<TYtMaterialize>(ctx, content.Pos()) + .World(materializeWorld) + .DataSink(maybeWrite.Cast().DataSink()) + .Input(content) + .Settings().Build() + .Done().Ptr(); + } else if (content.Raw()->IsCallable({TCoSort::CallableName(), TCoTopSort::CallableName()}) && !content.Raw()->GetConstraint<TSortedConstraintNode>()) { + // For Sorts by non members lambdas do it on YT side because of aux columns + // Before: YtWrite -> Sort/TopSort -> ... + // After: YtWrite -> Sort/TopSort -> YtMaterialize -> ... + auto materialize = Build<TYtMaterialize>(ctx, content.Pos()) + .World(materializeWorld) + .DataSink(maybeWrite.Cast().DataSink()) + .Input(content.Cast<TCoInputBase>().Input()) + .Settings().Build() + .Done(); + newContent = ctx.ChangeChild(content.Ref(), TCoInputBase::idx_Input, materialize.Ptr()); + } else { + // Materialize dq graph to yt table before YtWrite: + // Before: YtWrite -> Some callables ... + // After: YtWrite -> YtMaterialize -> Some callables ... + newContent = Build<TYtMaterialize>(ctx, content.Pos()) + .World(materializeWorld) + .DataSink(maybeWrite.Cast().DataSink()) + .Input(content) + .Settings().Build() + .Done().Ptr(); + } + if (content.Raw()->GetConstraint<TSortedConstraintNode>() || content.Raw()->GetConstraint<TDistinctConstraintNode>() || content.Raw()->GetConstraint<TUniqueConstraintNode>()) { + newContent = Build<TCoAssumeConstraints>(ctx, content.Pos()) + .Input(newContent) + .Value() + .Value(NYT::NodeToYsonString(content.Raw()->GetConstraintSet().ToYson(), NYson::EYsonFormat::Text), TNodeFlags::MultilineContent) + .Build() + .Done().Ptr(); + } + return Build<TYtWriteTable>(ctx, write->Pos()) + .InitFrom(maybeWrite.Cast()) + .Content(newContent) + .Done().Ptr(); + } + } + return write; + } + + void FillLookupSourceSettings(const TExprNode& node, ::google::protobuf::Any& settings, TString& sourceType) override { + const TDqLookupSourceWrap wrap(&node); + auto table = wrap.Input().Cast<TYtTable>(); + TYtTableBaseInfo::TPtr tableInfo{TYtTableBaseInfo::Parse(table)}; + auto codecSpec = tableInfo->GetCodecSpecNode({}); + TString rowSpec = NodeToYsonString(codecSpec, NYT::NYson::EYsonFormat::Text); + + NYt::NSource::TLookupSource source; + source.SetCluster(table.Cluster().StringValue()); + source.SetTable(table.Name().StringValue()); + source.SetRowSpec(rowSpec); + YQL_CLOG(INFO, ProviderYt) + << "Filling lookup source settings" + << ": cluster: " << source.cluster() + << ", table: " << source.table() + << ", RowSpec: " << rowSpec + ; + settings.PackFrom(source); + sourceType = "yt"; + } + + TMaybe<bool> CanWrite(const TExprNode& node, TExprContext& ctx) override { + if (auto maybeWrite = TMaybeNode<TYtWriteTable>(&node)) { + auto cluster = TString{maybeWrite.Cast().DataSink().Cluster().Value()}; + auto tableName = TString{TYtTableInfo::GetTableLabel(maybeWrite.Cast().Table())}; + auto epoch = TEpochInfo::Parse(maybeWrite.Cast().Table().CommitEpoch().Ref()); + + auto tableDesc = State_->TablesData->GetTable(cluster, tableName, epoch); + + if (!State_->Configuration->_EnableDq.Get(cluster).GetOrElse(true)) { + AddInfo(ctx, TStringBuilder() << "disabled for cluster " << cluster, false); + return false; + } + + if (!tableDesc.Meta) { + AddInfo(ctx, "write to table without meta", false); + return false; + } + if (tableDesc.Meta->IsDynamic) { + AddInfo(ctx, "write to dynamic table", false); + return false; + } + + if (!State_->Configuration->_EnableYtDqProcessWriteConstraints.Get().GetOrElse(DEFAULT_ENABLE_DQ_WRITE_CONSTRAINTS)) { + const auto content = maybeWrite.Cast().Content().Raw(); + if (const auto sorted = content->GetConstraint<TSortedConstraintNode>()) { + if (const auto distinct = content->GetConstraint<TDistinctConstraintNode>()) { + if (distinct->IsOrderBy(*sorted)) { + AddInfo(ctx, "unsupported write of unique data", false); + return false; + } + } + if (!content->IsCallable({"Sort", "TopSort", "AssumeSorted"})) { + AddInfo(ctx, "unsupported write of sorted data", false); + return false; + } + } + } + return true; + } + + return Nothing(); + } + + void RegisterMkqlCompiler(NCommon::TMkqlCallableCompilerBase& compiler) override { + RegisterDqYtMkqlCompilers(compiler, State_); + State_->Gateway->RegisterMkqlCompiler(compiler); + } + + bool CanFallback() override { + return true; + } + + void Annotate(const TExprNode& node, THashMap<TString, TString>& params) override { + if (!node.IsCallable("YtDqWideWrite")) { + return; + } + + YQL_ENSURE(!params.contains("yt.write"), "Duplicate 'yt.write' graph parameter"); + + TString server; + TString tx; + TString token; + + for (const auto& setting: node.Child(1)->Children()) { + if (setting->ChildrenSize() != 2) { + continue; + } + + if (setting->Child(0)->IsAtom("server")) { + server = setting->Child(1)->Content(); + } else if (setting->Child(0)->IsAtom("tx")) { + tx = setting->Child(1)->Content(); + } else if (setting->Child(0)->IsAtom("secureParams")) { + if (setting->ChildrenSize() > 1) { + TCoSecureParam secure(setting->Child(1)); + token = secure.Name().StringValue(); + } + } + } + YQL_ENSURE(server, "YtDqWideWrite: server parameter is expected"); + YQL_ENSURE(tx, "YtDqWideWrite: tx parameter is expected"); + + auto param = NYT::NodeToYsonString(NYT::TNode()("root_tx", tx)("server", server)("token", token)); + params["yt.write"] = param; + YQL_CLOG(INFO, ProviderYt) << "DQ annotate: adding yt.write=" << param; + } + + bool PrepareFullResultTableParams(const TExprNode& root, TExprContext& ctx, THashMap<TString, TString>& params, THashMap<TString, TString>& secureParams) override { + const auto resOrPull = TResOrPullBase(&root); + + if (FromString<bool>(resOrPull.Discard().Value())) { + return false; + } + + auto input = resOrPull.Input().Ptr(); + std::set<std::string_view> usedClusters; + VisitExpr(*input, [&usedClusters](const TExprNode& node) { + if (auto ds = TMaybeNode<TYtDSource>(&node)) { + usedClusters.insert(ds.Cast().Cluster().Value()); + return false; + } + if (auto ds = TMaybeNode<TYtDSink>(&node)) { + usedClusters.insert(ds.Cast().Cluster().Value()); + return false; + } + return true; + }); + TString cluster; + if (usedClusters.empty()) { + cluster = State_->Configuration->DefaultCluster.Get().GetOrElse(State_->Gateway->GetDefaultClusterName()); + } else { + cluster = TString{*usedClusters.begin()}; + } + + const auto type = GetSequenceItemType(input->Pos(), input->GetTypeAnn(), false, ctx); + YQL_ENSURE(type); + TYtOutTableInfo outTableInfo(type->Cast<TStructExprType>(), State_->Configuration->UseNativeYtTypes.Get().GetOrElse(DEFAULT_USE_NATIVE_YT_TYPES) ? NTCF_ALL : NTCF_NONE); + + const auto res = State_->Gateway->PrepareFullResultTable( + IYtGateway::TFullResultTableOptions(State_->SessionId) + .Cluster(cluster) + .Config(State_->Configuration->GetSettingsForNode(resOrPull.Origin().Ref())) + .OutTable(outTableInfo) + ); + + auto param = NYT::TNode()("cluster", cluster)("server", res.Server)("path", res.Path)("refName", res.RefName)("codecSpec", res.CodecSpec)("tableAttrs", res.TableAttrs); + if (res.RootTransactionId) { + param("root_tx", *res.RootTransactionId); + if (res.ExternalTransactionId) { + param("external_tx", *res.ExternalTransactionId); + } + } else if (auto externalTx = State_->Configuration->ExternalTx.Get().GetOrElse(TGUID())) { + param("external_tx", GetGuidAsString(externalTx)); + } + TString tokenName; + if (auto auth = State_->Configuration->Auth.Get().GetOrElse(TString())) { + tokenName = TString("cluster:default_").append(cluster); + if (!secureParams.contains(tokenName)) { + secureParams[tokenName] = auth; + } + } + param("token", tokenName); + + const auto strParam = NYT::NodeToYsonString(param); + params["yt.full_result_table"] = strParam; + YQL_CLOG(INFO, ProviderYt) << "DQ prepare full result table params: adding yt.full_result_table=" << strParam; + return true; + } + + void WriteFullResultTableRef(NYson::TYsonWriter& writer, const TVector<TString>& columns, const THashMap<TString, TString>& graphParams) override { + auto p = graphParams.FindPtr("yt.full_result_table"); + YQL_ENSURE(p, "Expected 'yt.full_result_table' parameter"); + auto param = NYT::NodeFromYsonString(*p); + const auto cluster = param["cluster"]; + YQL_ENSURE(cluster.IsString(), "Expected 'cluster' sub-parameter"); + const auto refName = param["refName"]; + YQL_ENSURE(refName.IsString(), "Expected 'refName' sub-parameter"); + NYql::WriteTableReference(writer, YtProviderName, cluster.AsString(), refName.AsString(), true, columns); + } + + virtual void ConfigurePeepholePipeline(bool beforeDqTransforms, const THashMap<TString, TString>& providerParams, TTransformationPipeline* pipeline) override { + if (!beforeDqTransforms) { + return; + } + + auto state = TYtState::TPtr(State_); + pipeline->Add(CreateFunctorTransformer([state](TExprNode::TPtr input, TExprNode::TPtr& output, TExprContext& ctx) { + return OptimizeExpr(input, output, [&](const TExprNode::TPtr& node, TExprContext& ctx) -> TExprNode::TPtr { + if (TYtReadTable::Match(node.Get()) && !node->Head().IsWorld()) { + YQL_CLOG(INFO, ProviderYt) << "Peephole-YtTrimWorld"; + return ctx.ChangeChild(*node, 0, ctx.NewWorld(node->Pos())); + } + return node; + }, ctx, TOptimizeExprSettings{state->Types}); + }), "YtTrimWorld", TIssuesIds::DEFAULT_ERROR); + + pipeline->Add(CreateSinglePassFunctorTransformer([state, providerParams](TExprNode::TPtr input, TExprNode::TPtr& output, TExprContext& ctx) { + output = input; + auto status = SubstTables(output, state, false, ctx); + if (status.Level != IGraphTransformer::TStatus::Error && input != output) { + YQL_CLOG(INFO, ProviderYt) << "Peephole-YtSubstTables"; + } + return status; + }), "YtSubstTables", TIssuesIds::DEFAULT_ERROR); + + pipeline->Add(CreateYtPeepholeTransformer(TYtState::TPtr(State_), providerParams), "YtPeepHole", TIssuesIds::DEFAULT_ERROR); + } + + static TString DqFallbackErrorMessageWrap(const TString& message) { + return "DQ cannot execute the query. Cause: " + message; + } + +private: + TYtState* State_; +}; + +THolder<IDqIntegration> CreateYtDqIntegration(TYtState* state) { + Y_ABORT_UNLESS(state); + return MakeHolder<TYtDqIntegration>(state); +} + +} diff --git a/yt/yql/providers/yt/provider/yql_yt_dq_integration.h b/yt/yql/providers/yt/provider/yql_yt_dq_integration.h new file mode 100644 index 0000000000..9fce4842cf --- /dev/null +++ b/yt/yql/providers/yt/provider/yql_yt_dq_integration.h @@ -0,0 +1,16 @@ +#pragma once + +#include "yql_yt_provider.h" + +#include <yql/essentials/core/dq_integration/yql_dq_integration.h> + +#include <util/generic/ptr.h> + +namespace NYql { + +THolder<IDqIntegration> CreateYtDqIntegration(TYtState* state); + +// TODO move to yql/core +bool CheckSupportedTypesOld(const TTypeAnnotationNode::TListType& typesToCheck, const TSet<TString>& supportedTypes, const TSet<NUdf::EDataSlot>& supportedDataTypes, std::function<void(const TString&)> unsupportedTypeHandler); + +} diff --git a/yt/yql/providers/yt/provider/yql_yt_dq_optimize.cpp b/yt/yql/providers/yt/provider/yql_yt_dq_optimize.cpp new file mode 100644 index 0000000000..c0e1dace64 --- /dev/null +++ b/yt/yql/providers/yt/provider/yql_yt_dq_optimize.cpp @@ -0,0 +1,219 @@ +#include "yql_yt_dq_optimize.h" +#include "yql_yt_helpers.h" +#include "yql_yt_optimize.h" + +#include <yt/yql/providers/yt/expr_nodes/yql_yt_expr_nodes.h> + +#include <yql/essentials/core/dqs_expr_nodes/dqs_expr_nodes.h> +#include <yql/essentials/core/yql_expr_optimize.h> +#include <yql/essentials/providers/common/dq/yql_dq_optimization_impl.h> +#include <yql/essentials/utils/log/log.h> + + +namespace NYql { + +using namespace NNodes; +using namespace NNodes::NDq; + +class TYtDqOptimizers: public TDqOptimizationBase { +public: + TYtDqOptimizers(TYtState::TPtr state) + : State_(state) + { + } + + TExprNode::TPtr RewriteRead(const TExprNode::TPtr& read, TExprContext& ctx) override { + if (auto disabledOpts = State_->Configuration->DisableOptimizers.Get(); disabledOpts && disabledOpts->contains(TStringBuilder() << "YtDqOptimizers-" << __FUNCTION__)) { + return read; + } + auto ytReadTable = TYtReadTable(read); + + TYtDSource dataSource = GetDataSource(ytReadTable, ctx); + if (!State_->Configuration->_EnableYtPartitioning.Get(dataSource.Cluster().StringValue()).GetOrElse(false)) { + return read; + } + + TSyncMap syncList; + auto ret = OptimizeReadWithSettings(read, false, false, syncList, State_, ctx); + YQL_ENSURE(syncList.empty()); + if (ret && ret != read) { + YQL_CLOG(DEBUG, ProviderYt) << __FUNCTION__; + } + return ret; + } + + TExprNode::TPtr RewriteLookupRead(const TExprNode::TPtr& read, TExprContext& ctx) override { + const auto ytReadTable = TYtReadTable(read); + //Presume that there is the only table for read + const auto ytSections = ytReadTable.Input(); + YQL_ENSURE(ytSections.Size() == 1); + const auto ytPaths = ytSections.Item(0).Paths(); + YQL_ENSURE(ytPaths.Size() == 1); + const auto ytTable = ytPaths.Item(0).Table().Maybe<TYtTable>(); + YQL_ENSURE(ytTable); + //read is of type: Tuple<World, InputSeq> + const auto inputSeqType = read->GetTypeAnn()->Cast<TTupleExprType>()->GetItems().at(1); + return Build<TDqLookupSourceWrap>(ctx, read->Pos()) + .Input(ytTable.Cast()) + .DataSource(ytReadTable.DataSource().Ptr()) + .RowType(ExpandType(read->Pos(), *GetSeqItemType(inputSeqType), ctx)) + .Settings(ytTable.Cast().Settings()) + .Done().Ptr(); + } + + TExprNode::TPtr ApplyExtractMembers(const TExprNode::TPtr& read, const TExprNode::TPtr& members, TExprContext& ctx) override { + if (auto disabledOpts = State_->Configuration->DisableOptimizers.Get(); disabledOpts && disabledOpts->contains(TStringBuilder() << "YtDqOptimizers-" << __FUNCTION__)) { + return read; + } + auto ytReadTable = TYtReadTable(read); + + TVector<TYtSection> sections; + for (auto section: ytReadTable.Input()) { + sections.push_back(UpdateInputFields(section, TExprBase(members), ctx)); + } + YQL_CLOG(DEBUG, ProviderYt) << __FUNCTION__; + return Build<TYtReadTable>(ctx, ytReadTable.Pos()) + .InitFrom(ytReadTable) + .Input() + .Add(sections) + .Build() + .Done().Ptr(); + } + + TExprNode::TPtr ApplyTakeOrSkip(const TExprNode::TPtr& read, const TExprNode::TPtr& countBase, TExprContext& ctx) override { + if (auto disabledOpts = State_->Configuration->DisableOptimizers.Get(); disabledOpts && disabledOpts->contains(TStringBuilder() << "YtDqOptimizers-" << __FUNCTION__)) { + return read; + } + auto ytReadTable = TYtReadTable(read); + auto count = TCoCountBase(countBase); + + if (ytReadTable.Input().Size() != 1) { + return read; + } + + TYtDSource dataSource = GetDataSource(ytReadTable, ctx); + TString cluster = dataSource.Cluster().StringValue(); + + if (!State_->Configuration->_EnableYtPartitioning.Get(cluster).GetOrElse(false)) { + return read; + } + + TSyncMap syncList; + if (!IsYtCompleteIsolatedLambda(count.Count().Ref(), syncList, cluster, false)) { + return read; + } + + TYtSection section = ytReadTable.Input().Item(0); + if (NYql::HasSetting(section.Settings().Ref(), EYtSettingType::Sample)) { + return read; + } + if (AnyOf(section.Paths(), [](const auto& path) { TYtPathInfo pathInfo(path); return (pathInfo.Table->Meta && pathInfo.Table->Meta->IsDynamic) || pathInfo.Ranges; })) { + return read; + } + + YQL_CLOG(DEBUG, ProviderYt) << __FUNCTION__; + EYtSettingType settingType = count.Maybe<TCoSkip>() ? EYtSettingType::Skip : EYtSettingType::Take; + return Build<TYtReadTable>(ctx, ytReadTable.Pos()) + .InitFrom(ytReadTable) + .Input() + .Add() + .InitFrom(section) + .Settings(NYql::AddSetting(section.Settings().Ref(), settingType, count.Count().Ptr(), ctx)) + .Build() + .Build() + .Done().Ptr(); + } + + TExprNode::TPtr ApplyUnordered(const TExprNode::TPtr& read, TExprContext& ctx) override { + if (auto disabledOpts = State_->Configuration->DisableOptimizers.Get(); disabledOpts && disabledOpts->contains(TStringBuilder() << "YtDqOptimizers-" << __FUNCTION__)) { + return read; + } + auto ytReadTable = TYtReadTable(read); + + TExprNode::TListType sections(ytReadTable.Input().Size()); + for (auto i = 0U; i < sections.size(); ++i) { + sections[i] = MakeUnorderedSection<true>(ytReadTable.Input().Item(i), ctx).Ptr(); + } + + YQL_CLOG(DEBUG, ProviderYt) << __FUNCTION__; + return Build<TYtReadTable>(ctx, ytReadTable.Pos()) + .InitFrom(ytReadTable) + .Input() + .Add(std::move(sections)) + .Build() + .Done().Ptr(); + } + + TExprNode::TListType ApplyExtend(const TExprNode::TListType& listOfRead, bool /*ordered*/, TExprContext& ctx) override { + if (auto disabledOpts = State_->Configuration->DisableOptimizers.Get(); disabledOpts && disabledOpts->contains(TStringBuilder() << "YtDqOptimizers-" << __FUNCTION__)) { + return listOfRead; + } + + // Group readres by cluster/settings + THashMap<std::pair<TStringBuf, TExprNode::TPtr>, std::vector<std::pair<size_t, TYtReadTable>>> reads; + for (size_t i = 0; i < listOfRead.size(); ++i) { + const auto child = listOfRead[i]; + auto ytReadTable = TYtReadTable(child); + if (ytReadTable.Input().Size() != 1) { + continue; + } + if (NYql::HasAnySetting(ytReadTable.Input().Item(0).Settings().Ref(), EYtSettingType::Take | EYtSettingType::Skip)) { + continue; + } + reads[std::make_pair(ytReadTable.DataSource().Cluster().Value(), ytReadTable.Input().Item(0).Settings().Ptr())].emplace_back(i, ytReadTable); + } + + if (reads.empty() || AllOf(reads, [](const auto& item) { return item.second.size() < 2; })) { + return listOfRead; + } + + YQL_CLOG(DEBUG, ProviderYt) << __FUNCTION__; + + TExprNode::TListType newListOfRead = listOfRead; + for (auto& item: reads) { + if (item.second.size() > 1) { + TSyncMap syncList; + TVector<TYtPath> paths; + for (auto& r: item.second) { + if (!r.second.World().Ref().IsWorld()) { + syncList.emplace(r.second.World().Ptr(), syncList.size()); + } + paths.insert(paths.end(), r.second.Input().Item(0).Paths().begin(), r.second.Input().Item(0).Paths().end()); + } + const auto ndx = item.second.front().first; + const auto& firstYtRead = item.second.front().second; + newListOfRead[ndx] = Build<TYtReadTable>(ctx, firstYtRead.Pos()) + .InitFrom(firstYtRead) + .World(ApplySyncListToWorld(ctx.NewWorld(firstYtRead.Pos()), syncList, ctx)) + .Input() + .Add() + .Paths() + .Add(paths) + .Build() + .Settings(firstYtRead.Input().Item(0).Settings()) + .Build() + .Build() + .Done().Ptr(); + + for (size_t i = 1; i < item.second.size(); ++i) { + newListOfRead[item.second[i].first] = nullptr; + } + } + } + + newListOfRead.erase(std::remove(newListOfRead.begin(), newListOfRead.end(), TExprNode::TPtr{}), newListOfRead.end()); + YQL_ENSURE(!newListOfRead.empty()); + + return newListOfRead; + } + +private: + TYtState::TPtr State_; +}; + +THolder<IDqOptimization> CreateYtDqOptimizers(TYtState::TPtr state) { + Y_ABORT_UNLESS(state); + return MakeHolder<TYtDqOptimizers>(state); +} + +} diff --git a/yt/yql/providers/yt/provider/yql_yt_dq_optimize.h b/yt/yql/providers/yt/provider/yql_yt_dq_optimize.h new file mode 100644 index 0000000000..4bc02b978e --- /dev/null +++ b/yt/yql/providers/yt/provider/yql_yt_dq_optimize.h @@ -0,0 +1,13 @@ +#pragma once + +#include "yql_yt_provider.h" + +#include <yql/essentials/core/dq_integration/yql_dq_optimization.h> + +#include <util/generic/ptr.h> + +namespace NYql { + +THolder<IDqOptimization> CreateYtDqOptimizers(TYtState::TPtr state); + +} diff --git a/yt/yql/providers/yt/provider/yql_yt_epoch.cpp b/yt/yql/providers/yt/provider/yql_yt_epoch.cpp new file mode 100644 index 0000000000..c348f0ea43 --- /dev/null +++ b/yt/yql/providers/yt/provider/yql_yt_epoch.cpp @@ -0,0 +1,437 @@ +#include "yql_yt_provider_impl.h" +#include "yql_yt_table.h" +#include "yql_yt_op_settings.h" + +#include <yt/yql/providers/yt/expr_nodes/yql_yt_expr_nodes.h> +#include <yt/yql/providers/yt/lib/graph_reorder/yql_graph_reorder.h> +#include <yql/essentials/providers/common/provider/yql_provider_names.h> +#include <yql/essentials/providers/common/provider/yql_provider.h> +#include <yql/essentials/core/yql_expr_optimize.h> +#include <yql/essentials/core/yql_graph_transformer.h> +#include <yql/essentials/utils/log/log.h> + +#include <util/generic/strbuf.h> +#include <util/string/builder.h> +#include <util/string/cast.h> + +namespace NYql { + +using namespace NNodes; + +class TYtEpochTransformer : public TSyncTransformerBase { +public: + TYtEpochTransformer(TYtState::TPtr state) + : State_(state) + { + } + + void Rewind() final { + State_->NextEpochId = 1; + } + + TStatus DoTransform(TExprNode::TPtr input, TExprNode::TPtr& output, TExprContext& ctx) final { + output = input; + if (ctx.Step.IsDone(TExprStep::Epochs)) { + return TStatus::Ok; + } + + TNodeMap<ui32> commitEpochs; + TStatus status = AssignCommitEpochs(input, output, ctx, commitEpochs); + if (status.Level == TStatus::Error) { + return status; + } + + TNodeMap<THashSet<TString>> tableWritesBeforeCommit; + status = status.Combine(AssignWriteEpochs(output, output, ctx, commitEpochs, tableWritesBeforeCommit)); + if (status.Level == TStatus::Error) { + return status; + } + + status = status.Combine(AssignUseEpochs(output, output, ctx, commitEpochs, tableWritesBeforeCommit)); + if (status.Level == TStatus::Error) { + return status; + } + + return status.Combine(TYtDependencyUpdater().ReorderGraph(output, output, ctx)); + } + +private: + class TYtDependencyUpdater: public TDependencyUpdater { + public: + TYtDependencyUpdater() + : TDependencyUpdater(YtProviderName, TYtConfigure::CallableName()) + { + } + + TMaybe<ui32> GetReadEpoch(const TExprNode::TPtr& readNode) const final { + TYtRead read(readNode); + TMaybe<ui32> maxEpoch; + if (auto list = read.Arg(2).Maybe<TExprList>()) { + for (auto item: list.Cast()) { + TMaybeNode<TYtTable> table = item.Maybe<TYtPath>().Table().Maybe<TYtTable>(); + if (!table) { + table = item.Maybe<TYtTable>(); + } + if (table) { + maxEpoch = Max(maxEpoch.GetOrElse(0), TEpochInfo::Parse(table.Cast().Epoch().Ref()).GetOrElse(0)); + } + } + } + return maxEpoch; + } + + TString GetWriteTarget(const TExprNode::TPtr& node) const final { + TYtWrite write(node); + return TStringBuilder() << "yt;" << write.DataSink().Cluster().Value() << ';' << TYtTableInfo(write.Arg(2)).Name; + } + }; + + TStatus AssignCommitEpochs(const TExprNode::TPtr& input, TExprNode::TPtr& output, TExprContext& ctx, + TNodeMap<ui32>& commitEpochs) + { + TOptimizeExprSettings settings(nullptr); + settings.VisitChanges = true; + auto status = OptimizeExpr(input, output, [&](const TExprNode::TPtr& node, TExprContext& ctx) -> TExprNode::TPtr { + // Assign uniq epoch for each commit + if (auto maybeCommit = TMaybeNode<TCoCommit>(node)) { + auto commit = maybeCommit.Cast(); + + if (!commit.DataSink().Maybe<TYtDSink>()) { + return node; + } + + auto settings = NCommon::ParseCommitSettings(commit, ctx); + if (settings.Epoch) { + commitEpochs[node.Get()] = FromString<ui32>(settings.Epoch.Cast().Value()); + return node; + } + + const ui32 commitEpoch = State_->NextEpochId++; + settings.Epoch = Build<TCoAtom>(ctx, commit.Pos()) + .Value(ToString(commitEpoch)) + .Done(); + + auto ret = Build<TCoCommit>(ctx, commit.Pos()) + .World(commit.World()) + .DataSink(commit.DataSink()) + .Settings(settings.BuildNode(ctx)) + .Done(); + + commitEpochs[ret.Raw()] = commitEpoch; + + return ret.Ptr(); + } + return node; + }, ctx, settings); + + if (input != output) { + YQL_CLOG(INFO, ProviderYt) << "Epoch-AssignCommitEpochs"; + } + + return status; + } + + TVector<TYtWrite> FindWritesBeforeCommit(const TCoCommit& commit) const { + auto cluster = commit.DataSink().Cast<TYtDSink>().Cluster().Value(); + + TVector<TYtWrite> writes; + VisitExprByFirst(commit.Ptr(), [&](const TExprNode::TPtr& node) { + if (auto maybeWrite = TMaybeNode<TYtWrite>(node)) { + if (auto ds = maybeWrite.DataSink()) { // Validate provider + if (ds.Cast().Cluster().Value() == cluster) { + writes.push_back(maybeWrite.Cast()); + } + } + } else if (auto ds = TMaybeNode<TCoCommit>(node).DataSink().Maybe<TYtDSink>()) { + if (ds.Cast().Cluster().Value() == cluster && commit.Ptr().Get() != node.Get()) { + return false; // Stop traversing + } + } + return true; + }); + + return writes; + } + + TStatus AssignWriteEpochs(const TExprNode::TPtr& input, TExprNode::TPtr& output, TExprContext& ctx, + const TNodeMap<ui32>& commitEpochs, TNodeMap<THashSet<TString>>& tableWritesBeforeCommit) + { + if (commitEpochs.empty()) { + return TStatus::Ok; + } + + TExprNode::TListType commits; + VisitExprByFirst(input, [&](const TExprNode::TPtr& node) { + if (auto ds = TMaybeNode<TCoCommit>(node).DataSink().Maybe<TYtDSink>()) { + commits.push_back(node); + } + return true; + }); + + TNodeMap<ui32> writeCommitEpochs; + // Find all writes before commits + for (auto& commitNode: commits) { + auto commit = TCoCommit(commitNode); + auto it = commitEpochs.find(commitNode.Get()); + YQL_ENSURE(it != commitEpochs.end()); + const ui32 commitEpoch = it->second; + + TVector<TYtWrite> writes = FindWritesBeforeCommit(commit); + THashMap<TString, TVector<TExprNode::TPtr>> writesByTable; + THashSet<TString> tableNames; + for (auto write: writes) { + TYtTableInfo tableInfo(write.Arg(2)); + if (!tableInfo.CommitEpoch.Defined()) { + writeCommitEpochs[write.Raw()] = commitEpoch; + } else if (*tableInfo.CommitEpoch != commitEpoch) { + ctx.AddError(TIssue(ctx.GetPosition(write.Pos()), TStringBuilder() + << "Table " << tableInfo.Name.Quote() << " write belongs to multiple commits with different epochs")); + return TStatus::Error; + } + tableNames.insert(tableInfo.Name); + + // Graph is traversed from the end, so store the last write for each table, which is + // actually executed first + writesByTable[tableInfo.Name].push_back(write.Ptr()); + } + for (auto& w: writesByTable) { + if (auto cnt = std::count_if(w.second.begin(), w.second.end(), [](const auto& p) { return NYql::HasSetting(*p->Child(4), EYtSettingType::Initial); })) { + YQL_ENSURE(cnt == 1, "Multiple 'initial' writes to the same table " << w.first); + continue; + } + if (w.second.size() > 1) { + std::stable_sort(w.second.begin(), w.second.end(), [](const auto& p1, const auto& p2) { return p1->UniqueId() < p2->UniqueId(); }); + } + w.second.front()->ChildRef(4) = NYql::AddSetting(*w.second.front()->Child(4), EYtSettingType::Initial, {}, ctx); + } + if (!tableNames.empty()) { + tableWritesBeforeCommit.emplace(commitNode.Get(), std::move(tableNames)); + } + } + + if (writeCommitEpochs.empty()) { + return TStatus::Ok; + } + + TOptimizeExprSettings settings(nullptr); + settings.VisitChanges = true; + auto status = OptimizeExpr(input, output, [&](const TExprNode::TPtr& node, TExprContext& ctx) -> TExprNode::TPtr { + if (node->IsCallable(TYtWrite::CallableName())) { + const auto it = writeCommitEpochs.find(node.Get()); + if (writeCommitEpochs.cend() != it) { + TYtTableInfo tableInfo = node->ChildPtr(2); + tableInfo.CommitEpoch = it->second; + node->ChildRef(2) = tableInfo.ToExprNode(ctx, node->Pos()).Ptr(); + } + } + return node; + }, ctx, settings); + + YQL_CLOG(INFO, ProviderYt) << "Epoch-AssignWriteEpochs"; + return status; + } + + TVector<const TExprNode*> FindCommitsBeforeNode(const TExprNode& startNode, TStringBuf cluster, + const TNodeMap<THashSet<TString>>& tableWritesBeforeCommit) + { + TVector<const TExprNode*> commits; + VisitExprByFirst(startNode, [&](const TExprNode& node) { + if (auto ds = TMaybeNode<TCoCommit>(&node).DataSink().Maybe<TYtDSink>()) { + if (tableWritesBeforeCommit.find(&node) != tableWritesBeforeCommit.end() && ds.Cast().Cluster().Value() == cluster) { + commits.push_back(&node); + } + } + return true; + }); + return commits; + } + + TStatus AssignUseEpochs(const TExprNode::TPtr& input, TExprNode::TPtr& output, TExprContext& ctx, + const TNodeMap<ui32>& commitEpochs, const TNodeMap<THashSet<TString>>& tableWritesBeforeCommit) + { + TNodeMap<TNodeMap<ui32>> ioEpochs; + VisitExpr(input, [&](const TExprNode::TPtr& node) { + if (auto ds = TMaybeNode<TYtRead>(node).DataSource()) { + if (tableWritesBeforeCommit.empty()) { + return true; + } + auto clusterName = ds.Cast().Cluster().Value(); + auto commitDeps = FindCommitsBeforeNode(*node, clusterName, tableWritesBeforeCommit); + if (commitDeps.empty()) { + return true; + } + + TYtRead read(node); + if (auto list = read.Arg(2).Maybe<TExprList>()) { + THashMap<TString, ui32> tableEpochs; + for (auto item: list.Cast()) { + TMaybeNode<TYtTable> tableNode = item.Maybe<TYtPath>().Table().Maybe<TYtTable>(); + if (!tableNode) { + tableNode = item.Maybe<TYtTable>(); + } + + if (tableNode) { + TYtTableInfo tableInfo(tableNode.Cast()); + if (!tableInfo.Epoch.Defined()) { + TMaybe<ui32> epoch; + if (auto p = tableEpochs.FindPtr(tableInfo.Name)) { + epoch = *p; + } + else { + for (auto commit: commitDeps) { + auto itWrites = tableWritesBeforeCommit.find(commit); + if (itWrites != tableWritesBeforeCommit.end() && itWrites->second.contains(tableInfo.Name)) { + auto itCommit = commitEpochs.find(commit); + YQL_ENSURE(itCommit != commitEpochs.end()); + epoch = Max<ui32>(itCommit->second, epoch.GetOrElse(0)); + } + } + } + tableEpochs[tableInfo.Name] = epoch.GetOrElse(0); + + if (0 != epoch.GetOrElse(0)) { + ioEpochs[node.Get()][tableNode.Raw()] = *epoch; + State_->EpochDependencies[*epoch].emplace(clusterName, tableInfo.Name); + } + } + } + } + } + } + else if (auto ds = TMaybeNode<TYtWrite>(node).DataSink()) { + if (tableWritesBeforeCommit.empty()) { + return true; + } + auto clusterName = ds.Cast().Cluster().Value(); + auto commitDeps = FindCommitsBeforeNode(*node, clusterName, tableWritesBeforeCommit); + if (commitDeps.empty()) { + return true; + } + + TYtWrite write(node); + if (write.Arg(2).Maybe<TYtTable>()) { + TYtTableInfo tableInfo(write.Arg(2)); + if (!tableInfo.Epoch.Defined()) { + + TMaybe<ui32> epoch; + for (auto commit: commitDeps) { + auto itWrites = tableWritesBeforeCommit.find(commit); + if (itWrites != tableWritesBeforeCommit.end() && itWrites->second.contains(tableInfo.Name)) { + auto itCommit = commitEpochs.find(commit); + YQL_ENSURE(itCommit != commitEpochs.end()); + epoch = Max<ui32>(itCommit->second, epoch.GetOrElse(0)); + } + } + + if (0 != epoch.GetOrElse(0)) { + ioEpochs[node.Get()][write.Arg(2).Raw()] = *epoch; + State_->EpochDependencies[*epoch].emplace(clusterName, tableInfo.Name); + } + } + } + } + else if (auto maybeRead = TMaybeNode<TYtReadTable>(node)) { + auto cluster = TString{maybeRead.Cast().DataSource().Cluster().Value()}; + for (auto section: maybeRead.Cast().Input()) { + for (auto path: section.Paths()) { + if (auto table = path.Table().Maybe<TYtTable>()) { + if (auto epoch = TEpochInfo::Parse(table.Epoch().Ref()).GetOrElse(0)) { + State_->EpochDependencies[epoch].emplace(cluster, table.Cast().Name().Value()); + } + } + } + } + } + else if (auto maybeOp = TMaybeNode<TYtTransientOpBase>(node)) { + auto cluster = TString{maybeOp.Cast().DataSink().Cluster().Value()}; + for (auto section: maybeOp.Cast().Input()) { + for (auto path: section.Paths()) { + if (auto table = path.Table().Maybe<TYtTable>()) { + if (auto epoch = TEpochInfo::Parse(table.Epoch().Ref()).GetOrElse(0)) { + State_->EpochDependencies[epoch].emplace(cluster, table.Cast().Name().Value()); + } + } + } + } + } + else if (auto maybePublish = TMaybeNode<TYtPublish>(node)) { + auto cluster = TString{maybePublish.Cast().DataSink().Cluster().Value()}; + auto table = maybePublish.Cast().Publish(); + if (auto epoch = TEpochInfo::Parse(table.Epoch().Ref()).GetOrElse(0)) { + State_->EpochDependencies[epoch].emplace(cluster, table.Name().Value()); + } + } + else if (auto maybeDrop = TMaybeNode<TYtDropTable>(node)) { + auto cluster = TString{maybeDrop.Cast().DataSink().Cluster().Value()}; + auto table = maybeDrop.Cast().Table(); + if (auto epoch = TEpochInfo::Parse(table.Epoch().Ref()).GetOrElse(0)) { + State_->EpochDependencies[epoch].emplace(cluster, table.Name().Value()); + } + } + else if (auto maybeScheme = TMaybeNode<TYtReadTableScheme>(node)) { + auto cluster = TString{maybeScheme.Cast().DataSource().Cluster().Value()}; + auto table = maybeScheme.Cast().Table(); + if (auto epoch = TEpochInfo::Parse(table.Epoch().Ref()).GetOrElse(0)) { + State_->EpochDependencies[epoch].emplace(cluster, table.Name().Value()); + } + } + else if (auto maybeWrite = TMaybeNode<TYtWriteTable>(node)) { + auto cluster = TString{maybeWrite.Cast().DataSink().Cluster().Value()}; + auto table = maybeWrite.Cast().Table(); + if (auto epoch = TEpochInfo::Parse(table.Epoch().Ref()).GetOrElse(0)) { + State_->EpochDependencies[epoch].emplace(cluster, table.Name().Value()); + } + } + return true; + }); + + if (ioEpochs.empty()) { + return TStatus::Ok; + } + + TOptimizeExprSettings settings(nullptr); + settings.VisitChanges = true; + auto status = OptimizeExpr(input, output, [&](const TExprNode::TPtr& node, TExprContext& ctx) -> TExprNode::TPtr { + if (TYtRead::Match(node.Get()) || TYtWrite::Match(node.Get())) { + const auto it = ioEpochs.find(node.Get()); + if (ioEpochs.cend() != it) { + auto tables = node->ChildPtr(2); + auto& tableEpochs = it->second; + TOptimizeExprSettings subSettings(nullptr); + subSettings.VisitChanges = true; + auto subStatus = OptimizeExpr(tables, tables, [&tableEpochs](const TExprNode::TPtr& subNode, TExprContext& ctx) -> TExprNode::TPtr { + if (TYtTable::Match(subNode.Get())) { + auto tableIt = tableEpochs.find(subNode.Get()); + if (tableEpochs.cend() != tableIt) { + TYtTableInfo tableInfo = subNode; + tableInfo.Epoch = tableIt->second; + return tableInfo.ToExprNode(ctx, subNode->Pos()).Ptr(); + } + } + return subNode; + }, ctx, subSettings); + if (subStatus.Level != TStatus::Ok) { + return {}; + } + + ioEpochs.erase(it); + node->ChildRef(2) = std::move(tables); + } + } + + return node; + }, ctx, settings); + + YQL_CLOG(INFO, ProviderYt) << "Epoch-AssignUseEpochs"; + return status; + } + +private: + TYtState::TPtr State_; +}; + +THolder<IGraphTransformer> CreateYtEpochTransformer(TYtState::TPtr state) { + return THolder(new TYtEpochTransformer(state)); +} + +} diff --git a/yt/yql/providers/yt/provider/yql_yt_gateway.cpp b/yt/yql/providers/yt/provider/yql_yt_gateway.cpp new file mode 100644 index 0000000000..fc217a7bb7 --- /dev/null +++ b/yt/yql/providers/yt/provider/yql_yt_gateway.cpp @@ -0,0 +1,4 @@ +#include "yql_yt_gateway.h" + +namespace NYql { +} diff --git a/yt/yql/providers/yt/provider/yql_yt_gateway.h b/yt/yql/providers/yt/provider/yql_yt_gateway.h new file mode 100644 index 0000000000..b331f2142c --- /dev/null +++ b/yt/yql/providers/yt/provider/yql_yt_gateway.h @@ -0,0 +1,678 @@ +#pragma once + +#include "yql_yt_table.h" +#include "yql_yt_table_desc.h" + +#include <yt/yql/providers/yt/common/yql_yt_settings.h> +#include <yt/yql/providers/yt/lib/row_spec/yql_row_spec.h> +#include <yql/providers/stat/uploader/yql_stat_uploader.h> + +#include <yql/essentials/providers/common/gateway/yql_provider_gateway.h> +#include <yql/essentials/core/expr_nodes/yql_expr_nodes.h> +#include <yql/essentials/core/yql_data_provider.h> +#include <yql/essentials/core/yql_type_annotation.h> +#include <yql/essentials/core/yql_execution.h> +#include <yql/essentials/core/file_storage/storage.h> + +#include <yt/cpp/mapreduce/interface/common.h> + +#include <library/cpp/time_provider/time_provider.h> +#include <library/cpp/random_provider/random_provider.h> +#include <library/cpp/yson/node/node.h> +#include <library/cpp/yson/public.h> +#include <library/cpp/threading/future/future.h> + +#include <util/generic/ptr.h> +#include <util/generic/typetraits.h> +#include <util/generic/string.h> +#include <util/generic/hash.h> +#include <util/generic/maybe.h> +#include <util/generic/vector.h> + +#include <utility> + +namespace NYql { + +class TYtClusterConfig; + +namespace NCommon { + class TMkqlCallableCompilerBase; +} + +class IYtGateway : public TThrRefBase { +public: + using TPtr = TIntrusivePtr<IYtGateway>; + +#define OPTION_FIELD_METHODS(type, name) \ + public: \ + TSelf& name(TTypeTraits<type>::TFuncParam arg##name)& { \ + name##_ = arg##name; \ + return *this; \ + } \ + TSelf&& name(TTypeTraits<type>::TFuncParam arg##name)&& { \ + name##_ = arg##name; \ + return std::move(*this); \ + } \ + TTypeTraits<type>::TFuncParam name() const { \ + return name##_; \ + } \ + type& name() { \ + return name##_; \ + } + +#define OPTION_FIELD(type, name) \ + private: \ + type name##_; \ + OPTION_FIELD_METHODS(type, name) + +#define OPTION_FIELD_DEFAULT(type, name, def) \ + private: \ + type name##_ = def; \ + OPTION_FIELD_METHODS(type, name) + + + ////////////////////////////////////////////////////////////// + + using TSecureParams = THashMap<TString, TString>; + + ////////////////////////////////////////////////////////////// + + struct TCommonOptions { + TString SessionId_; + + TCommonOptions(const TString& sessionId) + : SessionId_(sessionId) + { + } + + const TString& SessionId() const { + return SessionId_; + } + }; + + ////////////////////////////////////////////////////////////// + + struct TOpenSessionOptions : public TCommonOptions { + using TSelf = TOpenSessionOptions; + + TOpenSessionOptions(const TString& sessionId) + : TCommonOptions(sessionId) + { + } + + OPTION_FIELD(TString, UserName) + OPTION_FIELD(TOperationProgressWriter, ProgressWriter) + OPTION_FIELD(TYqlOperationOptions, OperationOptions) + OPTION_FIELD(TIntrusivePtr<IRandomProvider>, RandomProvider) + OPTION_FIELD(TIntrusivePtr<ITimeProvider>, TimeProvider) + OPTION_FIELD(TStatWriter, StatWriter) + OPTION_FIELD_DEFAULT(bool, CreateOperationTracker, true) + }; + + ////////////////////////////////////////////////////////////// + + struct TCloseSessionOptions : public TCommonOptions { + using TSelf = TCloseSessionOptions; + + TCloseSessionOptions(const TString& sessionId) + : TCommonOptions(sessionId) + { + } + }; + + ////////////////////////////////////////////////////////////// + + struct TCleanupSessionOptions : public TCommonOptions { + using TSelf = TCleanupSessionOptions; + + TCleanupSessionOptions(const TString& sessionId) + : TCommonOptions(sessionId) + { + } + }; + + ////////////////////////////////////////////////////////////// + + struct TFinalizeOptions : public TCommonOptions { + using TSelf = TFinalizeOptions; + + TFinalizeOptions(const TString& sessionId) + : TCommonOptions(sessionId) + { + } + + OPTION_FIELD(TYtSettings::TConstPtr, Config) + OPTION_FIELD_DEFAULT(bool, Abort, false) + OPTION_FIELD_DEFAULT(bool, DetachSnapshotTxs, false) + }; + + struct TFinalizeResult : public NCommon::TOperationResult { + }; + + ////////////////////////////////////////////////////////////// + + struct TCanonizeReq { + using TSelf = TCanonizeReq; + + OPTION_FIELD(TString, Cluster) + OPTION_FIELD(TString, Path) + OPTION_FIELD(TPosition, Pos) + }; + + struct TCanonizePathsOptions : public TCommonOptions { + using TSelf = TCanonizePathsOptions; + + TCanonizePathsOptions(const TString& sessionId) + : TCommonOptions(sessionId) + { + } + + OPTION_FIELD(TVector<TCanonizeReq>, Paths) + OPTION_FIELD(TYtSettings::TConstPtr, Config) + }; + + struct TCanonizedPath { + TString Path; + TMaybe<TVector<TString>> Columns; + TMaybe<TVector<NYT::TReadRange>> Ranges; + TMaybe<TString> AdditionalAttributes; + }; + + struct TCanonizePathsResult: public NCommon::TOperationResult { + TVector<TCanonizedPath> Data; + }; + + ////////////////////////////////////////////////////////////// + + struct TTableReq { + using TSelf = TTableReq; + + OPTION_FIELD(TString, Cluster) + OPTION_FIELD(TString, Table) + OPTION_FIELD_DEFAULT(bool, LockOnly, false) + OPTION_FIELD(TYtTableIntents, Intents) + OPTION_FIELD_DEFAULT(ui32, InferSchemaRows, 0) + OPTION_FIELD_DEFAULT(bool, ForceInferSchema, false) + OPTION_FIELD_DEFAULT(bool, Anonymous, false) + OPTION_FIELD_DEFAULT(bool, IgnoreYamrDsv, false) + OPTION_FIELD_DEFAULT(bool, IgnoreWeakSchema, false) + }; + + struct TGetTableInfoOptions : public TCommonOptions { + using TSelf = TGetTableInfoOptions; + + TGetTableInfoOptions(const TString& sessionId) + : TCommonOptions(sessionId) + { + } + + OPTION_FIELD(TVector<TTableReq>, Tables) + OPTION_FIELD(TYtSettings::TConstPtr, Config) + OPTION_FIELD_DEFAULT(bool, ReadOnly, false) + OPTION_FIELD_DEFAULT(ui32, Epoch, 0) + }; + + struct TTableInfoResult: public NCommon::TOperationResult { + struct TTableData { + TYtTableMetaInfo::TPtr Meta; + TYtTableStatInfo::TPtr Stat; + bool WriteLock = false; + }; + TVector<TTableData> Data; + }; + + struct TTableStatResult: public NCommon::TOperationResult { + TYtTableStatInfo::TPtr Data; + }; + + ////////////////////////////////////////////////////////////// + + struct TTableRangeOptions : public TCommonOptions { + using TSelf = TTableRangeOptions; + + TTableRangeOptions(const TString& sessionId) + : TCommonOptions(sessionId) + { + } + + OPTION_FIELD(TString, Cluster) + OPTION_FIELD(TString, Prefix) + OPTION_FIELD_DEFAULT(const TExprNode*, Filter, nullptr) + OPTION_FIELD_DEFAULT(TExprContext*, ExprCtx, nullptr) + OPTION_FIELD(TString, Suffix) + OPTION_FIELD(TUserDataTable, UserDataBlocks) + OPTION_FIELD(TUdfModulesTable, UdfModules) + OPTION_FIELD(IUdfResolver::TPtr, UdfResolver) + OPTION_FIELD_DEFAULT(NKikimr::NUdf::EValidateMode, UdfValidateMode, NKikimr::NUdf::EValidateMode::None) + OPTION_FIELD(TMaybe<ui32>, PublicId) + OPTION_FIELD(TYtSettings::TConstPtr, Config) + OPTION_FIELD(TString, OptLLVM) + OPTION_FIELD(TString, OperationHash) + OPTION_FIELD(TPosition, Pos) + OPTION_FIELD(TSecureParams, SecureParams) + }; + + struct TTableRangeResult : public NCommon::TOperationResult { + TVector<TCanonizedPath> Tables; + }; + + ////////////////////////////////////////////////////////////// + + struct TFolderOptions : public TCommonOptions { + using TSelf = TFolderOptions; + + TFolderOptions(const TString& sessionId) + : TCommonOptions(sessionId) + { + } + + OPTION_FIELD(TString, Cluster) + OPTION_FIELD(TString, Prefix) + OPTION_FIELD(TSet<TString>, Attributes) + OPTION_FIELD(TYtSettings::TConstPtr, Config) + OPTION_FIELD(TPosition, Pos) + }; + + struct TBatchFolderOptions : public TCommonOptions { + using TSelf = TBatchFolderOptions; + + TBatchFolderOptions(const TString& sessionId) + : TCommonOptions(sessionId) + { + } + + struct TFolderPrefixAttrs { + TString Prefix; + TSet<TString> AttrKeys; + }; + + OPTION_FIELD(TString, Cluster) + OPTION_FIELD(TVector<TFolderPrefixAttrs>, Folders) + OPTION_FIELD(TYtSettings::TConstPtr, Config) + OPTION_FIELD(TPosition, Pos) + }; + + struct TBatchFolderResult : public NCommon::TOperationResult { + struct TFolderItem { + TString Path; + TString Type; + NYT::TNode Attributes; + + }; + TVector<TFolderItem> Items; + }; + + struct TSerializedFolderItem { + }; + + struct TFolderResult : public NCommon::TOperationResult { + struct TFolderItem { + TString Path; + TString Type; + TString Attributes; + + auto operator<=>(const TFolderItem&) const = default; + }; + std::variant<TVector<TFolderItem>, TFileLinkPtr> ItemsOrFileLink; + }; + + ////////////////////////////////////////////////////////////// + + struct TResolveOptions : public TCommonOptions { + using TSelf = TResolveOptions; + + TResolveOptions(const TString& sessionId) + : TCommonOptions(sessionId) + { + } + + struct TItemWithReqAttrs { + TBatchFolderResult::TFolderItem Item; + TSet<TString> AttrKeys; + }; + + OPTION_FIELD(TString, Cluster) + OPTION_FIELD(TVector<TItemWithReqAttrs>, Items) + OPTION_FIELD(TYtSettings::TConstPtr, Config) + OPTION_FIELD(TPosition, Pos) + }; + + ////////////////////////////////////////////////////////////// + + struct TResOrPullOptions : public TCommonOptions { + using TSelf = TResOrPullOptions; + + TResOrPullOptions(const TString& sessionId) + : TCommonOptions(sessionId) + { + } + + OPTION_FIELD(TUserDataTable, UserDataBlocks) + OPTION_FIELD(TUdfModulesTable, UdfModules) + OPTION_FIELD(IUdfResolver::TPtr, UdfResolver) + OPTION_FIELD_DEFAULT(NKikimr::NUdf::EValidateMode, UdfValidateMode, NKikimr::NUdf::EValidateMode::None) + OPTION_FIELD(IDataProvider::TFillSettings, FillSettings) + OPTION_FIELD(TMaybe<ui32>, PublicId) + OPTION_FIELD(TYtSettings::TConstPtr, Config) + OPTION_FIELD(TString, UsedCluster) + OPTION_FIELD(TString, OptLLVM) + OPTION_FIELD(TString, OperationHash) + OPTION_FIELD(TSecureParams, SecureParams) + }; + + struct TResOrPullResult : public NCommon::TOperationResult { + TString Data; + }; + + ////////////////////////////////////////////////////////////// + + struct TRunOptions : public TCommonOptions { + using TSelf = TRunOptions; + + TRunOptions(const TString& sessionId) + : TCommonOptions(sessionId) + { + } + + OPTION_FIELD(TUserDataTable, UserDataBlocks) + OPTION_FIELD(TUdfModulesTable, UdfModules) + OPTION_FIELD(IUdfResolver::TPtr, UdfResolver) + OPTION_FIELD_DEFAULT(NKikimr::NUdf::EValidateMode, UdfValidateMode, NKikimr::NUdf::EValidateMode::None) + OPTION_FIELD(TMaybe<ui32>, PublicId); + OPTION_FIELD(TYtSettings::TConstPtr, Config) + OPTION_FIELD(TString, OptLLVM) + OPTION_FIELD(TString, OperationHash) + OPTION_FIELD(TSecureParams, SecureParams) + OPTION_FIELD_DEFAULT(TSet<TString>, AdditionalSecurityTags, {}) + }; + + struct TRunResult : public NCommon::TOperationResult { + // Return pair of table name, table stat for each output table + TVector<std::pair<TString, TYtTableStatInfo::TPtr>> OutTableStats; + }; + + + ////////////////////////////////////////////////////////////// + + struct TPrepareOptions : public TCommonOptions { + using TSelf = TPrepareOptions; + + TPrepareOptions(const TString& sessionId) + : TCommonOptions(sessionId) + { + } + + OPTION_FIELD(TMaybe<ui32>, PublicId) + OPTION_FIELD(TYtSettings::TConstPtr, Config) + OPTION_FIELD(TString, OperationHash) + OPTION_FIELD_DEFAULT(TSet<TString>, SecurityTags, {}) + }; + + ////////////////////////////////////////////////////////////// + + struct TCalcOptions : public TCommonOptions { + using TSelf = TCalcOptions; + + TCalcOptions(const TString& sessionId) + : TCommonOptions(sessionId) + { + } + + OPTION_FIELD(TString, Cluster) + OPTION_FIELD(TUserDataTable, UserDataBlocks) + OPTION_FIELD(TUdfModulesTable, UdfModules) + OPTION_FIELD(IUdfResolver::TPtr, UdfResolver) + OPTION_FIELD_DEFAULT(NKikimr::NUdf::EValidateMode, UdfValidateMode, NKikimr::NUdf::EValidateMode::None) + OPTION_FIELD(TMaybe<ui32>, PublicId); + OPTION_FIELD(TYtSettings::TConstPtr, Config) + OPTION_FIELD(TString, OptLLVM) + OPTION_FIELD(TString, OperationHash) + OPTION_FIELD(TSecureParams, SecureParams) + }; + + struct TCalcResult : public NCommon::TOperationResult { + TVector<NYT::TNode> Data; + }; + + ////////////////////////////////////////////////////////////// + + struct TPublishOptions : public TCommonOptions { + using TSelf = TPublishOptions; + + TPublishOptions(const TString& sessionId) + : TCommonOptions(sessionId) + { + } + + OPTION_FIELD(TMaybe<ui32>, PublicId) + OPTION_FIELD(TYqlRowSpecInfo::TPtr, DestinationRowSpec) + OPTION_FIELD(TYtSettings::TConstPtr, Config) + OPTION_FIELD(TString, OptLLVM) + OPTION_FIELD(TString, OperationHash) + }; + + struct TPublishResult : public NCommon::TOperationResult { + }; + + ////////////////////////////////////////////////////////////// + + struct TCommitOptions : public TCommonOptions { + using TSelf = TCommitOptions; + + TCommitOptions(const TString& sessionId) + : TCommonOptions(sessionId) + { + } + + OPTION_FIELD(TString, Cluster) + }; + + struct TCommitResult : public NCommon::TOperationResult { + }; + + ////////////////////////////////////////////////////////////// + + struct TDropTrackablesOptions : public TCommonOptions { + using TSelf = TDropTrackablesOptions; + + struct TClusterAndPath + { + TString Cluster; + TString Path; + }; + + TDropTrackablesOptions(const TString& sessionId) + : TCommonOptions(sessionId) + { + } + + OPTION_FIELD(TYtSettings::TConstPtr, Config) + OPTION_FIELD(TVector<TClusterAndPath>, Pathes) + }; + + struct TDropTrackablesResult : public NCommon::TOperationResult { + }; + + ////////////////////////////////////////////////////////////// + + struct TPathStatReq { + using TSelf = TPathStatReq; + + OPTION_FIELD(NYT::TRichYPath, Path) + OPTION_FIELD_DEFAULT(bool, IsTemp, false) + OPTION_FIELD_DEFAULT(bool, IsAnonymous, false) + OPTION_FIELD_DEFAULT(ui32, Epoch, 0) + }; + + struct TPathStatOptions : public TCommonOptions { + using TSelf = TPathStatOptions; + + TPathStatOptions(const TString& sessionId) + : TCommonOptions(sessionId) + { + } + + OPTION_FIELD(TString, Cluster) + OPTION_FIELD(TVector<TPathStatReq>, Paths) + OPTION_FIELD(TYtSettings::TConstPtr, Config) + OPTION_FIELD_DEFAULT(bool, Extended, false) + }; + + struct TPathStatResult: public NCommon::TOperationResult { + struct TExtendedResult { + THashMap<TString, i64> DataWeight; + THashMap<TString, ui64> EstimatedUniqueCounts; + }; + TVector<ui64> DataSize; + TVector<TMaybe<TExtendedResult>> Extended; + }; + + struct TFullResultTableOptions : public TCommonOptions { + using TSelf = TFullResultTableOptions; + + TFullResultTableOptions(const TString& sessionId) + : TCommonOptions(sessionId) + { + } + + OPTION_FIELD(TString, Cluster) + OPTION_FIELD(TYtSettings::TConstPtr, Config) + OPTION_FIELD(TYtOutTableInfo, OutTable) + }; + + struct TFullResultTableResult: public NCommon::TOperationResult { + TMaybe<TString> RootTransactionId; + TMaybe<TString> ExternalTransactionId; + TString Server; + TString Path; + TString RefName; + TString CodecSpec; + TString TableAttrs; + }; + + struct TGetTablePartitionsOptions : public TCommonOptions { + using TSelf = TGetTablePartitionsOptions; + + TGetTablePartitionsOptions(const TString& sessionId) + : TCommonOptions(sessionId) + { + } + + OPTION_FIELD(TString, Cluster) + OPTION_FIELD(TVector<TYtPathInfo::TPtr>, Paths) + OPTION_FIELD_DEFAULT(size_t, DataSizePerJob, 0) + OPTION_FIELD_DEFAULT(size_t, MaxPartitions, 0) + OPTION_FIELD_DEFAULT(bool, AdjustDataWeightPerPartition, true) + OPTION_FIELD(TYtSettings::TConstPtr, Config) + }; + + struct TGetTablePartitionsResult: public NCommon::TOperationResult { + NYT::TMultiTablePartitions Partitions; + }; + + struct TDownloadTablesReq { + using TSelf = TDownloadTablesReq; + + OPTION_FIELD(TString, Cluster) + OPTION_FIELD(TString, Table) + OPTION_FIELD_DEFAULT(bool, Anonymous, false) + OPTION_FIELD(TString, TargetPath) + }; + + struct TDownloadTablesOptions : public TCommonOptions { + using TSelf = TDownloadTablesOptions; + + TDownloadTablesOptions(const TString& sessionId) + : TCommonOptions(sessionId) + { + } + + OPTION_FIELD(TVector<TDownloadTablesReq>, Tables) + OPTION_FIELD_DEFAULT(ui32, Epoch, 0) + OPTION_FIELD(TYtSettings::TConstPtr, Config) + }; + + struct TDownloadTablesResult: public NCommon::TOperationResult { + }; + + struct TUploadTableOptions : public TCommonOptions { + using TSelf = TUploadTableOptions; + + TUploadTableOptions(const TString& sessionId) + : TCommonOptions(sessionId) + { + } + + OPTION_FIELD(TString, Cluster) + OPTION_FIELD(TString, Table) + OPTION_FIELD(TString, Path) + OPTION_FIELD(TString, Attrs) + OPTION_FIELD(TYtSettings::TConstPtr, Config) + }; + + struct TUploadTableResult: public NCommon::TOperationResult { + }; + +public: + virtual ~IYtGateway() = default; + + virtual void OpenSession(TOpenSessionOptions&& options) = 0; + + virtual NThreading::TFuture<void> CloseSession(TCloseSessionOptions&& options) = 0; + + virtual NThreading::TFuture<void> CleanupSession(TCleanupSessionOptions&& options) = 0; + + virtual NThreading::TFuture<TFinalizeResult> Finalize(TFinalizeOptions&& options) = 0; + + virtual NThreading::TFuture<TCanonizePathsResult> CanonizePaths(TCanonizePathsOptions&& options) = 0; + + virtual NThreading::TFuture<TTableInfoResult> GetTableInfo(TGetTableInfoOptions&& options) = 0; + + virtual NThreading::TFuture<TTableRangeResult> GetTableRange(TTableRangeOptions&& options) = 0; + + virtual NThreading::TFuture<TFolderResult> GetFolder(TFolderOptions&& options) = 0; + + virtual NThreading::TFuture<TBatchFolderResult> ResolveLinks(TResolveOptions&& options) = 0; + + virtual NThreading::TFuture<TBatchFolderResult> GetFolders(TBatchFolderOptions&& options) = 0; + + virtual NThreading::TFuture<TResOrPullResult> ResOrPull(const TExprNode::TPtr& node, TExprContext& ctx, TResOrPullOptions&& options) = 0; + + virtual NThreading::TFuture<TRunResult> Run(const TExprNode::TPtr& node, TExprContext& ctx, TRunOptions&& options) = 0; + + virtual NThreading::TFuture<TRunResult> Prepare(const TExprNode::TPtr& node, TExprContext& ctx, TPrepareOptions&& options) const = 0; + virtual NThreading::TFuture<TRunResult> GetTableStat(const TExprNode::TPtr& node, TExprContext& ctx, TPrepareOptions&& options) = 0 ; + + virtual NThreading::TFuture<TCalcResult> Calc(const TExprNode::TListType& nodes, TExprContext& ctx, TCalcOptions&& options) = 0; + + virtual NThreading::TFuture<TPublishResult> Publish(const TExprNode::TPtr& node, TExprContext& ctx, TPublishOptions&& options) = 0; + + virtual NThreading::TFuture<TCommitResult> Commit(TCommitOptions&& options) = 0; + + virtual NThreading::TFuture<TDropTrackablesResult> DropTrackables(TDropTrackablesOptions&& options) = 0; + + virtual NThreading::TFuture<TPathStatResult> PathStat(TPathStatOptions&& options) = 0; + virtual TPathStatResult TryPathStat(TPathStatOptions&& options) = 0; + + virtual bool TryParseYtUrl(const TString& url, TString* cluster, TString* path) const = 0; + + virtual TString GetDefaultClusterName() const = 0; + virtual TString GetClusterServer(const TString& cluster) const = 0; + virtual NYT::TRichYPath GetRealTable(const TString& sessionId, const TString& cluster, const TString& table, ui32 epoch, const TString& tmpFolder) const = 0; + virtual NYT::TRichYPath GetWriteTable(const TString& sessionId, const TString& cluster, const TString& table, const TString& tmpFolder) const = 0; + + virtual NThreading::TFuture<TDownloadTablesResult> DownloadTables(TDownloadTablesOptions&& options) = 0; + virtual NThreading::TFuture<TUploadTableResult> UploadTable(TUploadTableOptions&& options) = 0; + + virtual TFullResultTableResult PrepareFullResultTable(TFullResultTableOptions&& options) = 0; + + virtual void SetStatUploader(IStatUploader::TPtr statUploader) = 0; + + virtual void RegisterMkqlCompiler(NCommon::TMkqlCallableCompilerBase& compiler) = 0; + + virtual TGetTablePartitionsResult GetTablePartitions(TGetTablePartitionsOptions&& options) = 0; + + virtual void AddCluster(const TYtClusterConfig& cluster) = 0; +}; + +} diff --git a/yt/yql/providers/yt/provider/yql_yt_helpers.cpp b/yt/yql/providers/yt/provider/yql_yt_helpers.cpp new file mode 100644 index 0000000000..c25a4f8da1 --- /dev/null +++ b/yt/yql/providers/yt/provider/yql_yt_helpers.cpp @@ -0,0 +1,2192 @@ +#include "yql_yt_helpers.h" +#include "yql_yt_provider_impl.h" +#include "yql_yt_op_settings.h" +#include "yql_yt_op_hash.h" +#include "yql_yt_optimize.h" + +#include <yt/yql/providers/yt/lib/mkql_helpers/mkql_helpers.h> +#include <yt/yql/providers/yt/common/yql_configuration.h> +#include <yt/yql/providers/yt/opt/yql_yt_key_selector.h> +#include <yql/essentials/providers/common/provider/yql_provider.h> +#include <yql/essentials/providers/common/codec/yql_codec_type_flags.h> +#include <yql/essentials/providers/common/codec/yql_codec.h> +#include <yql/essentials/core/dq_expr_nodes/dq_expr_nodes.h> +#include <yql/essentials/core/dqs_expr_nodes/dqs_expr_nodes.h> +#include <yql/essentials/core/expr_nodes/yql_expr_nodes.h> +#include <yql/essentials/core/type_ann/type_ann_expr.h> +#include <yql/essentials/core/type_ann/type_ann_core.h> +#include <yql/essentials/core/issue/protos/issue_id.pb.h> +#include <yql/essentials/core/peephole_opt/yql_opt_peephole_physical.h> +#include <yql/essentials/core/yql_expr_optimize.h> +#include <yql/essentials/core/yql_expr_constraint.h> +#include <yql/essentials/core/yql_expr_csee.h> +#include <yql/essentials/core/yql_graph_transformer.h> +#include <yql/essentials/core/yql_opt_utils.h> +#include <yql/essentials/ast/yql_expr.h> +#include <yql/essentials/utils/log/log.h> + +#include <library/cpp/yson/node/node_io.h> + +#include <util/string/cast.h> +#include <util/string/hex.h> +#include <util/generic/xrange.h> +#include <util/generic/utility.h> +#include <util/generic/algorithm.h> +#include <util/generic/bitmap.h> + +namespace NYql { + +using namespace NNodes; +using namespace NNodes::NDq; + +namespace { + +bool IsYtIsolatedLambdaImpl(const TExprNode& lambdaBody, TSyncMap& syncList, TString* usedCluster, bool supportsDq, TNodeSet& visited) { + if (!visited.insert(&lambdaBody).second) { + return true; + } + + if (TMaybeNode<TCoTypeOf>(&lambdaBody)) { + return true; + } + + if (auto maybeLength = TMaybeNode<TYtLength>(&lambdaBody)) { + if (auto maybeRead = maybeLength.Input().Maybe<TYtReadTable>()) { + auto read = maybeRead.Cast(); + if (usedCluster && !UpdateUsedCluster(*usedCluster, TString{read.DataSource().Cluster().Value()})) { + return false; + } + syncList.emplace(read.Ptr(), syncList.size()); + } + if (auto maybeOutput = maybeLength.Input().Maybe<TYtOutput>()) { + auto output = maybeOutput.Cast(); + if (usedCluster && !UpdateUsedCluster(*usedCluster, TString{GetOutputOp(output).DataSink().Cluster().Value()})) { + return false; + } + syncList.emplace(output.Operation().Ptr(), syncList.size()); + } + return true; + } + + if (auto maybeContent = TMaybeNode<TYtTableContent>(&lambdaBody)) { + if (auto maybeRead = maybeContent.Input().Maybe<TYtReadTable>()) { + auto read = maybeRead.Cast(); + if (usedCluster && !UpdateUsedCluster(*usedCluster, TString{read.DataSource().Cluster().Value()})) { + return false; + } + syncList.emplace(read.Ptr(), syncList.size()); + } + if (auto maybeOutput = maybeContent.Input().Maybe<TYtOutput>()) { + auto output = maybeOutput.Cast(); + if (usedCluster && !UpdateUsedCluster(*usedCluster, TString{GetOutputOp(output).DataSink().Cluster().Value()})) { + return false; + } + syncList.emplace(output.Operation().Ptr(), syncList.size()); + } + return true; + } + + if (auto maybeContent = TMaybeNode<TDqReadWrapBase>(&lambdaBody)) { + if (!supportsDq) { + return false; + } + if (auto maybeRead = maybeContent.Input().Maybe<TYtReadTable>()) { + auto read = maybeRead.Cast(); + if (usedCluster && !UpdateUsedCluster(*usedCluster, TString{read.DataSource().Cluster().Value()})) { + return false; + } + syncList.emplace(read.Ptr(), syncList.size()); + } + if (auto maybeOutput = maybeContent.Input().Maybe<TYtOutput>()) { + auto output = maybeOutput.Cast(); + if (usedCluster && !UpdateUsedCluster(*usedCluster, TString{GetOutputOp(output).DataSink().Cluster().Value()})) { + return false; + } + syncList.emplace(output.Operation().Ptr(), syncList.size()); + } + return true; + } + + if (!supportsDq && (TDqConnection::Match(&lambdaBody) || TDqPhyPrecompute::Match(&lambdaBody) || TDqStageBase::Match(&lambdaBody) || TDqSourceWrapBase::Match(&lambdaBody))) { + return false; + } + + if (auto maybeRead = TMaybeNode<TCoRight>(&lambdaBody).Input().Maybe<TYtReadTable>()) { + auto read = maybeRead.Cast(); + if (usedCluster && !UpdateUsedCluster(*usedCluster, TString{read.DataSource().Cluster().Value()})) { + return false; + } + syncList.emplace(read.Ptr(), syncList.size()); + return true; + } else if (auto out = TMaybeNode<TYtOutput>(&lambdaBody)) { + auto op = GetOutputOp(out.Cast()); + if (usedCluster && !UpdateUsedCluster(*usedCluster, TString{op.DataSink().Cluster().Value()})) { + return false; + } + syncList.emplace(out.Cast().Operation().Ptr(), syncList.size()); + return true; + } + + if (auto right = TMaybeNode<TCoRight>(&lambdaBody).Input()) { + if (auto maybeCons = right.Maybe<TCoCons>()) { + syncList.emplace(maybeCons.Cast().World().Ptr(), syncList.size()); + return IsYtIsolatedLambdaImpl(maybeCons.Cast().Input().Ref(), syncList, usedCluster, supportsDq, visited); + } + + if (right.Cast().Raw()->IsCallable("PgReadTable!")) { + syncList.emplace(right.Cast().Raw()->HeadPtr(), syncList.size()); + return true; + } + } + + if (lambdaBody.IsCallable("WithWorld")) { + syncList.emplace(lambdaBody.ChildPtr(1), syncList.size()); + return true; + } + + if (!lambdaBody.GetTypeAnn()->IsComposable()) { + return false; + } + + for (auto& child : lambdaBody.Children()) { + if (!IsYtIsolatedLambdaImpl(*child, syncList, usedCluster, supportsDq, visited)) { + return false; + } + } + + return true; +} + +IGraphTransformer::TStatus EstimateDataSize(TVector<ui64>& result, TSet<TString>& requestedColumns, + const TString& cluster, const TVector<TYtPathInfo::TPtr>& paths, + const TMaybe<TVector<TString>>& columns, const TYtState& state, TExprContext& ctx, bool sync) +{ + result.clear(); + result.resize(paths.size(), 0); + requestedColumns.clear(); + + const bool useColumnarStat = GetJoinCollectColumnarStatisticsMode(*state.Configuration) != EJoinCollectColumnarStatisticsMode::Disable + && !state.Types->UseTableMetaFromGraph; + + TVector<size_t> reqMap; + TVector<IYtGateway::TPathStatReq> pathStatReqs; + for (size_t i: xrange(paths.size())) { + const TYtPathInfo::TPtr& pathInfo = paths[i]; + YQL_ENSURE(pathInfo->Table->Stat); + result[i] = pathInfo->Table->Stat->DataSize; + if (pathInfo->Ranges) { + if (auto usedRows = pathInfo->Ranges->GetUsedRows(pathInfo->Table->Stat->RecordsCount)) { + if (usedRows.GetRef() && pathInfo->Table->Stat->RecordsCount) { + result[i] *= double(usedRows.GetRef()) / double(pathInfo->Table->Stat->RecordsCount); + } else { + result[i] = 0; + } + } + } + + if (useColumnarStat) { + TMaybe<TVector<TString>> overrideColumns; + if (columns && pathInfo->Table->RowSpec && (pathInfo->Table->RowSpec->StrictSchema || nullptr == FindPtr(*columns, YqlOthersColumnName))) { + overrideColumns = columns; + } + + auto ytPath = BuildYtPathForStatRequest(cluster, *pathInfo, overrideColumns, state, ctx); + if (!ytPath) { + return IGraphTransformer::TStatus::Error; + } + + if (ytPath->Columns_) { + pathStatReqs.push_back( + IYtGateway::TPathStatReq() + .Path(*ytPath) + .IsTemp(pathInfo->Table->IsTemp) + .IsAnonymous(pathInfo->Table->IsAnonymous) + .Epoch(pathInfo->Table->Epoch.GetOrElse(0)) + ); + reqMap.push_back(i); + } + } + } + + if (!pathStatReqs.empty()) { + for (auto& req : pathStatReqs) { + YQL_ENSURE(req.Path().Columns_); + requestedColumns.insert(req.Path().Columns_->Parts_.begin(), req.Path().Columns_->Parts_.end()); + } + + IYtGateway::TPathStatResult pathStats; + IYtGateway::TPathStatOptions pathStatOptions = + IYtGateway::TPathStatOptions(state.SessionId) + .Cluster(cluster) + .Paths(pathStatReqs) + .Config(state.Configuration->Snapshot()); + if (sync) { + auto future = state.Gateway->PathStat(std::move(pathStatOptions)); + pathStats = future.GetValueSync(); + pathStats.ReportIssues(ctx.IssueManager); + if (!pathStats.Success()) { + return IGraphTransformer::TStatus::Error; + } + } else { + pathStats = state.Gateway->TryPathStat(std::move(pathStatOptions)); + if (!pathStats.Success()) { + return IGraphTransformer::TStatus::Repeat; + } + } + + YQL_ENSURE(pathStats.DataSize.size() == reqMap.size()); + for (size_t i: xrange(pathStats.DataSize.size())) { + result[reqMap[i]] = pathStats.DataSize[i]; + } + } + + return IGraphTransformer::TStatus::Ok; +} + +bool NeedCalc(NNodes::TExprBase node) { + auto type = node.Ref().GetTypeAnn(); + if (type->IsSingleton()) { + return false; + } + + if (type->GetKind() == ETypeAnnotationKind::Optional) { + if (node.Maybe<TCoNothing>()) { + return false; + } + if (auto maybeJust = node.Maybe<TCoJust>()) { + return NeedCalc(maybeJust.Cast().Input()); + } + return true; + } + + if (type->GetKind() == ETypeAnnotationKind::Tuple) { + if (auto maybeTuple = node.Maybe<TExprList>()) { + return AnyOf(maybeTuple.Cast(), [](const auto& item) { return NeedCalc(item); }); + } + return true; + } + + if (type->GetKind() == ETypeAnnotationKind::List) { + if (node.Maybe<TCoList>()) { + YQL_ENSURE(node.Ref().ChildrenSize() == 1, "Should be rewritten to AsList"); + return false; + } + if (auto maybeAsList = node.Maybe<TCoAsList>()) { + return AnyOf(maybeAsList.Cast().Args(), [](const auto& item) { return NeedCalc(NNodes::TExprBase(item)); }); + } + return true; + } + + YQL_ENSURE(type->GetKind() == ETypeAnnotationKind::Data, + "Object of type " << *type << " should not be considered for calculation"); + + return !node.Maybe<TCoDataCtor>(); +} + +} // unnamed + +bool UpdateUsedCluster(TString& usedCluster, const TString& newCluster) { + if (!usedCluster) { + usedCluster = newCluster; + } else if (usedCluster != newCluster) { + return false; + } + return true; +} + +bool IsYtIsolatedLambda(const TExprNode& lambdaBody, TSyncMap& syncList, bool supportsDq) { + TNodeSet visited; + return IsYtIsolatedLambdaImpl(lambdaBody, syncList, nullptr, supportsDq, visited); +} + +bool IsYtIsolatedLambda(const TExprNode& lambdaBody, TSyncMap& syncList, TString& usedCluster, bool supportsDq) { + TNodeSet visited; + return IsYtIsolatedLambdaImpl(lambdaBody, syncList, &usedCluster, supportsDq, visited); +} + +bool IsYtCompleteIsolatedLambda(const TExprNode& lambda, TSyncMap& syncList, bool supportsDq) { + return lambda.IsComplete() && IsYtIsolatedLambda(lambda, syncList, supportsDq); +} + +bool IsYtCompleteIsolatedLambda(const TExprNode& lambda, TSyncMap& syncList, TString& usedCluster, bool supportsDq) { + return lambda.IsComplete() && IsYtIsolatedLambda(lambda, syncList, usedCluster, supportsDq); +} + +TExprNode::TPtr YtCleanupWorld(const TExprNode::TPtr& input, TExprContext& ctx, TYtState::TPtr state) { + TExprNode::TPtr output = input; + + TNodeOnNodeOwnedMap remaps; + VisitExpr(output, [&remaps, &ctx](const TExprNode::TPtr& node) { + if (TYtLength::Match(node.Get())) { + return false; + } + + if (TYtTableContent::Match(node.Get())) { + return false; + } + + if (auto read = TMaybeNode<TCoRight>(node).Input().Maybe<TYtReadTable>()) { + remaps[node.Get()] = Build<TYtTableContent>(ctx, node->Pos()) + .Input(read.Cast()) + .Settings().Build() + .Done().Ptr(); + + return false; + } + + if (TYtReadTable::Match(node.Get())) { + return false; + } + + if (node->IsCallable("WithWorld")) { + remaps[node.Get()] = node->HeadPtr(); + return false; + } + + TDynBitMap outs; + for (size_t i = 0; i < node->ChildrenSize(); ++i) { + if (TYtOutput::Match(node->Child(i))) { + outs.Set(i); + } + } + if (!outs.Empty()) { + auto res = node; + Y_FOR_EACH_BIT(i, outs) { + res = ctx.ChangeChild(*res, i, + Build<TYtTableContent>(ctx, node->Pos()) + .Input(node->ChildPtr(i)) + .Settings().Build() + .Done().Ptr() + ); + } + remaps[node.Get()] = res; + } + + if (TYtOutput::Match(node.Get())) { + return false; + } + + if (auto right = TMaybeNode<TCoRight>(node)) { + auto cons = right.Cast().Input().Maybe<TCoCons>(); + if (cons) { + remaps[node.Get()] = cons.Cast().Input().Ptr(); + return false; + } + + if (right.Cast().Input().Ref().IsCallable("PgReadTable!")) { + const auto& read = right.Cast().Input().Ref(); + remaps[node.Get()] = ctx.Builder(node->Pos()) + .Callable("PgTableContent") + .Add(0, read.Child(1)->TailPtr()) + .Add(1, read.ChildPtr(2)) + .Add(2, read.ChildPtr(3)) + .Add(3, read.ChildPtr(4)) + .Seal() + .Build(); + } + } + + return true; + }); + + if (output->IsLambda() && TYtOutput::Match(output->Child(1))) { + remaps[output->Child(1)] = Build<TYtTableContent>(ctx, output->Child(1)->Pos()) + .Input(output->ChildPtr(1)) + .Settings().Build() + .Done().Ptr(); + } + + IGraphTransformer::TStatus status = IGraphTransformer::TStatus::Ok; + if (!remaps.empty()) { + TOptimizeExprSettings settings(state->Types); + settings.VisitChanges = true; + settings.VisitTuples = true; + status = RemapExpr(output, output, remaps, ctx, settings); + } + + remaps.clear(); + TNodeSet visitedReadTables; + ui64 sumSize = 0; + TMaybe<TPositionHandle> bigPos; + VisitExpr(output, [&remaps, &ctx, &visitedReadTables, &sumSize, &bigPos, state](const TExprNode::TPtr& node) { + if (auto maybeRead = TMaybeNode<TYtReadTable>(node)) { + if (state->Types->EvaluationInProgress && + state->Configuration->EvaluationTableSizeLimit.Get() && + visitedReadTables.emplace(maybeRead.Cast().Raw()).second) { + for (auto section : TYtSectionList(maybeRead.Cast().Input())) { + for (auto path : section.Paths()) { + auto info = TYtTableBaseInfo::Parse(path.Table()); + if (info && info->Stat) { + sumSize += info->Stat->DataSize; + if (info->Stat->DataSize > *state->Configuration->EvaluationTableSizeLimit.Get()) { + bigPos = path.Table().Pos(); + } + } + } + } + } + + if (maybeRead.Cast().World().Ref().Type() != TExprNode::World) { + remaps[node.Get()] = ctx.ChangeChild(*node, 0, ctx.NewWorld(node->Pos())); + } + return false; + } + if (TYtOutput::Match(node.Get())) { + return false; + } + return true; + }); + + if (state->Types->EvaluationInProgress && state->Configuration->EvaluationTableSizeLimit.Get()) { + if (sumSize > *state->Configuration->EvaluationTableSizeLimit.Get()) { + ctx.AddError(TIssue(ctx.GetPosition(bigPos.GetOrElse(input->Pos())), TStringBuilder() << "Too large table(s) for evaluation pass: " + << sumSize << " > " << *state->Configuration->EvaluationTableSizeLimit.Get())); + return nullptr; + } + } + + if (!remaps.empty()) { + TOptimizeExprSettings settings(state->Types); + settings.VisitChanges = true; + status = status.Combine(RemapExpr(output, output, remaps, ctx, settings)); + } + + YQL_ENSURE(status.Level != IGraphTransformer::TStatus::Error, "Bad input graph"); + + if (state->Types->EvaluationInProgress) { + status = status.Combine(SubstTables(output, state, false, ctx)); + YQL_ENSURE(status.Level != IGraphTransformer::TStatus::Error, "Subst tables failed"); + } + + return output; +} + +TYtOutputOpBase GetOutputOp(TYtOutput output) { + if (const auto tr = output.Operation().Maybe<TYtTryFirst>()) { + return tr.Cast().Second(); + } + return output.Operation().Cast<TYtOutputOpBase>(); +} + +TVector<TYtTableBaseInfo::TPtr> GetInputTableInfos(TExprBase input) { + TVector<TYtTableBaseInfo::TPtr> res; + if (auto out = input.Maybe<TYtOutput>()) { + res.push_back(MakeIntrusive<TYtOutTableInfo>(GetOutTable(out.Cast()))); + res.back()->IsUnordered = IsUnorderedOutput(out.Cast()); + } else { + auto read = input.Maybe<TCoRight>().Input().Maybe<TYtReadTable>(); + YQL_ENSURE(read, "Unknown operation input"); + for (auto section: read.Cast().Input()) { + for (auto path: section.Paths()) { + res.push_back(TYtTableBaseInfo::Parse(path.Table())); + } + } + } + return res; +} + +TVector<TYtPathInfo::TPtr> GetInputPaths(TExprBase input) { + TVector<TYtPathInfo::TPtr> res; + if (auto out = input.Maybe<TYtOutput>()) { + res.push_back(MakeIntrusive<TYtPathInfo>()); + res.back()->Table = MakeIntrusive<TYtOutTableInfo>(GetOutTable(out.Cast())); + res.back()->Table->IsUnordered = IsUnorderedOutput(out.Cast()); + } else { + auto read = input.Maybe<TCoRight>().Input().Maybe<TYtReadTable>(); + YQL_ENSURE(read, "Unknown operation input"); + for (auto section: read.Cast().Input()) { + for (auto path: section.Paths()) { + res.push_back(MakeIntrusive<TYtPathInfo>(path)); + } + } + } + return res; +} + +TStringBuf GetClusterName(NNodes::TExprBase input) { + if (auto read = input.Maybe<TCoRight>().Input().Maybe<TYtReadTable>()) { + return read.Cast().DataSource().Cluster().Value(); + } else if (auto output = input.Maybe<TYtOutput>()) { + return GetOutputOp(output.Cast()).DataSink().Cluster().Value(); + } else if (auto op = input.Maybe<TCoRight>().Input().Maybe<TYtOutputOpBase>()) { + return op.Cast().DataSink().Cluster().Value(); + } else { + YQL_ENSURE(false, "Unknown operation input"); + } +} + +bool IsYtProviderInput(NNodes::TExprBase input, bool withVariantList) { + if (input.Maybe<TYtOutput>()) { + return true; + } + if (auto maybeYtInput = input.Maybe<TCoRight>().Input()) { + if (withVariantList && maybeYtInput.Maybe<TYtOutputOpBase>()) { + return true; + } + if (auto maybeRead = maybeYtInput.Maybe<TYtReadTable>()) { + return withVariantList || maybeRead.Cast().Input().Size() == 1; + } + } + return false; +} + +bool IsConstExpSortDirections(NNodes::TExprBase sortDirections) { + if (sortDirections.Maybe<TCoBool>()) { + return true; + } else if (sortDirections.Maybe<TExprList>()) { + for (auto child: sortDirections.Cast<TExprList>()) { + if (!child.Maybe<TCoBool>()) { + return false; + } + } + return true; + } + return false; +} + +TExprNode::TListType GetNodesToCalculate(const TExprNode::TPtr& input) { + TExprNode::TListType needCalc; + TNodeSet uniqNodes; + VisitExpr(input, [&needCalc, &uniqNodes](const TExprNode::TPtr& node) { + if (auto maybeOp = TMaybeNode<TYtTransientOpBase>(node)) { + auto op = maybeOp.Cast(); + for (auto setting: op.Settings()) { + switch (FromString<EYtSettingType>(setting.Name().Value())) { + case EYtSettingType::Limit: + for (auto expr: setting.Value().Cast().Ref().Children()) { + for (auto item: expr->Children()) { + if (uniqNodes.insert(item->Child(1)).second) { + if (NeedCalc(TExprBase(item->Child(1)))) { + needCalc.push_back(item->ChildPtr(1)); + } + } + } + } + break; + default: + break; + } + } + } + else if (auto maybeSection = TMaybeNode<TYtSection>(node)) { + TYtSection section = maybeSection.Cast(); + for (auto setting: section.Settings()) { + switch (FromString<EYtSettingType>(setting.Name().Value())) { + case EYtSettingType::Take: + case EYtSettingType::Skip: + if (uniqNodes.insert(setting.Value().Cast().Raw()).second) { + if (NeedCalc(setting.Value().Cast())) { + needCalc.push_back(setting.Value().Cast().Ptr()); + } + } + break; + case EYtSettingType::KeyFilter: { + auto value = setting.Value().Cast<TExprList>(); + if (value.Size() > 0) { + for (auto member: value.Item(0).Cast<TCoNameValueTupleList>()) { + for (auto cmp: member.Value().Cast<TCoNameValueTupleList>()) { + if (cmp.Value() && uniqNodes.insert(cmp.Value().Cast().Raw()).second) { + if (NeedCalc(cmp.Value().Cast())) { + needCalc.push_back(cmp.Value().Cast().Ptr()); + } + } + } + } + } + break; + } + case EYtSettingType::KeyFilter2: { + auto value = setting.Value().Cast<TExprList>(); + if (value.Size() > 0) { + if (uniqNodes.insert(value.Item(0).Raw()).second && NeedCalc(value.Item(0))) { + needCalc.push_back(value.Item(0).Ptr()); + } + } + break; + } + default: + break; + } + } + } + else if (TMaybeNode<TYtOutput>(node)) { + // Stop traversing dependent operations + return false; + } + return true; + }); + return needCalc; +} + +bool HasNodesToCalculate(const TExprNode::TPtr& input) { + bool needCalc = false; + VisitExpr(input, [&needCalc](const TExprNode::TPtr& node) { + if (auto maybeOp = TMaybeNode<TYtTransientOpBase>(node)) { + auto op = maybeOp.Cast(); + for (auto setting: op.Settings()) { + switch (FromString<EYtSettingType>(setting.Name().Value())) { + case EYtSettingType::Limit: + for (auto expr: setting.Value().Cast().Ref().Children()) { + for (auto item: expr->Children()) { + if (NeedCalc(TExprBase(item->Child(1)))) { + needCalc = true; + return false; + } + } + } + break; + default: + break; + } + } + } + else if (auto maybeSection = TMaybeNode<TYtSection>(node)) { + TYtSection section = maybeSection.Cast(); + for (auto setting: section.Settings()) { + switch (FromString<EYtSettingType>(setting.Name().Value())) { + case EYtSettingType::Take: + case EYtSettingType::Skip: + if (NeedCalc(setting.Value().Cast())) { + needCalc = true; + return false; + } + break; + case EYtSettingType::KeyFilter: { + auto value = setting.Value().Cast<TExprList>(); + if (value.Size() > 0) { + for (auto member: value.Item(0).Cast<TCoNameValueTupleList>()) { + for (auto cmp: member.Value().Cast<TCoNameValueTupleList>()) { + if (cmp.Value() && NeedCalc(cmp.Value().Cast())) { + needCalc = true; + return false; + } + } + } + } + break; + } + case EYtSettingType::KeyFilter2: { + auto value = setting.Value().Cast<TExprList>(); + if (value.Size() > 0) { + if (value.Item(0).Raw() && NeedCalc(value.Item(0))) { + needCalc = true; + return false; + } + } + break; + } + default: + break; + } + } + } + else if (TMaybeNode<TYtOutput>(node)) { + // Stop traversing dependent operations + return false; + } + return !needCalc; + }); + return needCalc; +} + +std::pair<IGraphTransformer::TStatus, TAsyncTransformCallbackFuture> CalculateNodes(TYtState::TPtr state, + const TExprNode::TPtr& input, + const TString& cluster, + const TExprNode::TListType& needCalc, + TExprContext& ctx) +{ + YQL_ENSURE(!needCalc.empty()); + YQL_ENSURE(!input->HasResult(), "Infinitive calculation loop detected"); + TNodeMap<size_t> calcNodes; + TUserDataTable files; + + TExprNode::TPtr list = ctx.NewList(input->Pos(), TExprNode::TListType(needCalc)); + TTypeAnnotationNode::TListType tupleTypes; + std::transform(needCalc.cbegin(), needCalc.cend(), std::back_inserter(tupleTypes), [](const TExprNode::TPtr& n) { return n->GetTypeAnn(); }); + list->SetTypeAnn(ctx.MakeType<TTupleExprType>(tupleTypes)); + list->SetState(TExprNode::EState::ConstrComplete); + + auto status = SubstTables(list, state, /*anonOnly*/true, ctx); + if (status.Level == IGraphTransformer::TStatus::Error) { + return SyncStatus(status); + } + + auto callableTransformer = CreateExtCallableTypeAnnotationTransformer(*state->Types); + auto typeTransformer = CreateTypeAnnotationTransformer(callableTransformer, *state->Types); + + TExprNode::TPtr optimized; + status = UpdateTableContentMemoryUsage(list, optimized, state, ctx, true); + if (status.Level == IGraphTransformer::TStatus::Error) { + return SyncStatus(status); + } + + bool hasNonDeterministicFunctions = false; + status = PeepHoleOptimizeNode(list, optimized, ctx, *state->Types, typeTransformer.Get(), hasNonDeterministicFunctions); + if (status.Level == IGraphTransformer::TStatus::Error) { + return SyncStatus(status); + } + + auto filesRes = NCommon::FreezeUsedFiles(*optimized, files, *state->Types, ctx, MakeUserFilesDownloadFilter(*state->Gateway, cluster)); + if (filesRes.first.Level != IGraphTransformer::TStatus::Ok) { + return filesRes; + } + + TString calcHash; + auto config = state->Configuration->GetSettingsForNode(*input); + const auto queryCacheMode = config->QueryCacheMode.Get().GetOrElse(EQueryCacheMode::Disable); + if (queryCacheMode != EQueryCacheMode::Disable) { + if (!hasNonDeterministicFunctions && config->QueryCacheUseForCalc.Get().GetOrElse(true)) { + calcHash = TYtNodeHashCalculator(state, cluster, config).GetHash(*list); + } + YQL_CLOG(DEBUG, ProviderYt) << "Calc hash: " << HexEncode(calcHash).Quote() + << ", cache mode: " << queryCacheMode; + } + + for (size_t i: xrange(needCalc.size())) { + calcNodes.emplace(needCalc[i].Get(), i); + } + + THashMap<TString, TString> secureParams; + NCommon::FillSecureParams(input, *state->Types, secureParams); + + auto future = state->Gateway->Calc(optimized->ChildrenList(), ctx, + IYtGateway::TCalcOptions(state->SessionId) + .Cluster(cluster) + .UserDataBlocks(files) + .UdfModules(state->Types->UdfModules) + .UdfResolver(state->Types->UdfResolver) + .UdfValidateMode(state->Types->ValidateMode) + .PublicId(state->Types->TranslateOperationId(input->UniqueId())) + .Config(state->Configuration->GetSettingsForNode(*input)) + .OptLLVM(state->Types->OptLLVM.GetOrElse(TString())) + .OperationHash(calcHash) + .SecureParams(secureParams) + ); + return WrapFutureCallback(future, [state, calcNodes](const IYtGateway::TCalcResult& res, const TExprNode::TPtr& input, TExprNode::TPtr& output, TExprContext& ctx) { + YQL_ENSURE(res.Data.size() == calcNodes.size()); + + TProcessedNodesSet processedNodes; + if (TYtOpBase::Match(input.Get())) { + processedNodes.insert(input->Child(TYtOpBase::idx_World)->UniqueId()); + } + VisitExpr(input, [&processedNodes](const TExprNode::TPtr& node) { + if (TYtOutput::Match(node.Get())) { + // Stop traversing dependent operations + processedNodes.insert(node->UniqueId()); + return false; + } + return true; + }); + + TNodeOnNodeOwnedMap remaps; + for (auto& it: calcNodes) { + auto node = it.first; + auto type = node->GetTypeAnn(); + YQL_ENSURE(type); + NYT::TNode data = res.Data[it.second]; + remaps.emplace(node, NCommon::NodeToExprLiteral(node->Pos(), *type, data, ctx)); + } + TOptimizeExprSettings settings(state->Types); + settings.VisitChanges = true; + settings.VisitStarted = true; + settings.ProcessedNodes = &processedNodes; + auto status = RemapExpr(input, output, remaps, ctx, settings); + + if (status.Level == IGraphTransformer::TStatus::Error) { + return status; + } + input->SetState(TExprNode::EState::ExecutionComplete); + output->SetResult(ctx.NewAtom(output->Pos(), "calc")); // Special marker to check infinitive loop + return status.Combine(IGraphTransformer::TStatus::Repeat); + }); +} + +TMaybe<ui64> GetLimit(const TExprNode& settings) { + auto limitNode = NYql::GetSetting(settings, EYtSettingType::Limit); + if (!limitNode) { + return Nothing(); + } + limitNode = limitNode->ChildPtr(1); + + TMaybe<ui64> limit; + for (auto part: limitNode->Children()) { + TRecordsRange partialRange; + partialRange.Fill(*part); + if (!partialRange.Limit.Defined()) { + return Nothing(); + } + + // check overflow + if (std::numeric_limits<ui64>::max() - partialRange.Limit.GetRef() < partialRange.Offset.GetOrElse(0)) { + return Nothing(); + } + + if (!limit.Defined()) { + limit = partialRange.Limit.GetRef() + partialRange.Offset.GetOrElse(0); + } else { + limit = Max(limit.GetRef(), partialRange.Limit.GetRef() + partialRange.Offset.GetOrElse(0)); + } + } + + return limit == std::numeric_limits<ui64>::max() ? Nothing() : limit; +} + +TExprNode::TPtr GetLimitExpr(const TExprNode::TPtr& limitSetting, TExprContext& ctx) { + auto limitItems = limitSetting->ChildPtr(1); + TExprNode::TListType limitValues; + for (const auto& child : limitItems->Children()) { + TExprNode::TPtr skip, take; + for (auto& setting: child->Children()) { + if (setting->ChildrenSize() == 0) { + continue; + } + + auto settingName = setting->Child(0)->Content(); + if (settingName == TStringBuf("take")) { + take = setting->ChildPtr(1); + } else if (settingName == TStringBuf("skip")) { + skip = setting->ChildPtr(1); + } + } + + if (!take) { + return nullptr; + } + + if (skip) { + limitValues.push_back(ctx.NewCallable(child->Pos(), "+", { take, skip })); + } else { + limitValues.push_back(take); + } + } + + if (limitValues.empty()) { + return nullptr; + } + + if (limitValues.size() == 1) { + return limitValues.front(); + } + + return ctx.NewCallable(limitSetting->Pos(), "Max", std::move(limitValues)); +} + +IGraphTransformer::TStatus UpdateTableMeta(const TExprNode::TPtr& tableNode, TExprNode::TPtr& newTableNode, + const TYtTablesData::TPtr& tablesData, bool checkSqlView, bool updateRowSpecType, TExprContext& ctx) +{ + newTableNode = tableNode; + TYtTableInfo tableInfo = tableNode; + const TYtTableDescription& tableDesc = tablesData->GetTable(tableInfo.Cluster, tableInfo.Name, tableInfo.Epoch); + const bool withQB = NYql::HasSetting(tableInfo.Settings.Ref(), EYtSettingType::WithQB); + const bool hasUserSchema = NYql::HasSetting(tableInfo.Settings.Ref(), EYtSettingType::UserSchema); + const bool hasUserColumns = NYql::HasSetting(tableInfo.Settings.Ref(), EYtSettingType::UserColumns); + bool update = false; + + auto rowSpec = withQB ? tableDesc.QB2RowSpec : tableDesc.RowSpec; + if (updateRowSpecType) { + if (rowSpec && tableInfo.RowSpec && !rowSpec->GetType()) { + rowSpec->CopyType(*tableInfo.RowSpec); + rowSpec->SortedByTypes = tableInfo.RowSpec->SortedByTypes; + } + } + + if (!tableInfo.Stat) { + if (tableDesc.Stat) { + tableInfo.Stat = tableDesc.Stat; + update = true; + } + else if (tableDesc.Meta && tableDesc.Meta->DoesExist && tableInfo.Epoch.GetOrElse(0) == 0) { + ctx.AddError(TIssue(ctx.GetPosition(tableNode->Pos()), TStringBuilder() << + "Table " << tableInfo.Name << " stat was not loaded")); + return IGraphTransformer::TStatus::Error; + } + } + if (!tableInfo.Meta) { + if (!tableDesc.Meta) { + if (tableInfo.Epoch.GetOrElse(0) != 0) { + return IGraphTransformer::TStatus(IGraphTransformer::TStatus::Repeat, true); + } + ctx.AddError(TIssue(ctx.GetPosition(tableNode->Pos()), TStringBuilder() << + "Table " << tableInfo.Name << " metadata was not loaded")); + return IGraphTransformer::TStatus::Error; + } + + tableInfo.Meta = tableDesc.Meta; + tableInfo.RowSpec = rowSpec; + update = true; + } + else if (rowSpec && !tableInfo.RowSpec) { + tableInfo.RowSpec = rowSpec; + update = true; + } + + if (checkSqlView && tableInfo.Meta->SqlView) { + ctx.AddError(TIssue(ctx.GetPosition(tableNode->Pos()), TStringBuilder() + << "Reading from " << tableInfo.Name.Quote() << " view is not supported")); + return IGraphTransformer::TStatus::Error; + } + + if (hasUserSchema || hasUserColumns) { + const auto setting = GetSetting(tableInfo.Settings.Ref(), hasUserSchema ? EYtSettingType::UserSchema : EYtSettingType::UserColumns); + auto type = setting->Tail().GetTypeAnn()->Cast<TTypeExprType>()->GetType()->Cast<TStructExprType>(); + const auto prevRowSpec = tableInfo.RowSpec; + if (!(prevRowSpec && prevRowSpec->StrictSchema) && type->Cast<TStructExprType>()->FindItem("_other")) { + ctx.AddError(TIssue(ctx.GetPosition(setting->Tail().Pos()), "It is forbidden to specify the column '_other'.")); + return IGraphTransformer::TStatus::Error; + } + + TVector<TString> explicitYson; + if (prevRowSpec && hasUserColumns) { + const bool hasNativeFlags = prevRowSpec->GetNativeYtTypeFlags() != 0; + // patch original type + auto items = prevRowSpec->GetType()->GetItems(); + for (const auto& newItem : type->GetItems()) { + if (auto pos = prevRowSpec->GetType()->FindItem(newItem->GetName())) { + if (hasNativeFlags) { + bool isOptional = false; + const TDataExprType* dataType = nullptr; + if (IsDataOrOptionalOfData(items[*pos]->GetItemType(), isOptional, dataType) + && dataType->GetSlot() == EDataSlot::Yson + && !IsDataOrOptionalOfData(newItem->GetItemType())) + { + explicitYson.emplace_back(newItem->GetName()); + } + } + items[*pos] = ctx.MakeType<TItemExprType>(newItem->GetName(), newItem->GetItemType()); + } else { + items.push_back(newItem); + } + } + + type = ctx.MakeType<TStructExprType>(items); + } + + if ((prevRowSpec && !IsSameAnnotation(*prevRowSpec->GetType(), *type)) || (!prevRowSpec && hasUserSchema)) { + update = true; + + auto strict = hasUserSchema; + if (hasUserColumns) { + if (prevRowSpec) { + strict = prevRowSpec->StrictSchema; + } + } + + tableInfo.RowSpec = MakeIntrusive<TYqlRowSpecInfo>(); + tableInfo.RowSpec->SetType(type, prevRowSpec ? prevRowSpec->GetNativeYtTypeFlags() : 0ul); + tableInfo.RowSpec->UniqueKeys = false; + tableInfo.RowSpec->StrictSchema = strict; + tableInfo.RowSpec->ExplicitYson = explicitYson; + + if (prevRowSpec) { + if (auto nativeType = prevRowSpec->GetNativeYtType()) { + tableInfo.RowSpec->CopyTypeOrders(*nativeType); + } + if (prevRowSpec->IsSorted()) { + tableInfo.RowSpec->CopySortness(ctx, *prevRowSpec, TYqlRowSpecInfo::ECopySort::WithDesc); + tableInfo.RowSpec->MakeCommonSortness(ctx, *prevRowSpec); // Truncated keys with changed types + } + } + } + } else { + if (!update && rowSpec && tableInfo.RowSpec && (!rowSpec->CompareSortness(*tableInfo.RowSpec) || rowSpec->GetNativeYtType() != tableInfo.RowSpec->GetNativeYtType())) { + tableInfo.RowSpec = rowSpec; + update = true; + } + } + + if (update) { + newTableNode = tableInfo.ToExprNode(ctx, tableNode->Pos()).Ptr(); + return IGraphTransformer::TStatus::Repeat; + } + + return IGraphTransformer::TStatus::Ok; +} + +TExprNode::TPtr ValidateAndUpdateTablesMeta(const TExprNode::TPtr& input, TStringBuf cluster, const TYtTablesData::TPtr& tablesData, bool updateRowSpecType, TExprContext& ctx) { + TNodeSet tables; + VisitExpr(input, [&](const TExprNode::TPtr& node) { + if (auto maybeTable = TMaybeNode<TYtTable>(node)) { + tables.insert(maybeTable.Cast().Raw()); + return false; + } + else if (TMaybeNode<TYtOutput>(node)) { + // Don't traverse deeper to inner operations + return false; + } + return true; + }); + + if (!tables.empty()) { + bool valid = true; + for (auto table: tables) { + if (cluster != table->Child(TYtTable::idx_Cluster)->Content()) { + ctx.AddError(TIssue(ctx.GetPosition(table->Child(TYtTable::idx_Cluster)->Pos()), TStringBuilder() + << "Table " << TString{table->Child(TYtTable::idx_Name)->Content()}.Quote() + << " cluster doesn't match DataSource/DataSink cluster: " + << TString{table->Child(TYtTable::idx_Cluster)->Content()}.Quote() << " != " << TString{cluster}.Quote())); + valid = false; + } + } + if (!valid) { + return {}; + } + + TOptimizeExprSettings settings(nullptr); + settings.VisitChanges = true; + TExprNode::TPtr output = input; + auto status = OptimizeExpr(input, output, [&](const TExprNode::TPtr& node, TExprContext& ctx) -> TExprNode::TPtr { + if (tables.find(node.Get()) != tables.cend()) { + if (!TYtTableInfo::HasSubstAnonymousLabel(TExprBase(node))) { + TExprNode::TPtr newNode; + auto status = UpdateTableMeta(node, newNode, tablesData, true, updateRowSpecType, ctx); + if (IGraphTransformer::TStatus::Error == status.Level) { + return {}; + } + return newNode; + } + } + return node; + }, ctx, settings); + + if (IGraphTransformer::TStatus::Error == status.Level) { + return {}; + } + return output; + } + + return input; +} + +TExprNode::TPtr ResetTableMeta(const TExprNode::TPtr& tableNode, TExprContext& ctx) { + TExprNode::TListType children; + for (auto id: {TYtTable::idx_Meta, TYtTable::idx_Stat, TYtTable::idx_RowSpec}) { + if (!TCoVoid::Match(tableNode->Child(id))) { + if (children.empty()) { + children = tableNode->ChildrenList(); + } + children[id] = ctx.NewCallable(tableNode->Pos(), TCoVoid::CallableName(), {}); + } + } + if (children.empty()) { + return tableNode; + } + return ctx.ChangeChildren(*tableNode, std::move(children)); +} + +TExprNode::TPtr ResetOutTableMeta(const TExprNode::TPtr& tableNode, TExprContext& ctx) { + TExprNode::TListType children; + if (!TCoVoid::Match(tableNode->Child(TYtOutTable::idx_Stat))) { + if (children.empty()) { + children = tableNode->ChildrenList(); + } + children[TYtOutTable::idx_Stat] = ctx.NewCallable(tableNode->Pos(), TCoVoid::CallableName(), {}); + } + + if (tableNode->Child(TYtOutTable::idx_Name)->Content()) { + if (children.empty()) { + children = tableNode->ChildrenList(); + } + children[TYtOutTable::idx_Name] = ctx.NewAtom(tableNode->Pos(), TStringBuf()); + } + + if (children.empty()) { + return tableNode; + } + return ctx.ChangeChildren(*tableNode, std::move(children)); +} + +TExprNode::TPtr ResetTablesMeta(const TExprNode::TPtr& input, TExprContext& ctx, bool resetTmpOnly, bool isEvaluationInProgress) { + TNodeSet tables; + TNodeSet outTables; + VisitExpr(input, [&](const TExprNode::TPtr& node) { + if (auto maybeTable = TMaybeNode<TYtTable>(node)) { + const bool isAnonymous = NYql::HasSetting(maybeTable.Cast().Settings().Ref(), EYtSettingType::Anonymous); + if (!resetTmpOnly && !(isEvaluationInProgress && isAnonymous)) { + if (!TCoVoid::Match(maybeTable.Stat().Raw()) || !TCoVoid::Match(maybeTable.Meta().Raw()) || !TCoVoid::Match(maybeTable.RowSpec().Raw())) { + tables.insert(maybeTable.Raw()); + } + } + return false; + } + else if (auto maybeTable = TMaybeNode<TYtOutTable>(node)) { + if (!isEvaluationInProgress) { + if (!TCoVoid::Match(maybeTable.Stat().Raw()) || maybeTable.Cast().Name().Value()) { + outTables.insert(maybeTable.Raw()); + } + } + return false; + } + else if (TMaybeNode<TYtOutput>(node)) { + // Don't traverse deeper to inner operations + return false; + } + return true; + }); + + if (!tables.empty() || !outTables.empty()) { + TOptimizeExprSettings settings(nullptr); + settings.VisitChanges = true; + TExprNode::TPtr output = input; + auto status = OptimizeExpr(input, output, [&](const TExprNode::TPtr& node, TExprContext& ctx) -> TExprNode::TPtr { + if (tables.find(node.Get()) != tables.cend()) { + return ResetTableMeta(node, ctx); + } + else if (outTables.find(node.Get()) != outTables.cend()) { + return ResetOutTableMeta(node, ctx); + } + return node; + }, ctx, settings); + + if (IGraphTransformer::TStatus::Error == status.Level) { + return {}; + } + return output; + } + + return input; +} + +std::pair<TExprBase, TString> GetOutTableWithCluster(TExprBase ytOutput) { + const auto output = ytOutput.Cast<TYtOutput>(); + const auto op = GetOutputOp(output); + const auto cluster = TString{ op.DataSink().Cluster().Value() }; + size_t ndx = 0; + YQL_ENSURE(TryFromString<size_t>(output.OutIndex().Value(), ndx), "Bad " << TYtOutput::CallableName() << " output index value"); + const auto opOut = op.Output(); + YQL_ENSURE(ndx < opOut.Size()); + return { opOut.Item(ndx), cluster }; +} + +TExprBase GetOutTable(TExprBase ytOutput) { + return GetOutTableWithCluster(ytOutput).first; +} + +TMaybeNode<TCoFlatMapBase> GetFlatMapOverInputStream(TCoLambda opLambda, const TParentsMap& parentsMap) { + TMaybeNode<TCoFlatMapBase> map; + if (const auto it = parentsMap.find(opLambda.Args().Arg(0).Raw()); parentsMap.cend() != it) { + for (const auto& parent : it->second) { + if (!map) { + if (map = TMaybeNode<TCoFlatMapBase>(parent)) + continue; + } + + if (!TCoDependsOn::Match(parent)) { + map = {}; + break; + } + } + } + + return map; +} + +TMaybeNode<TCoFlatMapBase> GetFlatMapOverInputStream(TCoLambda opLambda) { + TParentsMap parentsMap; + GatherParents(opLambda.Body().Ref(), parentsMap); + return GetFlatMapOverInputStream(opLambda, parentsMap); +} + +TExprNode::TPtr ToOutTableWithHash(TExprBase output, const TYtState::TPtr& state, TExprContext& ctx) { + auto [outTableNode, cluster] = GetOutTableWithCluster(output); + auto outTable = outTableNode.Ptr(); + auto hash = TYtNodeHashCalculator(state, cluster, state->Configuration->Snapshot()).GetHash(output.Ref()); + outTable = ctx.ChangeChild(*outTable, TYtOutTable::idx_Settings, + NYql::AddSetting(*outTable->Child(TYtOutTable::idx_Settings), EYtSettingType::OpHash, ctx.NewAtom(output.Pos(), HexEncode(hash)), ctx) + ); + return outTable; +} + +IGraphTransformer::TStatus SubstTables(TExprNode::TPtr& input, const TYtState::TPtr& state, bool anonOnly, TExprContext& ctx) +{ + TProcessedNodesSet processedNodes; + VisitExpr(input, [&processedNodes](const TExprNode::TPtr& node) { + if (TYtOutput::Match(node.Get())) { + // Stop traversing dependent operations + processedNodes.insert(node->UniqueId()); + return false; + } + return true; + }); + + TOptimizeExprSettings settings(state->Types); + settings.VisitChanges = true; + settings.VisitStarted = true; + settings.CustomInstantTypeTransformer = state->Types->CustomInstantTypeTransformer.Get(); + settings.ProcessedNodes = &processedNodes; + + TExprNode::TPtr optimizedInput = input; + auto status = OptimizeExpr(optimizedInput, optimizedInput, [&](const TExprNode::TPtr& node, TExprContext& ctx) -> TExprNode::TPtr { + if (auto maybeTable = TMaybeNode<TYtTable>(node)) { + auto table = maybeTable.Cast(); + if (auto anon = NYql::GetSetting(table.Settings().Ref(), EYtSettingType::Anonymous)) { + if (anon->ChildrenSize() == 1) { + TString cluster = TString{table.Cluster().Value()}; + TString anonTableName = TString{table.Name().Value()}; + TString realTableName = state->AnonymousLabels.Value(std::make_pair(cluster, anonTableName), TString()); + if (!realTableName) { + ctx.AddError(TIssue(ctx.GetPosition(table.Pos()), TStringBuilder() << "Unaccounted anonymous table: " << cluster << '.' << anonTableName)); + return {}; + } + auto children = node->ChildrenList(); + children[TYtTable::idx_Name] = ctx.NewAtom(node->Pos(), realTableName); + children[TYtTable::idx_Settings] = NYql::AddSetting( + *NYql::RemoveSetting(table.Settings().Ref(), EYtSettingType::Anonymous, ctx), + EYtSettingType::Anonymous, ctx.NewAtom(node->Pos(), anonTableName), ctx); + return ctx.ChangeChildren(*node, std::move(children)); + } + } + } + + return node; + }, ctx, settings); + + if (status.Level == IGraphTransformer::TStatus::Error) { + return status; + } + + if (!anonOnly) { + const bool useQueryCache = state->Configuration->QueryCacheMode.Get().GetOrElse(EQueryCacheMode::Disable) != EQueryCacheMode::Disable + && state->Configuration->QueryCacheUseForCalc.Get().GetOrElse(true); + + TNodeOnNodeOwnedMap toOpt; + VisitExpr(optimizedInput, [&toOpt, &state, useQueryCache, &ctx](const TExprNode::TPtr& node) { + if (auto maybePath = TMaybeNode<TYtPath>(node)) { + if (maybePath.Table().Maybe<TYtOutput>()) { + auto path = maybePath.Cast(); + toOpt[node.Get()] = Build<TYtPath>(ctx, node->Pos()) + .InitFrom(path) + .Table(useQueryCache ? ToOutTableWithHash(path.Table(), state, ctx) : GetOutTable(path.Table()).Ptr()) + .Done().Ptr(); + } + return false; + } + if (TMaybeNode<TYtLength>(node).Input().Maybe<TYtOutput>()) { + auto length = TYtLength(node); + toOpt[node.Get()] = Build<TYtLength>(ctx, node->Pos()) + .InitFrom(length) + .Input<TYtReadTable>() + .World<TCoWorld>().Build() + .DataSource(ctx.RenameNode(GetOutputOp(length.Input().Cast<TYtOutput>()).DataSink().Ref(), TYtDSource::CallableName())) + .Input() + .Add() + .Paths() + .Add() + .Table(useQueryCache ? ToOutTableWithHash(length.Input(), state, ctx) : GetOutTable(length.Input()).Ptr()) + .Columns<TCoVoid>().Build() + .Ranges<TCoVoid>().Build() + .Stat<TCoVoid>().Build() + .Build() + .Build() + .Settings() + .Build() + .Build() + .Build() + .Build() + .Done().Ptr(); + return false; + } + if (TMaybeNode<TYtTableContent>(node).Input().Maybe<TYtOutput>()) { + auto content = TYtTableContent(node); + toOpt[node.Get()] = Build<TYtTableContent>(ctx, node->Pos()) + .InitFrom(content) + .Input<TYtReadTable>() + .World<TCoWorld>().Build() + .DataSource(ctx.RenameNode(GetOutputOp(content.Input().Cast<TYtOutput>()).DataSink().Ref(), TYtDSource::CallableName())) + .Input() + .Add() + .Paths() + .Add() + .Table(useQueryCache ? ToOutTableWithHash(content.Input(), state, ctx) : GetOutTable(content.Input()).Ptr()) + .Columns<TCoVoid>().Build() + .Ranges<TCoVoid>().Build() + .Stat<TCoVoid>().Build() + .Build() + .Build() + .Settings() + .Build() + .Build() + .Build() + .Build() + .Done().Ptr(); + return false; + } + if (auto maybeOut = TMaybeNode<TYtOutput>(node)) { + auto out = maybeOut.Cast(); + toOpt[node.Get()] = Build<TCoRight>(ctx, node->Pos()) + .Input<TYtReadTable>() + .World<TCoWorld>().Build() + .DataSource(ctx.RenameNode(GetOutputOp(out).DataSink().Ref(), TYtDSource::CallableName())) + .Input() + .Add() + .Paths() + .Add() + .Table(useQueryCache ? ToOutTableWithHash(out, state, ctx) : GetOutTable(out).Ptr()) + .Columns<TCoVoid>().Build() + .Ranges<TCoVoid>().Build() + .Stat<TCoVoid>().Build() + .Build() + .Build() + .Settings() + .Build() + .Build() + .Build() + .Build() + .Done().Ptr(); + return false; + } + return true; + }); + + if (!toOpt.empty()) { + settings.ProcessedNodes = nullptr; + status = RemapExpr(optimizedInput, optimizedInput, toOpt, ctx, settings); + if (status.Level == IGraphTransformer::TStatus::Error) { + return status; + } + } + } + + if (optimizedInput != input) { + auto typeTransformer = CreateTypeAnnotationTransformer(CreateExtCallableTypeAnnotationTransformer(*state->Types, true), *state->Types); + auto constrTransformer = CreateConstraintTransformer(*state->Types, true, true); + TVector<TTransformStage> transformers; + const auto issueCode = TIssuesIds::CORE_TYPE_ANN; + transformers.push_back(TTransformStage(typeTransformer, "TypeAnnotation", issueCode)); + transformers.push_back(TTransformStage( + CreateFunctorTransformer([](const TExprNode::TPtr& input, TExprNode::TPtr& output, TExprContext& ctx) { return UpdateCompletness(input, output, ctx); }), + "UpdateCompletness", issueCode)); + transformers.push_back(TTransformStage(constrTransformer, "Constraints", issueCode)); + auto fullTransformer = CreateCompositeGraphTransformer(transformers, false); + status = InstantTransform(*fullTransformer, optimizedInput, ctx); + if (status.Level == IGraphTransformer::TStatus::Error) { + return status; + } + + input = optimizedInput; + } + + return IGraphTransformer::TStatus::Ok; +} + +TYtPath CopyOrTrivialMap(TPositionHandle pos, TExprBase world, TYtDSink dataSink, const TTypeAnnotationNode& scheme, + TYtSection section, TYqlRowSpecInfo::TPtr outRowSpec, TExprContext& ctx, const TYtState::TPtr& state, const TCopyOrTrivialMapOpts& opts) +{ + bool tryKeepSortness = opts.TryKeepSortness; + const bool singleInput = section.Paths().Size() == 1; + bool needMap = false; + const auto sysColumns = NYql::GetSetting(section.Settings().Ref(), EYtSettingType::SysColumns); + bool useExplicitColumns = false; + bool exactCopySort = false; + bool hasAux = false; + TVector<std::pair<TYqlRowSpecInfo::TPtr, bool>> rowSpecs; + const ui64 outNativeYtTypeFlags = outRowSpec ? outRowSpec->GetNativeYtTypeFlags() : (state->Configuration->UseNativeYtTypes.Get().GetOrElse(DEFAULT_USE_NATIVE_YT_TYPES) ? NTCF_ALL : NTCF_NONE); + TYtOutTableInfo outTable(scheme.Cast<TStructExprType>(), outNativeYtTypeFlags); + outTable.RowSpec->SetConstraints(opts.Constraints); + TMaybe<NYT::TNode> outNativeType; + if (outRowSpec) { + outNativeType = outRowSpec->GetNativeYtType(); + } + bool first = !outRowSpec; + const bool useNativeDescSort = state->Configuration->UseNativeDescSort.Get().GetOrElse(DEFAULT_USE_NATIVE_DESC_SORT); + for (auto path: section.Paths()) { + TYtPathInfo pathInfo(path); + const bool hasRowSpec = !!pathInfo.Table->RowSpec; + const bool tableHasAux = hasRowSpec && pathInfo.Table->RowSpec->HasAuxColumns(); + TMaybe<NYT::TNode> currentNativeType; + if (hasRowSpec) { + currentNativeType = pathInfo.GetNativeYtType(); + } + if (first) { + outNativeType = currentNativeType; + first = false; + } + const bool needTableMap = pathInfo.RequiresRemap() || bool(sysColumns) + || outTable.RowSpec->GetNativeYtTypeFlags() != pathInfo.GetNativeYtTypeFlags() + || currentNativeType != outNativeType; + useExplicitColumns = useExplicitColumns || !pathInfo.Table->IsTemp || (tableHasAux && pathInfo.HasColumns()); + needMap = needMap || needTableMap; + hasAux = hasAux || tableHasAux; + if (tryKeepSortness) { + if (pathInfo.Table->IsUnordered || (opts.RangesResetSort && pathInfo.Ranges && pathInfo.Ranges->GetRangesCount() > 1)) { + tryKeepSortness = false; + } + rowSpecs.emplace_back(pathInfo.Table->RowSpec, needTableMap); + + exactCopySort = singleInput && pathInfo.Table->IsTemp && hasRowSpec + && IsSameAnnotation(scheme, *pathInfo.Table->RowSpec->GetType()); + } + } + if (!needMap && outNativeType) { + outTable.RowSpec->CopyTypeOrders(*outNativeType); + } + useExplicitColumns = useExplicitColumns || (!tryKeepSortness && hasAux); + + bool trimSort = false; + const bool sortConstraintEnabled = ctx.IsConstraintEnabled<TSortedConstraintNode>(); + if (tryKeepSortness) { + bool sortIsChanged = false; + for (size_t i = 0; i < rowSpecs.size(); ++i) { + if (!rowSpecs[i].first) { + sortIsChanged = outTable.RowSpec->ClearSortness(ctx); + continue; + } + if (0 == i) { + TYqlRowSpecInfo::ECopySort mode = TYqlRowSpecInfo::ECopySort::Pure; + if (rowSpecs[i].second) { + if (sortConstraintEnabled) { + mode = TYqlRowSpecInfo::ECopySort::WithDesc; + } + } else { + mode = exactCopySort + ? TYqlRowSpecInfo::ECopySort::Exact + : TYqlRowSpecInfo::ECopySort::WithDesc; + } + sortIsChanged = outTable.RowSpec->CopySortness(ctx, *rowSpecs[i].first, mode); + } else { + sortIsChanged = outTable.RowSpec->MakeCommonSortness(ctx, *rowSpecs[i].first) || sortIsChanged; + if (rowSpecs[i].second && !sortConstraintEnabled) { + sortIsChanged = outTable.RowSpec->KeepPureSortOnly(ctx) || sortIsChanged; + } + } + } + + useExplicitColumns = useExplicitColumns || (sortIsChanged && hasAux); + tryKeepSortness = outTable.RowSpec->IsSorted(); + trimSort = !tryKeepSortness; + } + outTable.SetUnique(opts.SectionUniq, pos, ctx); + + if (tryKeepSortness) { + if (needMap && !singleInput) { + auto settingsBuilder = Build<TCoNameValueTupleList>(ctx, pos); + settingsBuilder + .Add() + .Name() + .Value(ToString(EYtSettingType::Ordered)) + .Build() + .Build(); + if (!opts.LimitNodes.empty()) { + settingsBuilder + .Add() + .Name() + .Value(ToString(EYtSettingType::Limit)) + .Build() + .Value<TExprList>() + .Add(opts.LimitNodes) + .Build() + .Build(); + } + if (state->Configuration->UseFlow.Get().GetOrElse(DEFAULT_USE_FLOW)) { + settingsBuilder + .Add() + .Name() + .Value(ToString(EYtSettingType::Flow)) + .Build() + .Build(); + } + + TExprNode::TPtr mapSectionSettings = ctx.NewList(section.Pos(), {}); + TExprNode::TPtr sectionSettings = section.Settings().Ptr(); + if (sysColumns) { + mapSectionSettings = NYql::AddSetting(*mapSectionSettings, EYtSettingType::SysColumns, sysColumns->ChildPtr(1), ctx); + sectionSettings = NYql::RemoveSetting(*sectionSettings, EYtSettingType::SysColumns, ctx); + } + + auto getPathUniq = [] (const TYtPath& path) { + if (path.Ref().GetState() != TExprNode::EState::Initial) { + return path.Ref().GetConstraint<TDistinctConstraintNode>(); + } + // Dynamically constructed YtPath for YtOutput + return path.Table().Ref().GetConstraint<TDistinctConstraintNode>(); + }; + TVector<TYtPath> updatedPaths; + YQL_ENSURE(rowSpecs.size() == section.Paths().Size()); + for (size_t i = 0; i < section.Paths().Size(); ++i) { + auto path = section.Paths().Item(i); + if (rowSpecs[i].second) { + TYtOutTableInfo mapOutTable(scheme.Cast<TStructExprType>(), outNativeYtTypeFlags); + if (outNativeType) { + mapOutTable.RowSpec->CopyTypeOrders(*outNativeType); + } + YQL_ENSURE(rowSpecs[i].first); + mapOutTable.SetUnique(getPathUniq(path), path.Pos(), ctx); + auto mapper = Build<TCoLambda>(ctx, path.Pos()) + .Args({"stream"}) + .Body("stream") + .Done().Ptr(); + + mapOutTable.RowSpec->CopySortness(ctx, *rowSpecs[i].first, sortConstraintEnabled ? TYqlRowSpecInfo::ECopySort::WithDesc : TYqlRowSpecInfo::ECopySort::Pure); + if (sortConstraintEnabled) { + TKeySelectorBuilder builder(path.Pos(), ctx, useNativeDescSort, scheme.Cast<TStructExprType>()); + builder.ProcessRowSpec(*mapOutTable.RowSpec); + if (builder.NeedMap()) { + mapper = builder.MakeRemapLambda(true); + } + } + + path = Build<TYtPath>(ctx, path.Pos()) + .Table<TYtOutput>() + .Operation<TYtMap>() + .World(world) + .DataSink(dataSink) + .Input() + .Add() + .Paths() + .Add(path) + .Build() + .Settings(mapSectionSettings) + .Build() + .Build() + .Output() + .Add(mapOutTable.ToExprNode(ctx, path.Pos()).Cast<TYtOutTable>()) + .Build() + .Settings(settingsBuilder.Done()) + .Mapper(mapper) + .Build() + .OutIndex() + .Value("0") + .Build() + .Build() + .Columns<TCoVoid>().Build() + .Ranges<TCoVoid>().Build() + .Stat<TCoVoid>().Build() + .Done(); + } + updatedPaths.push_back(path); + } + section = Build<TYtSection>(ctx, section.Pos()) + .InitFrom(section) + .Paths() + .Add(updatedPaths) + .Build() + .Settings(sectionSettings) + .Done(); + needMap = false; + } + } else if (!trimSort) { + section = MakeUnorderedSection(section, ctx); + } + + if (needMap) { + auto settingsBuilder = Build<TCoNameValueTupleList>(ctx, pos); + settingsBuilder + .Add() + .Name() + .Value(ToString(EYtSettingType::Ordered)) + .Build() + .Build(); + if (!opts.LimitNodes.empty()) { + settingsBuilder + .Add() + .Name() + .Value(ToString(EYtSettingType::Limit)) + .Build() + .Value<TExprList>() + .Add(opts.LimitNodes) + .Build() + .Build(); + } + if (state->Configuration->UseFlow.Get().GetOrElse(DEFAULT_USE_FLOW)) { + settingsBuilder + .Add() + .Name() + .Value(ToString(EYtSettingType::Flow)) + .Build() + .Build(); + } + + auto mapper = Build<TCoLambda>(ctx, pos) + .Args({"stream"}) + .Body("stream") + .Done().Ptr(); + + if (sortConstraintEnabled && outTable.RowSpec->IsSorted()) { + TKeySelectorBuilder builder(pos, ctx, useNativeDescSort, scheme.Cast<TStructExprType>()); + builder.ProcessRowSpec(*outTable.RowSpec); + if (builder.NeedMap()) { + mapper = builder.MakeRemapLambda(true); + } + } + + return Build<TYtPath>(ctx, pos) + .Table<TYtOutput>() + .Operation<TYtMap>() + .World(world) + .DataSink(dataSink) + .Input() + .Add(section) + .Build() + .Output() + .Add(outTable.ToExprNode(ctx, pos).Cast<TYtOutTable>()) + .Build() + .Settings(settingsBuilder.Done()) + .Mapper(mapper) + .Build() + .OutIndex() + .Value("0") + .Build() + .Build() + .Columns<TCoVoid>().Build() + .Ranges<TCoVoid>().Build() + .Stat<TCoVoid>().Build() + .Done(); + } + + auto settingsBuilder = Build<TCoNameValueTupleList>(ctx, pos); + if (NYql::HasSetting(section.Settings().Ref(), EYtSettingType::Sample)) { + settingsBuilder + .Add() + .Name() + .Value(ToString(EYtSettingType::ForceTransform)) + .Build() + .Build(); + } + if (opts.CombineChunks) { + settingsBuilder + .Add() + .Name() + .Value(ToString(EYtSettingType::CombineChunks)) + .Build() + .Build(); + } + if (!opts.LimitNodes.empty()) { + settingsBuilder + .Add() + .Name() + .Value(ToString(EYtSettingType::Limit)) + .Build() + .Value<TExprList>() + .Add(opts.LimitNodes) + .Build() + .Build(); + } + + if (useExplicitColumns) { + TSet<TStringBuf> columns; + for (auto item: outTable.RowSpec->GetType()->GetItems()) { + columns.insert(item->GetName()); + } + for (auto item: outTable.RowSpec->GetAuxColumns()) { + columns.insert(item.first); + } + + section = UpdateInputFields(section, std::move(columns), ctx, false); + } + + return Build<TYtPath>(ctx, pos) + .Table<TYtOutput>() + .Operation<TYtMerge>() + .World(world) + .DataSink(dataSink) + .Input() + .Add(section) + .Build() + .Output() + .Add(outTable.ToExprNode(ctx, pos).Cast<TYtOutTable>()) + .Build() + .Settings(settingsBuilder.Done()) + .Build() + .OutIndex() + .Value(TStringBuf("0")) + .Build() + .Build() + .Columns<TCoVoid>().Build() + .Ranges<TCoVoid>().Build() + .Stat<TCoVoid>().Build() + .Done(); +} + +namespace { + +template <class T> +const TExprNode* GetSingleParent(const TExprNode* node, const TParentsMap& parentsMap) { + if (T::Match(node)) { + auto parentsIt = parentsMap.find(node); + YQL_ENSURE(parentsIt != parentsMap.cend()); + if (parentsIt->second.size() != 1) { + return nullptr; + } + return *parentsIt->second.begin(); + } + return node; +} + +} + + +bool IsOutputUsedMultipleTimes(const TExprNode& op, const TParentsMap& parentsMap) { + const TExprNode* node = &op; + node = GetSingleParent<TYtOutputOpBase>(node, parentsMap); + if (nullptr == node) { + return true; + } + node = GetSingleParent<TYtOutput>(node, parentsMap); + if (nullptr == node) { + return true; + } + node = GetSingleParent<TYtPath>(node, parentsMap); + if (nullptr == node) { + return true; + } + node = GetSingleParent<TYtPathList>(node, parentsMap); + if (nullptr == node) { + return true; + } + node = GetSingleParent<TYtSection>(node, parentsMap); + if (nullptr == node) { + return true; + } + node = GetSingleParent<TYtSectionList>(node, parentsMap); + return node == nullptr; +} + +TMaybe<NYT::TRichYPath> BuildYtPathForStatRequest(const TString& cluster, const TYtPathInfo& pathInfo, + const TMaybe<TVector<TString>>& overrideColumns, const TYtState& state, TExprContext& ctx) +{ + auto ytPath = NYT::TRichYPath(pathInfo.Table->Name); + pathInfo.FillRichYPath(ytPath); + if (overrideColumns) { + ytPath.Columns(*overrideColumns); + } + + if (ytPath.Columns_ && dynamic_cast<TYtTableInfo*>(pathInfo.Table.Get()) && pathInfo.Table->IsAnonymous + && !TYtTableInfo::HasSubstAnonymousLabel(pathInfo.Table->FromNode.Cast())) { + TString realTableName = state.AnonymousLabels.Value(std::make_pair(cluster, pathInfo.Table->Name), TString()); + if (!realTableName) { + TPositionHandle pos; + if (pathInfo.FromNode) { + pos = pathInfo.FromNode.Cast().Pos(); + } + ctx.AddError(TIssue(ctx.GetPosition(pos), TStringBuilder() << "Unaccounted anonymous table: " << cluster << '.' << pathInfo.Table->Name)); + return {}; + } + ytPath.Path_ = realTableName; + } + + return ytPath; +} + +TMaybe<TVector<ui64>> EstimateDataSize(const TString& cluster, const TVector<TYtPathInfo::TPtr>& paths, + const TMaybe<TVector<TString>>& columns, const TYtState& state, TExprContext& ctx) +{ + TVector<ui64> result; + TSet<TString> requestedColumns; + + bool sync = true; + + auto status = EstimateDataSize(result, requestedColumns, cluster, paths, columns, state, ctx, sync); + if (status != IGraphTransformer::TStatus::Ok) { + return {}; + } + + return result; +} + +IGraphTransformer::TStatus TryEstimateDataSize(TVector<ui64>& result, TSet<TString>& requestedColumns, + const TString& cluster, const TVector<TYtPathInfo::TPtr>& paths, + const TMaybe<TVector<TString>>& columns, const TYtState& state, TExprContext& ctx) +{ + bool sync = false; + return EstimateDataSize(result, requestedColumns, cluster, paths, columns, state, ctx, sync); +} + +TYtSection UpdateInputFields(TYtSection section, TExprBase fields, TExprContext& ctx) { + auto settings = section.Settings().Ptr(); + + auto sysColumns = NYql::GetSettingAsColumnList(*settings, EYtSettingType::SysColumns); + if (!sysColumns.empty()) { + if (auto list = fields.Maybe<TExprList>()) { + TMap<TStringBuf, TExprNode::TPtr> fieldMap; + for (auto item: list.Cast()) { + if (auto atom = item.Maybe<TCoAtom>()) { + fieldMap.emplace(atom.Cast().Value(), item.Ptr()); + } else { + fieldMap.emplace(item.Cast<TCoAtomList>().Item(0).Value(), item.Ptr()); + } + } + TVector<TString> updatedSysColumns; + for (auto sys: sysColumns) { + auto sysColName = TString(YqlSysColumnPrefix).append(sys); + if (fieldMap.contains(sysColName)) { + updatedSysColumns.push_back(sys); + fieldMap.erase(sysColName); + } + } + if (updatedSysColumns.size() != sysColumns.size()) { + settings = NYql::RemoveSetting(*settings, EYtSettingType::SysColumns, ctx); + if (!updatedSysColumns.empty()) { + settings = NYql::AddSettingAsColumnList(*settings, EYtSettingType::SysColumns, updatedSysColumns, ctx); + } + } + if (fieldMap.size() != list.Cast().Size()) { + TExprNode::TListType children; + std::transform(fieldMap.begin(), fieldMap.end(), std::back_inserter(children), [](const auto& pair) { return pair.second; }); + fields = TExprBase(ctx.NewList(fields.Pos(), std::move(children))); + } + } + } + + auto pathsBuilder = Build<TYtPathList>(ctx, section.Paths().Pos()); + for (const auto& path : section.Paths()) { + pathsBuilder.Add<TYtPath>() + .InitFrom(path) + .Columns(fields) + .Build(); + } + return Build<TYtSection>(ctx, section.Pos()) + .InitFrom(section) + .Paths(pathsBuilder.Done()) + .Settings(settings) + .Done(); +} + +TYtSection UpdateInputFields(TYtSection section, TSet<TStringBuf>&& members, TExprContext& ctx, bool hasWeakFields) { + auto settings = section.Settings().Ptr(); + + auto sysColumns = NYql::GetSettingAsColumnList(*settings, EYtSettingType::SysColumns); + if (!sysColumns.empty()) { + TVector<TString> updatedSysColumns; + for (auto sys: sysColumns) { + auto sysColName = TString(YqlSysColumnPrefix).append(sys); + if (members.contains(sysColName)) { + updatedSysColumns.push_back(sys); + members.erase(sysColName); + } + } + if (updatedSysColumns.size() != sysColumns.size()) { + settings = NYql::RemoveSetting(*settings, EYtSettingType::SysColumns, ctx); + if (!updatedSysColumns.empty()) { + settings = NYql::AddSettingAsColumnList(*settings, EYtSettingType::SysColumns, updatedSysColumns, ctx); + } + } + } + + auto fields = ToAtomList(members, section.Pos(), ctx); + auto pathsBuilder = Build<TYtPathList>(ctx, section.Paths().Pos()); + for (const auto& path : section.Paths()) { + if (!hasWeakFields || path.Columns().Maybe<TCoVoid>()) { + pathsBuilder.Add<TYtPath>() + .InitFrom(path) + .Columns(fields) + .Build(); + } else { + THashMap<TStringBuf, TExprNode::TPtr> weakFields; + for (auto col: path.Columns().Cast<TExprList>()) { + if (col.Ref().ChildrenSize() == 2) { + weakFields[col.Ref().Child(0)->Content()] = col.Ptr(); + } + } + TExprNode::TListType updatedColumns; + for (auto member: fields->Children()) { + if (auto p = weakFields.FindPtr(member->Content())) { + updatedColumns.push_back(*p); + } else { + updatedColumns.push_back(member); + } + } + pathsBuilder.Add<TYtPath>() + .InitFrom(path) + .Columns(ctx.NewList(path.Pos(), std::move(updatedColumns))) + .Build(); + } + } + return Build<TYtSection>(ctx, section.Pos()) + .InitFrom(section) + .Paths(pathsBuilder.Done()) + .Settings(settings) + .Done(); +} + +TYtPath MakeUnorderedPath(TYtPath path, bool hasLimits, TExprContext& ctx) { + bool makeUnordered = false; + bool keepSort = false; + if (auto maybeOut = path.Table().Maybe<TYtOutput>()) { + const auto out = maybeOut.Cast(); + if (!IsUnorderedOutput(out)) { + makeUnordered = true; + if (!path.Ranges().Maybe<TCoVoid>()) { + for (auto range: path.Ranges().Cast<TExprList>()) { + if (range.Maybe<TYtKeyExact>() || range.Maybe<TYtKeyRange>()) { + makeUnordered = false; + } else if (range.Maybe<TYtRow>() || range.Maybe<TYtRowRange>()) { + hasLimits = true; + } + } + } + } + if (auto settings = GetOutputOp(out).Maybe<TYtTransientOpBase>().Settings()) { + hasLimits = hasLimits || NYql::HasSetting(settings.Ref(), EYtSettingType::Limit); + keepSort = NYql::HasSetting(settings.Ref(), EYtSettingType::KeepSorted); + } else if (auto settings = GetOutputOp(out).Maybe<TYtFill>().Settings()) { + keepSort = NYql::HasSetting(settings.Ref(), EYtSettingType::KeepSorted); + } + keepSort = keepSort || GetOutputOp(out).Maybe<TYtSort>(); + } + if (makeUnordered && hasLimits && keepSort) { + makeUnordered = false; + } + if (makeUnordered) { + return Build<TYtPath>(ctx, path.Pos()) + .InitFrom(path) + .Table<TYtOutput>() + .InitFrom(path.Table().Cast<TYtOutput>()) + .Mode() + .Value(ToString(EYtSettingType::Unordered)) + .Build() + .Build() + .Done(); + } + return path; + +} + +template<bool WithUnorderedSetting> +TYtSection MakeUnorderedSection(TYtSection section, TExprContext& ctx) { + if (HasNonEmptyKeyFilter(section)) { + if constexpr (WithUnorderedSetting) + return Build<TYtSection>(ctx, section.Pos()) + .Paths(section.Paths()) + .Settings(NYql::AddSetting(section.Settings().Ref(), EYtSettingType::Unordered, {}, ctx)) + .Done(); + else + return section; + } + const bool hasLimits = NYql::HasAnySetting(section.Settings().Ref(), EYtSettingType::Take | EYtSettingType::Skip); + bool hasUpdated = false; + TVector<TYtPath> updatedPaths; + for (auto path: section.Paths()) { + updatedPaths.push_back(MakeUnorderedPath(path, hasLimits, ctx)); + hasUpdated = hasUpdated || updatedPaths.back().Raw() != path.Raw(); + } + + if constexpr (WithUnorderedSetting) { + return Build<TYtSection>(ctx, section.Pos()) + .Paths() + .Add(updatedPaths) + .Build() + .Settings(NYql::AddSetting(section.Settings().Ref(), EYtSettingType::Unordered, {}, ctx)) + .Done(); + } else { + if (!hasUpdated) + return section; + + return Build<TYtSection>(ctx, section.Pos()) + .Paths() + .Add(updatedPaths) + .Build() + .Settings(section.Settings()) + .Done(); + } +} + +template TYtSection MakeUnorderedSection<true>(TYtSection section, TExprContext& ctx); +template TYtSection MakeUnorderedSection<false>(TYtSection section, TExprContext& ctx); + +TYtSection ClearUnorderedSection(TYtSection section, TExprContext& ctx) { + const bool hasUnorderedOut = AnyOf(section.Paths(), [](const auto& path) { auto out = path.Table().template Maybe<TYtOutput>(); return out && IsUnorderedOutput(out.Cast()); }); + if (hasUnorderedOut) { + TVector<TYtPath> updatedPaths; + for (auto path: section.Paths()) { + if (auto out = path.Table().Maybe<TYtOutput>()) { + if (IsUnorderedOutput(out.Cast())) { + path = Build<TYtPath>(ctx, path.Pos()) + .InitFrom(path) + .Table<TYtOutput>() + .InitFrom(out.Cast()) + .Mode(TMaybeNode<TCoAtom>()) + .Build() + .Done(); + } + } + updatedPaths.push_back(path); + } + section = Build<TYtSection>(ctx, section.Pos()) + .InitFrom(section) + .Paths() + .Add(updatedPaths) + .Build() + .Done(); + } + if (NYql::HasSetting(section.Settings().Ref(), EYtSettingType::Unordered)) { + section = Build<TYtSection>(ctx, section.Pos()) + .InitFrom(section) + .Settings(NYql::RemoveSetting(section.Settings().Ref(), EYtSettingType::Unordered, ctx)) + .Done(); + } + return section; +} + +TYtDSource GetDataSource(TExprBase input, TExprContext& ctx) { + TMaybeNode<TExprBase> n = input; + if (auto right = input.Maybe<TCoRight>()) { + n = right.Input(); + } else if (auto content = input.Maybe<TYtTableContent>()) { + n = content.Input(); + } + if (auto read = n.Maybe<TYtReadTable>()) + return read.Cast().DataSource(); + if (auto out = n.Maybe<TYtOutput>()) { + return TYtDSource(ctx.RenameNode(GetOutputOp(out.Cast()).DataSink().Ref(), "DataSource")); + } else { + YQL_ENSURE(false, "Unknown operation input"); + } +} + +TExprNode::TPtr BuildEmptyTablesRead(TPositionHandle pos, const TExprNode& userSchema, TExprContext& ctx) { + if (!EnsureArgsCount(userSchema, 2, ctx)) { + return {}; + } + + return ctx.Builder(pos) + .Callable("Cons!") + .World(0) + .Callable(1, "List") + .Callable(0, "ListType") + .Add(0, userSchema.ChildPtr(1)) + .Seal() + .Seal() + .Seal() + .Build(); +} + +TExprNode::TPtr GetFlowSettings(TPositionHandle pos, const TYtState& state, TExprContext& ctx, TExprNode::TPtr settings) { + if (!settings) { + settings = ctx.NewList(pos, {}); + } + if (state.Configuration->UseFlow.Get().GetOrElse(DEFAULT_USE_FLOW)) { + settings = NYql::AddSetting(*settings, EYtSettingType::Flow, {}, ctx); + } + return settings; +} + +TVector<TStringBuf> GetKeyFilterColumns(const NNodes::TYtSection& section, EYtSettingTypes kind) { + TVector<TStringBuf> result; + if (kind.HasFlags(EYtSettingType::KeyFilter) && NYql::HasSetting(section.Settings().Ref(), EYtSettingType::KeyFilter)) { + for (auto keyFilter: NYql::GetAllSettingValues(section.Settings().Ref(), EYtSettingType::KeyFilter)) { + auto value = TExprList(keyFilter); + if (value.Size() > 0) { + for (auto member: value.Item(0).Cast<TCoNameValueTupleList>()) { + result.emplace_back(member.Name().Value()); + } + } + } + } + if (kind.HasFlags(EYtSettingType::KeyFilter2) && NYql::HasSetting(section.Settings().Ref(), EYtSettingType::KeyFilter2)) { + for (auto keyFilter: NYql::GetAllSettingValues(section.Settings().Ref(), EYtSettingType::KeyFilter2)) { + auto value = TExprList(keyFilter); + if (value.Size() > 0) { + for (auto member: value.Item(1).Cast<TCoNameValueTupleList>()) { + if (member.Name().Value() == "usedKeys") { + for (auto key : member.Value().Cast<TCoAtomList>()) { + result.emplace_back(key.Value()); + } + } + } + } + } + } + return result; +} + +bool HasNonEmptyKeyFilter(const NNodes::TYtSection& section) { + auto hasChildren = [](const auto& node) { return node->ChildrenSize() > 0; }; + return AnyOf(NYql::GetAllSettingValues(section.Settings().Ref(), EYtSettingType::KeyFilter), hasChildren) || + AnyOf(NYql::GetAllSettingValues(section.Settings().Ref(), EYtSettingType::KeyFilter2), hasChildren); +} + +TYtReadTable ConvertContentInputToRead(TExprBase input, TMaybeNode<TCoNameValueTupleList> settings, TExprContext& ctx, TMaybeNode<TCoAtomList> customFields) { + TExprNode::TPtr world; + TVector<TYtSection> sections; + TExprBase columns = customFields ? TExprBase(customFields.Cast()) : TExprBase(Build<TCoVoid>(ctx, input.Pos()).Done()); + if (auto out = input.Maybe<TYtOutput>()) { + world = ctx.NewWorld(input.Pos()); + if (!settings) { + settings = Build<TCoNameValueTupleList>(ctx, input.Pos()).Done(); + } + sections.push_back(Build<TYtSection>(ctx, input.Pos()) + .Paths() + .Add() + .Table(out.Cast()) + .Columns(columns) + .Ranges<TCoVoid>().Build() + .Stat<TCoVoid>().Build() + .Build() + .Build() + .Settings(settings.Cast()) + .Done()); + } + else { + auto read = input.Maybe<TYtReadTable>(); + YQL_ENSURE(read, "Unknown operation input"); + world = read.Cast().World().Ptr(); + for (auto section: read.Cast().Input()) { + if (settings) { + section = Build<TYtSection>(ctx, section.Pos()) + .InitFrom(section) + .Settings(MergeSettings(section.Settings().Ref(), settings.Cast().Ref(), ctx)) + .Done(); + } + + if (customFields) { + section = UpdateInputFields(section, customFields.Cast(), ctx); + } + + sections.push_back(section); + } + } + + return Build<TYtReadTable>(ctx, input.Pos()) + .World(world) + .DataSource(GetDataSource(input, ctx)) + .Input() + .Add(sections) + .Build() + .Done(); +} + +size_t GetMapDirectOutputsCount(const NNodes::TYtMapReduce& mapReduce) { + if (mapReduce.Mapper().Maybe<TCoVoid>()) { + return 0; + } + const auto& mapOutputType = GetSeqItemType(*mapReduce.Mapper().Ref().GetTypeAnn()); + if (mapOutputType.GetKind() != ETypeAnnotationKind::Variant) { + return 0; + } + + auto numVariants = mapOutputType.Cast<TVariantExprType>()->GetUnderlyingType()->Cast<TTupleExprType>()->GetSize(); + YQL_ENSURE(numVariants > 1); + return numVariants - 1; +} + +bool HasYtRowNumber(const TExprNode& node) { + bool hasRowNumber = false; + VisitExpr(node, [&hasRowNumber](const TExprNode& n) { + if (TYtRowNumber::Match(&n)) { + hasRowNumber = true; + } else if (TYtOutput::Match(&n)) { + return false; + } + return !hasRowNumber; + }); + return hasRowNumber; +} + +bool IsYtTableSuitableForArrowInput(NNodes::TExprBase tableNode, std::function<void(const TString&)> unsupportedHandler) { + auto meta = TYtTableBaseInfo::GetMeta(tableNode); + if (meta->InferredScheme) { + unsupportedHandler("can't use arrow input on tables with inferred schema"); + return false; + } + if (auto table = tableNode.Maybe<TYtTable>(); table && NYql::HasAnySetting(table.Cast().Settings().Ref(), EYtSettingType::UserColumns | EYtSettingType::UserSchema)) { + unsupportedHandler("can't use arrow input on tables with overridden schema/columns"); + return false; + } + if (meta->Attrs.contains(SCHEMA_MODE_ATTR_NAME) && meta->Attrs[SCHEMA_MODE_ATTR_NAME] == "weak") { + unsupportedHandler("can't use arrow input on tables with weak schema"); + return false; + } + + return true; +} + +} // NYql diff --git a/yt/yql/providers/yt/provider/yql_yt_helpers.h b/yt/yql/providers/yt/provider/yql_yt_helpers.h new file mode 100644 index 0000000000..cd9a54f7d5 --- /dev/null +++ b/yt/yql/providers/yt/provider/yql_yt_helpers.h @@ -0,0 +1,146 @@ +#pragma once + +#include "yql_yt_table.h" +#include "yql_yt_gateway.h" +#include "yql_yt_provider.h" +#include "yql_yt_op_settings.h" + +#include <yt/yql/providers/yt/common/yql_names.h> +#include <yt/yql/providers/yt/expr_nodes/yql_yt_expr_nodes.h> +#include <yql/essentials/core/yql_graph_transformer.h> +#include <yql/essentials/core/yql_type_annotation.h> +#include <yql/essentials/core/expr_nodes/yql_expr_nodes.h> +#include <yql/essentials/ast/yql_expr.h> + +#include <util/generic/string.h> +#include <util/generic/strbuf.h> +#include <util/generic/vector.h> +#include <util/generic/maybe.h> +#include <util/generic/set.h> + +namespace NYql { + +bool UpdateUsedCluster(TString& usedCluster, const TString& newCluster); +bool IsYtIsolatedLambda(const TExprNode& lambdaBody, TSyncMap& syncList, TString& usedCluster, bool supportsDq); +bool IsYtCompleteIsolatedLambda(const TExprNode& lambdaBody, TSyncMap& syncList, bool supportsDq); +bool IsYtCompleteIsolatedLambda(const TExprNode& lambdaBody, TSyncMap& syncList, TString& usedCluster, bool supportsDq); +TExprNode::TPtr YtCleanupWorld(const TExprNode::TPtr& input, TExprContext& ctx, TYtState::TPtr state); +TVector<TYtTableBaseInfo::TPtr> GetInputTableInfos(NNodes::TExprBase input); +TVector<TYtPathInfo::TPtr> GetInputPaths(NNodes::TExprBase input); +TStringBuf GetClusterName(NNodes::TExprBase input); +bool IsYtProviderInput(NNodes::TExprBase input, bool withVariantList = false); +bool IsConstExpSortDirections(NNodes::TExprBase sortDirections); +TExprNode::TListType GetNodesToCalculate(const TExprNode::TPtr& input); +bool HasNodesToCalculate(const TExprNode::TPtr& input); +std::pair<IGraphTransformer::TStatus, TAsyncTransformCallbackFuture> CalculateNodes(TYtState::TPtr state, const TExprNode::TPtr& input, + const TString& cluster, const TExprNode::TListType& needCalc, TExprContext& ctx); +TMaybe<ui64> GetLimit(const TExprNode& settings); +TExprNode::TPtr GetLimitExpr(const TExprNode::TPtr& limitSetting, TExprContext& ctx); +IGraphTransformer::TStatus UpdateTableMeta(const TExprNode::TPtr& tableNode, TExprNode::TPtr& newTableNode, + const TYtTablesData::TPtr& tablesData, bool checkSqlView, bool updateRowSpecType, TExprContext& ctx); +TExprNode::TPtr ValidateAndUpdateTablesMeta(const TExprNode::TPtr& input, TStringBuf cluster, const TYtTablesData::TPtr& tablesData, bool updateRowSpecType, TExprContext& ctx); +TExprNode::TPtr ResetTablesMeta(const TExprNode::TPtr& input, TExprContext& ctx, bool resetTmpOnly, bool isEvaluationInProgress); +NNodes::TExprBase GetOutTable(NNodes::TExprBase ytOutput); +std::pair<NNodes::TExprBase, TString> GetOutTableWithCluster(NNodes::TExprBase ytOutput); +NNodes::TMaybeNode<NNodes::TCoFlatMapBase> GetFlatMapOverInputStream(NNodes::TCoLambda opLambda, const TParentsMap& parentsMap); +NNodes::TMaybeNode<NNodes::TCoFlatMapBase> GetFlatMapOverInputStream(NNodes::TCoLambda opLambda); +IGraphTransformer::TStatus SubstTables(TExprNode::TPtr& input, const TYtState::TPtr& state, bool anonOnly, TExprContext& ctx); + +struct TCopyOrTrivialMapOpts { + TCopyOrTrivialMapOpts& SetLimitNodes(const TExprNode::TListType& limitNodes) { + LimitNodes = limitNodes; + return *this; + } + + TCopyOrTrivialMapOpts& SetTryKeepSortness(bool tryKeepSortness = true) { + TryKeepSortness = tryKeepSortness; + return *this; + } + + TCopyOrTrivialMapOpts& SetSectionUniq(const TDistinctConstraintNode* sectionUniq) { + SectionUniq = sectionUniq; + return *this; + } + + TCopyOrTrivialMapOpts& SetCombineChunks(bool combineChunks = true) { + CombineChunks = combineChunks; + return *this; + } + + TCopyOrTrivialMapOpts& SetRangesResetSort(bool rangesResetSort = true) { + RangesResetSort = rangesResetSort; + return *this; + } + + TCopyOrTrivialMapOpts& SetConstraints(const TConstraintSet& constraints) { + Constraints = constraints; + return *this; + } + + TExprNode::TListType LimitNodes; + bool TryKeepSortness = false; + const TDistinctConstraintNode* SectionUniq = nullptr; + bool CombineChunks = false; + bool RangesResetSort = true; + TConstraintSet Constraints; +}; + +NNodes::TYtPath CopyOrTrivialMap(TPositionHandle pos, NNodes::TExprBase world, NNodes::TYtDSink dataSink, + const TTypeAnnotationNode& scheme, NNodes::TYtSection section, TYqlRowSpecInfo::TPtr outRowSpec, TExprContext& ctx, const TYtState::TPtr& state, + const TCopyOrTrivialMapOpts& opts); +bool IsOutputUsedMultipleTimes(const TExprNode& op, const TParentsMap& parentsMap); + +TMaybe<TVector<ui64>> EstimateDataSize(const TString& cluster, const TVector<TYtPathInfo::TPtr>& paths, + const TMaybe<TVector<TString>>& columns, const TYtState& state, TExprContext& ctx); +IGraphTransformer::TStatus TryEstimateDataSize(TVector<ui64>& result, TSet<TString>& requestedColumns, + const TString& cluster, const TVector<TYtPathInfo::TPtr>& paths, + const TMaybe<TVector<TString>>& columns, const TYtState& state, TExprContext& ctx); +TMaybe<NYT::TRichYPath> BuildYtPathForStatRequest(const TString& cluster, const TYtPathInfo& pathInfo, + const TMaybe<TVector<TString>>& overrideColumns, const TYtState& state, TExprContext& ctx); + +NNodes::TYtSection UpdateInputFields(NNodes::TYtSection section, NNodes::TExprBase fields, TExprContext& ctx); +NNodes::TYtSection UpdateInputFields(NNodes::TYtSection section, TSet<TStringBuf>&& fields, TExprContext& ctx, bool hasWeakFields); +NNodes::TYtPath MakeUnorderedPath(NNodes::TYtPath path, bool hasLimits, TExprContext& ctx); +template<bool WithUnorderedSetting = false> +NNodes::TYtSection MakeUnorderedSection(NNodes::TYtSection section, TExprContext& ctx); +NNodes::TYtSection ClearUnorderedSection(NNodes::TYtSection section, TExprContext& ctx); +NNodes::TYtDSource GetDataSource(NNodes::TExprBase input, TExprContext& ctx); +TExprNode::TPtr BuildEmptyTablesRead(TPositionHandle pos, const TExprNode& userSchema, TExprContext& ctx); +TExprNode::TPtr GetFlowSettings(TPositionHandle pos, const TYtState& state, TExprContext& ctx, TExprNode::TPtr settings = {}); +TVector<TStringBuf> GetKeyFilterColumns(const NNodes::TYtSection& section, EYtSettingTypes kind); +bool HasNonEmptyKeyFilter(const NNodes::TYtSection& section); + +NNodes::TYtOutputOpBase GetOutputOp(NNodes::TYtOutput output); + +inline bool IsUnorderedOutput(NNodes::TYtOutput out) { + return out.Mode() && FromString<EYtSettingType>(out.Mode().Cast().Value()) == EYtSettingType::Unordered; +} + +template <typename TGateway> +std::function<bool(const TString&)> MakeUserFilesDownloadFilter(const TGateway& gateway, const TString& activeCluster) { + // hold activeCluster by value to support temp strings + return [&gateway, activeCluster](const TString& url) { + if (activeCluster.empty()) { + // todo: we lost our opportunity to skip download in this case, improve it + return true; + } + + TString extractedCluster; + TString extractedPath; + if (!gateway.TryParseYtUrl(url, &extractedCluster, &extractedPath/* don't remove - triggers additional check against allowed patterns*/)) { + return true; + } + + return (extractedCluster != activeCluster) && (extractedCluster != CurrentYtClusterShortcut); + }; +} + +NNodes::TYtReadTable ConvertContentInputToRead(NNodes::TExprBase input, NNodes::TMaybeNode<NNodes::TCoNameValueTupleList> settings, TExprContext& ctx, NNodes::TMaybeNode<NNodes::TCoAtomList> customFields = {}); + +size_t GetMapDirectOutputsCount(const NNodes::TYtMapReduce& mapReduce); + +bool HasYtRowNumber(const TExprNode& node); + +bool IsYtTableSuitableForArrowInput(NNodes::TExprBase table, std::function<void(const TString&)> unsupportedHandler); + +} diff --git a/yt/yql/providers/yt/provider/yql_yt_horizontal_join.cpp b/yt/yql/providers/yt/provider/yql_yt_horizontal_join.cpp new file mode 100644 index 0000000000..b35f82aca0 --- /dev/null +++ b/yt/yql/providers/yt/provider/yql_yt_horizontal_join.cpp @@ -0,0 +1,1925 @@ +#include "yql_yt_horizontal_join.h" +#include "yql_yt_helpers.h" +#include "yql_yt_optimize.h" + +#include <yt/yql/providers/yt/lib/expr_traits/yql_expr_traits.h> +#include <yt/yql/providers/yt/common/yql_configuration.h> +#include <yql/essentials/core/yql_opt_utils.h> +#include <yql/essentials/core/yql_expr_type_annotation.h> +#include <yql/essentials/utils/log/log.h> + +#include <util/generic/xrange.h> +#include <util/generic/algorithm.h> +#include <util/string/cast.h> +#include <util/digest/numeric.h> + + +template <> +struct THash<std::set<ui64>> { + inline size_t operator()(const std::set<ui64>& s) const { + size_t hash = s.size(); + for (auto& v: s) { + hash = CombineHashes(NumericHash(v), hash); + } + return hash; + } +}; + +namespace NYql { + +using namespace NNodes; + +namespace { + THashSet<TStringBuf> UDF_HORIZONTAL_JOIN_WHITE_LIST = { + "Streaming", + }; +} + +bool THorizontalJoinBase::IsGoodForHorizontalJoin(TYtMap map) const { + // Map already executed or in progress + if (map.Ref().GetState() == TExprNode::EState::ExecutionComplete + || map.Ref().GetState() == TExprNode::EState::ExecutionInProgress + || (map.Ref().HasResult() && map.Ref().GetResult().Type() == TExprNode::World)) { + return false; + } + + // Another node depends on this Map by world + if (HasWorldDeps.find(map.Raw()) != HasWorldDeps.end()) { + return false; + } + + // Map has multiple outputs or input sections + if (map.Input().Size() != 1 || map.Output().Size() != 1) { + return false; + } + + // Map has output limit or is sharded MapJoin + if (NYql::HasAnySetting(map.Settings().Ref(), EYtSettingType::Limit | EYtSettingType::SortLimitBy | EYtSettingType::Sharded | EYtSettingType::JobCount | EYtSettingType::BlockInputApplied)) { + return false; + } + + if (!IsYieldTransparent(map.Mapper().Ptr(), *State_->Types)) { + return false; + } + + bool good = true; + const TExprNode* innerLambdaArg = nullptr; + if (auto maybeInnerLambda = GetFlatMapOverInputStream(map.Mapper()).Lambda()) { + innerLambdaArg = maybeInnerLambda.Cast().Args().Arg(0).Raw(); + } + + VisitExpr(map.Mapper().Body().Ref(), [&good, innerLambdaArg](const TExprNode& node) -> bool { + if (!good) { + return false; + } + if (TYtOutput::Match(&node)) { + // Stop traversing dependent operations + return false; + } + else if (TYtRowNumber::Match(&node)) { + good = false; + } + else if (auto p = TMaybeNode<TYtTablePath>(&node)) { + // Support only YtTableProps in FlatMap over input stream. Other YtTableProps cannot be properly integrated into the Switch + if (p.Cast().DependsOn().Input().Raw() != innerLambdaArg) { + good = false; + } + } + else if (auto p = TMaybeNode<TYtTableRecord>(&node)) { + if (p.Cast().DependsOn().Input().Raw() != innerLambdaArg) { + good = false; + } + } + else if (auto p = TMaybeNode<TYtTableIndex>(&node)) { + if (p.Cast().DependsOn().Input().Raw() != innerLambdaArg) { + good = false; + } + } + return good; + }); + return good; +} + +TCoLambda THorizontalJoinBase::CleanupAuxColumns(TCoLambda lambda, TExprContext& ctx) const { + auto maybeInnerLambda = GetFlatMapOverInputStream(lambda).Lambda(); + if (!maybeInnerLambda) { + return lambda; + } + + TExprNode::TPtr innerLambda = maybeInnerLambda.Cast().Ptr(); + TExprNode::TPtr clonedInnerLambda = ctx.DeepCopyLambda(*innerLambda); + TExprNode::TPtr innerLambdaArg = TCoLambda(clonedInnerLambda).Args().Arg(0).Ptr(); + + TExprNode::TPtr extendedArg = ctx.NewArgument(lambda.Pos(), "extendedArg"); + + bool hasTablePath = false; + bool hasTableRecord = false; + bool hasTableIndex = false; + TNodeOnNodeOwnedMap replaces; + VisitExpr(*clonedInnerLambda, [&](const TExprNode& node) { + if (TYtOutput::Match(&node)) { + // Stop traversing dependent operations + return false; + } + if (auto p = TMaybeNode<TYtTablePath>(&node)) { + auto input = p.Cast().DependsOn().Input(); + if (input.Ptr() == innerLambdaArg) { + hasTablePath = true; + replaces[&node] = Build<TCoMember>(ctx, node.Pos()) + .Struct(extendedArg) + .Name() + .Value("_yql_table_path") + .Build() + .Done().Ptr(); + } + } + else if (auto p = TMaybeNode<TYtTableRecord>(&node)) { + auto input = p.Cast().DependsOn().Input(); + if (input.Ptr() == innerLambdaArg) { + hasTableRecord = true; + replaces[&node] = Build<TCoMember>(ctx, node.Pos()) + .Struct(extendedArg) + .Name() + .Value("_yql_table_record") + .Build() + .Done().Ptr(); + } + } + else if (auto p = TMaybeNode<TYtTableIndex>(&node)) { + auto input = p.Cast().DependsOn().Input(); + if (input.Ptr() == innerLambdaArg) { + hasTableIndex = true; + replaces[&node] = Build<TCoMember>(ctx, node.Pos()) + .Struct(extendedArg) + .Name() + .Value("_yql_table_index") + .Build() + .Done().Ptr(); + } + } + return true; + }); + + if (replaces.empty()) { + return lambda; + } + + TExprNode::TPtr cleanArg = extendedArg; + if (hasTablePath) { + cleanArg = Build<TCoForceRemoveMember>(ctx, cleanArg->Pos()) + .Struct(cleanArg) + .Name() + .Value("_yql_table_path") + .Build() + .Done().Ptr(); + } + if (hasTableRecord) { + cleanArg = Build<TCoForceRemoveMember>(ctx, cleanArg->Pos()) + .Struct(cleanArg) + .Name() + .Value("_yql_table_record") + .Build() + .Done().Ptr(); + } + if (hasTableIndex) { + cleanArg = Build<TCoForceRemoveMember>(ctx, cleanArg->Pos()) + .Struct(cleanArg) + .Name() + .Value("_yql_table_index") + .Build() + .Done().Ptr(); + } + + replaces[innerLambdaArg.Get()] = cleanArg; + + auto body = ctx.ReplaceNodes(clonedInnerLambda->TailPtr(), replaces); + + return Build<TCoLambda>(ctx, lambda.Pos()) + .Args({"stream"}) + .Body<TExprApplier>() + .Apply(lambda) + .With(lambda.Args().Arg(0), "stream") + .With(TExprBase(innerLambda), TExprBase(ctx.NewLambda(clonedInnerLambda->Pos(), ctx.NewArguments(extendedArg->Pos(), { extendedArg }), std::move(body)))) + .Build() + .Done(); +} + +void THorizontalJoinBase::ClearJoinGroup() { + DataSink = {}; + UsesTablePath.Clear(); + UsesTableRecord.Clear(); + UsesTableIndex.Clear(); + OpSettings = {}; + MemUsage.clear(); + JoinedMaps.clear(); + UsedFiles = 1; // jobstate +} + +void THorizontalJoinBase::AddToJoinGroup(TYtMap map) { + if (auto maybeInnerLambda = GetFlatMapOverInputStream(map.Mapper()).Lambda()) { + TExprNode::TPtr innerLambda = maybeInnerLambda.Cast().Ptr(); + TExprNode::TPtr innerLambdaArg = maybeInnerLambda.Cast().Args().Arg(0).Ptr(); + + bool hasTablePath = false; + bool hasTableRecord = false; + bool hasTableIndex = false; + VisitExpr(*innerLambda, [&](const TExprNode& node) { + if (TYtOutput::Match(&node)) { + // Stop traversing dependent operations + return false; + } + if (auto p = TMaybeNode<TYtTablePath>(&node)) { + auto input = p.Cast().DependsOn().Input(); + if (input.Ptr() == innerLambdaArg) { + hasTablePath = true; + } + } + else if (auto p = TMaybeNode<TYtTableRecord>(&node)) { + auto input = p.Cast().DependsOn().Input(); + if (input.Ptr() == innerLambdaArg) { + hasTableRecord = true; + } + } + else if (auto p = TMaybeNode<TYtTableIndex>(&node)) { + auto input = p.Cast().DependsOn().Input(); + if (input.Ptr() == innerLambdaArg) { + hasTableIndex = true; + } + } + return true; + }); + + if (hasTablePath) { + UsesTablePath.Set(JoinedMaps.size()); + } + if (hasTableRecord) { + UsesTableRecord.Set(JoinedMaps.size()); + } + if (hasTableIndex) { + UsesTableIndex.Set(JoinedMaps.size()); + } + } + + if (!DataSink) { + DataSink = map.DataSink(); + } + + if (NYql::HasSetting(map.Settings().Ref(), EYtSettingType::Ordered)) { + OpSettings |= EYtSettingType::Ordered; + } + if (NYql::HasSetting(map.Settings().Ref(), EYtSettingType::WeakFields)) { + OpSettings |= EYtSettingType::WeakFields; + } + if (NYql::HasSetting(map.Settings().Ref(), EYtSettingType::Flow)) { + OpSettings |= EYtSettingType::Flow; + } + + JoinedMaps.push_back(map); +} + +TCoLambda THorizontalJoinBase::BuildMapperWithAuxColumnsForSingleInput(TPositionHandle pos, bool ordered, TExprContext& ctx) const { + auto mapLambda = Build<TCoLambda>(ctx, pos) + .Args({"row"}) + .Body("row") + .Done(); + + if (!UsesTablePath.Empty()) { + mapLambda = Build<TCoLambda>(ctx, pos) + .Args({"row"}) + .Body<TCoAddMember>() + .Struct<TExprApplier>() + .Apply(mapLambda) + .With(0, "row") + .Build() + .Name() + .Value("_yql_table_path") + .Build() + .Item<TYtTablePath>() + .DependsOn() + .Input("row") + .Build() + .Build() + .Build() + .Done(); + } + if (!UsesTableRecord.Empty()) { + mapLambda = Build<TCoLambda>(ctx, pos) + .Args({"row"}) + .Body<TCoAddMember>() + .Struct<TExprApplier>() + .Apply(mapLambda) + .With(0, "row") + .Build() + .Name() + .Value("_yql_table_record") + .Build() + .Item<TYtTableRecord>() + .DependsOn() + .Input("row") + .Build() + .Build() + .Build() + .Done(); + } + if (!UsesTableIndex.Empty()) { + mapLambda = Build<TCoLambda>(ctx, pos) + .Args({"row"}) + .Body<TCoAddMember>() + .Struct<TExprApplier>() + .Apply(mapLambda) + .With(0, "row") + .Build() + .Name() + .Value("_yql_table_index") + .Build() + .Item<TYtTableIndex>() + .DependsOn() + .Input("row") + .Build() + .Build() + .Build() + .Done(); + } + return Build<TCoLambda>(ctx, pos) + .Args({"stream"}) + .Body<TCoMapBase>() + .CallableName(ordered ? TCoOrderedMap::CallableName() : TCoMap::CallableName()) + .Input("stream") + .Lambda(mapLambda) + .Build() + .Done(); +} + +TCoLambda THorizontalJoinBase::BuildMapperWithAuxColumnsForMultiInput(TPositionHandle pos, bool ordered, TExprContext& ctx) const { + TVector<TExprBase> tupleTypes; + for (size_t i: xrange(JoinedMaps.size())) { + auto itemType = JoinedMaps[i].Input().Item(0).Ref().GetTypeAnn()->Cast<TListExprType>()->GetItemType(); + TVector<const TItemExprType*> extraItems; + if (UsesTablePath.Get(i)) { + extraItems.push_back(ctx.MakeType<TItemExprType>("_yql_table_path", + ctx.MakeType<TDataExprType>(EDataSlot::String))); + } + if (UsesTableRecord.Get(i)) { + extraItems.push_back(ctx.MakeType<TItemExprType>("_yql_table_record", + ctx.MakeType<TDataExprType>(EDataSlot::Uint64))); + } + if (UsesTableIndex.Get(i)) { + extraItems.push_back(ctx.MakeType<TItemExprType>("_yql_table_index", + ctx.MakeType<TDataExprType>(EDataSlot::Uint32))); + } + if (!extraItems.empty()) { + auto items = itemType->Cast<TStructExprType>()->GetItems(); + items.insert(items.end(), extraItems.begin(), extraItems.end()); + itemType = ctx.MakeType<TStructExprType>(items); + } + + tupleTypes.push_back(TExprBase(ExpandType(pos, *itemType, ctx))); + } + TExprBase varType = Build<TCoVariantType>(ctx, pos) + .UnderlyingType<TCoTupleType>() + .Add(tupleTypes) + .Build() + .Done(); + + TVector<TExprBase> visitArgs; + for (size_t i: xrange(JoinedMaps.size())) { + visitArgs.push_back(Build<TCoAtom>(ctx, pos).Value(ToString(i)).Done()); + auto visitLambda = Build<TCoLambda>(ctx, pos) + .Args({"row"}) + .Body("row") + .Done(); + + if (UsesTablePath.Get(i)) { + visitLambda = Build<TCoLambda>(ctx, pos) + .Args({"row"}) + .Body<TCoAddMember>() + .Struct<TExprApplier>() + .Apply(visitLambda) + .With(0, "row") + .Build() + .Name() + .Value("_yql_table_path") + .Build() + .Item<TYtTablePath>() + .DependsOn() + .Input("row") + .Build() + .Build() + .Build() + .Done(); + } + + if (UsesTableRecord.Get(i)) { + visitLambda = Build<TCoLambda>(ctx, pos) + .Args({"row"}) + .Body<TCoAddMember>() + .Struct<TExprApplier>() + .Apply(visitLambda) + .With(0, "row") + .Build() + .Name() + .Value("_yql_table_record") + .Build() + .Item<TYtTableRecord>() + .DependsOn() + .Input("row") + .Build() + .Build() + .Build() + .Done(); + } + + if (UsesTableIndex.Get(i)) { + visitLambda = Build<TCoLambda>(ctx, pos) + .Args({"row"}) + .Body<TCoAddMember>() + .Struct<TExprApplier>() + .Apply(visitLambda) + .With(0, "row") + .Build() + .Name() + .Value("_yql_table_index") + .Build() + .Item<TYtTableIndex>() + .DependsOn() + .Input("row") + .Build() + .Build() + .Build() + .Done(); + } + + visitArgs.push_back(Build<TCoLambda>(ctx, pos) + .Args({"row"}) + .Body<TCoVariant>() + .Item<TExprApplier>() + .Apply(visitLambda) + .With(0, "row") + .Build() + .Index() + .Value(ToString(i)) + .Build() + .VarType(varType) + .Build() + .Done()); + } + + return Build<TCoLambda>(ctx, pos) + .Args({"stream"}) + .Body<TCoMapBase>() + .CallableName(ordered ? TCoOrderedMap::CallableName() : TCoMap::CallableName()) + .Input("stream") + .Lambda() + .Args({"var"}) + .Body<TCoVisit>() + .Input("var") + .FreeArgs() + .Add(visitArgs) + .Build() + .Build() + .Build() + .Build() + .Done(); +} + +TCoLambda THorizontalJoinBase::MakeSwitchLambda(size_t mapIndex, size_t fieldsCount, bool singleInput, TExprContext& ctx) const { + auto map = JoinedMaps[mapIndex]; + auto lambda = map.Mapper(); + + if (UsesTablePath.Get(mapIndex) || UsesTableRecord.Get(mapIndex) || UsesTableIndex.Get(mapIndex)) { + lambda = CleanupAuxColumns(lambda, ctx); + } + + if (singleInput) { + auto inputItemType = map.Input().Item(0).Ref().GetTypeAnn()->Cast<TListExprType>()->GetItemType()->Cast<TStructExprType>(); + if (inputItemType->GetSize() != fieldsCount || !UsesTablePath.Empty() || !UsesTableRecord.Empty() || !UsesTableIndex.Empty()) { + TVector<TStringBuf> fields; + for (auto itemType: inputItemType->GetItems()) { + fields.push_back(itemType->GetName()); + } + if (UsesTablePath.Get(mapIndex)) { + fields.push_back("_yql_table_path"); + } + if (UsesTableRecord.Get(mapIndex)) { + fields.push_back("_yql_table_record"); + } + if (UsesTableIndex.Get(mapIndex)) { + fields.push_back("_yql_table_index"); + } + + auto [placeHolder, lambdaWithPlaceholder] = ReplaceDependsOn(lambda.Ptr(), ctx, State_->Types); + YQL_ENSURE(placeHolder); + + lambda = Build<TCoLambda>(ctx, lambda.Pos()) + .Args({"stream"}) + .Body<TExprApplier>() + .Apply(TCoLambda(lambdaWithPlaceholder)) + .With<TCoExtractMembers>(0) + .Input("stream") + .Members(ToAtomList(fields, lambda.Pos(), ctx)) + .Build() + .With(TExprBase(placeHolder), "stream") + .Build() + .Done(); + } + } + + return lambda; +} + +////////////////////////////////////////////////////////////////////////////////////////////////////////////////////// + +IGraphTransformer::TStatus THorizontalJoinOptimizer::Optimize(TExprNode::TPtr input, TExprNode::TPtr& output, TExprContext& ctx) { + TOptimizeExprSettings settings(State_->Types); + settings.ProcessedNodes = ProcessedNodes; + return OptimizeExpr(input, output, [&](const TExprNode::TPtr& node, TExprContext& ctx) -> TExprNode::TPtr { + if (auto maybeRead = TMaybeNode<TCoRight>(node).Input().Maybe<TYtReadTable>()) { + auto read = maybeRead.Cast(); + auto readInput = read.Input().Ptr(); + auto newInput = HandleList(readInput, true, ctx); + if (newInput != readInput) { + if (newInput) { + YQL_CLOG(INFO, ProviderYt) << "PhysicalFinalizing-HorizontalJoin"; + return ctx.ChangeChild(*node, TCoRight::idx_Input, + ctx.ChangeChild(read.Ref(), TYtReadTable::idx_Input, std::move(newInput)) + ); + } + return {}; + } + } + if (auto maybeOp = TMaybeNode<TYtTransientOpBase>(node)) { + if (!node->HasResult() || node->GetResult().Type() != TExprNode::World) { + auto op = maybeOp.Cast(); + auto opInput = op.Input().Ptr(); + auto newInput = HandleList(opInput, true, ctx); + if (newInput != opInput) { + if (newInput) { + YQL_CLOG(INFO, ProviderYt) << "PhysicalFinalizing-HorizontalJoin"; + return ctx.ChangeChild(op.Ref(), TYtTransientOpBase::idx_Input, std::move(newInput)); + } + return {}; + } + } + } + if (auto maybePublish = TMaybeNode<TYtPublish>(node)) { + if (TExprNode::EState::ExecutionComplete != node->GetState() + && TExprNode::EState::ExecutionInProgress != node->GetState()) { + auto publish = maybePublish.Cast(); + auto pubInput = publish.Input().Ptr(); + if (pubInput->ChildrenSize() > 1) { + auto newInput = HandleList(pubInput, false, ctx); + if (newInput != pubInput) { + if (newInput) { + YQL_CLOG(INFO, ProviderYt) << "PhysicalFinalizing-HorizontalJoin"; + return ctx.ChangeChild(publish.Ref(), TYtPublish::idx_Input, std::move(newInput)); + } + return {}; + } + } + } + } + return node; + }, ctx, settings); +} + +TExprNode::TPtr THorizontalJoinOptimizer::HandleList(const TExprNode::TPtr& node, bool sectionList, TExprContext& ctx) { + auto groups = CollectGroups(node, sectionList); + if (groups.empty()) { + return node; + } + + for (auto& group: groups) { + if (group.second.size() < 2) { + continue; + } + + ClearJoinGroup(); + + for (auto& mapGrp: group.second) { + TYtMap map = std::get<0>(mapGrp); + const size_t sectionNum = std::get<1>(mapGrp); + const size_t pathNum = std::get<2>(mapGrp); + + size_t outNdx = JoinedMaps.size(); + const TExprNode* columns = nullptr; + const TExprNode* ranges = nullptr; + if (sectionList) { + auto path = TYtSection(node->Child(sectionNum)).Paths().Item(pathNum); + columns = path.Columns().Raw(); + ranges = path.Ranges().Raw(); + } + + auto uniqIt = UniqMaps.find(map.Raw()); + if (uniqIt != UniqMaps.end()) { + // Map output is used multiple times + // Move all {section,path} pairs to ExclusiveOuts + ExclusiveOuts[uniqIt->second].emplace_back(sectionNum, pathNum); + + auto it = GroupedOuts.find(std::make_tuple(sectionNum, columns, ranges)); + if (it != GroupedOuts.end()) { + auto itOut = it->second.find(uniqIt->second); + if (itOut != it->second.end()) { + ExclusiveOuts[uniqIt->second].emplace_back(sectionNum, itOut->second); + it->second.erase(itOut); + } + if (it->second.empty()) { + GroupedOuts.erase(it); + } + } + continue; + } + + const size_t mapInputCount = map.Input().Item(0).Paths().Size(); + if (const auto nextCount = InputCount + mapInputCount; nextCount > MaxTables) { + if (MakeJoinedMap(node->Pos(), ctx)) { + YQL_CLOG(INFO, ProviderYt) << "HorizontalJoin: split by max input tables: " << nextCount; + } + // Reinit outNdx because MakeJoinedMap() clears JoinedMaps + outNdx = JoinedMaps.size(); + } + + size_t outputCountIncrement = 0; + auto outRowSpec = TYtTableBaseInfo::GetRowSpec(map.Output().Item(0)); + const bool sortedOut = outRowSpec && outRowSpec->IsSorted(); + + if (sortedOut) { + if (ExclusiveOuts.find(outNdx) == ExclusiveOuts.end()) { + // Sorted output cannot be joined with others + outputCountIncrement = 1; + } + } + else if (GroupedOuts.find(std::make_tuple(sectionNum, columns, ranges)) == GroupedOuts.end()) { + outputCountIncrement = 1; + } + + if (ExclusiveOuts.size() + GroupedOuts.size() + outputCountIncrement > MaxOutTables) { + // MakeJoinedMap() clears ExclusiveOuts and GroupedOuts + const auto outCount = ExclusiveOuts.size() + GroupedOuts.size(); + if (MakeJoinedMap(node->Pos(), ctx)) { + YQL_CLOG(INFO, ProviderYt) << "HorizontalJoin: split by max output tables: " << outCount; + } + outputCountIncrement = 1; + // Reinit outNdx because MakeJoinedMap() clears JoinedMaps + outNdx = JoinedMaps.size(); + } + + TExprNode::TPtr updatedLambda = map.Mapper().Ptr(); + if (MaxJobMemoryLimit) { + auto status = UpdateTableContentMemoryUsage(map.Mapper().Ptr(), updatedLambda, State_, ctx, false); + if (status.Level != IGraphTransformer::TStatus::Ok) { + return {}; + } + } + + ScanResourceUsage(*updatedLambda, *State_->Configuration, State_->Types, &MemUsage, nullptr, &UsedFiles); + auto currMemory = Accumulate(MemUsage.begin(), MemUsage.end(), SwitchMemoryLimit, + [](ui64 sum, const std::pair<const TStringBuf, ui64>& val) { return sum + val.second; }); + + // Take into account codec input/output buffers (one for all inputs and one per output) + currMemory += YQL_JOB_CODEC_MEM * (ExclusiveOuts.size() + GroupedOuts.size() + outputCountIncrement + 1); + + if ((MaxJobMemoryLimit && currMemory > *MaxJobMemoryLimit) || UsedFiles > MaxOperationFiles) { + const auto usedFiles = UsedFiles; // Save value, because MakeJoinedMap will clear it + if (MakeJoinedMap(node->Pos(), ctx)) { + YQL_CLOG(INFO, ProviderYt) << "HorizontalJoin: split by limits. Memory: " << currMemory << ", files: " << usedFiles; + } + ScanResourceUsage(*updatedLambda, *State_->Configuration, State_->Types, &MemUsage, nullptr, &UsedFiles); + // Reinit outNdx because MakeJoinedMap() clears joinedMapOuts + outNdx = JoinedMaps.size(); + } + + InputCount += mapInputCount; + + if (map.World().Ref().Type() != TExprNode::World) { + Worlds.emplace(map.World().Ptr(), Worlds.size()); + } + + if (sortedOut) { + ExclusiveOuts[outNdx].emplace_back(sectionNum, pathNum); + } else { + GroupedOuts[std::make_tuple(sectionNum, columns, ranges)][outNdx] = pathNum; + } + UniqMaps.emplace(map.Raw(), outNdx); + + auto section = map.Input().Item(0); + if (section.Paths().Size() == 1) { + auto path = section.Paths().Item(0); + GroupedInputs.emplace(path.Table().Raw(), path.Ranges().Raw(), section.Settings().Raw(), NYql::HasSetting(map.Settings().Ref(), EYtSettingType::WeakFields)); + } else { + GroupedInputs.emplace(section.Raw(), nullptr, nullptr, NYql::HasSetting(map.Settings().Ref(), EYtSettingType::WeakFields)); + } + + AddToJoinGroup(map); + } + MakeJoinedMap(node->Pos(), ctx); + } + + if (InputSubsts.empty()) { + return node; + } + + return RebuildList(node, sectionList, ctx); +} + +void THorizontalJoinOptimizer::AddToGroups(TYtMap map, size_t s, size_t p, const TExprNode* section, + THashMap<TGroupKey, TVector<std::tuple<TYtMap, size_t, size_t>>>& groups, + TNodeMap<TMaybe<TGroupKey>>& processedMaps) const +{ + auto processedIt = processedMaps.find(map.Raw()); + if (processedIt != processedMaps.end()) { + if (processedIt->second) { + groups[*processedIt->second].emplace_back(map, s, p); + } + return; + } + + bool good = IsGoodForHorizontalJoin(map); + + // Map output has multiple readers + if (good) { + auto readersIt = OpDeps.find(map.Raw()); + if (readersIt == OpDeps.end()) { + good = false; + } + else if (readersIt->second.size() != 1) { + // Allow multiple readers from the same section + for (auto& r: readersIt->second) { + if (std::get<1>(r) != section) { + good = false; + break; + } + } + } + } + + // Gather not yet completed dependencies (input data and world). Not more than one is allowed + const TExprNode* dep = nullptr; + bool yamr = false; + bool qb2 = false; + if (good) { + TNodeSet deps; + for (auto path: map.Input().Item(0).Paths()) { + yamr = path.Table().Maybe<TYtTable>().RowSpec().Maybe<TCoVoid>().IsValid(); + if (auto setting = path.Table().Maybe<TYtTable>().Settings()) { + qb2 = qb2 || NYql::HasSetting(setting.Ref(), EYtSettingType::WithQB); + } + if (auto op = path.Table().Maybe<TYtOutput>().Operation()) { + const TExprNode* opNode = op.Cast().Raw(); + if (opNode->GetState() != TExprNode::EState::ExecutionComplete + || !opNode->HasResult() + || opNode->GetResult().Type() != TExprNode::World) { + deps.insert(opNode); + } + } + } + if (map.World().Ref().Type() != TExprNode::World + && map.World().Ref().GetState() != TExprNode::EState::ExecutionComplete) { + deps.insert(map.World().Raw()); + } + + if (deps.size() > 1) { + good = false; + } else if (deps.size() == 1) { + dep = *deps.begin(); + } + } + + if (good) { + ui32 flags = 0; + if (yamr) { + flags |= EFeatureFlags::YAMR; + } + if (qb2) { + flags |= EFeatureFlags::QB2; + } + if (NYql::HasSetting(map.Settings().Ref(), EYtSettingType::Flow)) { + flags |= EFeatureFlags::Flow; + } + auto key = TGroupKey { + TString{map.DataSink().Cluster().Value()}, + NYql::GetSampleParams(map.Input().Item(0).Settings().Ref()), + dep, + flags + }; + groups[key].emplace_back(map, s, p); + processedMaps.emplace_hint(processedIt, map.Raw(), key); + } else { + processedMaps.emplace_hint(processedIt, map.Raw(), Nothing()); + } +} + +THashMap<THorizontalJoinOptimizer::TGroupKey, TVector<std::tuple<TYtMap, size_t, size_t>>> THorizontalJoinOptimizer::CollectGroups(const TExprNode::TPtr& node, bool sectionList) const { + + THashMap<TGroupKey, TVector<std::tuple<TYtMap, size_t, size_t>>> groups; // group key -> Vector{Map, section_num, path_num} + TNodeMap<TMaybe<TGroupKey>> processedMaps; + + if (sectionList) { + auto sections = TYtSectionList(node); + for (size_t s = 0; s < sections.Size(); ++s) { + auto section = sections.Item(s); + for (size_t p = 0, pathCount = section.Paths().Size(); p < pathCount; ++p) { + auto path = section.Paths().Item(p); + if (auto maybeMap = path.Table().Maybe<TYtOutput>().Operation().Maybe<TYtMap>()) { + AddToGroups(maybeMap.Cast(), s, p, section.Raw(), groups, processedMaps); + } + } + } + } else { + auto list = TYtOutputList(node); + for (size_t s = 0; s < list.Size(); ++s) { + auto out = list.Item(s); + if (auto maybeMap = out.Operation().Maybe<TYtMap>()) { + // Treat each output as a separate section to prevent output concat + AddToGroups(maybeMap.Cast(), s, 0, nullptr, groups, processedMaps); + } + } + } + + return groups; +} + +void THorizontalJoinOptimizer::ClearJoinGroup() { + THorizontalJoinBase::ClearJoinGroup(); + Worlds.clear(); + UniqMaps.clear(); + GroupedOuts.clear(); + ExclusiveOuts.clear(); + InputCount = 0; + GroupedInputs.clear(); +} + +bool THorizontalJoinOptimizer::MakeJoinedMap(TPositionHandle pos, TExprContext& ctx) { + bool res = false; + if (JoinedMaps.size() > 1) { + + const bool singleInput = GroupedInputs.size() == 1; + const bool ordered = OpSettings.HasFlags(EYtSettingType::Ordered); + + TCoLambda mapper = (!UsesTablePath.Empty() || !UsesTableRecord.Empty() || !UsesTableIndex.Empty()) + ? (singleInput ? BuildMapperWithAuxColumnsForSingleInput(pos, ordered, ctx) : BuildMapperWithAuxColumnsForMultiInput(pos, ordered, ctx)) + : Build<TCoLambda>(ctx, pos) + .Args({"stream"}) + .Body("stream") + .Done(); + + TSet<TStringBuf> usedFields; + TSet<TStringBuf> weakFields; + if (singleInput) { + for (auto map: JoinedMaps) { + for (auto itemType: map.Input().Item(0).Ref().GetTypeAnn()->Cast<TListExprType>()->GetItemType()->Cast<TStructExprType>()->GetItems()) { + usedFields.insert(itemType->GetName()); + } + if (std::get<3>(*GroupedInputs.begin())) { + for (auto path: map.Input().Item(0).Paths()) { + if (auto columns = path.Columns().Maybe<TExprList>()) { + for (auto child: columns.Cast()) { + if (auto maybeTuple = child.Maybe<TCoAtomList>()) { + auto tuple = maybeTuple.Cast(); + if (tuple.Item(1).Value() == "weak") { + weakFields.insert(tuple.Item(0).Value()); + } + } + } + } + } + } + } + } + + TVector<TExprBase> switchArgs; + for (size_t i: xrange(JoinedMaps.size())) { + auto lambda = MakeSwitchLambda(i, usedFields.size(), singleInput, ctx); + + switchArgs.push_back( + Build<TCoAtomList>(ctx, lambda.Pos()) + .Add() + .Value(singleInput ? TString("0") : ToString(i)) + .Build() + .Done() + ); + switchArgs.push_back(lambda); + } + + auto settingsBuilder = Build<TCoNameValueTupleList>(ctx, pos); + if (OpSettings.HasFlags(EYtSettingType::Ordered)) { + settingsBuilder.Add() + .Name().Value(ToString(EYtSettingType::Ordered)).Build() + .Build(); + } + if (OpSettings.HasFlags(EYtSettingType::WeakFields)) { + settingsBuilder.Add() + .Name().Value(ToString(EYtSettingType::WeakFields)).Build() + .Build(); + } + if (OpSettings.HasFlags(EYtSettingType::Flow)) { + settingsBuilder.Add() + .Name().Value(ToString(EYtSettingType::Flow)).Build() + .Build(); + } + + mapper = Build<TCoLambda>(ctx, pos) + .Args({"stream"}) + .Body<TCoSwitch>() + .Input<TExprApplier>() + .Apply(mapper) + .With(0, "stream") + .Build() + .BufferBytes() + .Value(ToString(SwitchMemoryLimit)) + .Build() + .FreeArgs() + .Add(switchArgs) + .Build() + .Build() + .Done(); + + // outRemap[oldOutIndex] -> {newOutIndex, drop_flag, TVector{section_num, path_num}} + TVector<std::tuple<size_t, bool, TVector<std::pair<size_t, size_t>>>> outRemap; + outRemap.resize(JoinedMaps.size()); + const bool lessOuts = ExclusiveOuts.size() + GroupedOuts.size() < JoinedMaps.size(); + size_t nextNewOutIndex = 0; + TVector<TYtOutTable> joinedMapOuts; + for (auto& out: ExclusiveOuts) { + auto old_out_num = out.first; + auto& inputs = out.second; + if (lessOuts) { + outRemap[old_out_num] = std::make_tuple(nextNewOutIndex++, false, inputs); + joinedMapOuts.push_back(JoinedMaps[old_out_num].Output().Item(0)); + } else { + outRemap[old_out_num] = std::make_tuple(old_out_num, false, inputs); + } + } + for (auto& out: GroupedOuts) { + auto& grp = out.first; + auto& inputs = out.second; + TMaybe<size_t> newIndex; + size_t baseOldIndex = 0; + TVector<std::pair<size_t, size_t>> pairsSecPath; + for (auto in: inputs) { + auto old_out_num = in.first; + auto path_num = in.second; + if (!newIndex) { + newIndex = nextNewOutIndex++; + baseOldIndex = old_out_num; + } + std::get<0>(outRemap[old_out_num]) = *newIndex; + pairsSecPath.emplace_back(std::get<0>(grp), path_num); + } + if (lessOuts) { + outRemap[baseOldIndex] = std::make_tuple(*newIndex, true, std::move(pairsSecPath)); + joinedMapOuts.push_back(JoinedMaps[baseOldIndex].Output().Item(0)); + } else { + YQL_ENSURE(pairsSecPath.size() == 1); + outRemap[baseOldIndex] = std::make_tuple(baseOldIndex, true, std::move(pairsSecPath)); + } + } + + if (lessOuts) { + YQL_ENSURE(ExclusiveOuts.size() + GroupedOuts.size() == nextNewOutIndex, "Output table count mismatch: " << (ExclusiveOuts.size() + GroupedOuts.size()) << " != " << nextNewOutIndex); + if (nextNewOutIndex == 1) { + mapper = Build<TCoLambda>(ctx, pos) + .Args({"stream"}) + .Body<TCoFlatMapBase>() + .CallableName(ordered ? TCoOrderedFlatMap::CallableName() : TCoFlatMap::CallableName()) + .Input<TExprApplier>() + .Apply(mapper) + .With(0, "stream") + .Build() + .Lambda() + .Args({"item"}) + .Body<TCoJust>() + .Input<TCoVariantItem>() + .Variant("item") + .Build() + .Build() + .Build() + .Build() + .Done(); + } + else if (nextNewOutIndex < outRemap.size()) { + TVector<TExprBase> tupleTypes; + for (auto out: joinedMapOuts) { + auto itemType = out.Ref().GetTypeAnn()->Cast<TListExprType>()->GetItemType(); + tupleTypes.push_back(TExprBase(ExpandType(pos, *itemType, ctx))); + } + TExprBase varType = Build<TCoVariantType>(ctx, pos) + .UnderlyingType<TCoTupleType>() + .Add(tupleTypes) + .Build() + .Done(); + + TVector<TExprBase> visitArgs; + for (size_t i: xrange(outRemap.size())) { + visitArgs.push_back(Build<TCoAtom>(ctx, pos).Value(ToString(i)).Done()); + visitArgs.push_back(Build<TCoLambda>(ctx, pos) + .Args({"row"}) + .Body<TCoVariant>() + .Item("row") + .Index() + .Value(ToString(std::get<0>(outRemap[i]))) + .Build() + .VarType(varType) + .Build() + .Done()); + } + + mapper = Build<TCoLambda>(ctx, pos) + .Args({"stream"}) + .Body<TCoFlatMapBase>() + .CallableName(ordered ? TCoOrderedFlatMap::CallableName() : TCoFlatMap::CallableName()) + .Input<TExprApplier>() + .Apply(mapper) + .With(0, "stream") + .Build() + .Lambda() + .Args({"item"}) + .Body<TCoJust>() + .Input<TCoVisit>() + .Input("item") + .FreeArgs() + .Add(visitArgs) + .Build() + .Build() + .Build() + .Build() + .Build() + .Done(); + } + } else { + for (auto map: JoinedMaps) { + joinedMapOuts.push_back(map.Output().Item(0)); + } + } + + TVector<TYtSection> joinedMapSections; + if (singleInput) { + auto section = JoinedMaps.front().Input().Item(0); + auto itemType = section.Ref().GetTypeAnn()->Cast<TListExprType>()->GetItemType()->Cast<TStructExprType>(); + + TMaybeNode<TExprBase> columns; + if (!weakFields.empty()) { + auto columnsBuilder = Build<TExprList>(ctx, pos); + for (auto& field: usedFields) { + if (weakFields.contains(field)) { + columnsBuilder + .Add<TCoAtomList>() + .Add() + .Value(field) + .Build() + .Add() + .Value("weak") + .Build() + .Build(); + } + else { + columnsBuilder + .Add<TCoAtom>() + .Value(field) + .Build(); + } + } + columns = columnsBuilder.Done(); + } + else if (usedFields.size() != itemType->GetSize()) { + columns = TExprBase(ToAtomList(usedFields, pos, ctx)); + } + + if (columns) { + TVector<TYtPath> paths; + for (const auto& path : section.Paths()) { + paths.push_back(Build<TYtPath>(ctx, path.Pos()) + .InitFrom(path) + .Columns(columns.Cast()) + .Stat<TCoVoid>().Build() + .Done()); + } + + section = Build<TYtSection>(ctx, section.Pos()) + .InitFrom(section) + .Paths() + .Add(paths) + .Build() + .Done(); + } + joinedMapSections.push_back(section); + } + else { + for (auto map: JoinedMaps) { + joinedMapSections.push_back(map.Input().Item(0)); + } + } + + auto joinedMap = Build<TYtMap>(ctx, pos) + .World(NYql::ApplySyncListToWorld(ctx.NewWorld(pos), Worlds, ctx)) + .DataSink(DataSink.Cast()) + .Output() + .Add(joinedMapOuts) + .Build() + .Input() + .Add(joinedMapSections) + .Build() + .Settings(settingsBuilder.Done()) + .Mapper(mapper) + .Done(); + + // {section_num, path_num} -> {joined_map, out_num} + for (auto& r: outRemap) { + auto outIndex = std::get<0>(r); + auto dropFlag = std::get<1>(r); + auto& inputs = std::get<2>(r); + if (!inputs.empty()) { + InputSubsts.emplace(inputs.front(), std::make_pair(joinedMap, outIndex)); + if (dropFlag) { + for (size_t i = 1; i < inputs.size(); ++i) { + InputSubsts.emplace(inputs[i], Nothing()); + } + } + else { + for (size_t i = 1; i < inputs.size(); ++i) { + InputSubsts.emplace(inputs[i], std::make_pair(joinedMap, outIndex)); + } + } + } + } + res = true; + } + + ClearJoinGroup(); + return res; +} + +TExprNode::TPtr THorizontalJoinOptimizer::RebuildList(const TExprNode::TPtr& node, bool sectionList, TExprContext& ctx) { + TExprNode::TPtr res; + if (sectionList) { + TVector<TYtSection> updatedSections; + for (size_t s = 0; s < node->ChildrenSize(); ++s) { + auto section = TYtSection(node->ChildPtr(s)); + updatedSections.push_back(section); + + TVector<TYtPath> updatedPaths; + bool hasUpdatedPaths = false; + for (size_t p = 0, pathCount = section.Paths().Size(); p < pathCount; ++p) { + auto path = section.Paths().Item(p); + auto it = InputSubsts.find(std::make_pair(s, p)); + if (it != InputSubsts.end()) { + if (it->second.Defined()) { + updatedPaths.push_back(Build<TYtPath>(ctx, path.Pos()) + .InitFrom(path) + .Table<TYtOutput>() + .Operation(it->second->first) + .OutIndex() + .Value(ToString(it->second->second)) + .Build() + .Mode(path.Table().Maybe<TYtOutput>().Mode()) + .Build() + .Done()); + } + hasUpdatedPaths = true; + } + else { + updatedPaths.push_back(path); + } + } + + if (hasUpdatedPaths) { + updatedSections.back() = Build<TYtSection>(ctx, section.Pos()) + .InitFrom(section) + .Paths() + .Add(updatedPaths) + .Build() + .Done(); + } + } + res = Build<TYtSectionList>(ctx, node->Pos()).Add(updatedSections).Done().Ptr(); + } + else { + TVector<TYtOutput> updatedOuts; + for (size_t s = 0; s < node->ChildrenSize(); ++s) { + auto out = TYtOutput(node->ChildPtr(s)); + + auto it = InputSubsts.find(std::make_pair(s, 0)); + if (it != InputSubsts.end()) { + if (it->second.Defined()) { + updatedOuts.push_back(Build<TYtOutput>(ctx, out.Pos()) + .Operation(it->second->first) + .OutIndex() + .Value(ToString(it->second->second)) + .Build() + .Mode(out.Mode()) + .Done()); + } + } + else { + updatedOuts.push_back(out); + } + } + res = Build<TYtOutputList>(ctx, node->Pos()).Add(updatedOuts).Done().Ptr(); + } + InputSubsts.clear(); + return res; +} + +////////////////////////////////////////////////////////////////////////////////////////////////////////////////////// + +IGraphTransformer::TStatus TMultiHorizontalJoinOptimizer::Optimize(TExprNode::TPtr input, TExprNode::TPtr& output, TExprContext& ctx) { + THashMap<TGroupKey, TVector<TYtMap>> mapGroups; + + for (auto writer: OpDepsOrder) { + if (auto maybeMap = TMaybeNode<TYtMap>(writer)) { + auto map = maybeMap.Cast(); + if (IsGoodForHorizontalJoin(map)) { + bool good = true; + const TExprNode* dep = nullptr; + bool yamr = false; + bool qb2 = false; + TNodeSet deps; + for (auto path: map.Input().Item(0).Paths()) { + yamr = path.Table().Maybe<TYtTable>().RowSpec().Maybe<TCoVoid>().IsValid(); + if (auto setting = path.Table().Maybe<TYtTable>().Settings()) { + qb2 = qb2 || NYql::HasSetting(setting.Ref(), EYtSettingType::WithQB); + } + if (auto op = path.Table().Maybe<TYtOutput>().Operation()) { + const TExprNode* opNode = op.Cast().Raw(); + if (opNode->GetState() != TExprNode::EState::ExecutionComplete + || !opNode->HasResult() + || opNode->GetResult().Type() != TExprNode::World) { + deps.insert(opNode); + } + } + } + if (map.World().Ref().Type() != TExprNode::World + && map.World().Ref().GetState() != TExprNode::EState::ExecutionComplete) { + deps.insert(map.World().Raw()); + } + + if (deps.size() > 1) { + good = false; + } else if (deps.size() == 1) { + dep = *deps.begin(); + } + + if (good) { + std::set<ui64> readerIds; + for (auto& reader: OpDeps.at(writer)) { + readerIds.insert(std::get<0>(reader)->UniqueId()); + } + ui32 flags = 0; + if (yamr) { + flags |= EFeatureFlags::YAMR; + } + if (qb2) { + flags |= EFeatureFlags::QB2; + } + if (NYql::HasSetting(map.Settings().Ref(), EYtSettingType::Flow)) { + flags |= EFeatureFlags::Flow; + } + auto key = TGroupKey { + TString{map.DataSink().Cluster().Value()}, + readerIds, + NYql::GetSampleParams(map.Input().Item(0).Settings().Ref()), + dep, + flags + }; + mapGroups[key].push_back(map); + } + } + } + } + if (mapGroups.empty()) { + return IGraphTransformer::TStatus::Ok; + } + + for (auto& grp: mapGroups) { + if (grp.second.size() > 1) { + ::Sort(grp.second.begin(), grp.second.end(), + [](const TYtMap& m1, const TYtMap& m2) { return m1.Ref().UniqueId() < m2.Ref().UniqueId(); }); + + if (!HandleGroup(grp.second, ctx)) { + return IGraphTransformer::TStatus::Error; + } + if (!OutputSubsts.empty()) { + break; + } + } + } + + if (!OutputSubsts.empty()) { + TNodeOnNodeOwnedMap toOptimize; + VisitExpr(input, [&](const TExprNode::TPtr& node) { + if (auto maybeOut = TMaybeNode<TYtOutput>(node)) { + auto out = maybeOut.Cast(); + auto it = OutputSubsts.find(out.Operation().Raw()); + if (it != OutputSubsts.end()) { + YQL_ENSURE(out.OutIndex().Value() == "0"); + toOptimize[node.Get()] = Build<TYtOutput>(ctx, out.Pos()) + .Operation(it->second.first) + .OutIndex() + .Value(ToString(it->second.second)) + .Build() + .Mode(out.Mode()) + .Done().Ptr(); + } + } + return true; + }); + + YQL_CLOG(INFO, ProviderYt) << "PhysicalFinalizing-MultiHorizontalJoin"; + return RemapExpr(input, output, toOptimize, ctx, TOptimizeExprSettings(State_->Types)); + } + return IGraphTransformer::TStatus::Ok; +} + +bool TMultiHorizontalJoinOptimizer::HandleGroup(const TVector<TYtMap>& maps, TExprContext& ctx) { + for (TYtMap map: maps) { + const size_t mapInputCount = map.Input().Item(0).Paths().Size(); + if (const auto nextCount = InputCount + mapInputCount; nextCount > MaxTables) { + if (MakeJoinedMap(ctx)) { + YQL_CLOG(INFO, ProviderYt) << "MultiHorizontalJoin: split by max input tables: " << nextCount; + } + } + + if (const auto nextMapCount = JoinedMaps.size() + 1; nextMapCount > MaxOutTables) { + if (MakeJoinedMap(ctx)) { + YQL_CLOG(INFO, ProviderYt) << "MultiHorizontalJoin: split by max output tables: " << nextMapCount; + } + } + + TExprNode::TPtr updatedLambda = map.Mapper().Ptr(); + if (MaxJobMemoryLimit) { + if (UpdateTableContentMemoryUsage(map.Mapper().Ptr(), updatedLambda, State_, ctx, false).Level != IGraphTransformer::TStatus::Ok) { + return false; + } + } + + ScanResourceUsage(*updatedLambda, *State_->Configuration, State_->Types, &MemUsage, nullptr, &UsedFiles); + auto currMemory = Accumulate(MemUsage.begin(), MemUsage.end(), SwitchMemoryLimit, + [](ui64 sum, const std::pair<const TStringBuf, ui64>& val) { return sum + val.second; }); + + // Take into account codec input/output buffers (one for all inputs and one per output) + currMemory += YQL_JOB_CODEC_MEM * (JoinedMaps.size() + 2); + + if ((MaxJobMemoryLimit && currMemory > *MaxJobMemoryLimit) || UsedFiles > MaxOperationFiles) { + const auto usedFiles = UsedFiles; // Save value, because MakeJoinedMap will clear it + if (MakeJoinedMap(ctx)) { + YQL_CLOG(INFO, ProviderYt) << "MultiHorizontalJoin: split by limits. Memory: " << currMemory << ", files: " << usedFiles; + } + ScanResourceUsage(*updatedLambda, *State_->Configuration, State_->Types, &MemUsage, nullptr, &UsedFiles); + } + + InputCount += mapInputCount; + + if (map.World().Ref().Type() != TExprNode::World) { + Worlds.emplace(map.World().Ptr(), Worlds.size()); + } + + auto section = map.Input().Item(0); + if (section.Paths().Size() == 1) { + auto path = section.Paths().Item(0); + GroupedInputs.emplace(path.Table().Raw(), path.Ranges().Raw(), section.Settings().Raw(), NYql::HasSetting(map.Settings().Ref(), EYtSettingType::WeakFields)); + } else { + GroupedInputs.emplace(section.Raw(), nullptr, nullptr, NYql::HasSetting(map.Settings().Ref(), EYtSettingType::WeakFields)); + } + + AddToJoinGroup(map); + } + + MakeJoinedMap(ctx); + return true; +}; + +void TMultiHorizontalJoinOptimizer::ClearJoinGroup() { + THorizontalJoinBase::ClearJoinGroup(); + Worlds.clear(); + InputCount = 0; + GroupedInputs.clear(); +} + +bool TMultiHorizontalJoinOptimizer::MakeJoinedMap(TExprContext& ctx) { + bool res = false; + if (JoinedMaps.size() > 1) { + TPositionHandle pos = JoinedMaps.front().Ref().Pos(); + + const bool singleInput = GroupedInputs.size() == 1; + const bool ordered = OpSettings.HasFlags(EYtSettingType::Ordered); + + TCoLambda mapper = (!UsesTablePath.Empty() || !UsesTableRecord.Empty() || !UsesTableIndex.Empty()) + ? (singleInput ? BuildMapperWithAuxColumnsForSingleInput(pos, ordered, ctx) : BuildMapperWithAuxColumnsForMultiInput(pos, ordered, ctx)) + : Build<TCoLambda>(ctx, pos) + .Args({"stream"}) + .Body("stream") + .Done(); + + TSet<TStringBuf> usedFields; + TSet<TStringBuf> weakFields; + if (singleInput) { + for (auto map: JoinedMaps) { + for (auto itemType: map.Input().Item(0).Ref().GetTypeAnn()->Cast<TListExprType>()->GetItemType()->Cast<TStructExprType>()->GetItems()) { + usedFields.insert(itemType->GetName()); + } + if (std::get<3>(*GroupedInputs.begin())) { + for (auto path: map.Input().Item(0).Paths()) { + if (auto columns = path.Columns().Maybe<TExprList>()) { + for (auto child: columns.Cast()) { + if (auto maybeTuple = child.Maybe<TCoAtomList>()) { + auto tuple = maybeTuple.Cast(); + if (tuple.Item(1).Value() == "weak") { + weakFields.insert(tuple.Item(0).Value()); + } + } + } + } + } + } + } + } + + TVector<TExprBase> switchArgs; + for (size_t i: xrange(JoinedMaps.size())) { + auto lambda = MakeSwitchLambda(i, usedFields.size(), singleInput, ctx); + + switchArgs.push_back( + Build<TCoAtomList>(ctx, lambda.Pos()) + .Add() + .Value(singleInput ? TString("0") : ToString(i)) + .Build() + .Done() + ); + switchArgs.push_back(lambda); + } + + auto settingsBuilder = Build<TCoNameValueTupleList>(ctx, pos); + if (OpSettings.HasFlags(EYtSettingType::Ordered)) { + settingsBuilder.Add() + .Name().Value(ToString(EYtSettingType::Ordered)).Build() + .Build(); + } + if (OpSettings.HasFlags(EYtSettingType::WeakFields)) { + settingsBuilder.Add() + .Name().Value(ToString(EYtSettingType::WeakFields)).Build() + .Build(); + } + if (OpSettings.HasFlags(EYtSettingType::Flow)) { + settingsBuilder.Add() + .Name().Value(ToString(EYtSettingType::Flow)).Build() + .Build(); + } + + mapper = Build<TCoLambda>(ctx, pos) + .Args({"stream"}) + .Body<TCoSwitch>() + .Input<TExprApplier>() + .Apply(mapper) + .With(0, "stream") + .Build() + .BufferBytes() + .Value(ToString(SwitchMemoryLimit)) + .Build() + .FreeArgs() + .Add(switchArgs) + .Build() + .Build() + .Done(); + + TVector<TYtOutTable> joinedMapOuts; + for (auto map: JoinedMaps) { + joinedMapOuts.push_back(map.Output().Item(0)); + } + + TVector<TYtSection> joinedMapSections; + if (singleInput) { + auto section = JoinedMaps.front().Input().Item(0); + auto itemType = section.Ref().GetTypeAnn()->Cast<TListExprType>()->GetItemType()->Cast<TStructExprType>(); + + TMaybeNode<TExprBase> columns; + if (!weakFields.empty()) { + auto columnsBuilder = Build<TExprList>(ctx, pos); + for (auto& field: usedFields) { + if (weakFields.contains(field)) { + columnsBuilder + .Add<TCoAtomList>() + .Add() + .Value(field) + .Build() + .Add() + .Value("weak") + .Build() + .Build(); + } + else { + columnsBuilder + .Add<TCoAtom>() + .Value(field) + .Build(); + } + } + columns = columnsBuilder.Done(); + } + else if (usedFields.size() != itemType->GetSize()) { + columns = TExprBase(ToAtomList(usedFields, pos, ctx)); + } + + if (columns) { + TVector<TYtPath> paths; + for (const auto& path : section.Paths()) { + paths.push_back(Build<TYtPath>(ctx, path.Pos()) + .InitFrom(path) + .Columns(columns.Cast()) + .Stat<TCoVoid>().Build() + .Done()); + } + + section = Build<TYtSection>(ctx, section.Pos()) + .InitFrom(section) + .Paths() + .Add(paths) + .Build() + .Done(); + } + joinedMapSections.push_back(section); + } + else { + for (auto map: JoinedMaps) { + joinedMapSections.push_back(map.Input().Item(0)); + } + } + + auto joinedMap = Build<TYtMap>(ctx, pos) + .World(NYql::ApplySyncListToWorld(ctx.NewWorld(pos), Worlds, ctx)) + .DataSink(DataSink.Cast()) + .Output() + .Add(joinedMapOuts) + .Build() + .Input() + .Add(joinedMapSections) + .Build() + .Settings(settingsBuilder.Done()) + .Mapper(mapper) + .Done(); + + for (size_t i: xrange(JoinedMaps.size())) { + auto map = JoinedMaps[i]; + OutputSubsts.emplace(map.Raw(), std::make_pair(joinedMap, i)); + } + + res = true; + } + + ClearJoinGroup(); + return res; +} + +////////////////////////////////////////////////////////////////////////////////////////////////////////////////////// + +void TOutHorizontalJoinOptimizer::ClearJoinGroup() { + THorizontalJoinBase::ClearJoinGroup(); + UsedFields.clear(); + WeakFields.clear(); + UsedSysFields.clear(); +} + +bool TOutHorizontalJoinOptimizer::MakeJoinedMap(TPositionHandle pos, const TGroupKey& key, const TStructExprType* itemType, TExprContext& ctx) { + bool res = false; + if (JoinedMaps.size() > 1) { + const bool ordered = OpSettings.HasFlags(EYtSettingType::Ordered); + auto mapper = (!UsesTablePath.Empty() || !UsesTableRecord.Empty() || !UsesTableIndex.Empty()) + ? BuildMapperWithAuxColumnsForSingleInput(pos, ordered, ctx) + : Build<TCoLambda>(ctx, pos) + .Args({"stream"}) + .Body("stream") + .Done(); + + TVector<TExprBase> switchArgs; + for (size_t i: xrange(JoinedMaps.size())) { + + auto lambda = MakeSwitchLambda(i, UsedFields.size(), true, ctx); + + switchArgs.push_back( + Build<TCoAtomList>(ctx, lambda.Pos()) + .Add() + .Value("0") + .Build() + .Done() + ); + switchArgs.push_back(lambda); + } + + auto settingsBuilder = Build<TCoNameValueTupleList>(ctx, pos); + if (OpSettings.HasFlags(EYtSettingType::Ordered)) { + settingsBuilder.Add() + .Name().Value(ToString(EYtSettingType::Ordered)).Build() + .Build(); + } + if (OpSettings.HasFlags(EYtSettingType::WeakFields)) { + settingsBuilder.Add() + .Name().Value(ToString(EYtSettingType::WeakFields)).Build() + .Build(); + } + if (OpSettings.HasFlags(EYtSettingType::Flow)) { + settingsBuilder.Add() + .Name().Value(ToString(EYtSettingType::Flow)).Build() + .Build(); + } + + for (auto sys: UsedSysFields) { + UsedFields.erase(TString(YqlSysColumnPrefix).append(sys)); + } + + TExprBase columns = Build<TCoVoid>(ctx, pos).Done(); + if (!WeakFields.empty()) { + auto columnsBuilder = Build<TExprList>(ctx, pos); + for (auto& field: UsedFields) { + if (WeakFields.contains(field)) { + columnsBuilder + .Add<TCoAtomList>() + .Add() + .Value(field) + .Build() + .Add() + .Value("weak") + .Build() + .Build(); + } + else { + columnsBuilder + .Add<TCoAtom>() + .Value(field) + .Build(); + } + } + columns = columnsBuilder.Done(); + } + else if (UsedFields.size() != itemType->GetSize()) { + columns = TExprBase(ToAtomList(UsedFields, pos, ctx)); + } + + TVector<TYtOutTable> joinedMapOuts; + for (auto& map: JoinedMaps) { + joinedMapOuts.push_back(map.Output().Item(0)); + } + + auto sectionSettingsBuilder = Build<TCoNameValueTupleList>(ctx, pos); + if (std::get<4>(key)) { + sectionSettingsBuilder.Add(ctx.ShallowCopy(*std::get<4>(key))); + } + if (!UsedSysFields.empty()) { + sectionSettingsBuilder + .Add() + .Name() + .Value(ToString(EYtSettingType::SysColumns)) + .Build() + .Value(ToAtomList(UsedSysFields, pos, ctx)) + .Build() + ; + } + + auto joinedMap = Build<TYtMap>(ctx, pos) + .World(JoinedMaps.front().World()) + .DataSink(DataSink.Cast()) + .Output() + .Add(joinedMapOuts) + .Build() + .Input() + .Add() + .Paths() + .Add() + .Table(ctx.ShallowCopy(*std::get<2>(key))) + .Columns(columns) + .Ranges(ctx.ShallowCopy(*std::get<3>(key))) + .Stat<TCoVoid>().Build() + .Build() + .Build() + .Settings(sectionSettingsBuilder.Done()) + .Build() + .Build() + .Settings(settingsBuilder.Done()) + .Mapper() + .Args({"stream"}) + .Body<TCoSwitch>() + .Input<TExprApplier>() + .Apply(mapper) + .With(0, "stream") + .Build() + .BufferBytes() + .Value(ToString(SwitchMemoryLimit)) + .Build() + .FreeArgs() + .Add(switchArgs) + .Build() + .Build() + .Build() + .Done(); + + for (size_t i: xrange(JoinedMaps.size())) { + auto map = JoinedMaps[i]; + OutputSubsts.emplace(map.Raw(), std::make_pair(joinedMap, i)); + } + + res = true; + } + + ClearJoinGroup(); + + return res; +} + +bool TOutHorizontalJoinOptimizer::HandleGroup(TPositionHandle pos, const TGroupKey& key, const TVector<TYtMap>& maps, TExprContext& ctx) { + auto itemType = std::get<2>(key)->GetTypeAnn()->Cast<TListExprType>()->GetItemType()->Cast<TStructExprType>(); + + for (TYtMap map: maps) { + if (const auto nextMapCount = JoinedMaps.size() + 1; nextMapCount > MaxOutTables) { + if (MakeJoinedMap(pos, key, itemType, ctx)) { + YQL_CLOG(INFO, ProviderYt) << "OutHorizontalJoin: split by max output tables: " << nextMapCount; + } + } + + TExprNode::TPtr updatedLambda = map.Mapper().Ptr(); + if (MaxJobMemoryLimit) { + if (UpdateTableContentMemoryUsage(map.Mapper().Ptr(), updatedLambda, State_, ctx, false).Level != IGraphTransformer::TStatus::Ok) { + return false; + } + } + + ScanResourceUsage(*updatedLambda, *State_->Configuration, State_->Types, &MemUsage, nullptr, &UsedFiles); + auto currMemory = Accumulate(MemUsage.begin(), MemUsage.end(), SwitchMemoryLimit, + [](ui64 sum, const std::pair<const TStringBuf, ui64>& val) { return sum + val.second; }); + + // Take into account codec input/output buffers (one for all inputs and one per output) + currMemory += YQL_JOB_CODEC_MEM * (JoinedMaps.size() + 2); + + if ((MaxJobMemoryLimit && currMemory > *MaxJobMemoryLimit) || UsedFiles > MaxOperationFiles) { + const auto usedFiles = UsedFiles; // Save value, because MakeJoinedMap will clear it + if (MakeJoinedMap(pos, key, itemType, ctx)) { + YQL_CLOG(INFO, ProviderYt) << "OutHorizontalJoin: split by limits. Memory: " << currMemory << ", files: " << usedFiles; + } + ScanResourceUsage(*updatedLambda, *State_->Configuration, State_->Types, &MemUsage, nullptr, &UsedFiles); + } + + AddToJoinGroup(map); + + for (auto itemType: map.Input().Item(0).Ref().GetTypeAnn()->Cast<TListExprType>()->GetItemType()->Cast<TStructExprType>()->GetItems()) { + UsedFields.insert(itemType->GetName()); + } + + auto sysColumns = NYql::GetSettingAsColumnList(map.Input().Item(0).Settings().Ref(), EYtSettingType::SysColumns); + UsedSysFields.insert(sysColumns.begin(), sysColumns.end()); + + if (auto columns = map.Input().Item(0).Paths().Item(0).Columns().Maybe<TExprList>()) { + for (auto child: columns.Cast()) { + if (auto maybeTuple = child.Maybe<TCoAtomList>()) { + auto tuple = maybeTuple.Cast(); + if (tuple.Item(1).Value() == "weak") { + WeakFields.insert(tuple.Item(0).Value()); + } + } + } + } + } + + MakeJoinedMap(pos, key, itemType, ctx); + return true; +}; + +bool TOutHorizontalJoinOptimizer::IsGoodForOutHorizontalJoin(const TExprNode* op) { + auto it = ProcessedOps.find(op); + if (it != ProcessedOps.end()) { + return it->second; + } + + bool res = false; + if (auto maybeMap = TMaybeNode<TYtMap>(op)) { + auto map = maybeMap.Cast(); + + res = IsGoodForHorizontalJoin(map) + && map.Input().Item(0).Paths().Size() == 1 + && !NYql::HasAnySetting(map.Input().Item(0).Settings().Ref(), EYtSettingType::Take | EYtSettingType::Skip) + && !HasNonEmptyKeyFilter(map.Input().Item(0)); + } + ProcessedOps.emplace(op, res); + return res; +}; + +IGraphTransformer::TStatus TOutHorizontalJoinOptimizer::Optimize(TExprNode::TPtr input, TExprNode::TPtr& output, TExprContext& ctx) { + + THashMap<TGroupKey, TVector<TYtMap>> opGroups; + THashMap<TGroupKey, TVector<TYtMap>> tableGroups; + + for (auto writer: OpDepsOrder) { + if (IsGoodForOutHorizontalJoin(writer)) { + auto map = TYtMap(writer); + auto section = map.Input().Item(0); + auto path = section.Paths().Item(0); + + // Only input table. Temp outputs are processed above + if (auto table = path.Table().Maybe<TYtTable>()) { + ui32 flags = 0; + if (NYql::HasSetting(table.Settings().Ref(), EYtSettingType::WithQB)) { + flags |= EFeatureFlags::QB2; + } + if (NYql::HasSetting(map.Settings().Ref(), EYtSettingType::WeakFields)) { + flags |= EFeatureFlags::WeakField; + } + if (NYql::HasSetting(map.Settings().Ref(), EYtSettingType::Flow)) { + flags |= EFeatureFlags::Flow; + } + + auto key = TGroupKey{ + TString{map.DataSink().Cluster().Value()}, + map.World().Raw(), + table.Cast().Raw(), + path.Ranges().Raw(), + NYql::GetSetting(section.Settings().Ref(), EYtSettingType::Sample).Get(), + flags + }; + tableGroups[key].push_back(map); + } + } + + opGroups.clear(); + for (auto& reader: OpDeps.at(writer)) { + if (IsGoodForOutHorizontalJoin(std::get<0>(reader))) { + auto map = TYtMap(std::get<0>(reader)); + auto section = map.Input().Item(0); + + ui32 flags = 0; + if (NYql::HasSetting(map.Settings().Ref(), EYtSettingType::WeakFields)) { + flags |= EFeatureFlags::WeakField; + } + if (NYql::HasSetting(map.Settings().Ref(), EYtSettingType::Flow)) { + flags |= EFeatureFlags::Flow; + } + + auto key = TGroupKey{ + TString{map.DataSink().Cluster().Value()}, + map.World().Raw(), + std::get<2>(reader), + section.Paths().Item(0).Ranges().Raw(), + NYql::GetSetting(section.Settings().Ref(), EYtSettingType::Sample).Get(), + flags + }; + opGroups[key].push_back(map); + } + } + if (opGroups.empty()) { + continue; + } + + for (auto& group: opGroups) { + if (group.second.size() < 2) { + continue; + } + + if (!HandleGroup(writer->Pos(), group.first, group.second, ctx)) { + return IGraphTransformer::TStatus::Error; + } + if (!OutputSubsts.empty()) { + break; + } + } + + if (!OutputSubsts.empty()) { + break; + } + } + + if (OutputSubsts.empty() && !tableGroups.empty()) { + for (auto& group: tableGroups) { + if (group.second.size() < 2) { + continue; + } + + if (!HandleGroup(std::get<2>(group.first)->Pos(), group.first, group.second, ctx)) { + return IGraphTransformer::TStatus::Error; + } + if (!OutputSubsts.empty()) { + break; + } + } + } + + if (!OutputSubsts.empty()) { + TNodeOnNodeOwnedMap toOptimize; + VisitExpr(input, [&](const TExprNode::TPtr& node) { + if (auto maybeOut = TMaybeNode<TYtOutput>(node)) { + auto out = maybeOut.Cast(); + auto it = OutputSubsts.find(out.Operation().Raw()); + if (it != OutputSubsts.end()) { + YQL_ENSURE(out.OutIndex().Value() == "0"); + toOptimize[node.Get()] = Build<TYtOutput>(ctx, out.Pos()) + .Operation(it->second.first) + .OutIndex() + .Value(ToString(it->second.second)) + .Build() + .Mode(out.Mode()) + .Done().Ptr(); + } + } + return true; + }); + + YQL_CLOG(INFO, ProviderYt) << "PhysicalFinalizing-OutHorizontalJoin"; + return RemapExpr(input, output, toOptimize, ctx, TOptimizeExprSettings(State_->Types)); + } + return IGraphTransformer::TStatus::Ok; +} + +} // NYql diff --git a/yt/yql/providers/yt/provider/yql_yt_horizontal_join.h b/yt/yql/providers/yt/provider/yql_yt_horizontal_join.h new file mode 100644 index 0000000000..35c86fd7ea --- /dev/null +++ b/yt/yql/providers/yt/provider/yql_yt_horizontal_join.h @@ -0,0 +1,176 @@ +#pragma once + +#include "yql_yt_provider.h" +#include "yql_yt_optimize.h" +#include "yql_yt_op_settings.h" + +#include <yt/yql/providers/yt/expr_nodes/yql_yt_expr_nodes.h> +#include <yt/yql/providers/yt/common/yql_configuration.h> +#include <yql/essentials/core/yql_graph_transformer.h> +#include <yql/essentials/core/yql_expr_optimize.h> +#include <yql/essentials/ast/yql_expr.h> + +#include <util/generic/vector.h> +#include <util/generic/string.h> +#include <util/generic/strbuf.h> +#include <util/generic/bitmap.h> +#include <util/generic/map.h> +#include <util/generic/set.h> +#include <util/generic/maybe.h> +#include <util/generic/hash.h> +#include <util/generic/size_literals.h> + +#include <utility> +#include <tuple> + +namespace NYql { + +class THorizontalJoinBase { +public: + THorizontalJoinBase(const TYtState::TPtr& state, const std::vector<const TExprNode*>& opDepsOrder, const TOpDeps& opDeps, const TNodeSet& hasWorldDeps) + : State_(state) + , OpDepsOrder(opDepsOrder) + , OpDeps(opDeps) + , HasWorldDeps(hasWorldDeps) + , MaxTables(State_->Configuration->MaxInputTables.Get().GetOrElse(DEFAULT_MAX_INPUT_TABLES)) + , MaxOutTables(State_->Configuration->MaxOutputTables.Get().GetOrElse(DEFAULT_MAX_OUTPUT_TABLES)) + , SwitchMemoryLimit(State_->Configuration->SwitchLimit.Get().GetOrElse(DEFAULT_SWITCH_MEMORY_LIMIT)) + , MaxJobMemoryLimit(State_->Configuration->MaxExtraJobMemoryToFuseOperations.Get()) + , MaxOperationFiles(State_->Configuration->MaxOperationFiles.Get().GetOrElse(DEFAULT_MAX_OPERATION_FILES)) + , UsedFiles(1) // jobstate + { + } + +protected: + enum EFeatureFlags: ui32 { + YAMR = 1 << 1, + Flow = 1 << 2, + WeakField = 1 << 3, + QB2 = 1 << 4, + }; + + bool IsGoodForHorizontalJoin(NNodes::TYtMap map) const; + NNodes::TCoLambda CleanupAuxColumns(NNodes::TCoLambda lambda, TExprContext& ctx) const; + void ClearJoinGroup(); + void AddToJoinGroup(NNodes::TYtMap map); + NNodes::TCoLambda BuildMapperWithAuxColumnsForSingleInput(TPositionHandle pos, bool ordered, TExprContext& ctx) const; + NNodes::TCoLambda BuildMapperWithAuxColumnsForMultiInput(TPositionHandle pos, bool ordered, TExprContext& ctx) const; + NNodes::TCoLambda MakeSwitchLambda(size_t mapIndex, size_t fieldsCount, bool singleInput, TExprContext& ctx) const; + +protected: + TYtState::TPtr State_; + const std::vector<const TExprNode*>& OpDepsOrder; + const TOpDeps& OpDeps; + const TNodeSet& HasWorldDeps; + + const size_t MaxTables; + const size_t MaxOutTables; + const ui64 SwitchMemoryLimit; + const TMaybe<ui64> MaxJobMemoryLimit; + const size_t MaxOperationFiles; + + NNodes::TMaybeNode<NNodes::TYtDSink> DataSink; + TDynBitMap UsesTablePath; + TDynBitMap UsesTableRecord; + TDynBitMap UsesTableIndex; + EYtSettingTypes OpSettings; + TMap<TStringBuf, ui64> MemUsage; + size_t UsedFiles; + TVector<NNodes::TYtMap> JoinedMaps; +}; + +class THorizontalJoinOptimizer: public THorizontalJoinBase { + // Cluster, Sampling, Dependency, Flags + using TGroupKey = std::tuple<TString, TMaybe<TSampleParams>, const TExprNode*, ui32>; + +public: + THorizontalJoinOptimizer(const TYtState::TPtr& state, const std::vector<const TExprNode*>& opDepsOrder, const TOpDeps& opDeps, const TNodeSet& hasWorldDeps, TProcessedNodesSet* processedNodes) + : THorizontalJoinBase(state, opDepsOrder, opDeps, hasWorldDeps) + , ProcessedNodes(processedNodes) + { + } + + IGraphTransformer::TStatus Optimize(TExprNode::TPtr input, TExprNode::TPtr& output, TExprContext& ctx); + +private: + TExprNode::TPtr HandleList(const TExprNode::TPtr& node, bool sectionList, TExprContext& ctx); + void AddToGroups(NNodes::TYtMap map, size_t s, size_t p, const TExprNode* section, + THashMap<TGroupKey, TVector<std::tuple<NNodes::TYtMap, size_t, size_t>>>& groups, + TNodeMap<TMaybe<TGroupKey>>& processedMaps) const; + THashMap<TGroupKey, TVector<std::tuple<NNodes::TYtMap, size_t, size_t>>> CollectGroups(const TExprNode::TPtr& node, bool sectionList) const; + void ClearJoinGroup(); + bool MakeJoinedMap(TPositionHandle pos, TExprContext& ctx); + TExprNode::TPtr RebuildList(const TExprNode::TPtr& node, bool sectionList, TExprContext& ctx); + +private: + TSyncMap Worlds; + // operation -> joined_map_out + TNodeMap<size_t> UniqMaps; + // {section, columns, ranges} -> TMap{out_num -> path_num} + TMap<std::tuple<size_t, const TExprNode*, const TExprNode*>, TMap<size_t, size_t>> GroupedOuts; + // out_num -> TVector{section_num, path_num}. Cannot be joined with other outputs + TMap<size_t, TVector<std::pair<size_t, size_t>>> ExclusiveOuts; + size_t InputCount = 0; + // {section or table, ranges, section settings, WeakField} + TSet<std::tuple<const TExprNode*, const TExprNode*, const TExprNode*, bool>> GroupedInputs; + + // {section_num, path_num} -> {joined_map, out_num} + TMap<std::pair<size_t, size_t>, TMaybe<std::pair<NNodes::TYtMap, size_t>>> InputSubsts; + + TProcessedNodesSet* ProcessedNodes; +}; + +class TMultiHorizontalJoinOptimizer: public THorizontalJoinBase { + // Cluster, readers, Sampling, Dependency, Flags + using TGroupKey = std::tuple<TString, std::set<ui64>, TMaybe<TSampleParams>, const TExprNode*, ui32>; + +public: + TMultiHorizontalJoinOptimizer(const TYtState::TPtr& state, const std::vector<const TExprNode*>& opDepsOrder, const TOpDeps& opDeps, const TNodeSet& hasWorldDeps) + : THorizontalJoinBase(state, opDepsOrder, opDeps, hasWorldDeps) + { + } + + IGraphTransformer::TStatus Optimize(TExprNode::TPtr input, TExprNode::TPtr& output, TExprContext& ctx); + +private: + bool HandleGroup(const TVector<NNodes::TYtMap>& maps, TExprContext& ctx); + void ClearJoinGroup(); + bool MakeJoinedMap(TExprContext& ctx); + +private: + TSyncMap Worlds; + size_t InputCount = 0; + // {section or table, ranges, section settings, WeakField} + TSet<std::tuple<const TExprNode*, const TExprNode*, const TExprNode*, bool>> GroupedInputs; + + TNodeMap<std::pair<NNodes::TYtMap, size_t>> OutputSubsts; // original map -> joined map, out index +}; + +class TOutHorizontalJoinOptimizer: public THorizontalJoinBase { + // Group by: Cluster, World, Input, Range, Sampling, Flags + using TGroupKey = std::tuple<TString, const TExprNode*, const TExprNode*, const TExprNode*, const TExprNode*, ui32>; + +public: + TOutHorizontalJoinOptimizer(const TYtState::TPtr& state, const std::vector<const TExprNode*>& opDepsOrder, const TOpDeps& opDeps, const TNodeSet& hasWorldDeps) + : THorizontalJoinBase(state, opDepsOrder, opDeps, hasWorldDeps) + { + } + + IGraphTransformer::TStatus Optimize(TExprNode::TPtr input, TExprNode::TPtr& output, TExprContext& ctx); + +private: + void ClearJoinGroup(); + bool MakeJoinedMap(TPositionHandle pos, const TGroupKey& key, const TStructExprType* itemType, TExprContext& ctx); + bool HandleGroup(TPositionHandle pos, const TGroupKey& key, const TVector<NNodes::TYtMap>& maps, TExprContext& ctx); + bool IsGoodForOutHorizontalJoin(const TExprNode* op); + +private: + TSet<TStringBuf> UsedFields; + TSet<TStringBuf> WeakFields; + TSet<TString> UsedSysFields; + + TNodeMap<std::pair<NNodes::TYtMap, size_t>> OutputSubsts; // original map -> joined map, out index + TNodeMap<bool> ProcessedOps; +}; + +} // NYql diff --git a/yt/yql/providers/yt/provider/yql_yt_intent_determination.cpp b/yt/yql/providers/yt/provider/yql_yt_intent_determination.cpp new file mode 100644 index 0000000000..8b23660b7e --- /dev/null +++ b/yt/yql/providers/yt/provider/yql_yt_intent_determination.cpp @@ -0,0 +1,485 @@ +#include "yql_yt_provider_impl.h" +#include "yql_yt_table.h" +#include "yql_yt_table_desc.h" +#include "yql_yt_op_settings.h" +#include "yql_yt_helpers.h" + +#include <yt/yql/providers/yt/expr_nodes/yql_yt_expr_nodes.h> +#include <yql/essentials/providers/common/provider/yql_provider.h> +#include <yql/essentials/providers/common/transform/yql_visit.h> +#include <yql/essentials/core/yql_expr_type_annotation.h> +#include <yql/essentials/utils/log/log.h> + +#include <util/string/cast.h> + +namespace NYql { + +using namespace NNodes; + +const TString YtProvider_AnonTableName = "YtProvider_AnonTableName"; + +class TYtIntentDeterminationTransformer : public TVisitorTransformerBase { +public: + TYtIntentDeterminationTransformer(TYtState::TPtr state) + : TVisitorTransformerBase(false) + , State_(state) + { + // Handle initial callables after SQL parse + AddHandler({TYtRead::CallableName()}, Hndl(&TYtIntentDeterminationTransformer::HandleRead)); + AddHandler({TYtWrite::CallableName()}, Hndl(&TYtIntentDeterminationTransformer::HandleWrite)); + + // Handle callables for already parsed/optimized AST + AddHandler({TYtReadTable::CallableName()}, Hndl(&TYtIntentDeterminationTransformer::HandleReadTable)); + AddHandler({TYtReadTableScheme::CallableName()}, Hndl(&TYtIntentDeterminationTransformer::HandleReadTableScheme)); + AddHandler({TYtDropTable::CallableName()}, Hndl(&TYtIntentDeterminationTransformer::HandleDropTable)); + AddHandler({TYtPublish::CallableName()}, Hndl(&TYtIntentDeterminationTransformer::HandlePublish)); + AddHandler({TYtSort::CallableName()}, Hndl(&TYtIntentDeterminationTransformer::HandleOperation)); + AddHandler({TYtMap::CallableName()}, Hndl(&TYtIntentDeterminationTransformer::HandleOperation)); + AddHandler({TYtReduce::CallableName()}, Hndl(&TYtIntentDeterminationTransformer::HandleOperation)); + AddHandler({TYtMapReduce::CallableName()}, Hndl(&TYtIntentDeterminationTransformer::HandleOperation)); + AddHandler({TYtCopy::CallableName()}, Hndl(&TYtIntentDeterminationTransformer::HandleOperation)); + AddHandler({TYtMerge::CallableName()}, Hndl(&TYtIntentDeterminationTransformer::HandleOperation)); + AddHandler({TYtEquiJoin::CallableName()}, Hndl(&TYtIntentDeterminationTransformer::HandleOperation)); + AddHandler({TYtFill::CallableName()}, Hndl(&TYtIntentDeterminationTransformer::HandleOutOperation)); + AddHandler({TYtTouch::CallableName()}, Hndl(&TYtIntentDeterminationTransformer::HandleOutOperation)); + AddHandler({TYtWriteTable::CallableName()}, Hndl(&TYtIntentDeterminationTransformer::HandleWriteTable)); + } + + TStatus HandleRead(TExprBase input, TExprContext& ctx) { + TYtRead read = input.Cast<TYtRead>(); + if (!EnsureArgsCount(read.Ref(), 5, ctx)) { + return TStatus::Error; + } + + auto cluster = TString{read.DataSource().Cluster().Value()}; + + EYtSettingTypes acceptedSettings = EYtSettingType::View | EYtSettingType::Anonymous + | EYtSettingType::InferScheme | EYtSettingType::ForceInferScheme + | EYtSettingType::DoNotFailOnInvalidSchema | EYtSettingType::XLock + | EYtSettingType::UserSchema | EYtSettingType::UserColumns | EYtSettingType::IgnoreTypeV3; + for (auto path: read.Arg(2).Cast<TExprList>()) { + if (auto table = path.Maybe<TYtPath>().Table()) { + if (!TYtTableInfo::Validate(table.Cast().Ref(), acceptedSettings, ctx)) { + return TStatus::Error; + } + + TYtTableInfo tableInfo(table.Cast(), false); + if (!ProcessInputTableIntent(ctx.GetPosition(input.Pos()), cluster, tableInfo, ctx)) { + return TStatus::Error; + } + } + } + return TStatus::Ok; + } + + TStatus HandleWrite(TExprBase input, TExprContext& ctx) { + TYtWrite write = input.Cast<TYtWrite>(); + if (!EnsureArgsCount(write.Ref(), 5, ctx)) { + return TStatus::Error; + } + + if (!TYtTableInfo::Validate(write.Arg(2).Ref(), EYtSettingType::Mode | EYtSettingType::Initial | EYtSettingType::Anonymous, ctx)) { + return TStatus::Error; + } + + auto cluster = TString{write.DataSink().Cluster().Value()}; + TYtTableInfo tableInfo(write.Arg(2), false); + + TYtTableDescription& tableDesc = State_->TablesData->GetOrAddTable( + cluster, + tableInfo.Name, + tableInfo.Epoch + ); + + if (NYql::HasSetting(tableInfo.Settings.Cast().Ref(), EYtSettingType::Anonymous)) { + tableDesc.IsAnonymous = true; + RegisterAnonymouseTable(cluster, tableInfo.Name); + } + + if (auto mode = NYql::GetSetting(write.Arg(4).Ref(), EYtSettingType::Mode)) { + try { + switch (FromString<EYtWriteMode>(mode->Child(1)->Content())) { + case EYtWriteMode::Drop: + tableDesc.Intents |= TYtTableIntent::Drop; + break; + case EYtWriteMode::Append: + tableDesc.Intents |= TYtTableIntent::Append; + break; + case EYtWriteMode::Renew: + case EYtWriteMode::RenewKeepMeta: + tableDesc.Intents |= TYtTableIntent::Override; + break; + case EYtWriteMode::Flush: + tableDesc.Intents |= TYtTableIntent::Flush; + break; + default: + ctx.AddError(TIssue(ctx.GetPosition(mode->Child(1)->Pos()), TStringBuilder() << "Unsupported " + << TYtWrite::CallableName() << " mode: " << mode->Child(1)->Content())); + return TStatus::Error; + + } + } catch (const yexception& e) { + ctx.AddError(TIssue(ctx.GetPosition(mode->Child(1)->Pos()), TStringBuilder() << "Unsupported " + << TYtWrite::CallableName() << " mode: " << mode->Child(1)->Content() << ", " << e.what())); + return TStatus::Error; + } + } else { + tableDesc.Intents |= TYtTableIntent::Override; + } + + if (!ValidateOutputTableIntent(ctx.GetPosition(input.Pos()), tableDesc.Intents, cluster, tableInfo.Name, ctx)) { + return TStatus::Error; + } + + return TStatus::Ok; + } + + TStatus HandleReadTable(const TExprNode::TPtr& input, TExprNode::TPtr& output, TExprContext& ctx) { + TYtReadTable read = TYtReadTable(input); + + auto cluster = TString{read.DataSource().Cluster().Value()}; + + for (auto section: read.Input()) { + for (auto path: section.Paths()) { + if (auto table = path.Table().Maybe<TYtTable>()) { + TYtTableInfo tableInfo(table.Cast(), false); + if (!ProcessInputTableIntent(ctx.GetPosition(input->Pos()), cluster, tableInfo, ctx)) { + return TStatus::Error; + } + } + } + } + + output = ResetTablesMeta(input, ctx, State_->Types->UseTableMetaFromGraph, State_->Types->EvaluationInProgress > 0); + if (!output) { + return TStatus::Error; + } + return TStatus::Ok; + } + + TStatus HandleReadTableScheme(const TExprNode::TPtr& input, TExprNode::TPtr& output, TExprContext& ctx) { + TYtReadTableScheme scheme = TYtReadTableScheme(input); + + auto cluster = TString{scheme.DataSource().Cluster().Value()}; + + TYtTableInfo tableInfo(scheme.Table(), false); + if (!ProcessInputTableIntent(ctx.GetPosition(input->Pos()), cluster, tableInfo, ctx)) { + return TStatus::Error; + } + + output = ResetTablesMeta(input, ctx, State_->Types->UseTableMetaFromGraph, State_->Types->EvaluationInProgress > 0); + if (!output) { + return TStatus::Error; + } + + return TStatus::Ok; + } + + TStatus HandleOperation(const TExprNode::TPtr& input, TExprNode::TPtr& output, TExprContext& ctx) { + auto op = TYtTransientOpBase(input); + auto cluster = TString{op.DataSink().Cluster().Value()}; + + for (auto section: op.Input()) { + for (auto path: section.Paths()) { + if (auto table = path.Table().Maybe<TYtTable>()) { + TYtTableInfo tableInfo(table.Cast(), false); + if (!ProcessInputTableIntent(ctx.GetPosition(input->Pos()), cluster, tableInfo, ctx)) { + return TStatus::Error; + } + } + } + } + + output = ResetTablesMeta(input, ctx, State_->Types->UseTableMetaFromGraph, State_->Types->EvaluationInProgress > 0); + if (!output) { + return TStatus::Error; + } + return TStatus::Ok; + } + + TStatus HandleOutOperation(const TExprNode::TPtr& input, TExprNode::TPtr& output, TExprContext& ctx) { + output = ResetTablesMeta(input, ctx, State_->Types->UseTableMetaFromGraph, State_->Types->EvaluationInProgress > 0); + if (!output) { + return TStatus::Error; + } + return TStatus::Ok; + } + + TStatus HandleDropTable(const TExprNode::TPtr& input, TExprNode::TPtr& output, TExprContext& ctx) { + auto drop = TYtDropTable(input); + + auto cluster = TString{drop.DataSink().Cluster().Value()}; + TYtTableInfo tableInfo(drop.Table(), false); + + TYtTableDescription& tableDesc = State_->TablesData->GetOrAddTable( + cluster, + tableInfo.Name, + tableInfo.Epoch + ); + if (NYql::HasSetting(tableInfo.Settings.Cast().Ref(), EYtSettingType::Anonymous)) { + tableDesc.IsAnonymous = true; + RegisterAnonymouseTable(cluster, tableInfo.Name); + } + tableDesc.Intents |= TYtTableIntent::Drop; + + UpdateDescriptorMeta(tableDesc, tableInfo); + + output = ResetTablesMeta(input, ctx, State_->Types->UseTableMetaFromGraph, State_->Types->EvaluationInProgress > 0); + if (!output) { + return TStatus::Error; + } + return TStatus::Ok; + } + + TStatus HandlePublish(const TExprNode::TPtr& input, TExprNode::TPtr& output, TExprContext& ctx) { + auto publish = TYtPublish(input); + + auto cluster = TString{publish.DataSink().Cluster().Value()}; + TYtTableInfo tableInfo(publish.Publish(), false); + + TYtTableDescription& tableDesc = State_->TablesData->GetOrAddTable( + cluster, + tableInfo.Name, + tableInfo.Epoch + ); + if (NYql::HasSetting(tableInfo.Settings.Cast().Ref(), EYtSettingType::Anonymous)) { + tableDesc.IsAnonymous = true; + RegisterAnonymouseTable(cluster, tableInfo.Name); + } + if (auto mode = NYql::GetSetting(publish.Settings().Ref(), EYtSettingType::Mode)) { + try { + switch (FromString<EYtWriteMode>(mode->Child(1)->Content())) { + case EYtWriteMode::Append: + tableDesc.Intents |= TYtTableIntent::Append; + break; + case EYtWriteMode::Renew: + case EYtWriteMode::RenewKeepMeta: + tableDesc.Intents |= TYtTableIntent::Override; + break; + case EYtWriteMode::Flush: + tableDesc.Intents |= TYtTableIntent::Flush; + break; + default: + ctx.AddError(TIssue(ctx.GetPosition(mode->Child(1)->Pos()), TStringBuilder() << "Unsupported " + << TYtPublish::CallableName() << " mode: " << mode->Child(1)->Content())); + return TStatus::Error; + + } + } catch (const yexception& e) { + ctx.AddError(TIssue(ctx.GetPosition(mode->Child(1)->Pos()), TStringBuilder() << "Unsupported " + << TYtPublish::CallableName() << " mode: " << mode->Child(1)->Content() << ", " << e.what())); + return TStatus::Error; + } + } else { + tableDesc.Intents |= TYtTableIntent::Override; + } + + if (!ValidateOutputTableIntent(ctx.GetPosition(input->Pos()), tableDesc.Intents, cluster, tableInfo.Name, ctx)) { + return TStatus::Error; + } + + UpdateDescriptorMeta(tableDesc, tableInfo); + + output = ResetTablesMeta(input, ctx, State_->Types->UseTableMetaFromGraph, State_->Types->EvaluationInProgress > 0); + if (!output) { + return TStatus::Error; + } + return TStatus::Ok; + } + + TStatus HandleWriteTable(TExprBase input, TExprContext& ctx) { + TYtWriteTable write = input.Cast<TYtWriteTable>(); + + auto cluster = TString{write.DataSink().Cluster().Value()}; + TYtTableInfo tableInfo(write.Table(), false); + + TYtTableDescription& tableDesc = State_->TablesData->GetOrAddTable( + cluster, + tableInfo.Name, + tableInfo.Epoch + ); + + if (NYql::HasSetting(tableInfo.Settings.Cast().Ref(), EYtSettingType::Anonymous)) { + tableDesc.IsAnonymous = true; + RegisterAnonymouseTable(cluster, tableInfo.Name); + } + + if (auto mode = NYql::GetSetting(write.Settings().Ref(), EYtSettingType::Mode)) { + try { + switch (FromString<EYtWriteMode>(mode->Child(1)->Content())) { + case EYtWriteMode::Drop: + tableDesc.Intents |= TYtTableIntent::Drop; + break; + case EYtWriteMode::Append: + tableDesc.Intents |= TYtTableIntent::Append; + break; + case EYtWriteMode::Renew: + case EYtWriteMode::RenewKeepMeta: + tableDesc.Intents |= TYtTableIntent::Override; + break; + case EYtWriteMode::Flush: + tableDesc.Intents |= TYtTableIntent::Flush; + break; + default: + ctx.AddError(TIssue(ctx.GetPosition(mode->Child(1)->Pos()), TStringBuilder() << "Unsupported " + << TYtWrite::CallableName() << " mode: " << mode->Child(1)->Content())); + return TStatus::Error; + + } + } catch (const yexception& e) { + ctx.AddError(TIssue(ctx.GetPosition(mode->Child(1)->Pos()), TStringBuilder() << "Unsupported " + << TYtWrite::CallableName() << " mode: " << mode->Child(1)->Content() << ", " << e.what())); + return TStatus::Error; + } + } else { + tableDesc.Intents |= TYtTableIntent::Override; + } + + if (!ValidateOutputTableIntent(ctx.GetPosition(input.Pos()), tableDesc.Intents, cluster, tableInfo.Name, ctx)) { + return TStatus::Error; + } + + UpdateDescriptorMeta(tableDesc, tableInfo); + + return TStatus::Ok; + } + +private: + void UpdateDescriptorMeta(TYtTableDescription& tableDesc, const TYtTableInfo& tableInfo) const { + if (!State_->Types->UseTableMetaFromGraph) { + return; + } + + if (tableInfo.Stat && !tableDesc.Stat) { + if (tableDesc.Stat = tableInfo.Stat) { + tableDesc.Stat->FromNode = {}; + } + } + + if (tableInfo.Meta && !tableDesc.Meta) { + tableDesc.Meta = tableInfo.Meta; + tableDesc.Meta->FromNode = {}; + if (NYql::HasSetting(tableInfo.Settings.Ref(), EYtSettingType::WithQB)) { + tableDesc.QB2RowSpec = tableInfo.RowSpec; + tableDesc.QB2RowSpec->FromNode = {}; + } else { + if (tableDesc.RowSpec = tableInfo.RowSpec) { + tableDesc.RowSpec->FromNode = {}; + } + } + } + } + + bool ProcessInputTableIntent(TPosition pos, const TString& cluster, const TYtTableInfo& tableInfo, TExprContext& ctx) { + if (!State_->Checkpoints.empty() && State_->Checkpoints.contains(std::make_pair(cluster, tableInfo.Name))) { + ctx.AddError(TIssue(pos, TStringBuilder() << "Reading from checkpoint " << tableInfo.Name.Quote() << " is not allowed")); + return false; + } + + TYtTableDescription& tableDesc = State_->TablesData->GetOrAddTable( + cluster, + tableInfo.Name, + tableInfo.Epoch + ); + + if (NYql::HasSetting(tableInfo.Settings.Cast().Ref(), EYtSettingType::Anonymous)) { + tableDesc.IsAnonymous = true; + RegisterAnonymouseTable(cluster, tableInfo.Name); + } + + TYtTableIntents intents = TYtTableIntent::Read; + if (tableInfo.Settings) { + auto view = NYql::GetSetting(tableInfo.Settings.Cast().Ref(), EYtSettingType::View); + if (view) { + if (tableInfo.Epoch.GetOrElse(0)) { + ctx.AddError(TIssue(pos, TStringBuilder() + << "Table " << tableInfo.Name.Quote() << " cannot have any view after replacing its content")); + return false; + } + tableDesc.Views.insert({TString{view->Child(1)->Content()}, {}}); + intents = TYtTableIntent::View; // Override Read intent + } + if (NYql::HasSetting(tableInfo.Settings.Cast().Ref(), EYtSettingType::XLock)) { + intents |= TYtTableIntent::Override; + } + } + tableDesc.Intents |= intents; + if (tableInfo.Epoch.GetOrElse(0) == 0) { + if (!NYql::HasSetting(tableInfo.Settings.Cast().Ref(), EYtSettingType::UserSchema)) { + TExprNode::TPtr perForceInfer = NYql::GetSetting(tableInfo.Settings.Cast().Ref(), + EYtSettingType::ForceInferScheme); + TExprNode::TPtr perInfer = NYql::GetSetting(tableInfo.Settings.Cast().Ref(), + EYtSettingType::InferScheme); + tableDesc.InferSchemaRows = State_->Configuration->InferSchema.Get().OrElse(State_->Configuration->ForceInferSchema.Get()).GetOrElse(0); + if (perForceInfer) { + tableDesc.InferSchemaRows = (perForceInfer->ChildrenSize() == 2) + ? FromString<ui32>(perForceInfer->Tail().Content()) + : 1; + } else if (perInfer) { + tableDesc.InferSchemaRows = (perInfer->ChildrenSize() == 2) + ? FromString<ui32>(perInfer->Tail().Content()) + : 1; + } + tableDesc.ForceInferSchema = perForceInfer + || State_->Configuration->ForceInferSchema.Get().GetOrElse(0) > 0; + } + tableDesc.IgnoreTypeV3 = State_->Configuration->IgnoreTypeV3.Get().GetOrElse(false) + || NYql::HasSetting(tableInfo.Settings.Cast().Ref(), EYtSettingType::IgnoreTypeV3); + } + tableDesc.FailOnInvalidSchema = !NYql::HasSetting(tableInfo.Settings.Cast().Ref(), EYtSettingType::DoNotFailOnInvalidSchema); + + UpdateDescriptorMeta(tableDesc, tableInfo); + + return true; + } + + bool ValidateOutputTableIntent( + TPosition pos, + TYtTableIntents intents, + const TString& cluster, + const TString& tableName, + TExprContext& ctx) + { + if ( + !intents.HasFlags(TYtTableIntent::Flush) && + !State_->Checkpoints.empty() && + State_->Checkpoints.contains(std::make_pair(cluster, tableName))) + { + ctx.AddError(TIssue(pos, TStringBuilder() << "Writing to checkpoint " << tableName.Quote() << " is not allowed")); + return false; + } + + return true; + } + + void RegisterAnonymouseTable(const TString& cluster, const TString& label) { + auto& path = State_->AnonymousLabels[std::make_pair(cluster, label)]; + if (path.empty()) { + auto& qContext = State_->Types->QContext; + const TString key = cluster + "." + label; + if (qContext.CanRead()) { + auto res = qContext.GetReader()->Get({YtProvider_AnonTableName, key}).GetValueSync(); + if (!res) { + ythrow yexception() << "Missing replay data"; + } + + path = res->Value; + } else { + path = "tmp/" + GetGuidAsString(State_->Types->RandomProvider->GenGuid()); + if (qContext.CanWrite()) { + qContext.GetWriter()->Put({YtProvider_AnonTableName, key}, path).GetValueSync(); + } + } + + YQL_CLOG(INFO, ProviderYt) << "Anonymous label " << cluster << '.' << label << ": " << path; + } + } +private: + TYtState::TPtr State_; +}; + +THolder<IGraphTransformer> CreateYtIntentDeterminationTransformer(TYtState::TPtr state) { + return THolder(new TYtIntentDeterminationTransformer(state)); +} + +} diff --git a/yt/yql/providers/yt/provider/yql_yt_io_discovery.cpp b/yt/yql/providers/yt/provider/yql_yt_io_discovery.cpp new file mode 100644 index 0000000000..752c360e51 --- /dev/null +++ b/yt/yql/providers/yt/provider/yql_yt_io_discovery.cpp @@ -0,0 +1,908 @@ +#include "yql_yt_provider_impl.h" +#include "yql_yt_key.h" +#include "yql_yt_gateway.h" +#include "yql_yt_op_settings.h" +#include "yql_yt_helpers.h" +#include "yql_yt_io_discovery_walk_folders.h" + +#include <yt/yql/providers/yt/expr_nodes/yql_yt_expr_nodes.h> +#include <yt/yql/providers/yt/common/yql_names.h> +#include <yql/essentials/providers/common/provider/yql_provider_names.h> +#include <yql/essentials/providers/common/provider/yql_provider.h> +#include <yql/essentials/core/services/yql_eval_expr.h> +#include <yql/essentials/core/yql_expr_optimize.h> +#include <yql/essentials/core/yql_expr_type_annotation.h> +#include <yql/essentials/core/yql_expr_constraint.h> +#include <yql/essentials/core/type_ann/type_ann_core.h> +#include <yql/essentials/core/type_ann/type_ann_expr.h> +#include <yql/essentials/core/peephole_opt/yql_opt_peephole_physical.h> +#include <yql/essentials/core/issue/protos/issue_id.pb.h> +#include <yql/essentials/core/issue/yql_issue.h> +#include <yql/essentials/utils/log/log.h> + +#include <library/cpp/yson/node/node_io.h> +#include <library/cpp/threading/future/future.h> + +#include <util/generic/vector.h> +#include <util/generic/string.h> +#include <util/generic/hash.h> +#include <util/string/cast.h> +#include <util/string/strip.h> + + +namespace NYql { + +using namespace NNodes; + +class TYtIODiscoveryTransformer : public TGraphTransformerBase { +public: + TYtIODiscoveryTransformer(TYtState::TPtr state) + : State_(state) + { + } + + TStatus DoTransform(TExprNode::TPtr input, TExprNode::TPtr& output, TExprContext& ctx) final { + output = input; + if (ctx.Step.IsDone(TExprStep::DiscoveryIO)) { + return TStatus::Ok; + } + + TVector<IYtGateway::TCanonizeReq> paths; + const bool discoveryMode = State_->Types->DiscoveryMode; + const bool evaluationInProgress = State_->Types->EvaluationInProgress; + TOptimizeExprSettings settings(nullptr); + settings.VisitChanges = true; + auto status = OptimizeExpr(input, output, [&](const TExprNode::TPtr& node, TExprContext& ctx) -> TExprNode::TPtr { + if (auto maybeRead = TMaybeNode<TYtRead>(node)) { + if (!maybeRead.DataSource()) { // Validates provider + return node; + } + auto read = maybeRead.Cast(); + auto ds = read.DataSource(); + if (!EnsureArgsCount(read.Ref(), 5, ctx)) { + return {}; + } + + if (discoveryMode && evaluationInProgress) { + ctx.AddError(YqlIssue(ctx.GetPosition(read.Pos()), TIssuesIds::YQL_NOT_ALLOWED_IN_DISCOVERY, + TStringBuilder() << node->Content() << " is not allowed in Discovery mode")); + return {}; + } + + TYtInputKeys keys; + if (!keys.Parse(read.Arg(2).Ref(), ctx)) { + return {}; + } + + if (keys.IsProcessed()) { + // Already processed + return node; + } + + if (keys.GetKeys().empty()) { + auto userSchema = GetSetting(*read.Ref().Child(4), EYtSettingType::UserSchema); + if (userSchema) { + return BuildEmptyTablesRead(read.Pos(), *userSchema, ctx); + } + + ctx.AddError(TIssue(ctx.GetPosition(read.Arg(2).Pos()), "The list of tables is empty")); + return {}; + } + + if (keys.GetType() == TYtKey::EType::TableScheme) { + return ConvertTableScheme(read, keys.GetKeys().front(), ctx); + } + + if (discoveryMode) { + for (auto& key: keys.GetKeys()) { + auto keyPos = ctx.GetPosition(key.GetNode()->Pos()); + if (key.GetRange()) { + ctx.AddError(YqlIssue(ctx.GetPosition(read.Arg(2).Pos()), TIssuesIds::YQL_NOT_ALLOWED_IN_DISCOVERY, + TStringBuilder() << MrTableRangeName << '/' << MrTableRangeStrictName << " is not allowed in Discovery mode")); + return {}; + } + else if (key.GetFolder()) { + ctx.AddError(YqlIssue(ctx.GetPosition(read.Arg(2).Pos()), TIssuesIds::YQL_NOT_ALLOWED_IN_DISCOVERY, + TStringBuilder() << MrFolderName << " is not allowed in Discovery mode")); + return {}; + } + } + } + if (AllOf(keys.GetKeys(), [] (const TYtKey& key) { return key.IsAnonymous(); })) { + return ConvertTableRead(read, keys, ctx); + } + return node; + } + else if (auto maybeWrite = TMaybeNode<TYtWrite>(node)) { + if (!maybeWrite.DataSink()) { // Validates provider + return node; + } + auto write = maybeWrite.Cast(); + auto ds = write.DataSink(); + if (!EnsureArgsCount(write.Ref(), 5, ctx)) { + return {}; + } + + if (!EnsureTuple(write.Arg(4).MutableRef(), ctx)) { + return {}; + } + + TYtOutputKey key; + if (!key.Parse(write.Arg(2).Ref(), ctx)) { + return {}; + } + if (key.GetType() == TYtKey::EType::Undefined) { + // Already processed + return node; + } + + auto mode = NYql::GetSetting(*node->ChildPtr(4), EYtSettingType::Mode); + const bool flush = mode && FromString<EYtWriteMode>(mode->Child(1)->Content()) == EYtWriteMode::Flush; + + TYtTableInfo tableInfo(key, ds.Cluster().Value()); + if (key.IsAnonymous()) { + if (flush) { + ctx.AddError(TIssue( + ctx.GetPosition(write.Pos()), + TStringBuilder() << "Using anonymous tables as checkpoints is not allowed")); + return {}; + } + tableInfo.Settings = Build<TCoNameValueTupleList>(ctx, write.Pos()) + .Add() + .Name().Value(ToString(EYtSettingType::Anonymous)).Build() + .Build() + .Done(); + } else if (tableInfo.Name.StartsWith(NYT::TConfig::Get()->Prefix)) { + tableInfo.Name = tableInfo.Name.substr(NYT::TConfig::Get()->Prefix.size()); + } + + if (tableInfo.Name.empty()) { + ctx.AddError(TIssue(ctx.GetPosition(write.Pos()), "Table name must not be empty")); + return {}; + } + + if (flush) { + auto setKey = std::make_pair(ds.Cluster().Value(), tableInfo.Name); + if (State_->Checkpoints.contains(setKey)) { + ctx.AddError(TIssue( + ctx.GetPosition(write.Pos()), + TStringBuilder() << "Table " << tableInfo.Name.Quote() << " already used as checkpoint")); + return {}; + } + State_->Checkpoints.emplace(std::move(setKey)); + } + node->ChildRef(2) = tableInfo.ToExprNode(ctx, write.Pos()).Ptr(); + return node; + } + + return node; + }, ctx, settings); + + if (status.Level != TStatus::Ok) { + return status; + } + + status = VisitInputKeys(output, ctx, [this, &ctx, &paths] (TYtRead readNode, TYtInputKeys&& keys) -> TExprNode::TPtr { + if (keys.GetType() != TYtKey::EType::TableScheme) { + const auto cluster = TString{readNode.DataSource().Cluster().Value()}; + for (auto&& key: keys.ExtractKeys()) { + auto keyPos = ctx.GetPosition(key.GetNode()->Pos()); + if (key.GetRange()) { + PendingRanges_.emplace(std::make_pair(cluster, *key.GetRange()), std::make_pair(keyPos, NThreading::TFuture<IYtGateway::TTableRangeResult>())); + } + else if (key.GetFolder()) { + PendingFolders_.emplace(std::make_pair(cluster, *key.GetFolder()), std::make_pair(keyPos, NThreading::TFuture<IYtGateway::TFolderResult>())); + } + else if (key.GetWalkFolderArgs()) { + return ctx.ChangeChild(readNode.Ref(), 2, InitializeWalkFolders(std::move(key), cluster, keyPos, ctx)); + } + else if (key.GetWalkFolderImplArgs()) { + PendingWalkFoldersKeys_.insert(key.GetWalkFolderImplArgs()->StateKey); + } + else if (!key.IsAnonymous()) { + if (PendingCanonizations_.insert(std::make_pair(std::make_pair(cluster, key.GetPath()), paths.size())).second) { + paths.push_back(IYtGateway::TCanonizeReq() + .Cluster(cluster) + .Path(key.GetPath()) + .Pos(keyPos) + ); + } + } + } + } + return readNode.Ptr(); + }, /* visitChanges */ true); + + if (status.Level == TStatus::Error) { + PendingCanonizations_.clear(); + PendingFolders_.clear(); + PendingRanges_.clear(); + + for (const auto& key : PendingWalkFoldersKeys_) { + State_->WalkFoldersState.erase(key); + } + PendingWalkFoldersKeys_.clear(); + + YQL_CLOG(INFO, ProviderYt) << "YtIODiscovery - finish, status: " << (TStatus::ELevel)status.Level; + return status; + } + + if (PendingRanges_.empty() && PendingFolders_.empty() + && PendingCanonizations_.empty() && PendingWalkFoldersKeys_.empty()) { + YQL_CLOG(INFO, ProviderYt) << "YtIODiscovery - finish, status: " << (TStatus::ELevel)status.Level; + return status; + } + + TVector<NThreading::TFuture<void>> allFutures; + if (!PendingCanonizations_.empty()) { + CanonizeFuture_ = State_->Gateway->CanonizePaths( + IYtGateway::TCanonizePathsOptions(State_->SessionId) + .Paths(std::move(paths)) + .Config(State_->Configuration->Snapshot()) + ); + allFutures.push_back(CanonizeFuture_.IgnoreResult()); + } + + for (auto& x : PendingRanges_) { + auto& cluster = x.first.first; + auto& range = x.first.second; + auto filterLambda = range.Filter; + TUserDataTable files; + if (filterLambda) { + const auto transformer = CreateTypeAnnotationTransformer(CreateExtCallableTypeAnnotationTransformer(*State_->Types), *State_->Types); + const auto constraints = CreateConstraintTransformer(*State_->Types, true); + const auto peephole = MakePeepholeOptimization(State_->Types); + while (const auto stringType = ctx.MakeType<TDataExprType>(EDataSlot::String)) { + if (!UpdateLambdaAllArgumentsTypes(filterLambda, {stringType}, ctx)) { + return TStatus::Error; + } + + if (const auto status = transformer->Transform(filterLambda, filterLambda, ctx); status.Level == TStatus::Error) { + return status; + } else if (status.Level == TStatus::Repeat) { + continue; + } + + bool isOptional; + if (const TDataExprType* dataType = nullptr; + !(EnsureDataOrOptionalOfData(*filterLambda, isOptional, dataType, ctx) && EnsureSpecificDataType(filterLambda->Pos(), *dataType, EDataSlot::Bool, ctx))) { + return TStatus::Error; + } + + if (const auto status = UpdateLambdaConstraints(*filterLambda); status.Level == TStatus::Error) { + return status; + } + + if (const auto status = constraints->Transform(filterLambda, filterLambda, ctx); status.Level == TStatus::Error) { + return status; + } else if (status.Level == TStatus::Repeat) { + continue; + } + + if (const auto status = peephole->Transform(filterLambda, filterLambda, ctx); status.Level == TStatus::Error) { + return status; + } else if (status.Level == TStatus::Repeat) { + continue; + } + + break; + } + + if (!NCommon::FreezeUsedFilesSync(*filterLambda, files, *State_->Types, ctx, MakeUserFilesDownloadFilter(*State_->Gateway, cluster))) { + return TStatus::Error; + } + } + + auto result = State_->Gateway->GetTableRange( + IYtGateway::TTableRangeOptions(State_->SessionId) + .Cluster(cluster) + .Prefix(StripStringRight(range.Prefix, EqualsStripAdapter('/'))) + .Suffix(StripStringLeft(range.Suffix, EqualsStripAdapter('/'))) + .Filter(filterLambda.Get()) + .ExprCtx(filterLambda ? &ctx : nullptr) + .UserDataBlocks(files) + .UdfModules(State_->Types->UdfModules) + .UdfResolver(State_->Types->UdfResolver) + .UdfValidateMode(State_->Types->ValidateMode) + .Config(State_->Configuration->Snapshot()) + .OptLLVM(State_->Types->OptLLVM.GetOrElse(TString())) + .Pos(x.second.first) + ); + allFutures.push_back(result.IgnoreResult()); + x.second.second = result; + } + + for (auto& x : PendingFolders_) { + auto& cluster = x.first.first; + auto& folder = x.first.second; + auto result = State_->Gateway->GetFolder( + IYtGateway::TFolderOptions(State_->SessionId) + .Cluster(cluster) + .Prefix(folder.Prefix) + .Attributes(TSet<TString>(folder.Attributes.begin(), folder.Attributes.end())) + .Config(State_->Configuration->Snapshot()) + .Pos(x.second.first) + ); + allFutures.push_back(result.IgnoreResult()); + x.second.second = result; + } + + CanonizationRangesFoldersFuture_ = NThreading::WaitExceptionOrAll(allFutures); + return TStatus::Async; + } + + NThreading::TFuture<void> DoGetAsyncFuture(const TExprNode& input) final { + Y_UNUSED(input); + if (auto walkFoldersFuture = MaybeGetWalkFoldersFuture()) { + if (PendingCanonizations_.empty() && PendingRanges_.empty() && PendingFolders_.empty()) { + return walkFoldersFuture.GetRef(); + } + return NThreading::WaitExceptionOrAll(walkFoldersFuture.GetRef(), CanonizationRangesFoldersFuture_); + } + + return CanonizationRangesFoldersFuture_; + } + + TStatus DoApplyAsyncChanges(TExprNode::TPtr input, TExprNode::TPtr& output, TExprContext& ctx) final { + YQL_CLOG(INFO, ProviderYt) << "YtIODiscovery - DoApplyAsyncChanges start"; + output = input; + + if (!PendingCanonizations_.empty()) { + auto& res = CanonizeFuture_.GetValue(); + res.ReportIssues(ctx.IssueManager); + + if (!res.Success()) { + PendingCanonizations_.clear(); + PendingRanges_.clear(); + CanonizeFuture_ = {}; + CanonizationRangesFoldersFuture_ = {}; + + for (const auto& key : PendingWalkFoldersKeys_) { + State_->WalkFoldersState.erase(key); + } + PendingWalkFoldersKeys_.clear(); + + return TStatus::Error; + } + } + + TOptimizeExprSettings settings(nullptr); + settings.VisitChanges = true; + auto status = OptimizeExpr(input, output, [&](const TExprNode::TPtr& node, TExprContext& ctx) -> TExprNode::TPtr { + if (!TMaybeNode<TYtRead>(node).DataSource()) { + return node; + } + auto read = TYtRead(node); + auto cluster = TString{read.DataSource().Cluster().Value()}; + + TYtInputKeys keys; + if (!keys.Parse(*node->Child(2), ctx)) { + return {}; + } + + if (keys.GetType() == TYtKey::EType::Folder) { + const auto res = FetchFolderResult(ctx, cluster, *keys.GetKeys().front().GetFolder()); + if (!res) { + return {}; + } + if (auto file = std::get_if<TFileLinkPtr>(&res->ItemsOrFileLink)) { + TString alias; + if (auto p = FolderFileToAlias_.FindPtr(file->Get()->GetPath().GetPath())) { + alias = *p; + } else { + alias = TString("_yql_folder").append(ToString(FolderFileToAlias_.size())); + FolderFileToAlias_.emplace(file->Get()->GetPath().GetPath(), alias); + + TUserDataBlock tmpBlock; + tmpBlock.Type = EUserDataType::PATH; + tmpBlock.Data = file->Get()->GetPath().GetPath(); + tmpBlock.Usage.Set(EUserDataBlockUsage::Path); + tmpBlock.FrozenFile = file->Get(); + + State_->Types->UserDataStorage->AddUserDataBlock(alias, tmpBlock); + } + + auto folderListFromFile = ctx.Builder(node->Pos()) + .Callable("Collect") + .Callable(0, "Apply") + .Callable(0, "Udf") + .Atom(0, "File.FolderListFromFile") + .Seal() + .Callable(1, "FilePath") + .Atom(0, alias) + .Seal() + .Seal() + .Seal() + .Build(); + + return BuildFolderTableResExpr(ctx, node->Pos(), read.World(), folderListFromFile).Ptr(); + } + + auto items = std::get<TVector<IYtGateway::TFolderResult::TFolderItem>>(res->ItemsOrFileLink); + YQL_CLOG(INFO, ProviderYt) << "Got " << items.size() << " items for " << " GetFolder"; + TVector<TExprBase> listItems; + for (auto& item: items) { + listItems.push_back(BuildFolderListItemExpr(ctx, node->Pos(), item.Path, item.Type, item.Attributes)); + } + + return BuildFolderTableResExpr(ctx, node->Pos(), read.World(), BuildFolderListExpr(ctx, node->Pos(), listItems).Ptr()).Ptr(); + } + + if (keys.GetType() != TYtKey::EType::Table) { + return node; + } + + TVector<TExprBase> tableSettings; + TVector<TExprBase> readSettings; + SplitReadSettings(read, tableSettings, readSettings, ctx); + + bool hasErrors = false; + bool isStrict = keys.GetStrictConcat(); + TVector<TExprBase> tables; + + for (auto& key : keys.GetKeys()) { + if (key.GetRange()) { + auto p = PendingRanges_.FindPtr(std::make_pair(cluster, *key.GetRange())); + YQL_ENSURE(p); + auto& res = p->second.GetValue(); + res.ReportIssues(ctx.IssueManager); + + if (res.Success()) { + for (auto& oneTable: res.Tables) { + TYtTableInfo tableInfo; + tableInfo.Name = oneTable.Path; + tableInfo.Cluster = cluster; + auto settingsBuilder = Build<TCoNameValueTupleList>(ctx, key.GetNode()->Pos()).Add(tableSettings); + if (key.GetView()) { + settingsBuilder.Add() + .Name().Value(ToString(EYtSettingType::View)).Build() + .Value<TCoAtom>().Value(key.GetView()).Build() + .Build(); + } + tableInfo.Settings = settingsBuilder.Done(); + + TYtPathInfo pathInfo; + if (oneTable.Columns) { + pathInfo.SetColumns(*oneTable.Columns); + } + if (oneTable.Ranges) { + pathInfo.Ranges = MakeIntrusive<TYtRangesInfo>(); + pathInfo.Ranges->Parse(*oneTable.Ranges, ctx, key.GetNode()->Pos()); + } + tables.push_back(pathInfo.ToExprNode(ctx, key.GetNode()->Pos(), tableInfo.ToExprNode(ctx, key.GetNode()->Pos()))); + } + isStrict = isStrict && key.GetRange()->IsStrict; + } else { + hasErrors = true; + } + } + else if (key.IsAnonymous()) { + TYtTableInfo table(key, cluster); + table.Settings = Build<TCoNameValueTupleList>(ctx, read.Pos()) + .Add(tableSettings) + .Add() + .Name().Value(ToString(EYtSettingType::Anonymous)).Build() + .Build() + .Done(); + auto path = Build<TYtPath>(ctx, read.Pos()) + .Table(table.ToExprNode(ctx, read.Pos()).Cast<TYtTable>()) + .Columns<TCoVoid>().Build() + .Ranges<TCoVoid>().Build() + .Stat<TCoVoid>().Build() + .Done(); + tables.push_back(path); + } + else { + auto p = PendingCanonizations_.FindPtr(std::make_pair(cluster, key.GetPath())); + YQL_ENSURE(p); + auto& oneTable = CanonizeFuture_.GetValue().Data.at(*p); + if (oneTable.Path.empty()) { + ctx.AddError(TIssue(ctx.GetPosition(node->Pos()), TStringBuilder() << "Bad table name: " << key.GetPath())); + hasErrors = true; + continue; + } + TYtTableInfo tableInfo; + tableInfo.Name = oneTable.Path; + tableInfo.Cluster = cluster; + auto settingsBuilder = Build<TCoNameValueTupleList>(ctx, key.GetNode()->Pos()).Add(tableSettings); + if (key.GetView()) { + settingsBuilder.Add() + .Name().Value(ToString(EYtSettingType::View)).Build() + .Value<TCoAtom>().Value(key.GetView()).Build() + .Build(); + } + tableInfo.Settings = settingsBuilder.Done(); + + TYtPathInfo pathInfo; + if (oneTable.Columns) { + pathInfo.SetColumns(*oneTable.Columns); + } + if (oneTable.Ranges) { + pathInfo.Ranges = MakeIntrusive<TYtRangesInfo>(); + pathInfo.Ranges->Parse(*oneTable.Ranges, ctx, key.GetNode()->Pos()); + } + pathInfo.AdditionalAttributes = oneTable.AdditionalAttributes; + tables.push_back(pathInfo.ToExprNode(ctx, key.GetNode()->Pos(), tableInfo.ToExprNode(ctx, key.GetNode()->Pos()))); + } + } + if (hasErrors) { + return {}; + } + + if (!tables.size()) { + auto userSchema = GetSetting(read.Arg(4).Ref(), EYtSettingType::UserSchema); + if (userSchema) { + return BuildEmptyTablesRead(node->Pos(), *userSchema, ctx); + } + + ctx.AddError(TIssue(ctx.GetPosition(node->Pos()), "The list of tables is empty")); + return {}; + } + + if (!isStrict && tables.size() > 1) { + readSettings.push_back(Build<TCoNameValueTuple>(ctx, read.Pos()).Name().Value(ToString(EYtSettingType::WeakConcat)).Build().Done()); + } + + auto res = read.Ptr(); + res->ChildRef(2) = Build<TExprList>(ctx, read.Pos()).Add(tables).Done().Ptr(); + res->ChildRef(4) = Build<TCoNameValueTupleList>(ctx, read.Pos()).Add(readSettings).Done().Ptr(); + return res; + + }, ctx, settings); + + PendingCanonizations_.clear(); + PendingRanges_.clear(); + PendingFolders_.clear(); + CanonizeFuture_ = {}; + CanonizationRangesFoldersFuture_ = {}; + + if (status == TStatus::Ok && !PendingWalkFoldersKeys_.empty()) { + const auto walkFoldersStatus = RewriteWalkFoldersOnAsyncOrEvalChanges(output, ctx); + return walkFoldersStatus; + } + + YQL_CLOG(INFO, ProviderYt) << "YtIODiscovery DoApplyAsyncChanges - finish"; + return status; + } + + void Rewind() final { + YQL_CLOG(INFO, ProviderYt) << "Rewinding YtIODiscovery"; + PendingRanges_.clear(); + PendingFolders_.clear(); + PendingCanonizations_.clear(); + PendingWalkFoldersKeys_.clear(); + + CanonizeFuture_ = {}; + CanonizationRangesFoldersFuture_ = {}; + } + +private: + TExprNode::TPtr ConvertTableScheme(TYtRead read, const TYtKey& key, TExprContext& ctx) { + if (!read.Arg(3).Ref().IsCallable(TCoVoid::CallableName())) { + ctx.AddError(TIssue(ctx.GetPosition(read.Arg(3).Pos()), TStringBuilder() + << "Expected Void, but got: " << read.Arg(3).Ref().Content())); + return {}; + } + + if (!EnsureTuple(*read.Raw()->Child(4), ctx)) { + return {}; + } + + TYtTableInfo tableInfo(key, read.DataSource().Cluster().Value()); + + auto settings = NYql::RemoveSetting(read.Arg(4).Ref(), EYtSettingType::DoNotFailOnInvalidSchema, ctx); + if (key.GetView()) { + settings = NYql::AddSetting(*settings, EYtSettingType::View, ctx.NewAtom(key.GetNode()->Pos(), key.GetView()), ctx); + } + tableInfo.Settings = TExprBase(settings); + + auto res = read.Ptr(); + res->ChildRef(2) = Build<TExprList>(ctx, read.Pos()) + .Add<TYtPath>() + .Table(tableInfo.ToExprNode(ctx, read.Pos()).Cast<TYtTable>()) + .Columns<TCoVoid>().Build() + .Ranges<TCoVoid>().Build() + .Stat<TCoVoid>().Build() + .Build() + .Done().Ptr(); + res->ChildRef(4) = Build<TCoNameValueTupleList>(ctx, read.Pos()) + .Add() + .Name().Value(ToString(EYtSettingType::Scheme)).Build() + .Build() + .Done().Ptr(); + return res; + } + + void SplitReadSettings(TYtRead read, TVector<TExprBase>& tableSettings, TVector<TExprBase>& readSettings, TExprContext& ctx) { + if (auto list = read.Arg(4).Maybe<TCoNameValueTupleList>()) { + for (auto setting: list.Cast()) { + auto type = FromString<EYtSettingType>(setting.Name().Value()); + if (ToString(type) != setting.Name().Value()) { + // Normalize setting name + setting = Build<TCoNameValueTuple>(ctx, setting.Pos()) + .InitFrom(setting) + .Name() + .Value(ToString(type)) + .Build() + .Done(); + } + if (type & (EYtSettingType::InferScheme | EYtSettingType::ForceInferScheme | + EYtSettingType::DoNotFailOnInvalidSchema | EYtSettingType::XLock | + EYtSettingType::UserSchema | EYtSettingType::UserColumns | EYtSettingType::IgnoreTypeV3)) { + tableSettings.push_back(setting); + } else { + readSettings.push_back(setting); + } + } + } + } + + TExprNode::TPtr ConvertTableRead(TYtRead read, const TYtInputKeys& keys, TExprContext& ctx) { + TVector<TExprBase> tableSettings; + TVector<TExprBase> readSettings; + SplitReadSettings(read, tableSettings, readSettings, ctx); + + auto cluster = read.DataSource().Cluster().Value(); + TVector<TExprBase> tables; + for (auto& key: keys.GetKeys()) { + TYtTableInfo table(key, cluster); + auto settingsBuilder = Build<TCoNameValueTupleList>(ctx, read.Pos()).Add(tableSettings); + if (key.GetView()) { + settingsBuilder + .Add() + .Name().Value(ToString(EYtSettingType::View)).Build() + .Value<TCoAtom>().Value(key.GetView()).Build() + .Build(); + } + if (key.IsAnonymous()) { + settingsBuilder + .Add() + .Name().Value(ToString(EYtSettingType::Anonymous)).Build() + .Build(); + } + table.Settings = settingsBuilder.Done(); + auto path = Build<TYtPath>(ctx, read.Pos()) + .Table(table.ToExprNode(ctx, read.Pos()).Cast<TYtTable>()) + .Columns<TCoVoid>().Build() + .Ranges<TCoVoid>().Build() + .Stat<TCoVoid>().Build() + .Done(); + tables.push_back(path); + } + if (!keys.GetStrictConcat() && keys.GetKeys().size() > 1) { + readSettings.push_back(Build<TCoNameValueTuple>(ctx, read.Pos()).Name().Value(ToString(EYtSettingType::WeakConcat)).Build().Done()); + } + + auto res = read.Ptr(); + res->ChildRef(2) = Build<TExprList>(ctx, read.Pos()).Add(tables).Done().Ptr(); + res->ChildRef(4) = Build<TCoNameValueTupleList>(ctx, read.Pos()).Add(readSettings).Done().Ptr(); + return res; + } + + [[nodiscard]] + TExprNode::TPtr InitializeWalkFolders(TYtKey&& key, const TString& cluster, TPosition pos, TExprContext& ctx) { + auto& args = key.GetWalkFolderArgs().GetRef(); + + TWalkFoldersImpl walkFolders {State_->SessionId, cluster, State_->Configuration->Snapshot(), + pos, args, State_->Gateway}; + YQL_CLOG(INFO, ProviderYt) << "Initialized WalkFolders from " << cluster << ".`" + << args.InitialFolder.Prefix << "`" << " with root attributes cnt: " + << args.InitialFolder.Attributes.size(); + const auto instanceKey = ctx.NextUniqueId; + State_->WalkFoldersState.emplace(instanceKey, std::move(walkFolders)); + PendingWalkFoldersKeys_.insert(instanceKey); + + auto walkFoldersImplNode = Build<TYtWalkFoldersImpl>(ctx, key.GetNode()->Pos()) + .ProcessStateKey() + .Value(instanceKey) + .Build() + .PickledUserState(args.PickledUserState) + .UserStateType(args.UserStateType) + .Build() + .Value() + .Ptr(); + + return walkFoldersImplNode; + } + + TStatus RewriteWalkFoldersOnAsyncOrEvalChanges(TExprNode::TPtr& output, TExprContext& ctx) { + TStatus walkFoldersStatus = IGraphTransformer::TStatus::Ok; + + auto status = VisitInputKeys(output, ctx, [this, &ctx, &walkFoldersStatus] (TYtRead readNode, TYtInputKeys&& keys) -> TExprNode::TPtr { + if (keys.GetType() == TYtKey::EType::WalkFoldersImpl) { + YQL_CLOG(INFO, ProviderYt) << "YtIODiscovery - DoApplyAsyncChanges WalkFoldersImpl handling start"; + + auto parsedKey = keys.ExtractKeys().front(); + if (!parsedKey.GetWalkFolderImplArgs()) { + YQL_CLOG(ERROR, ProviderYt) << "Failed to parse WalkFolderImpl args"; + return {}; + } + const ui64 instanceKey = parsedKey.GetWalkFolderImplArgs()->StateKey; + if (*PendingWalkFoldersKeys_.begin() != instanceKey) { + return readNode.Ptr(); + } + + auto walkFoldersInstanceIt = this->State_->WalkFoldersState.find(instanceKey); + YQL_ENSURE(!walkFoldersInstanceIt.IsEnd()); + auto& walkFoldersImpl = walkFoldersInstanceIt->second; + + Y_ENSURE(walkFoldersImpl.GetAnyOpFuture().HasValue(), + "Called RewriteWalkFoldersOnAsyncChanges, but impl future is not ready"); + + auto nextState = parsedKey.GetWalkFolderImplArgs()->UserStateExpr; + walkFoldersStatus = walkFoldersImpl.GetNextStateExpr(ctx, std::move(parsedKey.GetWalkFolderImplArgs().GetRef()), nextState); + + if (walkFoldersStatus == TStatus::Error) { + return {}; + } + + if (walkFoldersImpl.IsFinished()) { + YQL_CLOG(INFO, ProviderYt) << "Building result expr for WalkFolders with key: " << instanceKey; + this->State_->WalkFoldersState.erase(instanceKey); + PendingWalkFoldersKeys_.erase(instanceKey); + + auto type = Build<TCoStructType>(ctx, readNode.Pos()) + .Add<TExprList>() + .Add<TCoAtom>() + .Value("State") + .Build() + .Add(parsedKey.GetWalkFolderImplArgs()->UserStateType) + .Build() + .DoBuild(); + + auto resList = Build<TCoList>(ctx, readNode.Pos()) + .ListType<TCoListType>() + .ItemType<TCoStructType>() + .InitFrom(type) + .Build() + .Build() + .FreeArgs() + .Add<TCoAsStruct>() + .Add() + .Add<TCoAtom>() + .Value("State") + .Build() + .Add(nextState) + .Build() + .Build() + .Build() + .DoBuild(); + + return Build<TCoCons>(ctx, readNode.Pos()) + .World(readNode.World()) + .Input<TCoAssumeColumnOrder>() + .Input(resList) + .ColumnOrder<TCoAtomList>() + .Add() + .Value("State") + .Build() + .Build() + .Build() + .Done() + .Ptr(); + } + + if (nextState == parsedKey.GetWalkFolderImplArgs()->UserStateExpr) { + return readNode.Ptr(); + } + + YQL_CLOG(TRACE, ProviderYt) << "State expr ast: " << ConvertToAst(*nextState, ctx, {}).Root->ToString(); + + auto walkFoldersImplNode = ctx.ChangeChild(*parsedKey.GetNode(), 0, std::move(nextState)); + return ctx.ChangeChild(readNode.Ref(), 2, std::move(walkFoldersImplNode)); + } + return readNode.Ptr(); + }); + + if (status == TStatus::Error) { + YQL_CLOG(ERROR, ProviderYt) << "WalkFolders error transforming"; + return status; + } + + YQL_CLOG(INFO, ProviderYt) << "WalkFolders next status: " << walkFoldersStatus; + return walkFoldersStatus; + } + + IGraphTransformer::TStatus VisitInputKeys(TExprNode::TPtr& output, + TExprContext& ctx, std::function<TExprNode::TPtr(TYtRead node, TYtInputKeys&&)> processKeys, bool visitChanges = false) { + TOptimizeExprSettings settings(nullptr); + settings.VisitChanges = visitChanges; + + const auto status = OptimizeExpr(output, output, [&](const TExprNode::TPtr& node, TExprContext& ctx) -> TExprNode::TPtr { + if (auto maybeRead = TMaybeNode<TYtRead>(node)) { + if (!maybeRead.DataSource()) { // Validates provider + return node; + } + auto read = maybeRead.Cast(); + auto ds = read.DataSource(); + if (!EnsureArgsCount(read.Ref(), 5, ctx)) { + return {}; + } + + TYtInputKeys keys; + auto& keysNode = read.Arg(2).Ref(); + if (!keys.Parse(keysNode, ctx)) { + return {}; + } + + if (keys.IsProcessed()) { + // Already processed + return node; + } + + if (keys.GetKeys().empty()) { + ctx.AddError(TIssue(ctx.GetPosition(read.Arg(2).Pos()), "The list of tables is empty")); + return {}; + } + return processKeys(read, std::move(keys)); + } + return node; + }, ctx, settings); + return status; + } + + TCoCons BuildFolderTableResExpr(TExprContext& ctx, NYql::TPositionHandle pos, const TExprBase& world, const TExprNodePtr& folderList) { + return Build<TCoCons>(ctx, pos) + .World(world) + .Input<TCoAssumeColumnOrder>() + .Input(folderList) + .ColumnOrder<TCoAtomList>() + .Add() + .Value("Path") + .Build() + .Add() + .Value("Type") + .Build() + .Add() + .Value("Attributes") + .Build() + .Build() + .Build() + .Done(); + } + + TMaybe<NYql::IYtGateway::TFolderResult> FetchFolderResult(TExprContext& ctx, const TString& cluster, const TYtKey::TFolderList& folder) { + auto p = PendingFolders_.FindPtr(std::make_pair(cluster, folder)); + YQL_ENSURE(p); + auto res = p->second.GetValue(); + res.ReportIssues(ctx.IssueManager); + if (!res.Success()) { + return {}; + } + return res; + } + + TWalkFoldersImpl& GetCurrentWalkFoldersInstance() const { + Y_ENSURE(!PendingWalkFoldersKeys_.empty()); + const auto key = PendingWalkFoldersKeys_.begin(); + auto stateIt = State_->WalkFoldersState.find(*key); + YQL_ENSURE(stateIt != State_->WalkFoldersState.end()); + return stateIt->second; + } + + TMaybe<NThreading::TFuture<void>> MaybeGetWalkFoldersFuture() const { + // inflight 1 + if (!PendingWalkFoldersKeys_.empty()) { + return GetCurrentWalkFoldersInstance().GetAnyOpFuture(); + } + return Nothing(); + } + +private: + TYtState::TPtr State_; + + THashMap<std::pair<TString, TYtKey::TRange>, std::pair<TPosition, NThreading::TFuture<IYtGateway::TTableRangeResult>>> PendingRanges_; + THashMap<std::pair<TString, TYtKey::TFolderList>, std::pair<TPosition, NThreading::TFuture<IYtGateway::TFolderResult>>> PendingFolders_; + THashMap<std::pair<TString, TString>, size_t> PendingCanonizations_; // cluster, original table path -> positions in canon result + TSet<ui64> PendingWalkFoldersKeys_; + NThreading::TFuture<IYtGateway::TCanonizePathsResult> CanonizeFuture_; + NThreading::TFuture<void> CanonizationRangesFoldersFuture_; + + THashMap<TString, TString> FolderFileToAlias_; +}; + +THolder<IGraphTransformer> CreateYtIODiscoveryTransformer(TYtState::TPtr state) { + return THolder(new TYtIODiscoveryTransformer(state)); +} + +} diff --git a/yt/yql/providers/yt/provider/yql_yt_io_discovery_walk_folders.cpp b/yt/yql/providers/yt/provider/yql_yt_io_discovery_walk_folders.cpp new file mode 100644 index 0000000000..acb16851c8 --- /dev/null +++ b/yt/yql/providers/yt/provider/yql_yt_io_discovery_walk_folders.cpp @@ -0,0 +1,589 @@ +#include "yql_yt_io_discovery_walk_folders.h" + +#include <yt/yql/providers/yt/gateway/native/yql_yt_native_folders.h> +#include <yt/yql/providers/yt/provider/yql_yt_gateway.h> +#include <yql/essentials/minikql/computation/mkql_computation_node_pack.h> +#include <yql/essentials/utils/log/log.h> + +#include <util/string/split.h> + +namespace NYql { +using namespace NNodes; + + +NNodes::TCoStructType +BuildFolderItemStructType(TExprContext& ctx, NYql::TPositionHandle pos) { + return Build<TCoStructType>(ctx, pos) + .Add<TExprList>() + .Add<TCoAtom>() + .Value("Path") + .Build() + .Add<TCoDataType>() + .Type() + .Value("String") + .Build() + .Build() + .Build() + .Add<TExprList>() + .Add<TCoAtom>() + .Value("Type") + .Build() + .Add<TCoDataType>() + .Type() + .Value("String") + .Build() + .Build() + .Build() + .Add<TExprList>() + .Add<TCoAtom>() + .Value("Attributes") + .Build() + .Add<TCoDataType>() + .Type() + .Value("Yson") + .Build() + .Build() + .Build() + .Done(); +} + +TCoList +BuildFolderListExpr(TExprContext& ctx, NYql::TPositionHandle pos, const TVector<NNodes::TExprBase>& folderItems) { + return Build<TCoList>(ctx, pos) + .ListType<TCoListType>() + .ItemType<TCoStructType>() + .InitFrom(BuildFolderItemStructType(ctx, pos)) + .Build() + .Build() + .FreeArgs() + .Add(folderItems) + .Build() + .Build() + .Value(); +} + +TExprBase +BuildFolderListItemExpr(TExprContext &ctx, NYql::TPositionHandle pos, + const TString &path, const TString &type, + const TString &attributesYson) { + return Build<TCoAsStruct>(ctx, pos) + .Add() + .Add<TCoAtom>() + .Value("Path") + .Build() + .Add<TCoString>() + .Literal() + .Value(path) + .Build() + .Build() + .Build() + .Add() + .Add<TCoAtom>() + .Value("Type") + .Build() + .Add<TCoString>() + .Literal() + .Value(type) + .Build() + .Build() + .Build() + .Add() + .Add<TCoAtom>() + .Value("Attributes") + .Build() + .Add<TCoYson>() + .Literal() + .Value(attributesYson) + .Build() + .Build() + .Build() + .Done(); +} + +TWalkFoldersImpl::TWalkFoldersImpl(const TString& sessionId, const TString& cluster, TYtSettings::TConstPtr config, + TPosition pos, const TYtKey::TWalkFoldersArgs& args, const IYtGateway::TPtr gateway): + Pos_(pos), SessionId_(sessionId), Cluster_(cluster), Config_(config), Gateway_(gateway) { + + PreHandler_ = args.PreHandler->IsCallable("Void") ? Nothing() : MakeMaybe(args.PreHandler); + ResolveHandler_ = args.ResolveHandler; + DiveHandler_ = args.DiveHandler; + PostHandler_ = args.PostHandler->IsCallable("Void") ? Nothing() : MakeMaybe(args.PostHandler); + + ProcessFoldersQueue_.emplace_back(TFolderQueueItem { + .Folder = args.InitialFolder, + }); + IYtGateway::TBatchFolderOptions::TFolderPrefixAttrs folder { + std::move(args.InitialFolder.Prefix), + TSet<TString>(args.InitialFolder.Attributes.begin(), args.InitialFolder.Attributes.end()) + }; + DoFolderListOperation({folder}); +} + +IGraphTransformer::TStatus TWalkFoldersImpl::GetNextStateExpr(TExprContext& ctx, const TYtKey::TWalkFoldersImplArgs& args, TExprNode::TPtr& state) { + YQL_CLOG(INFO, ProviderYt) << "Current processing state: " << int(ProcessingState_); + switch (ProcessingState_) { + case WaitingListFolderOp: { + return AfterListFolderOp(ctx, args, state); + } + case PreHandling: { + return PreHandleVisitedInSingleFolder(ctx, args, ProcessFoldersQueue_.front(), state); + } + case ResolveHandling: { + return ResolveHandleInSingleFolder(ctx, args, ProcessFoldersQueue_.front(), state); + } + case AfterResolveHandling: { + return AfterResolveHandle(ctx, args, ProcessFoldersQueue_.front(), state); + } + case WaitingResolveLinkOp: { + return HandleAfterResolveFuture(ctx, args, ProcessFoldersQueue_.front(), state); + } + case DiveHandling: { + return DiveHandleInSingleFolder(ctx, args, ProcessFoldersQueue_.front(), state); + } + case AfterDiveHandling: { + return AfterDiveHandle(ctx, args, ProcessFoldersQueue_.front(), state); + } + case PostHandling: { + return PostHandleVisitedInSingleFolder(ctx, args, ProcessFoldersQueue_.front(), state); + } + case FinishingHandling: { + return BuildFinishedState(ctx, args, state); + } + case FinishedHandling: { + return IGraphTransformer::TStatus::Ok; + } + } + return IGraphTransformer::TStatus::Ok; +} + +void TWalkFoldersImpl::DoFolderListOperation(TVector<IYtGateway::TBatchFolderOptions::TFolderPrefixAttrs>&& folders) { + YQL_CLOG(INFO, ProviderYt) << "Sending folder list batch with " << folders.size() << " items"; + auto options = IYtGateway::TBatchFolderOptions(SessionId_) + .Pos(Pos_) + .Cluster(Cluster_) + .Config(Config_) + .Folders(folders); + BatchFolderListFuture_ = Gateway_->GetFolders(std::move(options)); +} + +IGraphTransformer::TStatus TWalkFoldersImpl::EvaluateNextUserStateExpr(TExprContext& ctx, const TExprNode::TPtr& userStateType, const TExprNode::TPtr userStateExpr, std::function<TExprNode::TPtr(const NNodes::TExprBase&)> nextStateFunc, TExprNode::TPtr& state) { + const auto userStateUnpickled = Build<TCoUnpickle>(ctx, PosHandle_) + .Type(userStateType) + .Buffer(userStateExpr) + .Build(); + + const auto nextUserStatePickled = Build<TCoPickle>(ctx, PosHandle_) + .Value(nextStateFunc(userStateUnpickled.Value())) + .Build() + .Value() + .Ptr(); + + ctx.Step.Repeat(TExprStep::ExprEval); + + YQL_CLOG(TRACE, ProviderYt) << "WalkFolders - next evaluate ast: " << ConvertToAst(*nextUserStatePickled, ctx, {}).Root->ToString(); + + state = ctx.Builder(PosHandle_) + .Callable("EvaluateExpr") + .Add(0, nextUserStatePickled) + .Seal() + .Build(); + return IGraphTransformer::TStatus::Ok; +} + +IGraphTransformer::TStatus TWalkFoldersImpl::AfterListFolderOp(TExprContext& ctx, const TYtKey::TWalkFoldersImplArgs& args, TExprNode::TPtr& state) { + if (!BatchFolderListFuture_) { + YQL_CLOG(INFO, ProviderYt) << "Folder queue is empty, finishing WalkFolders with key: " << args.StateKey; + ProcessingState_ = FinishingHandling; + return GetNextStateExpr(ctx, args, state); + } else { + if (!BatchFolderListFuture_->HasValue()) { + YQL_CLOG(INFO, ProviderYt) << "Batch list future is not ready"; + return IGraphTransformer::TStatus::Repeat; + } + + Y_ENSURE(!ProcessFoldersQueue_.empty(), "Got future result for Yt List but no folder in queue"); + auto folderListVal = BatchFolderListFuture_->GetValueSync(); + if (folderListVal.Success()) { + auto& folder = ProcessFoldersQueue_.front(); + YQL_CLOG(INFO, ProviderYt) << "Got " << folderListVal.Items.size() << " results for list op at `" << folder.Folder.Prefix << "`"; + folder.ItemsToPreHandle = std::move(folderListVal.Items); + folder.PreHandleItemsFetched = true; + ProcessingState_ = PreHandling; + } else { + folderListVal.ReportIssues(ctx.IssueManager); + } + + BatchFolderListFuture_ = Nothing(); + } + return PreHandleVisitedInSingleFolder(ctx, args, ProcessFoldersQueue_.front(), state); +} + +IGraphTransformer::TStatus TWalkFoldersImpl::PreHandleVisitedInSingleFolder(TExprContext& ctx, const TYtKey::TWalkFoldersImplArgs& args, TFolderQueueItem& folder, TExprNode::TPtr& state) { + YQL_CLOG(INFO, ProviderYt) << "Processing preHandler at " << folder.Folder.Prefix + << " for WalkFolders with key: " << args.StateKey; + + if (!folder.PreHandleItemsFetched) { + YQL_CLOG(INFO, ProviderYt) << "Waiting for folder list: `" << folder.Folder.Prefix << "`"; + ProcessingState_ = WaitingListFolderOp; + return GetNextStateExpr(ctx, args, state); + } + + if (folder.ItemsToPreHandle.empty()) { + YQL_CLOG(INFO, ProviderYt) << "Items to preHandle are empty, skipping " << folder.Folder.Prefix; + ProcessFoldersQueue_.pop_front(); + ProcessingState_ = WaitingListFolderOp; + return GetNextStateExpr(ctx, args, state); + } + + TVector<TExprBase> folderListItems; + for (auto&& item : folder.ItemsToPreHandle) { + if (PreHandler_) { + folderListItems.push_back( + BuildFolderListItemExpr(ctx, PosHandle_, item.Path,item.Type, + NYT::NodeToYsonString(item.Attributes))); + } + + if (item.Type == "link") { + folder.LinksToResolveHandle.emplace_back(std::move(item)); + } else if (item.Type == "map_node") { + folder.ItemsToDiveHandle.emplace_back(std::move(item)); + } else { + folder.ItemsToPostHandle.emplace_back(std::move(item)); + } + } + + if (!PreHandler_) { + YQL_CLOG(INFO, ProviderYt) << "No preHandler defined, skipping for WalkFolders with key: " << args.StateKey; + ProcessingState_ = ResolveHandling; + return GetNextStateExpr(ctx, args, state); + } + + const auto folderListExpr = BuildFolderListExpr(ctx, PosHandle_, folderListItems); + + const auto makeNextUserState = [&] (const TExprBase& userStateUnpickled) { + return Build<TCoApply>(ctx, PosHandle_) + .Callable(PreHandler_.GetRef()) + .FreeArgs() + .Add(folderListExpr) + .Add(userStateUnpickled) + .Add<TCoInt64>() + .Literal() + .Value(ToString(folder.Level)) + .Build() + .Build() + .Build() + .Build() + .Value() + .Ptr(); + }; + + ProcessingState_ = ResolveHandling; + return EvaluateNextUserStateExpr(ctx, args.UserStateType, args.UserStateExpr, makeNextUserState, state); +} + +IGraphTransformer::TStatus TWalkFoldersImpl::ResolveHandleInSingleFolder(TExprContext& ctx, const TYtKey::TWalkFoldersImplArgs& args, TFolderQueueItem& folder, TExprNode::TPtr& state) { + YQL_CLOG(INFO, ProviderYt) << "Processing resolveHandler at " << folder.Folder.Prefix + << "for WalkFolders with key: " << args.StateKey; + ProcessingState_ = AfterResolveHandling; + return BuildDiveOrResolveHandlerEval(ctx, args, ResolveHandler_.GetRef(), folder.LinksToResolveHandle, folder.Folder.Attributes, folder.Level, state); +} + +IGraphTransformer::TStatus TWalkFoldersImpl::BuildDiveOrResolveHandlerEval(TExprContext& ctx, const TYtKey::TWalkFoldersImplArgs& args, TExprNode::TPtr& handler, + const TVector<IYtGateway::TBatchFolderResult::TFolderItem>& res, const TVector<TString>& attributes, ui64 level, TExprNode::TPtr& state) { + using namespace NNodes; + + TVector<TExprBase> items; + items.reserve(res.size()); + for (auto& link : res) { + auto itemsExpr = + BuildFolderListItemExpr(ctx, PosHandle_, + link.Path, link.Type, NYT::NodeToYsonString(link.Attributes)); + items.push_back(itemsExpr); + } + const auto itemsNode = BuildFolderListExpr(ctx, PosHandle_, items); + + TVector<TExprBase> attributeExprs; + for (auto& attr : attributes) { + const auto attributeExpr = Build<TCoString>(ctx, PosHandle_) + .Literal() + .Value(attr) + .Build() + .Build() + .Value(); + attributeExprs.push_back(attributeExpr); + } + + const auto userStateUnpickled = Build<TCoUnpickle>(ctx, PosHandle_) + .Type(args.UserStateType) + .Buffer(args.UserStateExpr) + .DoBuild(); + + const auto handlerResult = Build<TCoApply>(ctx, PosHandle_) + .Callable(handler) + .FreeArgs() + .Add(itemsNode) + .Add(userStateUnpickled) + .Add<TCoAsList>() + .Add(attributeExprs) + .Build() + .Add<TCoInt64>() + .Literal() + .Value(ToString(level)) + .Build() + .Build() + .Build() + .Build() + .Value() + .Ptr(); + + const auto resolveHandlerResPickled = ctx.Builder(PosHandle_) + .Callable("StaticMap") + .Add(0, handlerResult) + .Lambda(1) + .Param("item") + .Callable("Pickle") + .Arg(0, "item") + .Seal() + .Seal() + .Seal() + .Build(); + + ctx.Step.Repeat(TExprStep::ExprEval); + state = ctx.Builder(PosHandle_) + .Callable("EvaluateExpr") + .Add(0, resolveHandlerResPickled) + .Seal() + .Build(); + return IGraphTransformer::TStatus::Ok; +} + +void ParseNameAttributesPickledList(TStringBuf pickledTupleList, std::function<void(TString&&, TSet<TString>)> handleParsedNameAndAttrs) { + using namespace NKikimr::NMiniKQL; + + TScopedAlloc alloc(__LOCATION__); + TTypeEnvironment env(alloc); + TMemoryUsageInfo memInfo("Yt WalkFolders"); + THolderFactory holderFactory(alloc.Ref(), memInfo); + + TSmallVec<TType*> nodeToResolveWithAttrListTypes; + auto stringType = TDataType::Create(NUdf::TDataType<char*>::Id, env); + nodeToResolveWithAttrListTypes.push_back(stringType); + nodeToResolveWithAttrListTypes.push_back(TListType::Create(stringType, env)); + + auto nodeToResolveTuple = TTupleType::Create(2, nodeToResolveWithAttrListTypes.data(), env); + TValuePacker packer(false, TListType::Create(nodeToResolveTuple, env)); + auto parsedList = packer.Unpack(pickledTupleList, holderFactory); + + YQL_CLOG(INFO, ProviderYt) << "Parsing list with length: " << parsedList.GetListLength(); + + for (size_t i = 0; i < parsedList.GetListLength(); ++i) { + const auto requestedTuple = parsedList.GetElement(i); + const auto nameEl = requestedTuple.GetElement(0); + const auto name = nameEl.AsStringRef(); + YQL_CLOG(INFO, ProviderYt) << "Parsed dive or resolve item name: " << name; + + auto requestedAttrsVal = requestedTuple.GetElement(1); + TSet<TString> attrs; + for (size_t j = 0; j < requestedAttrsVal.GetListLength(); ++j) { + const auto attrEl = requestedAttrsVal.GetElement(j); + YQL_CLOG(INFO, ProviderYt) << "Parsed requested attribute: " << attrEl.AsStringRef(); + attrs.insert(TString(attrEl.AsStringRef())); + } + handleParsedNameAndAttrs(TString(name), attrs); + } +} + +IGraphTransformer::TStatus TWalkFoldersImpl::AfterResolveHandle(TExprContext& ctx, TYtKey::TWalkFoldersImplArgs args, TFolderQueueItem& folder, TExprNode::TPtr& state) { + EnsureTupleSize(*args.UserStateExpr, 2, ctx); + YQL_CLOG(INFO, ProviderYt) << "After resolveHandler EvaluateExpr"; + + TCoString pickledLinksToResolve(args.UserStateExpr->Child(0)); + THashMap<TString, TSet<TString>> nameAndRequestedAttrs; + ParseNameAttributesPickledList(pickledLinksToResolve.Literal().StringValue(), + [&nameAndRequestedAttrs] (TString name, TSet<TString> attrs) { + nameAndRequestedAttrs[name] = std::move(attrs); + }); + + TVector<IYtGateway::TResolveOptions::TItemWithReqAttrs> links; + links.reserve(nameAndRequestedAttrs.size()); + for (auto&& linkToResolve : folder.LinksToResolveHandle) { + auto it = nameAndRequestedAttrs.find(linkToResolve.Path); + if (it == nameAndRequestedAttrs.end()) { + continue; + } + + IYtGateway::TResolveOptions::TItemWithReqAttrs link { + .Item = std::move(linkToResolve), + .AttrKeys = std::move(it->second), + }; + links.emplace_back(std::move(link)); + } + + if (links.empty()) { + YQL_CLOG(INFO, ProviderYt) << "Links to visit are empty"; + args.UserStateExpr = args.UserStateExpr->Child(1); + state = args.UserStateExpr; + + ProcessingState_ = DiveHandling; + return GetNextStateExpr(ctx, args, state); + } + + ProcessingState_ = WaitingResolveLinkOp; + auto options = IYtGateway::TResolveOptions(SessionId_) + .Pos(Pos_) + .Cluster(Cluster_) + .Config(Config_) + .Items(links); + BatchResolveFuture_ = Gateway_->ResolveLinks(std::move(options)); + + state = args.UserStateExpr->Child(1); + return IGraphTransformer::TStatus::Repeat; +} + +IGraphTransformer::TStatus TWalkFoldersImpl::HandleAfterResolveFuture(TExprContext& ctx, const TYtKey::TWalkFoldersImplArgs& args, TFolderQueueItem& folder, TExprNode::TPtr& state) { + YQL_CLOG(INFO, ProviderYt) << "After resolve future result"; + + if (!BatchResolveFuture_) { + ctx.AddError(TIssue(Pos_, TStringBuilder() << "Resolve future not set for WalkFolders in: " << folder.Folder.Prefix)); + return IGraphTransformer::TStatus::Error; + } + if (!BatchResolveFuture_->HasValue() && !BatchResolveFuture_->HasException()) { + YQL_CLOG(INFO, ProviderYt) << "Batch resolve future is not ready"; + return IGraphTransformer::TStatus::Repeat; + } + + auto res = BatchResolveFuture_->GetValueSync(); + BatchResolveFuture_ = Nothing(); + YQL_CLOG(INFO, ProviderYt) << "Added items to handle after batch resolve future completion"; + + for (auto&& node : res.Items) { + if (node.Type == "map_node") { + folder.ItemsToDiveHandle.emplace_back(std::move(node)); + } else { + folder.ItemsToPostHandle.emplace_back(std::move(node)); + } + } + + ProcessingState_ = DiveHandling; + return GetNextStateExpr(ctx, args, state); +} + +IGraphTransformer::TStatus TWalkFoldersImpl::DiveHandleInSingleFolder(TExprContext& ctx, const TYtKey::TWalkFoldersImplArgs& args, TFolderQueueItem& folder, TExprNode::TPtr& state) { + YQL_CLOG(INFO, ProviderYt) << "Processing diveHandler at " << folder.Folder.Prefix + << " for WalkFolders with key: " << args.StateKey; + ProcessingState_ = AfterDiveHandling; + return BuildDiveOrResolveHandlerEval(ctx, args, DiveHandler_.GetRef(), folder.ItemsToDiveHandle, folder.Folder.Attributes, folder.Level, state); +} + +IGraphTransformer::TStatus TWalkFoldersImpl::AfterDiveHandle(TExprContext& ctx, TYtKey::TWalkFoldersImplArgs args, TFolderQueueItem& folder, TExprNode::TPtr& state) { + using namespace NKikimr::NMiniKQL; + + EnsureTupleSize(*args.UserStateExpr, 2, ctx); + YQL_CLOG(INFO, ProviderYt) << "After diveHandler EvaluateExpr"; + + TVector<IYtGateway::TBatchFolderOptions::TFolderPrefixAttrs> diveItems; + TCoString pickledLinksToResolve(args.UserStateExpr->Child(0)); + THashMap<TString, TSet<TString>> nameAndRequestedAttrs; + ParseNameAttributesPickledList(pickledLinksToResolve.Literal().StringValue(), + [&queue=ProcessFoldersQueue_, &diveItems, nextLevel = folder.Level + 1] (TString path, TSet<TString> attrs) { + diveItems.push_back({.Prefix = path, .AttrKeys = attrs}); + queue.push_back({ + .Folder = {.Prefix = std::move(path), .Attributes = TVector<TString>(attrs.begin(), attrs.end())}, + .Level = nextLevel, + }); + }); + + folder.ItemsToPostHandle.insert(folder.ItemsToPostHandle.end(), + std::make_move_iterator(folder.ItemsToDiveHandle.begin()), + std::make_move_iterator(folder.ItemsToDiveHandle.end())); + folder.ItemsToDiveHandle.clear(); + + args.UserStateExpr = args.UserStateExpr->Child(1); + state = args.UserStateExpr; + ProcessingState_ = PostHandling; + + if (diveItems.empty()) { + YQL_CLOG(INFO, ProviderYt) << "Nodes to dive are empty"; + return GetNextStateExpr(ctx, args, state); + } + + auto options = IYtGateway::TBatchFolderOptions(SessionId_) + .Pos(Pos_) + .Cluster(Cluster_) + .Config(Config_) + .Folders(diveItems); + Y_ENSURE(!BatchFolderListFuture_, "Single inflight batch folder request allowed"); + BatchFolderListFuture_ = Gateway_->GetFolders(std::move(options)); + + return GetNextStateExpr(ctx, args, state); +} + +IGraphTransformer::TStatus TWalkFoldersImpl::PostHandleVisitedInSingleFolder(TExprContext& ctx, const TYtKey::TWalkFoldersImplArgs& args, TFolderQueueItem& folder, TExprNode::TPtr& state) { + if (!PostHandler_) { + YQL_CLOG(INFO, ProviderYt) << "No postHandler defined, skipping for WalkFolders with key: " << args.StateKey; + ProcessingState_ = WaitingListFolderOp; + return IGraphTransformer::TStatus::Repeat; + } + + YQL_CLOG(INFO, ProviderYt) << "Processing postHandler at " << folder.Folder.Prefix + << " for WalkFolders with key: " << args.StateKey; + + TVector<TExprBase> folderListItems; + for (auto&& item : folder.ItemsToPostHandle) { + folderListItems.push_back( + BuildFolderListItemExpr(ctx, + PosHandle_, + item.Path, + item.Type, + NYT::NodeToYsonString(item.Attributes))); + + } + + const auto folderListExpr = BuildFolderListExpr(ctx, PosHandle_, folderListItems); + + const auto makeNextUserState = [&] (const TExprBase& userStateUnpickled) { + return Build<TCoApply>(ctx, PosHandle_) + .Callable(PostHandler_.GetRef()) + .FreeArgs() + .Add(folderListExpr) + .Add(userStateUnpickled) + .Add<TCoInt64>() + .Literal() + .Value(ToString(folder.Level)) + .Build() + .Build() + .Build() + .Build() + .Value() + .Ptr(); + }; + + ProcessingState_ = WaitingListFolderOp; + + ProcessFoldersQueue_.pop_front(); + return EvaluateNextUserStateExpr(ctx, args.UserStateType, args.UserStateExpr, makeNextUserState, state); +} + +IGraphTransformer::TStatus TWalkFoldersImpl::BuildFinishedState(TExprContext& ctx, const TYtKey::TWalkFoldersImplArgs& args, TExprNode::TPtr& state) { + // TODO: Dump large user state to file + // const auto dataLen = args.UserStateExpr->IsCallable("String") + // ? TCoString(args.UserStateExpr).Literal().StringValue().Size() + // : 0; + + const auto userStateUnpickled = Build<TCoUnpickle>(ctx, PosHandle_) + .Type(args.UserStateType) + .Buffer(args.UserStateExpr) + .DoBuild() + .Ptr(); + + ctx.Step.Repeat(TExprStep::ExprEval); + ProcessingState_ = FinishedHandling; + + state = ctx.Builder(PosHandle_) + .Callable("EvaluateExpr") + .Add(0, userStateUnpickled) + .Seal() + .Build(); + return IGraphTransformer::TStatus::Ok; +} +} // namespace NYql diff --git a/yt/yql/providers/yt/provider/yql_yt_io_discovery_walk_folders.h b/yt/yql/providers/yt/provider/yql_yt_io_discovery_walk_folders.h new file mode 100644 index 0000000000..cdfd5c5993 --- /dev/null +++ b/yt/yql/providers/yt/provider/yql_yt_io_discovery_walk_folders.h @@ -0,0 +1,128 @@ +#pragma once + +#include <yt/yql/providers/yt/provider/yql_yt_gateway.h> +#include <yt/yql/providers/yt/provider/yql_yt_key.h> + +#include <library/cpp/yson/node/node_io.h> +#include <library/cpp/threading/future/core/future.h> + +namespace NYql { +NNodes::TExprBase +BuildFolderListItemExpr(TExprContext &ctx, NYql::TPositionHandle pos, + const TString &path, const TString &type, + const TString &attributesYson); + +NNodes::TCoList +BuildFolderListExpr(TExprContext& ctx, NYql::TPositionHandle pos, + const TVector<NNodes::TExprBase>& folderItems); + +NNodes::TCoStructType +BuildFolderItemStructType(TExprContext& ctx, NYql::TPositionHandle pos); + +class TWalkFoldersImpl { +public: + TWalkFoldersImpl(const TString& sessionId, const TString& cluster, TYtSettings::TConstPtr config, + TPosition pos, const TYtKey::TWalkFoldersArgs& args, const IYtGateway::TPtr gateway); + + IGraphTransformer::TStatus GetNextStateExpr(TExprContext& ctx, const TYtKey::TWalkFoldersImplArgs& args, TExprNode::TPtr& state); + + enum EProcessingState { + WaitingListFolderOp, + PreHandling, + ResolveHandling, + AfterResolveHandling, + WaitingResolveLinkOp, + DiveHandling, + AfterDiveHandling, + PostHandling, + FinishingHandling, + FinishedHandling + }; + + EProcessingState GetProcessingState() const { + return ProcessingState_; + } + + bool IsFinished() const { + return ProcessingState_ == FinishedHandling; + } + + NThreading::TFuture<void> GetAnyOpFuture() const { + TVector<NThreading::TFuture<void>> futures; + if (BatchFolderListFuture_ && BatchFolderListFuture_->Initialized()) { + futures.push_back(BatchFolderListFuture_->IgnoreResult()); + } + if (BatchResolveFuture_&& BatchResolveFuture_->Initialized()) { + futures.push_back(BatchResolveFuture_->IgnoreResult()); + } + return NThreading::WaitAny(futures); + } + + TWalkFoldersImpl& operator=(const TWalkFoldersImpl&) = delete; + TWalkFoldersImpl(const TWalkFoldersImpl&) = delete; + + TWalkFoldersImpl(TWalkFoldersImpl&&) = default; + TWalkFoldersImpl& operator=(TWalkFoldersImpl&&) = default; + +private: + static constexpr size_t LARGE_USER_STATE = 8192; + + TPosition Pos_; + TPositionHandle PosHandle_; + + TMaybe<TExprNode::TPtr> PreHandler_; + TMaybe<TExprNode::TPtr> ResolveHandler_; + TMaybe<TExprNode::TPtr> DiveHandler_; + TMaybe<TExprNode::TPtr> PostHandler_; + + struct TFolderQueueItem { + TYtKey::TFolderList Folder; + + bool PreHandleItemsFetched = false; + + TVector<IYtGateway::TBatchFolderResult::TFolderItem> ItemsToPreHandle; + TVector<IYtGateway::TBatchFolderResult::TFolderItem> LinksToResolveHandle; + TVector<IYtGateway::TBatchFolderResult::TFolderItem> ItemsToDiveHandle; + TVector<IYtGateway::TBatchFolderResult::TFolderItem> ItemsToPostHandle; + + ui64 Level = 0; + }; + TDeque<TFolderQueueItem> ProcessFoldersQueue_; + + EProcessingState ProcessingState_ = WaitingListFolderOp; + + TString SessionId_; + TString Cluster_; + TYtSettings::TConstPtr Config_; + + IYtGateway::TPtr Gateway_; + + TMaybe<NThreading::TFuture<IYtGateway::TBatchFolderResult>> BatchFolderListFuture_; + TMaybe<NThreading::TFuture<IYtGateway::TBatchFolderResult>> BatchResolveFuture_; + + void DoFolderListOperation(TVector<IYtGateway::TBatchFolderOptions::TFolderPrefixAttrs>&& folders); + + IGraphTransformer::TStatus EvaluateNextUserStateExpr(TExprContext& ctx, const TExprNode::TPtr& userStateType, const TExprNode::TPtr userStateExpr, std::function<TExprNode::TPtr(const NNodes::TExprBase&)> nextStateFunc, TExprNode::TPtr& state); + + IGraphTransformer::TStatus AfterListFolderOp(TExprContext& ctx, const TYtKey::TWalkFoldersImplArgs& args, TExprNode::TPtr& state); + + IGraphTransformer::TStatus PreHandleVisitedInSingleFolder(TExprContext& ctx, const TYtKey::TWalkFoldersImplArgs& args, TFolderQueueItem& folder, TExprNode::TPtr& state); + + IGraphTransformer::TStatus ResolveHandleInSingleFolder(TExprContext& ctx, const TYtKey::TWalkFoldersImplArgs& args, TFolderQueueItem& folder, TExprNode::TPtr& state); + + IGraphTransformer::TStatus BuildDiveOrResolveHandlerEval(TExprContext& ctx, const TYtKey::TWalkFoldersImplArgs& args, TExprNode::TPtr& handler, + const TVector<IYtGateway::TBatchFolderResult::TFolderItem>& res, const TVector<TString>& attributes, ui64 level, TExprNode::TPtr& state); + + IGraphTransformer::TStatus AfterResolveHandle(TExprContext& ctx, TYtKey::TWalkFoldersImplArgs args, TFolderQueueItem& folder, TExprNode::TPtr& state); + + IGraphTransformer::TStatus HandleAfterResolveFuture(TExprContext& ctx, const TYtKey::TWalkFoldersImplArgs& args, TFolderQueueItem& folder, TExprNode::TPtr& state); + + IGraphTransformer::TStatus DiveHandleInSingleFolder(TExprContext& ctx, const TYtKey::TWalkFoldersImplArgs& args, TFolderQueueItem& folder, TExprNode::TPtr& state); + + IGraphTransformer::TStatus AfterDiveHandle(TExprContext& ctx, TYtKey::TWalkFoldersImplArgs args, TFolderQueueItem& folder, TExprNode::TPtr& state); + + IGraphTransformer::TStatus PostHandleVisitedInSingleFolder(TExprContext& ctx, const TYtKey::TWalkFoldersImplArgs& args, TFolderQueueItem& folder, TExprNode::TPtr& state); + + IGraphTransformer::TStatus BuildFinishedState(TExprContext& ctx, const TYtKey::TWalkFoldersImplArgs& args, TExprNode::TPtr& state); +}; +} // namespace NYql diff --git a/yt/yql/providers/yt/provider/yql_yt_join_impl.cpp b/yt/yql/providers/yt/provider/yql_yt_join_impl.cpp new file mode 100644 index 0000000000..b40c14e5a3 --- /dev/null +++ b/yt/yql/providers/yt/provider/yql_yt_join_impl.cpp @@ -0,0 +1,4924 @@ +#include "yql_yt_join_impl.h" +#include "yql_yt_op_settings.h" +#include "yql_yt_helpers.h" +#include "yql_yt_table.h" + +#include <yt/yql/providers/yt/opt/yql_yt_join.h> +#include <yt/yql/providers/yt/lib/expr_traits/yql_expr_traits.h> +#include <yt/yql/providers/yt/common/yql_configuration.h> +#include <yql/essentials/providers/common/codec/yql_codec_type_flags.h> +#include <yql/essentials/core/yql_join.h> +#include <yql/essentials/core/yql_expr_type_annotation.h> +#include <yql/essentials/core/yql_expr_optimize.h> +#include <yql/essentials/core/yql_opt_utils.h> +#include <yql/essentials/core/yql_type_helpers.h> +#include <yql/essentials/utils/log/log.h> + +#include <util/string/join.h> +#include <util/string/cast.h> +#include <util/string/builder.h> +#include <util/generic/xrange.h> +#include <util/generic/algorithm.h> +#include <util/generic/yexception.h> +#include <util/generic/hash_set.h> +#include <util/generic/vector.h> +#include <util/generic/map.h> +#include <util/generic/size_literals.h> +#include <util/generic/strbuf.h> + +namespace NYql { + +using TStatus = IGraphTransformer::TStatus; + +namespace { + +TYtJoinNode::TPtr FindYtEquiJoinLeaf(const std::vector<TYtJoinNodeLeaf::TPtr>& leaves, TStringBuf table) { + for (auto& leaf : leaves) { + if (!leaf) { + continue; + } + + if (Find(leaf->Scope.begin(), leaf->Scope.end(), table) != leaf->Scope.end()) { + return leaf; + } + } + + ythrow yexception() << "Table " << TString{table}.Quote() << " not found"; +} + +void GatherEquiJoinKeyColumns(const TExprNode::TPtr& columns, THashSet<TString>& keyColumns) { + for (ui32 i = 0; i < columns->ChildrenSize(); i += 2) { + auto table = columns->Child(i)->Content(); + auto column = columns->Child(i + 1)->Content(); + keyColumns.insert({ FullColumnName(table, column) }); + } +} + +TYtJoinNodeOp::TPtr ImportYtEquiJoinRecursive(const TVector<TYtJoinNodeLeaf::TPtr>& leaves, const TYtJoinNodeOp* parent, + THashSet<TString>& drops, const TExprNode& joinTree, TExprContext& ctx) +{ + TYtJoinNodeOp::TPtr result(new TYtJoinNodeOp()); + result->Parent = parent; + result->JoinKind = joinTree.HeadPtr(); + result->LeftLabel = joinTree.ChildPtr(3); + result->RightLabel = joinTree.ChildPtr(4); + result->LinkSettings = GetEquiJoinLinkSettings(*joinTree.Child(5)); + + THashSet<TString> leftKeys, rightKeys; + GatherEquiJoinKeyColumns(result->LeftLabel, leftKeys); + GatherEquiJoinKeyColumns(result->RightLabel, rightKeys); + if (!result->JoinKind->IsAtom({"RightOnly", "RightSemi"})) { + for (const auto& key : leftKeys) { + if (drops.contains(key)) { + result->OutputRemoveColumns.insert(key); + } + } + } + + if (!result->JoinKind->IsAtom({"LeftOnly", "LeftSemi"})) { + for (const auto& key : rightKeys) { + if (drops.contains(key)) { + result->OutputRemoveColumns.insert(key); + } + } + } + + std::vector<std::string_view> lCheck, rCheck; + + lCheck.reserve(leftKeys.size()); + for (const auto& key : leftKeys) { + drops.erase(key); + lCheck.emplace_back(key); + } + + rCheck.reserve(rightKeys.size()); + for (const auto& key : rightKeys) { + drops.erase(key); + rCheck.emplace_back(key); + } + + result->Left = joinTree.Child(1)->IsAtom() ? + FindYtEquiJoinLeaf(leaves, joinTree.Child(1)->Content()): + ImportYtEquiJoinRecursive(leaves, result.Get(), drops, *joinTree.Child(1), ctx); + + result->Right = joinTree.Child(2)->IsAtom() ? + FindYtEquiJoinLeaf(leaves, joinTree.Child(2)->Content()): + ImportYtEquiJoinRecursive(leaves, result.Get(), drops, *joinTree.Child(2), ctx); + + result->Scope.insert(result->Scope.end(), result->Left->Scope.cbegin(), result->Left->Scope.cend()); + result->Scope.insert(result->Scope.end(), result->Right->Scope.cbegin(), result->Right->Scope.cend()); + + const std::string_view joinKind = result->JoinKind->Content(); + const bool singleSide = joinKind.ends_with("Only") || joinKind.ends_with("Semi"); + const bool rightSide = joinKind.starts_with("Right"); + const bool leftSide = joinKind.starts_with("Left"); + + const auto lUnique = result->Left->Constraints.GetConstraint<TUniqueConstraintNode>(); + const auto rUnique = result->Right->Constraints.GetConstraint<TUniqueConstraintNode>(); + + const bool lAny = result->LinkSettings.LeftHints.contains("unique") || result->LinkSettings.LeftHints.contains("any"); + const bool rAny = result->LinkSettings.RightHints.contains("unique") || result->LinkSettings.RightHints.contains("any"); + + const bool lOneRow = lAny || lUnique && lUnique->ContainsCompleteSet(lCheck); + const bool rOneRow = rAny || rUnique && rUnique->ContainsCompleteSet(rCheck); + + if (singleSide) { + if (leftSide) + result->Constraints.AddConstraint(lUnique); + else if (rightSide) + result->Constraints.AddConstraint(rUnique); + } else if (!result->JoinKind->IsAtom("Cross")) { + const bool exclusion = result->JoinKind->IsAtom("Exclusion"); + const bool useLeft = lUnique && (rOneRow || exclusion); + const bool useRight = rUnique && (lOneRow || exclusion); + + if (useLeft && !useRight) + result->Constraints.AddConstraint(lUnique); + else if (useRight && !useLeft) + result->Constraints.AddConstraint(rUnique); + else if (useLeft && useRight) + result->Constraints.AddConstraint(TUniqueConstraintNode::Merge(lUnique, rUnique, ctx)); + } + + const auto lDistinct = result->Left->Constraints.GetConstraint<TDistinctConstraintNode>(); + const auto rDistinct = result->Right->Constraints.GetConstraint<TDistinctConstraintNode>(); + + if (singleSide) { + if (leftSide) + result->Constraints.AddConstraint(lDistinct); + else if (rightSide) + result->Constraints.AddConstraint(rDistinct); + } else if (!result->JoinKind->IsAtom("Cross")) { + const bool inner = result->JoinKind->IsAtom("Inner"); + const bool useLeft = lDistinct && rOneRow && (inner || leftSide); + const bool useRight = rDistinct && lOneRow && (inner || rightSide); + + if (useLeft && !useRight) + result->Constraints.AddConstraint(lDistinct); + else if (useRight && !useLeft) + result->Constraints.AddConstraint(rDistinct); + else if (useLeft && useRight) + result->Constraints.AddConstraint(TDistinctConstraintNode::Merge(lDistinct, rDistinct, ctx)); + } + + const auto lEmpty = result->Left->Constraints.GetConstraint<TEmptyConstraintNode>(); + const auto rEmpty = result->Right->Constraints.GetConstraint<TEmptyConstraintNode>(); + + if (lEmpty || rEmpty) { + if (result->JoinKind->IsAtom({"Inner", "LeftSemi", "RightSemi", "Cross"})) + result->Constraints.AddConstraint(ctx.MakeConstraint<TEmptyConstraintNode>()); + else if (leftSide && lEmpty) + result->Constraints.AddConstraint(lEmpty); + else if (rightSide && rEmpty) + result->Constraints.AddConstraint(rEmpty); + else if (lEmpty && rEmpty) + result->Constraints.AddConstraint(ctx.MakeConstraint<TEmptyConstraintNode>()); + } + + return result; +} + +bool IsEffectivelyUnique(const TEquiJoinLinkSettings& linkSettings, const TMapJoinSettings& settings, bool isLeft) { + auto& hints = isLeft ? linkSettings.LeftHints : linkSettings.RightHints; + return hints.contains("unique") || hints.contains("any") || (isLeft ? settings.LeftUnique : settings.RightUnique); +} + +bool HasNonTrivialAny(const TEquiJoinLinkSettings& linkSettings, const TMapJoinSettings& settings, TChoice side) { + YQL_ENSURE(IsLeftOrRight(side)); + auto& hints = (side == TChoice::Left) ? linkSettings.LeftHints : linkSettings.RightHints; + bool unique = ((side == TChoice::Left) ? settings.LeftUnique : settings.RightUnique) || hints.contains("unique"); + return hints.contains("any") && !unique; +} + +IGraphTransformer::TStatus TryEstimateDataSizeChecked(TVector<ui64>& result, TYtSection& inputSection, const TString& cluster, + const TVector<TYtPathInfo::TPtr>& paths, const TMaybe<TVector<TString>>& columns, const TYtState& state, TExprContext& ctx) +{ + if (GetJoinCollectColumnarStatisticsMode(*state.Configuration) == EJoinCollectColumnarStatisticsMode::Sync) { + auto syncResult = EstimateDataSize(cluster, paths, columns, state, ctx); + if (!syncResult) { + return IGraphTransformer::TStatus::Error; + } + result = std::move(*syncResult); + return IGraphTransformer::TStatus::Ok; + } + + TSet<TString> requestedColumns; + auto status = TryEstimateDataSize(result, requestedColumns, cluster, paths, columns, state, ctx); + auto settings = inputSection.Settings().Ptr(); + if (status == TStatus::Repeat) { + bool hasStatColumns = NYql::HasSetting(inputSection.Settings().Ref(), EYtSettingType::StatColumns); + if (hasStatColumns) { + auto oldColumns = NYql::GetSettingAsColumnList(*settings, EYtSettingType::StatColumns); + TSet<TString> oldColumnSet(oldColumns.begin(), oldColumns.end()); + + bool alreadyRequested = AllOf(requestedColumns, [&](const auto& c) { + return oldColumnSet.contains(c); + }); + + YQL_ENSURE(!alreadyRequested); + + settings = NYql::RemoveSetting(*settings, EYtSettingType::StatColumns, ctx); + } + + YQL_CLOG(INFO, ProviderYt) << "Stat missing for columns: " << JoinSeq(", ", requestedColumns) << ", rebuilding section"; + TVector<TString> requestedColumnList(requestedColumns.begin(), requestedColumns.end()); + + inputSection = Build<TYtSection>(ctx, inputSection.Ref().Pos()) + .InitFrom(inputSection) + .Settings(NYql::AddSettingAsColumnList(*settings, EYtSettingType::StatColumns, requestedColumnList, ctx)) + .Done(); + } + return status; +} + +TStatus UpdateInMemorySizeSetting(TMapJoinSettings& settings, TYtSection& inputSection, const TJoinLabels& labels, + const TYtJoinNodeOp& op, TExprContext& ctx, bool isLeft, + const TStructExprType* itemType, const TVector<TString>& joinKeyList, const TYtState::TPtr& state, const TString& cluster, + const TVector<TYtPathInfo::TPtr>& tables, bool mapJoinUseFlow) +{ + ui64 size = 0; + auto status = CalculateJoinLeafSize(size, settings, inputSection, op, ctx, isLeft, itemType, joinKeyList, state, cluster, tables); + if (status != TStatus::Ok) { + return status; + } + + const ui64 rows = isLeft ? settings.LeftRows : settings.RightRows; + const auto& label = labels.Inputs[isLeft ? 0 : 1]; + const bool needPayload = op.JoinKind->IsAtom("Inner") || op.JoinKind->IsAtom(isLeft ? "Right" : "Left"); + + ui64 result = 0; + if (op.JoinKind->IsAtom("Cross")) { + if (mapJoinUseFlow) { + result = size + rows * (1ULL + label.InputType->GetSize()) * sizeof(NKikimr::NUdf::TUnboxedValuePod); // Table content after Collect + } else { + ui64 avgOtherSideWeight = (isLeft ? settings.RightSize : settings.LeftSize) / (isLeft ? settings.RightRows : settings.LeftRows); + + ui64 rowFactor = (1 + label.InputType->GetSize()) * sizeof(NKikimr::NUdf::TUnboxedValuePod); // Table content after Collect + rowFactor += (1 + label.InputType->GetSize() + labels.Inputs[isLeft ? 1 : 0].InputType->GetSize()) * sizeof(NKikimr::NUdf::TUnboxedValuePod); // Table content after Map with added left side + rowFactor += avgOtherSideWeight; // Average added left side for each row after Map + + result = 2 * size + rows * rowFactor; + } + } else { + result = CalcInMemorySizeNoCrossJoin(labels.Inputs[isLeft ? 0 : 1], op, + settings, isLeft, ctx, needPayload, size); + } + + (isLeft ? settings.LeftMemSize : settings.RightMemSize) = result; + return TStatus::Ok; +} + +TYtJoinNodeLeaf::TPtr ConvertYtEquiJoinToLeaf(const TYtJoinNodeOp& op, TPositionHandle pos, TExprContext& ctx) { + auto joinLabelBuilder = Build<TCoAtomList>(ctx, pos); + for (auto& x : op.Scope) { + joinLabelBuilder.Add().Value(x).Build(); + } + auto label = joinLabelBuilder.Done().Ptr(); + auto section = Build<TYtSection>(ctx, pos) + .Paths() + .Add() + .Table<TYtOutput>() + .Operation(op.Output.Cast()) + .OutIndex().Value("0").Build() + .Build() + .Columns<TCoVoid>().Build() + .Ranges<TCoVoid>().Build() + .Stat<TCoVoid>().Build() + .Build() + .Build() + .Settings() + .Add() + .Name() + .Value(ToString(EYtSettingType::JoinLabel)) + .Build() + .Value(label) + .Build() + .Build() + .Done(); + + TYtJoinNodeLeaf::TPtr leaf = MakeIntrusive<TYtJoinNodeLeaf>(section, TMaybeNode<TCoLambda>()); + leaf->Scope = op.Scope; + leaf->Label = label; + return leaf; +} + +void UpdateSortPrefix(bool initial, TVector<TString>& sortedKeys, const TYqlRowSpecInfo::TPtr& rowSpec, + bool& areUniqueKeys, const THashSet<TString>& keySet, TMaybeNode<TCoLambda> premap) +{ + if (!rowSpec) { + areUniqueKeys = false; + sortedKeys.clear(); + return; + } + if (initial) { + sortedKeys = rowSpec->SortedBy; + TMaybe<THashSet<TStringBuf>> passthroughFields; + bool isPassthrough = true; + + if (premap) { + isPassthrough = IsPassthroughLambda(premap.Cast(), &passthroughFields); + } + + for (size_t pos = 0; isPassthrough && pos < rowSpec->SortDirections.size(); ++pos) { + if (!rowSpec->SortDirections[pos] || + (premap && passthroughFields && !(*passthroughFields).contains(rowSpec->SortedBy[pos]))) + { + sortedKeys.resize(pos); + break; + } + } + areUniqueKeys = areUniqueKeys && (rowSpec->SortedBy.size() == sortedKeys.size()); + } + else { + size_t newPrefixLength = Min(sortedKeys.size(), rowSpec->SortedBy.size()); + for (size_t pos = 0; pos < newPrefixLength; ++pos) { + if (sortedKeys[pos] != rowSpec->SortedBy[pos] || !rowSpec->SortDirections[pos]) { + newPrefixLength = pos; + break; + } + } + + areUniqueKeys = areUniqueKeys && (newPrefixLength == sortedKeys.size()); + sortedKeys.resize(newPrefixLength); + } + + areUniqueKeys = areUniqueKeys && rowSpec->UniqueKeys && sortedKeys.size() == keySet.size(); + ui32 foundKeys = 0; + for (auto key : sortedKeys) { + if (keySet.contains(key)) { + ++foundKeys; + } else { + break; + } + } + + if (foundKeys != keySet.size()) { + areUniqueKeys = false; + sortedKeys.clear(); + } +} + +TVector<TString> BuildCompatibleSortWith(const TVector<TString>& otherSortedKeys, const TVector<TString>& otherJoinKeyList, + const TVector<TString>& myJoinKeyList) +{ + YQL_ENSURE(myJoinKeyList.size() == otherJoinKeyList.size()); + YQL_ENSURE(otherSortedKeys.size() >= otherJoinKeyList.size()); + + THashMap<TString, size_t> otherJoinListPos; + for (size_t i = 0; i < otherJoinKeyList.size(); ++i) { + otherJoinListPos[otherJoinKeyList[i]] = i; + } + + YQL_ENSURE(otherJoinListPos.size() == otherJoinKeyList.size()); + + + TVector<TString> mySortedKeys; + mySortedKeys.reserve(myJoinKeyList.size()); + + for (size_t i = 0; i < Min(myJoinKeyList.size(), otherSortedKeys.size()); ++i) { + auto otherKey = otherSortedKeys[i]; + auto joinPos = otherJoinListPos.find(otherKey); + YQL_ENSURE(joinPos != otherJoinListPos.end()); + + mySortedKeys.push_back(myJoinKeyList[joinPos->second]); + } + + + return mySortedKeys; +} + + +TVector<TString> BuildCommonSortPrefix(const TVector<TString>& leftSortedKeys, const TVector<TString>& rightSortedKeys, + const TVector<TString>& leftJoinKeyList, const TVector<TString>& rightJoinKeyList, + THashMap<TString, TString>& leftKeyRenames, THashMap<TString, TString>& rightKeyRenames, + bool allowColumnRenames) +{ + TVector<TString> result; + + YQL_ENSURE(leftJoinKeyList.size() == rightJoinKeyList.size()); + size_t joinSize = leftJoinKeyList.size(); + + THashMap<TString, size_t> leftJoinListPos; + THashMap<TString, size_t> rightJoinListPos; + if (allowColumnRenames) { + for (size_t i = 0; i < joinSize; ++i) { + leftJoinListPos[leftJoinKeyList[i]] = i; + rightJoinListPos[rightJoinKeyList[i]] = i; + } + } + + for (size_t pos = 0; pos < Min(leftSortedKeys.size(), rightSortedKeys.size(), joinSize); ++pos) { + auto left = leftSortedKeys[pos]; + auto right = rightSortedKeys[pos]; + if (left == right) { + result.push_back(left); + continue; + } + + if (!allowColumnRenames) { + break; + } + + auto l = leftJoinListPos.find(left); + YQL_ENSURE(l != leftJoinListPos.end()); + auto leftPos = l->second; + + auto r = rightJoinListPos.find(right); + YQL_ENSURE(r != rightJoinListPos.end()); + auto rightPos = r->second; + + if (leftPos != rightPos) { + break; + } + + TString rename = TStringBuilder() << "_yql_join_renamed_column_" << leftPos; + leftKeyRenames[left] = rename; + rightKeyRenames[right] = rename; + + result.push_back(rename); + } + + return result; +} + +TYtSection SectionApplyRenames(const TYtSection& section, const THashMap<TString, TString>& renames, TExprContext& ctx) { + if (!renames) { + return section; + } + + TVector<TYtPath> updatedPaths; + for (size_t p = 0; p < section.Paths().Size(); ++p) { + auto path = section.Paths().Item(p); + + TYtColumnsInfo columnsInfo(path.Columns()); + columnsInfo.SetRenames(renames); + + updatedPaths.push_back(Build<TYtPath>(ctx, path.Pos()) + .InitFrom(path) + .Columns(columnsInfo.ToExprNode(ctx, path.Columns().Pos())) + .Done() + ); + } + + return Build<TYtSection>(ctx, section.Pos()) + .InitFrom(section) + .Paths() + .Add(updatedPaths) + .Build() + .Done(); +} + +TVector<TString> KeysApplyInputRenames(const TVector<TString>& keys, const THashMap<TString, TString>& renames) { + auto result = keys; + for (auto& i : result) { + auto r = renames.find(i); + if (r != renames.end()) { + i = r->second; + } + } + return result; +} + +TExprNode::TPtr BuildReverseRenamingLambda(TPositionHandle pos, const THashMap<TString, TString>& renames, TExprContext& ctx) { + auto arg = ctx.NewArgument(pos, "item"); + auto body = arg; + + for (auto& r : renames) { + auto from = r.second; + auto to = r.first; + body = ctx.Builder(body->Pos()) + .Callable("RemoveMember") + .Callable(0, "AddMember") + .Add(0, body) + .Atom(1, to) + .Callable(2, "Member") + .Add(0, body) + .Atom(1, from) + .Seal() + .Seal() + .Atom(1, from) + .Seal() + .Build(); + } + + return ctx.NewLambda(pos, ctx.NewArguments(pos, { arg }), std::move(body)); +} + +TMaybeNode<TCoLambda> GetPremapLambda(const TYtJoinNodeLeaf& leaf) { + return leaf.Premap; +} + +ETypeAnnotationKind DeriveCommonSequenceKind(ETypeAnnotationKind one, ETypeAnnotationKind two) { + if (one == ETypeAnnotationKind::Stream || two == ETypeAnnotationKind::Stream) { + return ETypeAnnotationKind::Stream; + } + + if (one == ETypeAnnotationKind::List || two == ETypeAnnotationKind::List) { + return ETypeAnnotationKind::List; + } + + return ETypeAnnotationKind::Optional; +} + +void ApplyInputPremap(TExprNode::TPtr& lambda, const TMaybeNode<TCoLambda>& premap, ETypeAnnotationKind commonKind, + const THashMap<TString, TString>& renames, TExprContext& ctx) +{ + TStringBuf converter; + if (commonKind == ETypeAnnotationKind::Stream) { + converter = "ToStream"; + } else if (commonKind == ETypeAnnotationKind::List) { + converter = "ToList"; + } else if (commonKind == ETypeAnnotationKind::Optional) { + converter = "ToSequence"; + } else { + YQL_ENSURE(false, "unexpected common kind"); + } + + auto reverseRenamingLambda = BuildReverseRenamingLambda(lambda->Pos(), renames, ctx); + + if (premap) { + lambda = ctx.Builder(lambda->Pos()) + .Lambda() + .Param("item") + .Callable(converter) + .Callable(0, "FlatMap") + .Apply(0, premap.Cast().Ptr()) + .With(0) + .Apply(reverseRenamingLambda) + .With(0, "item") + .Seal() + .Done() + .Seal() + .Add(1, lambda) + .Seal() + .Seal() + .Seal() + .Build(); + } else { + lambda = ctx.Builder(lambda->Pos()) + .Lambda() + .Param("item") + .Callable(converter) + .Apply(0, lambda) + .With(0) + .Apply(reverseRenamingLambda) + .With(0, "item") + .Seal() + .Done() + .Seal() + .Seal() + .Seal() + .Build(); + } +} + +void ApplyInputPremap(TExprNode::TPtr& lambda, const TYtJoinNodeLeaf& leaf, + const TYtJoinNodeLeaf& otherLeaf, TExprContext& ctx, + const THashMap<TString, TString>& renames = THashMap<TString, TString>()) +{ + auto premap = GetPremapLambda(leaf); + auto otherPremap = GetPremapLambda(otherLeaf); + + auto commonKind = ETypeAnnotationKind::Optional; + if (premap) { + YQL_ENSURE(EnsureSeqOrOptionalType(premap.Cast().Ref(), ctx)); + commonKind = DeriveCommonSequenceKind(commonKind, premap.Cast().Ref().GetTypeAnn()->GetKind()); + } + if (otherPremap) { + YQL_ENSURE(EnsureSeqOrOptionalType(otherPremap.Cast().Ref(), ctx)); + commonKind = DeriveCommonSequenceKind(commonKind, otherPremap.Cast().Ref().GetTypeAnn()->GetKind()); + } + + ApplyInputPremap(lambda, premap, commonKind, renames, ctx); +} + + + +TString RenamesToString(const THashMap<TString, TString>& renames) { + TVector<TString> renamesStr; + renamesStr.reserve(renames.size()); + + for (auto& r : renames) { + renamesStr.emplace_back(r.first + " -> " + r.second); + } + + return JoinSeq(", ", renamesStr); +} + +struct TEquivKeys { + TSet<TString> Keys; + const TTypeAnnotationNode* Type = nullptr; +}; + +TTypeAnnotationNode::TListType GetTypes(const TVector<TEquivKeys>& input) { + TTypeAnnotationNode::TListType result; + result.reserve(input.size()); + for (auto& i : input) { + result.push_back(i.Type); + } + return result; +} + +void FlattenKeys(TVector<TString>& keys, TTypeAnnotationNode::TListType& types, const TVector<TEquivKeys>& input) { + for (auto& i : input) { + auto type = i.Type; + for (auto& key : i.Keys) { + keys.push_back(key); + types.push_back(type); + } + } +} + +void LogOutputSideSort(const TVector<TEquivKeys>& outSort, bool isLeft) { + TVector<TString> strs; + for (auto& s : outSort) { + strs.push_back(TString::Join("{", JoinSeq(", ", s.Keys), "}")); + } + YQL_CLOG(INFO, ProviderYt) << "Output sort for " << (isLeft ? "left" : "right") + << " side: [" << JoinSeq(", ", strs) << "]"; +} + +void BuildOutputSideSort(TVector<TEquivKeys>& outSort, bool isLeft, TStringBuf joinType, + const TStructExprType& outType, const TMap<TStringBuf, TVector<TStringBuf>>& renameMap, + const TJoinLabel& label, const TVector<TString>& inputSort) +{ + YQL_ENSURE(joinType != "Cross"); + + if (joinType == "Exclusion" || joinType == "Full") { + // output is not sorted for either side + return; + } + + if (isLeft && (joinType == "Right" || joinType == "RightOnly" || joinType == "RightSemi")) { + // no data or not sorted + return; + } + + if (!isLeft && (joinType == "Left" || joinType == "LeftOnly" || joinType == "LeftSemi")) { + // no data or not sorted + return; + } + + for (auto& key : inputSort) { + auto name = label.FullName(key); + + TVector<TStringBuf> newNames; + newNames.push_back(name); + if (auto renamed = renameMap.FindPtr(name)) { + newNames = *renamed; + } + + if (newNames.empty()) { + // stop on first deleted sort key + return; + } + + TEquivKeys keys; + + for (auto n : newNames) { + auto maybeIdx = outType.FindItem(n); + if (!maybeIdx.Defined()) { + return; + } + + bool inserted = keys.Keys.insert(ToString(n)).second; + YQL_ENSURE(inserted, "Duplicate key: " << n); + + if (!keys.Type) { + keys.Type = outType.GetItems()[*maybeIdx]->GetItemType(); + } else { + YQL_ENSURE(keys.Type == outType.GetItems()[*maybeIdx]->GetItemType()); + } + } + + outSort.emplace_back(std::move(keys)); + } +} + +TVector<TString> MatchSort(const THashSet<TString>& desiredKeys, const TVector<TEquivKeys>& sideSort) { + TVector<TString> result; + + THashSet<TString> matchedKeys; + for (auto& k : sideSort) { + // only single alternative + YQL_ENSURE(k.Keys.size() == 1); + auto key = *k.Keys.begin(); + if (desiredKeys.contains(key)) { + matchedKeys.insert(key); + result.push_back(key); + } else { + break; + } + } + + if (matchedKeys.size() != desiredKeys.size()) { + result.clear(); + } + + return result; +} + +TVector<TStringBuf> MatchSort(TTypeAnnotationNode::TListType& types, const TVector<TStringBuf>& desiredSort, const TVector<TEquivKeys>& sideSort) { + TVector<TStringBuf> result; + types.clear(); + for (size_t i = 0, j = 0; i < desiredSort.size() && j < sideSort.size(); ++j) { + auto key = desiredSort[i++]; + if (!sideSort[j].Keys.contains(key)) { + break; + } + while (true) { + result.push_back(key); + types.push_back(sideSort[j].Type); + if (i >= desiredSort.size()) { + break; + } + key = desiredSort[i]; + if (!sideSort[j].Keys.contains(key)) { + break; + } + i++; + } + } + + if (result.size() != desiredSort.size()) { + result.clear(); + types.clear(); + } + + return result; +} + +size_t GetSortWeight(TMap<TVector<TStringBuf>, size_t>& weights, const TVector<TStringBuf>& sort, const TSet<TVector<TStringBuf>>& sorts) { + if (sort.empty() || !sorts.contains(sort)) { + return 0; + } + + if (auto w = weights.FindPtr(sort)) { + return *w; + } + + size_t weight = 1; + auto currSort = sort; + for (size_t i = currSort.size() - 1; i > 0; --i) { + currSort.resize(i); + auto currWeight = GetSortWeight(weights, currSort, sorts); + if (currWeight) { + weight += currWeight; + break; + } + } + + weights[sort] = weight; + return weight; +} + + +void BuildOutputSort(const TYtOutTableInfo& outTableInfo, const TYtJoinNodeOp& op, TStringBuf joinType, bool setTopLevelFullSort, + const TStructExprType& outItemType, const TMap<TStringBuf, TVector<TStringBuf>>& renameMap, + const TSet<TVector<TStringBuf>>& topLevelSorts, + const TVector<TString>& leftSortedKeys, const TVector<TString>& rightSortedKeys, + const TJoinLabel& leftLabel, const TJoinLabel& rightLabel) +{ + { + TVector<TString> strs; + for (auto& r : renameMap) { + strs.push_back(TString::Join(r.first, " -> {", JoinSeq(", ", r.second), "}")); + } + YQL_CLOG(INFO, ProviderYt) << "Join renames: [" << JoinSeq(", ", strs) << "]"; + } + + { + TVector<TString> strs; + for (auto& s: topLevelSorts) { + strs.push_back(TString::Join("{", JoinSeq(", ", s), "}")); + } + YQL_CLOG(INFO, ProviderYt) << "Top level sorts: [" << JoinSeq(", ", strs) << "]"; + } + + YQL_CLOG(INFO, ProviderYt) << "Output Type: " << static_cast<const TTypeAnnotationNode&>(outItemType); + + YQL_CLOG(INFO, ProviderYt) << "Deriving output sort order for " << (op.Parent ? "intermediate " : "final ") + << joinType << " join. Input left side sort: [" << JoinSeq(", ", leftSortedKeys) + << "], input right side sort: [" << JoinSeq(", ", rightSortedKeys) << "]"; + + TVector<TEquivKeys> outLeftSorted; + BuildOutputSideSort(outLeftSorted, true, joinType, outItemType, renameMap, leftLabel, leftSortedKeys); + LogOutputSideSort(outLeftSorted, true); + + TVector<TEquivKeys> outRightSorted; + BuildOutputSideSort(outRightSorted, false, joinType, outItemType, renameMap, rightLabel, rightSortedKeys); + LogOutputSideSort(outRightSorted, false); + + TVector<TString> outSorted; + TTypeAnnotationNode::TListType outSortTypes; + + if (op.Parent) { + if (op.Parent->JoinKind->Content() != "Cross") { + THashSet<TString> desiredOutputSortKeys; + YQL_ENSURE(&op == op.Parent->Left.Get() || &op == op.Parent->Right.Get()); + auto parentLabel = (&op == op.Parent->Left.Get()) ? op.Parent->LeftLabel : op.Parent->RightLabel; + for (ui32 i = 0; i < parentLabel->ChildrenSize(); i += 2) { + auto table = parentLabel->Child(i)->Content(); + auto column = parentLabel->Child(i + 1)->Content(); + desiredOutputSortKeys.insert(FullColumnName(table, column)); + } + + YQL_ENSURE(!desiredOutputSortKeys.empty()); + + YQL_CLOG(INFO, ProviderYt) << "Desired output sort keys for next join: [" << JoinSeq(", ", desiredOutputSortKeys) << "]"; + + THashSet<TString> matchedSortKeys; + + auto leftMatch = MatchSort(desiredOutputSortKeys, outLeftSorted); + auto rightMatch = MatchSort(desiredOutputSortKeys, outRightSorted); + + // choose longest sort + if (leftMatch.size() > rightMatch.size() || (leftMatch.size() == rightMatch.size() && leftMatch <= rightMatch)) { + outSorted = leftMatch; + outSortTypes = GetTypes(outLeftSorted); + } else { + outSorted = rightMatch; + outSortTypes = GetTypes(outRightSorted); + } + } + } else if (setTopLevelFullSort) { + FlattenKeys(outSorted, outSortTypes, outLeftSorted); + FlattenKeys(outSorted, outSortTypes, outRightSorted); + } else { + TMap<TVector<TStringBuf>, size_t> weights; + + size_t bestWeight = 0; + TVector<TStringBuf> bestSort; + + for (auto& requestedSort : topLevelSorts) { + TTypeAnnotationNode::TListType leftMatchTypes; + auto leftMatch = MatchSort(leftMatchTypes, requestedSort, outLeftSorted); + auto weight = GetSortWeight(weights, leftMatch, topLevelSorts); + if (weight > bestWeight) { + bestWeight = weight; + bestSort = leftMatch; + outSortTypes = leftMatchTypes; + } + + TTypeAnnotationNode::TListType rightMatchTypes; + auto rightMatch = MatchSort(rightMatchTypes, requestedSort, outRightSorted); + weight = GetSortWeight(weights, rightMatch, topLevelSorts); + if (weight > bestWeight) { + bestWeight = weight; + bestSort = rightMatch; + outSortTypes = rightMatchTypes; + } + } + + outSorted.clear(); + outSorted.reserve(bestSort.size()); + for (auto& k : bestSort) { + outSorted.push_back(ToString(k)); + } + } + + YQL_ENSURE(outSortTypes.size() >= outSorted.size()); + outSortTypes.resize(outSorted.size()); + if (!outSorted.empty()) { + outTableInfo.RowSpec->SortMembers = outSorted; + outTableInfo.RowSpec->SortedBy = outSorted; + outTableInfo.RowSpec->SortedByTypes = outSortTypes; + outTableInfo.RowSpec->SortDirections = TVector<bool>(outSorted.size(), true); + } + + YQL_CLOG(INFO, ProviderYt) << "Resulting output sort: [" << JoinSeq(", ", outTableInfo.RowSpec->SortedBy) << "]"; +} + +const TStructExprType* MakeIntermediateEquiJoinTableType(TPositionHandle pos, const TExprNode& joinTree, + const TJoinLabels& labels, const THashSet<TString>& outputRemoveColumns, TExprContext& ctx) +{ + auto reduceColumnTypes = GetJoinColumnTypes(joinTree, labels, ctx); + TVector<const TItemExprType*> structItems; + for (auto& x : reduceColumnTypes) { + if (!outputRemoveColumns.contains(x.first)) { + structItems.push_back(ctx.MakeType<TItemExprType>(x.first, x.second)); + } + } + + auto reduceResultType = ctx.MakeType<TStructExprType>(structItems); + if (!reduceResultType->Validate(pos, ctx)) { + return nullptr; + } + + return reduceResultType; +} + +void AddAnyJoinOptionsToCommonJoinCore(TExprNode::TListType& options, bool swapTables, const TEquiJoinLinkSettings& linkSettings, TPositionHandle pos, TExprContext& ctx) { + TExprNode::TListType anySettings; + + const auto& leftHints = swapTables ? linkSettings.RightHints : linkSettings.LeftHints; + const auto& rightHints = swapTables ? linkSettings.LeftHints : linkSettings.RightHints; + + if (leftHints.contains("any")) { + anySettings.push_back(ctx.NewAtom(pos, "left", TNodeFlags::Default)); + } + + if (rightHints.contains("any")) { + anySettings.push_back(ctx.NewAtom(pos, "right", TNodeFlags::Default)); + } + + if (!anySettings.empty()) { + options.push_back( + ctx.Builder(pos) + .List() + .Atom(0, "any", TNodeFlags::Default) + .Add(1, ctx.NewList(pos, std::move(anySettings))) + .Seal() + .Build()); + } +} + +TExprNode::TPtr BuildYtReduceLambda(TPositionHandle pos, const TExprNode::TPtr& groupArg, TExprNode::TPtr&& flatMapLambdaBody, const bool sysColumns, TExprContext& ctx) +{ + TExprNode::TPtr chopperHandler = ctx.NewLambda(pos, ctx.NewArguments(pos, {ctx.NewArgument(pos, "stup"), groupArg }), std::move(flatMapLambdaBody)); + TExprNode::TPtr chopperSwitch; + + if (sysColumns) { + chopperSwitch = ctx.Builder(pos) + .Lambda() + .Param("key") + .Param("item") + .Callable("SqlExtractKey") + .Arg(0, "item") + .Lambda(1) + .Param("row") + .Callable("Member") + .Arg(0, "row") + .Atom(1, YqlSysColumnKeySwitch, TNodeFlags::Default) + .Seal() + .Seal() + .Seal() + .Seal() + .Build(); + + chopperHandler = ctx.Builder(pos) + .Lambda() + .Param("key") + .Param("group") + .Apply(chopperHandler) + .With(0, "key") + .With(1) + .Callable("RemovePrefixMembers") + .Arg(0, "group") + .List(1) + .Atom(0, YqlSysColumnKeySwitch, TNodeFlags::Default) + .Seal() + .Seal() + .Done() + .Seal() + .Seal() + .Build(); + } + else { + chopperSwitch = ctx.Builder(pos) + .Lambda() + .Param("key") + .Param("item") + .Callable("YtIsKeySwitch") + .Callable(0, "DependsOn") + .Arg(0, "item") + .Seal() + .Seal() + .Seal() + .Build(); + } + + return ctx.Builder(pos) + .Lambda() + .Param("flow") + .Callable("Chopper") + .Arg(0, "flow") + .Lambda(1) + .Param("item") + .Callable("Uint64") // Fake const key + .Atom(0, "0", TNodeFlags::Default) + .Seal() + .Seal() + .Add(2, chopperSwitch) + .Add(3, chopperHandler) + .Seal() + .Seal() + .Build(); +} + +struct TMergeJoinSortInfo { + TChoice AdditionalSort = TChoice::None; + TChoice NeedRemapBeforeSort = TChoice::None; + + const TStructExprType* LeftBeforePremap = nullptr; + const TStructExprType* RightBeforePremap = nullptr; + + TVector<TString> LeftSortedKeys; + TVector<TString> RightSortedKeys; + + TVector<TString> CommonSortedKeys; + + THashMap<TString, TString> LeftKeyRenames; + THashMap<TString, TString> RightKeyRenames; +}; + +TMergeJoinSortInfo Invert(const TMergeJoinSortInfo& info) { + auto result = info; + result.AdditionalSort = Invert(result.AdditionalSort); + result.NeedRemapBeforeSort = Invert(result.NeedRemapBeforeSort); + std::swap(result.LeftBeforePremap, result.RightBeforePremap); + std::swap(result.LeftSortedKeys, result.RightSortedKeys); + std::swap(result.LeftKeyRenames, result.RightKeyRenames); + return result; +} + +TYtSection SectionApplyAdditionalSort(const TYtSection& section, const TYtEquiJoin& equiJoin, const TVector<TString>& sortTableOrder, const TStructExprType* sortTableType, + bool needRemapBeforeSort, const TYtState& state, TExprContext& ctx) +{ + auto pos = equiJoin.Pos(); + auto inputWorld = equiJoin.World(); + auto inputSection = section; + + TTypeAnnotationNode::TListType sortedByTypes; + for (auto& column: sortTableOrder) { + auto ndx = sortTableType->FindItem(column); + YQL_ENSURE(ndx.Defined(), "Missing column " << column); + sortedByTypes.push_back(sortTableType->GetItems()[*ndx]->GetItemType()); + } + + TVector<bool> sortDirections(sortTableOrder.size(), true); + ui64 nativeTypeFlags = state.Configuration->UseNativeYtTypes.Get().GetOrElse(DEFAULT_USE_NATIVE_YT_TYPES) ? NTCF_ALL : NTCF_NONE; + TMaybe<NYT::TNode> nativeType; + + if (needRemapBeforeSort) { + inputSection = Build<TYtSection>(ctx, pos) + .Paths() + .Add() + .Table<TYtOutput>() + .Operation<TYtMap>() + .World(inputWorld) + .DataSink(equiJoin.DataSink()) + .Input() + .Add(inputSection) + .Build() + .Output() + .Add(TYtOutTableInfo(sortTableType, state.Configuration->UseNativeYtTypes.Get().GetOrElse(DEFAULT_USE_NATIVE_YT_TYPES) ? NTCF_ALL : NTCF_NONE) + .ToExprNode(ctx, pos).Cast<TYtOutTable>()) + .Build() + .Settings(GetFlowSettings(pos, state, ctx)) + .Mapper() + .Args({"list"}) + .Body("list") + .Build() + .Build() + .OutIndex().Value("0").Build() + .Build() + .Columns<TCoVoid>().Build() + .Ranges<TCoVoid>().Build() + .Stat<TCoVoid>().Build() + .Build() + .Build() + .Settings().Build() + .Done(); + + inputWorld = Build<TCoWorld>(ctx, pos).Done(); + } else { + auto inputRowSpec = TYtTableBaseInfo::GetRowSpec(section.Paths().Item(0).Table()); + // Use types from first input only, because all of them shoud be equal (otherwise remap is required) + nativeTypeFlags = inputRowSpec->GetNativeYtTypeFlags(); + nativeType = inputRowSpec->GetNativeYtType(); + } + + TYtOutTableInfo sortOut(sortTableType, nativeTypeFlags); + sortOut.RowSpec->SortMembers = sortTableOrder; + sortOut.RowSpec->SortedBy = sortTableOrder; + sortOut.RowSpec->SortedByTypes = sortedByTypes; + sortOut.RowSpec->SortDirections = sortDirections; + + if (nativeType) { + sortOut.RowSpec->CopyTypeOrders(*nativeType); + } + + return Build<TYtSection>(ctx, pos) + .Paths() + .Add() + .Table<TYtOutput>() + .Operation<TYtSort>() + .World(inputWorld) + .DataSink(equiJoin.DataSink()) + .Input() + .Add(inputSection) + .Build() + .Output() + .Add(sortOut.ToExprNode(ctx, pos).Cast<TYtOutTable>()) + .Build() + .Settings().Build() + .Build() + .OutIndex().Value("0").Build() + .Build() + .Columns<TCoVoid>().Build() + .Ranges<TCoVoid>().Build() + .Stat<TCoVoid>().Build() + .Build() + .Build() + .Settings().Build() + .Done(); +} + + +bool RewriteYtMergeJoin(TYtEquiJoin equiJoin, const TJoinLabels& labels, TYtJoinNodeOp& op, + const TYtJoinNodeLeaf& leftLeaf, const TYtJoinNodeLeaf& rightLeaf, const TYtState::TPtr& state, TExprContext& ctx, + bool swapTables, bool joinReduce, bool tryFirstAsPrimary, bool joinReduceForSecond, + const TMergeJoinSortInfo sortInfo, bool& skipped) +{ + skipped = false; + auto pos = equiJoin.Pos(); + + auto leftKeyColumns = op.LeftLabel; + auto rightKeyColumns = op.RightLabel; + if (swapTables) { + DoSwap(leftKeyColumns, rightKeyColumns); + } + + auto joinType = op.JoinKind; + if (swapTables) { + SwapJoinType(pos, joinType, ctx); + } + + auto& mainLabel = labels.Inputs[swapTables ? 1 : 0]; + auto& smallLabel = labels.Inputs[swapTables ? 0 : 1]; + + auto joinTree = ctx.NewList(pos, { + joinType, + ctx.NewAtom(pos, leftLeaf.Scope[0]), + ctx.NewAtom(pos, rightLeaf.Scope[0]), + leftKeyColumns, + rightKeyColumns, + ctx.NewList(pos, {}) + }); + + auto columnTypes = GetJoinColumnTypes(*joinTree, labels, "Inner", ctx); + auto finalColumnTypes = GetJoinColumnTypes(*joinTree, labels, ctx); + + auto outputLeftSchemeType = MakeOutputJoinColumns(columnTypes, mainLabel, ctx); + auto outputRightSchemeType = MakeOutputJoinColumns(columnTypes, smallLabel, ctx); + + auto inputKeyTypeLeft = BuildJoinKeyType(mainLabel, *leftKeyColumns); + auto inputKeyTypeRight = BuildJoinKeyType(smallLabel, *rightKeyColumns); + auto filteredKeyTypeLeft = AsDictKeyType(RemoveNullsFromJoinKeyType(inputKeyTypeLeft), ctx); + auto filteredKeyTypeRight = AsDictKeyType(RemoveNullsFromJoinKeyType(inputKeyTypeRight), ctx); + if (!IsSameAnnotation(*filteredKeyTypeLeft, *filteredKeyTypeRight)) { + YQL_CLOG(INFO, ProviderYt) << "Mismatch key types, left: " << *AsDictKeyType(inputKeyTypeLeft, ctx) << ", right: " << *AsDictKeyType(inputKeyTypeRight, ctx); + skipped = true; + return true; + } + + auto outputKeyType = UnifyJoinKeyType(pos, inputKeyTypeLeft, inputKeyTypeRight, ctx); + + TExprNode::TListType leftMembersNodes; + TExprNode::TListType rightMembersNodes; + TExprNode::TListType requiredMembersNodes; + bool hasData[2] = { false, false }; + TMap<TStringBuf, TVector<TStringBuf>> renameMap; + TSet<TVector<TStringBuf>> topLevelSorts; + if (!op.Parent) { + renameMap = LoadJoinRenameMap(equiJoin.JoinOptions().Ref()); + topLevelSorts = LoadJoinSortSets(equiJoin.JoinOptions().Ref()); + } + + if (joinType->Content() != "RightSemi" && joinType->Content() != "RightOnly") { + hasData[0] = true; + for (auto x : outputLeftSchemeType->GetItems()) { + auto name = ctx.NewAtom(pos, x->GetName()); + if (auto renamed = renameMap.FindPtr(x->GetName())) { + if (renamed->empty()) { + continue; + } + } + + auto finalColumnType = finalColumnTypes[x->GetName()]; + if (!finalColumnType->IsOptionalOrNull()) { + requiredMembersNodes.push_back(name); + } + + leftMembersNodes.push_back(name); + } + } + + if (joinType->Content() != "LeftSemi" && joinType->Content() != "LeftOnly") { + hasData[1] = true; + for (auto x : outputRightSchemeType->GetItems()) { + auto name = ctx.NewAtom(pos, x->GetName()); + if (auto renamed = renameMap.FindPtr(x->GetName())) { + if (renamed->empty()) { + continue; + } + } + + auto finalColumnType = finalColumnTypes[x->GetName()]; + if (!finalColumnType->IsOptionalOrNull()) { + requiredMembersNodes.push_back(name); + } + + rightMembersNodes.push_back(name); + } + } + + TCommonJoinCoreLambdas cjcLambdas[2]; + TVector<TString> keys = KeysApplyInputRenames(BuildJoinKeyList(mainLabel, *leftKeyColumns), sortInfo.LeftKeyRenames); + for (ui32 index = 0; index < 2; ++index) { + auto& label = (index == 0) ? mainLabel : smallLabel; + + auto keyColumnsNode = (index == 0) ? leftKeyColumns : rightKeyColumns; + + auto myOutputSchemeType = (index == 0) ? outputLeftSchemeType : outputRightSchemeType; + auto otherOutputSchemeType = (index == 1) ? outputLeftSchemeType : outputRightSchemeType; + + cjcLambdas[index] = MakeCommonJoinCoreReduceLambda(pos, ctx, label, outputKeyType, *keyColumnsNode, + joinType->Content(), myOutputSchemeType, otherOutputSchemeType, index, false, + 0, renameMap, hasData[index], hasData[1 - index], keys); + + auto& leaf = (index == 0) ? leftLeaf : rightLeaf; + auto& otherLeaf = (index == 1) ? leftLeaf : rightLeaf; + auto& keyRenames = (index == 0) ? sortInfo.LeftKeyRenames : sortInfo.RightKeyRenames; + ApplyInputPremap(cjcLambdas[index].ReduceLambda, leaf, otherLeaf, ctx, keyRenames); + } + YQL_ENSURE(cjcLambdas[0].CommonJoinCoreInputType == cjcLambdas[1].CommonJoinCoreInputType, "Must be same type from both side of join."); + + auto groupArg = ctx.NewArgument(pos, "group"); + auto convertedList = ctx.Builder(pos) + .Callable("FlatMap") + .Add(0, groupArg) + .Lambda(1) + .Param("item") + .Callable("Visit") + .Arg(0, "item") + .Atom(1, 0U) + .Add(2, cjcLambdas[0].ReduceLambda) + .Atom(3, 1U) + .Add(4, cjcLambdas[1].ReduceLambda) + .Seal() + .Seal() + .Seal() + .Build(); + + TExprNode::TListType optionNodes; + AddAnyJoinOptionsToCommonJoinCore(optionNodes, swapTables, op.LinkSettings, pos, ctx); + if (auto memLimit = state->Configuration->CommonJoinCoreLimit.Get()) { + optionNodes.push_back(ctx.Builder(pos) + .List() + .Atom(0, "memLimit", TNodeFlags::Default) + .Atom(1, ToString(*memLimit), TNodeFlags::Default) + .Seal() + .Build()); + } + optionNodes.push_back(ctx.Builder(pos) + .List() + .Atom(0, "sorted", TNodeFlags::Default) + .Atom(1, "left", TNodeFlags::Default) + .Seal() + .Build()); + + TExprNode::TListType keyMembersNodes; + YQL_ENSURE(sortInfo.CommonSortedKeys.size() == outputKeyType.size()); + for (auto& x : sortInfo.CommonSortedKeys) { + keyMembersNodes.push_back(ctx.NewAtom(pos, x)); + } + + auto joinedRawStream = ctx.NewCallable(pos, "CommonJoinCore", { convertedList, joinType, + ctx.NewList(pos, std::move(leftMembersNodes)), ctx.NewList(pos, std::move(rightMembersNodes)), + ctx.NewList(pos, std::move(requiredMembersNodes)), ctx.NewList(pos, std::move(keyMembersNodes)), + ctx.NewList(pos, std::move(optionNodes)), ctx.NewAtom(pos, "_yql_table_index", TNodeFlags::Default) }); + auto joinedRaw = joinedRawStream; + + const TStructExprType* outItemType = nullptr; + if (!op.Parent) { + if (auto type = GetSequenceItemType(equiJoin.Pos(), + equiJoin.Ref().GetTypeAnn()->Cast<TTupleExprType>()->GetItems()[1], + false, ctx)) { + outItemType = type->Cast<TStructExprType>(); + } else { + return false; + } + } else { + outItemType = MakeIntermediateEquiJoinTableType(pos, *joinTree, labels, op.OutputRemoveColumns, ctx); + if (!outItemType) { + return false; + } + } + + auto joined = ctx.Builder(pos) + .Callable("Map") + .Add(0, joinedRaw) + .Add(1, BuildJoinRenameLambda(pos, renameMap, *outItemType, ctx).Ptr()) + .Seal() + .Build(); + + TYtOutTableInfo outTableInfo(outItemType, state->Configuration->UseNativeYtTypes.Get().GetOrElse(DEFAULT_USE_NATIVE_YT_TYPES) ? NTCF_ALL : NTCF_NONE); + outTableInfo.RowSpec->SetConstraints(op.Constraints); + outTableInfo.SetUnique(op.Constraints.GetConstraint<TDistinctConstraintNode>(), pos, ctx); + const bool setTopLevelFullSort = state->Configuration->JoinMergeSetTopLevelFullSort.Get().GetOrElse(false); + + BuildOutputSort(outTableInfo, op, joinType->Content(), setTopLevelFullSort, *outItemType, renameMap, topLevelSorts, + sortInfo.LeftSortedKeys, sortInfo.RightSortedKeys, mainLabel, smallLabel); + + auto reduceWorld = equiJoin.World(); + auto leftSection = Build<TYtSection>(ctx, leftLeaf.Section.Pos()) + .InitFrom(leftLeaf.Section) + .Settings(NYql::RemoveSettings(leftLeaf.Section.Settings().Ref(), EYtSettingType::JoinLabel | EYtSettingType::StatColumns, ctx)) + .Done(); + auto rightSection = Build<TYtSection>(ctx, rightLeaf.Section.Pos()) + .InitFrom(rightLeaf.Section) + .Settings(NYql::RemoveSettings(rightLeaf.Section.Settings().Ref(), EYtSettingType::JoinLabel | EYtSettingType::StatColumns, ctx)) + .Done(); + + + if (sortInfo.AdditionalSort != TChoice::None) { + if (sortInfo.AdditionalSort == TChoice::Left || sortInfo.AdditionalSort == TChoice::Both) { + bool needRemapBeforeSort = sortInfo.NeedRemapBeforeSort == TChoice::Left || sortInfo.NeedRemapBeforeSort == TChoice::Both; + leftSection = SectionApplyAdditionalSort(leftSection, equiJoin, sortInfo.LeftSortedKeys, sortInfo.LeftBeforePremap, needRemapBeforeSort, *state, ctx); + } + + if (sortInfo.AdditionalSort == TChoice::Right || sortInfo.AdditionalSort == TChoice::Both) { + bool needRemapBeforeSort = sortInfo.NeedRemapBeforeSort == TChoice::Right || sortInfo.NeedRemapBeforeSort == TChoice::Both; + rightSection = SectionApplyAdditionalSort(rightSection, equiJoin, sortInfo.RightSortedKeys, sortInfo.RightBeforePremap, needRemapBeforeSort, *state, ctx); + } + + reduceWorld = Build<TCoWorld>(ctx, pos).Done(); + } + + leftSection = SectionApplyRenames(leftSection, sortInfo.LeftKeyRenames, ctx); + rightSection = SectionApplyRenames(rightSection, sortInfo.RightKeyRenames, ctx); + + auto settingsBuilder = Build<TCoNameValueTupleList>(ctx, pos) + .Add() + .Name() + .Value(ToString(EYtSettingType::ReduceBy), TNodeFlags::Default) + .Build() + .Value(ToAtomList(sortInfo.CommonSortedKeys, pos, ctx)) + .Build(); + + if (joinReduce) { + settingsBuilder + .Add() + .Name() + .Value(ToString(EYtSettingType::JoinReduce), TNodeFlags::Default) + .Build() + .Build(); + } + + if (tryFirstAsPrimary) { + const ui64 maxJobSize = state->Configuration->JoinMergeReduceJobMaxSize.Get().GetOrElse(8_GB); + auto subSettingsBuilder = Build<TCoNameValueTupleList>(ctx, pos) + .Add() + .Name() + .Value(MaxJobSizeForFirstAsPrimaryName, TNodeFlags::Default) + .Build() + .Value<TCoAtom>() + .Value(ToString(maxJobSize), TNodeFlags::Default) + .Build() + .Build(); + if (joinReduceForSecond) { + subSettingsBuilder + .Add() + .Name() + .Value(JoinReduceForSecondAsPrimaryName, TNodeFlags::Default) + .Build() + .Build(); + } + + settingsBuilder + .Add() + .Name() + .Value(ToString(EYtSettingType::FirstAsPrimary), TNodeFlags::Default) + .Build() + .Value(subSettingsBuilder.Done()) + .Build(); + } + + if (state->Configuration->UseFlow.Get().GetOrElse(DEFAULT_USE_FLOW)) { + settingsBuilder + .Add() + .Name() + .Value(ToString(EYtSettingType::Flow), TNodeFlags::Default) + .Build() + .Build(); + } + + const auto useSystemColumns = state->Configuration->UseSystemColumns.Get().GetOrElse(DEFAULT_USE_SYS_COLUMNS); + if (useSystemColumns) { + settingsBuilder + .Add() + .Name() + .Value(ToString(EYtSettingType::KeySwitch), TNodeFlags::Default) + .Build() + .Build(); + } + + op.Output = Build<TYtReduce>(ctx, pos) + .World(reduceWorld) + .DataSink(equiJoin.DataSink()) + .Input() + .Add(leftSection) + .Add(rightSection) + .Build() + .Output() + .Add(outTableInfo.ToExprNode(ctx, pos).Cast<TYtOutTable>()) + .Build() + .Settings(settingsBuilder.Done()) + .Reducer(BuildYtReduceLambda(pos, groupArg, std::move(joined), useSystemColumns, ctx)) + .Done(); + + return true; +} + +bool RewriteYtMapJoin(TYtEquiJoin equiJoin, const TJoinLabels& labels, bool isLookupJoin, + TYtJoinNodeOp& op, const TYtJoinNodeLeaf& leftLeaf, const TYtJoinNodeLeaf& rightLeaf, + TExprContext& ctx, const TMapJoinSettings& settings, bool useShards, const TYtState::TPtr& state) +{ + auto pos = equiJoin.Pos(); + auto joinType = op.JoinKind; + if (settings.SwapTables) { + SwapJoinType(pos, joinType, ctx); + } + + auto& mainLabel = labels.Inputs[settings.SwapTables ? 1 : 0]; + auto& smallLabel = labels.Inputs[settings.SwapTables ? 0 : 1]; + + TStringBuf strategyName = isLookupJoin ? "LookupJoin" : "MapJoin"; + + auto const& leftHints = settings.SwapTables ? op.LinkSettings.RightHints : op.LinkSettings.LeftHints; + auto const& rightHints = settings.SwapTables ? op.LinkSettings.LeftHints : op.LinkSettings.RightHints; + + YQL_ENSURE(!leftHints.contains("any")); + + const bool isUniqueKey = rightHints.contains("unique") || settings.RightUnique || rightHints.contains("any"); + if (isUniqueKey) { + YQL_CLOG(INFO, ProviderYt) << strategyName << " assumes unique keys for the small table"; + } + + ui64 partCount = 1; + ui64 partRows = settings.RightRows; + if ((settings.RightSize > 0) && useShards) { + partCount = (settings.RightMemSize + settings.MapJoinLimit - 1) / settings.MapJoinLimit; + partRows = (settings.RightRows + partCount - 1) / partCount; + } + + if (partCount > 1) { + YQL_ENSURE(!isLookupJoin); + YQL_CLOG(INFO, ProviderYt) << strategyName << " sharded into " << partCount << " parts, each " << partRows << " rows"; + } + + auto leftKeyColumns = settings.SwapTables ? op.RightLabel : op.LeftLabel; + auto rightKeyColumns = settings.SwapTables ? op.LeftLabel : op.RightLabel; + auto joinTree = ctx.NewList(pos, { + joinType, + ctx.NewAtom(pos, leftLeaf.Scope[0]), + ctx.NewAtom(pos, rightLeaf.Scope[0]), + leftKeyColumns, + rightKeyColumns, + ctx.NewList(pos, {}) + }); + + auto columnTypes = GetJoinColumnTypes(*joinTree, labels, ctx); + auto outputLeftSchemeType = MakeOutputJoinColumns(columnTypes, mainLabel, ctx); + auto outputRightSchemeType = MakeOutputJoinColumns(columnTypes, smallLabel, ctx); + + auto inputKeyTypeLeft = BuildJoinKeyType(mainLabel, *leftKeyColumns); + auto inputKeyTypeRight = BuildJoinKeyType(smallLabel, *rightKeyColumns); + auto outputKeyType = UnifyJoinKeyType(pos, inputKeyTypeLeft, inputKeyTypeRight, ctx); + + TMap<TStringBuf, TVector<TStringBuf>> renameMap; + if (!op.Parent) { + renameMap = LoadJoinRenameMap(equiJoin.JoinOptions().Ref()); + } + + const TStructExprType* outItemType = nullptr; + if (!op.Parent) { + if (auto type = GetSequenceItemType(equiJoin.Pos(), + equiJoin.Ref().GetTypeAnn()->Cast<TTupleExprType>()->GetItems()[1], + false, ctx)) { + outItemType = type->Cast<TStructExprType>(); + } else { + return false; + } + } else { + outItemType = MakeIntermediateEquiJoinTableType(pos, *joinTree, labels, op.OutputRemoveColumns, ctx); + if (!outItemType) { + return false; + } + } + + auto mainPaths = MakeUnorderedSection(leftLeaf.Section, ctx).Paths(); + auto mainSettings = NYql::RemoveSettings(leftLeaf.Section.Settings().Ref(), EYtSettingType::JoinLabel | EYtSettingType::StatColumns, ctx); + auto smallPaths = MakeUnorderedSection(rightLeaf.Section, ctx).Paths(); + auto smallSettings = NYql::RemoveSettings(rightLeaf.Section.Settings().Ref(), EYtSettingType::JoinLabel | EYtSettingType::StatColumns, ctx); + if (!NYql::HasSetting(*smallSettings, EYtSettingType::Unordered)) { + smallSettings = NYql::AddSetting(*smallSettings, EYtSettingType::Unordered, {}, ctx); + } + auto smallKeyColumns = rightKeyColumns; + + TSyncMap syncList; + for (auto path: smallPaths) { + if (auto out = path.Table().Maybe<TYtOutput>()) { + syncList.emplace(GetOutputOp(out.Cast()).Ptr(), syncList.size()); + } + } + auto mapWorld = ApplySyncListToWorld(equiJoin.World().Ptr(), syncList, ctx); + + auto mapSettingsBuilder = Build<TCoNameValueTupleList>(ctx, pos); + if (partCount > 1) { + mapSettingsBuilder + .Add() + .Name() + .Value(ToString(EYtSettingType::Sharded)) + .Build() + .Build(); + } + + const bool isCross = joinType->IsAtom("Cross"); + + auto tableContentSettings = ctx.NewList(pos, {}); + if (isCross) { + ui64 rowFactor = (1 + smallLabel.InputType->GetSize()) * sizeof(NKikimr::NUdf::TUnboxedValuePod); // Table content after Collect + rowFactor += (1 + smallLabel.InputType->GetSize() + mainLabel.InputType->GetSize()) * sizeof(NKikimr::NUdf::TUnboxedValuePod); // Table content after Map with added left side + rowFactor += settings.LeftSize / settings.LeftRows; // Average added left side for each row after Map + + tableContentSettings = NYql::AddSetting(*tableContentSettings, EYtSettingType::RowFactor, ctx.NewAtom(pos, ToString(rowFactor), TNodeFlags::Default), ctx); + } + + auto mapJoinUseFlow = state->Configuration->MapJoinUseFlow.Get().GetOrElse(DEFAULT_MAP_JOIN_USE_FLOW); + + TYtOutTableInfo outTableInfo(outItemType, state->Configuration->UseNativeYtTypes.Get().GetOrElse(DEFAULT_USE_NATIVE_YT_TYPES) ? NTCF_ALL : NTCF_NONE); + outTableInfo.RowSpec->SetConstraints(op.Constraints); + outTableInfo.SetUnique(op.Constraints.GetConstraint<TDistinctConstraintNode>(), pos, ctx); + + TVector<TYtMap> maps; + for (ui64 partNo = 0; partNo < partCount; ++partNo) { + auto listArg = ctx.NewArgument(pos, "list"); + + ui64 dictItemsCount = settings.RightRows; + auto readSettings = smallSettings; + + if (partCount > 1) { + ui64 start = partRows * partNo; + ui64 finish = Min(start + partRows, settings.RightRows); + readSettings = NYql::AddSetting(*readSettings, EYtSettingType::Skip, + ctx.Builder(pos) + .Callable("Uint64") + .Atom(0, ToString(start), TNodeFlags::Default) + .Seal() + .Build(), ctx); + + readSettings = NYql::AddSetting(*readSettings, EYtSettingType::Take, + ctx.Builder(pos) + .Callable("Uint64") + .Atom(0, ToString(finish - start), TNodeFlags::Default) + .Seal() + .Build(), ctx); + + readSettings = NYql::AddSetting(*readSettings, EYtSettingType::Unordered, {}, ctx); + + dictItemsCount = finish - start; + } + + auto tableContent = Build<TYtTableContent>(ctx, pos) + .Input<TYtReadTable>() + .World<TCoWorld>().Build() + .DataSource(ctx.RenameNode(equiJoin.DataSink().Ref(), TYtDSource::CallableName())) + .Input() + .Add() + .Paths(smallPaths) + .Settings(readSettings) + .Build() + .Build() + .Build() + .Settings(tableContentSettings) + .Done().Ptr(); + + TExprNode::TPtr lookupJoinFilterLambda; + if (isLookupJoin) { + TExprNode::TPtr tableContentAsJoinKeysTupleList; + YQL_ENSURE(leftKeyColumns->ChildrenSize() == rightKeyColumns->ChildrenSize()); + + TVector<TString> mainJoinMembers; + TVector<TString> smallJoinMembers; + + for (ui32 i = 0; i < leftKeyColumns->ChildrenSize(); i += 2) { + TString mainMemberName = mainLabel.MemberName(leftKeyColumns->Child(i)->Content(), + leftKeyColumns->Child(i + 1)->Content()); + + TString smallMemberName = smallLabel.MemberName(rightKeyColumns->Child(i)->Content(), + rightKeyColumns->Child(i + 1)->Content()); + + mainJoinMembers.push_back(mainMemberName); + smallJoinMembers.push_back(smallMemberName); + } + + auto status = SubstTables(tableContent, state, false, ctx); + if (status.Level == IGraphTransformer::TStatus::Error) { + return false; + } + + ctx.Step.Repeat(TExprStep::ExprEval); + + tableContent = ctx.Builder(pos) + .Callable("EvaluateExpr") + .Add(0, tableContent) + .Seal() + .Build(); + + lookupJoinFilterLambda = ctx.Builder(pos) + .Lambda() + .Param("item") + .Callable("Coalesce") + .Callable(0, "SqlIn") + .Callable(0, "EvaluateExpr") + .Callable(0, "FlatMap") + .Add(0, tableContent) + .Lambda(1) + .Param("input") + .Callable("Just") + .Do([&](TExprNodeBuilder& parent) -> TExprNodeBuilder& { + auto builder = parent.List(0); + for (ui32 i = 0; i < smallJoinMembers.size(); ++i) { + builder + .Callable(i, "Member") + .Arg(0, "input") + .Atom(1, smallJoinMembers[i]) + .Seal(); + } + return builder.Seal(); + }) + .Seal() + .Seal() + .Seal() + .Seal() + .Do([&](TExprNodeBuilder& parent) -> TExprNodeBuilder& { + auto builder = parent.List(1); + for (ui32 i = 0; i < mainJoinMembers.size(); ++i) { + builder + .Callable(i, "Member") + .Arg(0, "item") + .Atom(1, mainJoinMembers[i]) + .Seal(); + } + return builder.Seal(); + }) + .List(2) + .Seal() + .Seal() + .Callable(1, "Bool") + .Atom(0, "false", TNodeFlags::Default) + .Seal() + .Seal() + .Seal() + .Build(); + } + + if (auto premap = GetPremapLambda(rightLeaf)) { + tableContent = ctx.Builder(tableContent->Pos()) + .Callable("FlatMap") + .Add(0, tableContent) + .Add(1, premap.Cast().Ptr()) + .Seal() + .Build(); + } + + const bool needPayload = joinType->IsAtom({"Inner", "Left"}); + + // don't produce nulls + TExprNode::TListType remappedMembers; + TExprNode::TListType remappedMembersToSkipNull; + + TExprNode::TPtr smallKeySelector; + if (!isCross) { + tableContent = RemapNonConvertibleItems(tableContent, smallLabel, *rightKeyColumns, outputKeyType, remappedMembers, remappedMembersToSkipNull, ctx); + if (!remappedMembersToSkipNull.empty()) { + tableContent = ctx.NewCallable(pos, "SkipNullMembers", { tableContent, ctx.NewList(pos, std::move(remappedMembersToSkipNull)) }); + } + + auto arg = ctx.NewArgument(pos, "item"); + TExprNode::TListType tupleItems; + YQL_ENSURE(2 * remappedMembers.size() == rightKeyColumns->ChildrenSize()); + for (auto memberNameNode : remappedMembers) { + auto member = ctx.Builder(pos) + .Callable("Member") + .Add(0, arg) + .Add(1, memberNameNode) + .Seal() + .Build(); + + tupleItems.push_back(member); + } + + TExprNode::TPtr lambdaBody; + if (tupleItems.size() == 1) { + lambdaBody = tupleItems.front(); + } else { + lambdaBody = ctx.NewList(pos, std::move(tupleItems)); + } + + smallKeySelector = ctx.NewLambda(pos, ctx.NewArguments(pos, { arg }), std::move(lambdaBody)); + } else { + tableContent = Build<TCoCollect>(ctx, pos) + .Input(tableContent) + .Done().Ptr(); + } + + // may produce null in keys + TExprNode::TPtr smallPayloadSelector; + if (!isCross) { + if (needPayload) { + smallPayloadSelector = ctx.Builder(pos) + .Lambda() + .Param("item") + .Callable("AsStruct") + .Do([&](TExprNodeBuilder& parent)->TExprNodeBuilder& { + ui32 index = 0; + for (auto x : smallLabel.EnumerateAllMembers()) { + parent.List(index++) + .Atom(0, x) + .Callable(1, "Member") + .Arg(0, "item") + .Atom(1, x) + .Seal(); + } + + return parent; + }) + .Seal() + .Seal() + .Build(); + } + else { + smallPayloadSelector = ctx.Builder(pos) + .Lambda() + .Param("item") + .Callable("Void") + .Seal() + .Seal() + .Build(); + } + } + + TExprNode::TPtr dict; + if (!isCross && !mapJoinUseFlow) { + dict = ctx.Builder(pos) + .Callable("ToDict") + .Add(0, tableContent) + .Add(1, smallKeySelector) + .Add(2, smallPayloadSelector) + .List(3) + .Atom(0, "Hashed") + .Atom(1, needPayload && !isUniqueKey ? "Many" : "One") + .Atom(2, "Compact") + .List(3) + .Atom(0, "ItemsCount") + .Atom(1, ToString(dictItemsCount)) + .Seal() + .Seal() + .Seal() + .Build(); + } + + auto mainArg = ctx.NewArgument(pos, "mainRow"); + auto lookupArg = ctx.NewArgument(pos, "lookupRow"); + TExprNode::TListType joinedOutNodes; + TExprNode::TListType rightRenameNodes; + TExprNode::TListType leftRenameNodes; + for (ui32 index = 0; index < outputLeftSchemeType->GetSize(); ++index) { + auto item = outputLeftSchemeType->GetItems()[index]; + TVector<TStringBuf> newNames; + newNames.push_back(item->GetName()); + TStringBuf part1; + TStringBuf part2; + SplitTableName(item->GetName(), part1, part2); + TString memberName = mainLabel.MemberName(part1, part2); + if (!op.Parent) { + if (auto renamed = renameMap.FindPtr(item->GetName())) { + newNames = *renamed; + } + } else if (op.OutputRemoveColumns.contains(item->GetName())) { + newNames = {}; + } + + for (auto newName : newNames) { + leftRenameNodes.push_back(ctx.NewAtom(pos, memberName)); + leftRenameNodes.push_back(ctx.NewAtom(pos, newName)); + AddJoinRemappedColumn(pos, mainArg, joinedOutNodes, memberName, newName, ctx); + } + } + + if (needPayload || isCross) { + for (ui32 index = 0; index < outputRightSchemeType->GetSize(); ++index) { + auto item = outputRightSchemeType->GetItems()[index]; + TVector<TStringBuf> newNames; + newNames.push_back(item->GetName()); + TStringBuf part1; + TStringBuf part2; + SplitTableName(item->GetName(), part1, part2); + TString memberName = smallLabel.MemberName(part1, part2); + if (!op.Parent) { + if (auto renamed = renameMap.FindPtr(item->GetName())) { + newNames = *renamed; + } + } else if (op.OutputRemoveColumns.contains(item->GetName())) { + newNames = {}; + } + + for (auto newName : newNames) { + rightRenameNodes.push_back(ctx.NewAtom(pos, memberName)); + rightRenameNodes.push_back(ctx.NewAtom(pos, newName)); + AddJoinRemappedColumn(pos, lookupArg, joinedOutNodes, memberName, newName, ctx); + } + } + } + + TExprNode::TPtr joined; + if (!isCross) { + TExprNode::TListType leftKeyColumnNodes; + TExprNode::TListType leftKeyColumnNodesNullable; + auto mapInput = RemapNonConvertibleItems(listArg, mainLabel, *leftKeyColumns, outputKeyType, leftKeyColumnNodes, leftKeyColumnNodesNullable, ctx); + if (mapJoinUseFlow) { + joined = ctx.Builder(pos) + .Callable("FlatMap") + .Callable(0, "SqueezeToDict") + .Callable(0, "ToFlow") + .Add(0, std::move(tableContent)) + .Callable(1, "DependsOn") + .Add(0, listArg) + .Seal() + .Seal() + .Add(1, std::move(smallKeySelector)) + .Add(2, std::move(smallPayloadSelector)) + .List(3) + .Atom(0, "Hashed", TNodeFlags::Default) + .Atom(1, needPayload && !isUniqueKey ? "Many" : "One", TNodeFlags::Default) + .Atom(2, "Compact", TNodeFlags::Default) + .List(3) + .Atom(0, "ItemsCount", TNodeFlags::Default) + .Atom(1, ToString(dictItemsCount), TNodeFlags::Default) + .Seal() + .Seal() + .Seal() + .Lambda(1) + .Param("dict") + .Callable("MapJoinCore") + .Add(0, std::move(mapInput)) + .Arg(1, "dict") + .Add(2, joinType) + .Add(3, ctx.NewList(pos, std::move(leftKeyColumnNodes))) + .Add(4, ctx.NewList(pos, std::move(remappedMembers))) + .Add(5, ctx.NewList(pos, std::move(leftRenameNodes))) + .Add(6, ctx.NewList(pos, std::move(rightRenameNodes))) + .Add(7, leftKeyColumns) + .Add(8, rightKeyColumns) + .Seal() + .Seal() + .Seal() + .Build(); + } else { + joined = ctx.Builder(pos) + .Callable("MapJoinCore") + .Add(0, mapInput) + .Add(1, dict) + .Add(2, joinType) + .Add(3, ctx.NewList(pos, std::move(leftKeyColumnNodes))) + .Add(4, ctx.NewList(pos, std::move(remappedMembers))) + .Add(5, ctx.NewList(pos, std::move(leftRenameNodes))) + .Add(6, ctx.NewList(pos, std::move(rightRenameNodes))) + .Add(7, leftKeyColumns) + .Add(8, rightKeyColumns) + .Seal() + .Build(); + } + } + else { + auto joinedOut = ctx.NewCallable(pos, "AsStruct", std::move(joinedOutNodes)); + auto joinedBody = ctx.Builder(pos) + .Callable("Map") + .Callable(0, "ToFlow") + .Add(0, std::move(tableContent)) + .Callable(1, "DependsOn") + .Add(0, listArg) + .Seal() + .Seal() + .Lambda(1) + .Param("smallRow") + .ApplyPartial(nullptr, std::move(joinedOut)).WithNode(*lookupArg, "smallRow").Seal() + .Seal() + .Seal() + .Build(); + + auto joinedLambda = ctx.NewLambda(pos, ctx.NewArguments(pos, { mainArg }), std::move(joinedBody)); + joined = ctx.Builder(pos) + .Callable("FlatMap") + .Add(0, listArg) + .Add(1, std::move(joinedLambda)) + .Seal() + .Build(); + } + + auto mapLambda = ctx.NewLambda(pos, ctx.NewArguments(pos, {std::move(listArg)}), std::move(joined)); + if (const auto premap = GetPremapLambda(leftLeaf)) { + TExprNode::TPtr placeHolder; + std::tie(placeHolder, mapLambda) = ReplaceDependsOn(mapLambda, ctx, state->Types); + + mapLambda = ctx.Builder(mapLambda->Pos()) + .Lambda() + .Param("list") + .Apply(mapLambda) + .With(0) + .Callable("FlatMap") + .Arg(0, "list") + .Add(1, premap.Cast().Ptr()) + .Seal() + .Done() + .WithNode(*placeHolder, "list") + .Seal() + .Seal() + .Build(); + } + + // since premap doesn't affect key columns we can apply lookup join filter before premap + if (lookupJoinFilterLambda) { + TExprNode::TPtr placeHolder; + std::tie(placeHolder, mapLambda) = ReplaceDependsOn(mapLambda, ctx, state->Types); + + mapLambda = ctx.Builder(mapLambda->Pos()) + .Lambda() + .Param("list") + .Apply(mapLambda) + .With(0) + .Callable("Filter") + .Arg(0, "list") + .Add(1, std::move(lookupJoinFilterLambda)) + .Seal() + .Done() + .WithNode(*placeHolder, "list") + .Seal() + .Seal() + .Build(); + } + + if (state->Configuration->UseFlow.Get().GetOrElse(DEFAULT_USE_FLOW)) { + mapSettingsBuilder + .Add() + .Name() + .Value(ToString(EYtSettingType::Flow), TNodeFlags::Default) + .Build() + .Build(); + } + + auto map = Build<TYtMap>(ctx, pos) + .World(mapWorld) + .DataSink(equiJoin.DataSink()) + .Input() + .Add() + .Paths(mainPaths) + .Settings(mainSettings) + .Build() + .Build() + .Output() + .Add(outTableInfo.ToExprNode(ctx, pos).Cast<TYtOutTable>()) + .Build() + .Settings(mapSettingsBuilder.Done()) + .Mapper(mapLambda) + .Done(); + + maps.push_back(map); + } + + if (maps.size() == 1) { + op.Output = maps.front(); + } + else { + TVector<TYtPath> paths; + for (auto map: maps) { + paths.push_back(Build<TYtPath>(ctx, pos) + .Table<TYtOutput>() + .Operation(map) + .OutIndex().Value(0U).Build() + .Build() + .Columns<TCoVoid>().Build() + .Ranges<TCoVoid>().Build() + .Stat<TCoVoid>().Build() + .Done() + ); + } + op.Output = Build<TYtMerge>(ctx, pos) + .World<TCoWorld>().Build() + .DataSink(equiJoin.DataSink()) + .Input() + .Add() + .Paths() + .Add(paths) + .Build() + .Settings() + .Build() + .Build() + .Build() + .Output() + .Add(outTableInfo.ToExprNode(ctx, pos).Cast<TYtOutTable>()) + .Build() + .Settings() + .Build() + .Done(); + } + + return true; +} + +TCoLambda BuildIdentityLambda(TPositionHandle pos, TExprContext& ctx) { + return Build<TCoLambda>(ctx, pos) + .Args({"item"}) + .Body("item") + .Done(); +} + +TCoLambda BuildMapCombinerSideLambda(const TYtState::TPtr& state, const TMap<TString, const TTypeAnnotationNode*>& keys, TPositionHandle pos, TExprContext& ctx) { + + auto pickleLambda = ctx.Builder(pos) + .Lambda() + .Param("item") + .Callable("StablePickle") + .Arg(0, "item") + .Seal() + .Seal() + .Build(); + + auto identityLambda = BuildIdentityLambda(pos, ctx).Ptr(); + + TCoLambda keyExtractor(ctx.Builder(pos) + .Lambda() + .Param("item") + .Do([&](TExprNodeBuilder& parent) -> TExprNodeBuilder& { + auto builder = parent.List(); + size_t i = 0; + for (const auto& [name, type] : keys) { + bool needPickle = RemoveOptionalType(type)->GetKind() != ETypeAnnotationKind::Data; + builder + .Apply(i++, needPickle ? pickleLambda : identityLambda) + .With(0) + .Callable("Member") + .Arg(0, "item") + .Atom(1, name) + .Seal() + .Done() + .Seal(); + } + return builder.Seal(); + }) + .Seal() + .Build()); + + + return Build<TCoLambda>(ctx, pos) + .Args({"stream"}) + .Body<TCoCombineCore>() + .Input("stream") + .KeyExtractor(keyExtractor) + .InitHandler() + .Args({"key", "item"}) + .Body("item") + .Build() + .UpdateHandler() + .Args({"key", "item", "state"}) + .Body("state") + .Build() + .FinishHandler() + .Args({"key", "state"}) + .Body<TCoJust>() + .Input("state") + .Build() + .Build() + .MemLimit<TCoAtom>() + .Value(ToString(state->Configuration->CombineCoreLimit.Get().GetOrElse(0))) + .Build() + .Build() + .Done(); +} + +TCoLambda BuildMapCombinerLambda(const TYtState::TPtr& state, const TJoinLabels& labels, const TYtJoinNodeOp& op, TPositionHandle pos, TExprContext& ctx) +{ + TCoLambda identityLambda = BuildIdentityLambda(pos, ctx); + + bool leftAny = op.LinkSettings.LeftHints.contains("any"); + bool rightAny = op.LinkSettings.RightHints.contains("any"); + + if (!leftAny && !rightAny) { + return identityLambda; + } + + auto leftKeyColumns = op.LeftLabel; + auto rightKeyColumns = op.RightLabel; + + YQL_ENSURE(leftKeyColumns); + YQL_ENSURE(rightKeyColumns); + + auto leftKeyTypes = BuildJoinKeyTypeMap(labels.Inputs[0], *leftKeyColumns); + auto rightKeyTypes = BuildJoinKeyTypeMap(labels.Inputs[1], *rightKeyColumns); + + return Build<TCoLambda>(ctx, pos) + .Args({"stream"}) + .Body<TCoSwitch>() + .Input("stream") + .BufferBytes() + .Value(ToString(state->Configuration->SwitchLimit.Get().GetOrElse(DEFAULT_SWITCH_MEMORY_LIMIT))) + .Build() + .FreeArgs() + .Add<TCoAtomList>() + .Add() + .Value(0U) + .Build() + .Build() + .Add(leftAny ? BuildMapCombinerSideLambda(state, leftKeyTypes, pos, ctx) : identityLambda) + .Add<TCoAtomList>() + .Add() + .Value(1U) + .Build() + .Build() + .Add(rightAny ? BuildMapCombinerSideLambda(state, rightKeyTypes, pos, ctx) : identityLambda) + .Build() + .Build() + .Done(); +} + +bool JoinKeysMayHaveNulls(const TVector<const TTypeAnnotationNode*>& inputKeyTypes, const TVector<const TTypeAnnotationNode*>& unifiedKeyTypes) { + YQL_ENSURE(inputKeyTypes.size() == unifiedKeyTypes.size()); + for (size_t i = 0; i < inputKeyTypes.size(); ++i) { + if (inputKeyTypes[i]->HasOptionalOrNull()) { + return true; + } + + NUdf::TCastResultOptions options = CastResult<true>(inputKeyTypes[i], unifiedKeyTypes[i]); + YQL_ENSURE(!(options & NKikimr::NUdf::ECastOptions::Impossible)); + if (options & NKikimr::NUdf::ECastOptions::MayFail) { + return true; + } + } + return false; +} + +TExprNode::TPtr BuildSideSplitNullsLambda(TPositionHandle pos, bool mayHaveNulls, const TExprNode::TPtr& inputItem, + const TVector<TString>& keyColumns, const TString& sidePrefix, + const TCoLambda& joinRenamingLambda, const TStructExprType& joinOutputType, + const TExprNode::TPtr& outputVariantType, size_t outputVariantIndex, TExprContext& ctx) +{ + if (!mayHaveNulls) { + return ctx.Builder(pos) + .Lambda() + .Param("side") + .Callable("Variant") + .Add(0, inputItem) + .Atom(1, 0U) + .Add(2, outputVariantType) + .Seal() + .Seal() + .Build(); + } + + return ctx.Builder(pos) + .Lambda() + .Param("side") + .Callable("If") + .Callable(0, "HasNull") + .List(0) + .Do([&](TExprNodeBuilder& parent) -> TExprNodeBuilder& { + for (ui32 i = 0; i < keyColumns.size(); ++i) { + parent + .Callable(i, "Member") + .Add(0, inputItem) + .Atom(1, keyColumns[i]) + .Seal(); + } + return parent; + }) + .Seal() + .Seal() + .Callable(1, "Variant") + .Apply(0, joinRenamingLambda.Ptr()) + .With(0) + .Callable("StrictCast") + .Callable(0, "FlattenMembers") + .List(0) + .Atom(0, sidePrefix) + .Arg(1, "side") + .Seal() + .Seal() + .Add(1, ExpandType(pos, joinOutputType, ctx)) + .Seal() + .Done() + .Seal() + .Atom(1, ToString(outputVariantIndex), TNodeFlags::Default) + .Add(2, outputVariantType) + .Seal() + .Callable(2, "Variant") + .Add(0, inputItem) + .Atom(1, 0U) + .Add(2, outputVariantType) + .Seal() + .Seal() + .Seal() + .Build(); +} + +bool RewriteYtCommonJoin(TYtEquiJoin equiJoin, const TJoinLabels& labels, TYtJoinNodeOp& op, + const TYtJoinNodeLeaf& leftLeaf, const TYtJoinNodeLeaf& rightLeaf, const TYtState::TPtr& state, TExprContext& ctx, + bool leftUnique, bool rightUnique, ui64 leftSize, ui64 rightSize) +{ + const auto pos = equiJoin.Pos(); + + const auto leftNotFat = leftUnique || op.LinkSettings.LeftHints.contains("unique") || op.LinkSettings.LeftHints.contains("small"); + const auto rightNotFat = rightUnique || op.LinkSettings.RightHints.contains("unique") || op.LinkSettings.RightHints.contains("small"); + bool leftFirst = false; + if (leftNotFat != rightNotFat) { + // non-fat will be first + leftFirst = leftNotFat; + } else { + // small table will be first + leftFirst = leftSize < rightSize; + } + + YQL_CLOG(INFO, ProviderYt) << "CommonJoin, << " << (leftFirst ? "left" : "right") << " table moved first"; + + auto leftKeyColumns = op.LeftLabel; + auto rightKeyColumns = op.RightLabel; + + auto joinType = op.JoinKind; + auto joinTree = ctx.NewList(pos, { + joinType, + ctx.NewAtom(pos, leftLeaf.Scope[0]), + ctx.NewAtom(pos, rightLeaf.Scope[0]), + leftKeyColumns, + rightKeyColumns, + ctx.NewList(pos, {}) + }); + + auto columnTypes = GetJoinColumnTypes(*joinTree, labels, "Inner", ctx); + auto finalColumnTypes = GetJoinColumnTypes(*joinTree, labels, ctx); + auto outputLeftSchemeType = MakeOutputJoinColumns(columnTypes, labels.Inputs[0], ctx); + auto outputRightSchemeType = MakeOutputJoinColumns(columnTypes, labels.Inputs[1], ctx); + + TVector<const TTypeAnnotationNode*> outputKeyType; + TVector<const TTypeAnnotationNode*> inputKeyTypeLeft; + TVector<const TTypeAnnotationNode*> inputKeyTypeRight; + const bool isCrossJoin = joinType->IsAtom("Cross"); + if (isCrossJoin) { + outputKeyType = TVector<const TTypeAnnotationNode*>(1, ctx.MakeType<TDataExprType>(EDataSlot::Uint32)); + } else { + inputKeyTypeLeft = BuildJoinKeyType(labels.Inputs[0], *leftKeyColumns); + inputKeyTypeRight = BuildJoinKeyType(labels.Inputs[1], *rightKeyColumns); + outputKeyType = UnifyJoinKeyType(pos, inputKeyTypeLeft, inputKeyTypeRight, ctx); + } + + TVector<TString> ytReduceByColumns; + for (size_t i = 0; i < outputKeyType.size(); ++i) { + ytReduceByColumns.push_back(TStringBuilder() << "_yql_join_column_" << i); + } + + TExprNode::TListType leftMembersNodes; + TExprNode::TListType rightMembersNodes; + TExprNode::TListType requiredMembersNodes; + bool hasData[2] = { false, false }; + TMap<TStringBuf, TVector<TStringBuf>> renameMap; + if (!op.Parent) { + renameMap = LoadJoinRenameMap(equiJoin.JoinOptions().Ref()); + } + + if (!joinType->IsAtom({"RightSemi", "RightOnly"})) { + hasData[0] = true; + for (auto x : outputLeftSchemeType->GetItems()) { + auto name = ctx.NewAtom(pos, x->GetName()); + if (auto renamed = renameMap.FindPtr(x->GetName())) { + if (renamed->empty()) { + continue; + } + } + + auto finalColumnType = finalColumnTypes[x->GetName()]; + if (!finalColumnType->IsOptionalOrNull()) { + requiredMembersNodes.push_back(name); + } + + leftMembersNodes.push_back(name); + } + } + + if (!joinType->IsAtom({"LeftSemi", "LeftOnly"})) { + hasData[1] = true; + for (auto x : outputRightSchemeType->GetItems()) { + auto name = ctx.NewAtom(pos, x->GetName()); + if (auto renamed = renameMap.FindPtr(x->GetName())) { + if (renamed->empty()) { + continue; + } + } + + auto finalColumnType = finalColumnTypes[x->GetName()]; + if (!finalColumnType->IsOptionalOrNull()) { + requiredMembersNodes.push_back(name); + } + + rightMembersNodes.push_back(name); + } + } + + TCommonJoinCoreLambdas cjcLambdas[2]; + for (ui32 index = 0; index < 2; ++index) { + auto keyColumnsNode = (index == 0) ? leftKeyColumns : rightKeyColumns; + + auto& label = labels.Inputs[index]; + auto& otherLabel = labels.Inputs[1 - index]; + + auto myOutputSchemeType = (index == 0) ? outputLeftSchemeType : outputRightSchemeType; + auto otherOutputSchemeType = (index == 1) ? outputLeftSchemeType : outputRightSchemeType; + + cjcLambdas[index] = MakeCommonJoinCoreLambdas(pos, ctx, label, otherLabel, outputKeyType, *keyColumnsNode, + joinType->Content(), myOutputSchemeType, otherOutputSchemeType, index, true, + leftFirst ? index : 1 - index, renameMap, hasData[index], hasData[1 - index], ytReduceByColumns); + + auto& leaf = (index == 0) ? leftLeaf : rightLeaf; + auto& otherLeaf = (index == 1) ? leftLeaf : rightLeaf; + ApplyInputPremap(cjcLambdas[index].MapLambda, leaf, otherLeaf, ctx); + } + YQL_ENSURE(cjcLambdas[0].CommonJoinCoreInputType == cjcLambdas[1].CommonJoinCoreInputType, "Must be same type from both side of join."); + + auto groupArg = ctx.NewArgument(pos, "group"); + + TExprNode::TListType optionNodes; + AddAnyJoinOptionsToCommonJoinCore(optionNodes, false, op.LinkSettings, pos, ctx); + if (auto memLimit = state->Configuration->CommonJoinCoreLimit.Get()) { + optionNodes.push_back(ctx.Builder(pos) + .List() + .Atom(0, "memLimit", TNodeFlags::Default) + .Atom(1, ToString(*memLimit), TNodeFlags::Default) + .Seal() + .Build()); + } + optionNodes.push_back(ctx.Builder(pos) + .List() + .Atom(0, "sorted", TNodeFlags::Default) + .Atom(1, leftFirst ? "left" : "right", TNodeFlags::Default) + .Seal() + .Build()); + + TExprNode::TListType keyMembersNodes; + if (!isCrossJoin) { + for (auto& x : ytReduceByColumns) { + keyMembersNodes.push_back(ctx.NewAtom(pos, x)); + } + } + + auto convertedList = PrepareForCommonJoinCore(pos, ctx, groupArg, cjcLambdas[0].ReduceLambda, + cjcLambdas[1].ReduceLambda); + auto joinedRawStream = ctx.NewCallable(pos, "CommonJoinCore", { convertedList, joinType, + ctx.NewList(pos, std::move(leftMembersNodes)), ctx.NewList(pos, std::move(rightMembersNodes)), + ctx.NewList(pos, std::move(requiredMembersNodes)), ctx.NewList(pos, std::move(keyMembersNodes)), + ctx.NewList(pos, std::move(optionNodes)), ctx.NewAtom(pos, "_yql_table_index", TNodeFlags::Default) }); + auto joinedRaw = joinedRawStream; + + const TStructExprType* outItemTypeBeforeRename = MakeIntermediateEquiJoinTableType(pos, *joinTree, labels, {}, ctx); + if (!outItemTypeBeforeRename) { + return false; + } + const TStructExprType* outItemType = nullptr; + if (!op.Parent) { + if (auto type = GetSequenceItemType(equiJoin.Pos(), + equiJoin.Ref().GetTypeAnn()->Cast<TTupleExprType>()->GetItems()[1], + false, ctx)) { + outItemType = type->Cast<TStructExprType>(); + } else { + return false; + } + } else { + outItemType = MakeIntermediateEquiJoinTableType(pos, *joinTree, labels, op.OutputRemoveColumns, ctx); + if (!outItemType) { + return false; + } + } + + const TCoLambda joinRenamingLambda = BuildJoinRenameLambda(pos, renameMap, *outItemType, ctx); + + auto joined = ctx.Builder(pos) + .Callable("Map") + .Add(0, joinedRaw) + .Add(1, joinRenamingLambda.Ptr()) + .Seal() + .Build(); + + TVector<TString> ytSortByColumns; + ytSortByColumns = ytReduceByColumns; + ytSortByColumns.push_back("_yql_sort"); + + auto mapCombinerLambda = BuildMapCombinerLambda(state, labels, op, pos, ctx); + + TExprNode::TPtr chopperHandler = ctx.NewLambda(pos, ctx.NewArguments(pos, {ctx.NewArgument(pos, "stup"), groupArg }), std::move(joined)); + TExprNode::TPtr chopperSwitch; + auto settingsBuilder = Build<TCoNameValueTupleList>(ctx, pos); + + if (state->Configuration->UseSystemColumns.Get().GetOrElse(DEFAULT_USE_SYS_COLUMNS)) { + chopperSwitch = ctx.Builder(pos) + .Lambda() + .Param("key") + .Param("item") + .Callable("SqlExtractKey") + .Arg(0, "item") + .Lambda(1) + .Param("row") + .Callable("Member") + .Arg(0, "row") + .Atom(1, YqlSysColumnKeySwitch, TNodeFlags::Default) + .Seal() + .Seal() + .Seal() + .Seal() + .Build(); + + chopperHandler = ctx.Builder(pos) + .Lambda() + .Param("key") + .Param("group") + .Apply(chopperHandler) + .With(0, "key") + .With(1) + .Callable("RemovePrefixMembers") + .Arg(0, "group") + .List(1) + .Atom(0, YqlSysColumnPrefix, TNodeFlags::Default) + .Seal() + .Seal() + .Done() + .Seal() + .Seal() + .Build(); + + settingsBuilder + .Add() + .Name() + .Value(ToString(EYtSettingType::KeySwitch), TNodeFlags::Default) + .Build() + .Build(); + } + else { + chopperSwitch = Build<TCoLambda>(ctx, pos) + .Args({"key", "item"}) + .Body<TYtIsKeySwitch>() + .DependsOn() + .Input("item") + .Build() + .Build() + .Done().Ptr(); + } + + auto reducer = Build<TCoLambda>(ctx, pos) + .Args({"stream"}) + .Body<TCoChopper>() + .Input("stream") + .KeyExtractor() + .Args({"item"}) + .Body<TCoUint64>() + .Literal() + .Value(0U) + .Build() + .Build() + .Build() + .GroupSwitch(chopperSwitch) + .Handler(chopperHandler) + .Build() + .Done(); + + settingsBuilder + .Add() + .Name() + .Value(ToString(EYtSettingType::ReduceBy), TNodeFlags::Default) + .Build() + .Value(ToAtomList(ytReduceByColumns, pos, ctx)) + .Build() + .Add() + .Name() + .Value(ToString(EYtSettingType::SortBy), TNodeFlags::Default) + .Build() + .Value(ToAtomList(ytSortByColumns, pos, ctx)) + .Build(); + + auto mapLambda = Build<TCoLambda>(ctx, pos) + .Args({"flow"}) + .Body<TCoOrderedFlatMap>() + .Input<TExprApplier>() + .Apply(mapCombinerLambda) + .With(0, "flow") + .Build() + .Lambda() + .Args({"item"}) + .Body<TCoVisit>() + .Input("item") + .FreeArgs() + .Add<TCoAtom>() + .Value(0U) + .Build() + .Add(cjcLambdas[0].MapLambda) + .Add<TCoAtom>() + .Value(1U) + .Build() + .Add(cjcLambdas[1].MapLambda) + .Build() + .Build() + .Build() + .Build().Done().Ptr(); + + TYtOutTableInfo outInfo(outItemType, state->Configuration->UseNativeYtTypes.Get().GetOrElse(DEFAULT_USE_NATIVE_YT_TYPES) ? NTCF_ALL : NTCF_NONE); + outInfo.RowSpec->SetConstraints(op.Constraints); + outInfo.SetUnique(op.Constraints.GetConstraint<TDistinctConstraintNode>(), pos, ctx); + const auto outTableInfo = outInfo.ToExprNode(ctx, pos).Ptr(); + + TExprNode::TListType mapReduceOutputs; + TVector<bool> mapReduceOutputsSingleValue; + + mapReduceOutputs.push_back(outTableInfo); + if (!isCrossJoin && state->Configuration->JoinCommonUseMapMultiOut.Get().GetOrElse(DEFAULT_JOIN_COMMON_USE_MULTI_OUT)) { + bool leftNulls = false; + if (!leftNotFat && joinType->IsAtom({"Left", "Full", "Exclusion"})) { + leftNulls = JoinKeysMayHaveNulls(inputKeyTypeLeft, outputKeyType); + if (leftNulls) { + mapReduceOutputs.push_back(outTableInfo); + mapReduceOutputsSingleValue.push_back(op.LinkSettings.LeftHints.contains("any")); + } + } + + bool rightNulls = false; + if (!rightNotFat && joinType->IsAtom({"Right", "Full", "Exclusion"})) { + rightNulls = JoinKeysMayHaveNulls(inputKeyTypeRight, outputKeyType); + if (rightNulls) { + mapReduceOutputs.push_back(outTableInfo); + mapReduceOutputsSingleValue.push_back(op.LinkSettings.RightHints.contains("any")); + } + } + + if (leftNulls || rightNulls) { + TExprNode::TPtr itemArg = ctx.NewArgument(pos, "item"); + TExprNode::TListType outputVarTypeItems; + + // output to reducer + outputVarTypeItems.push_back(ctx.NewCallable(pos, "TypeOf", { itemArg })); + + // direct outputs + size_t leftOutputIndex = 0; + if (leftNulls) { + leftOutputIndex = outputVarTypeItems.size(); + outputVarTypeItems.push_back(ExpandType(pos, *outItemType, ctx)); + } + size_t rightOutputIndex = 0; + if (rightNulls) { + rightOutputIndex = outputVarTypeItems.size(); + outputVarTypeItems.push_back(ExpandType(pos, *outItemType, ctx)); + } + + auto variantType = ctx.Builder(pos) + .Callable("VariantType") + .Add(0, ctx.NewCallable(pos, "TupleType", std::move(outputVarTypeItems))) + .Seal() + .Build(); + + const TString leftSidePrefix = labels.Inputs[0].AddLabel ? (TStringBuilder() << labels.Inputs[0].Tables[0] << ".") : TString(); + TExprNode::TPtr leftVisitLambda = BuildSideSplitNullsLambda(pos, leftNulls, itemArg, ytReduceByColumns, leftSidePrefix, + joinRenamingLambda, *outItemTypeBeforeRename, variantType, leftOutputIndex, ctx); + + const TString rightSidePrefix = labels.Inputs[1].AddLabel ? (TStringBuilder() << labels.Inputs[1].Tables[0] << ".") : TString(); + TExprNode::TPtr rightVisitLambda = BuildSideSplitNullsLambda(pos, rightNulls, itemArg, ytReduceByColumns, rightSidePrefix, + joinRenamingLambda, *outItemTypeBeforeRename, variantType, rightOutputIndex, ctx); + + auto splitLambdaBody = ctx.Builder(pos) + .Callable("Visit") + .Callable(0, "Member") + .Add(0, itemArg) + .Atom(1, "_yql_join_payload", TNodeFlags::Default) + .Seal() + .Atom(1, 0U) + .Add(2, leftVisitLambda) + .Atom(3, 1U) + .Add(4, rightVisitLambda) + .Seal() + .Build(); + + mapLambda = ctx.Builder(pos) + .Lambda() + .Param("stream") + .Callable("OrderedMap") + .Apply(0, mapLambda) + .With(0, "stream") + .Seal() + .Add(1, ctx.NewLambda(pos, ctx.NewArguments(pos, { itemArg }), std::move(splitLambdaBody))) + .Seal() + .Seal() + .Build(); + } + } + + if (state->Configuration->UseFlow.Get().GetOrElse(DEFAULT_USE_FLOW)) { + settingsBuilder + .Add() + .Name() + .Value(ToString(EYtSettingType::Flow), TNodeFlags::Default) + .Build() + .Build(); + } + + const auto mapreduce = Build<TYtMapReduce>(ctx, pos) + .World(equiJoin.World()) + .DataSink(equiJoin.DataSink()) + .Input() + .Add() + .Paths(MakeUnorderedSection(leftLeaf.Section, ctx).Paths()) + .Settings(NYql::RemoveSettings(leftLeaf.Section.Settings().Ref(), EYtSettingType::JoinLabel | EYtSettingType::StatColumns, ctx)) + .Build() + .Add() + .Paths(MakeUnorderedSection(rightLeaf.Section, ctx).Paths()) + .Settings(NYql::RemoveSettings(rightLeaf.Section.Settings().Ref(), EYtSettingType::JoinLabel | EYtSettingType::StatColumns, ctx)) + .Build() + .Build() + .Output(ctx.NewList(pos, std::move(mapReduceOutputs))) + .Settings(settingsBuilder.Done()) + .Mapper(mapLambda) + .Reducer(reducer) + .Done(); + + if (mapReduceOutputsSingleValue.empty()) { + op.Output = mapreduce; + } else { + TVector<TYtPath> paths; + ui32 idx = 0U; + for (bool single: mapReduceOutputsSingleValue) { + TExprBase ranges = Build<TCoVoid>(ctx, pos).Done(); + if (single) { + ranges = Build<TExprList>(ctx, pos) + .Add<TYtRow>() + .Index<TCoUint64>() + .Literal() + .Value(0U) + .Build() + .Build() + .Build() + .Done(); + } + + paths.push_back(Build<TYtPath>(ctx, pos) + .Table<TYtOutput>() + .Operation(mapreduce) + .OutIndex().Value(idx++).Build() + .Build() + .Columns<TCoVoid>().Build() + .Ranges(ranges) + .Stat<TCoVoid>().Build() + .Done() + ); + } + + paths.push_back(Build<TYtPath>(ctx, pos) + .Table<TYtOutput>() + .Operation(mapreduce) + .OutIndex().Value(idx++).Build() + .Build() + .Columns<TCoVoid>().Build() + .Ranges<TCoVoid>().Build() + .Stat<TCoVoid>().Build() + .Done() + ); + + op.Output = Build<TYtMerge>(ctx, pos) + .World(equiJoin.World()) + .DataSink(equiJoin.DataSink()) + .Input() + .Add() + .Paths() + .Add(paths) + .Build() + .Settings() + .Build() + .Build() + .Build() + .Output() + .Add(outTableInfo) + .Build() + .Settings() + .Build() + .Done(); + } + + return true; +} + +bool RewriteYtEmptyJoin(TYtEquiJoin equiJoin, const TJoinLabels& labels, TYtJoinNodeOp& op, + const TYtJoinNodeLeaf& leftLeaf, const TYtJoinNodeLeaf& rightLeaf, const TYtState::TPtr& state, TExprContext& ctx) +{ + auto pos = equiJoin.Pos(); + + YQL_CLOG(INFO, ProviderYt) << "EmptyJoin"; + + const TStructExprType* outItemType = nullptr; + if (!op.Parent) { + if (auto type = GetSequenceItemType(pos, equiJoin.Ref().GetTypeAnn()->Cast<TTupleExprType>()->GetItems()[1], false, ctx)) { + outItemType = type->Cast<TStructExprType>(); + } else { + return false; + } + } else { + auto joinTree = ctx.NewList(pos, { + op.JoinKind, + ctx.NewAtom(pos, leftLeaf.Scope[0]), + ctx.NewAtom(pos, rightLeaf.Scope[0]), + op.LeftLabel, + op.RightLabel, + ctx.NewList(pos, {}) + }); + + outItemType = MakeIntermediateEquiJoinTableType(pos, *joinTree, labels, op.OutputRemoveColumns, ctx); + if (!outItemType) { + return false; + } + } + + TSyncMap syncList; + for (auto path: leftLeaf.Section.Paths()) { + if (auto out = path.Table().Maybe<TYtOutput>()) { + syncList.emplace(out.Cast().Operation().Ptr(), syncList.size()); + } + } + for (auto path: rightLeaf.Section.Paths()) { + if (auto out = path.Table().Maybe<TYtOutput>()) { + syncList.emplace(out.Cast().Operation().Ptr(), syncList.size()); + } + } + + TYtOutTableInfo outTableInfo(outItemType, state->Configuration->UseNativeYtTypes.Get().GetOrElse(DEFAULT_USE_NATIVE_YT_TYPES) ? NTCF_ALL : NTCF_NONE); + outTableInfo.RowSpec->SetConstraints(op.Constraints); + outTableInfo.SetUnique(op.Constraints.GetConstraint<TDistinctConstraintNode>(), pos, ctx); + + op.Output = Build<TYtTouch>(ctx, pos) + .World(ApplySyncListToWorld(equiJoin.World().Ptr(), syncList, ctx)) + .DataSink(equiJoin.DataSink()) + .Output() + .Add(outTableInfo.ToExprNode(ctx, pos).Cast<TYtOutTable>()) + .Build() + .Done(); + + return true; +} + +TStatus CollectJoinSideStats(ESizeStatCollectMode sizeMode, TJoinSideStats& stats, TYtSection& inputSection, + const TYtState& state, const TString& cluster, + const TVector<TYtPathInfo::TPtr>& tableInfo, const THashSet<TString>& joinKeys, + bool isCross, TMaybeNode<TCoLambda> premap, TExprContext& ctx) +{ + stats = {}; + + stats.HasUniqueKeys = !isCross; + stats.IsDynamic = AnyOf(tableInfo, [](const TYtPathInfo::TPtr& path) { + return path->Table->Meta->IsDynamic; + }); + const ui64 nativeTypeFlags = state.Configuration->UseNativeYtTypes.Get().GetOrElse(DEFAULT_USE_NATIVE_YT_TYPES) && inputSection.Ref().GetTypeAnn() + ? GetNativeYtTypeFlags(*inputSection.Ref().GetTypeAnn()->Cast<TListExprType>()->GetItemType()->Cast<TStructExprType>()) + : 0ul; + TMaybe<NYT::TNode> firstNativeType; + if (!tableInfo.empty()) { + firstNativeType = tableInfo.front()->GetNativeYtType(); + } + stats.NeedsRemap = NYql::HasSetting(inputSection.Settings().Ref(), EYtSettingType::SysColumns) + || AnyOf(tableInfo, [nativeTypeFlags, firstNativeType](const TYtPathInfo::TPtr& path) { + return path->RequiresRemap() + || path->Table->RowSpec->HasAuxColumns() // TODO: remove + || nativeTypeFlags != path->GetNativeYtTypeFlags() + || firstNativeType != path->GetNativeYtType(); + }); + + bool first = true; + for (auto& path: tableInfo) { + if (sizeMode != ESizeStatCollectMode::NoSize) { + YQL_ENSURE(path->Table->Stat); + auto tableRecords = path->Table->Stat->RecordsCount; + if (path->Ranges) { + tableRecords = path->Ranges->GetUsedRows(tableRecords).GetOrElse(tableRecords); + } + stats.RowsCount += tableRecords; + stats.Size += path->Table->Stat->DataSize; + } + + if (!stats.TableNames.empty()) { + stats.TableNames += " | "; + } + + stats.TableNames += path->Table->Name; + if (!isCross) { + UpdateSortPrefix(first, stats.SortedKeys, path->Table->RowSpec, stats.HasUniqueKeys, joinKeys, premap); + } + first = false; + } + + if (sizeMode != ESizeStatCollectMode::ColumnarSize) { + return TStatus::Ok; + } + + TVector<ui64> dataSizes; + auto status = TryEstimateDataSizeChecked(dataSizes, inputSection, cluster, tableInfo, {}, state, ctx); + if (status.Level != TStatus::Ok) { + return status; + } + + stats.Size = Accumulate(dataSizes.begin(), dataSizes.end(), 0ull, [](ui64 sum, ui64 v) { return sum + v; }); + return TStatus::Ok; +} + +TStatus CollectPathsAndLabels(TVector<TYtPathInfo::TPtr>& tables, TJoinLabels& labels, + const TStructExprType*& itemType, const TStructExprType*& itemTypeBeforePremap, + const TYtJoinNodeLeaf& leaf, TExprContext& ctx) +{ + tables = {}; + itemType = nullptr; + itemTypeBeforePremap = nullptr; + + TExprBase input = leaf.Section; + if (auto type = GetSequenceItemType(input, false, ctx)) { + itemTypeBeforePremap = type->Cast<TStructExprType>(); + } else { + return TStatus::Error; + } + + if (leaf.Premap) { + input = leaf.Premap.Cast(); + } + + if (auto type = GetSequenceItemType(input, false, ctx)) { + itemType = type->Cast<TStructExprType>(); + } else { + return TStatus::Error; + } + + if (auto err = labels.Add(ctx, *leaf.Label, itemType)) { + ctx.AddError(*err); + return TStatus::Error; + } + + for (auto path: leaf.Section.Paths()) { + auto pathInfo = MakeIntrusive<TYtPathInfo>(path); + tables.push_back(pathInfo); + } + + return TStatus::Ok; +} + +TStatus RewriteYtEquiJoinLeaf(TYtEquiJoin equiJoin, TYtJoinNodeOp& op, TYtJoinNodeLeaf& leftLeaf, + TYtJoinNodeLeaf& rightLeaf, const TYtState::TPtr& state, TExprContext& ctx) +{ + TJoinLabels labels; + + bool leftTablesReady = false; + TVector<TYtPathInfo::TPtr> leftTables; + bool rightTablesReady = false; + TVector<TYtPathInfo::TPtr> rightTables; + const TStructExprType* leftItemType = nullptr; + const TStructExprType* leftItemTypeBeforePremap = nullptr; + const TStructExprType* rightItemType = nullptr; + const TStructExprType* rightItemTypeBeforePremap = nullptr; + + { + auto status = CollectPathsAndLabelsReady(leftTablesReady, leftTables, labels, leftItemType, leftItemTypeBeforePremap, leftLeaf, ctx); + if (status != TStatus::Ok) { + YQL_ENSURE(status.Level == TStatus::Error); + return status; + } + + status = CollectPathsAndLabelsReady(rightTablesReady, rightTables, labels, rightItemType, rightItemTypeBeforePremap, rightLeaf, ctx); + if (status != TStatus::Ok) { + YQL_ENSURE(status.Level == TStatus::Error); + return status; + } + } + + const auto joinType = op.JoinKind->Content(); + const auto disableOptimizers = state->Configuration->DisableOptimizers.Get().GetOrElse(TSet<TString>()); + + bool empty = false; + if (leftLeaf.Section.Ref().GetConstraint<TEmptyConstraintNode>() != nullptr + && AllOf(leftTables, [](const TYtPathInfo::TPtr& p) { return !p->Table->Meta->IsDynamic; }) + ) { + if (joinType == "Inner" || joinType == "Left" || joinType == "LeftOnly" || joinType == "LeftSemi" || joinType == "RightSemi" || joinType == "Cross") { + empty = true; + YQL_CLOG(INFO, ProviderYt) << "Left join side is empty"; + } + } + if (!empty + && rightLeaf.Section.Ref().GetConstraint<TEmptyConstraintNode>() != nullptr + && AllOf(rightTables, [](const TYtPathInfo::TPtr& p) { return !p->Table->Meta->IsDynamic; }) + ) { + if (joinType == "Inner" || joinType == "Right" || joinType == "RightOnly" || joinType == "RightSemi" || joinType == "LeftSemi" || joinType == "Cross") { + empty = true; + YQL_CLOG(INFO, ProviderYt) << "Right join side is empty"; + } + } + if (empty) { + return RewriteYtEmptyJoin(equiJoin, labels, op, leftLeaf, rightLeaf, state, ctx) + ? TStatus::Ok + : TStatus::Error; + } + + const bool isCross = joinType == "Cross"; + const unsigned readyCount = unsigned(leftTablesReady) + rightTablesReady; + if (isCross && readyCount < 2) { + return TStatus::Repeat; + } + + auto leftJoinKeys = BuildJoinKeys(labels.Inputs[0], *op.LeftLabel); + auto rightJoinKeys = BuildJoinKeys(labels.Inputs[1], *op.RightLabel); + auto leftJoinKeyList = BuildJoinKeyList(labels.Inputs[0], *op.LeftLabel); + auto rightJoinKeyList = BuildJoinKeyList(labels.Inputs[1], *op.RightLabel); + YQL_ENSURE(leftJoinKeys.size() <= leftJoinKeyList.size()); + YQL_ENSURE(rightJoinKeys.size() <= rightJoinKeyList.size()); + YQL_ENSURE(leftJoinKeyList.size() == rightJoinKeyList.size()); + if (!isCross) { + YQL_CLOG(INFO, ProviderYt) << "leftJoinKeys: " << JoinSeq(",", leftJoinKeyList) + << ", rightJoinKeys: " << JoinSeq(",", rightJoinKeyList); + } + + auto const& linkSettings = op.LinkSettings; + ui64 mergeTablesLimit = 0; + TMaybe<bool> mergeUseSmallAsPrimary; + double mergeUnsortedFactor = 0.2; + bool forceMergeJoin = false; + if (auto limit = state->Configuration->JoinMergeTablesLimit.Get()) { + YQL_CLOG(INFO, ProviderYt) << "JoinMergeTablesLimit: " << *limit; + mergeTablesLimit = *limit; + + if (mergeUseSmallAsPrimary = state->Configuration->JoinMergeUseSmallAsPrimary.Get()) { + YQL_CLOG(INFO, ProviderYt) << "JoinMergeUseSmallAsPrimary: " << *mergeUseSmallAsPrimary; + } + + if (auto factor = state->Configuration->JoinMergeUnsortedFactor.Get()) { + YQL_CLOG(INFO, ProviderYt) << "JoinMergeUnsortedFactor: " << *factor; + mergeUnsortedFactor = *factor; + } + + if (auto force = state->Configuration->JoinMergeForce.Get()) { + YQL_CLOG(INFO, ProviderYt) << "JoinMergeForce: " << *force; + forceMergeJoin = *force; + } else if (linkSettings.ForceSortedMerge) { + YQL_CLOG(INFO, ProviderYt) << "Got forceSortedMerge from link settings"; + forceMergeJoin = true; + } + } + if (!readyCount && !forceMergeJoin) { + return TStatus::Repeat; + } + + auto cluster = TString{equiJoin.DataSink().Cluster().Value()}; + + TMapJoinSettings mapSettings; + TJoinSideStats leftStats; + TJoinSideStats rightStats; + + const bool allowLookupJoin = !isCross && leftTablesReady && rightTablesReady && !forceMergeJoin; + if (allowLookupJoin) { + auto status = CollectStatsAndMapJoinSettings(ESizeStatCollectMode::RawSize, mapSettings, leftStats, rightStats, + leftTablesReady, leftTables, leftJoinKeys, rightTablesReady, rightTables, rightJoinKeys, + &leftLeaf, &rightLeaf, *state, isCross, cluster, ctx); + if (status.Level != TStatus::Ok) { + return (status.Level == TStatus::Repeat) ? TStatus::Ok : status; + } + + YQL_CLOG(INFO, ProviderYt) << "Considering LookupJoin: left table(s): " + << leftStats.TableNames << " with size: " << mapSettings.LeftSize << ", rows: " + << mapSettings.LeftRows << ", dynamic : " << leftStats.IsDynamic << ", right table(s): " + << rightStats.TableNames << " with size: " << mapSettings.RightSize << ", rows: " + << mapSettings.RightRows << ", dynamic : " << rightStats.IsDynamic; + + YQL_CLOG(INFO, ProviderYt) << "Join kind: " << op.JoinKind->Content() << ", left unique: " << leftStats.HasUniqueKeys + << ", right unique: " << rightStats.HasUniqueKeys << ", left sorted prefix: [" + << JoinSeq(",", leftStats.SortedKeys) << "], right sorted prefix: [" + << JoinSeq(",", rightStats.SortedKeys) << "]"; + + auto lookupJoinLimit = Min(state->Configuration->LookupJoinLimit.Get().GetOrElse(0), + state->Configuration->EvaluationTableSizeLimit.Get().GetOrElse(Max<ui64>())); + auto lookupJoinMaxRows = state->Configuration->LookupJoinMaxRows.Get().GetOrElse(0); + + bool isLeftSorted = leftStats.SortedKeys.size() >= leftJoinKeys.size(); + bool isRightSorted = rightStats.SortedKeys.size() >= rightJoinKeys.size(); + + // TODO: support ANY for left side via Reduce with PartitionByKey + bool leftAny = linkSettings.LeftHints.contains("any"); + bool rightAny = linkSettings.RightHints.contains("any"); + + bool isLeftAllowLookupJoin = !leftAny && isLeftSorted && (mapSettings.RightRows < lookupJoinMaxRows) && + (mapSettings.RightMemSize < lookupJoinLimit) && + (joinType == "Inner" || joinType == "LeftSemi") && !rightStats.IsDynamic; + + bool isRightAllowLookupJoin = !rightAny && isRightSorted && (mapSettings.LeftRows < lookupJoinMaxRows) && + (mapSettings.LeftMemSize < lookupJoinLimit) && + (joinType == "Inner" || joinType == "RightSemi") && !leftStats.IsDynamic; + + YQL_CLOG(INFO, ProviderYt) << "LookupJoin: isLeftAllowLookupJoin: " << isLeftAllowLookupJoin + << ", isRightAllowLookupJoin: " << isRightAllowLookupJoin; + + if (isLeftAllowLookupJoin || isRightAllowLookupJoin) { + bool swapTables = false; + if (isLeftAllowLookupJoin && isRightAllowLookupJoin) { + swapTables = (mapSettings.LeftSize < mapSettings.RightSize); + } + else if (!isLeftAllowLookupJoin) { + swapTables = true; + } + + mapSettings.SwapTables = swapTables; + + if (swapTables) { + DoSwap(mapSettings.LeftRows, mapSettings.RightRows); + DoSwap(mapSettings.LeftSize, mapSettings.RightSize); + DoSwap(mapSettings.LeftMemSize, mapSettings.RightMemSize); + DoSwap(mapSettings.LeftUnique, mapSettings.RightUnique); + YQL_CLOG(INFO, ProviderYt) << "Selected LookupJoin: filter over the right table, use content of the left one, " << (op.LinkSettings.JoinAlgo != EJoinAlgoType::Undefined ? ToString(op.LinkSettings.JoinAlgo).c_str() : "no") << " cbo algo"; + + return RewriteYtMapJoin(equiJoin, labels, true, op, rightLeaf, leftLeaf, ctx, mapSettings, false, state) ? + TStatus::Ok : TStatus::Error; + } else { + YQL_CLOG(INFO, ProviderYt) << "Selected LookupJoin: filter over the left table, use content of the right one, " << (op.LinkSettings.JoinAlgo != EJoinAlgoType::Undefined ? ToString(op.LinkSettings.JoinAlgo).c_str() : "no") << " cbo algo"; + + return RewriteYtMapJoin(equiJoin, labels, true, op, leftLeaf, rightLeaf, ctx, mapSettings, false, state) ? + TStatus::Ok : TStatus::Error; + } + } + } + + { + auto status = CollectStatsAndMapJoinSettings(ESizeStatCollectMode::ColumnarSize, mapSettings, leftStats, rightStats, + leftTablesReady, leftTables, leftJoinKeys, rightTablesReady, rightTables, rightJoinKeys, + &leftLeaf, &rightLeaf, *state, isCross, cluster, ctx); + if (status.Level != TStatus::Ok) { + return (status.Level == TStatus::Repeat) ? TStatus::Ok : status; + } + } + + YQL_CLOG(INFO, ProviderYt) << "Left table(s): " + << (leftTablesReady ? leftStats.TableNames : "(not ready)") << " with size: " << mapSettings.LeftSize << ", rows: " + << mapSettings.LeftRows << ", dynamic : " << leftStats.IsDynamic << ", right table(s): " + << (rightTablesReady ? rightStats.TableNames : "(not ready)") << " with size: " << mapSettings.RightSize << ", rows: " + << mapSettings.RightRows << ", dynamic : " << rightStats.IsDynamic; + + YQL_CLOG(INFO, ProviderYt) << "Join kind: " << op.JoinKind->Content() << ", left hints: " << + (linkSettings.LeftHints ? JoinSeq(",", linkSettings.LeftHints) : "none") << ", right hints: " << + (linkSettings.RightHints ? JoinSeq(",", linkSettings.RightHints) : "none") << ", left unique: " << leftStats.HasUniqueKeys + << ", right unique: " << rightStats.HasUniqueKeys << ", left sorted prefix: [" + << JoinSeq(",", leftStats.SortedKeys) << "], right sorted prefix: [" + << JoinSeq(",", rightStats.SortedKeys) << "]"; + + bool allowOrderedJoin = !isCross && ((leftTablesReady && rightTablesReady) || forceMergeJoin); + + TMergeJoinSortInfo sortInfo; + sortInfo.LeftSortedKeys = leftStats.SortedKeys; + sortInfo.RightSortedKeys = rightStats.SortedKeys; + + sortInfo.LeftBeforePremap = leftItemTypeBeforePremap; + sortInfo.RightBeforePremap = rightItemTypeBeforePremap; + + if (allowOrderedJoin) { + bool isLeftSorted = sortInfo.LeftSortedKeys.size() >= leftJoinKeys.size(); + bool isRightSorted = sortInfo.RightSortedKeys.size() >= rightJoinKeys.size(); + + if (allowOrderedJoin && !isLeftSorted && !isRightSorted && !forceMergeJoin) { + YQL_CLOG(INFO, ProviderYt) << "Skipped OrderedJoin, because both sides are unsorted"; + allowOrderedJoin = false; + } + + if (allowOrderedJoin && (leftJoinKeys.size() != leftJoinKeyList.size() || + rightJoinKeys.size() != rightJoinKeyList.size())) + { + YQL_CLOG(INFO, ProviderYt) << "Skipped OrderedJoin, because side(s) contain duplicate join keys"; + allowOrderedJoin = false; + } + + if (allowOrderedJoin && !forceMergeJoin && (mapSettings.LeftCount + mapSettings.RightCount > mergeTablesLimit)) { + YQL_CLOG(INFO, ProviderYt) << "Skipped OrderedJoin, because there are too many tables, mergeTablesLimit: " << mergeTablesLimit; + allowOrderedJoin = false; + } + + if (allowOrderedJoin && (!isLeftSorted || !isRightSorted)) { + if (!isLeftSorted && !isRightSorted) { + YQL_ENSURE(forceMergeJoin); + sortInfo.AdditionalSort = TChoice::Both; + sortInfo.LeftSortedKeys = leftJoinKeyList; + sortInfo.RightSortedKeys = BuildCompatibleSortWith(sortInfo.LeftSortedKeys, leftJoinKeyList, rightJoinKeyList); + } else if (!isRightSorted) { + if (!forceMergeJoin && mapSettings.RightSize > mapSettings.LeftSize * mergeUnsortedFactor) { + YQL_CLOG(INFO, ProviderYt) << "Skipped OrderedJoin, because unsorted right table is too big"; + allowOrderedJoin = false; + } else { + sortInfo.AdditionalSort = TChoice::Right; + sortInfo.RightSortedKeys = BuildCompatibleSortWith(sortInfo.LeftSortedKeys, leftJoinKeyList, rightJoinKeyList); + } + } else { + if (!forceMergeJoin && mapSettings.LeftSize > mapSettings.RightSize * mergeUnsortedFactor) { + YQL_CLOG(INFO, ProviderYt) << "Skipped OrderedJoin, because unsorted left table is too big"; + allowOrderedJoin = false; + } else { + sortInfo.AdditionalSort = TChoice::Left; + sortInfo.LeftSortedKeys = BuildCompatibleSortWith(sortInfo.RightSortedKeys, rightJoinKeyList, leftJoinKeyList); + } + } + + if (allowOrderedJoin && !isLeftSorted) { + isLeftSorted = true; + YQL_ENSURE(sortInfo.LeftSortedKeys.size() == leftJoinKeys.size()); + if (leftStats.NeedsRemap) { + sortInfo.NeedRemapBeforeSort = Merge(sortInfo.NeedRemapBeforeSort, TChoice::Left); + } + YQL_CLOG(INFO, ProviderYt) << "Added sort of the left table" + << (leftStats.NeedsRemap ? " with additional remapping" : ""); + } + + if (allowOrderedJoin && !isRightSorted) { + isRightSorted = true; + YQL_ENSURE(sortInfo.RightSortedKeys.size() == rightJoinKeys.size()); + if (rightStats.NeedsRemap) { + sortInfo.NeedRemapBeforeSort = Merge(sortInfo.NeedRemapBeforeSort, TChoice::Right); + } + YQL_CLOG(INFO, ProviderYt) << "Added sort of the right table" + << (rightStats.NeedsRemap ? " with additional remapping" : ""); + } + } + + if (allowOrderedJoin) { + YQL_ENSURE(isLeftSorted); + YQL_ENSURE(isRightSorted); + + YQL_ENSURE(sortInfo.LeftSortedKeys.size() >= leftJoinKeys.size()); + YQL_ENSURE(sortInfo.RightSortedKeys.size() >= rightJoinKeys.size()); + + // TODO: in some cases we can allow output to be sorted more strictly + sortInfo.LeftSortedKeys.resize(leftJoinKeys.size()); + sortInfo.RightSortedKeys.resize(rightJoinKeys.size()); + + const bool allowColumnRenames = state->Configuration->JoinAllowColumnRenames.Get().GetOrElse(true) + // TODO: remove next line after https://st.yandex-team.ru/YT-12738 is fixed + && !leftStats.IsDynamic && !rightStats.IsDynamic; + + sortInfo.CommonSortedKeys = BuildCommonSortPrefix(sortInfo.LeftSortedKeys, sortInfo.RightSortedKeys, + leftJoinKeyList, rightJoinKeyList, sortInfo.LeftKeyRenames, sortInfo.RightKeyRenames, allowColumnRenames); + + YQL_CLOG(INFO, ProviderYt) << "Common sorted prefix (with JoinAllowColumnRenames: " << allowColumnRenames << ") is [" + << JoinSeq(", ", sortInfo.CommonSortedKeys) << "], left sorted prefix: [" + << JoinSeq(", ", sortInfo.LeftSortedKeys) << "], right sorted prefix: [" + << JoinSeq(", ", sortInfo.RightSortedKeys) << "], left renames: [" + << RenamesToString(sortInfo.LeftKeyRenames) << "], right renames: [" + << RenamesToString(sortInfo.RightKeyRenames) << "]"; + + if (sortInfo.CommonSortedKeys.size() < leftJoinKeys.size()) { + YQL_CLOG(INFO, ProviderYt) << "Skipped OrderedJoin, because table sort prefixes are incompatible with " + "join keys"; + allowOrderedJoin = false; + } + } + } + + if (allowOrderedJoin) { + if (joinType.EndsWith("Semi")) { + const bool leftSemi = joinType == "LeftSemi"; + const auto& dictLabel = leftSemi ? labels.Inputs[1] : labels.Inputs[0]; + auto& dictHints = leftSemi ? op.LinkSettings.RightHints : op.LinkSettings.LeftHints; + + op.JoinKind = ctx.NewAtom(op.JoinKind->Pos(), "Inner"); + dictHints.insert("any"); + + auto columnsToRemove = dictLabel.EnumerateAllColumns(); + op.OutputRemoveColumns.insert(columnsToRemove.begin(), columnsToRemove.end()); + + YQL_CLOG(INFO, ProviderYt) << "Will rewrite ordered " << joinType << " to ANY Inner + remove columns [" + << JoinSeq(", ", columnsToRemove) << "]"; + return TStatus::Ok; + } + + bool allowPrimaryLeft = (joinType == "Inner" || joinType == "Left" || joinType == "LeftOnly"); + bool allowPrimaryRight = (joinType == "Inner" || joinType == "Right" || joinType == "RightOnly"); + + bool swapTables = false; + bool useJoinReduce = false; + bool useJoinReduceForSecond = false; + bool tryFirstAsPrimary = false; + + if (!linkSettings.Compact) { + if (allowPrimaryLeft != allowPrimaryRight) { + swapTables = allowPrimaryLeft; + auto primary = swapTables ? TChoice::Left : TChoice::Right; + useJoinReduce = !HasNonTrivialAny(linkSettings, mapSettings, primary); + } else if (allowPrimaryLeft) { + YQL_ENSURE(allowPrimaryRight); + // both tables can be chosen as primary + bool biggerHasUniqueKeys = mapSettings.RightSize > mapSettings.LeftSize ? + mapSettings.RightUnique : mapSettings.LeftUnique; + + if (biggerHasUniqueKeys) { + // it is safe to use smaller table as primary + swapTables = mapSettings.RightSize > mapSettings.LeftSize; + } else if (mergeUseSmallAsPrimary) { + // explicit setting + if (*mergeUseSmallAsPrimary) { + // use smaller table as primary + swapTables = mapSettings.RightSize > mapSettings.LeftSize; + } else { + // use bigger table as primary + swapTables = mapSettings.LeftSize > mapSettings.RightSize; + } + } else { + // make bigger table last one, and try first (smaller) as primary + swapTables = mapSettings.LeftSize > mapSettings.RightSize; + tryFirstAsPrimary = true; + } + + auto primary = swapTables ? TChoice::Left : TChoice::Right; + if (tryFirstAsPrimary) { + useJoinReduceForSecond = !HasNonTrivialAny(linkSettings, mapSettings, primary); + useJoinReduce = !HasNonTrivialAny(linkSettings, mapSettings, Invert(primary)); + } else { + useJoinReduce = !HasNonTrivialAny(linkSettings, mapSettings, primary); + } + } else { + // try to move non-fat table to the left, otherwise keep them as is + if (mapSettings.LeftUnique != mapSettings.RightUnique) { + swapTables = mapSettings.RightUnique; + } else { + swapTables = mapSettings.LeftSize > mapSettings.RightSize; + } + } + } + + YQL_CLOG(INFO, ProviderYt) << "OrderedJoin allowPrimaryLeft : " << allowPrimaryLeft + << ", allowPrimaryRight " << allowPrimaryRight; + + YQL_CLOG(INFO, ProviderYt) << "Selected OrderedJoin over the " << (swapTables ? "left" : "right") + << " table as primary, will use join reduce: " << useJoinReduce << ", will try other primary: " << tryFirstAsPrimary + << ", will use join reduce for other primary: " << useJoinReduceForSecond + << ", " << (op.LinkSettings.JoinAlgo != EJoinAlgoType::Undefined ? ToString(op.LinkSettings.JoinAlgo).c_str() : "no") << " cbo algo"; + + bool skipped = false; + if (!RewriteYtMergeJoin(equiJoin, labels, op, + swapTables ? rightLeaf : leftLeaf, + swapTables ? leftLeaf : rightLeaf, + state, ctx, swapTables, useJoinReduce, tryFirstAsPrimary, useJoinReduceForSecond, + swapTables ? Invert(sortInfo) : sortInfo, + skipped)) + { + return TStatus::Error; + } + + if (!skipped) { + return TStatus::Ok; + } + } + + if (auto mapJoinLimit = state->Configuration->MapJoinLimit.Get(); mapJoinLimit && !forceMergeJoin) { + YQL_CLOG(INFO, ProviderYt) << "MapJoinLimit: " << *mapJoinLimit; + mapSettings.MapJoinLimit = *mapJoinLimit; + + if (mapSettings.MapJoinLimit) { + mapSettings.MapJoinShardMinRows = state->Configuration->MapJoinShardMinRows.Get().GetOrElse(1); + mapSettings.MapJoinShardCount = state->Configuration->MapJoinShardCount.Get().GetOrElse(1); + + ui64 rightLimit = mapSettings.MapJoinLimit; + ui64 leftLimit = mapSettings.MapJoinLimit; + auto leftPartSize = mapSettings.CalculatePartSize(mapSettings.LeftRows); + auto rightPartSize = mapSettings.CalculatePartSize(mapSettings.RightRows); + TMaybe<ui64> leftPartCount; + TMaybe<ui64> rightPartCount; + if (leftPartSize) { + YQL_ENSURE(leftTablesReady); + leftPartCount = (mapSettings.LeftRows + *leftPartSize - 1) / *leftPartSize; + } + + if (rightPartSize) { + YQL_ENSURE(rightTablesReady); + rightPartCount = (mapSettings.RightRows + *rightPartSize - 1) / *rightPartSize; + } + + bool allowShardRight = false; + const bool rightUnique = linkSettings.RightHints.contains("unique") || mapSettings.RightUnique; + const bool denyShardRight = linkSettings.RightHints.contains("any") && !rightUnique; + // TODO: currently we disable sharding when other side is not ready + if (leftTablesReady && rightPartCount && !denyShardRight && ((joinType == "Inner") || (joinType == "Cross") || + (joinType == "LeftSemi" && rightUnique))) { + allowShardRight = true; + rightLimit *= *rightPartCount; + } + + bool allowShardLeft = false; + const bool leftUnique = linkSettings.LeftHints.contains("unique") || mapSettings.LeftUnique; + const bool denyShardLeft = linkSettings.LeftHints.contains("any") && !leftUnique; + // TODO: currently we disable sharding when other side is not ready + if (rightTablesReady && leftPartCount && !denyShardLeft && ((joinType == "Inner") || (joinType == "Cross") || + (joinType == "RightSemi" && leftUnique))) { + allowShardLeft = true; + leftLimit *= *leftPartCount; + } + + auto mapJoinUseFlow = state->Configuration->MapJoinUseFlow.Get().GetOrElse(DEFAULT_MAP_JOIN_USE_FLOW); + if (leftTablesReady) { + auto status = UpdateInMemorySizeSetting(mapSettings, leftLeaf.Section, labels, op, ctx, true, leftItemType, leftJoinKeyList, state, cluster, leftTables, mapJoinUseFlow); + if (status.Level != TStatus::Ok) { + return (status.Level == TStatus::Repeat) ? TStatus::Ok : status; + } + } + + if (rightTablesReady) { + auto status = UpdateInMemorySizeSetting(mapSettings, rightLeaf.Section, labels, op, ctx, false, rightItemType, rightJoinKeyList, state, cluster, rightTables, mapJoinUseFlow); + if (status.Level != TStatus::Ok) { + return (status.Level == TStatus::Repeat) ? TStatus::Ok : status; + } + } + + YQL_CLOG(INFO, ProviderYt) << "MapJoinShardMinRows: " << mapSettings.MapJoinShardMinRows + << ", MapJoinShardCount: " << mapSettings.MapJoinShardCount + << ", left is present: " << leftTablesReady + << ", left size limit: " << leftLimit << ", left mem size:" << mapSettings.LeftMemSize + << ", right is present: " << rightTablesReady + << ", right size limit: " << rightLimit << ", right mem size: " << mapSettings.RightMemSize; + + bool leftAny = linkSettings.LeftHints.contains("any"); + bool rightAny = linkSettings.RightHints.contains("any"); + + const bool isLeftAllowMapJoin = !leftAny && rightTablesReady && (mapSettings.RightMemSize <= rightLimit) && + (joinType == "Inner" || joinType == "Left" || joinType == "LeftOnly" || joinType == "LeftSemi" || joinType == "Cross") + && !rightStats.IsDynamic; + const bool isRightAllowMapJoin = !rightAny && leftTablesReady && (mapSettings.LeftMemSize <= leftLimit) && + (joinType == "Inner" || joinType == "Right" || joinType == "RightOnly" || joinType == "RightSemi" || joinType == "Cross") + && !leftStats.IsDynamic; + YQL_CLOG(INFO, ProviderYt) << "MapJoin: isLeftAllowMapJoin: " << isLeftAllowMapJoin + << ", isRightAllowMapJoin: " << isRightAllowMapJoin; + + if (isLeftAllowMapJoin || isRightAllowMapJoin) { + bool swapTables = false; + if (isLeftAllowMapJoin && isRightAllowMapJoin) { + swapTables = (mapSettings.LeftSize < mapSettings.RightSize); + } + else if (!isLeftAllowMapJoin) { + swapTables = true; + } + + mapSettings.SwapTables = swapTables; + + if (swapTables) { + DoSwap(mapSettings.LeftRows, mapSettings.RightRows); + DoSwap(mapSettings.LeftSize, mapSettings.RightSize); + DoSwap(mapSettings.LeftMemSize, mapSettings.RightMemSize); + DoSwap(mapSettings.LeftUnique, mapSettings.RightUnique); + YQL_CLOG(INFO, ProviderYt) << "Selected MapJoin: map over the right table, use content of the left one, " << (op.LinkSettings.JoinAlgo != EJoinAlgoType::Undefined ? ToString(op.LinkSettings.JoinAlgo).c_str() : "no") << " cbo algo"; + return RewriteYtMapJoin(equiJoin, labels, false, op, rightLeaf, leftLeaf, ctx, mapSettings, allowShardLeft, state) ? + TStatus::Ok : TStatus::Error; + } else { + YQL_CLOG(INFO, ProviderYt) << "Selected MapJoin: map over the left table, use content of the right one, " << (op.LinkSettings.JoinAlgo != EJoinAlgoType::Undefined ? ToString(op.LinkSettings.JoinAlgo).c_str() : "no") << " cbo algo"; + return RewriteYtMapJoin(equiJoin, labels, false, op, leftLeaf, rightLeaf, ctx, mapSettings, allowShardRight, state) ? + TStatus::Ok : TStatus::Error; + } + } + } + } + + if (leftTablesReady && rightTablesReady) { + YQL_CLOG(INFO, ProviderYt) << "Selected CommonJoin," << (op.LinkSettings.JoinAlgo != EJoinAlgoType::Undefined ? ToString(op.LinkSettings.JoinAlgo).c_str() : "no") << " cbo algo"; + + return RewriteYtCommonJoin(equiJoin, labels, op, leftLeaf, rightLeaf, state, ctx, leftStats.HasUniqueKeys, + rightStats.HasUniqueKeys, mapSettings.LeftSize, mapSettings.RightSize) ? + TStatus::Ok : TStatus::Error; + } else { + return TStatus::Repeat; + } +} + +TExprBase MakePremap(const TYtJoinNodeLeaf& leaf, TExprContext& ctx) { + if (leaf.Premap) { + return leaf.Premap.Cast(); + } + return Build<TCoVoid>(ctx, leaf.Section.Pos()).Done(); +} + +TExprNode::TPtr ExportJoinTree(const TYtJoinNodeOp& op, TPositionHandle pos, TVector<TYtSection>& inputSections, + TVector<TExprBase>& premaps, THashSet<TString>& outputRemoveColumns, TExprContext& ctx) +{ + TExprNode::TPtr left; + if (auto leaf = dynamic_cast<const TYtJoinNodeLeaf*>(op.Left.Get())) { + left = ctx.NewAtom(pos, op.Left->Scope[0]); + inputSections.push_back(leaf->Section); + premaps.push_back(MakePremap(*leaf, ctx)); + } else { + left = ExportJoinTree(*dynamic_cast<const TYtJoinNodeOp*>(op.Left.Get()), pos, inputSections, premaps, + outputRemoveColumns, ctx); + } + + TExprNode::TPtr right; + if (auto leaf = dynamic_cast<TYtJoinNodeLeaf*>(op.Right.Get())) { + right = ctx.NewAtom(pos, op.Right->Scope[0]); + inputSections.push_back(leaf->Section); + premaps.push_back(MakePremap(*leaf, ctx)); + } + else { + right = ExportJoinTree(*dynamic_cast<const TYtJoinNodeOp*>(op.Right.Get()), pos, inputSections, premaps, + outputRemoveColumns, ctx); + } + + outputRemoveColumns.insert(op.OutputRemoveColumns.begin(), op.OutputRemoveColumns.end()); + + return ctx.NewList(pos, { + op.JoinKind, + left, + right, + op.LeftLabel, + op.RightLabel, + BuildEquiJoinLinkSettings(op.LinkSettings, ctx), + }); +} + +void CombineJoinStatus(TStatus& status, TStatus other) { + YQL_ENSURE(status == TStatus::Repeat || status == TStatus::Ok || status == TStatus::Error); + + switch (other.Level) { + case TStatus::Error: + status = TStatus::Error; + break; + case TStatus::Ok: + if (status != TStatus::Error) { + status = TStatus::Ok; + } + break; + case TStatus::Repeat: + break; + default: + YQL_ENSURE(false, "Unexpected join status"); + } +} + +enum class EStarRewriteStatus { + WaitInput, + None, + Ok, + Error, +}; + +bool IsJoinKindCompatibleWithStar(TStringBuf kind) { + return kind == "Inner" || + kind == "Left" || kind == "LeftSemi" || kind == "LeftOnly" || + kind == "Right" || kind == "RightSemi" || kind == "RightOnly"; +} + +bool IsSideSuitableForStarJoin(TStringBuf joinKind, const TEquiJoinLinkSettings& linkSettings, const TMapJoinSettings& mapJoinSettings, bool isLeft) +{ + YQL_ENSURE(IsJoinKindCompatibleWithStar(joinKind)); + + if (joinKind == (isLeft ? "Left" : "Right") || joinKind == "Inner") + { + // other side should be unique + return IsEffectivelyUnique(linkSettings, mapJoinSettings, !isLeft); + } else if (joinKind.StartsWith(isLeft ? "Left" : "Right")) { + return true; + } + + return false; +} + +bool ExtractJoinKeysForStarJoin(const TExprNode& labelNode, TString& label, TVector<TString>& keyList) { + YQL_ENSURE(labelNode.ChildrenSize() > 0); + YQL_ENSURE(labelNode.ChildrenSize() % 2 == 0); + label = {}; + keyList = {}; + keyList.reserve(labelNode.ChildrenSize() / 2); + + for (size_t i = 0; i < labelNode.ChildrenSize(); i += 2) { + YQL_ENSURE(labelNode.Child(i)->IsAtom()); + YQL_ENSURE(labelNode.Child(i + 1)->IsAtom()); + auto table = labelNode.Child(i)->Content(); + auto key = labelNode.Child(i + 1)->Content(); + + if (!label) { + label = table; + } else if (label != table) { + return false; + } + + keyList.emplace_back(key); + } + return true; +} + +TVector<const TTypeAnnotationNode*> BuildJoinKeyType(const TStructExprType& input, const TVector<TString>& keys) { + TVector<const TTypeAnnotationNode*> result; + result.reserve(keys.size()); + for (auto& key : keys) { + auto maybeIndex = input.FindItem(key); + YQL_ENSURE(maybeIndex); + result.push_back(input.GetItems()[*maybeIndex]->GetItemType()); + } + return result; +} + +constexpr auto joinFixedArgsCount = 7U; + +const TStructExprType* GetJoinInputType(TYtEquiJoin equiJoin, size_t inputIndex, TExprContext& ctx) { + const auto premap = TMaybeNode<TCoLambda>(equiJoin.Ref().ChildPtr(joinFixedArgsCount + inputIndex)); + TExprBase input = premap ? TExprBase(premap.Cast()) : TExprBase(equiJoin.Input().Item(inputIndex)); + if (auto type = GetSequenceItemType(input, false, ctx)) { + return type->Cast<TStructExprType>(); + } + return nullptr; +} + +void CollectPossibleStarJoins(const TYtEquiJoin& equiJoin, TYtJoinNodeOp& op, const TYtState::TPtr& state, EStarRewriteStatus& collectStatus, TExprContext& ctx) { + YQL_ENSURE(!op.StarOptions); + if (collectStatus != EStarRewriteStatus::Ok) { + return; + } + + auto leftLeaf = dynamic_cast<TYtJoinNodeLeaf*>(op.Left.Get()); + auto rightLeaf = dynamic_cast<TYtJoinNodeLeaf*>(op.Right.Get()); + + auto leftOp = dynamic_cast<TYtJoinNodeOp*>(op.Left.Get()); + auto rightOp = dynamic_cast<TYtJoinNodeOp*>(op.Right.Get()); + + if (!leftLeaf) { + YQL_ENSURE(leftOp); + CollectPossibleStarJoins(equiJoin, *leftOp, state, collectStatus, ctx); + } + + if (!rightLeaf) { + YQL_ENSURE(rightOp); + CollectPossibleStarJoins(equiJoin, *rightOp, state, collectStatus, ctx); + } + + if (collectStatus != EStarRewriteStatus::Ok) { + return; + } + + if (!leftLeaf && !rightLeaf) { + // join of two joins is never a star join + return; + } + + auto joinKind = op.JoinKind->Content(); + if (!IsJoinKindCompatibleWithStar(joinKind) || + (leftLeaf && leftLeaf->Scope.size() != 1) || + (rightLeaf && rightLeaf->Scope.size() != 1)) + { + return; + } + + YQL_ENSURE(!leftLeaf || leftLeaf->Label->IsAtom()); + YQL_ENSURE(!rightLeaf || rightLeaf->Label->IsAtom()); + + TJoinLabels labels; + + bool leftTablesReady = false; + TVector<TYtPathInfo::TPtr> leftTables; + bool rightTablesReady = false; + TVector<TYtPathInfo::TPtr> rightTables; + const TStructExprType* leftItemType = nullptr; + const TStructExprType* leftItemTypeBeforePremap = nullptr; + const TStructExprType* rightItemType = nullptr; + const TStructExprType* rightItemTypeBeforePremap = nullptr; + + THashSet<TString> leftJoinKeys; + THashSet<TString> rightJoinKeys; + + TVector<TString> leftJoinKeyList; + TVector<TString> rightJoinKeyList; + + if (leftLeaf) { + leftTablesReady = true; + auto status = CollectPathsAndLabels(leftTables, labels, leftItemType, leftItemTypeBeforePremap, *leftLeaf, ctx); + if (status != TStatus::Ok) { + YQL_ENSURE(status.Level == TStatus::Error); + collectStatus = EStarRewriteStatus::Error; + return; + } + + leftJoinKeys = BuildJoinKeys(labels.Inputs[0], *op.LeftLabel); + leftJoinKeyList = BuildJoinKeyList(labels.Inputs[0], *op.LeftLabel); + YQL_ENSURE(leftJoinKeys.size() <= leftJoinKeyList.size()); + } + + if (rightLeaf) { + rightTablesReady = true; + auto status = CollectPathsAndLabels(rightTables, labels, rightItemType, rightItemTypeBeforePremap, *rightLeaf, ctx); + if (status != TStatus::Ok) { + YQL_ENSURE(status.Level == TStatus::Error); + collectStatus = EStarRewriteStatus::Error; + return; + } + + rightJoinKeys = BuildJoinKeys(labels.Inputs[leftLeaf ? 1 : 0], *op.RightLabel); + rightJoinKeyList = BuildJoinKeyList(labels.Inputs[leftLeaf ? 1 : 0], *op.RightLabel); + } + + + auto cluster = TString{equiJoin.DataSink().Cluster().Value()}; + + TMapJoinSettings mapSettings; + TJoinSideStats leftStats; + TJoinSideStats rightStats; + + { + bool isCross = false; + auto status = CollectStatsAndMapJoinSettings(ESizeStatCollectMode::NoSize, mapSettings, leftStats, rightStats, + leftTablesReady, leftTables, leftJoinKeys, rightTablesReady, rightTables, rightJoinKeys, + leftLeaf, rightLeaf, *state, isCross, cluster, ctx); + + switch (status.Level) { + case TStatus::Error: + collectStatus = EStarRewriteStatus::Error; + break; + case TStatus::Ok: + break; + default: + YQL_ENSURE(false, "Unexpected collect stats status"); + } + + if (collectStatus != EStarRewriteStatus::Ok) { + return; + } + } + + if (leftLeaf) { + if (leftStats.SortedKeys.size() < leftJoinKeys.size()) { + // left is not sorted + return; + } + + if (leftJoinKeyList.size() != leftJoinKeys.size()) { + // right side contains duplicate join keys + return; + } + } + + if (rightLeaf) { + if (rightStats.SortedKeys.size() < rightJoinKeys.size()) { + // right is not sorted + return; + } + + if (rightJoinKeyList.size() != rightJoinKeys.size()) { + // right side contains duplicate join keys + return; + } + } + + auto addStarOption = [&](bool isLeft) { + + const auto& joinKeys = isLeft ? leftJoinKeys : rightJoinKeys; + + TYtStarJoinOption starJoinOption; + starJoinOption.StarKeys.insert(joinKeys.begin(), joinKeys.end()); + starJoinOption.StarInputIndex = isLeft ? leftLeaf->Index : rightLeaf->Index; + starJoinOption.StarLabel = isLeft ? leftLeaf->Label->Content() : rightLeaf->Label->Content(); + starJoinOption.StarSortedKeys = isLeft ? leftStats.SortedKeys : rightStats.SortedKeys; + + YQL_CLOG(INFO, ProviderYt) << "Adding " << (isLeft ? rightLeaf->Label->Content() : leftLeaf->Label->Content()) + << " [" << JoinSeq(", ", isLeft ? rightJoinKeyList : leftJoinKeyList) + << "] to star " << starJoinOption.StarLabel << " [" << JoinSeq(", ", starJoinOption.StarKeys) + << "]"; + + op.StarOptions.emplace_back(std::move(starJoinOption)); + }; + + if (leftLeaf && rightLeaf) { + YQL_ENSURE(leftLeaf->Label->Content() != rightLeaf->Label->Content()); + + YQL_ENSURE(leftItemType && rightItemType); + auto inputKeyTypeLeft = BuildJoinKeyType(*leftItemType, leftJoinKeyList); + auto inputKeyTypeRight = BuildJoinKeyType(*rightItemType, rightJoinKeyList); + if (!IsSameAnnotation(*AsDictKeyType(RemoveNullsFromJoinKeyType(inputKeyTypeLeft), ctx), + *AsDictKeyType(RemoveNullsFromJoinKeyType(inputKeyTypeRight), ctx))) + { + // key types should match for merge star join to work + return; + } + + if (IsSideSuitableForStarJoin(joinKind, op.LinkSettings, mapSettings, true)) { + addStarOption(true); + } + + if (IsSideSuitableForStarJoin(joinKind, op.LinkSettings, mapSettings, false)) { + addStarOption(false); + } + } else { + + auto childOp = leftLeaf ? rightOp : leftOp; + + bool allowNonUnique = joinKind.EndsWith("Semi") || joinKind.EndsWith("Only"); + bool allowKind = true; + if (joinKind.StartsWith("Left")) { + allowKind = leftLeaf == nullptr; + } else if (joinKind.StartsWith("Right")) { + allowKind = leftLeaf != nullptr; + } + + if (childOp->StarOptions && allowKind && (allowNonUnique || IsEffectivelyUnique(op.LinkSettings, mapSettings, leftLeaf != nullptr))) { + + const auto& leafJoinKeyList = leftLeaf ? leftJoinKeyList : rightJoinKeyList; + TString leafLabel = leftLeaf ? TString{leftLeaf->Label->Content()} : TString{rightLeaf->Label->Content()}; + + auto inputKeyTypeLeaf = BuildJoinKeyType(leftLeaf ? *leftItemType : *rightItemType, leafJoinKeyList); + + TString childLabel; + TVector<TString> childKeyList; + + if (ExtractJoinKeysForStarJoin(leftLeaf ? *op.RightLabel : *op.LeftLabel, childLabel, childKeyList)) { + TSet<TString> childKeys(childKeyList.begin(), childKeyList.end()); + for (const auto& childOption : childOp->StarOptions) { + if (leafLabel == childOption.StarLabel || childLabel != childOption.StarLabel || + childKeys != childOption.StarKeys) + { + continue; + } + + auto starInputType = GetJoinInputType(equiJoin, childOption.StarInputIndex, ctx); + YQL_ENSURE(starInputType); + auto inputKeyTypeChild = BuildJoinKeyType(*starInputType, childKeyList); + + if (!IsSameAnnotation(*AsDictKeyType(RemoveNullsFromJoinKeyType(inputKeyTypeChild), ctx), + *AsDictKeyType(RemoveNullsFromJoinKeyType(inputKeyTypeLeaf), ctx))) + { + // key types should match for merge star join to work + return; + } + + + TYtStarJoinOption option = childOption; + YQL_CLOG(INFO, ProviderYt) << "Adding " << leafLabel << " [" << JoinSeq(", ", leafJoinKeyList) + << "] to star " << option.StarLabel << " [" << JoinSeq(", ", option.StarKeys) + << "]"; + + op.StarOptions.emplace_back(option); + } + YQL_ENSURE(op.StarOptions.size() <= 1); + } + } + } +} + +const TStructExprType* AddLabelToStructType(const TStructExprType& input, TStringBuf label, TExprContext& ctx) { + TVector<const TItemExprType*> structItems; + for (auto& i : input.GetItems()) { + structItems.push_back(ctx.MakeType<TItemExprType>(FullColumnName(label, i->GetName()), i->GetItemType())); + } + return ctx.MakeType<TStructExprType>(structItems); +} + +const TStructExprType* MakeStructMembersOptional(const TStructExprType& input, TExprContext& ctx) { + TVector<const TItemExprType*> structItems; + for (auto& i : input.GetItems()) { + if (i->GetItemType()->GetKind() == ETypeAnnotationKind::Optional) { + structItems.push_back(i); + } else { + structItems.push_back(ctx.MakeType<TItemExprType>(i->GetName(), ctx.MakeType<TOptionalExprType>(i->GetItemType()))); + } + } + return ctx.MakeType<TStructExprType>(structItems); +} + +EStarRewriteStatus RewriteYtEquiJoinStarSingleChain(TYtEquiJoin equiJoin, TYtJoinNodeOp& op, const TYtState::TPtr& state, TExprContext& ctx) { + + YQL_ENSURE(op.StarOptions.size() == 1); + const auto& starOption = op.StarOptions.front(); + + auto starLabel = starOption.StarLabel; + const auto& starKeys = starOption.StarKeys; + const auto& starSortedKeys = starOption.StarSortedKeys; + const auto starInputIndex = starOption.StarInputIndex; + const auto starPremap = TMaybeNode<TCoLambda>(equiJoin.Ref().ChildPtr(joinFixedArgsCount + starInputIndex)); + + auto starInputType = GetJoinInputType(equiJoin, starInputIndex, ctx); + if (!starInputType) { + return EStarRewriteStatus::Error; + } + + THashMap<TString, TString> starRenames; + TSet<TString> renamedStarKeys; + TVector<TString> renamedStarSortedKeys; + { + size_t idx = 0; + for (auto& c : starOption.StarKeys) { + auto renamed = TStringBuilder() << "_yql_star_join_column_" << idx++; + starRenames[c] = renamed; + renamedStarKeys.insert(renamed); + } + + for (auto& c : starOption.StarSortedKeys) { + if (!starKeys.contains(c)) { + break; + } + YQL_ENSURE(starRenames.contains(c)); + renamedStarSortedKeys.push_back(starRenames[c]); + } + } + + TYtJoinNodeOp* currOp = &op; + + struct TStarJoinStep { + TStringBuf JoinKind; + TString Label; + + size_t ReduceIndex = Max<size_t>(); + TMaybeNode<TCoLambda> Premap; + + const TStructExprType* InputType = nullptr; + + TVector<TString> StarKeyList; + TVector<TString> KeyList; + + THashMap<TString, TString> Renames; + }; + + TVector<TStarJoinStep> starChain; + TVector<TYtSection> reduceSections; + ETypeAnnotationKind commonPremapKind = ETypeAnnotationKind::Optional; + if (starPremap) { + commonPremapKind = DeriveCommonSequenceKind(commonPremapKind, starPremap.Cast().Ref().GetTypeAnn()->GetKind()); + } + + THashSet<TString> unusedKeys; + while (currOp) { + const TYtStarJoinOption* option = nullptr; + for (auto& opt : currOp->StarOptions) { + if (opt.StarLabel == starLabel) { + option = &opt; + } + } + YQL_ENSURE(option); + YQL_ENSURE(starLabel == option->StarLabel); + YQL_ENSURE(starKeys == option->StarKeys); + YQL_ENSURE(starInputIndex == option->StarInputIndex); + + auto leftLeaf = dynamic_cast<TYtJoinNodeLeaf*>(currOp->Left.Get()); + auto rightLeaf = dynamic_cast<TYtJoinNodeLeaf*>(currOp->Right.Get()); + TYtJoinNodeLeaf* leaf = nullptr; + const TExprNode* leafLabelNode = nullptr; + const TExprNode* starLabelNode = nullptr; + + TYtJoinNodeOp* nextOp = nullptr; + if (leftLeaf && rightLeaf) { + leaf = (option->StarInputIndex == leftLeaf->Index) ? rightLeaf : leftLeaf; + leafLabelNode = (option->StarInputIndex == leftLeaf->Index) ? currOp->RightLabel.Get() : currOp->LeftLabel.Get(); + starLabelNode = (option->StarInputIndex == leftLeaf->Index) ? currOp->LeftLabel.Get() : currOp->RightLabel.Get(); + } else { + leaf = leftLeaf ? leftLeaf : rightLeaf; + leafLabelNode = leftLeaf ? currOp->LeftLabel.Get() : currOp->RightLabel.Get(); + starLabelNode = leftLeaf ? currOp->RightLabel.Get() : currOp->LeftLabel.Get(); + + nextOp = leftLeaf ? dynamic_cast<TYtJoinNodeOp*>(currOp->Right.Get()) : + dynamic_cast<TYtJoinNodeOp*>(currOp->Left.Get()); + YQL_ENSURE(nextOp); + } + YQL_ENSURE(leaf); + YQL_ENSURE(leafLabelNode); + YQL_ENSURE(starLabelNode); + + TStarJoinStep starJoinStep; + starJoinStep.JoinKind = currOp->JoinKind->Content(); + starJoinStep.InputType = GetJoinInputType(equiJoin, leaf->Index, ctx); + if (!starJoinStep.InputType) { + return EStarRewriteStatus::Error; + } + + YQL_ENSURE(ExtractJoinKeysForStarJoin(*leafLabelNode, starJoinStep.Label, starJoinStep.KeyList)); + TString sl; + YQL_ENSURE(ExtractJoinKeysForStarJoin(*starLabelNode, sl, starJoinStep.StarKeyList)); + YQL_ENSURE(sl == starLabel); + YQL_ENSURE(starJoinStep.KeyList.size() == starJoinStep.StarKeyList.size()); + + for (auto i : xrange(starJoinStep.KeyList.size())) { + auto starColumn = starJoinStep.StarKeyList[i]; + auto otherColumn = starJoinStep.KeyList[i]; + + YQL_ENSURE(starRenames.contains(starColumn)); + auto starRenamed = starRenames[starColumn]; + YQL_ENSURE(starRenamed); + + auto& renamed = starJoinStep.Renames[otherColumn]; + YQL_ENSURE(!renamed || renamed == starRenamed); + renamed = starRenamed; + } + + starJoinStep.ReduceIndex = leaf->Index; // will be updated + starJoinStep.Premap = TMaybeNode<TCoLambda>(equiJoin.Ref().ChildPtr(joinFixedArgsCount + leaf->Index)); + if (starJoinStep.Premap) { + YQL_ENSURE(EnsureSeqOrOptionalType(starJoinStep.Premap.Cast().Ref(), ctx)); + commonPremapKind = DeriveCommonSequenceKind(commonPremapKind, starJoinStep.Premap.Cast().Ref().GetTypeAnn()->GetKind()); + } + starChain.emplace_back(std::move(starJoinStep)); + + unusedKeys.insert(currOp->OutputRemoveColumns.begin(), currOp->OutputRemoveColumns.end()); + + currOp = nextOp; + } + + // we start doing joins from leafs + std::reverse(starChain.begin(), starChain.end()); + + const TStructExprType* chainOutputType = AddLabelToStructType(*starInputType, starLabel, ctx); + reduceSections.reserve(starChain.size() + 1); + + TVector<size_t> innerIndexes; + TVector<size_t> semiIndexes; + TVector<size_t> onlyIndexes; + TVector<size_t> leftIndexes; + for (auto& item : starChain) { + auto section = equiJoin.Input().Item(item.ReduceIndex); + section = Build<TYtSection>(ctx, section.Pos()) + .InitFrom(section) + .Settings(NYql::RemoveSettings(section.Settings().Ref(), EYtSettingType::JoinLabel | EYtSettingType::StatColumns, ctx)) + .Done(); + + section = SectionApplyRenames(section, item.Renames, ctx); + + reduceSections.push_back(section); + item.ReduceIndex = reduceSections.size() - 1; + + if (item.JoinKind == "Inner") { + innerIndexes.push_back(item.ReduceIndex); + } else if (item.JoinKind.EndsWith("Semi")) { + semiIndexes.push_back(item.ReduceIndex); + } else if (item.JoinKind.EndsWith("Only")) { + onlyIndexes.push_back(item.ReduceIndex); + } else if (item.JoinKind == "Left" || item.JoinKind == "Right") { + leftIndexes.push_back(item.ReduceIndex); + } else { + YQL_ENSURE(false, "Unexpected join type in Star JOIN"); + } + + if (item.JoinKind.EndsWith("Semi") || item.JoinKind.EndsWith("Only")) { + // output type remains the same as input + continue; + } + + const TStructExprType* inputType = AddLabelToStructType(*item.InputType, item.Label, ctx); + if (item.JoinKind != "Inner") { + inputType = MakeStructMembersOptional(*inputType, ctx); + } + + TVector<const TItemExprType*> outputTypeItems = chainOutputType->GetItems(); + const auto& inputTypeItems = inputType->GetItems(); + outputTypeItems.insert(outputTypeItems.end(), inputTypeItems.begin(), inputTypeItems.end()); + + chainOutputType = ctx.MakeType<TStructExprType>(outputTypeItems); + } + + { + auto section = equiJoin.Input().Item(starInputIndex); + section = Build<TYtSection>(ctx, section.Pos()) + .InitFrom(section) + .Settings(NYql::RemoveSettings(section.Settings().Ref(), EYtSettingType::JoinLabel | EYtSettingType::StatColumns, ctx)) + .Done(); + + section = SectionApplyRenames(section, starRenames, ctx); + reduceSections.push_back(section); + } + + YQL_ENSURE(reduceSections.size() == starChain.size() + 1); + YQL_ENSURE(innerIndexes.size() + semiIndexes.size() + onlyIndexes.size() + leftIndexes.size() == starChain.size()); + + const auto pos = equiJoin.Pos(); + + const TStructExprType* outItemType = nullptr; + if (!op.Parent) { + if (auto type = GetSequenceItemType(pos, + equiJoin.Ref().GetTypeAnn()->Cast<TTupleExprType>()->GetItems()[1], + false, ctx)) + { + outItemType = type->Cast<TStructExprType>(); + } else { + return EStarRewriteStatus::Error; + } + } else { + TVector<const TItemExprType*> structItems = chainOutputType->GetItems(); + EraseIf(structItems, [&](const TItemExprType* item) { return unusedKeys.contains(item->GetName()); }); + outItemType = ctx.MakeType<TStructExprType>(structItems); + } + + const TVariantExprType* inputVariant = nullptr; + { + TTypeAnnotationNode::TListType items; + items.reserve(reduceSections.size()); + for (const auto& item: starChain) { + items.push_back(item.InputType); + } + items.push_back(starInputType); + inputVariant = ctx.MakeType<TVariantExprType>(ctx.MakeType<TTupleExprType>(items)); + } + + TYtOutTableInfo outTableInfo(outItemType, state->Configuration->UseNativeYtTypes.Get().GetOrElse(DEFAULT_USE_NATIVE_YT_TYPES) ? NTCF_ALL : NTCF_NONE); + outTableInfo.RowSpec->SetConstraints(equiJoin.Ref().GetConstraintSet()); + outTableInfo.SetUnique(equiJoin.Ref().GetConstraint<TDistinctConstraintNode>(), pos, ctx); + // TODO: mark output sorted + Y_UNUSED(starSortedKeys); + + + YQL_CLOG(INFO, ProviderYt) << "Processing star join " << starLabel << ":[" << JoinSeq(",", starKeys) << "] of length " << starChain.size(); + for (auto& item : starChain) { + YQL_CLOG(INFO, ProviderYt) << "Join: " << item.JoinKind << ", " << item.Label << ":[" << JoinSeq(",", item.KeyList) + << "] -> " << starLabel << ":[" << JoinSeq(",", item.StarKeyList) << "]"; + } + YQL_CLOG(INFO, ProviderYt) << "StarJoin result type is " << *(const TTypeAnnotationNode*)chainOutputType; + + TExprNode::TPtr groupArg = ctx.NewArgument(pos, "group"); + TExprNode::TPtr nullFilteredRenamedAndPremappedStream = ctx.Builder(pos) + .Callable("FlatMap") + .Add(0, groupArg) + .Lambda(1) + .Param("item") + .Callable("Visit") + .Arg(0, "item") + .Do([&](TExprNodeBuilder& parent) -> TExprNodeBuilder& { + for (ui32 i = 0; i < reduceSections.size(); ++i) { + bool isStarInput = (i == reduceSections.size() - 1); + TSet<TString> skipNullMemberColumns; + if (!isStarInput || innerIndexes) { + // skip null for all inputs except for star input when there are no inner joins + skipNullMemberColumns = renamedStarKeys; + } + + auto premapLambda = ctx.Builder(pos) + .Lambda() + .Param("item") + .Callable("Just") + .Arg(0, "item") + .Seal() + .Seal() + .Build(); + + ApplyInputPremap(premapLambda, isStarInput ? starPremap : starChain[i].Premap, + commonPremapKind, isStarInput ? starRenames : starChain[i].Renames, ctx); + + parent + .Atom(2 * i + 1, i) + .Lambda(2 * i + 2) + .Param("unpackedVariant") + .Callable("FlatMap") + .Callable(0, "FlatMap") + .Callable(0, "SkipNullMembers") + .Callable(0, "Just") + .Arg(0, "unpackedVariant") + .Seal() + .Add(1, ToAtomList(skipNullMemberColumns, pos, ctx)) + .Seal() + .Lambda(1) + .Param("filtered") + .Apply(premapLambda) + .With(0, "filtered") + .Seal() + .Seal() + .Seal() + .Lambda(1) + .Param("filteredRenamedAndPremapped") + .Callable("Just") + .Callable(0, "Variant") + .Arg(0, "filteredRenamedAndPremapped") + .Atom(1, i) + .Add(2, ExpandType(pos, *inputVariant, ctx)) + .Seal() + .Seal() + .Seal() + .Seal() + .Seal(); + } + return parent; + }) + .Seal() + .Seal() + .Seal() + .Build(); + + auto buildOptionalInnersFromState = ctx.Builder(pos) + .Lambda() + .Param("state") + .Callable("TryRemoveAllOptionals") + .List(0) + .Do([&](TExprNodeBuilder& parent) -> TExprNodeBuilder& { + for (ui32 i = 0; i < innerIndexes.size(); ++i) { + parent + .Callable(i, "Nth") + .Arg(0, "state") + .Atom(1, ToString(innerIndexes[i]), TNodeFlags::Default) + .Seal(); + } + return parent; + }) + .Seal() + .Seal() + .Seal() + .Build(); + + TExprNode::TPtr buildResultRow = ctx.Builder(pos) + .Lambda() + .Param("starItem") + .Param("inners") + .Param("state") + .Callable("FlattenMembers") + .Do([&](TExprNodeBuilder& parent) -> TExprNodeBuilder& { + ui32 seqno = 0; + parent + .List(seqno++) + .Atom(0, starLabel + ".") + .Arg(1, "starItem") + .Seal(); + + for (ui32 i = 0; i < innerIndexes.size(); ++i) { + parent + .List(seqno++) + .Atom(0, starChain[innerIndexes[i]].Label + ".") + .Callable(1, "Nth") + .Arg(0, "inners") + .Atom(1, i) + .Seal() + .Seal(); + } + + for (ui32 i = 0; i < leftIndexes.size(); ++i) { + parent + .List(seqno++) + .Atom(0, starChain[leftIndexes[i]].Label + ".") + .Callable(1, "Nth") + .Arg(0, "state") + .Atom(1, ToString(leftIndexes[i]), TNodeFlags::Default) + .Seal() + .Seal(); + } + + return parent; + }) + .Seal() + .Seal() + .Build(); + + TExprNode::TPtr performStarChainLambda = ctx.Builder(pos) + .Lambda() + .Param("starItem") + .Param("state") + .Callable("FlatMap") + .Apply(0, buildOptionalInnersFromState) + .With(0, "state") + .Seal() + .Lambda(1) + .Param("inners") + .Callable("Filter") + .Callable(0, "Just") + .Apply(0, buildResultRow) + .With(0, "starItem") + .With(1, "inners") + .With(2, "state") + .Seal() + .Seal() + .Lambda(1) + .Param("unused") + .Callable("And") + .Do([&](TExprNodeBuilder& parent) -> TExprNodeBuilder& { + ui32 seqno = 0; + for (auto i : onlyIndexes) { + parent + .Callable(seqno++, "Not") + .Callable(0, "Exists") + .Callable(0, "Nth") + .Arg(0, "state") + .Atom(1, i) + .Seal() + .Seal() + .Seal(); + } + + for (auto i : semiIndexes) { + parent + .Callable(seqno++, "Exists") + .Callable(0, "Nth") + .Arg(0, "state") + .Atom(1, i) + .Seal() + .Seal(); + } + + if (seqno == 0) { + parent + .Callable(seqno, "Bool") + .Atom(0, "true", TNodeFlags::Default) + .Seal(); + } + return parent; + }) + .Seal() + .Seal() + .Seal() + .Seal() + .Seal() + .Seal() + .Build(); + + auto foldMapStateNode = ctx.Builder(pos) + .List() + .Do([&](TExprNodeBuilder& parent) -> TExprNodeBuilder& { + for (ui32 i = 0; i < starChain.size(); ++i) { + auto joinKind = starChain[i].JoinKind; + bool isEmptyStructInState = joinKind.EndsWith("Semi") || joinKind.EndsWith("Only"); + TVector<const TItemExprType*> noItems; + auto stateItemType = isEmptyStructInState ? ctx.MakeType<TStructExprType>(noItems) : starChain[i].InputType; + parent + .Callable(i, "Nothing") + .Add(0, ExpandType(pos, *ctx.MakeType<TOptionalExprType>(stateItemType), ctx)) + .Seal(); + } + return parent; + }) + .Seal() + .Build(); + + TExprNode::TPtr emptyResultNode = ctx.Builder(pos) + .Callable("Nothing") + .Add(0, ExpandType(pos, *ctx.MakeType<TOptionalExprType>(chainOutputType), ctx)) + .Seal() + .Build(); + + auto updateStateLambdaBuilder = [&](size_t inputIndex, size_t tupleSize, bool useEmptyStruct) { + YQL_ENSURE(inputIndex < tupleSize); + return ctx.Builder(pos) + .Lambda() + .Param("item") + .Param("state") + .List() + .Do([&](TExprNodeBuilder& parent) -> TExprNodeBuilder& { + for (ui32 i = 0; i < tupleSize; ++i) { + if (i != inputIndex) { + parent + .Callable(i, "Nth") + .Arg(0, "state") + .Atom(1, ToString(i), TNodeFlags::Default) + .Seal(); + } else if (useEmptyStruct) { + parent + .Callable(i, "Just") + .Callable(0, "AsStruct") + .Seal() + .Seal(); + } else { + parent + .Callable(i, "Just") + .Arg(0, "item") + .Seal(); + } + } + return parent; + }) + .Seal() + .Seal() + .Build(); + }; + + TExprNode::TPtr foldMapVisitLambda = ctx.Builder(pos) + .Lambda() + .Param("item") + .Param("state") + .Callable("Visit") + .Arg(0, "item") + .Do([&](TExprNodeBuilder& parent) -> TExprNodeBuilder& { + for (ui32 i = 0; i < starChain.size(); ++i) { + auto joinKind = starChain[i].JoinKind; + bool isEmptyStructInState = joinKind.EndsWith("Semi") || joinKind.EndsWith("Only"); + parent + .Atom(2 * i + 1, i) + .Lambda(2 * i + 2) + .Param("var") + .List() + .Add(0, emptyResultNode) + .Callable(1, "If") + .Callable(0, "Exists") + .Callable(0, "Nth") + .Arg(0, "state") + .Atom(1, i) + .Seal() + .Seal() + .Arg(1, "state") + .Apply(2, updateStateLambdaBuilder(i, starChain.size(), isEmptyStructInState)) + .With(0, "var") + .With(1, "state") + .Seal() + .Seal() + .Seal() + .Seal(); + } + return parent; + }) + .Atom(2 * starChain.size() + 1, ToString(starChain.size())) + .Lambda(2 * starChain.size() + 2) + .Param("var") + .List() + .Apply(0, performStarChainLambda) + .With(0, "var") + .With(1, "state") + .Seal() + .Arg(1, "state") + .Seal() + .Seal() + .Seal() + .Seal() + .Build(); + + TExprNode::TPtr foldedStream = ctx.Builder(pos) + .Callable("FoldMap") + .Add(0, nullFilteredRenamedAndPremappedStream) + .Add(1, foldMapStateNode) + .Lambda(2) + .Param("item") + .Param("state") + .List() + .Callable(0, "Nth") + .Apply(0, foldMapVisitLambda) + .With(0, "item") + .With(1, "state") + .Seal() + .Atom(1, 0U) + .Seal() + .Callable(1, "Nth") + .Apply(0, foldMapVisitLambda) + .With(0, "item") + .With(1, "state") + .Seal() + .Atom(1, 1U) + .Seal() + .Seal() + .Seal() + .Seal() + .Build(); + + TMap<TStringBuf, TVector<TStringBuf>> renameMap; + if (!op.Parent) { + renameMap = LoadJoinRenameMap(equiJoin.JoinOptions().Ref()); + } + TExprNode::TPtr finalRenamingLambda = BuildJoinRenameLambda(pos, renameMap, *outItemType, ctx).Ptr(); + + TExprNode::TPtr finalRenamedStream = ctx.Builder(pos) + .Callable("FlatMap") + .Add(0, foldedStream) + .Lambda(1) + .Param("optItem") + .Callable("Map") + .Arg(0, "optItem") + .Lambda(1) + .Param("item") + .Apply(finalRenamingLambda) + .With(0, "item") + .Seal() + .Seal() + .Seal() + .Seal() + .Seal() + .Build(); + + auto settingsBuilder = Build<TCoNameValueTupleList>(ctx, pos) + .Add() + .Name() + .Value(ToString(EYtSettingType::ReduceBy)) + .Build() + .Value(ToAtomList(renamedStarSortedKeys, pos, ctx)) + .Build() + .Add() + .Name() + .Value(ToString(EYtSettingType::JoinReduce)) + .Build() + .Build(); + + if (state->Configuration->UseFlow.Get().GetOrElse(DEFAULT_USE_FLOW)) { + settingsBuilder + .Add() + .Name() + .Value(ToString(EYtSettingType::Flow)) + .Build() + .Build(); + } + + const auto useSystemColumns = state->Configuration->UseSystemColumns.Get().GetOrElse(DEFAULT_USE_SYS_COLUMNS); + if (useSystemColumns) { + settingsBuilder + .Add() + .Name() + .Value(ToString(EYtSettingType::KeySwitch)) + .Build() + .Build(); + } + + auto reduceOp = Build<TYtReduce>(ctx, pos) + .World(equiJoin.World()) + .DataSink(equiJoin.DataSink()) + .Input() + .Add(reduceSections) + .Build() + .Output() + .Add(outTableInfo.ToExprNode(ctx, pos).Cast<TYtOutTable>()) + .Build() + .Settings(settingsBuilder.Done()) + .Reducer(BuildYtReduceLambda(pos, groupArg, std::move(finalRenamedStream), useSystemColumns, ctx)) + .Done(); + + op.Output = reduceOp; + return EStarRewriteStatus::Ok; +} + +void CombineStarStatus(EStarRewriteStatus& status, EStarRewriteStatus other) { + YQL_ENSURE(status == EStarRewriteStatus::Error || status == EStarRewriteStatus::None || + status == EStarRewriteStatus::Ok); + switch (other) { + case EStarRewriteStatus::Error: + status = other; + break; + case EStarRewriteStatus::Ok: + if (status != EStarRewriteStatus::Error) { + status = other; + } + break; + case EStarRewriteStatus::None: + break; + default: + YQL_ENSURE(false, "Unexpected star join status"); + } +} + +EStarRewriteStatus RewriteYtEquiJoinStarChains(TYtEquiJoin equiJoin, TYtJoinNodeOp& op, const TYtState::TPtr& state, TExprContext& ctx) { + TYtJoinNodeLeaf* leftLeaf = dynamic_cast<TYtJoinNodeLeaf*>(op.Left.Get()); + TYtJoinNodeLeaf* rightLeaf = dynamic_cast<TYtJoinNodeLeaf*>(op.Right.Get()); + + EStarRewriteStatus result = EStarRewriteStatus::None; + if (op.StarOptions) { + if (leftLeaf && rightLeaf) { + // too trivial star join - let RewriteYtEquiJoinLeaves() to handle it + return result; + } + return RewriteYtEquiJoinStarSingleChain(equiJoin, op, state, ctx); + } + + if (!leftLeaf) { + auto leftOp = dynamic_cast<TYtJoinNodeOp*>(op.Left.Get()); + YQL_ENSURE(leftOp); + CombineStarStatus(result, RewriteYtEquiJoinStarChains(equiJoin, *leftOp, state, ctx)); + if (result == EStarRewriteStatus::Error) { + return result; + } + if (result == EStarRewriteStatus::Ok && leftOp->Output) { + op.Left = ConvertYtEquiJoinToLeaf(*leftOp, equiJoin.Pos(), ctx); + } + } + + if (!rightLeaf) { + auto rightOp = dynamic_cast<TYtJoinNodeOp*>(op.Right.Get()); + YQL_ENSURE(rightOp); + CombineStarStatus(result, RewriteYtEquiJoinStarChains(equiJoin, *rightOp, state, ctx)); + if (result == EStarRewriteStatus::Error) { + return result; + } + if (result == EStarRewriteStatus::Ok && rightOp->Output) { + op.Right = ConvertYtEquiJoinToLeaf(*rightOp, equiJoin.Pos(), ctx); + } + } + + return result; +} + +EStarRewriteStatus RewriteYtEquiJoinStar(TYtEquiJoin equiJoin, TYtJoinNodeOp& op, const TYtState::TPtr& state, TExprContext& ctx) { + const bool enableStarJoin = state->Configuration->JoinEnableStarJoin.Get().GetOrElse(false); + if (!enableStarJoin) { + return EStarRewriteStatus::None; + } + + const bool allowColumnRenames = state->Configuration->JoinAllowColumnRenames.Get().GetOrElse(true); + if (!allowColumnRenames) { + return EStarRewriteStatus::None; + } + + EStarRewriteStatus collectStatus = EStarRewriteStatus::Ok; + CollectPossibleStarJoins(equiJoin, op, state, collectStatus, ctx); + if (collectStatus != EStarRewriteStatus::Ok) { + return collectStatus; + } + + return RewriteYtEquiJoinStarChains(equiJoin, op, state, ctx); +} + +} // namespace + +ui64 CalcInMemorySizeNoCrossJoin(const TJoinLabel& label, const TYtJoinNodeOp& op, const TMapJoinSettings& settings, bool isLeft, TExprContext& ctx, bool needPayload, ui64 size) +{ + const auto& keys = *(isLeft ? op.LeftLabel : op.RightLabel); + const ui64 rows = isLeft ? settings.LeftRows : settings.RightRows; + + const bool isUniqueKey = IsEffectivelyUnique(op.LinkSettings, settings, isLeft); + + bool many = needPayload && !isUniqueKey; + + auto inputKeyType = BuildJoinKeyType(label, keys); + auto keyType = AsDictKeyType(inputKeyType, ctx); + const TTypeAnnotationNode* payloadType = needPayload ? label.InputType : (const TTypeAnnotationNode*)ctx.MakeType<TVoidExprType>(); + + double sizeFactor = 1.; + ui64 rowFactor = 0; + CalcToDictFactors(keyType, payloadType, EDictType::Hashed, many, true, sizeFactor, rowFactor); + + return size * sizeFactor + rows * rowFactor; +} + +TStatus CollectStatsAndMapJoinSettings(ESizeStatCollectMode sizeMode, TMapJoinSettings& mapSettings, + TJoinSideStats& leftStats, TJoinSideStats& rightStats, + bool leftTablesReady, const TVector<TYtPathInfo::TPtr>& leftTables, const THashSet<TString>& leftJoinKeys, + bool rightTablesReady, const TVector<TYtPathInfo::TPtr>& rightTables, const THashSet<TString>& rightJoinKeys, + TYtJoinNodeLeaf* leftLeaf, TYtJoinNodeLeaf* rightLeaf, const TYtState& state, bool isCross, + TString cluster, TExprContext& ctx) +{ + mapSettings = {}; + leftStats = {}; + rightStats = {}; + + if (leftLeaf) { + auto premap = GetPremapLambda(*leftLeaf); + auto joinSideStatus = CollectJoinSideStats(leftTablesReady ? sizeMode : ESizeStatCollectMode::NoSize, leftStats, leftLeaf->Section, state, cluster, + leftTables, leftJoinKeys, isCross, premap, ctx); + if (joinSideStatus.Level != TStatus::Ok) { + return joinSideStatus; + } + + if (leftTablesReady) { + mapSettings.LeftRows = leftStats.RowsCount; + mapSettings.LeftSize = leftStats.Size; + mapSettings.LeftCount = leftTables.size(); + mapSettings.LeftUnique = leftStats.HasUniqueKeys && mapSettings.LeftCount == 1; + } + } + + if (rightLeaf) { + auto premap = GetPremapLambda(*rightLeaf); + auto joinSideStatus = CollectJoinSideStats(rightTablesReady ? sizeMode : ESizeStatCollectMode::NoSize, rightStats, rightLeaf->Section, state, cluster, + rightTables, rightJoinKeys, isCross, premap, ctx); + if (joinSideStatus.Level != TStatus::Ok) { + return joinSideStatus; + } + + if (rightTablesReady) { + mapSettings.RightRows = rightStats.RowsCount; + mapSettings.RightSize = rightStats.Size; + mapSettings.RightCount = rightTables.size(); + mapSettings.RightUnique = rightStats.HasUniqueKeys && mapSettings.RightCount == 1; + } + } + + if (sizeMode == ESizeStatCollectMode::RawSize) { + mapSettings.LeftMemSize = mapSettings.LeftSize; + mapSettings.RightMemSize = mapSettings.RightSize; + } + + return TStatus::Ok; +} + +TStatus CalculateJoinLeafSize(ui64& result, TMapJoinSettings& settings, TYtSection& inputSection, + const TYtJoinNodeOp& op, TExprContext& ctx, bool isLeft, + const TStructExprType* itemType, const TVector<TString>& joinKeyList, const TYtState::TPtr& state, const TString& cluster, + const TVector<TYtPathInfo::TPtr>& tables) +{ + result = isLeft ? settings.LeftSize : settings.RightSize; + const bool needPayload = op.JoinKind->IsAtom("Inner") || op.JoinKind->IsAtom(isLeft ? "Right" : "Left"); + + if (!needPayload && !op.JoinKind->IsAtom("Cross")) { + if (joinKeyList.size() < itemType->GetSize()) { + TVector<ui64> dataSizes; + auto status = TryEstimateDataSizeChecked(dataSizes, inputSection, cluster, tables, joinKeyList, *state, ctx); + if (status.Level != TStatus::Ok) { + return status; + } + result = Accumulate(dataSizes.begin(), dataSizes.end(), 0ull, [](ui64 sum, ui64 v) { return sum + v; });; + } + } + + return TStatus::Ok; +} + +IGraphTransformer::TStatus CollectPathsAndLabelsReady( + bool& ready, TVector<TYtPathInfo::TPtr>& tables, TJoinLabels& labels, + const TStructExprType*& itemType, const TStructExprType*& itemTypeBeforePremap, + const TYtJoinNodeLeaf& leaf, TExprContext& ctx) +{ + ready = false; + TStatus result = CollectPathsAndLabels(tables, labels, itemType, itemTypeBeforePremap, leaf, ctx); + if (result != TStatus::Ok) { + return result; + } + + ready = AllOf(tables, [](const auto& pathInfo) { return bool(pathInfo->Table->Stat); }); + return TStatus::Ok; +} + +TYtJoinNodeOp::TPtr ImportYtEquiJoin(TYtEquiJoin equiJoin, TExprContext& ctx) { + TVector<TYtJoinNodeLeaf::TPtr> leaves; + TMaybeNode<TExprBase> world; + for (const size_t i: xrange(equiJoin.Input().Size())) { + auto leaf = MakeIntrusive<TYtJoinNodeLeaf>(equiJoin.Input().Item(i), TMaybeNode<TCoLambda>(equiJoin.Ref().ChildPtr(joinFixedArgsCount + i))); + leaf->Label = NYql::GetSetting(leaf->Section.Settings().Ref(), EYtSettingType::JoinLabel)->Child(1); + leaf->Index = i; + TPartOfConstraintBase::TPathReduce rename; + if (leaf->Label->IsAtom()) { + leaf->Scope.emplace_back(leaf->Label->Content()); + rename = [&](const TPartOfConstraintBase::TPathType& path) -> std::vector<TPartOfConstraintBase::TPathType> { + auto result = path; + TStringBuilder sb; + sb << leaf->Scope.front() << '.' << result.front(); + result.front() = ctx.AppendString(sb); + return {result}; + }; + } else { + for (const auto& x : leaf->Label->Children()) { + leaf->Scope.emplace_back(x->Content()); + } + } + + const auto sourceConstraints = leaf->Section.Ref().GetConstraintSet(); + if (const auto distinct = sourceConstraints.GetConstraint<TDistinctConstraintNode>()) + if (const auto complete = leaf->Premap ? TPartOfDistinctConstraintNode::MakeComplete(ctx, leaf->Premap.Cast().Body().Ref().GetConstraint<TPartOfDistinctConstraintNode>(), distinct) : distinct) + if (const auto d = complete->RenameFields(ctx, rename)) + leaf->Constraints.AddConstraint(d); + if (const auto unique = sourceConstraints.GetConstraint<TUniqueConstraintNode>()) + if (const auto complete = leaf->Premap ? TPartOfUniqueConstraintNode::MakeComplete(ctx, leaf->Premap.Cast().Body().Ref().GetConstraint<TPartOfUniqueConstraintNode>(), unique) : unique) + if (const auto u = complete->RenameFields(ctx, rename)) + leaf->Constraints.AddConstraint(u); + if (const auto empty = sourceConstraints.GetConstraint<TEmptyConstraintNode>()) + leaf->Constraints.AddConstraint(empty); + + leaves.push_back(std::move(leaf)); + } + + const auto& renameMap = LoadJoinRenameMap(equiJoin.JoinOptions().Ref()); + THashSet<TString> drops; + for (const auto& [column, renames] : renameMap) { + if (renames.empty()) { + drops.insert(ToString(column)); + } + } + + const auto root = ImportYtEquiJoinRecursive(leaves, nullptr, drops, equiJoin.Joins().Ref(), ctx); + if (const auto renames = LoadJoinRenameMap(equiJoin.JoinOptions().Ref()); !renames.empty()) { + const auto rename = [&renames](const TPartOfConstraintBase::TPathType& path) -> std::vector<TPartOfConstraintBase::TPathType> { + if (path.empty()) + return {}; + + const auto it = renames.find(path.front()); + if (renames.cend() == it) + return {path}; + if (it->second.empty()) + return {}; + + std::vector<TPartOfConstraintBase::TPathType> res(it->second.size()); + std::transform(it->second.cbegin(), it->second.cend(), res.begin(), [&path](const std::string_view& newName) { + auto newPath = path; + newPath.front() = newName; + return newPath; + }); + return res; + }; + + TConstraintSet set; + if (const auto unique = root->Constraints.GetConstraint<TUniqueConstraintNode>()) + if (const auto u = unique->RenameFields(ctx, rename)) + set.AddConstraint(u); + if (const auto distinct = root->Constraints.GetConstraint<TDistinctConstraintNode>()) + if (const auto d = distinct->RenameFields(ctx, rename)) + set.AddConstraint(d); + if (const auto empty = root->Constraints.GetConstraint<TEmptyConstraintNode>()) + set.AddConstraint(empty); + root->Constraints = set; + } + + root->CostBasedOptPassed = HasSetting(equiJoin.JoinOptions().Ref(), "cbo_passed"); + return root; +} + +IGraphTransformer::TStatus CollectCboStatsLeaf( + const THashMap<TString, THashSet<TString>>& relJoinColumns, + const TString& cluster, + TYtJoinNodeLeaf& leaf, + const TYtState::TPtr& state, + TExprContext& ctx) { + + const TMaybe<ui64> maxChunkCountExtendedStats = state->Configuration->ExtendedStatsMaxChunkCount.Get(); + TVector<TYtPathInfo::TPtr> tables; + if (maxChunkCountExtendedStats) { + TVector<TString> requestedColumnList; + auto columnsPos = relJoinColumns.find(JoinLeafLabel(leaf.Label)); + if (columnsPos != relJoinColumns.end()) { + requestedColumnList.assign(columnsPos->second.begin(), columnsPos->second.end()); + } + + THashSet<TString> memSizeColumns(requestedColumnList.begin(), requestedColumnList.end()); + TVector<IYtGateway::TPathStatReq> pathStatReqs; + + ui64 sectionChunkCount = 0; + for (auto path: leaf.Section.Paths()) { + auto pathInfo = MakeIntrusive<TYtPathInfo>(path); + tables.push_back(pathInfo); + sectionChunkCount += pathInfo->Table->Stat->ChunkCount; + + if (pathInfo->HasColumns()) { + NYT::TRichYPath path; + pathInfo->FillRichYPath(path); + std::copy(path.Columns_->Parts_.begin(), path.Columns_->Parts_.end(), std::inserter(memSizeColumns, memSizeColumns.end())); + } + + auto ytPath = BuildYtPathForStatRequest(cluster, *pathInfo, requestedColumnList, *state, ctx); + + if (!ytPath) { + return IGraphTransformer::TStatus::Error; + } + + pathStatReqs.push_back( + IYtGateway::TPathStatReq() + .Path(*ytPath) + .IsTemp(pathInfo->Table->IsTemp) + .IsAnonymous(pathInfo->Table->IsAnonymous) + .Epoch(pathInfo->Table->Epoch.GetOrElse(0))); + } + + if (!pathStatReqs.empty() && (*maxChunkCountExtendedStats == 0 || sectionChunkCount <= *maxChunkCountExtendedStats)) { + IYtGateway::TPathStatOptions pathStatOptions = + IYtGateway::TPathStatOptions(state->SessionId) + .Cluster(cluster) + .Paths(pathStatReqs) + .Config(state->Configuration->Snapshot()) + .Extended(true); + + IYtGateway::TPathStatResult pathStats = state->Gateway->TryPathStat(std::move(pathStatOptions)); + + if (!pathStats.Success()) { + leaf.Section = Build<TYtSection>(ctx, leaf.Section.Ref().Pos()) + .InitFrom(leaf.Section) + .Settings(NYql::AddSettingAsColumnList(leaf.Section.Settings().Ref(), EYtSettingType::StatColumns, requestedColumnList, ctx)) + .Done(); + return TStatus::Repeat; + } + } + } + + TVector<ui64> dataSize; + return TryEstimateDataSizeChecked(dataSize, leaf.Section, cluster, tables, {}, *state, ctx); +} + +void AddJoinColumns(THashMap<TString, THashSet<TString>>& relJoinColumns, const TYtJoinNodeOp& op) { + for (ui32 i = 0; i < op.LeftLabel->ChildrenSize(); i += 2) { + auto ltable = op.LeftLabel->Child(i)->Content(); + auto lcolumn = op.LeftLabel->Child(i + 1)->Content(); + auto rtable = op.RightLabel->Child(i)->Content(); + auto rcolumn = op.RightLabel->Child(i + 1)->Content(); + + relJoinColumns[TString(ltable)].insert(TString(lcolumn)); + relJoinColumns[TString(rtable)].insert(TString(rcolumn)); + } +} + +IGraphTransformer::TStatus CollectCboStatsNode(THashMap<TString, THashSet<TString>>& relJoinColumns, const TString& cluster, TYtJoinNodeOp& op, const TYtState::TPtr& state, TExprContext& ctx) { + IGraphTransformer::TStatus result = TStatus::Ok; + TYtJoinNodeLeaf* leftLeaf = dynamic_cast<TYtJoinNodeLeaf*>(op.Left.Get()); + TYtJoinNodeLeaf* rightLeaf = dynamic_cast<TYtJoinNodeLeaf*>(op.Right.Get()); + AddJoinColumns(relJoinColumns, op); + if (leftLeaf) { + result = result.Combine(CollectCboStatsLeaf(relJoinColumns, cluster, *leftLeaf, state, ctx)); + } else { + auto& leftOp = *dynamic_cast<TYtJoinNodeOp*>(op.Left.Get()); + result = result.Combine(CollectCboStatsNode(relJoinColumns, cluster, leftOp, state, ctx)); + } + if (rightLeaf) { + result = result.Combine(CollectCboStatsLeaf(relJoinColumns, cluster, *rightLeaf, state, ctx)); + } else { + auto& rightOp = *dynamic_cast<TYtJoinNodeOp*>(op.Right.Get()); + result = result.Combine(CollectCboStatsNode(relJoinColumns, cluster, rightOp, state, ctx)); + } + return result; +} + +IGraphTransformer::TStatus CollectCboStats(const TString& cluster, TYtJoinNodeOp& op, const TYtState::TPtr& state, TExprContext& ctx) { + THashMap<TString, THashSet<TString>> relJoinColumns; + return CollectCboStatsNode(relJoinColumns, cluster, op, state, ctx); +} + +IGraphTransformer::TStatus RewriteYtEquiJoin(TYtEquiJoin equiJoin, TYtJoinNodeOp& op, const TYtState::TPtr& state, TExprContext& ctx) { + switch (RewriteYtEquiJoinStar(equiJoin, op, state, ctx)) { + case EStarRewriteStatus::Error: + return IGraphTransformer::TStatus::Error; + case EStarRewriteStatus::Ok: + return IGraphTransformer::TStatus::Ok; + case EStarRewriteStatus::WaitInput: + return IGraphTransformer::TStatus::Repeat; + case EStarRewriteStatus::None: + break; + } + + return RewriteYtEquiJoinLeaves(equiJoin, op, state, ctx); +} + +TStatus RewriteYtEquiJoinLeaves(TYtEquiJoin equiJoin, TYtJoinNodeOp& op, const TYtState::TPtr& state, TExprContext& ctx) { + TYtJoinNodeLeaf* leftLeaf = dynamic_cast<TYtJoinNodeLeaf*>(op.Left.Get()); + TYtJoinNodeLeaf* rightLeaf = dynamic_cast<TYtJoinNodeLeaf*>(op.Right.Get()); + + TStatus result = TStatus::Repeat; + if (!leftLeaf) { + auto& leftOp = *dynamic_cast<TYtJoinNodeOp*>(op.Left.Get()); + CombineJoinStatus(result, RewriteYtEquiJoinLeaves(equiJoin, leftOp, state, ctx)); + if (result.Level == TStatus::Error) { + return result; + } + if (result.Level == TStatus::Ok && leftOp.Output) { + // convert to leaf + op.Left = ConvertYtEquiJoinToLeaf(leftOp, equiJoin.Pos(), ctx); + } + } + + if (!rightLeaf) { + auto& rightOp = *dynamic_cast<TYtJoinNodeOp*>(op.Right.Get()); + CombineJoinStatus(result, RewriteYtEquiJoinLeaves(equiJoin, rightOp, state, ctx)); + if (result.Level == TStatus::Error) { + return result; + } + if (result.Level == TStatus::Ok && rightOp.Output) { + // convert to leaf + op.Right = ConvertYtEquiJoinToLeaf(rightOp, equiJoin.Pos(), ctx); + } + } + + if (leftLeaf && rightLeaf) { + CombineJoinStatus(result, RewriteYtEquiJoinLeaf(equiJoin, op, *leftLeaf, *rightLeaf, state, ctx)); + } + + return result; +} + +TMaybeNode<TExprBase> ExportYtEquiJoin(TYtEquiJoin equiJoin, const TYtJoinNodeOp& op, TExprContext& ctx, + const TYtState::TPtr& state) { + if (op.Output) { + return op.Output.Cast(); + } + + TVector<TYtSection> sections; + TVector<TExprBase> premaps; + THashSet<TString> outputRemoveColumns; + auto joinTree = ExportJoinTree(op, equiJoin.Pos(), sections, premaps, outputRemoveColumns, ctx); + + TExprNode::TPtr joinSettings = equiJoin.JoinOptions().Ptr(); + if (outputRemoveColumns) { + auto renameMap = LoadJoinRenameMap(*joinSettings); + for (auto& c : outputRemoveColumns) { + YQL_ENSURE(renameMap[c].empty(), "Rename map contains non-trivial renames for column " << c); + } + + joinSettings = RemoveSetting(*joinSettings, "rename", ctx); + TExprNode::TListType joinSettingsNodes = joinSettings->ChildrenList(); + + AppendEquiJoinRenameMap(joinSettings->Pos(), renameMap, joinSettingsNodes, ctx); + joinSettings = ctx.ChangeChildren(*joinSettings, std::move(joinSettingsNodes)); + } + + if (!HasSetting(*joinSettings, "cbo_passed") && op.CostBasedOptPassed) { + joinSettings = AddSetting(*joinSettings, joinSettings->Pos(), "cbo_passed", {}, ctx); + } + + auto outItemType = GetSequenceItemType(equiJoin.Pos(), + equiJoin.Ref().GetTypeAnn()->Cast<TTupleExprType>()->GetItems()[1], + false, ctx); + if (!outItemType) { + return {}; + } + + const auto join = Build<TYtEquiJoin>(ctx, equiJoin.Pos()) + .World(equiJoin.World()) + .DataSink(equiJoin.DataSink()) + .Input() + .Add(sections) + .Build() + .Output() + .Add(TYtOutTableInfo(outItemType->Cast<TStructExprType>(), + state->Configuration->UseNativeYtTypes.Get().GetOrElse(DEFAULT_USE_NATIVE_YT_TYPES) ? NTCF_ALL : NTCF_NONE).ToExprNode(ctx, equiJoin.Pos()).Cast<TYtOutTable>()) + .Build() + .Settings() + .Build() + .Joins(joinTree) + .JoinOptions(joinSettings) + .Done(); + auto children = join.Ref().ChildrenList(); + children.reserve(children.size() + premaps.size()); + std::transform(premaps.cbegin(), premaps.cend(), std::back_inserter(children), std::bind(&TExprBase::Ptr, std::placeholders::_1)); + return TExprBase(ctx.ChangeChildren(join.Ref(), std::move(children))); +} + +TString JoinLeafLabel(TExprNode::TPtr label) { + if (label->ChildrenSize() == 0) { + return TString(label->Content()); + } + TString result; + for (ui32 i = 0; i < label->ChildrenSize(); ++i) { + result += label->Child(i)->Content(); + if (i+1 != label->ChildrenSize()) { + result += ","; + } + } + + return result; +} + +} diff --git a/yt/yql/providers/yt/provider/yql_yt_join_impl.h b/yt/yql/providers/yt/provider/yql_yt_join_impl.h new file mode 100644 index 0000000000..c8c05a6162 --- /dev/null +++ b/yt/yql/providers/yt/provider/yql_yt_join_impl.h @@ -0,0 +1,126 @@ +#pragma once + +#include <yql/essentials/core/yql_join.h> + +#include <yt/yql/providers/yt/opt/yql_yt_join.h> +#include <yt/yql/providers/yt/provider/yql_yt_provider.h> +#include <yt/yql/providers/yt/expr_nodes/yql_yt_expr_nodes.h> + +#include <util/generic/ptr.h> +#include <util/generic/vector.h> +#include <util/generic/ylimits.h> + +namespace NYql { + +using namespace NNodes; + +struct TYtJoinNodeOp; + +struct TYtJoinNode: public TRefCounted<TYtJoinNode, TSimpleCounter> { + using TPtr = TIntrusivePtr<TYtJoinNode>; + virtual ~TYtJoinNode() = default; + + TVector<TString> Scope; + TConstraintSet Constraints; +}; + +struct TYtJoinNodeLeaf : TYtJoinNode { + using TPtr = TIntrusivePtr<TYtJoinNodeLeaf>; + + TYtJoinNodeLeaf(TYtSection section, TMaybeNode<TCoLambda> premap) + : Section(section) + , Premap(premap) + { + } + + TYtSection Section; + TMaybeNode<TCoLambda> Premap; + TExprNode::TPtr Label; + size_t Index = Max<size_t>(); +}; + +struct TYtStarJoinOption { + TSet<TString> StarKeys; + TVector<TString> StarSortedKeys; + size_t StarInputIndex = Max<size_t>(); + TString StarLabel; +}; + +struct TYtJoinNodeOp : TYtJoinNode { + using TPtr = TIntrusivePtr<TYtJoinNodeOp>; + + TYtJoinNode::TPtr Left; + TYtJoinNode::TPtr Right; + TExprNode::TPtr JoinKind; + TExprNode::TPtr LeftLabel; + TExprNode::TPtr RightLabel; + TEquiJoinLinkSettings LinkSettings; + const TYtJoinNodeOp* Parent = nullptr; + TVector<TYtStarJoinOption> StarOptions; + TMaybeNode<TYtOutputOpBase> Output; + THashSet<TString> OutputRemoveColumns; + bool CostBasedOptPassed = false; +}; + +struct TOptimizerLinkSettings { + bool HasForceSortedMerge = false; + bool HasHints = false; +}; + +TYtJoinNodeOp::TPtr ImportYtEquiJoin(TYtEquiJoin equiJoin, TExprContext& ctx); + +IGraphTransformer::TStatus CollectCboStats(const TString& cluster, TYtJoinNodeOp& op, const TYtState::TPtr& state, TExprContext& ctx); + +IGraphTransformer::TStatus RewriteYtEquiJoinLeaves(TYtEquiJoin equiJoin, TYtJoinNodeOp& op, const TYtState::TPtr& state, TExprContext& ctx); +IGraphTransformer::TStatus RewriteYtEquiJoin(TYtEquiJoin equiJoin, TYtJoinNodeOp& op, const TYtState::TPtr& state, TExprContext& ctx); +TMaybeNode<TExprBase> ExportYtEquiJoin(TYtEquiJoin equiJoin, const TYtJoinNodeOp& op, TExprContext& ctx, const TYtState::TPtr& state); +TYtJoinNodeOp::TPtr OrderJoins(TYtJoinNodeOp::TPtr op, const TYtState::TPtr& state, const TString& cluster, TExprContext& ctx, bool debug = false); +TString JoinLeafLabel(TExprNode::TPtr label); + +struct IBaseOptimizerNode; +struct IProviderContext; + +void BuildOptimizerJoinTree(TYtState::TPtr state, const TString& cluster, std::shared_ptr<IBaseOptimizerNode>& tree, std::shared_ptr<IProviderContext>& providerCtx, TOptimizerLinkSettings& settings, TYtJoinNodeOp::TPtr op, TExprContext& ctx); +TYtJoinNode::TPtr BuildYtJoinTree(std::shared_ptr<IBaseOptimizerNode> node, TExprContext& ctx, TPositionHandle pos); + +bool AreSimilarTrees(TYtJoinNode::TPtr node1, TYtJoinNode::TPtr node2); + +IGraphTransformer::TStatus CollectPathsAndLabelsReady( + bool& ready, TVector<TYtPathInfo::TPtr>& tables, TJoinLabels& labels, + const TStructExprType*& itemType, const TStructExprType*& itemTypeBeforePremap, + const TYtJoinNodeLeaf& leaf, TExprContext& ctx); + +IGraphTransformer::TStatus CalculateJoinLeafSize(ui64& result, TMapJoinSettings& settings, TYtSection& inputSection, + const TYtJoinNodeOp& op, TExprContext& ctx, bool isLeft, + const TStructExprType* itemType, const TVector<TString>& joinKeyList, const TYtState::TPtr& state, const TString& cluster, + const TVector<TYtPathInfo::TPtr>& tables); + +enum class ESizeStatCollectMode { + NoSize, + RawSize, + ColumnarSize, +}; + +struct TJoinSideStats { + TString TableNames; + bool HasUniqueKeys = false; + bool IsDynamic = false; + bool NeedsRemap = false; + + TVector<TString> SortedKeys; + + ui64 RowsCount = 0; + ui64 Size = 0; +}; + +IGraphTransformer::TStatus CollectStatsAndMapJoinSettings(ESizeStatCollectMode sizeMode, TMapJoinSettings& mapSettings, + TJoinSideStats& leftStats, TJoinSideStats& rightStats, + bool leftTablesReady, const TVector<TYtPathInfo::TPtr>& leftTables, const THashSet<TString>& leftJoinKeys, + bool rightTablesReady, const TVector<TYtPathInfo::TPtr>& rightTables, const THashSet<TString>& rightJoinKeys, + TYtJoinNodeLeaf* leftLeaf, TYtJoinNodeLeaf* rightLeaf, const TYtState& state, bool isCross, + TString cluster, TExprContext& ctx); + +ui64 CalcInMemorySizeNoCrossJoin(const TJoinLabel& label, const TYtJoinNodeOp& op, const TMapJoinSettings& settings, bool isLeft, + TExprContext& ctx, bool needPayload, ui64 size); + +} diff --git a/yt/yql/providers/yt/provider/yql_yt_join_reorder.cpp b/yt/yql/providers/yt/provider/yql_yt_join_reorder.cpp new file mode 100644 index 0000000000..1e34482155 --- /dev/null +++ b/yt/yql/providers/yt/provider/yql_yt_join_reorder.cpp @@ -0,0 +1,683 @@ + +#include "yql_yt_provider_context.h" +#include "yql_yt_join_impl.h" +#include "yql_yt_helpers.h" + +#include <yql/essentials/core/cbo/cbo_optimizer_new.h> +#include <yql/essentials/core/yql_graph_transformer.h> +#include <yql/essentials/parser/pg_wrapper/interface/optimizer.h> +#include <yql/essentials/providers/common/provider/yql_provider.h> +#include <yt/yql/providers/yt/opt/yql_yt_join.h> +#include <yt/yql/providers/yt/provider/yql_yt_provider_context.h> +#include <yql/essentials/utils/log/log.h> + +#include <yt/cpp/mapreduce/common/helpers.h> + +namespace NYql { + +namespace { + +void DebugPrint(TYtJoinNode::TPtr node, TExprContext& ctx, int level) { + auto* op = dynamic_cast<TYtJoinNodeOp*>(node.Get()); + auto printScope = [](const TVector<TString>& scope) -> TString { + TStringBuilder b; + for (auto& s : scope) { + b << s << ","; + } + return b; + }; + TString prefix; + for (int i = 0; i < level; i++) { + prefix += ' '; + } + if (op) { + Cerr << prefix + << "Op: " + << "Type: " << NCommon::ExprToPrettyString(ctx, *op->JoinKind) + << "Left: " << NCommon::ExprToPrettyString(ctx, *op->LeftLabel) + << "Right: " << NCommon::ExprToPrettyString(ctx, *op->RightLabel) + << "Scope: " << printScope(op->Scope) << "\n" + << "\n"; + DebugPrint(op->Left, ctx, level+1); + DebugPrint(op->Right, ctx, level+1); + } else { + auto* leaf = dynamic_cast<TYtJoinNodeLeaf*>(node.Get()); + Cerr << prefix + << "Leaf: " + << "Section: " << NCommon::ExprToPrettyString(ctx, *leaf->Section.Ptr()) + << "Label: " << NCommon::ExprToPrettyString(ctx, *leaf->Label) + << "Scope: " << printScope(leaf->Scope) << "\n" + << "\n"; + } +} + +class TJoinReorderer { +public: + TJoinReorderer( + TYtJoinNodeOp::TPtr op, + const TYtState::TPtr& state, + const TString& cluster, + TExprContext& ctx, + bool debug = false) + : Root(op) + , State(state) + , Cluster(cluster) + , Ctx(ctx) + , Debug(debug) + { + Y_UNUSED(State); + + if (Debug) { + DebugPrint(Root, Ctx, 0); + } + } + + TYtJoinNodeOp::TPtr Do() { + std::shared_ptr<IBaseOptimizerNode> tree; + TOptimizerLinkSettings linkSettings; + std::shared_ptr<IProviderContext> providerCtx; + BuildOptimizerJoinTree(State, Cluster, tree, providerCtx, linkSettings, Root, Ctx); + auto ytCtx = std::static_pointer_cast<TYtProviderContext>(providerCtx); + + std::function<void(const TString& str)> log; + + log = [](const TString& str) { + YQL_CLOG(INFO, ProviderYt) << str; + }; + + IOptimizerNew::TPtr opt; + + switch (State->Types->CostBasedOptimizer) { + case ECostBasedOptimizerType::PG: + if (linkSettings.HasForceSortedMerge || linkSettings.HasHints) { + YQL_CLOG(ERROR, ProviderYt) << "PG CBO does not support link settings"; + return Root; + } + opt = State->OptimizerFactory_->MakeJoinCostBasedOptimizerPG(*providerCtx, Ctx, {.Logger = log}); + break; + case ECostBasedOptimizerType::Native: + if (linkSettings.HasHints) { + YQL_CLOG(ERROR, ProviderYt) << "Native CBO does not suppor link hints"; + return Root; + } + opt = State->OptimizerFactory_->MakeJoinCostBasedOptimizerNative(*providerCtx, Ctx, {.MaxDPhypDPTableSize = 100000}); + break; + case ECostBasedOptimizerType::Disable: + YQL_CLOG(DEBUG, ProviderYt) << "CBO disabled"; + return Root; + } + + std::shared_ptr<TJoinOptimizerNode> result; + + try { + result = opt->JoinSearch(std::dynamic_pointer_cast<TJoinOptimizerNode>(tree)); + if (tree == result) { return Root; } + } catch (...) { + YQL_CLOG(ERROR, ProviderYt) << "Cannot do join search " << CurrentExceptionMessage(); + return Root; + } + + std::stringstream ss; + result->Print(ss); + + YQL_CLOG(INFO, ProviderYt) << "Result: " << ss.str(); + + TVector<TString> scope; + TYtJoinNodeOp::TPtr res = dynamic_cast<TYtJoinNodeOp*>(BuildYtJoinTree(result, Ctx, {}).Get()); + res->CostBasedOptPassed = true; + + YQL_ENSURE(res); + if (Debug) { + DebugPrint(res, Ctx, 0); + } + + return res; + } + +private: + TYtJoinNodeOp::TPtr Root; + const TYtState::TPtr& State; + TString Cluster; + TExprContext& Ctx; + bool Debug; +}; + +class TYtRelOptimizerNode: public TRelOptimizerNode { +public: + TYtRelOptimizerNode(TString label, TOptimizerStatistics stats, TYtJoinNodeLeaf* leaf) + : TRelOptimizerNode(std::move(label), std::move(stats)) + , OriginalLeaf(leaf) + { } + + TYtJoinNodeLeaf* OriginalLeaf; +}; + +class TYtJoinOptimizerNode: public TJoinOptimizerNode { +public: + TYtJoinOptimizerNode(const std::shared_ptr<IBaseOptimizerNode>& left, + const std::shared_ptr<IBaseOptimizerNode>& right, + const TVector<NDq::TJoinColumn>& leftKeys, + const TVector<NDq::TJoinColumn>& rightKeys, + const EJoinKind joinType, + const EJoinAlgoType joinAlgo, + TYtJoinNodeOp* originalOp) + : TJoinOptimizerNode(left, right, leftKeys, rightKeys, joinType, joinAlgo, + originalOp ? originalOp->LinkSettings.LeftHints.contains("any") : false, + originalOp ? originalOp->LinkSettings.RightHints.contains("any") : false, + originalOp != nullptr) + , OriginalOp(originalOp) + { } + + TYtJoinNodeOp* OriginalOp; // Only for nonReorderable +}; + +class TOptimizerTreeBuilder +{ +public: + TOptimizerLinkSettings LinkSettings; + TOptimizerTreeBuilder(TYtState::TPtr state, const TString& cluster, std::shared_ptr<IBaseOptimizerNode>& tree, std::shared_ptr<IProviderContext>& providerCtx, TYtJoinNodeOp::TPtr inputTree, TExprContext& ctx) + : State(state) + , Cluster(cluster) + , Tree(tree) + , OutProviderCtx(providerCtx) + , InputTree(inputTree) + , Ctx(ctx) + { } + + void Do() { + Tree = ProcessNode(InputTree, TRelSizeInfo{}); + auto joinMergeForce = State->Configuration->JoinMergeForce.Get(); + TYtProviderContext::TJoinAlgoLimits limits{ + .MapJoinMemLimit = 0, + .LookupJoinMemLimit = 0, + .LookupJoinMaxRows = 0 + }; + + if (State->Configuration->MapJoinLimit.Get()) { + limits.MapJoinMemLimit = *State->Configuration->MapJoinLimit.Get(); + } + if (LinkSettings.HasForceSortedMerge || joinMergeForce && *joinMergeForce) { + limits.MapJoinMemLimit = 0; + } + limits.LookupJoinMaxRows = State->Configuration->LookupJoinMaxRows.Get().GetOrElse(0); + if (State->Configuration->UseNewPredicateExtraction.Get().GetOrElse(DEFAULT_USE_NEW_PREDICATE_EXTRACTION)) { + limits.LookupJoinMaxRows = std::min( + limits.LookupJoinMaxRows, + State->Configuration->MaxKeyRangeCount.Get().GetOrElse(DEFAULT_MAX_KEY_RANGE_COUNT)); + } + limits.LookupJoinMemLimit = Min(State->Configuration->LookupJoinLimit.Get().GetOrElse(0), + State->Configuration->EvaluationTableSizeLimit.Get().GetOrElse(Max<ui64>())); + OutProviderCtx = std::make_shared<TYtProviderContext>(limits, std::move(ProviderRelInfo_)); + } + +private: + TVector<TString> GetJoinColumns(const TString& label) { + auto pos = RelJoinColumns.find(label); + if (pos == RelJoinColumns.end()) { + return TVector<TString>{}; + } + + return TVector<TString>(pos->second.begin(), pos->second.end()); + } + + std::shared_ptr<IBaseOptimizerNode> ProcessNode(TYtJoinNode::TPtr node, TRelSizeInfo sizeInfo) { + if (auto* op = dynamic_cast<TYtJoinNodeOp*>(node.Get())) { + return OnOp(op); + } else if (auto* leaf = dynamic_cast<TYtJoinNodeLeaf*>(node.Get())) { + return OnLeaf(leaf, sizeInfo); + } else { + YQL_ENSURE("Unknown node type"); + return nullptr; + } + } + + std::shared_ptr<IBaseOptimizerNode> OnOp(TYtJoinNodeOp* op) { + auto joinKind = ConvertToJoinKind(TString(op->JoinKind->Content())); + YQL_ENSURE(op->LeftLabel->ChildrenSize() == op->RightLabel->ChildrenSize()); + TVector<NDq::TJoinColumn> leftKeys; + TVector<NDq::TJoinColumn> rightKeys; + for (ui32 i = 0; i < op->LeftLabel->ChildrenSize(); i += 2) { + auto ltable = op->LeftLabel->Child(i)->Content(); + auto lcolumn = op->LeftLabel->Child(i + 1)->Content(); + auto rtable = op->RightLabel->Child(i)->Content(); + auto rcolumn = op->RightLabel->Child(i + 1)->Content(); + AddRelJoinColumn(TString(ltable), TString(lcolumn)); + AddRelJoinColumn(TString(rtable), TString(rcolumn)); + NDq::TJoinColumn lcol{TString(ltable), TString(lcolumn)}; + NDq::TJoinColumn rcol{TString(rtable), TString(rcolumn)}; + leftKeys.push_back(lcol); + rightKeys.push_back(rcol); + } + TRelSizeInfo leftSizeInfo; + TRelSizeInfo rightSizeInfo; + ExtractMapJoinStats(leftSizeInfo, rightSizeInfo, op); + + auto left = ProcessNode(op->Left, leftSizeInfo); + auto right = ProcessNode(op->Right, rightSizeInfo); + bool nonReorderable = op->LinkSettings.ForceSortedMerge; + LinkSettings.HasForceSortedMerge = LinkSettings.HasForceSortedMerge || op->LinkSettings.ForceSortedMerge; + LinkSettings.HasHints = LinkSettings.HasHints || !op->LinkSettings.LeftHints.empty() || !op->LinkSettings.RightHints.empty(); + + return std::make_shared<TYtJoinOptimizerNode>( + left, right, leftKeys, rightKeys, joinKind, EJoinAlgoType::GraceJoin, nonReorderable ? op : nullptr + ); + } + + std::shared_ptr<IBaseOptimizerNode> OnLeaf(TYtJoinNodeLeaf* leaf, TRelSizeInfo sizeInfo) { + TString label = JoinLeafLabel(leaf->Label); + + const TMaybe<ui64> maxChunkCountExtendedStats = State->Configuration->ExtendedStatsMaxChunkCount.Get(); + + TVector<TString> keyList = GetJoinColumns(label); + + TYtSection section{leaf->Section}; + auto stat = std::make_shared<TOptimizerStatistics>(); + stat->Ncols = std::ssize(keyList); + stat->ColumnStatistics = TIntrusivePtr<TOptimizerStatistics::TColumnStatMap>( + new TOptimizerStatistics::TColumnStatMap()); + auto providerStats = std::make_unique<TYtProviderStatistic>(); + + if (Y_UNLIKELY(!section.Settings().Empty()) && Y_UNLIKELY(section.Settings().Item(0).Name() == "Test")) { + for (const auto& setting : section.Settings()) { + if (setting.Name() == "Rows") { + stat->Nrows += FromString<ui64>(setting.Value().Ref().Content()); + } + } + } else { + for (auto path: section.Paths()) { + auto pathInfo = MakeIntrusive<TYtPathInfo>(path); + if (pathInfo->HasColumns()) { + NYT::TRichYPath ytPath; + pathInfo->FillRichYPath(ytPath); + stat->Ncols = std::max<int>(stat->Ncols, std::ssize(ytPath.Columns_->Parts_)); + } + auto tableStat = TYtTableBaseInfo::GetStat(path.Table()); + stat->ByteSize += tableStat->DataSize; + stat->Nrows += tableStat->RecordsCount; + } + if (section.Ref().GetState() >= TExprNode::EState::ConstrComplete) { + auto sorted = section.Ref().GetConstraint<TSortedConstraintNode>(); + if (sorted) { + TVector<TString> key; + for (const auto& item : sorted->GetContent()) { + for (const auto& path : item.first) { + const auto& column = path.front(); + key.push_back(TString(column)); + } + } + providerStats->SortColumns = key; + } + } + } + + TVector<TYtColumnStatistic> columnInfo; + + if (maxChunkCountExtendedStats) { + TVector<TMaybe<IYtGateway::TPathStatResult::TExtendedResult>> extendedStats; + extendedStats = GetStatsFromCache(leaf, keyList, *maxChunkCountExtendedStats); + columnInfo = ExtractColumnInfo(extendedStats); + } + + TDynBitMap relBitmap; + relBitmap.Set(std::ssize(ProviderRelInfo_)); + providerStats->RelBitmap = relBitmap; + providerStats->SizeInfo = sizeInfo; + + ProviderRelInfo_.push_back(TYtProviderRelInfo{ + .Label = label, + .ColumnInfo = columnInfo, + .SortColumns = providerStats->SortColumns + }); + + stat->Specific = std::move(providerStats); + return std::make_shared<TYtRelOptimizerNode>(std::move(label), std::move(*stat), leaf); + } + + void ExtractInMemorySize( + TMaybe<ui64>& leftMemorySize, + TMaybe<ui64>& rightMemorySize, + ESizeStatCollectMode mode, + TYtJoinNodeOp* op, + const TJoinLabels& labels, + int numLeaves, + TYtJoinNodeLeaf* leftLeaf, + bool leftTablesReady, + const TVector<TYtPathInfo::TPtr>& leftTables, + const THashSet<TString>& leftJoinKeys, + const TStructExprType* leftItemType, + TYtJoinNodeLeaf* rightLeaf, + bool rightTablesReady, + const TVector<TYtPathInfo::TPtr>& rightTables, + const THashSet<TString>& rightJoinKeys, + const TStructExprType* rightItemType) + { + TMapJoinSettings mapSettings; + TJoinSideStats leftStats; + TJoinSideStats rightStats; + bool isCross = false; + auto status = CollectStatsAndMapJoinSettings(mode, mapSettings, leftStats, rightStats, + leftTablesReady, leftTables, leftJoinKeys, rightTablesReady, rightTables, rightJoinKeys, + leftLeaf, rightLeaf, *State, isCross, Cluster, Ctx); + if (status != IGraphTransformer::TStatus::Ok) { + YQL_CLOG(WARN, ProviderYt) << "Unable to collect paths and labels: " << status; + return; + } + if (leftLeaf) { + const bool needPayload = op->JoinKind->IsAtom("Inner") || op->JoinKind->IsAtom("Right"); + const auto& label = labels.Inputs[0]; + TVector<TString> leftJoinKeyList(leftJoinKeys.begin(), leftJoinKeys.end()); + TYtSection inputSection{leftLeaf->Section}; + const ui64 rows = mapSettings.LeftRows; + ui64 size = 0; + auto status = CalculateJoinLeafSize(size, mapSettings, inputSection, *op, Ctx, true, leftItemType, leftJoinKeyList, State, Cluster, leftTables); + if (status != IGraphTransformer::TStatus::Ok) { + YQL_CLOG(WARN, ProviderYt) << "Unable to calculate join leaf size: " << status; + return; + } + if (op->JoinKind->IsAtom("Cross")) { + leftMemorySize = size + rows * (1ULL + label.InputType->GetSize()) * sizeof(NKikimr::NUdf::TUnboxedValuePod); + } else { + leftMemorySize = CalcInMemorySizeNoCrossJoin( + label, *op, mapSettings, true, Ctx, needPayload, size); + } + } + + if (rightLeaf) { + const bool needPayload = op->JoinKind->IsAtom("Inner") || op->JoinKind->IsAtom("Left"); + const auto& label = labels.Inputs[numLeaves - 1]; + TVector<TString> rightJoinKeyList(rightJoinKeys.begin(), rightJoinKeys.end()); + TYtSection inputSection{rightLeaf->Section}; + const ui64 rows = mapSettings.RightRows; + ui64 size = 0; + + auto status = CalculateJoinLeafSize(size, mapSettings, inputSection, *op, Ctx, false, rightItemType, rightJoinKeyList, State, Cluster, rightTables); + if (status != IGraphTransformer::TStatus::Ok) { + YQL_CLOG(WARN, ProviderYt) << "Unable to calculate join leaf size: " << status; + return; + } + if (op->JoinKind->IsAtom("Cross")) { + rightMemorySize = size + rows * (1ULL + label.InputType->GetSize()) * sizeof(NKikimr::NUdf::TUnboxedValuePod); + } else { + rightMemorySize = CalcInMemorySizeNoCrossJoin( + label, *op, mapSettings, false, Ctx, needPayload, size); + } + } + } + + void ExtractMapJoinStats(TRelSizeInfo& leftSizeInfo, TRelSizeInfo& rightSizeInfo, TYtJoinNodeOp* op) { + auto mapJoinUseFlow = State->Configuration->MapJoinUseFlow.Get().GetOrElse(DEFAULT_MAP_JOIN_USE_FLOW); + if (!mapJoinUseFlow) { + // Only support flow map joins in CBO. + return; + } + + TYtJoinNodeLeaf* leftLeaf = dynamic_cast<TYtJoinNodeLeaf*>(op->Left.Get()); + TYtJoinNodeLeaf* rightLeaf = dynamic_cast<TYtJoinNodeLeaf*>(op->Right.Get()); + + bool leftTablesReady = false; + TVector<TYtPathInfo::TPtr> leftTables; + bool rightTablesReady = false; + TVector<TYtPathInfo::TPtr> rightTables; + THashSet<TString> leftJoinKeys, rightJoinKeys; + int numLeaves = 0; + TJoinLabels labels; + const TStructExprType* leftItemType = nullptr; + const TStructExprType* leftItemTypeBeforePremap = nullptr; + const TStructExprType* rightItemType = nullptr; + const TStructExprType* rightItemTypeBeforePremap = nullptr; + + { + if (leftLeaf) { + TYtSection section{leftLeaf->Section}; + if (Y_UNLIKELY(!section.Settings().Empty() && section.Settings().Item(0).Name() == "Test")) { + return; + } + + auto status = CollectPathsAndLabelsReady(leftTablesReady, leftTables, labels, leftItemType, leftItemTypeBeforePremap, *leftLeaf, Ctx); + if (status != IGraphTransformer::TStatus::Ok) { + YQL_CLOG(WARN, ProviderYt) << "Unable to collect paths and labels: " << status; + return; + } + leftJoinKeys = BuildJoinKeys(labels.Inputs[0], *op->LeftLabel); + ++numLeaves; + } + if (rightLeaf) { + TYtSection section{rightLeaf->Section}; + if (Y_UNLIKELY(!section.Settings().Empty() && section.Settings().Item(0).Name() == "Test")) { + return; + } + auto status = CollectPathsAndLabelsReady(rightTablesReady, rightTables, labels, rightItemType, rightItemTypeBeforePremap, *rightLeaf, Ctx); + if (status != IGraphTransformer::TStatus::Ok) { + YQL_CLOG(WARN, ProviderYt) << "Unable to collect paths and labels: " << status; + return; + } + rightJoinKeys = BuildJoinKeys(labels.Inputs[1], *op->RightLabel); + ++numLeaves; + } + } + + if (numLeaves == 0) { + return; + } + + ExtractInMemorySize(leftSizeInfo.MapJoinMemSize, rightSizeInfo.MapJoinMemSize, ESizeStatCollectMode::ColumnarSize, op, labels, + numLeaves, leftLeaf, leftTablesReady, leftTables, leftJoinKeys, leftItemType, + rightLeaf, rightTablesReady, rightTables, rightJoinKeys, rightItemType); + ExtractInMemorySize(leftSizeInfo.LookupJoinMemSize, rightSizeInfo.LookupJoinMemSize, ESizeStatCollectMode::RawSize, op, labels, + numLeaves, leftLeaf, leftTablesReady, leftTables, leftJoinKeys, leftItemType, + rightLeaf, rightTablesReady, rightTables, rightJoinKeys, rightItemType); + } + + TVector<TMaybe<IYtGateway::TPathStatResult::TExtendedResult>> GetStatsFromCache( + TYtJoinNodeLeaf* nodeLeaf, const TVector<TString>& columns, ui64 maxChunkCount) { + TVector<IYtGateway::TPathStatReq> pathStatReqs; + TYtSection section{nodeLeaf->Section}; + ui64 totalChunkCount = 0; + for (auto path: section.Paths()) { + auto pathInfo = MakeIntrusive<TYtPathInfo>(path); + + totalChunkCount += pathInfo->Table->Stat->ChunkCount; + + auto ytPath = BuildYtPathForStatRequest(Cluster, *pathInfo, columns, *State, Ctx); + YQL_ENSURE(ytPath); + + pathStatReqs.push_back( + IYtGateway::TPathStatReq() + .Path(*ytPath) + .IsTemp(pathInfo->Table->IsTemp) + .IsAnonymous(pathInfo->Table->IsAnonymous) + .Epoch(pathInfo->Table->Epoch.GetOrElse(0))); + + } + if (pathStatReqs.empty() || (maxChunkCount != 0 && totalChunkCount > maxChunkCount)) { + return {}; + } + + IYtGateway::TPathStatOptions pathStatOptions = + IYtGateway::TPathStatOptions(State->SessionId) + .Cluster(Cluster) + .Paths(pathStatReqs) + .Config(State->Configuration->Snapshot()) + .Extended(true); + + IYtGateway::TPathStatResult pathStats = State->Gateway->TryPathStat(std::move(pathStatOptions)); + if (!pathStats.Success()) { + YQL_CLOG(WARN, ProviderYt) << "Unable to read path stats that must be already present in cache"; + return {}; + } + return pathStats.Extended; + } + + TVector<TYtColumnStatistic> ExtractColumnInfo(TVector<TMaybe<IYtGateway::TPathStatResult::TExtendedResult>> extendedResults) + { + THashMap<TString, size_t> columns; + TVector<TString> columnNames; + TVector<TMaybe<i64>> dataWeight; + TVector<TMaybe<ui64>> estimatedUniqueCounts; + + for (const auto& result : extendedResults) { + if (!result) { + continue; + } + for (const auto& entry : result->DataWeight) { + auto insResult = columns.insert(std::make_pair(entry.first, columns.size())); + size_t index = insResult.first->second; + if (insResult.second) { + dataWeight.push_back(0); + estimatedUniqueCounts.push_back(Nothing()); + columnNames.push_back(entry.first); + } + dataWeight[index] = (dataWeight[index].GetOrElse(0) + entry.second); + } + for (const auto& entry : result->EstimatedUniqueCounts) { + auto insResult = columns.insert(std::make_pair(entry.first, columns.size())); + size_t index = insResult.first->second; + if (insResult.second) { + dataWeight.push_back(Nothing()); + estimatedUniqueCounts.push_back(0); + columnNames.push_back(entry.first); + } + estimatedUniqueCounts[index] = std::max(estimatedUniqueCounts[index].GetOrElse(0), entry.second); + } + } + TVector<TYtColumnStatistic> result; + result.reserve(std::ssize(columns)); + for (int i = 0; i < std::ssize(columns); i++) { + result.push_back(TYtColumnStatistic{ + .ColumnName = columnNames[i], + .EstimatedUniqueCount = estimatedUniqueCounts[i], + .DataWeight = dataWeight[i] + }); + } + + return result; + } + + void AddRelJoinColumn(const TString& rtable, const TString& rcolumn) { + auto entry = RelJoinColumns.insert(std::make_pair(rtable, THashSet<TString>{})); + entry.first->second.insert(rcolumn); + } + + TString JoinLeafLabel(TExprNode::TPtr label) { + if (label->ChildrenSize() == 0) { + return TString(label->Content()); + } + TString result; + for (ui32 i = 0; i < label->ChildrenSize(); ++i) { + result += label->Child(i)->Content(); + if (i+1 != label->ChildrenSize()) { + result += ","; + } + } + + return result; + } + + TYtState::TPtr State; + const TString Cluster; + std::shared_ptr<IBaseOptimizerNode>& Tree; + std::shared_ptr<IProviderContext>& OutProviderCtx; + THashMap<TString, THashSet<TString>> RelJoinColumns; + TYtJoinNodeOp::TPtr InputTree; + TExprContext& Ctx; + TVector<TYtProviderRelInfo> ProviderRelInfo_; +}; + +TYtJoinNode::TPtr BuildYtJoinTree(std::shared_ptr<IBaseOptimizerNode> node, TVector<TString>& scope, TExprContext& ctx, TPositionHandle pos) { + if (node->Kind == RelNodeType) { + auto* leaf = static_cast<TYtRelOptimizerNode*>(node.get())->OriginalLeaf; + scope.insert(scope.end(), leaf->Scope.begin(), leaf->Scope.end()); + return leaf; + } else if (node->Kind == JoinNodeType) { + auto* op = static_cast<TJoinOptimizerNode*>(node.get()); + auto* ytop = dynamic_cast<TYtJoinOptimizerNode*>(op); + TYtJoinNodeOp::TPtr ret; + if (ytop && !ytop->IsReorderable) { + ret = ytop->OriginalOp; + } else { + ret = MakeIntrusive<TYtJoinNodeOp>(); + ret->JoinKind = ctx.NewAtom(pos, ConvertToJoinString(op->JoinType)); + ret->LinkSettings.JoinAlgo = op->JoinAlgo; + TVector<TExprNodePtr> leftLabel, rightLabel; + leftLabel.reserve(op->LeftJoinKeys.size() * 2); + rightLabel.reserve(op->RightJoinKeys.size() * 2); + for (auto& left : op->LeftJoinKeys) { + leftLabel.emplace_back(ctx.NewAtom(pos, left.RelName)); + leftLabel.emplace_back(ctx.NewAtom(pos, left.AttributeName)); + } + for (auto& right : op->RightJoinKeys) { + rightLabel.emplace_back(ctx.NewAtom(pos, right.RelName)); + rightLabel.emplace_back(ctx.NewAtom(pos, right.AttributeName)); + } + ret->LeftLabel = Build<TCoAtomList>(ctx, pos) + .Add(leftLabel) + .Done() + .Ptr(); + ret->RightLabel = Build<TCoAtomList>(ctx, pos) + .Add(rightLabel) + .Done() + .Ptr(); + } + int index = scope.size(); + ret->Left = BuildYtJoinTree(op->LeftArg, scope, ctx, pos); + ret->Right = BuildYtJoinTree(op->RightArg, scope, ctx, pos); + ret->Scope.insert(ret->Scope.end(), scope.begin() + index, scope.end()); + return ret; + } else { + YQL_ENSURE(false, "Unknown node type"); + } +} + +} // namespace + +bool AreSimilarTrees(TYtJoinNode::TPtr node1, TYtJoinNode::TPtr node2) { + if (node1 == node2) { + return true; + } + if (node1 && !node2) { + return false; + } + if (node2 && !node1) { + return false; + } + if (node1->Scope != node2->Scope) { + return false; + } + auto opLeft = dynamic_cast<TYtJoinNodeOp*>(node1.Get()); + auto opRight = dynamic_cast<TYtJoinNodeOp*>(node2.Get()); + if (opLeft && opRight) { + return AreSimilarTrees(opLeft->Left, opRight->Left) + && AreSimilarTrees(opLeft->Right, opRight->Right); + } else if (!opLeft && !opRight) { + return true; + } else { + return false; + } +} + +void BuildOptimizerJoinTree(TYtState::TPtr state, const TString& cluster, std::shared_ptr<IBaseOptimizerNode>& tree, std::shared_ptr<IProviderContext>& providerCtx, TOptimizerLinkSettings& linkSettings, TYtJoinNodeOp::TPtr op, TExprContext& ctx) +{ + TOptimizerTreeBuilder builder(state, cluster, tree, providerCtx, op, ctx); + builder.Do(); + linkSettings = builder.LinkSettings; +} + +TYtJoinNode::TPtr BuildYtJoinTree(std::shared_ptr<IBaseOptimizerNode> node, TExprContext& ctx, TPositionHandle pos) { + TVector<TString> scope; + return BuildYtJoinTree(node, scope, ctx, pos); +} + +TYtJoinNodeOp::TPtr OrderJoins(TYtJoinNodeOp::TPtr op, const TYtState::TPtr& state, const TString& cluster, TExprContext& ctx, bool debug) +{ + if (state->Types->CostBasedOptimizer == ECostBasedOptimizerType::Disable || op->CostBasedOptPassed) { + return op; + } + + auto result = TJoinReorderer(op, state, cluster, ctx, debug).Do(); + if (!debug && AreSimilarTrees(result, op)) { + return op; + } + return result; +} + +} // namespace NYql diff --git a/yt/yql/providers/yt/provider/yql_yt_key.cpp b/yt/yql/providers/yt/provider/yql_yt_key.cpp new file mode 100644 index 0000000000..3c2f99b26d --- /dev/null +++ b/yt/yql/providers/yt/provider/yql_yt_key.cpp @@ -0,0 +1,290 @@ +#include "yql_yt_key.h" +#include <library/cpp/yson/node/node_io.h> + +#include <yt/yql/providers/yt/common/yql_names.h> +#include <yt/yql/providers/yt/expr_nodes/yql_yt_expr_nodes.h> +#include <yql/essentials/core/yql_expr_type_annotation.h> + +#include <util/string/builder.h> +#include <util/string/cast.h> +#include <util/string/split.h> +#include <util/generic/strbuf.h> + +namespace NYql { + +namespace { + +THashSet<TStringBuf> EXT_KEY_CALLABLES = { + TStringBuf("Key"), + TStringBuf("TempTable"), + MrFolderName, + MrWalkFoldersName, + MrWalkFoldersImplName +}; + +THashSet<TStringBuf> KEY_CALLABLES = { + TStringBuf("Key"), + TStringBuf("TempTable"), +}; + +} + +bool TYtKey::Parse(const TExprNode& key, TExprContext& ctx, bool isOutput) { + using namespace NNodes; + if (!key.IsCallable(EXT_KEY_CALLABLES)) { + ctx.AddError(TIssue(ctx.GetPosition(key.Pos()), TStringBuf("Expected key"))); + return false; + } + + KeyNode = &key; + if (key.IsCallable(TStringBuf("TempTable"))) { + if (key.ChildrenSize() != 1) { + ctx.AddError(TIssue(ctx.GetPosition(key.Pos()), TStringBuf("TempTable must have a label"))); + return false; + } + + if (!EnsureAtom(*key.Child(0), ctx)) { + return false; + } + Type = EType::Table; + Anonymous = true; + Path = TString("@").append(key.Child(0)->Content()); + return true; + } + else if (key.IsCallable(MrFolderName)) { + if (key.ChildrenSize() != 2 || !key.Child(0)->IsAtom() || !key.Child(1)->IsAtom()) { + ctx.AddError(TIssue(ctx.GetPosition(key.Pos()), TStringBuilder() << "Incorrect format of " << MrFolderName)); + return false; + } + Folder.ConstructInPlace(); + Folder->Prefix = key.Child(0)->Content(); + Split(TString(key.Child(1)->Content()), ";", Folder->Attributes); + Type = EType::Folder; + return true; + } + + if (key.ChildrenSize() < 1) { + ctx.AddError(TIssue(ctx.GetPosition(key.Pos()), TStringBuf("Key must have at least one component - table or tablescheme, " + " and may have second tag - view"))); + return false; + } + else if (const auto maybeWalkFolders = TMaybeNode<TYtWalkFolders>(&key)) { + Type = EType::WalkFolders; + const auto walkFolders = maybeWalkFolders.Cast(); + + TFolderList initialListFolder; + initialListFolder.Prefix = walkFolders.Prefix(); + Split(TString(walkFolders.Attributes().StringValue()), ";", initialListFolder.Attributes); + + WalkFolderArgs = MakeMaybe(TWalkFoldersArgs{ + .InitialFolder = std::move(initialListFolder), + .PickledUserState = walkFolders.PickledUserState().Ptr(), + .UserStateType = walkFolders.UserStateType().Ptr(), + .PreHandler = walkFolders.PreHandler().Ptr(), + .ResolveHandler = walkFolders.ResolveHandler().Ptr(), + .DiveHandler = walkFolders.DiveHandler().Ptr(), + .PostHandler = walkFolders.PostHandler().Ptr(), + }); + + return true; + } + else if (const auto maybeWalkFolders = TMaybeNode<TYtWalkFoldersImpl>(&key)) { + Type = EType::WalkFoldersImpl; + const auto walkFolders = maybeWalkFolders.Cast(); + + ui64 stateKey; + if (!TryFromString(walkFolders.ProcessStateKey().StringValue(), stateKey)) { + ctx.AddError(TIssue(ctx.GetPosition(key.Pos()), + TStringBuilder() << MrWalkFoldersImplName << ": incorrect format of state map key")); + return false; + } + + WalkFolderImplArgs = MakeMaybe(TWalkFoldersImplArgs{ + .UserStateExpr = walkFolders.PickledUserState().Ptr(), + .UserStateType = walkFolders.UserStateType().Ptr(), + .StateKey = stateKey, + }); + + Type = EType::WalkFoldersImpl; + return true; + } + + auto tagName = key.Child(0)->Child(0)->Content(); + if (tagName == TStringBuf("table")) { + Type = EType::Table; + if (key.ChildrenSize() > 3) { + ctx.AddError(TIssue(ctx.GetPosition(key.Pos()), "Too many tags")); + return false; + } + } else if (tagName == TStringBuf("tablescheme")) { + Type = EType::TableScheme; + if (key.ChildrenSize() > 3) { + ctx.AddError(TIssue(ctx.GetPosition(key.Pos()), "Too many tags")); + return false; + } + if (isOutput) { + Type = EType::Table; + } + } else { + ctx.AddError(TIssue(ctx.GetPosition(key.Child(0)->Pos()), TString("Unexpected tag: ") + tagName)); + return false; + } + + const TExprNode* nameNode = key.Child(0)->Child(1); + if (nameNode->IsCallable("String")) { + if (!EnsureArgsCount(*nameNode, 1, ctx)) { + return false; + } + + if (!EnsureAtom(*nameNode->Child(0), ctx)) { + return false; + } + + const TExprNode* tableName = nameNode->Child(0); + + Path = tableName->Content(); + } + else if (nameNode->IsCallable(MrTableRangeName) || nameNode->IsCallable(MrTableRangeStrictName)) { + if (EType::TableScheme == Type) { + ctx.AddError(TIssue(ctx.GetPosition(nameNode->Pos()), "MrTableRange[Strict] must not be used with tablescheme tag")); + return false; + } + + if (!EnsureMinArgsCount(*nameNode, 1, ctx)) { + return false; + } + + if (!EnsureMaxArgsCount(*nameNode, 3, ctx)) { + return false; + } + + if (!EnsureAtom(*nameNode->Child(0), ctx)) { + return false; + } + + if (nameNode->ChildrenSize() > 2){ + if (!EnsureAtom(*nameNode->Child(2), ctx)) { + return false; + } + } + + Range.ConstructInPlace(); + Range->Prefix = nameNode->Child(0)->Content(); + if (nameNode->ChildrenSize() > 1) { + Range->Filter = nameNode->Child(1); + } + if (nameNode->ChildrenSize() > 2) { + Range->Suffix = nameNode->Child(2)->Content(); + } + Range->IsStrict = nameNode->Content() == MrTableRangeStrictName; + + } else { + ctx.AddError(TIssue(ctx.GetPosition(key.Pos()), "Expected String or MrTableRange[Strict]")); + } + + if (key.ChildrenSize() > 1) { + for (ui32 i = 1; i < key.ChildrenSize(); ++i) { + auto tag = key.Child(i)->Child(0); + if (tag->Content() == TStringBuf("view")) { + const TExprNode* viewNode = key.Child(i)->Child(1); + if (!viewNode->IsCallable("String")) { + ctx.AddError(TIssue(ctx.GetPosition(viewNode->Pos()), "Expected String")); + return false; + } + + if (viewNode->ChildrenSize() != 1 || !EnsureAtom(*viewNode->Child(0), ctx)) { + ctx.AddError(TIssue(ctx.GetPosition(viewNode->Child(0)->Pos()), "Dynamic views names are not supported")); + return false; + } + + View = viewNode->Child(0)->Content(); + if (View.empty()) { + ctx.AddError(TIssue(ctx.GetPosition(viewNode->Child(0)->Pos()), "View name must not be empty")); + return false; + } + } else { + ctx.AddError(TIssue(ctx.GetPosition(tag->Pos()), TStringBuilder() << "Unexpected tag: " << tag->Content())); + return false; + } + } + } + return true; +} + +bool TYtInputKeys::Parse(const TExprNode& keysNode, TExprContext& ctx) { + bool hasNonKeys = false; + TVector<const TExprNode*> keys; + if (keysNode.Type() == TExprNode::List) { + keysNode.ForEachChild([&](const TExprNode& k) { + if (k.IsCallable(KEY_CALLABLES)) { + keys.push_back(&k); + } else { + hasNonKeys = true; + } + }); + } else if (keysNode.IsCallable(MrTableConcatName)) { + keysNode.ForEachChild([&](const TExprNode& k){ + keys.push_back(&k); + }); + StrictConcat = false; + } else { + if (keysNode.IsCallable(EXT_KEY_CALLABLES)) { + keys.push_back(&keysNode); + } else { + hasNonKeys = true; + } + } + if (hasNonKeys) { + if (!keys.empty()) { + ctx.AddError(TIssue(ctx.GetPosition(keysNode.Pos()), "Mixed keys")); + return false; + } + HasNonKeys = true; + return true; + } + for (auto k: keys) { + Keys.emplace_back(); + if (!Keys.back().Parse(*k, ctx)) { + return false; + } + if (TYtKey::EType::Undefined == Type) { + Type = Keys.back().GetType(); + } else if (Type != Keys.back().GetType()) { + ctx.AddError(TIssue(ctx.GetPosition(k->Child(0)->Pos()), "Multiple keys must have the same tag")); + return false; + } + } + if (TYtKey::EType::Table != Type && Keys.size() > 1) { + ctx.AddError(TIssue(ctx.GetPosition(keysNode.Pos()), "Expected single key")); + return false; + } + return true; +} + +bool TYtOutputKey::Parse(const TExprNode& keyNode, TExprContext& ctx) { + if (!keyNode.IsCallable(KEY_CALLABLES)) { + return true; + } + if (!TYtKey::Parse(keyNode, ctx, true)) { + return false; + } + + if (GetType() != TYtKey::EType::Table) { + ctx.AddError(TIssue(ctx.GetPosition(keyNode.Child(0)->Child(0)->Pos()), + TStringBuilder() << "Unexpected tag: " << keyNode.Child(0)->Child(0)->Content())); + return false; + } + + if (!GetView().empty()) { + ctx.AddError(TIssue(ctx.GetPosition(keyNode.Pos()), "Unexpected view for output key")); + return false; + } + + if (GetRange().Defined()) { + ctx.AddError(TIssue(ctx.GetPosition(keyNode.Pos()), "Unexpected MrTableRange[Strict] for output key")); + return false; + } + return true; +} + +} diff --git a/yt/yql/providers/yt/provider/yql_yt_key.h b/yt/yql/providers/yt/provider/yql_yt_key.h new file mode 100644 index 0000000000..fd672ceaf5 --- /dev/null +++ b/yt/yql/providers/yt/provider/yql_yt_key.h @@ -0,0 +1,194 @@ +#pragma once + +#include <yql/essentials/utils/yql_panic.h> +#include <yql/essentials/ast/yql_expr.h> + +#include <util/generic/hash.h> +#include <util/generic/string.h> +#include <util/generic/maybe.h> +#include <util/generic/vector.h> +#include <util/digest/numeric.h> +#include <util/str_stl.h> + +namespace NYql { + +class TYtKey { +public: + enum class EType { + Undefined, + Table, + TableScheme, + Folder, + WalkFolders, + WalkFoldersImpl, + }; + + struct TRange { + TString Prefix; + TExprNode::TPtr Filter; + TString Suffix; + bool IsStrict = true; + + friend bool operator ==(const TRange& left, const TRange& right) { + return left.Prefix == right.Prefix + && left.Filter == right.Filter + && left.Suffix == right.Suffix + && left.IsStrict == right.IsStrict; + } + }; + + struct TFolderList { + TString Prefix; + TVector<TString> Attributes; + + friend bool operator ==(const TFolderList& left, const TFolderList& right) { + return left.Prefix == right.Prefix + && left.Attributes == right.Attributes; + } + }; + + struct TWalkFoldersArgs { + TFolderList InitialFolder; + + TExprNode::TPtr PickledUserState; + TExprNode::TPtr UserStateType; + + TExprNode::TPtr PreHandler; + TExprNode::TPtr ResolveHandler; + TExprNode::TPtr DiveHandler; + TExprNode::TPtr PostHandler; + }; + + struct TWalkFoldersImplArgs { + TExprNode::TPtr UserStateExpr; + TExprNode::TPtr UserStateType; + + ui64 StateKey; + }; + +public: + TYtKey() { + } + + EType GetType() const { + return Type; + } + + const TExprNode* GetNode() const { + return KeyNode; + } + + const TString& GetPath() const { + YQL_ENSURE(Type != EType::Undefined); + return Path; + } + + const TString& GetView() const { + YQL_ENSURE(Type != EType::Undefined); + return View; + } + + bool IsAnonymous() const { + return Anonymous; + } + + const TMaybe<TRange>& GetRange() const { + YQL_ENSURE(Type != EType::Undefined); + return Range; + } + + const TMaybe<TFolderList>& GetFolder() const { + YQL_ENSURE(Type != EType::Undefined); + return Folder; + } + + TMaybe<TWalkFoldersArgs>& GetWalkFolderArgs() { + YQL_ENSURE(Type != EType::Undefined); + return WalkFolderArgs; + } + + TMaybe<TWalkFoldersImplArgs>& GetWalkFolderImplArgs() { + YQL_ENSURE(Type != EType::Undefined); + return WalkFolderImplArgs; + } + + bool Parse(const TExprNode& key, TExprContext& ctx, bool isOutput = false); + +private: + EType Type = EType::Undefined; + const TExprNode* KeyNode = nullptr; + TString Path; + TString View; + bool Anonymous = false; + TMaybe<TRange> Range; + TMaybe<TFolderList> Folder; + TMaybe<TWalkFoldersArgs> WalkFolderArgs; + TMaybe<TWalkFoldersImplArgs> WalkFolderImplArgs; +}; + +class TYtInputKeys { +public: + TYtInputKeys() { + } + + TYtKey::EType GetType() const { + return Type; + } + + const TVector<TYtKey>& GetKeys() const { + return Keys; + } + + TVector<TYtKey>&& ExtractKeys() { + return std::move(Keys); + } + + bool IsProcessed() const { + return HasNonKeys; + } + + bool GetStrictConcat() const { + return StrictConcat; + } + + bool Parse(const TExprNode& keysNode, TExprContext& ctx); + +private: + TYtKey::EType Type = TYtKey::EType::Undefined; + TVector<TYtKey> Keys; + bool StrictConcat = true; + bool HasNonKeys = false; +}; + +class TYtOutputKey: public TYtKey { +public: + TYtOutputKey() { + } + + bool Parse(const TExprNode& keyNode, TExprContext& ctx); +}; + +} + +template <> +struct hash<NYql::TYtKey::TRange> { + inline size_t operator()(const NYql::TYtKey::TRange& r) const { + size_t res = ComputeHash(r.Prefix); + res = CombineHashes(res, NumericHash(r.Filter.Get())); + res = CombineHashes(res, ComputeHash(r.Suffix)); + res = CombineHashes(res, size_t(r.IsStrict)); + return res; + } +}; + +template <> +struct hash<NYql::TYtKey::TFolderList> { + inline size_t operator()(const NYql::TYtKey::TFolderList& r) const { + size_t res = ComputeHash(r.Prefix); + res = CombineHashes(res, r.Attributes.size()); + for (auto& a: r.Attributes) { + res = CombineHashes(res, ComputeHash(a)); + } + return res; + } +}; diff --git a/yt/yql/providers/yt/provider/yql_yt_load_columnar_stats.cpp b/yt/yql/providers/yt/provider/yql_yt_load_columnar_stats.cpp new file mode 100644 index 0000000000..789938e02f --- /dev/null +++ b/yt/yql/providers/yt/provider/yql_yt_load_columnar_stats.cpp @@ -0,0 +1,237 @@ +#include "yql_yt_provider_impl.h" +#include "yql_yt_gateway.h" +#include "yql_yt_helpers.h" + +#include <library/cpp/yson/node/node_io.h> +#include <yt/cpp/mapreduce/common/helpers.h> + +#include <yt/yql/providers/yt/expr_nodes/yql_yt_expr_nodes.h> + +#include <yql/essentials/core/yql_graph_transformer.h> +#include <yql/essentials/core/yql_expr_optimize.h> + +#include <library/cpp/threading/future/async.h> +#include <yql/essentials/utils/log/log.h> + + +namespace NYql { + +namespace { + +using namespace NNodes; + +class TPathStatusState : public TThrRefBase { +public: + using TPtr = TIntrusivePtr<TPathStatusState>; + + void EnsureNoInflightRequests() const { + TGuard<TMutex> guard(Lock); + YQL_ENSURE(PathStatResults.empty()); + } + + TNodeMap<IYtGateway::TPathStatResult> PullPathStatResults() { + TNodeMap<IYtGateway::TPathStatResult> results; + TGuard<TMutex> guard(Lock); + results.swap(PathStatResults); + return results; + } + + void MarkReady(TExprNode* node, const IYtGateway::TPathStatResult& result) { + TGuard<TMutex> guard(Lock); + YQL_ENSURE(PathStatResults.count(node) == 0); + PathStatResults[node] = result; + } + +private: + mutable TMutex Lock; + TNodeMap<IYtGateway::TPathStatResult> PathStatResults; +}; + +class TYtLoadColumnarStatsTransformer : public TGraphTransformerBase { +public: + TYtLoadColumnarStatsTransformer(TYtState::TPtr state) + : State_(state) + , PathStatusState(new TPathStatusState) + { + } + + void Rewind() final { + PathStatusState = MakeIntrusive<TPathStatusState>(); + AsyncFuture = {}; + } + +private: + TStatus DoTransform(TExprNode::TPtr input, TExprNode::TPtr& output, TExprContext& ctx) override { + output = input; + + PathStatusState->EnsureNoInflightRequests(); + TVector<std::pair<IYtGateway::TPathStatOptions, TExprNode*>> pathStatArgs; + bool hasError = false; + TNodeOnNodeOwnedMap sectionRewrites; + VisitExpr(input, [this, &pathStatArgs, &hasError, §ionRewrites, &ctx](const TExprNode::TPtr& node) { + const TMaybe<ui64> maxChunkCountExtendedStats = State_->Configuration->ExtendedStatsMaxChunkCount.Get(); + if (auto maybeSection = TMaybeNode<TYtSection>(node)) { + TYtSection section = maybeSection.Cast(); + if (NYql::HasSetting(section.Settings().Ref(), EYtSettingType::StatColumns)) { + auto columnList = NYql::GetSettingAsColumnList(section.Settings().Ref(), EYtSettingType::StatColumns); + + TMaybe<TString> cluster; + TVector<IYtGateway::TPathStatReq> pathStatReqs; + size_t idx = 0; + ui64 totalChunkCount = 0; + for (auto path: section.Paths()) { + bool hasStat = false; + if (path.Table().Maybe<TYtTable>().Stat().Maybe<TYtStat>()) { + hasStat = true; + } else if (path.Table().Maybe<TYtOutTable>().Stat().Maybe<TYtStat>()) { + hasStat = true; + } else if (auto ytOutput = path.Table().Maybe<TYtOutput>()) { + auto outTable = GetOutTable(ytOutput.Cast()); + if (outTable.Maybe<TYtOutTable>().Stat().Maybe<TYtStat>()) { + hasStat = true; + } + } + + if (!hasStat) { + YQL_CLOG(INFO, ProviderYt) << "Removing columnar stat from YtSection #" << section.Ref().UniqueId() + << " due to missing stats in path #" << idx; + + sectionRewrites[section.Raw()] = Build<TYtSection>(ctx, section.Ref().Pos()) + .InitFrom(section) + .Settings(NYql::RemoveSetting(section.Settings().Ref(), EYtSettingType::StatColumns, ctx)) + .Done().Ptr(); + } + + if (!sectionRewrites.empty()) { + // no need to prepare columnar stat requests in this case + return !hasError; + } + + TYtPathInfo pathInfo(path); + YQL_ENSURE(pathInfo.Table->Stat); + totalChunkCount += pathInfo.Table->Stat->ChunkCount; + + TString currCluster; + if (auto ytTable = path.Table().Maybe<TYtTable>()) { + currCluster = TString{ytTable.Cast().Cluster().Value()}; + } else { + currCluster = TString{GetOutputOp(path.Table().Cast<TYtOutput>()).DataSink().Cluster().Value()}; + } + YQL_ENSURE(currCluster); + + if (cluster) { + YQL_ENSURE(currCluster == *cluster); + } else { + cluster = currCluster; + } + + auto ytPath = BuildYtPathForStatRequest(*cluster, pathInfo, columnList, *State_, ctx); + if (!ytPath) { + hasError = true; + return false; + } + + pathStatReqs.push_back( + IYtGateway::TPathStatReq() + .Path(*ytPath) + .IsTemp(pathInfo.Table->IsTemp) + .IsAnonymous(pathInfo.Table->IsAnonymous) + .Epoch(pathInfo.Table->Epoch.GetOrElse(0)) + ); + + ++idx; + } + + bool requestExtendedStats = maxChunkCountExtendedStats && + (*maxChunkCountExtendedStats == 0 || totalChunkCount <= *maxChunkCountExtendedStats); + + if (pathStatReqs) { + auto pathStatOptions = IYtGateway::TPathStatOptions(State_->SessionId) + .Cluster(*cluster) + .Paths(pathStatReqs) + .Config(State_->Configuration->Snapshot()) + .Extended(requestExtendedStats); + + auto tryResult = State_->Gateway->TryPathStat(IYtGateway::TPathStatOptions(pathStatOptions)); + if (!tryResult.Success()) { + pathStatArgs.emplace_back(std::move(pathStatOptions), node.Get()); + } + } + } + } + return !hasError; + }); + + if (hasError) { + return TStatus::Error; + } + + if (!sectionRewrites.empty()) { + auto status = RemapExpr(input, output, sectionRewrites, ctx, TOptimizeExprSettings(State_->Types)); + YQL_ENSURE(status.Level != TStatus::Ok); + return status; + } + + if (pathStatArgs.empty()) { + return TStatus::Ok; + } + + TVector<NThreading::TFuture<void>> futures; + YQL_CLOG(INFO, ProviderYt) << "Starting " << pathStatArgs.size() << " requests for columnar stats"; + for (auto& arg : pathStatArgs) { + IYtGateway::TPathStatOptions& options = arg.first; + TExprNode* node = arg.second; + + auto future = State_->Gateway->PathStat(std::move(options)); + + futures.push_back(future.Apply([pathStatusState = PathStatusState, node](const NThreading::TFuture<IYtGateway::TPathStatResult>& result) { + pathStatusState->MarkReady(node, result.GetValueSync()); + })); + } + + AsyncFuture = WaitExceptionOrAll(futures); + return TStatus::Async; + } + + NThreading::TFuture<void> DoGetAsyncFuture(const TExprNode& input) override { + Y_UNUSED(input); + return AsyncFuture; + } + + TStatus DoApplyAsyncChanges(TExprNode::TPtr input, TExprNode::TPtr& output, TExprContext& ctx) override { + output = input; + + TNodeMap<IYtGateway::TPathStatResult> results = PathStatusState->PullPathStatResults(); + YQL_ENSURE(!results.empty()); + + for (auto& item : results) { + auto& node = item.first; + auto& result = item.second; + if (!result.Success()) { + TIssueScopeGuard issueScope(ctx.IssueManager, [&]() { + return MakeIntrusive<TIssue>( + ctx.GetPosition(node->Pos()), + TStringBuilder() << "Execution of node: " << node->Content() + ); + }); + result.ReportIssues(ctx.IssueManager); + return TStatus::Error; + } + } + + YQL_CLOG(INFO, ProviderYt) << "Applied " << results.size() << " results of columnar stats"; + return TStatus::Repeat; + } + + TYtState::TPtr State_; + TPathStatusState::TPtr PathStatusState; + NThreading::TFuture<void> AsyncFuture; +}; + +} // namespace + +THolder<IGraphTransformer> CreateYtLoadColumnarStatsTransformer(TYtState::TPtr state) { + return THolder(new TYtLoadColumnarStatsTransformer(state)); +} + +} // namespace NYql diff --git a/yt/yql/providers/yt/provider/yql_yt_load_table_meta.cpp b/yt/yql/providers/yt/provider/yql_yt_load_table_meta.cpp new file mode 100644 index 0000000000..2d735c3f25 --- /dev/null +++ b/yt/yql/providers/yt/provider/yql_yt_load_table_meta.cpp @@ -0,0 +1,328 @@ +#include "yql_yt_provider_impl.h" +#include "yql_yt_gateway.h" + +#include <yt/yql/providers/yt/expr_nodes/yql_yt_expr_nodes.h> +#include <yt/yql/providers/yt/common/yql_names.h> +#include <yt/yql/providers/yt/common/yql_configuration.h> +#include <yt/yql/providers/yt/lib/row_spec/yql_row_spec.h> +#include <yql/essentials/providers/common/provider/yql_provider.h> + +#include <yql/essentials/core/yql_graph_transformer.h> +#include <yql/essentials/core/yql_type_annotation.h> +#include <yql/essentials/core/yql_expr_optimize.h> +#include <yql/essentials/core/expr_nodes/yql_expr_nodes.h> + +#include <library/cpp/threading/future/async.h> +#include <library/cpp/yson/node/node_io.h> + +#include <util/generic/ptr.h> +#include <util/generic/string.h> +#include <util/generic/algorithm.h> +#include <util/string/join.h> + +#include <utility> + +namespace NYql { + +namespace { + +using namespace NNodes; + +class TYtLoadTableMetadataTransformer : public TGraphTransformerBase { +private: + struct TLoadContext: public TThrRefBase { + using TPtr = TIntrusivePtr<TLoadContext>; + + ui32 Epoch = 0; + IYtGateway::TTableInfoResult Result; + TVector<std::pair<std::pair<TString, TString>, TYtTableDescription*>> Tables; + }; +public: + TYtLoadTableMetadataTransformer(TYtState::TPtr state) + : State_(state) + { + } + + TStatus DoTransform(TExprNode::TPtr input, TExprNode::TPtr& output, TExprContext& ctx) final { + output = input; + + if (ctx.Step.IsDone(TExprStep::LoadTablesMetadata)) { + return TStatus::Ok; + } + + ui32 loadEpoch = 0; + size_t settingsVer = 0; + if (State_->LoadEpochMetadata.Defined()) { + std::tie(loadEpoch, settingsVer) = *State_->LoadEpochMetadata; + State_->LoadEpochMetadata.Clear(); + } + + LoadCtx = MakeIntrusive<TLoadContext>(); + LoadCtx->Epoch = loadEpoch; + IYtGateway::TGetTableInfoOptions opts{State_->SessionId}; + + if (0 != loadEpoch) { + auto p = State_->EpochDependencies.FindPtr(loadEpoch); + YQL_ENSURE(p && !p->empty()); + for (auto& clusterAndTable: *p) { + TYtTableDescription& tableDesc = State_->TablesData->GetModifTable(clusterAndTable.first, clusterAndTable.second, loadEpoch); + TString tableName = clusterAndTable.second; + if (tableDesc.IsAnonymous) { + tableName = State_->AnonymousLabels.Value(std::make_pair(clusterAndTable.first, tableName), TString()); + YQL_ENSURE(tableName, "Unaccounted anonymous table: " << clusterAndTable.first << '.' << clusterAndTable.second); + } + opts.Tables().push_back(IYtGateway::TTableReq() + .Cluster(clusterAndTable.first) + .Table(tableName) + .Intents(tableDesc.Intents) + .InferSchemaRows(0) + .ForceInferSchema(false) + .Anonymous(tableDesc.IsAnonymous) + .IgnoreYamrDsv(State_->Configuration->IgnoreYamrDsv.Get().GetOrElse(false)) + .IgnoreWeakSchema(State_->Configuration->IgnoreWeakSchema.Get().GetOrElse(false)) + ); + LoadCtx->Tables.emplace_back(clusterAndTable, &tableDesc); + } + } else { + for (auto& clusterAndTable: State_->TablesData->GetAllZeroEpochTables()) { + TYtTableDescription& tableDesc = State_->TablesData->GetModifTable(clusterAndTable.first, clusterAndTable.second, loadEpoch); + if (tableDesc.Meta) { + if (!tableDesc.HasWriteLock && HasModifyIntents(tableDesc.Intents)) { + TStringBuilder msg; + msg << "Table " << clusterAndTable.second.Quote() + << " is locked exclusively after taking snapshot lock." + << " This may lead query failure if YQL detected concurrent table modification before taking exclusive lock." + << " Add WITH XLOCK hint to all reads from this table to resolve this warning"; + if (!ctx.AddWarning(YqlIssue(TPosition(), EYqlIssueCode::TIssuesIds_EIssueCode_YT_LATE_TABLE_XLOCK, msg))) { + return TStatus::Error; + } + } + + // skip if table already has loaded metadata and has only read intents + if (State_->Types->IsReadOnly || State_->Types->UseTableMetaFromGraph || tableDesc.HasWriteLock || !HasModifyIntents(tableDesc.Intents)) { + // Intents/views can be updated since evaluation phase + if (!tableDesc.FillViews( + clusterAndTable.first, clusterAndTable.second, State_->Types->QContext, ctx, + State_->Types->Modules.get(), State_->Types->UrlListerManager.Get(), *State_->Types->RandomProvider, + State_->Configuration->ViewIsolation.Get().GetOrElse(false), + State_->Types->UdfResolver + )) { + return TStatus::Error; + } + continue; + } + } + TString tableName = clusterAndTable.second; + if (tableDesc.IsAnonymous) { + tableName = State_->AnonymousLabels.Value(std::make_pair(clusterAndTable.first, tableName), TString()); + YQL_ENSURE(tableName, "Unaccounted anonymous table: " << clusterAndTable.first << '.' << clusterAndTable.second); + } + opts.Tables().push_back(IYtGateway::TTableReq() + .Cluster(clusterAndTable.first) + .Table(tableName) + .LockOnly(bool(tableDesc.Meta)) + .Intents(tableDesc.Intents) + .InferSchemaRows(tableDesc.InferSchemaRows) + .ForceInferSchema(tableDesc.ForceInferSchema) + .Anonymous(tableDesc.IsAnonymous) + .IgnoreYamrDsv(State_->Configuration->IgnoreYamrDsv.Get().GetOrElse(false)) + .IgnoreWeakSchema(State_->Configuration->IgnoreWeakSchema.Get().GetOrElse(false)) + ); + LoadCtx->Tables.emplace_back(clusterAndTable, &tableDesc); + } + } + + if (opts.Tables().empty()) { + return TStatus::Ok; + } + + auto config = loadEpoch ? State_->Configuration->GetSettingsVer(settingsVer) : State_->Configuration->Snapshot(); + opts.Config(config).ReadOnly(State_->Types->IsReadOnly).Epoch(loadEpoch); + + auto future = State_->Gateway->GetTableInfo(std::move(opts)); + auto loadCtx = LoadCtx; + + AsyncFuture = future.Apply( + [loadCtx](const NThreading::TFuture<IYtGateway::TTableInfoResult>& completedFuture) { + loadCtx->Result = completedFuture.GetValue(); + YQL_ENSURE(!loadCtx->Result.Success() || loadCtx->Result.Data.size() == loadCtx->Tables.size()); + }); + + return TStatus::Async; + } + + NThreading::TFuture<void> DoGetAsyncFuture(const TExprNode& input) final { + Y_UNUSED(input); + return AsyncFuture; + } + + TStatus DoApplyAsyncChanges(TExprNode::TPtr input, TExprNode::TPtr& output, TExprContext& ctx) final { + output = input; + // Raise errors if any + AsyncFuture.GetValue(); + + LoadCtx->Result.ReportIssues(ctx.IssueManager); + if (!LoadCtx->Result.Success()) { + LoadCtx.Reset(); + return TStatus::Error; + } + + ctx.Step.Repeat(TExprStep::ExpandApplyForLambdas); + THashMap<std::pair<TString, TString>, TYtTableDescription*> tableDescrs; + for (size_t i = 0 ; i < LoadCtx->Result.Data.size(); ++i) { + tableDescrs.emplace(LoadCtx->Tables[i]); + + TString cluster = LoadCtx->Tables[i].first.first; + TString tableName = LoadCtx->Tables[i].first.second; + TYtTableDescription& tableDesc = *LoadCtx->Tables[i].second; + + if (LoadCtx->Result.Data[i].WriteLock) { + tableDesc.HasWriteLock = true; + } + + TIssueScopeGuard issueScope(ctx.IssueManager, [tableName]() { + return MakeIntrusive<TIssue>(TStringBuilder() << "Table " << tableName); + }); + + if (auto meta = LoadCtx->Result.Data[i].Meta) { + tableDesc.Meta = meta; + const auto schemaAttrs = std::initializer_list<TStringBuf>{YqlRowSpecAttribute, SCHEMA_ATTR_NAME, READ_SCHEMA_ATTR_NAME, INFER_SCHEMA_ATTR_NAME}; + if (AnyOf(schemaAttrs, [&tableDesc](TStringBuf attr) { return tableDesc.Meta->Attrs.contains(attr); })) { + auto rowSpec = MakeIntrusive<TYqlRowSpecInfo>(); + if (!rowSpec->Parse(tableDesc.Meta->Attrs, ctx)) { + return TStatus::Error; + } + if (!State_->Configuration->UseNativeDescSort.Get().GetOrElse(false) && rowSpec->ClearNativeDescendingSort(ctx)) { + if (!ctx.AddWarning(YqlIssue(TPosition(), EYqlIssueCode::TIssuesIds_EIssueCode_YT_NATIVE_DESC_SORT_IGNORED, "Native descending sort is ignored"))) { + return TStatus::Error; + } + } + // Some sanity checks + if (tableDesc.RowSpec && tableDesc.RowSpec->IsSorted()) { + YQL_ENSURE(rowSpec->IsSorted(), "Bad predicted sort for the table '" + << cluster << "." << tableName << "' (epoch=" << LoadCtx->Epoch << "), expected sorted, but actually not."); + YQL_ENSURE(rowSpec->SortedBy.size() >= tableDesc.RowSpec->SortedBy.size() + && Equal(tableDesc.RowSpec->SortedBy.begin(), tableDesc.RowSpec->SortedBy.end(), rowSpec->SortedBy.begin()), + "Bad predicted SortedBy for the table '" + << cluster << "." << tableName << "' (epoch=" << LoadCtx->Epoch << "), expected: [" << JoinSeq(",", tableDesc.RowSpec->SortedBy) + << "], actual: [" << JoinSeq(",", rowSpec->SortedBy) << ']'); + YQL_ENSURE(rowSpec->SortMembers.size() >= tableDesc.RowSpec->SortMembers.size() + && Equal(tableDesc.RowSpec->SortMembers.begin(), tableDesc.RowSpec->SortMembers.end(), rowSpec->SortMembers.begin()), + "Bad predicted SortMembers for the table '" + << cluster << "." << tableName << "' (epoch=" << LoadCtx->Epoch << "), expected: [" << JoinSeq(",", tableDesc.RowSpec->SortMembers) + << "], actual: [" << JoinSeq(",", rowSpec->SortMembers) << ']'); + } + tableDesc.RowSpec = rowSpec; + ForEach(std::begin(schemaAttrs), std::end(schemaAttrs), [&tableDesc](TStringBuf attr) { tableDesc.Meta->Attrs.erase(attr); }); + + if (LoadCtx->Epoch != 0 && tableDesc.RawRowType) { + // Some sanity checks + YQL_ENSURE(IsSameAnnotation(*tableDesc.RawRowType, *tableDesc.RowSpec->GetType()), "Scheme diff: " << GetTypeDiff(*tableDesc.RawRowType, *tableDesc.RowSpec->GetType())); + } + } + if (auto rowSpecAttr = tableDesc.Meta->Attrs.FindPtr(TString{YqlRowSpecAttribute}.append("_qb2"))) { + auto rowSpec = MakeIntrusive<TYqlRowSpecInfo>(); + if (!rowSpec->Parse(*rowSpecAttr, ctx)) { + return TStatus::Error; + } + tableDesc.QB2RowSpec = rowSpec; + tableDesc.Meta->Attrs.erase(TString{YqlRowSpecAttribute}.append("_qb2")); + } + + if (0 == LoadCtx->Epoch) { + if (!tableDesc.Fill( + cluster, tableName, State_->Types->QContext, ctx, + State_->Types->Modules.get(), State_->Types->UrlListerManager.Get(), *State_->Types->RandomProvider, + State_->Configuration->ViewIsolation.Get().GetOrElse(false), + State_->Types->UdfResolver + )) { + return TStatus::Error; + } + } + } + + if (auto stat = LoadCtx->Result.Data[i].Stat) { + tableDesc.Stat = stat; + if (HasReadIntents(tableDesc.Intents)) { + const auto& securityTagsSet = tableDesc.Stat->SecurityTags; + const TString tmpFolder = GetTablesTmpFolder(*State_->Configuration); + if (!securityTagsSet.empty() && tmpFolder.empty()) { + TStringBuilder msg; + msg << "Table " << cluster << "." << tableName + << " contains sensitive data, but is used with the default tmp folder." + << " This may lead to sensitive data being leaked, consider using a protected tmp folder with the TmpFolder pragma."; + auto issue = YqlIssue(TPosition(), EYqlIssueCode::TIssuesIds_EIssueCode_YT_SECURE_DATA_IN_COMMON_TMP, msg); + if (State_->Configuration->ForceTmpSecurity.Get().GetOrElse(false)) { + ctx.AddError(issue); + return TStatus::Error; + } else { + if (!ctx.AddWarning(issue)) { + return TStatus::Error; + } + } + } + } + } + } + + TStatus status = TStatus::Ok; + if (LoadCtx->Epoch) { + TNodeMap<TYtTableDescription*> tableNodes; + VisitExpr(input, [&](const TExprNode::TPtr& node) { + TString cluster; + if (auto maybeTable = TMaybeNode<TYtTable>(node)) { + TYtTable table = maybeTable.Cast(); + if (auto p = tableDescrs.FindPtr(std::make_pair(TString{table.Cluster().Value()}, TString{table.Name().Value()}))) { + if (TEpochInfo::Parse(table.Epoch().Ref()).GetOrElse(0) == LoadCtx->Epoch) { + tableNodes.emplace(node.Get(), *p); + } + } + return false; + } + return true; + }); + + TOptimizeExprSettings settings(nullptr); + settings.VisitChanges = true; + settings.VisitStarted = true; + status = OptimizeExpr(input, output, [&](const TExprNode::TPtr& node, TExprContext& ctx) -> TExprNode::TPtr { + auto it = tableNodes.find(node.Get()); + if (it != tableNodes.cend()) { + TYtTableInfo table = node; + + const TYtTableDescription& tableDesc = *it->second; + table.Stat = tableDesc.Stat; + table.Meta = tableDesc.Meta; + table.RowSpec = tableDesc.RowSpec; + + return table.ToExprNode(ctx, node->Pos()).Ptr(); + } + return node; + }, ctx, settings); + if (input != output) { + status = status.Combine(TStatus(TStatus::Repeat, true)); + } + } + + LoadCtx.Reset(); + return status; + } + + void Rewind() final { + LoadCtx.Reset(); + AsyncFuture = {}; + } + +private: + TYtState::TPtr State_; + TLoadContext::TPtr LoadCtx; + NThreading::TFuture<void> AsyncFuture; +}; + +} // namespace + +THolder<IGraphTransformer> CreateYtLoadTableMetadataTransformer(TYtState::TPtr state) { + return THolder(new TYtLoadTableMetadataTransformer(state)); +} + +} // namespace NYql diff --git a/yt/yql/providers/yt/provider/yql_yt_logical_optimize.cpp b/yt/yql/providers/yt/provider/yql_yt_logical_optimize.cpp new file mode 100644 index 0000000000..8577696de2 --- /dev/null +++ b/yt/yql/providers/yt/provider/yql_yt_logical_optimize.cpp @@ -0,0 +1,2569 @@ +#include "yql_yt_provider_impl.h" +#include "yql_yt_helpers.h" +#include "yql_yt_table.h" + +#include <yt/yql/providers/yt/expr_nodes/yql_yt_expr_nodes.h> +#include <yt/yql/providers/yt/opt/yql_yt_join.h> +#include <yt/yql/providers/yt/opt/yql_yt_key_selector.h> +#include <yt/yql/providers/yt/common/yql_configuration.h> +#include <yql/essentials/providers/common/transform/yql_optimize.h> +#include <yql/essentials/providers/common/provider/yql_provider.h> +#include <yql/essentials/providers/common/codec/yql_codec_type_flags.h> +#include <yql/essentials/core/yql_aggregate_expander.h> +#include <yql/essentials/core/yql_expr_type_annotation.h> +#include <yql/essentials/core/yql_expr_optimize.h> +#include <yql/essentials/core/yql_opt_window.h> +#include <yql/essentials/core/yql_opt_utils.h> +#include <yql/essentials/core/yql_opt_match_recognize.h> +#include <yql/essentials/core/yql_join.h> +#include <yql/essentials/core/yql_type_helpers.h> +#include <yql/essentials/utils/log/log.h> + +#include <library/cpp/disjoint_sets/disjoint_sets.h> + +#include <util/generic/set.h> +#include <util/generic/algorithm.h> +#include <util/generic/vector.h> +#include <util/generic/map.h> + +#include <utility> + +namespace NYql { + +using namespace NNodes; + +class TYtLogicalOptProposalTransformer : public TOptimizeTransformerBase { +public: + TYtLogicalOptProposalTransformer(TYtState::TPtr state) + : TOptimizeTransformerBase(state->Types, NLog::EComponent::ProviderYt, state->Configuration->DisableOptimizers.Get().GetOrElse(TSet<TString>())) + , State_(state) + { +#define HNDL(name) "LogicalOptimizer-"#name, Hndl(&TYtLogicalOptProposalTransformer::name) + AddHandler(0, &TYtMap::Match, HNDL(DirectRow)); + AddHandler(0, Names({TYtReduce::CallableName(), TYtMapReduce::CallableName()}), HNDL(IsKeySwitch)); + AddHandler(0, &TCoLeft::Match, HNDL(TrimReadWorld)); + AddHandler(0, &TCoCalcOverWindowBase::Match, HNDL(CalcOverWindow)); + AddHandler(0, &TCoCalcOverWindowGroup::Match, HNDL(CalcOverWindow)); + AddHandler(0, &TCoSort::Match, HNDL(SortOverAlreadySorted<false>)); + AddHandler(0, &TCoTopSort::Match, HNDL(SortOverAlreadySorted<true>)); + AddHandler(0, &TCoNth::Match, HNDL(Demux)); + AddHandler(0, &TYtMap::Match, HNDL(VarianItemOverInput)); + AddHandler(0, &TYtWithUserJobsOpBase::Match, HNDL(VisitOverInputWithEqualLambdas)); + AddHandler(0, &TYtWithUserJobsOpBase::Match, HNDL(UnorderedOverInput)); + AddHandler(0, &TCoFlatMapBase::Match, HNDL(DirectRowInFlatMap)); + AddHandler(0, &TCoUnorderedBase::Match, HNDL(Unordered)); + AddHandler(0, &TCoAggregate::Match, HNDL(CountAggregate)); + AddHandler(0, &TYtReadTable::Match, HNDL(ZeroSampleToZeroLimit)); + AddHandler(0, &TCoMatchRecognize::Match, HNDL(MatchRecognize)); + + AddHandler(1, &TCoFilterNullMembers::Match, HNDL(FilterNullMemebers<TCoFilterNullMembers>)); + AddHandler(1, &TCoSkipNullMembers::Match, HNDL(FilterNullMemebers<TCoSkipNullMembers>)); + AddHandler(1, &TCoFlatMapBase::Match, HNDL(FuseFlatmaps)); + AddHandler(1, &TCoZip::Match, HNDL(Zip)); + AddHandler(1, &TCoZipAll::Match, HNDL(ZipAll)); + AddHandler(1, &TYtWithUserJobsOpBase::Match, HNDL(OutputInFlatMap)); + AddHandler(1, &TYtOutput::Match, HNDL(BypassCopy)); + AddHandler(1, &TCoAggregateBase::Match, HNDL(Aggregate)); + AddHandler(1, &TCoExtractMembers::Match, HNDL(ExtractMembers)); + AddHandler(1, &TCoExtractMembers::Match, HNDL(ExtractMembersOverContent)); + AddHandler(1, &TCoRight::Match, HNDL(PushdownReadColumns)); + AddHandler(1, &TYtTransientOpBase::Match, HNDL(PushdownOpColumns)); + AddHandler(1, &TCoCountBase::Match, HNDL(TakeOrSkip)); + AddHandler(1, &TCoEquiJoin::Match, HNDL(SelfInnerJoinWithSameKeys)); + AddHandler(1, &TCoExtendBase::Match, HNDL(ExtendOverSameMap)); + AddHandler(1, &TCoFlatMapBase::Match, HNDL(FlatMapOverExtend)); + AddHandler(1, &TCoTake::Match, HNDL(TakeOverExtend)); + + AddHandler(2, &TCoEquiJoin::Match, HNDL(ConvertToCommonTypeForForcedMergeJoin)); + AddHandler(2, &TCoShuffleByKeys::Match, HNDL(ShuffleByKeys)); +#undef HNDL + } + +protected: + TYtSection PushdownSectionColumns(TYtSection section, TExprContext& ctx, const TGetParents& getParents) const { + if (HasNonEmptyKeyFilter(section)) { + // wait until key filter values are calculated and pushed to Path/Ranges + return section; + } + bool hasNewPath = false; + TVector<TExprBase> paths; + const bool useNativeDescSort = State_->Configuration->UseNativeDescSort.Get().GetOrElse(DEFAULT_USE_NATIVE_DESC_SORT); + for (auto path: section.Paths()) { + paths.push_back(path); + + auto columns = TYtColumnsInfo(path.Columns()); + if (!columns.HasColumns()) { + // No column filter + continue; + } + + auto type = path.Table().Ref().GetTypeAnn()->Cast<TListExprType>()->GetItemType()->Cast<TStructExprType>(); + if (type->GetSize() <= columns.GetColumns()->size()) { + // The same column set as original type + continue; + } + + // Use operation output only + if (auto op = path.Table().Maybe<TYtOutput>().Operation().Maybe<TYtTransientOpBase>()) { + if (op.Raw()->StartsExecution() || op.Raw()->HasResult()) { + // Operation is already executed + continue; + } + + if (IsOutputUsedMultipleTimes(op.Cast().Ref(), *getParents())) { + // Operation output is used multiple times + continue; + } + + if (op.Cast().Ref().GetTypeAnn()->Cast<TTupleExprType>()->GetItems()[1]->Cast<TListExprType>()->GetItemType()->GetKind() == ETypeAnnotationKind::Variant) { + // Operation with multi-output + continue; + } + + if (NYql::HasSetting(op.Settings().Ref(), EYtSettingType::SortLimitBy)) { + continue; + } + + const TYqlRowSpecInfo::TPtr rowSpec = TYtTableBaseInfo::GetRowSpec(path.Table()); + YQL_ENSURE(rowSpec); + + if (rowSpec->HasAuxColumns()) { + continue; + } + + TSet<TString> effectiveColumns; + bool keepColumns = columns.GetRenames().Defined(); + for (auto& column : columns.GetColumns().GetRef()) { + keepColumns = keepColumns || !column.Type.empty(); + effectiveColumns.insert(column.Name); + } + + if (NYql::HasSetting(op.Settings().Ref(), EYtSettingType::KeepSorted)) { + for (size_t i = 0; i < rowSpec->SortedBy.size(); ++i) { + const bool inserted = effectiveColumns.insert(rowSpec->SortedBy[i]).second; + keepColumns = keepColumns || inserted; + } + } + + if (!path.Ranges().Maybe<TCoVoid>()) { + // add columns which are implicitly used by path.Ranges(), but not included in path.Columns(); + const auto ranges = TYtRangesInfo(path.Ranges()); + const size_t usedKeyPrefix = ranges.GetUsedKeyPrefixLength(); + YQL_ENSURE(usedKeyPrefix <= rowSpec->SortedBy.size()); + for (size_t i = 0; i < usedKeyPrefix; ++i) { + const bool inserted = effectiveColumns.insert(rowSpec->SortedBy[i]).second; + keepColumns = keepColumns || inserted; + } + } + + if (type->GetSize() <= effectiveColumns.size()) { + // The same column set as original type + continue; + } + + if (auto maybeMap = op.Maybe<TYtMap>()) { + TYtMap map = maybeMap.Cast(); + TVector<const TItemExprType*> structItems; + + auto mapper = ctx.Builder(map.Mapper().Pos()) + .Lambda() + .Param("stream") + .Callable(NYql::HasSetting(map.Settings().Ref(), EYtSettingType::Ordered) ? TCoOrderedFlatMap::CallableName() : TCoFlatMap::CallableName()) + .Apply(0, map.Mapper().Ref()) + .With(0, "stream") + .Seal() + .Lambda(1) + .Param("item") + .Callable(TCoJust::CallableName()) + .Callable(0, TCoAsStruct::CallableName()) + .Do([&](TExprNodeBuilder& parent) -> TExprNodeBuilder& { + size_t index = 0; + for (const auto& column: effectiveColumns) { + auto pos = type->FindItem(column); + YQL_ENSURE(pos); + structItems.push_back(type->GetItems()[*pos]); + parent + .List(index++) + .Atom(0, column) + .Callable(1, TCoMember::CallableName()) + .Arg(0, "item") + .Atom(1, column) + .Seal() + .Seal(); + } + return parent; + }) + .Seal() + .Seal() + .Seal() + .Seal() + .Seal() + .Build(); + + auto outStructType = ctx.MakeType<TStructExprType>(structItems); + TYtOutTableInfo mapOut(outStructType, State_->Configuration->UseNativeYtTypes.Get().GetOrElse(DEFAULT_USE_NATIVE_YT_TYPES) ? NTCF_ALL : NTCF_NONE); + + if (ctx.IsConstraintEnabled<TSortedConstraintNode>()) { + if (const auto s = path.Table().Ref().GetConstraint<TSortedConstraintNode>()) { + if (const auto sorted = s->FilterFields(ctx, [outStructType](const TPartOfConstraintBase::TPathType& path) { return !path.empty() && outStructType->FindItem(path.front()); }) ) { + TKeySelectorBuilder builder(map.Mapper().Pos(), ctx, useNativeDescSort, outStructType); + builder.ProcessConstraint(*sorted); + builder.FillRowSpecSort(*mapOut.RowSpec); + + if (builder.NeedMap()) { + mapper = ctx.Builder(map.Mapper().Pos()) + .Lambda() + .Param("stream") + .Apply(builder.MakeRemapLambda(true)) + .With(0) + .Apply(*mapper) + .With(0, "stream") + .Seal() + .Done() + .Seal() + .Seal() + .Build(); + } + } + } + } else { + mapOut.RowSpec->CopySortness(ctx, TYqlRowSpecInfo(map.Output().Item(0).RowSpec())); + } + mapOut.SetUnique(path.Ref().GetConstraint<TDistinctConstraintNode>(), map.Mapper().Pos(), ctx); + + TExprBase newColumns = Build<TCoVoid>(ctx, path.Pos()).Done(); + if (keepColumns) { + newColumns = path.Columns(); + } + hasNewPath = true; + paths.back() = Build<TYtPath>(ctx, path.Pos()) + .InitFrom(path) + .Table<TYtOutput>() + .Operation<TYtMap>() + .InitFrom(map) + .Output() + .Add(mapOut.ToExprNode(ctx, map.Pos()).Cast<TYtOutTable>()) + .Build() + .Mapper(mapper) + .Build() + .OutIndex().Value(TStringBuf("0")).Build() + .Mode(path.Table().Cast<TYtOutput>().Mode()) + .Build() + .Columns(newColumns) + .Stat<TCoVoid>().Build() + .Done(); + } + else if (auto maybeMerge = op.Maybe<TYtMerge>()) { + TYtMerge merge = maybeMerge.Cast(); + + auto prevRowSpec = TYqlRowSpecInfo(merge.Output().Item(0).RowSpec()); + auto prevOutType = merge.Output().Item(0).Ref().GetTypeAnn()->Cast<TListExprType>()->GetItemType()->Cast<TStructExprType>(); + TVector<const TItemExprType*> structItems; + for (const auto& column: effectiveColumns) { + auto pos = prevOutType->FindItem(column); + YQL_ENSURE(pos); + structItems.push_back(prevOutType->GetItems()[*pos]); + } + + TYtOutTableInfo mergeOut(ctx.MakeType<TStructExprType>(structItems), prevRowSpec.GetNativeYtTypeFlags()); + mergeOut.RowSpec->CopySortness(ctx, prevRowSpec, TYqlRowSpecInfo::ECopySort::WithDesc); + if (auto nativeType = prevRowSpec.GetNativeYtType()) { + mergeOut.RowSpec->CopyTypeOrders(*nativeType); + } + mergeOut.SetUnique(path.Ref().GetConstraint<TDistinctConstraintNode>(), merge.Pos(), ctx); + + TSet<TStringBuf> columnSet(effectiveColumns.begin(), effectiveColumns.end()); + if (mergeOut.RowSpec->HasAuxColumns()) { + for (auto item: mergeOut.RowSpec->GetAuxColumns()) { + columnSet.insert(item.first); + } + } + TExprBase newColumns = Build<TCoVoid>(ctx, path.Pos()).Done(); + if (keepColumns) { + newColumns = path.Columns(); + } + + hasNewPath = true; + paths.back() = Build<TYtPath>(ctx, path.Pos()) + .InitFrom(path) + .Table<TYtOutput>() + .Operation<TYtMerge>() + .InitFrom(merge) + .Input() + .Add(UpdateInputFields(merge.Input().Item(0), std::move(columnSet), ctx, false)) + .Build() + .Output() + .Add(mergeOut.ToExprNode(ctx, merge.Pos()).Cast<TYtOutTable>()) + .Build() + .Build() + .OutIndex().Value(TStringBuf("0")).Build() + .Mode(path.Table().Cast<TYtOutput>().Mode()) + .Build() + .Columns(newColumns) + .Stat<TCoVoid>().Build() + .Done(); + } + } + } + if (hasNewPath) { + return Build<TYtSection>(ctx, section.Pos()) + .InitFrom(section) + .Paths() + .Add(paths) + .Build() + .Done(); + } + return section; + } + +protected: + TMaybeNode<TExprBase> Aggregate(TExprBase node, TExprContext& ctx) const { + auto aggregate = node.Cast<TCoAggregateBase>(); + + auto input = aggregate.Input(); + if (!IsYtProviderInput(input)) { + return node; + } + + auto cluster = TString{GetClusterName(input)}; + TSyncMap syncList; + + for (auto handler: aggregate.Handlers()) { + auto trait = handler.Trait(); + if (auto maybeAggTrait = trait.Maybe<TCoAggregationTraits>()) { + const auto& t = maybeAggTrait.Cast(); + TVector<TExprBase> lambdas = { + t.InitHandler(), + t.UpdateHandler(), + t.SaveHandler(), + t.LoadHandler(), + t.MergeHandler(), + t.FinishHandler(), + }; + for (auto lambda : lambdas) { + if (!IsYtCompleteIsolatedLambda(lambda.Ref(), syncList, cluster, false)) { + return node; + } + } + } else if (trait.Ref().IsCallable("AggApply")) { + if (!IsYtCompleteIsolatedLambda(*trait.Ref().Child(2), syncList, cluster, false)) { + return node; + } + } + } + + auto usePhases = State_->Configuration->UseAggPhases.Get().GetOrElse(false); + auto usePartitionsByKeys = State_->Configuration->UsePartitionsByKeysForFinalAgg.Get().GetOrElse(true); + TAggregateExpander aggExpander(usePartitionsByKeys, false, node.Ptr(), ctx, *State_->Types, false, false, + usePhases, State_->Types->UseBlocks || State_->Types->BlockEngineMode == EBlockEngineMode::Force); + return aggExpander.ExpandAggregate(); + } + + TMaybeNode<TExprBase> DirectRow(TExprBase node, TExprContext& ctx) const { + if (State_->Configuration->UseSystemColumns.Get().GetOrElse(DEFAULT_USE_SYS_COLUMNS)) { + return node; + } + + auto map = node.Cast<TYtMap>(); + + auto mapper = map.Mapper(); + auto maybeLambda = GetFlatMapOverInputStream(mapper).Lambda(); + if (!maybeLambda) { + return node; + } + + TCoLambda lambda = maybeLambda.Cast(); + auto arg = lambda.Args().Arg(0); + + TNodeSet nodesToOptimize; + TProcessedNodesSet processedNodes; + processedNodes.insert(map.World().Ref().UniqueId()); + VisitExpr(lambda.Ptr(), [&nodesToOptimize, &processedNodes, arg](const TExprNode::TPtr& node) { + if (TCoTablePath::Match(node.Get())) { + if (node->ChildrenSize() == 0 || node->Child(0)->Child(0) == arg.Raw()) { + nodesToOptimize.insert(node.Get()); + } + } + else if (TCoTableRecord::Match(node.Get())) { + if (node->ChildrenSize() == 0 || node->Child(0)->Child(0) == arg.Raw()) { + nodesToOptimize.insert(node.Get()); + } + } + else if (TYtOutput::Match(node.Get())) { + // Stop traversing dependent operations + processedNodes.insert(node->UniqueId()); + return false; + } + return true; + }); + if (nodesToOptimize.empty()) { + return node; + } + + TExprNode::TPtr newBody = lambda.Body().Ptr(); + TPositionHandle tablePos; + TOptimizeExprSettings settings(State_->Types); + settings.ProcessedNodes = &processedNodes; // Prevent optimizer to go deeper than current operation + auto status = OptimizeExpr(newBody, newBody, [&nodesToOptimize, &tablePos, arg](const TExprNode::TPtr& input, TExprContext& ctx) -> TExprNode::TPtr { + if (nodesToOptimize.find(input.Get()) != nodesToOptimize.end()) { + if (TCoTablePath::Match(input.Get())) { + tablePos = input->Pos(); + if (input->ChildrenSize() == 1) { + return ctx.RenameNode(*input, TYtTablePath::CallableName()); + } + return Build<TYtTablePath>(ctx, input->Pos()) + .DependsOn() + .Input(arg) + .Build() + .Done().Ptr(); + } + else if (TCoTableRecord::Match(input.Get())) { + tablePos = input->Pos(); + if (input->ChildrenSize() == 1) { + return ctx.RenameNode(*input, TYtTableRecord::CallableName()); + } + return Build<TYtTableRecord>(ctx, input->Pos()) + .DependsOn() + .Input(arg) + .Build() + .Done().Ptr(); + } + } + + return input; + }, ctx, settings); + + if (status.Level == IGraphTransformer::TStatus::Error) { + return {}; + } + + if (status.Level == IGraphTransformer::TStatus::Ok) { + return node; + } + + auto newLambda = Build<TCoLambda>(ctx, lambda.Pos()) + .Args({TStringBuf("row")}) + .Body<TExprApplier>() + .Apply(TExprBase(newBody)) + .With(arg, TStringBuf("row")) + .Build() + .Done(); + + auto newMapper = Build<TCoLambda>(ctx, mapper.Pos()) + .Args({TStringBuf("stream")}) + .Body<TExprApplier>() + .Apply(mapper) + .With(mapper.Args().Arg(0), TStringBuf("stream")) + .With(lambda, newLambda) + .Build() + .Done(); + + bool stop = false; + for (auto section: map.Input()) { + for (auto path: section.Paths()) { + if (path.Table().Maybe<TYtOutput>()) { + auto issue = TIssue(ctx.GetPosition(tablePos), "TablePath(), TableName() and TableRecordIndex() will be empty for temporary tables.\n" + "Please consult documentation https://yql.yandex-team.ru/docs/yt/builtins/basic#tablepath for possible workaround"); + SetIssueCode(EYqlIssueCode::TIssuesIds_EIssueCode_YT_TABLE_PATH_RECORD_FOR_TMP, issue); + if (!ctx.AddWarning(issue)) { + return nullptr; + } + stop = true; + break; + } + } + if (stop) { + break; + } + } + + return ctx.ChangeChild(node.Ref(), TYtMap::idx_Mapper, newMapper.Ptr()); + } + + TMaybeNode<TExprBase> IsKeySwitch(TExprBase node, TExprContext& ctx) const { + if (State_->Configuration->UseSystemColumns.Get().GetOrElse(DEFAULT_USE_SYS_COLUMNS)) { + return node; + } + + auto lambda = node.Maybe<TYtMapReduce>() ? node.Cast<TYtMapReduce>().Reducer() : node.Cast<TYtReduce>().Reducer(); + TNodeSet nodesToOptimize; + TProcessedNodesSet processedNodes; + processedNodes.insert(node.Cast<TYtWithUserJobsOpBase>().World().Ref().UniqueId()); + VisitExpr(lambda.Ptr(), [&nodesToOptimize, &processedNodes](const TExprNode::TPtr& node) { + if (TCoIsKeySwitch::Match(node.Get())) { + nodesToOptimize.insert(node.Get()); + } + else if (TYtOutput::Match(node.Get())) { + // Stop traversing dependent operations + processedNodes.insert(node->UniqueId()); + return false; + } + return true; + }); + + if (nodesToOptimize.empty()) { + return node; + } + + TExprNode::TPtr newBody = lambda.Body().Ptr(); + TPosition tablePos; + TOptimizeExprSettings settings(State_->Types); + settings.ProcessedNodes = &processedNodes; // Prevent optimizer to go deeper than current operation + TStatus status = OptimizeExpr(newBody, newBody, [&](const TExprNode::TPtr& input, TExprContext& ctx) -> TExprNode::TPtr { + if (nodesToOptimize.find(input.Get()) != nodesToOptimize.end()) { + if (TCoIsKeySwitch::Match(input.Get())) { + return + Build<TYtIsKeySwitch>(ctx, input->Pos()) + .DependsOn() + .Input(input->HeadPtr()) + .Build() + .Done().Ptr(); + } + } + return input; + }, ctx, settings); + + if (status.Level == TStatus::Error) { + return {}; + } + + if (status.Level == TStatus::Ok) { + return node; + } + + auto newLambda = Build<TCoLambda>(ctx, lambda.Pos()) + .Args({TStringBuf("stream")}) + .Body<TExprApplier>() + .Apply(TExprBase(newBody)) + .With(lambda.Args().Arg(0), TStringBuf("stream")) + .Build() + .Done(); + + return ctx.ChangeChild(node.Ref(), node.Maybe<TYtMapReduce>() ? TYtMapReduce::idx_Reducer : TYtReduce::idx_Reducer, newLambda.Ptr()); + } + + template <typename T> + TMaybeNode<TExprBase> FilterNullMemebers(TExprBase node, TExprContext& ctx) const { + auto filterNullMembers = node.Cast<T>(); + if (!IsYtProviderInput(filterNullMembers.Input())) { + return node; + } + + YQL_ENSURE(filterNullMembers.Ptr()->GetTypeAnn()->GetKind() == ETypeAnnotationKind::List); + + return Build<TCoOrderedFlatMap>(ctx, filterNullMembers.Pos()) + .Input(filterNullMembers.Input()) + .Lambda() + .Args({"item"}) + .template Body<T>() + .template Input<TCoJust>() + .Input("item") + .Build() + .Members(filterNullMembers.Members()) + .Build() + .Build() + .Done(); + } + + TMaybeNode<TExprBase> TrimReadWorld(TExprBase node, TExprContext& ctx) const { + auto maybeRead = node.Cast<TCoLeft>().Input().Maybe<TYtReadTable>(); + if (!maybeRead) { + return node; + } + + auto read = maybeRead.Cast(); + TExprNode::TListType worlds(1, read.World().Ptr()); + for (auto section: read.Input()) { + for (auto path: section.Paths()) { + if (auto out = path.Table().Maybe<TYtOutput>()) { + worlds.push_back( + Build<TCoLeft>(ctx, node.Pos()) + .Input(out.Cast().Operation()) + .Done().Ptr() + ); + } + } + } + + return TExprBase(worlds.size() == 1 ? worlds.front() : ctx.NewCallable(node.Pos(), TCoSync::CallableName(), std::move(worlds))); + } + + TMaybeNode<TExprBase> CalcOverWindow(TExprBase node, TExprContext& ctx) const { + auto list = node.Cast<TCoInputBase>().Input(); + if (!IsYtProviderInput(list)) { + return node; + } + + TVector<TYtTableBaseInfo::TPtr> tableInfos = GetInputTableInfos(list); + if (AllOf(tableInfos, [](const TYtTableBaseInfo::TPtr& info) { return !info->Meta->IsDynamic; })) { + TExprNodeList calcs = ExtractCalcsOverWindow(node.Ptr(), ctx); + TSet<TStringBuf> rowNumberCols; + for (auto& calcNode : calcs) { + TCoCalcOverWindowTuple calc(calcNode); + if (calc.Keys().Size() != 0 || !calc.SessionSpec().Maybe<TCoVoid>() || !calc.SortSpec().Maybe<TCoVoid>()) { + continue; + } + bool needOptimize = false; + auto frames = calc.Frames().Ref().ChildrenList(); + for (const auto& win : frames) { + YQL_ENSURE(TCoWinOnBase::Match(win.Get())); + auto args = win->ChildrenList(); + needOptimize = args.size() > 1 && + // We rewrite RowNumber() into YtMap if it is the only window function for some frame + // (hence AllOf is used below) + // If the frame consist of RowNumber() and other window functions, we just calculate RowNumber() along with them + AllOf(args.begin() + 1, args.end(), + [](const auto& arg) { + return arg->Child(1)->IsCallable("RowNumber"); + }); + if (needOptimize) { + break; + } + } + + if (!needOptimize) { + continue; + } + + for (auto& win : frames) { + YQL_ENSURE(TCoWinOnBase::Match(win.Get())); + auto winOnArgs = win->ChildrenList(); + + TExprNodeList newWinOnArgs; + newWinOnArgs.push_back(std::move(winOnArgs[0])); + + for (size_t i = 1; i < winOnArgs.size(); ++i) { + auto labelAtom = winOnArgs[i]->Child(0); + YQL_ENSURE(labelAtom->IsAtom()); + auto trait = winOnArgs[i]->Child(1); + + if (trait->IsCallable("RowNumber")) { + rowNumberCols.insert(labelAtom->Content()); + } else { + newWinOnArgs.push_back(std::move(winOnArgs[i])); + } + } + + win = ctx.ChangeChildren(*win, std::move(newWinOnArgs)); + } + + calcNode = Build<TCoCalcOverWindowTuple>(ctx, calc.Pos()) + .Keys(calc.Keys()) + .SortSpec(calc.SortSpec()) + .Frames(ctx.NewList(calc.Frames().Pos(), std::move(frames))) + .SessionSpec(calc.SessionSpec()) + .SessionColumns(calc.SessionColumns()) + .Done().Ptr(); + } + + if (rowNumberCols) { + auto rowArg = ctx.NewArgument(node.Pos(), "row"); + auto body = rowArg; + + const bool useSysColumns = State_->Configuration->UseSystemColumns.Get().GetOrElse(DEFAULT_USE_SYS_COLUMNS); + const auto sysColumnNum = TString(YqlSysColumnPrefix).append("num"); + if (useSysColumns) { + for (auto& col : rowNumberCols) { + body = ctx.Builder(node.Pos()) + .Callable("AddMember") + .Add(0, body) + .Atom(1, col) + .Callable(2, "Member") + .Add(0, rowArg) + .Atom(1, sysColumnNum) + .Seal() + .Seal() + .Build(); + } + body = ctx.Builder(node.Pos()) + .Callable("ForceRemoveMember") + .Add(0, body) + .Atom(1, sysColumnNum) + .Seal() + .Build(); + + auto settings = Build<TCoNameValueTupleList>(ctx, list.Pos()) + .Add() + .Name() + .Value(ToString(EYtSettingType::SysColumns)) + .Build() + .Value(ToAtomList(TVector<TStringBuf>{"num"}, list.Pos(), ctx)) + .Build() + .Done(); + + if (auto right = list.Maybe<TCoRight>()) { + list = right.Cast().Input(); + } + + list = Build<TCoRight>(ctx, list.Pos()) + .Input(ConvertContentInputToRead(list, settings, ctx)) + .Done(); + } else { + for (auto& col : rowNumberCols) { + body = ctx.Builder(node.Pos()) + .Callable("AddMember") + .Add(0, body) + .Atom(1, col) + .Callable(2, "YtRowNumber") + .Callable(0, "DependsOn") + .Add(0, rowArg) + .Seal() + .Seal() + .Seal() + .Build(); + } + } + + auto input = ctx.Builder(node.Pos()) + .Callable("OrderedMap") + .Add(0, list.Ptr()) + .Add(1, ctx.NewLambda(node.Pos(), ctx.NewArguments(node.Pos(), {rowArg}), std::move(body))) + .Seal() + .Build(); + + YQL_CLOG(INFO, ProviderYt) << "Replaced " << rowNumberCols.size() << " RowNumber()s with " << (useSysColumns ? sysColumnNum : TString("YtRowNumber()")); + + return RebuildCalcOverWindowGroup(node.Pos(), input, calcs, ctx); + } + } + + return ExpandCalcOverWindow(node.Ptr(), ctx, *State_->Types); + } + + template<bool IsTop> + TMaybeNode<TExprBase> SortOverAlreadySorted(TExprBase node, TExprContext& ctx) const { + const auto sort = node.Cast<std::conditional_t<IsTop, TCoTopBase, TCoSort>>(); + + if (!IsConstExpSortDirections(sort.SortDirections())) { + return node; + } + + auto list = sort.Input(); + if (!IsYtProviderInput(list)) { + return node; + } + + TVector<TYtPathInfo::TPtr> pathInfos = GetInputPaths(list); + if (pathInfos.size() > 1) { + return node; + } + TYtPathInfo::TPtr pathInfo = pathInfos.front(); + if (pathInfo->Columns || pathInfo->Ranges || !pathInfo->Table->RowSpec || !pathInfo->Table->RowSpec->IsSorted()) { + return node; + } + const TStructExprType* itemType = nullptr; + if (auto type = GetSequenceItemType(node, false, ctx)) { + itemType = type->Cast<TStructExprType>(); + } else { + return {}; + } + + const bool useNativeDescSort = State_->Configuration->UseNativeDescSort.Get().GetOrElse(DEFAULT_USE_NATIVE_DESC_SORT); + TKeySelectorBuilder builder(node.Pos(), ctx, useNativeDescSort, itemType); + builder.ProcessKeySelector(sort.KeySelectorLambda().Ptr(), sort.SortDirections().Ptr()); + + if (builder.Members().size() < builder.Columns().size()) { + return node; + } + + auto rowSpec = pathInfo->Table->RowSpec; + for (size_t i = 0; i < builder.Members().size(); ++i) { + if (i >= rowSpec->SortMembers.size()) { + return node; + } + + if (rowSpec->SortMembers[i] != builder.Members()[i]) { + return node; + } + + if (!rowSpec->SortDirections[i] || !builder.SortDirections()[i]) { + return node; + } + } + + if constexpr (IsTop) + return Build<TCoTake>(ctx, sort.Pos()) + .Input(list) + .Count(sort.Count()) + .Done(); + else + return list; + } + + TMaybeNode<TExprBase> Demux(TExprBase node, TExprContext& ctx) const { + auto nth = node.Cast<TCoNth>(); + auto input = nth.Tuple().Maybe<TCoDemux>().Input().Maybe<TCoRight>().Input(); + if (!input) { + return node; + } + + if (auto op = input.Maybe<TYtOutputOpBase>()) { + return Build<TYtOutput>(ctx, node.Pos()) + .Operation(op.Cast()) + .OutIndex(nth.Index()) + .Done(); + } + if (auto maybeRead = input.Maybe<TYtReadTable>()) { + auto read = maybeRead.Cast(); + auto ndx = FromString<size_t>(nth.Index().Value()); + YQL_ENSURE(ndx < read.Input().Size()); + return Build<TCoRight>(ctx, node.Pos()) + .Input<TYtReadTable>() + .InitFrom(read) + .Input() + .Add(read.Input().Item(ndx)) + .Build() + .Build() + .Done(); + } + return node; + + } + + TMaybeNode<TExprBase> VarianItemOverInput(TExprBase node, TExprContext& ctx) const { + auto map = node.Cast<TYtMap>(); + if (map.Input().Size() == 1) { + return node; + } + + // All sections should have equal settings + const TExprNode* sectionSettings = nullptr; + for (auto section: map.Input()) { + if (nullptr == sectionSettings) { + sectionSettings = section.Settings().Raw(); + } else if (sectionSettings != section.Settings().Raw()) { + return node; + } + } + + auto mapper = map.Mapper(); + + TParentsMap parentsMap; + GatherParents(mapper.Body().Ref(), parentsMap); + + auto maybeLambda = GetFlatMapOverInputStream(mapper, parentsMap).Lambda(); + if (!maybeLambda) { + return node; + } + + TCoLambda lambda = maybeLambda.Cast(); + auto arg = lambda.Args().Arg(0); + + // Check arg is used only in VariantItem + auto it = parentsMap.find(arg.Raw()); + if (it == parentsMap.cend() || it->second.size() != 1 || !TCoVariantItem::Match(*it->second.begin())) { + return node; + } + + // Substitute VariantItem(arg) by arg + TNodeOnNodeOwnedMap nodesToOptimize; + nodesToOptimize.emplace(*it->second.begin(), arg.Ptr()); + TExprNode::TPtr newMapper; + auto status = RemapExpr(mapper.Ptr(), newMapper, nodesToOptimize, ctx, TOptimizeExprSettings{nullptr}); + + if (status.Level == IGraphTransformer::TStatus::Error) { + return {}; + } + + // Join all paths to single section + auto newPaths = Build<TYtPathList>(ctx, map.Input().Pos()); + for (auto section: map.Input()) { + newPaths.Add(section.Paths().Ref().ChildrenList()); + } + + return Build<TYtMap>(ctx, map.Pos()) + .InitFrom(map) + .Input() + .Add() + .Paths(newPaths.Done()) + .Settings(map.Input().Item(0).Settings()) + .Build() + .Build() + .Mapper(TCoLambda(newMapper)) + .Done(); + } + + TMaybeNode<TExprBase> VisitOverInputWithEqualLambdas(TExprBase node, TExprContext& ctx) const { + auto op = node.Cast<TYtWithUserJobsOpBase>(); + if (op.Input().Size() == 1) { + return node; + } + + if (op.Maybe<TYtMapReduce>().Mapper().Maybe<TCoVoid>()) { + return node; + } + + size_t lambdaIdx = op.Maybe<TYtMapReduce>() + ? TYtMapReduce::idx_Mapper + : op.Maybe<TYtReduce>() + ? TYtReduce::idx_Reducer + : TYtMap::idx_Mapper; + + auto opLambda = TCoLambda(op.Ref().ChildPtr(lambdaIdx)); + + TParentsMap parentsMap; + GatherParents(opLambda.Body().Ref(), parentsMap); + + auto maybeLambda = GetFlatMapOverInputStream(opLambda, parentsMap).Lambda(); + if (!maybeLambda) { + return node; + } + + TCoLambda lambda = maybeLambda.Cast(); + auto arg = lambda.Args().Arg(0); + + // Check arg is used only in Visit + auto it = parentsMap.find(arg.Raw()); + if (it == parentsMap.cend() || it->second.size() != 1 || !TCoVisit::Match(*it->second.begin())) { + return node; + } + + using TGroupKey = std::pair<const TExprNode*, const TExprNode*>; // lambda, section settings + TMap<TGroupKey, TVector<size_t>> groupedInputs; // key -> {section id} + TVector<TExprNode::TPtr> visitLambdas; + visitLambdas.resize(op.Input().Size()); + + const TExprNode* visit = *it->second.begin(); + for (ui32 index = 1; index < visit->ChildrenSize(); ++index) { + if (visit->Child(index)->IsAtom()) { + size_t inputNum = FromString<size_t>(visit->Child(index)->Content()); + YQL_ENSURE(inputNum < op.Input().Size()); + + ++index; + + groupedInputs[std::make_pair(visit->Child(index), op.Input().Item(inputNum).Settings().Raw())].push_back(inputNum); + visitLambdas[inputNum] = visit->ChildPtr(index); + } + } + + if (groupedInputs.empty() || AllOf(groupedInputs, [](const decltype(groupedInputs)::value_type& grp) { return grp.second.size() <= 1; })) { + return node; + } + + TVector<TVector<size_t>> groups; + groups.reserve(groupedInputs.size()); + for (auto& grp: groupedInputs) { + groups.push_back(std::move(grp.second)); + } + ::Sort(groups.begin(), groups.end(), + [] (const decltype(groups)::value_type& v1, const decltype(groups)::value_type& v2) { + return v1.front() < v2.front(); + } + ); + + // Rebuild input + TVector<TYtSection> newSections; + for (auto& grp: groups) { + TVector<TYtPath> paths; + for (size_t sectionNum: grp) { + auto oldSection = op.Input().Item(sectionNum); + paths.insert(paths.end(), oldSection.Paths().begin(), oldSection.Paths().end()); + } + auto firstOldSection = op.Input().Item(grp.front()); + newSections.push_back(Build<TYtSection>(ctx, firstOldSection.Pos()) + .InitFrom(firstOldSection) + .Paths() + .Add(paths) + .Build() + .Done()); + } + + // Rebuild lambda + TExprNode::TPtr newVisit = ctx.Builder(visit->Pos()) + .Callable(TCoVisit::CallableName()) + .Add(0, visit->ChildPtr(0)) + .Do([&](TExprNodeBuilder& builder) -> TExprNodeBuilder& { + for (size_t i = 0; i < groups.size(); ++i) { + builder.Atom(i * 2 + 1, ToString(i)); + builder.Add(i * 2 + 2, visitLambdas.at(groups[i].front())); + } + if (visit->ChildrenSize() % 2 == 0) { // has default value + builder.Add(groups.size() * 2 + 1, visit->TailPtr()); + } + return builder; + }) + .Seal() + .Build(); + TNodeOnNodeOwnedMap nodesToOptimize; + nodesToOptimize.emplace(visit, newVisit); + TExprNode::TPtr newOpLambda; + auto status = RemapExpr(opLambda.Ptr(), newOpLambda, nodesToOptimize, ctx, TOptimizeExprSettings{nullptr}); + if (status.Level == IGraphTransformer::TStatus::Error) { + return {}; + } + + auto res = ctx.ChangeChild(op.Ref(), TYtWithUserJobsOpBase::idx_Input, + Build<TYtSectionList>(ctx, op.Input().Pos()) + .Add(newSections) + .Done().Ptr()); + + res = ctx.ChangeChild(*res, lambdaIdx, std::move(newOpLambda)); + return TExprBase(res); + } + + TMaybeNode<TExprBase> UnorderedOverInput(TExprBase node, TExprContext& ctx) const { + auto op = node.Cast<TYtWithUserJobsOpBase>(); + + if (op.Maybe<TYtMapReduce>().Mapper().Maybe<TCoVoid>()) { + return node; + } + + size_t lambdaIdx = op.Maybe<TYtMapReduce>() + ? TYtMapReduce::idx_Mapper + : op.Maybe<TYtReduce>() + ? TYtReduce::idx_Reducer + : TYtMap::idx_Mapper; + + auto opLambda = TCoLambda(op.Ref().ChildPtr(lambdaIdx)); + auto arg = opLambda.Args().Arg(0); + + TParentsMap parentsMap; + GatherParents(opLambda.Body().Ref(), parentsMap); + + auto it = parentsMap.find(arg.Raw()); + if (it == parentsMap.cend()) { + return node; + } + + // Substitute Unordered(arg) by arg + TNodeOnNodeOwnedMap nodesToOptimize; + for (auto n: it->second) { + if (TCoUnordered::Match(n)) { + nodesToOptimize.emplace(n, arg.Ptr()); + } else if (!TCoDependsOn::Match(n)) { + return node; + } + } + if (nodesToOptimize.empty()) { + return node; + } + + TExprNode::TPtr newOpLambda; + auto status = RemapExpr(opLambda.Ptr(), newOpLambda, nodesToOptimize, ctx, TOptimizeExprSettings{nullptr}); + if (status.Level == IGraphTransformer::TStatus::Error) { + return {}; + } + + auto res = ctx.ChangeChild(op.Ref(), lambdaIdx, std::move(newOpLambda)); + + TVector<TYtSection> updatedSections; + for (auto section: op.Input()) { + updatedSections.push_back(MakeUnorderedSection(section, ctx)); + } + res = ctx.ChangeChild(*res, TYtWithUserJobsOpBase::idx_Input, + Build<TYtSectionList>(ctx, op.Input().Pos()) + .Add(updatedSections) + .Done().Ptr()); + + if (NYql::HasSetting(op.Settings().Ref(), EYtSettingType::Ordered)) { + res = ctx.ChangeChild(*res, TYtWithUserJobsOpBase::idx_Settings, NYql::RemoveSettings(op.Settings().Ref(), EYtSettingType::Ordered, ctx)); + } + + return TExprBase(res); + } + + TMaybeNode<TExprBase> FuseFlatmaps(TExprBase node, TExprContext& ctx, const TGetParents& getParents) const { + auto outerFlatmap = node.Cast<TCoFlatMapBase>(); + if (!outerFlatmap.Input().Maybe<TCoFlatMapBase>()) { + return node; + } + + auto innerFlatmap = outerFlatmap.Input().Cast<TCoFlatMapBase>(); + if (!IsYtProviderInput(innerFlatmap.Input())) { + return node; + } + + if (FindNode(innerFlatmap.Lambda().Body().Ptr(), + [](const TExprNode::TPtr& node) { return !TYtOutput::Match(node.Get()); }, + [](const TExprNode::TPtr& node) { return TCoNonDeterministicBase::Match(node.Get()); })) { + + // If inner FlatMap uses non-deterministic functions then disallow to split it in case of multiusage + const TParentsMap* parentsMap = getParents(); + auto parentsIt = parentsMap->find(innerFlatmap.Raw()); + YQL_ENSURE(parentsIt != parentsMap->cend()); + if (parentsIt->second.size() > 1) { + return node; + } + } + + const auto flatMapName = outerFlatmap.Maybe<TCoOrderedFlatMap>() && innerFlatmap.Maybe<TCoOrderedFlatMap>() + ? TCoOrderedFlatMap::CallableName() + : TCoFlatMap::CallableName(); + + auto [placeHolder, lambdaWithPlaceholder] = ReplaceDependsOn(outerFlatmap.Lambda().Ptr(), ctx, State_->Types); + if (!placeHolder) { + return {}; + } + + return Build<TCoFlatMapBase>(ctx, outerFlatmap.Pos()) + .CallableName(flatMapName) + .Input(innerFlatmap.Input()) + .Lambda() + .Args({"item"}) + .Body<TCoFlatMapBase>() + .CallableName(flatMapName) + .Input<TExprApplier>() + .Apply(innerFlatmap.Lambda()) + .With(0, "item") + .Build() + .Lambda() + .Args({"outerItem"}) + .Body<TExprApplier>() + .Apply(TCoLambda(lambdaWithPlaceholder)) + .With(0, "outerItem") + .With(TExprBase(placeHolder), "item") + .Build() + .Build() + .Build() + .Build() + .Done(); + } + + TMaybeNode<TExprBase> Zip(TExprBase node, TExprContext& ctx) const { + auto zip = node.Cast<TCoZip>(); + if (zip.ArgCount() != 2) { + return node; + } + + auto lhsList = zip.Arg(0); + auto rhsList = zip.Arg(1); + if (!IsYtProviderInput(lhsList) || !IsYtProviderInput(rhsList)) { + return node; + } + + return Build<TCoMap>(ctx, zip.Pos()) + .Input<TCoJoin>() + .LeftInput<TCoEnumerate>() + .Input(lhsList) + .Build() + .RightInput<TCoEnumerate>() + .Input(rhsList) + .Build() + .LeftLambda() + .Args({"p"}) + .Body<TCoNth>() + .Tuple("p") + .Index() + .Value("0") + .Build() + .Build() + .Build() + .RightLambda() + .Args({"p"}) + .Body<TCoNth>() + .Tuple("p") + .Index() + .Value("0") + .Build() + .Build() + .Build() + .JoinKind() + .Value("Inner") + .Build() + .Build() + .Lambda() + .Args({"m"}) + .Body<TExprList>() + .Add<TCoNth>() + .Tuple<TCoNth>() + .Tuple("m") + .Index() + .Value("0") + .Build() + .Build() + .Index() + .Value("1") + .Build() + .Build() + .Add<TCoNth>() + .Tuple<TCoNth>() + .Tuple("m") + .Index() + .Value("1") + .Build() + .Build() + .Index() + .Value("1") + .Build() + .Build() + .Build() + .Build() + .Done(); + } + + TMaybeNode<TExprBase> ZipAll(TExprBase node, TExprContext& ctx) const { + auto zip = node.Cast<TCoZipAll>(); + if (zip.ArgCount() != 2) { + return node; + } + + auto lhsList = zip.Arg(0); + auto rhsList = zip.Arg(1); + if (!IsYtProviderInput(lhsList) || !IsYtProviderInput(rhsList)) { + return node; + } + + return Build<TCoMap>(ctx, zip.Pos()) + .Input<TCoJoin>() + .LeftInput<TCoEnumerate>() + .Input(lhsList) + .Build() + .RightInput<TCoEnumerate>() + .Input(rhsList) + .Build() + .LeftLambda() + .Args({"p"}) + .Body<TCoNth>() + .Tuple("p") + .Index() + .Value("0") + .Build() + .Build() + .Build() + .RightLambda() + .Args({"p"}) + .Body<TCoNth>() + .Tuple("p") + .Index() + .Value("0") + .Build() + .Build() + .Build() + .JoinKind() + .Value("Full") + .Build() + .Build() + .Lambda() + .Args({"m"}) + .Body<TExprList>() + .Add<TCoMap>() + .Input<TCoNth>() + .Tuple("m") + .Index() + .Value("0") + .Build() + .Build() + .Lambda() + .Args({"p"}) + .Body<TCoNth>() + .Tuple("p") + .Index() + .Value("1") + .Build() + .Build() + .Build() + .Build() + .Add<TCoMap>() + .Input<TCoNth>() + .Tuple("m") + .Index() + .Value("1") + .Build() + .Build() + .Lambda() + .Args({"p"}) + .Body<TCoNth>() + .Tuple("p") + .Index() + .Value("1") + .Build() + .Build() + .Build() + .Build() + .Build() + .Build() + .Done(); + } + + TMaybeNode<TExprBase> ExtractMembers(TExprBase node, TExprContext& ctx) const { + auto extract = node.Cast<TCoExtractMembers>(); + auto input = extract.Input(); + if (!IsYtProviderInput(input)) { + return node; + } + + TExprNode::TPtr world; + TVector<TYtSection> sections; + if (auto out = input.Maybe<TYtOutput>()) { + world = ctx.NewWorld(input.Pos()); + sections.push_back(Build<TYtSection>(ctx, input.Pos()) + .Paths() + .Add() + .Table(out.Cast()) + .Columns(extract.Members()) + .Ranges<TCoVoid>().Build() + .Stat<TCoVoid>().Build() + .Build() + .Build() + .Settings().Build() + .Done()); + } + else { + auto read = input.Maybe<TCoRight>().Input().Maybe<TYtReadTable>(); + YQL_ENSURE(read, "Unknown operation input"); + world = read.Cast().World().Ptr(); + + for (auto section: read.Cast().Input()) { + sections.push_back(UpdateInputFields(section, extract.Members(), ctx)); + } + } + + return Build<TCoRight>(ctx, extract.Pos()) + .Input<TYtReadTable>() + .World(world) + .DataSource(GetDataSource(input, ctx)) + .Input() + .Add(sections) + .Build() + .Build() + .Done(); + } + + TMaybeNode<TExprBase> ExtractMembersOverContent(TExprBase node, TExprContext& ctx) const { + auto extractMembers = node.Cast<TCoExtractMembers>(); + + TExprBase tableContent = extractMembers.Input(); + if (!tableContent.Maybe<TYtTableContent>()) { + return node; + } + + return Build<TYtTableContent>(ctx, tableContent.Pos()) + .InitFrom(tableContent.Cast<TYtTableContent>()) + .Input(ConvertContentInputToRead(tableContent.Cast<TYtTableContent>().Input(), {}, ctx, extractMembers.Members())) + .Done(); + } + + TMaybeNode<TExprBase> TakeOrSkip(TExprBase node, TExprContext& ctx) const { + auto countBase = node.Cast<TCoCountBase>(); + auto input = countBase.Input(); + if (!input.Maybe<TYtTableContent>()) { + return node; + } + + input = input.Cast<TYtTableContent>().Input(); + + TYtDSource dataSource = GetDataSource(input, ctx); + TString cluster = TString{dataSource.Cluster().Value()}; + TSyncMap syncList; + if (!IsYtCompleteIsolatedLambda(countBase.Count().Ref(), syncList, cluster, false)) { + return node; + } + + EYtSettingType settingType = node.Maybe<TCoSkip>() ? EYtSettingType::Skip : EYtSettingType::Take; + + auto settings = Build<TCoNameValueTupleList>(ctx, countBase.Pos()) + .Add() + .Name() + .Value(ToString(settingType)) + .Build() + .Value(countBase.Count()) + .Build() + .Done(); + + return ctx.ChangeChild(countBase.Input().Ref(), TYtTableContent::idx_Input, ConvertContentInputToRead(input, settings, ctx).Ptr()); + } + + TMaybeNode<TExprBase> BypassCopy(TExprBase node, TExprContext& ctx, const TGetParents& getParents) const { + auto srcOut = node.Cast<TYtOutput>(); + auto maybeCopy = srcOut.Operation().Maybe<TYtCopy>(); + if (!maybeCopy) { + return node; + } + auto copy = maybeCopy.Cast(); + if (copy.World().Ref().Type() != TExprNode::World) { + return node; + } + + if (copy.Ref().HasResult()) { + return node; + } + + TYtPath path = copy.Input().Item(0).Paths().Item(0); + if (!path.Table().Maybe<TYtOutput>()) { + return node; + } + + const auto parentsMap = getParents(); + auto parentsIt = parentsMap->find(copy.Raw()); + for (auto p: parentsIt->second) { + if (TCoLeft::Match(p)) { + return node; + } + } + + + auto res = path.Table().Cast<TYtOutput>(); + if (IsUnorderedOutput(srcOut)) { + res = Build<TYtOutput>(ctx, res.Pos()) + .InitFrom(res) + .Mode(srcOut.Mode()) + .Done(); + } + return res; + } + + TMaybeNode<TExprBase> PushdownReadColumns(TExprBase node, TExprContext& ctx, const TGetParents& getParents) const { + auto maybeRead = node.Cast<TCoRight>().Input().Maybe<TYtReadTable>(); + if (!maybeRead) { + return node; + } + + auto read = maybeRead.Cast(); + + bool hasNewSection = false; + TVector<TYtSection> sections; + for (auto section: read.Input()) { + sections.push_back(PushdownSectionColumns(section, ctx, getParents)); + if (section.Raw() != sections.back().Raw()) { + hasNewSection = true; + } + } + + if (!hasNewSection) { + return node; + } + + return Build<TCoRight>(ctx, node.Pos()) + .Input<TYtReadTable>() + .InitFrom(read) + .Input() + .Add(sections) + .Build() + .Build() + .Done(); + } + + TMaybeNode<TExprBase> PushdownOpColumns(TExprBase node, TExprContext& ctx, const TGetParents& getParents) const { + auto op = node.Cast<TYtTransientOpBase>(); + + bool hasNewSection = false; + TVector<TYtSection> sections; + for (auto section: op.Input()) { + sections.push_back(PushdownSectionColumns(section, ctx, getParents)); + if (section.Raw() != sections.back().Raw()) { + hasNewSection = true; + } + } + + if (!hasNewSection) { + return node; + } + + return TExprBase(ctx.ChangeChild(node.Ref(), TYtTransientOpBase::idx_Input, Build<TYtSectionList>(ctx, op.Input().Pos()).Add(sections).Done().Ptr())); + } + + struct TExtendOverSameMapGroupLess { + bool operator() (const std::pair<bool, TCoLambda>& left, const std::pair<bool, TCoLambda>& right) const { + return std::make_pair(left.first, left.second.Ref().UniqueId()) < std::make_pair(right.first, right.second.Ref().UniqueId()); + } + }; + + TMaybeNode<TExprBase> ExtendOverSameMap(TExprBase node, TExprContext& ctx) const { + auto extend = node.Cast<TCoExtendBase>(); + + // Don't apply to OrderedExtend because input is reordered + if (extend.Maybe<TCoOrderedExtend>()) { + return node; + } + + TVector<TExprBase> retChildren; + TVector<TCoFlatMapBase> flatMaps; + bool keepOrder = extend.Maybe<TCoMerge>().IsValid(); + for (auto child : extend) { + if (auto maybeFlatMap = child.Maybe<TCoFlatMapBase>()) { + auto flatMap = maybeFlatMap.Cast(); + keepOrder = keepOrder && flatMap.Maybe<TCoOrderedFlatMap>(); + auto input = flatMap.Input(); + if (!IsYtProviderInput(input)) { + return node; + } + if (input.Ref().UseCount() > 2) { // Additional reference is owned by 'input' local var + retChildren.push_back(child); + } else { + flatMaps.push_back(flatMap); + } + } + else { + return node; + } + } + + // group by YAMR input and lambda nodes + std::map<std::pair<bool, TCoLambda>, TVector<TCoFlatMapBase>, TExtendOverSameMapGroupLess> grouped; + + for (auto flatmap : flatMaps) { + // All YtRead inputs either YAMR or not, so check only first one. YtOutput cannot be YAMR + const bool yamr = flatmap.Input().Maybe<TCoRight>().Input().Maybe<TYtReadTable>().Input() + .Item(0).Paths().Item(0).Table().Maybe<TYtTable>().RowSpec().Maybe<TCoVoid>().IsValid(); + + grouped[std::make_pair(yamr, flatmap.Lambda())].emplace_back(flatmap); + } + + size_t singleWithoutContextDependent = 0; + TVector<std::pair<bool, TCoLambda>> contextDependentLambdas; + for (auto& x : grouped) { + if (IsTablePropsDependent(x.first.second.Ref())) { + contextDependentLambdas.push_back(x.first); + } else if (x.second.size() == 1) { + ++singleWithoutContextDependent; + } + } + + if (grouped.size() == flatMaps.size() || + contextDependentLambdas.size() + singleWithoutContextDependent == grouped.size()) + { + return node; + } + + for (auto& x : contextDependentLambdas) { + retChildren.insert(retChildren.end(), grouped[x].begin(), grouped[x].end()); + grouped.erase(x); + } + + for (auto& x: grouped) { + TExprNode::TListType subChildren; + + for (auto flatMap: x.second) { + subChildren.push_back(flatMap.Input().Ptr()); + } + + auto flatMapInput = ctx.NewCallable(node.Pos(), extend.Ref().Content(), std::move(subChildren)); + + retChildren.push_back(TExprBase(ctx.Builder(node.Pos()) + .Callable(keepOrder ? TCoOrderedFlatMap::CallableName() : TCoFlatMap::CallableName()) + .Add(0, flatMapInput) + .Add(1, x.first.second.Ptr()) + .Seal() + .Build())); + } + + if (extend.Maybe<TCoMerge>()) { + return Build<TCoMerge>(ctx, node.Pos()).Add(retChildren).Done(); + } + + return Build<TCoExtend>(ctx, node.Pos()).Add(retChildren).Done(); + } + + static bool IsExtendWithFlatMaps(TExprBase node, bool requireChildFlatMap) { + if (!node.Maybe<TCoExtendBase>()) { + return false; + } + + auto extend = node.Cast<TCoExtendBase>(); + auto type = extend.Ref().GetTypeAnn(); + if (type->GetKind() != ETypeAnnotationKind::List || + type->Cast<TListExprType>()->GetItemType()->GetKind() != ETypeAnnotationKind::Struct) { + return false; + } + + bool hasFlatMap = false; + for (auto child : extend) { + if (IsYtProviderInput(child)) { + continue; + } + + if (auto mayFlatMap = child.Maybe<TCoFlatMapBase>()) { + if (IsYtProviderInput(mayFlatMap.Cast().Input())) { + hasFlatMap = true; + continue; + } + } + return false; + } + + return !requireChildFlatMap || hasFlatMap; + } + + TMaybeNode<TExprBase> FlatMapOverExtend(TExprBase node, TExprContext& ctx, const TGetParents& getParents) const { + auto flatMap = node.Cast<TCoFlatMapBase>(); + + auto input = flatMap.Input(); + TVector<TCoInputBase> intermediates; + while (input.Maybe<TCoCountBase>() || input.Maybe<TCoFilterNullMembersBase>()) { + intermediates.push_back(input.Cast<TCoInputBase>()); + input = input.Cast<TCoInputBase>().Input(); + } + if (!IsExtendWithFlatMaps(input, true)) { + return node; + } + + const TParentsMap* parentsMap = getParents(); + auto parentsIt = parentsMap->find(input.Raw()); + YQL_ENSURE(parentsIt != parentsMap->cend()); + if (parentsIt->second.size() > 1) { + return node; + } + + const bool ordered = flatMap.Maybe<TCoOrderedFlatMap>() && !input.Maybe<TCoExtend>(); + TExprNode::TListType extendChildren; + for (auto child: input.Ref().Children()) { + extendChildren.push_back(ctx.Builder(child->Pos()) + .Callable(ordered ? TCoOrderedFlatMap::CallableName() : TCoFlatMap::CallableName()) + .Add(0, child) + .Add(1, flatMap.Lambda().Ptr()) + .Seal() + .Build()); + } + TStringBuf extendName = input.Maybe<TCoMerge>() + ? TCoMerge::CallableName() + : (ordered ? TCoOrderedExtend::CallableName() : TCoExtend::CallableName()); + + auto res = ctx.NewCallable(node.Pos(), extendName, std::move(extendChildren)); + for (auto it = intermediates.rbegin(); it != intermediates.rend(); ++it) { + res = ctx.ChangeChild(it->Ref(), TCoInputBase::idx_Input, std::move(res)); + } + + return TExprBase(res); + } + + TMaybeNode<TExprBase> TakeOverExtend(TExprBase node, TExprContext& ctx) const { + auto take = node.Cast<TCoTake>(); + + if (!IsExtendWithFlatMaps(take.Input(), true)) { + return node; + } + + TExprNode::TListType extendChildren; + for (auto child : take.Input().Ref().Children()) { + extendChildren.push_back(ctx.Builder(child->Pos()) + .Callable(TCoTake::CallableName()) + .Add(0, child) + .Add(1, take.Count().Ptr()) + .Seal() + .Build()); + } + + return Build<TCoLimit>(ctx, node.Pos()) + .Input(ctx.NewCallable(node.Pos(), take.Input().Ref().Content(), std::move(extendChildren))) + .Count(take.Count()) + .Done(); + } + + TMaybeNode<TExprBase> DirectRowInFlatMap(TExprBase node, TExprContext& ctx) const { + if (State_->Configuration->UseSystemColumns.Get().GetOrElse(DEFAULT_USE_SYS_COLUMNS)) { + return node; + } + + auto flatMap = node.Cast<TCoFlatMapBase>(); + + const auto& outItemType = GetSeqItemType(*flatMap.Ref().GetTypeAnn()); + if (outItemType.GetKind() != ETypeAnnotationKind::Struct) { + return node; + } + + auto input = flatMap.Input(); + TVector<TCoInputBase> intermediates; + while (input.Maybe<TCoCountBase>() || input.Maybe<TCoFilterNullMembersBase>()) { + intermediates.push_back(input.Cast<TCoInputBase>()); + input = input.Cast<TCoInputBase>().Input(); + } + if (!IsExtendWithFlatMaps(input, false)) { + return node; + } + + TNodeSet nodesToOptimize; + TProcessedNodesSet processedNodes; + auto originalArg = flatMap.Lambda().Args().Arg(0).Raw(); + VisitExpr(flatMap.Lambda().Ptr(), [&nodesToOptimize, &processedNodes, originalArg](const TExprNode::TPtr& node) { + if (TCoTablePath::Match(node.Get()) && (node->ChildrenSize() == 0 || node->Child(0)->Child(0) == originalArg)) { + nodesToOptimize.insert(node.Get()); + } + else if (TCoTableRecord::Match(node.Get()) && (node->ChildrenSize() == 0 || node->Child(0)->Child(0) == originalArg)) { + nodesToOptimize.insert(node.Get()); + } + else if (TYtOutput::Match(node.Get())) { + // Stop traversing dependent operations + processedNodes.insert(node->UniqueId()); + return false; + } + return true; + }); + + if (nodesToOptimize.empty()) { + return node; + } + + // move TablePath/TableRecord if any + const auto& extend = input.Ref(); + const bool ordered = flatMap.Maybe<TCoOrderedFlatMap>() && !input.Maybe<TCoExtend>(); + TExprNode::TListType updatedExtendInputs; + for (auto& x : extend.Children()) { + auto updatedInput = ctx.Builder(flatMap.Pos()) + .Callable(ordered ? "OrderedMap" : "Map") + .Add(0, x) + .Lambda(1) + .Param("row") + .Callable("AddMember") + .Callable(0, "AddMember") + .Arg(0, "row") + .Atom(1, "_yql_table_path") + .Callable(2, "TablePath") + .Callable(0, "DependsOn") + .Arg(0, "row") + .Seal() + .Seal() + .Seal() + .Atom(1, "_yql_table_record") + .Callable(2, "TableRecord") + .Callable(0, "DependsOn") + .Arg(0, "row") + .Seal() + .Seal() + .Seal() + .Seal() + .Seal() + .Build(); + + updatedExtendInputs.push_back(updatedInput); + } + + TStringBuf extendName = input.Maybe<TCoMerge>() + ? TCoMerge::CallableName() + : (ordered ? TCoOrderedExtend::CallableName() : TCoExtend::CallableName()); + + auto newInput = ctx.NewCallable(extend.Pos(), extendName, std::move(updatedExtendInputs)); + for (auto it = intermediates.rbegin(); it != intermediates.rend(); ++it) { + newInput = ctx.ChangeChild(it->Ref(), TCoInputBase::idx_Input, std::move(newInput)); + } + auto arg = ctx.NewArgument(flatMap.Pos(), "row"); + TExprNode::TPtr newBody = flatMap.Lambda().Body().Ptr(); + + TOptimizeExprSettings settings(State_->Types); + settings.ProcessedNodes = &processedNodes; // Prevent optimizer to go deeper than current operation + auto status = OptimizeExpr(newBody, newBody, [&](const TExprNode::TPtr& input, TExprContext& ctx)->TExprNode::TPtr { + if (nodesToOptimize.find(input.Get()) != nodesToOptimize.end()) { + if (input->IsCallable("TablePath")) { + return ctx.NewCallable(input->Pos(), "Member", { arg, ctx.NewAtom(input->Pos(), "_yql_table_path") }); + } + + if (input->IsCallable("TableRecord")) { + return ctx.NewCallable(input->Pos(), "Member", { arg, ctx.NewAtom(input->Pos(), "_yql_table_record") }); + } + } + + return input; + }, ctx, settings); + + if (status.Level == IGraphTransformer::TStatus::Error) { + return {}; + } + + newBody = ctx.ReplaceNode(std::move(newBody), flatMap.Lambda().Args().Arg(0).Ref(), arg); + auto newLambda = ctx.NewLambda(flatMap.Pos(), ctx.NewArguments(flatMap.Pos(), { arg }), std::move(newBody)); + auto res = ctx.NewCallable(flatMap.Pos(), ordered ? "OrderedFlatMap" : "FlatMap", { std::move(newInput), std::move(newLambda) }); + + res = ctx.Builder(flatMap.Pos()) + .Callable(ordered ? "OrderedMap" : "Map") + .Add(0, res) + .Lambda(1) + .Param("row") + .Callable("ForceRemoveMember") + .Callable(0, "ForceRemoveMember") + .Arg(0, "row") + .Atom(1, "_yql_table_path") + .Seal() + .Atom(1, "_yql_table_record") + .Seal() + .Seal() + .Seal() + .Build(); + + return TExprBase(res); + } + + TMaybeNode<TExprBase> OutputInFlatMap(TExprBase node, TExprContext& ctx) const { + auto op = node.Cast<TYtWithUserJobsOpBase>(); + if (op.Input().Size() != 1) { + return node; + } + + TMaybeNode<TCoLambda> maybeOpLambda; + size_t opLambdaIdx = 0; + if (auto mapReduce = op.Maybe<TYtMapReduce>()) { + maybeOpLambda = mapReduce.Mapper().Maybe<TCoLambda>(); + opLambdaIdx = TYtMapReduce::idx_Mapper; + } else if (auto reduce = op.Maybe<TYtReduce>()) { + maybeOpLambda = reduce.Reducer(); + opLambdaIdx = TYtReduce::idx_Reducer; + } else { + maybeOpLambda = op.Maybe<TYtMap>().Mapper(); + opLambdaIdx = TYtMap::idx_Mapper; + } + if (!maybeOpLambda) { + return node; + } + auto opLambda = maybeOpLambda.Cast(); + + auto maybeFlatMap = GetFlatMapOverInputStream(opLambda); + if (!maybeFlatMap) { + return node; + } + + auto flatMap = maybeFlatMap.Cast(); + TCoLambda lambda = flatMap.Lambda(); + + auto finalNode = lambda.Body(); + const bool isListIf = finalNode.Maybe<TCoListIf>().IsValid(); + const bool isAsList1 = finalNode.Maybe<TCoAsList>() && finalNode.Cast<TCoAsList>().ArgCount() == 1; + const bool removeLastOp = finalNode.Maybe<TCoToList>() || + finalNode.Maybe<TCoForwardList>() || finalNode.Maybe<TCoIterator>(); + if (!isListIf && !isAsList1 && !removeLastOp) { + return node; + } + + TNodeOnNodeOwnedMap nodesToOptimize; + if (isAsList1) { + nodesToOptimize[flatMap.Raw()] = Build<TCoFlatMapBase>(ctx, flatMap.Pos()) + .CallableName(flatMap.CallableName()) + .Input(flatMap.Input()) + .Lambda() + .Args({"stream"}) + .Body<TExprApplier>() + .Apply(TExprBase(ctx.NewCallable(lambda.Pos(), "Just", {finalNode.Ref().HeadPtr()}))) + .With(lambda.Args().Arg(0), "stream") + .Build() + .Build() + .Done().Ptr(); + } + + if (isListIf) { + nodesToOptimize[flatMap.Raw()] = Build<TCoFlatMapBase>(ctx, flatMap.Pos()) + .CallableName(flatMap.CallableName()) + .Input(flatMap.Input()) + .Lambda() + .Args({"stream"}) + .Body<TExprApplier>() + .Apply(TExprBase(ctx.NewCallable(lambda.Pos(), "OptionalIf", {finalNode.Ref().HeadPtr(), finalNode.Ref().TailPtr()}))) + .With(lambda.Args().Arg(0), "stream") + .Build() + .Build() + .Done().Ptr(); + } + + if (removeLastOp) { + nodesToOptimize[flatMap.Raw()] = Build<TCoFlatMapBase>(ctx, flatMap.Pos()) + .CallableName(flatMap.CallableName()) + .Input(flatMap.Input()) + .Lambda() + .Args({"stream"}) + .Body<TExprApplier>() + .Apply(TExprBase(finalNode.Ref().HeadPtr())) + .With(lambda.Args().Arg(0), "stream") + .Build() + .Build() + .Done().Ptr(); + } + + TProcessedNodesSet processedNodes; + VisitExpr(opLambda.Ptr(), [&processedNodes](const TExprNode::TPtr& node) { + if (TYtOutput::Match(node.Get())) { + // Stop traversing dependent operations + processedNodes.insert(node->UniqueId()); + return false; + } + return true; + }); + + TOptimizeExprSettings settings(State_->Types); + settings.ProcessedNodes = &processedNodes; // Prevent optimizer to go deeper than current operation + TExprNode::TPtr newOpLambda; + auto status = RemapExpr(opLambda.Ptr(), newOpLambda, nodesToOptimize, ctx, settings); + if (status.Level == IGraphTransformer::TStatus::Error) { + return {}; + } + + return TExprBase(ctx.ChangeChild(op.Ref(), opLambdaIdx, ctx.DeepCopyLambda(*newOpLambda))); + } + + static void CollectEquiJoinLinks(TCoEquiJoinTuple joinTree, TVector<std::pair<TString, TString>>& links, + const std::function<bool(TExprBase linkSettings)>& collectPred) + { + if (!joinTree.LeftScope().Maybe<TCoAtom>()) { + CollectEquiJoinLinks(joinTree.LeftScope().Cast<TCoEquiJoinTuple>(), links, collectPred); + } + + if (!joinTree.RightScope().Maybe<TCoAtom>()) { + CollectEquiJoinLinks(joinTree.RightScope().Cast<TCoEquiJoinTuple>(), links, collectPred); + } + + if (collectPred(joinTree.Options())) { + YQL_ENSURE(joinTree.LeftKeys().Size() == joinTree.RightKeys().Size()); + YQL_ENSURE(joinTree.LeftKeys().Size() % 2 == 0); + for (ui32 i = 0; i < joinTree.LeftKeys().Size(); i += 2) { + + auto leftKey = FullColumnName(joinTree.LeftKeys().Item(i).Value(), joinTree.LeftKeys().Item(i + 1).Value()); + auto rightKey = FullColumnName(joinTree.RightKeys().Item(i).Value(), joinTree.RightKeys().Item(i + 1).Value()); + + links.emplace_back(leftKey, rightKey); + } + } + } + + struct TRemapTarget { + TString Name; + const TTypeAnnotationNode* Type; + }; + + static void ApplyJoinKeyRemapsLeaf(TExprNode::TPtr& keysNode, const THashMap<TStringBuf, THashMap<TStringBuf, TRemapTarget>>& memberRemapsByLabel, + TExprContext& ctx) + { + YQL_ENSURE(keysNode->IsList()); + TExprNodeList keys = keysNode->ChildrenList(); + YQL_ENSURE(keys.size() % 2 == 0); + for (size_t i = 0; i < keys.size(); i += 2) { + TCoAtom label(keys[i]); + if (auto it = memberRemapsByLabel.find(label.Value()); it != memberRemapsByLabel.end()) { + TCoAtom column(keys[i + 1]); + if (auto remapIt = it->second.find(column.Value()); remapIt != it->second.end()) { + keys[i + 1] = ctx.NewAtom(column.Pos(), remapIt->second.Name); + } + } + } + keysNode = ctx.NewList(keysNode->Pos(), std::move(keys)); + } + + static void ApplyJoinKeyRemaps(TExprNode::TPtr& joinTree, const THashMap<TStringBuf, THashMap<TStringBuf, TRemapTarget>>& memberRemapsByLabel, + TExprContext& ctx) + { + YQL_ENSURE(joinTree->IsList()); + TExprNodeList children = joinTree->ChildrenList(); + + auto& leftScope = children[TCoEquiJoinTuple::idx_LeftScope]; + if (!leftScope->IsAtom()) { + ApplyJoinKeyRemaps(leftScope, memberRemapsByLabel, ctx); + } + + auto& rightScope = children[TCoEquiJoinTuple::idx_RightScope]; + if (!rightScope->IsAtom()) { + ApplyJoinKeyRemaps(rightScope, memberRemapsByLabel, ctx); + } + + auto& leftKeys = children[TCoEquiJoinTuple::idx_LeftKeys]; + ApplyJoinKeyRemapsLeaf(leftKeys, memberRemapsByLabel, ctx); + + auto& rightKeys = children[TCoEquiJoinTuple::idx_RightKeys]; + ApplyJoinKeyRemapsLeaf(rightKeys, memberRemapsByLabel, ctx); + + joinTree = ctx.NewList(joinTree->Pos(), std::move(children)); + } + + TMaybeNode<TExprBase> ConvertToCommonTypeForForcedMergeJoin(TExprBase node, TExprContext& ctx) const { + auto equiJoin = node.Cast<TCoEquiJoin>(); + YQL_ENSURE(equiJoin.ArgCount() >= 4); + + bool hasYtInputs = false; + THashMap<TStringBuf, size_t> inputPosByLabel; + TVector<const TStructExprType*> inputTypes; + for (size_t i = 0; i < equiJoin.ArgCount() - 2; ++i) { + auto input = equiJoin.Arg(i).Cast<TCoEquiJoinInput>(); + if (!input.Scope().Maybe<TCoAtom>()) { + return node; + } + TStringBuf label = input.Scope().Cast<TCoAtom>().Value(); + bool inserted = inputPosByLabel.emplace(label, i).second; + YQL_ENSURE(inserted); + hasYtInputs = hasYtInputs || IsYtProviderInput(input.List()); + inputTypes.push_back(GetSeqItemType(*input.List().Ref().GetTypeAnn()).Cast<TStructExprType>()); + } + + if (!hasYtInputs) { + return node; + } + + bool forceSortedMerge = false; + if (auto force = State_->Configuration->JoinMergeForce.Get()) { + forceSortedMerge = *force; + } + + auto collectPred = [forceSortedMerge](TExprBase linkSettingsNode) { + return forceSortedMerge || GetEquiJoinLinkSettings(linkSettingsNode.Ref()).ForceSortedMerge; + }; + + TVector<std::pair<TString, TString>> links; + CollectEquiJoinLinks(equiJoin.Arg(equiJoin.ArgCount() - 2).Cast<TCoEquiJoinTuple>(), links, collectPred); + + TVector<TStringBuf> joinKeys; + for (const auto& [left, right] : links) { + joinKeys.push_back(left); + joinKeys.push_back(right); + } + SortUnique(joinKeys); + + TDisjointSets disjointSets(joinKeys.size()); + for (const auto& [left, right] : links) { + size_t leftPos = LowerBound(joinKeys.begin(), joinKeys.end(), left) - joinKeys.begin(); + size_t rightPos = LowerBound(joinKeys.begin(), joinKeys.end(), right) - joinKeys.begin(); + + YQL_ENSURE(leftPos < joinKeys.size()); + YQL_ENSURE(rightPos < joinKeys.size()); + + disjointSets.UnionSets(leftPos, rightPos); + } + + TMap<size_t, TSet<size_t>> keySetsByCanonicElement; + for (size_t i = 0; i < joinKeys.size(); ++i) { + size_t canonicElement = disjointSets.CanonicSetElement(i); + keySetsByCanonicElement[canonicElement].insert(i); + } + + THashMap<TStringBuf, THashMap<TStringBuf, TRemapTarget>> memberRemapsByLabel; + size_t genIdx = 0; + TVector<TString> toDrop; + for (auto& [_, keySet] : keySetsByCanonicElement) { + TVector<const TTypeAnnotationNode*> srcKeyTypes; + TVector<TStringBuf> srcKeyNames; + TVector<const TStructExprType*> srcInputTypes; + for (auto& keyIdx : keySet) { + const TStringBuf key = joinKeys[keyIdx]; + TStringBuf label; + TStringBuf column; + SplitTableName(key, label, column); + + auto it = inputPosByLabel.find(label); + YQL_ENSURE(it != inputPosByLabel.end()); + + const size_t inputIdx = it->second; + YQL_ENSURE(inputIdx < inputTypes.size()); + + auto columnType = inputTypes[inputIdx]->FindItemType(column); + YQL_ENSURE(columnType); + + srcKeyNames.push_back(key); + srcKeyTypes.push_back(columnType); + srcInputTypes.push_back(inputTypes[inputIdx]); + } + + // derive common type for all join keys in key set + const TTypeAnnotationNode* commonType = UnifyJoinKeyType(equiJoin.Pos(), srcKeyTypes, ctx); + YQL_ENSURE(commonType); + + const TTypeAnnotationNode* commonTypeNoOpt = RemoveOptionalType(commonType); + + bool needRemap = !IsDataOrOptionalOfData(commonType); + for (size_t i = 0; i < srcKeyNames.size(); ++i) { + TStringBuf srcKey = srcKeyNames[i]; + const TTypeAnnotationNode* srcKeyType = srcKeyTypes[i]; + const TStructExprType* inputType = srcInputTypes[i]; + + if (needRemap || !IsSameAnnotation(*RemoveOptionalType(srcKeyType), *commonTypeNoOpt)) { + TStringBuf label; + TStringBuf column; + SplitTableName(srcKey, label, column); + + TString targetName; + do { + targetName = TStringBuilder() << "_yql_normalized_join_key" << genIdx++; + } while (inputType->FindItem(targetName)); + + TRemapTarget target; + target.Type = commonType; + target.Name = targetName; + toDrop.push_back(FullColumnName(label, targetName)); + + memberRemapsByLabel[label][column] = target; + } + } + } + + TExprNodeList equiJoinArgs = equiJoin.Ref().ChildrenList(); + for (auto& [label, remaps] : memberRemapsByLabel) { + auto it = inputPosByLabel.find(label); + YQL_ENSURE(it != inputPosByLabel.end()); + + const size_t inputPos = it->second; + + TExprNode::TPtr& equiJoinInput = equiJoinArgs[inputPos]; + + auto row = ctx.NewArgument(equiJoinInput->Pos(), "row"); + auto body = row; + for (auto& [srcColumn, target] : remaps) { + auto srcType = inputTypes[inputPos]->FindItemType(srcColumn); + YQL_ENSURE(srcType); + auto srcValue = ctx.Builder(body->Pos()) + .Callable("Member") + .Add(0, row) + .Atom(1, srcColumn) + .Seal() + .Build(); + auto targetValue = RemapNonConvertibleMemberForJoin(srcValue->Pos(), srcValue, *srcType, *target.Type, ctx); + body = ctx.Builder(body->Pos()) + .Callable("AddMember") + .Add(0, body) + .Atom(1, target.Name) + .Add(2, targetValue) + .Seal() + .Build(); + } + + auto remapLambda = ctx.NewLambda(row->Pos(), ctx.NewArguments(row->Pos(), { row }), std::move(body)); + equiJoinInput = ctx.Builder(equiJoinInput->Pos()) + .List() + .Callable(0, "OrderedMap") + .Add(0, equiJoinInput->HeadPtr()) + .Add(1, remapLambda) + .Seal() + .Add(1, equiJoinInput->TailPtr()) + .Seal() + .Build(); + } + + if (!memberRemapsByLabel.empty()) { + ApplyJoinKeyRemaps(equiJoinArgs[equiJoin.ArgCount() - 2], memberRemapsByLabel, ctx); + TExprNode::TPtr result = ctx.ChangeChildren(equiJoin.Ref(), std::move(equiJoinArgs)); + + auto row = ctx.NewArgument(equiJoin.Pos(), "row"); + auto removed = row; + for (auto column : toDrop) { + removed = ctx.Builder(equiJoin.Pos()) + .Callable("ForceRemoveMember") + .Add(0, removed) + .Atom(1, column) + .Seal() + .Build(); + } + + auto removeLambda = ctx.NewLambda(equiJoin.Pos(), ctx.NewArguments(equiJoin.Pos(), { row }), std::move(removed)); + + result = ctx.Builder(equiJoin.Pos()) + .Callable("OrderedMap") + .Add(0, result) + .Add(1, removeLambda) + .Seal() + .Build(); + + return result; + } + + return node; + } + + TMaybeNode<TExprBase> SelfInnerJoinWithSameKeys(TExprBase node, TExprContext& ctx) const { + auto equiJoin = node.Cast<TCoEquiJoin>(); + // optimize self intersect (inner join) over (filtered) tables with unique keys + if (equiJoin.ArgCount() != 4) { + return node; + } + + auto tree = equiJoin.Arg(2).Cast<TCoEquiJoinTuple>(); + if (tree.Type().Value() != "Inner") { + return node; + } + + auto left = equiJoin.Arg(0).Cast<TCoEquiJoinInput>().List(); + auto right = equiJoin.Arg(1).Cast<TCoEquiJoinInput>().List(); + TMaybeNode<TExprBase> leftInput; + TMaybeNode<TExprBase> rightInput; + TMaybe<THashSet<TStringBuf>> leftPassthroughFields; + TMaybe<THashSet<TStringBuf>> rightPassthroughFields; + TMaybe<TSet<TString>> leftReadFields; + TMaybe<TSet<TString>> rightReadFields; + TExprNode::TPtr leftPredicate; + TExprNode::TPtr leftArg; + TExprNode::TPtr leftValue; + TExprNode::TPtr rightPredicate; + TExprNode::TPtr rightArg; + TExprNode::TPtr rightValue; + + if (auto extract = left.Maybe<TCoExtractMembers>()) { + leftReadFields.ConstructInPlace(); + for (auto f: extract.Cast().Members()) { + leftReadFields->emplace(f.Value()); + } + left = extract.Cast().Input(); + } + + if (auto extract = right.Maybe<TCoExtractMembers>()) { + rightReadFields.ConstructInPlace(); + for (auto f: extract.Cast().Members()) { + rightReadFields->emplace(f.Value()); + } + right = extract.Cast().Input(); + } + + if (IsYtProviderInput(left)) { + leftInput = left; + } else if (auto maybeFlatMap = left.Maybe<TCoFlatMapBase>()) { + auto flatMap = maybeFlatMap.Cast(); + const auto& input = flatMap.Input(); + if (IsYtProviderInput(input) && IsPassthroughFlatMap(flatMap, &leftPassthroughFields)) { + const auto& lambda = flatMap.Lambda(); + const auto& body = lambda.Body(); + leftArg = lambda.Args().Arg(0).Ptr(); + if (body.Maybe<TCoOptionalIf>() || body.Maybe<TCoListIf>()) { + leftValue = body.Cast<TCoConditionalValueBase>().Value().Ptr(); + leftPredicate = body.Cast<TCoConditionalValueBase>().Predicate().Ptr(); + } else { + leftValue = body.Ref().ChildPtr(0); + } + + leftInput = input; + } + } + + if (IsYtProviderInput(right)) { + rightInput = right; + } else if (auto maybeFlatMap = right.Maybe<TCoFlatMapBase>()) { + auto flatMap = maybeFlatMap.Cast(); + const auto& input = flatMap.Input(); + if (IsYtProviderInput(input) && IsPassthroughFlatMap(flatMap, &rightPassthroughFields)) { + const auto& lambda = flatMap.Lambda(); + const auto& body = lambda.Body(); + rightArg = lambda.Args().Arg(0).Ptr(); + if (body.Maybe<TCoOptionalIf>() || body.Maybe<TCoListIf>()) { + rightValue = body.Cast<TCoConditionalValueBase>().Value().Ptr(); + rightPredicate = body.Cast<TCoConditionalValueBase>().Predicate().Ptr(); + } else { + rightValue = body.Ref().ChildPtr(0); + } + + rightInput = input; + } + } + + if (!leftInput || !rightInput) { + return node; + } + + auto leftCluster = GetClusterName(leftInput.Cast()); + auto rightCluster = GetClusterName(rightInput.Cast()); + if (leftCluster != rightCluster) { + return node; + } + + // check that key and settings are same + if (leftInput.Maybe<TYtOutput>() && rightInput.Maybe<TYtOutput>()) { + if (leftInput.Cast().Raw() != rightInput.Cast().Raw()) { + return node; + } + } + else { + auto leftSection = leftInput.Maybe<TCoRight>().Input().Maybe<TYtReadTable>().Input().Item(0); + auto rightSection = rightInput.Maybe<TCoRight>().Input().Maybe<TYtReadTable>().Input().Item(0); + if (!leftSection || !rightSection) { + return node; + } + + if (!EqualSettingsExcept(leftSection.Settings().Cast().Ref(), rightSection.Settings().Cast().Ref(), EYtSettingType::Unordered)) { + return node; + } + + // only one table + if (leftSection.Cast().Paths().Size() > 1 || rightSection.Cast().Paths().Size() > 1) { + return node; + } + + auto leftPath = leftSection.Cast().Paths().Item(0); + auto rightPath = rightSection.Cast().Paths().Item(0); + + if (leftPath.Table().Raw() != rightPath.Table().Raw()) { + return node; + } + + if (leftPath.Ranges().Raw() != rightPath.Ranges().Raw()) { + return node; + } + } + + TYtTableBaseInfo::TPtr tableInfo = GetInputTableInfos(leftInput.Cast()).front(); + if (tableInfo->IsUnordered || !tableInfo->RowSpec || !tableInfo->RowSpec->IsSorted() || !tableInfo->RowSpec->UniqueKeys) { + return node; + } + + // check keys + THashSet<TStringBuf> joinColumns; + for (ui32 i = 0; i < tree.LeftKeys().Size(); i += 2) { + const auto& leftColumn = tree.LeftKeys().Item(i + 1).Value(); + const auto& rightColumn = tree.RightKeys().Item(i + 1).Value(); + if (leftColumn != rightColumn) { + return node; + } + + joinColumns.emplace(leftColumn); + } + + if (tableInfo->RowSpec->SortedBy.size() != joinColumns.size()) { + return node; + } + + for (auto& field : tableInfo->RowSpec->SortedBy) { + if (leftPassthroughFields && !leftPassthroughFields->contains(field)) { + return node; + } + + if (rightPassthroughFields && !rightPassthroughFields->contains(field)) { + return node; + } + + if (!joinColumns.contains(field)) { + return node; + } + } + + // now we could rewrite join with single flatmap + TExprBase commonInput = leftInput.Cast(); + if (leftInput.Maybe<TCoRight>()) { + auto read = leftInput.Maybe<TCoRight>().Input().Maybe<TYtReadTable>().Cast(); + if (!leftReadFields) { + if (auto columns = leftInput.Maybe<TCoRight>().Input().Maybe<TYtReadTable>().Input().Item(0).Paths().Item(0).Columns().Maybe<TCoAtomList>()) { + leftReadFields.ConstructInPlace(); + for (auto f: columns.Cast()) { + leftReadFields->emplace(f.Value()); + } + } + } + if (!rightReadFields) { + if (auto columns = rightInput.Maybe<TCoRight>().Input().Maybe<TYtReadTable>().Input().Item(0).Paths().Item(0).Columns().Maybe<TCoAtomList>()) { + rightReadFields.ConstructInPlace(); + for (auto f: columns.Cast()) { + rightReadFields->emplace(f.Value()); + } + } + } + if (leftReadFields && rightReadFields) { + TSet<TString> commonFields; + commonFields.insert(leftReadFields->begin(), leftReadFields->end()); + commonFields.insert(rightReadFields->begin(), rightReadFields->end()); + auto columnsBuilder = Build<TCoAtomList>(ctx, node.Pos()); + for (auto f: commonFields) { + columnsBuilder.Add().Value(f).Build(); + } + commonInput = Build<TCoRight>(ctx, leftInput.Cast().Pos()) + .Input<TYtReadTable>() + .World(read.World()) + .DataSource(read.DataSource()) + .Input() + .Add() + .Paths() + .Add() + .InitFrom(read.Input().Item(0).Paths().Item(0)) + .Columns(columnsBuilder.Done()) + .Build() + .Build() + .Settings(read.Input().Item(0).Settings()) + .Build() + .Build() + .Build() + .Done(); + } + } + + auto joinLambdaArg = ctx.NewArgument(node.Pos(), "read"); + auto leftReadArg = joinLambdaArg; + if (leftReadFields) { + leftReadArg = FilterByFields(node.Pos(), leftReadArg, *leftReadFields, ctx, true); + } + + auto rightReadArg = joinLambdaArg; + if (rightReadFields) { + rightReadArg = FilterByFields(node.Pos(), rightReadArg, *rightReadFields, ctx, true); + } + + auto truth = ctx.NewCallable(node.Pos(), "Bool", { ctx.NewAtom(node.Pos(), "true") }); + auto leftPrefix = equiJoin.Arg(0).Cast<TCoEquiJoinInput>().Scope().Cast<TCoAtom>().Value(); + auto rightPrefix = equiJoin.Arg(1).Cast<TCoEquiJoinInput>().Scope().Cast<TCoAtom>().Value(); + auto joinLambdaBody = ctx.Builder(node.Pos()) + .Callable("OptionalIf") + .Callable(0, "And") + .Add(0, leftPredicate ? ctx.ReplaceNode(std::move(leftPredicate), *leftArg, leftReadArg) : truth) + .Add(1, rightPredicate ? ctx.ReplaceNode(std::move(rightPredicate), *rightArg, rightReadArg) : truth) + .Seal() + .Callable(1, "FlattenMembers") + .List(0) + .Atom(0, TString(leftPrefix) + ".") + .Add(1, leftValue ? ctx.ReplaceNode(std::move(leftValue), *leftArg, leftReadArg) : leftReadArg) + .Seal() + .List(1) + .Atom(0, TString(rightPrefix) + ".") + .Add(1, rightValue ? ctx.ReplaceNode(std::move(rightValue), *rightArg, rightReadArg) : rightReadArg) + .Seal() + .Seal() + .Seal() + .Build(); + + auto joinLambda = ctx.NewLambda(node.Pos(), ctx.NewArguments(node.Pos(), { joinLambdaArg }), std::move(joinLambdaBody)); + auto joinFlatMap = Build<TCoFlatMap>(ctx, node.Pos()) + .Input(commonInput) + .Lambda(joinLambda) + .Done(); + + auto ret = joinFlatMap.Ptr(); + + // have to apply renames + auto settings = equiJoin.Arg(equiJoin.ArgCount() - 1); + auto renameMap = LoadJoinRenameMap(settings.Ref()); + + if (!renameMap.empty()) { + const TStructExprType* itemType = nullptr; + if (auto type = GetSequenceItemType(node, false, ctx)) { + itemType = type->Cast<TStructExprType>(); + } else { + return {}; + } + + ret = ctx.Builder(node.Pos()) + .Callable("Map") + .Add(0, ret) + .Add(1, BuildJoinRenameLambda(node.Pos(), renameMap, *itemType, ctx).Ptr()) + .Seal() + .Build(); + } + + return TExprBase(ret); + } + + TMaybeNode<TExprBase> Unordered(TExprBase node, TExprContext& ctx) const { + const auto unordered = node.Cast<TCoUnorderedBase>(); + if (const auto tableContent = unordered.Input().Maybe<TYtTableContent>()) { + if (const auto out = tableContent.Input().Maybe<TYtOutput>()) { + if (IsUnorderedOutput(out.Cast())) + return tableContent; + return Build<TYtTableContent>(ctx, node.Pos()) + .InitFrom(tableContent.Cast()) + .Input<TYtOutput>() + .InitFrom(out.Cast()) + .Mode() + .Value(ToString(EYtSettingType::Unordered), TNodeFlags::Default) + .Build() + .Build() + .Done(); + } else if (const auto read = tableContent.Input().Maybe<TYtReadTable>()) { + TVector<TYtSection> sections; + for (auto section: read.Cast().Input()) { + sections.push_back(MakeUnorderedSection<true>(section, ctx)); + } + return Build<TYtTableContent>(ctx, node.Pos()) + .InitFrom(tableContent.Cast()) + .Input<TYtReadTable>() + .InitFrom(read.Cast()) + .Input() + .Add(sections) + .Build() + .Build() + .Done(); + } + } else if (const auto input = unordered.Input(); IsYtProviderInput(input, true)) { + if (const auto out = input.Maybe<TYtOutput>()) { + if (unordered.Maybe<TCoUnorderedSubquery>() && out.Cast().Operation().Maybe<TYtSort>()) { + if (!WarnUnroderedSubquery(node.Ref(), ctx)) { + return {}; + } + } + if (IsUnorderedOutput(out.Cast())) { + // Already unordered. Just drop + return input; + } + return Build<TYtOutput>(ctx, node.Pos()) + .InitFrom(out.Cast()) + .Mode() + .Value(ToString(EYtSettingType::Unordered), TNodeFlags::Default) + .Build() + .Done(); + } else if (const auto read = input.Maybe<TCoRight>().Input().Maybe<TYtReadTable>()) { + TExprNode::TListType sections(read.Cast().Input().Size()); + for (auto i = 0U; i < sections.size(); ++i) { + sections[i] = MakeUnorderedSection<true>(read.Cast().Input().Item(i), ctx).Ptr(); + } + + return Build<TCoRight>(ctx, node.Pos()) + .Input<TYtReadTable>() + .InitFrom(read.Cast()) + .Input() + .Add(std::move(sections)) + .Build() + .Build() + .Done(); + } + } + + return node; + } + + TMaybeNode<TExprBase> ShuffleByKeys(TExprBase node, TExprContext& ctx) const { + auto list = node.Cast<TCoInputBase>().Input(); + if (!IsYtProviderInput(list)) { + return node; + } + + auto shuffle = node.Cast<TCoShuffleByKeys>(); + + return Build<TCoPartitionsByKeys>(ctx, node.Pos()) + .Input(list) + .SortDirections<TCoVoid>() + .Build() + .SortKeySelectorLambda<TCoVoid>() + .Build() + .KeySelectorLambda(shuffle.KeySelectorLambda()) + .ListHandlerLambda<TCoLambda>() + .Args({ TStringBuf("stream") }) + .Body<TCoForwardList>() + .Stream<TCoToStream>() + .Input<TExprApplier>() + .Apply(shuffle.ListHandlerLambda().Body()) + .With(shuffle.ListHandlerLambda().Args().Arg(0), TStringBuf("stream")) + .Build() + .Build() + .Build() + .Build() + .Done(); + } + + TMaybeNode<TExprBase> MatchRecognize(TExprBase node, TExprContext& ctx) { + return ExpandMatchRecognize(node.Ptr(), ctx, *Types); + } + + TMaybeNode<TExprBase> CountAggregate(TExprBase node, TExprContext& ctx) const { + auto aggregate = node.Cast<TCoAggregate>(); + + auto input = aggregate.Input(); + if (!IsYtProviderInput(input)) { + return node; + } + + return TAggregateExpander::CountAggregateRewrite(aggregate, ctx, + State_->Types->UseBlocks || State_->Types->BlockEngineMode == EBlockEngineMode::Force); + } + + TMaybeNode<TExprBase> ZeroSampleToZeroLimit(TExprBase node, TExprContext& ctx) const { + auto read = node.Cast<TYtReadTable>(); + + bool hasUpdates = false; + TVector<TExprBase> updatedSections; + for (auto section: read.Input()) { + TMaybe<TSampleParams> sampling = NYql::GetSampleParams(section.Settings().Ref()); + if (sampling && sampling->Percentage == 0.0) { + hasUpdates = true; + section = Build<TYtSection>(ctx, section.Pos()) + .InitFrom(section) + .Settings( + NYql::AddSetting( + *NYql::RemoveSetting(section.Settings().Ref(), EYtSettingType::Sample, ctx), + EYtSettingType::Take, + Build<TCoUint64>(ctx, section.Pos()).Literal().Value(0).Build().Done().Ptr(), + ctx + ) + ) + .Done(); + } + updatedSections.push_back(section); + } + + if (!hasUpdates) { + return node; + } + + return ctx.ChangeChild(read.Ref(), TYtReadTable::idx_Input, + Build<TYtSectionList>(ctx, read.Input().Pos()) + .Add(updatedSections) + .Done().Ptr()); + } + +private: + TYtState::TPtr State_; +}; + + +THolder<IGraphTransformer> CreateYtLogicalOptProposalTransformer(TYtState::TPtr state) { + return THolder(new TYtLogicalOptProposalTransformer(state)); +} + +} diff --git a/yt/yql/providers/yt/provider/yql_yt_mkql_compiler.cpp b/yt/yql/providers/yt/provider/yql_yt_mkql_compiler.cpp new file mode 100644 index 0000000000..902af3f9b9 --- /dev/null +++ b/yt/yql/providers/yt/provider/yql_yt_mkql_compiler.cpp @@ -0,0 +1,586 @@ +#include "yql_yt_mkql_compiler.h" +#include "yql_yt_helpers.h" +#include "yql_yt_op_settings.h" + +#include <yt/yql/providers/yt/expr_nodes/yql_yt_expr_nodes.h> +#include <yt/yql/providers/yt/lib/row_spec/yql_row_spec.h> +#include <yt/yql/providers/yt/lib/skiff/yql_skiff_schema.h> +#include <yt/yql/providers/yt/lib/mkql_helpers/mkql_helpers.h> + +#include <yql/essentials/core/dq_expr_nodes/dq_expr_nodes.h> +#include <yql/essentials/core/dqs_expr_nodes/dqs_expr_nodes.h> +#include <yql/essentials/core/yql_opt_utils.h> +#include <yql/essentials/providers/common/codec/yql_codec_type_flags.h> +#include <yql/essentials/providers/common/mkql/yql_type_mkql.h> +#include <yql/essentials/minikql/mkql_program_builder.h> +#include <yql/essentials/minikql/mkql_node_cast.h> +#include <yql/essentials/minikql/defs.h> +#include <yql/essentials/utils/log/context.h> + +#include <library/cpp/yson/node/node_io.h> +#include <yt/cpp/mapreduce/common/helpers.h> + +#include <util/generic/yexception.h> +#include <util/generic/xrange.h> +#include <util/string/cast.h> + +namespace NYql { + +using namespace NKikimr; +using namespace NKikimr::NMiniKQL; +using namespace NNodes; +using namespace NNodes::NDq; + +TRuntimeNode BuildTableContentCall(TStringBuf callName, + TType* outItemType, + TStringBuf clusterName, + const TExprNode& input, + const TMaybe<ui64>& itemsCount, + NCommon::TMkqlBuildContext& ctx, + bool forceColumns, + const THashSet<TString>& extraSysColumns, + bool forceKeyColumns) +{ + forceColumns = forceColumns || forceKeyColumns; + TType* const strType = ctx.ProgramBuilder.NewDataType(NUdf::TDataType<char*>::Id); + TType* const boolType = ctx.ProgramBuilder.NewDataType(NUdf::TDataType<bool>::Id); + TType* const ui64Type = ctx.ProgramBuilder.NewDataType(NUdf::TDataType<ui64>::Id); + TType* const ui32Type = ctx.ProgramBuilder.NewDataType(NUdf::TDataType<ui32>::Id); + TType* const tupleTypeTables = ctx.ProgramBuilder.NewTupleType({strType, boolType, strType, ui64Type, ui64Type, boolType, ui32Type}); + TType* const listTypeGroup = ctx.ProgramBuilder.NewListType(tupleTypeTables); + + const TExprNode* settings = nullptr; + TMaybe<TSampleParams> sampling; + TVector<TRuntimeNode> groups; + if (input.IsCallable(TYtOutput::CallableName())) { + YQL_ENSURE(!forceKeyColumns); + auto outTableInfo = TYtOutTableInfo(GetOutTable(TExprBase(&input))); + YQL_ENSURE(outTableInfo.Stat, "Table " << outTableInfo.Name.Quote() << " has no Stat"); + auto richYPath = NYT::TRichYPath(outTableInfo.Name); + if (forceColumns && outTableInfo.RowSpec->HasAuxColumns()) { + NYT::TSortColumns columns; + for (auto& item: outTableInfo.RowSpec->GetType()->GetItems()) { + columns.Add(TString{item->GetName()}); + } + richYPath.Columns(columns); + } + TString spec; + if (!extraSysColumns.empty()) { + auto specNode = outTableInfo.GetCodecSpecNode(); + auto structType = AS_TYPE(TStructType, outItemType); + NYT::TNode columns; + for (auto& col: extraSysColumns) { + const auto fullName = TString(YqlSysColumnPrefix).append(col); + if (!structType->FindMemberIndex(fullName)) { + columns.Add(col); + outItemType = ctx.ProgramBuilder.NewStructType(outItemType, fullName, ctx.ProgramBuilder.NewDataType(GetSysColumnTypeId(col))); + } + } + if (!columns.IsUndefined()) { + specNode[YqlSysColumnPrefix] = std::move(columns); + } + spec = NYT::NodeToCanonicalYsonString(specNode); + } else { + spec = outTableInfo.RowSpec->ToYsonString(); + } + groups.push_back( + ctx.ProgramBuilder.NewList(tupleTypeTables, {ctx.ProgramBuilder.NewTuple(tupleTypeTables, { + ctx.ProgramBuilder.NewDataLiteral<NUdf::EDataSlot::String>(NYT::NodeToYsonString(NYT::PathToNode(richYPath))), + ctx.ProgramBuilder.NewDataLiteral(true), + ctx.ProgramBuilder.NewDataLiteral<NUdf::EDataSlot::String>(spec), + ctx.ProgramBuilder.NewDataLiteral(outTableInfo.Stat->ChunkCount), + ctx.ProgramBuilder.NewDataLiteral(outTableInfo.Stat->RecordsCount), + ctx.ProgramBuilder.NewDataLiteral(false), + ctx.ProgramBuilder.NewDataLiteral(ui32(0)), + })}) + ); + } + else { + auto sectionList = TYtSectionList(&input); + TVector<TType*> sectionTypes; + bool rebuildType = false; + for (size_t i: xrange(sectionList.Size())) { + auto section = sectionList.Item(i); + TType* secType = outItemType; + if (sectionList.Size() > 1) { + const auto varType = AS_TYPE(TVariantType, outItemType); + const auto tupleType = AS_TYPE(TTupleType, varType->GetUnderlyingType()); + secType = tupleType->GetElementType(i); + } + TVector<TRuntimeNode> tableTuples; + TVector<TString> columns; + if (forceColumns) { + for (auto& colType: section.Ref().GetTypeAnn()->Cast<TListExprType>()->GetItemType()->Cast<TStructExprType>()->GetItems()) { + columns.push_back(ToString(colType->GetName())); + } + } + + TStructType* structType = AS_TYPE(TStructType, secType); + if (forceKeyColumns) { + TMap<TString, const TTypeAnnotationNode*> extraKeyColumns; + for (auto p: section.Paths()) { + TYtPathInfo pathInfo(p); + if (!pathInfo.Ranges || !pathInfo.Table->RowSpec || !pathInfo.Table->RowSpec->IsSorted()) { + continue; + } + size_t usedKeyPrefix = pathInfo.Ranges->GetUsedKeyPrefixLength(); + for (size_t i = 0; i < usedKeyPrefix; ++i) { + TString key = pathInfo.Table->RowSpec->SortedBy[i]; + if (!structType->FindMemberIndex(key)) { + auto itemType = pathInfo.Table->RowSpec->GetType()->FindItemType(key); + YQL_ENSURE(itemType); + + auto it = extraKeyColumns.find(key); + if (it == extraKeyColumns.end()) { + extraKeyColumns[key] = itemType; + } else { + YQL_ENSURE(IsSameAnnotation(*it->second, *itemType), + "Extra key columns should be of same type in all paths"); + } + } + } + } + + for (auto& [key, keyType] : extraKeyColumns) { + for (auto p: section.Paths()) { + TYtPathInfo pathInfo(p); + auto currKeyType = pathInfo.Table->RowSpec->GetType()->FindItemType(key); + YQL_ENSURE(currKeyType, + "Column " << key << + " is used only in key filter in one YtPath and missing in another YPath in same section"); + YQL_ENSURE(IsSameAnnotation(*keyType, *currKeyType), + "Extra key columns should be of same type in all paths"); + } + + secType = ctx.ProgramBuilder.NewStructType(secType, key, + NCommon::BuildType(section.Ref(), *keyType, ctx.ProgramBuilder)); + rebuildType = true; + } + + for (auto& k : extraKeyColumns) { + columns.push_back(k.first); + } + } + + NYT::TNode sysColumns; + for (auto& col: extraSysColumns) { + const auto fullName = TString(YqlSysColumnPrefix).append(col); + if (!structType->FindMemberIndex(fullName)) { + sysColumns.Add(col); + secType = ctx.ProgramBuilder.NewStructType(secType, fullName, ctx.ProgramBuilder.NewDataType(GetSysColumnTypeId(col))); + rebuildType = true; + } + } + sectionTypes.push_back(secType); + for (auto col: NYql::GetSettingAsColumnList(section.Settings().Ref(), EYtSettingType::SysColumns)) { + sysColumns.Add(col); + } + + for (auto p: section.Paths()) { + TYtPathInfo pathInfo(p); + YQL_ENSURE(pathInfo.Table->Stat, "Table " << pathInfo.Table->Name.Quote() << " has no Stat"); + // Table may have aux columns. Exclude them by specifying explicit columns from the type + if (forceColumns && pathInfo.Table->RowSpec && (forceKeyColumns || !pathInfo.HasColumns())) { + pathInfo.SetColumns(columns); + } + TString spec; + if (!sysColumns.IsUndefined()) { + auto specNode = pathInfo.GetCodecSpecNode(); + specNode[YqlSysColumnPrefix] = sysColumns; + spec = NYT::NodeToCanonicalYsonString(specNode); + } else { + spec = pathInfo.GetCodecSpecStr(); + } + + TVector<TRuntimeNode> tupleItems; + NYT::TRichYPath richTPath(pathInfo.Table->Name); + pathInfo.FillRichYPath(richTPath); + tupleItems.push_back(ctx.ProgramBuilder.NewDataLiteral<NUdf::EDataSlot::String>(NYT::NodeToYsonString(NYT::PathToNode(richTPath)))); + tupleItems.push_back(ctx.ProgramBuilder.NewDataLiteral(pathInfo.Table->IsTemp)); + tupleItems.push_back(ctx.ProgramBuilder.NewDataLiteral<NUdf::EDataSlot::String>(spec)); + tupleItems.push_back(ctx.ProgramBuilder.NewDataLiteral(pathInfo.Table->Stat->ChunkCount)); + tupleItems.push_back(ctx.ProgramBuilder.NewDataLiteral(pathInfo.Table->Stat->RecordsCount)); + tupleItems.push_back(ctx.ProgramBuilder.NewDataLiteral(pathInfo.Table->IsAnonymous)); + tupleItems.push_back(ctx.ProgramBuilder.NewDataLiteral(pathInfo.Table->Epoch.GetOrElse(0))); + + tableTuples.push_back(ctx.ProgramBuilder.NewTuple(tupleTypeTables, tupleItems)); + } + groups.push_back(ctx.ProgramBuilder.NewList(tupleTypeTables, tableTuples)); + // All sections have the same sampling settings + sampling = GetSampleParams(section.Settings().Ref()); + } + if (sectionList.Size() == 1) { + settings = sectionList.Item(0).Settings().Raw(); + if (rebuildType) { + outItemType = sectionTypes.front(); + } + } else if (rebuildType) { + outItemType = ctx.ProgramBuilder.NewVariantType(ctx.ProgramBuilder.NewTupleType(sectionTypes)); + } + } + + TVector<TRuntimeNode> samplingTupleItems; + if (sampling) { + samplingTupleItems.push_back(ctx.ProgramBuilder.NewDataLiteral(sampling->Percentage)); + samplingTupleItems.push_back(ctx.ProgramBuilder.NewDataLiteral(sampling->Repeat)); + bool isSystemSampling = sampling->Mode == EYtSampleMode::System; + samplingTupleItems.push_back(ctx.ProgramBuilder.NewDataLiteral(isSystemSampling)); + } + + auto outListType = ctx.ProgramBuilder.NewListType(outItemType); + + TCallableBuilder call(ctx.ProgramBuilder.GetTypeEnvironment(), callName, outListType); + + call.Add(ctx.ProgramBuilder.NewDataLiteral<NUdf::EDataSlot::String>(clusterName)); // cluster name + call.Add(ctx.ProgramBuilder.NewList(listTypeGroup, groups)); + call.Add(ctx.ProgramBuilder.NewTuple(samplingTupleItems)); + + if (itemsCount) { + call.Add(ctx.ProgramBuilder.NewTuple({ctx.ProgramBuilder.NewDataLiteral(*itemsCount)})); + } else { + call.Add(ctx.ProgramBuilder.NewEmptyTuple()); + } + + auto res = TRuntimeNode(call.Build(), false); + + if (settings) { + for (auto child: settings->Children()) { + switch (FromString<EYtSettingType>(child->Child(0)->Content())) { + case EYtSettingType::Take: + res = ctx.ProgramBuilder.Take(res, NCommon::MkqlBuildExpr(*child->Child(1), ctx)); + break; + case EYtSettingType::Skip: + res = ctx.ProgramBuilder.Skip(res, NCommon::MkqlBuildExpr(*child->Child(1), ctx)); + break; + case EYtSettingType::Sample: + case EYtSettingType::DirectRead: + case EYtSettingType::KeyFilter: + case EYtSettingType::KeyFilter2: + case EYtSettingType::Unordered: + case EYtSettingType::NonUnique: + case EYtSettingType::SysColumns: + break; + default: + YQL_LOG_CTX_THROW yexception() << "Unsupported table content setting " << TString{child->Child(0)->Content()}.Quote(); + } + } + } + + return res; +} + +TRuntimeNode BuildTableContentCall(TType* outItemType, + TStringBuf clusterName, + const TExprNode& input, + const TMaybe<ui64>& itemsCount, + NCommon::TMkqlBuildContext& ctx, + bool forceColumns, + const THashSet<TString>& extraSysColumns, + bool forceKeyColumns) +{ + return BuildTableContentCall(TYtTableContent::CallableName(), outItemType, clusterName, input, itemsCount, ctx, forceColumns, extraSysColumns, forceKeyColumns); +} + +template<bool NeedPartitionRanges> +TRuntimeNode BuildDqYtInputCall( + TType* outputType, + TType* itemType, + const TString& clusterName, + const TString& tokenName, + const TYtSectionList& sectionList, + const TYtState::TPtr& state, + NCommon::TMkqlBuildContext& ctx, + size_t inflight, + size_t timeout, + bool enableBlockReader) +{ + NYT::TNode specNode = NYT::TNode::CreateMap(); + NYT::TNode& tablesNode = specNode[YqlIOSpecTables]; + NYT::TNode& registryNode = specNode[YqlIOSpecRegistry]; + THashMap<TString, TString> uniqSpecs; + NYT::TNode samplingSpec; + const ui64 nativeTypeCompat = state->Configuration->NativeYtTypeCompatibility.Get(clusterName).GetOrElse(NTCF_LEGACY); + + auto updateFlags = [nativeTypeCompat](NYT::TNode& spec) { + if (spec.HasKey(YqlRowSpecAttribute)) { + auto& rowSpec = spec[YqlRowSpecAttribute]; + ui64 nativeYtTypeFlags = 0; + if (rowSpec.HasKey(RowSpecAttrNativeYtTypeFlags)) { + nativeYtTypeFlags = rowSpec[RowSpecAttrNativeYtTypeFlags].AsUint64(); + } else { + if (rowSpec.HasKey(RowSpecAttrUseNativeYtTypes)) { + nativeYtTypeFlags = rowSpec[RowSpecAttrUseNativeYtTypes].AsBool() ? NTCF_LEGACY : NTCF_NONE; + } else if (rowSpec.HasKey(RowSpecAttrUseTypeV2)) { + nativeYtTypeFlags = rowSpec[RowSpecAttrUseTypeV2].AsBool() ? NTCF_LEGACY : NTCF_NONE; + } + } + rowSpec[RowSpecAttrNativeYtTypeFlags] = ui64(nativeYtTypeFlags & nativeTypeCompat); + } + }; + + TVector<TRuntimeNode> groups; + for (size_t i: xrange(sectionList.Size())) { + auto section = sectionList.Item(i); + for (auto& child: section.Settings().Ref().Children()) { + switch (FromString<EYtSettingType>(child->Child(0)->Content())) { + case EYtSettingType::Sample: + case EYtSettingType::SysColumns: + case EYtSettingType::Unordered: + case EYtSettingType::NonUnique: + break; + case EYtSettingType::KeyFilter: + case EYtSettingType::KeyFilter2: + YQL_ENSURE(child->Child(1)->ChildrenSize() == 0, "Unsupported KeyFilter setting"); + break; + default: + YQL_ENSURE(false, "Unsupported settings"); + break; + } + } + + TVector<TStringBuf> columns; + THashMap<TString, ui32> structColumns; + ui32 index = 0; + for (auto& colType: section.Ref().GetTypeAnn()->Cast<TListExprType>()->GetItemType()->Cast<TStructExprType>()->GetItems()) { + columns.push_back(colType->GetName()); + structColumns.emplace(colType->GetName(), index++); + } + + NYT::TNode sysColumns; + for (auto col: NYql::GetSettingAsColumnList(section.Settings().Ref(), EYtSettingType::SysColumns)) { + sysColumns.Add(col); + } + + TVector<TRuntimeNode> tableTuples; + ui64 tableOffset = 0; + for (auto p: section.Paths()) { + TYtPathInfo pathInfo(p); + // Table may have aux columns. Exclude them by specifying explicit columns from the type + if (pathInfo.Table->RowSpec && !pathInfo.HasColumns()) { + pathInfo.SetColumns(columns); + } + auto specNode = pathInfo.GetCodecSpecNode(); + if (!sysColumns.IsUndefined()) { + specNode[YqlSysColumnPrefix] = sysColumns; + } + updateFlags(specNode); + TString refName = TStringBuilder() << "$table" << uniqSpecs.size(); + auto res = uniqSpecs.emplace(NYT::NodeToCanonicalYsonString(specNode), refName); + if (res.second) { + registryNode[refName] = specNode; + } else { + refName = res.first->second; + } + tablesNode.Add(refName); + // TODO() Enable range indexes + auto skiffNode = SingleTableSpecToInputSkiff(specNode, structColumns, !enableBlockReader, !enableBlockReader, false); + const auto tmpFolder = GetTablesTmpFolder(*state->Configuration); + auto tableName = pathInfo.Table->Name; + if (pathInfo.Table->IsAnonymous && !TYtTableInfo::HasSubstAnonymousLabel(pathInfo.Table->FromNode.Cast())) { + tableName = state->AnonymousLabels.Value(std::make_pair(clusterName, tableName), TString()); + YQL_ENSURE(tableName, "Unaccounted anonymous table: " << pathInfo.Table->Name); + } + + NYT::TRichYPath richYPath = state->Gateway->GetRealTable(state->SessionId, clusterName, tableName, pathInfo.Table->Epoch.GetOrElse(0), tmpFolder); + pathInfo.FillRichYPath(richYPath); + auto pathNode = NYT::PathToNode(richYPath); + + tableTuples.push_back(ctx.ProgramBuilder.NewTuple({ + ctx.ProgramBuilder.NewDataLiteral<NUdf::EDataSlot::String>(pathInfo.Table->IsTemp ? TString() : tableName), + ctx.ProgramBuilder.NewDataLiteral<NUdf::EDataSlot::String>(NYT::NodeToYsonString(pathNode)), + ctx.ProgramBuilder.NewDataLiteral<NUdf::EDataSlot::String>(NYT::NodeToYsonString(skiffNode)), + ctx.ProgramBuilder.NewDataLiteral(tableOffset) + })); + YQL_ENSURE(pathInfo.Table->Stat); + tableOffset += pathInfo.Table->Stat->RecordsCount; + } + groups.push_back(ctx.ProgramBuilder.NewList(tableTuples.front().GetStaticType(), tableTuples)); + // All sections have the same sampling settings + if (samplingSpec.IsUndefined()) { + if (auto sampling = GetSampleParams(section.Settings().Ref())) { + samplingSpec["sampling_rate"] = sampling->Percentage / 100.; + if (sampling->Repeat) { + samplingSpec["sampling_seed"] = static_cast<i64>(sampling->Repeat); + } + if (sampling->Mode == EYtSampleMode::System) { + samplingSpec["sampling_mode"] = "block"; + } + } + } + } + + auto server = state->Gateway->GetClusterServer(clusterName); + YQL_ENSURE(server, "Invalid YT cluster: " << clusterName); + + TCallableBuilder call(ctx.ProgramBuilder.GetTypeEnvironment(), enableBlockReader ? "DqYtBlockRead" : "DqYtRead", outputType); + + call.Add(ctx.ProgramBuilder.NewDataLiteral<NUdf::EDataSlot::String>(server)); + call.Add(ctx.ProgramBuilder.NewDataLiteral<NUdf::EDataSlot::String>(tokenName)); + call.Add(ctx.ProgramBuilder.NewDataLiteral<NUdf::EDataSlot::String>(NYT::NodeToYsonString(specNode))); + call.Add(ctx.ProgramBuilder.NewDataLiteral<NUdf::EDataSlot::String>(samplingSpec.IsUndefined() ? TString() : NYT::NodeToYsonString(samplingSpec))); + call.Add(ctx.ProgramBuilder.NewList(groups.front().GetStaticType(), groups)); + call.Add(TRuntimeNode(itemType, true)); + + call.Add(ctx.ProgramBuilder.NewDataLiteral(inflight)); + call.Add(ctx.ProgramBuilder.NewDataLiteral(timeout)); + if constexpr (NeedPartitionRanges) + call.Add(ctx.ProgramBuilder.NewVoid()); + + return TRuntimeNode(call.Build(), false); +} + +void RegisterYtMkqlCompilers(NCommon::TMkqlCallableCompilerBase& compiler) { + compiler.AddCallable(TYtLength::CallableName(), + [](const TExprNode& node, NCommon::TMkqlBuildContext& ctx) { + auto length = TYtLength(&node); + ui64 lengthRes = 0; + if (auto out = length.Input().Maybe<TYtOutput>()) { + auto info = TYtOutTableInfo(GetOutTable(out.Cast())); + YQL_ENSURE(info.Stat); + lengthRes = info.Stat->RecordsCount; + } else { + auto read = length.Input().Maybe<TYtReadTable>(); + YQL_ENSURE(read, "Unknown length input"); + YQL_ENSURE(read.Cast().Input().Size() == 1, "Unsupported read with multiple sections"); + for (auto path: read.Cast().Input().Item(0).Paths()) { + auto info = TYtTableBaseInfo::Parse(path.Table()); + YQL_ENSURE(info->Stat); + lengthRes += info->Stat->RecordsCount; + } + } + return ctx.ProgramBuilder.NewDataLiteral<ui64>(lengthRes); + }); + + compiler.AddCallable(TYtTableContent::CallableName(), + [](const TExprNode& node, NCommon::TMkqlBuildContext& ctx) { + TYtTableContent tableContent(&node); + if (node.GetConstraint<TEmptyConstraintNode>()) { + const auto itemType = NCommon::BuildType(node, GetSeqItemType(*node.GetTypeAnn()), ctx.ProgramBuilder); + return ctx.ProgramBuilder.NewEmptyList(itemType); + } + TMaybe<ui64> itemsCount; + TString name = ToString(TYtTableContent::CallableName()); + if (auto setting = NYql::GetSetting(tableContent.Settings().Ref(), EYtSettingType::ItemsCount)) { + itemsCount = FromString<ui64>(setting->Child(1)->Content()); + } + if (NYql::HasSetting(tableContent.Settings().Ref(), EYtSettingType::Small)) { + name.prepend("Small"); + } + if (auto maybeRead = tableContent.Input().Maybe<TYtReadTable>()) { + auto read = maybeRead.Cast(); + return BuildTableContentCall(name, + NCommon::BuildType(node, *node.GetTypeAnn()->Cast<TListExprType>()->GetItemType(), ctx.ProgramBuilder), + read.DataSource().Cluster().Value(), read.Input().Ref(), itemsCount, ctx, true); + } else { + auto output = tableContent.Input().Cast<TYtOutput>(); + return BuildTableContentCall(name, + NCommon::BuildType(node, *node.GetTypeAnn()->Cast<TListExprType>()->GetItemType(), ctx.ProgramBuilder), + GetOutputOp(output).DataSink().Cluster().Value(), output.Ref(), itemsCount, ctx, true); + } + }); + + compiler.AddCallable({TYtTablePath::CallableName(), TYtTableRecord::CallableName(), TYtTableIndex::CallableName(), TYtIsKeySwitch::CallableName(), TYtRowNumber::CallableName()}, + [](const TExprNode& node, NCommon::TMkqlBuildContext& ctx) { + auto dataSlot = node.GetTypeAnn()->Cast<TDataExprType>()->GetSlot(); + + TCallableBuilder call(ctx.ProgramBuilder.GetTypeEnvironment(), node.Content(), + ctx.ProgramBuilder.NewDataType(dataSlot)); + + call.Add(NCommon::MkqlBuildExpr(*node.Child(0), ctx)); + return TRuntimeNode(call.Build(), false); + }); +} + +void RegisterDqYtMkqlCompilers(NCommon::TMkqlCallableCompilerBase& compiler, const TYtState::TPtr& state) { + + compiler.ChainCallable(TDqReadBlockWideWrap::CallableName(), + [state](const TExprNode& node, NCommon::TMkqlBuildContext& ctx) { + if (const auto& wrapper = TDqReadBlockWideWrap(&node); wrapper.Input().Maybe<TYtReadTable>().IsValid()) { + const auto ytRead = wrapper.Input().Cast<TYtReadTable>(); + const auto readType = ytRead.Ref().GetTypeAnn()->Cast<TTupleExprType>()->GetItems().back(); + const auto inputItemType = NCommon::BuildType(wrapper.Input().Ref(), GetSeqItemType(*readType), ctx.ProgramBuilder); + const auto cluster = ytRead.DataSource().Cluster().StringValue(); + const bool useRPCReaderDefault = DEFAULT_USE_RPC_READER_IN_DQ || state->Types->BlockEngineMode != EBlockEngineMode::Disable; + size_t inflight = state->Configuration->UseRPCReaderInDQ.Get(cluster).GetOrElse(useRPCReaderDefault) ? state->Configuration->DQRPCReaderInflight.Get(cluster).GetOrElse(DEFAULT_RPC_READER_INFLIGHT) : 0; + size_t timeout = state->Configuration->DQRPCReaderTimeout.Get(cluster).GetOrElse(DEFAULT_RPC_READER_TIMEOUT).MilliSeconds(); + const auto outputType = NCommon::BuildType(wrapper.Ref(), *wrapper.Ref().GetTypeAnn(), ctx.ProgramBuilder); + TString tokenName; + if (auto secureParams = wrapper.Token()) { + tokenName = secureParams.Cast().Name().StringValue(); + } + + bool solid = false; + for (const auto& flag : wrapper.Flags()) + if (solid = flag.Value() == "Solid") + break; + // at this moment, we know, that rpc reader is enabled (see dq_opts + CanBlockRead at integration) + return ctx.ProgramBuilder.BlockExpandChunked( + solid + ? BuildDqYtInputCall<false>(outputType, inputItemType, cluster, tokenName, ytRead.Input(), state, ctx, 1, timeout, true && inflight) + : BuildDqYtInputCall<true>(outputType, inputItemType, cluster, tokenName, ytRead.Input(), state, ctx, inflight, timeout, true && inflight) + ); + } + + return TRuntimeNode(); + }); + + compiler.ChainCallable(TDqReadWideWrap::CallableName(), + [state](const TExprNode& node, NCommon::TMkqlBuildContext& ctx) { + if (const auto& wrapper = TDqReadWideWrap(&node); wrapper.Input().Maybe<TYtReadTable>().IsValid()) { + const auto ytRead = wrapper.Input().Cast<TYtReadTable>(); + const auto readType = ytRead.Ref().GetTypeAnn()->Cast<TTupleExprType>()->GetItems().back(); + const auto inputItemType = NCommon::BuildType(wrapper.Input().Ref(), GetSeqItemType(*readType), ctx.ProgramBuilder); + const auto cluster = ytRead.DataSource().Cluster().StringValue(); + size_t isRPC = state->Configuration->UseRPCReaderInDQ.Get(cluster).GetOrElse(DEFAULT_USE_RPC_READER_IN_DQ) ? state->Configuration->DQRPCReaderInflight.Get(cluster).GetOrElse(DEFAULT_RPC_READER_INFLIGHT) : 0; + size_t timeout = state->Configuration->DQRPCReaderTimeout.Get(cluster).GetOrElse(DEFAULT_RPC_READER_TIMEOUT).MilliSeconds(); + const auto outputType = NCommon::BuildType(wrapper.Ref(), *wrapper.Ref().GetTypeAnn(), ctx.ProgramBuilder); + TString tokenName; + if (auto secureParams = wrapper.Token()) { + tokenName = secureParams.Cast().Name().StringValue(); + } + + bool solid = false; + for (const auto& flag : wrapper.Flags()) + if (solid = flag.Value() == "Solid") + break; + + if (solid) + return BuildDqYtInputCall<false>(outputType, inputItemType, cluster, tokenName, ytRead.Input(), state, ctx, isRPC > 0 ? 1 : 0, timeout, false); + else + return BuildDqYtInputCall<true>(outputType, inputItemType, cluster, tokenName, ytRead.Input(), state, ctx, isRPC, timeout, false); + } + + return TRuntimeNode(); + }); + + compiler.AddCallable(TYtDqWideWrite::CallableName(), + [](const TExprNode& node, NCommon::TMkqlBuildContext& ctx) { + const auto write = TYtDqWideWrite(&node); + const auto outType = NCommon::BuildType(write.Ref(), *write.Ref().GetTypeAnn(), ctx.ProgramBuilder); + const auto arg = MkqlBuildExpr(write.Input().Ref(), ctx); + + TString server{GetSetting(write.Settings().Ref(), "server")->Child(1)->Content()}; + TString table{GetSetting(write.Settings().Ref(), "table")->Child(1)->Content()}; + TString outSpec{GetSetting(write.Settings().Ref(), "outSpec")->Child(1)->Content()}; + auto secureSetting = GetSetting(write.Settings().Ref(), "secureParams"); + TString writerOptions{GetSetting(write.Settings().Ref(), "writerOptions")->Child(1)->Content()}; + + TString tokenName; + if (secureSetting->ChildrenSize() > 1) { + TCoSecureParam secure(secureSetting->Child(1)); + tokenName = secure.Name().StringValue(); + } + + TCallableBuilder call(ctx.ProgramBuilder.GetTypeEnvironment(), "YtDqRowsWideWrite", outType); + call.Add(arg); + call.Add(ctx.ProgramBuilder.NewDataLiteral<NUdf::EDataSlot::String>(server)); + call.Add(ctx.ProgramBuilder.NewDataLiteral<NUdf::EDataSlot::String>(tokenName)); + call.Add(ctx.ProgramBuilder.NewDataLiteral<NUdf::EDataSlot::String>(table)); + call.Add(ctx.ProgramBuilder.NewDataLiteral<NUdf::EDataSlot::String>(outSpec)); + call.Add(ctx.ProgramBuilder.NewDataLiteral<NUdf::EDataSlot::String>(writerOptions)); + + return TRuntimeNode(call.Build(), false); + }); +} + +} diff --git a/yt/yql/providers/yt/provider/yql_yt_mkql_compiler.h b/yt/yql/providers/yt/provider/yql_yt_mkql_compiler.h new file mode 100644 index 0000000000..61d12e44f0 --- /dev/null +++ b/yt/yql/providers/yt/provider/yql_yt_mkql_compiler.h @@ -0,0 +1,41 @@ +#pragma once + +#include "yql_yt_provider.h" + +#include <yql/essentials/providers/common/mkql/yql_provider_mkql.h> + +#include <util/generic/strbuf.h> +#include <util/generic/string.h> +#include <util/generic/vector.h> +#include <util/generic/maybe.h> + +#include <utility> + +namespace NYql { + +class TExprNode; +class TTypeAnnotationNode; + +NKikimr::NMiniKQL::TRuntimeNode BuildTableContentCall(NKikimr::NMiniKQL::TType* outItemType, + TStringBuf clusterName, + const TExprNode& input, + const TMaybe<ui64>& itemsCount, + NCommon::TMkqlBuildContext& ctx, + bool forceColumns, + const THashSet<TString>& extraSysColumns = {}, + bool forceKeyColumns = false); + +NKikimr::NMiniKQL::TRuntimeNode BuildTableContentCall(TStringBuf callName, + NKikimr::NMiniKQL::TType* outItemType, + TStringBuf clusterName, + const TExprNode& input, + const TMaybe<ui64>& itemsCount, + NCommon::TMkqlBuildContext& ctx, + bool forceColumns, + const THashSet<TString>& extraSysColumns = {}, + bool forceKeyColumns = false); + +void RegisterYtMkqlCompilers(NCommon::TMkqlCallableCompilerBase& compiler); +void RegisterDqYtMkqlCompilers(NCommon::TMkqlCallableCompilerBase& compiler, const TYtState::TPtr& state); + +} diff --git a/yt/yql/providers/yt/provider/yql_yt_op_hash.cpp b/yt/yql/providers/yt/provider/yql_yt_op_hash.cpp new file mode 100644 index 0000000000..c05b7fae39 --- /dev/null +++ b/yt/yql/providers/yt/provider/yql_yt_op_hash.cpp @@ -0,0 +1,267 @@ +#include "yql_yt_helpers.h" +#include "yql_yt_op_hash.h" +#include "yql_yt_op_settings.h" + +#include <yt/yql/providers/yt/expr_nodes/yql_yt_expr_nodes.h> +#include <yt/yql/providers/yt/lib/hash/yql_hash_builder.h> + +#include <yql/essentials/core/dq_expr_nodes/dq_expr_nodes.h> +#include <yql/essentials/utils/log/log.h> + +#include <yql/essentials/utils/yql_panic.h> + +#include <library/cpp/yson/node/node_io.h> + +#include <util/string/cast.h> +#include <util/string/hex.h> + +namespace NYql { + +using namespace NNodes; +using namespace NNodes::NDq; + +TYtNodeHashCalculator::TYtNodeHashCalculator(const TYtState::TPtr& state, const TString& cluster, const TYtSettings::TConstPtr& config) + : TNodeHashCalculator(*state->Types, state->NodeHash, MakeSalt(config, cluster)) + , DontFailOnMissingParentOpHash(state->Types->EvaluationInProgress) + , State(state) + , Cluster(cluster) + , Configuration(config) +{ + auto opHasher = [this] (const TExprNode& node, TArgIndex& argIndex, ui32 frameLevel) { + return GetOperationHash(node, argIndex, frameLevel); + }; + Hashers[TYtReadTable::CallableName()] = opHasher; + Hashers[TYtFill::CallableName()] = opHasher; + Hashers[TYtSort::CallableName()] = opHasher; + Hashers[TYtMerge::CallableName()] = opHasher; + Hashers[TYtMap::CallableName()] = opHasher; + Hashers[TYtReduce::CallableName()] = opHasher; + Hashers[TYtMapReduce::CallableName()] = opHasher; + Hashers[TYtCopy::CallableName()] = opHasher; + Hashers[TYtTouch::CallableName()] = opHasher; + Hashers[TYtDqProcessWrite::CallableName()] = [this] (const TExprNode& node, TArgIndex& argIndex, ui32 frameLevel) { + if (const auto flags = TYtDqProcessWrite(&node).Flags()) { + // Only for hybrid for now. + for (const auto& atom : flags.Cast()) { + if (atom.Value() == "FallbackOnError") { + return GetOperationHash(node, argIndex, frameLevel); + } + } + } + return TString(); + }; + + if (State->DqHelper) { + Hashers[TDqStage::CallableName()] = [this] (const TExprNode& node, TArgIndex& argIndex, ui32 frameLevel) { + THashBuilder builder; + builder << node.Content(); + for (size_t i = 0; i < node.ChildrenSize(); ++i) { + if (i == TDqStageBase::idx_Settings) { + // skip _logical_id setting from hashing + const auto& settings = State->DqHelper->RemoveVariadicDqStageSettings(*node.Child(i)); + for (const auto& s: settings) { + if (auto partHash = GetHashImpl(*s, argIndex, frameLevel)) { + builder << partHash; + } + else { + return TString(); + } + } + } else { + if (auto partHash = GetHashImpl(*node.Child(i), argIndex, frameLevel)) { + builder << partHash; + } + else { + return TString(); + } + } + } + return builder.Finish(); + }; + } + + Hashers[TYtOutput::CallableName()] = [this] (const TExprNode& node, TArgIndex& argIndex, ui32 frameLevel) { + return GetOutputHash(node, argIndex, frameLevel); + }; + Hashers[TYtOutTable::CallableName()] = [this] (const TExprNode& node, TArgIndex& argIndex, ui32 frameLevel) { + return GetOutTableHash(node, argIndex, frameLevel); + }; + Hashers[TYtTable::CallableName()] = [this] (const TExprNode& node, TArgIndex& argIndex, ui32 frameLevel) { + return GetTableHash(node, argIndex, frameLevel); + }; + + Hashers[TYtStat::CallableName()] = [this] (const TExprNode& node, TArgIndex& argIndex, ui32 frameLevel) { + Y_UNUSED(argIndex); + Y_UNUSED(frameLevel); + + TYtTableStatInfo stat{TExprBase(&node)}; + + const bool ignoreRevision = Configuration->QueryCacheIgnoreTableRevision.Get().GetOrElse(false); + + THashBuilder builder; + builder << stat.Id; + if (!ignoreRevision) { + builder << stat.Revision; + // TODO remove after https://st.yandex-team.ru/YT-9914 + builder << stat.ChunkCount; + builder << stat.DataSize; + builder << stat.RecordsCount; + } + + return builder.Finish(); + }; +} + +TString TYtNodeHashCalculator::MakeSalt(const TYtSettings::TConstPtr& config, const TString& cluster) { + auto salt = config->QueryCacheSalt.Get().GetOrElse(TString()); + THashBuilder builder; + bool update = false; + if (auto val = config->LayerPaths.Get(cluster)) { + update = true; + for (const auto& path : *val) { + builder << path; + } + } + if (auto val = config->DockerImage.Get(cluster)) { + update = true; + builder << *val; + } + if (auto val = config->NativeYtTypeCompatibility.Get(cluster)) { + update = true; + builder << *val; + } + if (auto val = config->OptimizeFor.Get(cluster)) { + update = true; + builder << *val; + } + if (update) { + builder << salt; + salt = builder.Finish(); + } + return salt; +} + +TString TYtNodeHashCalculator::GetOutputHash(const TExprNode& node, TArgIndex& argIndex, ui32 frameLevel) const { + Y_UNUSED(argIndex); + Y_UNUSED(frameLevel); + + auto output = TYtOutput(&node); + auto opUniqueId = GetOutputOp(output).Ref().UniqueId(); + auto it = NodeHash.find(opUniqueId); + if (it == NodeHash.end()) { + if (DontFailOnMissingParentOpHash) { + return TString(); + } + YQL_ENSURE(false, "Cannot find hash for operation " << GetOutputOp(output).Ref().Content() + << ", #" << opUniqueId); + } + if (it->second.empty()) { + return TString(); + } + return (THashBuilder() << it->second << FromString<size_t>(output.OutIndex().Value())).Finish(); +} + +TString TYtNodeHashCalculator::GetOutTableHash(const TExprNode& node, TArgIndex& argIndex, ui32 frameLevel) const { + auto opHash = NYql::GetSetting(*node.Child(TYtOutTable::idx_Settings), EYtSettingType::OpHash); + if (opHash && opHash->Child(1)->Content().empty()) { + return TString(); + } + THashBuilder builder; + builder << node.Content(); + for (size_t i = 0; i < node.ChildrenSize(); ++i) { + if (i != TYtOutTable::idx_Name && i != TYtOutTable::idx_Stat && i != TYtOutTable::idx_Settings) { + if (auto partHash = GetHashImpl(*node.Child(i), argIndex, frameLevel)) { + builder << partHash; + } + else { + return TString(); + } + } + } + + for (auto& setting : node.Child(TYtOutTable::idx_Settings)->Children()) { + if (auto partHash = GetHashImpl(*setting, argIndex, frameLevel)) { + builder << partHash; + } + else { + return TString(); + } + } + + // OpHash is present for substitutions YtOutput->YtOutTable. It already includes publish-related hashes from the parent operation + if (!opHash) { + if (auto compressionCodec = Configuration->TemporaryCompressionCodec.Get(Cluster)) { + builder << *compressionCodec; + } + if (auto erasureCodec = Configuration->TemporaryErasureCodec.Get(Cluster)) { + builder << ToString(*erasureCodec); + } + if (auto media = Configuration->TemporaryMedia.Get(Cluster)) { + builder << NYT::NodeToYsonString(*media); + } + if (auto primaryMedium = Configuration->TemporaryPrimaryMedium.Get(Cluster)) { + builder << *primaryMedium; + } + if (auto replicationFactor = Configuration->TemporaryReplicationFactor.Get(Cluster)) { + builder << *replicationFactor; + } + if (auto optimizeFor = Configuration->OptimizeFor.Get(Cluster)) { + builder << *optimizeFor; + } + } + + return builder.Finish(); +} + +TString TYtNodeHashCalculator::GetTableHash(const TExprNode& node, TArgIndex& argIndex, ui32 frameLevel) const { + TYtTable ytTable{ &node }; + auto epoch = TEpochInfo::Parse(ytTable.Epoch().Ref()).GetOrElse(0); + const auto tableLabel = TString(TYtTableInfo::GetTableLabel(ytTable)); + const auto& tableDescription = State->TablesData->GetTable(TString(ytTable.Cluster().Value()), tableLabel, epoch); + if (tableDescription.Hash) { + if (!tableDescription.Hash.Empty()) { + YQL_CLOG(INFO, ProviderYt) << "Use hash " << HexEncode(*tableDescription.Hash) << " for table " << + ytTable.Cluster().Value() << "." << tableLabel << "#" << epoch; + } + + return *tableDescription.Hash; + } + + const bool ignoreRevision = Configuration->QueryCacheIgnoreTableRevision.Get().GetOrElse(false); + if (!ignoreRevision && TYtTableMetaInfo(ytTable.Meta()).IsDynamic) { + return TString(); + } + + THashBuilder builder; + builder << node.Content(); + builder << tableLabel; + for (size_t i = 0; i < node.ChildrenSize(); ++i) { + if (i != TYtTable::idx_Name && i != TYtTable::idx_Meta) { + if (auto partHash = GetHashImpl(*node.Child(i), argIndex, frameLevel)) { + builder << partHash; + } + else { + return TString(); + } + } + } + return builder.Finish(); +} + +TString TYtNodeHashCalculator::GetOperationHash(const TExprNode& node, TArgIndex& argIndex, ui32 frameLevel) const { + THashBuilder builder; + builder << node.Content(); + for (size_t i = 0; i < node.ChildrenSize(); ++i) { + if (i != TYtReadTable::idx_World) { + if (auto partHash = GetHashImpl(*node.Child(i), argIndex, frameLevel)) { + builder << partHash; + } + else { + return TString(); + } + } + } + return builder.Finish(); +} + +} // NYql diff --git a/yt/yql/providers/yt/provider/yql_yt_op_hash.h b/yt/yql/providers/yt/provider/yql_yt_op_hash.h new file mode 100644 index 0000000000..91d6aeaadc --- /dev/null +++ b/yt/yql/providers/yt/provider/yql_yt_op_hash.h @@ -0,0 +1,28 @@ +#pragma once + +#include "yql_yt_provider.h" + +#include <yt/yql/providers/yt/lib/hash/yql_op_hash.h> + +namespace NYql { + +class TYtNodeHashCalculator: public TNodeHashCalculator { +public: + TYtNodeHashCalculator(const TYtState::TPtr& state, const TString& cluster, const TYtSettings::TConstPtr& config); + + static TString MakeSalt(const TYtSettings::TConstPtr& config, const TString& cluster); + +private: + TString GetOutputHash(const TExprNode& node, TArgIndex& argIndex, ui32 frameLevel) const; + TString GetOutTableHash(const TExprNode& node, TArgIndex& argIndex, ui32 frameLevel) const; + TString GetTableHash(const TExprNode& node, TArgIndex& argIndex, ui32 frameLevel) const; + TString GetOperationHash(const TExprNode& node, TArgIndex& argIndex, ui32 frameLevel) const; + +private: + const bool DontFailOnMissingParentOpHash; + const TYtState::TPtr State; + const TString Cluster; + const TYtSettings::TConstPtr Configuration; +}; + +} // NYql diff --git a/yt/yql/providers/yt/provider/yql_yt_op_settings.cpp b/yt/yql/providers/yt/provider/yql_yt_op_settings.cpp new file mode 100644 index 0000000000..2f9619fb16 --- /dev/null +++ b/yt/yql/providers/yt/provider/yql_yt_op_settings.cpp @@ -0,0 +1,1270 @@ +#include "yql_yt_op_settings.h" + +#include <yt/yql/providers/yt/lib/expr_traits/yql_expr_traits.h> +#include <yt/yql/providers/yt/common/yql_yt_settings.h> +#include <yql/essentials/providers/common/provider/yql_provider.h> +#include <yql/essentials/core/yql_expr_type_annotation.h> +#include <yql/essentials/core/yql_opt_utils.h> + +#include <util/string/cast.h> +#include <util/generic/hash_set.h> + +#include <library/cpp/yson/node/node_io.h> + + +namespace NYql { + +namespace { + +bool ValidateColumnSettings(TExprNode& columnsSettings, TExprContext& ctx, TVector<TString>& columns, bool maybeEmpty) { + if (maybeEmpty) { + if (!EnsureTuple(columnsSettings, ctx)) { + return false; + } + } else { + if (!EnsureTupleMinSize(columnsSettings, 1, ctx)) { + return false; + } + } + + for (auto& child : columnsSettings.Children()) { + if (!EnsureAtom(*child, ctx)) { + return false; + } + columns.push_back(TString(child->Content())); + } + return true; +} + +bool ValidateColumnPairSettings(TExprNode& columnsSettings, TExprContext& ctx, TVector<TString>& columns) { + if (!EnsureTupleMinSize(columnsSettings, 1, ctx)) { + return false; + } + + for (auto& child : columnsSettings.Children()) { + if (child->IsList()) { + if (!EnsureTupleSize(*child, 2, ctx)) { + return false; + } + if (!EnsureAtom(*child->Child(0), ctx)) { + return false; + } + if (!EnsureAtom(*child->Child(1), ctx)) { + return false; + } + bool res = false; + if (!TryFromString(child->Child(1)->Content(), res)) { + ctx.AddError(TIssue(ctx.GetPosition(child->Child(1)->Pos()), TStringBuilder() << "Expected bool value, but got: " << child->Child(1)->Content())); + return false; + } + columns.emplace_back(child->Child(0)->Content()); + } else { + if (HasError(child->GetTypeAnn(), ctx)) { + return false; + } + + if (!child->IsAtom()) { + ctx.AddError(TIssue(ctx.GetPosition(child->Pos()), TStringBuilder() << "Expected tuple or atom, but got: " << child->Type())); + return false; + } + columns.emplace_back(child->Content()); + } + } + return true; +} + +bool ValidateColumnSubset(const TPositionHandle& pos, TExprContext& ctx, const TVector<TString>& sortBy, const TVector<TString>& reduceBy) { + + if (sortBy.size() < reduceBy.size()) { + ctx.AddError(TIssue(ctx.GetPosition(pos), "reduceBy column list must be prefix of sortBy column list")); + return false; + } + for (size_t i = 0; i < reduceBy.size(); ++i) { + if (sortBy[i] != reduceBy[i]) { + ctx.AddError(TIssue(ctx.GetPosition(pos), "reduceBy column list must be prefix of sortBy column list")); + return false; + } + } + return true; +} + +const THashSet<TStringBuf> SYS_COLUMNS = { + "path", + "record", + "index", + "num", +}; + +} // unnamed + +bool ValidateSettings(const TExprNode& settingsNode, EYtSettingTypes accepted, TExprContext& ctx) { + TMaybe<TVector<TString>> sortBy; + TMaybe<TVector<TString>> reduceBy; + bool isTruncate = true; + EYtSettingTypes used; + for (auto setting: settingsNode.Children()) { + if (!EnsureTupleMinSize(*setting, 1, ctx)) { + return false; + } + auto nameNode = setting->Child(0); + if (!EnsureAtom(*nameNode, ctx)) { + return false; + } + EYtSettingType type; + if (!TryFromString(nameNode->Content(), type)) { + ctx.AddError(TIssue(ctx.GetPosition(nameNode->Pos()), TStringBuilder() << "Unknown setting name: " << nameNode->Content())); + return false; + } + if (!accepted.HasFlags(type)) { + ctx.AddError(TIssue(ctx.GetPosition(nameNode->Pos()), TStringBuilder() << "Setting " << nameNode->Content() << " is not accepted here")); + return false; + } + used |= type; + + TIssueScopeGuard issueScope(ctx.IssueManager, [setting, type, &ctx]() { + return MakeIntrusive<TIssue>(ctx.GetPosition(setting->Pos()), TStringBuilder() << "Setting " << type); + }); + + switch (type) { + case EYtSettingType::KeyFilter: { + if (!EnsureTupleSize(*setting, 2, ctx)) { + return false; + } + auto value = setting->Child(1); + if (!EnsureTupleMaxSize(*value, 2, ctx)) { + return false; + } + if (value->ChildrenSize() > 0) { + if (value->ChildrenSize() == 2) { + if (!EnsureAtom(*value->Child(1), ctx)) { + return false; + } + size_t tableIndex = 0; + if (!TryFromString(value->Child(1)->Content(), tableIndex)) { + ctx.AddError(TIssue(ctx.GetPosition(value->Child(1)->Pos()), TStringBuilder() + << "Bad table index value: " << value->Child(1)->Content())); + return false; + } + } + auto andList = value->Child(0); + if (!EnsureTupleMinSize(*andList, 1, ctx)) { + return false; + } + for (auto keyPredicates: andList->Children()) { + if (!EnsureTupleSize(*keyPredicates, 2, ctx)) { + return false; + } + + if (!EnsureAtom(*keyPredicates->Child(0), ctx)) { + return false; + } + if (keyPredicates->Child(0)->Content().empty()) { + ctx.AddError(TIssue(ctx.GetPosition(keyPredicates->Child(0)->Pos()), "Column name should be a non-empty atom")); + return false; + } + if (!EnsureTupleMinSize(*keyPredicates->Child(1), 1, ctx)) { + return false; + } + + for (auto cmp: keyPredicates->Child(1)->Children()) { + if (!EnsureTupleSize(*cmp, 2, ctx)) { + return false; + } + + if (!EnsureAtom(*cmp->Child(0), ctx)) { + return false; + } + if (cmp->Child(0)->Content().empty()) { + ctx.AddError(TIssue(ctx.GetPosition(cmp->Child(0)->Pos()), "Comparison should be a non-empty atom")); + return false; + } + if (!IsRangeComparison(cmp->Child(0)->Content())) { + ctx.AddError(TIssue(ctx.GetPosition(cmp->Child(0)->Pos()), TStringBuilder() + << "Unsupported compare operation " + << TString{cmp->Child(0)->Content()}.Quote())); + return false; + } + + if (!IsNull(*cmp->Child(1))) { + bool isOptional = false; + const TDataExprType* dataType = nullptr; + if (!EnsureDataOrOptionalOfData(*cmp->Child(1), isOptional, dataType, ctx)) { + return false; + } + if (!EnsureComparableDataType(cmp->Child(1)->Pos(), dataType->GetSlot(), ctx)) { + return false; + } + } + } + } + } + break; + } + case EYtSettingType::KeyFilter2: { + if (!EnsureTupleSize(*setting, 2, ctx)) { + return false; + } + auto value = setting->Child(1); + if (!EnsureTupleMaxSize(*value, 3, ctx)) { + return false; + } + if (value->ChildrenSize() > 0) { + if (!EnsureTupleMinSize(*value, 2, ctx)) { + return false; + } + if (value->ChildrenSize() == 3) { + if (!EnsureTupleOfAtoms(value->Tail(), ctx)) { + return false; + } + + THashSet<size_t> indexes; + for (auto& indexNode : value->Tail().ChildrenList()) { + size_t tableIndex = 0; + if (!TryFromString(indexNode->Content(), tableIndex)) { + ctx.AddError(TIssue(ctx.GetPosition(indexNode->Pos()), TStringBuilder() + << "Bad table index value: " << indexNode->Content())); + return false; + } + if (!indexes.insert(tableIndex).second) { + ctx.AddError(TIssue(ctx.GetPosition(indexNode->Pos()), TStringBuilder() + << "Duplicate table index value: " << tableIndex)); + return false; + } + } + } + + auto& computeNode = value->Head(); + const TTypeAnnotationNode* computeNodeType = computeNode.GetTypeAnn(); + YQL_ENSURE(computeNodeType); + if (!EnsureListType(computeNode.Pos(), *computeNodeType, ctx)) { + return false; + } + + auto computeNodeListItemType = computeNodeType->Cast<TListExprType>()->GetItemType(); + if (!EnsureTupleTypeSize(computeNode.Pos(), computeNodeListItemType, 2, ctx)) { + return false; + } + + auto computeNodeTupleItems = computeNodeListItemType->Cast<TTupleExprType>()->GetItems(); + YQL_ENSURE(computeNodeTupleItems.size() == 2); + if (!IsSameAnnotation(*computeNodeTupleItems[0], *computeNodeTupleItems[1])) { + ctx.AddError(TIssue(ctx.GetPosition(computeNode.Pos()), TStringBuilder() + << "Expecting compute expression to return list of 2-element tuples of same type, got: " + << *computeNodeType)); + return false; + } + + if (!EnsureTupleType(computeNode.Pos(), *computeNodeTupleItems.front(), ctx)) { + return false; + } + + auto rangeTypeItems = computeNodeTupleItems.front()->Cast<TTupleExprType>()->GetItems(); + if (rangeTypeItems.size() < 2) { + ctx.AddError(TIssue(ctx.GetPosition(computeNode.Pos()), TStringBuilder() + << "Expecting range boundary to be a tuple with at least two elements, got: " + << *computeNodeTupleItems.front())); + return false; + } + + if (!EnsureSpecificDataType(computeNode.Pos(), *rangeTypeItems.back(), EDataSlot::Int32,ctx)) { + return false; + } + + for (size_t i = 0; i + 1 < rangeTypeItems.size(); ++i) { + auto boundaryType = rangeTypeItems[i]; + YQL_ENSURE(boundaryType); + if (!EnsureOptionalType(computeNode.Pos(), *boundaryType, ctx)) { + return false; + } + + bool isOptional; + const TDataExprType* data = nullptr; + if (!EnsureDataOrOptionalOfData(computeNode.Pos(), RemoveOptionalType(boundaryType), isOptional, data, ctx)) { + return false; + } + + if (!EnsureComparableDataType(computeNode.Pos(), data->GetSlot(), ctx)) { + return false; + } + } + + auto& settingsNode = *value->Child(1); + if (!EnsureTuple(settingsNode, ctx)) { + return false; + } + + for (auto& setting : settingsNode.ChildrenList()) { + if (!EnsureTupleMinSize(*setting, 1, ctx)) { + return false; + } + + if (!EnsureAtom(setting->Head(), ctx)) { + return false; + } + + auto name = setting->Head().Content(); + if (name == "usedKeys") { + if (!EnsureTupleSize(*setting, 2, ctx)) { + return false; + } + + if (!EnsureTupleMinSize(setting->Tail(), 1, ctx)) { + return false; + } + + THashSet<TStringBuf> keys; + for (auto& key : setting->Tail().ChildrenList()) { + if (!EnsureAtom(*key, ctx)) { + return false; + } + + if (!keys.insert(key->Content()).second) { + ctx.AddError(TIssue(ctx.GetPosition(key->Pos()), TStringBuilder() + << "Duplicate key column name: " << key->Content())); + return false; + } + } + } else { + ctx.AddError(TIssue(ctx.GetPosition(setting->Head().Pos()), TStringBuilder() + << "Unknown option : '" << setting->Head().Content() << "' for " + << ToString(EYtSettingType::KeyFilter).Quote())); + return false; + } + } + } + break; + } + case EYtSettingType::Take: + case EYtSettingType::Skip: + if (!EnsureTupleSize(*setting, 2, ctx)) { + return false; + } + if (!EnsureSpecificDataType(*setting->Child(1), EDataSlot::Uint64, ctx)) { + return false; + } + break; + case EYtSettingType::UserSchema: + case EYtSettingType::UserColumns: { + if (!EnsureTupleSize(*setting, 2, ctx)) { + return false; + } + if (setting->Child(1)->GetTypeAnn()) { + if (!EnsureType(*setting->Child(1), ctx)) { + return false; + } + auto type = setting->Child(1)->GetTypeAnn()->Cast<TTypeExprType>()->GetType(); + if (!EnsureStructType(setting->Child(1)->Pos(), *type, ctx)) { + return false; + } + + if (!EnsurePersistableType(setting->Child(1)->Pos(), *type, ctx)) { + return false; + } + } + + break; + } + case EYtSettingType::StatColumns: { + if (!EnsureTupleSize(*setting, 2, ctx)) { + return false; + } + TVector<TString> columns; + const bool maybeEmpty = true; + if (!ValidateColumnSettings(*setting->Child(1), ctx, columns, maybeEmpty)) { + return false; + } + break; + } + case EYtSettingType::SysColumns: { + if (!EnsureTupleSize(*setting, 2, ctx)) { + return false; + } + TVector<TString> columns; + const bool maybeEmpty = false; + if (!ValidateColumnSettings(*setting->Child(1), ctx, columns, maybeEmpty)) { + return false; + } + for (auto col: columns) { + if (!SYS_COLUMNS.contains(col)) { + ctx.AddError(TIssue(ctx.GetPosition(setting->Child(1)->Pos()), TStringBuilder() + << "Unsupported system column " << col.Quote())); + return false; + } + } + break; + } + case EYtSettingType::InferScheme: + case EYtSettingType::ForceInferScheme: + if (!EnsureTupleMinSize(*setting, 1, ctx)) { + return false; + } + if (!EnsureTupleMaxSize(*setting, 2, ctx)) { + return false; + } + if (setting->ChildrenSize() == 2) { + if (!EnsureAtom(setting->Tail(), ctx)) { + return false; + } + ui32 val; + if (!TryFromString(setting->Tail().Content(), val) || val < 1 || val > 1000) { + ctx.AddError(TIssue(ctx.GetPosition(setting->Tail().Pos()), TStringBuilder() + << "Bad value " << TString{setting->Tail().Content()}.Quote())); + return false; + } + } + break; + case EYtSettingType::Ordered: + case EYtSettingType::Initial: + case EYtSettingType::DoNotFailOnInvalidSchema: + case EYtSettingType::DirectRead: + case EYtSettingType::Scheme: + case EYtSettingType::WeakConcat: + case EYtSettingType::IgnoreNonExisting: + case EYtSettingType::WarnNonExisting: + case EYtSettingType::ForceTransform: + case EYtSettingType::TransformColGroups: + case EYtSettingType::CombineChunks: + case EYtSettingType::WithQB: + case EYtSettingType::Inline: + case EYtSettingType::WeakFields: + case EYtSettingType::Sharded: + case EYtSettingType::XLock: + case EYtSettingType::JoinReduce: + case EYtSettingType::Unordered: + case EYtSettingType::NonUnique: + case EYtSettingType::KeepSorted: + case EYtSettingType::KeySwitch: + case EYtSettingType::IgnoreTypeV3: + case EYtSettingType::NoDq: + case EYtSettingType::Split: + case EYtSettingType::KeepMeta: + case EYtSettingType::MonotonicKeys: + case EYtSettingType::BlockInputReady: + case EYtSettingType::BlockInputApplied: + case EYtSettingType::Small: + if (!EnsureTupleSize(*setting, 1, ctx)) { + return false; + } + break; + case EYtSettingType::Flow: + case EYtSettingType::Anonymous: + if (!EnsureTupleMinSize(*setting, 1, ctx)) { + return false; + } + if (!EnsureTupleMaxSize(*setting, 2, ctx)) { + return false; + } + if (setting->ChildrenSize() == 2) { + if (!EnsureAtom(*setting->Child(1), ctx)) { + return false; + } + } + break; + case EYtSettingType::FirstAsPrimary: + if (!EnsureTupleMinSize(*setting, 1, ctx)) { + return false; + } + if (!EnsureTupleMaxSize(*setting, 2, ctx)) { + return false; + } + if (setting->ChildrenSize() == 2) { + for (auto childSetting : setting->Child(1)->Children()) { + if (!EnsureTupleMinSize(*childSetting, 1, ctx)) { + return false; + } + if (!EnsureAtom(*childSetting->Child(0), ctx)) { + return false; + } + + const auto childSettingName = childSetting->Child(0)->Content(); + if (childSettingName == MaxJobSizeForFirstAsPrimaryName) { + if (!EnsureTupleSize(*childSetting, 2, ctx)) { + return false; + } + ui64 value; + if (!EnsureAtom(*childSetting->Child(1), ctx)) { + return false; + } + if (!TryFromString(childSetting->Child(1)->Content(), value)) { + ctx.AddError(TIssue(ctx.GetPosition(childSetting->Child(1)->Pos()), TStringBuilder() + << "Bad value " << TString{childSetting->Child(1)->Content()}.Quote())); + return false; + } + if (!value) { + ctx.AddError(TIssue(ctx.GetPosition(childSetting->Child(1)->Pos()), TStringBuilder() + << MaxJobSizeForFirstAsPrimaryName << " value should not be zero")); + return false; + } + } else if (childSettingName == JoinReduceForSecondAsPrimaryName) { + if (!EnsureTupleSize(*childSetting, 1, ctx)) { + return false; + } + } else { + ctx.AddError(TIssue(ctx.GetPosition(childSetting->Pos()), TStringBuilder() + << "Unknown " << ToString(EYtSettingType::FirstAsPrimary) << " subsetting " << TString{childSettingName}.Quote())); + return false; + } + } + } + break; + case EYtSettingType::JobCount: + case EYtSettingType::MemUsage: + case EYtSettingType::ItemsCount: + case EYtSettingType::RowFactor: { + ui64 value; + if (!EnsureAtom(*setting->Child(1), ctx)) { + return false; + } + if (!TryFromString(setting->Child(1)->Content(), value)) { + ctx.AddError(TIssue(ctx.GetPosition(setting->Child(1)->Pos()), TStringBuilder() + << "Bad value " << TString{setting->Child(1)->Content()}.Quote())); + return false; + } + if ((EYtSettingType::JobCount == type || EYtSettingType::FirstAsPrimary == type) && 0 == value) { + ctx.AddError(TIssue(ctx.GetPosition(setting->Child(1)->Pos()), "Value should not be zero")); + return false; + } + break; + } + case EYtSettingType::View: + case EYtSettingType::OpHash: + if (!EnsureArgsCount(*setting, 2, ctx)) { + return false; + } + if (!EnsureAtom(*setting->Child(1), ctx)) { + return false; + } + break; + case EYtSettingType::Mode: { + if (!EnsureTupleSize(*setting, 2, ctx)) { + return false; + } + if (!EnsureAtom(*setting->Child(1), ctx)) { + return false; + } + EYtWriteMode mode; + if (!TryFromString(setting->Child(1)->Content(), mode)) { + ctx.AddError(TIssue(ctx.GetPosition(setting->Child(1)->Pos()), TStringBuilder() + << "Unsupported mode value " << TString{setting->Child(1)->Content()}.Quote())); + return false; + } + isTruncate = EYtWriteMode::Renew == mode; + break; + } + case EYtSettingType::Limit: + if (!EnsureTupleSize(*setting, 2, ctx)) { + return false; + } + if (!EnsureTuple(*setting->Child(1), ctx)) { + return false; + } + for (auto expr: setting->Child(1)->Children()) { + if (!EnsureTuple(*expr, ctx)) { + return false; + } + for (auto item: expr->Children()) { + if (!EnsureTupleMinSize(*item, 1, ctx)) { + return false; + } + if (!EnsureAtom(*item->Child(0), ctx)) { + return false; + } + if (FromString<EYtSettingType>(item->Child(0)->Content()) & (EYtSettingType::Take | EYtSettingType::Skip)) { + if (!EnsureArgsCount(*item, 2, ctx)) { + return false; + } + + if (!EnsureSpecificDataType(*item->Child(1), EDataSlot::Uint64, ctx)) { + return false; + } + } else { + ctx.AddError(TIssue(ctx.GetPosition(item->Pos()), TStringBuilder() + << "Unsupported item " << TString{item->Child(0)->Content()}.Quote())); + return false; + } + } + } + break; + case EYtSettingType::ReduceFilterBy: + case EYtSettingType::UniqueBy: { + if (!EnsureTupleSize(*setting, 2, ctx)) { + return false; + } + TVector<TString> columns; + if (!ValidateColumnSettings(*setting->Child(1), ctx, columns, EYtSettingType::ReduceFilterBy == type)) { + return false; + } + break; + } + case EYtSettingType::SortLimitBy: { + if (!EnsureTupleSize(*setting, 2, ctx)) { + return false; + } + TVector<TString> columns; + if (!ValidateColumnPairSettings(*setting->Child(1), ctx, columns)) { + return false; + } + break; + } + case EYtSettingType::SortBy: + if (!EnsureTupleSize(*setting, 2, ctx)) { + return false; + } + + sortBy.ConstructInPlace(); + if (!ValidateColumnPairSettings(*setting->Child(1), ctx, *sortBy)) { + return false; + } + if (reduceBy.Defined() && !ValidateColumnSubset(setting->Child(1)->Pos(), ctx, *sortBy, *reduceBy)) { + return false; + } + break; + case EYtSettingType::ReduceBy: + if (!EnsureTupleSize(*setting, 2, ctx)) { + return false; + } + + reduceBy.ConstructInPlace(); + if (!ValidateColumnPairSettings(*setting->Child(1), ctx, *reduceBy)) { + return false; + } + + if (sortBy.Defined() && !ValidateColumnSubset(setting->Child(1)->Pos(), ctx, *sortBy, *reduceBy)) { + return false; + } + break; + case EYtSettingType::Sample: + if (!EnsureTupleSize(*setting, 2, ctx)) { + return false; + } + if (!EnsureTupleSize(*setting->Child(1), 3, ctx)) { + return false; + } + if (!EnsureAtom(*setting->Child(1)->Child(0), ctx)) { + return false; + } + { + EYtSampleMode sampleMode = EYtSampleMode::System; + if (!TryFromString(setting->Child(1)->Child(0)->Content(), sampleMode)) { + ctx.AddError(TIssue(ctx.GetPosition(setting->Child(1)->Child(0)->Pos()), TStringBuilder() + << "Bad sample mode value: " << setting->Child(1)->Child(0)->Content())); + return false; + } + } + if (!EnsureAtom(*setting->Child(1)->Child(1), ctx)) { + return false; + } + { + double samplePercentage = 0; + if (!TryFromString(setting->Child(1)->Child(1)->Content(), samplePercentage) || + !(samplePercentage >= 0. && samplePercentage <= 100.)) + { + ctx.AddError(TIssue(ctx.GetPosition(setting->Child(1)->Child(1)->Pos()), TStringBuilder() + << "Bad sample percentage value: " << setting->Child(1)->Child(1)->Content())); + return false; + } + } + if (!EnsureAtom(*setting->Child(1)->Child(2), ctx)) { + return false; + } + { + ui64 sampleRepeat = 0; + if (!TryFromString(setting->Child(1)->Child(2)->Content(), sampleRepeat)) { + ctx.AddError(TIssue(ctx.GetPosition(setting->Child(1)->Child(2)->Pos()), TStringBuilder() + << "Bad sample repeat value: " << setting->Child(1)->Child(2)->Content())); + return false; + } + } + break; + case EYtSettingType::JoinLabel: + if (!EnsureTupleSize(*setting, 2, ctx)) { + return false; + } + break; + case EYtSettingType::MapOutputType: + case EYtSettingType::ReduceInputType: + if (!EnsureTupleSize(*setting, 2, ctx)) { + return false; + } + if (!EnsureType(setting->Tail(), ctx)) { + return false; + } + break; + case EYtSettingType::ErasureCodec: { + if (!EnsureTupleSize(*setting, 2, ctx)) { + return false; + } + NYT::EErasureCodecAttr codec; + if (!TryFromString(setting->Tail().Content(), codec)) { + ctx.AddError(TIssue(ctx.GetPosition(setting->Tail().Pos()), TStringBuilder() + << "Unsupported erasure codec value " << TString{setting->Tail().Content()}.Quote())); + return false; + } + break; + } + case EYtSettingType::CompressionCodec: + if (!EnsureTupleSize(*setting, 2, ctx)) { + return false; + } + if (!ValidateCompressionCodecValue(setting->Tail().Content())) { + ctx.AddError(TIssue(ctx.GetPosition(setting->Tail().Pos()), TStringBuilder() + << "Unsupported compression codec value " << TString{setting->Tail().Content()}.Quote())); + return false; + } + break; + case EYtSettingType::Expiration: { + if (!EnsureTupleSize(*setting, 2, ctx) || !EnsureAtom(setting->Tail(), ctx)) { + return false; + } + TInstant ti; + TDuration td; + if (!TInstant::TryParseIso8601(setting->Tail().Content(), ti) && + !TDuration::TryParse(setting->Tail().Content(), td)) + { + ctx.AddError(TIssue(ctx.GetPosition(setting->Tail().Pos()), TStringBuilder() + << "Unsupported Expiration value " << TString{setting->Tail().Content()}.Quote())); + return false; + } + break; + } + case EYtSettingType::ReplicationFactor: { + if (!EnsureTupleSize(*setting, 2, ctx)) { + return false; + } + if (!EnsureAtom(setting->Tail(), ctx)) { + return false; + } + ui32 replicationFactor; + if (!TryFromString(setting->Tail().Content(), replicationFactor) || + replicationFactor < 1 || replicationFactor > 10) + { + ctx.AddError(TIssue(ctx.GetPosition(setting->Tail().Pos()), TStringBuilder() + << "Unsupported Replication Factor value:" + << TString{setting->Tail().Content()}.Quote())); + return false; + } + break; + } + case EYtSettingType::UserAttrs: { + if (!EnsureTupleSize(*setting, 2, ctx)) { + return false; + } + if (!EnsureAtom(setting->Tail(), ctx)) { + return false; + } + NYT::TNode mapNode; + try { + mapNode = NYT::NodeFromYsonString(setting->Tail().Content()); + } catch (const std::exception& e) { + ctx.AddError(TIssue(ctx.GetPosition(setting->Tail().Pos()), TStringBuilder() + << "Failed to parse Yson: " << e.what())); + return false; + } + if (!mapNode.IsMap()) { + ctx.AddError(TIssue(ctx.GetPosition(setting->Tail().Pos()), TStringBuilder() + << "Expected Yson map, got: " << mapNode.GetType())); + return false; + } + const auto& map = mapNode.AsMap(); + for (auto it = map.cbegin(); it != map.cend(); ++it) { + if (!it->second.HasValue()) { + ctx.AddError(TIssue(ctx.GetPosition(setting->Tail().Pos()), TStringBuilder() + << "Expected Yson map key having value: " + << it->first.Quote())); + return false; + } + } + break; + } + case EYtSettingType::Media: + if (!EnsureTupleSize(*setting, 2, ctx)) { + return false; + } + if (!EnsureAtom(setting->Tail(), ctx)) { + return false; + } + try { + MediaValidator(NYT::NodeFromYsonString(setting->Tail().Content())); + } catch (const std::exception& e) { + ctx.AddError(TIssue(ctx.GetPosition(setting->Tail().Pos()), TStringBuilder() + << "Incorrect media: " << e.what())); + return false; + } + break; + case EYtSettingType::PrimaryMedium: + if (!EnsureTupleSize(*setting, 2, ctx)) { + return false; + } + if (!EnsureAtom(setting->Tail(), ctx)) { + return false; + } + break; + case EYtSettingType::MutationId: + if (!EnsureTupleSize(*setting, 2, ctx)) { + return false; + } + if (!EnsureAtom(*setting->Child(1), ctx)) { + return false; + } + ui32 mutationId; + if (!TryFromString(setting->Child(1)->Content(), mutationId)) { + ctx.AddError(TIssue(ctx.GetPosition(setting->Child(1)->Pos()), TStringBuilder() + << "Expected a number, but got: " << TString{setting->Child(1)->Content()}.Quote())); + return false; + } + break; + case EYtSettingType::ColumnGroups: { + if (!EnsureTupleSize(*setting, 2, ctx)) { + return false; + } + if (!EnsureAtom(setting->Tail(), ctx)) { + return false; + } + NYT::TNode mapNode; + try { + mapNode = NYT::NodeFromYsonString(setting->Tail().Content()); + } catch (const std::exception& e) { + ctx.AddError(TIssue(ctx.GetPosition(setting->Tail().Pos()), TStringBuilder() + << "Failed to parse Yson: " << e.what())); + return false; + } + if (!mapNode.IsMap()) { + ctx.AddError(TIssue(ctx.GetPosition(setting->Tail().Pos()), TStringBuilder() + << "Expected Yson map, got: " << mapNode.GetType())); + return false; + } + bool hasDef = false; + std::unordered_set<TString> uniqColumns; + const auto& map = mapNode.AsMap(); + for (auto it = map.cbegin(); it != map.cend(); ++it) { + if (it->second.IsEntity()) { + if (hasDef) { + ctx.AddError(TIssue(ctx.GetPosition(setting->Tail().Pos()), TStringBuilder() + << "Not more than one group should have # value: " + << it->first.Quote())); + return false; + } + hasDef = true; + } else if (!it->second.IsList()) { + ctx.AddError(TIssue(ctx.GetPosition(setting->Tail().Pos()), TStringBuilder() + << "Expected list value, group: " + << it->first.Quote())); + return false; + } else if (it->second.AsList().size() < 2) { + ctx.AddError(TIssue(ctx.GetPosition(setting->Tail().Pos()), TStringBuilder() + << "Expected list with at least two columns, group: " + << it->first.Quote())); + return false; + } else { + for (const auto& item: it->second.AsList()) { + if (!item.IsString()) { + ctx.AddError(TIssue(ctx.GetPosition(setting->Tail().Pos()), TStringBuilder() + << "Expected string value in list, found " + << item.GetType() << ", group: " << it->first.Quote())); + return false; + } + if (!uniqColumns.insert(item.AsString()).second) { + ctx.AddError(TIssue(ctx.GetPosition(setting->Tail().Pos()), TStringBuilder() + << "Duplicate column " << item.AsString().Quote())); + return false; + } + } + } + } + break; + } + case EYtSettingType::SecurityTags: { + if (!EnsureTupleSize(*setting, 2, ctx)) { + return false; + } + if (!EnsureAtom(setting->Tail(), ctx)) { + return false; + } + NYT::TNode securityTagsNode; + try { + securityTagsNode = NYT::NodeFromYsonString(setting->Tail().Content()); + } catch (const std::exception& e) { + ctx.AddError(TIssue(ctx.GetPosition(setting->Tail().Pos()), TStringBuilder() + << "Failed to parse Yson: " << e.what())); + return false; + } + if (!securityTagsNode.IsList()) { + ctx.AddError(TIssue(ctx.GetPosition(setting->Tail().Pos()), TStringBuilder() + << "Expected YSON list of strings")); + return false; + } + for (const auto &child : securityTagsNode.AsList()) { + if (!child.IsString()) { + ctx.AddError(TIssue(ctx.GetPosition(setting->Tail().Pos()), TStringBuilder() + << "Expected YSON list of strings")); + return false; + } + } + return true; + } + case EYtSettingType::LAST: { + YQL_ENSURE(false, "Unexpected EYtSettingType"); + } + } + } + + if (used.HasFlags(EYtSettingType::InferScheme) && used.HasFlags(EYtSettingType::ForceInferScheme)) { + ctx.AddError(TIssue(ctx.GetPosition(settingsNode.Pos()), TStringBuilder() + << "Setting " << ToString(EYtSettingType::InferScheme).Quote() + << " is not allowed with " << ToString(EYtSettingType::ForceInferScheme).Quote())); + return false; + } + if (used.HasFlags(EYtSettingType::SortLimitBy) && !used.HasFlags(EYtSettingType::Limit)) { + ctx.AddError(TIssue(ctx.GetPosition(settingsNode.Pos()), TStringBuilder() + << "Setting " << ToString(EYtSettingType::SortLimitBy).Quote() + << " is not allowed without " << ToString(EYtSettingType::Limit).Quote())); + return false; + } + if (used.HasFlags(EYtSettingType::KeyFilter) && used.HasFlags(EYtSettingType::KeyFilter2)) { + ctx.AddError(TIssue(ctx.GetPosition(settingsNode.Pos()), TStringBuilder() + << "Settings " << ToString(EYtSettingType::KeyFilter).Quote() + << " and " << ToString(EYtSettingType::KeyFilter2).Quote() << " can not be used together")); + return false; + } + if (used.HasFlags(EYtSettingType::Anonymous) && used.HasFlags(EYtSettingType::Expiration)) { + ctx.AddError(TIssue(ctx.GetPosition(settingsNode.Pos()), TStringBuilder() + << ToString(EYtSettingType::Expiration).Quote() + << " setting cannot be used for anonymous tables")); + return false; + } + + if (isTruncate) { + for (auto type: {EYtSettingType::MonotonicKeys}) { + if (used.HasFlags(type)) { + ctx.AddError(TIssue(ctx.GetPosition(settingsNode.Pos()), TStringBuilder() + << ToString(type).Quote() + << " setting can not be used with TRUNCATE mode")); + return false; + } + } + } else { + for (auto type: {EYtSettingType::Expiration, EYtSettingType::Media, EYtSettingType::PrimaryMedium, EYtSettingType::KeepMeta}) { + if (used.HasFlags(type)) { + ctx.AddError(TIssue(ctx.GetPosition(settingsNode.Pos()), TStringBuilder() + << ToString(type).Quote() + << " setting can only be used with TRUNCATE mode")); + return false; + } + } + } + + return true; +} + +bool ValidateColumnGroups(const TExprNode& setting, const TStructExprType& rowType, TExprContext& ctx) { + const auto columnGroups = NYT::NodeFromYsonString(setting.Tail().Content()); + TIssueScopeGuard issueScope(ctx.IssueManager, [&]() { + return MakeIntrusive<TIssue>(ctx.GetPosition(setting.Pos()), TStringBuilder() << "Setting " << setting.Head().Content()); + }); + + for (const auto& grp: columnGroups.AsMap()) { + if (!grp.second.IsEntity()) { + for (const auto& col: grp.second.AsList()) { + if (!rowType.FindItem(col.AsString())) { + ctx.AddError(TIssue(ctx.GetPosition(setting.Pos()), TStringBuilder() + << "Column group " << grp.first.Quote() << " refers to unknown column " << col.AsString().Quote())); + return false; + } + } + } + } + return true; +} + +TString NormalizeColumnGroupSpec(const TStringBuf spec) { + try { + auto columnGroups = NYT::NodeFromYsonString(spec); + for (auto& grp: columnGroups.AsMap()) { + if (!grp.second.IsEntity()) { + std::stable_sort(grp.second.AsList().begin(), grp.second.AsList().end(), [](const auto& l, const auto& r) { return l.AsString() < r.AsString(); }); + } + } + return NYT::NodeToCanonicalYsonString(columnGroups); + } catch (...) { + // Keep as is. Type annotation will add user friendly error later + return TString{spec}; + } +} + +const TString& GetSingleColumnGroupSpec() { + static TString GROUP = NYT::NodeToCanonicalYsonString(NYT::TNode::CreateMap()("default", NYT::TNode::CreateEntity()), NYson::EYsonFormat::Text); + return GROUP; +} + +TExprNode::TPtr GetSetting(const TExprNode& settings, EYtSettingType type) { + for (auto& setting : settings.Children()) { + if (setting->ChildrenSize() != 0 && FromString<EYtSettingType>(setting->Child(0)->Content()) == type) { + return setting; + } + } + return nullptr; +} + +TExprNode::TPtr UpdateSettingValue(const TExprNode& settings, EYtSettingType type, TExprNode::TPtr&& value, TExprContext& ctx) { + for (ui32 index = 0U; index < settings.ChildrenSize(); ++index) { + if (settings.Child(index)->ChildrenSize() != 0 && FromString<EYtSettingType>(settings.Child(index)->Head().Content()) == type) { + const auto setting = settings.Child(index); + + auto newSetting = ctx.ChangeChildren(*setting, value ? TExprNode::TListType{setting->HeadPtr(), std::move(value)} : TExprNode::TListType{setting->HeadPtr()}); + return ctx.ChangeChild(settings, index, std::move(newSetting)); + } + } + return {}; +} + +TExprNode::TPtr AddOrUpdateSettingValue(const TExprNode& settings, EYtSettingType type, TExprNode::TPtr&& value, TExprContext& ctx) { + for (ui32 index = 0U; index < settings.ChildrenSize(); ++index) { + if (settings.Child(index)->ChildrenSize() != 0 && FromString<EYtSettingType>(settings.Child(index)->Head().Content()) == type) { + const auto setting = settings.Child(index); + + auto newSetting = ctx.ChangeChildren(*setting, value ? TExprNode::TListType{setting->HeadPtr(), std::move(value)} : TExprNode::TListType{setting->HeadPtr()}); + return ctx.ChangeChild(settings, index, std::move(newSetting)); + } + } + return AddSetting(settings, type, value, ctx); +} + +TExprNode::TListType GetAllSettingValues(const TExprNode& settings, EYtSettingType type) { + TExprNode::TListType res; + for (auto& setting : settings.Children()) { + if (setting->ChildrenSize() == 2 && FromString<EYtSettingType>(setting->Child(0)->Content()) == type) { + res.push_back(setting->ChildPtr(1)); + } + } + return res; +} + +bool HasSetting(const TExprNode& settings, EYtSettingType type) { + return bool(GetSetting(settings, type)); +} + +bool HasAnySetting(const TExprNode& settings, EYtSettingTypes types) { + for (auto& child: settings.Children()) { + if (child->ChildrenSize() != 0 && types.HasFlags(FromString<EYtSettingType>(child->Child(0)->Content()))) { + return true; + } + } + return false; +} + +bool HasSettingsExcept(const TExprNode& settings, EYtSettingTypes types) { + for (auto& child: settings.Children()) { + if (child->ChildrenSize() != 0 && !types.HasFlags(FromString<EYtSettingType>(child->Child(0)->Content()))) { + return true; + } + } + return false; +} + +bool EqualSettingsExcept(const TExprNode& lhs, const TExprNode& rhs, EYtSettingTypes types) { + if (&lhs == &rhs) + return true; + + TNodeSet set(lhs.ChildrenSize()); + for (const auto& child: lhs.Children()) { + if (child->ChildrenSize() != 0 && !types.HasFlags(FromString<EYtSettingType>(child->Head().Content()))) { + set.emplace(child.Get()); + } + } + for (const auto& child: rhs.Children()) { + if (child->ChildrenSize() != 0 && !types.HasFlags(FromString<EYtSettingType>(child->Head().Content()))) { + if (!set.erase(child.Get())) + return false; + } + } + return set.empty(); +} + +TExprNode::TPtr RemoveSetting(const TExprNode& settings, EYtSettingType type, TExprContext& ctx) { + TExprNode::TListType children; + for (auto child: settings.Children()) { + if (child->ChildrenSize() != 0 && FromString<EYtSettingType>(child->Child(0)->Content()) == type) { + continue; + } + children.push_back(child); + } + + return ctx.NewList(settings.Pos(), std::move(children)); +} + +TExprNode::TPtr RemoveSettings(const TExprNode& settings, EYtSettingTypes types, TExprContext& ctx) { + TExprNode::TListType children; + for (auto child: settings.Children()) { + if (child->ChildrenSize() != 0 && types.HasFlags(FromString<EYtSettingType>(child->Child(0)->Content()))) { + continue; + } + children.push_back(child); + } + + return ctx.NewList(settings.Pos(), std::move(children)); +} + +TExprNode::TPtr KeepOnlySettings(const TExprNode& settings, EYtSettingTypes types, TExprContext& ctx) { + TExprNode::TListType children; + for (auto child: settings.Children()) { + if (child->ChildrenSize() != 0 && !types.HasFlags(FromString<EYtSettingType>(child->Child(0)->Content()))) { + continue; + } + children.push_back(child); + } + + return ctx.NewList(settings.Pos(), std::move(children)); +} + +TExprNode::TPtr AddSetting(const TExprNode& settings, EYtSettingType type, const TExprNode::TPtr& value, TExprContext& ctx) { + return AddSetting(settings, settings.Pos(), ToString(type), value, ctx); +} + +TVector<TString> GetSettingAsColumnList(const TExprNode& settings, EYtSettingType type) { + TVector<TString> result; + if (auto node = GetSetting(settings, type)) { + for (auto& column : node->Child(1)->Children()) { + if (column->IsAtom()) { + result.emplace_back(column->Content()); + } else { + YQL_ENSURE(column->IsList() && column->ChildrenSize() > 0); + result.emplace_back(column->Child(0)->Content()); + } + } + } + return result; +} + +TVector<std::pair<TString, bool>> GetSettingAsColumnPairList(const TExprNode& settings, EYtSettingType type) { + TVector<std::pair<TString, bool>> result; + if (auto node = GetSetting(settings, type)) { + for (auto& column : node->Child(1)->Children()) { + if (column->IsAtom()) { + result.emplace_back(column->Content(), true); + } else { + result.emplace_back(column->Child(0)->Content(), FromString<bool>(column->Child(1)->Content())); + } + } + } + return result; +} + +TExprNode::TListType GetSettingAsColumnAtomList(const TExprNode& settings, EYtSettingType type) { + TExprNode::TListType result; + if (const auto node = GetSetting(settings, type)) { + for (const auto& column : node->Child(1)->Children()) { + if (column->IsAtom()) { + result.emplace_back(column); + } else { + YQL_ENSURE(column->IsList() && column->ChildrenSize() > 0); + result.emplace_back(column->HeadPtr()); + } + } + } + return result; +} + +std::vector<std::pair<TExprNode::TPtr, bool>> GetSettingAsColumnAtomPairList(const TExprNode& settings, EYtSettingType type) { + std::vector<std::pair<TExprNode::TPtr, bool>> result; + if (const auto node = GetSetting(settings, type)) { + for (const auto& column : node->Child(1)->Children()) { + if (column->IsAtom()) { + result.emplace_back(column, true); + } else { + result.emplace_back(column->HeadPtr(), FromString<bool>(column->Child(1)->Content())); + } + } + } + return result; +} + +TExprNode::TPtr AddSettingAsColumnList(const TExprNode& settings, EYtSettingType type, + const TVector<TString>& columns, TExprContext& ctx) +{ + return AddSetting(settings, type, ToAtomList(columns, settings.Pos(), ctx), ctx); +} + +TExprNode::TPtr ToColumnPairList(const TVector<std::pair<TString, bool>>& columns, TPositionHandle pos, TExprContext& ctx) { + if (AllOf(columns, [](const auto& pair) { return pair.second; })) { + TExprNode::TListType children; + for (auto& pair: columns) { + children.push_back(ctx.NewAtom(pos, pair.first)); + } + + return ctx.NewList(pos, std::move(children)); + } else { + TExprNode::TListType children; + for (auto& pair: columns) { + children.push_back(ctx.NewList(pos, { + ctx.NewAtom(pos, pair.first), + ctx.NewAtom(pos, pair.second ? "1" : "0", TNodeFlags::Default), + })); + } + + return ctx.NewList(pos, std::move(children)); + } +} + +TExprNode::TPtr AddSettingAsColumnPairList(const TExprNode& settings, EYtSettingType type, + const TVector<std::pair<TString, bool>>& columns, TExprContext& ctx) +{ + return AddSetting(settings, type, ToColumnPairList(columns, settings.Pos(), ctx), ctx); +} + +TMaybe<TSampleParams> GetSampleParams(const TExprNode& settings) { + if (auto setting = GetSetting(settings, EYtSettingType::Sample)) { + auto value = setting->Child(1); + YQL_ENSURE(value->ChildrenSize() == 3); + return TSampleParams { + FromString<EYtSampleMode>(value->Child(0)->Content()), + FromString<double>(value->Child(1)->Content()), + FromString<ui64>(value->Child(2)->Content()) + }; + } + + return Nothing(); +} + +TMaybe<ui64> GetMaxJobSizeForFirstAsPrimary(const TExprNode& settings) { + if (auto setting = NYql::GetSetting(settings, EYtSettingType::FirstAsPrimary)) { + if (setting->ChildrenSize() > 1) { + if (auto subsetting = NYql::GetSetting(*setting->Child(1), MaxJobSizeForFirstAsPrimaryName)) { + return FromString<ui64>(subsetting->Child(1)->Content()); + } + } + } + + return Nothing(); +} + +bool UseJoinReduceForSecondAsPrimary(const TExprNode &settings) { + if (auto setting = NYql::GetSetting(settings, EYtSettingType::FirstAsPrimary)) { + if (setting->ChildrenSize() > 1) { + return NYql::HasSetting(*setting->Child(1), JoinReduceForSecondAsPrimaryName); + } + } + + return false; +} + +ui32 GetMinChildrenForIndexedKeyFilter(EYtSettingType type) { + if (type == EYtSettingType::KeyFilter) { + return 2u; + } + YQL_ENSURE(type == EYtSettingType::KeyFilter2); + return 3u; +} + +EYtSettingTypes operator|(EYtSettingTypes left, const EYtSettingTypes& right) { + return left |= right; +} + +EYtSettingTypes operator&(EYtSettingTypes left, const EYtSettingTypes& right) { + return left &= right; +} + +EYtSettingTypes operator|(EYtSettingType left, EYtSettingType right) { + return EYtSettingTypes(left) | EYtSettingTypes(right); +} + +} // NYql diff --git a/yt/yql/providers/yt/provider/yql_yt_op_settings.h b/yt/yql/providers/yt/provider/yql_yt_op_settings.h new file mode 100644 index 0000000000..5c42dbc734 --- /dev/null +++ b/yt/yql/providers/yt/provider/yql_yt_op_settings.h @@ -0,0 +1,236 @@ +#pragma once + +#include <yql/essentials/ast/yql_expr.h> + +#include <util/generic/flags.h> +#include <util/generic/strbuf.h> +#include <util/system/types.h> +#include <util/string/cast.h> +#include <util/str_stl.h> + +#include <utility> +#include <bitset> + +namespace NYql { + +/////////////////////////////////////////////////////////////////////////////////////////////// + +enum class EYtWriteMode: ui32 { + Renew /* "renew" */, + RenewKeepMeta /* "renew_keep_meta" */, + Append /* "append" */, + Drop /* "drop" */, + Flush /* "flush" */, +}; + +/////////////////////////////////////////////////////////////////////////////////////////////// + +enum class EYtSampleMode: ui32 { + System /* "system" */, + Bernoulli /* "bernoulli" */, +}; + +struct TSampleParams { + EYtSampleMode Mode; + double Percentage; + ui64 Repeat; + + friend bool operator==(const TSampleParams& l, const TSampleParams& r) { + return l.Mode == r.Mode + && l.Percentage == r.Percentage + && l.Repeat == r.Repeat; + } +}; + +/////////////////////////////////////////////////////////////////////////////////////////////// + +enum class EYtSettingType: ui64 { + // Table reads + Initial /* "initial" */, + InferScheme /* "infer_scheme" "inferscheme" "infer_schema" "inferschema" */, + ForceInferScheme /* "force_infer_schema" "forceinferschema" */, + DoNotFailOnInvalidSchema /* "do_not_fail_on_invalid_schema" */, + DirectRead /* "direct_read" "directread"*/, + View /* "view" */, + Mode /* "mode" */, + Scheme /* "scheme" */, + WeakConcat /* "weak_concat" */, + Anonymous /* "anonymous" */, + WithQB /* "with_qb" */, + Inline /* "inline" */, + Sample /* "sample" */, + JoinLabel /* "joinLabel" */, + IgnoreNonExisting /* "ignore_non_existing" "ignorenonexisting" */, + WarnNonExisting /* "warn_non_existing" "warnnonexisting" */, + XLock /* "xlock" */, + Unordered /* "unordered" */, + NonUnique /* "nonUnique" */, + UserSchema /* "userschema" */, + UserColumns /* "usercolumns" */, + StatColumns /* "statcolumns" */, + SysColumns /* "syscolumns" */, + IgnoreTypeV3 /* "ignoretypev3" "ignore_type_v3" */, + // Table content + MemUsage /* "memUsage" */, + ItemsCount /* "itemsCount" */, + RowFactor /* "rowFactor" */, + Small /* "small" */, + // Operations + Ordered /* "ordered" */, // hybrid supported + KeyFilter /* "keyFilter" */, + KeyFilter2 /* "keyFilter2" */, + Take /* "take" */, + Skip /* "skip" */, + Limit /* "limit" */, // hybrid supported + SortLimitBy /* "sortLimitBy" */, // hybrid supported + SortBy /* "sortBy" */, // hybrid supported + ReduceBy /* "reduceBy" */, // hybrid supported + ReduceFilterBy /* "reduceFilterBy" */, + ForceTransform /* "forceTransform" */, // hybrid supported + TransformColGroups /* "transformColGroups" */, // hybrid supported + WeakFields /* "weakFields" */, + Sharded /* "sharded" */, + CombineChunks /* "combineChunks" */, + JobCount /* "jobCount" */, // hybrid supported + JoinReduce /* "joinReduce" */, // hybrid supported + FirstAsPrimary /* "firstAsPrimary" */, // hybrid supported + Flow /* "flow" */, // hybrid supported + KeepSorted /* "keepSorted" */, // hybrid supported + KeySwitch /* "keySwitch" */, // hybrid supported + BlockInputReady /* "blockInputReady" */, // hybrid supported + BlockInputApplied /* "blockInputApplied" */, // hybrid supported + // Out tables + UniqueBy /* "uniqueBy" */, + OpHash /* "opHash" */, + // Operations + MapOutputType /* "mapOutputType" */, // hybrid supported + ReduceInputType /* "reduceInputType" */, // hybrid supported + NoDq /* "noDq" */, + // Read + Split /* "split" */, + // Write hints + CompressionCodec /* "compression_codec" "compressioncodec"*/, + ErasureCodec /* "erasure_codec" "erasurecodec" */, + Expiration /* "expiration" */, + ReplicationFactor /* "replication_factor" "replicationfactor" */, + UserAttrs /* "user_attrs", "userattrs" */, + Media /* "media" */, + PrimaryMedium /* "primary_medium", "primarymedium" */, + KeepMeta /* "keep_meta", "keepmeta" */, + MonotonicKeys /* "monotonic_keys", "monotonickeys" */, + MutationId /* "mutationid", "mutation_id" */, + ColumnGroups /* "column_groups", "columngroups" */, + SecurityTags /* "security_tags", "securitytags" */, + + LAST +}; + +constexpr auto YtSettingTypesCount = static_cast<ui64>(EYtSettingType::LAST); + +class EYtSettingTypes : std::bitset<YtSettingTypesCount> { +using TBase = std::bitset<YtSettingTypesCount>; + explicit EYtSettingTypes(const std::bitset<YtSettingTypesCount>& bitset) + : TBase(bitset) + {} + +public: + using ::NYql::EYtSettingTypes::bitset::bitset; + + EYtSettingTypes(EYtSettingType type) + : TBase(std::bitset<YtSettingTypesCount>(1) << static_cast<ui64>(type)) + {} + + EYtSettingTypes& operator|=(const EYtSettingTypes& other) { + TBase::operator|=(other); + return *this; + } + + friend EYtSettingTypes operator|(EYtSettingTypes, const EYtSettingTypes&); + + EYtSettingTypes& operator&=(const EYtSettingTypes& other) { + TBase::operator&=(other); + return *this; + } + + friend EYtSettingTypes operator&(EYtSettingTypes, const EYtSettingTypes&); + + bool HasFlags(const EYtSettingTypes& other) { + return *this & other; + } + + operator bool() const { + return count() != 0; + } +}; + +EYtSettingTypes operator|(EYtSettingType left, EYtSettingType right); + +const auto DqReadSupportedSettings = EYtSettingType::SysColumns | EYtSettingType::Sample | EYtSettingType::Unordered | EYtSettingType::NonUnique | EYtSettingType::KeyFilter2; +const auto DqOpSupportedSettings = EYtSettingType::Ordered | EYtSettingType::Limit | EYtSettingType::SortLimitBy | EYtSettingType::SortBy | + EYtSettingType::ReduceBy | EYtSettingType::ForceTransform | EYtSettingType::JobCount | EYtSettingType::JoinReduce | + EYtSettingType::FirstAsPrimary | EYtSettingType::Flow | EYtSettingType::BlockInputReady | EYtSettingType::BlockInputApplied | EYtSettingType::KeepSorted | EYtSettingType::KeySwitch | + EYtSettingType::ReduceInputType | EYtSettingType::MapOutputType | EYtSettingType::Sharded | EYtSettingType::TransformColGroups; + +/////////////////////////////////////////////////////////////////////////////////////////////// + +bool ValidateSettings(const TExprNode& settingsNode, EYtSettingTypes accepted, TExprContext& ctx); + +template <class TContainer> +TExprNode::TPtr ToAtomList(const TContainer& columns, TPositionHandle pos, TExprContext& ctx) { + TExprNode::TListType children; + for (auto& column : columns) { + children.push_back(ctx.NewAtom(pos, column)); + } + + return ctx.NewList(pos, std::move(children)); +} + +bool ValidateColumnGroups(const TExprNode& setting, const TStructExprType& rowType, TExprContext& ctx); +TString NormalizeColumnGroupSpec(const TStringBuf spec); +const TString& GetSingleColumnGroupSpec(); + +TExprNode::TPtr ToColumnPairList(const TVector<std::pair<TString, bool>>& columns, TPositionHandle pos, TExprContext& ctx); + +TExprNode::TPtr GetSetting(const TExprNode& settings, EYtSettingType type); +TExprNode::TPtr UpdateSettingValue(const TExprNode& settings, EYtSettingType type, TExprNode::TPtr&& value, TExprContext& ctx); +TExprNode::TPtr AddOrUpdateSettingValue(const TExprNode& settings, EYtSettingType type, TExprNode::TPtr&& value, TExprContext& ctx); + +TExprNode::TListType GetAllSettingValues(const TExprNode& settings, EYtSettingType type); +TVector<TString> GetSettingAsColumnList(const TExprNode& settings, EYtSettingType type); +TVector<std::pair<TString, bool>> GetSettingAsColumnPairList(const TExprNode& settings, EYtSettingType type); + +TExprNode::TListType GetSettingAsColumnAtomList(const TExprNode& settings, EYtSettingType type); +std::vector<std::pair<TExprNode::TPtr, bool>> GetSettingAsColumnAtomPairList(const TExprNode& settings, EYtSettingType type); + +bool HasSetting(const TExprNode& settings, EYtSettingType type); +bool HasAnySetting(const TExprNode& settings, EYtSettingTypes types); +bool HasSettingsExcept(const TExprNode& settings, EYtSettingTypes types); +bool EqualSettingsExcept(const TExprNode& lhs, const TExprNode& rhs, EYtSettingTypes types); + +TExprNode::TPtr RemoveSetting(const TExprNode& settings, EYtSettingType type, TExprContext& ctx); +TExprNode::TPtr RemoveSettings(const TExprNode& settings, EYtSettingTypes types, TExprContext& ctx); +TExprNode::TPtr KeepOnlySettings(const TExprNode& settings, EYtSettingTypes types, TExprContext& ctx); +TExprNode::TPtr AddSetting(const TExprNode& settings, EYtSettingType type, const TExprNode::TPtr& value, TExprContext& ctx); +TExprNode::TPtr AddSettingAsColumnList(const TExprNode& settings, EYtSettingType type, + const TVector<TString>& columns, TExprContext& ctx); +TExprNode::TPtr AddSettingAsColumnPairList(const TExprNode& settings, EYtSettingType type, + const TVector<std::pair<TString, bool>>& columns, TExprContext& ctx); +TMaybe<TSampleParams> GetSampleParams(const TExprNode& settings); + +const TStringBuf JoinReduceForSecondAsPrimaryName = "joinReduceForSecond"; +const TStringBuf MaxJobSizeForFirstAsPrimaryName = "maxJobSize"; + +TMaybe<ui64> GetMaxJobSizeForFirstAsPrimary(const TExprNode& settings); +bool UseJoinReduceForSecondAsPrimary(const TExprNode& settings); + +ui32 GetMinChildrenForIndexedKeyFilter(EYtSettingType type); + +} // NYql + + +template <> +struct THash<NYql::TSampleParams> { + size_t operator()(const NYql::TSampleParams& p) const { + return CombineHashes(NumericHash(static_cast<ui32>(p.Mode)), CombineHashes(THash<double>()(p.Percentage), NumericHash(p.Repeat))); + } +}; diff --git a/yt/yql/providers/yt/provider/yql_yt_optimize.cpp b/yt/yql/providers/yt/provider/yql_yt_optimize.cpp new file mode 100644 index 0000000000..94c455dc0c --- /dev/null +++ b/yt/yql/providers/yt/provider/yql_yt_optimize.cpp @@ -0,0 +1,674 @@ +#include "yql_yt_optimize.h" +#include "yql_yt_op_settings.h" +#include "yql_yt_table.h" +#include "yql_yt_helpers.h" +#include "yql_yt_provider_impl.h" + +#include <yt/yql/providers/yt/lib/res_pull/table_limiter.h> +#include <yt/yql/providers/yt/lib/expr_traits/yql_expr_traits.h> +#include <yt/yql/providers/yt/common/yql_configuration.h> +#include <yql/essentials/providers/common/codec/yql_codec_type_flags.h> +#include <yql/essentials/core/expr_nodes/yql_expr_nodes.h> +#include <yql/essentials/core/type_ann/type_ann_expr.h> +#include <yql/essentials/core/yql_expr_optimize.h> +#include <yql/essentials/core/yql_type_helpers.h> +#include <yql/essentials/core/yql_expr_constraint.h> +#include <yql/essentials/core/yql_graph_transformer.h> +#include <yql/essentials/core/yql_expr_csee.h> +#include <yql/essentials/public/udf/udf_value.h> +#include <yql/essentials/utils/log/log.h> +#include <yql/essentials/core/services/yql_transform_pipeline.h> + +#include <util/generic/xrange.h> +#include <util/generic/ptr.h> +#include <util/generic/vector.h> +#include <util/generic/size_literals.h> +#include <util/generic/maybe.h> + +#include <utility> + +namespace NYql { + +using namespace NNodes; +namespace { +TMaybeNode<TYtSection> MaterializeSectionIfRequired(TExprBase world, TYtSection section, TYtDSink dataSink, TYqlRowSpecInfo::TPtr outRowSpec, bool keepSortness, + const TExprNode::TListType& limitNodes, const TYtState::TPtr& state, TExprContext& ctx) +{ + const bool hasLimit = NYql::HasAnySetting(section.Settings().Ref(), EYtSettingType::Take | EYtSettingType::Skip); + bool needMaterialize = hasLimit && NYql::HasSetting(section.Settings().Ref(), EYtSettingType::Sample); + bool hasDynamic = false; + if (!needMaterialize) { + bool hasRanges = false; + for (TYtPath path: section.Paths()) { + TYtPathInfo pathInfo(path); + hasDynamic = hasDynamic || (pathInfo.Table->Meta && pathInfo.Table->Meta->IsDynamic); + hasRanges = hasRanges || pathInfo.Ranges; + } + needMaterialize = hasRanges || (hasLimit && hasDynamic); + } + + if (needMaterialize) { + auto scheme = section.Ref().GetTypeAnn()->Cast<TListExprType>()->GetItemType(); + auto path = CopyOrTrivialMap(section.Pos(), + world, dataSink, + *scheme, + Build<TYtSection>(ctx, section.Pos()) + .Paths(section.Paths()) + .Settings(NYql::RemoveSettings(section.Settings().Ref(), + EYtSettingType::Take | EYtSettingType::Skip | + EYtSettingType::KeyFilter | EYtSettingType::KeyFilter2 | EYtSettingType::JoinLabel | + EYtSettingType::Unordered | EYtSettingType::NonUnique | EYtSettingType::StatColumns, ctx)) + .Done(), + outRowSpec, + ctx, + state, + TCopyOrTrivialMapOpts() + .SetTryKeepSortness(keepSortness || (!ctx.IsConstraintEnabled<TSortedConstraintNode>() && (!hasDynamic || (!hasLimit && limitNodes.empty())))) + .SetRangesResetSort(false) + .SetSectionUniq(section.Ref().GetConstraint<TDistinctConstraintNode>()) + .SetLimitNodes(limitNodes) + ); + + return Build<TYtSection>(ctx, section.Pos()) + .Paths() + .Add(path) + .Build() + .Settings(NYql::RemoveSetting(section.Settings().Ref(), EYtSettingType::Sample, ctx)) + .Done(); + } + + return {}; +} + +TMaybeNode<TYtSection> UpdateSectionWithRange(TExprBase world, TYtSection section, const TRecordsRange& range, + TYtDSink dataSink, TYqlRowSpecInfo::TPtr outRowSpec, bool keepSortness, bool allowWorldDeps, bool allowMaterialize, + TSyncMap& syncList, const TYtState::TPtr& state, TExprContext& ctx) +{ + bool isEmptyInput = allowWorldDeps; + TVector<TYtPath> updatedPaths; + TVector<TYtPath> skippedPaths; + if (auto limiter = TTableLimiter(range)) { + if (auto materialized = MaterializeSectionIfRequired(world, section, dataSink, outRowSpec, keepSortness, + {NYql::KeepOnlySettings(section.Settings().Ref(), EYtSettingType::Take | EYtSettingType::Skip | EYtSettingType::SysColumns, ctx)}, state, ctx)) + { + if (!allowMaterialize || state->Types->EvaluationInProgress) { + // Keep section as is + return {}; + } + if (!allowWorldDeps) { + if (const auto out = materialized.Paths().Item(0).Table().Maybe<TYtOutput>()) { + syncList.emplace(GetOutputOp(out.Cast()).Ptr(), syncList.size()); + } + } + return materialized; + } + + for (size_t i: xrange(section.Paths().Size())) { + auto path = section.Paths().Item(i); + TYtPathInfo pathInfo(path); + if (!pathInfo.Table->Stat) { + // Not all tables have required info + return {}; + } + + ui64 startRecordInTable = 0; + ui64 endRecordInTable = 0; + + if (pathInfo.Table->Stat->RecordsCount) { + if (!limiter.NextTable(pathInfo.Table->Stat->RecordsCount)) { + if (allowWorldDeps) { + skippedPaths.push_back(path); + } else { + pathInfo.Stat.Drop(); + pathInfo.Ranges = TYtRangesInfo::MakeEmptyRange(); + updatedPaths.push_back(pathInfo.ToExprNode(ctx, path.Pos(), path.Table()).Cast<TYtPath>()); + } + continue; + } + startRecordInTable = limiter.GetTableStart(); + endRecordInTable = limiter.GetTableZEnd(); // 0 means the entire table usage + } + + if (startRecordInTable || endRecordInTable) { + pathInfo.Stat.Drop(); + pathInfo.Ranges = MakeIntrusive<TYtRangesInfo>(); + TYtRangesInfo::TRowRange range; + if (startRecordInTable) { + range.Lower = startRecordInTable; + } + if (endRecordInTable) { + range.Upper = endRecordInTable; + } + pathInfo.Ranges->AddRowRange(range); + updatedPaths.push_back(pathInfo.ToExprNode(ctx, path.Pos(), path.Table()).Cast<TYtPath>()); + } else { + updatedPaths.push_back(path); + } + isEmptyInput = false; + if (limiter.Exceed()) { + if (allowWorldDeps) { + for (size_t j = i + 1; j < section.Paths().Size(); ++j) { + skippedPaths.push_back(section.Paths().Item(j)); + } + } else { + for (size_t j = i + 1; j < section.Paths().Size(); ++j) { + auto path = section.Paths().Item(j); + path = Build<TYtPath>(ctx, path.Pos()) + .InitFrom(path) + .Ranges<TExprList>() + .Build() + .Stat<TCoVoid>().Build() + .Done(); + updatedPaths.push_back(path); + } + } + break; + } + } + } else if (!allowWorldDeps) { + for (auto path: section.Paths()) { + updatedPaths.push_back(Build<TYtPath>(ctx, path.Pos()) + .InitFrom(path) + .Ranges<TExprList>() + .Build() + .Stat<TCoVoid>().Build() + .Done()); + } + } + + if (isEmptyInput) { + skippedPaths.assign(section.Paths().begin(), section.Paths().end()); + } + for (auto path: skippedPaths) { + if (auto out = path.Table().Maybe<TYtOutput>()) { + syncList.emplace(GetOutputOp(out.Cast()).Ptr(), syncList.size()); + } + } + + if (isEmptyInput) { + return MakeEmptySection(section, dataSink, keepSortness, state, ctx); + } + + return Build<TYtSection>(ctx, section.Pos()) + .Paths() + .Add(updatedPaths) + .Build() + .Settings(NYql::RemoveSettings(section.Settings().Ref(), EYtSettingType::Take | EYtSettingType::Skip, ctx)) + .Done(); +} + +void EnableKeyBoundApi(TYtPathInfo& pathInfo, const TYtState::TPtr& state) { + if (!pathInfo.Ranges) { + return; + } + YQL_ENSURE(pathInfo.Table); + const bool useKeyBoundApi = + state->Configuration->_UseKeyBoundApi.Get(pathInfo.Table->Cluster).GetOrElse(DEFAULT_USE_KEY_BOUND_API); + pathInfo.Ranges->SetUseKeyBoundApi(useKeyBoundApi); +} + +TMaybeNode<TYtSection> UpdateSectionWithLegacyFilters(TYtSection section, const TVector<TExprBase>& filters, const TYtState::TPtr& state, TExprContext& ctx) +{ + TVector<TExprBase> commonFilters; + TMap<size_t, TVector<TExprBase>> tableFilters; + for (auto filter: filters) { + auto filterList = filter.Cast<TExprList>(); + if (filterList.Size() == 2) { + tableFilters[FromString<size_t>(filterList.Item(1).Cast<TCoAtom>().Value())].push_back(filterList.Item(0)); + } + else { + commonFilters.push_back(filterList.Item(0)); + } + } + + TVector<TYtPath> updatedPaths; + size_t tableIndex = 0; + for (auto path: section.Paths()) { + if (commonFilters.size() == filters.size()) { + TYtPathInfo pathInfo(path); + pathInfo.Stat.Drop(); + pathInfo.Ranges = TYtRangesInfo::ApplyLegacyKeyFilters(commonFilters, pathInfo.Table->RowSpec, ctx); + EnableKeyBoundApi(pathInfo, state); + updatedPaths.push_back(pathInfo.ToExprNode(ctx, path.Pos(), path.Table()).Cast<TYtPath>()); + } + else { + TVector<TExprBase> pathFilters = commonFilters; + if (auto p = tableFilters.FindPtr(tableIndex)) { + pathFilters.insert(pathFilters.end(), p->begin(), p->end()); + } + if (pathFilters.empty()) { + updatedPaths.push_back(path); + } + else { + TYtPathInfo pathInfo(path); + pathInfo.Stat.Drop(); + pathInfo.Ranges = TYtRangesInfo::ApplyLegacyKeyFilters(pathFilters, pathInfo.Table->RowSpec, ctx); + EnableKeyBoundApi(pathInfo, state); + updatedPaths.push_back(pathInfo.ToExprNode(ctx, path.Pos(), path.Table()).Cast<TYtPath>()); + } + } + ++tableIndex; + } + + auto updatedSettings = NYql::RemoveSetting(section.Settings().Ref(), EYtSettingType::KeyFilter, ctx); + updatedSettings = NYql::AddSetting(*updatedSettings, EYtSettingType::KeyFilter, ctx.NewList(section.Pos(), {}), ctx); + + return Build<TYtSection>(ctx, section.Pos()) + .Paths() + .Add(updatedPaths) + .Build() + .Settings(updatedSettings) + .Done(); +} + +TMaybeNode<TYtSection> UpdateSectionWithFilters(TYtSection section, const TVector<TExprBase>& filters, const TYtState::TPtr& state, TExprContext& ctx) { + TMap<size_t, TExprNode::TPtr> filtersByTableIndex; + TExprNode::TPtr commonFilter; + for (auto filter: filters) { + auto filterList = filter.Cast<TExprList>(); + auto computedFilter = filterList.Item(0).Ptr(); + if (filterList.Size() == 3) { + for (auto idxNode : filterList.Item(2).Cast<TCoAtomList>()) { + size_t idx = FromString<size_t>(idxNode.Value()); + YQL_ENSURE(!filtersByTableIndex.contains(idx)); + filtersByTableIndex[idx] = computedFilter; + } + } else { + YQL_ENSURE(!commonFilter); + commonFilter = computedFilter; + } + } + + YQL_ENSURE(filtersByTableIndex.empty() && commonFilter || !commonFilter && !filtersByTableIndex.empty()); + + TVector<TYtPath> updatedPaths; + size_t tableIndex = 0; + for (auto path: section.Paths()) { + TExprNode::TPtr filter; + if (commonFilter) { + filter = commonFilter; + } else { + auto it = filtersByTableIndex.find(tableIndex); + if (it != filtersByTableIndex.end()) { + filter = it->second; + } + } + + if (!filter) { + updatedPaths.push_back(path); + } else { + TYtPathInfo pathInfo(path); + pathInfo.Stat.Drop(); + pathInfo.Ranges = TYtRangesInfo::ApplyKeyFilter(*filter); + EnableKeyBoundApi(pathInfo, state); + updatedPaths.push_back(pathInfo.ToExprNode(ctx, path.Pos(), path.Table()).Cast<TYtPath>()); + } + ++tableIndex; + } + + auto updatedSettings = NYql::RemoveSetting(section.Settings().Ref(), EYtSettingType::KeyFilter2, ctx); + updatedSettings = NYql::AddSetting(*updatedSettings, EYtSettingType::KeyFilter2, ctx.NewList(section.Pos(), {}), ctx); + + return Build<TYtSection>(ctx, section.Pos()) + .Paths() + .Add(updatedPaths) + .Build() + .Settings(updatedSettings) + .Done(); +} + +} //namespace + +TMaybeNode<TYtSection> UpdateSectionWithSettings(TExprBase world, TYtSection section, TYtDSink dataSink, TYqlRowSpecInfo::TPtr outRowSpec, bool keepSortness, bool allowWorldDeps, bool allowMaterialize, + TSyncMap& syncList, const TYtState::TPtr& state, TExprContext& ctx) +{ + if (NYql::HasSetting(section.Settings().Ref(), EYtSettingType::DirectRead)) { + return {}; + } + + if (!NYql::HasAnySetting(section.Settings().Ref(), EYtSettingType::Take | EYtSettingType::Skip | EYtSettingType::KeyFilter | EYtSettingType::KeyFilter2)) { + return {}; + } + + if (HasNodesToCalculate(section.Ptr())) { + return {}; + } + + TRecordsRange range; + TVector<TExprBase> keyFilters; + bool legacyKeyFilters = false; + for (auto s: section.Settings()) { + switch (FromString<EYtSettingType>(s.Name().Value())) { + case EYtSettingType::KeyFilter: + legacyKeyFilters = true; + [[fallthrough]]; + case EYtSettingType::KeyFilter2: + if (s.Value().Cast<TExprList>().Size() > 0) { + keyFilters.push_back(s.Value().Cast()); + } + break; + default: + // Skip other settings + break; + } + } + range.Fill(section.Settings().Ref()); + + if (range.Limit || range.Offset) { + return UpdateSectionWithRange(world, section, range, dataSink, outRowSpec, keepSortness, allowWorldDeps, allowMaterialize, syncList, state, ctx); + } + if (!keyFilters.empty()) { + return legacyKeyFilters ? UpdateSectionWithLegacyFilters(section, keyFilters, state, ctx) : UpdateSectionWithFilters(section, keyFilters, state, ctx); + } + + return {}; +} + +TYtSection MakeEmptySection(TYtSection section, NNodes::TYtDSink dataSink, bool keepSortness, const TYtState::TPtr& state, TExprContext& ctx) { + TYtOutTableInfo outTable(GetSequenceItemType(section, false)->Cast<TStructExprType>(), + state->Configuration->UseNativeYtTypes.Get().GetOrElse(DEFAULT_USE_NATIVE_YT_TYPES) ? NTCF_ALL : NTCF_NONE); + if (section.Paths().Size() == 1) { + auto srcTableInfo = TYtTableBaseInfo::Parse(section.Paths().Item(0).Table()); + if (keepSortness && srcTableInfo->RowSpec && srcTableInfo->RowSpec->IsSorted()) { + outTable.RowSpec->CopySortness(ctx, *srcTableInfo->RowSpec, TYqlRowSpecInfo::ECopySort::WithCalc); + } + } + outTable.SetUnique(section.Ref().GetConstraint<TDistinctConstraintNode>(), section.Pos(), ctx); + return Build<TYtSection>(ctx, section.Pos()) + .Paths() + .Add() + .Table<TYtOutput>() + .Operation<TYtTouch>() + .World<TCoWorld>().Build() + .DataSink(dataSink) + .Output() + .Add(outTable.ToExprNode(ctx, section.Pos()).Cast<TYtOutTable>()) + .Build() + .Build() + .OutIndex().Value("0").Build() + .Build() + .Columns<TCoVoid>().Build() + .Ranges<TCoVoid>().Build() + .Stat<TCoVoid>().Build() + .Build() + .Build() + .Settings(NYql::RemoveSettings(section.Settings().Ref(), EYtSettingType::Take | EYtSettingType::Skip | EYtSettingType::Sample, ctx)) + .Done(); +} + +TExprNode::TPtr OptimizeReadWithSettings(const TExprNode::TPtr& node, bool allowWorldDeps, bool allowMaterialize, TSyncMap& syncList, + const TYtState::TPtr& state, TExprContext& ctx) +{ + auto read = TYtReadTable(node); + auto dataSink = TYtDSink(ctx.RenameNode(read.DataSource().Ref(), "DataSink")); + + bool hasUpdates = false; + TVector<TExprBase> updatedSections; + for (auto section: read.Input()) { + updatedSections.push_back(section); + const bool keepSort = ctx.IsConstraintEnabled<TSortedConstraintNode>() && !NYql::HasSetting(section.Settings().Ref(), EYtSettingType::Unordered); + if (auto updatedSection = UpdateSectionWithSettings(read.World(), section, dataSink, {}, keepSort, allowWorldDeps, allowMaterialize, syncList, state, ctx)) { + updatedSections.back() = updatedSection.Cast(); + hasUpdates = true; + } + } + if (!hasUpdates) { + return node; + } + + auto res = ctx.ChangeChild(read.Ref(), TYtReadTable::idx_Input, + Build<TYtSectionList>(ctx, read.Input().Pos()) + .Add(updatedSections) + .Done().Ptr()); + + return res; +} + +IGraphTransformer::TStatus UpdateTableContentMemoryUsage(const TExprNode::TPtr& input, TExprNode::TPtr& output, const TYtState::TPtr& state, + TExprContext& ctx, bool estimateTableContentWeight) +{ + auto current = input; + output.Reset(); + for (;;) { + TProcessedNodesSet ignoreNodes; + VisitExpr(current, [&ignoreNodes](const TExprNode::TPtr& node) { + if (TYtOutput::Match(node.Get())) { + // Stop traversing dependent operations + ignoreNodes.insert(node->UniqueId()); + return false; + } + return true; + }); + + TOptimizeExprSettings settings(state->Types); + settings.CustomInstantTypeTransformer = state->Types->CustomInstantTypeTransformer.Get(); + settings.ProcessedNodes = &ignoreNodes; + + TParentsMap parentsMap; + GatherParents(*current, parentsMap); + + TExprNode::TPtr newCurrent; + auto status = OptimizeExpr(current, newCurrent, + [&parentsMap, current, state, estimateTableContentWeight](const TExprNode::TPtr& node, TExprContext& ctx) -> TExprNode::TPtr { + if (auto maybeContent = TMaybeNode<TYtTableContent>(node)) { + auto content = maybeContent.Cast(); + if (NYql::HasAnySetting(content.Settings().Ref(), EYtSettingType::MemUsage | EYtSettingType::Small)) { + return node; + } + + ui64 collectRowFactor = 0; + if (auto setting = NYql::GetSetting(content.Settings().Ref(), EYtSettingType::RowFactor)) { + collectRowFactor = FromString<ui64>(setting->Child(1)->Content()); + } else { + const auto contentItemType = content.Ref().GetTypeAnn()->Cast<TListExprType>()->GetItemType(); + size_t fieldsCount = 0; + switch (contentItemType->GetKind()) { + case ETypeAnnotationKind::Struct: + fieldsCount = contentItemType->Cast<TStructExprType>()->GetSize(); + break; + case ETypeAnnotationKind::Tuple: + fieldsCount = contentItemType->Cast<TTupleExprType>()->GetSize(); + break; + default: + break; + } + collectRowFactor = 2 * (1 + fieldsCount) * sizeof(NKikimr::NUdf::TUnboxedValuePod); + } + + bool wrapToCollect = false; + TVector<std::pair<double, ui64>> factors; // first: sizeFactor, second: rowFactor + TNodeSet tableContentConsumers; + if (!GetTableContentConsumerNodes(*node, *current, parentsMap, tableContentConsumers)) { + wrapToCollect = true; + factors.emplace_back(2., collectRowFactor); + } + else { + for (auto consumer: tableContentConsumers) { + if (consumer->IsCallable({"ToDict","SqueezeToDict", "SqlIn"})) { + double sizeFactor = 1.; + ui64 rowFactor = 0ULL; + if (auto err = CalcToDictFactors(*consumer, ctx, sizeFactor, rowFactor)) { + ctx.AddError(*err); + return {}; + } + factors.emplace_back(sizeFactor, rowFactor); + } + else if (consumer->IsCallable("Collect")) { + factors.emplace_back(2., collectRowFactor); + } + } + } + + ui64 memUsage = 0; + ui64 dataWeight = 0; + ui64 itemsCount = 0; + bool useItemsCount = !NYql::HasSetting(content.Settings().Ref(), EYtSettingType::ItemsCount); + + if (factors.empty()) { + // No ToDict or Collect consumers. Assume memory usage equals to max row size on YT + memUsage = 16_MB; + useItemsCount = false; + } + if (estimateTableContentWeight || !factors.empty()) { + if (auto maybeRead = content.Input().Maybe<TYtReadTable>()) { + TVector<ui64> records; + TVector<TYtPathInfo::TPtr> tableInfos; + bool hasNotCalculated = false; + for (auto section: maybeRead.Cast().Input()) { + for (auto path: section.Paths()) { + TYtPathInfo::TPtr info = MakeIntrusive<TYtPathInfo>(path); + if (info->Table->Stat) { + ui64 tableRecord = info->Table->Stat->RecordsCount; + if (info->Ranges) { + const auto used = info->Ranges->GetUsedRows(tableRecord); + tableRecord = used.GetOrElse(tableRecord); + if (used) { + itemsCount += *used; + } else { + useItemsCount = false; + } + } else { + itemsCount += tableRecord; + } + if (info->Table->Meta->IsDynamic) { + useItemsCount = false; + } + records.push_back(tableRecord); + tableInfos.push_back(info); + } + else { + YQL_CLOG(INFO, ProviderYt) << "Assume 1Gb memory usage for YtTableContent #" + << node->UniqueId() << " because input table is not calculated yet"; + memUsage += 1_GB; + hasNotCalculated = true; + useItemsCount = false; + break; + } + } + if (NYql::HasSetting(section.Settings().Ref(), EYtSettingType::Sample)) { + useItemsCount = false; + } + if (hasNotCalculated) { + break; + } + } + if (!hasNotCalculated && !tableInfos.empty()) { + if (auto dataSizes = EstimateDataSize(TString{maybeRead.Cast().DataSource().Cluster().Value()}, tableInfos, Nothing(), *state, ctx)) { + YQL_ENSURE(dataSizes->size() == records.size()); + for (size_t i: xrange(records.size())) { + for (auto& factor: factors) { + memUsage += factor.first * dataSizes->at(i) + factor.second * records.at(i); + } + dataWeight += dataSizes->at(i); + } + } else { + return {}; + } + } + } + else { + TYtOutTableInfo info(GetOutTable(content.Input().Cast<TYtOutput>())); + if (info.Stat) { + const ui64 dataSize = info.Stat->DataSize; + const ui64 records = info.Stat->RecordsCount; + for (auto& factor: factors) { + memUsage += factor.first * dataSize + factor.second * records; + } + itemsCount += records; + dataWeight += dataSize; + } + else { + YQL_CLOG(INFO, ProviderYt) << "Assume 1Gb memory usage for YtTableContent #" + << node->UniqueId() << " because input table is not calculated yet"; + memUsage += 1_GB; + useItemsCount = false; + } + } + } + + auto settings = content.Settings().Ptr(); + settings = NYql::AddSetting(*settings, EYtSettingType::MemUsage, ctx.NewAtom(node->Pos(), ToString(memUsage), TNodeFlags::Default), ctx); + if (useItemsCount) { + settings = NYql::AddSetting(*settings, EYtSettingType::ItemsCount, ctx.NewAtom(node->Pos(), ToString(itemsCount), TNodeFlags::Default), ctx); + } + if (estimateTableContentWeight && dataWeight < state->Configuration->TableContentLocalExecution.Get().GetOrElse(DEFAULT_TABLE_CONTENT_LOCAL_EXEC)) { + settings = NYql::AddSetting(*settings, EYtSettingType::Small, {}, ctx); + } + + return ctx.WrapByCallableIf(wrapToCollect, "Collect", ctx.ChangeChild(*node, TYtTableContent::idx_Settings, std::move(settings))); + } + return node; + }, + ctx, settings); + + if (IGraphTransformer::TStatus::Error == status.Level) { + ctx.AddError(TIssue(ctx.GetPosition(current->Pos()), TStringBuilder() << "Failed to update YtTableContent memory usage in node: " << current->Content())); + return status; + } + + if (newCurrent != current) { + if (current->IsLambda()) { + YQL_ENSURE(newCurrent->IsLambda()); + YQL_ENSURE(newCurrent->Head().ChildrenSize() == current->Head().ChildrenSize()); + for (size_t i = 0; i < newCurrent->Head().ChildrenSize(); ++i) { + newCurrent->Head().Child(i)->SetTypeAnn(current->Head().Child(i)->GetTypeAnn()); + newCurrent->Head().Child(i)->CopyConstraints(*current->Head().Child(i)); + } + } + auto typeTransformer = CreateTypeAnnotationTransformer(CreateExtCallableTypeAnnotationTransformer(*state->Types, true), *state->Types); + auto constrTransformer = CreateConstraintTransformer(*state->Types, true, true); + TVector<TTransformStage> transformers; + const auto issueCode = TIssuesIds::CORE_TYPE_ANN; + transformers.push_back(TTransformStage(typeTransformer, "TypeAnnotation", issueCode)); + transformers.push_back(TTransformStage( + CreateFunctorTransformer([](const TExprNode::TPtr& input, TExprNode::TPtr& output, TExprContext& ctx) { return UpdateCompletness(input, output, ctx); }), + "UpdateCompletness", issueCode)); + transformers.push_back(TTransformStage(constrTransformer, "Constraints", issueCode)); + auto fullTransformer = CreateCompositeGraphTransformer(transformers, false); + status = InstantTransform(*fullTransformer, newCurrent, ctx); + if (status.Level == IGraphTransformer::TStatus::Error) { + return status; + } + + current = newCurrent; + continue; + } + + output = current; + return IGraphTransformer::TStatus::Ok; + } +} + +struct TPeepholePipelineConfigurator : public IPipelineConfigurator { + TPeepholePipelineConfigurator(TYtState::TPtr state) + : State_(std::move(state)) + {} +private: + void AfterCreate(TTransformationPipeline*) const final {} + + void AfterTypeAnnotation(TTransformationPipeline* pipeline) const final { + pipeline->Add(CreateYtPeepholeTransformer(State_, {}), "Peephole"); + pipeline->Add(CreateYtWideFlowTransformer(State_), "WideFlow"); + pipeline->Add(CreateYtBlockInputTransformer(State_), "BlockInput"); + } + + void AfterOptimize(TTransformationPipeline*) const final {} + + const TYtState::TPtr State_; +}; + +IGraphTransformer::TStatus PeepHoleOptimizeBeforeExec(TExprNode::TPtr input, TExprNode::TPtr& output, + const TYtState::TPtr& state, bool& hasNonDeterministicFunctions, TExprContext& ctx, bool estimateTableContentWeight) +{ + if (const auto status = UpdateTableContentMemoryUsage(input, output, state, ctx, estimateTableContentWeight); + status.Level != IGraphTransformer::TStatus::Ok) { + return status; + } + + const TPeepholePipelineConfigurator wideFlowTransformers(state); + TPeepholeSettings peepholeSettings; + peepholeSettings.CommonConfig = &wideFlowTransformers; + return PeepHoleOptimizeNode(output, output, ctx, *state->Types, nullptr, hasNonDeterministicFunctions, peepholeSettings); +} + +} // NYql diff --git a/yt/yql/providers/yt/provider/yql_yt_optimize.h b/yt/yql/providers/yt/provider/yql_yt_optimize.h new file mode 100644 index 0000000000..e9766d87f6 --- /dev/null +++ b/yt/yql/providers/yt/provider/yql_yt_optimize.h @@ -0,0 +1,37 @@ +#pragma once + +#include "yql_yt_provider.h" +#include "yql_yt_provider_impl.h" + +#include <yt/yql/providers/yt/expr_nodes/yql_yt_expr_nodes.h> +#include <yt/yql/providers/yt/lib/mkql_helpers/mkql_helpers.h> +#include <yql/essentials/ast/yql_expr.h> +#include <yql/essentials/core/yql_graph_transformer.h> +#include <yql/essentials/core/peephole_opt/yql_opt_peephole_physical.h> + +#include <util/generic/string.h> +#include <util/generic/vector.h> + +#include <tuple> + +namespace NYql { + +// writer -> (reader, YtSection, YtOutput, YtPath) +using TOpDeps = TNodeMap<TVector<std::tuple<const TExprNode*, const TExprNode*, const TExprNode*, const TExprNode*>>>; + +NNodes::TMaybeNode<NNodes::TYtSection> UpdateSectionWithSettings(NNodes::TExprBase world, NNodes::TYtSection section, NNodes::TYtDSink dataSink, + TYqlRowSpecInfo::TPtr outRowSpec, bool keepSortness, bool allowWorldDeps, bool allowMaterialize, TSyncMap& syncList, const TYtState::TPtr& state, TExprContext& ctx); + +NNodes::TYtSection MakeEmptySection(NNodes::TYtSection section, NNodes::TYtDSink dataSink, + bool keepSortness, const TYtState::TPtr& state, TExprContext& ctx); + +TExprNode::TPtr OptimizeReadWithSettings(const TExprNode::TPtr& node, bool allowWorldDeps, bool allowMaterialize, TSyncMap& syncList, + const TYtState::TPtr& state, TExprContext& ctx); + +IGraphTransformer::TStatus UpdateTableContentMemoryUsage(const TExprNode::TPtr& input, TExprNode::TPtr& output, + const TYtState::TPtr& state, TExprContext& ctx, bool estimateTableContentWeight); + +IGraphTransformer::TStatus PeepHoleOptimizeBeforeExec(TExprNode::TPtr input, TExprNode::TPtr& output, + const TYtState::TPtr& state, bool& hasNonDeterministicFunctions, TExprContext& ctx, bool estimateTableContentWeight); + +} //NYql diff --git a/yt/yql/providers/yt/provider/yql_yt_peephole.cpp b/yt/yql/providers/yt/provider/yql_yt_peephole.cpp new file mode 100644 index 0000000000..d3dbdfd48b --- /dev/null +++ b/yt/yql/providers/yt/provider/yql_yt_peephole.cpp @@ -0,0 +1,162 @@ +#include "yql_yt_provider_impl.h" + +#include <yql/essentials/ast/yql_expr.h> +#include <yql/essentials/core/yql_graph_transformer.h> +#include <yql/essentials/core/yql_opt_utils.h> +#include <yql/essentials/providers/common/transform/yql_optimize.h> +#include <yt/yql/providers/yt/common/yql_configuration.h> +#include <yt/yql/providers/yt/expr_nodes/yql_yt_expr_nodes.h> +#include <yt/yql/providers/yt/common/yql_names.h> + +#include <library/cpp/yson/node/node_io.h> +#include <yt/cpp/mapreduce/common/helpers.h> + +namespace NYql { + +using namespace NNodes; + +namespace { + +class TYtPeepholeTransformer : public TOptimizeTransformerBase { +public: + TYtPeepholeTransformer(TYtState::TPtr state, const THashMap<TString, TString>& settings) + : TOptimizeTransformerBase(state ? state->Types : nullptr, NLog::EComponent::ProviderYt, {}) + , State_(state) + , Settings_(settings) + { +#define HNDL(name) "Peephole-"#name, Hndl(&TYtPeepholeTransformer::name) + AddHandler(0, &TYtLength::Match, HNDL(OptimizeLength)); + AddHandler(0, &TYtDqWrite::Match, HNDL(OptimizeYtDqWrite)); +#undef HNDL + } + +private: + TMaybeNode<TExprBase> OptimizeLength(TExprBase node, TExprContext& ctx) { + std::optional<size_t> lengthRes; + const auto& input = node.Cast<TYtLength>().Input(); + if (const auto& out = input.Maybe<TYtOutput>()) { + if (const auto& info = TYtTableBaseInfo::Parse(out.Cast()); info->Stat) { + lengthRes = info->Stat->RecordsCount; + } + } else if (const auto& read = input.Maybe<TYtReadTable>()) { + lengthRes = 0ULL; + for (auto path: read.Cast().Input().Item(0).Paths()) { + if (const auto& info = TYtTableBaseInfo::Parse(path.Table()); info->Stat && info->Meta && !info->Meta->IsDynamic && path.Ranges().Maybe<TCoVoid>()) + lengthRes = *lengthRes + info->Stat->RecordsCount; + else { + lengthRes = std::nullopt; + break; + } + } + } + + if (lengthRes) { + return Build<TCoUint64>(ctx, node.Pos()) + .Literal().Value(ToString(*lengthRes), TNodeFlags::Default).Build() + .Done(); + } + + return node; + } + + TMaybeNode<TExprBase> OptimizeYtDqWrite(TExprBase node, TExprContext& ctx) { + const auto write = node.Cast<TYtDqWrite>(); + + const auto& items = GetSeqItemType(write.Input().Ref().GetTypeAnn())->Cast<TStructExprType>()->GetItems(); + auto expand = ctx.Builder(write.Pos()) + .Callable("ExpandMap") + .Add(0, write.Input().Ptr()) + .Lambda(1) + .Param("item") + .Do([&](TExprNodeBuilder& lambda) -> TExprNodeBuilder& { + ui32 i = 0U; + for (const auto& item : items) { + lambda.Callable(i++, "Member") + .Arg(0, "item") + .Atom(1, item->GetName()) + .Seal(); + } + return lambda; + }) + .Seal() + .Seal().Build(); + + TYtDqWideWrite wideWrite = Build<TYtDqWideWrite>(ctx, write.Pos()) + .Input(std::move(expand)) + .Settings(write.Settings()) + .Done(); + + if (Settings_.empty()) { + return Build<TCoDiscard>(ctx, write.Pos()) + .Input(wideWrite) + .Done(); + } + + auto cluster = Settings_.at("yt_cluster"); + auto server = Settings_.at("yt_server"); + auto table = Settings_.at("yt_table"); + auto tableName = Settings_.at("yt_tableName"); + auto tableType = Settings_.at("yt_tableType"); + auto writeOptions = Settings_.at("yt_writeOptions"); + auto outSpec = Settings_.at("yt_outSpec"); + auto tx = Settings_.at("yt_tx"); + + YQL_ENSURE(!HasSetting(wideWrite.Settings().Ref(), "table")); + + TMaybeNode<TCoSecureParam> secParams; + if (State_->Configuration->Auth.Get().GetOrElse(TString())) { + secParams = Build<TCoSecureParam>(ctx, node.Pos()).Name().Build(TString("cluster:default_").append(cluster)).Done(); + } + + auto settings = Build<TCoNameValueTupleList>(ctx, node.Pos()) + .Add() + .Name().Value("table", TNodeFlags::Default).Build() + .Value<TCoAtom>().Value(table).Build() + .Build() + .Add() + .Name().Value("server", TNodeFlags::Default).Build() + .Value<TCoAtom>().Value(server).Build() + .Build() + .Add() + .Name().Value("outSpec", TNodeFlags::Default).Build() + .Value<TCoAtom>().Value(outSpec).Build() + .Build() + .Add() + .Name().Value("secureParams", TNodeFlags::Default).Build() + .Value(secParams) + .Build() + .Add() + .Name().Value("writerOptions", TNodeFlags::Default).Build() + .Value<TCoAtom>().Value(writeOptions).Build() + .Build() + .Add() + .Name().Value("tx", TNodeFlags::Default).Build() + .Value<TCoAtom>().Value(tx).Build() + .Build() + .Add() // yt_file gateway specific + .Name().Value("tableName", TNodeFlags::Default).Build() + .Value<TCoAtom>().Value(tableName).Build() + .Build() + .Add() // yt_file gateway specific + .Name().Value("tableType", TNodeFlags::Default).Build() + .Value<TCoAtom>().Value(tableType).Build() + .Build() + .Done().Ptr(); + + return Build<TCoDiscard>(ctx, write.Pos()) + .Input(ctx.ChangeChild(wideWrite.Ref(), TYtDqWideWrite::idx_Settings, std::move(settings))) + .Done(); + } + + const TYtState::TPtr State_; + const THashMap<TString, TString> Settings_; +}; + +} + +THolder<IGraphTransformer> CreateYtPeepholeTransformer(TYtState::TPtr state, const THashMap<TString, TString>& settings) { + return MakeHolder<TYtPeepholeTransformer>(std::move(state), settings); +} + +} // namespace NYql + diff --git a/yt/yql/providers/yt/provider/yql_yt_physical_finalizing.cpp b/yt/yql/providers/yt/provider/yql_yt_physical_finalizing.cpp new file mode 100644 index 0000000000..01cf4708e7 --- /dev/null +++ b/yt/yql/providers/yt/provider/yql_yt_physical_finalizing.cpp @@ -0,0 +1,3060 @@ +#include "yql_yt_provider_impl.h" +#include "yql_yt_provider.h" +#include "yql_yt_helpers.h" +#include "yql_yt_optimize.h" +#include "yql_yt_horizontal_join.h" + +#include <yt/yql/providers/yt/expr_nodes/yql_yt_expr_nodes.h> +#include <yt/yql/providers/yt/common/yql_configuration.h> +#include <yt/yql/providers/yt/opt/yql_yt_key_selector.h> +#include <yt/yql/providers/yt/lib/expr_traits/yql_expr_traits.h> +#include <yql/essentials/providers/result/expr_nodes/yql_res_expr_nodes.h> +#include <yql/essentials/providers/common/codec/yql_codec_type_flags.h> +#include <yql/essentials/providers/common/provider/yql_provider.h> +#include <yql/essentials/core/yql_expr_optimize.h> +#include <yql/essentials/core/yql_graph_transformer.h> +#include <yql/essentials/core/yql_expr_type_annotation.h> +#include <yql/essentials/core/yql_opt_utils.h> +#include <yql/essentials/ast/yql_expr.h> +#include <yql/essentials/utils/log/log.h> + +#include <util/generic/vector.h> +#include <util/generic/string.h> +#include <util/generic/xrange.h> +#include <util/generic/bitmap.h> +#include <util/generic/map.h> +#include <util/generic/hash.h> +#include <util/generic/algorithm.h> +#include <util/string/cast.h> +#include <util/str_stl.h> + +#include <utility> +#include <unordered_set> +#include <iterator> +#include <algorithm> + +namespace NYql { + +using namespace NNodes; + +namespace { + +TYtOutputOpBase GetRealOperation(TExprBase op) { + if (const auto mayTry = op.Maybe<TYtTryFirst>()) + return mayTry.Cast().First(); + return op.Cast<TYtOutputOpBase>(); +} + +class TYtPhysicalFinalizingTransformer : public TSyncTransformerBase { +public: + TYtPhysicalFinalizingTransformer(TYtState::TPtr state) + : State_(state) + { + } + + TStatus DoTransform(TExprNode::TPtr input, TExprNode::TPtr& output, TExprContext& ctx) final { + output = input; + + TOpDeps opDeps; + std::vector<const TExprNode*> opDepsOrder; + TNodeSet hasWorldDeps; // Operations, which have world dependencies on them from another nodes + TNodeSet toCombine; + TNodeSet neverCombine; + TNodeSet lefts; + TNodeMap<TString> commits; + TExprNode::TListType publishNodes; + + bool enableChunkCombining = IsChunkCombiningEnabled(); + + auto storeDep = [&opDeps, &opDepsOrder](const TYtOutput& out, const TExprNode* reader, const TExprNode* sec, const TExprNode* path) { + const auto op = out.Operation().Raw(); + auto& res = opDeps[op]; + if (res.empty()) { + opDepsOrder.push_back(op); + } + res.emplace_back(reader, sec, out.Raw(), path); + }; + + TParentsMap parentsMap; + GatherParents(*input, parentsMap); + + TNodeSet visitedOutParents; + std::vector<const TExprNode*> outs; + VisitExpr(input, [&](const TExprNode::TPtr& node)->bool { + if (auto maybeOp = TMaybeNode<TYtTransientOpBase>(node)) { + auto op = maybeOp.Cast(); + for (auto section: op.Input()) { + for (auto path: section.Paths()) { + if (auto maybeOutput = path.Table().Maybe<TYtOutput>()) { + visitedOutParents.insert(path.Raw()); + auto out = maybeOutput.Cast(); + storeDep(out, op.Raw(), section.Raw(), path.Raw()); + if (enableChunkCombining) { + CollectForCombine(out, toCombine, neverCombine); + } + } + } + } + } + else if (auto maybeRead = TMaybeNode<TYtReadTable>(node)) { + auto read = maybeRead.Cast(); + for (auto section: read.Input()) { + for (auto path: section.Paths()) { + if (auto maybeOutput = path.Table().Maybe<TYtOutput>()) { + visitedOutParents.insert(path.Raw()); + auto out = maybeOutput.Cast(); + storeDep(out, read.Raw(), section.Raw(), path.Raw()); + if (enableChunkCombining) { + CollectForCombine(out, toCombine, neverCombine); + } + } + } + } + } + else if (auto maybePublish = TMaybeNode<TYtPublish>(node)) { + auto publish = maybePublish.Cast(); + visitedOutParents.insert(publish.Input().Raw()); + publishNodes.push_back(publish.Ptr()); + for (auto out: publish.Input()) { + storeDep(out, publish.Raw(), nullptr, nullptr); + } + } + else if (auto maybeOutput = TMaybeNode<TYtOutput>(node)) { + outs.push_back(node.Get()); + } + else if (auto maybeLeft = TMaybeNode<TCoLeft>(node)) { + if (auto maybeOp = maybeLeft.Input().Maybe<TYtOutputOpBase>()) { + hasWorldDeps.insert(maybeOp.Cast().Raw()); + } + lefts.insert(maybeLeft.Raw()); + } + else if (auto maybeCommit = TMaybeNode<TCoCommit>(node)) { + if (auto ds = maybeCommit.DataSink().Maybe<TYtDSink>()) { + if (ProcessedMergePublish.find(node->UniqueId()) == ProcessedMergePublish.end()) { + commits.emplace(node.Get(), ds.Cast().Cluster().Value()); + } + } + } + + return true; + }); + + for (auto out: outs) { + std::vector<const TExprNode*> readers; + if (auto it = parentsMap.find(out); it != parentsMap.end()) { + std::copy_if(it->second.begin(), it->second.end(), + std::back_inserter(readers), + [&visitedOutParents](auto n) { + return !visitedOutParents.contains(n); + } + ); + } + + if (!readers.empty()) { + std::stable_sort(readers.begin(), readers.end(), [](auto l, auto r) { return l->UniqueId() < r->UniqueId(); }); + for (auto n: readers) { + YQL_ENSURE(!TYtPath::Match(n)); // All YtPath usages must be gathered in previous VisitExpr + storeDep(TYtOutput(out), n, nullptr, nullptr); + if (enableChunkCombining && (TYtTableContent::Match(n) || TResWriteBase::Match(n) || TYtStatOut::Match(n))) { + CollectForCombine(TYtOutput(out), toCombine, neverCombine); + } + } + } + } + + YQL_ENSURE(opDeps.size() == opDepsOrder.size()); + + const auto disableOptimizers = State_->Configuration->DisableOptimizers.Get().GetOrElse(TSet<TString>()); + + TStatus status = TStatus::Ok; + if (!disableOptimizers.contains("UnorderedPublishTarget")) { + status = UnorderedPublishTarget(input, output, publishNodes, ctx); + if (status.Level != TStatus::Ok) { + return status; + } + } + + if (!disableOptimizers.contains("MergePublish")) { + status = MergePublish(input, output, commits, ctx); + if (status.Level != TStatus::Ok) { + return status; + } + } + + if (!disableOptimizers.contains("BypassMergeBeforeLength")) { + status = BypassMergeOrCopyBeforeLength(input, output, opDeps, lefts, ctx); + if (status.Level != TStatus::Ok) { + return status; + } + } + + if (!disableOptimizers.contains("AlignPublishTypes")) { + status = AlignPublishTypes(input, output, opDeps, ctx); + if (status.Level != TStatus::Ok) { + return status; + } + } + + if (!disableOptimizers.contains("SuppressOuts")) { + status = OptimizeUnusedOuts(input, output, opDeps, lefts, ctx); + if (status.Level != TStatus::Ok) { + return status; + } + } + + if (!disableOptimizers.contains("UnorderedOuts") && ctx.IsConstraintEnabled<TSortedConstraintNode>()) { + status = OptimizeUnorderedOuts(input, output, opDepsOrder, opDeps, lefts, ctx); + if (status.Level != TStatus::Ok) { + return status; + } + } + + if (!disableOptimizers.contains("FieldSubsetForMultiUsage")) { + status = OptimizeFieldSubsetForMultiUsage(input, output, opDeps, lefts, ctx); + if (status.Level != TStatus::Ok) { + return status; + } + } + + if (State_->Configuration->MaxInputTables.Get().Defined() || State_->Configuration->MaxInputTablesForSortedMerge.Get().Defined()) { + // Run it after UnorderedOuts, because sorted YtMerge may become unsorted after UnorderedOuts + status = SplitLargeInputs(input, output, ctx, !disableOptimizers.contains("SplitLargeMapInputs")); + if (status.Level != TStatus::Ok) { + return status; + } + } + + if (!disableOptimizers.contains("MergeMultiOuts")) { + status = OptimizeMultiOuts(input, output, opDeps, lefts, ctx); + if (status.Level != TStatus::Ok) { + return status; + } + } + + if (!State_->Configuration->DisableFuseOperations.Get().GetOrElse(DEFAULT_DISABLE_FUSE_OPERATIONS) && + !disableOptimizers.contains("FuseMultiOutsWithOuterMaps")) { + status = FuseMultiOutsWithOuterMaps(input, output, opDeps, lefts, hasWorldDeps, ctx); + if (status.Level != TStatus::Ok) { + return status; + } + } + + if (enableChunkCombining) { + EraseNodesIf(toCombine, [&neverCombine](const auto& entry) { return neverCombine.count(entry) != 0; }); + if (!toCombine.empty()) { + status = AddChunkCombining(input, output, toCombine, lefts, opDeps, ctx); + if (status.Level != TStatus::Ok) { + return status; + } + } + toCombine.clear(); + neverCombine.clear(); + } + lefts.clear(); + + TParentsMap limits; // op -> settings with limits + TNodeSet noLimits; // writers + for (auto& x: opDeps) { + auto writer = x.first; + if (!TYtTransientOpBase::Match(writer)) { + continue; + } + if (NYql::HasSetting(*writer->Child(TYtTransientOpBase::idx_Settings), EYtSettingType::Limit)) { + continue; + } + if (writer->HasResult() && writer->GetResult().Type() == TExprNode::World) { + continue; + } + + bool canHaveLimit = TYtTransientOpBase(writer).Output().Size() == 1; + if (canHaveLimit) { + TString usedCluster; + for (auto item: x.second) { + if (!std::get<1>(item)) { // YtLength, YtPublish + canHaveLimit = false; + break; + } + for (auto path: TYtSection(std::get<1>(item)).Paths()) { + if (!path.Ranges().Maybe<TCoVoid>()) { + canHaveLimit = false; + break; + } + } + if (!canHaveLimit) { + break; + } + bool hasTake = false; + for (auto setting: TYtSection(std::get<1>(item)).Settings()) { + auto kind = FromString<EYtSettingType>(setting.Name().Value()); + if (EYtSettingType::Take == kind || EYtSettingType::Skip == kind) { + TSyncMap syncList; + if (!IsYtCompleteIsolatedLambda(setting.Value().Ref(), syncList, usedCluster, false) || !syncList.empty()) { + hasTake = false; + break; + } + if (EYtSettingType::Take == kind) { + hasTake = true; + } + } + } + + if (!hasTake) { + canHaveLimit = false; + break; + } + } + } + + if (!canHaveLimit) { + noLimits.insert(writer); + continue; + } + + auto& limit = limits[writer]; + for (auto item: x.second) { + limit.insert(TYtSection(std::get<1>(item)).Settings().Raw()); + } + } + + for (auto writer : noLimits) { + limits.erase(writer); + } + + status = OptimizeExpr(output, output, [&](const TExprNode::TPtr& node, TExprContext& ctx) { + if (!node->HasResult() || node->GetResult().Type() != TExprNode::World) { + const bool opWithJobs = TYtWithUserJobsOpBase::Match(node.Get()); + if (opWithJobs || TYtSort::Match(node.Get()) || TYtMerge::Match(node.Get())) { + auto ret = Limits(node, limits, ctx); + if (ret != node) { + if (ret) { + YQL_CLOG(INFO, ProviderYt) << "PhysicalFinalizing-Limits"; + } + return ret; + } + + if (opWithJobs) { + ret = LengthOverPhysicalList(node, opDeps, ctx); + if (ret != node) { + if (ret) { + YQL_CLOG(INFO, ProviderYt) << "PhysicalFinalizing-LengthOverPhysicalList"; + } + return ret; + } + ret = TopSortForProducers(node, opDeps, ctx); + if (ret != node) { + if (ret) { + YQL_CLOG(INFO, ProviderYt) << "PhysicalFinalizing-TopSortForProducers"; + } + return ret; + } + } + } + } + return node; + }, ctx, TOptimizeExprSettings(nullptr)); + + if (status.Level != TStatus::Ok) { + return status; + } + + if (!disableOptimizers.contains("HorizontalJoin")) { + status = THorizontalJoinOptimizer(State_, opDepsOrder, opDeps, hasWorldDeps, &ProcessedHorizontalJoin).Optimize(output, output, ctx); + if (status.Level != TStatus::Ok) { + return status; + } + } + + if (!disableOptimizers.contains("MultiHorizontalJoin")) { + status = TMultiHorizontalJoinOptimizer(State_, opDepsOrder, opDeps, hasWorldDeps).Optimize(output, output, ctx); + if (status.Level != TStatus::Ok) { + return status; + } + } + + if (!disableOptimizers.contains("OutHorizontalJoin")) { + status = TOutHorizontalJoinOptimizer(State_, opDepsOrder, opDeps, hasWorldDeps).Optimize(output, output, ctx); + if (status.Level != TStatus::Ok) { + return status; + } + } + + if (const auto mode = State_->Configuration->ColumnGroupMode.Get().GetOrElse(EColumnGroupMode::Disable); mode != EColumnGroupMode::Disable) { + status = CalculateColumnGroups(input, output, opDepsOrder, opDeps, mode, ctx); + if (status.Level != TStatus::Ok) { + return status; + } + } + + return status; + } + + void Rewind() final { + ProcessedMergePublish.clear(); + ProcessedSplitLargeInputs.clear(); + ProcessedUnusedOuts.clear(); + ProcessedMultiOuts.clear(); + ProcessedHorizontalJoin.clear(); + ProcessedFieldSubsetForMultiUsage.clear(); + ProcessedFuseWithOuterMaps.clear(); + ProcessedCalculateColumnGroups.clear(); + } + +private: + + static bool IsBeingExecuted(const TExprNode& op) { + if (TYtTryFirst::Match(&op)) { + return op.Head().StartsExecution() || op.Head().HasResult(); + } else { + return op.StartsExecution() || op.HasResult(); + } + } + + static THashSet<TStringBuf> OPS_WITH_SORTED_OUTPUT; + + void CollectForCombine(const TYtOutput& output, TNodeSet& toCombine, TNodeSet& neverCombine) + { + if (neverCombine.find(output.Raw()) != neverCombine.end()) { + return; + } + + const auto op = GetOutputOp(output); + + if (auto maybeMerge = op.Maybe<TYtMerge>()) { + auto merge = maybeMerge.Cast(); + + if (NYql::HasSetting(merge.Settings().Ref(), EYtSettingType::CombineChunks)) { + + // inputs of combining merge should never be combined + for (auto section: merge.Input()) { + for (auto path: section.Paths()) { + if (auto maybeOut = path.Table().Maybe<TYtOutput>()) { + auto out = maybeOut.Cast(); + neverCombine.insert(out.Raw()); + } + } + } + return; + } + } + + if (!op.Ref().HasResult()) { + return; + } + + auto outTable = GetOutTable(output).Cast<TYtOutTable>(); + auto tableInfo = TYtTableBaseInfo::Parse(outTable); + + TMaybe<ui64> maybeLimit = State_->Configuration->MinTempAvgChunkSize.Get(); + + if (!maybeLimit.Defined()) { + return; + } + + ui64 limit = *maybeLimit.Get(); + + if (limit == 0) { + // always combine + toCombine.insert(output.Raw()); + return; + + } + + YQL_ENSURE(tableInfo->Stat); + ui64 chunksCount = tableInfo->Stat->ChunkCount; + ui64 dataSize = tableInfo->Stat->DataSize; + if (chunksCount > 1 && dataSize > chunksCount) { + ui64 chunkSize = dataSize / chunksCount; + + if (chunkSize < limit) { + toCombine.insert(output.Raw()); + } + } + } + + bool IsChunkCombiningEnabled() + { + return State_->Configuration->MinTempAvgChunkSize.Get().Defined(); + } + + TStatus AddChunkCombining(TExprNode::TPtr input, TExprNode::TPtr& output, const TNodeSet& toCombine, + const TNodeSet& lefts, const TOpDeps& opDeps, TExprContext& ctx) + { + TNodeOnNodeOwnedMap replaces; + TNodeOnNodeOwnedMap newOps; // Old output -> new op + for (auto node: toCombine) { + TYtOutput ytOutput(node); + const auto oldOp = GetOutputOp(ytOutput); + + auto combiningOp = + Build<TYtMerge>(ctx, oldOp.Pos()) + .World<TCoWorld>().Build() + .DataSink(oldOp.DataSink()) + .Output() + .Add() + .InitFrom(GetOutTable(ytOutput).Cast<TYtOutTable>()) + .Name().Value("").Build() + .Stat<TCoVoid>().Build() + .Build() + .Build() + .Input() + .Add() + .Paths() + .Add() + .Table<TYtOutput>() // clone to exclude infinitive loop in RemapExpr + .InitFrom(ytOutput) + .Build() + .Columns<TCoVoid>().Build() + .Ranges<TCoVoid>().Build() + .Stat<TCoVoid>().Build() + .Build() + .Build() + .Settings<TCoNameValueTupleList>() + .Build() + .Build() + .Build() + .Settings() + .Add() + .Name().Value(ToString(EYtSettingType::CombineChunks)).Build() + .Build() + .Build() + .Done(); + + auto newYtOutput = + Build<TYtOutput>(ctx, ytOutput.Pos()) + .Operation(combiningOp) + .OutIndex().Value("0").Build() + .Done(); + + replaces[node] = newYtOutput.Ptr(); + newOps[node] = combiningOp.Ptr(); + } + + if (!newOps.empty()) { + for (auto node: lefts) { + TCoLeft left(node); + + if (auto maybeOp = left.Input().Maybe<TYtOutputOpBase>()) { + auto op = maybeOp.Cast(); + auto depsIt = opDeps.find(op.Raw()); + if (depsIt != opDeps.end()) { + TExprNode::TListType toSync; + for (const auto& dep : depsIt->second) { + const TExprNode* oldOutput = std::get<2>(dep); + auto it = newOps.find(oldOutput); + if (it != newOps.end()) { + auto world = ctx.NewCallable(left.Pos(), TCoLeft::CallableName(), { it->second }); + toSync.push_back(world); + } + } + if (!toSync.empty()) { + auto newLeft = ctx.NewCallable(left.Pos(), TCoSync::CallableName(), std::move(toSync)); + YQL_CLOG(INFO, ProviderYt) << "PhysicalFinalizing-AddChunkCombining-Worlds"; + replaces[node] = newLeft; + } + } + } + } + } + + if (!replaces.empty()) { + YQL_CLOG(INFO, ProviderYt) << "PhysicalFinalizing-AddChunkCombining"; + return RemapExpr(input, output, replaces, ctx, TOptimizeExprSettings(State_->Types)); + } + return TStatus::Ok; + } + + TStatus UnorderedPublishTarget(TExprNode::TPtr input, TExprNode::TPtr& output, const TExprNode::TListType& publishNodes, TExprContext& ctx) { + TNodeOnNodeOwnedMap replaceMap; + for (auto n: publishNodes) { + auto publish = TYtPublish(n); + + auto cluster = publish.DataSink().Cluster().StringValue(); + auto pubTableInfo = TYtTableInfo(publish.Publish()); + if (auto commitEpoch = pubTableInfo.CommitEpoch.GetOrElse(0)) { + const TYtTableDescription& nextDescription = State_->TablesData->GetTable(cluster, pubTableInfo.Name, commitEpoch); + YQL_ENSURE(nextDescription.RowSpec); + if (nextDescription.RowSpecSortReady && !nextDescription.RowSpec->IsSorted()) { + bool modified = false; + TVector<TYtOutput> outs; + for (auto out: publish.Input()) { + if (!IsUnorderedOutput(out) && TYqlRowSpecInfo(GetOutTable(out).Cast<TYtOutTable>().RowSpec()).IsSorted()) { + outs.push_back(Build<TYtOutput>(ctx, out.Pos()) + .InitFrom(out) + .Mode() + .Value(ToString(EYtSettingType::Unordered)) + .Build() + .Done()); + modified = true; + } else { + outs.push_back(out); + } + } + if (modified) { + replaceMap[publish.Raw()] = Build<TYtPublish>(ctx, publish.Pos()) + .InitFrom(publish) + .Input() + .Add(outs) + .Build() + .Done().Ptr(); + } + } + } + } + if (!replaceMap.empty()) { + YQL_CLOG(INFO, ProviderYt) << "PhysicalFinalizing-UnorderedPublishTarget"; + return RemapExpr(input, output, replaceMap, ctx, TOptimizeExprSettings(State_->Types)); + } + return TStatus::Ok; + } + + TStatus MergePublish(TExprNode::TPtr input, TExprNode::TPtr& output, const TNodeMap<TString>& commits, TExprContext& ctx) { + for (auto& c: commits) { + THashMap<TString, TVector<const TExprNode*>> groupedPublishesByDst; + VisitExprByFirst(*c.first, [&](const TExprNode& node) { + if (auto maybePub = TMaybeNode<TYtPublish>(&node)) { + auto pub = maybePub.Cast(); + if (pub.DataSink().Cluster().Value() == c.second && !NYql::HasSetting(pub.Settings().Ref(), EYtSettingType::MonotonicKeys)) { + groupedPublishesByDst[TString{pub.Publish().Name().Value()}].push_back(&node); + } + } else if (auto maybeCommit = TMaybeNode<TCoCommit>(&node)) { + if (auto maybeSink = maybeCommit.DataSink().Maybe<TYtDSink>()) { + // Stop traversing when got another commit on the same cluster + return maybeSink.Cast().Cluster().Value() != c.second || &node == c.first; + } + } + return true; + }); + TVector<TString> dstTables; + for (auto& grp: groupedPublishesByDst) { + if (grp.second.size() > 1) { + dstTables.push_back(grp.first); + } + } + if (dstTables.size() > 1) { + ::Sort(dstTables); + } + for (auto& tbl: dstTables) { + auto& grp = groupedPublishesByDst[tbl]; + while (grp.size() > 1) { + TNodeOnNodeOwnedMap replaceMap; + // Optimize only two YtPublish nodes at once to properly update world dependencies + auto last = TYtPublish(grp[0]); + auto prev = TYtPublish(grp[1]); + if (last.World().Raw() != prev.Raw()) { + // Has extra dependencies. Don't merge + grp.erase(grp.begin()); + continue; + } + + auto mode = NYql::GetSetting(last.Settings().Ref(), EYtSettingType::Mode); + YQL_ENSURE(mode && FromString<EYtWriteMode>(mode->Child(1)->Content()) == EYtWriteMode::Append); + YQL_ENSURE(!NYql::HasSetting(last.Settings().Ref(), EYtSettingType::Initial)); + + replaceMap.emplace(grp[0], + Build<TYtPublish>(ctx, grp[0]->Pos()) + .InitFrom(prev) + .Input() + .Add(prev.Input().Ref().ChildrenList()) + .Add(last.Input().Ref().ChildrenList()) + .Build() + .Done().Ptr() + ); + replaceMap.emplace(grp[1], prev.World().Ptr()); + + YQL_CLOG(INFO, ProviderYt) << "PhysicalFinalizing-MergePublish"; + return RemapExpr(input, output, replaceMap, ctx, TOptimizeExprSettings(State_->Types)); + } + } + ProcessedMergePublish.insert(c.first->UniqueId()); + } + return TStatus::Ok; + }; + + TStatus OptimizeFieldSubsetForMultiUsage(TExprNode::TPtr input, TExprNode::TPtr& output, const TOpDeps& opDeps, const TNodeSet& lefts, TExprContext& ctx) { + TVector<std::pair<const TOpDeps::value_type*, THashSet<TString>>> matchedOps; + const bool useNativeDescSort = State_->Configuration->UseNativeDescSort.Get().GetOrElse(DEFAULT_USE_NATIVE_DESC_SORT); + for (auto& x: opDeps) { + auto writer = x.first; + + if (!ProcessedFieldSubsetForMultiUsage.insert(writer->UniqueId()).second) { + continue; + } + + if (IsBeingExecuted(*writer)) { + continue; + } + if (!TYtMap::Match(writer) && !TYtMerge::Match(writer)) { + continue; + } + if (writer->GetTypeAnn()->Cast<TTupleExprType>()->GetItems()[1]->Cast<TListExprType>()->GetItemType()->GetKind() == ETypeAnnotationKind::Variant) { + // Operation with multi-output + continue; + } + if (NYql::HasSetting(*writer->Child(TYtTransientOpBase::idx_Settings), EYtSettingType::SortLimitBy)) { + continue; + } + + auto outTable = TYtTransientOpBase(writer).Output().Item(0); + const TYqlRowSpecInfo rowSpec(outTable.RowSpec()); + if (rowSpec.HasAuxColumns()) { + continue; + } + + auto type = outTable.Ref().GetTypeAnn()->Cast<TListExprType>()->GetItemType()->Cast<TStructExprType>(); + + bool good = true; + THashSet<TString> usedColumns; + if (NYql::HasSetting(*writer->Child(TYtTransientOpBase::idx_Settings), EYtSettingType::KeepSorted)) { + for (size_t i = 0; i < rowSpec.SortedBy.size(); ++i) { + usedColumns.insert(rowSpec.SortedBy[i]); + } + } + + for (auto& item: x.second) { + if (auto rawSection = std::get<1>(item)) { + if (HasNonEmptyKeyFilter(TYtSection(rawSection))) { + // wait until key filter values are calculated and pushed to Path/Ranges + good = false; + break; + } + } + auto rawPath = std::get<3>(item); + if (!rawPath) { + if (TYtLength::Match(std::get<0>(item))) { + continue; + } + good = false; + break; + } + auto path = TYtPath(rawPath); + + auto columns = TYtColumnsInfo(path.Columns()); + if (!columns.HasColumns()) { + good = false; + break; + } + + if (type->GetSize() <= columns.GetColumns()->size()) { + good = false; + break; + } + std::transform(columns.GetColumns()->cbegin(), columns.GetColumns()->cend(), + std::inserter(usedColumns, usedColumns.end()), + [] (const TYtColumnsInfo::TColumn& c) { return c.Name; } + ); + + if (!path.Ranges().Maybe<TCoVoid>()) { + // add columns which are implicitly used by path.Ranges(), but not included in path.Columns(); + const auto ranges = TYtRangesInfo(path.Ranges()); + const size_t usedKeyPrefix = ranges.GetUsedKeyPrefixLength(); + YQL_ENSURE(usedKeyPrefix <= rowSpec.SortedBy.size()); + for (size_t i = 0; i < usedKeyPrefix; ++i) { + usedColumns.insert(rowSpec.SortedBy[i]); + } + } + + if (type->GetSize() <= usedColumns.size()) { + good = false; + break; + } + } + + if (good && usedColumns.size() < type->GetSize()) { + matchedOps.emplace_back(&x, std::move(usedColumns)); + } + } + + if (matchedOps.empty()) { + return TStatus::Ok; + } + + TNodeOnNodeOwnedMap replaces; + TNodeOnNodeOwnedMap newOps; + for (auto& item: matchedOps) { + auto writer = item.first->first; + auto& columns = item.second; + auto outTable = TYtTransientOpBase(writer).Output().Item(0); + auto type = outTable.Ref().GetTypeAnn()->Cast<TListExprType>()->GetItemType()->Cast<TStructExprType>(); + + TVector<const TItemExprType*> structItems; + for (auto& column: columns) { + auto pos = type->FindItem(column); + YQL_ENSURE(pos); + structItems.push_back(type->GetItems()[*pos]); + } + auto outStructType = ctx.MakeType<TStructExprType>(structItems); + auto distinct = outTable.Ref().GetConstraint<TDistinctConstraintNode>(); + if (distinct) { + distinct = distinct->FilterFields(ctx, [&columns](const TPartOfConstraintBase::TPathType& path) { return !path.empty() && columns.contains(path.front()); }); + } + + TExprNode::TPtr newOp; + if (auto maybeMap = TMaybeNode<TYtMap>(writer)) { + TYtMap map = maybeMap.Cast(); + + auto mapper = ctx.Builder(map.Mapper().Pos()) + .Lambda() + .Param("stream") + .Callable(NYql::HasSetting(map.Settings().Ref(), EYtSettingType::Ordered) ? TCoOrderedFlatMap::CallableName() : TCoFlatMap::CallableName()) + .Apply(0, map.Mapper().Ref()) + .With(0, "stream") + .Seal() + .Lambda(1) + .Param("item") + .Callable(TCoJust::CallableName()) + .Callable(0, TCoAsStruct::CallableName()) + .Do([&](TExprNodeBuilder& parent) -> TExprNodeBuilder& { + size_t index = 0; + for (auto& column: columns) { + parent + .List(index++) + .Atom(0, column) + .Callable(1, TCoMember::CallableName()) + .Arg(0, "item") + .Atom(1, column) + .Seal() + .Seal(); + } + return parent; + }) + .Seal() + .Seal() + .Seal() + .Seal() + .Seal() + .Build(); + + TYtOutTableInfo mapOut(outStructType, State_->Configuration->UseNativeYtTypes.Get().GetOrElse(DEFAULT_USE_NATIVE_YT_TYPES) ? NTCF_ALL : NTCF_NONE); + + if (ctx.IsConstraintEnabled<TSortedConstraintNode>()) { + if (auto sorted = outTable.Ref().GetConstraint<TSortedConstraintNode>()) { + auto prefixLength = sorted->GetContent().size(); + for (size_t i = 0; i < prefixLength; ++i) { + bool found = false; + for (const auto& path : sorted->GetContent()[i].first) + if (found = path.size() == 1U && columns.contains(path.front())) + break; + + if (!found) + prefixLength = i; + } + + if (sorted = sorted->CutPrefix(prefixLength, ctx)) { + if (sorted = sorted->FilterFields(ctx, [&columns](const TPartOfConstraintBase::TPathType& path) { return !path.empty() && columns.contains(path.front()); })) { + TKeySelectorBuilder builder(map.Mapper().Pos(), ctx, useNativeDescSort, outStructType); + builder.ProcessConstraint(*sorted); + builder.FillRowSpecSort(*mapOut.RowSpec); + + if (builder.NeedMap()) { + mapper = ctx.Builder(map.Mapper().Pos()) + .Lambda() + .Param("stream") + .Apply(builder.MakeRemapLambda(true)) + .With(0) + .Apply(mapper) + .With(0, "stream") + .Seal() + .Done() + .Seal() + .Seal() + .Build(); + } + } + } + } + } else { + mapOut.RowSpec->CopySortness(ctx, TYqlRowSpecInfo(outTable.RowSpec())); + } + mapOut.SetUnique(distinct, map.Mapper().Pos(), ctx); + mapOut.RowSpec->SetConstraints(outTable.Ref().GetConstraintSet()); + + newOp = Build<TYtMap>(ctx, map.Pos()) + .InitFrom(map) + .Output() + .Add(mapOut.ToExprNode(ctx, map.Pos()).Cast<TYtOutTable>()) + .Build() + .Mapper(std::move(mapper)) + .Done().Ptr(); + } + else { + auto merge = TYtMerge(writer); + auto prevRowSpec = TYqlRowSpecInfo(merge.Output().Item(0).RowSpec()); + TYtOutTableInfo mergeOut(outStructType, prevRowSpec.GetNativeYtTypeFlags()); + mergeOut.RowSpec->CopySortness(ctx, prevRowSpec, TYqlRowSpecInfo::ECopySort::WithDesc); + mergeOut.SetUnique(distinct, merge.Pos(), ctx); + mergeOut.RowSpec->SetConstraints(outTable.Ref().GetConstraintSet()); + + if (auto nativeType = prevRowSpec.GetNativeYtType()) { + mergeOut.RowSpec->CopyTypeOrders(*nativeType); + } + + TSet<TStringBuf> columnSet; + for (auto& column: columns) { + columnSet.insert(column); + } + if (mergeOut.RowSpec->HasAuxColumns()) { + for (auto item: mergeOut.RowSpec->GetAuxColumns()) { + columnSet.insert(item.first); + } + } + + newOp = Build<TYtMerge>(ctx, merge.Pos()) + .InitFrom(merge) + .Input() + .Add(UpdateInputFields(merge.Input().Item(0), std::move(columnSet), ctx, false)) + .Build() + .Output() + .Add(mergeOut.ToExprNode(ctx, merge.Pos()).Cast<TYtOutTable>()) + .Build() + .Done().Ptr(); + } + + newOps[writer] = newOp; + for (auto& reader: item.first->second) { + if (TYtLength::Match(std::get<0>(reader))) { + auto rawLen = std::get<0>(reader); + auto len = TYtLength(rawLen); + replaces[rawLen] = Build<TYtLength>(ctx, len.Pos()) + .InitFrom(len) + .Input(ctx.ChangeChild(len.Input().Ref(), TYtOutput::idx_Operation, TExprNode::TPtr(newOp))) + .Done().Ptr(); + } else { + auto rawPath = std::get<3>(reader); + auto path = TYtPath(rawPath); + replaces[rawPath] = Build<TYtPath>(ctx, path.Pos()) + .InitFrom(path) + .Table(ctx.ChangeChild(path.Table().Ref(), TYtOutput::idx_Operation, TExprNode::TPtr(newOp))) + .Done().Ptr(); + } + } + } + if (!lefts.empty() && !newOps.empty()) { + for (auto node: lefts) { + TCoLeft left(node); + auto newIt = newOps.find(left.Input().Raw()); + if (newIt != newOps.end()) { + replaces[node] = ctx.ChangeChild(*node, TCoLeft::idx_Input, TExprNode::TPtr(newIt->second)); + } + } + } + + if (!replaces.empty()) { + YQL_CLOG(INFO, ProviderYt) << "PhysicalFinalizing-FieldSubsetForMultiUsage"; + return RemapExpr(input, output, replaces, ctx, TOptimizeExprSettings{State_->Types}); + } + return TStatus::Ok; + } + + TStatus OptimizeUnorderedOuts(TExprNode::TPtr input, TExprNode::TPtr& output, const std::vector<const TExprNode*>& opDepsOrder, const TOpDeps& opDeps, const TNodeSet& lefts, TExprContext& ctx) { + std::vector<const TExprNode*> matchedOps; + for (auto writer: opDepsOrder) { + if (!TYtEquiJoin::Match(writer) && !IsBeingExecuted(*writer)) { + matchedOps.push_back(writer); + } + } + + if (matchedOps.empty()) { + return TStatus::Ok; + } + + TNodeOnNodeOwnedMap replaces; + TNodeOnNodeOwnedMap newOps; + for (auto writer: matchedOps) { + TDynBitMap orderedOuts; + TDynBitMap unorderedOuts; + const auto& readers = opDeps.at(writer); + for (auto& item: readers) { + auto out = TYtOutput(std::get<2>(item)); + if (IsUnorderedOutput(out)) { + unorderedOuts.Set(FromString<size_t>(out.OutIndex().Value())); + } else { + orderedOuts.Set(FromString<size_t>(out.OutIndex().Value())); + } + } + if (!unorderedOuts.Empty()) { + if (orderedOuts.Empty() || !writer->IsCallable(OPS_WITH_SORTED_OUTPUT)) { + TExprNode::TPtr newOp; + if (const auto mayTry = TExprBase(writer).Maybe<TYtTryFirst>()) { + TExprNode::TPtr newOpFirst = MakeUnorderedOp(mayTry.Cast().First().Ref(), unorderedOuts, ctx); + TExprNode::TPtr newOpSecond = MakeUnorderedOp(mayTry.Cast().Second().Ref(), unorderedOuts, ctx); + if (newOpFirst || newOpSecond) { + newOp = Build<TYtTryFirst>(ctx, writer->Pos()) + .First(newOpFirst ? std::move(newOpFirst) : mayTry.Cast().First().Ptr()) + .Second(newOpSecond ? std::move(newOpSecond) : mayTry.Cast().Second().Ptr()) + .Done().Ptr(); + } + } else { + newOp = MakeUnorderedOp(*writer, unorderedOuts, ctx); + } + if (newOp) { + newOps[writer] = newOp; + } + for (auto& item: readers) { + auto out = std::get<2>(item); + replaces[out] = Build<TYtOutput>(ctx, out->Pos()) + .Operation(newOp ? newOp : out->ChildPtr(TYtOutput::idx_Operation)) + .OutIndex(out->ChildPtr(TYtOutput::idx_OutIndex)) + .Done().Ptr(); + } + } + } + } + if (!lefts.empty() && !newOps.empty()) { + for (auto node: lefts) { + TCoLeft left(node); + auto newIt = newOps.find(left.Input().Raw()); + if (newIt != newOps.end()) { + replaces[node] = ctx.ChangeChild(*node, TCoLeft::idx_Input, TExprNode::TPtr(newIt->second)); + } + } + } + + if (!replaces.empty()) { + YQL_CLOG(INFO, ProviderYt) << "PhysicalFinalizing-UnorderedOuts"; + return RemapExpr(input, output, replaces, ctx, TOptimizeExprSettings{State_->Types}); + } + return TStatus::Ok; + } + + TExprNode::TPtr MakeUnorderedOp(const TExprNode& node, const TDynBitMap& unorderedOuts, TExprContext& ctx) const { + if (!node.IsCallable(OPS_WITH_SORTED_OUTPUT)) { + return {}; + } + auto op = TYtOutputOpBase(&node); + + bool hasOtherSortedOuts = false; + bool changedOutSort = false; + TVector<TYtOutTable> outTables; + TExprNode::TListType filterColumns(op.Output().Size()); + for (size_t i = 0; i < op.Output().Size(); ++i) { + auto out = op.Output().Item(i); + if (unorderedOuts.Test(i)) { + auto rowSpec = TYtTableBaseInfo::GetRowSpec(out); + YQL_ENSURE(rowSpec); + if (rowSpec->IsSorted()) { + if (rowSpec->HasAuxColumns()) { + TVector<TString> columns; + for (auto item: rowSpec->GetType()->GetItems()) { + columns.emplace_back(item->GetName()); + } + filterColumns[i] = ToAtomList(columns, node.Pos(), ctx); + } + rowSpec->ClearSortness(ctx); + outTables.push_back(TYtOutTable(ctx.ChangeChild(out.Ref(), TYtOutTable::idx_RowSpec, rowSpec->ToExprNode(ctx, out.Pos()).Ptr()))); + changedOutSort = true; + } else { + outTables.push_back(out); + } + } else { + if (TYtTableBaseInfo::GetRowSpec(out)->IsSorted()) { + hasOtherSortedOuts = true; + } + outTables.push_back(out); + } + } + + bool isFill = false; + bool isYtDqProcessWrite = false; + int lambdaIdx = -1; + TExprNode::TPtr lambda; + if (TYtMap::Match(&node)) { + lambdaIdx = TYtMap::idx_Mapper; + } else if (TYtReduce::Match(&node)) { + lambdaIdx = TYtReduce::idx_Reducer; + } else if (TYtFill::Match(&node)) { + lambdaIdx = TYtFill::idx_Content; + isFill = true; + } else if (TYtDqProcessWrite::Match(&node)) { + lambdaIdx = TYtDqProcessWrite::idx_Input; + isYtDqProcessWrite = true; + } + if (-1 != lambdaIdx && !hasOtherSortedOuts) { + if (isFill) { + if (node.ChildPtr(lambdaIdx)->GetConstraint<TSortedConstraintNode>()) { + lambda = Build<TCoLambda>(ctx, node.ChildPtr(lambdaIdx)->Pos()) + .Args({}) + .Body<TCoUnordered>() + .Input<TExprApplier>() + .Apply(TCoLambda(node.ChildPtr(lambdaIdx))) + .Build() + .Build() + .Done().Ptr(); + } + } else if (isYtDqProcessWrite) { + TProcessedNodesSet processedNodes; + TNodeOnNodeOwnedMap remaps; + VisitExpr(node.ChildPtr(lambdaIdx), [&processedNodes, &remaps, &ctx](const TExprNode::TPtr& n) { + if (TYtOutput::Match(n.Get())) { + // Stop traversing dependent operations + processedNodes.insert(n->UniqueId()); + return false; + } + if (TYtDqWrite::Match(n.Get())) { + auto newInput = Build<TCoUnordered>(ctx, n->Pos()) + .Input(n->ChildPtr(TYtDqWrite::idx_Input)) + .Done(); + remaps[n.Get()] = ctx.ChangeChild(*n, TYtDqWrite::idx_Input, newInput.Ptr()); + } + return true; + }); + if (!remaps.empty()) { + TOptimizeExprSettings settings{State_->Types}; + settings.ProcessedNodes = &processedNodes; + auto status = RemapExpr(node.ChildPtr(lambdaIdx), lambda, remaps, ctx, settings); + if (status.Level == IGraphTransformer::TStatus::Error) { + return {}; + } + } + + } else { + TProcessedNodesSet processedNodes; + TNodeOnNodeOwnedMap remaps; + VisitExpr(node.ChildPtr(lambdaIdx), [&processedNodes, &remaps, &ctx](const TExprNode::TPtr& n) { + if (TYtOutput::Match(n.Get())) { + // Stop traversing dependent operations + processedNodes.insert(n->UniqueId()); + return false; + } + auto name = n->Content(); + if (n->IsCallable() && n->ChildrenSize() > 0 && name.SkipPrefix("Ordered")) { + const auto inputKind = n->Child(0)->GetTypeAnn()->GetKind(); + if (inputKind == ETypeAnnotationKind::Stream || inputKind == ETypeAnnotationKind::Flow) { + remaps[n.Get()] = ctx.RenameNode(*n, name); + } + } + return true; + }); + if (!remaps.empty()) { + TOptimizeExprSettings settings{State_->Types}; + settings.ProcessedNodes = &processedNodes; + auto status = RemapExpr(node.ChildPtr(lambdaIdx), lambda, remaps, ctx, settings); + if (status.Level == IGraphTransformer::TStatus::Error) { + return {}; + } + } + } + } + + TExprNode::TPtr res; + if (changedOutSort) { + res = ctx.ChangeChild(node, TYtOutputOpBase::idx_Output, + Build<TYtOutSection>(ctx, op.Pos()).Add(outTables).Done().Ptr()); + + if (TYtSort::Match(&node)) { + res = ctx.RenameNode(*res, TYtMerge::CallableName()); + } + + if (lambdaIdx != -1 && AnyOf(filterColumns, [](const TExprNode::TPtr& p) { return !!p; })) { + + TExprNode::TPtr extractLambda; + if (op.Output().Size() == 1) { + extractLambda = Build<TCoLambda>(ctx, lambda->Pos()) + .Args({"stream"}) + .Body<TCoExtractMembers>() + .Input("stream") + .Members(filterColumns[0]) + .Build() + .Done().Ptr(); + } else { + auto varType = ExpandType(lambda->Pos(), *GetSeqItemType(node.Child(lambdaIdx)->GetTypeAnn()), ctx); + TVector<TExprBase> visitArgs; + for (size_t i = 0; i < op.Output().Size(); ++i) { + visitArgs.push_back(Build<TCoAtom>(ctx, lambda->Pos()).Value(ToString(i)).Done()); + if (filterColumns[i]) { + visitArgs.push_back(Build<TCoLambda>(ctx, lambda->Pos()) + .Args({"row"}) + .Body<TCoVariant>() + .Item<TCoUnwrap>() + .Optional<TCoExtractMembers>() + .Input<TCoJust>() + .Input("row") + .Build() + .Members(filterColumns[i]) + .Build() + .Build() + .Index() + .Value(ToString(i)) + .Build() + .VarType(varType) + .Build() + .Done()); + } else { + visitArgs.push_back(Build<TCoLambda>(ctx, lambda->Pos()) + .Args({"row"}) + .Body<TCoVariant>() + .Item("row") + .Index() + .Value(ToString(i)) + .Build() + .VarType(varType) + .Build() + .Done()); + } + } + + extractLambda = Build<TCoLambda>(ctx, lambda->Pos()) + .Args({"stream"}) + .Body<TCoFlatMapBase>() + .CallableName(hasOtherSortedOuts ? TCoOrderedFlatMap::CallableName() : TCoFlatMap::CallableName()) + .Input("stream") + .Lambda() + .Args({"var"}) + .Body<TCoJust>() + .Input<TCoVisit>() + .Input("var") + .FreeArgs() + .Add(visitArgs) + .Build() + .Build() + .Build() + .Build() + .Build() + .Done().Ptr(); + } + + + if (!lambda) { + lambda = node.ChildPtr(lambdaIdx); + } + if (isYtDqProcessWrite) { + TProcessedNodesSet processedNodes; + TNodeOnNodeOwnedMap remaps; + VisitExpr(lambda, [&processedNodes, &remaps, extractLambda, &ctx](const TExprNode::TPtr& n) { + if (TYtOutput::Match(n.Get())) { + // Stop traversing dependent operations + processedNodes.insert(n->UniqueId()); + return false; + } + if (auto dqWrite = TMaybeNode<TYtDqWrite>(n)) { + auto newWrite = Build<TYtDqWrite>(ctx, n->Pos()) + .InitFrom(dqWrite.Cast()) + .Input<TExprApplier>() + .Apply(TCoLambda(extractLambda)) + .With(0, dqWrite.Cast().Input()) + .Build() + .Done(); + remaps[n.Get()] = newWrite.Ptr(); + } + return true; + }); + if (!remaps.empty()) { + TOptimizeExprSettings settings{State_->Types}; + settings.ProcessedNodes = &processedNodes; + auto status = RemapExpr(lambda, lambda, remaps, ctx, settings); + if (status.Level == IGraphTransformer::TStatus::Error) { + return {}; + } + } + + } else { + lambda = ctx.FuseLambdas(*extractLambda, *lambda); + } + } + } + + if (lambda) { + res = ctx.ChangeChild(res ? *res : node, lambdaIdx, std::move(lambda)); + } + + if (op.Maybe<TYtTransientOpBase>()) { + auto trOp = TYtTransientOpBase(&node); + if (!hasOtherSortedOuts && NYql::HasSetting(trOp.Settings().Ref(), EYtSettingType::Ordered)) { + res = ctx.ChangeChild(res ? *res : node, TYtTransientOpBase::idx_Settings, + NYql::RemoveSetting(trOp.Settings().Ref(), EYtSettingType::Ordered, ctx)); + } + + if (TYtMap::Match(&node)) { + Fill(filterColumns.begin(), filterColumns.end(), TExprNode::TPtr()); + filterColumns.resize(trOp.Input().Size()); + } + + if (!TYtReduce::Match(&node)) { + // Push Unordered and columns to operation inputs + bool changedInput = false; + TVector<TYtSection> updatedSections; + for (size_t i = 0; i < trOp.Input().Size(); ++i) { + auto section = trOp.Input().Item(i); + if (!hasOtherSortedOuts) { + section = MakeUnorderedSection(section, ctx); + if (section.Raw() != trOp.Input().Item(i).Raw()) { + changedInput = true; + } + } + if (filterColumns[i]) { + section = UpdateInputFields(section, TExprBase(filterColumns[i]), ctx); + changedInput = true; + } + updatedSections.push_back(section); + } + + if (changedInput) { + res = ctx.ChangeChild(res ? *res : node, TYtTransientOpBase::idx_Input, + Build<TYtSectionList>(ctx, trOp.Pos()).Add(updatedSections).Done().Ptr()); + } + } + } + + return res; + } + + TStatus SplitLargeInputs(TExprNode::TPtr input, TExprNode::TPtr& output, TExprContext& ctx, bool splitMap) { + auto maxTables = State_->Configuration->MaxInputTables.Get(); + auto maxSortedTables = State_->Configuration->MaxInputTablesForSortedMerge.Get(); + + TOptimizeExprSettings settings(State_->Types); + settings.ProcessedNodes = &ProcessedSplitLargeInputs; + + return OptimizeExpr(input, output, [maxTables, maxSortedTables, splitMap, this](const TExprNode::TPtr& node, TExprContext& ctx) { + if (TYtTransientOpBase::Match(node.Get()) && !IsBeingExecuted(*node)) { + auto op = TYtTransientOpBase(node); + auto outRowSpec = MakeIntrusive<TYqlRowSpecInfo>(op.Output().Item(0).RowSpec()); + const bool sortedMerge = TYtMerge::Match(node.Get()) && outRowSpec->IsSorted(); + const bool keepSort = sortedMerge || TYtReduce::Match(node.Get()) || TYtEquiJoin::Match(node.Get()); + auto limit = maxTables; + if (maxSortedTables.Defined() && sortedMerge) { + limit = maxSortedTables; + } + + if (limit) { + if (splitMap && TYtMap::Match(node.Get()) + && op.Input().Size() == 1 && op.Output().Size() == 1 + && !NYql::HasAnySetting(op.Settings().Ref(), EYtSettingType::SortLimitBy | EYtSettingType::Sharded | EYtSettingType::JobCount | EYtSettingType::Ordered | EYtSettingType::KeepSorted) + && op.Input().Item(0).Paths().Size() > *limit + && !NYql::HasAnySetting(op.Input().Item(0).Settings().Ref(), EYtSettingType::Take | EYtSettingType::Skip)) + { + auto section = op.Input().Item(0); + TVector<TYtPath> newPaths; + TVector<TYtPath> paths; + TVector<TYtPath> prevPaths(section.Paths().begin(), section.Paths().end()); + while (!prevPaths.empty()) { + size_t count = Min<size_t>(*limit, prevPaths.size()); + YQL_ENSURE(count > 0); + + newPaths.push_back( + Build<TYtPath>(ctx, op.Pos()) + .Table<TYtOutput>() + .Operation<TYtMap>() + .InitFrom(op.Cast<TYtMap>()) + .Input() + .Add() + .Paths() + .Add(TVector<TYtPath>(prevPaths.begin(), prevPaths.begin() + count)) + .Build() + .Settings(section.Settings()) + .Build() + .Build() + .Build() + .OutIndex().Value("0").Build() + .Build() + .Columns<TCoVoid>().Build() + .Ranges<TCoVoid>().Build() + .Stat<TCoVoid>().Build() + .Done() + ); + prevPaths.erase(prevPaths.begin(), prevPaths.begin() + count); + } + + YQL_CLOG(INFO, ProviderYt) << "PhysicalFinalizing-SplitLargeMapInputs"; + return Build<TYtMerge>(ctx, op.Pos()) + .World<TCoWorld>().Build() + .DataSink(op.DataSink()) + .Input() + .Add() + .Paths() + .Add(newPaths) + .Build() + .Settings() + .Build() + .Build() + .Build() + .Output() + .Add(op.Output().Item(0)) + .Build() + .Settings(NYql::KeepOnlySettings(op.Settings().Ref(), EYtSettingType::Limit, ctx)) + .Done().Ptr(); + } + + TVector<TYtSection> updatedSections; + bool hasUpdates = false; + for (auto section: op.Input()) { + const EYtSettingType kfType = NYql::HasSetting(section.Settings().Ref(), EYtSettingType::KeyFilter2) ? + EYtSettingType::KeyFilter2 : EYtSettingType::KeyFilter; + const auto keyFiltersValues = NYql::GetAllSettingValues(section.Settings().Ref(), kfType); + const bool hasKeyFilters = AnyOf(keyFiltersValues, + [](const TExprNode::TPtr& keyFilter) { return keyFilter->ChildrenSize() > 0; }); + const bool hasTableKeyFilters = AnyOf(keyFiltersValues, + [kfType](const TExprNode::TPtr& keyFilter) { + return keyFilter->ChildrenSize() >= GetMinChildrenForIndexedKeyFilter(kfType); + }); + + if (hasTableKeyFilters) { + // Cannot optimize it + updatedSections.push_back(section); + } else if (section.Paths().Size() > *limit) { + auto scheme = section.Ref().GetTypeAnn()->Cast<TListExprType>()->GetItemType(); + TVector<TYtPath> paths; + TVector<TYtPath> prevPaths(section.Paths().begin(), section.Paths().end()); + const size_t min = NYql::HasSetting(section.Settings().Ref(), EYtSettingType::SysColumns) ? 0 : 1; + while (prevPaths.size() > min) { + size_t count = Min<size_t>(*limit, prevPaths.size()); + YQL_ENSURE(count > 0); + + auto path = CopyOrTrivialMap(section.Pos(), + op.World(), op.DataSink(), + *scheme, + Build<TYtSection>(ctx, section.Pos()) + .Paths() + .Add(TVector<TYtPath>(prevPaths.begin(), prevPaths.begin() + count)) + .Build() + .Settings(NYql::KeepOnlySettings(section.Settings().Ref(), EYtSettingType::KeyFilter | EYtSettingType::KeyFilter2 | EYtSettingType::SysColumns, ctx)) + .Done(), + outRowSpec, + ctx, State_, + TCopyOrTrivialMapOpts().SetTryKeepSortness(keepSort).SetRangesResetSort(false)); + + paths.push_back(path); + prevPaths.erase(prevPaths.begin(), prevPaths.begin() + count); + } + auto settings = section.Settings().Ptr(); + if (!prevPaths.empty()) { + if (hasKeyFilters) { + // Modify keyFilters to point to remaining tables only + const size_t from = paths.size(); + const size_t to = from + prevPaths.size(); + auto keyFiltersValues = NYql::GetAllSettingValues(section.Settings().Ref(), EYtSettingType::KeyFilter); + settings = NYql::RemoveSetting(*settings, EYtSettingType::KeyFilter, ctx); + for (auto val: keyFiltersValues) { + for (size_t i = from; i < to; ++i) { + auto children = val->ChildrenList(); + children.push_back(ctx.NewAtom(section.Pos(), ToString(i), TNodeFlags::Default)); + settings = NYql::AddSetting(*settings, EYtSettingType::KeyFilter, + ctx.NewList(section.Pos(), std::move(children)), ctx); + } + } + keyFiltersValues = NYql::GetAllSettingValues(section.Settings().Ref(), EYtSettingType::KeyFilter2); + settings = NYql::RemoveSetting(*settings, EYtSettingType::KeyFilter2, ctx); + for (auto val: keyFiltersValues) { + TExprNode::TListType indicies; + for (size_t i = from; i < to; ++i) { + indicies.push_back(ctx.NewAtom(section.Pos(), ToString(i), TNodeFlags::Default)); + } + auto children = val->ChildrenList(); + children.push_back(ctx.NewList(section.Pos(), std::move(indicies))); + settings = NYql::AddSetting(*settings, EYtSettingType::KeyFilter2, + ctx.NewList(section.Pos(), std::move(children)), ctx); + } + } + paths.insert(paths.end(), prevPaths.begin(), prevPaths.end()); + } else { + // All keyFilters are pushed to children operations. Can remove them from the current + if (hasKeyFilters) { + settings = NYql::RemoveSettings(*settings, EYtSettingType::KeyFilter | EYtSettingType::KeyFilter2, ctx); + } + } + updatedSections.push_back(Build<TYtSection>(ctx, section.Pos()) + .InitFrom(section) + .Paths() + .Add(paths) + .Build() + .Settings(settings) + .Done()); + hasUpdates = true; + } else { + updatedSections.push_back(section); + } + } + if (hasUpdates) { + YQL_CLOG(INFO, ProviderYt) << "PhysicalFinalizing-SplitLargeInputs"; + return ctx.ChangeChild(*node, TYtTransientOpBase::idx_Input, + Build<TYtSectionList>(ctx, op.Input().Pos()).Add(updatedSections).Done().Ptr()); + } + } + } + return node; + }, ctx, settings); + } + + static TExprNode::TPtr RebuildLambdaWithLessOuts(TExprNode::TPtr lambda, bool withArg, bool ordered, const TDynBitMap& usedOuts, + const TYtOutSection& outs, TVector<TYtOutTable>& resTables, TExprContext& ctx) + { + if (usedOuts.Count() == 1) { + auto ndx = usedOuts.FirstNonZeroBit(); + resTables.push_back(outs.Item(ndx)); + if (withArg) { + auto oldLambda = TCoLambda(lambda); + lambda = Build<TCoLambda>(ctx, lambda->Pos()) + .Args({"stream"}) + .Body<TCoFlatMapBase>() + .CallableName(ordered ? TCoOrderedFlatMap::CallableName() : TCoFlatMap::CallableName()) + .template Input<TExprApplier>() + .Apply(oldLambda) + .With(oldLambda.Args().Arg(0), "stream") + .Build() + .Lambda() + .Args({"item"}) + .template Body<TCoGuess>() + .Variant("item") + .Index() + .Value(ToString(ndx)) + .Build() + .Build() + .Build() + .Build() + .Done().Ptr(); + } + else { + lambda = Build<TCoLambda>(ctx, lambda->Pos()) + .Args({}) + .Body<TCoFlatMapBase>() + .CallableName(ordered ? TCoOrderedFlatMap::CallableName() : TCoFlatMap::CallableName()) + .template Input<TExprApplier>() + .Apply(TCoLambda(lambda)) + .Build() + .Lambda() + .Args({"item"}) + .template Body<TCoGuess>() + .Variant("item") + .Index() + .Value(ToString(ndx)) + .Build() + .Build() + .Build() + .Build() + .Done().Ptr(); + } + } + else { + TVector<TExprBase> tupleTypes; + for (size_t i = 0; i < outs.Size(); ++i) { + if (usedOuts.Test(i)) { + auto out = outs.Item(i); + tupleTypes.push_back(TExprBase(ExpandType(out.Pos(), *out.Ref().GetTypeAnn()->Cast<TListExprType>()->GetItemType(), ctx))); + } + } + TExprBase varType = Build<TCoVariantType>(ctx, lambda->Pos()) + .UnderlyingType<TCoTupleType>() + .Add(tupleTypes) + .Build() + .Done(); + + TVector<TExprBase> visitArgs; + size_t nextVarIndex = 0; + for (size_t i = 0; i < outs.Size(); ++i) { + if (usedOuts.Test(i)) { + visitArgs.push_back(Build<TCoAtom>(ctx, lambda->Pos()).Value(ToString(i)).Done()); + visitArgs.push_back(Build<TCoLambda>(ctx, lambda->Pos()) + .Args({"item"}) + .template Body<TCoJust>() + .template Input<TCoVariant>() + .Item("item") + .Index() + .Value(ToString(nextVarIndex++)) + .Build() + .VarType(varType) + .Build() + .Build() + .Done() + ); + resTables.push_back(outs.Item(i)); + } + } + visitArgs.push_back(Build<TCoNothing>(ctx, lambda->Pos()) + .template OptionalType<TCoOptionalType>() + .ItemType(varType) + .Build() + .Done()); + + if (withArg) { + auto oldLambda = TCoLambda(lambda); + lambda = Build<TCoLambda>(ctx, lambda->Pos()) + .Args({"stream"}) + .Body<TCoFlatMapBase>() + .CallableName(ordered ? TCoOrderedFlatMap::CallableName() : TCoFlatMap::CallableName()) + .template Input<TExprApplier>() + .Apply(oldLambda) + .With(oldLambda.Args().Arg(0), "stream") + .Build() + .Lambda() + .Args({"item"}) + .template Body<TCoVisit>() + .Input("item") + .FreeArgs() + .Add(visitArgs) + .Build() + .Build() + .Build() + .Build() + .Done().Ptr(); + } + else { + lambda = Build<TCoLambda>(ctx, lambda->Pos()) + .Args({}) + .Body<TCoFlatMapBase>() + .CallableName(ordered ? TCoOrderedFlatMap::CallableName() : TCoFlatMap::CallableName()) + .template Input<TExprApplier>() + .Apply(TCoLambda(lambda)) + .Build() + .Lambda() + .Args({"item"}) + .template Body<TCoVisit>() + .Input("item") + .FreeArgs() + .Add(visitArgs) + .Build() + .Build() + .Build() + .Build() + .Done().Ptr(); + } + } + return lambda; + } + + TStatus BypassMergeOrCopyBeforeLength(TExprNode::TPtr input, TExprNode::TPtr& output, const TOpDeps& opDeps, const TNodeSet& lefts, TExprContext& ctx) { + TNodeOnNodeOwnedMap replaces; + for (auto& x: opDeps) { + if ((TYtMerge::Match(x.first) || TYtCopy::Match(x.first)) && x.second.size() > 0 && AllOf(x.second, [](const auto& item) { return TYtLength::Match(std::get<0>(item)); } )) { + auto op = TYtTransientOpBase(x.first); + if (op.Ref().HasResult()) { + continue; + } + + auto section = op.Input().Item(0); + if (op.Maybe<TYtMerge>()) { + if (NYql::HasSetting(op.Settings().Ref(), EYtSettingType::Limit)) { + continue; + } + + if (NYql::HasAnySetting(section.Settings().Ref(), EYtSettingType::Take | EYtSettingType::Skip | EYtSettingType::Sample)) { + continue; + } + if (NYql::HasNonEmptyKeyFilter(section)) { + continue; + } + + if (AnyOf(section.Paths(), [](const TYtPath& path) { return !path.Ranges().Maybe<TCoVoid>() || TYtTableBaseInfo::GetMeta(path.Table())->IsDynamic; })) { + continue; + } + // Dependency on more than 1 operation + if (1 < Accumulate(section.Paths(), 0ull, [](ui64 val, const TYtPath& path) { return val + path.Table().Maybe<TYtOutput>().IsValid(); })) { + continue; + } + } + + TSyncMap syncList; + for (auto path: section.Paths()) { + if (auto out = path.Table().Maybe<TYtOutput>()) { + syncList.emplace(GetOutputOp(out.Cast()).Ptr(), syncList.size()); + } + } + auto newWorld = ApplySyncListToWorld(op.World().Ptr(), syncList, ctx); + for (auto node: lefts) { + TCoLeft left(node); + if (left.Input().Raw() == op.Raw()) { + replaces[node] = newWorld; + } + } + + for (auto& item : x.second) { + auto len = TYtLength(std::get<0>(item)); + auto out = len.Input().Cast<TYtOutput>(); + + TExprNode::TPtr newLen; + if (section.Paths().Size() == 1) { + if (auto maybeOp = section.Paths().Item(0).Table().Maybe<TYtOutput>()) { + if (IsUnorderedOutput(out)) { + newLen = Build<TYtLength>(ctx, len.Pos()) + .InitFrom(len) + .Input<TYtOutput>() + .InitFrom(maybeOp.Cast()) + .Mode(out.Mode()) + .Build() + .Done().Ptr(); + } else { + newLen = Build<TYtLength>(ctx, len.Pos()) + .InitFrom(len) + .Input(maybeOp.Cast()) + .Done().Ptr(); + } + } + } + if (!newLen) { + newLen = Build<TYtLength>(ctx, len.Pos()) + .InitFrom(len) + .Input<TYtReadTable>() + .World(ctx.NewWorld(len.Pos())) + .DataSource(GetDataSource(len.Input(), ctx)) + .Input() + .Add(IsUnorderedOutput(out) ? MakeUnorderedSection(section, ctx) : section) + .Build() + .Build() + .Done().Ptr(); + } + + replaces[len.Raw()] = newLen; + } + } + } + + if (!replaces.empty()) { + YQL_CLOG(INFO, ProviderYt) << "PhysicalFinalizing-BypassMergeOrCopyBeforeLength"; + return RemapExpr(input, output, replaces, ctx, TOptimizeExprSettings(State_->Types)); + } + + return TStatus::Ok; + } + + TStatus OptimizeUnusedOuts(TExprNode::TPtr input, TExprNode::TPtr& output, const TOpDeps& opDeps, const TNodeSet& lefts, TExprContext& ctx) { + TNodeOnNodeOwnedMap replaces; + TNodeOnNodeOwnedMap newOps; + for (auto& x: opDeps) { + auto writer = x.first; + if (const size_t outCount = GetRealOperation(TExprBase(writer)).Output().Size(); outCount > 1 && writer->GetState() != TExprNode::EState::ExecutionComplete + && writer->GetState() != TExprNode::EState::ExecutionInProgress + && (!writer->HasResult() || writer->GetResult().Type() != TExprNode::World) + && ProcessedUnusedOuts.find(writer->UniqueId()) == ProcessedUnusedOuts.end()) + { + TDynBitMap usedOuts; + for (auto& item: x.second) { + usedOuts.Set(FromString<size_t>(std::get<2>(item)->Child(TYtOutput::idx_OutIndex)->Content())); + } + if (!usedOuts.Empty() && usedOuts.Count() < outCount) { + TExprNode::TPtr newOp; + if (const auto mayTry = TExprBase(writer).Maybe<TYtTryFirst>()) { + const auto opSecond = mayTry.Cast().Second().Raw(); + TExprNode::TPtr newOpSecond = SuppressUnusedOuts(*opSecond, usedOuts, ctx); + if (newOpSecond) { + newOp = Build<TYtTryFirst>(ctx, writer->Pos()) + .First(mayTry.Cast().First().Ptr()) + .Second(std::move(newOpSecond)) + .Done().Ptr(); + } + } else { + newOp = SuppressUnusedOuts(*writer, usedOuts, ctx); + } + newOps[writer] = newOp; + TVector<size_t> remappedIndicies(outCount, Max<size_t>()); + size_t newIndex = 0; + for (size_t i = 0; i < outCount; ++i) { + if (usedOuts.Test(i)) { + remappedIndicies[i] = newIndex++; + } + } + for (auto& item: x.second) { + auto oldOutput = TYtOutput(std::get<2>(item)); + auto oldNdx = FromString<size_t>(oldOutput.OutIndex().Value()); + YQL_ENSURE(oldNdx < remappedIndicies.size()); + auto newNdx = remappedIndicies[oldNdx]; + YQL_ENSURE(newNdx != Max<size_t>()); + replaces[oldOutput.Raw()] = Build<TYtOutput>(ctx, oldOutput.Pos()) + .Operation(newOp) + .OutIndex() + .Value(ToString(newNdx)) + .Build() + .Mode(oldOutput.Mode()) + .Done().Ptr(); + } + } + } + ProcessedUnusedOuts.insert(writer->UniqueId()); + } + if (!lefts.empty() && !newOps.empty()) { + for (auto node: lefts) { + TCoLeft left(node); + auto newIt = newOps.find(left.Input().Raw()); + if (newIt != newOps.end()) { + replaces[node] = ctx.ChangeChild(*node, TCoLeft::idx_Input, TExprNode::TPtr(newIt->second)); + } + } + } + + if (!replaces.empty()) { + return RemapExpr(input, output, replaces, ctx, TOptimizeExprSettings(State_->Types)); + } + return TStatus::Ok; + } + + TStatus AlignPublishTypes(TExprNode::TPtr input, TExprNode::TPtr& output, const TOpDeps& opDeps, TExprContext& ctx) { + TNodeOnNodeOwnedMap remap; + TMap<size_t, size_t> outUsage; + TNodeMap<TNodeOnNodeOwnedMap> updatePublish; + const bool useNativeDescSort = State_->Configuration->UseNativeDescSort.Get().GetOrElse(DEFAULT_USE_NATIVE_DESC_SORT); + for (auto& x : opDeps) { + outUsage.clear(); + bool hasPublish = false; + for (auto& item : x.second) { + auto reader = std::get<0>(item); + if (TYtPublish::Match(reader) || TYtCopy::Match(reader) || TYtMerge::Match(reader) || TYtSort::Match(reader)) { + const auto opIndex = FromString<size_t>(std::get<2>(item)->Child(TYtOutput::idx_OutIndex)->Content()); + ++outUsage[opIndex]; + hasPublish = hasPublish || TYtPublish::Match(reader); + } + } + if (!hasPublish) { + continue; + } + + const TYtOutputOpBase operation = GetRealOperation(TExprBase(x.first)); + const bool canUpdateOp = !IsBeingExecuted(*x.first) && !operation.Maybe<TYtCopy>(); + const bool canChangeNativeTypeForOp = !operation.Maybe<TYtMerge>() && !operation.Maybe<TYtSort>(); + + auto origOutput = operation.Output().Ptr(); + auto newOutput = origOutput; + for (auto& item : x.second) { + auto reader = std::get<0>(item); + if (auto maybePublish = TMaybeNode<TYtPublish>(reader)) { + auto publish = maybePublish.Cast(); + const auto out = TYtOutput(std::get<2>(item)); + const size_t opIndex = FromString<size_t>(out.OutIndex().Value()); + const bool unordered = IsUnorderedOutput(out); + TYtTableInfo dstInfo = publish.Publish(); + const auto& desc = State_->TablesData->GetTable(dstInfo.Cluster, dstInfo.Name, dstInfo.CommitEpoch); + if (!desc.RowSpec) { + continue; + } + auto table = operation.Output().Item(opIndex); + if (auto outRowSpec = TYtTableBaseInfo::GetRowSpec(table)) { + auto mode = NYql::GetSetting(publish.Settings().Ref(), EYtSettingType::Mode); + const bool append = mode && FromString<EYtWriteMode>(mode->Child(1)->Content()) == EYtWriteMode::Append; + + const bool diffNativeType = desc.RowSpec->GetNativeYtTypeFlags() != outRowSpec->GetNativeYtTypeFlags() + || desc.RowSpec->GetNativeYtType() != outRowSpec->GetNativeYtType(); + const bool diffColumnOrder = State_->Types->OrderedColumns && !append && desc.RowSpec->GetTypeNode() != outRowSpec->GetTypeNode(); + + if (diffNativeType || diffColumnOrder) { + outRowSpec->CopyType(*desc.RowSpec); + TExprNode::TPtr newTable = ctx.ChangeChild(table.Ref(), TYtOutTable::idx_RowSpec, + outRowSpec->ToExprNode(ctx, table.RowSpec().Pos()).Ptr()); + + if (canUpdateOp && outUsage[opIndex] <= 1 && (!diffNativeType || canChangeNativeTypeForOp)) { + YQL_CLOG(INFO, ProviderYt) << "AlignPublishTypes: change " << opIndex << " output of " << operation.Ref().Content(); + newOutput = ctx.ChangeChild(*newOutput, opIndex, std::move(newTable)); + } else if (diffNativeType) { + YQL_CLOG(INFO, ProviderYt) << "AlignPublishTypes: add remap op for " << opIndex << " output of " << operation.Ref().Content(); + auto settingsBuilder = Build<TCoNameValueTupleList>(ctx, out.Pos()); + auto mapper = Build<TCoLambda>(ctx, out.Pos()) + .Args({"stream"}) + .Body("stream") + .Done().Ptr(); + + if (!unordered && outRowSpec->IsSorted()) { + settingsBuilder + .Add() + .Name() + .Value(ToString(EYtSettingType::Ordered), TNodeFlags::Default) + .Build() + .Build(); + TKeySelectorBuilder builder(out.Pos(), ctx, useNativeDescSort, outRowSpec->GetType()); + builder.ProcessRowSpec(*outRowSpec); + if (builder.NeedMap()) { + mapper = builder.MakeRemapLambda(true); + } + } + if (State_->Configuration->UseFlow.Get().GetOrElse(DEFAULT_USE_FLOW)) { + settingsBuilder + .Add() + .Name() + .Value(ToString(EYtSettingType::Flow), TNodeFlags::Default) + .Build() + .Build(); + } + + updatePublish[reader][out.Raw()] = Build<TYtOutput>(ctx, out.Pos()) + .Operation<TYtMap>() + .World<TCoWorld>().Build() + .DataSink(operation.DataSink()) + .Input() + .Add() + .Paths() + .Add() + .Table<TYtOutput>() + .InitFrom(out) + .Build() + .Columns<TCoVoid>().Build() + .Ranges<TCoVoid>().Build() + .Stat<TCoVoid>().Build() + .Build() + .Build() + .Settings().Build() + .Build() + .Build() + .Output() + .Add(std::move(newTable)) + .Build() + .Settings(settingsBuilder.Done()) + .Mapper(mapper) + .Build() + .OutIndex().Build(0U) + .Mode(out.Mode()) + .Done().Ptr(); + } + } + } + } + } + if (newOutput != origOutput) { + remap[operation.Raw()] = ctx.ChangeChild(operation.Ref(), TYtOutputOpBase::idx_Output, std::move(newOutput)); + } + } + + if (!updatePublish.empty()) { + for (auto& item: updatePublish) { + auto publish = TYtPublish(item.first); + const TNodeOnNodeOwnedMap& updateOuts = item.second; + auto origInput = publish.Input().Ptr(); + auto newInput = origInput; + for (size_t i = 0; i < publish.Input().Size(); ++i) { + if (auto it = updateOuts.find(publish.Input().Item(i).Raw()); it != updateOuts.end()) { + newInput = ctx.ChangeChild(*newInput, i, TExprNode::TPtr(it->second)); + } + } + YQL_ENSURE(newInput != origInput); + remap[publish.Raw()] = ctx.ChangeChild(publish.Ref(), TYtPublish::idx_Input, std::move(newInput)); + } + } + + if (!remap.empty()) { + YQL_CLOG(INFO, ProviderYt) << "PhysicalFinalizing-AlignPublishTypes"; + return RemapExpr(input, output, remap, ctx, TOptimizeExprSettings(State_->Types)); + } + + return TStatus::Ok; + } + + static TExprNode::TPtr SuppressUnusedOuts(const TExprNode& node, const TDynBitMap& usedOuts, TExprContext& ctx) { + auto op = TYtOutputOpBase(&node); + size_t lambdaNdx = 0; + bool withArg = true; + bool ordered = false; + if (auto ytMap = op.Maybe<TYtMap>()) { + lambdaNdx = TYtMap::idx_Mapper; + ordered = NYql::HasSetting(ytMap.Cast().Settings().Ref(), EYtSettingType::Ordered); + } else if (op.Maybe<TYtReduce>()) { + lambdaNdx = TYtReduce::idx_Reducer; + } else if (op.Maybe<TYtMapReduce>()) { + lambdaNdx = TYtMapReduce::idx_Reducer; + } else if (op.Maybe<TYtFill>()) { + lambdaNdx = TYtFill::idx_Content; + withArg = false; + } else { + YQL_ENSURE(false, "Unsupported operation " << node.Content()); + } + TExprNode::TPtr lambda = node.ChildPtr(lambdaNdx); + TVector<TYtOutTable> resTables; + + lambda = RebuildLambdaWithLessOuts(lambda, withArg, ordered, usedOuts, op.Output(), resTables, ctx); + + auto res = ctx.ChangeChild(node, TYtOutputOpBase::idx_Output, + Build<TYtOutSection>(ctx, op.Pos()).Add(resTables).Done().Ptr()); + + res = ctx.ChangeChild(*res, lambdaNdx, std::move(lambda)); + + YQL_CLOG(INFO, ProviderYt) << "PhysicalFinalizing-SuppressOuts"; + return res; + } + + static TExprNode::TPtr Limits(const TExprNode::TPtr& node, const TParentsMap& limits, TExprContext& ctx) { + auto limitIt = limits.find(node.Get()); + if (limitIt == limits.end()) { + return node; + } + auto op = TYtTransientOpBase(node); + TExprNode::TListType limitNodeChildren; + const TNodeSet& limitNodesSet = limitIt->second; + TVector<const TExprNode*> limitNodes(limitNodesSet.cbegin(), limitNodesSet.cend()); + std::stable_sort(limitNodes.begin(), limitNodes.end(), [](const auto& p1, const auto& p2) { return p1->UniqueId() < p2->UniqueId(); }); + for (const auto& readerSettings: limitNodes) { + TExprNode::TListType readerNodeChildren; + for (auto setting : readerSettings->Children()) { + if (setting->ChildrenSize() > 0) { + auto kind = FromString<EYtSettingType>(setting->Child(0)->Content()); + if (EYtSettingType::Take == kind || EYtSettingType::Skip == kind) { + readerNodeChildren.push_back(setting); + } + } + } + auto readerNode = ctx.NewList(op.Pos(), std::move(readerNodeChildren)); + limitNodeChildren.push_back(readerNode); + } + + return ctx.ChangeChild(*node, TYtTransientOpBase::idx_Settings, + NYql::AddSetting(op.Settings().Ref(), EYtSettingType::Limit, ctx.NewList(op.Pos(), std::move(limitNodeChildren)), ctx)); + } + + static TExprNode::TPtr LengthOverPhysicalList(const TExprNode::TPtr& node, const TOpDeps& opDeps, TExprContext& ctx) { + auto op = TYtWithUserJobsOpBase(node); + + // TODO: support multi-output + if (op.Output().Size() != 1) { + return node; + } + auto outTable = op.Output().Item(0); + if (outTable.Ref().GetTypeAnn()->Cast<TListExprType>()->GetItemType()->Cast<TStructExprType>()->GetSize() == 0) { + // already empty struct + return node; + } + + auto readersIt = opDeps.find(op.Raw()); + if (readersIt == opDeps.end() || readersIt->second.size() != 1) { + return node; + } + auto reader = std::get<0>(readersIt->second.front()); + if (!TYtLength::Match(reader)) { + return node; + } + + // only YtLength is used, rewrite + auto lambdaIdx = op.Maybe<TYtMap>() + ? TYtMap::idx_Mapper + : op.Maybe<TYtReduce>() + ? TYtReduce::idx_Reducer + : TYtMapReduce::idx_Reducer; + + // Rebuild lambda + auto lambda = TCoLambda(node->ChildPtr(lambdaIdx)); + lambda = Build<TCoLambda>(ctx, lambda.Pos()) + .Args({"stream"}) + .Body<TCoMap>() + .Input<TExprApplier>() + .Apply(lambda) + .With(0, "stream") + .Build() + .Lambda() + .Args({"item"}) + .Body<TCoAsStruct>() + .Build() + .Build() + .Build() + .Done(); + + // Rebuild output table + TYtOutTableInfo outTableInfo(outTable); + outTableInfo.RowSpec->ClearSortness(ctx); + outTableInfo.RowSpec->SetType(ctx.MakeType<TStructExprType>(TVector<const TItemExprType*>())); + + auto newOp = ctx.ShallowCopy(*node); + if (NYql::HasSetting(op.Settings().Ref(), EYtSettingType::Ordered)) { + newOp->ChildRef(TYtWithUserJobsOpBase::idx_Settings) = NYql::RemoveSetting(op.Settings().Ref(), EYtSettingType::Ordered, ctx); + } + + newOp->ChildRef(lambdaIdx) = lambda.Ptr(); + newOp->ChildRef(TYtWithUserJobsOpBase::idx_Output) = + Build<TYtOutSection>(ctx, op.Output().Pos()) + .Add(outTableInfo.ToExprNode(ctx, outTable.Pos()).Cast<TYtOutTable>()) + .Done().Ptr(); + + return newOp; + } + + static TExprNode::TPtr TopSortForProducers(const TExprNode::TPtr& node, const TOpDeps& opDeps, TExprContext& ctx) { + auto op = TYtWithUserJobsOpBase(node); + + if (op.Output().Size() != 1) { + return node; + } + + if (NYql::HasAnySetting(op.Settings().Ref(), EYtSettingType::SortLimitBy | EYtSettingType::Limit)) { + return node; + } + + auto readersIt = opDeps.find(op.Raw()); + if (readersIt == opDeps.end()) { + return node; + } + + TExprNode::TListType limits; + TVector<std::pair<TString, bool>> sortBy; + for (auto item: readersIt->second) { + if (!TYtSort::Match(std::get<0>(item))) { + return node; + } + auto sort = TYtSort(std::get<0>(item)); + if (!NYql::HasSetting(sort.Settings().Ref(), EYtSettingType::Limit)) { + return node; + } + + auto opSortBy = TYtOutTableInfo(sort.Output().Item(0)).RowSpec->GetForeignSort(); + if (sortBy.empty()) { + sortBy = std::move(opSortBy); + } + else if (opSortBy != sortBy) { + return node; + } + auto limitValues = NYql::GetSetting(sort.Settings().Ref(), EYtSettingType::Limit)->Child(1); + limits.insert(limits.end(), limitValues->Children().begin(), limitValues->Children().end()); + } + + if (limits.empty()) { + return node; + } + + auto newSettings = NYql::AddSetting(op.Settings().Ref(), EYtSettingType::Limit, + ctx.NewList(node->Pos(), std::move(limits)), ctx); + newSettings = NYql::AddSettingAsColumnPairList(*newSettings, EYtSettingType::SortLimitBy, sortBy, ctx); + newSettings = NYql::RemoveSetting(*newSettings, EYtSettingType::Ordered, ctx); + + auto newOp = ctx.ShallowCopy(*node); + newOp->ChildRef(TYtWithUserJobsOpBase::idx_Settings) = newSettings; + + auto outTable = op.Output().Item(0); + TYtOutTableInfo outTableInfo(outTable); + if (outTableInfo.RowSpec->IsSorted()) { + outTableInfo.RowSpec->ClearSortness(ctx); + newOp->ChildRef(TYtWithUserJobsOpBase::idx_Output) = + Build<TYtOutSection>(ctx, op.Output().Pos()) + .Add(outTableInfo.ToExprNode(ctx, outTable.Pos()).Cast<TYtOutTable>()) + .Done().Ptr(); + } + + return newOp; + } + + TStatus OptimizeMultiOuts(TExprNode::TPtr input, TExprNode::TPtr& output, const TOpDeps& opDeps, const TNodeSet& lefts, TExprContext& ctx) { + TNodeOnNodeOwnedMap rewrite; + TNodeOnNodeOwnedMap newOps; + for (auto& x: opDeps) { + auto writer = x.first; + const TYtOutputOpBase op = GetRealOperation(TExprBase(writer)); + if (const size_t outCount = op.Output().Size(); outCount > 1 && !BeingExecuted(*writer) + && (!op.Maybe<TYtMapReduce>() || GetMapDirectOutputsCount(op.Maybe<TYtMapReduce>().Cast()) == 0) // TODO: optimize this case + && ProcessedMultiOuts.find(writer->UniqueId()) == ProcessedMultiOuts.end()) + { + // Outputs cannot be merged if: 1) there are ranges over output, 2) output is used multiple times in the same section, 3) output is sorted + TSet<size_t> exclusiveOuts; + TVector<const TTypeAnnotationNode*> outTypes; + for (size_t i: xrange(op.Output().Size())) { + if (TYqlRowSpecInfo(op.Output().Item(i).RowSpec()).IsSorted()) { + exclusiveOuts.insert(i); + } + outTypes.push_back(op.Output().Item(i).Ref().GetTypeAnn()); + } + + TMap<size_t, TSet<const TExprNode*>> outUsage; // output num -> set of readers (sections or operations) + THashMap<size_t, TSet<std::pair<const TExprNode*, const TExprNode*>>> duplicateCheck; // output num -> set of readers (operation + section) + for (auto& reader: x.second) { + const auto out = TYtOutput(std::get<2>(reader)); + const auto outIndex = FromString<size_t>(out.OutIndex().Value()); + YQL_ENSURE(outIndex < outTypes.size()); + const auto path = std::get<3>(reader); + if (path && !TCoVoid::Match(path->Child(TYtPath::idx_Ranges))) { + exclusiveOuts.insert(outIndex); + } + auto section = std::get<1>(reader); // section + auto op = std::get<0>(reader); // operation + // Section may be used multiple times in different operations + // So, check only unique pair of operation + section + if (!duplicateCheck[outIndex].insert(std::make_pair(op, section)).second) { + exclusiveOuts.insert(outIndex); + } + outUsage[outIndex].insert(section ? section : op); + } + + // Group by {{set of sections}, type} + TMap<std::pair<TSet<const TExprNode*>, const TTypeAnnotationNode*>, TVector<size_t>> groupedOuts; + for (auto& item: outUsage) { + if (!exclusiveOuts.contains(item.first)) { + groupedOuts[std::make_pair(item.second, outTypes[item.first])].push_back(item.first); + } + } + if (AnyOf(groupedOuts, [](const auto& item) { return item.second.size() > 1; })) { + size_t nextNewOutIndex = 0; + TVector<TYtOutTable> joinedOutTables; + TVector<std::pair<size_t, bool>> outRemap; // old index -> new index, drop flag + outRemap.resize(outCount); + for (auto old_out_num: exclusiveOuts) { + outRemap[old_out_num] = std::make_pair(nextNewOutIndex++, false); + joinedOutTables.push_back(op.Output().Item(old_out_num)); + } + TVector<TVector<size_t>> sortedGroups; + sortedGroups.reserve(groupedOuts.size()); + for (auto& item: groupedOuts) { + sortedGroups.push_back(std::move(item.second)); + } + ::Sort(sortedGroups, [](const TVector<size_t>& v1, const TVector<size_t>& v2) { return v1.front() < v2.front(); }); + for (auto& outs: sortedGroups) { + TMaybe<size_t> newIndex; + bool drop = false; + for (auto old_out_num: outs) { + if (!newIndex) { + newIndex = nextNewOutIndex++; + joinedOutTables.push_back(op.Output().Item(old_out_num)); + } + outRemap[old_out_num] = std::make_pair(*newIndex, drop); + drop = true; + } + } + + size_t lambdaNdx = 0; + bool ordered = false; + if (auto ytMap = op.Maybe<TYtMap>()) { + lambdaNdx = TYtMap::idx_Mapper; + ordered = NYql::HasSetting(ytMap.Cast().Settings().Ref(), EYtSettingType::Ordered); + } else if (op.Maybe<TYtReduce>()) { + lambdaNdx = TYtReduce::idx_Reducer; + } else if (op.Maybe<TYtMapReduce>()) { + lambdaNdx = TYtMapReduce::idx_Reducer; + } else if (op.Maybe<TYtFill>()) { + lambdaNdx = TYtFill::idx_Content; + } else { + YQL_ENSURE(false, "Unsupported operation " << writer->Content()); + } + TExprNode::TPtr lambda = writer->ChildPtr(lambdaNdx); + lambda = RebuildLambdaWithMergedOuts(lambda, nextNewOutIndex == 1, ordered, joinedOutTables, outRemap, ctx); + + auto newOp = ctx.ChangeChild(op.Ref(), TYtOutputOpBase::idx_Output, + Build<TYtOutSection>(ctx, op.Pos()).Add(joinedOutTables).Done().Ptr()); + + newOp = ctx.ChangeChild(*newOp, lambdaNdx, std::move(lambda)); + + rewrite[writer] = newOp; + newOps[writer] = newOp; + + TNodeOnNodeOwnedMap newOuts; + TNodeSet processed; + for (auto& reader: x.second) { + auto oldOutput = TYtOutput(std::get<2>(reader)); + if (processed.insert(oldOutput.Raw()).second) { + auto oldNdx = FromString<size_t>(oldOutput.OutIndex().Value()); + YQL_ENSURE(oldNdx < outRemap.size()); + auto newOut = Build<TYtOutput>(ctx, oldOutput.Pos()) + .Operation(newOp) + .OutIndex() + .Value(outRemap[oldNdx].second ? TString("canary") : ToString(outRemap[oldNdx].first)) // Insert invalid "canary" index for dropped outputs + .Build() + .Mode(oldOutput.Mode()) + .Done().Ptr(); + rewrite[oldOutput.Raw()] = newOut; + if (!outRemap[oldNdx].second) { + newOuts[oldOutput.Raw()] = newOut; + } + } + } + + for (auto& reader: x.second) { + if (auto rawSection = std::get<1>(reader)) { + if (processed.insert(rawSection).second) { + auto section = TYtSection(rawSection); + bool updated = false; + TVector<TYtPath> updatedPaths; + for (auto path: section.Paths()) { + if (path.Table().Maybe<TYtOutput>().Operation().Raw() == writer) { + if (auto it = newOuts.find(path.Table().Cast<TYtOutput>().Raw()); it != newOuts.cend()) { + updatedPaths.push_back(Build<TYtPath>(ctx, path.Pos()) + .InitFrom(path) + .Table(it->second) + .Done()); + } + updated = true; + } else { + updatedPaths.push_back(path); + } + } + if (updated) { + YQL_ENSURE(!updatedPaths.empty()); + rewrite[rawSection] = Build<TYtSection>(ctx, section.Pos()) + .InitFrom(section) + .Paths() + .Add(updatedPaths) + .Build() + .Done().Ptr(); + } + } + } + else if (TYtPublish::Match(std::get<0>(reader))) { + auto publish = TYtPublish(std::get<0>(reader)); + if (processed.insert(publish.Raw()).second) { + bool updated = false; + TExprNode::TListType updatedOuts; + for (auto out: publish.Input()) { + if (out.Operation().Raw() == writer) { + if (auto it = newOuts.find(out.Raw()); it != newOuts.cend()) { + updatedOuts.push_back(it->second); + } + updated = true; + } else { + updatedOuts.push_back(out.Ptr()); + } + } + if (updated) { + YQL_ENSURE(!updatedOuts.empty()); + rewrite[publish.Raw()] = Build<TYtPublish>(ctx, publish.Pos()) + .InitFrom(publish) + .Input() + .Add(updatedOuts) + .Build() + .Done().Ptr(); + } + } + } + } + break; + } + } + ProcessedMultiOuts.insert(writer->UniqueId()); + } + if (!lefts.empty() && !newOps.empty()) { + for (auto node: lefts) { + TCoLeft left(node); + auto newIt = newOps.find(left.Input().Raw()); + if (newIt != newOps.end()) { + rewrite[node] = ctx.ChangeChild(*node, TCoLeft::idx_Input, TExprNode::TPtr(newIt->second)); + } + } + } + + if (!rewrite.empty()) { + YQL_CLOG(INFO, ProviderYt) << "PhysicalFinalizing-MergeMultiOuts"; + return RemapExpr(input, output, rewrite, ctx, TOptimizeExprSettings(State_->Types)); + } + return TStatus::Ok; + } + + static TExprNode::TPtr RebuildLambdaWithMergedOuts(const TExprNode::TPtr& lambda, bool singleOutput, bool ordered, + const TVector<TYtOutTable>& joinedOutTables, const TVector<std::pair<size_t, bool>>& outRemap, TExprContext& ctx) + { + if (singleOutput) { + if (lambda->Child(0)->ChildrenSize()) { + return Build<TCoLambda>(ctx, lambda->Pos()) + .Args({"stream"}) + .Body<TCoFlatMapBase>() + .CallableName(ordered ? TCoOrderedFlatMap::CallableName() : TCoFlatMap::CallableName()) + .Input<TExprApplier>() + .Apply(TCoLambda(lambda)) + .With(0, "stream") + .Build() + .Lambda() + .Args({"item"}) + .Body<TCoJust>() + .Input<TCoVariantItem>() + .Variant("item") + .Build() + .Build() + .Build() + .Build() + .Done().Ptr(); + } else { + return Build<TCoLambda>(ctx, lambda->Pos()) + .Args({}) + .Body<TCoFlatMapBase>() + .CallableName(ordered ? TCoOrderedFlatMap::CallableName() : TCoFlatMap::CallableName()) + .Input<TExprApplier>() + .Apply(TCoLambda(lambda)) + .Build() + .Lambda() + .Args({"item"}) + .Body<TCoJust>() + .Input<TCoVariantItem>() + .Variant("item") + .Build() + .Build() + .Build() + .Build() + .Done().Ptr(); + } + } else { // nextNewOutIndex > 1 + TVector<TExprBase> tupleTypes; + for (auto out: joinedOutTables) { + auto itemType = out.Ref().GetTypeAnn()->Cast<TListExprType>()->GetItemType(); + tupleTypes.push_back(TExprBase(ExpandType(out.Pos(), *itemType, ctx))); + } + TExprBase varType = Build<TCoVariantType>(ctx, lambda->Pos()) + .UnderlyingType<TCoTupleType>() + .Add(tupleTypes) + .Build() + .Done(); + + TVector<TExprBase> visitArgs; + for (size_t i: xrange(outRemap.size())) { + visitArgs.push_back(Build<TCoAtom>(ctx, lambda->Pos()).Value(ToString(i)).Done()); + visitArgs.push_back(Build<TCoLambda>(ctx, lambda->Pos()) + .Args({"row"}) + .Body<TCoVariant>() + .Item("row") + .Index() + .Value(ToString(outRemap[i].first)) + .Build() + .VarType(varType) + .Build() + .Done()); + } + + if (lambda->Child(0)->ChildrenSize()) { + return Build<TCoLambda>(ctx, lambda->Pos()) + .Args({"stream"}) + .Body<TCoFlatMapBase>() + .CallableName(ordered ? TCoOrderedFlatMap::CallableName() : TCoFlatMap::CallableName()) + .Input<TExprApplier>() + .Apply(TCoLambda(lambda)) + .With(0, "stream") + .Build() + .Lambda() + .Args({"item"}) + .Body<TCoJust>() + .Input<TCoVisit>() + .Input("item") + .FreeArgs() + .Add(visitArgs) + .Build() + .Build() + .Build() + .Build() + .Build() + .Done().Ptr(); + } else { + return Build<TCoLambda>(ctx, lambda->Pos()) + .Args({}) + .Body<TCoFlatMapBase>() + .CallableName(ordered ? TCoOrderedFlatMap::CallableName() : TCoFlatMap::CallableName()) + .Input<TExprApplier>() + .Apply(TCoLambda(lambda)) + .Build() + .Lambda() + .Args({"item"}) + .Body<TCoJust>() + .Input<TCoVisit>() + .Input("item") + .FreeArgs() + .Add(visitArgs) + .Build() + .Build() + .Build() + .Build() + .Build() + .Done().Ptr(); + } + } + } + + TStatus FuseMultiOutsWithOuterMaps(TExprNode::TPtr input, TExprNode::TPtr& output, const TOpDeps& opDeps, const TNodeSet& lefts, const TNodeSet& hasWorldDeps, TExprContext& ctx) { + const ui64 switchLimit = State_->Configuration->SwitchLimit.Get().GetOrElse(DEFAULT_SWITCH_MEMORY_LIMIT); + const auto maxOperationFiles = State_->Configuration->MaxOperationFiles.Get().GetOrElse(DEFAULT_MAX_OPERATION_FILES); + const auto maxJobMemoryLimit = State_->Configuration->MaxExtraJobMemoryToFuseOperations.Get(); + const auto maxOutTables = State_->Configuration->MaxOutputTables.Get().GetOrElse(DEFAULT_MAX_OUTPUT_TABLES); + const auto maxOuterCpuUsage = State_->Configuration->MaxCpuUsageToFuseMultiOuts.Get().GetOrElse(2.0); + const auto maxOuterReplicationFactor = State_->Configuration->MaxReplicationFactorToFuseMultiOuts.Get().GetOrElse(2.0); + + for (auto& x: opDeps) { + if (TYtWithUserJobsOpBase::Match(x.first) && !BeingExecuted(*x.first) && !hasWorldDeps.contains(x.first) + && !NYql::HasAnySetting(*x.first->Child(TYtWithUserJobsOpBase::idx_Settings), EYtSettingType::Limit | EYtSettingType::SortLimitBy | EYtSettingType::JobCount)) { + + const auto op = TYtWithUserJobsOpBase(x.first); + + if (op.Maybe<TYtMapReduce>() && GetMapDirectOutputsCount(op.Cast<TYtMapReduce>()) != 0) { // TODO: optimize this case + continue; + } + + size_t lambdaNdx = 0; + if (auto ytMap = op.Maybe<TYtMap>()) { + lambdaNdx = TYtMap::idx_Mapper; + } else if (op.Maybe<TYtReduce>()) { + lambdaNdx = TYtReduce::idx_Reducer; + } else if (op.Maybe<TYtMapReduce>()) { + lambdaNdx = TYtMapReduce::idx_Reducer; + } else { + YQL_ENSURE(false, "Unsupported operation " << op.Ref().Content()); + } + + auto lambda = TCoLambda(op.Ref().ChildPtr(lambdaNdx)); + if (!IsYieldTransparent(lambda.Ptr(), *State_->Types)) { + continue; + } + + const size_t opOutTables = op.Output().Size(); + std::map<size_t, std::pair<std::vector<const TExprNode*>, std::vector<const TExprNode*>>> maps; // output -> pair<vector<YtMap>, vector<other YtOutput's>> + for (size_t i = 0; i < x.second.size(); ++i) { + auto reader = std::get<0>(x.second[i]); + if (BeingExecuted(*reader)) { + maps.clear(); + break; + } + const auto out = std::get<2>(x.second[i]); + const auto opIndex = FromString<size_t>(out->Child(TYtOutput::idx_OutIndex)->Content()); + auto& item = maps[opIndex]; + const TExprNode* matched = nullptr; + const auto newPair = ProcessedFuseWithOuterMaps.insert(std::make_pair(x.first->UniqueId(), reader->UniqueId())).second; + if (newPair && TYtMap::Match(reader)) { + const auto outerMap = TYtMap(reader); + if ((outerMap.World().Ref().IsWorld() || outerMap.World().Raw() == op.World().Raw()) + && outerMap.Input().Size() == 1 + && outerMap.Output().Size() + item.first.size() <= maxOutTables // fast check for too many operations + && outerMap.DataSink().Cluster().Value() == op.DataSink().Cluster().Value() + && NYql::HasSetting(op.Settings().Ref(), EYtSettingType::Flow) == NYql::HasSetting(outerMap.Settings().Ref(), EYtSettingType::Flow) + && !NYql::HasSetting(op.Settings().Ref(), EYtSettingType::JobCount) + && !NYql::HasAnySetting(outerMap.Settings().Ref(), EYtSettingType::JobCount | EYtSettingType::BlockInputApplied) + && !HasYtRowNumber(outerMap.Mapper().Body().Ref()) + && IsYieldTransparent(outerMap.Mapper().Ptr(), *State_->Types) + && (!op.Maybe<TYtMapReduce>() || AllOf(outerMap.Output(), [](const auto& out) { return !TYtTableBaseInfo::GetRowSpec(out)->IsSorted(); }))) { + + const auto outerSection = outerMap.Input().Item(0); + if (outerSection.Paths().Size() == 1 && outerSection.Settings().Size() == 0) { + const auto outerPath = outerSection.Paths().Item(0); + if (outerPath.Ranges().Maybe<TCoVoid>()) { + matched = reader; + } + } + } + } + if (matched) { + item.first.push_back(matched); + } else { + item.second.push_back(out); + } + } + + // Check limits + if (AnyOf(maps, [](const auto& item) { return item.second.first.size() > 0; })) { + TMap<TStringBuf, ui64> memUsage; + size_t currenFiles = 1; // jobstate. Take into account only once + size_t currOutTables = opOutTables; + + TExprNode::TPtr updatedBody = lambda.Body().Ptr(); + if (maxJobMemoryLimit) { + auto status = UpdateTableContentMemoryUsage(lambda.Body().Ptr(), updatedBody, State_, ctx, false); + if (status.Level != TStatus::Ok) { + return status; + } + } + ScanResourceUsage(*updatedBody, *State_->Configuration, State_->Types, maxJobMemoryLimit ? &memUsage : nullptr, nullptr, ¤Files); + + TMap<TStringBuf, ui64> newMemUsage; + TMap<TStringBuf, double> cpuUsage; + for (auto& item: maps) { + if (!item.second.first.empty()) { + size_t otherTablesDelta = item.second.second.empty() ? 1 : 0; + for (auto it = item.second.first.begin(); it != item.second.first.end(); ) { + const auto outerMap = TYtMap(*it); + + const size_t outTablesDelta = outerMap.Output().Size() - otherTablesDelta; + + updatedBody = outerMap.Mapper().Body().Ptr(); + if (maxJobMemoryLimit) { + auto status = UpdateTableContentMemoryUsage(outerMap.Mapper().Body().Ptr(), updatedBody, State_, ctx, false); + if (status.Level != TStatus::Ok) { + return status; + } + } + TMap<TStringBuf, ui64>* pMemUsage = nullptr; + if (maxJobMemoryLimit) { + pMemUsage = &newMemUsage; + newMemUsage = memUsage; + } + size_t newCurrenFiles = currenFiles; + cpuUsage.clear(); + ScanResourceUsage(*updatedBody, *State_->Configuration, State_->Types, pMemUsage, &cpuUsage, &newCurrenFiles); + + auto usedMemory = Accumulate(newMemUsage.begin(), newMemUsage.end(), switchLimit, + [](ui64 sum, const std::pair<const TStringBuf, ui64>& val) { return sum + val.second; }); + + // Take into account codec input/output buffers (one for all inputs and one per output) + usedMemory += YQL_JOB_CODEC_MEM * (currOutTables + outTablesDelta + 1); + + const auto usedCpu = Accumulate(cpuUsage.begin(), cpuUsage.end(), 1.0, + [](double prod, const std::pair<const TStringBuf, double>& val) { return prod * val.second; }); + + const auto replicationFactor = NCommon::GetDataReplicationFactor(outerMap.Mapper().Ref(), ctx); + + bool skip = false; + if (replicationFactor > maxOuterReplicationFactor) { + YQL_CLOG(DEBUG, ProviderYt) << "FuseMultiOutsWithOuterMaps: skip by replication factor limites " << replicationFactor << " > " << maxOuterReplicationFactor; + skip = true; + } + else if (outTablesDelta && currOutTables + outTablesDelta > maxOutTables) { + YQL_CLOG(DEBUG, ProviderYt) << "FuseMultiOutsWithOuterMaps: skip by out table limits " << (currOutTables + outTablesDelta) << " > " << maxOutTables; + skip = true; + } + else if (newCurrenFiles > maxOperationFiles) { + YQL_CLOG(DEBUG, ProviderYt) << "FuseMultiOutsWithOuterMaps: skip by files limits " << newCurrenFiles << " > " << maxOperationFiles; + skip = true; + } + else if (maxJobMemoryLimit && usedMemory > *maxJobMemoryLimit) { + YQL_CLOG(DEBUG, ProviderYt) << "FuseMultiOutsWithOuterMaps: skip by memory limits " << usedMemory << " > " << *maxJobMemoryLimit; + skip = true; + } + else if (usedCpu > maxOuterCpuUsage) { + YQL_CLOG(DEBUG, ProviderYt) << "FuseMultiOutsWithOuterMaps: skip by cpu limits " << usedCpu << " > " << maxOuterCpuUsage; + skip = true; + } + if (skip) { + // Move to other usages + it = item.second.first.erase(it); + if (item.second.second.empty()) { + ++currOutTables; + } + item.second.second.push_back(outerMap.Input().Item(0).Paths().Item(0).Table().Raw()); + continue; + } + currenFiles = newCurrenFiles; + memUsage = std::move(newMemUsage); + currOutTables += outTablesDelta; + otherTablesDelta = 0; // Take into account only once + ++it; + } + } + } + } + + if (AnyOf(maps, [](const auto& item) { return item.second.first.size() > 0; })) { + TNodeOnNodeOwnedMap remaps; + TNodeSet removed; + TNodeMap<size_t> newOutIndicies; // old YtOutput -> new index + + TVector<TExprBase> switchArgs; + TVector<TYtOutTable> outs; + for (size_t i = 0; i < op.Output().Size(); ++i) { + auto& item = maps[i]; + + if (item.first.empty() || !item.second.empty()) { + for (const auto out: item.second) { + newOutIndicies[out] = outs.size(); + } + switchArgs.push_back( + Build<TCoAtomList>(ctx, lambda.Pos()) + .Add() + .Value(ctx.GetIndexAsString(i)) + .Build() + .Done()); + switchArgs.push_back( + Build<TCoLambda>(ctx, lambda.Pos()) + .Args({"flow"}) + .Body("flow") + .Done()); + outs.push_back(op.Output().Item(i)); + } + + for (auto reader: item.first) { + if (auto it = opDeps.find(reader); it != opDeps.end()) { + for (auto& deps: it->second) { + const auto out = std::get<2>(deps); + const auto oldIndex = FromString<size_t>(out->Child(TYtOutput::idx_OutIndex)->Content()); + newOutIndicies[out] = outs.size() + oldIndex; + } + } + const auto outerMap = TYtMap(reader); + switchArgs.push_back( + Build<TCoAtomList>(ctx, lambda.Pos()) + .Add() + .Value(ctx.GetIndexAsString(i)) + .Build() + .Done()); + + TExprNode::TListType members; + for (auto item : GetSeqItemType(*outerMap.Mapper().Args().Arg(0).Ref().GetTypeAnn()).Cast<TStructExprType>()->GetItems()) { + members.push_back(ctx.NewAtom(outerMap.Mapper().Pos(), item->GetName())); + } + + auto [placeHolder, lambdaWithPlaceholder] = ReplaceDependsOn(outerMap.Mapper().Ptr(), ctx, State_->Types); + YQL_ENSURE(placeHolder); + + switchArgs.push_back(Build<TCoLambda>(ctx, outerMap.Mapper().Pos()) + .Args({"flow"}) + .Body<TExprApplier>() + .Apply(TCoLambda(lambdaWithPlaceholder)) + .With<TCoExtractMembers>(0) + .Input("flow") + .Members() + .Add(members) + .Build() + .Build() + .With(TExprBase(placeHolder), "flow") + .Build() + .Done()); + + outs.insert(outs.end(), outerMap.Output().begin(), outerMap.Output().end()); + removed.insert(reader); + } + } + + lambda = Build<TCoLambda>(ctx, lambda.Pos()) + .Args({"flow"}) + .Body<TCoSwitch>() + .Input<TExprApplier>() + .Apply(lambda) + .With(0, "flow") + .Build() + .BufferBytes() + .Value(ToString(switchLimit)) + .Build() + .FreeArgs() + .Add(switchArgs) + .Build() + .Build() + .Done(); + + auto newOp = ctx.ChangeChild(op.Ref(), TYtOutputOpBase::idx_Output, + Build<TYtOutSection>(ctx, op.Pos()).Add(outs).Done().Ptr()); + + newOp = ctx.ChangeChild(*newOp, lambdaNdx, lambda.Ptr()); + + remaps[op.Raw()] = newOp; + for (auto node: lefts) { + TCoLeft left(node); + if (removed.contains(left.Input().Raw())) { + remaps[node] = ctx.ChangeChild(*node, TCoLeft::idx_Input, TExprNode::TPtr(newOp)); + } + } + for (auto& item: newOutIndicies) { + remaps[item.first] = Build<TYtOutput>(ctx, item.first->Pos()) + .InitFrom(TYtOutput(item.first)) + .Operation(newOp) + .OutIndex() + .Value(ctx.GetIndexAsString(item.second)) + .Build() + .Done().Ptr(); + } + YQL_CLOG(INFO, ProviderYt) << "PhysicalFinalizing-FuseMultiOutsWithOuterMaps"; + return RemapExpr(input, output, remaps, ctx, TOptimizeExprSettings(State_->Types)); + } + } + } + return TStatus::Ok; + } + + TExprNode::TPtr UpdateColumnGroups(const TYtOutputOpBase& op, const std::map<size_t, TString>& groupSpecs, TExprContext& ctx) { + auto origOutput = op.Output().Ptr(); + auto newOutput = origOutput; + for (const auto& item: groupSpecs) { + const auto table = op.Output().Item(item.first); + auto currentGroup = GetSetting(table.Settings().Ref(), EYtSettingType::ColumnGroups); + if (!currentGroup || currentGroup->Tail().Content() != item.second) { + auto newSettings = AddOrUpdateSettingValue(table.Settings().Ref(), + EYtSettingType::ColumnGroups, + ctx.NewAtom(table.Settings().Pos(), item.second, TNodeFlags::MultilineContent), + ctx); + auto newTable = ctx.ChangeChild(table.Ref(), TYtOutTable::idx_Settings, std::move(newSettings)); + newOutput = ctx.ChangeChild(*newOutput, item.first, std::move(newTable)); + } + } + if (newOutput != origOutput) { + return ctx.ChangeChild(op.Ref(), TYtOutputOpBase::idx_Output, std::move(newOutput)); + } + if (auto copy = op.Maybe<TYtCopy>()) { + TStringBuf inputColGroup; + const auto& path = copy.Cast().Input().Item(0).Paths().Item(0); + if (auto table = path.Table().Maybe<TYtTable>()) { + if (auto tableDesc = State_->TablesData->FindTable(copy.Cast().DataSink().Cluster().StringValue(), TString{TYtTableInfo::GetTableLabel(table.Cast())}, TEpochInfo::Parse(table.Cast().Epoch().Ref()))) { + inputColGroup = tableDesc->ColumnGroupSpec; + } + } else if (auto out = path.Table().Maybe<TYtOutput>()) { + if (auto setting = NYql::GetSetting(GetOutputOp(out.Cast()).Output().Item(FromString<ui32>(out.Cast().OutIndex().Value())).Settings().Ref(), EYtSettingType::ColumnGroups)) { + inputColGroup = setting->Tail().Content(); + } + } + TStringBuf outGroup; + if (auto setting = GetSetting(op.Output().Item(0).Settings().Ref(), EYtSettingType::ColumnGroups)) { + outGroup = setting->Tail().Content(); + } + if (inputColGroup != outGroup) { + return ctx.RenameNode(op.Ref(), TYtMerge::CallableName()); + } + } + return {}; + } + + struct TColumnUsage { + bool GenerateGroups; + std::vector<const TStructExprType*> OutTypes; + std::vector<std::unordered_map<TString, std::set<size_t>>> ColumnUsage; + std::vector<bool> FullUsage; + std::vector<std::unordered_set<TString>> PublishUsage; + std::vector<std::vector<const TExprNode*>> UsedByMerges; + }; + + void GatherColumnUsage(EColumnGroupMode mode, const TExprNode* writer, const TOpDeps::mapped_type& readers, const TOpDeps& opDeps, TColumnUsage& usage, TNodeMap<size_t>& uniquePaths) { + for (const auto& outTable: GetRealOperation(TExprBase(writer)).Output()) { + usage.OutTypes.push_back(outTable.Ref().GetTypeAnn()->Cast<TListExprType>()->GetItemType()->Cast<TStructExprType>()); + } + + usage.ColumnUsage.resize(usage.OutTypes.size()); + usage.FullUsage.resize(usage.OutTypes.size()); + usage.PublishUsage.resize(usage.OutTypes.size()); + usage.UsedByMerges.resize(usage.OutTypes.size()); + // Collect column usage per consumer + for (auto& item: readers) { + const auto out = TYtOutput(std::get<2>(item)); + const auto outIndex = FromString<size_t>(out.OutIndex().Value()); + YQL_ENSURE(outIndex < usage.OutTypes.size()); + auto rawPath = std::get<3>(item); + if (!rawPath) { + if (TYtLength::Match(std::get<0>(item))) { + continue; + } + if (auto maybePublish = TMaybeNode<TYtPublish>(std::get<0>(item))) { + TYtTableInfo dstInfo = maybePublish.Cast().Publish(); + const auto& desc = State_->TablesData->GetTable(dstInfo.Cluster, dstInfo.Name, dstInfo.CommitEpoch); + usage.PublishUsage[outIndex].insert(desc.ColumnGroupSpec); + } else if (TResPull::Match(std::get<0>(item))) { + usage.PublishUsage[outIndex].emplace(); + } else { + usage.FullUsage[outIndex] = true; + } + } else if (auto maybeMerge = TMaybeNode<TYtMerge>(std::get<0>(item)); maybeMerge && AllOf(maybeMerge.Cast().Input().Item(0).Paths(), + [](const TYtPath& path) { return path.Ref().GetTypeAnn()->Equals(*path.Table().Ref().GetTypeAnn()); })) { + + // YtMerge may have no usage in the graph (only via Left!) + if (opDeps.contains(std::get<0>(item))) { + usage.UsedByMerges[outIndex].push_back(std::get<0>(item)); + } + + } else if (TYtCopy::Match(std::get<0>(item))) { + + // YtCopy may have no usage in the graph (only via Left!) + if (opDeps.contains(std::get<0>(item))) { + usage.UsedByMerges[outIndex].push_back(std::get<0>(item)); + } + + } else if (EColumnGroupMode::Single == mode) { + usage.FullUsage[outIndex] = true; + } else { + auto path = TYtPath(rawPath); + auto columns = TYtColumnsInfo(path.Columns()); + if (!columns.HasColumns() || usage.OutTypes[outIndex]->GetSize() <= columns.GetColumns()->size()) { + usage.FullUsage[outIndex] = true; + } else { + const size_t pathNdx = uniquePaths.emplace(rawPath, uniquePaths.size()).first->second; + auto& cu = usage.ColumnUsage[outIndex]; + std::for_each(columns.GetColumns()->cbegin(), columns.GetColumns()->cend(), + [&cu, pathNdx](const TYtColumnsInfo::TColumn& c) { + cu[c.Name].insert(pathNdx); + } + ); + } + } + } + } + + TStatus CalculateColumnGroups(TExprNode::TPtr input, TExprNode::TPtr& output, std::vector<const TExprNode*> opDepsOrder, const TOpDeps& opDeps, EColumnGroupMode mode, TExprContext& ctx) { + const auto maxGroups = State_->Configuration->MaxColumnGroups.Get().GetOrElse(DEFAULT_MAX_COLUMN_GROUPS); + const auto minGroupSize = State_->Configuration->MinColumnGroupSize.Get().GetOrElse(DEFAULT_MIN_COLUMN_GROUP_SIZE); + + TNodeMap<TColumnUsage> colUsages; + TNodeMap<size_t> uniquePaths; + std::vector<const TExprNode*> mergesToProcess; + std::vector<const TExprNode*> withMergeDeps; + + for (auto writer: opDepsOrder) { + if (TYtEquiJoin::Match(writer) || IsBeingExecuted(*writer)) { + continue; + } + const auto& readers = opDeps.at(writer); + + // Check all counsumers are known + auto& processed = ProcessedCalculateColumnGroups[writer]; + if (processed.size() == readers.size() && AllOf(readers, [&processed](const auto& item) { return processed.contains(std::get<0>(item)->UniqueId()); })) { + continue; + } + processed.clear(); + std::transform(readers.begin(), readers.end(), + std::inserter(processed, processed.end()), + [](const auto& item) { return std::get<0>(item)->UniqueId(); } + ); + + TColumnUsage& usage = colUsages[writer]; + usage.GenerateGroups = true; + + GatherColumnUsage(mode, writer, readers, opDeps, usage, uniquePaths); + bool hasMergeDep = false; + for (const auto& item: usage.UsedByMerges) { + hasMergeDep = hasMergeDep || !item.empty(); + mergesToProcess.insert(mergesToProcess.end(), item.begin(), item.end()); + } + if (hasMergeDep) { + withMergeDeps.push_back(writer); + } + } + + while (!mergesToProcess.empty()) { + std::vector<const TExprNode*> nextMergesToProcess; + for (auto merge: mergesToProcess) { + auto res = colUsages.emplace(merge, TColumnUsage{}); + if (res.second) { // Not processed before + TColumnUsage& usage = res.first->second; + usage.GenerateGroups = TYtCopy::Match(merge); // Maybe we need to rewrite YtCopy to YtMerge + GatherColumnUsage(mode, merge, opDeps.at(merge), opDeps, usage, uniquePaths); + bool hasMergeDep = false; + for (const auto& item: usage.UsedByMerges) { + hasMergeDep = hasMergeDep || !item.empty(); + nextMergesToProcess.insert(nextMergesToProcess.end(), item.begin(), item.end()); + } + if (hasMergeDep) { + withMergeDeps.push_back(merge); + } + } + } + nextMergesToProcess.swap(mergesToProcess); + } + + // In case of [YtOp -> YtMerge -> ...consumers] inherit column usage from YtMerge to YtOp + // Iterate in reverse order - from top-level operations to deeper + for (auto ri = withMergeDeps.rbegin(); ri != withMergeDeps.rend(); ++ri) { + TColumnUsage& usage = colUsages.at(*ri); + for (size_t outIndex = 0; outIndex < usage.UsedByMerges.size(); ++outIndex) { + for (auto merge: usage.UsedByMerges[outIndex]) { + const TColumnUsage& mergeUsage = colUsages.at(merge); + if (TYtCopy::Match(merge)) { + usage.FullUsage[outIndex] = mergeUsage.FullUsage.at(0); + usage.PublishUsage[outIndex] = mergeUsage.PublishUsage.at(0); + usage.ColumnUsage[outIndex] = mergeUsage.ColumnUsage.at(0); + break; // Don't process others. YtCopy enforces exact the same column groups + } else { + usage.FullUsage[outIndex] = usage.FullUsage[outIndex] || mergeUsage.FullUsage.at(0); + usage.PublishUsage[outIndex].insert(mergeUsage.PublishUsage.at(0).cbegin(), mergeUsage.PublishUsage.at(0).cend()); + auto& cu = usage.ColumnUsage[outIndex]; + for (const auto& p: mergeUsage.ColumnUsage.at(0)) { + cu[p.first].insert(p.second.cbegin(), p.second.cend()); + } + } + } + } + } + + TNodeOnNodeOwnedMap remap; + for (auto& x: colUsages) { + auto writer = x.first; + TColumnUsage& usage = x.second; + if (usage.GenerateGroups) { + + std::map<size_t, TString> groupSpecs; + for (size_t i = 0; i < usage.OutTypes.size(); ++i) { + if (!usage.PublishUsage[i].empty()) { + if (usage.PublishUsage[i].size() == 1) { + if (auto spec = *usage.PublishUsage[i].cbegin(); !spec.empty()) { + groupSpecs[i] = spec; + } + } + continue; + } + if (EColumnGroupMode::Single == mode) { + if (usage.FullUsage[i]) { + groupSpecs[i] = NYql::GetSingleColumnGroupSpec(); + } + } else { + if (usage.FullUsage[i]) { + // Add all columns for tables with entire usage + const size_t pathNdx = uniquePaths.emplace(nullptr, uniquePaths.size()).first->second; + auto& cu = usage.ColumnUsage[i]; + std::for_each(usage.OutTypes[i]->GetItems().cbegin(), usage.OutTypes[i]->GetItems().cend(), + [&cu, pathNdx](const TItemExprType* itemType) { + cu[TString{itemType->GetName()}].insert(pathNdx); + } + ); + } + + if (!usage.ColumnUsage[i].empty()) { + auto groupSpec = NYT::TNode(); + + // Find unique groups. Use ordered collections for stable names + std::map<std::set<size_t>, std::set<TString>> groups; + for (const auto& item: usage.ColumnUsage[i]) { + groups[item.second].insert(item.first); + if (groups.size() > maxGroups) { + groups.clear(); + break; + } + } + if (!groups.empty()) { + bool allGroups = true; + size_t maxSize = 0; + auto maxGrpIt = groups.end(); + // Delete too short groups and find a group with max size + for (auto it = groups.begin(); it != groups.end();) { + if (it->second.size() < minGroupSize) { + it = groups.erase(it); + allGroups = false; + } else { + if (it->second.size() > maxSize) { + maxSize = it->second.size(); + maxGrpIt = it; + } + ++it; + } + } + if (!groups.empty()) { + groupSpec = NYT::TNode::CreateMap(); + // If we keep all groups then use the group with max size as default + if (allGroups && maxGrpIt != groups.end()) { + groupSpec["default"] = NYT::TNode::CreateEntity(); + groups.erase(maxGrpIt); + } + TStringBuilder nameBuilder; + nameBuilder.reserve(8); // "group" + 2 digit number + zero-terminator + nameBuilder << "group"; + size_t num = 0; + for (const auto& g: groups) { + nameBuilder.resize(5); + nameBuilder << num++; + auto columns = NYT::TNode::CreateList(); + for (const auto& n: g.second) { + columns.Add(n); + } + groupSpec[nameBuilder] = std::move(columns); + } + } + } + if (!groupSpec.IsUndefined()) { + groupSpecs[i] = NYT::NodeToCanonicalYsonString(groupSpec, NYson::EYsonFormat::Text); + } + } + } + } + if (!groupSpecs.empty()) { + TExprNode::TPtr newOp; + if (const auto mayTry = TExprBase(writer).Maybe<TYtTryFirst>()) { + TExprNode::TPtr newOpFirst = UpdateColumnGroups(mayTry.Cast().First(), groupSpecs, ctx); + TExprNode::TPtr newOpSecond = UpdateColumnGroups(mayTry.Cast().Second(), groupSpecs, ctx); + if (newOpFirst || newOpSecond) { + newOp = Build<TYtTryFirst>(ctx, writer->Pos()) + .First(newOpFirst ? std::move(newOpFirst) : mayTry.Cast().First().Ptr()) + .Second(newOpSecond ? std::move(newOpSecond) : mayTry.Cast().Second().Ptr()) + .Done().Ptr(); + } + } else { + newOp = UpdateColumnGroups(TYtOutputOpBase(writer), groupSpecs, ctx); + } + if (newOp) { + remap[writer] = newOp; + } + } + } + } + + if (!remap.empty()) { + YQL_CLOG(INFO, ProviderYt) << "PhysicalFinalizing-CalculateColumnGroups"; + return RemapExpr(input, output, remap, ctx, TOptimizeExprSettings{State_->Types}); + } + return TStatus::Ok; + } + + bool BeingExecuted(const TExprNode& node) { + return node.GetState() > TExprNode::EState::ExecutionRequired || node.HasResult(); + } + +private: + TYtState::TPtr State_; + TProcessedNodesSet ProcessedMergePublish; + TProcessedNodesSet ProcessedSplitLargeInputs; + TProcessedNodesSet ProcessedUnusedOuts; + TProcessedNodesSet ProcessedMultiOuts; + TProcessedNodesSet ProcessedHorizontalJoin; + TProcessedNodesSet ProcessedFieldSubsetForMultiUsage; + TNodeMap<TProcessedNodesSet> ProcessedCalculateColumnGroups; + std::unordered_set<std::pair<ui64, ui64>, THash<std::pair<ui64, ui64>>> ProcessedFuseWithOuterMaps; +}; + +THashSet<TStringBuf> TYtPhysicalFinalizingTransformer::OPS_WITH_SORTED_OUTPUT = { + TYtMerge::CallableName(), + TYtMap::CallableName(), + TYtSort::CallableName(), + TYtReduce::CallableName(), + TYtFill::CallableName(), + TYtDqProcessWrite::CallableName(), + TYtTryFirst::CallableName(), +}; + +} + +THolder<IGraphTransformer> CreateYtPhysicalFinalizingTransformer(TYtState::TPtr state) { + return THolder(new TYtPhysicalFinalizingTransformer(state)); +} + +} // NYql + diff --git a/yt/yql/providers/yt/provider/yql_yt_physical_optimize.cpp b/yt/yql/providers/yt/provider/yql_yt_physical_optimize.cpp new file mode 100644 index 0000000000..1c66050072 --- /dev/null +++ b/yt/yql/providers/yt/provider/yql_yt_physical_optimize.cpp @@ -0,0 +1,56 @@ +#include "phy_opt/yql_yt_phy_opt.h" + +#include <yql/essentials/core/yql_graph_transformer.h> + + +#include <util/generic/ptr.h> + + +namespace NYql { + +namespace { + +using namespace NNodes; + + +class TAsyncSyncCompositeTransformer : public TGraphTransformerBase { +public: + TAsyncSyncCompositeTransformer(THolder<IGraphTransformer>&& async, THolder<IGraphTransformer>&& sync) + : Async(std::move(async)) + , Sync(std::move(sync)) + { + } +private: + TStatus DoTransform(TExprNode::TPtr input, TExprNode::TPtr& output, TExprContext& ctx) override { + auto status = Async->Transform(input, output, ctx); + if (status.Level != TStatus::Ok) { + return status; + } + return InstantTransform(*Sync, output, ctx, true); + } + + NThreading::TFuture<void> DoGetAsyncFuture(const TExprNode& input) override { + return Async->GetAsyncFuture(input); + } + + TStatus DoApplyAsyncChanges(TExprNode::TPtr input, TExprNode::TPtr& output, TExprContext& ctx) override { + return Async->ApplyAsyncChanges(input, output, ctx); + } + + void Rewind() final { + Async->Rewind(); + Sync->Rewind(); + } + + const THolder<IGraphTransformer> Async; + const THolder<IGraphTransformer> Sync; + +}; + +} // namespce + +THolder<IGraphTransformer> CreateYtPhysicalOptProposalTransformer(TYtState::TPtr state) { + return MakeHolder<TAsyncSyncCompositeTransformer>(CreateYtLoadColumnarStatsTransformer(state), MakeHolder<TYtPhysicalOptProposalTransformer>(state)); +} + +} // namespace NYql diff --git a/yt/yql/providers/yt/provider/yql_yt_provider.cpp b/yt/yql/providers/yt/provider/yql_yt_provider.cpp new file mode 100644 index 0000000000..44c70d483a --- /dev/null +++ b/yt/yql/providers/yt/provider/yql_yt_provider.cpp @@ -0,0 +1,552 @@ +#include "yql_yt_provider.h" +#include "yql_yt_dq_integration.h" + +#include <yt/yql/providers/yt/expr_nodes/yql_yt_expr_nodes.h> +#include <yt/yql/providers/yt/common/yql_names.h> +#include <yql/essentials/providers/common/provider/yql_provider_names.h> +#include <yql/essentials/providers/common/proto/gateways_config.pb.h> +#include <yql/essentials/providers/common/activation/yql_activation.h> +#include <yql/essentials/providers/common/schema/expr/yql_expr_schema.h> +#include <yt/yql/providers/yt/gateway/qplayer/yql_yt_qplayer_gateway.h> + +#include <util/generic/singleton.h> + +namespace NYql { + +bool TYtTableDescription::Fill( + const TString& cluster, const TString& table, const TQContext& qContext, TExprContext& ctx, + IModuleResolver* moduleResolver, IUrlListerManager* urlListerManager, IRandomProvider& randomProvider, + bool allowViewIsolation, IUdfResolver::TPtr udfResolver) { + const TStructExprType* type = RowSpec ? RowSpec->GetType() : nullptr; + if (!type) { + TVector<const TItemExprType*> items; + if (Meta->YqlCompatibleScheme) { + for (auto& name : YAMR_FIELDS) { + items.push_back(ctx.MakeType<TItemExprType>(name, ctx.MakeType<TDataExprType>(EDataSlot::String))); + } + } + type = ctx.MakeType<TStructExprType>(items); + } + + if (!TYtTableDescriptionBase::Fill(TString{YtProviderName}, cluster, + table, type, Meta->SqlView, Meta->SqlViewSyntaxVersion, qContext, Meta->Attrs, ctx, + moduleResolver, urlListerManager, randomProvider, allowViewIsolation, udfResolver)) { + return false; + } + if (QB2RowSpec) { + RowType = QB2RowSpec->GetType(); + } + return true; +} + +void TYtTableDescription::ToYson(NYson::TYsonWriter& writer, const TString& cluster, const TString& table, const TString& view) const +{ + YQL_ENSURE(Stat); + YQL_ENSURE(Meta); + const bool isView = !view.empty() || View.Defined(); + const TYtViewDescription* viewMeta = !view.empty() ? Views.FindPtr(view) : View.Get(); + + writer.OnBeginMap(); + writer.OnKeyedItem("Cluster"); + writer.OnStringScalar(cluster); + writer.OnKeyedItem("Name"); + writer.OnStringScalar(table); + if (isView) { + YQL_ENSURE(viewMeta); + writer.OnKeyedItem("View"); + writer.OnStringScalar(!view.empty() ? view : table); + writer.OnKeyedItem("Sql"); + writer.OnStringScalar(viewMeta->Sql); + } else { + writer.OnKeyedItem("DoesExist"); + writer.OnBooleanScalar(Meta->DoesExist); + writer.OnKeyedItem("IsEmpty"); + writer.OnBooleanScalar(Stat->IsEmpty()); + writer.OnKeyedItem("IsSorted"); + writer.OnBooleanScalar(RowSpec && RowSpec->IsSorted()); + writer.OnKeyedItem("IsDynamic"); + writer.OnBooleanScalar(Meta->IsDynamic); + writer.OnKeyedItem("UniqueKeys"); + writer.OnBooleanScalar(RowSpec && RowSpec->UniqueKeys); + writer.OnKeyedItem("CanWrite"); + writer.OnBooleanScalar(Meta->CanWrite); + writer.OnKeyedItem("RecordsCount"); + writer.OnInt64Scalar(Stat->RecordsCount); + writer.OnKeyedItem("DataSize"); + writer.OnInt64Scalar(Stat->DataSize); + writer.OnKeyedItem("ChunkCount"); + writer.OnInt64Scalar(Stat->ChunkCount); + writer.OnKeyedItem("ModifyTime"); + writer.OnInt64Scalar(Stat->ModifyTime); + writer.OnKeyedItem("Id"); + writer.OnStringScalar(Stat->Id); + writer.OnKeyedItem("Revision"); + writer.OnUint64Scalar(Stat->Revision); + writer.OnKeyedItem("IsRealData"); + + bool isRealData = !Meta->Attrs.contains(QB2Premapper) && !Meta->Attrs.contains(YqlReadUdfAttribute) && !IgnoreTypeV3; + if (isRealData) { + for (auto& x: Meta->Attrs) { + if (x.first.StartsWith(YqlProtoFieldPrefixAttribute)) { + isRealData = false; + break; + } + } + } + writer.OnBooleanScalar(isRealData); + writer.OnKeyedItem("YqlCompatibleSchema"); + writer.OnBooleanScalar(Meta->YqlCompatibleScheme); + } + + const TTypeAnnotationNode* rowType = viewMeta + ? viewMeta->RowType + : QB2RowSpec + ? QB2RowSpec->GetType() + : RowType; + auto rowSpec = QB2RowSpec ? QB2RowSpec : RowSpec; + + // fields + + auto writeSortOrder = [&writer](TMaybe<size_t> order, TMaybe<bool> ascending) { + writer.OnKeyedItem("ClusterSortOrder"); + writer.OnBeginList(); + if (order) { + writer.OnListItem(); + writer.OnInt64Scalar(*order); + } + writer.OnEndList(); + + writer.OnKeyedItem("Ascending"); + writer.OnBeginList(); + if (ascending) { + writer.OnListItem(); + writer.OnBooleanScalar(*ascending); + } + writer.OnEndList(); + }; + + writer.OnKeyedItem("Fields"); + writer.OnBeginList(); + if ((isView || Meta->DoesExist) && rowType->GetKind() == ETypeAnnotationKind::Struct) { + for (auto& item: rowType->Cast<TStructExprType>()->GetItems()) { + writer.OnListItem(); + + auto name = item->GetName(); + writer.OnBeginMap(); + + writer.OnKeyedItem("Name"); + writer.OnStringScalar(name); + + writer.OnKeyedItem("Type"); + NCommon::WriteTypeToYson(writer, item->GetItemType()); + + size_t fieldIdx = rowSpec && rowSpec->IsSorted() + ? FindIndex(rowSpec->SortMembers, name) + : NPOS; + + if (fieldIdx != NPOS) { + bool ascending = !rowSpec->SortDirections.empty() + ? rowSpec->SortDirections.at(fieldIdx) + : true; + writeSortOrder(fieldIdx, ascending); + } else { + writeSortOrder(Nothing(), Nothing()); + } + writer.OnEndMap(); + } + } + writer.OnEndList(); + + writer.OnKeyedItem("RowType"); + NCommon::WriteTypeToYson(writer, rowType); + + if (!isView) { + // meta attr + writer.OnKeyedItem("MetaAttr"); + writer.OnBeginMap(); + + for (const auto& attr : Meta->Attrs) { + if (attr.first.StartsWith("_yql")) { + continue; + } + + writer.OnKeyedItem(attr.first); + writer.OnStringScalar(attr.second); + } + + writer.OnEndMap(); + + // views + writer.OnKeyedItem("Views"); + TVector<TString> views; + for (const auto& attr : Meta->Attrs) { + if (!attr.first.StartsWith(YqlViewPrefixAttribute) || attr.first.size() == YqlViewPrefixAttribute.size()) { + continue; + } + views.push_back(attr.first.substr(YqlViewPrefixAttribute.size())); + } + std::sort(begin(views), end(views)); + writer.OnBeginList(); + for (const auto& v: views) { + writer.OnListItem(); + writer.OnStringScalar(v); + } + writer.OnEndList(); + } + + writer.OnEndMap(); +} + +bool TYtTableDescription::Validate(TPosition pos, TStringBuf cluster, TStringBuf tableName, bool withQB, + const THashMap<std::pair<TString, TString>, TString>& anonymousLabels, TExprContext& ctx) const { + auto rowSpec = withQB ? QB2RowSpec : RowSpec; + if (FailOnInvalidSchema + && !rowSpec + && !Meta->YqlCompatibleScheme + && !Meta->InferredScheme + ) { + TMaybe<TString> anonLabel; + for (const auto& x : anonymousLabels) { + if (x.first.first == cluster && x.second == tableName) { + anonLabel = x.first.second; + break; + } + } + + if (anonLabel) { + ctx.AddError(TIssue(pos, TStringBuilder() << "Anonymous table '" << *anonLabel << "' must be materialized. Use COMMIT before reading from it.")); + } else if (InferSchemaRows > 0) { + ctx.AddError(TIssue(pos, TStringBuilder() << "Cannot infer schema for table " + << TString{tableName}.Quote() << ", table is empty").SetCode(TIssuesIds::YT_SCHEMELESS_TABLE, TSeverityIds::S_ERROR)); + } else { + ctx.AddError(TIssue(pos, TStringBuilder() << "Table " + << TString{tableName}.Quote() + << " does not have any scheme attribute supported by YQL," + << " you can add \"PRAGMA yt.InferSchema = '1';\" to" + << " your query in order to use types of first data row" + << " as scheme.").SetCode(TIssuesIds::YT_SCHEMELESS_TABLE, TSeverityIds::S_ERROR)); + } + return false; + } + return true; +} + +void TYtTableDescription::SetConstraintsReady() { + ConstraintsReady = true; + if (RowSpec && Constraints) { + RowSpec->SetConstraints(Constraints); + if (const auto sorted = Constraints.GetConstraint<TSortedConstraintNode>()) { + if (const auto distinct = Constraints.GetConstraint<TDistinctConstraintNode>()) { + RowSpec->UniqueKeys = distinct->IsOrderBy(*sorted); + } + } + } +} + +bool TYtTableDescription::FillViews( + const TString& cluster, const TString& table, const TQContext& qContext, TExprContext& ctx, + IModuleResolver* moduleResolver, IUrlListerManager* urlListerManager, IRandomProvider& randomProvider, + bool allowViewIsolation, IUdfResolver::TPtr udfResolver) { + return TYtTableDescriptionBase::FillViews( + TString{YtProviderName}, cluster, table, Meta->Attrs, qContext, ctx, + moduleResolver, urlListerManager, randomProvider, allowViewIsolation, udfResolver); +} + +const TYtTableDescription& TYtTablesData::GetTable(const TString& cluster, const TString& table, TMaybe<ui32> epoch) const { + auto p = Tables.FindPtr(std::make_tuple(cluster, table, epoch.GetOrElse(0))); + YQL_ENSURE(p, "Table description is not defined: " << cluster << '.' << table << "[epoch=" << epoch.GetOrElse(0) << ']'); + return *p; +} + +const TYtTableDescription* TYtTablesData::FindTable(const TString& cluster, const TString& table, TMaybe<ui32> epoch) const { + return Tables.FindPtr(std::make_tuple(cluster, table, epoch.GetOrElse(0))); +} + +TYtTableDescription& TYtTablesData::GetOrAddTable(const TString& cluster, const TString& table, TMaybe<ui32> epoch) { + return Tables[std::make_tuple(cluster, table, epoch.GetOrElse(0))]; +} + +TYtTableDescription& TYtTablesData::GetModifTable(const TString& cluster, const TString& table, TMaybe<ui32> epoch) { + auto p = Tables.FindPtr(std::make_tuple(cluster, table, epoch.GetOrElse(0))); + YQL_ENSURE(p, "Table description is not defined: " << cluster << '.' << table << "[epoch=" << epoch.GetOrElse(0) << ']'); + return *p; +} + +TVector<std::pair<TString, TString>> TYtTablesData::GetAllEpochTables(ui32 epoch) const { + TVector<std::pair<TString, TString>> res; + res.reserve(Tables.size()); + for (const auto& item: Tables) { + if (std::get<2>(item.first) == epoch) { + res.emplace_back(std::get<0>(item.first), std::get<1>(item.first)); + } + } + return res; +} + +TVector<std::pair<TString, TString>> TYtTablesData::GetAllZeroEpochTables() const { + return GetAllEpochTables(0U); +} + +void TYtTablesData::ForEach(const std::function<void(const TString&, const TString&, ui32, const TYtTableDescription&)>& cb) const { + for (const auto& item: Tables) { + cb(std::get<0>(item.first), std::get<1>(item.first), std::get<2>(item.first), item.second); + } +} + +void TYtTablesData::CleanupCompiledSQL() { + for (auto& item: Tables) { + item.second.CleanupCompiledSQL(); + } +} + +void TYtState::Reset() { + LoadEpochMetadata.Clear(); + EpochDependencies.clear(); + Configuration->ClearVersions(); + TablesData = MakeIntrusive<TYtTablesData>(); + AnonymousLabels.clear(); + NodeHash.clear(); + Checkpoints.clear(); + WalkFoldersState.clear(); + NextEpochId = 1; + FlowDependsOnId = 0; +} + +void TYtState::EnterEvaluation(ui64 id) { + bool res = ConfigurationEvalStates_.emplace(id, Configuration->GetState()).second; + YQL_ENSURE(res, "Duplicate evaluation state " << id); + + res = EpochEvalStates_.emplace(id, NextEpochId).second; + YQL_ENSURE(res, "Duplicate evaluation state " << id); +} + +void TYtState::LeaveEvaluation(ui64 id) { + { + auto it = ConfigurationEvalStates_.find(id); + YQL_ENSURE(it != ConfigurationEvalStates_.end()); + Configuration->RestoreState(std::move(it->second)); + ConfigurationEvalStates_.erase(it); + } + + { + auto it = EpochEvalStates_.find(id); + YQL_ENSURE(it != EpochEvalStates_.end()); + NextEpochId = it->second; + EpochEvalStates_.erase(it); + } +} + +std::pair<TIntrusivePtr<TYtState>, TStatWriter> CreateYtNativeState(IYtGateway::TPtr gateway, const TString& userName, const TString& sessionId, + const TYtGatewayConfig* ytGatewayConfig, TIntrusivePtr<TTypeAnnotationContext> typeCtx, + const IOptimizerFactory::TPtr& optFactory, const IDqHelper::TPtr& helper) +{ + auto ytState = MakeIntrusive<TYtState>(typeCtx.Get()); + ytState->SessionId = sessionId; + ytState->Gateway = gateway; + ytState->DqIntegration_ = CreateYtDqIntegration(ytState.Get()); + ytState->OptimizerFactory_ = optFactory; + ytState->DqHelper = helper; + + if (ytGatewayConfig) { + std::unordered_set<std::string_view> groups; + if (ytState->Types->Credentials != nullptr) { + groups.insert(ytState->Types->Credentials->GetGroups().begin(), ytState->Types->Credentials->GetGroups().end()); + } + auto filter = [userName, ytState, groups = std::move(groups)](const NYql::TAttr& attr) -> bool { + if (!attr.HasActivation()) { + return true; + } + if (NConfig::Allow(attr.GetActivation(), userName, groups)) { + with_lock(ytState->StatisticsMutex) { + ytState->Statistics[Max<ui32>()].Entries.emplace_back(TStringBuilder() << "Activation:" << attr.GetName(), 0, 0, 0, 0, 1); + } + return true; + } + return false; + }; + + ytState->Configuration->Init(*ytGatewayConfig, filter, *typeCtx); + } + + TStatWriter statWriter = [ytState](ui32 publicId, const TVector<TOperationStatistics::TEntry>& stat) { + with_lock(ytState->StatisticsMutex) { + for (size_t i = 0; i < stat.size(); ++i) { + ytState->Statistics[publicId].Entries.push_back(stat[i]); + } + } + }; + + return {ytState, statWriter}; +} + +TDataProviderInitializer GetYtNativeDataProviderInitializer(IYtGateway::TPtr gateway, IOptimizerFactory::TPtr optFactory, IDqHelper::TPtr helper, ui32 planLimits) { + return [originalGateway = gateway, optFactory, helper, planLimits] ( + const TString& userName, + const TString& sessionId, + const TGatewaysConfig* gatewaysConfig, + const NKikimr::NMiniKQL::IFunctionRegistry* functionRegistry, + TIntrusivePtr<IRandomProvider> randomProvider, + TIntrusivePtr<TTypeAnnotationContext> typeCtx, + const TOperationProgressWriter& progressWriter, + const TYqlOperationOptions& operationOptions, + THiddenQueryAborter hiddenAborter, + const TQContext& qContext + ) { + Y_UNUSED(functionRegistry); + Y_UNUSED(randomProvider); + Y_UNUSED(progressWriter); + Y_UNUSED(operationOptions); + Y_UNUSED(hiddenAborter); + auto gateway = originalGateway; + if (qContext) { + gateway = WrapYtGatewayWithQContext(originalGateway, qContext, + typeCtx->RandomProvider, typeCtx->FileStorage); + } + + TDataProviderInfo info; + info.SupportsHidden = true; + + const TYtGatewayConfig* ytGatewayConfig = gatewaysConfig ? &gatewaysConfig->GetYt() : nullptr; + TIntrusivePtr<TYtState> ytState; + TStatWriter statWriter; + std::tie(ytState, statWriter) = CreateYtNativeState(gateway, userName, sessionId, ytGatewayConfig, typeCtx, optFactory, helper); + ytState->PlanLimits = planLimits; + + info.Names.insert({TString{YtProviderName}}); + info.Source = CreateYtDataSource(ytState); + info.Sink = CreateYtDataSink(ytState); + info.SupportFullResultDataSink = true; + info.OpenSession = [gateway, statWriter](const TString& sessionId, const TString& username, + const TOperationProgressWriter& progressWriter, const TYqlOperationOptions& operationOptions, + TIntrusivePtr<IRandomProvider> randomProvider, TIntrusivePtr<ITimeProvider> timeProvider) { + gateway->OpenSession( + IYtGateway::TOpenSessionOptions(sessionId) + .UserName(username) + .ProgressWriter(progressWriter) + .OperationOptions(operationOptions) + .RandomProvider(randomProvider) + .TimeProvider(timeProvider) + .StatWriter(statWriter) + ); + return NThreading::MakeFuture(); + }; + + info.CleanupSessionAsync = [ytState, gateway](const TString& sessionId) { + return gateway->CleanupSession(IYtGateway::TCleanupSessionOptions(sessionId)); + }; + + info.CloseSessionAsync = [ytState, gateway](const TString& sessionId) { + return gateway->CloseSession(IYtGateway::TCloseSessionOptions(sessionId)).Apply([ytState](const NThreading::TFuture<void>& future) { + // do manual cleanup; otherwise there may be dead nodes at program termination + // in setup with several providers + ytState->TablesData->CleanupCompiledSQL(); + + future.TryRethrow(); + }); + }; + + info.TokenResolver = [ytState, gateway](const TString& url, const TString& alias) -> TString { + TString cluster; + // assume it is not a YT link at all + if (!gateway->TryParseYtUrl(url, &cluster, nullptr) && !url.StartsWith("yt:") && alias != "yt") { + return {}; + } + + // todo: get token by cluster name from Auth when it will be implemented + if (auto token = ytState->Configuration->Auth.Get()) { + return *token; + } + + if (cluster) { + if (auto p = ytState->Configuration->Tokens.FindPtr(cluster)) { + return *p; + } + } + return {}; + }; + + return info; + }; +} + +namespace { + +using namespace NNodes; + +struct TYtDataSourceFunctions { + THashSet<TStringBuf> Names; + + TYtDataSourceFunctions() { + Names.insert(TEpoch::CallableName()); + Names.insert(TYtMeta::CallableName()); + Names.insert(TYtStat::CallableName()); + Names.insert(TYqlRowSpec::CallableName()); + Names.insert(TYtTable::CallableName()); + Names.insert(TYtRow::CallableName()); + Names.insert(TYtRowRange::CallableName()); + Names.insert(TYtKeyExact::CallableName()); + Names.insert(TYtKeyRange::CallableName()); + Names.insert(TYtPath::CallableName()); + Names.insert(TYtSection::CallableName()); + Names.insert(TYtReadTable::CallableName()); + Names.insert(TYtReadTableScheme::CallableName()); + Names.insert(TYtTableContent::CallableName()); + Names.insert(TYtLength::CallableName()); + Names.insert(TYtConfigure::CallableName()); + Names.insert(TYtTablePath::CallableName()); + Names.insert(TYtTableRecord::CallableName()); + Names.insert(TYtTableIndex::CallableName()); + Names.insert(TYtIsKeySwitch::CallableName()); + Names.insert(TYtRowNumber::CallableName()); + Names.insert(TYtStatOutTable::CallableName()); + } +}; + +struct TYtDataSinkFunctions { + THashSet<TStringBuf> Names; + + TYtDataSinkFunctions() { + Names.insert(TYtOutTable::CallableName()); + Names.insert(TYtOutput::CallableName()); + Names.insert(TYtSort::CallableName()); + Names.insert(TYtCopy::CallableName()); + Names.insert(TYtMerge::CallableName()); + Names.insert(TYtMap::CallableName()); + Names.insert(TYtReduce::CallableName()); + Names.insert(TYtMapReduce::CallableName()); + Names.insert(TYtWriteTable::CallableName()); + Names.insert(TYtFill::CallableName()); + Names.insert(TYtTouch::CallableName()); + Names.insert(TYtDropTable::CallableName()); + Names.insert(TCoCommit::CallableName()); + Names.insert(TYtPublish::CallableName()); + Names.insert(TYtEquiJoin::CallableName()); + Names.insert(TYtStatOut::CallableName()); + } +}; + +} + +const THashSet<TStringBuf>& YtDataSourceFunctions() { + return Default<TYtDataSourceFunctions>().Names; +} + +const THashSet<TStringBuf>& YtDataSinkFunctions() { + return Default<TYtDataSinkFunctions>().Names; +} + +bool TYtState::IsHybridEnabled() const { + return Types->PureResultDataSource == DqProviderName + && Configuration->HybridDqExecution.Get().GetOrElse(DefaultHybridDqExecution) && Types->HiddenMode == EHiddenMode::Disable; +} + +bool TYtState::IsHybridEnabledForCluster(const std::string_view& cluster) const { + return !OnlyNativeExecution && Configuration->_EnableDq.Get(TString(cluster)).GetOrElse(true); +} + +bool TYtState::HybridTakesTooLong() const { + return TimeSpentInHybrid + (HybridInFlightOprations.empty() ? TDuration::Zero() : NMonotonic::TMonotonic::Now() - HybridStartTime) + > Configuration->HybridDqTimeSpentLimit.Get().GetOrElse(TDuration::Minutes(20)); +} + +} diff --git a/yt/yql/providers/yt/provider/yql_yt_provider.h b/yt/yql/providers/yt/provider/yql_yt_provider.h new file mode 100644 index 0000000000..e79c41cb13 --- /dev/null +++ b/yt/yql/providers/yt/provider/yql_yt_provider.h @@ -0,0 +1,144 @@ +#pragma once + +#include "yql_yt_gateway.h" +#include "yql_yt_table_desc.h" +#include "yql_yt_table.h" +#include "yql_yt_io_discovery_walk_folders.h" + +#include <yt/yql/providers/yt/common/yql_yt_settings.h> +#include <yt/yql/providers/yt/lib/row_spec/yql_row_spec.h> +#include <yql/essentials/core/cbo/cbo_optimizer_new.h> +#include <yql/essentials/core/dq_integration/yql_dq_integration.h> +#include <yql/essentials/core/dq_integration/yql_dq_helper.h> +#include <yql/essentials/core/yql_data_provider.h> +#include <yql/essentials/core/yql_execution.h> +#include <yql/essentials/ast/yql_constraint.h> + +#include <library/cpp/time_provider/monotonic.h> +#include <library/cpp/yson/writer.h> + +#include <util/generic/string.h> +#include <util/generic/set.h> +#include <util/generic/hash.h> +#include <util/generic/ptr.h> +#include <util/generic/vector.h> +#include <util/generic/maybe.h> +#include <util/generic/hash_set.h> +#include <util/generic/strbuf.h> +#include <util/system/mutex.h> +#include <util/str_stl.h> + +#include <utility> +#include <tuple> +#include <unordered_map> + +namespace NYql { + +struct TYtTableDescription: public TYtTableDescriptionBase { + TYtTableStatInfo::TPtr Stat; + TYtTableMetaInfo::TPtr Meta; + TYqlRowSpecInfo::TPtr RowSpec; + TYqlRowSpecInfo::TPtr QB2RowSpec; + TConstraintSet Constraints; + bool ConstraintsReady = false; + bool IsAnonymous = false; + bool IsReplaced = false; + TMaybe<bool> MonotonicKeys; + std::unordered_map<ui32, size_t> WriteValidateCount; // mutationId -> validate count + TMaybe<TString> Hash; + TString ColumnGroupSpec; + bool RowSpecSortReady = false; + + bool Fill( + const TString& cluster, const TString& table, const TQContext& qContext, TExprContext& ctx, + IModuleResolver* moduleResolver, IUrlListerManager* urlListerManager, IRandomProvider& randomProvider, + bool allowViewIsolation, IUdfResolver::TPtr udfResolver); + void ToYson(NYson::TYsonWriter& writer, const TString& cluster, const TString& table, const TString& view) const; + bool Validate(TPosition pos, TStringBuf cluster, TStringBuf tableName, bool withQB, + const THashMap<std::pair<TString, TString>, TString>& anonymousLabels, TExprContext& ctx) const; + void SetConstraintsReady(); + bool FillViews( + const TString& cluster, const TString& table, const TQContext& qContext, TExprContext& ctx, + IModuleResolver* moduleResolver, IUrlListerManager* urlListerManager, IRandomProvider& randomProvider, + bool allowViewIsolation, IUdfResolver::TPtr udfResolver); +}; + +// Anonymous tables are kept by labels +class TYtTablesData: public TThrRefBase { +public: + using TPtr = TIntrusivePtr<TYtTablesData>; + + const TYtTableDescription& GetTable(const TString& cluster, const TString& table, TMaybe<ui32> epoch) const; + const TYtTableDescription* FindTable(const TString& cluster, const TString& table, TMaybe<ui32> epoch) const; + TYtTableDescription& GetOrAddTable(const TString& cluster, const TString& table, TMaybe<ui32> epoch); + TYtTableDescription& GetModifTable(const TString& cluster, const TString& table, TMaybe<ui32> epoch); + TVector<std::pair<TString, TString>> GetAllEpochTables(ui32 epoch) const; + TVector<std::pair<TString, TString>> GetAllZeroEpochTables() const; + void CleanupCompiledSQL(); + void ForEach(const std::function<void(const TString&, const TString&, ui32, const TYtTableDescription&)>& cb) const; +private: + using TTableKey = std::tuple<TString, TString, ui32>; // cluster + table + epoch + THashMap<TTableKey, TYtTableDescription> Tables; +}; + + +struct TYtState : public TThrRefBase { + using TPtr = TIntrusivePtr<TYtState>; + + void Reset(); + void EnterEvaluation(ui64 id); + void LeaveEvaluation(ui64 id); + bool IsHybridEnabled() const; + bool IsHybridEnabledForCluster(const std::string_view& cluster) const; + bool HybridTakesTooLong() const; + + TYtState(TTypeAnnotationContext* types) { + Types = types; + Configuration = MakeIntrusive<TYtVersionedConfiguration>(*types); + } + + TString SessionId; + IYtGateway::TPtr Gateway; + TTypeAnnotationContext* Types = nullptr; + TMaybe<std::pair<ui32, size_t>> LoadEpochMetadata; // Epoch being committed, settings versions + THashMap<ui32, TSet<std::pair<TString, TString>>> EpochDependencies; // List of tables, which have to be updated after committing specific epoch + TYtVersionedConfiguration::TPtr Configuration; + TYtTablesData::TPtr TablesData = MakeIntrusive<TYtTablesData>(); + THashMap<std::pair<TString, TString>, TString> AnonymousLabels; // cluster + label -> name + std::unordered_map<ui64, TString> NodeHash; // unique id -> hash + THashMap<ui32, TOperationStatistics> Statistics; // public id -> stat + THashMap<TString, TOperationStatistics> HybridStatistics; // subfolder -> stat + THashMap<TString, THashMap<TString, TOperationStatistics>> HybridOpStatistics; // operation name -> subfolder -> stat + TMutex StatisticsMutex; + THashSet<std::pair<TString, TString>> Checkpoints; // Set of checkpoint tables + THolder<IDqIntegration> DqIntegration_; + ui32 NextEpochId = 1; + bool OnlyNativeExecution = false; + bool PassiveExecution = false; + TDuration TimeSpentInHybrid; + NMonotonic::TMonotonic HybridStartTime; + std::unordered_set<ui32> HybridInFlightOprations; + THashMap<ui64, TWalkFoldersImpl> WalkFoldersState; + ui32 PlanLimits = 10; + i32 FlowDependsOnId = 0; + IOptimizerFactory::TPtr OptimizerFactory_; + IDqHelper::TPtr DqHelper; +private: + std::unordered_map<ui64, TYtVersionedConfiguration::TState> ConfigurationEvalStates_; + std::unordered_map<ui64, ui32> EpochEvalStates_; +}; + + +class TYtGatewayConfig; +std::pair<TIntrusivePtr<TYtState>, TStatWriter> CreateYtNativeState(IYtGateway::TPtr gateway, const TString& userName, const TString& sessionId, + const TYtGatewayConfig* ytGatewayConfig, TIntrusivePtr<TTypeAnnotationContext> typeCtx, + const IOptimizerFactory::TPtr& optFactory, const IDqHelper::TPtr& helper); +TIntrusivePtr<IDataProvider> CreateYtDataSource(TYtState::TPtr state); +TIntrusivePtr<IDataProvider> CreateYtDataSink(TYtState::TPtr state); + +TDataProviderInitializer GetYtNativeDataProviderInitializer(IYtGateway::TPtr gateway, IOptimizerFactory::TPtr optFactory, IDqHelper::TPtr helper, ui32 planLimits = 10); + +const THashSet<TStringBuf>& YtDataSourceFunctions(); +const THashSet<TStringBuf>& YtDataSinkFunctions(); + +} // NYql diff --git a/yt/yql/providers/yt/provider/yql_yt_provider_context.cpp b/yt/yql/providers/yt/provider/yql_yt_provider_context.cpp new file mode 100644 index 0000000000..3c51fc5a72 --- /dev/null +++ b/yt/yql/providers/yt/provider/yql_yt_provider_context.cpp @@ -0,0 +1,214 @@ + +#include <yt/yql/providers/yt/provider/yql_yt_provider_context.h> + +#include <yql/essentials/utils/log/log.h> +#include <yql/essentials/utils/yql_panic.h> + +#include <util/generic/bitmap.h> +#include <util/generic/hash_set.h> +#include <util/generic/maybe.h> + +namespace NYql { + +TMaybe<double> TYtProviderContext::ColumnNumUniqueValues(const TDynBitMap& relMap, const NDq::TJoinColumn& joinColumn) const { + auto entry = ColumnIndex_.find(joinColumn.AttributeName); + if (entry == ColumnIndex_.end()) { + return Nothing(); + } + for (const auto& relColEntry : entry->second) { + int relIndex = relColEntry.first; + int colIndex = relColEntry.second; + if (relMap[relIndex] && RelInfo_[relIndex].Label == joinColumn.RelName && + RelInfo_[relIndex].ColumnInfo[colIndex].EstimatedUniqueCount) { + return RelInfo_[relIndex].ColumnInfo[colIndex].EstimatedUniqueCount; + } + } + return Nothing(); +} + +TString TYtProviderContext::DebugStatistic(const TVector<NDq::TJoinColumn>& columns, const TOptimizerStatistics& stat) const { + std::stringstream ss; + const auto* specific = static_cast<TYtProviderStatistic*>(stat.Specific.get()); + ss << "Rels: ["; + int i = 0; + for (size_t pos = specific->RelBitmap.FirstNonZeroBit(); pos != specific->RelBitmap.Size(); pos = specific->RelBitmap.NextNonZeroBit(pos)) { + if (i > 0) { + ss << ", "; + } + ss << RelInfo_[pos].Label; + ++i; + } + ss << "]\n"; + ss << "Columns = ["; + i = 0; + for (const auto& joinColumn : columns) { + if (i > 0) { + ss << ", "; + } + ss << joinColumn.AttributeName; + auto numUniques = ColumnNumUniqueValues(specific->RelBitmap, joinColumn); + if (numUniques) { + ss << " (" << *numUniques << " uniques)"; + } else { + ss << " (? uniques)"; + } + ++i; + } + ss << "]\n"; + ss << "SortColumns = ["; + for (int i = 0; i < std::ssize(specific->SortColumns); ++i) { + if (i > 0) { + ss << ", "; + } + ss << specific->SortColumns[i]; + } + ss << "]\n"; + ss << "Ncols = " << stat.Ncols << ", ByteSize = " << stat.ByteSize << ", Nrows = " + << stat.Nrows << ", Cost = " << stat.Cost << "\n"; + return ss.str(); +} + +TMaybe<double> TYtProviderContext::FindMaxUniqueVals(const TYtProviderStatistic& specific, const TVector<NDq::TJoinColumn>& columns) const { + TMaybe<double> result; + for (const auto& joinColumn : columns) { + auto val = ColumnNumUniqueValues(specific.RelBitmap, joinColumn); + if (val && (!result || *val > *result)) { + result = val; + } + } + return result; +} + +double ComputeCardinality(TMaybe<double> maxUniques, double leftCardinality, double rightCardinality) { + if (!maxUniques) { + double result = 0.2 * leftCardinality * rightCardinality; + return result; + } + if (*maxUniques == 0.0) { + return 0.0; + } + double result = leftCardinality / *maxUniques * rightCardinality; + return result; +} + +TYtProviderContext::TYtProviderContext(TJoinAlgoLimits limits, TVector<TYtProviderRelInfo> relInfo) + : Limits_(limits) + , RelInfo_(std::move(relInfo)) { + for (int relIndex = 0; relIndex < std::ssize(RelInfo_); ++relIndex) { + const auto& rel = RelInfo_[relIndex]; + for (int colIndex = 0; colIndex < std::ssize(rel.ColumnInfo); ++colIndex) { + const auto& column = rel.ColumnInfo[colIndex].ColumnName; + ColumnIndex_[column].insert(std::make_pair(relIndex, colIndex)); + } + } +} + +bool TYtProviderContext::IsMapJoinApplicable(const TOptimizerStatistics& stat) const { + const TYtProviderStatistic* specific = static_cast<const TYtProviderStatistic*>(stat.Specific.get()); + return stat.Type == EStatisticsType::BaseTable && specific->SizeInfo.MapJoinMemSize && + *specific->SizeInfo.MapJoinMemSize < Limits_.MapJoinMemLimit; +} + +bool TYtProviderContext::IsLookupJoinApplicable(const TOptimizerStatistics& table, const TOptimizerStatistics& lookupTable, const TVector<NDq::TJoinColumn>& tableJoinKeys) const { + const TYtProviderStatistic* tableSpecific = static_cast<const TYtProviderStatistic*>(table.Specific.get()); + const TYtProviderStatistic* lookupTableSpecific = static_cast<const TYtProviderStatistic*>(lookupTable.Specific.get()); + if (table.Type != EStatisticsType::BaseTable || tableSpecific->SortColumns.empty()) { + return false; + } + const auto& relName = RelInfo_[tableSpecific->RelBitmap.FirstNonZeroBit()].Label; + NDq::TJoinColumn sortColumn(relName, tableSpecific->SortColumns[0]); + if (std::find(tableJoinKeys.begin(), tableJoinKeys.end(), sortColumn) == tableJoinKeys.end()) { + return false; + } + if (lookupTable.Nrows > Limits_.LookupJoinMaxRows || + !lookupTableSpecific->SizeInfo.LookupJoinMemSize || *lookupTableSpecific->SizeInfo.LookupJoinMemSize >= Limits_.LookupJoinMemLimit) { + return false; + } + + return true; +} + +bool TYtProviderContext::IsJoinApplicable( + const std::shared_ptr<IBaseOptimizerNode>& left, + const std::shared_ptr<IBaseOptimizerNode>& right, + const TVector<NDq::TJoinColumn>& leftJoinKeys, + const TVector<NDq::TJoinColumn>& /*rightJoinKeys*/, + EJoinAlgoType joinAlgo, + EJoinKind /*joinKind*/) { + if (joinAlgo == EJoinAlgoType::LookupJoin) { + return !leftJoinKeys.empty() && IsLookupJoinApplicable(left->Stats, right->Stats, leftJoinKeys); + } + if (joinAlgo == EJoinAlgoType::MapJoin) { + return IsMapJoinApplicable(right->Stats); + } + return joinAlgo == EJoinAlgoType::MergeJoin; +} + +TOptimizerStatistics TYtProviderContext::ComputeJoinStats( + const TOptimizerStatistics& leftStats, + const TOptimizerStatistics& rightStats, + const TVector<NDq::TJoinColumn>& leftJoinKeys, + const TVector<NDq::TJoinColumn>& rightJoinKeys, + EJoinAlgoType joinAlgo, + EJoinKind /*joinKind*/, + TCardinalityHints::TCardinalityHint* /*maybeHint*/) const { + + const TYtProviderStatistic* leftSpecific = static_cast<const TYtProviderStatistic*>(leftStats.Specific.get()); + const TYtProviderStatistic* rightSpecific = static_cast<const TYtProviderStatistic*>(rightStats.Specific.get()); + + TMaybe<double> maxUniques; + auto leftMaxUniques = FindMaxUniqueVals(*leftSpecific, leftJoinKeys); + if (leftMaxUniques && (!maxUniques || *maxUniques < *leftMaxUniques)) { + maxUniques = *leftMaxUniques; + } + auto rightMaxUniques = FindMaxUniqueVals(*rightSpecific, rightJoinKeys); + if (rightMaxUniques && (!maxUniques || *maxUniques < *rightMaxUniques)) { + maxUniques = *rightMaxUniques; + } + + auto resultSpecific = std::make_unique<TYtProviderStatistic>(); + + resultSpecific->RelBitmap = leftSpecific->RelBitmap | rightSpecific->RelBitmap; + resultSpecific->JoinAlgo = joinAlgo; + + double outputCardinality = ComputeCardinality(maxUniques, leftStats.Nrows, rightStats.Nrows); + double leftCardinalityFactor = leftStats.Nrows != 0.0 ? outputCardinality / leftStats.Nrows : 0.0; + double rightCardinalityFactor = rightStats.Nrows != 0.0 ? outputCardinality / rightStats.Nrows : 0.0; + + double outputByteSize = leftCardinalityFactor * leftStats.ByteSize + rightCardinalityFactor * rightStats.ByteSize; + + auto leftReadBytes = leftStats.ByteSize; + double rightReadBytes = rightStats.ByteSize; + + if (joinAlgo == EJoinAlgoType::LookupJoin) { + auto leftJoinUniques = FindMaxUniqueVals(*leftSpecific, TVector<NDq::TJoinColumn>{leftJoinKeys[0]}); + auto rightJoinUniques = FindMaxUniqueVals(*rightSpecific, TVector<NDq::TJoinColumn>{rightJoinKeys[0]}); + if (leftJoinUniques && rightJoinUniques && *rightJoinUniques < *leftJoinUniques) { + leftReadBytes *= *rightJoinUniques / *leftJoinUniques; + } + } + + if (joinAlgo == EJoinAlgoType::MapJoin && + (leftSpecific->JoinAlgo == EJoinAlgoType::MapJoin || leftSpecific->JoinAlgo == EJoinAlgoType::LookupJoin) && + rightStats.Type == EStatisticsType::BaseTable) { + // Optimistically assume that consecutive MapJoins are fused together. + leftReadBytes = 0; + } + + double outputCost = + leftStats.Cost + rightStats.Cost + + leftReadBytes + rightReadBytes + + outputByteSize; + + TOptimizerStatistics result( + EStatisticsType::ManyManyJoin, + outputCardinality, + leftStats.Ncols + rightStats.Ncols, + outputByteSize, + outputCost, + {}, {}, EStorageType::NA, std::move(resultSpecific)); + + return result; +} + +} // namespace NYql diff --git a/yt/yql/providers/yt/provider/yql_yt_provider_context.h b/yt/yql/providers/yt/provider/yql_yt_provider_context.h new file mode 100644 index 0000000000..fcf1d176b3 --- /dev/null +++ b/yt/yql/providers/yt/provider/yql_yt_provider_context.h @@ -0,0 +1,81 @@ +#pragma once + +#include <util/generic/bitmap.h> +#include <util/generic/maybe.h> + +#include <yql/essentials/core/cbo/cbo_optimizer_new.h> + +namespace NYql { + +struct TYtColumnStatistic { + TString ColumnName; + TMaybe<int64_t> EstimatedUniqueCount; + TMaybe<int64_t> DataWeight; +}; + +struct TRelSizeInfo { + TMaybe<ui64> MapJoinMemSize; + TMaybe<ui64> LookupJoinMemSize; +}; + +struct TYtProviderStatistic : public IProviderStatistics { + TDynBitMap RelBitmap; + TVector<TString> SortColumns; + TMaybe<EJoinAlgoType> JoinAlgo; + TRelSizeInfo SizeInfo; +}; + +struct TYtProviderRelInfo { + TString Label; + TVector<TYtColumnStatistic> ColumnInfo; + TVector<TString> SortColumns; +}; + +class TYtProviderContext : public TBaseProviderContext { +public: + struct TJoinAlgoLimits { + ui64 MapJoinMemLimit; + ui64 LookupJoinMemLimit; + ui64 LookupJoinMaxRows; + }; + + TYtProviderContext(TJoinAlgoLimits limits, TVector<TYtProviderRelInfo> relInfo); + + virtual TOptimizerStatistics ComputeJoinStats( + const TOptimizerStatistics& leftStats, + const TOptimizerStatistics& rightStats, + const TVector<NDq::TJoinColumn>& leftJoinKeys, + const TVector<NDq::TJoinColumn>& rightJoinKeys, + EJoinAlgoType joinAlgo, + EJoinKind joinKind, + TCardinalityHints::TCardinalityHint* maybeHint = nullptr) const override; + + bool IsJoinApplicable( + const std::shared_ptr<IBaseOptimizerNode>& leftStats, + const std::shared_ptr<IBaseOptimizerNode>& rightStats, + const TVector<NDq::TJoinColumn>& leftJoinKeys, + const TVector<NDq::TJoinColumn>& rightJoinKeys, + EJoinAlgoType joinAlgo, + EJoinKind joinKind) override; + +private: + bool IsLookupJoinApplicable(const TOptimizerStatistics& table, const TOptimizerStatistics& lookupTable, const TVector<NDq::TJoinColumn>& tableJoinKeys) const; + + bool IsMapJoinApplicable(const TOptimizerStatistics& table) const; + + TDynBitMap ExtractColumnsBitmap(const TDynBitMap& columnBitmap, const TVector<TString>& columns) const; + + TVector<TYtColumnStatistic> MergeColumnStatistics(const TYtProviderStatistic& leftSpecific, const TYtProviderStatistic& rightSpecific, const TDynBitMap& outputBitmap) const; + + TMaybe<double> FindMaxUniqueVals(const TYtProviderStatistic& specific, const TVector<NDq::TJoinColumn>& columns) const; + + TMaybe<double> ColumnNumUniqueValues(const TDynBitMap& relMap, const NDq::TJoinColumn& columnName) const; + + TString DebugStatistic(const TVector<NDq::TJoinColumn>& columns, const TOptimizerStatistics& stat) const; + + const TJoinAlgoLimits Limits_; + TVector<TYtProviderRelInfo> RelInfo_; + THashMap<TString, THashSet<std::pair<int, int>>> ColumnIndex_; +}; + +} // namespace NYql diff --git a/yt/yql/providers/yt/provider/yql_yt_provider_impl.cpp b/yt/yql/providers/yt/provider/yql_yt_provider_impl.cpp new file mode 100644 index 0000000000..c3a60b42e4 --- /dev/null +++ b/yt/yql/providers/yt/provider/yql_yt_provider_impl.cpp @@ -0,0 +1,90 @@ +#include "yql_yt_provider_impl.h" +#include "yql_yt_op_settings.h" +#include "yql_yt_table.h" +#include "yql_yt_helpers.h" + +#include <yt/yql/providers/yt/expr_nodes/yql_yt_expr_nodes.h> +#include <yql/essentials/core/yql_expr_optimize.h> + +#include <util/string/builder.h> + +namespace NYql { + +using namespace NNodes; + +void ScanPlanDependencies(const TExprNode::TPtr& input, TExprNode::TListType& children) { + VisitExpr(input, [&children](const TExprNode::TPtr& node) { + if (TMaybeNode<TYtReadTable>(node)) { + children.push_back(node); + return false; + } + if (const auto maybeOutput = TMaybeNode<TYtOutput>(node)) { + const auto& output = maybeOutput.Cast(); + if (const auto& maybeTryFirst = output.Operation().Maybe<TYtTryFirst>()) { + const auto& tryFirst = maybeTryFirst.Cast(); + children.emplace_back(tryFirst.Second().Ptr()); + children.emplace_back(tryFirst.First().Ptr()); + } else + children.emplace_back(GetOutputOp(output).Ptr()); + return false; + } + if (node->IsCallable("DqCnResult")) { // For TYtDqProcessWrite. + children.emplace_back(node->HeadPtr()); + return false; + } + return true; + }); +} + +void ScanForUsedOutputTables(const TExprNode& input, TVector<TString>& usedNodeIds) +{ + VisitExpr(input, [&usedNodeIds](const TExprNode& node) { + if (auto maybeYtOutput = TMaybeNode<TYtOutput>(&node)) { + + auto ytOutput = maybeYtOutput.Cast(); + + TString cluster = TString{GetOutputOp(ytOutput).DataSink().Cluster().Value()}; + TString table = TString{GetOutTable(ytOutput).Cast<TYtOutTable>().Name().Value()}; + + if (!cluster.empty() && !table.empty()) { + usedNodeIds.push_back(MakeUsedNodeId(cluster, table)); + } + return false; + } + return true; + }); + +} + +TString MakeUsedNodeId(const TString& cluster, const TString& table) +{ + YQL_ENSURE(!cluster.empty()); + YQL_ENSURE(!table.empty()); + + return cluster + "." + table; +} + +TString MakeTableDisplayName(NNodes::TExprBase table, bool isOutput) { + TStringBuilder name; + if (table.Maybe<TYtTable>()) { + auto ytTable = table.Cast<TYtTable>(); + name << ytTable.Cluster().Value() << "."; + if (NYql::HasSetting(ytTable.Settings().Ref(), EYtSettingType::Anonymous)) { + name << ytTable.Name().Value(); + } + else { + name << '`' << ytTable.Name().Value() << '`'; + } + auto epoch = isOutput ? ytTable.CommitEpoch() : ytTable.Epoch(); + if (auto epochVal = TEpochInfo::Parse(epoch.Ref()).GetOrElse(0)) { + name << " #" << epochVal; + } + } + else { + name << "(tmp)"; + } + return name; +} + + +} // NYql diff --git a/yt/yql/providers/yt/provider/yql_yt_provider_impl.h b/yt/yql/providers/yt/provider/yql_yt_provider_impl.h new file mode 100644 index 0000000000..512a20f9fb --- /dev/null +++ b/yt/yql/providers/yt/provider/yql_yt_provider_impl.h @@ -0,0 +1,47 @@ +#pragma once + +#include "yql_yt_provider.h" + +#include <yql/essentials/providers/common/transform/yql_visit.h> +#include <yql/essentials/providers/common/transform/yql_exec.h> +#include <yql/essentials/core/yql_graph_transformer.h> +#include <yql/essentials/ast/yql_expr.h> +#include <yql/essentials/core/expr_nodes_gen/yql_expr_nodes_gen.h> +#include <yt/yql/providers/yt/expr_nodes/yql_yt_expr_nodes.h> + +#include <util/generic/ptr.h> + +namespace NYql { + +THolder<IGraphTransformer> CreateYtIODiscoveryTransformer(TYtState::TPtr state); +THolder<IGraphTransformer> CreateYtEpochTransformer(TYtState::TPtr state); +THolder<IGraphTransformer> CreateYtIntentDeterminationTransformer(TYtState::TPtr state); +THolder<IGraphTransformer> CreateYtLoadTableMetadataTransformer(TYtState::TPtr state); +THolder<IGraphTransformer> CreateYtLoadColumnarStatsTransformer(TYtState::TPtr state); + +THolder<TVisitorTransformerBase> CreateYtDataSourceTypeAnnotationTransformer(TYtState::TPtr state); +THolder<IGraphTransformer> CreateYtDataSourceConstraintTransformer(TYtState::TPtr state); +THolder<TExecTransformerBase> CreateYtDataSourceExecTransformer(TYtState::TPtr state); + +THolder<TVisitorTransformerBase> CreateYtDataSinkTypeAnnotationTransformer(TYtState::TPtr state); +THolder<IGraphTransformer> CreateYtDataSinkConstraintTransformer(TYtState::TPtr state, bool subGraph); +THolder<TExecTransformerBase> CreateYtDataSinkExecTransformer(TYtState::TPtr state); +THolder<IGraphTransformer> CreateYtDataSinkTrackableNodesCleanupTransformer(TYtState::TPtr state); +THolder<IGraphTransformer> CreateYtDataSinkFinalizingTransformer(TYtState::TPtr state); + +THolder<IGraphTransformer> CreateYtLogicalOptProposalTransformer(TYtState::TPtr state); +THolder<IGraphTransformer> CreateYtPhysicalOptProposalTransformer(TYtState::TPtr state); +THolder<IGraphTransformer> CreateYtPhysicalFinalizingTransformer(TYtState::TPtr state); +THolder<IGraphTransformer> CreateYtPeepholeTransformer(TYtState::TPtr state, const THashMap<TString, TString>& settings); +THolder<IGraphTransformer> CreateYtWideFlowTransformer(TYtState::TPtr state); +THolder<IGraphTransformer> CreateYtDqHybridTransformer(TYtState::TPtr state, THolder<IGraphTransformer>&& finalizer); +THolder<IGraphTransformer> CreateYtBlockInputFilterTransformer(TYtState::TPtr state, THolder<IGraphTransformer>&& finalizer); +THolder<IGraphTransformer> CreateYtBlockInputTransformer(TYtState::TPtr state); + +void ScanPlanDependencies(const TExprNode::TPtr& input, TExprNode::TListType& children); +TString MakeTableDisplayName(NNodes::TExprBase table, bool isOutput); + +void ScanForUsedOutputTables(const TExprNode& input, TVector<TString>& usedNodeIds); +TString MakeUsedNodeId(const TString& cluster, const TString& table); + +} // NYql diff --git a/yt/yql/providers/yt/provider/yql_yt_table.cpp b/yt/yql/providers/yt/provider/yql_yt_table.cpp new file mode 100644 index 0000000000..d793aee0a1 --- /dev/null +++ b/yt/yql/providers/yt/provider/yql_yt_table.cpp @@ -0,0 +1,3083 @@ +#include "yql_yt_table.h" +#include "yql_yt_key.h" +#include "yql_yt_helpers.h" +#include "yql_yt_op_settings.h" + +#include <yql/essentials/core/expr_nodes/yql_expr_nodes.h> +#include <yql/essentials/core/yql_expr_type_annotation.h> +#include <yt/yql/providers/yt/expr_nodes/yql_yt_expr_nodes.h> +#include <yt/yql/providers/yt/common/yql_names.h> +#include <yt/yql/providers/yt/gateway/lib/yt_helpers.h> +#include <yql/essentials/utils/log/log.h> +#include <yql/essentials/public/udf/tz/udf_tz.h> +#include <yql/essentials/public/decimal/yql_decimal.h> +#include <yql/essentials/public/decimal/yql_decimal_serialize.h> +#include <yql/essentials/minikql/mkql_type_ops.h> +#include <yql/essentials/utils/utf8.h> + +#include <library/cpp/yson/node/node_io.h> +#include <yt/cpp/mapreduce/common/helpers.h> +#include <yt/cpp/mapreduce/interface/serialize.h> + +#include <util/stream/output.h> +#include <util/stream/str.h> +#include <util/string/builder.h> +#include <util/string/cast.h> +#include <util/string/split.h> +#include <util/generic/overloaded.h> +#include <util/generic/strbuf.h> +#include <util/generic/hash_set.h> +#include <util/generic/xrange.h> +#include <util/generic/singleton.h> +#include <util/generic/utility.h> + +#include <limits> + +namespace NYql { + +using namespace NNodes; +using namespace NKikimr; +using namespace NKikimr::NUdf; +using namespace std::string_view_literals; + +class TExprDataToYtNodeConverter { +public: + using TConverter = std::function<NYT::TNode(const TExprNode&)>; + + TExprDataToYtNodeConverter() { + Converters.emplace(TCoUint8::CallableName(), [](const TExprNode& node) { + return NYT::TNode((ui64)NYql::FromString<ui8>(*node.Child(0), EDataSlot::Uint8)); + }); + Converters.emplace(TCoInt8::CallableName(), [](const TExprNode& node) { + return NYT::TNode((i64)NYql::FromString<i8>(*node.Child(0), EDataSlot::Int8)); + }); + Converters.emplace(TCoUint16::CallableName(), [](const TExprNode& node) { + return NYT::TNode((ui64)NYql::FromString<ui16>(*node.Child(0), EDataSlot::Uint16)); + }); + Converters.emplace(TCoInt16::CallableName(), [](const TExprNode& node) { + return NYT::TNode((i64)NYql::FromString<i16>(*node.Child(0), EDataSlot::Int16)); + }); + Converters.emplace(TCoInt32::CallableName(), [](const TExprNode& node) { + return NYT::TNode((i64)NYql::FromString<i32>(*node.Child(0), EDataSlot::Int32)); + }); + Converters.emplace(TCoUint32::CallableName(), [](const TExprNode& node) { + return NYT::TNode((ui64)NYql::FromString<ui32>(*node.Child(0), EDataSlot::Uint32)); + }); + Converters.emplace(TCoInt64::CallableName(), [](const TExprNode& node) { + return NYT::TNode(NYql::FromString<i64>(*node.Child(0), EDataSlot::Int64)); + }); + Converters.emplace(TCoUint64::CallableName(), [](const TExprNode& node) { + return NYT::TNode(NYql::FromString<ui64>(*node.Child(0), EDataSlot::Uint64)); + }); + Converters.emplace(TCoString::CallableName(), [](const TExprNode& node) { + return NYT::TNode(node.Child(0)->Content()); + }); + Converters.emplace(TCoUtf8::CallableName(), [](const TExprNode& node) { + return NYT::TNode(node.Child(0)->Content()); + }); + Converters.emplace(TCoJson::CallableName(), [](const TExprNode& node) { + return NYT::TNode(node.Child(0)->Content()); + }); + Converters.emplace(TCoYson::CallableName(), [](const TExprNode& node) { + return NYT::NodeFromYsonString(TString{node.Child(0)->Content()}); + }); + Converters.emplace(TCoDecimal::CallableName(), [](const TExprNode& node) { + char data[sizeof(NDecimal::TInt128)]; + const ui32 size = NDecimal::Serialize( + NDecimal::FromString(node.Child(0)->Content(), + ::FromString<ui8>(node.Child(1)->Content()), + ::FromString<ui8>(node.Child(2)->Content())), + data); + return NYT::TNode(TStringBuf(data, size)); + }); + Converters.emplace(TCoBool::CallableName(), [](const TExprNode& node) { + return NYT::TNode(NYql::FromString<bool>(*node.Child(0), EDataSlot::Bool)); + }); + Converters.emplace(TCoFloat::CallableName(), [](const TExprNode& node) { + return NYT::TNode((double)NYql::FromString<float>(*node.Child(0), EDataSlot::Float)); + }); + Converters.emplace(TCoDouble::CallableName(), [](const TExprNode& node) { + return NYT::TNode(NYql::FromString<double>(*node.Child(0), EDataSlot::Double)); + }); + Converters.emplace(TCoNull::CallableName(), [](const TExprNode& /*node*/) { + return NYT::TNode::CreateEntity(); + }); + Converters.emplace(TCoNothing::CallableName(), [](const TExprNode& /*node*/) { + return NYT::TNode::CreateEntity(); + }); + Converters.emplace(TCoDate::CallableName(), [](const TExprNode& node) { + return NYT::TNode((ui64)NYql::FromString<ui16>(*node.Child(0), EDataSlot::Date)); + }); + Converters.emplace(TCoDatetime::CallableName(), [](const TExprNode& node) { + return NYT::TNode((ui64)NYql::FromString<ui32>(*node.Child(0), EDataSlot::Datetime)); + }); + Converters.emplace(TCoTimestamp::CallableName(), [](const TExprNode& node) { + return NYT::TNode(NYql::FromString<ui64>(*node.Child(0), EDataSlot::Timestamp)); + }); + Converters.emplace(TCoInterval::CallableName(), [](const TExprNode& node) { + return NYT::TNode(NYql::FromString<i64>(*node.Child(0), EDataSlot::Interval)); + }); + Converters.emplace(TCoDate32::CallableName(), [](const TExprNode& node) { + return NYT::TNode((i64)NYql::FromString<i32>(*node.Child(0), EDataSlot::Date32)); + }); + Converters.emplace(TCoDatetime64::CallableName(), [](const TExprNode& node) { + return NYT::TNode(NYql::FromString<i64>(*node.Child(0), EDataSlot::Datetime64)); + }); + Converters.emplace(TCoTimestamp64::CallableName(), [](const TExprNode& node) { + return NYT::TNode(NYql::FromString<i64>(*node.Child(0), EDataSlot::Timestamp64)); + }); + Converters.emplace(TCoInterval64::CallableName(), [](const TExprNode& node) { + return NYT::TNode(NYql::FromString<i64>(*node.Child(0), EDataSlot::Interval64)); + }); + Converters.emplace(TCoTzDate::CallableName(), [](const TExprNode& node) { + TStringBuf tzName = node.Child(0)->Content(); + TStringBuf valueStr; + GetNext(tzName, ',', valueStr); + TStringStream out; + NMiniKQL::SerializeTzDate(::FromString<ui16>(valueStr), NMiniKQL::GetTimezoneId(tzName), out); + return NYT::TNode(out.Str()); + }); + Converters.emplace(TCoTzDatetime::CallableName(), [](const TExprNode& node) { + TStringBuf tzName = node.Child(0)->Content(); + TStringBuf valueStr; + GetNext(tzName, ',', valueStr); + TStringStream out; + NMiniKQL::SerializeTzDatetime(::FromString<ui32>(valueStr), NMiniKQL::GetTimezoneId(tzName), out); + return NYT::TNode(out.Str()); + }); + Converters.emplace(TCoTzTimestamp::CallableName(), [](const TExprNode& node) { + TStringBuf tzName = node.Child(0)->Content(); + TStringBuf valueStr; + GetNext(tzName, ',', valueStr); + TStringStream out; + NMiniKQL::SerializeTzTimestamp(::FromString<ui64>(valueStr), NMiniKQL::GetTimezoneId(tzName), out); + return NYT::TNode(out.Str()); + }); + Converters.emplace(TCoTzDate32::CallableName(), [](const TExprNode& node) { + TStringBuf tzName = node.Child(0)->Content(); + TStringBuf valueStr; + GetNext(tzName, ',', valueStr); + TStringStream out; + NMiniKQL::SerializeTzDate32(::FromString<i32>(valueStr), NMiniKQL::GetTimezoneId(tzName), out); + return NYT::TNode(out.Str()); + }); + Converters.emplace(TCoTzDatetime64::CallableName(), [](const TExprNode& node) { + TStringBuf tzName = node.Child(0)->Content(); + TStringBuf valueStr; + GetNext(tzName, ',', valueStr); + TStringStream out; + NMiniKQL::SerializeTzDatetime64(::FromString<i64>(valueStr), NMiniKQL::GetTimezoneId(tzName), out); + return NYT::TNode(out.Str()); + }); + Converters.emplace(TCoTzTimestamp64::CallableName(), [](const TExprNode& node) { + TStringBuf tzName = node.Child(0)->Content(); + TStringBuf valueStr; + GetNext(tzName, ',', valueStr); + TStringStream out; + NMiniKQL::SerializeTzTimestamp64(::FromString<i64>(valueStr), NMiniKQL::GetTimezoneId(tzName), out); + return NYT::TNode(out.Str()); + }); + Converters.emplace(TCoUuid::CallableName(), [](const TExprNode& node) { + return NYT::TNode(node.Child(0)->Content()); + }); + } + + NYT::TNode Convert(const TExprNode& node) const { + if (auto p = Converters.FindPtr(node.Content())) { + return (*p)(node); + } + return NYT::TNode(); + } +private: + THashMap<TStringBuf, TConverter> Converters; +}; + +// Converts ExprNode representation to YT table format +NYT::TNode ExprNodeToYtNode(const TExprNode& node) { + return Default<TExprDataToYtNodeConverter>().Convert(node.IsCallable("Just") ? node.Head() : node); +} + +TExprNode::TPtr YtNodeToExprNode(const NYT::TNode& node, TExprContext& ctx, TPositionHandle pos) { + switch (node.GetType()) { + case NYT::TNode::String: + return Build<TCoString>(ctx, pos) + .Literal() + .Value(node.AsString()) + .Build() + .Done().Ptr(); + case NYT::TNode::Int64: + return Build<TCoInt64>(ctx, pos) + .Literal() + .Value(ToString(node.AsInt64())) + .Build() + .Done().Ptr(); + case NYT::TNode::Uint64: + return Build<TCoUint64>(ctx, pos) + .Literal() + .Value(ToString(node.AsUint64())) + .Build() + .Done().Ptr(); + case NYT::TNode::Double: + return Build<TCoDouble>(ctx, pos) + .Literal() + .Value(ToString(node.AsDouble())) + .Build() + .Done().Ptr(); + case NYT::TNode::Bool: + return Build<TCoBool>(ctx, pos) + .Literal() + .Value(ToString(node.AsBool())) + .Build() + .Done().Ptr(); + case NYT::TNode::Null: + return Build<TCoNull>(ctx, pos) + .Done().Ptr(); + case NYT::TNode::Undefined: + ythrow yexception() << "Cannot convert UNDEFINED TNode to expr node"; + default: + return ctx.Builder(pos).Callable("Yson").Atom(0, NYT::NodeToYsonString(node)).Seal().Build(); + } +} + +///////////////////////////////////////////////////////////////////////////////////////////////////////// + +bool TYtTableStatInfo::Validate(const TExprNode& node, TExprContext& ctx) { + if (!EnsureCallable(node, ctx)) { + return false; + } + if (!node.IsCallable(TYtStat::CallableName())) { + ctx.AddError(TIssue(ctx.GetPosition(node.Pos()), TStringBuilder() << "Expected " << TYtStat::CallableName() + << " callable, but got " << node.Content())); + return false; + } + + for (auto& child: node.Children()) { + if (!EnsureTupleSize(*child, 2, ctx)) { + return false; + } + const TExprNode* name = child->Child(0); + const TExprNode* value = child->Child(1); + if (!EnsureAtom(*name, ctx)) { + return false; + } + +#define VALIDATE_FIELD(field) \ + if (name->Content() == TStringBuf(#field)) { \ + if (!EnsureAtom(*value, ctx)) { \ + return false; \ + } \ + decltype(TYtTableStatInfo::field) _##field; \ + if (!TryFromString(value->Content(), _##field)) { \ + ctx.AddError(TIssue(ctx.GetPosition(value->Pos()), \ + TStringBuilder() << "Bad value of '" #field "' attribute: " \ + << value->Content())); \ + return false; \ + } \ + } + + if (name->Content() == "Id") { + if (!EnsureAtom(*value, ctx)) { + return false; + } + if (value->Content().empty()) { + ctx.AddError(TIssue(ctx.GetPosition(value->Pos()), + TStringBuilder() << "Empty value of 'Id' attribute")); + return false; + } + } + else + VALIDATE_FIELD(RecordsCount) + else + VALIDATE_FIELD(DataSize) + else + VALIDATE_FIELD(ChunkCount) + else + VALIDATE_FIELD(ModifyTime) + else + VALIDATE_FIELD(Revision) + else if (name->Content() == "SecurityTags") { + if (!value->IsList()) { + ctx.AddError(TIssue(ctx.GetPosition(value->Pos()), + TStringBuilder() << "Expected list")); + return false; + } + for (const auto& tagAtom : value->Children()) { + if (!EnsureAtom(*tagAtom, ctx)) { + return false; + } + } + } else { + ctx.AddError(TIssue(ctx.GetPosition(child->Pos()), TStringBuilder() << "Unsupported table stat option: " << name->Content())); + return false; + } +#undef VALIDATE_FIELD + + } + return true; +} + +void TYtTableStatInfo::Parse(TExprBase node) { + *this = {}; + FromNode = node.Maybe<TExprBase>(); + for (auto child: node.Cast<TYtStat>()) { + auto setting = child.Cast<TCoNameValueTuple>(); +#define HANDLE_FIELD(field) \ + if (setting.Name().Value() == TStringBuf(#field)) { \ + field = FromString(setting.Value().Cast<TCoAtom>().Value()); \ + } + + if (setting.Name().Value() == "Id") { + Id = TString{setting.Value().Cast<TCoAtom>().Value()}; + } + else + HANDLE_FIELD(RecordsCount) + else + HANDLE_FIELD(DataSize) + else + HANDLE_FIELD(ChunkCount) + else + HANDLE_FIELD(ModifyTime) + else + HANDLE_FIELD(Revision) + else if (setting.Name().Value() == "SecurityTags") { + SecurityTags = {}; + for (const auto& tagAtom : setting.Value().Cast<TListBase<TCoAtom>>()) { + SecurityTags.emplace(tagAtom.Value()); + } + } else { + YQL_ENSURE(false, "Unexpected option " << setting.Name().Value()); + } +#undef HANDLE_FIELD + } +} + +TExprBase TYtTableStatInfo::ToExprNode(TExprContext& ctx, const TPositionHandle& pos) const { + auto statBuilder = Build<TYtStat>(ctx, pos); + +#define ADD_FIELD(field) \ + .Add() \ + .Name() \ + .Value(TStringBuf(#field), TNodeFlags::Default) \ + .Build() \ + .Value<TCoAtom>() \ + .Value(ToString(field)) \ + .Build() \ + .Build() + + statBuilder + ADD_FIELD(Id) + ADD_FIELD(RecordsCount) + ADD_FIELD(DataSize) + ADD_FIELD(ChunkCount) + ADD_FIELD(ModifyTime) + ADD_FIELD(Revision) + ; + +#undef ADD_FIELD + + if (!SecurityTags.empty()) { + statBuilder + .Add() + .Name() + .Value("SecurityTags") + .Build() + .Value(ToAtomList(SecurityTags, pos, ctx)) + .Build(); + } + + return statBuilder.Done(); +} + +///////////////////////////////////////////////////////////////////////////////////////////////////////// + +bool TYtTableMetaInfo::Validate(const TExprNode& node, TExprContext& ctx) { + if (!EnsureCallable(node, ctx)) { + return false; + } + if (!node.IsCallable(TYtMeta::CallableName())) { + ctx.AddError(TIssue(ctx.GetPosition(node.Pos()), TStringBuilder() << "Expected " << TYtMeta::CallableName() + << " callable, but got " << node.Content())); + return false; + } + + for (auto& child: node.Children()) { + if (!EnsureTupleSize(*child, 2, ctx)) { + return false; + } + const TExprNode* name = child->Child(0); + TExprNode* value = child->Child(1); + if (!EnsureAtom(*name, ctx)) { + return false; + } + +#define VALIDATE_FIELD(field) \ + if (name->Content() == TStringBuf(#field)) { \ + if (!EnsureAtom(*value, ctx)) { \ + return false; \ + } \ + decltype(TYtTableMetaInfo::field) _##field; \ + if (!TryFromString(value->Content(), _##field)) { \ + ctx.AddError(TIssue(ctx.GetPosition(value->Pos()), \ + TStringBuilder() << "Bad value of '" #field "' attribute: " \ + << value->Content())); \ + return false; \ + } \ + } + + if (name->Content() == TStringBuf("Attrs")) { + if (!EnsureTuple(*value, ctx)) { + return false; + } + for (auto& item: value->Children()) { + if (!EnsureTupleSize(*item, 2, ctx)) { + return false; + } + if (!EnsureAtom(*item->Child(0), ctx) || !EnsureAtom(*item->Child(1), ctx)) { + return false; + } + } + } + else + VALIDATE_FIELD(CanWrite) + else + VALIDATE_FIELD(DoesExist) + else + VALIDATE_FIELD(YqlCompatibleScheme) + else + VALIDATE_FIELD(InferredScheme) + else + VALIDATE_FIELD(IsDynamic) + else + VALIDATE_FIELD(SqlView) + else + VALIDATE_FIELD(SqlViewSyntaxVersion) + else { + ctx.AddError(TIssue(ctx.GetPosition(child->Pos()), TStringBuilder() << "Unsupported table meta option: " << name->Content())); + return false; + } +#undef VALIDATE_FIELD + + } + return true; +} + +void TYtTableMetaInfo::Parse(TExprBase node) { + *this = {}; + FromNode = node.Maybe<TExprBase>(); + for (auto child: node.Cast<TYtMeta>()) { + auto setting = child.Cast<TCoNameValueTuple>(); +#define HANDLE_FIELD(field) \ + if (setting.Name().Value() == TStringBuf(#field)) { \ + field = FromString(setting.Value().Cast<TCoAtom>().Value()); \ + } + + if (setting.Name().Value() == TStringBuf("Attrs")) { + for (auto item: setting.Value().Cast<TCoNameValueTupleList>()) { + Attrs.insert({TString{item.Name().Value()}, TString{item.Value().Cast<TCoAtom>().Value()}}); + } + } + else if (setting.Name().Value() == TStringBuf("SqlView")) { + SqlView = TString{setting.Value().Cast<TCoAtom>().Value()}; + } + else + HANDLE_FIELD(SqlViewSyntaxVersion) + else + HANDLE_FIELD(CanWrite) + else + HANDLE_FIELD(DoesExist) + else + HANDLE_FIELD(YqlCompatibleScheme) + else + HANDLE_FIELD(InferredScheme) + else + HANDLE_FIELD(IsDynamic) + else { + YQL_ENSURE(false, "Unexpected option " << setting.Name().Value()); + } +#undef HANDLE_FIELD + + } +} + +TExprBase TYtTableMetaInfo::ToExprNode(TExprContext& ctx, const TPositionHandle& pos) const { + auto metaBuilder = Build<TYtMeta>(ctx, pos); + +#define ADD_BOOL_FIELD(field) \ + .Add() \ + .Name() \ + .Value(TStringBuf(#field), TNodeFlags::Default) \ + .Build() \ + .Value<TCoAtom>() \ + .Value(field ? TStringBuf("1") : TStringBuf("0"), TNodeFlags::Default) \ + .Build() \ + .Build() + + metaBuilder + ADD_BOOL_FIELD(CanWrite) + ADD_BOOL_FIELD(DoesExist) + ADD_BOOL_FIELD(YqlCompatibleScheme) + ADD_BOOL_FIELD(InferredScheme) + ADD_BOOL_FIELD(IsDynamic) + ; + +#undef ADD_BOOL_FIELD + + if (SqlView) { + metaBuilder + .Add() + .Name() + .Value(TStringBuf("SqlView"), TNodeFlags::Default) + .Build() + .Value<TCoAtom>() + .Value(SqlView) + .Build() + .Build(); + + metaBuilder + .Add() + .Name() + .Value(TStringBuf("SqlViewSyntaxVersion"), TNodeFlags::Default) + .Build() + .Value<TCoAtom>() + .Value(ToString(SqlViewSyntaxVersion), TNodeFlags::Default) + .Build() + .Build(); + } + + if (!Attrs.empty()) { + auto attrsBuilder = Build<TCoNameValueTupleList>(ctx, pos); + for (auto& attr: Attrs) { + attrsBuilder.Add() + .Name() + .Value(attr.first) + .Build() + .Value<TCoAtom>() + .Value(attr.second) + .Build() + .Build(); + } + metaBuilder + .Add() + .Name() + .Value(TStringBuf("Attrs"), TNodeFlags::Default) + .Build() + .Value(attrsBuilder.Done()) + .Build(); + } + + return metaBuilder.Done(); +} + +///////////////////////////////////////////////////////////////////////////////////////////////////////// + +bool TEpochInfo::Validate(const TExprNode& node, TExprContext& ctx) { + if (!node.IsCallable(TEpoch::CallableName())) { + ctx.AddError(TIssue(ctx.GetPosition(node.Pos()), TStringBuilder() << "Expected " << TEpoch::CallableName())); + return false; + } + if (!EnsureArgsCount(node, 1, ctx)) { + return false; + } + + auto epochValue = node.Child(TEpoch::idx_Value); + if (!EnsureAtom(*epochValue, ctx)) { + return false; + } + + ui32 epoch = 0; + if (!TryFromString(epochValue->Content(), epoch)) { + ctx.AddError(TIssue(ctx.GetPosition(epochValue->Pos()), TStringBuilder() << "Bad epoch value: " << epochValue->Content())); + return false; + } + return true; +} + +TMaybe<ui32> TEpochInfo::Parse(const TExprNode& node) { + if (auto maybeEpoch = TMaybeNode<TEpoch>(&node)) { + return FromString<ui32>(maybeEpoch.Cast().Value().Value()); + } + return Nothing(); +} + +TExprBase TEpochInfo::ToExprNode(const TMaybe<ui32>& epoch, TExprContext& ctx, const TPositionHandle& pos) { + return epoch + ? Build<TEpoch>(ctx, pos).Value().Value(ToString(*epoch), TNodeFlags::Default).Build().Done().Cast<TExprBase>() + : Build<TCoVoid>(ctx, pos).Done().Cast<TExprBase>(); +} + +///////////////////////////////////////////////////////////////////////////////////////////////////////// + +TYtTableBaseInfo::TYtTableBaseInfo(const TYtKey& key, TStringBuf cluster) + : Name(key.GetPath()) + , Cluster(cluster) +{ +} + +NYT::TNode TYtTableBaseInfo::GetCodecSpecNode(const NCommon::TStructMemberMapper& mapper) const { + NYT::TNode res = NYT::TNode::CreateMap(); + if (RowSpec) { + RowSpec->FillCodecNode(res[YqlRowSpecAttribute], mapper); + } + if (Meta) { + if (auto p = Meta->Attrs.FindPtr(FORMAT_ATTR_NAME)) { + res[FORMAT_ATTR_NAME] = NYT::NodeFromYsonString(*p); + } + if (Meta->IsDynamic) { + res[YqlDynamicAttribute] = true; + } + } + return res; +} + +NYT::TNode TYtTableBaseInfo::GetAttrSpecNode(ui64 nativeTypeCompatibility, bool rowSpecCompactForm) const { + NYT::TNode res = NYT::TNode::CreateMap(); + if (RowSpec) { + RowSpec->FillAttrNode(res[YqlRowSpecAttribute], nativeTypeCompatibility, rowSpecCompactForm); + } + return res; +} + +TYtTableBaseInfo::TPtr TYtTableBaseInfo::Parse(TExprBase node) { + if (node.Maybe<TYtOutTable>()) { + return MakeIntrusive<TYtOutTableInfo>(node); + } else if (auto out = node.Maybe<TYtOutput>()) { + auto tableWithCluster = GetOutTableWithCluster(node); + auto res = MakeIntrusive<TYtOutTableInfo>(tableWithCluster.first); + res->Cluster = tableWithCluster.second; + res->IsUnordered = IsUnorderedOutput(out.Cast()); + return res; + } else if (node.Maybe<TYtTable>()) { + return MakeIntrusive<TYtTableInfo>(node); + } else { + ythrow yexception() << "Not a table node " << (node.Raw() ? TString{node.Ref().Content()}.Quote() : TStringBuf("\"null\"")); + } +} + +TYtTableMetaInfo::TPtr TYtTableBaseInfo::GetMeta(TExprBase node) { + TMaybeNode<TExprBase> meta; + if (auto outTable = node.Maybe<TYtOutTable>()) { + meta = outTable.Cast().Meta(); + } else if (node.Maybe<TYtOutput>()) { + meta = GetOutTable(node).Cast<TYtOutTable>().Meta(); + } else if (auto table = node.Maybe<TYtTable>()) { + meta = table.Cast().Meta(); + } else { + ythrow yexception() << "Not a table node " << (node.Raw() ? TString{node.Ref().Content()}.Quote() : TStringBuf("\"null\"")); + } + + if (meta.Maybe<TCoVoid>()) { + return {}; + } + return MakeIntrusive<TYtTableMetaInfo>(meta.Cast()); +} + +TYqlRowSpecInfo::TPtr TYtTableBaseInfo::GetRowSpec(TExprBase node) { + TMaybeNode<TExprBase> rowSpec; + if (auto outTable = node.Maybe<TYtOutTable>()) { + rowSpec = outTable.Cast().RowSpec(); + } else if (auto out = node.Maybe<TYtOutput>()) { + rowSpec = GetOutTable(node).Cast<TYtOutTable>().RowSpec(); + } else if (auto table = node.Maybe<TYtTable>()) { + rowSpec = table.Cast().RowSpec(); + } else { + ythrow yexception() << "Not a table node " << (node.Raw() ? TString{node.Ref().Content()}.Quote() : TStringBuf("\"null\"")); + } + + if (rowSpec.Maybe<TCoVoid>()) { + return {}; + } + return MakeIntrusive<TYqlRowSpecInfo>(rowSpec.Cast()); +} + +TYtTableStatInfo::TPtr TYtTableBaseInfo::GetStat(NNodes::TExprBase node) { + TExprNode::TPtr statNode; + if (node.Maybe<TYtOutTable>()) { + statNode = node.Cast<TYtOutTable>().Stat().Ptr(); + } else if (node.Maybe<TYtTable>()) { + statNode = node.Cast<TYtTable>().Stat().Ptr(); + } else if (node.Maybe<TYtOutput>()) { + auto tableWithCluster = GetOutTableWithCluster(node); + statNode = tableWithCluster.first.Cast<TYtOutTable>().Stat().Ptr(); + } else { + ythrow yexception() << "Not a table node " << (node.Raw() ? TString{node.Ref().Content()}.Quote() : TStringBuf("\"null\"")); + } + return MakeIntrusive<TYtTableStatInfo>(statNode); +} + +TStringBuf TYtTableBaseInfo::GetTableName(NNodes::TExprBase node) { + TMaybeNode<TYtTableBase> tableBase; + if (auto outTable = node.Maybe<TYtOutTable>()) { + tableBase = outTable.Cast(); + } else if (node.Maybe<TYtOutput>()) { + tableBase = GetOutTable(node).Cast<TYtOutTable>(); + } else if (auto table = node.Maybe<TYtTable>()) { + tableBase = table.Cast(); + } else { + ythrow yexception() << "Not a table node " << (node.Raw() ? TString{node.Ref().Content()}.Quote() : TStringBuf("\"null\"")); + } + + return tableBase.Cast().Name().Value(); +} + +bool TYtTableBaseInfo::RequiresRemap() const { + return Meta->InferredScheme + // || !Meta->YqlCompatibleScheme -- Non compatuble schemas do not have RowSpec + || Meta->Attrs.contains(QB2Premapper) + || !RowSpec + || !RowSpec->StrictSchema + || !RowSpec->DefaultValues.empty(); +} + +bool TYtTableBaseInfo::HasSameScheme(const TTypeAnnotationNode& scheme) const { + if (!RowSpec) { + if (scheme.GetKind() != ETypeAnnotationKind::Struct) { + return false; + } + const TVector<const TItemExprType*>& items = scheme.Cast<TStructExprType>()->GetItems(); + if (YAMR_FIELDS.size() != items.size()) { + return false; + } + for (size_t i: xrange(YAMR_FIELDS.size())) { + if (items[i]->GetName() != YAMR_FIELDS[i] + || items[i]->GetItemType()->GetKind() != ETypeAnnotationKind::Data + || items[i]->GetItemType()->Cast<TDataExprType>()->GetSlot() != EDataSlot::String) + { + return false; + } + } + + return true; + } else { + return IsSameAnnotation(*RowSpec->GetType(), scheme); + } +} + +bool TYtTableBaseInfo::HasSamePhysicalScheme(const TYtTableBaseInfo& info) const { + if (bool(RowSpec) != bool(info.RowSpec)) { + return false; + } + if (RowSpec) { + if (!IsSameAnnotation(*RowSpec->GetType(), *info.RowSpec->GetType())) { + return false; + } + return RowSpec->GetAuxColumns() == info.RowSpec->GetAuxColumns(); + } + else { + return true; + } +} + +///////////////////////////////////////////////////////////////////////////////////////////////////////// + +TYtTableInfo::TYtTableInfo(const TYtKey& key, TStringBuf cluster) + : TYtTableBaseInfo(key, cluster) +{ +} + +bool TYtTableInfo::Validate(const TExprNode& node, EYtSettingTypes accepted, TExprContext& ctx) { + if (!node.IsCallable(TYtTable::CallableName())) { + ctx.AddError(TIssue(ctx.GetPosition(node.Pos()), TStringBuilder() << "Expected " << TYtTable::CallableName())); + return false; + } + + if (!EnsureArgsCount(node, 8, ctx)) { + return false; + } + + if (!EnsureAtom(*node.Child(TYtTable::idx_Name), ctx)) { + return false; + } + + if (node.Child(TYtTable::idx_Name)->Content().empty()) { + ctx.AddError(TIssue(ctx.GetPosition(node.Child(TYtTable::idx_Name)->Pos()), TStringBuilder() << "Expected non-empty table name")); + return false; + } + +#define VALIDATE_OPT_FIELD(idx, TFunc) \ + if (!node.Child(idx)->IsCallable(TFunc::CallableName()) \ + && !node.Child(idx)->IsCallable(TStringBuf("Void"))) { \ + ctx.AddError(TIssue(ctx.GetPosition(node.Child(idx)->Pos()), TStringBuilder() \ + << "Expected " << TFunc::CallableName() \ + << " or Void")); \ + return false; \ + } + + VALIDATE_OPT_FIELD(TYtTable::idx_RowSpec, TYqlRowSpec) + VALIDATE_OPT_FIELD(TYtTable::idx_Meta, TYtMeta) + VALIDATE_OPT_FIELD(TYtTable::idx_Stat, TYtStat) + VALIDATE_OPT_FIELD(TYtTable::idx_Epoch, TEpoch) + VALIDATE_OPT_FIELD(TYtTable::idx_CommitEpoch, TEpoch) + +#undef VALIDATE_OPT_FIELD + + if (!EnsureTuple(*node.Child(TYtTable::idx_Settings), ctx)) { + return false; + } + + if (!ValidateSettings(*node.Child(TYtTable::idx_Settings), accepted, ctx)) { + return false; + } + + if (!EnsureAtom(*node.Child(TYtTable::idx_Cluster), ctx)) { + return false; + } + + if (node.Child(TYtTable::idx_Cluster)->Content().empty()) { + ctx.AddError(TIssue(ctx.GetPosition(node.Child(TYtTable::idx_Cluster)->Pos()), TStringBuilder() << "Expected non-empty cluster name")); + return false; + } + + return true; +} + +void TYtTableInfo::Parse(TExprBase node, bool useTypes) { + *this = {}; + FromNode = node.Maybe<TExprBase>(); + TYtTable table = node.Cast<TYtTable>(); + Name = table.Name().Value(); + if (table.RowSpec().Maybe<TYqlRowSpec>()) { + RowSpec = MakeIntrusive<TYqlRowSpecInfo>(table.RowSpec(), useTypes); + } + if (table.Meta().Maybe<TYtMeta>()) { + Meta = MakeIntrusive<TYtTableMetaInfo>(table.Meta()); + } + if (table.Stat().Maybe<TYtStat>()) { + Stat = MakeIntrusive<TYtTableStatInfo>(table.Stat()); + } + Epoch = TEpochInfo::Parse(table.Epoch().Ref()); + CommitEpoch = TEpochInfo::Parse(table.CommitEpoch().Ref()); + Settings = table.Settings(); + Cluster = table.Cluster().Value(); + IsTemp = IsAnonymous = NYql::HasSetting(table.Settings().Ref(), EYtSettingType::Anonymous); +} + +TExprBase TYtTableInfo::ToExprNode(TExprContext& ctx, const TPositionHandle& pos) const { + auto tableBuilder = Build<TYtTable>(ctx, pos); + YQL_ENSURE(!Name.empty()); + tableBuilder.Name().Value(Name).Build(); + if (RowSpec) { + tableBuilder.RowSpec(RowSpec->ToExprNode(ctx, pos)); + } else { + tableBuilder.RowSpec<TCoVoid>().Build(); + } + if (Meta) { + tableBuilder.Meta(Meta->ToExprNode(ctx, pos)); + } else { + tableBuilder.Meta<TCoVoid>().Build(); + } + if (Stat) { + tableBuilder.Stat(Stat->ToExprNode(ctx, pos)); + } else { + tableBuilder.Stat<TCoVoid>().Build(); + } + tableBuilder.Epoch(TEpochInfo::ToExprNode(Epoch, ctx, pos)); + tableBuilder.CommitEpoch(TEpochInfo::ToExprNode(CommitEpoch, ctx, pos)); + if (Settings) { + tableBuilder.Settings(Settings.Cast<TCoNameValueTupleList>()); + } else { + tableBuilder.Settings().Build(); + } + tableBuilder.Cluster().Value(Cluster).Build(); + + return tableBuilder.Done(); +} + +TStringBuf TYtTableInfo::GetTableLabel(NNodes::TExprBase node) { + if (auto ann = NYql::GetSetting(node.Cast<TYtTable>().Settings().Ref(), EYtSettingType::Anonymous)) { + if (ann->ChildrenSize() == 2) { + return ann->Child(1)->Content(); + } + } + return node.Cast<TYtTable>().Name().Value(); +} + +bool TYtTableInfo::HasSubstAnonymousLabel(NNodes::TExprBase node) { + if (auto ann = NYql::GetSetting(node.Cast<TYtTable>().Settings().Ref(), EYtSettingType::Anonymous)) { + return ann->ChildrenSize() == 2; + } + return false; +} + +///////////////////////////////////////////////////////////////////////////////////////////////////////// + +TYtOutTableInfo::TYtOutTableInfo(const TStructExprType* type, ui64 nativeYtTypeFlags) { + RowSpec = MakeIntrusive<TYqlRowSpecInfo>(); + RowSpec->SetType(type, nativeYtTypeFlags); + + Meta = MakeIntrusive<TYtTableMetaInfo>(); + Meta->CanWrite = true; + Meta->DoesExist = true; + Meta->YqlCompatibleScheme = true; + + IsTemp = true; +} + +bool TYtOutTableInfo::Validate(const TExprNode& node, TExprContext& ctx) { + if (!node.IsCallable(TYtOutTable::CallableName())) { + ctx.AddError(TIssue(ctx.GetPosition(node.Pos()), TStringBuilder() << "Expected " << TYtOutTable::CallableName())); + return false; + } + if (!EnsureArgsCount(node, 5, ctx)) { + return false; + } + + if (!EnsureAtom(*node.Child(TYtOutTable::idx_Name), ctx)) { + return false; + } + +#define VALIDATE_OPT_FIELD(idx, TFunc) \ + if (!node.Child(idx)->IsCallable(TFunc::CallableName()) \ + && !node.Child(idx)->IsCallable(TStringBuf("Void"))) { \ + ctx.AddError(TIssue(ctx.GetPosition(node.Child(idx)->Pos()), TStringBuilder() \ + << "Expected " << TFunc::CallableName() \ + << " or Void")); \ + return false; \ + } +#define VALIDATE_REQ_FIELD(idx, TFunc) \ + if (!node.Child(idx)->IsCallable(TFunc::CallableName())) { \ + ctx.AddError(TIssue(ctx.GetPosition(node.Child(idx)->Pos()), TStringBuilder() \ + << "Expected " << TFunc::CallableName())); \ + return false; \ + } + + VALIDATE_REQ_FIELD(TYtOutTable::idx_RowSpec, TYqlRowSpec) + VALIDATE_REQ_FIELD(TYtOutTable::idx_Meta, TYtMeta) + VALIDATE_OPT_FIELD(TYtOutTable::idx_Stat, TYtStat) + +#undef VALIDATE_OPT_FIELD +#undef VALIDATE_REQ_FIELD + + if (!EnsureTuple(*node.Child(TYtOutTable::idx_Settings), ctx)) { + return false; + } + + if (!ValidateSettings(*node.Child(TYtOutTable::idx_Settings), EYtSettingType::UniqueBy | EYtSettingType::OpHash | EYtSettingType::ColumnGroups, ctx)) { + return false; + } + + if (auto setting = NYql::GetSetting(*node.Child(TYtOutTable::idx_Settings), EYtSettingType::ColumnGroups)) { + if (!ValidateColumnGroups(*setting, *node.Child(TYtOutTable::idx_RowSpec)->GetTypeAnn()->Cast<TStructExprType>(), ctx)) { + return false; + } + } + + return true; +} + +void TYtOutTableInfo::Parse(TExprBase node) { + *this = {}; + FromNode = node.Maybe<TExprBase>(); + TYtOutTable table = node.Cast<TYtOutTable>(); + Name = table.Name().Value(); + RowSpec = MakeIntrusive<TYqlRowSpecInfo>(table.RowSpec()); + Meta = MakeIntrusive<TYtTableMetaInfo>(table.Meta()); + if (table.Stat().Maybe<TYtStat>()) { + Stat = MakeIntrusive<TYtTableStatInfo>(table.Stat()); + } + Settings = table.Settings(); +} + +TExprBase TYtOutTableInfo::ToExprNode(TExprContext& ctx, const TPositionHandle& pos) const { + auto tableBuilder = Build<TYtOutTable>(ctx, pos); + tableBuilder.Name().Value(Name).Build(); + YQL_ENSURE(RowSpec); + tableBuilder.RowSpec(RowSpec->ToExprNode(ctx, pos)); + YQL_ENSURE(Meta); + tableBuilder.Meta(Meta->ToExprNode(ctx, pos)); + if (Stat) { + tableBuilder.Stat(Stat->ToExprNode(ctx, pos)); + } else { + tableBuilder.Stat<TCoVoid>().Build(); + } + if (Settings) { + tableBuilder.Settings(Settings.Cast<TCoNameValueTupleList>()); + } else { + tableBuilder.Settings().Build(); + } + return tableBuilder.Done(); +} + +TYtOutTableInfo& TYtOutTableInfo::SetUnique(const TDistinctConstraintNode* distinct, const TPositionHandle& pos, TExprContext& ctx) { + if (distinct) { + RowSpec->UniqueKeys = !RowSpec->SortMembers.empty() && RowSpec->SortMembers.size() == RowSpec->SortedBy.size() + && distinct->IsOrderBy(*RowSpec->MakeSortConstraint(ctx)); + if (!Settings) { + Settings = Build<TCoNameValueTupleList>(ctx, pos).Done(); + } + if (const auto columns = NYql::GetSettingAsColumnList(Settings.Ref(), EYtSettingType::UniqueBy)) { + YQL_ENSURE(distinct->ContainsCompleteSet(std::vector<std::string_view>(columns.cbegin(), columns.cend()))); + } else if (const auto simple = distinct->FilterFields(ctx, [](const TPartOfConstraintBase::TPathType& path) { return 1U == path.size(); })) { + auto content = simple->GetContent(); + if (1U < content.size()) { + std::unordered_set<std::string_view> sorted(RowSpec->SortMembers.cbegin(), RowSpec->SortMembers.cend()); + if (const auto filtered = simple->FilterFields(ctx, [&sorted](const TPartOfConstraintBase::TPathType& path) { return 1U == path.size() && sorted.contains(path.front()); })) + content = filtered->GetContent(); + } + std::vector<std::string_view> uniques(content.front().size()); + std::transform(content.front().cbegin(), content.front().cend(), uniques.begin(), [&](const TPartOfConstraintBase::TSetType& set) { return set.front().front(); }); + Settings = TExprBase(NYql::AddSetting(Settings.Ref(), EYtSettingType::UniqueBy, ToAtomList(uniques, pos, ctx), ctx)); + } + } + return *this; +} + +NYT::TNode TYtOutTableInfo::GetColumnGroups() const { + if (Settings) { + if (auto setting = NYql::GetSetting(Settings.Ref(), EYtSettingType::ColumnGroups)) { + return NYT::NodeFromYsonString(setting->Tail().Content()); + } + } + return {}; +} + +///////////////////////////////////////////////////////////////////////////////////////////////////////// + +bool TYtRangesInfo::Validate(const TExprNode& node, TExprContext& ctx, bool exists, const TYqlRowSpecInfo::TPtr& rowSpec) { + auto validatKey = [&rowSpec, &ctx] (TExprNode& key) { + if (!EnsureTupleMaxSize(key, (ui32)rowSpec->SortMembers.size(), ctx)) { + return false; + } + for (size_t i: xrange(key.ChildrenSize())) { + auto keyChild = key.Child(i); + bool isOptional = false; + const TDataExprType* columnType = nullptr; + if (!EnsureDataOrOptionalOfData(keyChild->Pos(), rowSpec->SortedByTypes[i], isOptional, columnType, ctx)) { + ctx.AddError(TIssue(ctx.GetPosition(keyChild->Pos()), TStringBuilder() + << "Table column " << rowSpec->SortedBy[i] << " is not data or optional of data")); + return false; + } + if (IsNull(*keyChild)) { + if (!isOptional) { + ctx.AddError(TIssue(ctx.GetPosition(keyChild->Pos()), TStringBuilder() + << "Column " << rowSpec->SortMembers[i] << " type " + << *rowSpec->SortedByTypes[i] << " cannot be compared with Null")); + return false; + } + } else if (keyChild->GetTypeAnn()->GetKind() == ETypeAnnotationKind::Optional) { + if (!IsSameAnnotation(*keyChild->GetTypeAnn(), *rowSpec->SortedByTypes[i])) { + ctx.AddError(TIssue(ctx.GetPosition(keyChild->Pos()), TStringBuilder() + << "Column " << rowSpec->SortMembers[i] << " type " + << *rowSpec->SortedByTypes[i] << " should be equal to compare type: " + << *keyChild->GetTypeAnn())); + return false; + } + } else { + auto keyType = keyChild->GetTypeAnn()->Cast<TDataExprType>(); + if (keyType->GetSlot() != columnType->GetSlot() + && !GetSuperType(keyType->GetSlot(), columnType->GetSlot())) + { + ctx.AddError(TIssue(ctx.GetPosition(keyChild->Pos()), TStringBuilder() + << "Column " << rowSpec->SortMembers[i] << " type " + << *rowSpec->SortedByTypes[i] << " cannot be compared with " + << *keyChild->GetTypeAnn())); + return false; + } + } + } + return true; + }; + + for (auto& child: node.Children()) { + if (!TYtRangeItemBase::Match(child.Get())) { + ctx.AddError(TIssue(ctx.GetPosition(child->Pos()), TStringBuilder() + << "Expected one of YtRangeItemBase, but got " << child->Content())); + + } + if (rowSpec) { + if (child->IsCallable(TYtKeyExact::CallableName())) { + if (!rowSpec->IsSorted()) { + ctx.AddError(TIssue(ctx.GetPosition(child->Pos()), TStringBuilder() + << TYtKeyExact::CallableName() << " cannot be used with unsorted table")); + return false; + } + + if (!validatKey(*child->Child(TYtKeyExact::idx_Key))) { + return false; + } + } + else if (child->IsCallable(TYtKeyRange::CallableName())) { + if (!rowSpec->IsSorted()) { + ctx.AddError(TIssue(ctx.GetPosition(child->Pos()), TStringBuilder() + << TYtKeyRange::CallableName() << " cannot be used with unsorted table")); + return false; + } + if (!validatKey(*child->Child(TYtKeyRange::idx_Lower))) { + return false; + } + if (!validatKey(*child->Child(TYtKeyRange::idx_Upper))) { + return false; + } + } + } else if (exists) { + ctx.AddError(TIssue(ctx.GetPosition(child->Pos()), TStringBuilder() + << child->Content() << " cannot be used with YAMR table")); + return false; + } + } + return true; +} + +void TYtRangesInfo::Parse(TExprBase node) { + *this = {}; + + for (auto item: node.Cast<TExprList>()) { + if (item.Maybe<TYtRow>()) { + Ranges.emplace_back(TRowSingle{FromString<ui64>(item.Cast<TYtRow>().Index().Literal().Value())}); + } else if (item.Maybe<TYtRowRange>()) { + TYtRowRange rangeNode = item.Cast<TYtRowRange>(); + TRowRange range; + if (!rangeNode.Lower().Maybe<TCoVoid>()) { + range.Lower = FromString<ui64>(rangeNode.Lower().Cast<TCoUint64>().Literal().Value()); + } + if (!rangeNode.Upper().Maybe<TCoVoid>()) { + range.Upper = FromString<ui64>(rangeNode.Upper().Cast<TCoUint64>().Literal().Value()); + } + Ranges.push_back(std::move(range)); + } else if (item.Maybe<TYtKeyExact>()) { + TYtKeyExact keyNode = item.Cast<TYtKeyExact>(); + Ranges.emplace_back(TKeySingle{TVector<NNodes::TExprBase>{keyNode.Key().begin(), keyNode.Key().end()}}); + } else if (item.Maybe<TYtKeyRange>()) { + TYtKeyRange rangeNode = item.Cast<TYtKeyRange>(); + TKeyRange range; + range.Lower.assign(rangeNode.Lower().begin(), rangeNode.Lower().end()); + range.Upper.assign(rangeNode.Upper().begin(), rangeNode.Upper().end()); + // if useKeyBound setting is missed, useKeyBoundApi should be set to false, + // irrespective of DEFAULT_USE_KEY_BOUND_API value + range.UseKeyBoundApi = false; + if (rangeNode.Flags()) { + for (auto atom: rangeNode.Flags().Cast()) { + if (atom.Value() == TStringBuf("excludeLower")) { + range.LowerInclude = false; + } else if (atom.Value() == TStringBuf("includeUpper")) { + range.UpperInclude = true; + } else if (atom.Value() == TStringBuf("useKeyBound")) { + range.UseKeyBoundApi = true; + } + } + } + Ranges.push_back(std::move(range)); + } + } +} + +void TYtRangesInfo::Parse(const TVector<NYT::TReadRange>& ranges, TExprContext& ctx, const TPositionHandle& pos) { + *this = {}; + + for (const NYT::TReadRange& item: ranges) { + YQL_ENSURE(!NYT::IsTrivial(item.Exact_) || !NYT::IsTrivial(item.LowerLimit_) || !NYT::IsTrivial(item.UpperLimit_), "Full scan range is not supported"); + + if (!NYT::IsTrivial(item.Exact_)) { + YQL_ENSURE(NYT::IsTrivial(item.LowerLimit_) && NYT::IsTrivial(item.UpperLimit_)); + if (item.Exact_.RowIndex_) { + YQL_ENSURE(!item.Exact_.Key_ && !item.Exact_.Offset_); + Ranges.emplace_back(TRowSingle{(ui64)*item.Exact_.RowIndex_}); + } else if (item.Exact_.Key_) { + YQL_ENSURE(!item.Exact_.Offset_); + TVector<TExprBase> key; + for (auto& keyPart: item.Exact_.Key_->Parts_) { + key.emplace_back(YtNodeToExprNode(keyPart, ctx, pos)); + } + Ranges.emplace_back(TKeySingle{std::move(key)}); + } + } else { + YQL_ENSURE(!NYT::IsTrivial(item.LowerLimit_) || !NYT::IsTrivial(item.UpperLimit_)); + if (item.LowerLimit_.RowIndex_ || item.UpperLimit_.RowIndex_) { + TRowRange range; + if (item.LowerLimit_.RowIndex_) { + range.Lower.ConstructInPlace(*item.LowerLimit_.RowIndex_); + } + if (item.UpperLimit_.RowIndex_) { + range.Upper.ConstructInPlace(*item.UpperLimit_.RowIndex_); + } + Ranges.emplace_back(std::move(range)); + } else if (item.LowerLimit_.Key_ || item.UpperLimit_.Key_) { + TKeyRange range; + range.UseKeyBoundApi = false; + if (item.LowerLimit_.Key_ && !item.LowerLimit_.Key_->Parts_.empty()) { + size_t count = item.LowerLimit_.Key_->Parts_.size(); + if (item.LowerLimit_.Key_->Parts_.back().IsEntity() && !item.LowerLimit_.Key_->Parts_.back().HasAttributes()) { + range.LowerInclude = false; + --count; + } + for (size_t i = 0; i < count; ++i) { + range.Lower.emplace_back(YtNodeToExprNode(item.LowerLimit_.Key_->Parts_[i], ctx, pos)); + } + } + if (item.UpperLimit_.Key_ && !item.UpperLimit_.Key_->Parts_.empty()) { + size_t count = item.UpperLimit_.Key_->Parts_.size(); + if (item.UpperLimit_.Key_->Parts_.back().IsEntity() + && item.UpperLimit_.Key_->Parts_.back().HasAttributes() + && item.UpperLimit_.Key_->Parts_.back().GetAttributes().AsMap().Value("type", "") == "max") { + range.UpperInclude = true; + --count; + } + for (size_t i = 0; i < count; ++i) { + range.Upper.emplace_back(YtNodeToExprNode(item.UpperLimit_.Key_->Parts_[i], ctx, pos)); + } + } + Ranges.emplace_back(std::move(range)); + } else { + YQL_ENSURE(item.LowerLimit_.KeyBound_ || item.UpperLimit_.KeyBound_); + TKeyRange range; + range.UseKeyBoundApi = true; + if (item.LowerLimit_.KeyBound_) { + auto relation = item.LowerLimit_.KeyBound_->Relation(); + YQL_ENSURE(relation == NYT::ERelation::Greater || relation == NYT::ERelation::GreaterOrEqual); + range.LowerInclude = relation == NYT::ERelation::GreaterOrEqual; + YQL_ENSURE(!item.LowerLimit_.KeyBound_->Key().Parts_.empty()); + for (auto& key : item.LowerLimit_.KeyBound_->Key().Parts_) { + range.Lower.emplace_back(YtNodeToExprNode(key, ctx, pos)); + } + } + + if (item.UpperLimit_.KeyBound_) { + auto relation = item.UpperLimit_.KeyBound_->Relation(); + YQL_ENSURE(relation == NYT::ERelation::Less || relation == NYT::ERelation::LessOrEqual); + range.UpperInclude = relation == NYT::ERelation::LessOrEqual; + YQL_ENSURE(!item.UpperLimit_.KeyBound_->Key().Parts_.empty()); + for (auto& key : item.UpperLimit_.KeyBound_->Key().Parts_) { + range.Upper.emplace_back(YtNodeToExprNode(key, ctx, pos)); + } + } + Ranges.emplace_back(std::move(range)); + } + } + } +} + +void TYtRangesInfo::AddRowRange(const TRowRange& range) { + Ranges.push_back(range); +} + +void TYtRangesInfo::SetUseKeyBoundApi(bool useKeyBoundApi) { + for (auto& range: Ranges) { + if (auto* data = std::get_if<TKeyRange>(&range)) { + data->UseKeyBoundApi = useKeyBoundApi; + } + } +} + +TExprBase TYtRangesInfo::ToExprNode(TExprContext& ctx, const TPositionHandle& pos) const { + auto rangesBuilder = Build<TExprList>(ctx, pos); + for (auto& range: Ranges) { + std::visit(TOverloaded{ + [&](const TRowSingle& row) { + rangesBuilder.Add<TYtRow>() + .Index() + .Literal() + .Value(ToString(row.Offset), TNodeFlags::Default) + .Build() + .Build() + .Build(); + }, + [&](const TRowRange& data) { + auto builder = rangesBuilder.Add<TYtRowRange>(); + if (data.Lower) { + builder.Lower<TCoUint64>() + .Literal() + .Value(ToString(*data.Lower), TNodeFlags::Default) + .Build() + .Build(); + } else { + builder.Lower<TCoVoid>().Build(); + } + if (data.Upper) { + builder.Upper<TCoUint64>() + .Literal() + .Value(ToString(*data.Upper), TNodeFlags::Default) + .Build() + .Build(); + } else { + builder.Upper<TCoVoid>().Build(); + } + builder.Build(); + }, + [&](const TKeySingle& key) { + rangesBuilder.Add<TYtKeyExact>() + .Key() + .Add(key.Key) + .Build() + .Build(); + }, + [&](const TKeyRange& data) { + auto builder = rangesBuilder.Add<TYtKeyRange>(); + builder.Lower().Add(data.Lower).Build(); + builder.Upper().Add(data.Upper).Build(); + if (!data.LowerInclude || data.UpperInclude) { + auto flagsBuilder = builder.Flags(); + if (!data.LowerInclude) { + flagsBuilder.Add().Value("excludeLower", TNodeFlags::Default).Build(); + } + if (data.UpperInclude) { + flagsBuilder.Add().Value("includeUpper", TNodeFlags::Default).Build(); + } + if (data.UseKeyBoundApi) { + flagsBuilder.Add().Value("useKeyBound", TNodeFlags::Default).Build(); + } + flagsBuilder.Build(); + } + builder.Build(); + } + }, range); + } + return rangesBuilder.Done(); +} + +void TYtRangesInfo::FillRichYPath(NYT::TRichYPath& path, size_t keyColumnsCount) const { + path.MutableRanges().ConstructInPlace(); + for (auto& range: Ranges) { + std::visit(TOverloaded{ + [&](const TRowSingle& row) { + path.AddRange(NYT::TReadRange().Exact(NYT::TReadLimit().RowIndex(row.Offset))); + }, + [&](const TRowRange& data) { + NYT::TReadRange ytRange; + if (data.Lower) { + ytRange.LowerLimit(NYT::TReadLimit().RowIndex(*data.Lower)); + } + if (data.Upper) { + ytRange.UpperLimit(NYT::TReadLimit().RowIndex(*data.Upper)); + } + path.AddRange(ytRange); + }, + [&](const TKeySingle& data) { + NYT::TKey key; + for (auto& node: data.Key) { + auto keyPart = ExprNodeToYtNode(node.Ref()); + YQL_ENSURE(!keyPart.IsUndefined(), "Unsupported range node: " << node.Ref().Content()); + key.Add(std::move(keyPart)); + } + path.AddRange(NYT::TReadRange().Exact(NYT::TReadLimit().Key(key))); + }, + [&](const TKeyRange& data) { + YQL_ENSURE(keyColumnsCount > 0); + YQL_ENSURE(data.Lower.size() <= keyColumnsCount); + YQL_ENSURE(data.Upper.size() <= keyColumnsCount); + NYT::TReadRange ytRange; + if (!data.Lower.empty()) { + NYT::TKey key; + for (auto& node: data.Lower) { + auto keyPart = ExprNodeToYtNode(node.Ref()); + YQL_ENSURE(!keyPart.IsUndefined(), "Unsupported range node: " << node.Ref().Content()); + key.Add(std::move(keyPart)); + } + if (data.UseKeyBoundApi) { + NYT::TKeyBound lower(data.LowerInclude ? NYT::ERelation::GreaterOrEqual : NYT::ERelation::Greater, key); + ytRange.LowerLimit(NYT::TReadLimit().KeyBound(lower)); + } else { + if (!data.LowerInclude) { + size_t toAddMaxs = keyColumnsCount - data.Lower.size(); + for (size_t i = 0; i < toAddMaxs; ++i) { + auto entity = NYT::TNode::CreateEntity(); + entity.Attributes().AsMap()[TStringBuf("type")] = TStringBuf("max"); + key.Add(entity); + } + if (toAddMaxs == 0) { + key.Add(NYT::TNode::CreateEntity()); + } + } + ytRange.LowerLimit(NYT::TReadLimit().Key(key)); + } + } + if (!data.Upper.empty()) { + NYT::TKey key; + for (auto& node: data.Upper) { + auto keyPart = ExprNodeToYtNode(node.Ref()); + YQL_ENSURE(!keyPart.IsUndefined(), "Unsupported range node: " << node.Ref().Content()); + key.Add(std::move(keyPart)); + } + if (data.UseKeyBoundApi) { + NYT::TKeyBound upper(data.UpperInclude ? NYT::ERelation::LessOrEqual : NYT::ERelation::Less, key); + ytRange.UpperLimit(NYT::TReadLimit().KeyBound(upper)); + } else { + if (data.UpperInclude) { + auto entity = NYT::TNode::CreateEntity(); + entity.Attributes().AsMap()[TStringBuf("type")] = TStringBuf("max"); + key.Add(entity); + } + ytRange.UpperLimit(NYT::TReadLimit().Key(key)); + } + } + path.AddRange(ytRange); + } + }, range); + } +} + +TMaybe<ui64> TYtRangesInfo::GetUsedRows(ui64 tableRowCount) const { + ui64 rows = 0; + if (tableRowCount) { + for (auto& range: Ranges) { + if (std::holds_alternative<TRowSingle>(range)) { + ++rows; + } else if (const auto* data = std::get_if<TRowRange>(&range)) { + ui64 range = tableRowCount; + if (data->Upper) { + range = *data->Upper; + } + if (data->Lower) { + range -= Min(range, *data->Lower); + } + rows += range; + } else { + return Nothing(); + } + } + } + return rows; +} + +size_t TYtRangesInfo::GetRangesCount() const { + return Ranges.size(); +} + +size_t TYtRangesInfo::GetUsedKeyPrefixLength() const { + size_t used = 0; + for (auto& range: Ranges) { + if (const auto* key = std::get_if<TKeySingle>(&range)) { + used = std::max(used, key->Key.size()); + } else if (const auto* keyRange = std::get_if<TKeyRange>(&range)) { + used = std::max(used, keyRange->Lower.size()); + used = std::max(used, keyRange->Upper.size()); + } + } + return used; +} + +TVector<TYtRangesInfo::TRange> TYtRangesInfo::GetRanges() const { + return Ranges; +} + +bool TYtRangesInfo::IsEmpty() const { + return Ranges.empty() || AllOf(Ranges, [](const TRange& r) { + if (const TRowRange* data = ::std::get_if<TRowRange>(&r)) { + return (data->Lower && data->Upper && *data->Lower >= *data->Upper) || + (data->Upper && *data->Upper == 0ul); + } + return false; + }); +} + +namespace { + +template <typename TLayerType> +NUdf::TUnboxedValuePod GetTzValue(TStringBuf atom) { + TStringBuf valueStr; + GetNext(atom, ',', valueStr); + return NUdf::TUnboxedValuePod(::FromString<TLayerType>(valueStr)); +} + +NUdf::TUnboxedValuePod GetDecimalValue(TCoDecimal decimal) { + return NUdf::TUnboxedValuePod(NDecimal::FromString(decimal.Literal().Value(), + ::FromString<ui8>(decimal.Precision().Value()), + ::FromString<ui8>(decimal.Scale().Value()))); +} + +NKikimr::NUdf::EDataSlot GetCompareType(TStringBuf type) { + using namespace NKikimr::NUdf; + auto cmpSlot = GetDataSlot(type); + switch (cmpSlot) { + case EDataSlot::Int8: + case EDataSlot::Uint8: + case EDataSlot::Int16: + case EDataSlot::Uint16: + case EDataSlot::Int32: + case EDataSlot::Uint32: + case EDataSlot::Int64: + cmpSlot = EDataSlot::Int64; + break; + case EDataSlot::Uint64: + cmpSlot = EDataSlot::Uint64; + break; + default: + break; + } + return cmpSlot; +} + +int CompareDataNodes(NNodes::TExprBase left, NNodes::TExprBase right) +{ + using namespace NNodes; + using namespace NKikimr::NUdf; + + if (left.Maybe<TCoNull>()) { + return right.Maybe<TCoNull>().IsValid() ? 0 : -1; + } else if (right.Maybe<TCoNull>()) { + return 1; + } + + YQL_ENSURE(left.Maybe<TCoDataCtor>().IsValid()); + YQL_ENSURE(right.Maybe<TCoDataCtor>().IsValid()); + + auto leftDataCtor = left.Cast<TCoDataCtor>(); + auto rightDataCtor = right.Cast<TCoDataCtor>(); + + auto leftType = GetCompareType(left.Ref().Content()); + auto rightType = GetCompareType(right.Ref().Content()); + if (leftType == rightType) { + switch (leftType) { + case EDataSlot::Bool: + return NUdf::CompareValues(leftType, + TUnboxedValuePod(NYql::FromString<bool>(leftDataCtor.Literal().Ref(), leftType)), + TUnboxedValuePod(NYql::FromString<bool>(rightDataCtor.Literal().Ref(), leftType))); + case EDataSlot::Int64: + case EDataSlot::Interval: + case EDataSlot::Date: + case EDataSlot::Datetime: + case EDataSlot::Date32: + case EDataSlot::Datetime64: + case EDataSlot::Timestamp64: + case EDataSlot::Interval64: + return NUdf::CompareValues(leftType, + TUnboxedValuePod(NYql::FromString<i64>(leftDataCtor.Literal().Ref(), leftType)), + TUnboxedValuePod(NYql::FromString<i64>(rightDataCtor.Literal().Ref(), leftType))); + case EDataSlot::Uint64: + case EDataSlot::Timestamp: + return NUdf::CompareValues(leftType, + TUnboxedValuePod(NYql::FromString<ui64>(leftDataCtor.Literal().Ref(), leftType)), + TUnboxedValuePod(NYql::FromString<ui64>(rightDataCtor.Literal().Ref(), leftType))); + case EDataSlot::Float: + return NUdf::CompareValues(leftType, + TUnboxedValuePod(NYql::FromString<float>(leftDataCtor.Literal().Ref(), leftType)), + TUnboxedValuePod(NYql::FromString<float>(rightDataCtor.Literal().Ref(), leftType))); + case EDataSlot::Double: + return NUdf::CompareValues(leftType, + TUnboxedValuePod(NYql::FromString<double>(leftDataCtor.Literal().Ref(), leftType)), + TUnboxedValuePod(NYql::FromString<double>(rightDataCtor.Literal().Ref(), leftType))); + case EDataSlot::String: + case EDataSlot::Utf8: + case EDataSlot::Uuid: + return leftDataCtor.Literal().Value().compare(rightDataCtor.Literal().Value()); + case EDataSlot::TzDate: + return NUdf::CompareValues(leftType, + GetTzValue<ui16>(leftDataCtor.Literal().Value()), + GetTzValue<ui16>(rightDataCtor.Literal().Value())); + case EDataSlot::TzDatetime: + return NUdf::CompareValues(leftType, + GetTzValue<ui32>(leftDataCtor.Literal().Value()), + GetTzValue<ui32>(rightDataCtor.Literal().Value())); + case EDataSlot::TzTimestamp: + return NUdf::CompareValues(leftType, + GetTzValue<ui64>(leftDataCtor.Literal().Value()), + GetTzValue<ui64>(rightDataCtor.Literal().Value())); + case EDataSlot::Decimal: + return NUdf::CompareValues(leftType, + GetDecimalValue(left.Cast<TCoDecimal>()), + GetDecimalValue(right.Cast<TCoDecimal>())); + default: + break; + } + } + + YQL_LOG_CTX_THROW yexception() << "Cannot compare " << left.Ref().Content() << " and " << right.Ref().Content(); +} + +bool AdjacentDataNodes(NNodes::TExprBase left, NNodes::TExprBase right) +{ + using namespace NNodes; + using namespace NKikimr::NUdf; + + YQL_ENSURE(left.Maybe<TCoDataCtor>().IsValid()); + YQL_ENSURE(right.Maybe<TCoDataCtor>().IsValid()); + + auto leftLiteral = left.Cast<TCoDataCtor>().Literal(); + auto rightLiteral = right.Cast<TCoDataCtor>().Literal(); + + auto leftType = GetCompareType(left.Ref().Content()); + auto rightType = GetCompareType(right.Ref().Content()); + if (leftType == rightType) { + switch (leftType) { + case EDataSlot::Bool: + return !FromString<bool>(leftLiteral.Ref(), leftType) && FromString<bool>(rightLiteral.Ref(), rightType); + case EDataSlot::Int64: + return FromString<i64>(leftLiteral.Ref(), leftType) + 1 == FromString<i64>(rightLiteral.Ref(), rightType); + case EDataSlot::Uint64: + return FromString<ui64>(leftLiteral.Ref(), leftType) + 1 == FromString<ui64>(rightLiteral.Ref(), rightType); + default: + return false; + } + } + + YQL_LOG_CTX_THROW yexception() << "Cannot compare " << left.Ref().Content() << " and " << right.Ref().Content(); +} + +template <bool UpperBound> +void ScaleDate(ui64& val, bool& includeBound, EDataSlot srcDataSlot, EDataSlot targetDataSlot) { + switch (srcDataSlot) { + case EDataSlot::Date: + switch (targetDataSlot) { + case EDataSlot::Datetime: + val *= 86400ull; + break; + case EDataSlot::Timestamp: + val *= 86400000000ull; + break; + default: + break; + } + break; + case EDataSlot::Datetime: + switch (targetDataSlot) { + case EDataSlot::Date: + if (val % 86400ull) { + includeBound = UpperBound; + } + val /= 86400ull; + break; + case EDataSlot::Timestamp: + val *= 1000000ull; + break; + default: + break; + } + break; + case EDataSlot::Timestamp: + switch (targetDataSlot) { + case EDataSlot::Date: + if (val % 86400000000ull) { + includeBound = UpperBound; + } + val /= 86400000000ull; + break; + case EDataSlot::Datetime: + if (val % 1000000ULL) { + includeBound = UpperBound; + } + val /= 1000000ULL; + break; + default: + break; + } + break; + default: + break; + } +} + +bool AdjustLowerValue(TString& lowerValue, bool& lowerInclude, EDataSlot lowerDataSlot, EDataSlot targetDataSlot) { + if (EDataSlot::Interval == targetDataSlot + || (GetDataTypeInfo(targetDataSlot).Features & NUdf::SignedIntegralType)) + { + // Target is signed integer + if (GetDataTypeInfo(lowerDataSlot).Features & NUdf::FloatType) { + double dVal = FromString<double>(lowerValue); + i64 val = 0; + if (dVal <= static_cast<double>(std::numeric_limits<i64>::min())) { + if (dVal < static_cast<double>(std::numeric_limits<i64>::min())) { + lowerInclude = true; + } + val = std::numeric_limits<i64>::min(); + } + else if (dVal == static_cast<double>(std::numeric_limits<i64>::max())) { + val = std::numeric_limits<i64>::max(); + } + else if (dVal > static_cast<double>(std::numeric_limits<i64>::max())) { + return false; + } + else if (std::ceil(dVal) != dVal) { + lowerInclude = true; + val = static_cast<i64>(std::ceil(dVal)); + } + else { + val = static_cast<i64>(dVal); + } + + TMaybe<i64> valMin; + TMaybe<i64> valMax; + switch (targetDataSlot) { + case EDataSlot::Int8: + valMin = static_cast<i64>(std::numeric_limits<i8>::min()); + valMax = static_cast<i64>(std::numeric_limits<i8>::max()); + break; + case EDataSlot::Int16: + valMin = static_cast<i64>(std::numeric_limits<i16>::min()); + valMax = static_cast<i64>(std::numeric_limits<i16>::max()); + break; + case EDataSlot::Int32: + valMin = static_cast<i64>(std::numeric_limits<i32>::min()); + valMax = static_cast<i64>(std::numeric_limits<i32>::max()); + break; + case EDataSlot::Int64: + case EDataSlot::Interval: + valMin = static_cast<i64>(std::numeric_limits<i64>::min()); + valMax = static_cast<i64>(std::numeric_limits<i64>::max()); + break; + default: + break; + } + + if (valMax && (val > *valMax || (!lowerInclude && val == *valMax))) { + return false; + } + + if (valMin && val < *valMin) { + lowerInclude = true; + val = *valMin; + } + + if (!lowerInclude) { + lowerInclude = true; + ++val; + } + + lowerValue = ToString(val); + } + else if (GetDataTypeInfo(lowerDataSlot).Features & NUdf::UnsignedIntegralType) { + ui64 val = FromString<ui64>(lowerValue); + TMaybe<ui64> valMax; + switch (targetDataSlot) { + case EDataSlot::Int8: + valMax = static_cast<ui64>(std::numeric_limits<i8>::max()); + break; + case EDataSlot::Int16: + valMax = static_cast<ui64>(std::numeric_limits<i16>::max()); + break; + case EDataSlot::Int32: + valMax = static_cast<ui64>(std::numeric_limits<i32>::max()); + break; + case EDataSlot::Int64: + case EDataSlot::Interval: + valMax = static_cast<ui64>(std::numeric_limits<i64>::max()); + break; + default: + break; + } + + if (valMax && (val > *valMax || (!lowerInclude && val == *valMax))) { + return false; + } + + if (!lowerInclude) { + lowerInclude = true; + ++val; + lowerValue = ToString(val); + } + } + else if (GetDataTypeInfo(lowerDataSlot).Features & NUdf::SignedIntegralType) { + i64 val = FromString<i64>(lowerValue); + + TMaybe<i64> valMin; + TMaybe<i64> valMax; + switch (targetDataSlot) { + case EDataSlot::Int8: + valMin = static_cast<i64>(std::numeric_limits<i8>::min()); + valMax = static_cast<i64>(std::numeric_limits<i8>::max()); + break; + case EDataSlot::Int16: + valMin = static_cast<i64>(std::numeric_limits<i16>::min()); + valMax = static_cast<i64>(std::numeric_limits<i16>::max()); + break; + case EDataSlot::Int32: + valMin = static_cast<i64>(std::numeric_limits<i32>::min()); + valMax = static_cast<i64>(std::numeric_limits<i32>::max()); + break; + case EDataSlot::Int64: + case EDataSlot::Interval: + valMin = static_cast<i64>(std::numeric_limits<i64>::min()); + valMax = static_cast<i64>(std::numeric_limits<i64>::max()); + break; + default: + break; + } + + if (valMax && (val > *valMax || (!lowerInclude && val == *valMax))) { + return false; + } + + if (valMin && val < *valMin) { + lowerInclude = true; + val = *valMin; + } + + if (!lowerInclude) { + lowerInclude = true; + ++val; + } + + lowerValue = ToString(val); + } + } + else if (GetDataTypeInfo(targetDataSlot).Features & (NUdf::UnsignedIntegralType | NUdf::DateType)) { + // Target is unsigned integer + ui64 val = 0; + if (GetDataTypeInfo(lowerDataSlot).Features & NUdf::SignedIntegralType) { + if (FromString<i64>(lowerValue) < 0) { + lowerInclude = true; + val = 0; + } + else { + val = FromString<ui64>(lowerValue); + } + } + else if (GetDataTypeInfo(lowerDataSlot).Features & NUdf::FloatType) { + double dVal = FromString<double>(lowerValue); + + if (dVal > static_cast<double>(std::numeric_limits<ui64>::max())) { + return false; + } + else if (dVal == static_cast<double>(std::numeric_limits<ui64>::max())) { + val = std::numeric_limits<ui64>::max(); + } + else if (dVal < 0. || std::ceil(dVal) != dVal) { + lowerInclude = true; + val = static_cast<ui64>(std::ceil(Max<double>(0., dVal))); + } + else { + val = static_cast<ui64>(dVal); + } + } + else if (GetDataTypeInfo(lowerDataSlot).Features & NUdf::UnsignedIntegralType) { + val = FromString<ui64>(lowerValue); + } + else if ((GetDataTypeInfo(lowerDataSlot).Features & NUdf::DateType) && (GetDataTypeInfo(targetDataSlot).Features & NUdf::DateType) + && lowerDataSlot != targetDataSlot) { + + val = FromString<ui64>(lowerValue); + ScaleDate<false>(val, lowerInclude, lowerDataSlot, targetDataSlot); + } + else { + // Nothing to change for other types + return true; + } + TMaybe<ui64> valMax; + switch (targetDataSlot) { + case EDataSlot::Uint8: + valMax = static_cast<ui64>(std::numeric_limits<ui8>::max()); + break; + case EDataSlot::Uint16: + valMax = static_cast<ui64>(std::numeric_limits<ui16>::max()); + break; + case EDataSlot::Uint32: + valMax = static_cast<ui64>(std::numeric_limits<ui32>::max()); + break; + case EDataSlot::Uint64: + valMax = static_cast<ui64>(std::numeric_limits<ui64>::max()); + break; + case EDataSlot::Date: + valMax = static_cast<ui64>(NUdf::MAX_DATE); + break; + case EDataSlot::Datetime: + valMax = static_cast<ui64>(NUdf::MAX_DATETIME); + break; + case EDataSlot::Timestamp: + valMax = static_cast<ui64>(NUdf::MAX_TIMESTAMP); + break; + default: + break; + } + + if (valMax && (val > *valMax || (!lowerInclude && val == *valMax))) { + return false; + } + + if (!lowerInclude) { + lowerInclude = true; + ++val; + } + + lowerValue = ToString(val); + } + return true; +} + +bool AdjustUpperValue(TString& upperValue, bool& upperInclude, EDataSlot upperDataSlot, EDataSlot targetDataSlot) { + if (EDataSlot::Interval == targetDataSlot + || (GetDataTypeInfo(targetDataSlot).Features & NUdf::SignedIntegralType)) + { + // Target is signed integer + if (GetDataTypeInfo(upperDataSlot).Features & NUdf::FloatType) { + double dVal = FromString<double>(upperValue); + i64 val = 0; + if (dVal >= static_cast<double>(std::numeric_limits<i64>::max())) { + if (dVal > static_cast<double>(std::numeric_limits<i64>::max())) { + upperInclude = true; + } + val = std::numeric_limits<i64>::max(); + } + else if (dVal < static_cast<double>(std::numeric_limits<i64>::min())) { + return false; + } + else if (dVal == static_cast<double>(std::numeric_limits<i64>::min())) { + val = std::numeric_limits<i64>::min(); + } + else if (std::floor(dVal) != dVal) { + upperInclude = true; + val = static_cast<ui64>(std::floor(dVal)); + } + else { + val = static_cast<i64>(dVal); + } + + TMaybe<i64> valMin; + TMaybe<i64> valMax; + switch (targetDataSlot) { + case EDataSlot::Int8: + valMin = static_cast<i64>(std::numeric_limits<i8>::min()); + valMax = static_cast<i64>(std::numeric_limits<i8>::max()); + break; + case EDataSlot::Int16: + valMin = static_cast<i64>(std::numeric_limits<i16>::min()); + valMax = static_cast<i64>(std::numeric_limits<i16>::max()); + break; + case EDataSlot::Int32: + valMin = static_cast<i64>(std::numeric_limits<i32>::min()); + valMax = static_cast<i64>(std::numeric_limits<i32>::max()); + break; + case EDataSlot::Int64: + case EDataSlot::Interval: + valMin = static_cast<i64>(std::numeric_limits<i64>::min()); + valMax = static_cast<i64>(std::numeric_limits<i64>::max()); + break; + default: + break; + } + + if (valMin && (val < *valMin || (!upperInclude && val == *valMin))) { + return false; + } + + if (valMax && val > *valMax) { + upperInclude = true; + val = *valMax; + } + else if (upperInclude && (!valMax || val < *valMax)) { + upperInclude = false; + ++val; + } + + upperValue = ToString(val); + } + else if (GetDataTypeInfo(upperDataSlot).Features & NUdf::UnsignedIntegralType) { + ui64 val = FromString<ui64>(upperValue); + TMaybe<ui64> valMax; + switch (targetDataSlot) { + case EDataSlot::Int8: + valMax = static_cast<ui64>(std::numeric_limits<i8>::max()); + break; + case EDataSlot::Int16: + valMax = static_cast<ui64>(std::numeric_limits<i16>::max()); + break; + case EDataSlot::Int32: + valMax = static_cast<ui64>(std::numeric_limits<i32>::max()); + break; + case EDataSlot::Int64: + case EDataSlot::Interval: + valMax = static_cast<ui64>(std::numeric_limits<i64>::max()); + break; + default: + break; + } + + if (valMax && val > *valMax) { + upperInclude = true; + val = *valMax; + upperValue = ToString(val); + } + else if (upperInclude && (!valMax || val < *valMax)) { + upperInclude = false; + ++val; + upperValue = ToString(val); + } + } + else if (GetDataTypeInfo(upperDataSlot).Features & NUdf::SignedIntegralType) { + i64 val = FromString<i64>(upperValue); + + TMaybe<i64> valMin; + TMaybe<i64> valMax; + switch (targetDataSlot) { + case EDataSlot::Int8: + valMin = static_cast<i64>(std::numeric_limits<i8>::min()); + valMax = static_cast<i64>(std::numeric_limits<i8>::max()); + break; + case EDataSlot::Int16: + valMin = static_cast<i64>(std::numeric_limits<i16>::min()); + valMax = static_cast<i64>(std::numeric_limits<i16>::max()); + break; + case EDataSlot::Int32: + valMin = static_cast<i64>(std::numeric_limits<i32>::min()); + valMax = static_cast<i64>(std::numeric_limits<i32>::max()); + break; + case EDataSlot::Int64: + case EDataSlot::Interval: + valMin = static_cast<i64>(std::numeric_limits<i64>::min()); + valMax = static_cast<i64>(std::numeric_limits<i64>::max()); + break; + default: + break; + } + + if (valMin && (val < *valMin || (!upperInclude && val == *valMin))) { + return false; + } + + if (valMax && val > *valMax) { + upperInclude = true; + val = *valMax; + } + else if (upperInclude && (!valMax || val < *valMax)) { + upperInclude = false; + ++val; + } + + upperValue = ToString(val); + } + } + else if (GetDataTypeInfo(targetDataSlot).Features & (NUdf::UnsignedIntegralType | NUdf::DateType)) { + // Target is unsigned integer + ui64 val = 0; + if (GetDataTypeInfo(upperDataSlot).Features & NUdf::SignedIntegralType) { + if (FromString<i64>(upperValue) < 0) { + return false; + } + else { + val = FromString<ui64>(upperValue); + } + } + else if (GetDataTypeInfo(upperDataSlot).Features & NUdf::FloatType) { + double dVal = FromString<double>(upperValue); + if (dVal < 0.) { + return false; + } + else if (dVal >= static_cast<double>(std::numeric_limits<ui64>::max())) { + if (dVal > static_cast<double>(std::numeric_limits<ui64>::max())) { + upperInclude = true; + } + val = std::numeric_limits<ui64>::max(); + } + else if (std::floor(dVal) != dVal) { + upperInclude = true; + val = static_cast<ui64>(std::floor(dVal)); + } + else { + val = static_cast<ui64>(dVal); + } + } + else if (GetDataTypeInfo(upperDataSlot).Features & NUdf::UnsignedIntegralType) { + val = FromString<ui64>(upperValue); + } + else if ((GetDataTypeInfo(upperDataSlot).Features & NUdf::DateType) && (GetDataTypeInfo(targetDataSlot).Features & NUdf::DateType) + && upperDataSlot != targetDataSlot) { + + val = FromString<ui64>(upperValue); + ScaleDate<true>(val, upperInclude, upperDataSlot, targetDataSlot); + } + else { + // Nothing to change for other types + return true; + } + + TMaybe<ui64> valMax; + switch (targetDataSlot) { + case EDataSlot::Uint8: + valMax = static_cast<ui64>(std::numeric_limits<ui8>::max()); + break; + case EDataSlot::Uint16: + valMax = static_cast<ui64>(std::numeric_limits<ui16>::max()); + break; + case EDataSlot::Uint32: + valMax = static_cast<ui64>(std::numeric_limits<ui32>::max()); + break; + case EDataSlot::Uint64: + valMax = static_cast<ui64>(std::numeric_limits<ui64>::max()); + break; + case EDataSlot::Date: + valMax = static_cast<ui64>(NUdf::MAX_DATE); + break; + case EDataSlot::Datetime: + valMax = static_cast<ui64>(NUdf::MAX_DATETIME); + break; + case EDataSlot::Timestamp: + valMax = static_cast<ui64>(NUdf::MAX_TIMESTAMP); + break; + default: + break; + } + + if (valMax && val > *valMax) { + upperInclude = true; + val = *valMax; + } + else if (upperInclude && (!valMax || val < *valMax)) { + upperInclude = false; + ++val; + } + + upperValue = ToString(val); + } + return true; +} + + +struct TKeyRangeInternal { + TVector<TExprBase> ExactPart; + TMaybeNode<TExprBase> LowerLeaf; + TMaybeNode<TExprBase> UpperLeaf; + bool LowerInclude = false; + bool UpperInclude = false; + + bool IsFullScan() const { + return ExactPart.empty() && !LowerLeaf && !UpperLeaf; + } + + bool IsExact() const { + return (!ExactPart.empty() && !LowerLeaf && !UpperLeaf) + || (LowerLeaf.Maybe<TCoNull>() && UpperLeaf.Maybe<TCoNull>()) + || (LowerLeaf.IsValid() && UpperLeaf.IsValid() && LowerInclude + && ((UpperInclude && 0 == CompareDataNodes(LowerLeaf.Cast(), UpperLeaf.Cast())) + || (!UpperInclude && AdjacentDataNodes(LowerLeaf.Cast(), UpperLeaf.Cast()))) + ); + } + + bool IsLowerInf() const { + return ExactPart.empty() && !LowerLeaf; + } + + bool IsUpperInf() const { + return ExactPart.empty() && !UpperLeaf; + } + + void NormalizeExact() { + if (LowerLeaf && UpperLeaf) { + ExactPart.push_back(LowerLeaf.Cast()); + LowerLeaf = UpperLeaf = {}; + } + } + + TYtRangesInfo::TKeyRange ToKeyRange() const { + TYtRangesInfo::TKeyRange range; + YQL_ENSURE(LowerLeaf || UpperLeaf); + range.Lower.assign(ExactPart.begin(), ExactPart.end()); + range.Upper.assign(ExactPart.begin(), ExactPart.end()); + if (LowerLeaf) { + range.Lower.push_back(LowerLeaf.Cast()); + } + if (UpperLeaf) { + range.Upper.push_back(UpperLeaf.Cast()); + } + range.LowerInclude = LowerInclude || !LowerLeaf; + range.UpperInclude = (UpperInclude && UpperLeaf) || (!UpperLeaf && !ExactPart.empty()); + return range; + } +}; + +template <bool leftLower, bool rightLower> +bool CompareKeyRange(const TKeyRangeInternal& l, const TKeyRangeInternal& r) { + if (leftLower && l.IsLowerInf()) { + // Left lower is -inf, always less than right + return !rightLower || !r.IsLowerInf(); + } + if (!leftLower && l.IsUpperInf()) { + // Left upper is +inf, always greater than right + return false; + } + if (rightLower && r.IsLowerInf()) { + // Right lower is -inf, always less than left + return false; + } + if (!rightLower && r.IsUpperInf()) { + // Right upper is +inf, always greater than left + return leftLower || !l.IsUpperInf(); + } + auto& lBound = leftLower ? l.LowerLeaf : l.UpperLeaf; + auto& rBound = rightLower ? r.LowerLeaf : r.UpperLeaf; + int cmp = 0; + for (size_t i = 0; i < Min(l.ExactPart.size(), r.ExactPart.size()) && 0 == cmp; ++i) { + cmp = CompareDataNodes(l.ExactPart[i], r.ExactPart[i]); + } + if (0 == cmp) { + if (l.ExactPart.size() < r.ExactPart.size() && lBound) { + cmp = CompareDataNodes(lBound.Cast(), r.ExactPart[l.ExactPart.size()]); + } + else if (l.ExactPart.size() > r.ExactPart.size() && rBound) { + cmp = CompareDataNodes(l.ExactPart[r.ExactPart.size()], rBound.Cast()); + } + else if (lBound && rBound) { + cmp = CompareDataNodes(lBound.Cast(), rBound.Cast()); + } + } + if (cmp < 0) { + return true; + } + else if (cmp > 0) { + return false; + } + + if (l.ExactPart.size() == r.ExactPart.size()) { + if (leftLower && !l.LowerLeaf) { + // Left lower is -inf, always less than right + return !rightLower || r.LowerLeaf; + } + if (!leftLower && !l.UpperLeaf) { + // Left upper is +inf, always greater than right + return false; + } + if (rightLower && !r.LowerLeaf) { + // Right lower is -inf, always less than left + return false; + } + if (!rightLower && !r.UpperLeaf) { + // Right upper is +inf, always greater than left + return leftLower || l.UpperLeaf; + } + } + + auto lInclude = leftLower ? !l.LowerInclude : l.UpperInclude; + auto rInclude = rightLower ? !r.LowerInclude : r.UpperInclude; + + if (rightLower) { + return lInclude == rInclude + ? (l.ExactPart.size() + lBound.IsValid()) < (r.ExactPart.size() + rBound.IsValid()) + : lInclude < rInclude; + } else { + return lInclude == rInclude + ? (l.ExactPart.size() + lBound.IsValid()) > (r.ExactPart.size() + rBound.IsValid()) + : lInclude > rInclude; + } +} + +void DeduplicateRanges(TVector<TKeyRangeInternal>& ranges) { + std::stable_sort(ranges.begin(), ranges.end(), CompareKeyRange<true, true>); + + size_t i = 0; + while (i + 1 < ranges.size()) { + TKeyRangeInternal& current = ranges[i]; + TKeyRangeInternal& next = ranges[i + 1]; + if (!CompareKeyRange<false, true>(current, next)) { // current.Upper >= next.Lower + if (CompareKeyRange<false, false>(current, next)) { // current.Upper < next.Upper + YQL_ENSURE(current.ExactPart.size() == next.ExactPart.size()); + DoSwap(current.UpperLeaf, next.UpperLeaf); + DoSwap(current.UpperInclude, next.UpperInclude); + } + ranges.erase(ranges.begin() + i + 1); + if (ranges[i].IsFullScan()) { + ranges.clear(); + return; + } + } + else { + ++i; + } + } +} + +TVector<NNodes::TExprBase> ConvertBoundaryNode(const TExprNode& boundary, bool& isIncluded) { + isIncluded = false; + YQL_ENSURE(boundary.IsList()); + TExprNodeList items = boundary.ChildrenList(); + YQL_ENSURE(items.size() >= 2); + + YQL_ENSURE(items.back()->IsCallable("Int32")); + isIncluded = FromString<i32>(items.back()->Head().Content()) != 0; + items.pop_back(); + + TVector<NNodes::TExprBase> result; + for (auto item : items) { + if (item->IsCallable("Nothing")) { + break; + } + YQL_ENSURE(item->IsCallable("Just")); + item = item->HeadPtr(); + + result.push_back(NNodes::TExprBase(std::move(item))); + YQL_ENSURE(result.back().Maybe<TCoDataCtor>() || + result.back().Maybe<TCoNothing>() || + result.back().Maybe<TCoJust>()); + } + + return result; +} + +TExprBase RoundTz(const TExprBase& node, bool down, TExprContext& ctx) { + if (auto maybeTz = node.Maybe<TCoTzDateBase>()) { + TStringBuf tzName = maybeTz.Cast().Literal().Value(); + TStringBuf value; + GetNext(tzName, ',', value); + + const auto& names = NUdf::GetTimezones(); + YQL_ENSURE(!names.empty()); + + TStringBuf targetName; + if (down) { + targetName = names.front(); + } else { + for (auto it = names.rbegin(); it != names.rend(); ++it) { + if (!it->empty()) { + targetName = *it; + break; + } + } + } + YQL_ENSURE(!tzName.empty()); + YQL_ENSURE(!targetName.empty()); + + if (tzName != targetName) { + return TExprBase(ctx.Builder(node.Pos()) + .Callable(node.Ref().Content()) + .Atom(0, TStringBuilder() << value << "," << targetName) + .Seal() + .Build()); + } + } + return node; +} + +TMaybe<TYtRangesInfo::TKeyRange> WidenTzKeys(const TYtRangesInfo::TKeySingle& single, TExprContext& ctx) { + if (AllOf(single.Key, [](const auto& key){ return !TCoTzDateBase::Match(key.Raw()); })) { + return {}; + } + TYtRangesInfo::TKeyRange result; + result.LowerInclude = result.UpperInclude = true; + for (auto& key : single.Key) { + result.Lower.push_back(RoundTz(key, true, ctx)); + result.Upper.push_back(RoundTz(key, false, ctx)); + } + return result; +} + +void WidenTzKeys(TYtRangesInfo::TKeyRange& range, TExprContext& ctx) { + for (auto& key : range.Lower) { + key = RoundTz(key, true, ctx); + } + for (auto& key : range.Upper) { + key = RoundTz(key, false, ctx); + } +} + +} // unnamed + +TYtRangesInfo::TPtr TYtRangesInfo::ApplyLegacyKeyFilters(const TVector<TExprBase>& keyFilters, + const TYqlRowSpecInfo::TPtr& rowSpec, TExprContext& ctx) +{ + YQL_ENSURE(rowSpec && rowSpec->IsSorted()); + TVector<TKeyRangeInternal> ranges; + for (auto andGrp: keyFilters) { + bool emptyMatch = false; + TKeyRangeInternal key; + size_t memberIndex = 0; + for (auto keyPredicates: andGrp.Cast<TCoNameValueTupleList>()) { + YQL_ENSURE(memberIndex < rowSpec->SortMembers.size() && rowSpec->SortMembers[memberIndex] == keyPredicates.Name().Value()); + const TTypeAnnotationNode* columnType = rowSpec->SortedByTypes[memberIndex]; + if (columnType->GetKind() == ETypeAnnotationKind::Optional) { + columnType = columnType->Cast<TOptionalExprType>()->GetItemType(); + } + auto columnDataSlot = columnType->Cast<TDataExprType>()->GetSlot(); + ++memberIndex; + + key.NormalizeExact(); + key.LowerLeaf = key.UpperLeaf = {}; + key.LowerInclude = false; + key.UpperInclude = false; + + auto isWithinRange = [&key](TExprBase value, bool includeBounds) -> int { + if (key.UpperLeaf) { + int cmp = CompareDataNodes(value, key.UpperLeaf.Cast()); + if (cmp > 0 || (cmp == 0 && !key.UpperInclude && includeBounds)) { + return 1; // out of range: greater than upper bound + } + } + if (key.LowerLeaf) { + int cmp = CompareDataNodes(key.LowerLeaf.Cast(), value); + if (cmp > 0 || (cmp == 0 && !key.LowerInclude && includeBounds)) { + return -1; // out of range: less than lower bound + } + } + return 0; + }; + + for (auto cmp: keyPredicates.Value().Cast<TCoNameValueTupleList>()) { + auto operation = cmp.Name().Value(); + TExprBase value = cmp.Value().Cast(); + if (value.Maybe<TCoNothing>()) { + emptyMatch = true; + break; + } + if (value.Maybe<TCoNull>()) { + if ((key.LowerLeaf.IsValid() && !key.LowerLeaf.Maybe<TCoNull>()) + || (key.UpperLeaf.IsValid() && !key.UpperLeaf.Maybe<TCoNull>())) + { + emptyMatch = true; + break; + } + key.LowerLeaf = key.UpperLeaf = value; + key.LowerInclude = key.UpperInclude = true; + } + else { + if (key.LowerLeaf.Maybe<TCoNull>() || key.UpperLeaf.Maybe<TCoNull>()) { + emptyMatch = true; + break; + } + + if (value.Maybe<TCoJust>()) { + value = value.Cast<TCoJust>().Input(); + } + + if (operation == TStringBuf("<") || operation == TStringBuf("<=")) { + bool includeBound = (operation == TStringBuf("<=")); + int cmp = isWithinRange(value, includeBound); + if (cmp < 0) /* lower than existing lower bound yields an empty match */ { + emptyMatch = true; + break; + } + else if (cmp == 0) { + key.UpperLeaf = value; + key.UpperInclude = includeBound; + } + } + else if (operation == TStringBuf(">") || operation == TStringBuf(">=")) { + bool includeBound = (operation == TStringBuf(">=")); + int cmp = isWithinRange(value, includeBound); + if (cmp > 0) /* upper than existing upper bound yields an empty match */ { + emptyMatch = true; + break; + } + else if (cmp == 0) { + key.LowerLeaf = value; + key.LowerInclude = includeBound; + } + } + else if (operation == TStringBuf("==")) { + int cmp = isWithinRange(value, true); + if (cmp != 0) /* value out of the bounds */ { + emptyMatch = true; + break; + } + else { + key.UpperLeaf = key.LowerLeaf = value; + key.UpperInclude = key.LowerInclude = true; + } + } + else if (operation == TStringBuf("StartsWith")) { + int cmp = isWithinRange(value, true); + if (cmp != 0) /* value out of the bounds */ { + emptyMatch = true; + break; + } + + key.LowerLeaf = value; + key.LowerInclude = true; + YQL_ENSURE(value.Ref().IsCallable({"String", "Utf8"})); + if (auto content = TString{key.LowerLeaf.Cast<TCoDataCtor>().Literal().Value()}; !content.empty()) { + std::optional<std::string> incremented; + if (columnDataSlot == EDataSlot::String) { + key.LowerLeaf = value = TExprBase(ctx.RenameNode(value.Ref(), "String")); + incremented = NextLexicographicString(content); + } else { + YQL_ENSURE(columnDataSlot == EDataSlot::Utf8); + if (IsUtf8(content)) { + incremented = NextValidUtf8(content); + } else { + emptyMatch = true; + break; + } + } + + if (incremented) { + key.UpperLeaf = ctx.ChangeChildren(value.Ref(), {ctx.NewAtom(value.Ref().Head().Pos(), *incremented)}); + key.UpperInclude = false; + } + } + } + } + } + + if (!emptyMatch && key.LowerLeaf.Maybe<TCoDataCtor>()) { + auto lowerDataSlot = GetDataSlot(key.LowerLeaf.Cast().Ref().Content()); + if (GetDataTypeInfo(lowerDataSlot).Features & (NUdf::NumericType | NUdf::DateType)) { + TString newLowerValue = TString{key.LowerLeaf.Cast<TCoDataCtor>().Literal().Value()}; + if (!AdjustLowerValue(newLowerValue, key.LowerInclude, lowerDataSlot, columnDataSlot)) { + emptyMatch = true; + } + else if (columnDataSlot != lowerDataSlot + || newLowerValue != key.LowerLeaf.Cast<TCoDataCtor>().Literal().Value()) + { + key.LowerLeaf = TExprBase(ctx.Builder(key.LowerLeaf.Cast().Pos()) + .Callable(GetDataTypeInfo(columnDataSlot).Name) + .Atom(0, newLowerValue) + .Seal() + .Build()); + } + } + } + + if (!emptyMatch && key.UpperLeaf.Maybe<TCoDataCtor>()) { + auto upperDataSlot = GetDataSlot(key.UpperLeaf.Cast().Ref().Content()); + if (GetDataTypeInfo(upperDataSlot).Features & (NUdf::NumericType | NUdf::DateType)) { + TString newUpperValue = TString{key.UpperLeaf.Cast<TCoDataCtor>().Literal().Value()}; + if (!AdjustUpperValue(newUpperValue, key.UpperInclude, upperDataSlot, columnDataSlot)) { + emptyMatch = true; + } + else if (columnDataSlot != upperDataSlot + || newUpperValue != key.UpperLeaf.Cast<TCoDataCtor>().Literal().Value()) + { + key.UpperLeaf = TExprBase(ctx.Builder(key.UpperLeaf.Cast().Pos()) + .Callable(GetDataTypeInfo(columnDataSlot).Name) + .Atom(0, newUpperValue) + .Seal() + .Build()); + } + } + } + + if (emptyMatch) { + break; + } + + if (!key.IsExact()) { + break; + } + } + + if (!emptyMatch) { + if (key.IsFullScan()) { + return TYtRangesInfo::TPtr(); + } + + ranges.push_back(std::move(key)); + } + } + + TYtRangesInfo::TPtr rangesInfo = MakeIntrusive<TYtRangesInfo>(); + if (!ranges.empty()) { + if (ranges.size() > 1) { + DeduplicateRanges(ranges); + if (ranges.empty()) { + return TYtRangesInfo::TPtr(); // Full scan + } + } + for (TKeyRangeInternal& range: ranges) { + if (range.IsExact()) { + range.NormalizeExact(); + TKeySingle single{std::move(range.ExactPart)}; + auto maybeKeyRange = WidenTzKeys(single, ctx); + if (maybeKeyRange) { + rangesInfo->Ranges.emplace_back(*maybeKeyRange); + } else { + rangesInfo->Ranges.emplace_back(std::move(single)); + } + } + else { + TKeyRange r = range.ToKeyRange(); + WidenTzKeys(r, ctx); + rangesInfo->Ranges.emplace_back(std::move(r)); + } + } + } + return rangesInfo; +} + +TYtRangesInfo::TPtr TYtRangesInfo::ApplyKeyFilter(const TExprNode& keyFilter) { + // keyFilter is List<Tuple<Left, Right>>, Left/Right are key boundaries + if (keyFilter.IsCallable("List")) { + return MakeEmptyRange(); + } + + YQL_ENSURE(keyFilter.IsCallable("AsList")); + auto res = MakeIntrusive<TYtRangesInfo>(); + for (auto& interval : keyFilter.ChildrenList()) { + YQL_ENSURE(interval->IsList()); + YQL_ENSURE(interval->ChildrenSize() == 2); + + TKeyRange range; + range.Lower = ConvertBoundaryNode(interval->Head(), range.LowerInclude); + if (range.Lower.empty()) { + // just a convention + range.LowerInclude = true; + } + range.Upper = ConvertBoundaryNode(interval->Tail(), range.UpperInclude); + + if (range.Lower.empty() && range.Upper.empty()) { + // full range + return {}; + } + + bool sameBounds = range.Lower.size() == range.Upper.size(); + for (size_t i = 0; sameBounds && i < range.Lower.size(); ++i) { + sameBounds = range.Lower[i].Raw() == range.Upper[i].Raw(); + } + + if (sameBounds) { + YQL_ENSURE(range.LowerInclude && range.UpperInclude, "Not expecting empty regions here"); + res->Ranges.emplace_back(TKeySingle{std::move(range.Lower)}); + } else { + res->Ranges.emplace_back(std::move(range)); + } + } + + return res; +} + +TYtRangesInfo::TPtr TYtRangesInfo::MakeEmptyRange() { + return MakeIntrusive<TYtRangesInfo>(); +} + +///////////////////////////////////////////////////////////////////////////////////////////////////////// + +bool TYtColumnsInfo::Validate(TExprNode& node, TExprContext& ctx) { + if (!node.IsCallable(TCoVoid::CallableName())) { + if (!EnsureTuple(node, ctx)) { + return false; + } + THashSet<TStringBuf> columns; + THashSet<TStringBuf> renameFrom; + THashSet<TStringBuf> renameTo; + + for (auto& child: node.Children()) { + TStringBuf name; + TStringBuf type; + TStringBuf originalName; + + if (child->Type() == TExprNode::Atom) { + name = child->Content(); + } else { + if (!EnsureTupleMinSize(*child, 2, ctx)) { + return false; + } + if (!EnsureAtom(*child->Child(0), ctx)) { + return false; + } + if (!EnsureAtom(*child->Child(1), ctx)) { + return false; + } + + name = child->Child(0)->Content(); + type = child->Child(1)->Content(); + + if (type == "weak") { + if (!EnsureTupleSize(*child, 2, ctx)) { + return false; + } + } else if (type == "rename") { + if (!EnsureTupleSize(*child, 3, ctx)) { + return false; + } + originalName = child->Child(2)->Content(); + } else { + ctx.AddError(TIssue(ctx.GetPosition(child->Pos()), TStringBuilder() << "Unknown column type: " << type)); + return false; + } + } + + if (type != "rename") { + if (!columns.insert(name).second) { + ctx.AddError(TIssue(ctx.GetPosition(child->Pos()), TStringBuilder() << "Duplicate column name: " << name)); + return false; + } + } else { + if (!renameFrom.insert(originalName).second) { + ctx.AddError(TIssue(ctx.GetPosition(child->Pos()), TStringBuilder() << "Duplicate original column name: " << originalName)); + return false; + } + if (renameTo.find(originalName) != renameTo.end()) { + ctx.AddError(TIssue(ctx.GetPosition(child->Pos()), TStringBuilder() << "Rename chain detected with original column name: " << originalName)); + return false; + } + + if (!renameTo.insert(name).second) { + ctx.AddError(TIssue(ctx.GetPosition(child->Pos()), TStringBuilder() << "Duplicate target column name: " << name)); + return false; + } + if (renameFrom.find(name) != renameFrom.end()) { + ctx.AddError(TIssue(ctx.GetPosition(child->Pos()), TStringBuilder() << "Rename chain detected with target column name: " << name)); + return false; + } + } + } + } + return true; +} + +void TYtColumnsInfo::Parse(NNodes::TExprBase node) { + Renames = {}; + Columns = {}; + Others = false; + if (node.Maybe<TExprList>()) { + for (auto child: node.Cast<TExprList>()) { + TColumn column; + if (child.Maybe<TCoAtom>()) { + column.Name = child.Cast<TCoAtom>().Value(); + if (!Columns) { + Columns.ConstructInPlace(); + } + Columns->emplace_back(column); + UpdateOthers(column.Name); + continue; + } + auto tuple = child.Cast<TCoAtomList>(); + column.Name = tuple.Item(0).Value(); + column.Type = tuple.Item(1).Value(); + if (tuple.Ref().ChildrenSize() == 2) { + if (!Columns) { + Columns.ConstructInPlace(); + } + Columns->emplace_back(column); + UpdateOthers(column.Name); + continue; + } + + auto from = tuple.Item(2).Value(); + if (!Renames) { + Renames.ConstructInPlace(); + } + (*Renames)[from] = column.Name; + } + + if (!Columns && !Renames) { + Columns.ConstructInPlace(); + } + } +} + +NNodes::TExprBase TYtColumnsInfo::ToExprNode(TExprContext& ctx, const TPositionHandle& pos, const THashSet<TString>* filter) const { + if (!Columns && !Renames) { + return Build<TCoVoid>(ctx, pos).Done(); + } + + auto listBuilder = Build<TExprList>(ctx, pos); + if (Renames) { + for (auto& r: *Renames) { + listBuilder + .Add<TCoAtomList>() + .Add() + .Value(r.second) + .Build() + .Add() + .Value("rename", TNodeFlags::Default) + .Build() + .Add() + .Value(r.first) + .Build() + .Build(); + } + } + + if (Columns) { + for (auto& c: *Columns) { + if (filter && !filter->contains(c.Name)) { + continue; + } + if (c.Type == "weak") { + listBuilder + .Add<TCoAtomList>() + .Add() + .Value(c.Name) + .Build() + .Add() + .Value(c.Type, TNodeFlags::Default) + .Build() + .Build(); + } else { + listBuilder + .Add<TCoAtom>() + .Value(c.Name) + .Build(); + } + } + } + + return listBuilder.Done(); +} + +void TYtColumnsInfo::UpdateOthers(TStringBuf col) { + if (!Others) { + Others = (col == YqlOthersColumnName); + } +} + +void TYtColumnsInfo::FillRichYPath(NYT::TRichYPath& path, bool withColumns) const { + if (Columns && withColumns) { + TVector<TString> columns; + for (auto& c: *Columns) { + columns.push_back(c.Name); + } + path.Columns(columns); + } + + if (Renames) { + path.RenameColumns(*Renames); + } +} + + +///////////////////////////////////////////////////////////////////////////////////////////////////////// + +bool TYtPathInfo::Validate(const TExprNode& node, TExprContext& ctx) { + if (!node.IsCallable(TYtPath::CallableName())) { + ctx.AddError(TIssue(ctx.GetPosition(node.Pos()), TStringBuilder() << "Expected " << TYtPath::CallableName())); + return false; + } + if (!EnsureMinMaxArgsCount(node, 4, 5, ctx)) { + return false; + } + + if (!node.Child(TYtPath::idx_Table)->IsCallable(TYtTable::CallableName()) + && !node.Child(TYtPath::idx_Table)->IsCallable(TYtOutput::CallableName()) + && !node.Child(TYtPath::idx_Table)->IsCallable(TYtOutTable::CallableName())) { + ctx.AddError(TIssue(ctx.GetPosition(node.Child(TYtPath::idx_Table)->Pos()), TStringBuilder() << "Unexpected " + << node.Child(TYtPath::idx_Table)->Content())); + return false; + } + + if (!TYtColumnsInfo::Validate(*node.Child(TYtPath::idx_Columns), ctx)) { + return false; + } + + if (!node.Child(TYtPath::idx_Ranges)->IsCallable(TCoVoid::CallableName())) { + TYqlRowSpecInfo::TPtr rowSpec = TYtTableBaseInfo::GetRowSpec(TExprBase(node.ChildPtr(TYtPath::idx_Table))); + TYtTableMetaInfo::TPtr meta = TYtTableBaseInfo::GetMeta(TExprBase(node.ChildPtr(TYtPath::idx_Table))); + if (!TYtRangesInfo::Validate(*node.Child(TYtPath::idx_Ranges), ctx, meta && meta->DoesExist, rowSpec)) { + return false; + } + } + + if (!node.Child(TYtPath::idx_Stat)->IsCallable(TYtStat::CallableName()) + && !node.Child(TYtPath::idx_Stat)->IsCallable(TCoVoid::CallableName())) { + + ctx.AddError(TIssue(ctx.GetPosition(node.Child(TYtPath::idx_Stat)->Pos()), TStringBuilder() + << "Expected " << TYtStat::CallableName() + << " or Void")); + return false; + } + + if (node.ChildrenSize() > TYtPath::idx_AdditionalAttributes && !EnsureAtom(*node.Child(TYtPath::idx_AdditionalAttributes), ctx)) { + return false; + } + + return true; +} + +void TYtPathInfo::Parse(TExprBase node) { + *this = {}; + FromNode = node.Maybe<TExprBase>(); + TYtPath path = node.Cast<TYtPath>(); + Table = TYtTableBaseInfo::Parse(path.Table()); + + if (path.Columns().Maybe<TExprList>()) { + Columns = MakeIntrusive<TYtColumnsInfo>(path.Columns()); + } + + if (path.Ranges().Maybe<TExprList>()) { + Ranges = MakeIntrusive<TYtRangesInfo>(path.Ranges()); + } + + if (path.Stat().Maybe<TYtStat>()) { + Stat = MakeIntrusive<TYtTableStatInfo>(path.Stat().Ptr()); + } + if (path.AdditionalAttributes().Maybe<TCoAtom>()) { + AdditionalAttributes = path.AdditionalAttributes().Cast().Value(); + } +} + +TExprBase TYtPathInfo::ToExprNode(TExprContext& ctx, const TPositionHandle& pos, NNodes::TExprBase table) const { + auto pathBuilder = Build<TYtPath>(ctx, pos); + pathBuilder.Table(table); + if (Columns) { + pathBuilder.Columns(Columns->ToExprNode(ctx, pos)); + } else { + pathBuilder.Columns<TCoVoid>().Build(); + } + if (Ranges) { + pathBuilder.Ranges(Ranges->ToExprNode(ctx, pos)); + } else { + pathBuilder.Ranges<TCoVoid>().Build(); + } + if (Stat) { + pathBuilder.Stat(Stat->ToExprNode(ctx, pos)); + } else { + pathBuilder.Stat<TCoVoid>().Build(); + } + if (AdditionalAttributes) { + pathBuilder.AdditionalAttributes<TCoAtom>() + .Value(*AdditionalAttributes, TNodeFlags::MultilineContent) + .Build(); + } + + return pathBuilder.Done(); +} + +void TYtPathInfo::FillRichYPath(NYT::TRichYPath& path) const { + if (AdditionalAttributes) { + DeserializeRichYPathAttrs(*AdditionalAttributes, path); + } + if (Columns) { + // Should have the same criteria as in TYtPathInfo::GetCodecSpecNode() + bool useAllColumns = !Table->RowSpec; // Always use all columns for YAMR format + if (Table->RowSpec && !Table->RowSpec->StrictSchema && Columns->HasOthers()) { + useAllColumns = true; + } + + Columns->FillRichYPath(path, /* withColumns = */ !useAllColumns); + } + if (Ranges) { + YQL_ENSURE(Table); + YQL_ENSURE(Table->RowSpec); + Ranges->FillRichYPath(path, Table->RowSpec->SortedBy.size()); + } +} + +IGraphTransformer::TStatus TYtPathInfo::GetType(const TTypeAnnotationNode*& filtered, TExprNode::TPtr& newFields, TExprContext& ctx, const TPositionHandle& pos) const { + YQL_ENSURE(Table, "TYtPathInfo::Parse() must be called"); + + filtered = nullptr; + newFields = {}; + const TStructExprType* inputType = Table->FromNode.Ref().GetTypeAnn()->Cast<TListExprType>()->GetItemType()->Cast<TStructExprType>(); + if (!Columns) { + filtered = inputType; + return IGraphTransformer::TStatus::Ok; + } else { + auto& renames = Columns->GetRenames(); + if (renames) { + auto inputTypeItems = inputType->GetItems(); + for (auto& r : *renames) { + auto from = r.first; + auto to = r.second; + + auto idx = inputType->FindItem(from); + if (!idx) { + ctx.AddError(TIssue(ctx.GetPosition(pos), TStringBuilder() << "Unknown original member: " << from)); + return IGraphTransformer::TStatus::Error; + } + inputTypeItems[*idx] = ctx.MakeType<TItemExprType>(to, inputTypeItems[*idx]->GetItemType()); + } + YQL_ENSURE(!inputTypeItems.empty()); + inputType = ctx.MakeType<TStructExprType>(inputTypeItems); + } + + auto& columns = Columns->GetColumns(); + if (!columns) { + filtered = inputType; + return IGraphTransformer::TStatus::Ok; + } + + THashSet<TString> auxColumns; + if (Table->RowSpec) { + for (auto& aux: Table->RowSpec->GetAuxColumns()) { + auxColumns.insert(aux.first); + } + } + bool hasMissingColumns = false; + THashSet<TString> fieldNames; + for (auto& item: *columns) { + if (inputType->FindItem(item.Name) || item.Type == "weak") { + fieldNames.insert(item.Name); + } else if (!auxColumns.contains(item.Name)) { + hasMissingColumns = true; + } + } + + if (hasMissingColumns) { + newFields = Columns->ToExprNode(ctx, pos, &fieldNames).Ptr(); + return IGraphTransformer::TStatus::Repeat; + } + + TVector<const TItemExprType*> items; + for (auto& item : inputType->GetItems()) { + if (fieldNames.contains(item->GetName())) { + items.push_back(item); + } + } + auto weakType = ctx.MakeType<TOptionalExprType>(ctx.MakeType<TDataExprType>(EDataSlot::Yson)); + for (auto& item: *columns) { + if (!inputType->FindItem(item.Name) && item.Type == "weak") { + items.push_back(ctx.MakeType<TItemExprType>(item.Name, weakType)); + } + } + + auto itemType = ctx.MakeType<TStructExprType>(items); + if (!itemType->Validate(pos, ctx)) { + return IGraphTransformer::TStatus::Error; + } + filtered = itemType; + return IGraphTransformer::TStatus::Ok; + } +} + +const NCommon::TStructMemberMapper& TYtPathInfo::GetColumnMapper() { + if (Mapper_) { + return *Mapper_; + } + THashSet<TStringBuf> filterColumns; + THashMap<TString, TString> renames; + if (Columns && Columns->GetRenames()) { + renames = *Columns->GetRenames(); + } + + bool useAllColumns = true; // Should have the same criteria as in TYtPathInfo::FillRichYPath() + if (HasColumns()) { + useAllColumns = !Table->RowSpec; // Always use all columns for YAMR format + if (Table->RowSpec && !Table->RowSpec->StrictSchema && Columns->HasOthers()) { + useAllColumns = true; + } + + if (!useAllColumns) { + for (auto& c: *Columns->GetColumns()) { + filterColumns.insert(c.Name); + } + } + } + if (useAllColumns && renames.empty()) { + Mapper_.ConstructInPlace(); + return *Mapper_; + } + + Mapper_ = [renames, filterColumns, useAllColumns, strict = Table->RowSpec->StrictSchema](TStringBuf name) -> TMaybe<TStringBuf> { + if (auto r = renames.find(name); r != renames.cend()) { + name = r->second; + } + if (strict && !useAllColumns && !filterColumns.contains(name)) { + return Nothing(); + } + if (!strict && name == YqlOthersColumnName) { + return Nothing(); + } + return MakeMaybe(name); + }; + return *Mapper_; +} + +NYT::TNode TYtPathInfo::GetCodecSpecNode() { + auto specNode = Table->GetCodecSpecNode(GetColumnMapper()); + if (Table->RowSpec && !Table->RowSpec->StrictSchema && HasColumns()) { + auto list = NYT::TNode::CreateList(); + for (auto& col: *Columns->GetColumns()) { + if (col.Type == "weak") { + list.Add(col.Name); + } + } + if (!list.AsList().empty()) { + specNode[YqlRowSpecAttribute][RowSpecAttrWeakFields] = list; + } + } + + return specNode; +} + +bool TYtPathInfo::RequiresRemap() const { + if (!Table->RequiresRemap()) { + return false; + } + const auto meta = Table->Meta; + const auto rowSpec = Table->RowSpec; + if (!meta->Attrs.contains(QB2Premapper) && rowSpec && rowSpec->DefaultValues.empty() && !rowSpec->StrictSchema) { + if (HasColumns() && !Columns->HasOthers()) { + return false; + } + } + + return true; +} + +TString TYtPathInfo::GetCodecSpecStr() { + return NYT::NodeToCanonicalYsonString(GetCodecSpecNode(), NYson::EYsonFormat::Text); +} + +ui64 TYtPathInfo::GetNativeYtTypeFlags() { + return Table->RowSpec ? Table->RowSpec->GetNativeYtTypeFlags(GetColumnMapper()) : 0; +} + +TMaybe<NYT::TNode> TYtPathInfo::GetNativeYtType() { + return Table->RowSpec ? Table->RowSpec->GetNativeYtType(GetColumnMapper()) : Nothing(); +} + +///////////////////////////////////////////////////////////////////////////////////////////////////////// + +} // NYql diff --git a/yt/yql/providers/yt/provider/yql_yt_table.h b/yt/yql/providers/yt/provider/yql_yt_table.h new file mode 100644 index 0000000000..9675ce7c74 --- /dev/null +++ b/yt/yql/providers/yt/provider/yql_yt_table.h @@ -0,0 +1,356 @@ +#pragma once + +#include "yql_yt_op_settings.h" + +#include <yt/yql/providers/yt/lib/row_spec/yql_row_spec.h> +#include <yt/yql/providers/yt/common/yql_configuration.h> +#include <yql/essentials/providers/common/schema/expr/yql_expr_schema.h> +#include <yql/essentials/core/yql_graph_transformer.h> +#include <yql/essentials/core/expr_nodes_gen/yql_expr_nodes_gen.h> +#include <yql/essentials/ast/yql_expr.h> + +#include <yt/cpp/mapreduce/interface/node.h> +#include <yt/cpp/mapreduce/interface/common.h> + +#include <util/generic/ptr.h> +#include <util/generic/string.h> +#include <util/generic/strbuf.h> +#include <util/generic/vector.h> +#include <util/generic/variant.h> +#include <util/generic/maybe.h> +#include <util/generic/hash_set.h> +#include <util/string/cast.h> + +#include <utility> + +namespace NYql { + +class TYtKey; + +struct TYtTableStatInfo: public TThrRefBase { + using TPtr = TIntrusivePtr<TYtTableStatInfo>; + + TYtTableStatInfo() { + } + + TYtTableStatInfo(NNodes::TExprBase node) { + Parse(node); + } + TYtTableStatInfo(TExprNode::TPtr node) + : TYtTableStatInfo(NNodes::TExprBase(node)) + { + } + + static bool Validate(const TExprNode& node, TExprContext& ctx); + void Parse(NNodes::TExprBase node); + NNodes::TExprBase ToExprNode(TExprContext& ctx, const TPositionHandle& pos) const; + + bool IsEmpty() const { + return 0 == RecordsCount; + } + + NNodes::TMaybeNode<NNodes::TExprBase> FromNode; + TString Id; + ui64 RecordsCount = 0; + ui64 DataSize = 0; + ui64 ChunkCount = 0; + ui64 ModifyTime = 0; + ui64 Revision = 0; + TSet<TString> SecurityTags = {}; + ui64 TableRevision = 0; // Not serializable +}; + +struct TYtTableMetaInfo: public TThrRefBase { + using TPtr = TIntrusivePtr<TYtTableMetaInfo>; + + TYtTableMetaInfo() { + } + TYtTableMetaInfo(NNodes::TExprBase node) { + Parse(node); + } + TYtTableMetaInfo(TExprNode::TPtr node) + : TYtTableMetaInfo(NNodes::TExprBase(node)) + { + } + + static bool Validate(const TExprNode& node, TExprContext& ctx); + void Parse(NNodes::TExprBase node); + NNodes::TExprBase ToExprNode(TExprContext& ctx, const TPositionHandle& pos) const; + + NNodes::TMaybeNode<NNodes::TExprBase> FromNode; + bool CanWrite = true; + bool DoesExist = false; + bool YqlCompatibleScheme = false; + bool InferredScheme = false; + bool IsDynamic = false; + TString SqlView; + ui16 SqlViewSyntaxVersion = 1; + + THashMap<TString, TString> Attrs; +}; + +struct TEpochInfo { + static bool Validate(const TExprNode& node, TExprContext& ctx); + static TMaybe<ui32> Parse(const TExprNode& node); + static NNodes::TExprBase ToExprNode(const TMaybe<ui32>& epoch, TExprContext& ctx, const TPositionHandle& pos); +}; + +struct TYtTableBaseInfo: public TThrRefBase { + using TPtr = TIntrusivePtr<TYtTableBaseInfo>; + + TYtTableBaseInfo() = default; + TYtTableBaseInfo(const TYtKey& key, TStringBuf cluster); + + virtual NNodes::TExprBase ToExprNode(TExprContext& ctx, const TPositionHandle& pos) const = 0; + + NYT::TNode GetCodecSpecNode(const NCommon::TStructMemberMapper& mapper = {}) const; + NYT::TNode GetAttrSpecNode(ui64 nativeTypeCompatibility, bool rowSpecCompactForm) const; + + static TPtr Parse(NNodes::TExprBase node); + static TYtTableMetaInfo::TPtr GetMeta(NNodes::TExprBase node); + static TYqlRowSpecInfo::TPtr GetRowSpec(NNodes::TExprBase node); + static TYtTableStatInfo::TPtr GetStat(NNodes::TExprBase node); + static TStringBuf GetTableName(NNodes::TExprBase node); + + bool RequiresRemap() const; + bool HasSameScheme(const TTypeAnnotationNode& scheme) const; + bool HasSamePhysicalScheme(const TYtTableBaseInfo& info) const; + + NNodes::TMaybeNode<NNodes::TExprBase> FromNode; + TString Name; + TString Cluster; + TYqlRowSpecInfo::TPtr RowSpec; + TYtTableMetaInfo::TPtr Meta; + TYtTableStatInfo::TPtr Stat; + NNodes::TMaybeNode<NNodes::TExprBase> Settings; + bool IsTemp = false; + bool IsAnonymous = false; + bool IsUnordered = false; + TMaybe<ui32> Epoch; // Epoch, from which the table is read + TMaybe<ui32> CommitEpoch; // Epoch, in which the table modifications became accessible +}; + +struct TYtTableInfo: public TYtTableBaseInfo { + using TPtr = TIntrusivePtr<TYtTableInfo>; + + TYtTableInfo() = default; + TYtTableInfo(const TYtKey& key, TStringBuf cluster); + TYtTableInfo(NNodes::TExprBase node, bool useTypes = true) { + Parse(node, useTypes); + } + TYtTableInfo(TExprNode::TPtr node) + : TYtTableInfo(NNodes::TExprBase(node)) + { + } + + static TStringBuf GetTableLabel(NNodes::TExprBase node); + static bool HasSubstAnonymousLabel(NNodes::TExprBase node); + + static bool Validate(const TExprNode& node, EYtSettingTypes accepted, TExprContext& ctx); + void Parse(NNodes::TExprBase node, bool useTypes = true); + NNodes::TExprBase ToExprNode(TExprContext& ctx, const TPositionHandle& pos) const override; +}; + +struct TYtOutTableInfo: public TYtTableBaseInfo { + using TPtr = TIntrusivePtr<TYtOutTableInfo>; + + TYtOutTableInfo() { + IsTemp = true; + } + TYtOutTableInfo(const TStructExprType* type, ui64 nativeYtTypeFlags); + TYtOutTableInfo(NNodes::TExprBase node) { + Parse(node); + IsTemp = true; + } + TYtOutTableInfo(TExprNode::TPtr node) + : TYtOutTableInfo(NNodes::TExprBase(node)) + { + } + + static bool Validate(const TExprNode& node, TExprContext& ctx); + void Parse(NNodes::TExprBase node); + NNodes::TExprBase ToExprNode(TExprContext& ctx, const TPositionHandle& pos) const override; + + TYtOutTableInfo& SetUnique(const TDistinctConstraintNode* distinct, const TPositionHandle& pos, TExprContext& ctx); + NYT::TNode GetColumnGroups() const; +}; + +struct TYtRangesInfo: public TThrRefBase { + using TPtr = TIntrusivePtr<TYtRangesInfo>; + + struct TRowSingle { + ui64 Offset = Max<ui64>(); + }; + struct TRowRange { + TMaybe<ui64> Lower; + TMaybe<ui64> Upper; + }; + struct TKeySingle { + TVector<NNodes::TExprBase> Key; + }; + struct TKeyRange { + TVector<NNodes::TExprBase> Lower; + bool LowerInclude = true; + TVector<NNodes::TExprBase> Upper; + bool UpperInclude = false; + bool UseKeyBoundApi = DEFAULT_USE_KEY_BOUND_API; + }; + using TRange = std::variant<TRowSingle, TRowRange, TKeySingle, TKeyRange>; + + TYtRangesInfo() { + } + TYtRangesInfo(NNodes::TExprBase node) { + Parse(node); + } + TYtRangesInfo(TExprNode::TPtr node) + : TYtRangesInfo(NNodes::TExprBase(node)) + { + } + + static bool Validate(const TExprNode& node, TExprContext& ctx, bool exists, const TYqlRowSpecInfo::TPtr& rowSpec = {}); + void Parse(NNodes::TExprBase node); + void Parse(const TVector<NYT::TReadRange>& ranges, TExprContext& ctx, const TPositionHandle& pos = {}); + void AddRowRange(const TRowRange& range); + void SetUseKeyBoundApi(bool useKeyBoundApi); + NNodes::TExprBase ToExprNode(TExprContext& ctx, const TPositionHandle& pos) const; + void FillRichYPath(NYT::TRichYPath& path, size_t keyColumnsCount) const; + TMaybe<ui64> GetUsedRows(ui64 tableRowCount) const; + size_t GetRangesCount() const; + size_t GetUsedKeyPrefixLength() const; + bool IsEmpty() const; + TVector<TRange> GetRanges() const; + static TYtRangesInfo::TPtr ApplyLegacyKeyFilters(const TVector<NNodes::TExprBase>& keyFilters, + const TYqlRowSpecInfo::TPtr& rowSpec, TExprContext& ctx); + static TYtRangesInfo::TPtr ApplyKeyFilter(const TExprNode& keyFilter); + static TYtRangesInfo::TPtr MakeEmptyRange(); + +private: + TVector<TRange> Ranges; +}; + +struct TYtColumnsInfo: public TThrRefBase { + using TPtr = TIntrusivePtr<TYtColumnsInfo>; + + struct TColumn { + TString Name; + TString Type; + }; + + TYtColumnsInfo() = default; + + TYtColumnsInfo(NNodes::TExprBase node) + { + Parse(node); + } + + TYtColumnsInfo(TExprNode::TPtr node) + : TYtColumnsInfo(NNodes::TExprBase(node)) + { + } + + static bool Validate(TExprNode& node, TExprContext& ctx); + void Parse(NNodes::TExprBase node); + + NNodes::TExprBase ToExprNode(TExprContext& ctx, const TPositionHandle& pos, const THashSet<TString>* filter = nullptr) const; + + bool HasOthers() const { + return Others; + } + + void FillRichYPath(NYT::TRichYPath& path, bool withColumns) const; + + bool HasColumns() const { + return Columns.Defined(); + } + + template <class TContainer> + void SetColumns(const TContainer& columns) { + Columns.ConstructInPlace(); + Others = false; + for (auto& col: columns) { + Columns->emplace_back(TColumn{::ToString(col), TString()}); + UpdateOthers(col); + } + } + + void SetRenames(const THashMap<TString, TString>& renames) { + YQL_ENSURE(!Renames.Defined()); + if (HasColumns()) { + for (auto& col : *Columns) { + auto r = renames.find(col.Name); + if (r != renames.end()) { + col.Name = r->second; + } + } + } + Renames.ConstructInPlace(renames); + } + + const TMaybe<TVector<TColumn>>& GetColumns() const { + return Columns; + }; + + const TMaybe<THashMap<TString, TString>>& GetRenames() const { + return Renames; + }; + +private: + void UpdateOthers(TStringBuf col); + + TMaybe<THashMap<TString, TString>> Renames; + TMaybe<TVector<TColumn>> Columns; + bool Others = false; +}; + +// TODO: 1) additional fields, which missing in original table 2) field type converting +struct TYtPathInfo: public TThrRefBase { + using TPtr = TIntrusivePtr<TYtPathInfo>; + + TYtPathInfo() { + } + TYtPathInfo(NNodes::TExprBase node) { + Parse(node); + } + TYtPathInfo(TExprNode::TPtr node) + : TYtPathInfo(NNodes::TExprBase(node)) + { + } + + static bool Validate(const TExprNode& node, TExprContext& ctx); + void Parse(NNodes::TExprBase node); + NNodes::TExprBase ToExprNode(TExprContext& ctx, const TPositionHandle& pos, NNodes::TExprBase table) const; + void FillRichYPath(NYT::TRichYPath& path) const; + IGraphTransformer::TStatus GetType(const TTypeAnnotationNode*& filtered, TExprNode::TPtr& newFields, + TExprContext& ctx, const TPositionHandle& pos) const; + + bool HasColumns() const { + return Columns && Columns->HasColumns(); + } + + template <class TContainer> + void SetColumns(const TContainer& columns) { + if (!Columns) { + Columns = MakeIntrusive<TYtColumnsInfo>(); + } + Columns->SetColumns(columns); + } + + NYT::TNode GetCodecSpecNode(); + TString GetCodecSpecStr(); + bool RequiresRemap() const; + ui64 GetNativeYtTypeFlags(); + TMaybe<NYT::TNode> GetNativeYtType(); + + NNodes::TMaybeNode<NNodes::TExprBase> FromNode; + TYtTableBaseInfo::TPtr Table; + TYtColumnsInfo::TPtr Columns; + TYtRangesInfo::TPtr Ranges; + TYtTableStatInfo::TPtr Stat; + TMaybe<TString> AdditionalAttributes; +private: + const NCommon::TStructMemberMapper& GetColumnMapper(); + + TMaybe<NCommon::TStructMemberMapper> Mapper_; +}; + +} diff --git a/yt/yql/providers/yt/provider/yql_yt_table_desc.cpp b/yt/yql/providers/yt/provider/yql_yt_table_desc.cpp new file mode 100644 index 0000000000..1e5db30816 --- /dev/null +++ b/yt/yql/providers/yt/provider/yql_yt_table_desc.cpp @@ -0,0 +1,457 @@ +#include "yql_yt_table_desc.h" + +#include <yt/yql/providers/yt/common/yql_names.h> +#include <yql/essentials/core/issue/protos/issue_id.pb.h> +#include <yql/essentials/core/yql_expr_optimize.h> +#include <yql/essentials/core/yql_expr_type_annotation.h> +#include <yql/essentials/core/qplayer/storage/interface/yql_qstorage.h> +#include <yql/essentials/core/issue/yql_issue.h> +#include <yql/essentials/sql/sql.h> +#include <yql/essentials/utils/yql_panic.h> + +#include <util/generic/scope.h> + +namespace NYql { + +namespace { + +const TString RAW_VIEW_SQL = "select * from self_raw"; +const TString YtView_Component = "YtView"; + +TExprNode::TPtr BuildProtoRemapper(const TMap<TString, TString>& protoFields, TExprContext& ctx) { + auto rowArg = ctx.NewArgument(TPosition(), TStringBuf("row")); + auto result = rowArg; + for (auto& x : protoFields) { + result = ctx.Builder(TPositionHandle()) + .Callable("ReplaceMember") + .Add(0, result) + .Atom(1, x.first) + .Callable(2, "Apply") + .Callable(0, "Udf") + .Atom(0, "Protobuf.Parse") + .Callable(1, "Void") + .Seal() + .Callable(2, "Void") + .Seal() + .Atom(3, x.second) + .Seal() + .Callable(1, "Member") + .Add(0, result) + .Atom(1, x.first) + .Seal() + .Seal() + .Seal() + .Build(); + } + + return ctx.NewLambda(TPosition(), + ctx.NewArguments(TPosition(), { rowArg }), std::move(result)); +} + +TExprNode::TPtr BuildUdfRemapper(const THashMap<TString, TString>& metaAttrs, TExprContext& ctx) { + return ctx.Builder(TPositionHandle()) + .Lambda() + .Param(TStringBuf("row")) + .Callable(TStringBuf("Apply")) + .Callable(0, TStringBuf("Udf")) + .Atom(0, metaAttrs.at(YqlReadUdfAttribute)) + .Do([&](TExprNodeBuilder& parent) -> TExprNodeBuilder& { + if (auto runConfigValue = metaAttrs.FindPtr(YqlReadUdfRunConfigAttribute)) { + parent.Callable(1, TStringBuf("String")) + .Atom(0, *runConfigValue) + .Seal(); + } else { + parent.Callable(1, TStringBuf("Void")) + .Seal(); + } + return parent; + }) + .Callable(2, TStringBuf("Void")) // User type + .Seal() + .Atom(3, metaAttrs.Value(YqlReadUdfTypeConfigAttribute, TString())) + .Seal() + .Arg(1, TStringBuf("row")) + .Seal() + .Seal() + .Build(); +} + +TExprNode::TPtr BuildIgnoreTypeV3Remapper(const TStructExprType* rowType, TExprContext& ctx) { + auto rowArg = ctx.NewArgument(TPosition(), TStringBuf("row")); + auto result = rowArg; + for (const TItemExprType* itemType : rowType->GetItems()) { + auto untaggedType = itemType->GetItemType(); + while (untaggedType->GetKind() == ETypeAnnotationKind::Tagged) { + untaggedType = untaggedType->Cast<TTaggedExprType>()->GetBaseType(); + } + + auto unwrappedType = untaggedType; + if (unwrappedType->GetKind() == ETypeAnnotationKind::Optional) { + unwrappedType = unwrappedType->Cast<TOptionalExprType>()->GetItemType(); + } + + if (unwrappedType->GetKind() != ETypeAnnotationKind::Data) { + + auto argumentsType = ctx.MakeType<TTupleExprType>(TTypeAnnotationNode::TListType{untaggedType}); + + auto udfArgumentsType = ctx.MakeType<TTupleExprType>(TTypeAnnotationNode::TListType{ + argumentsType, + ctx.MakeType<TStructExprType>(TVector<const TItemExprType*>{}), + ctx.MakeType<TTupleExprType>(TTypeAnnotationNode::TListType{}) + }); + + auto member = ctx.Builder(TPositionHandle()) + .Callable("Member") + .Add(0, result) + .Atom(1, itemType->GetName()) + .Seal() + .Build(); + + for (auto type = itemType->GetItemType(); type->GetKind() == ETypeAnnotationKind::Tagged; type = type->Cast<TTaggedExprType>()->GetBaseType()) { + member = ctx.Builder(TPositionHandle()) + .Callable("Untag") + .Add(0, member) + .Atom(1, type->Cast<TTaggedExprType>()->GetTag()) + .Seal() + .Build(); + } + + result = ctx.Builder(TPositionHandle()) + .Callable("ReplaceMember") + .Add(0, result) + .Atom(1, itemType->GetName()) + .Callable(2, "Apply") + .Callable(0, "Udf") + .Atom(0, "Yson2.Serialize") + .Callable(1, "Void") + .Seal() + .Callable(2, "Void") + .Seal() + .Seal() + .Callable(1, "Apply") + .Callable(0, "Udf") + .Atom(0, "Yson2.From") + .Callable(1, "Void") + .Seal() + .Add(2, ExpandType(TPositionHandle(), *udfArgumentsType, ctx)) + .Seal() + .Add(1, member) + .Seal() + .Seal() + .Seal() + .Build(); + } + } + + if (result == rowArg) { + // No items to remap + return {}; + } + return ctx.NewLambda(TPosition(), + ctx.NewArguments(TPosition(), { rowArg }), std::move(result)); +} + +TExprNode::TPtr CompileViewSql(const TString& provider, const TString& cluster, const TString& sql, ui16 syntaxVersion, + const TString& viewId, const TQContext& qContext, + TExprContext& ctx, IModuleResolver* moduleResolver, IUrlListerManager* urlListerManager, + IRandomProvider& randomProvider, bool enableViewIsolation, IUdfResolver::TPtr udfResolver) +{ + NSQLTranslation::TTranslationSettings settings; + settings.Mode = NSQLTranslation::ESqlMode::LIMITED_VIEW; + settings.DefaultCluster = cluster.empty() ? "view" : cluster; + settings.ClusterMapping[settings.DefaultCluster] = cluster.empty() ? "data" : provider; + settings.SyntaxVersion = syntaxVersion; + settings.V0Behavior = NSQLTranslation::EV0Behavior::Disable; + if (qContext.CanRead() && enableViewIsolation) { + auto res = qContext.GetReader()->Get({YtView_Component, viewId}).GetValueSync(); + if (!res) { + ythrow yexception() << "Missing replay data"; + } + + settings.FileAliasPrefix = res->Value; + } else { + settings.FileAliasPrefix = "view_" + randomProvider.GenUuid4().AsGuidString() + "/"; + if (!enableViewIsolation) { + settings.FileAliasPrefix.clear(); // disable FileAliasPrefix while preserving number of randomProvider calls + } + + if (enableViewIsolation && qContext.CanWrite()) { + qContext.GetWriter()->Put({YtView_Component, viewId}, settings.FileAliasPrefix).GetValueSync(); + } + } + + NYql::TAstParseResult sqlRes = NSQLTranslation::SqlToYql(sql, settings); + ctx.IssueManager.RaiseIssues(sqlRes.Issues); + if (!sqlRes.IsOk()) { + return {}; + } + + TString oldAliasPrefix = moduleResolver->GetFileAliasPrefix(); + moduleResolver->SetFileAliasPrefix(TString{settings.FileAliasPrefix}); + Y_DEFER { + moduleResolver->SetFileAliasPrefix(std::move(oldAliasPrefix)); + }; + TExprNode::TPtr exprRoot; + if (!CompileExpr(*sqlRes.Root, exprRoot, ctx, moduleResolver, urlListerManager, false, Max<ui32>(), syntaxVersion)) { + return {}; + } + + if (!enableViewIsolation) { + return exprRoot; + } + + constexpr TStringBuf OuterFuncs[] = { + "SecureParam", + "CurrentOperationId", + "CurrentOperationSharedId", + "CurrentAuthenticatedUser", + }; + + constexpr TStringBuf CodegenFuncs[] = { + "FilePath", + "FileContent", + "FolderPath", + "Files", + "Configure!", + "Udf", + "ScriptUdf", + "SqlCall", + }; + + TOptimizeExprSettings optSettings(nullptr); + optSettings.VisitChanges = true; + auto status = OptimizeExpr(exprRoot, exprRoot, [&](const TExprNode::TPtr& node, TExprContext& ctx) -> TExprNode::TPtr { + for (const auto& name : OuterFuncs) { + if (node->IsCallable(name)) { + ctx.AddError(TIssue(ctx.GetPosition(node->Pos()), TStringBuilder() << name << " function can't be used in views")); + return nullptr; + } + } + + if (node->IsCallable("FuncCode") && node->ChildrenSize() > 0) { + if (!node->Head().IsCallable("String")) { + ctx.AddError(TIssue(ctx.GetPosition(node->Pos()), "FuncCode should have constant function name in views")); + return nullptr; + } + + if (node->Head().Head().IsAtom()) { + for (const auto& name : OuterFuncs) { + if (node->Head().Head().Content() == name) { + ctx.AddError(TIssue(ctx.GetPosition(node->Pos()), TStringBuilder() << name << " function can't be used in views")); + return nullptr; + } + } + + for (const auto& name : CodegenFuncs) { + if (node->Head().Head().Content() == name) { + ctx.AddError(TIssue(ctx.GetPosition(node->Pos()), TStringBuilder() << name << " function can't be used inside generated code in views")); + return nullptr; + } + } + } + } + + if (node->IsCallable("ScriptUdf") && node->ChildrenSize() > 0 && node->Head().Content().StartsWith("CustomPython")) { + ctx.AddError(TIssue(ctx.GetPosition(node->Pos()), "CustomPython module can't be used in views")); + return nullptr; + } + + if (node->IsCallable({"Udf","SqlCall"}) && node->Head().IsAtom()) { + auto origFunc = node->Head().Content(); + TStringBuf moduleName, funcName; + if (!SplitUdfName(origFunc, moduleName, funcName)) { + return node; + } + + if (udfResolver->ContainsModule(TString(moduleName))) { + return node; + } + + return ctx.ChangeChild(*node, 0, + ctx.NewAtom(node->Head().Pos(), settings.FileAliasPrefix + origFunc)); + } + + return node; + }, ctx, optSettings); + + if (status == IGraphTransformer::TStatus::Error) { + return nullptr; + }; + + return exprRoot; +} + +} // unnamed + + +bool TYtViewDescription::Fill(const TString& provider, const TString& cluster, const TString& sql, ui16 syntaxVersion, + const TString& viewId, const TQContext& qContext, TExprContext& ctx, + IModuleResolver* moduleResolver, IUrlListerManager* urlListerManager, IRandomProvider& randomProvider, bool enableViewIsolation, + IUdfResolver::TPtr udfResolver) +{ + Sql = sql; + CompiledSql = CompileViewSql(provider, cluster, sql, syntaxVersion, viewId, qContext, + ctx, moduleResolver, urlListerManager, randomProvider, enableViewIsolation, udfResolver); + return bool(CompiledSql); +} + +void TYtViewDescription::CleanupCompiledSQL() +{ + CompiledSql.Reset(); +} + +bool TYtTableDescriptionBase::Fill(const TString& provider, const TString& cluster, const TString& table, + const TStructExprType* type, const TString& viewSql, ui16 syntaxVersion, const TQContext& qContext, + const THashMap<TString, TString>& metaAttrs, + TExprContext& ctx, IModuleResolver* moduleResolver, IUrlListerManager* urlListerManager, IRandomProvider& randomProvider, bool enableViewIsolation, + IUdfResolver::TPtr udfResolver) +{ + // (1) row type + RawRowType = type; + YQL_ENSURE(RawRowType && RawRowType->GetKind() == ETypeAnnotationKind::Struct); + RowType = RawRowType; + + bool onlyRawView = false; + if (TYtTableIntent::View == Intents) { + for (auto& view : Views) { + if (view.first == TStringBuf("raw")) { + onlyRawView = true; + } else { + onlyRawView = false; + break; + } + } + } + + // (2) UDF remapper / proto fields + if (!onlyRawView) { + for (auto& x: metaAttrs) { + if (x.first.StartsWith(YqlProtoFieldPrefixAttribute)) { + auto fieldName = x.first.substr(YqlProtoFieldPrefixAttribute.size()); + if (fieldName.empty()) { + ctx.AddError(TIssue(TPosition(), + TStringBuilder() << "Proto field name should not be empty. Table: " + << cluster << '.' << table)); + return false; + } + if (type->FindItem(fieldName)) { // ignore nonexisting fields + ProtoFields.insert({fieldName, x.second}); + } else { + ctx.AddWarning(YqlIssue(TPosition(), EYqlIssueCode::TIssuesIds_EIssueCode_YT_MISSING_PROTO_FIELD, + TStringBuilder() << "Proto field name for missing column " << fieldName << ". Table: " + << cluster << '.' << table)); + } + } + } + + const TString* udfName = metaAttrs.FindPtr(YqlReadUdfAttribute); + if (udfName && !ProtoFields.empty()) { + ctx.AddError(TIssue(TPosition(), + TStringBuilder() << "UDF remapper and proto fields cannot be declared simultaneously. Table: " + << cluster << '.' << table)); + return false; + } + + if (IgnoreTypeV3) { + UdfApplyLambda = BuildIgnoreTypeV3Remapper(RawRowType->Cast<TStructExprType>(), ctx); + } + TExprNode::TPtr lambda; + if (udfName) { + lambda = BuildUdfRemapper(metaAttrs, ctx); + } else if (!ProtoFields.empty()) { + lambda = BuildProtoRemapper(ProtoFields, ctx); + } + if (lambda) { + if (UdfApplyLambda) { + UdfApplyLambda = ctx.Builder(TPositionHandle()) + .Lambda() + .Param("row") + .Apply(lambda) + .With(0) + .Apply(UdfApplyLambda) + .With(0, "row") + .Seal() + .Done() + .Seal() + .Seal() + .Build(); + } else { + UdfApplyLambda = lambda; + } + } + HasUdfApply = (bool)UdfApplyLambda; + } + + // (3) views + if (!FillViews(provider, cluster, table, metaAttrs, qContext, ctx, moduleResolver, urlListerManager, randomProvider, enableViewIsolation, udfResolver)) { + return false; + } + + if (viewSql) { + if (!View) { + auto viewId = cluster + "/" + table; + if (!View.ConstructInPlace().Fill(provider, cluster, viewSql, syntaxVersion, viewId, qContext, ctx, moduleResolver, urlListerManager, randomProvider, enableViewIsolation, udfResolver)) { + ctx.AddError(TIssue(TPosition(), + TStringBuilder() << "Can't load sql view, table: " << cluster << '.' << table)); + return false; + } + } + } + + return true; +} + +bool TYtTableDescriptionBase::FillViews(const TString& provider, const TString& cluster, const TString& table, + const THashMap<TString, TString>& metaAttrs, const TQContext& qContext, TExprContext& ctx, IModuleResolver* moduleResolver, IUrlListerManager* urlListerManager, + IRandomProvider& randomProvider, bool allowViewIsolation, IUdfResolver::TPtr udfResolver) +{ + for (auto& view: Views) { + TYtViewDescription& viewDesc = view.second; + + if (!viewDesc.CompiledSql) { + TString viewSql; + ui16 syntaxVersion = 1; + if (view.first == "raw") { + viewSql = RAW_VIEW_SQL; + } else { + auto sql = metaAttrs.FindPtr(YqlViewPrefixAttribute + view.first); + if (!sql) { + ctx.AddError(TIssue(TPosition(), + TStringBuilder() << "View " << view.first + << " not found in table " << cluster << '.' << table + << " metadata").SetCode(TIssuesIds::YT_VIEW_NOT_FOUND, TSeverityIds::S_ERROR)); + return false; + } + + viewSql = *sql; + auto sqlSyntaxVersion = metaAttrs.FindPtr("_yql_syntax_version_" + view.first); + if (sqlSyntaxVersion) { + syntaxVersion = FromString<ui16>(*sqlSyntaxVersion); + } + } + + auto viewId = cluster + "/" + table + "/" + view.first; + if (!viewDesc.Fill(provider, cluster, viewSql, syntaxVersion, viewId, qContext, ctx, moduleResolver, urlListerManager, randomProvider, allowViewIsolation, udfResolver)) { + ctx.AddError(TIssue(TPosition(), + TStringBuilder() << "Can't load sql view " << viewSql.Quote() + << ", table: " << cluster << '.' << table + << ", view: " << view.first)); + return false; + } + } + } + return true; +} + +void TYtTableDescriptionBase::CleanupCompiledSQL() +{ + UdfApplyLambda.Reset(); + for (auto& view : Views) { + view.second.CleanupCompiledSQL(); + } + if (View) { + View->CleanupCompiledSQL(); + } +} + +} diff --git a/yt/yql/providers/yt/provider/yql_yt_table_desc.h b/yt/yql/providers/yt/provider/yql_yt_table_desc.h new file mode 100644 index 0000000000..3acf35d0e4 --- /dev/null +++ b/yt/yql/providers/yt/provider/yql_yt_table_desc.h @@ -0,0 +1,86 @@ +#pragma once + +#include <yql/essentials/ast/yql_expr.h> +#include <yql/essentials/core/url_lister/interface/url_lister_manager.h> +#include <yql/essentials/core/yql_udf_resolver.h> + +#include <library/cpp/random_provider/random_provider.h> + +#include <util/system/types.h> +#include <util/generic/flags.h> +#include <util/generic/string.h> +#include <util/generic/map.h> +#include <util/generic/maybe.h> +#include <util/generic/hash.h> +#include <util/generic/ptr.h> + +#include <utility> + +namespace NYql { + +class TQContext; + +enum class TYtTableIntent: ui32 { + Read = 1 << 0, + View = 1 << 1, // Read via view + Override = 1 << 2, + Append = 1 << 3, + Create = 1 << 4, // Reserved. Not implemented yet + Drop = 1 << 5, + Flush = 1 << 6, // Untransactional write +}; + +Y_DECLARE_FLAGS(TYtTableIntents, TYtTableIntent); +Y_DECLARE_OPERATORS_FOR_FLAGS(TYtTableIntents); + +inline bool HasReadIntents(TYtTableIntents intents) { + return intents & (TYtTableIntent::Read | TYtTableIntent::View); +} + +inline bool HasModifyIntents(TYtTableIntents intents) { + return intents & (TYtTableIntent::Override | TYtTableIntent::Append | TYtTableIntent::Drop | TYtTableIntent::Flush); +} + +inline bool HasExclusiveModifyIntents(TYtTableIntents intents) { + return intents & (TYtTableIntent::Override | TYtTableIntent::Drop | TYtTableIntent::Flush); +} + +struct TYtViewDescription { + TString Sql; + ui16 SyntaxVersion = 1; + TExprNode::TPtr CompiledSql; // contains Read! to self/self_raw tables + const TTypeAnnotationNode* RowType = nullptr; // Filled only if scheme requested + + bool Fill(const TString& provider, const TString& cluster, const TString& sql, ui16 syntaxVersion, + const TString& viewId, const TQContext& qContext, TExprContext& ctx, + IModuleResolver* moduleResolver, IUrlListerManager* urlListerManager, IRandomProvider& randomProvider, + bool enableViewIsolation, IUdfResolver::TPtr udfResolver); + void CleanupCompiledSQL(); +}; + +struct TYtTableDescriptionBase { + const TTypeAnnotationNode* RawRowType = nullptr; + const TTypeAnnotationNode* RowType = nullptr; // may be customized by UDF if scheme requested + TExprNode::TPtr UdfApplyLambda; // convert table row by UDF + bool HasUdfApply = false; + TMap<TString, TYtViewDescription> Views; + TMaybe<TYtViewDescription> View; + TMap<TString, TString> ProtoFields; + TYtTableIntents Intents; + ui32 InferSchemaRows = 0; + bool ForceInferSchema = false; + bool FailOnInvalidSchema = true; + bool HasWriteLock = false; + bool IgnoreTypeV3 = false; + + bool Fill(const TString& provider, const TString& cluster, const TString& table, const TStructExprType* type, + const TString& viewSql, ui16 syntaxVersion, const TQContext& qContext, const THashMap<TString, TString>& metaAttrs, TExprContext& ctx, + IModuleResolver* moduleResolver, IUrlListerManager* urlListerManager, IRandomProvider& randomProvider, + bool enableViewIsolation, IUdfResolver::TPtr udfResolver); + void CleanupCompiledSQL(); + bool FillViews(const TString& provider, const TString& cluster, const TString& table, const THashMap<TString, TString>& metaAttrs, + const TQContext& qContext, TExprContext& ctx, IModuleResolver* moduleResolver, IUrlListerManager* urlListerManager, IRandomProvider& randomProvider, + bool enableViewIsolation, IUdfResolver::TPtr udfResolver); +}; + +} diff --git a/yt/yql/providers/yt/provider/yql_yt_wide_flow.cpp b/yt/yql/providers/yt/provider/yql_yt_wide_flow.cpp new file mode 100644 index 0000000000..4577d5455d --- /dev/null +++ b/yt/yql/providers/yt/provider/yql_yt_wide_flow.cpp @@ -0,0 +1,256 @@ +#include "yql_yt_provider_impl.h" + +#include <yt/yql/providers/yt/expr_nodes/yql_yt_expr_nodes.h> +#include <yql/essentials/ast/yql_expr.h> +#include <yql/essentials/core/yql_graph_transformer.h> +#include <yql/essentials/core/yql_opt_utils.h> +#include <yql/essentials/providers/common/transform/yql_optimize.h> +#include <yt/yql/providers/yt/common/yql_configuration.h> + +namespace NYql { + +using namespace NNodes; + +namespace { + +TExprNode::TPtr MakeWideLambda(const TExprNode& lambda, ui32 limit, TExprContext& ctx) { + if (lambda.IsCallable("Void")) + return {}; + + if (const auto inStructType = dynamic_cast<const TStructExprType*>(GetSeqItemType(lambda.Head().Head().GetTypeAnn())), outStructType = dynamic_cast<const TStructExprType*>(GetSeqItemType(lambda.GetTypeAnn())); + inStructType && outStructType && limit > std::max(inStructType->GetSize(), outStructType->GetSize()) && 0U < std::min(inStructType->GetSize(), outStructType->GetSize())) { + + return ctx.Builder(lambda.Pos()) + .Lambda() + .Param("wide") + .Callable("ExpandMap") + .Apply(0, lambda) + .With(0) + .Callable("NarrowMap") + .Arg(0, "wide") + .Lambda(1) + .Params("fields", inStructType->GetSize()) + .Callable("AsStruct") + .Do([&](TExprNodeBuilder& parent) -> TExprNodeBuilder& { + ui32 i = 0U; + for (const auto& item : inStructType->GetItems()) { + parent.List(i) + .Atom(0, item->GetName()) + .Arg(1, "fields", i) + .Seal(); + ++i; + } + return parent; + }) + .Seal() + .Seal() + .Seal() + .Done() + .Seal() + .Lambda(1) + .Param("item") + .Do([&](TExprNodeBuilder& lambda) -> TExprNodeBuilder& { + ui32 i = 0U; + for (const auto& item : outStructType->GetItems()) { + lambda.Callable(i++, "Member") + .Arg(0, "item") + .Atom(1, item->GetName()) + .Seal(); + } + return lambda; + }) + .Seal() + .Seal() + .Seal().Build(); + } else if (inStructType && limit > inStructType->GetSize() && 0U < inStructType->GetSize()) { + return ctx.Builder(lambda.Pos()) + .Lambda() + .Param("wide") + .Apply(lambda) + .With(0) + .Callable("NarrowMap") + .Arg(0, "wide") + .Lambda(1) + .Params("fields", inStructType->GetSize()) + .Callable("AsStruct") + .Do([&](TExprNodeBuilder& parent) -> TExprNodeBuilder& { + ui32 i = 0U; + for (const auto& item : inStructType->GetItems()) { + parent.List(i) + .Atom(0, item->GetName()) + .Arg(1, "fields", i) + .Seal(); + ++i; + } + return parent; + }) + .Seal() + .Seal() + .Seal() + .Done() + .Seal() + .Seal().Build(); + } else if (outStructType && limit > outStructType->GetSize() && 0U < outStructType->GetSize()) { + return ctx.Builder(lambda.Pos()) + .Lambda() + .Param("flow") + .Callable("ExpandMap") + .Apply(0, lambda) + .With(0, "flow") + .Seal() + .Lambda(1) + .Param("item") + .Do([&](TExprNodeBuilder& lambda) -> TExprNodeBuilder& { + ui32 i = 0U; + for (const auto& item : outStructType->GetItems()) { + lambda.Callable(i++, "Member") + .Arg(0, "item") + .Atom(1, item->GetName()) + .Seal(); + } + return lambda; + }) + .Seal() + .Seal() + .Seal().Build(); + } + + return {}; +} + +TExprNode::TPtr MakeWideLambdaNoArg(const TExprNode& lambda, ui32 limit, TExprContext& ctx) { + if (const auto outStructType = dynamic_cast<const TStructExprType*>(GetSeqItemType(lambda.GetTypeAnn())); + outStructType && limit > outStructType->GetSize() && 0U < outStructType->GetSize()) { + return ctx.Builder(lambda.Pos()) + .Lambda() + .Callable("ExpandMap") + .Add(0, lambda.TailPtr()) + .Lambda(1) + .Param("item") + .Do([&](TExprNodeBuilder& lambda) -> TExprNodeBuilder& { + ui32 i = 0U; + for (const auto& item : outStructType->GetItems()) { + lambda.Callable(i++, "Member") + .Arg(0, "item") + .Atom(1, item->GetName()) + .Seal(); + } + return lambda; + }) + .Seal() + .Seal() + .Seal().Build(); + } + + return {}; +} + +class TYtWideFlowTransformer : public TOptimizeTransformerBase { +public: + TYtWideFlowTransformer(TYtState::TPtr state) + : TOptimizeTransformerBase(state ? state->Types : nullptr, NLog::EComponent::ProviderYt, {}) + , Limit_(state ? state->Configuration->WideFlowLimit.Get().GetOrElse(DEFAULT_WIDE_FLOW_LIMIT) : DEFAULT_WIDE_FLOW_LIMIT) + { + if (Limit_) { +#define HNDL(name) "WideFlow-"#name, Hndl(&TYtWideFlowTransformer::name) + AddHandler(0, &TYtFill::Match, HNDL(OptimizeFill)); + AddHandler(0, &TYtMap::Match, HNDL(OptimizeMap)); + AddHandler(0, &TYtReduce::Match, HNDL(OptimizeReduce)); + AddHandler(0, &TYtMapReduce::Match, HNDL(OptimizeMapReduce)); +#undef HNDL + } + } + + TMaybeNode<TExprBase> OptimizeFill(TExprBase node, TExprContext& ctx) { + if (const auto fill = node.Cast<TYtFill>(); auto wideContent = MakeWideLambdaNoArg(fill.Content().Ref(), Limit_, ctx)) { + if (auto settings = UpdateSettingValue(fill.Settings().Ref(), EYtSettingType::Flow, ctx.NewAtom(fill.Pos(), ToString(Limit_), TNodeFlags::Default), ctx)) { + const auto wideFill = Build<TYtFill>(ctx, fill.Pos()) + .InitFrom(fill) + .Content(std::move(wideContent)) + .Settings(std::move(settings)) + .Done(); + return wideFill.Ptr(); + } + } + + return node; + } + + TMaybeNode<TExprBase> OptimizeMap(TExprBase node, TExprContext& ctx) { + if (const auto map = node.Cast<TYtMap>(); auto wideMapper = MakeWideLambda(map.Mapper().Ref(), Limit_, ctx)) { + if (auto settings = UpdateSettingValue(map.Settings().Ref(), EYtSettingType::Flow, ctx.NewAtom(map.Pos(), ToString(Limit_), TNodeFlags::Default), ctx)) { + const auto wideMap = Build<TYtMap>(ctx, map.Pos()) + .InitFrom(map) + .Mapper(std::move(wideMapper)) + .Settings(std::move(settings)) + .Done(); + return wideMap.Ptr(); + } + } + + return node; + } + + TMaybeNode<TExprBase> OptimizeReduce(TExprBase node, TExprContext& ctx) { + if (const auto reduce = node.Cast<TYtReduce>(); auto wideReducer = MakeWideLambda(reduce.Reducer().Ref(), Limit_, ctx)) { + auto settingsBuilder = Build<TCoNameValueTupleList>(ctx, reduce.Pos()).InitFrom(reduce.Settings()) + .Add() + .Name().Build(ToString(EYtSettingType::ReduceInputType)) + .Value(TExprBase(ExpandType(reduce.Pos(), GetSeqItemType(*reduce.Reducer().Args().Arg(0).Ref().GetTypeAnn()), ctx))) + .Build(); + + if (auto settings = UpdateSettingValue(settingsBuilder.Done().Ref(), EYtSettingType::Flow, ctx.NewAtom(reduce.Pos(), ToString(Limit_), TNodeFlags::Default), ctx)) { + const auto wideReduce = Build<TYtReduce>(ctx, reduce.Pos()) + .InitFrom(reduce) + .Reducer(std::move(wideReducer)) + .Settings(std::move(settings)) + .Done(); + return wideReduce.Ptr(); + } + } + + return node; + } + + TMaybeNode<TExprBase> OptimizeMapReduce(TExprBase node, TExprContext& ctx) { + const auto mapReduce = node.Cast<TYtMapReduce>(); + if (auto wideMapper = MakeWideLambda(mapReduce.Mapper().Ref(), Limit_, ctx), wideReducer = MakeWideLambda(mapReduce.Reducer().Ref(), Limit_, ctx); wideMapper || wideReducer) { + + auto settingsBuilder = Build<TCoNameValueTupleList>(ctx, mapReduce.Pos()).InitFrom(mapReduce.Settings()) + .Add() + .Name().Build(ToString(EYtSettingType::ReduceInputType)) + .Value(TExprBase(ExpandType(mapReduce.Pos(), GetSeqItemType(*mapReduce.Reducer().Args().Arg(0).Ref().GetTypeAnn()), ctx))) + .Build(); + + if (wideMapper) { + settingsBuilder + .Add() + .Name().Build(ToString(EYtSettingType::MapOutputType)) + .Value(TExprBase(ExpandType(mapReduce.Pos(), GetSeqItemType(*mapReduce.Mapper().Ref().GetTypeAnn()), ctx))) + .Build(); + } + + if (auto settings = UpdateSettingValue(settingsBuilder.Done().Ref(), EYtSettingType::Flow, ctx.NewAtom(mapReduce.Pos(), ToString(Limit_), TNodeFlags::Default), ctx)) { + const auto wideMapReduce = Build<TYtMapReduce>(ctx, mapReduce.Pos()) + .InitFrom(mapReduce) + .Mapper(wideMapper ? std::move(wideMapper) : mapReduce.Mapper().Ptr()) + .Reducer(wideReducer ? std::move(wideReducer) : mapReduce.Reducer().Ptr()) + .Settings(std::move(settings)) + .Done(); + return wideMapReduce.Ptr(); + } + } + + return node; + } +private: + const ui16 Limit_; +}; + +} + +THolder<IGraphTransformer> CreateYtWideFlowTransformer(TYtState::TPtr state) { + return MakeHolder<TYtWideFlowTransformer>(std::move(state)); +} + +} // namespace NYql |