From bef4cb2c2ffd2f23686d0327fea83e1f1761fce8 Mon Sep 17 00:00:00 2001 From: jackylee Date: Tue, 4 Feb 2025 01:39:52 +0800 Subject: [PATCH 01/22] [MIRROR] Fix build faile don Macos with INSTALL_PREFIX not set (#8654) --- dev/builddeps-veloxbe.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/dev/builddeps-veloxbe.sh b/dev/builddeps-veloxbe.sh index ae392d427661..2f000a22d24a 100755 --- a/dev/builddeps-veloxbe.sh +++ b/dev/builddeps-veloxbe.sh @@ -30,6 +30,7 @@ VELOX_HOME="" VELOX_PARAMETER="" BUILD_ARROW=ON SPARK_VERSION=ALL +INSTALL_PREFIX=${INSTALL_PREFIX:-} # set default number of threads as cpu cores minus 2 if [[ "$(uname)" == "Darwin" ]]; then From 546e63afb7ab1391d0c332f2ef8f7155f07fcd0d Mon Sep 17 00:00:00 2001 From: Gluten Performance Bot <137994563+GlutenPerfBot@users.noreply.github.com> Date: Tue, 4 Feb 2025 04:06:25 +0800 Subject: [PATCH 02/22] [GLUTEN-6887][VL] Daily Update Velox Version (2025_02_01) (#8646) 174a62842 by Peter Enescu, fix(array functions): Add support to array_has_duplicates for input of type json (12207) 15d1ce0e3 by Wei He, build: Set up nightly runs of memory arbitration fuzzer (12191) eb91ba6eb by Xiaoxuan Meng, feat: Add sequence storage connector with index lookup join integration (12227) 3578ef66f by Max Katsev, Fix static initialization order fiasco in velox (12204) b127d51e0 by Xiaoxuan Meng, fix: Fix a null bit set in left index lookup join (12226) 832583d3b by Xiao Du, fix: Update the Description of map_top_n (12216) 6fc81cf79 by Xiaoxuan Meng, fix: Fix build failure and change lookup result iterator return type (12224) dcafd3292 by Xiaoxuan Meng, feat: Add initial index lookup join operator implementation (12218) 256ed01be by Christian Zentgraf, misc: Update python versions used for pyvelox (12157) 4bd122114 by Jacob Khaliqi, fix: Fix Compression file check output error messages (12009) d9a58fc18 by Xiaoxuan Meng, feat: Add test index storage connector for index lookup join unit test (12215) 9c31eab07 by Xiaoxuan Meng, feat: Extend local planner to support index lookup join (12214) --- cpp/velox/compute/VeloxBackend.cc | 10 +++++----- ep/build-velox/src/get_velox.sh | 2 +- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/cpp/velox/compute/VeloxBackend.cc b/cpp/velox/compute/VeloxBackend.cc index a6dc8bc770b5..9b7dab4c66ee 100644 --- a/cpp/velox/compute/VeloxBackend.cc +++ b/cpp/velox/compute/VeloxBackend.cc @@ -112,21 +112,21 @@ void VeloxBackend::init(const std::unordered_map& conf google::InitGoogleLogging("gluten"); // Allow growing buffer in another task through its memory pool. - config::globalConfig.memoryPoolCapacityTransferAcrossTasks = true; + config::globalConfig().memoryPoolCapacityTransferAcrossTasks = true; // Avoid creating too many shared leaf pools. - config::globalConfig.memoryNumSharedLeafPools = 0; + config::globalConfig().memoryNumSharedLeafPools = 0; // Set velox_exception_user_stacktrace_enabled. - config::globalConfig.exceptionUserStacktraceEnabled = + config::globalConfig().exceptionUserStacktraceEnabled = backendConf_->get(kEnableUserExceptionStacktrace, kEnableUserExceptionStacktraceDefault); // Set velox_exception_system_stacktrace_enabled. - config::globalConfig.exceptionSystemStacktraceEnabled = + config::globalConfig().exceptionSystemStacktraceEnabled = backendConf_->get(kEnableSystemExceptionStacktrace, kEnableSystemExceptionStacktraceDefault); // Set velox_memory_use_hugepages. - config::globalConfig.memoryUseHugepages = backendConf_->get(kMemoryUseHugePages, kMemoryUseHugePagesDefault); + config::globalConfig().memoryUseHugepages = backendConf_->get(kMemoryUseHugePages, kMemoryUseHugePagesDefault); // Async timeout. FLAGS_gluten_velox_aysnc_timeout_on_task_stopping = diff --git a/ep/build-velox/src/get_velox.sh b/ep/build-velox/src/get_velox.sh index 46f1a754868d..dd27160529e2 100755 --- a/ep/build-velox/src/get_velox.sh +++ b/ep/build-velox/src/get_velox.sh @@ -17,7 +17,7 @@ set -exu VELOX_REPO=https://github.com/oap-project/velox.git -VELOX_BRANCH=2025_01_31 +VELOX_BRANCH=2025_02_01 VELOX_HOME="" OS=`uname -s` From bd920ce9cb2ea86f6d05955a327f60d1b0b6a90d Mon Sep 17 00:00:00 2001 From: Kyligence Git Date: Tue, 4 Feb 2025 05:31:59 -0600 Subject: [PATCH 03/22] [GLUTEN-1632][CH]Daily Update Clickhouse Version (20250204) (#8658) * [GLUTEN-1632][CH]Daily Update Clickhouse Version (20250204) * Fix Build due to https://github.com/ClickHouse/ClickHouse/pull/75218 --------- Co-authored-by: kyligence-git Co-authored-by: Chang Chen --- cpp-ch/clickhouse.version | 4 ++-- cpp-ch/local-engine/Parser/RelParsers/AggregateRelParser.cpp | 1 + 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/cpp-ch/clickhouse.version b/cpp-ch/clickhouse.version index c887227fbb33..ad7aa8b85332 100644 --- a/cpp-ch/clickhouse.version +++ b/cpp-ch/clickhouse.version @@ -1,3 +1,3 @@ CH_ORG=Kyligence -CH_BRANCH=rebase_ch/20250201 -CH_COMMIT=6a77c5fc048 +CH_BRANCH=rebase_ch/20250204 +CH_COMMIT=69b2d923f37 diff --git a/cpp-ch/local-engine/Parser/RelParsers/AggregateRelParser.cpp b/cpp-ch/local-engine/Parser/RelParsers/AggregateRelParser.cpp index 223583c9af03..ef98bba618e6 100644 --- a/cpp-ch/local-engine/Parser/RelParsers/AggregateRelParser.cpp +++ b/cpp-ch/local-engine/Parser/RelParsers/AggregateRelParser.cpp @@ -31,6 +31,7 @@ #include #include #include +#include #include #include #include From 1caa88cfa1d482bc3fd0438b3a5f11bb5d5c6bb7 Mon Sep 17 00:00:00 2001 From: Gluten Performance Bot <137994563+GlutenPerfBot@users.noreply.github.com> Date: Wed, 5 Feb 2025 00:32:04 +0800 Subject: [PATCH 04/22] [GLUTEN-6887][VL] Daily Update Velox Version (2025_02_04) (#8657) a8ea2c9b7 by Kevin Wilfong, fix: Remove StringWriter that modifies input in place (12219) 1a8d336c0 by Pradeep Vaka, Add xxhash64 internal function for Date, Varchar, BigInt types (12170) e4846794b by aditi-pandit, fix: "Memory" is misspelt (12222) f1dec31a2 by Pedro Eugenio Rocha Pedreira, feat(python): PyVelox bindings for LocalRunner (12225) 0c9c94577 by Deepak Majeti, fix(parquet): Clear column chunk metadata of rowgroups not selected (10225) 2e316dd7d by Yenda Li, refactor: use tryFindOrCompile instead of findOrCompile for regex [1/n] (12234) 8a3aa63e9 by Yiyang Chen, Add capacity() getter to ApproxMostFrequentStreamSummary (12212) 3e3f90e11 by Jimmy Lu, fix: Support dictionary with nulls in RowVector::pushDictionaryToRowVectorLeaves (12220) ed351a8dd by Ke, feat: Collect storage stat for Tables scan (12210) 1bbb94447 by Pedro Eugenio Rocha Pedreira, feat(python): PyVelox bindings for Files (12233) --- cpp/velox/jni/JniFileSystem.cc | 7 ++++++- ep/build-velox/src/get_velox.sh | 2 +- 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/cpp/velox/jni/JniFileSystem.cc b/cpp/velox/jni/JniFileSystem.cc index 40a499a87837..c01d870e5c3b 100644 --- a/cpp/velox/jni/JniFileSystem.cc +++ b/cpp/velox/jni/JniFileSystem.cc @@ -17,6 +17,7 @@ #include "JniFileSystem.h" #include "jni/JniCommon.h" +#include "velox/common/io/IoStatistics.h" namespace { constexpr std::string_view kJniFsScheme("jni:"); @@ -84,7 +85,11 @@ class JniReadFile : public facebook::velox::ReadFile { } } - std::string_view pread(uint64_t offset, uint64_t length, void* buf) const override { + std::string_view pread( + uint64_t offset, + uint64_t length, + void* buf, + facebook::velox::io::IoStatistics* stats = nullptr) const override { JNIEnv* env = nullptr; attachCurrentThreadAsDaemonOrThrow(vm, &env); env->CallVoidMethod( diff --git a/ep/build-velox/src/get_velox.sh b/ep/build-velox/src/get_velox.sh index dd27160529e2..124ba6155004 100755 --- a/ep/build-velox/src/get_velox.sh +++ b/ep/build-velox/src/get_velox.sh @@ -17,7 +17,7 @@ set -exu VELOX_REPO=https://github.com/oap-project/velox.git -VELOX_BRANCH=2025_02_01 +VELOX_BRANCH=2025_02_04 VELOX_HOME="" OS=`uname -s` From a0a3c6f2596049ce7b14b67828e6146369fb333a Mon Sep 17 00:00:00 2001 From: Chang chen Date: Wed, 5 Feb 2025 13:19:46 +0800 Subject: [PATCH 05/22] [GLUTEN-8623][CH] Support File meta and row index for parquet (#8624) * [Minor Refactor] remove unnecessary headers and fix typo * [Feature] Init implementation * [Feature] Introduce ParquetInputFormat * [Feature] Introduce ColumnIndexRowRangesProvider * [Refactor] Don't use PARQUET_TEST_DATA in gtest * [Refactor] Rename supportPageindexReader to onlyHasFlatType * [Feature] Support Meta * [Fix] Reduce dependency: ParquetMeta don't depend FormatFile. * Fix failed Ut when reading complex type * Fix falied uts, when column filters pruning all datas. * Fix failed uts, when substrait::ReadRel::LocalFiles's items is empty. * Fix ParquetBlockInputFormat don't skip Row Groups * Normalize benchmark file path * Remove TODO, we already run benchmark, no regression --- .../GlutenClickhouseFunctionSuite.scala | 2 +- .../backendsapi/velox/VeloxBackend.scala | 4 - cpp-ch/local-engine/Common/BlockTypeUtils.h | 6 + cpp-ch/local-engine/Common/DebugUtils.cpp | 27 +- .../local-engine/Common/GlutenStringUtils.cpp | 17 - .../local-engine/Common/GlutenStringUtils.h | 53 +- .../Parser/InputFileNameParser.cpp | 38 +- .../local-engine/Parser/InputFileNameParser.h | 11 +- .../Parser/RelParsers/ReadRelParser.cpp | 2 +- .../Parser/RelParsers/ReadRelParser.h | 12 +- .../Parser/SerializedPlanParser.cpp | 4 +- .../Storages/Parquet/ParquetMeta.cpp | 189 +++++ .../Storages/Parquet/ParquetMeta.h | 168 ++++ .../local-engine/Storages/Parquet/RowRanges.h | 2 + .../Parquet/VectorizedParquetRecordReader.cpp | 184 ++-- .../Parquet/VectorizedParquetRecordReader.h | 65 +- .../Parquet/VirtualColumnRowIndexReader.h | 134 +++ .../SubstraitSource/ExcelTextFormatFile.cpp | 16 +- .../SubstraitSource/ExcelTextFormatFile.h | 13 +- .../Storages/SubstraitSource/FormatFile.cpp | 68 +- .../Storages/SubstraitSource/FormatFile.h | 98 ++- .../SubstraitSource/JSONFormatFile.cpp | 18 +- .../SubstraitSource/ORCFormatFile.cpp | 17 +- .../SubstraitSource/ParquetFormatFile.cpp | 239 +++--- .../SubstraitSource/ParquetFormatFile.h | 34 +- .../SubstraitSource/SubstraitFileSource.cpp | 273 +++--- .../SubstraitSource/SubstraitFileSource.h | 95 +-- .../SubstraitSource/TextFormatFile.cpp | 7 +- .../tests/benchmark_local_engine.cpp | 7 +- .../tests/benchmark_parquet_read.cpp | 38 +- .../tests/data/all_meta/data/f0/._SUCCESS.crc | Bin 0 -> 8 bytes ...-8bdd-a6911d0d465a-c000.snappy.parquet.crc | Bin 0 -> 20 bytes .../tests/data/all_meta/data/f0/_SUCCESS | 0 ...4f9b-8bdd-a6911d0d465a-c000.snappy.parquet | Bin 0 -> 1282 bytes .../tests/data/all_meta/data/f1/._SUCCESS.crc | Bin 0 -> 8 bytes ...-9cf1-aa914dc7cd57-c000.snappy.parquet.crc | Bin 0 -> 20 bytes .../tests/data/all_meta/data/f1/_SUCCESS | 0 ...495b-9cf1-aa914dc7cd57-c000.snappy.parquet | Bin 0 -> 1282 bytes ...4f9b-8bdd-a6911d0d465a-c000.snappy.parquet | Bin 0 -> 1282 bytes .../decimal_filter_push_down/18_2.json | 0 .../18_2_flba.snappy.parquet | Bin .../tests/data/input_filename.snappy.parquet | Bin 0 -> 443 bytes .../data/metadata.rowindex.snappy.parquet | Bin 0 -> 1767 bytes .../tests/data/rowindex_in.snappy.parquet | Bin 0 -> 125451 bytes .../local-engine/tests/gluten_test_util.cpp | 25 +- cpp-ch/local-engine/tests/gluten_test_util.h | 11 +- .../local-engine/tests/gtest_ch_functions.cpp | 43 +- .../tests/gtest_clickhouse_pr_verify.cpp | 13 +- .../tests/gtest_parquet_columnindex.cpp | 104 ++- .../tests/gtest_parquet_columnindex_bug.cpp | 123 ++- .../local-engine/tests/gtest_parquet_read.cpp | 76 +- cpp-ch/local-engine/tests/gtest_parser.cpp | 2 +- .../tests/gtest_write_pipeline.cpp | 2 +- .../tests/gtest_write_pipeline_mergetree.cpp | 5 +- .../tests/json/gtest_local_engine_config.json | 1 + .../json/parquet_metadata/input_filename.json | 194 +++++ .../input_filename_no_real_column.json | 171 ++++ .../parquet_metadata/read_metadata.all.json | 790 ++++++++++++++++++ .../read_metadata.row_index.json | 384 +++++++++ .../json/parquet_metadata/rowindex_in.json | 1 + cpp-ch/local-engine/tests/testConfig.h.in | 8 +- .../backendsapi/BackendSettingsApi.scala | 4 +- .../clickhouse/ClickHouseTestSettings.scala | 4 + 63 files changed, 3061 insertions(+), 741 deletions(-) create mode 100644 cpp-ch/local-engine/Storages/Parquet/ParquetMeta.cpp create mode 100644 cpp-ch/local-engine/Storages/Parquet/ParquetMeta.h create mode 100644 cpp-ch/local-engine/Storages/Parquet/VirtualColumnRowIndexReader.h create mode 100644 cpp-ch/local-engine/tests/data/all_meta/data/f0/._SUCCESS.crc create mode 100644 cpp-ch/local-engine/tests/data/all_meta/data/f0/.part-00000-92bb25d0-7446-4f9b-8bdd-a6911d0d465a-c000.snappy.parquet.crc create mode 100644 cpp-ch/local-engine/tests/data/all_meta/data/f0/_SUCCESS create mode 100644 cpp-ch/local-engine/tests/data/all_meta/data/f0/part-00000-92bb25d0-7446-4f9b-8bdd-a6911d0d465a-c000.snappy.parquet create mode 100644 cpp-ch/local-engine/tests/data/all_meta/data/f1/._SUCCESS.crc create mode 100644 cpp-ch/local-engine/tests/data/all_meta/data/f1/.part-00000-963c0854-a1e2-495b-9cf1-aa914dc7cd57-c000.snappy.parquet.crc create mode 100644 cpp-ch/local-engine/tests/data/all_meta/data/f1/_SUCCESS create mode 100644 cpp-ch/local-engine/tests/data/all_meta/data/f1/part-00000-963c0854-a1e2-495b-9cf1-aa914dc7cd57-c000.snappy.parquet create mode 100644 cpp-ch/local-engine/tests/data/all_meta/part-00000-92bb25d0-7446-4f9b-8bdd-a6911d0d465a-c000.snappy.parquet rename cpp-ch/local-engine/tests/{ => data}/decimal_filter_push_down/18_2.json (100%) rename cpp-ch/local-engine/tests/{ => data}/decimal_filter_push_down/18_2_flba.snappy.parquet (100%) create mode 100644 cpp-ch/local-engine/tests/data/input_filename.snappy.parquet create mode 100644 cpp-ch/local-engine/tests/data/metadata.rowindex.snappy.parquet create mode 100644 cpp-ch/local-engine/tests/data/rowindex_in.snappy.parquet create mode 100644 cpp-ch/local-engine/tests/json/parquet_metadata/input_filename.json create mode 100644 cpp-ch/local-engine/tests/json/parquet_metadata/input_filename_no_real_column.json create mode 100644 cpp-ch/local-engine/tests/json/parquet_metadata/read_metadata.all.json create mode 100644 cpp-ch/local-engine/tests/json/parquet_metadata/read_metadata.row_index.json create mode 100644 cpp-ch/local-engine/tests/json/parquet_metadata/rowindex_in.json diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/compatibility/GlutenClickhouseFunctionSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/compatibility/GlutenClickhouseFunctionSuite.scala index eac4c54cfc7a..a8ffc90dc33e 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/compatibility/GlutenClickhouseFunctionSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/compatibility/GlutenClickhouseFunctionSuite.scala @@ -234,7 +234,7 @@ class GlutenClickhouseFunctionSuite extends GlutenClickHouseTPCHAbstractSuite { test("function_input_file_expr") { withTable("test_table") { sql("create table test_table(a int) using parquet") - sql("insert into test_table values(1)") + sql("insert into test_table values (1), (2)") compareResultsAgainstVanillaSpark( """ |select a,input_file_name(), input_file_block_start(), diff --git a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxBackend.scala b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxBackend.scala index d893eea9e93a..61ed9a2de17f 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxBackend.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxBackend.scala @@ -395,10 +395,6 @@ object VeloxBackendSettings extends BackendSettingsApi { true } - override def supportNativeMetadataColumns(): Boolean = true - - override def supportNativeRowIndexColumn(): Boolean = true - override def supportExpandExec(): Boolean = true override def supportSortExec(): Boolean = true diff --git a/cpp-ch/local-engine/Common/BlockTypeUtils.h b/cpp-ch/local-engine/Common/BlockTypeUtils.h index bbb0940b853a..bed08c8c5323 100644 --- a/cpp-ch/local-engine/Common/BlockTypeUtils.h +++ b/cpp-ch/local-engine/Common/BlockTypeUtils.h @@ -19,6 +19,7 @@ #include #include +#include #include #include #include @@ -73,6 +74,11 @@ inline DB::DataTypePtr DATE() return std::make_shared(); } +inline DB::DataTypePtr TIMESTAMP() +{ + return std::make_shared(6); +} + inline DB::Block makeBlockHeader(const DB::ColumnsWithTypeAndName & data) { return DB::Block(data); diff --git a/cpp-ch/local-engine/Common/DebugUtils.cpp b/cpp-ch/local-engine/Common/DebugUtils.cpp index 513a4ee557d2..69b4fe2ce849 100644 --- a/cpp-ch/local-engine/Common/DebugUtils.cpp +++ b/cpp-ch/local-engine/Common/DebugUtils.cpp @@ -22,6 +22,7 @@ #include #include #include +#include #include #include #include @@ -32,8 +33,10 @@ #include #include #include -#include "Functions/IFunction.h" -#include +#include +#include +#include +#include namespace pb_util = google::protobuf::util; @@ -119,6 +122,24 @@ static std::string truncate(const std::string & str, size_t width) using NameAndColumn = std::pair; using NameAndColumns = std::vector; +template +const T& toAggType(DB::ConstAggregateDataPtr data) +{ + return *reinterpret_cast(data); +} + +std::string get(const DB::ColumnAggregateFunction & agg, size_t row) +{ + auto funcName = agg.getAggregateFunction()->getName(); + + if (funcName == "count") + { + DB::ConstAggregateDataPtr data = agg.getData()[row]; + return std::to_string(toAggType(data).count); + } + return "Nan"; +} + /** * Get rows represented in Sequence by specific truncate and vertical requirement. * @@ -158,7 +179,7 @@ static std::vector> getRows(const NameAndColumns & bloc const auto * const col = column.second.get(); DB::WhichDataType which(getDataType(col)); if (which.isAggregateFunction()) - currentRow.emplace_back("Nan"); + currentRow.emplace_back(get(static_cast(*col), row)); else { if (col->isNullAt(row)) diff --git a/cpp-ch/local-engine/Common/GlutenStringUtils.cpp b/cpp-ch/local-engine/Common/GlutenStringUtils.cpp index 858099fff920..56ed040769de 100644 --- a/cpp-ch/local-engine/Common/GlutenStringUtils.cpp +++ b/cpp-ch/local-engine/Common/GlutenStringUtils.cpp @@ -14,9 +14,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -#include -#include -#include #include "GlutenStringUtils.h" @@ -28,18 +25,4 @@ bool GlutenStringUtils::isNullPartitionValue(const std::string & value) return value == "__HIVE_DEFAULT_PARTITION__"; } -std::string GlutenStringUtils::dumpPartitionValues(const std::map & values) -{ - std::string res; - res += "["; - - for (const auto & [key, value] : values) - { - res += key + "=" + value + ", "; - } - - res += "]"; - return res; -} - } diff --git a/cpp-ch/local-engine/Common/GlutenStringUtils.h b/cpp-ch/local-engine/Common/GlutenStringUtils.h index 0de185d83cc6..5dba3157f5d6 100644 --- a/cpp-ch/local-engine/Common/GlutenStringUtils.h +++ b/cpp-ch/local-engine/Common/GlutenStringUtils.h @@ -15,17 +15,58 @@ * limitations under the License. */ #pragma once +#include +#include #include -#include -#include +#include namespace local_engine { -class GlutenStringUtils +namespace GlutenStringUtils { -public: - static bool isNullPartitionValue(const std::string & value); +bool isNullPartitionValue(const std::string & value); + +template +struct is_pair : std::false_type +{ +}; + +template +struct is_pair> : std::true_type +{ +}; + +template +using is_kv_container = is_pair; + +template +std::string mkString( + const Container & container, + const std::string & delimiter = ", ", + const std::string & prefix = "[", + const std::string & suffix = "]", + const std::string & pairSeparator = "=") +{ + if (container.empty()) + return ""; + + std::string result = prefix; + + bool first = true; + for (const auto & element : container) + { + if (!first) + result += delimiter; + if constexpr (is_kv_container::value) + result += fmt::format("{}{}{}", element.first, pairSeparator, element.second); + else + result += fmt::format("{}", element); + first = false; + } + + result += suffix; + return result; +} - static std::string dumpPartitionValues(const std::map & values); }; } diff --git a/cpp-ch/local-engine/Parser/InputFileNameParser.cpp b/cpp-ch/local-engine/Parser/InputFileNameParser.cpp index e6fb2fa6b3c6..ebb707ada407 100644 --- a/cpp-ch/local-engine/Parser/InputFileNameParser.cpp +++ b/cpp-ch/local-engine/Parser/InputFileNameParser.cpp @@ -25,6 +25,7 @@ #include #include + namespace local_engine { static DB::ITransformingStep::Traits getTraits() @@ -48,14 +49,11 @@ static DB::Block createOutputHeader( { DB::Block output_header{header}; if (file_name.has_value()) - output_header.insert(DB::ColumnWithTypeAndName{std::make_shared(), InputFileNameParser::INPUT_FILE_NAME}); + output_header.insert(DB::ColumnWithTypeAndName{std::make_shared(), FileMetaColumns::INPUT_FILE_NAME}); if (block_start.has_value()) - output_header.insert(DB::ColumnWithTypeAndName{std::make_shared(), InputFileNameParser::INPUT_FILE_BLOCK_START}); + output_header.insert(DB::ColumnWithTypeAndName{std::make_shared(), FileMetaColumns::INPUT_FILE_BLOCK_START}); if (block_length.has_value()) - { - output_header.insert( - DB::ColumnWithTypeAndName{std::make_shared(), InputFileNameParser::INPUT_FILE_BLOCK_LENGTH}); - } + output_header.insert(DB::ColumnWithTypeAndName{std::make_shared(), FileMetaColumns::INPUT_FILE_BLOCK_LENGTH}); return output_header; } @@ -123,20 +121,17 @@ class InputFileExprProjectStep : public DB::ITransformingStep bool InputFileNameParser::hasInputFileNameColumn(const DB::Block & block) { - auto names = block.getNames(); - return std::find(names.begin(), names.end(), INPUT_FILE_NAME) != names.end(); + return block.findByName(FileMetaColumns::INPUT_FILE_NAME) != nullptr; } bool InputFileNameParser::hasInputFileBlockStartColumn(const DB::Block & block) { - auto names = block.getNames(); - return std::find(names.begin(), names.end(), INPUT_FILE_BLOCK_START) != names.end(); + return block.findByName(FileMetaColumns::INPUT_FILE_BLOCK_START) != nullptr; } bool InputFileNameParser::hasInputFileBlockLengthColumn(const DB::Block & block) { - auto names = block.getNames(); - return std::find(names.begin(), names.end(), INPUT_FILE_BLOCK_LENGTH) != names.end(); + return block.findByName(FileMetaColumns::INPUT_FILE_BLOCK_LENGTH) != nullptr; } void InputFileNameParser::addInputFileColumnsToChunk( @@ -150,7 +145,7 @@ void InputFileNameParser::addInputFileColumnsToChunk( for (size_t i = 0; i < header.columns(); ++i) { const auto & column = header.getByPosition(i); - if (column.name == INPUT_FILE_NAME) + if (column.name == FileMetaColumns::INPUT_FILE_NAME) { if (!file_name.has_value()) throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Input file name is not set"); @@ -158,7 +153,7 @@ void InputFileNameParser::addInputFileColumnsToChunk( auto file_name_column = type_string->createColumnConst(chunk.getNumRows(), file_name.value()); output_columns.insert(output_columns.begin() + i, std::move(file_name_column)); } - else if (column.name == INPUT_FILE_BLOCK_START) + else if (column.name == FileMetaColumns::INPUT_FILE_BLOCK_START) { if (!block_start.has_value()) throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "block_start is not set"); @@ -166,7 +161,7 @@ void InputFileNameParser::addInputFileColumnsToChunk( auto block_start_column = type_int64->createColumnConst(chunk.getNumRows(), block_start.value()); output_columns.insert(output_columns.begin() + i, std::move(block_start_column)); } - else if (column.name == INPUT_FILE_BLOCK_LENGTH) + else if (column.name == FileMetaColumns::INPUT_FILE_BLOCK_LENGTH) { if (!block_length.has_value()) throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "block_length is not set"); @@ -180,17 +175,12 @@ void InputFileNameParser::addInputFileColumnsToChunk( bool InputFileNameParser::containsInputFileColumns(const DB::Block & block) { - return hasInputFileNameColumn(block) || hasInputFileBlockStartColumn(block) || hasInputFileBlockLengthColumn(block); + return FileMetaColumns::hasVirtualColumns(block); } DB::Block InputFileNameParser::removeInputFileColumn(const DB::Block & block) { - const auto & columns = block.getColumnsWithTypeAndName(); - DB::ColumnsWithTypeAndName result_columns; - for (const auto & column : columns) - if (!INPUT_FILE_COLUMNS_SET.contains(column.name)) - result_columns.push_back(column); - return result_columns; + return FileMetaColumns::removeVirtualColumns(block); } std::optional InputFileNameParser::addInputFileProjectStep(DB::QueryPlan & plan) @@ -204,8 +194,4 @@ std::optional InputFileNameParser::addInputFileProjectStep return result; } -void InputFileNameParser::addInputFileColumnsToChunk(const DB::Block & header, DB::Chunk & chunk) const -{ - addInputFileColumnsToChunk(header, chunk, file_name, block_start, block_length); -} } diff --git a/cpp-ch/local-engine/Parser/InputFileNameParser.h b/cpp-ch/local-engine/Parser/InputFileNameParser.h index ab52ba107222..a716f91c7c24 100644 --- a/cpp-ch/local-engine/Parser/InputFileNameParser.h +++ b/cpp-ch/local-engine/Parser/InputFileNameParser.h @@ -16,10 +16,11 @@ */ #pragma once #include +#include namespace DB { - class Chunk; +class Chunk; } namespace local_engine @@ -27,11 +28,6 @@ namespace local_engine class InputFileNameParser { public: - static inline const String & INPUT_FILE_NAME = "input_file_name"; - static inline const String & INPUT_FILE_BLOCK_START = "input_file_block_start"; - static inline const String & INPUT_FILE_BLOCK_LENGTH = "input_file_block_length"; - static inline std::unordered_set INPUT_FILE_COLUMNS_SET = {INPUT_FILE_NAME, INPUT_FILE_BLOCK_START, INPUT_FILE_BLOCK_LENGTH}; - static bool hasInputFileNameColumn(const DB::Block & block); static bool hasInputFileBlockStartColumn(const DB::Block & block); static bool hasInputFileBlockLengthColumn(const DB::Block & block); @@ -46,13 +42,10 @@ class InputFileNameParser void setFileName(const String & file_name) { this->file_name = file_name; } - void setBlockStart(const Int64 block_start) { this->block_start = block_start; } - void setBlockLength(const Int64 block_length) { this->block_length = block_length; } [[nodiscard]] std::optional addInputFileProjectStep(DB::QueryPlan & plan); - void addInputFileColumnsToChunk(const DB::Block & header, DB::Chunk & chunk) const; private: std::optional file_name; diff --git a/cpp-ch/local-engine/Parser/RelParsers/ReadRelParser.cpp b/cpp-ch/local-engine/Parser/RelParsers/ReadRelParser.cpp index f65584eaea03..0a663a88a6cf 100644 --- a/cpp-ch/local-engine/Parser/RelParsers/ReadRelParser.cpp +++ b/cpp-ch/local-engine/Parser/RelParsers/ReadRelParser.cpp @@ -144,7 +144,7 @@ DB::QueryPlanStepPtr ReadRelParser::parseReadRelWithJavaIter(const substrait::Re return source_step; } -QueryPlanStepPtr ReadRelParser::parseReadRelWithLocalFile(const substrait::ReadRel & rel) +QueryPlanStepPtr ReadRelParser::parseReadRelWithLocalFile(const substrait::ReadRel & rel) const { auto header = TypeParser::buildBlockFromNamedStruct(rel.base_schema()); substrait::ReadRel::LocalFiles local_files; diff --git a/cpp-ch/local-engine/Parser/RelParsers/ReadRelParser.h b/cpp-ch/local-engine/Parser/RelParsers/ReadRelParser.h index a6ad920947b2..d50c0dfb0b0b 100644 --- a/cpp-ch/local-engine/Parser/RelParsers/ReadRelParser.h +++ b/cpp-ch/local-engine/Parser/RelParsers/ReadRelParser.h @@ -38,10 +38,10 @@ class ReadRelParser : public RelParser // This is source node, there is no input std::optional getSingleInput(const substrait::Rel & rel) override { return {}; } - bool isReadFromDefault(const substrait::ReadRel & rel); - bool isReadRelFromJava(const substrait::ReadRel & rel); - bool isReadFromMergeTree(const substrait::ReadRel & rel); - bool isReadFromStreamKafka(const substrait::ReadRel & rel); + static bool isReadFromDefault(const substrait::ReadRel & rel); + static bool isReadRelFromJava(const substrait::ReadRel & rel); + static bool isReadFromMergeTree(const substrait::ReadRel & rel); + static bool isReadFromStreamKafka(const substrait::ReadRel & rel); void setInputIter(jobject input_iter_, bool is_materialze) { @@ -49,13 +49,13 @@ class ReadRelParser : public RelParser is_input_iter_materialize = is_materialze; } - void setSplitInfo(String split_info_) { split_info = split_info_; } + void setSplitInfo(const String & split_info_) { split_info = split_info_; } private: jobject input_iter; bool is_input_iter_materialize; String split_info; DB::QueryPlanStepPtr parseReadRelWithJavaIter(const substrait::ReadRel & rel); - DB::QueryPlanStepPtr parseReadRelWithLocalFile(const substrait::ReadRel & rel); + DB::QueryPlanStepPtr parseReadRelWithLocalFile(const substrait::ReadRel & rel) const; }; } diff --git a/cpp-ch/local-engine/Parser/SerializedPlanParser.cpp b/cpp-ch/local-engine/Parser/SerializedPlanParser.cpp index e1733e915a8a..e99c7c05b5b5 100644 --- a/cpp-ch/local-engine/Parser/SerializedPlanParser.cpp +++ b/cpp-ch/local-engine/Parser/SerializedPlanParser.cpp @@ -243,7 +243,7 @@ QueryPlanPtr SerializedPlanParser::parseOp(const substrait::Rel & rel, std::list const auto & read = rel.read(); if (read.has_local_files()) { - if (read_rel_parser->isReadRelFromJava(read)) + if (ReadRelParser::isReadRelFromJava(read)) { auto iter = read.local_files().items().at(0).uri_file(); auto pos = iter.find(':'); @@ -252,7 +252,7 @@ QueryPlanPtr SerializedPlanParser::parseOp(const substrait::Rel & rel, std::list read_rel_parser->setInputIter(input_iter, materalize_input); } } - else if (read_rel_parser->isReadFromMergeTree(read)) + else if (ReadRelParser::isReadFromMergeTree(read)) { if (!read.has_extension_table()) { diff --git a/cpp-ch/local-engine/Storages/Parquet/ParquetMeta.cpp b/cpp-ch/local-engine/Storages/Parquet/ParquetMeta.cpp new file mode 100644 index 000000000000..f8e461f2acae --- /dev/null +++ b/cpp-ch/local-engine/Storages/Parquet/ParquetMeta.cpp @@ -0,0 +1,189 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "ParquetMeta.h" +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ +namespace ErrorCodes +{ +extern const int BAD_ARGUMENTS; +} +} + + +namespace local_engine +{ +std::unique_ptr ParquetMetaBuilder::openInputParquetFile(DB::ReadBuffer * read_buffer) +{ + const DB::FormatSettings format_settings{ + .seekable_read = true, + }; + std::atomic is_stopped{0}; + auto arrow_file = asArrowFile(*read_buffer, format_settings, is_stopped, "Parquet", PARQUET_MAGIC_BYTES); + + return parquet::ParquetFileReader::Open(arrow_file, parquet::default_reader_properties(), nullptr); +} + +std::vector ParquetMetaBuilder::pruneColumn( + const DB::Block & header, const parquet::FileMetaData & metadata, bool case_insensitive, bool allow_missing_columns) +{ + std::shared_ptr schema; + THROW_ARROW_NOT_OK(parquet::arrow::FromParquetSchema(metadata.schema(), &schema)); + + DB::ArrowFieldIndexUtil field_util(case_insensitive, allow_missing_columns); + auto index_mapping = field_util.findRequiredIndices(header, *schema, metadata); + + std::vector column_indices; + for (const auto & [clickhouse_header_index, parquet_indexes] : index_mapping) + for (auto parquet_index : parquet_indexes) + column_indices.push_back(parquet_index); + return column_indices; +} + +std::unique_ptr ParquetMetaBuilder::collectColumnIndex( + const parquet::RowGroupMetaData & rgMeta, + parquet::RowGroupPageIndexReader & rowGroupPageIndex, + const std::vector & column_indices, + bool case_insensitive) +{ + auto result = std::make_unique(); + ColumnIndexStore & column_index_store = *result; + column_index_store.reserve(column_indices.size()); + + for (auto const column_index : column_indices) + { + const auto * col_desc = rgMeta.schema()->Column(column_index); + const auto col_index = rowGroupPageIndex.GetColumnIndex(column_index); + const auto offset_index = rowGroupPageIndex.GetOffsetIndex(column_index); + const std::string columnName = case_insensitive ? boost::to_lower_copy(col_desc->name()) : col_desc->name(); + column_index_store[columnName] = ColumnIndex::create(col_desc, col_index, offset_index); + } + return result; +} + + +ParquetMetaBuilder & ParquetMetaBuilder::buildRequiredRowGroups( + const parquet::FileMetaData & file_meta, const std::function & should_include_row_group) +{ + Int32 total_row_groups = file_meta.num_row_groups(); + + readRowGroups.reserve(total_row_groups); + + auto get_column_start_offset = [&](const parquet::ColumnChunkMetaData & metadata_) -> Int64 + { + Int64 offset = metadata_.data_page_offset(); + if (metadata_.has_dictionary_page() && offset > metadata_.dictionary_page_offset()) + offset = metadata_.dictionary_page_offset(); + return offset; + }; + + UInt64 rowStartIndexOffset = 0; + for (int i = 0; i < total_row_groups; ++i) + { + const auto row_group_meta = file_meta.RowGroup(i); + Int64 start_offset = get_column_start_offset(*row_group_meta->ColumnChunk(0)); + Int64 total_bytes = row_group_meta->total_compressed_size(); + + if (!total_bytes) + for (int j = 0; j < row_group_meta->num_columns(); ++j) + total_bytes += row_group_meta->ColumnChunk(j)->total_compressed_size(); + + const UInt64 midpoint_offset = static_cast(start_offset + total_bytes / 2); + + if (should_include_row_group(midpoint_offset)) + { + RowGroupInformation info; + info.index = i; + info.num_rows = row_group_meta->num_rows(); + info.start = row_group_meta->file_offset(); + info.total_compressed_size = row_group_meta->total_compressed_size(); + info.total_size = row_group_meta->total_byte_size(); + info.rowStartIndexOffset = rowStartIndexOffset; + readRowGroups.emplace_back(std::move(info)); + } + rowStartIndexOffset += row_group_meta->num_rows(); + } + return *this; +} + +ParquetMetaBuilder & ParquetMetaBuilder::buildSkipRowGroup(const parquet::FileMetaData & file_meta) +{ + if (collectSkipRowGroup) + { + Int32 total_row_groups = file_meta.num_row_groups(); + + std::vector total_row_group_indices(total_row_groups); + std::iota(total_row_group_indices.begin(), total_row_group_indices.end(), 0); + + std::vector required_row_group_indices(readRowGroups.size()); + for (size_t i = 0; i < readRowGroups.size(); ++i) + required_row_group_indices[i] = readRowGroups[i].index; + + std::ranges::set_difference(total_row_group_indices, required_row_group_indices, std::back_inserter(skipRowGroups)); + } + return *this; +} + +ParquetMetaBuilder & ParquetMetaBuilder::buildRowRange( + parquet::ParquetFileReader & reader, + const parquet::FileMetaData & file_meta, + const DB::Block & readBlock, + const ColumnIndexFilter * column_index_filter) +{ + if (collectPageIndex) + { + readColumns = pruneColumn(readBlock, file_meta, case_insensitive, allow_missing_columns); + for (auto & row_group : readRowGroups) + { + const auto rgMeta = file_meta.RowGroup(row_group.index); + const auto pageIndex = reader.GetPageIndexReader(); + const auto rowGroupPageIndex = pageIndex == nullptr ? nullptr : pageIndex->RowGroup(row_group.index); + if (column_index_filter == nullptr || rowGroupPageIndex == nullptr) + row_group.rowRanges = RowRanges::createSingle(row_group.num_rows); + else + { + auto columnIndex = collectColumnIndex(*rgMeta, *rowGroupPageIndex, readColumns, case_insensitive); + row_group.rowRanges = column_index_filter->calculateRowRanges(*columnIndex, row_group.num_rows); + row_group.columnIndexStore = std::move(columnIndex); + } + } + } + return *this; +} + +ParquetMetaBuilder & ParquetMetaBuilder::build( + DB::ReadBuffer * read_buffer, + const DB::Block * readBlock, + const ColumnIndexFilter * column_index_filter, + const std::function & should_include_row_group) +{ + auto reader = openInputParquetFile(read_buffer); + const auto file_meta = reader->metadata(); + return buildRequiredRowGroups(*file_meta, should_include_row_group) + .buildSkipRowGroup(*file_meta) + .buildRowRange(*reader, *file_meta, *readBlock, column_index_filter); +} + +} \ No newline at end of file diff --git a/cpp-ch/local-engine/Storages/Parquet/ParquetMeta.h b/cpp-ch/local-engine/Storages/Parquet/ParquetMeta.h new file mode 100644 index 000000000000..d722f8b5b5d8 --- /dev/null +++ b/cpp-ch/local-engine/Storages/Parquet/ParquetMeta.h @@ -0,0 +1,168 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +#pragma once + +#include +#include +#include +#include +#include + +namespace local_engine +{ + +struct RowGroupInformation +{ + UInt32 index = 0; + UInt64 start = 0; + UInt64 total_compressed_size = 0; + UInt64 total_size = 0; + UInt64 num_rows = 0; + UInt64 rowStartIndexOffset = 0; + + std::unique_ptr columnIndexStore; + RowRanges rowRanges; +}; + +struct ParquetMetaBuilder +{ + // control flag + bool case_insensitive = false; + bool allow_missing_columns = false; + bool collectSkipRowGroup = false; + bool collectPageIndex = false; + + // + std::vector readRowGroups; + + // collectSkipRowGroup + std::vector skipRowGroups; + + // collectPageIndex + std::vector readColumns; + + ParquetMetaBuilder & build( + DB::ReadBuffer * read_buffer, + const DB::Block * readBlock, + const ColumnIndexFilter * column_index_filter, + const std::function & should_include_row_group); + + static std::unique_ptr openInputParquetFile(DB::ReadBuffer * read_buffer); + +private: + ParquetMetaBuilder & + buildRequiredRowGroups(const parquet::FileMetaData & file_meta, const std::function & should_include_row_group); + ParquetMetaBuilder & buildSkipRowGroup(const parquet::FileMetaData & file_meta); + ParquetMetaBuilder & buildRowRange( + parquet::ParquetFileReader & reader, + const parquet::FileMetaData & file_meta, + const DB::Block & readBlock, + const ColumnIndexFilter * column_index_filter); + + static std::vector + pruneColumn(const DB::Block & header, const parquet::FileMetaData & metadata, bool case_insensitive, bool allow_missing_columns); + static std::unique_ptr collectColumnIndex( + const parquet::RowGroupMetaData & rgMeta, + parquet::RowGroupPageIndexReader & rowGroupPageIndex, + const std::vector & column_indices, + bool case_insensitive = false); +}; + +namespace ParquetVirtualMeta +{ +inline constexpr auto TMP_ROWINDEX = "_tmp_metadata_row_index"; +inline bool hasMetaColumns(const DB::Block & header) +{ + return header.findByName(TMP_ROWINDEX) != nullptr; +} +inline DB::DataTypePtr getMetaColumnType(const DB::Block & header) +{ + return header.findByName(TMP_ROWINDEX)->type; +} +inline DB::Block removeMetaColumns(const DB::Block & header) +{ + DB::Block new_header; + for (const auto & col : header) + if (col.name != TMP_ROWINDEX) + new_header.insert(col); + return new_header; +} +} + +class ColumnIndexRowRangesProvider +{ + ColumnIndexRowRangesProvider(std::vector rowGroupInfos, std::vector readColumns) + : startRowGroupIndex_(rowGroupInfos[0].index), rowGroupInfos_(std::move(rowGroupInfos)), readColumns_(std::move(readColumns)) + { + for (const auto & rg : rowGroupInfos_) + readRowGroups_.push_back(rg.index); + + /// Currently, we only filter row group based on the 'midpoint_offset' of the file, so that the row groups are continuous. + /// We utilized this feature by using vectors instead of map. + assert(rowGroupInfos_.size() == rowGroupInfos_.back().index - startRowGroupIndex_ + 1); + } + +public: + /// Used in UT, in case of testing VirtualColumnRowIndexReader. + explicit ColumnIndexRowRangesProvider(std::vector rowGroupInfos) + : ColumnIndexRowRangesProvider(std::move(rowGroupInfos), {}) + { + } + explicit ColumnIndexRowRangesProvider(ParquetMetaBuilder & meta_collect) + : ColumnIndexRowRangesProvider(std::move(meta_collect.readRowGroups), std::move(meta_collect.readColumns)) + { + } + + std::optional getRowRanges(Int32 row_group_index) const + { + auto index = adjustRowIndex(row_group_index); + auto ranges = rowGroupInfos_[index].rowRanges; + auto rgCount = rowGroupInfos_[index].num_rows; + if (rgCount == 0 || ranges.rowCount() == 0) + return std::nullopt; + return ranges; + } + + UInt64 getRowGroupStartIndex(Int32 row_group_index) const + { + return rowGroupInfos_[adjustRowIndex(row_group_index)].rowStartIndexOffset; + } + const ColumnIndexStore & getColumnIndexStore(Int32 row_group_index) const + { + if (!rowGroupInfos_[adjustRowIndex(row_group_index)].columnIndexStore) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "ColumnIndexStore is not available"); + return *rowGroupInfos_[adjustRowIndex(row_group_index)].columnIndexStore; + } + + const std::vector & getReadRowGroups() const { return readRowGroups_; }; + const std::vector & getReadColumns() const { return readColumns_; }; + +private: + Int32 adjustRowIndex(Int32 row_group_index) const + { + Int32 realIndex = row_group_index - startRowGroupIndex_; + assert(realIndex >= 0 || realIndex < static_cast(rowGroupInfos_.size())); + return realIndex; + }; + + const UInt32 startRowGroupIndex_; + const std::vector rowGroupInfos_; + std::vector readRowGroups_; + const std::vector readColumns_; +}; + +} \ No newline at end of file diff --git a/cpp-ch/local-engine/Storages/Parquet/RowRanges.h b/cpp-ch/local-engine/Storages/Parquet/RowRanges.h index 0ca77f1cfe46..946d1c74025f 100644 --- a/cpp-ch/local-engine/Storages/Parquet/RowRanges.h +++ b/cpp-ch/local-engine/Storages/Parquet/RowRanges.h @@ -189,8 +189,10 @@ class RowRanges } const std::vector & getRanges() const { return ranges; } + std::vector & getRanges() { return ranges; } const Range & getRange(size_t index) const { return ranges[index]; } + Range & getRange(size_t index) { return ranges[index]; } std::string toString() const { diff --git a/cpp-ch/local-engine/Storages/Parquet/VectorizedParquetRecordReader.cpp b/cpp-ch/local-engine/Storages/Parquet/VectorizedParquetRecordReader.cpp index 5208ace66564..cacd7932aac2 100644 --- a/cpp-ch/local-engine/Storages/Parquet/VectorizedParquetRecordReader.cpp +++ b/cpp-ch/local-engine/Storages/Parquet/VectorizedParquetRecordReader.cpp @@ -23,9 +23,9 @@ #include #include #include +#include #include #include -#include #include #include #include @@ -93,6 +93,8 @@ std::string lowerColumnNameIfNeed(const std::string & column_name, const DB::For namespace local_engine { +using namespace ParquetVirtualMeta; + VectorizedColumnReader::VectorizedColumnReader( const parquet::arrow::SchemaField & field, ParquetFileReaderExt * reader, const std::vector & row_groups) : arrow_field_(field.field) @@ -105,7 +107,7 @@ VectorizedColumnReader::VectorizedColumnReader( void VectorizedColumnReader::nextRowGroup() { - input_.nextChunkWithRowRange().and_then( + input_.nextRowGroup().and_then( [&](ColumnChunkPageRead && read) -> std::optional { setPageReader(std::move(read.first), read.second); @@ -162,10 +164,22 @@ std::shared_ptr VectorizedColumnReader::readBatch(int64_t b return result; } +parquet::arrow::SchemaManifest VectorizedParquetRecordReader::createSchemaManifest(const parquet::FileMetaData & metadata) +{ + const parquet::SchemaDescriptor * parquet_schema = metadata.schema(); + const auto & keyValueMetadata = metadata.key_value_metadata(); + const parquet::ArrowReaderProperties properties; + parquet::arrow::SchemaManifest manifest; + THROW_ARROW_NOT_OK(parquet::arrow::SchemaManifest::Make(parquet_schema, keyValueMetadata, properties, &manifest)); + return manifest; +} + + VectorizedParquetRecordReader::VectorizedParquetRecordReader(const DB::Block & header, const DB::FormatSettings & format_settings) - : format_settings_(format_settings) + : parquet_header_(header) + , format_settings_(format_settings) , arrow_column_to_ch_column_( - header, + parquet_header_, "Parquet", format_settings.parquet.allow_missing_columns, format_settings.null_as_default, @@ -175,54 +189,23 @@ VectorizedParquetRecordReader::VectorizedParquetRecordReader(const DB::Block & h } bool VectorizedParquetRecordReader::initialize( - const DB::Block & header, const std::shared_ptr & arrow_file, - const ColumnIndexFilterPtr & column_index_filter, + const ColumnIndexRowRangesProvider & row_ranges_provider, const std::shared_ptr & metadata) { auto file_reader = parquet::ParquetFileReader::Open(arrow_file, parquet::default_reader_properties(), metadata); - const parquet::ArrowReaderProperties properties; - const parquet::FileMetaData & file_metadata = *(file_reader->metadata()); - const parquet::SchemaDescriptor * parquet_schema = file_metadata.schema(); - const auto keyValueMetadata = file_metadata.key_value_metadata(); - parquet::arrow::SchemaManifest manifest; - THROW_ARROW_NOT_OK(parquet::arrow::SchemaManifest::Make(parquet_schema, keyValueMetadata, properties, &manifest)); - std::vector> fields; - fields.reserve(manifest.schema_fields.size()); - for (auto const & schema_field : manifest.schema_fields) - fields.emplace_back(schema_field.field); - const arrow::Schema schema(fields, keyValueMetadata); - - /// column pruning - DB::ArrowFieldIndexUtil field_util( - format_settings_.parquet.case_insensitive_column_matching, format_settings_.parquet.allow_missing_columns); - auto index_mapping = field_util.findRequiredIndices(header, schema, *metadata); - - std::vector column_indices; - for (const auto & [clickhouse_header_index, parquet_indexes] : index_mapping) - { - for (auto parquet_index : parquet_indexes) - { - column_indices.push_back(parquet_index); - } - } + const parquet::FileMetaData & file_metadata = *file_reader->metadata(); - THROW_ARROW_NOT_OK_OR_ASSIGN(std::vector field_indices, manifest.GetFieldIndices(column_indices)); - - /// row groups pruning - std::vector row_groups(boost::counting_iterator(0), boost::counting_iterator(file_metadata.num_row_groups())); - if (!format_settings_.parquet.skip_row_groups.empty()) - { - row_groups.erase( - std::ranges::remove_if(row_groups, [&](const Int32 i) { return format_settings_.parquet.skip_row_groups.contains(i); }).begin(), - row_groups.end()); - } + const std::vector & column_indices = row_ranges_provider.getReadColumns(); + const std::vector & row_groups = row_ranges_provider.getReadRowGroups(); if (row_groups.empty()) return false; - file_reader_ - = std::make_unique(arrow_file, std::move(file_reader), column_index_filter, field_indices, format_settings_); + // initialize File Reader + parquet::arrow::SchemaManifest manifest = createSchemaManifest(file_metadata); + THROW_ARROW_NOT_OK_OR_ASSIGN(std::vector field_indices, manifest.GetFieldIndices(column_indices)); + file_reader_ = std::make_unique(arrow_file, std::move(file_reader), row_ranges_provider, format_settings_); column_readers_.reserve(field_indices.size()); for (auto const & column_index : field_indices) @@ -246,10 +229,8 @@ DB::Chunk VectorizedParquetRecordReader::nextBatch() ::arrow::ChunkedArrayVector columns(column_readers_.size()); DB::ArrowColumnToCHColumn::NameToArrowColumn name_to_column_ptr; for (auto & vectorized_column_reader : column_readers_) - { name_to_column_ptr[lowerColumnNameIfNeed(vectorized_column_reader.columnName(), format_settings_)] = {vectorized_column_reader.readBatch(format_settings_.parquet.max_block_size), vectorized_column_reader.arrowField()}; - } if (const size_t num_rows = name_to_column_ptr.begin()->second.column->length(); num_rows > 0) return arrow_column_to_ch_column_.arrowColumnsToCHChunk(name_to_column_ptr, num_rows, nullptr); @@ -259,56 +240,49 @@ DB::Chunk VectorizedParquetRecordReader::nextBatch() ParquetFileReaderExt::ParquetFileReaderExt( const std::shared_ptr & source, std::unique_ptr parquetFileReader, - const ColumnIndexFilterPtr & column_index_filter, - const std::vector & column_indices, + const ColumnIndexRowRangesProvider & row_ranges_provider, const DB::FormatSettings & format_settings) : source_(source) , file_reader_(std::move(parquetFileReader)) - , column_index_filter_(column_index_filter) , format_settings_(format_settings) - , column_indices_(column_indices.begin(), column_indices.end()) + , row_ranges_provider_(row_ranges_provider) { THROW_ARROW_NOT_OK_OR_ASSIGN(const int64_t source_size, source_->GetSize()); source_size_ = source_size; } -std::optional PageIterator::nextChunkWithRowRange() + +std::optional PageIterator::nextRowGroup() { while (!row_groups_.empty()) { const Int32 row_group_index = row_groups_.front(); - const auto rg = reader_ext_->rowGroup(row_group_index); - const auto rg_count = rg->num_rows(); - - if (rg_count == 0) - { - row_groups_.pop_front(); - continue; - } - - const RowRanges row_ranges - = reader_ext_->canPruningPage(row_group_index) ? reader_ext_->getRowRanges(row_group_index) : RowRanges::createSingle(rg_count); - - if (row_ranges.rowCount() == 0) - { - row_groups_.pop_front(); - continue; - } - - const BuildRead readWithRowRange = [&](const arrow::io::ReadRange & col_range) - { - const ColumnIndexStore & column_index_store = reader_ext_->getColumnIndexStore(row_group_index); - const ColumnIndex & index - = *(column_index_store.find(lowerColumnNameIfNeed(descr()->name(), reader_ext_->format_settings_))->second); - return buildRead(rg_count, col_range, index.offsetIndex().page_locations(), row_ranges); - }; - const BuildRead readAll = [&](const arrow::io::ReadRange & col_range) { return buildAllRead(rg_count, col_range); }; - - const auto read = row_ranges.rowCount() == rg_count ? readAll : readWithRowRange; - auto result = reader_ext_->readColumnChunkPageBase(*rg, column_index_, read); + auto result = reader_ext_->nextRowGroup(row_group_index, column_index_, descr()->name()); row_groups_.pop_front(); - return result; + if (result) + return result; } - return {}; + return std::nullopt; +} + +std::optional +ParquetFileReaderExt::nextRowGroup(int32_t row_group_index, int32_t column_index, const std::string & column_name) const +{ + return row_ranges_provider_.getRowRanges(row_group_index) + .transform( + [&](const RowRanges & row_ranges) + { + const auto rg = fileMeta()->RowGroup(row_group_index); + const auto rg_count = rg->num_rows(); + const BuildRead readAll = [&](const arrow::io::ReadRange & col_range) { return buildAllRead(rg_count, col_range); }; + const BuildRead read = row_ranges.rowCount() == rg_count ? readAll : [&](const arrow::io::ReadRange & col_range) + { + const ColumnIndexStore & column_index_store = row_ranges_provider_.getColumnIndexStore(row_group_index); + const ColumnIndex & index = *column_index_store.find(lowerColumnNameIfNeed(column_name, format_settings_))->second; + return buildRead(rg_count, col_range, index.offsetIndex().page_locations(), row_ranges); + }; + + return readColumnChunkPageBase(*rg, column_index, read); + }); } ColumnChunkPageRead ParquetFileReaderExt::readColumnChunkPageBase( @@ -331,45 +305,13 @@ ColumnChunkPageRead ParquetFileReaderExt::readColumnChunkPageBase( read_sequence); } -const RowRanges & ParquetFileReaderExt::getRowRanges(const Int32 row_group) -{ - if (!row_group_row_ranges_.contains(row_group)) - { - const auto rowGroupMeta = rowGroup(row_group); - const ColumnIndexStore & column_index_store = getColumnIndexStore(row_group); - row_group_row_ranges_[row_group] = calculateRowRanges(column_index_store, rowGroupMeta->num_rows()); - } - return *(row_group_row_ranges_[row_group]); -} - -const ColumnIndexStore & ParquetFileReaderExt::getColumnIndexStore(const Int32 row_group) -{ - if (!row_group_column_index_stores_.contains(row_group)) - { - const auto rowGroupMeta = rowGroup(row_group); - const auto rowGroupIndex = rowGroupPageIndexReader(row_group); - - auto result = std::make_unique(); - ColumnIndexStore & column_index_store = *result; - column_index_store.reserve(column_indices_.size()); - - for (auto const column_index : column_indices_) - { - const auto * col_desc = rowGroupMeta->schema()->Column(column_index); - const auto col_index = rowGroupIndex->GetColumnIndex(column_index); - const auto offset_index = rowGroupIndex->GetOffsetIndex(column_index); - column_index_store[lowerColumnNameIfNeed(col_desc->name(), format_settings_)] - = ColumnIndex::create(col_desc, col_index, offset_index); - } - row_group_column_index_stores_[row_group] = std::move(result); - } - return *(row_group_column_index_stores_[row_group]); -} - /// input format VectorizedParquetBlockInputFormat::VectorizedParquetBlockInputFormat( - DB::ReadBuffer & in_, const DB::Block & header_, const DB::FormatSettings & format_settings) - : DB::IInputFormat(header_, &in_), record_reader_(getPort().getHeader(), format_settings) + DB::ReadBuffer & in_, + const DB::Block & header_, + const ColumnIndexRowRangesProvider & row_ranges_provider, + const DB::FormatSettings & format_settings) + : DB::IInputFormat(header_, &in_), record_reader_(getPort().getHeader(), format_settings), row_ranges_provider_(row_ranges_provider) { } @@ -389,7 +331,7 @@ DB::Chunk VectorizedParquetBlockInputFormat::read() const auto arrow_file = DB::asArrowFile(*in, record_reader_.format_settings_, is_stopped, "Parquet", PARQUET_MAGIC_BYTES); if (is_stopped != 0) return {}; - if (!record_reader_.initialize(getPort().getHeader(), arrow_file, column_index_filter_)) + if (!record_reader_.initialize(arrow_file, row_ranges_provider_)) return {}; } return record_reader_.nextBatch(); @@ -459,7 +401,7 @@ ColumnReadState buildRead( const size_t lastRowIndexInPage = page_row_ranges[i].to; size_t readRowIndexInPage = page_row_ranges[i].from; - /// [readRowIndexInPage ,rowIndex-1] - [rowIndex, rowIndex+readNumber-1] - [rowIndex+readNumber, lastRowIndexInPage] + /// [readRowIndexInPage, rowIndex-1] - [rowIndex, rowIndex+readNumber-1] - [rowIndex+readNumber, lastRowIndexInPage] if (rowIndex <= lastRowIndexInPage) { assert(rowIndex >= readRowIndexInPage); @@ -472,7 +414,7 @@ ColumnReadState buildRead( rowIndex += readNumber; readRowIndexInPage = rowIndex; - /// we already read cuurent page, so we need to read next page. + /// we already read current page, so we need to read next page. if (row_range_begin->to > lastRowIndexInPage) { assert(readRowIndexInPage > lastRowIndexInPage); diff --git a/cpp-ch/local-engine/Storages/Parquet/VectorizedParquetRecordReader.h b/cpp-ch/local-engine/Storages/Parquet/VectorizedParquetRecordReader.h index 06f0e61225fa..faa9e3db5929 100644 --- a/cpp-ch/local-engine/Storages/Parquet/VectorizedParquetRecordReader.h +++ b/cpp-ch/local-engine/Storages/Parquet/VectorizedParquetRecordReader.h @@ -15,6 +15,7 @@ * limitations under the License. */ #pragma once + #include #if USE_PARQUET @@ -22,8 +23,6 @@ #include #include #include -#include -#include #include #include @@ -33,6 +32,7 @@ struct PageLocation; } namespace DB { +class DataTypeNullable; class Block; } @@ -48,8 +48,8 @@ using ReadSequence = std::vector; using ColumnReadState = std::pair; using ColumnChunkPageRead = std::pair, ReadSequence>; -class ColumnIndexFilter; class VectorizedParquetBlockInputFormat; +class ColumnIndexRowRangesProvider; ColumnReadState buildAllRead(int64_t rg_count, const arrow::io::ReadRange & chunk_range); ColumnReadState buildRead( @@ -111,53 +111,28 @@ class ParquetReadState }; using BuildRead = std::function; -class PageIterator; class ParquetFileReaderExt { - using RowRangesMap = absl::flat_hash_map>; - using ColumnIndexStoreMap = absl::flat_hash_map>; - friend class PageIterator; + /// Members std::shared_ptr<::arrow::io::RandomAccessFile> source_; int64_t source_size_; std::unique_ptr file_reader_; - ColumnIndexFilterPtr column_index_filter_; - RowRangesMap row_group_row_ranges_; - ColumnIndexStoreMap row_group_column_index_stores_; const DB::FormatSettings & format_settings_; + const ColumnIndexRowRangesProvider & row_ranges_provider_; -protected: - std::unordered_set column_indices_; - const RowRanges & getRowRanges(Int32 row_group); - const ColumnIndexStore & getColumnIndexStore(Int32 row_group); - - bool canPruningPage(const Int32 row_group) const { return column_index_filter_ && rowGroupPageIndexReader(row_group) != nullptr; } - std::unique_ptr calculateRowRanges(const ColumnIndexStore & index_store, const size_t rowgroup_count) const - { - return std::make_unique(column_index_filter_->calculateRowRanges(index_store, rowgroup_count)); - } - std::unique_ptr rowGroup(const Int32 row_group) const - { - const auto file_metadata = file_reader_->metadata(); - return file_metadata->RowGroup(row_group); - } - - std::shared_ptr rowGroupPageIndexReader(const Int32 row_group) const - { - const auto pageIndex = file_reader_->GetPageIndexReader(); - return pageIndex == nullptr ? nullptr : pageIndex->RowGroup(row_group); - } - + std::shared_ptr fileMeta() const { return file_reader_->metadata(); } ColumnChunkPageRead readColumnChunkPageBase(const parquet::RowGroupMetaData & rg, Int32 column_index, const BuildRead & build_read) const; public: ParquetFileReaderExt( - const std::shared_ptr<::arrow::io::RandomAccessFile> & source, + const std::shared_ptr & source, std::unique_ptr parquetFileReader, - const ColumnIndexFilterPtr & column_index_filter, - const std::vector & column_indices, + const ColumnIndexRowRangesProvider & row_ranges_provider, const DB::FormatSettings & format_settings); + std::optional nextRowGroup(int32_t row_group_index, int32_t column_index, const std::string & column_name) const; + parquet::ParquetFileReader * fileReader() const { return file_reader_.get(); } }; class PageIterator final : public parquet::arrow::FileColumnIterator @@ -166,13 +141,13 @@ class PageIterator final : public parquet::arrow::FileColumnIterator public: PageIterator(const int column_index, ParquetFileReaderExt * reader_ext, const std::vector & row_groups) - : FileColumnIterator(column_index, reader_ext->file_reader_.get(), row_groups), reader_ext_(reader_ext) + : FileColumnIterator(column_index, reader_ext->fileReader(), row_groups), reader_ext_(reader_ext) { } ~PageIterator() override = default; - std::optional nextChunkWithRowRange(); + std::optional nextRowGroup(); }; class VectorizedColumnReader @@ -195,6 +170,7 @@ class VectorizedColumnReader class VectorizedParquetRecordReader { + const DB::Block parquet_header_; const DB::FormatSettings format_settings_; DB::ArrowColumnToCHColumn arrow_column_to_ch_column_; @@ -203,16 +179,18 @@ class VectorizedParquetRecordReader // parquet::arrow::SchemaManifest manifest_; /// columns to read from Parquet file. std::vector column_readers_; + friend class VectorizedParquetBlockInputFormat; + static parquet::arrow::SchemaManifest createSchemaManifest(const parquet::FileMetaData & metadata); + public: VectorizedParquetRecordReader(const DB::Block & header, const DB::FormatSettings & format_settings); ~VectorizedParquetRecordReader() = default; bool initialize( - const DB::Block & header, const std::shared_ptr & arrow_file, - const ColumnIndexFilterPtr & column_index_filter, + const ColumnIndexRowRangesProvider & row_ranges_provider, const std::shared_ptr & metadata = nullptr); DB::Chunk nextBatch(); @@ -231,14 +209,17 @@ class VectorizedParquetBlockInputFormat final : public DB::IInputFormat { std::atomic is_stopped{0}; VectorizedParquetRecordReader record_reader_; - ColumnIndexFilterPtr column_index_filter_; + const ColumnIndexRowRangesProvider & row_ranges_provider_; protected: void onCancel() noexcept override { is_stopped = 1; } public: - VectorizedParquetBlockInputFormat(DB::ReadBuffer & in_, const DB::Block & header_, const DB::FormatSettings & format_settings); - void setColumnIndexFilter(const ColumnIndexFilterPtr & column_index_filter) { column_index_filter_ = column_index_filter; } + VectorizedParquetBlockInputFormat( + DB::ReadBuffer & in_, + const DB::Block & header_, + const ColumnIndexRowRangesProvider & row_ranges_provider, + const DB::FormatSettings & format_settings); String getName() const override { return "VectorizedParquetBlockInputFormat"; } void resetParser() override; diff --git a/cpp-ch/local-engine/Storages/Parquet/VirtualColumnRowIndexReader.h b/cpp-ch/local-engine/Storages/Parquet/VirtualColumnRowIndexReader.h new file mode 100644 index 000000000000..bf5fc0f7f771 --- /dev/null +++ b/cpp-ch/local-engine/Storages/Parquet/VirtualColumnRowIndexReader.h @@ -0,0 +1,134 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace local_engine +{ + +class RowIndexGenerator +{ + RowRanges row_ranges_; + size_t start_rowRange_ = 0; + +public: + RowIndexGenerator(const RowRanges & row_ranges, UInt64 startingRowIdx) : row_ranges_(row_ranges) + { + assert(!row_ranges_.getRanges().empty()); + for (auto & range : row_ranges_.getRanges()) + { + range.from += startingRowIdx; + range.to += startingRowIdx; + } + } + + size_t populateRowIndices(Int64 * row_indices, size_t batchSize) + { + size_t count = 0; + for (; start_rowRange_ < row_ranges_.getRanges().size(); ++start_rowRange_) + { + auto & range = row_ranges_.getRange(start_rowRange_); + for (size_t j = range.from; j <= range.to; ++j) + { + *row_indices++ = j; + if (++count >= batchSize) + { + range.from = j + 1; // Advance range.from + return count; + } + } + } + return count; + } +}; + +class VirtualColumnRowIndexReader +{ + const ColumnIndexRowRangesProvider & row_ranges_provider_; + std::deque row_groups_; + std::optional row_index_generator_; + DB::DataTypePtr column_type_; + + std::optional nextRowGroup() + { + while (!row_groups_.empty()) + { + const Int32 row_group_index = row_groups_.front(); + auto result = row_ranges_provider_.getRowRanges(row_group_index); + row_groups_.pop_front(); + if (result) + return RowIndexGenerator{result.value(), row_ranges_provider_.getRowGroupStartIndex(row_group_index)}; + } + return std::nullopt; + } + +public: + VirtualColumnRowIndexReader(const ColumnIndexRowRangesProvider & row_ranges_provider, const DB::DataTypePtr & column_type) + : row_ranges_provider_(row_ranges_provider) + , row_groups_(row_ranges_provider.getReadRowGroups().begin(), row_ranges_provider.getReadRowGroups().end()) + , row_index_generator_(nextRowGroup()) + , column_type_(column_type) + { + } + + ~VirtualColumnRowIndexReader() = default; + + DB::ColumnPtr readBatch(const int64_t batch_size) + { + if (column_type_->isNullable()) + { + auto internal_type = typeid_cast(*column_type_).getNestedType(); + auto nested_column = readBatchNonNullable(internal_type, batch_size); + auto nullmap_column = DB::ColumnUInt8::create(nested_column->size(), 0); + return DB::ColumnNullable::create(nested_column, std::move(nullmap_column)); + } + return readBatchNonNullable(column_type_, batch_size); + } + + DB::ColumnPtr readBatchNonNullable(const DB::DataTypePtr & notNullType, const int64_t batch_size) + { + assert(DB::WhichDataType(notNullType).isInt64()); + auto column = DB::ColumnInt64::create(batch_size); + DB::ColumnInt64::Container & vec = column->getData(); + int64_t readCount = 0; + int64_t remaining = batch_size; + while (remaining > 0 && row_index_generator_) + { + Int64 * pos = vec.data() + readCount; + readCount += row_index_generator_->populateRowIndices(pos, remaining); + remaining = batch_size - readCount; + if (remaining > 0) + row_index_generator_ = nextRowGroup(); + } + + if (remaining) // we know that we have read all the rows, but we can't fill the container + vec.resize(readCount); + assert(readCount + remaining == batch_size); + assert(readCount == column->size()); + return column; + } +}; +} \ No newline at end of file diff --git a/cpp-ch/local-engine/Storages/SubstraitSource/ExcelTextFormatFile.cpp b/cpp-ch/local-engine/Storages/SubstraitSource/ExcelTextFormatFile.cpp index e1ff4bda5ed5..f094bf915d4f 100644 --- a/cpp-ch/local-engine/Storages/SubstraitSource/ExcelTextFormatFile.cpp +++ b/cpp-ch/local-engine/Storages/SubstraitSource/ExcelTextFormatFile.cpp @@ -64,28 +64,24 @@ bool ExcelTextFormatFile::useThis(const DB::ContextPtr & context) FormatFile::InputFormatPtr ExcelTextFormatFile::createInputFormat(const DB::Block & header) { - auto res = std::make_shared(); - res->read_buffer = read_buffer_builder->build(file_info); + auto read_buffer = read_buffer_builder->build(file_info); DB::FormatSettings format_settings = createFormatSettings(); size_t max_block_size = file_info.text().max_block_size(); DB::RowInputFormatParams params = {.max_block_size = max_block_size}; - std::shared_ptr buffer = std::make_unique(*(res->read_buffer)); + std::shared_ptr buffer = std::make_unique(*read_buffer); DB::Names column_names; column_names.reserve(file_info.schema().names_size()); for (const auto & item : file_info.schema().names()) - { column_names.push_back(item); - } - std::shared_ptr txt_input_format = std::make_shared( - header, buffer, params, format_settings, column_names, file_info.text().escape()); - res->input = txt_input_format; - return res; + auto txt_input_format + = std::make_shared(header, buffer, params, format_settings, column_names, file_info.text().escape()); + return std::make_shared(std::move(read_buffer), txt_input_format); } -DB::FormatSettings ExcelTextFormatFile::createFormatSettings() +DB::FormatSettings ExcelTextFormatFile::createFormatSettings() const { DB::FormatSettings format_settings = DB::getFormatSettings(context); format_settings.csv.trim_whitespaces = true; diff --git a/cpp-ch/local-engine/Storages/SubstraitSource/ExcelTextFormatFile.h b/cpp-ch/local-engine/Storages/SubstraitSource/ExcelTextFormatFile.h index 6fc4183de092..5e70d22eeb96 100644 --- a/cpp-ch/local-engine/Storages/SubstraitSource/ExcelTextFormatFile.h +++ b/cpp-ch/local-engine/Storages/SubstraitSource/ExcelTextFormatFile.h @@ -41,7 +41,9 @@ class ExcelTextFormatFile : public FormatFile public: explicit ExcelTextFormatFile( DB::ContextPtr context_, const substrait::ReadRel::LocalFiles::FileOrFiles & file_info_, ReadBufferBuilderPtr read_buffer_builder_) - : FormatFile(context_, file_info_, read_buffer_builder_){} + : FormatFile(context_, file_info_, read_buffer_builder_) + { + } ~ExcelTextFormatFile() override = default; @@ -51,7 +53,7 @@ class ExcelTextFormatFile : public FormatFile String getFileFormat() const override { return "ExcelText"; } private: - DB::FormatSettings createFormatSettings(); + DB::FormatSettings createFormatSettings() const; }; @@ -82,7 +84,12 @@ class ExcelTextFormatReader final : public DB::CSVFormatReader std::vector readTypes() override; void skipFieldDelimiter() override; void skipRowEndDelimiter() override; - bool readField(DB::IColumn & column, const DB::DataTypePtr & type, const DB::SerializationPtr & serialization, bool is_last_file_column, const String & column_name) override; + bool readField( + DB::IColumn & column, + const DB::DataTypePtr & type, + const DB::SerializationPtr & serialization, + bool is_last_file_column, + const String & column_name) override; void skipField(size_t /*file_column*/) override { skipField(); } void skipField(); diff --git a/cpp-ch/local-engine/Storages/SubstraitSource/FormatFile.cpp b/cpp-ch/local-engine/Storages/SubstraitSource/FormatFile.cpp index ee54a8ff5e9a..0fb5294e6073 100644 --- a/cpp-ch/local-engine/Storages/SubstraitSource/FormatFile.cpp +++ b/cpp-ch/local-engine/Storages/SubstraitSource/FormatFile.cpp @@ -15,7 +15,6 @@ * limitations under the License. */ #include "FormatFile.h" -#include #include #include #include @@ -47,11 +46,65 @@ extern const int NOT_IMPLEMENTED; } namespace local_engine { -FormatFile::FormatFile( - DB::ContextPtr context_, - const substrait::ReadRel::LocalFiles::FileOrFiles & file_info_, - const ReadBufferBuilderPtr & read_buffer_builder_) - : context(context_), file_info(file_info_), read_buffer_builder(read_buffer_builder_) +using namespace DB; +// Initialize the static variable outside the class definition +std::map> FileMetaColumns::BASE_METADATA_EXTRACTORS + = {{FILE_PATH, [](const std::string & metadata) { return metadata; }}, + {FILE_NAME, [](const std::string & metadata) { return metadata; }}, + {FILE_SIZE, [](const std::string & value) { return std::strtoll(value.c_str(), nullptr, 10); }}, + {FILE_BLOCK_START, [](const std::string & value) { return std::strtoll(value.c_str(), nullptr, 10); }}, + {FILE_BLOCK_LENGTH, [](const std::string & value) { return std::strtoll(value.c_str(), nullptr, 10); }}, + {FILE_MODIFICATION_TIME, + [](const std::string & metadata) + { + DB::ReadBufferFromString in(metadata); + DateTime64 time = 0; + readDateTime64Text(time, 6, in, DateLUT::instance("UTC")); + return DecimalField(time, 6); + }}}; + +// Initialize the static variable outside the class definition +std::map> FileMetaColumns::INPUT_FUNCTION_EXTRACTORS + = {{INPUT_FILE_NAME, [](const substraitInputFile & file) { return file.uri_file(); }}, + {INPUT_FILE_BLOCK_START, [](const substraitInputFile & file) { return file.start(); }}, + {INPUT_FILE_BLOCK_LENGTH, [](const substraitInputFile & file) { return file.length(); }}}; + +FileMetaColumns::FileMetaColumns(const substraitInputFile & file) +{ + for (const auto & column : file.metadata_columns()) + { + if (!BASE_METADATA_EXTRACTORS.contains(column.key())) + continue; + + assert(BASE_METADATA_EXTRACTORS.contains(column.key())); + metadata_columns_map[column.key()] = BASE_METADATA_EXTRACTORS[column.key()](column.value()); + } + + for (const auto & inputExtractor : INPUT_FUNCTION_EXTRACTORS) + { + assert(!metadata_columns_map.contains(inputExtractor.first)); + metadata_columns_map[inputExtractor.first] = inputExtractor.second(file); + } +} + +DB::ColumnPtr FileMetaColumns::createMetaColumn(const String & columnName, const DB::DataTypePtr & type, size_t rows) const +{ + assert(metadata_columns_map.contains(columnName)); + const auto field = metadata_columns_map.at(columnName); + + if (INPUT_FILE_COLUMNS_SET.contains(columnName)) + { + /// copied from InputFileNameParser::addInputFileColumnsToChunk() + /// TODO: check whether using const column is correct or not. + return type->createColumnConst(rows, field); + } + auto mutable_column = type->createColumn(); + mutable_column->insertMany(field, rows); + return mutable_column; +} + +FormatFile::FormatFile(DB::ContextPtr context_, const substraitInputFile & file_info_, const ReadBufferBuilderPtr & read_buffer_builder_) + : context(context_), file_info(file_info_), read_buffer_builder(read_buffer_builder_), meta_columns(file_info_) { if (file_info.partition_columns_size()) { @@ -64,7 +117,6 @@ FormatFile::FormatFile( Poco::URI::decode(partition_column.key(), unescaped_key); Poco::URI::decode(partition_column.value(), unescaped_value); - partition_keys.push_back(unescaped_key); partition_values[unescaped_key] = unescaped_value; std::string normalized_key = unescaped_key; @@ -80,7 +132,7 @@ FormatFile::FormatFile( file_info.file_format_case(), std::to_string(file_info.start()) + "-" + std::to_string(file_info.start() + file_info.length()), file_info.partition_index(), - GlutenStringUtils::dumpPartitionValues(partition_values)); + GlutenStringUtils::mkString(partition_values)); } FormatFilePtr FormatFileUtil::createFile( diff --git a/cpp-ch/local-engine/Storages/SubstraitSource/FormatFile.h b/cpp-ch/local-engine/Storages/SubstraitSource/FormatFile.h index f93abd916b52..32323d094023 100644 --- a/cpp-ch/local-engine/Storages/SubstraitSource/FormatFile.h +++ b/cpp-ch/local-engine/Storages/SubstraitSource/FormatFile.h @@ -19,14 +19,13 @@ #include #include #include -#include - #include #include #include #include #include #include +#include namespace DB { @@ -38,20 +37,87 @@ extern const int NOT_IMPLEMENTED; namespace local_engine { + +class FormatFile; +using substraitInputFile = substrait::ReadRel::LocalFiles::FileOrFiles; + +class FileMetaColumns +{ +public: + inline static const std::string FILE_PATH = "file_path"; + inline static const std::string FILE_NAME = "file_name"; + inline static const std::string FILE_BLOCK_START = "file_block_start"; + inline static const std::string FILE_BLOCK_LENGTH = "file_block_length"; + inline static const std::string FILE_SIZE = "file_size"; + inline static const std::string FILE_MODIFICATION_TIME = "file_modification_time"; + + inline static const std::string METADATA_NAME = "_metadata"; + + static inline const std::string INPUT_FILE_NAME = "input_file_name"; + static inline const std::string INPUT_FILE_BLOCK_START = "input_file_block_start"; + static inline const std::string INPUT_FILE_BLOCK_LENGTH = "input_file_block_length"; + + static inline std::unordered_set INPUT_FILE_COLUMNS_SET = {INPUT_FILE_NAME, INPUT_FILE_BLOCK_START, INPUT_FILE_BLOCK_LENGTH}; + static inline std::unordered_set METADATA_COLUMNS_SET + = {FILE_PATH, FILE_NAME, FILE_BLOCK_START, FILE_BLOCK_LENGTH, FILE_SIZE, FILE_MODIFICATION_TIME}; + + /// Caution: only used in InputFileNameParser + static bool isVirtualColumn(const std::string & column_name) + { + return METADATA_COLUMNS_SET.contains(column_name) || INPUT_FILE_COLUMNS_SET.contains(column_name); + } + static bool hasVirtualColumns(const DB::Block & block) + { + return std::ranges::any_of(block, [](const auto & column) { return isVirtualColumn(column.name); }); + } + + static DB::Block removeVirtualColumns(const DB::Block & block) + { + DB::ColumnsWithTypeAndName result_columns; + std::ranges::copy_if( + block.getColumnsWithTypeAndName(), + std::back_inserter(result_columns), + [](const auto & column) { return !isVirtualColumn(column.name); }); + return result_columns; + } + /// + + explicit FileMetaColumns(const substraitInputFile & file); + DB::ColumnPtr createMetaColumn(const String & columnName, const DB::DataTypePtr & type, size_t rows) const; + + bool virtualColumn(const std::string & column_name) const { return metadata_columns_map.contains(column_name); } + +protected: + static std::map> BASE_METADATA_EXTRACTORS; + + /// InputFileName, InputFileBlockStart and InputFileBlockLength, + static std::map> INPUT_FUNCTION_EXTRACTORS; + + std::unordered_map metadata_columns_map; +}; + class FormatFile { public: - struct InputFormat + class InputFormat { + protected: std::unique_ptr read_buffer; DB::InputFormatPtr input; + + public: + virtual ~InputFormat() = default; + DB::IInputFormat & inputFormat() const { return *input; } + void cancel() const noexcept { return input->cancel(); } + virtual DB::Chunk generate() { return input->generate(); } + InputFormat(std::unique_ptr read_buffer_, const DB::InputFormatPtr & input_) + : read_buffer(std::move(read_buffer_)), input(input_) + { + } }; using InputFormatPtr = std::shared_ptr; - FormatFile( - DB::ContextPtr context_, - const substrait::ReadRel::LocalFiles::FileOrFiles & file_info_, - const ReadBufferBuilderPtr & read_buffer_builder_); + FormatFile(DB::ContextPtr context_, const substraitInputFile & file_info_, const ReadBufferBuilderPtr & read_buffer_builder_); virtual ~FormatFile() = default; /// Create a new input format for reading this file @@ -64,29 +130,27 @@ class FormatFile /// Try to get rows from file metadata virtual std::optional getTotalRows() { return {}; } - /// Get partition keys from file path - const std::vector & getFilePartitionKeys() const { return partition_keys; } + virtual String getFileFormat() const = 0; + /// Get partition keys from a file path const std::map & getFilePartitionValues() const { return partition_values; } - const std::map & getFileNormalizedPartitionValues() const { return normalized_partition_values; } - virtual String getURIPath() const { return file_info.uri_file(); } - - virtual size_t getStartOffset() const { return file_info.start(); } - virtual size_t getLength() const { return file_info.length(); } - virtual String getFileFormat() const = 0; + String getURIPath() const { return file_info.uri_file(); } + size_t getStartOffset() const { return file_info.start(); } + const FileMetaColumns & fileMetaColumns() const { return meta_columns; } protected: DB::ContextPtr context; - substrait::ReadRel::LocalFiles::FileOrFiles file_info; + const substraitInputFile file_info; ReadBufferBuilderPtr read_buffer_builder; - std::vector partition_keys; std::map partition_values; /// partition keys are normalized to lower cases for partition column case-insensitive matching std::map normalized_partition_values; std::shared_ptr key_condition; + const FileMetaColumns meta_columns; }; + using FormatFilePtr = std::shared_ptr; using FormatFiles = std::vector; diff --git a/cpp-ch/local-engine/Storages/SubstraitSource/JSONFormatFile.cpp b/cpp-ch/local-engine/Storages/SubstraitSource/JSONFormatFile.cpp index e6e7573be7af..5cbab2bbda3b 100644 --- a/cpp-ch/local-engine/Storages/SubstraitSource/JSONFormatFile.cpp +++ b/cpp-ch/local-engine/Storages/SubstraitSource/JSONFormatFile.cpp @@ -25,23 +25,25 @@ namespace local_engine { -JSONFormatFile::JSONFormatFile(DB::ContextPtr context_, const substrait::ReadRel::LocalFiles::FileOrFiles & file_info_, ReadBufferBuilderPtr read_buffer_builder_) - :FormatFile(context_, file_info_, read_buffer_builder_) {} +JSONFormatFile::JSONFormatFile( + DB::ContextPtr context_, const substrait::ReadRel::LocalFiles::FileOrFiles & file_info_, ReadBufferBuilderPtr read_buffer_builder_) + : FormatFile(context_, file_info_, read_buffer_builder_) +{ +} FormatFile::InputFormatPtr JSONFormatFile::createInputFormat(const DB::Block & header) { - auto res = std::make_shared(); - res->read_buffer = read_buffer_builder->buildWithCompressionWrapper(file_info); + auto read_buffer = read_buffer_builder->buildWithCompressionWrapper(file_info); DB::FormatSettings format_settings = DB::getFormatSettings(context); format_settings.with_names_use_header = true; format_settings.skip_unknown_fields = true; size_t max_block_size = file_info.json().max_block_size(); DB::RowInputFormatParams in_params = {max_block_size}; - std::shared_ptr json_input_format = - std::make_shared(*(res->read_buffer), header, in_params, format_settings, false); - res->input = json_input_format; - return res; + std::shared_ptr json_input_format + = std::make_shared(*read_buffer, header, in_params, format_settings, false); + + return std::make_shared(std::move(read_buffer), json_input_format); } } diff --git a/cpp-ch/local-engine/Storages/SubstraitSource/ORCFormatFile.cpp b/cpp-ch/local-engine/Storages/SubstraitSource/ORCFormatFile.cpp index 4751088df5e0..ec17f69d09ea 100644 --- a/cpp-ch/local-engine/Storages/SubstraitSource/ORCFormatFile.cpp +++ b/cpp-ch/local-engine/Storages/SubstraitSource/ORCFormatFile.cpp @@ -38,12 +38,11 @@ ORCFormatFile::ORCFormatFile( FormatFile::InputFormatPtr ORCFormatFile::createInputFormat(const DB::Block & header) { - auto file_format = std::make_shared(); - file_format->read_buffer = read_buffer_builder->build(file_info); + auto read_buffer = read_buffer_builder->build(file_info); std::vector stripes; [[maybe_unused]] UInt64 total_stripes = 0; - if (auto * seekable_in = dynamic_cast(file_format->read_buffer.get())) + if (auto * seekable_in = dynamic_cast(read_buffer.get())) { stripes = collectRequiredStripes(seekable_in, total_stripes); seekable_in->seek(0, SEEK_SET); @@ -61,12 +60,7 @@ FormatFile::InputFormatPtr ORCFormatFile::createInputFormat(const DB::Block & he required_stripe_indices[i] = stripes[i].index; std::vector skip_stripe_indices; - std::set_difference( - total_stripe_indices.begin(), - total_stripe_indices.end(), - required_stripe_indices.begin(), - required_stripe_indices.end(), - std::back_inserter(skip_stripe_indices)); + std::ranges::set_difference(total_stripe_indices, required_stripe_indices, std::back_inserter(skip_stripe_indices)); format_settings.orc.skip_stripes = std::unordered_set(skip_stripe_indices.begin(), skip_stripe_indices.end()); if (context->getConfigRef().has("timezone")) @@ -75,9 +69,8 @@ FormatFile::InputFormatPtr ORCFormatFile::createInputFormat(const DB::Block & he const String mapped_timezone = DateTimeUtil::convertTimeZone(config_timezone); format_settings.orc.reader_time_zone_name = mapped_timezone; } - auto input_format = std::make_shared(*file_format->read_buffer, header, format_settings); - file_format->input = input_format; - return file_format; + auto input_format = std::make_shared(*read_buffer, header, format_settings); + return std::make_shared(std::move(read_buffer), input_format); } std::optional ORCFormatFile::getTotalRows() diff --git a/cpp-ch/local-engine/Storages/SubstraitSource/ParquetFormatFile.cpp b/cpp-ch/local-engine/Storages/SubstraitSource/ParquetFormatFile.cpp index 212a196550be..e3e28d3fcbb0 100644 --- a/cpp-ch/local-engine/Storages/SubstraitSource/ParquetFormatFile.cpp +++ b/cpp-ch/local-engine/Storages/SubstraitSource/ParquetFormatFile.cpp @@ -19,8 +19,6 @@ #if USE_PARQUET #include -#include -#include #include #include @@ -31,8 +29,7 @@ #include #include #include -#include -#include +#include #include namespace DB @@ -50,9 +47,86 @@ extern const int UNKNOWN_TYPE; } } + namespace local_engine { +struct ShouldIncludeRowGroup +{ + const substraitInputFile & file_info; + + explicit ShouldIncludeRowGroup(const substraitInputFile & file_info) : file_info(file_info) { } + + bool operator()(UInt64 midpoint_offset) const + { + return file_info.start() <= midpoint_offset && midpoint_offset < file_info.start() + file_info.length(); + } +}; + +namespace +{ +ParquetMetaBuilder collectRequiredRowGroups(DB::ReadBuffer * read_buffer, const substraitInputFile & file_info) +{ + ParquetMetaBuilder result; + ShouldIncludeRowGroup should_include_row_group{file_info}; + result.build(read_buffer, nullptr, nullptr, should_include_row_group); + return result; +} +} + +using namespace ParquetVirtualMeta; +class ParquetInputFormat : public FormatFile::InputFormat +{ + const DB::Block readHeader; + const DB::Block outputHeader; + std::unique_ptr rowRangesProvider; + std::optional row_index_reader; + +public: + ParquetInputFormat( + std::unique_ptr read_buffer_, + const DB::InputFormatPtr & input_, + std::unique_ptr provider, + DB::Block readHeader_, + DB::Block outputHeader_) + : InputFormat(std::move(read_buffer_), input_) + , readHeader(std::move(readHeader_)) + , outputHeader(std::move(outputHeader_)) + , rowRangesProvider(std::move(provider)) + , row_index_reader( + outputHeader.columns() > readHeader.columns() + ? std::make_optional(*rowRangesProvider, getMetaColumnType(outputHeader)) + : std::nullopt) + { + } + + DB::Chunk generate() override + { + if (readHeader.columns() == 0) + { + assert(outputHeader.columns()); + assert(row_index_reader); + // TODO: format_settings_.parquet.max_block_size + DB::Columns cols{row_index_reader->readBatch(8192)}; + size_t rows = cols[0]->size(); + return DB::Chunk(std::move(cols), rows); + } + auto chunk = input->generate(); + size_t num_rows = chunk.getNumRows(); + if (row_index_reader && num_rows) + { + auto row_index_column = row_index_reader->readBatch(num_rows); + assert(outputHeader.columns() == readHeader.columns() + 1); + size_t column_pos = outputHeader.getPositionByName(TMP_ROWINDEX); + if (column_pos < chunk.getNumColumns()) + chunk.addColumn(column_pos, std::move(row_index_column)); + else + chunk.addColumn(std::move(row_index_column)); + } + return chunk; + } +}; + ParquetFormatFile::ParquetFormatFile( const DB::ContextPtr & context_, const substrait::ReadRel::LocalFiles::FileOrFiles & file_info_, @@ -62,54 +136,74 @@ ParquetFormatFile::ParquetFormatFile( { } -FormatFile::InputFormatPtr ParquetFormatFile::createInputFormat(const DB::Block & header) +FormatFile::InputFormatPtr ParquetFormatFile::createInputFormat( + const DB::Block & header, + const std::shared_ptr & key_condition, + const ColumnIndexFilterPtr & column_index_filter) const { - auto res = std::make_shared(); - res->read_buffer = read_buffer_builder->build(file_info); + bool readRowIndex = hasMetaColumns(header); + bool usePageIndexReader = (use_pageindex_reader || readRowIndex) && onlyHasFlatType(header); + auto read_buffer = read_buffer_builder->build(file_info); + auto format_settings = DB::getFormatSettings(context); - std::vector required_row_groups; - int total_row_groups = 0; - if (auto * seekable_in = dynamic_cast(res->read_buffer.get())) + DB::Block output_header = header; + DB::Block read_header = removeMetaColumns(header); + + ParquetMetaBuilder metaBuilder{ + .collectPageIndex = usePageIndexReader || readRowIndex, + .collectSkipRowGroup = !usePageIndexReader, + .case_insensitive = format_settings.parquet.case_insensitive_column_matching, + .allow_missing_columns = format_settings.parquet.allow_missing_columns}; + + ShouldIncludeRowGroup should_include_row_group{file_info}; + if (auto * seekable_in = dynamic_cast(read_buffer.get())) { // reuse the read_buffer to avoid opening the file twice. // especially,the cost of opening a hdfs file is large. - required_row_groups = collectRequiredRowGroups(seekable_in, total_row_groups); + metaBuilder.build(seekable_in, &read_header, column_index_filter.get(), should_include_row_group); seekable_in->seek(0, SEEK_SET); } else - required_row_groups = collectRequiredRowGroups(total_row_groups); - - auto format_settings = DB::getFormatSettings(context); - - std::vector total_row_group_indices(total_row_groups); - std::iota(total_row_group_indices.begin(), total_row_group_indices.end(), 0); + { + const auto in = read_buffer_builder->build(file_info); + metaBuilder.build(in.get(), &read_header, column_index_filter.get(), should_include_row_group); + } - std::vector required_row_group_indices(required_row_groups.size()); - for (size_t i = 0; i < required_row_groups.size(); ++i) - required_row_group_indices[i] = required_row_groups[i].index; + if (metaBuilder.readRowGroups.empty()) + return nullptr; - std::vector skip_row_group_indices; - std::ranges::set_difference(total_row_group_indices, required_row_group_indices, std::back_inserter(skip_row_group_indices)); + auto provider = usePageIndexReader || readRowIndex ? std::make_unique(metaBuilder) : nullptr; - format_settings.parquet.skip_row_groups = std::unordered_set(skip_row_group_indices.begin(), skip_row_group_indices.end()); + if (usePageIndexReader) + { + auto input = std::make_shared(*read_buffer, read_header, *provider, format_settings); + return std::make_shared( + std::move(read_buffer), input, std::move(provider), std::move(read_header), std::move(output_header)); + } const DB::Settings & settings = context->getSettingsRef(); + format_settings.parquet.skip_row_groups = std::unordered_set(metaBuilder.skipRowGroups.begin(), metaBuilder.skipRowGroups.end()); - if (use_pageindex_reader && supportPageindexReader(header)) + if (readRowIndex) { - res->input = std::make_shared(*(res->read_buffer), header, format_settings); - } - else - { - res->input = std::make_shared( - *(res->read_buffer), - header, - format_settings, - settings[DB::Setting::max_parsing_threads], - settings[DB::Setting::max_download_threads], - 8192); + assert(provider); + /// In case of readRowIndex, we need to preserve the order of the rows + format_settings.parquet.preserve_order = true; + + /// TODO: enable filter push down again + format_settings.parquet.filter_push_down = false; } - return res; + + auto input = std::make_shared( + *read_buffer, + read_header, + format_settings, + settings[DB::Setting::max_parsing_threads], + settings[DB::Setting::max_download_threads], + 8192); + input->setKeyCondition(key_condition); + return std::make_shared( + std::move(read_buffer), input, std::move(provider), std::move(read_header), std::move(output_header)); } std::optional ParquetFormatFile::getTotalRows() @@ -120,10 +214,11 @@ std::optional ParquetFormatFile::getTotalRows() return total_rows; } - int _; - auto rowgroups = collectRequiredRowGroups(_); + auto in = read_buffer_builder->build(file_info); + auto result = collectRequiredRowGroups(in.get(), file_info); + size_t rows = 0; - for (const auto & rowgroup : rowgroups) + for (const auto & rowgroup : result.readRowGroups) rows += rowgroup.num_rows; { @@ -133,78 +228,18 @@ std::optional ParquetFormatFile::getTotalRows() } } -bool ParquetFormatFile::supportPageindexReader(const DB::Block & header) +bool ParquetFormatFile::onlyHasFlatType(const DB::Block & header) { - const auto result = std::ranges::find_if( + return std::ranges::all_of( header, [](DB::ColumnWithTypeAndName const & col) { const DB::DataTypePtr type_not_nullable = DB::removeNullable(col.type); const DB::WhichDataType which(type_not_nullable); - return DB::isArray(which) || DB::isMap(which) || DB::isTuple(which); + return !DB::isArray(which) && !DB::isMap(which) && !DB::isTuple(which); }); - - return result == header.end(); } -std::vector ParquetFormatFile::collectRequiredRowGroups(int & total_row_groups) const -{ - auto in = read_buffer_builder->build(file_info); - return collectRequiredRowGroups(in.get(), total_row_groups); -} - -std::vector ParquetFormatFile::collectRequiredRowGroups(DB::ReadBuffer * read_buffer, int & total_row_groups) const -{ - const DB::FormatSettings format_settings{ - .seekable_read = true, - }; - std::atomic is_stopped{0}; - std::unique_ptr reader; - auto status = parquet::arrow::OpenFile( - asArrowFile(*read_buffer, format_settings, is_stopped, "Parquet", PARQUET_MAGIC_BYTES), arrow::default_memory_pool(), &reader); - if (!status.ok()) - throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Open file({}) failed. {}", file_info.uri_file(), status.ToString()); - - const auto file_meta = reader->parquet_reader()->metadata(); - total_row_groups = file_meta->num_row_groups(); - - std::vector row_group_metadatas; - row_group_metadatas.reserve(total_row_groups); - - auto get_column_start_offset = [&](const parquet::ColumnChunkMetaData & metadata_) - { - Int64 offset = metadata_.data_page_offset(); - if (metadata_.has_dictionary_page() && offset > metadata_.dictionary_page_offset()) - offset = metadata_.dictionary_page_offset(); - return offset; - }; - - for (int i = 0; i < total_row_groups; ++i) - { - const auto row_group_meta = file_meta->RowGroup(i); - Int64 start_offset = 0; - Int64 total_bytes = 0; - start_offset = get_column_start_offset(*row_group_meta->ColumnChunk(0)); - total_bytes = row_group_meta->total_compressed_size(); - if (!total_bytes) - for (int j = 0; j < row_group_meta->num_columns(); ++j) - total_bytes += row_group_meta->ColumnChunk(j)->total_compressed_size(); - - const UInt64 midpoint_offset = static_cast(start_offset + total_bytes / 2); - /// Current row group has intersection with the required range. - if (file_info.start() <= midpoint_offset && midpoint_offset < file_info.start() + file_info.length()) - { - RowGroupInformation info; - info.index = i; - info.num_rows = row_group_meta->num_rows(); - info.start = row_group_meta->file_offset(); - info.total_compressed_size = row_group_meta->total_compressed_size(); - info.total_size = row_group_meta->total_byte_size(); - row_group_metadatas.emplace_back(std::move(info)); - } - } - return row_group_metadatas; -} } #endif diff --git a/cpp-ch/local-engine/Storages/SubstraitSource/ParquetFormatFile.h b/cpp-ch/local-engine/Storages/SubstraitSource/ParquetFormatFile.h index bed03439421f..ade3739f3c91 100644 --- a/cpp-ch/local-engine/Storages/SubstraitSource/ParquetFormatFile.h +++ b/cpp-ch/local-engine/Storages/SubstraitSource/ParquetFormatFile.h @@ -19,25 +19,13 @@ #include "config.h" #if USE_PARQUET -#include -#include -#include -#include -#include +#include #include -#include -#include namespace local_engine { -struct RowGroupInformation -{ - UInt32 index = 0; - UInt64 start = 0; - UInt64 total_compressed_size = 0; - UInt64 total_size = 0; - UInt64 num_rows = 0; -}; + + class ParquetFormatFile : public FormatFile { public: @@ -48,22 +36,28 @@ class ParquetFormatFile : public FormatFile bool use_local_format_); ~ParquetFormatFile() override = default; - FormatFile::InputFormatPtr createInputFormat(const DB::Block & header) override; + InputFormatPtr createInputFormat(const DB::Block & /*header*/) override + { + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Use createInputFormat with key_condition and column_index_filter"); + } + + InputFormatPtr createInputFormat( + const DB::Block & header, + const std::shared_ptr & key_condition = nullptr, + const ColumnIndexFilterPtr & column_index_filter = nullptr) const; + std::optional getTotalRows() override; bool supportSplit() const override { return true; } String getFileFormat() const override { return "Parquet"; } - static bool supportPageindexReader(const DB::Block & header); + static bool onlyHasFlatType(const DB::Block & header); private: bool use_pageindex_reader; std::mutex mutex; std::optional total_rows; - - std::vector collectRequiredRowGroups(int & total_row_groups) const; - std::vector collectRequiredRowGroups(DB::ReadBuffer * read_buffer, int & total_row_groups) const; }; } diff --git a/cpp-ch/local-engine/Storages/SubstraitSource/SubstraitFileSource.cpp b/cpp-ch/local-engine/Storages/SubstraitSource/SubstraitFileSource.cpp index ef350472b64c..d173d712b7ba 100644 --- a/cpp-ch/local-engine/Storages/SubstraitSource/SubstraitFileSource.cpp +++ b/cpp-ch/local-engine/Storages/SubstraitSource/SubstraitFileSource.cpp @@ -17,26 +17,26 @@ #include #include -#include -#include -#include -#include - #include #include #include #include #include +#include #include #include #include +#include #include +#include #include +#include +#include +#include #include #include #include #include -#include "DataTypes/DataTypesDecimal.h" namespace DB { @@ -49,53 +49,53 @@ extern const int LOGICAL_ERROR; namespace local_engine { -// When run query "select count(*) from t", there is no any column to be read. -// The number of rows is the only needed information. To handle these cases, we -// build blocks with a const virtual column to indicate how many rows is in it. + +/// When run query "select count(*) from t", there is no any column to be read. The only necessary information is the number of rows. +/// To handle these cases, we build blocks with a const virtual column to indicate how many rows are in it. static DB::Block getRealHeader(const DB::Block & header) { - auto header_without_input_file_columns = InputFileNameParser::removeInputFileColumn(header); - auto result_header = header; - if (!header_without_input_file_columns.columns()) - { - auto virtual_header = BlockUtil::buildRowCountHeader(); - for (const auto & column_with_type_and_name : virtual_header.getColumnsWithTypeAndName()) - { - result_header.insert(column_with_type_and_name); - } - } - return result_header; + return header ? header : BlockUtil::buildRowCountHeader(); +} + +static std::vector initializeFiles(const substrait::ReadRel::LocalFiles & file_infos, const DB::ContextPtr & context) +{ + if (file_infos.items().empty()) + return {}; + std::vector files; + const Poco::URI file_uri(file_infos.items().Get(0).uri_file()); + ReadBufferBuilderPtr read_buffer_builder = ReadBufferBuilderFactory::instance().createBuilder(file_uri.getScheme(), context); + for (const auto & item : file_infos.items()) + files.emplace_back(FormatFileUtil::createFile(context, read_buffer_builder, item)); + return files; +} + +static DB::Block initReadHeader(const DB::Block & block, const FormatFiles & files) +{ + if (files.empty()) + return block; + const auto & partitions = files[0]->getFilePartitionValues(); + const auto & fileMetaColumns = files[0]->fileMetaColumns(); + DB::ColumnsWithTypeAndName result_columns; + std::ranges::copy_if( + block.getColumnsWithTypeAndName(), + std::back_inserter(result_columns), + [&partitions, &fileMetaColumns](const auto & column) + { return !partitions.contains(column.name) && !fileMetaColumns.virtualColumn(column.name); }); + return result_columns; } SubstraitFileSource::SubstraitFileSource( - const DB::ContextPtr & context_, - const DB::Block & header_, - const substrait::ReadRel::LocalFiles & file_infos) - : DB::SourceWithKeyCondition(getRealHeader(header_), false) - , context(context_) - , output_header(InputFileNameParser::removeInputFileColumn(header_)) - , to_read_header(output_header) - , input_file_name(InputFileNameParser::containsInputFileColumns(header_)) + const DB::ContextPtr & context_, const DB::Block & outputHeader_, const substrait::ReadRel::LocalFiles & file_infos) + : DB::SourceWithKeyCondition(getRealHeader(outputHeader_), false) + , files(initializeFiles(file_infos, context_)) + , outputHeader(outputHeader_) + , readHeader(initReadHeader(outputHeader, files)) { - if (file_infos.items_size()) - { - /// Initialize files - const Poco::URI file_uri(file_infos.items().Get(0).uri_file()); - read_buffer_builder = ReadBufferBuilderFactory::instance().createBuilder(file_uri.getScheme(), context); - for (const auto & item : file_infos.items()) - files.emplace_back(FormatFileUtil::createFile(context, read_buffer_builder, item)); - - /// File partition keys are read from the file path - const auto partition_keys = files[0]->getFilePartitionKeys(); - for (const auto & key : partition_keys) - if (const auto * col = to_read_header.findByName(key, true)) - to_read_header.erase(col->name); - } } void SubstraitFileSource::setKeyCondition(const std::optional & filter_actions_dag, DB::ContextPtr context_) { - setKeyConditionImpl(filter_actions_dag, context_, to_read_header); + setKeyConditionImpl(filter_actions_dag, context_, readHeader); if (filter_actions_dag) column_index_filter = std::make_shared(filter_actions_dag.value(), context_); } @@ -112,11 +112,7 @@ DB::Chunk SubstraitFileSource::generate() DB::Chunk chunk; if (file_reader->pull(chunk)) - { - if (input_file_name) - input_file_name_parser.addInputFileColumnsToChunk(output.getHeader(), chunk); return chunk; - } /// try to read from next file file_reader.reset(); @@ -131,39 +127,31 @@ bool SubstraitFileSource::tryPrepareReader() if (file_reader) return true; - if (current_file_index >= files.size()) - return false; - - auto current_file = files[current_file_index]; - current_file_index += 1; - - if (!current_file->supportSplit() && current_file->getStartOffset()) + while (current_file_index < files.size()) { + auto current_file = files[current_file_index]; + current_file_index += 1; /// For the files do not support split strategy, the task with not 0 offset will generate empty data - file_reader = std::make_unique(current_file); - return true; - } + if (!current_file->supportSplit() && current_file->getStartOffset()) + continue; - if (!to_read_header) - { - auto total_rows = current_file->getTotalRows(); - if (total_rows.has_value()) - file_reader = std::make_unique(current_file, context, output_header, *total_rows); - else + if (!readHeader) { - /// For text/json format file, we can't get total rows from file metadata. - /// So we add a dummy column to indicate the number of rows. - file_reader - = std::make_unique(current_file, context, getRealHeader(to_read_header), getRealHeader(output_header)); + if (auto totalRows = current_file->getTotalRows()) + file_reader = std::make_unique(current_file, outputHeader, *totalRows); + else + { + /// If we can't get total rows from file metadata (i.e. text/json format file), adding a dummy column to + /// indicate the number of rows. + file_reader = NormalFileReader::create(current_file, getRealHeader(readHeader), getRealHeader(outputHeader)); + } } + else + file_reader = NormalFileReader::create(current_file, readHeader, outputHeader, key_condition, column_index_filter); + if (file_reader) + return true; } - else - file_reader = std::make_unique(current_file, context, to_read_header, output_header); - input_file_name_parser.setFileName(current_file->getURIPath()); - input_file_name_parser.setBlockStart(current_file->getStartOffset()); - input_file_name_parser.setBlockLength(current_file->getLength()); - file_reader->applyKeyCondition(key_condition, column_index_filter); - return true; + return false; } @@ -173,7 +161,36 @@ void SubstraitFileSource::onCancel() noexcept file_reader->cancel(); } -DB::ColumnPtr FileReaderWrapper::createConstColumn(DB::DataTypePtr data_type, const DB::Field & field, size_t rows) +DB::Columns BaseReader::addVirtualColumn(DB::Chunk dataChunk, size_t rowNum) const +{ + // dataChunk may be empty + const size_t rows = dataChunk.empty() ? rowNum : dataChunk.getNumRows(); + assert(rows && "read 0 rows from file"); + + auto read_columns = dataChunk.detachColumns(); + const auto & columns = outputHeader.getColumnsWithTypeAndName(); + const auto & normalized_partition_values = file->getFileNormalizedPartitionValues(); + + DB::Columns res_columns; + res_columns.reserve(columns.size()); + std::ranges::transform( + columns, + std::back_inserter(res_columns), + [&](const auto & column) -> DB::ColumnPtr + { + if (readHeader.has(column.name)) + return read_columns[readHeader.getPositionByName(column.name)]; + if (auto it = normalized_partition_values.find(boost::to_lower_copy(column.name)); it != normalized_partition_values.end()) + return createPartitionColumn(it->second, column.type, rows); + if (file->fileMetaColumns().virtualColumn(column.name)) + return file->fileMetaColumns().createMetaColumn(column.name, column.type, rows); + throw DB::Exception( + DB::ErrorCodes::LOGICAL_ERROR, "Not found column = {} when reading file: {}.", column.name, file->getURIPath()); + }); + return res_columns; +} + +DB::ColumnPtr BaseReader::createConstColumn(DB::DataTypePtr data_type, const DB::Field & field, size_t rows) { auto nested_type = DB::removeNullable(data_type); auto column = nested_type->createColumnConst(rows, field); @@ -183,7 +200,7 @@ DB::ColumnPtr FileReaderWrapper::createConstColumn(DB::DataTypePtr data_type, co return column; } -DB::ColumnPtr FileReaderWrapper::createColumn(const String & value, DB::DataTypePtr type, size_t rows) +DB::ColumnPtr BaseReader::createPartitionColumn(const String & value, const DB::DataTypePtr & type, size_t rows) { if (GlutenStringUtils::isNullPartitionValue(value)) { @@ -216,7 +233,7 @@ DB::ColumnPtr FileReaderWrapper::createColumn(const String & value, DB::DataType return DB::Field(value); \ } -DB::Field FileReaderWrapper::buildFieldFromString(const String & str_value, DB::DataTypePtr type) +DB::Field BaseReader::buildFieldFromString(const String & str_value, DB::DataTypePtr type) { using FieldBuilder = std::function; static std::map field_builders @@ -276,25 +293,25 @@ DB::Field FileReaderWrapper::buildFieldFromString(const String & str_value, DB:: DB::WhichDataType which(nested_type->getTypeId()); if (which.isDecimal32()) { - auto & dataTypeDecimal = static_cast &>(*nested_type); + const auto & dataTypeDecimal = static_cast &>(*nested_type); DB::Decimal32 value = dataTypeDecimal.parseFromString(str_value); return DB::DecimalField(value, dataTypeDecimal.getScale()); } else if (which.isDecimal64()) { - auto & dataTypeDecimal = static_cast &>(*nested_type); + const auto & dataTypeDecimal = static_cast &>(*nested_type); DB::Decimal64 value = dataTypeDecimal.parseFromString(str_value); return DB::DecimalField(value, dataTypeDecimal.getScale()); } else if (which.isDecimal128()) { - auto & dataTypeDecimal = static_cast &>(*nested_type); + const auto & dataTypeDecimal = static_cast &>(*nested_type); DB::Decimal128 value = dataTypeDecimal.parseFromString(str_value); return DB::DecimalField(value, dataTypeDecimal.getScale()); } else if (which.isDecimal256()) { - auto & dataTypeDecimal = static_cast &>(*nested_type); + const auto & dataTypeDecimal = static_cast &>(*nested_type); DB::Decimal256 value = dataTypeDecimal.parseFromString(str_value); return DB::DecimalField(value, dataTypeDecimal.getScale()); } @@ -304,16 +321,11 @@ DB::Field FileReaderWrapper::buildFieldFromString(const String & str_value, DB:: return it->second(read_buffer, str_value); } -ConstColumnsFileReader::ConstColumnsFileReader(FormatFilePtr file_, DB::ContextPtr context_, const DB::Block & header_, size_t block_size_) - : FileReaderWrapper(file_), context(context_), header(header_), remained_rows(0), block_size(block_size_) +ConstColumnsFileReader::ConstColumnsFileReader(const FormatFilePtr & file_, const DB::Block & header_, size_t blockSize) + : BaseReader(file_, {}, header_), remained_rows(file->getTotalRows().value()), block_size(blockSize) { - auto rows = file->getTotalRows(); - if (!rows) - throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Cannot get total rows number from file : {}", file->getURIPath()); - remained_rows = *rows; } - bool ConstColumnsFileReader::pull(DB::Chunk & chunk) { if (isCancelled()) @@ -334,39 +346,46 @@ bool ConstColumnsFileReader::pull(DB::Chunk & chunk) remained_rows -= block_size; } - DB::Columns res_columns; - if (const size_t col_num = header.columns()) - { - res_columns.reserve(col_num); - const auto & normalized_partition_values = file->getFileNormalizedPartitionValues(); - for (size_t pos = 0; pos < col_num; ++pos) - { - const auto & column = header.getByPosition(pos); - const auto & type = column.type; - const auto & name = column.name; + /// If the original output header is empty, build a block to represent the row count. + DB::Columns res_columns + = outputHeader.columns() > 0 ? addVirtualColumn({}, to_read_rows) : BlockUtil::buildRowCountChunk(to_read_rows).detachColumns(); - auto it = normalized_partition_values.find(boost::to_lower_copy(name)); - if (it == normalized_partition_values.end()) - throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Unknow partition column : {}", name); + chunk = DB::Chunk(std::move(res_columns), to_read_rows); + return true; +} - res_columns.emplace_back(createColumn(it->second, type, to_read_rows)); - } +std::unique_ptr NormalFileReader::create( + const FormatFilePtr & file, + const DB::Block & to_read_header_, + const DB::Block & output_header_, + const std::shared_ptr & key_condition, + const ColumnIndexFilterPtr & column_index_filter) +{ + FormatFile::InputFormatPtr input_format; + if (auto * parquetFile = dynamic_cast(file.get())) + { + /// Apply key condition to the reader. + /// If use_local_format is true, column_index_filter will be used otherwise it will be ignored + input_format = parquetFile->createInputFormat(to_read_header_, key_condition, column_index_filter); } else { - // the original header is empty, build a block to represent the row count. - res_columns = BlockUtil::buildRowCountChunk(to_read_rows).detachColumns(); + input_format = file->createInputFormat(to_read_header_); + if (key_condition) + input_format->inputFormat().setKeyCondition(key_condition); } - - chunk = DB::Chunk(std::move(res_columns), to_read_rows); - return true; + if (!input_format) + return nullptr; + return std::make_unique(file, to_read_header_, output_header_, input_format); } NormalFileReader::NormalFileReader( - const FormatFilePtr & file_, const DB::ContextPtr & context_, const DB::Block & to_read_header_, const DB::Block & output_header_) - : FileReaderWrapper(file_), context(context_), to_read_header(to_read_header_), output_header(output_header_) + const FormatFilePtr & file_, + const DB::Block & to_read_header_, + const DB::Block & output_header_, + const FormatFile::InputFormatPtr & input_format_) + : BaseReader(file_, to_read_header_, output_header_), input_format(input_format_) { - input_format = file->createInputFormat(to_read_header); } bool NormalFileReader::pull(DB::Chunk & chunk) @@ -374,36 +393,12 @@ bool NormalFileReader::pull(DB::Chunk & chunk) if (isCancelled()) return false; - DB::Chunk raw_chunk = input_format->input->generate(); - const size_t rows = raw_chunk.getNumRows(); + /// read read real data chunk from input. + DB::Chunk dataChunk = input_format->generate(); + const size_t rows = dataChunk.getNumRows(); if (!rows) return false; - - auto read_columns = raw_chunk.detachColumns(); - const auto & columns = output_header.getColumnsWithTypeAndName(); - const auto & normalized_partition_values = file->getFileNormalizedPartitionValues(); - - DB::Columns res_columns; - res_columns.reserve(columns.size()); - for (auto & column : columns) - { - if (to_read_header.has(column.name)) - { - auto pos = to_read_header.getPositionByName(column.name); - res_columns.push_back(read_columns[pos]); - } - else - { - auto it = normalized_partition_values.find(boost::to_lower_copy(column.name)); - if (it == normalized_partition_values.end()) - throw DB::Exception( - DB::ErrorCodes::LOGICAL_ERROR, "Not found column({}) from file({}) partition keys.", column.name, file->getURIPath()); - - res_columns.push_back(createColumn(it->second, column.type, rows)); - } - } - - chunk = DB::Chunk(std::move(res_columns), rows); + chunk = DB::Chunk(addVirtualColumn(std::move(dataChunk)), rows); return true; } diff --git a/cpp-ch/local-engine/Storages/SubstraitSource/SubstraitFileSource.h b/cpp-ch/local-engine/Storages/SubstraitSource/SubstraitFileSource.h index d436a30d73b2..1a6ae6535d4b 100644 --- a/cpp-ch/local-engine/Storages/SubstraitSource/SubstraitFileSource.h +++ b/cpp-ch/local-engine/Storages/SubstraitSource/SubstraitFileSource.h @@ -20,21 +20,22 @@ #include #include #include -#include -#include #include -#include #include -#include namespace local_engine { -class FileReaderWrapper +class ColumnIndexFilter; +using ColumnIndexFilterPtr = std::shared_ptr; + +class BaseReader { public: - explicit FileReaderWrapper(const FormatFilePtr & file_) : file(file_) { } - virtual ~FileReaderWrapper() = default; - virtual bool pull(DB::Chunk & chunk) = 0; + explicit BaseReader(const FormatFilePtr & file_, const DB::Block & to_read_header_, const DB::Block & header_) + : file(file_), readHeader(to_read_header_), outputHeader(header_) + { + } + virtual ~BaseReader() = default; void cancel() { @@ -43,79 +44,60 @@ class FileReaderWrapper onCancel(); } + virtual bool pull(DB::Chunk & chunk) = 0; bool isCancelled() const { return is_cancelled.load(std::memory_order_acquire); } - /// Apply key condition to the reader, if use_local_format is true, column_index_filter will be used - /// otherwise it will be ignored - virtual void applyKeyCondition( - const std::shared_ptr & /*key_condition*/, const ColumnIndexFilterPtr & /*column_index_filter*/) - { - } - protected: - virtual void onCancel() {}; + virtual void onCancel() { }; + + DB::Columns addVirtualColumn(DB::Chunk dataChunk, size_t rowNum = 0) const; FormatFilePtr file; + DB::Block readHeader; + DB::Block outputHeader; + std::atomic is_cancelled{false}; static DB::ColumnPtr createConstColumn(DB::DataTypePtr type, const DB::Field & field, size_t rows); - static DB::ColumnPtr createColumn(const String & value, DB::DataTypePtr type, size_t rows); + static DB::ColumnPtr createPartitionColumn(const String & value, const DB::DataTypePtr & type, size_t rows); static DB::Field buildFieldFromString(const String & value, DB::DataTypePtr type); }; -class NormalFileReader : public FileReaderWrapper +class NormalFileReader : public BaseReader { public: + static std::unique_ptr create( + const FormatFilePtr & file, + const DB::Block & to_read_header_, + const DB::Block & output_header_, + const std::shared_ptr & key_condition = nullptr, + const ColumnIndexFilterPtr & column_index_filter = nullptr); NormalFileReader( - const FormatFilePtr & file_, const DB::ContextPtr & context_, const DB::Block & to_read_header_, const DB::Block & output_header_); + const FormatFilePtr & file_, + const DB::Block & to_read_header_, + const DB::Block & output_header_, + const FormatFile::InputFormatPtr & input_format_); ~NormalFileReader() override = default; bool pull(DB::Chunk & chunk) override; - void applyKeyCondition( - const std::shared_ptr & key_condition, const ColumnIndexFilterPtr & column_index_filter) override - { - if (auto * const vectorized = dynamic_cast(input_format->input.get())) - vectorized->setColumnIndexFilter(column_index_filter); - else - input_format->input->setKeyCondition(key_condition); - } - private: - void onCancel() override - { - input_format->input->cancel(); - } - - DB::ContextPtr context; - DB::Block to_read_header; - DB::Block output_header; + void onCancel() override { input_format->cancel(); } FormatFile::InputFormatPtr input_format; }; -class EmptyFileReader : public FileReaderWrapper +class ConstColumnsFileReader : public BaseReader { public: - explicit EmptyFileReader(FormatFilePtr file_) : FileReaderWrapper(file_) { } - ~EmptyFileReader() override = default; - bool pull(DB::Chunk &) override { return false; } -}; - -class ConstColumnsFileReader : public FileReaderWrapper -{ -public: - ConstColumnsFileReader( - FormatFilePtr file_, DB::ContextPtr context_, const DB::Block & header_, size_t block_size_ = DB::DEFAULT_BLOCK_SIZE); + ConstColumnsFileReader(const FormatFilePtr & file_, const DB::Block & header_, size_t blockSize = DB::DEFAULT_BLOCK_SIZE); ~ConstColumnsFileReader() override = default; bool pull(DB::Chunk & chunk) override; private: - DB::ContextPtr context; - DB::Block header; size_t remained_rows; - size_t block_size; + const size_t block_size; }; class SubstraitFileSource : public DB::SourceWithKeyCondition @@ -134,17 +116,14 @@ class SubstraitFileSource : public DB::SourceWithKeyCondition private: bool tryPrepareReader(); void onCancel() noexcept override; - - DB::ContextPtr context; - DB::Block output_header; /// Sample header may contains partitions keys - DB::Block to_read_header; // Sample header not include partition keys FormatFiles files; - bool input_file_name = false; - InputFileNameParser input_file_name_parser; + + DB::Block outputHeader; /// Sample header may contain partitions columns and file meta-columns + DB::Block readHeader; /// Sample header doesn't include partition columns and file meta-columns UInt32 current_file_index = 0; - std::unique_ptr file_reader; - ReadBufferBuilderPtr read_buffer_builder; + + std::unique_ptr file_reader; ColumnIndexFilterPtr column_index_filter; }; } diff --git a/cpp-ch/local-engine/Storages/SubstraitSource/TextFormatFile.cpp b/cpp-ch/local-engine/Storages/SubstraitSource/TextFormatFile.cpp index a05a150c2863..fcbe650d4655 100644 --- a/cpp-ch/local-engine/Storages/SubstraitSource/TextFormatFile.cpp +++ b/cpp-ch/local-engine/Storages/SubstraitSource/TextFormatFile.cpp @@ -36,8 +36,7 @@ TextFormatFile::TextFormatFile( FormatFile::InputFormatPtr TextFormatFile::createInputFormat(const DB::Block & header) { - auto res = std::make_shared(); - res->read_buffer = read_buffer_builder->buildWithCompressionWrapper(file_info); + auto read_buffer = read_buffer_builder->buildWithCompressionWrapper(file_info); /// Initialize format params size_t max_block_size = file_info.text().max_block_size(); @@ -70,8 +69,8 @@ FormatFile::InputFormatPtr TextFormatFile::createInputFormat(const DB::Block & h format_settings.csv.allow_single_quotes = false; format_settings.csv.allow_double_quotes = false; } - res->input = std::make_shared(header, *(res->read_buffer), params, format_settings); - return res; + auto input = std::make_shared(header, *read_buffer, params, format_settings); + return std::make_shared(std::move(read_buffer), input); } } diff --git a/cpp-ch/local-engine/tests/benchmark_local_engine.cpp b/cpp-ch/local-engine/tests/benchmark_local_engine.cpp index 97029dd3328d..59acf42cf327 100644 --- a/cpp-ch/local-engine/tests/benchmark_local_engine.cpp +++ b/cpp-ch/local-engine/tests/benchmark_local_engine.cpp @@ -83,8 +83,7 @@ DB::ContextMutablePtr global_context; { substrait::ReadRel::LocalFiles files; substrait::ReadRel::LocalFiles::FileOrFiles * file = files.add_items(); - std::string file_path = "file:///home/hongbin/code/gluten/jvm/src/test/resources/tpch-data/lineitem/" - "part-00000-d08071cb-0dfa-42dc-9198-83cb334ccda3-c000.snappy.parquet"; + std::string file_path{GLUTEN_SOURCE_TPCH_URI("lineitem/part-00000-d08071cb-0dfa-42dc-9198-83cb334ccda3-c000.snappy.parquet")}; file->set_uri_file(file_path); substrait::ReadRel::LocalFiles::FileOrFiles::ParquetReadOptions parquet_format; file->mutable_parquet()->CopyFrom(parquet_format); @@ -846,8 +845,8 @@ QueryPlanPtr joinPlan(QueryPlanPtr left, QueryPlanPtr right, String left_key, St } auto hash_join = std::make_shared(join, right->getCurrentHeader()); - QueryPlanStepPtr join_step - = std::make_unique(left->getCurrentHeader(), right->getCurrentHeader(), hash_join, block_size, 8192, 1, NameSet{}, false, false); + QueryPlanStepPtr join_step = std::make_unique( + left->getCurrentHeader(), right->getCurrentHeader(), hash_join, block_size, 8192, 1, NameSet{}, false, false); std::vector plans; plans.emplace_back(std::move(left)); diff --git a/cpp-ch/local-engine/tests/benchmark_parquet_read.cpp b/cpp-ch/local-engine/tests/benchmark_parquet_read.cpp index 0cb9df3c7ed2..2f62e5067681 100644 --- a/cpp-ch/local-engine/tests/benchmark_parquet_read.cpp +++ b/cpp-ch/local-engine/tests/benchmark_parquet_read.cpp @@ -23,6 +23,8 @@ #include #include #include +#include +#include #include #include #include @@ -39,15 +41,22 @@ void BM_ColumnIndexRead_NoFilter(benchmark::State & state) { using namespace DB; - std::string file = "/home/chang/test/tpch/parquet/s100/lineitem1/" - "part-00000-9395e12a-3620-4085-9677-c63b920353f4-c000.snappy.parquet"; + std::string file = local_engine::test::third_party_data( + "benchmark/column_index/lineitem/part-00000-9395e12a-3620-4085-9677-c63b920353f4-c000.snappy.parquet"); Block header{toBlockRowType(local_engine::test::readParquetSchema(file))}; FormatSettings format_settings; Block res; for (auto _ : state) { - auto in = std::make_unique(file); - auto format = std::make_shared(*in, header, format_settings); + local_engine::ParquetMetaBuilder metaBuilder{ + .collectPageIndex = true, + .collectSkipRowGroup = false, + .case_insensitive = format_settings.parquet.case_insensitive_column_matching, + .allow_missing_columns = format_settings.parquet.allow_missing_columns}; + ReadBufferFromFilePRead fileReader(file); + metaBuilder.build(&fileReader, &header, nullptr, [](UInt64 /*midpoint_offset*/) -> bool { return true; }); + local_engine::ColumnIndexRowRangesProvider provider{metaBuilder}; + auto format = std::make_shared(fileReader, header, provider, format_settings); auto pipeline = QueryPipeline(std::move(format)); auto reader = std::make_unique(pipeline); while (reader->pull(res)) @@ -61,15 +70,15 @@ void BM_ColumnIndexRead_Old(benchmark::State & state) { using namespace DB; - std::string file = "/home/chang/test/tpch/parquet/s100/lineitem1/" - "part-00000-9395e12a-3620-4085-9677-c63b920353f4-c000.snappy.parquet"; + std::string file = local_engine::test::third_party_data( + "benchmark/column_index/lineitem/part-00000-9395e12a-3620-4085-9677-c63b920353f4-c000.snappy.parquet"); Block header{toBlockRowType(local_engine::test::readParquetSchema(file))}; FormatSettings format_settings; Block res; for (auto _ : state) { - auto in = std::make_unique(file); - auto format = std::make_shared(*in, header, format_settings, 1, 1, 8192); + ReadBufferFromFilePRead fileReader(file); + auto format = std::make_shared(fileReader, header, format_settings, 1, 1, 8192); auto pipeline = QueryPipeline(std::move(format)); auto reader = std::make_unique(pipeline); while (reader->pull(res)) @@ -86,8 +95,7 @@ void BM_ParquetReadDate32(benchmark::State & state) ColumnWithTypeAndName(DataTypeDate32().createColumn(), std::make_shared(), "l_shipdate"), ColumnWithTypeAndName(DataTypeDate32().createColumn(), std::make_shared(), "l_commitdate"), ColumnWithTypeAndName(DataTypeDate32().createColumn(), std::make_shared(), "l_receiptdate")}; - std::string file = "/data1/liyang/cppproject/gluten/jvm/src/test/resources/tpch-data/lineitem/" - "part-00000-d08071cb-0dfa-42dc-9198-83cb334ccda3-c000.snappy.parquet"; + std::string file{GLUTEN_SOURCE_TPCH_DIR("lineitem/part-00000-d08071cb-0dfa-42dc-9198-83cb334ccda3-c000.snappy.parquet")}; FormatSettings format_settings; Block res; for (auto _ : state) @@ -110,8 +118,7 @@ void BM_OptimizedParquetReadString(benchmark::State & state) Block header{ ColumnWithTypeAndName(DataTypeString().createColumn(), std::make_shared(), "l_returnflag"), ColumnWithTypeAndName(DataTypeString().createColumn(), std::make_shared(), "l_linestatus")}; - std::string file = "file:///data1/liyang/cppproject/gluten/jvm/src/test/resources/tpch-data/lineitem/" - "part-00000-d08071cb-0dfa-42dc-9198-83cb334ccda3-c000.snappy.parquet"; + std::string file{GLUTEN_SOURCE_TPCH_URI("lineitem/part-00000-d08071cb-0dfa-42dc-9198-83cb334ccda3-c000.snappy.parquet")}; Block res; for (auto _ : state) @@ -142,8 +149,7 @@ void BM_OptimizedParquetReadDate32(benchmark::State & state) ColumnWithTypeAndName(DataTypeDate32().createColumn(), std::make_shared(), "l_shipdate"), ColumnWithTypeAndName(DataTypeDate32().createColumn(), std::make_shared(), "l_commitdate"), ColumnWithTypeAndName(DataTypeDate32().createColumn(), std::make_shared(), "l_receiptdate")}; - std::string file = "file:///data1/liyang/cppproject/gluten/jvm/src/test/resources/tpch-data/lineitem/" - "part-00000-d08071cb-0dfa-42dc-9198-83cb334ccda3-c000.snappy.parquet"; + std::string file{GLUTEN_SOURCE_TPCH_URI("lineitem/part-00000-d08071cb-0dfa-42dc-9198-83cb334ccda3-c000.snappy.parquet")}; Block res; for (auto _ : state) @@ -209,7 +215,7 @@ void BM_ColumnIndexRead_Filter_ReturnAllResult(benchmark::State & state) { using namespace DB; - const std::string filename = local_engine::test::data_file( + const std::string filename = local_engine::test::third_party_data( "benchmark/column_index/lineitem/part-00000-9395e12a-3620-4085-9677-c63b920353f4-c000.snappy.parquet"); const std::string filter1 = "l_shipdate is not null AND l_shipdate <= toDate32('1998-09-01')"; const substrait::ReadRel::LocalFiles files = createLocalFiles(filename, true); @@ -226,7 +232,7 @@ void BM_ColumnIndexRead_Filter_ReturnHalfResult(benchmark::State & state) { using namespace DB; - const std::string filename = local_engine::test::data_file( + const std::string filename = local_engine::test::third_party_data( "benchmark/column_index/lineitem/part-00000-9395e12a-3620-4085-9677-c63b920353f4-c000.snappy.parquet"); const std::string filter1 = "l_orderkey is not null AND l_orderkey > 300977829"; const substrait::ReadRel::LocalFiles files = createLocalFiles(filename, true); diff --git a/cpp-ch/local-engine/tests/data/all_meta/data/f0/._SUCCESS.crc b/cpp-ch/local-engine/tests/data/all_meta/data/f0/._SUCCESS.crc new file mode 100644 index 0000000000000000000000000000000000000000..3b7b044936a890cd8d651d349a752d819d71d22c GIT binary patch literal 8 PcmYc;N@ieSU}69O2$TUk literal 0 HcmV?d00001 diff --git a/cpp-ch/local-engine/tests/data/all_meta/data/f0/.part-00000-92bb25d0-7446-4f9b-8bdd-a6911d0d465a-c000.snappy.parquet.crc b/cpp-ch/local-engine/tests/data/all_meta/data/f0/.part-00000-92bb25d0-7446-4f9b-8bdd-a6911d0d465a-c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..a4db2dd16fbfd36d990c93aab7bb43e3a7834945 GIT binary patch literal 20 bcmYc;N@ieSU}E?y{70`$(_*XATQgGtJrf3d literal 0 HcmV?d00001 diff --git a/cpp-ch/local-engine/tests/data/all_meta/data/f0/_SUCCESS b/cpp-ch/local-engine/tests/data/all_meta/data/f0/_SUCCESS new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/cpp-ch/local-engine/tests/data/all_meta/data/f0/part-00000-92bb25d0-7446-4f9b-8bdd-a6911d0d465a-c000.snappy.parquet b/cpp-ch/local-engine/tests/data/all_meta/data/f0/part-00000-92bb25d0-7446-4f9b-8bdd-a6911d0d465a-c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..1e702dce838679b7bba4478846f6ffc9bc3556bb GIT binary patch literal 1282 zcmb7E-)qxQ6u!NQC5+M`>|7EEQi2Pc)X>=0bz=znFl3BTeCUe|Nt?SGm!?~qDr58? z5Pb2$r#<;F@JaAdMDQ;VVZMrQP8j$fi039*67a!U%KdTfcfRkQ?$lU24ivX#Pzb5EC?P~D$v8f7274G^02O}yIC)zwz~-(eFco14xdsM2Kl%BwT!74X zB*8?$Bw?Y4;aD~gD;E-)nk@@i zl3-QEmbVJgZW?$`c{uT+!!(IeP1m+eMTd&kR!AAE5|lt0 zWt(ii*?Q8(t7T5zCz;6{XAL}IFW&|R}$PI<~a(ag(m_9 zhcXY&QAo5c9*{m;7fC+|(o+d;5cV3)GZprZE?{fxsyYsbmb33*#@fRp zL-Y=h;;CK$dchEZ!PNJizK;k@<#pqN)mS^4&FWb;HVwh4iWfwD$ivHLiw7@DzszMl zH-j7g_EhS#4{|FVqyPW_ literal 0 HcmV?d00001 diff --git a/cpp-ch/local-engine/tests/data/all_meta/data/f1/._SUCCESS.crc b/cpp-ch/local-engine/tests/data/all_meta/data/f1/._SUCCESS.crc new file mode 100644 index 0000000000000000000000000000000000000000..3b7b044936a890cd8d651d349a752d819d71d22c GIT binary patch literal 8 PcmYc;N@ieSU}69O2$TUk literal 0 HcmV?d00001 diff --git a/cpp-ch/local-engine/tests/data/all_meta/data/f1/.part-00000-963c0854-a1e2-495b-9cf1-aa914dc7cd57-c000.snappy.parquet.crc b/cpp-ch/local-engine/tests/data/all_meta/data/f1/.part-00000-963c0854-a1e2-495b-9cf1-aa914dc7cd57-c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..98fb6204411d2f24b0b80d5b57792a05b3e32f47 GIT binary patch literal 20 bcmYc;N@ieSU}BhA^-I7k@}9iXTQgGtJM;!# literal 0 HcmV?d00001 diff --git a/cpp-ch/local-engine/tests/data/all_meta/data/f1/_SUCCESS b/cpp-ch/local-engine/tests/data/all_meta/data/f1/_SUCCESS new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/cpp-ch/local-engine/tests/data/all_meta/data/f1/part-00000-963c0854-a1e2-495b-9cf1-aa914dc7cd57-c000.snappy.parquet b/cpp-ch/local-engine/tests/data/all_meta/data/f1/part-00000-963c0854-a1e2-495b-9cf1-aa914dc7cd57-c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..4cdaff7168a77f7aef33ac16cd9f9897b7faab44 GIT binary patch literal 1282 zcmb7EO>5Lp6n*m&Ybd26^ga>@Is_Y98N7u{DMe)8QO#&{g!|*=tJNKM-&rQ;~bKeF6I#l7= z_m^)k>OgTj0}3JYO-cxnDl(3~=T9)c04n_a{o^?;!Dg?eu&W3|$Q3Z)`0JO`GUQTQ z5=@lXM>yjzQ~dbr^&6!`HQ&r|Vv2iR?K0I=S!ibyP>m6ihJ_x66WJoHoJnYEzAR-) zf>jk`7Sx#n+sH{Yo$bIzwmPgpCtIFW(sRW7_vXeUCw%;H+& zwscV##en4WIced67wio|CLm*wdTt7zjRWrq5650K$&wh=bZyH-C+4)4LT0ckK?#&m zw#l|lWj!?^9T{0lEJ=5GZ!{Bi31O!aEU-@$EMo4Us~MNLO+s~-ZJ0YqB^*mfK~?O^ zR4-(tI!>q#D>}$2*HTguR+8{-v5t9ISucDpR!E$?Uqs6S?`2@@l?1nmd4d92;i*8$ zq1=NL6cTMq2Bgo|Wzx@r^hkp1guO%aOoe@*3)tG4st&`x<%}F`oR##@N?I@s)C&!( zW!)s|@?VJYzHec>_hTMe;qa+7;F04xk)s~d10$M@xUpu8qcHA8M$OpsxbKdQwFd`= z=p7y_H_f<6L+*!P{Tj|fcVHRFucSUZ?bYk4*{4Z)d;7eu_z!}Dj02QN=Q&t*L~ zha3Lze=b@)%>@Ixa*kqmr~hzccS_XFtOGX&VKr-6)wb-0WmlKFvFE#0x7)Pa4ZFVT rwz?~gChxUY8+Et6Qt!DAw{O|JS#P^;*ICxY8=C*gH~h$S@ay*n^|c}{ literal 0 HcmV?d00001 diff --git a/cpp-ch/local-engine/tests/data/all_meta/part-00000-92bb25d0-7446-4f9b-8bdd-a6911d0d465a-c000.snappy.parquet b/cpp-ch/local-engine/tests/data/all_meta/part-00000-92bb25d0-7446-4f9b-8bdd-a6911d0d465a-c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..1e702dce838679b7bba4478846f6ffc9bc3556bb GIT binary patch literal 1282 zcmb7E-)qxQ6u!NQC5+M`>|7EEQi2Pc)X>=0bz=znFl3BTeCUe|Nt?SGm!?~qDr58? z5Pb2$r#<;F@JaAdMDQ;VVZMrQP8j$fi039*67a!U%KdTfcfRkQ?$lU24ivX#Pzb5EC?P~D$v8f7274G^02O}yIC)zwz~-(eFco14xdsM2Kl%BwT!74X zB*8?$Bw?Y4;aD~gD;E-)nk@@i zl3-QEmbVJgZW?$`c{uT+!!(IeP1m+eMTd&kR!AAE5|lt0 zWt(ii*?Q8(t7T5zCz;6{XAL}IFW&|R}$PI<~a(ag(m_9 zhcXY&QAo5c9*{m;7fC+|(o+d;5cV3)GZprZE?{fxsyYsbmb33*#@fRp zL-Y=h;;CK$dchEZ!PNJizK;k@<#pqN)mS^4&FWb;HVwh4iWfwD$ivHLiw7@DzszMl zH-j7g_EhS#4{|FVqyPW_ literal 0 HcmV?d00001 diff --git a/cpp-ch/local-engine/tests/decimal_filter_push_down/18_2.json b/cpp-ch/local-engine/tests/data/decimal_filter_push_down/18_2.json similarity index 100% rename from cpp-ch/local-engine/tests/decimal_filter_push_down/18_2.json rename to cpp-ch/local-engine/tests/data/decimal_filter_push_down/18_2.json diff --git a/cpp-ch/local-engine/tests/decimal_filter_push_down/18_2_flba.snappy.parquet b/cpp-ch/local-engine/tests/data/decimal_filter_push_down/18_2_flba.snappy.parquet similarity index 100% rename from cpp-ch/local-engine/tests/decimal_filter_push_down/18_2_flba.snappy.parquet rename to cpp-ch/local-engine/tests/data/decimal_filter_push_down/18_2_flba.snappy.parquet diff --git a/cpp-ch/local-engine/tests/data/input_filename.snappy.parquet b/cpp-ch/local-engine/tests/data/input_filename.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..3b6f61f1971f530807f113ce1f655221ed3f6ce8 GIT binary patch literal 443 zcmZWmO-sW-6x=Qqg7hMEmjrSM3oR|gC2RT_yy!_h6>lOUo9wm*lQhkTBBg)Ei@(pa z7oSNzdD(q4@9oU6Z*_Gi2qFgQk&n;suTz`g1{w%;dmV(3b%rgo+f?mEutkBksX+oL z5McMFSLU*Kc`Bo4ol1g%93ZnJ5yh5m*+TOr#tW)gS zFZ{L2P%rJ$i0BY{T z5CF2Q!=&$B`1Uj_-&keCGJ9$$_$qk>5FK&<&1#aXnpkmEL+qW&4wmCn~!ig&@3tAlGR-!RczpJt^iL zwgwpXkft=eWFg#mv0f~;g&PX5Di*`Qsk$v6m@=+tYo#NQj&?0RRq=G)YIx8(S8aHU z!1KDHAGN5#RDB$%E;*RO^ZYsC*N8=DdRq1b=CEy5;od3ci4{hA0Dg-y&c9^XUa)Rp zrBiI-rr6gBIuJaqghtcyNbteae(;BsX}miu!Mn;T_%EdBBkL-EN%mZleU8XhR0C^M zQHN2cpY@HXd>&dWs`YZnh#MvT#)P7_iFcqE@n)!~5=tc};P0ha53j_e*dIcoK zK(Qi@H-z6O7>DVk4X5EPi-**1@U&mMuQHu?S=5eqvpjE6{wb5k7Adr)L&KXk$m&^CEOSCv>!r%90tLZwqtU?1aIB z8=6IPt|5G{V@@AgGMmo4bTvFYPQv_FvyQvwg6}&uAH8#q-w{PKg*2y^mY0jGj|%ti zA04_+_#q+G;WzoDFO^)(9@Ky^zrZ=R4BEk@5Ig3BLQ=Ux$u5=c zQej(dq2YT4uQpklDwihqc$M1j@}zJpd&(2u)b0t_b3|!ZNlZ>md0x%gt{*i3`af_5 Ke`#m&FYY^=;97G4 literal 0 HcmV?d00001 diff --git a/cpp-ch/local-engine/tests/data/rowindex_in.snappy.parquet b/cpp-ch/local-engine/tests/data/rowindex_in.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..e85374feee65c5ab8b081fbb9dac6bba9e686704 GIT binary patch literal 125451 zcmeF)1(a6h-aqbP=*|H|L_}IdKZY z?RjPx5&8Xn&+B>5@f`PB*V^lN9)|J0{qE1cWACR-v9bjt!XjoRiD&ZCr|B#-eprFeoTd5Y3J%`-ep8J?po&r^=_RNw_F@**$sGOzF|m8eV= zUQ3=RvT9i9cs=^8i+U%HI1(>n`}*dW<7SR%f3?$A#fw)hEODH$Hxeg|7gpULf0G)# zMNQu39p0rD@9{n#@F5>1PZaOt;3s?%{De>alh62^FZhzL_?mC{mhY%d9qLk#`ZS;+ zjc800n$nEsw4f!eXiXd1(vJ3Ypd+2=Oc%P+jqdcIC%x!RANtad{tRFsgBZ*ZhB8cp z4v%ig5it!KFCuG1_K3@?r|*j&=|BH?VPUBvga0oM;lxF&XN<@kvGek>Q;DMrC0;~= zpiq+K3;y3lapJ_O74}ll6Pf+6W{ZaRP)>R#`co+i2cK#mCr)^}IJLsl#Pzw62_nWP zh-k36%iMU;g%|bnI>pT${R91husHFP1;0R?w5Ps{7h`&56F-b^DGLn;m zl%ygxX-G>tB1lgLGIA4{xS7n{LKbc%E4Pu2+sRH2a*~VObJWeT|;7OjMG*9yk&r*ixD9iJdqdXOO zfr`AyOT5f0yh+=(3W@2{jRnLf`8M4 z!N2KG{>jfA;xI=z$}x^}f|H!$7fy4Avz+5R7r4kJE_21-5EdtK;{Q#KN2ZFSyBw*b z&w5V&{1h+Z#K_gdT)TdqKZ(-B35y@+@3`0Y9ONf{<`9QD!cmTKoD-bn6u)qqGo0ld=efW|E^(PF1oshf2q!M_P$XdqNJt`F z1;Ub$lw`Q~4og8wf&vVRPxpobsP)$E4c&x972b6Lqyodzk}miYq$dLzxrt2NOlEE& z3%8P$+sMZ4WG4qX$whARke55i$DQQoE(%bPLflPZ?%`hU<9;6CK^`KKhbh7%6r~u& zd6W{Af$p!T$;0Gg5@;>5{53zAU&l{qSR*RJpA4%NCsnwQ1u->k{rFCy8-r^GNx|@^Quw{G>C=7n z)FHna{KTe@5)6BlN>ru_uThoPsm2>r=S^zx7BzXBcX*dtyvO@|z=wRq$9%%4e8%T| z!Iyl+*L=gbd`E5SP?vhtrvVLVL}QxJlx8%i1ubbsYueD3cC@Dh9qB}8y3mzwbf*VB z=|yk)(3gJnX8;2k#9)Rnlwk~K1S1*6XvQ#>ag1jI6Pd(hzGn(knZ|U20t|{z--gm> zQ_t>A6Wu6%RAAU_<}jCe%x3`$S;S(Nu#{yiX9X)+#cI~DmUXOW0~^`IX11`EAK1or zcJL!R*~M=5u$O)8=Ku%!iJv*dVUBQ=V;tuMCppD0oaPK?ImdY}aFI(~<_f`lK^($~ zOFZI}fP@6q7gV0sls+>OM9gm8wQY*uq)(FY;Qk<~f=xzpQjn5Vq$Uk%Nk;_f$v{SK zA`>^0nOn%htz_jkvT-}v$w5wXk()f^R44mxVV&gds|+~Bt1pB(pp#U-?uu#c=3-HdO< zG-oJ%E?r!DD0b`AZhTls-)Y-a~QvXfoxW)FMW$9@iQke~ROLmcJ^M>)oEPH>V_{K9F@aF%nN z=K>eG#AU7!EXf7al;OlB9`Q*)LK2afBqSvn$w@&HGCW6Fo~In;slW?V zaF4WrdQT3>+*xyrX*R5OsSyO6% zG{z$r>8jvZ9DNRjkBtW_ruJ(0+o8{rsBZfgETR4Lr_|!*81<*5ePo(oNym{k`W!v= zwrFj$E%C1oeuBtJMQYLz9Y32#ZJ!mIE4wa!(utqPV&bRvm5FDrS^S9|{NxUQCpLZ# zj_kC~=b}pQy1|bc411FryhTmk<{jRp7Vq&sAMha`@iCw9DWCB!nJ#pt8{O$aPkPatKJ=v@{TaYO z1~Hf+3}qO@8Nop~% zOcIikjO3&sC8CI_~c!v>OiFBWFc7 zS=Q?0ZXBI*6RKq$7g#QRk?A1qWLl8xoo?i;keszJg!` z6t`UYDsd_X85B&^ox8N5!S8Q>i<>^YR@_wae9rba`wzA(+q5^%jZ#P{h6TS=dNPoa zo5;k?WabvKa4T84jcnXbc5;xDT;wJXdAWmp+(~}!q5uUc#N8C;9`5Bn?&kp>Kg;%LW zWvcKRRe7Ciyg_x|qy}$Mlec+?cd5mDyw3-G$VYt4Cj?~}l%QSzRBbuF9g@;HzAEMOsv_!sV1`{&fMJ{ofD}?EpID`|Ic*G|G2}wj^l8}^S zBqs$aNkwYXkd|~rke&==@{MDa<|G%YEF>13bt>MDj32c!Z)9qd1RJf|5MO zwo#a^T;*2Dmd0gpLJVk``h>q z-Dao!%@Al^RJW~ni?D%>Y+^H8*vb!VV>>(ek)7;fH+$I2KKAoZ->#M*N9F(1xR1;n zY;AMg5`B)IgKJ#-(%o0%Cy1O(+)QR-f@N5TKG*C&h#miwHU$@VXkH%-a;p2zeh)=; z&2`(~s)QGgO`nbPT6T+>ezv6z=wRq$9%%4e8%T|!Iyl+*L=gbd`E5SP?vht zrvVLVL}QxJlx8%i1ubbsYueD3cC@Dh9qB}8y3mzwbf*VB=|yk)(3gJnX8;2k#9)Rn zlwk~K1S1*6XvQ#>ag1jI6Pd(hzGn(knZ|TxFq2u#W)5?i$9xvBkVPzJ2}@bVa#paC zRjei`z@YeaZ76*v^gb1=S^WMAkO~Z2&jvQKiOp|rna*v|nD z@)JLEh{GJ=D91R?2~Ki~UpUPf&T@|PT;L*?xXcy8R6!iViAy}SnxXVLyS453-=~U?P*4%%89$CUQx1LoSVJ(CE$A>o%W{9sLB6a|>Cx6|El3s)ws&a7=DHQ2 z2gOg;m=&NShZ^j?X8aSo0`yGyW3lnGZRw(>H(UWygJHLlmD|Y1?PMnhImtzC@{pH1 z$j6=J=Pn9RkV4!|Vea8x?&E$Q;6WZDl7}h6BNU|=#d(wxl;klUrxZ`{Bu`PAr+J2F zDZ_J=<$20co(jA`MPB43Ugi~Er4p5?!fRCJb*k|O)p?T|yhTmk<{jRp7Vq&sAMha` z@iCw9DWCBlxi$tXrMhOvxe zJQJA6BqsAcQ<%y$rZavw#>UU_ z`IkrCFn-iv*m72|l2xo`4QpA)dN#0;O>AZhTls-)Y-a~QvXfoxW)FMW$9@iQke~RO zLmcJ^M>)oEPH>V_{K9F@aF%nN=K>eG#AU7!rorP7PF&&~- zsYydx(h)&=GLVs*$i&TL<`%MWD_OaXY}`(Ea*&f;fJVYc9Q-nt-N->J_C?zP#V?0hNp5RHIqBKwQ z49`-A=P1kbl%qTqc!7$%$V|;L%ILJ@@ z%pneQgrgkeI43yCDSqKJXE@6_&U1l_T;eiUf)_p{%8~NlGVLRC29J|Eaz&q`=iWMu zI&vz`)ycOYa&9L(If#y*rF{m^43)}t@snQsl!%F+z0=k#7b`}l*TqlX@S3slbLmjq z_BWh-Q-fh&@ipJ@E#Fa_I@F~e^=Uvu8qt_0G^H8MX+cX`(V8~2r5)|*Ku0>!nJ#pt z8{O$aPkPatKJ=v@{TaYO1~Hf+3}qO@8NoR)Pu)s_$z2v~AIJgZ(1E zKl!Ev!*=i^JK4o<_OO?I?B@Un`H7!7#9@wblw%y{1SdJgFP!EKXF11tE^v`cT;>YF zQ`2z>Cob`bPXZE>h{PlzDalZ~VJS#SDpHe%w4@_=)l_;ikdd3n#LZ;p7P4?FS-FjD z+)j3K5R_j~dROD8Zr3Skev>}AJuDvQfBFdM-}l(M?^8bGbH3n9{ue18*;G#jN3-a2 zD18ilrcWMRC$zgN_QeOyqq?nyzp^E*XiXd1(vJ3Ypd+2=Oc%P+jqdcIC%x!RANtad zf72PYKN|Lt>!O=+eN1zPHoyHi??C+&S2w=}d6Sb|dpRVq=LD!fKjUZ)ywP@Ola!CTbiZQkKsYVjWL z^8p|75g+pjpYj=>^95h>6<_lW-|`)`sY6}rQJ)4hq!Ep2LQsc64eGVf0#NO-4L02H z0aS$;)`FI_qBU)3OFP=rfsS;dGhOIPH@eev5W$tqT}hPA9? zJz6ug{q5+w9dlx)0ixeLnrK7R)of$(zrgx+;vCQZTi$E`Z{Fze>o||RKX@45@j&!h z$Az}P&1|`|UTB#iHijNd9NaxTq~juam?AtvQHoKVM=8O7(9GH&P5sEa(al&dra411 zaCLf3o1OgX5GarK&dVL-Lz@TN3ih6yFf#PC+%-cR`>*3v@%kH|3W93$xJzrUnScA$ z_O}j0tA%Hc9Rp1r+HYm(nWE^8RGs3+zJEs%hHYdMo7uuveqbBh*};$OWEZ>H!(R5W zp937^Cw}G-hdIJgj&Yn5oa7Y0aGEomItA?~Iy_i!)w5forhe7ZN3Kj#;2yb`nadfgP93JiOQNFJsLk5H6i6z5S&P?E=Z zoKif&lRQOfp5_^zr3}wemggx)c`EP%6?u`Dc$rstl}c2m3a?R>*Qv%EROd}<@D??B zn|FAZTD-^me87i%#K(NXr+miee8HD|#n*hpw|qx!>QI+@XwA?DgyY@zeEpj-Q2nT@ zSOZt6hBTrvO=wCp{=^0B|HPZ^KWl+3a-W_Gj{VVRy%ySkFs5VOW51n&I}p`v|J0ea zB=KkdrPDsLK=4`}N5SZG^bFjfrJWlmyc$12bBdRI;v8l(g4bfgoV=|We!(VZUjq!+#ELtpyQp8*VH5Q7=QP=+y_5sYLMqZz|k#xb4= zOk@(1`JO3EWg63&!Axc`n>oy79`jkiLKd-@B`jqb%UQunR*>Tp@TVR2;&IOFZI}fP^F>F-b^DGLn;ml%ygxX-G>tB1lgLGIA4{ zxS7n{LKbc%E4Pu2+sRH2a*~VO?5~FH{*_&<_rye z+V0sjBjwfX3G$`@1u29!5B9zdZoPEiHAA1+H;+aNpDZz}Kg+wFoe;V&xGsFkgqMg7 zpBXt@-RhsgrXFqIFC|-l03%al;R1Vv5W$tqT}hPA9? zJsa4_CN{H$t^B|?wzGp!_{>TWai)F4+kZ3i`7!Du_Agz$P8#1cXmO0O!ih^f;*)@c zBqFh8?j%VQm-(x_BlXuaYJYSST4eFy^_PxEqtBuE3AP_}Skum(^EI2VOC$~M8h$mS z7Ozy&__xOUEALkOqjPG1bU#dF^XP_b5!0Zd09w%K;;sZ&11JccyD7{)L4DA0aq>P;Dj?PM3b*~4D;v7ZAR z zQjn5Vq$Uk%Nk;_f$v{SKA`>^0nOn%htz_jkvT-}v$w5wXk()f^qxR?95p9gr5hlu21itq?UDMnC%LG@h?pk1@ujfq)$y)J;1U|2~W<8eyy1W)o5 zrFoiXc$P9eM_Hby9ObFN3smGqUgBk5;Z-V8nJTQSEtG^7!YX+l$)(VP~90?6j;38S~o ziK&5@rJt5j*Rj9g0(QJ%PTq``|X<#PDFLwKeS&h zS$qxm2O|Hk{!N3C4+gImbvzV(j-GuxwxHp;_*Vlc2%USmkNb%Zpg~hcp1382l3r%nz{&|DWl`NC_&2GE*K>!!pM-*qD(RT$Qa*0iB5?PyO2I?{>GbfGKV z=uQuM(u>~op)dXD&j1E8h`|hDD8m@e2u3oB(Trg%;~38bCNhc1e9si7GL7lXU?#Je z%^c=3kNGTMA&Xed5|*-z<*Z;Ot60q%*0PTEY+xgs*vuBT@&nu0&JKQLC%f3q9`>@2 z{T$#RKk+k%ILr}_5>#MNeOCi$M!ln*^xE(5e^Y{CCppD0oaPK?ImdY}aFI(~<_f_( z*5eRPT;dU*1SBL8iAh3Il98Mgq$CxoNkdxF5kYz~kdd3n#LZ;p7P4?FS-FjD+)j3K zkds{GCJ%YJgM8dce(s_G1u4Yc6y_f8ZwqLJqKY6U{*fra)Ba;O; z(tj^q|I4?XNA}cJ!O<)F9Eu+s;954CGB)&n|Ja=ey`#FVkFnu@$cD8)I<59cU*!;a zAi5b3#x!SW1Kft4-8&?{x&bZ-oCkQ2hloVG2m1=9v}<-WR4mcK61)Brw`lolamvMP zfZH%|XQ$9CP4ow2cfe%|?SKnD8QKBYuj!HVH{5}s6vG~-2#-*dVif05N>Gx=c$`u^ z!IL~iX`bd8o}~=WQI_W^M|mpn0u_0Ymw1_1c$G?2rV6i7mDj1p8&u~_YVZ~{d7F25 zms-5X`+UHMe8k6m!l!)3=X}AJe8ty%!?%1#ZR${$deo-@4QWJUn$VPHG^YhEX+>+= z(3Wv5W$tqT} zhPA9?Jsa4_CN{H$t^B|?wzGpD*~u<;vxmLxV?PHth{m)AWEyVe(y_zbuw4^EKSf>5 z{wWWKB#%Gpzv5>5_nz^L_ekL(IQI+Uk2cKI{K{VLd!r6h9B8sPUxC>xMz|F zw-0}TXF>n$*|k5qCnmCEbTf8}Y0l6r+^OA7vs@hm1+numMR){F9_%bQx_{TcYa%H2 z%eSlg9yp&0f~xPi{;h7b0j~Ve>fu>p2SE)_*9*QyBdYeU8w4rBu%9`^VUBQ=V;tuM zCppD0oaPK?ImdY}aFI(~<_f`=OT;0ZxWpqq2}npH5|f0aBqKQ~NJ%PElZLdUBZBl~ zAR{-CiJQsHEo9+VvT_^QxSj0eASb!VO&;=c2l=>@{MDa<|G%YEF>13bt> zMDj32c!Z)9qd1RJf|5MOMQr5Vj>K}%ZEnl`kh9qs8rM>?T3LwgX;4;oeL zH}TUs>LT`E^mNFdeY5>%J>wa9UQY$bh3Kf%vwGD}^=`NYP7!)Q%iTEL=uQuM(u>~op)dXD z&j1E8h`|hDD8m@e2u3oB(Trg%;~38bCNhc1e9si7GL7lXU?#Je%^c=3kNGTMA&Xed z5|*-z<*Z;Ot60q%*0PTEY+xgs*vuBT@&nu0&JKQLC%f3q9`>@2{T$#RKk+k%ILr}_ za*X4g;3TK`h0~njEawPHFetyP@w0hyi_fJVYc9Q-nt-N->J_C_()NwRbgsCNwzQH>L(+7J(j1 z5!^&SZZfA7Pw*Fg;mW`2Ci~BN$}@7Xt_qGJ(dSV9n1k!F{7nCtB8bVKp;6s7On3c@ zUa$5?9|rwb#eL+7=w>__)10BPPyNPw2FAVsAxNA@DM3lJd9Vd;O3&$=uSuWS1)w6* zCv!~r4Bphe)eW}5l?^W*8$M%?4%>Fav5yK2dz?}{!IL~iX`bd8o}~=WQI_W^M|mpn z0u_0Ymw1_1c$G?2rV6i7mDj1p8&u~_YVZ~{d7F25ms-5X`+UHMe8k6m!l!)3=X}AJ ze8ty%!?%1#ZR${$deo-@4QWJUn$VPHG^YhEX+>+=(3W9ONf{<`9QD!cmTKoD-ZR6h3w!EIU4G@NehdPDNeC{zordNg2QL z-*?%?T=)Ovr#mAn1Yd^ccp>^6iXU6x`gY&Y?6*rm6;lRx4KMmN{*PX-7VqPf@pr^C zs{hkp{_xLw&@8fdbVK%uY0yvrja}Sxdid4hPY^nf@i?W34xmOeCbYi67Px!@D1S@< zt#018b7*?zx*Z5b!^_16P_Na~Yu_+{RAJaJoaPK?ImdY}aFI(~<_f_#wZ@{MDa<|G%YEF>13bt>MDj32c!Z)9qd1RJf|5MOYe=*rnqOJkmWxNh{L1jAnBC0^zgUZoP1slsbi<#np@2Gx0! z8oWhK-sTy5`9`$KJLmJVTCN!lP z&1pePTG5&|w51*G=|D$1(U~rEr5oMpK~H+on?76(pw&%xw~qP#tC-PG->B=@f51Ce z{#Ccx|M;8JQk6V%3%-fd5kLAI3LxX3xyL*A4PE47_aG!l6$H?)_Nf2WH>~~9 zS+zgCIsZHJEC;Q7=1;nic~XYrzyBSLZZJ3Fw|U|2u;Gk}2%VlYD($}omAf{~13G-DXc zIL0%9iA-WL-!p}&Ok+ATn8_?=Gl#j%V?GO5$RZZAgrzKFIV)JnDps?GwX9=38`#Ju zHnWAT{J=K0vx6Vm$u4%YhrR4$KL;KdyG)HsY2Dr?j4RFDajD7`N^9f6DI0#aTVb4*P z=P5^dD)0god6AcRnOAs~N>ru_uThoPsm2>r=S^zx7BzXBcX*dtyvO@|z=wRq$9%%4 ze8%T|!Iyl+*L=gbd`E5SP?vhtrvVLVL}QxJlx8%i1ubbsYueD3cC@Dh9qB}8y3mzw zbf*VB=|yk)(3gJnX8;2k#9)Rnlwk~K1S1*6XvPo}Vo-#7Ewlrn!Q~dMZ@3LxC5BC4 zB9oZR_e^0b)0oZ-W-^P}%waC`n9l+hvWUejVJXX4&I(qtiq))PE$dj%1~#&Z&1_*S zKd_DM?BGXsvWwm9VK4jG&jAkd6F+l^!yMr#$2iUjPI8K0IL#T(a*p#{;3AjMnsy+V zg*&rl&5lrE#l8jMa@6H4>UGAcqDv zrnc^X%@WYBUIABrNR9B!v7?}4Lr*NaVg4w>uq%Z5P2&(wT;dU*1SBL8iAh3Il98Mg zq$CxoNkdxF5kYz~kdd3n#LZ;p7P4?FS-FjD+)j3Kkds{GCJ%YJgM8dce(s_G1u4Yc z6y_f8dpRVq=Lpa6s7yP7{UPW|*lsDq=&uD|{U)T%!DI@NfC>byw}-l8UN^A7J) zi}!e+5BQLe_?S=ll+XB_FZhzL_?mC{mhY%d9qLk#`ZS;+jc800n$nEsw4f!eXiXd1 z(vJ3Ypd+2=Oc%P+jqdcIC%x!RANtad{tRFsgBZ*ZhBAz+`EzOS_FTW&0XIDADmKCu zXe6T;%^1cqj`93`xBWl(R)RDod;LQ(9+@JIE07~)^!aN1G;7p#`)}WakSa|OKdE(2 z8q$)E2-1^*jNC*fZYDFgkcC^x%57xhcCwR$oa7=mdC1EhE~ajBdskW12G*KO1|W9-ZLo99&uL{XFF;k2Vj6KuZo!?G}1y^tv5z zWutb$#pKlCrUzSx*5|HU1F9aLKQ?~0cIiIthVS7~gkd8X$tXrMhOvxeJQJA6BqsAc zQ<%y$rZarCLVmEu(%RcsVfP?(R&m7_~M>xtcj&p*OoZ=TwbB42=<2)C*$R#dwg)luHhj8K& zkN6}YA&E##5|WaPJjjg8B<;FBCr};YK$YxivJg9XtGaDe5xz zvP;A(yh;p-lPU^ zQIofMhj*#Pd%VvFe8@+9%qM)xXMD~Ve92dQ%{P3@chsg1b*V>v8qknNG^PnnX-0Ee z(2`cPrVVXrM|(QZkxq1`3tj0(cY4s1Ui799ed$Mk1~8C83}y&J8OCr%Fp^P>W(;E) z$9N_%kx5MEd!{gzpaO&HyBa^;CidDKbKiH}76c_2Hj`P*W)5?i$9xvBkVPzJ2}@bV za#paCRjg(WYgxy7Hn5RRY-S5v`GIY0X9qvBlU?j)4}00iehzSupZJ+W9Oei|ImU5L zaFSE}!fDQMmUEov0vEZ&Wv&pWyy6f}T;dU*p!|Z;yBa^udkky)+YQ)>(gt_YiA78j zl9G(%q#z}!`1>yVpZ-msku`NyaJ(IT4#kfR2rCE7>>P6s7PAKQPE@!37tX2u(eL$+ z93S0`6JnY(H24`^xAuv6SHmX=oELeCmx&Ia4x{HD4Na9rhf3@Qg#5xMUrhLnU%4Q9 zYshusQ#`zEZ1`**zI)>hHz26Mu*4)GDalAq3R04a)TALT>4+db8OX>@Wa4Hra|>Cx zm8{%GHf|?7Imk&aa+8O=+(AC>BtLgifPxg_ZVGb`_i`Wi^8gR>5Rp7g5gwr^#VF3B zl%OP!@i?V;f+u;3(mc&GJWCm#qb$!;j`CFC1uF6)FYz+3@G6z4Och?EDz8(GH>l2= z)Zi_G`U`6BYWR%ny`uIFUje58!``J9@9{n#@F5@ZF`w`$pYb_g@FidIHQ(?p-%*=7 z)TJKvX+T37(U>MQr5Vj>K}%ZEnl`kh9qs8rM>^4&E_9_E-RVJ3deNIc^ravD8NfgW zF_<9?Wf;R5!AM3inlX%J9OIclD17E5i0HX$>ABE^LhRUQV$@aa-*^Q((e|j{aEA!| zk)7;fH+$I2KK65fgZ#wL9O5uXILa}ObApqc;ulVHhO?aGJQujgH4k{E`zL?bXJp27 zu7Hl4qR&?YXjb!$n_`|shzX!f>4N))|H4_dKl;tykyWFc@%5PI37fQrNf(6OOydSA2uAolUo0^!wS1E|izX2FAu7L$Hw z^rH&HCNY`snZi`2F`XIAWEQiT!(8Sup9L&r5sO*EQkJot6|7_xt69TZ*0G)qY-AIg z*}_(SU>j;MYzIHGlU?j)4}00iehzSupZJ+W9Oei|ImU5LaFSE}!fDQMmUEov0vEZ& zWv&pWBjXTGT;dU*1SBL8iAh3Il98Mgq$CxoNkdxF5kYz~5Y%5#d-^D}_n`ffCL?2R z{H`1QD8R6r$;>Te;a0M88`-#>?BpOPxyVf(@^T0HxRd1KE+|L6% z$U{W(FhzKTq7PdUm{ffuOAi@e0kyuzzg zqB2!@jiCI3($kuuw;zn1H}L3hC*P_?UB&)hx=!R-@kkHW8rS2ulW&Wnx^1yvcL_^b#&TA$l2xo`4QpA)dN#0;O>AZhTls-)Z0Fx1 z?IUu`{D;=iVutOJsUkuRnmYPyek^xHzKCw4dhUvS30x34m8rsOR7J}N3qa>jcNwJJ zWA`8YdJkN=8ga@585Hb+YqjOf(rX?9ihT)OmZ&{&iGv>*+5>lB$KK~|6hmq;>Sn zGn&(amb9WZZD>n7+S7rKbfPm|=t?)b(}SM$qBni$OF#NEfPoBRFhdy1ForXNk&I$A zV;IXg#xsG5Oky(MGli*4V>&aK$t-3whq=sSK0zf0RjAWKF?6Wg$yqUbf3Axmr5LuD zB`jqb%UQunR3#U25S$gJ%4A+I%v#uK)xr~&Gg?(*onNmDUo+KCGzg3MBd$$=#SoP|5g4bfgoV=|We!(VZUjq!+#ELtpyQp8@=nQ$KP>bTiJ3 zY0h|2k+FT|q&Z2ij)8*Ed7Wy!fi@2|z)f6zbjS@hz*P^c8nXee(YWR_LKmm#k zzzwJwo+)+^w4l@Ub}=`AQP;6we+Q}(3`ioD26yv!@SN+l{&h1aOc>r~?ns`Dl_c#E36%{#nH zE#BjOKHx(>A}GM1_^zf;$L539#Y{e2mp&>m>@z;+3%=wlzUCXgzr)R7%rx`U75R*Q$qAp{zT@L0j zmwC)*0Sj5gfArmU`W#pOS9jZg)*@MCw)8H5j@zTpSM#UImgRL~RzhRu;IgL=ZX0sw zoSfw1FTP{#&yM@ZccPo|-I(SK<1O&kj8za@{IWjqrSNeJV8gY13rv@X!zZok=)F7&eB&GwZXy#mlbKt{!mVWGHi7~SichbFMn5fUFTWBq|8QOYsKBrs*Qv%EROd}<@D??Bo1p%J+S8h$(a(|j zYZv@Be`-Zt#@=)JdY=#YkdOG7PxzF-Z?pgD&v{0!(pAB+I{K{JLX&Xgx=mRfQv@;T zvnHzB{*sex8G`R`isL`3aN-h=_#_}9iAc=8$eg#R$=hi4%!Ww|rfpm8$&c6xxY$hy#pO@NaGwj}=fw8SvrQ=c-VV45;YDKO zXVk*c^<$m@zi#-W2E*RteLmnrKH_6O;Zr{2bH3n9zT#`X;ak3=Hg%{=J?hhdhBTrv zO=wCpn$v=ow4ya_XiGcV(}9k3qBC9SN;kUGgP!!FH~$}dcLE>7{Xc$u6BEXLRMxuh zP*qe_RnrzlRZ+babt`qZD7}M-h`8_jiu=CfzVDleD~Je!AR-7N{;&5O*;$)BzWIN@ z{rz(E|sVI09x$Q~$rAKO;%evZdq z4>aHRxphD6g0_FZ~+%_36Z#rC|p4_uHqW5BL+8c6SokH+qi?f zh{HX^<31Ab01xp9iFk}Bc#0%ELo%Mj#_r$&Ph@}sFJwd}sK|^g$ck*pjvUB|T*!?) zcnNvo4LkC|f&7r&Pj)@3sdqoem*0!m_bKNfghKh`U36jg_97^XVknLhD2Y<|>s|Ig z{q#=X-`P|${bAg)+4SxwcEX`eFLpnE+Ggv=rs>bWh#ip6E1&Yea#$_B*LCpSXB2Y3 zSy27sPuSqCp_x*;AL(+spewq8s!R8CbmoKc9Ct~r`>90tQ_k#u0yiu>oksUl+q1E= z`xzP-bXfmBDV0w@?1Q$#D1xFWhT4J@Ie`r#mgv%S5O|Wq5>+S5?(`PypAe( z16A=Rs^KkE$J_A5J9roGp$2NA7T!l~)InW*fO_~4_3;rJpdlLJV>HGm_!OVvb9{j> z@fDijYcxeOG{-k+ftF~6)@XyaXovRr79G$L-=PyaqYJvC8@i(hzQ+&fiC*Z9AJGSW z@e_W=FZdO|LH0n|`%=1}vy+n7>08&>YSzsWfkAVolAOvDCf-nTZ7>Z#S zju9A%Q5cOe7>jWjj|rHFNtlc&n2Kqbjv1JVS(uFw%)wmD!+b2jLM*~!EWuJN!*Z;^ zO02?atif8WLnzi`12$q4He(C6VjH$&2X7w_O*yoVa7iCTCcwNVFk@d4^l z!w(%^rbLv+btR<>l5VFvdf3A zHM^j(Hy%t%V-#Fv&o<63X#ar7qG^pjU^lcK#33BU5gf%a9LEWqL>NvX9H(&xXK@ba z5rGT1h)amXWklf$qHz`1a2+wYft$F6Slq@P+(jJjAs+XUfCqSpM@YnDJi${W;Te+g z95(9i0Z(Lr0xx7lCaB1aEXay%$c`MyiCoByJa`Ft;SD?T!GZiJfPyH5!YG2GD2C!F zfs!bN((pkUl*P*^hgTpwpzM4pJp4$f?_CQ-jRKjbhjMq^GZ=fpPL^ZsH z>UbNzcn9y|J=8!=)WZ9yjXJ1{4^R&uqCP%C12jYhCB8xve2u1P zhUWMNEzlCJ&>C&f7VXd;-=YIL;yZLgXLLbVbVGOa!1wq8J<$ujA$yiJ6-{XaRbmfr5(w^V+;prwslV-VcR!3Q=w2f;~?^8ah+}88{e++K3x36K_#J=15B<;|1K^K=2*4l&VlaX*1i=`JVHl1P7>Q9BjWHOD zaTt#Yn21T3j47CkX_$@~n2A}KjS$SiT+G9KEWko6!eT7JQY^!AtiVdF!fLF+TC77T z)?))UViPuF3$|h#wqpl&Vi$H}5B6do_TvB!;t&qw2#(?yj^hMQA`GVxj?*}Uvyfd; zc0;zBejx7TyrHMfD>1tDL+p#T3%H0&h{RR071wYbF}Q)7xP@5U#vR;69PS|= z_mO}Hc!)81=D#=yE=b#K|3!n=(tC6*y`y4$PZ(u9X_iy}2(&T$($SnL zAAzLP`4N547t~z7+*y=(CQ(0P;4-#gpNEG>M_VfoT1hvx@BH|=`j-T$41ycm#na*Z z3^e%W>HcY62Z3GCRt9D9GRolHhl38-o<;U zftsj=_fZ>lP!}Jd9zH~Ue1ryQh(`DrjqwRS#b@{&U*Jo8g(mnKP0?&yK<@dJ9I7kcAI^g&r+ z;E#a_z#zyDC_5jUSMPkz&x_lU=3zMYK-&-mV?!849l?sE3pczu?B0g4xw0&4cLfH*o-aM zif!1A9oUIo*o{5di+$LS12~97IE*7WieosA6F3Q~N$0~LxPGB49=$jSAxwK2JH@-> zaGb`!{V5^^O1J#Cdbj;&eNX0FwE!=GOm7;uY&iXUpe+;5@6a#!Z+v}sOi`;9khcy0 z;K8;3#{WF1{9d3=MMZ|6`}!G$>}M8K?}2WvIXNyzN)IHx&QJImzrg5QwhVi|HI0uz zU&=7JEM`|VHgdz$G={--@+{}1}N}C zMr4AD%*cYQ$cF65ft<*N+{lBMkQd&tBOe^dj{+!&LMV(PD2iezjuI$|QYZ}{ltEd% zjBaHB`pysDd{jyP)iTDLqik-C0ld54fp(5yU=ddkfX^Hhl38-o<;U zftsj=_fZ>lP!}Jd9zH~Ue1ryQh(`DrjqwRS#b@{&U*Jo8g(mnKP0?&yK<@dJ9I7kcAI^g&~r~kQLdG9XXH_xsV%q@DlRE8+PP_1Nl(^1yKlvQ3OR%48>6bB~c2c;e#?LixcRpMO zE@)FWJ`2-!Dcii0Z+&UljK-+7mjMq^GZ=fpP zL^ZsH>UbNzcn9y|J=8!=)WZ9yjXJ1{4^R&uqCP%C12jYhCB8xv ze2u1PhUWMNEzlCJ&>C&f7VXd;-=YIL;yZLgXLLbVbVGOa!1wq8J<$uj@gw@6FMh($ z_yxb>H~fx2;D>(bj{)$?!849l?sE3pczu?B0g z4xw0&4cLfH*o-aMif!1A9oUIo*o{5di+$LS12~97IE*7WieosA6F7-5oI*HG;|$K? z9L^&G7oc}O9DBPnb-3S)$KEb#uVVie-(VN|Ctqak-~E3Q_|_=I>mO51<2I%H*?ll# zrg!%yR18&xEJ54?2gxNte?f1KvWy)Qbu?NoXr;2AQXZLgN=-Cq7VwB2#TT@ilYQdq7+KQ2W3zeFQXh@L3zB2 z3aE%mcny{DI;!9eRK=U9hPO~1Z^IYw;9a~2+5cqUOX+@=?OEN+82RHi?125x_C9K( z4(j3q)We6UkB`s*4bcc6qcJ|gr}zw?;|qL=uh0ZvqbZu9Ile&)v_vbkMjNz6JG95Q z=zxy+4xP{$UCcPSJ$^t>^g?g^h(73xpYSt&!LRrYzvB=1p&$BV0Q@l!0T_fp z3`P)!Af@*iz9%BcdB~a4vxm-J`(T%5Ag_zc#J1_iX=QkGM+P}+E&=>kN?OcYU%wKM84Gu z>jizwxHSgAZCE^eQ?``!|7)!7e%di|PuoA1V8f&l?>kx|d*no}Lgw5E3t=NX` z*nyqch27YLz1WBSIDmsVgu^(3qd11+IDwN0!zqO0G|u2G&fz>FZ~+%_36Z#rC|p4_ zuHqW5BL+8c6SokH+mKyRcEgm8C@6HuX7kHex1NZ7(RL54J@Ie`r#mgv%S5O|Wq5>+S5?(`PypAe(16A=RWIvRBQ16NO1}OUR(Y`MX zfU90u-cA3Vr-%IYZkyNiKk4#mzEjvzGEFsZ^=^n`a0^z9-T&eMxM|vEn@;+F(N)&| zRmaxSd(l+iYepHbo8{C$1a0nrFEC@u_dn9<3`P)!fSStyxX^vuH|l+b+W@#uwiYq~ zPQE2NwrJ~AKH4~kU+dRO7~|pLnaiV_r``D>=;XH1r}Yn2shoz&9%y?TzIX@k;yu(r zP1M5ssEs5S95UC`Nk~`=_2AQp9WPfAyQ~KjSP<>BfEQ7vc4f z=_BKo4X2O4K6LozR{dqpIsUCd5qa0pknX1uK1O4Ff=}@oKF1gM5?`SSzD83tLvwtC z7HEl9XpJ^#i*{&_Z_xoA@f|v$GrFKFy5ZmODo(y*j53Zj%c*xi$Hom`uBLQ9(&YqW zD29QW%jch4tJfXT#|yZOtY-YRXA>Fy=4tj*3&x%tmd1BLT|CP=`=3X<5BsG#`i-5? zb^|wY3$eJ3JGhHD+(SI>BLNTa5RZ_E$9RILNWwED<2h_pzyqGh00myih)htC8Cj4O z*^nJMkQ2F(8+q^&^1>T-zLGAN6eQ4X)5JYGcw zR754bhRS#yRqzI?;!RY;Td0n=;fr_hF5W{8)I=@3kJ_k%y7&O~@F8Rel$|f7{~0%7 z`U>8KXy@O#^*`)^wuWegkI@*P;8T2t&+!Gm#8+s7uhA6E&>Y{O1zMsNTB8lxq8-}f zTXaB2e1}fxj4tSkZs?94_#Qu?CwiebencPi#ZUMdzu;H=hTriA{Ll~mF#!G;hyV;i zAO<4{LlBIi7>3~(fsq)6(HMiVNa=r$$8VYU;!(J9+RNB@UP2~dBL2;;W&H2{F55iI zQ;8{3XSni@oagDA$i|ZCv2m;SKOBWyb>PB|7eD?y(KcMFN7ssaG5(m#KiE8x0Sdg3 z5t*POGycKLtflvK&_8Q{`v31`9eisQ)eHK*achi!8+h*4-K;4+kn}pkF#;n&)#am2 zeBiU$`nNW2gAnS{1LZZp1qxanRL=Sd$oVBuQ_t$o{%1wl*g37fG6 zTd@t>u>(7?3%jugd$AAuaR3K#2#0Y5M{x|taRMh1hEoW~X`I1XoWprU-~uk<5+ZRK zQMiI=T*Wn9M+|P@CT<}Xw{Zt|5r=z-$9*K=0UqKJWCxU;FQxz4;CE(YnxB8z18q-{ zgl9;`bJ+N6^MEHZK!F!BA`?_(Miyj6He^Q*)X1y}ylua)XMmn|jJ zJmXgHd^q+NuyJVUi(diF*EZV%Hq$~Z!eT7JQY^!AtiVdF!fLF+TC77T)?))UViPuF z3$|h#wqpl&Vi$H}5B6do_TvEl4KL#48)ua9o>@-)!_UE!D=uV6>3XEQ8HLdp1EcGC zI`-66≥P074bI9v`#o*?iix}Yn%p*woud;EZ&=!M?+5q;1XKjCNmf?x3)e#am1LqGJ#0Qh4d0x$@H7>pnc zK`@457=~j6Mq(63V+_V(9L8e;CSnpMV+y8X8m40gW?~j*BLs6G`=9K4DSgk%F}G%$ zZ~ELm`>+Gr7GNP3VKJ6qDVAY5R$wJoVKvrZE!H6v>#+eFu?d^81zWKV+pz;Xu?xGg z2Yay(`*8pVaR`TT1V?cU$8iED5r$I;$7!6wS)9XpMBoB0;u0cp8Bw@`Xk5iLTt^IU z;3jS%7PoN+cM*qs(EA?7zXgmN7p=dYa}K_a*Ivc`t}n2Qd3FEmEp|;_du!oO7>MTk zc`;u9n7$~c-N=U1kG;)0HDV?oTD3y}<_Lr@i^;o&|G{Hw>HP;pzSE2{PB+V`cR#lW zjUD40el1pb-WE=dVjRX}0w!V-CSwYwVj89+rTf`(`Op|TbC=IPGqhK+nY@6`!fb?K z4(4JW=Hst7+5f}~rxmaBTN7>c6=N}?1>!v|$h7B8b5UO{=hiVFBYx>Bn3W4mtyqmT{Fg6jVU^;*)7 zfqeeSBL9IQZ-epFd;%t75+=jwglw_J2W^`#S%3&8EWJvBCs@o%MM{2O^_ zsgHjfFgrfd{QgyYM{WH_bTf+az8teK8zGp3xtNFfSb&9CgvD5brC5gLSb>#Th1FPt zwOEHxtj7jy#3pRU7Hq{fY{w4l#4hZ{9_+NvX9H(&x zXK@ba5rGT1h)amXWklf$qHz`1a2+wYft$F6Slq@P+(jJjAs+XUfCqSpM@YnDJi${W z;Te+g95((?JRp0b?1gMKeGI~tbqgZW{0@j+(dLDW$OILckp)?i4cU+B~TKjP#QicgR*!T&u>a-%@6&fVn@Xk?#x0vo{}dEw*-J~7Jp z)GVj|DQLi!sdICt9E6ijX9}ib8mPGpfb-ux|A_vM&E?aLTmi0&t+_b>ZpWPcQH;}e z9)fcYfXl-GxV+9!LCZIdlFRg@eh`j5(DpGJ;}d*}&+s|Ez?b+6P4G3Eq8Xaw8?-=6 zv_fmNL0hy#dwh!y=!oyo37ydeUC|BQ(F5P(2lPZQ^u~|qgTD9)KjRntir?@%{(v9) zp+5$|9|IA9K?uZP1Yrn*F%-iv93wCiqc9p{Fc#x59uqJTlQ0=mFcs4<9WyW!voISW zn1i{Phxu55g;<2eSc0Wk2HF2)-%IIyW+q>loo4654rp72)mVeIScg!o#|CV~CTzwQ zY{fQg#}4eoF6_o0?8QFp#{nF~AsogL9K|sl#|fN77)~J^r*Q^naSrDZfeW~ZONhi} zMBxgeaTV8a9Wl6po4AEo+{PW;MI7!S9`})e2Y84_NW^12!Bgm+4+r5kpW1xgJQ84j z1C*q_in*K}Qo^g~UkBIz#Di<;9f{`Kq6DvhOf8Mul53Z&6zkB=6H_EucET`W61nj+ZJ5x&cBVEpP%)m?- z-OrSJ%MR;fb=t4Bi@?j6OJwq~{!^YF~ zfG09Qffq6&6I5hI7Gy;>WJeC8KuMHB zY51TF%Hm~|!z(C{S5W~KQ3n*s^e|=;vKw;_fP{hQ48;*HtL`* zK0rNui2C>l4bTvc@G%-%ff7Rzh&ZQQ|Kr1LFy+NXGyth4O@ zs}uLlR+5(mNaw893LSA^oj(l(+KMJ5A z3ZXEH;6L!dnymrPyCE9kg^%pYXC>>DQOw@n_e-OYUzr8fyPqA0;+Ew|>3(KX@mZLS z5Kwg){Wfj%(`))WHMj48>e2n=F~0*^yZy=~4%0b5|2T&rH219L?0|MJA3r?JLvQSb zwnbQsC0L4OSdJA~iB(vQHCT&v2*rAAz(#DsW^BP$Y{Pc!z)tMKZtTHc?8AN>z(E|s zVI09x9K&&(z)6JR6vA;DXK)tha2^r3fQz_q7VwB2#TT@ilYQdq7+KQ2W3zeFQXh@ zL3zB23aE%mcny{DI;!9eRK=U9hPO~1Z^IYw;9b0j8mNg{cptS<2X*lQ>fuAw$46*@ zhG>M3(HNiLQ+$Tc@dae}lU+~mfEfO^Vdl#T zh1FPtwOEHxtcUD^viqgwWe={p?QPj%+HKzB6w1{)h3`Q`ap!_u|K&PTFSsA3Ua( z-tOLarBTLJW;yi_Kiem5JFljE_>m50J{DjhjLzq7_=AKru0L9e&Znx``OF)*@|phS zfLrHN&9k|)^Le;_-8J(&SGS{Y?18pSP>~r~kQLdG9XXH_xsV%q@DlRE8+PP_1Nl(^ z1yKlvQ3OR%48>6bB~c2c;e#?Lif<9cKtnXb$7qaC@F_mS=lB9&;wv=4*Jz4nXpV2t z0xi)Bt z{D?m2i=Xf_e!;K!4Zq_L_@N*AV*vay5CIs3Knz9@h9DS2F$}{o0wXaBqcH|!F%IJ~ z0TVF^lQ9KTF%8o(12ZuTvk`(hn2ULsj|EtWMOcg_Sc+v>julvmRnR*hzWTX%;PKHH zKm4rLUd8^0&I1@OsA6lgntJaOo-}KcXG-rQJ4J@Ie`r#mgv%S5O|Wq5>+S5?(`PypAe(16A=Rs^KkE$J_A5J9roGp$2NA z7T!l~)InW*fO_~4_3;rJpdlLJV>HGm_!OVvb9{j>@fDijYcxeOG{-k+ftHZHPxd^j zsSkRKJhDN)L2wy;-9~#6`|E4#fBC3$-(74fnRXkuY&QLipZF~g=ILX5ok!mGXq#;> zA4>OOKmJY60R6}NdY{rq|E|8?H;0d2(45Atap3LJ?a(0}DZS59+APCztN>M)VQ-g{ z4<)2A?5&%vsX6Rz$iBsCUH2_7!`|{ZyPq*zCoD*_`(YonwM9F$$G7N!j`$9p&>3CO z72VJsJ@7q#Ku`2SZ~TZp=!>85Gk(FZ_zl0~5BQ-U`eOk6F%SV5gg^{N5QZQaLop1) zF#;no3ZpRwV=)fnF#!`X36n7eQ!x$GF#|I(3$qb|Ihc!in2!Zmh(%b8C0L4OSdJA~ ziB(vQHCT&v2*rAAz(#DsW^BP$Y{Pc!z)r{>D0^Q@_cLPbzD;R<_F)&a?ZrOq#{nF~ zAsogL9K|sl#|fN77)~J^r*Q^naSrDZfeW~ZONhi}MBxgeaTV8a9Wl6po4AEo+{PW; zMI7!S9`})e2Y84_NW^12!BZsR8ItiFHg+Emcp?K7cp)P)K}BX{K~`i#cF680yIxB7 zb86GF)h~Yb$>k&Oq5t~g`d@yzG~XZCR5JB6ZuRbm18-5`r;hTSwab~Oy|m5Nn{MYv z^g&r+;E#a_z#s%-FoG}y!5E5R7>*GbiBTAhF&K++7>|Fm z%QpFLGRnBwET{g_XJmBLdgqy^(&4PcDy)Xl`ONU&6RIC5b2*?UW3HRqx=80E*M?g> zY3*EnFrv{)zc#jf=aXT%Z)sE33=fSJMzJS{3w8e zD1^c&f}$vf;wXWVD23ASK^c_A%P5CeP#&+M0xF^sUPEQPjw*NqRq-aO;Vo3h+wjFZ zco*-X25O=f-bZcJL0x=+diW6a@evxJAsXRhG{z_R6rbU9e1R|V6`J5{G(|Hs$2Vw! zmS~06XoI$BhxYgu9ncZqp%Xfz3%a5kx}yiahwOl|^RapL4?mCg4cuxT@pk+0!yahs zjUUkmeen~1#xM94zu|ZM0YCIZe++;>1|k515QxDD!Vm;wD28D;MqngHVKl~IEXH9x zCSW2aVKSy*DyCsNW?&{}VKzcA2XiqG^RWO6u?UN?1WU0D%drA0u?nlP25Yenp;(U% z*oaNo40##L;=R<%J8sFwJ$#f5k=;l8+Gm89x_<0`J~r~kQLdG9XTKyEq`%O&-`Uo0%?=l z&X{3YTXUHlmy^lcGrwfiAHtR;n}^<KBHFWm0J(5osdP6zvF~HHs>qz+_VMNGK)} z=Qd{5X2y(bRLP@AdZNmWl93;-O(d!!5osdP6p7^0L~U-wR<{{5B;IN+%j`)2CYOh3 zc@!xXQRPO-?AAo0DiV<<5>1gvE>YAhMr^g4F+<|6=CXChjLSo`JW5tQvD_%x+!9Mv zMIw?|qA3!|C5jr#5}DLOQb8sX7c^$o_Qs6ML$o|f4y~znqvT}XwTVPkBqE6=nj(>0 znyB9zvDE{{42k!f%U;s{kY0F*mPg60Czcx}k6U7isz^iq{d5D%rDWWHq8>OgQVu`9qL=sChMIyOGQ4g_1 zCbd!~eTKvpjal_4W5(qnS{|i1t%)i(N(p|rHj$``M5KvCQzVj06ZK~!wtCT+A@K!s zSsCLM)#V{t9;K9?SZLQ zC@;GumZ*wEq=`gRB$8|5aF)oVR+kq(CKA78%&G&78JCA>d6aTmQ|(46&%A3BiK<9M z5=%5iBDpkC{f*daj4?yv>*li8vdB|;;UQWcr2>nIDmO|+wB zkw`94)SE1kNo|x}pCNHWV^$q+%(y&6%cHzQYof}H@*Y22n@CheBGN>nDH6%0i8{fE zttJ^WBz|fxt0TRxTU5!T)Z_yi;W&aeiY~ZEMW9JVeW*e4{nhZj=_xyEc)iibN!_L{lV^ zOB1!75nJ77%#e7mxh%K0Je3z7qUBLqv6!fGqqO0NOH@UoDiV<<5>1gvE=|-tMr`#* zV}``N&1GAS8JCA>d9)9oL~SdH+q)%}sES0Si9}N*l1mhI8%tzTOUdT}CK8u4X4S67 zjLSo`JW2ux(w%?d>d5D%r>8dA|8>O3DVu`9qL=sCh zMIyOGQ4g?0Cbfb?pCR$9#;n@Mm~nZCmPhG9Yof}H@;yIXn@CheBGN>nDH6%0iQ3nQ ztwtC#BtCC0D=iXi9}T- zB26TkB9UC8sFjS^>KJ2&#G}n+^1tF`a(RfBNAY7ZQRPPI#}C&g5>=6iG?8eE_Fv~+ zj&sSsxRyz+B_FSuNL{N0*JR7E1vM4~AY$t8*^|I$<@ zRsKt+OeB_nttpfGR$+a{kwh9%U$ti7GeBFt;WWRgs7^k!XrUa*3kKf3T2Am4AvL6N%+NDafS0FGn|+ zTppt3QAV(ssB)u>bZa6}6^Td_iKa*-mndp&Bep76#FmM~av5!zRJj7QOfC=6@+hNO zOjNm1#<(?+sES0Si9}PhYe~EORm+8IWm3N^rrk>7FN|4rx-sLLSn??2_(4>;QO5Jb zwTVPkBqB{Dnj(>0ny52aB9p2ez+r~O8H#I*Y-&Se#^oVe9%Z7|RJ&0oGw<3&qAC)R z#1c)BNG?s(Ml6v@mFphKL}IyEl1!>x=SL=&hiG|}DJ&+c+$huBnn+YdBGN>nDcTi- zTuLn$D3M8RB}c26NZitxRplBFGPx#}Jjx7y5OuoD&*X<|6N#!wM4Cu6MY~#rOJX@O zT_!b0NqvUI*^OECD`Unrs^n2-^Mj~zqlECowTVPkBqB{Dnj(>0ny5`!B9kg7xyeLg zIWtZs)vJ^~Rc@4pZcQYrA`xjK(G=~}0GCqBNMxB*ha9kDB5^)rR&8y}xF(i7%3`gl zcH{mPiy~1KiAZ9Jrbr~0C~6xcwkpG#WFoPQg_23l>7&oMJVaT_LS-30h$=VA3VygY zk*JD9q=`gRB$7)LHJ1@v?Pbi6xTm>HzQL8rBuQ8*I-g6K9PQ_Ry$H$RkT#;<%$niAc6aNgUv5)a7Q7r=>0@MtB+< zXr!mJx_7vvM>#ja(^B1sQ$5XX6yj;!MhlH7GZtB*#BxWEa%QD9O2RsGdE^%Bb2+xl zQ{P6jcFTY_+_xxWnc!gF2NZr)jc#SgI?V!qcb?G8^awvO4uZ8l(^{`jBOM( zH-n|RM;04VrY*5Vxycf}UM~UjM zSgM;mNHLb*9Hv<652+|i>%PO(NwnyDDu=;nOIvW-qGRg67gnI*~{ z?&wj@t+hsZxKS~-QOpkOa|zpLLdEEXkJl-|OW#KG z$9p-q(G%_%Q5c)!d@lV%7+R&h{Kqe}Mme`!9?X_ryh6g6ue;pQqeO>Vqxfy{GPcp3 zU0#+geeQslzKs?hv_yHx9X(3a32T(((`J;|i`M577HvccxoU|L#~nRNM65MR;ynf) z=^bLCHA;B05oPXkOO$x-=us{PWVF=f$&if38pVvVzAuQFkWsJ8(uo&Y@>@SGwR!D;VKDdR5yt``g4ilIy=s7X^z&m}(6x<(hTWz@IP((9HePq?E;xpv1IrGJ7MCGLqeN`zk~J<8&K znViohkvn>n=)g>tHS!yl$=F5@$7Hhny+lniqO6>3iIU76{ViI|ENc}1`I(Gu6u;CO z<=W~@`Zfw(lgYV_g4deQC2^BA%9tJIa=-nVEZZpNh&%=MfYnDOoKf9?V`l64aBGxd z5t)o_6l}xzKu2)UA4^vup}6u>!PdR^k@x+0*?ikx>n>9+n9`m^* z9JYhsxe$o;pi95zNI%kdIA8AH;c+I*-H)D+`8*XQ|Y@<77lqU(+DC3@FHva1T z`(?4LQEWgKeH(=i%HrHcfq_|!=aMiii>10F#$++7+kdh(%1uTA>sxxJ%otm`|12}g zo%z-%K}*djNvpG1wo%Lmc?$f!gl@D%@!w=diQ8d~61*>qxgQ%kFd7(8<*8sqdQZwTIzCcR#v^bb7xC9qq+~dqeqEaXpNG*%#0GdHmhYzhix*V z%-EdO`HC6O9X-m$UCt`(S*^^WplW7zT+vX48aKUC(Un2*qY!AJB4^|JNs(fWtZ zEqk`@*uHCP?~FEY##}i(ziY<__|l$^%iK&XD%{dSIH4j?&jLBBW zp^db4@OUEFI8w>2evji3jHy<Rs37^Muntv<$34YOAW}O2}&W z&qiBchjuW@nMl^qOypdL;9c3g%iC{i8SLXqQ4}B5{!qR-reevgY$_!im5tv}Gjo8i zBV`=0)XaUUwPvAOzU61Mnw{h9|Fmir;bD*Q@Hyw>Gs1_OWz?R$)JkgRyq8+d$WYck z?yOd`#4_ouSztDw#4^-Op1ngmJSK^p_pIv>yjyA(p=Gd7&}x>P-`>AKUQa2SUzJpg zHX<9pp=jpdWJk&vW+|HcQftCcE#a(ldeLg6U9>YE_GpS$!)HVdik4X`niNac#d$Ba zqLHDjJ@C9%w1k@JFIqxPiYCwAp~&cIEm6un>pBGQk)j>ZGT6sxMf0m-Ps-1DT#A;| zFs-5`HKb_fz-mXz_-iSe`%-JdP%Yt@2(4)0P17!#zITQ<^$BiD(PTWCUOHJ9=e^X5 zMuxKXCuCsnjBl3yqQy6(X!7hGij4f0M9zEGbqL-oMcZ~kG8lVjQbT)k{=6BbXiqx0 z7j2K-K8)SJ14T0jg*#Hlj!V(pms%5sY6$}_YDGKI>u(k9L@%GfUKEX?RIG~>OV-7C zFSVkPp{zZY3>587@AMb#PH&1P&)%WPsC7x?yk}j9;C)iGm6x=l>3?ZYI>=udFa5qg z9I|w;8vOwKC%>X>=8$$r%E)&qoBL90%1}~PvL}*~o`8d&f2(dZ@d==rLmRQK*H2c6 zAxf;C)Gw9F#c1nwF;c4>$5PW@IvP_uSp|n81Mel3^PY7@#_X3B3At?fgQF_)7nenf z$C28!ipQ@go;fVvp_-%jrFia3tvPpTId77a;&IG3?c&9F*b_XYiH|?c9NMURy@awx z@^(VMmrC&xNK)Q@EJ`aLhmg}#y!fp40J?<*Qao7&haxK_shsz$D>CDN6mQ%Wt$4Gv zmpH0o4=Rv1n-q@&)M*uuUr{{s)B;D!83Q1|OdWT9ApJ6m} zXlE$sC6qOiu_XGvREpQ!BS}Pt$&eKyo-iOeq(wO4ODmWB50YXwa?^#!5F*z#P z1FveuGdk5Z9(KP1d2?vRJDPU!j!N;&(<2-yXHiJ;+?QH&Mrb*YkW>EhBFyzlf zvC_n67|k5o*$;XNWsMj@!TM>#aopd-1s{^)t-huek1q_;UpyL9JXr;YA}4W3D(5}x ziiDD*lKqL6!{}W3B12yDa!K*{IwGyVJbp#-%u_rZs(B`e6wiIBHD`>LbK7;Tczm&u z_Ftaf!P3Mhh-MD$3=X}7vPLopLBE&EU*3enQoOk_TJiXvB>lysF~yTra451;lFE6{ zx+1&CQOO>wd8BxJ3zJsy_!Y%7Pg8NEoXsM|b6;xBnV{udc|$86-{<_T z;?cw>kY*0;EEK(jvPLDe;%VbJ-2d`Ij!5w)+|-K4*F)(q9*rrUtb#+4lW8QC^PY7@ zj*+91{i2q`C>~Xj;^md%@#Rxm#p72L&pg$}k#a_k6wiIBHK*TE$r*A>E8ek%=6a>F z+vWSq`W;SRx2Lgilx7JG^Tj63Jyh12$EFJ zd)5_+&{pJ_mcu9>RgvP^rFeYj=3YGB;00xqUr{{sbRkE|Sw&Jj_oddH!N;U{<8JH4 z+hwj-D#g3$VP}`+3qGIZol?A)wQ@<(WR0BnQYqdhl9ab6k)*E8Ct>7H1X+AGlzC&nO;I! zqsm(Gq=cyyFOnqX?I-SQ#p4_A^cRoD6i-%xg#46L&U@As2@aFuO-Rs+ryp>lDpI^6 zQarv-cP}1)d1JE4uPB~*`kW)>EITQl`%-JpO~Yk;*1n;(o?N?51*Pz((zV>#uQIh!J)_rqms&b&$=R; z$Wh52spT;K@~DdRwmw?%1}W};d3tZlFdB+yo=)mWIqOu4=f2dMGgiwvH%TiVqi)hJ zp7B;?jN%haGlzDzs9r)@qgq<=q=cyyZ^~IIUhp%mcnso6&tIN?V1ULHPgcR9$Vsn~ z%6ZSaBKycu$$mo1VHR(YB5#t*O7R#X=f2dM zb3}v`FNvJ=wkM*@^-8mNH1P?fnL|7CSTCWh(MMYGq=cyyZyQO<+oSqx#bcye`inPA5f9ipN00w2P;A7&P%2MKgzX2Dn~AS))(2;z>bMDc%{9l(+8*(2FFL8yL zBpt=0F~yTra42#zzNB*Av#!X9NGV>jmcw|>qbjo7R*~W{*fXuaJbp#-%v1FpDQEOc z@!XeMb8gde?i#EY?|`{pX};z$veah;%^cd9_<9Ltjhbl1lM<#5X~6spp{FCCTrxpmrC)HNK)QDC0HvSgKyJc zJQ`CxSp|n8D2KT3)hrR6Y+M^)slN;N4SLwi##9&c6n6~!|znczsdvVs)P zeW^9)87=4hp<406lg;%?vv@S|38tAtyCQ;KLRq60S}CQ3sT42piWG0yFs*ouIZl7^ zXiV{B6 O*g;Y`?^#!5H90ET!?YYm@u-TtRjDq;V?=YR#pA6CzoK~NWgi?VSBH?| zxi7WmjMj2a8m<+OvC(P2=IOmHO?-lA=FqO{pqEhAsEt-UDPbzb8xt+X^BbWRkD=D- zDW3k>mc|rMR>7fNoI+AL?^#!58#yZ3_h>nc&$d*>KCD1qUnw2~xl=73yDh(>c;=-m z94S}0km9*7wdRb|at4poipP-fw2PLoJz^7s|S zGcUj4NV!Uf6wiIBHD{8RGnt(9wigeX>y^rGYZUJg#iN-+>8KS?iY9C1yq8Mx7GIO% zMUU2tci2plj@~xuuoO>L!J)`SLL`;*o^?gekfV});TTKtsEWK*si76`T-wDuC&e=_ zA>v56(ufq#eW^8P;C0CvM@}i;HFLdEDW3k>mL@)9Xy(wa5Tch*)~K^qN-1F~#S0}# zd3*R+t$3VIkp90s8dE%31&1ORI+0Y)d)5_+)>dT7IIVd4k!1gK8Kt+aCB@@xg|z9T;zf~@;&I+X+Pke@Jev57q8aO=l}m~yYvjC_O7V7) zq`ZCKc&&JxMv?yF(U{`ND)5auKP8p(o^?gyv=tdP!BRY`BE_pM#p5K6REsx(QH;aL zk#Z>)N6Hmnqgop}9*+GK z=EL0;#$jSwtwk36BD=DPh={oBvXjoV8=0mxFM^axEFrj3N-UuSTH_eAYuC-zxVCHT z#`;zy6hV;?N(oYelq_a_sj+6?eV+fBbI;k{bL~}%@IU9@dGh>v?%X?f?s1~I@{K(8DUDnb z4P`#tf{Gg5)c#Y#!vDYT{XehrL4i;h`DuXhBCbMJZSLM#feN?-S8zUgv=dFOu&j zJnTeD!!ut1VG8hUERaspg9tA@>T8}6&6g$w9`-a{4IWzJNwjH6{V@pXXs@6o!T`KA zh!oZ6a{>=rr1HZ7~ zj-C>fL>PcK^(En*`@O)!_QyQ%%-I&rfJX~DVwe}DT;&`25E^AQ@uI*pXIoHFW1HIV zCp>JfOv5v0TTB6-jr!6_A{gPNM}5tqOVs=Xnt+FGn%VHoVT+cyfi^4$!9{3vuJVm6iILGicq4-g*ql&0aCN6at< zcs8y~C#hzHmmc*s4}3*y3p!$`8l_z28(9$}`~N8L%x!p3;b+?i1>Pa32A-0_VT)b1fM+AubdtiD#Q*RUda;57)3 z7Ieg5I7+$7H?k&1)Q4f1 zdDi;MyfJ{5cpPn7(&r39IvNvrL>Pd#0Fk0<92a=l+MNGA56yr_3p!%B9i?358yUJx zBNs)(!kga6!}gbihmF(0mB+K~dxZB~2Y+_xwAq9_!b^|(nlqv~{j$Kr7VPYmXWsMB z5+6nz9w4ETK%*m91>jwPNKqY{5qQ|qogW^W0go1RL{pSx5xe9`@wNlW5bDWM~l5(S*Pw!T`K= zh!oXjh}_i@WB-2sm4{})qXiu?D3MaG@{LUWgYf1~P@}o>K!q2{-Gqm9fFL{^wwMAu z8>6V(Y}z8>rAK|u1<~Ap(yY9O^%Z!}YkoXyToX6Yh6hLl7opLSs{-(b|4Dc&5CJ^I z7UYMAX27Eb9nln}T;&^?fks&!|BJvgAJ0IA@T!D|P=p{n9JZJOJR1e6+iYSa;iX4? z%?qNr22Eb7oU*lYTi}23BCh(BfkRKkJ0go1RL{pS< zm2YGY8fDd(^;RCJ@Y}N;goiAMAUr(Wh$+Ccah5tsy(GN!sIR#yniru-cnj87HoOJE zLz|YgN`sJ&n2~9!0KB9BCcO7wH}Dp%NRIcs;YGru1$@g6gp_iXZ)6@CWp(J32M<(u zsq!0vw;YDY8w1OPXCpavlH5sn=}}*ET{O=@6YvnGakck6w8Z0R!$J^TghofM3cx$| zAHqBRSAmCYj{GYR&45P>NG`ztDdj5P$O1IVYIx3vw=9S4BZPOY?|F5?L*Pjop80r&DZsPQs5(hdCA{>guQ~cZ z!karS@DQDn4bQyip(P$go0jBLmn`uLsF^AN?=6TF)#RH34@oTf;h`DuXaV1{10nyN zT;&^C6(cL6(LCFN3a|5e2@g3hL3lW9F$H)w-c=_lu!NT$^)-(UT?cr_-x7F8jLC*) zZo{J`ej05i($yfOBWA{$Dgf_&h!oY>+X4>(H2LA78SrR9M+~8*l&gFr>tf`*XxIdh zmw(7xQhNvw@ite$n`%`H*Qn(UoZ`y+5sw<^BvH16mmam6Kfn)j?8)nH*rXcoTzkV$ zT5zw)`nvLj0i`F@NGm(yr4PvNB!z?-4Y}$HcGP$W8d`k}jS}8h6OHWi1=K#;pq4hU znF|lTbi|-tO1aASabA2Jc$XS#>UuTRN+{~gwe5ZQiL~C`x3g5)Q$gfUNz#8xkJbA6 zdv^Eg4PB)zrCVC`O{tFlN9!d71wA@g8z>?Ar?jg!P^nf1D*A7&dkX`P@2M5G75WGI z2Dc9siiI6LwR*L`u8@jTN2T6hD;Dfa7Pjr( zx38G|)g*z)y+Tj#Ky7EO@Bdx|+S#uWNGek!J+pCpY=dGQ!&E>7tYIo(P8}GnP SMf0cSw?D)0S-$hW&Tj#w^CuVp literal 0 HcmV?d00001 diff --git a/cpp-ch/local-engine/tests/gluten_test_util.cpp b/cpp-ch/local-engine/tests/gluten_test_util.cpp index 455abdb62e41..b89c861a63cf 100644 --- a/cpp-ch/local-engine/tests/gluten_test_util.cpp +++ b/cpp-ch/local-engine/tests/gluten_test_util.cpp @@ -93,6 +93,7 @@ std::pair> create_plan_and_execu return create_plan_and_executor(json_plan, split, context); } +// /home/chang/SourceCode/gluten_backend/utils/extern-local-engine/tests/data const char * get_data_dir() { const auto * const result = std::getenv("PARQUET_TEST_DATA"); @@ -104,17 +105,37 @@ const char * get_data_dir() return result; } -std::string data_file(const char * file) +std::string internal_data_file(const char * file, const std::string & dir_string) { const fs::path parquet_path = file; if (parquet_path.is_absolute()) return file; - const std::string dir_string(get_data_dir()); std::stringstream ss; ss << dir_string << "/" << file; return ss.str(); } +/// used with the PARQUET_TEST_DATA environment variable +std::string third_party_data(const char * file) +{ + return internal_data_file(file, get_data_dir()); +} + +/// Used with the SOURCE_DIR macro defined in config.h. +/// It represents a test data file in 'utils/extern-local-engine/tests/data' +std::string gtest_data(const char * file) +{ +#define DATA_SOURCE_DIR SOURCE_DIR "/utils/extern-local-engine/tests/data" + return internal_data_file(file, DATA_SOURCE_DIR); +} + +/// It represents a test data file in 'utils/extern-local-engine/tests/data' with 'file://' schema +std::string gtest_uri(const char * file) +{ +#define GLUTEN_DATA_DIR(file) "file://" SOURCE_DIR file + return internal_data_file(file, GLUTEN_DATA_DIR("/utils/extern-local-engine/tests/data")); +} + std::shared_ptr asArrowFileForParquet(DB::ReadBuffer & in, const DB::FormatSettings & settings) { std::atomic is_stopped{0}; diff --git a/cpp-ch/local-engine/tests/gluten_test_util.h b/cpp-ch/local-engine/tests/gluten_test_util.h index 7a4f2912ccd9..6db4e09c13f2 100644 --- a/cpp-ch/local-engine/tests/gluten_test_util.h +++ b/cpp-ch/local-engine/tests/gluten_test_util.h @@ -40,9 +40,6 @@ using BlockFieldType = DB::ColumnWithTypeAndName; using AnotherRowType = DB::NamesAndTypesList; using AnotherFieldType = DB::NameAndTypePair; - -#define GLUTEN_DATA_DIR(file) "file://" SOURCE_DIR file - namespace parquet { class ColumnDescriptor; @@ -61,8 +58,10 @@ class RandomAccessFile; namespace local_engine::test { -const char * get_data_dir(); -std::string data_file(const char * file); + +std::string third_party_data(const char * file); +std::string gtest_data(const char * file); +std::string gtest_uri(const char * file); std::shared_ptr asArrowFileForParquet(DB::ReadBuffer & in, const DB::FormatSettings & settings); @@ -94,7 +93,7 @@ inline std::string replaceLocalFilesWildcards(const std::string_view haystack, c inline std::string replaceLocalFilesWithTPCH(const std::string_view haystack) { static constexpr auto wildcard = "{replace_GLUTEN_SOURCE_TPCH_DIR}"; - constexpr std::string_view replaced = GLUTEN_SOURCE_TPCH_DIR(""); + constexpr std::string_view replaced = GLUTEN_SOURCE_TPCH_URI(""); return boost::replace_all_copy(std::string{haystack}, wildcard, replaced); } diff --git a/cpp-ch/local-engine/tests/gtest_ch_functions.cpp b/cpp-ch/local-engine/tests/gtest_ch_functions.cpp index a442c4039fbd..6f91323d63db 100644 --- a/cpp-ch/local-engine/tests/gtest_ch_functions.cpp +++ b/cpp-ch/local-engine/tests/gtest_ch_functions.cpp @@ -20,10 +20,12 @@ #include #include #include +#include #include #include +#include "IO/ReadBufferFromString.h" -TEST(TestFuntion, Hash) +TEST(TestFunction, murmurHash2_64) { using namespace DB; auto & factory = FunctionFactory::instance(); @@ -53,6 +55,45 @@ TEST(TestFuntion, Hash) ASSERT_EQ(result->getUInt(0), result->getUInt(1)); } +TEST(TestFunction, toDateTime64) +{ + using namespace DB; + auto & factory = FunctionFactory::instance(); + auto function = factory.get("toDateTime64", local_engine::QueryContext::globalContext()); + + auto d0 = local_engine::STRING(); + auto c0 = d0->createColumn(); + c0->insert("2025-01-21 12:58:13.106"); + + auto d1 = local_engine::UINT(); + auto c1 = d1->createColumnConst(1, 6); + + auto d2 = local_engine::STRING(); + auto c2 = d0->createColumnConst(1, "UTC"); + + + ColumnsWithTypeAndName columns + = {ColumnWithTypeAndName(std::move(c0), d0, "string0"), ColumnWithTypeAndName(c1, d1, "int0"), ColumnWithTypeAndName(c2, d2, "tz")}; + + Block block(columns); + std::cerr << "input:\n"; + debug::headBlock(block); + auto executable = function->build(block.getColumnsWithTypeAndName()); + auto result = executable->execute(block.getColumnsWithTypeAndName(), executable->getResultType(), block.rows(), false); + std::cerr << "output:\n"; + debug::headColumn(result); + + DateTime64 time = 0; + { + std::string parsedTimeStamp = "2025-01-21 12:58:13.106"; + DB::ReadBufferFromString in(parsedTimeStamp); + readDateTime64Text(time, 6, in, DateLUT::instance("UTC")); + } + Field expected = Field(DecimalField(time, 6)); + + ASSERT_EQ((*result.get())[0], expected); +} + TEST(TestFunction, In) { using namespace DB; diff --git a/cpp-ch/local-engine/tests/gtest_clickhouse_pr_verify.cpp b/cpp-ch/local-engine/tests/gtest_clickhouse_pr_verify.cpp index 1e023a4cd1d5..a0775964e5b9 100644 --- a/cpp-ch/local-engine/tests/gtest_clickhouse_pr_verify.cpp +++ b/cpp-ch/local-engine/tests/gtest_clickhouse_pr_verify.cpp @@ -41,11 +41,13 @@ TEST(Clickhouse, PR54881) const auto context1 = DB::Context::createCopy(QueryContext::globalContext()); // context1->setSetting("enable_named_columns_in_function_tuple", DB::Field(true)); auto settings = context1->getSettingsRef(); - EXPECT_FALSE(settings[Setting::enable_named_columns_in_function_tuple]) << "GLUTEN NEED set enable_named_columns_in_function_tuple to false"; + EXPECT_FALSE(settings[Setting::enable_named_columns_in_function_tuple]) + << "GLUTEN NEED set enable_named_columns_in_function_tuple to false"; constexpr std::string_view split_template = R"({"items":[{"uriFile":"{replace_local_files}","partitionIndex":"0","length":"1529","parquet":{},"schema":{},"metadataColumns":[{}]}]})"; - constexpr std::string_view file{GLUTEN_DATA_DIR("/utils/extern-local-engine/tests/data/54881.snappy.parquet")}; + + const std::string file{test::gtest_uri("54881.snappy.parquet")}; auto [_, local_executor] = test::create_plan_and_executor(EMBEDDED_PLAN(_pr_54881_), split_template, file, context1); EXPECT_TRUE(local_executor->hasNext()); @@ -100,7 +102,7 @@ TEST(Clickhouse, PR68135) { constexpr std::string_view split_template = R"({"items":[{"uriFile":"{replace_local_files}","partitionIndex":"0","length":"461","parquet":{},"schema":{},"metadataColumns":[{}]}]})"; - constexpr std::string_view file{GLUTEN_DATA_DIR("/utils/extern-local-engine/tests/data/68135.snappy.parquet")}; + const std::string file{test::gtest_uri("68135.snappy.parquet")}; auto [_, local_executor] = test::create_plan_and_executor(EMBEDDED_PLAN(_pr_68135_), split_template, file); EXPECT_TRUE(local_executor->hasNext()); @@ -113,9 +115,8 @@ TEST(Clickhouse, PR68131) { constexpr std::string_view split_template = R"({"items":[{"uriFile":"{replace_local_files}","partitionIndex":"0","length":"289","parquet":{},"schema":{},"metadataColumns":[{}]}]})"; - auto [_, local_executor] = test::create_plan_and_executor( - EMBEDDED_PLAN(_pr_68131_), split_template, GLUTEN_DATA_DIR("/utils/extern-local-engine/tests/data/68131.parquet")); + auto [_, local_executor] = test::create_plan_and_executor(EMBEDDED_PLAN(_pr_68131_), split_template, test::gtest_uri("68131.parquet")); EXPECT_TRUE(local_executor->hasNext()); const Block & x = *local_executor->nextColumnar(); debug::headBlock(x); -} +} \ No newline at end of file diff --git a/cpp-ch/local-engine/tests/gtest_parquet_columnindex.cpp b/cpp-ch/local-engine/tests/gtest_parquet_columnindex.cpp index 5a39580c19fd..4d9bad05665c 100644 --- a/cpp-ch/local-engine/tests/gtest_parquet_columnindex.cpp +++ b/cpp-ch/local-engine/tests/gtest_parquet_columnindex.cpp @@ -20,7 +20,6 @@ #include #include #include -#include #include #include #include @@ -31,6 +30,7 @@ #include #include #include +#include #include #include #include @@ -199,14 +199,14 @@ class CIBuilder class OIBuilder { - size_t previouse_count_ = 0; + size_t previous_count_ = 0; std::vector row_index_; public: OIBuilder & addPage(size_t row_count) { - row_index_.push_back(previouse_count_); - previouse_count_ += row_count; + row_index_.push_back(previous_count_); + previous_count_ += row_count; return *this; } @@ -220,7 +220,7 @@ class OIBuilder parquet::OffsetIndexPtr build() const { parquet::OffsetIndexBuilderPtr builder = parquet::OffsetIndexBuilder::Make(); - // we don't care about the offset and comprewssed_page_size. + // we don't care about the offset and compressed_page_size. std::ranges::for_each(row_index_, [&](const auto & row_index) { builder->AddPage(1, 1, row_index); }); constexpr int64_t final_position = 4096; builder->Finish(final_position); @@ -367,21 +367,21 @@ local_engine::RowRanges buildTestRowRanges(const std::vector & rowIndexes local_engine::RowRanges result; const parquet::OffsetIndexBuilderPtr builder = parquet::OffsetIndexBuilder::Make(); - local_engine::PageIndexs pageIndexs; + local_engine::PageIndexs pageIndexes; for (Int32 i = 0, n = rowIndexes.size(); i < n; i += 2) { const int64_t from = rowIndexes[i]; const int64_t to = rowIndexes[i + 1]; builder->AddPage(0, 0, from); builder->AddPage(0, 0, to + 1); - pageIndexs.push_back(i); + pageIndexes.push_back(i); } constexpr int64_t final_position = 4096; builder->Finish(final_position); const auto offset_index = builder->Build(); const Int32 rgCount = rowIndexes.back() - 1; - return local_engine::RowRangesBuilder(rgCount, offset_index->page_locations()).toRowRanges(pageIndexs); + return local_engine::RowRangesBuilder(rgCount, offset_index->page_locations()).toRowRanges(pageIndexes); } void assertRows(const local_engine::RowRanges & ranges, const std::vector & expectedRows) @@ -393,13 +393,18 @@ void assertRows(const local_engine::RowRanges & ranges, const std::vector & expectedRows) +local_engine::RowRanges calculateRowRangesForTest(const std::string & exp) { static const AnotherRowType name_and_types = buildTestRowType(); static const local_engine::ColumnIndexStore column_index_store = buildTestColumnIndexStore(); const local_engine::ColumnIndexFilter filter( local_engine::test::parseFilter(exp, name_and_types).value(), local_engine::QueryContext::globalContext()); - assertRows(filter.calculateRowRanges(column_index_store, TOTALSIZE), expectedRows); + return filter.calculateRowRanges(column_index_store, TOTALSIZE); +} + +void testCondition(const std::string & exp, const std::vector & expectedRows) +{ + assertRows(calculateRowRangesForTest(exp), expectedRows); } void testCondition(const std::string & exp, size_t rowCount) @@ -433,6 +438,7 @@ TEST(RowRanges, Create) ASSERT_EQ(0, empty.rowCount()); ASSERT_FALSE(empty.isOverlapping(0, std::numeric_limits::max())); } + TEST(RowRanges, Union) { using local_engine::RowRanges; @@ -499,6 +505,28 @@ TEST(ColumnIndex, Filtering) testCondition("column1 >= 7 and column1 < 11 and column2 > 'Romeo' and column2 <= 'Tango'", {7, 11, 12, 13}); } + +TEST(RowIndex, VirtualColumnRowIndexReader) +{ + local_engine::RowGroupInformation rg_info{ + .index = 0, + .num_rows = 30, + .rowStartIndexOffset = 0, + .columnIndexStore = nullptr, + .rowRanges{calculateRowRangesForTest("column1 in (7)")}}; + std::vector rowGroups; + rowGroups.push_back(std::move(rg_info)); + + local_engine::ColumnIndexRowRangesProvider provider({std::move(rowGroups)}); + local_engine::VirtualColumnRowIndexReader reader(provider, local_engine::BIGINT()); + + DB::ColumnPtr col = reader.readBatch(TOTALSIZE); + const auto & col_str = typeid_cast(*col); + std::vector result; + std::ranges::for_each(col_str.getData(), [&](const auto & val) { result.push_back(val); }); + ASSERT_EQ(result, std::vector({7, 8, 9, 10, 11, 12, 13})); +} + TEST(ColumnIndex, FilteringWithAllNullPages) { using namespace test_utils; @@ -549,7 +577,7 @@ using ParquetValue = std::variant< parquet::ByteArrayType::c_type>; template -void doComapre( +void doCompare( const parquet::ColumnDescriptor & descriptor, const DB::Field & value, const std::function & compare) { local_engine::ToParquet to_parquet; @@ -562,7 +590,7 @@ void with_actual(const DB::Field & value, const parquet::ColumnDescriptor & desc switch (desc.physical_type()) { case parquet::Type::BOOLEAN: - doComapre(desc, value, compare); + doCompare(desc, value, compare); return; case parquet::Type::INT32: { switch (desc.converted_type()) @@ -574,7 +602,7 @@ void with_actual(const DB::Field & value, const parquet::ColumnDescriptor & desc case parquet::ConvertedType::INT_16: case parquet::ConvertedType::INT_32: case parquet::ConvertedType::NONE: - doComapre(desc, value, compare); + doCompare(desc, value, compare); return; default: break; @@ -587,33 +615,33 @@ void with_actual(const DB::Field & value, const parquet::ColumnDescriptor & desc case parquet::ConvertedType::INT_64: case parquet::ConvertedType::UINT_64: case parquet::ConvertedType::NONE: - doComapre(desc, value, compare); + doCompare(desc, value, compare); return; default: break; } break; case parquet::Type::INT96: - // doComapre(desc, value, compare); + // doCompare(desc, value, compare); break; case parquet::Type::FLOAT: - doComapre(desc, value, compare); + doCompare(desc, value, compare); return; case parquet::Type::DOUBLE: - doComapre(desc, value, compare); + doCompare(desc, value, compare); return; case parquet::Type::BYTE_ARRAY: switch (desc.converted_type()) { case parquet::ConvertedType::UTF8: - doComapre(desc, value, compare); + doCompare(desc, value, compare); return; default: break; } break; case parquet::Type::FIXED_LEN_BYTE_ARRAY: - // doComapre(desc, value, compare); + // doCompare(desc, value, compare); break; case parquet::Type::UNDEFINED: break; @@ -705,17 +733,17 @@ TEST(ColumnIndex, Field) }); const std::vector> primitive_fields{ - {"f_bool", UInt8(1)}, - {"f_byte", Int8(1)}, - {"f_short", Int16(2)}, - {"f_int", Int32(3)}, - {"f_long", Int64(4)}, - {"f_float", Float32(5.5)}, + {"f_bool", static_cast(1)}, + {"f_byte", static_cast(1)}, + {"f_short", static_cast(2)}, + {"f_int", static_cast(3)}, + {"f_long", static_cast(4)}, + {"f_float", static_cast(5.5)}, {"f_double", Float64{6.6}}, {"f_string", "hello world"}, {"f_binary", "hello world"}, {"f_decimal", DecimalField(777, 2)}, - {"f_date", Int32(18262)}, + {"f_date", static_cast(18262)}, {"f_timestamp", DecimalField(1666162060000000L, 6)}}; // 2022-09-01 12:34:20.000000 std::ranges::for_each( @@ -733,7 +761,7 @@ struct ReadStatesParam ReadStatesParam() = default; ReadStatesParam(local_engine::RowRanges ranges, std::shared_ptr states) - : row_ranges(std::move(ranges)), read_states(std::move(states)) {}; + : row_ranges(std::move(ranges)), read_states(std::move(states)) { }; local_engine::RowRanges row_ranges; std::shared_ptr read_states; @@ -1088,22 +1116,30 @@ TEST_P(TestBuildPageReadStates, BuildPageReadStates) TEST(ColumnIndex, VectorizedParquetRecordReader) { using namespace local_engine; + //TODO: move test parquet to s3 and download to CI machine. const std::string filename = "/home/chang/test/tpch/parquet/Index/60001/part-00000-76ef9b89-f292-495f-9d0d-98325f3d8956-c000.snappy.parquet"; - ReadBufferFromFilePRead in(filename); + const FormatSettings format_settings{}; - auto arrow_file = local_engine::test::asArrowFileForParquet(in, format_settings); + static const AnotherRowType name_and_types{{"11", BIGINT()}}; - const auto filterAction = local_engine::test::parseFilter("`11` = 10 or `11` = 50", name_and_types); - auto column_index_filter - = std::make_shared(filterAction.value(), local_engine::QueryContext::globalContext()); + const auto filterAction = test::parseFilter("`11` = 10 or `11` = 50", name_and_types); + auto column_index_filter = std::make_shared(filterAction.value(), local_engine::QueryContext::globalContext()); Block blockHeader({{BIGINT(), "11"}, {STRING(), "18"}}); - local_engine::VectorizedParquetRecordReader recordReader(blockHeader, format_settings); - recordReader.initialize(blockHeader, arrow_file, column_index_filter); + ReadBufferFromFilePRead in(filename); + + ParquetMetaBuilder metaBuilder{.collectPageIndex = true}; + metaBuilder.build(&in, &blockHeader, column_index_filter.get(), [](UInt64 /*midpoint_offset*/) -> bool { return true; }); + ColumnIndexRowRangesProvider provider{metaBuilder}; + + VectorizedParquetRecordReader recordReader(blockHeader, format_settings); + auto arrow_file = test::asArrowFileForParquet(in, format_settings); + recordReader.initialize(arrow_file, provider); + auto chunk{recordReader.nextBatch()}; ASSERT_EQ(chunk.getNumColumns(), 2); ASSERT_EQ(chunk.getNumRows(), format_settings.parquet.max_block_size); diff --git a/cpp-ch/local-engine/tests/gtest_parquet_columnindex_bug.cpp b/cpp-ch/local-engine/tests/gtest_parquet_columnindex_bug.cpp index 51a5a0556ea8..40e397be401b 100644 --- a/cpp-ch/local-engine/tests/gtest_parquet_columnindex_bug.cpp +++ b/cpp-ch/local-engine/tests/gtest_parquet_columnindex_bug.cpp @@ -20,6 +20,7 @@ #include #include #include +#include #include #include #include @@ -27,23 +28,133 @@ #include using namespace local_engine; - using namespace DB; -INCBIN(_pr_18_2, SOURCE_DIR "/utils/extern-local-engine/tests/decimal_filter_push_down/18_2.json"); +INCBIN(_pr_18_2, SOURCE_DIR "/utils/extern-local-engine/tests/data/decimal_filter_push_down/18_2.json"); TEST(ColumnIndex, Decimal182) { // [precision,scale] = [18,2] - const auto context1 = DB::Context::createCopy(QueryContext::globalMutableContext()); - const auto config = ExecutorConfig::loadFromContext(context1); + auto query_id = QueryContext::instance().initializeQuery("RowIndex"); + SCOPE_EXIT({ QueryContext::instance().finalizeQuery(query_id); }); + const auto context = QueryContext::instance().currentQueryContext(); + const auto config = ExecutorConfig::loadFromContext(context); EXPECT_TRUE(config.use_local_format) << "gtest need set use_local_format to true"; constexpr std::string_view split_template = R"({"items":[{"uriFile":"{replace_local_files}","partitionIndex":"0","length":"488","parquet":{},"schema":{},"metadataColumns":[{}]}]})"; - constexpr std::string_view file{GLUTEN_DATA_DIR("/utils/extern-local-engine/tests/decimal_filter_push_down/18_2_flba.snappy.parquet")}; - auto [_, local_executor] = test::create_plan_and_executor(EMBEDDED_PLAN(_pr_18_2), split_template, file, context1); + const std::string file{test::gtest_uri("decimal_filter_push_down/18_2_flba.snappy.parquet")}; + auto [_, local_executor] = test::create_plan_and_executor(EMBEDDED_PLAN(_pr_18_2), split_template, file, context); EXPECT_TRUE(local_executor->hasNext()); const Block & x = *local_executor->nextColumnar(); debug::headBlock(x); +} + +void readFile( + std::string_view json_plan, + std::string_view split_template, + std::string_view file, + const std::function & callback, + const TestSettings & test_settings = {{"input_format_parquet_allow_missing_columns", false}}) +{ + auto query_id = QueryContext::instance().initializeQuery("RowIndex"); + SCOPE_EXIT({ QueryContext::instance().finalizeQuery(query_id); }); + const auto context = QueryContext::instance().currentQueryContext(); + for (const auto & x : test_settings) + context->setSetting(x.first, x.second); + auto [_, local_executor] = test::create_plan_and_executor(json_plan, split_template, file, context); + callback(*local_executor); +} + +INCBIN(_read_metadata, SOURCE_DIR "/utils/extern-local-engine/tests/json/parquet_metadata/read_metadata.row_index.json"); +TEST(RowIndex, Basic) +{ + const std::string file{test::gtest_uri("metadata.rowindex.snappy.parquet")}; + constexpr std::string_view split_template + = R"({"items":[{"uriFile":"{replace_local_files}","length":"1767","parquet":{},"partitionColumns":[{"key":"pb","value":"1003"}],"schema":{},"metadataColumns":[{"key":"input_file_name","value":"{replace_local_files}"},{"key":"input_file_block_length","value":"1767"},{"key":"input_file_block_start","value":"0"}],"properties":{"fileSize":"1767","modificationTime":"1736847651881"}}]})"; + readFile( + EMBEDDED_PLAN(_read_metadata), + split_template, + file, + [&](LocalExecutor & local_executor) + { + EXPECT_TRUE(local_executor.hasNext()); + debug::headBlock(*local_executor.nextColumnar()); + }); +} + +INCBIN(_rowindex_in, SOURCE_DIR "/utils/extern-local-engine/tests/json/parquet_metadata/rowindex_in.json"); +TEST(RowIndex, In) +{ + const std::string file{test::gtest_uri("rowindex_in.snappy.parquet")}; + /// all row gorups are ignored + constexpr std::string_view split_template_ignore_all_rg + = R"({"items":[{"uriFile":"{replace_local_files}","length":"256","parquet":{},"schema":{},"metadataColumns":[{"key":"input_file_name","value":"{replace_local_files}"},{"key":"input_file_block_length","value":"256"},{"key":"input_file_block_start","value":"0"}],"properties":{"fileSize":"125451","modificationTime":"1737104830724"}}]})"; + + readFile( + EMBEDDED_PLAN(_rowindex_in), + split_template_ignore_all_rg, + file, + [&](LocalExecutor & local_executor) { EXPECT_FALSE(local_executor.hasNext()); }); + + constexpr std::string_view split_template + = R"({"items":[{"uriFile":"{replace_local_files}","length":"125451","parquet":{},"schema":{},"metadataColumns":[{"key":"input_file_name","value":"{replace_local_files}"},{"key":"input_file_block_length","value":"256"},{"key":"input_file_block_start","value":"0"}],"properties":{"fileSize":"125451","modificationTime":"1737104830724"}}]})"; + + readFile( + EMBEDDED_PLAN(_rowindex_in), + split_template, + file, + [&](LocalExecutor & local_executor) + { + EXPECT_TRUE(local_executor.hasNext()); + debug::headBlock(*local_executor.nextColumnar()); + }); +} + +INCBIN(_all_meta, SOURCE_DIR "/utils/extern-local-engine/tests/json/parquet_metadata/read_metadata.all.json"); +TEST(RowIndex, AllMeta) +{ + const std::string file{test::gtest_uri("all_meta/part-00000-92bb25d0-7446-4f9b-8bdd-a6911d0d465a-c000.snappy.parquet")}; + constexpr std::string_view split_template + = R"({"items":[{"uriFile":"{replace_local_files}","length":"1282","parquet":{},"schema":{},"metadataColumns":[{"key":"file_path","value":"{replace_local_files}"},{"key":"file_block_length","value":"1282"},{"key":"input_file_name","value":"{replace_local_files}"},{"key":"input_file_block_length","value":"1282"},{"key":"file_name","value":"part-00000-484a7344-cf25-4367-bf46-8123a6a7b71e-c000.snappy.parquet"},{"key":"file_modification_time","value":"2025-01-19 05:09:48.664"},{"key":"file_block_start","value":"0"},{"key":"input_file_block_start","value":"0"},{"key":"file_size","value":"1282"}],"properties":{"fileSize":"1282","modificationTime":"1737263388664"}}]})"; + + readFile( + EMBEDDED_PLAN(_all_meta), + split_template, + file, + [&](LocalExecutor & local_executor) + { + EXPECT_TRUE(local_executor.hasNext()); + debug::headBlock(*local_executor.nextColumnar()); + }); +} + +INCBIN(_input_filename, SOURCE_DIR "/utils/extern-local-engine/tests/json/parquet_metadata/input_filename.json"); +INCBIN( + _input_filename_no_real_column, SOURCE_DIR "/utils/extern-local-engine/tests/json/parquet_metadata/input_filename_no_real_column.json"); +TEST(RowIndex, InputFileName) +{ + const std::string file{test::gtest_uri("input_filename.snappy.parquet")}; + constexpr std::string_view split_template + = R"({"items":[{"uriFile":"{replace_local_files}","length":"443","parquet":{},"schema":{},"metadataColumns":[{"key":"input_file_name","value":"{replace_local_files}"},{"key":"input_file_block_length","value":"443"},{"key":"input_file_block_start","value":"0"}],"properties":{"fileSize":"443","modificationTime":"1737445386987"}}]})"; + + readFile( + EMBEDDED_PLAN(_input_filename), + split_template, + file, + [&](LocalExecutor & local_executor) + { + EXPECT_TRUE(local_executor.hasNext()); + debug::headBlock(*local_executor.nextColumnar()); + }); + + readFile( + EMBEDDED_PLAN(_input_filename_no_real_column), + split_template, + file, + [&](LocalExecutor & local_executor) + { + EXPECT_TRUE(local_executor.hasNext()); + debug::headBlock(*local_executor.nextColumnar()); + }); } \ No newline at end of file diff --git a/cpp-ch/local-engine/tests/gtest_parquet_read.cpp b/cpp-ch/local-engine/tests/gtest_parquet_read.cpp index b75cee57488c..c47ec6d05de1 100644 --- a/cpp-ch/local-engine/tests/gtest_parquet_read.cpp +++ b/cpp-ch/local-engine/tests/gtest_parquet_read.cpp @@ -47,10 +47,11 @@ #include using namespace DB; +using namespace local_engine; void readSchema(const String & path) { - auto name_and_types = local_engine::test::readParquetSchema(local_engine::test::data_file(path.c_str())); + auto name_and_types = test::readParquetSchema(test::gtest_data(path.c_str())); auto & factory = DataTypeFactory::instance(); auto check_type = [&name_and_types, &factory](const String & column, const String & expect_str_type) @@ -95,7 +96,7 @@ void readSchema(const String & path) BlockRowType createColumn(const String & full_path, const std::map & fields) { - const auto name_and_types = local_engine::test::readParquetSchema(full_path); + const auto name_and_types = test::readParquetSchema(full_path); auto is_selected = [&fields](const auto & name_and_type) { return fields.contains(name_and_type.name); }; return toBlockRowType(name_and_types, is_selected); } @@ -103,7 +104,7 @@ BlockRowType createColumn(const String & full_path, const std::map void readData(const String & path, const std::map & fields) { - String full_path = local_engine::test::data_file(path.c_str()); + String full_path = test::gtest_data(path.c_str()); FormatSettings settings; ColumnsWithTypeAndName columns = createColumn(full_path, fields); Block header(columns); @@ -144,28 +145,20 @@ TEST(ParquetRead, ReadSchema) TEST(ParquetRead, VerifyPageindexReaderSupport) { - EXPECT_FALSE(local_engine::ParquetFormatFile::supportPageindexReader( - toBlockRowType(local_engine::test::readParquetSchema(local_engine::test::data_file("alltypes/alltypes_notnull.parquet"))))); - EXPECT_FALSE(local_engine::ParquetFormatFile::supportPageindexReader( - toBlockRowType(local_engine::test::readParquetSchema(local_engine::test::data_file("alltypes/alltypes_null.parquet"))))); - - - EXPECT_FALSE(local_engine::ParquetFormatFile::supportPageindexReader( - toBlockRowType(local_engine::test::readParquetSchema(local_engine::test::data_file("array.parquet"))))); - EXPECT_TRUE(local_engine::ParquetFormatFile::supportPageindexReader( - toBlockRowType(local_engine::test::readParquetSchema(local_engine::test::data_file("date.parquet"))))); - EXPECT_TRUE(local_engine::ParquetFormatFile::supportPageindexReader( - toBlockRowType(local_engine::test::readParquetSchema(local_engine::test::data_file("datetime64.parquet"))))); - EXPECT_TRUE(local_engine::ParquetFormatFile::supportPageindexReader( - toBlockRowType(local_engine::test::readParquetSchema(local_engine::test::data_file("decimal.parquet"))))); - EXPECT_TRUE(local_engine::ParquetFormatFile::supportPageindexReader( - toBlockRowType(local_engine::test::readParquetSchema(local_engine::test::data_file("iris.parquet"))))); - EXPECT_FALSE(local_engine::ParquetFormatFile::supportPageindexReader( - toBlockRowType(local_engine::test::readParquetSchema(local_engine::test::data_file("map.parquet"))))); - EXPECT_TRUE(local_engine::ParquetFormatFile::supportPageindexReader( - toBlockRowType(local_engine::test::readParquetSchema(local_engine::test::data_file("sample.parquet"))))); - EXPECT_FALSE(local_engine::ParquetFormatFile::supportPageindexReader( - toBlockRowType(local_engine::test::readParquetSchema(local_engine::test::data_file("struct.parquet"))))); + EXPECT_FALSE( + ParquetFormatFile::onlyHasFlatType(toBlockRowType(test::readParquetSchema(test::gtest_data("alltypes/alltypes_notnull.parquet"))))); + EXPECT_FALSE( + ParquetFormatFile::onlyHasFlatType(toBlockRowType(test::readParquetSchema(test::gtest_data("alltypes/alltypes_null.parquet"))))); + + + EXPECT_FALSE(ParquetFormatFile::onlyHasFlatType(toBlockRowType(test::readParquetSchema(test::gtest_data("array.parquet"))))); + EXPECT_TRUE(ParquetFormatFile::onlyHasFlatType(toBlockRowType(test::readParquetSchema(test::gtest_data("date.parquet"))))); + EXPECT_TRUE(ParquetFormatFile::onlyHasFlatType(toBlockRowType(test::readParquetSchema(test::gtest_data("datetime64.parquet"))))); + EXPECT_TRUE(ParquetFormatFile::onlyHasFlatType(toBlockRowType(test::readParquetSchema(test::gtest_data("decimal.parquet"))))); + EXPECT_TRUE(ParquetFormatFile::onlyHasFlatType(toBlockRowType(test::readParquetSchema(test::gtest_data("iris.parquet"))))); + EXPECT_FALSE(ParquetFormatFile::onlyHasFlatType(toBlockRowType(test::readParquetSchema(test::gtest_data("map.parquet"))))); + EXPECT_TRUE(ParquetFormatFile::onlyHasFlatType(toBlockRowType(test::readParquetSchema(test::gtest_data("sample.parquet"))))); + EXPECT_FALSE(ParquetFormatFile::onlyHasFlatType(toBlockRowType(test::readParquetSchema(test::gtest_data("struct.parquet"))))); } TEST(ParquetRead, ReadDataNotNull) @@ -348,10 +341,10 @@ TEST(ParquetRead, ArrowRead) // a: [1..20] // b: [1.0..20.0] - const std::string sample(local_engine::test::data_file("sample.parquet")); + const std::string sample(test::gtest_data("sample.parquet")); ReadBufferFromFile in(sample); const FormatSettings format_settings{}; - auto arrow_file = local_engine::test::asArrowFileForParquet(in, format_settings); + auto arrow_file = test::asArrowFileForParquet(in, format_settings); // std::shared_ptr metadata = parquet::ReadMetaData(arrow_file); std::unique_ptr reader; @@ -362,7 +355,7 @@ TEST(ParquetRead, ArrowRead) EXPECT_EQ(table->num_rows(), 20); EXPECT_EQ(table->num_columns(), 2); - auto columns = toBlockRowType(local_engine::test::readParquetSchema(sample)); + auto columns = toBlockRowType(test::readParquetSchema(sample)); Block header(columns); ArrowColumnToCHColumn converter( @@ -390,7 +383,7 @@ TEST(ParquetRead, ArrowRead) TEST(ParquetRead, LowLevelRead) { - const std::string sample(local_engine::test::data_file("sample.parquet")); + const std::string sample(test::gtest_data("sample.parquet")); // Create a ParquetReader instance const std::unique_ptr parquet_reader = parquet::ParquetFileReader::OpenFile(sample, false); @@ -407,7 +400,7 @@ TEST(ParquetRead, LowLevelRead) const parquet::SchemaDescriptor & schema = *(file_metadata->schema()); const parquet::ColumnDescriptor & column_a_descr = *(schema.Column(col_a)); EXPECT_EQ(column_a_descr.name(), "a"); - const parquet::internal::LevelInfo level_info = local_engine::computeLevelInfo(&column_a_descr); + const parquet::internal::LevelInfo level_info = computeLevelInfo(&column_a_descr); const auto reader = parquet::internal::RecordReader::Make(&column_a_descr, level_info); // Iterate over all the RowGroups in the file @@ -425,13 +418,20 @@ TEST(ParquetRead, LowLevelRead) TEST(ParquetRead, VectorizedColumnReader) { - const std::string sample(local_engine::test::data_file("sample.parquet")); - Block blockHeader({{local_engine::DOUBLE(), "b"}, {local_engine::BIGINT(), "a"}}); - ReadBufferFromFile in(sample); + const std::string sample(test::gtest_data("sample.parquet")); const FormatSettings format_settings{}; - auto arrow_file = local_engine::test::asArrowFileForParquet(in, format_settings); - local_engine::VectorizedParquetRecordReader recordReader(blockHeader, format_settings); - recordReader.initialize(blockHeader, arrow_file, nullptr); + Block blockHeader({{DOUBLE(), "b"}, {BIGINT(), "a"}}); + + ReadBufferFromFile in(sample); + + ParquetMetaBuilder metaBuilder{.collectPageIndex = true}; + metaBuilder.build(&in, &blockHeader, nullptr, [](UInt64 /*midpoint_offset*/) -> bool { return true; }); + ColumnIndexRowRangesProvider provider{metaBuilder}; + VectorizedParquetRecordReader recordReader(blockHeader, format_settings); + + auto arrow_file = test::asArrowFileForParquet(in, format_settings); + recordReader.initialize(arrow_file, provider); + auto chunk{recordReader.nextBatch()}; ASSERT_EQ(chunk.getNumColumns(), 2); ASSERT_EQ(chunk.getNumRows(), 20); @@ -453,8 +453,8 @@ TEST(ParquetRead, UpperColRead) { constexpr std::string_view split_template = R"({"items":[{"uriFile":"{replace_local_files}","length":"459","parquet":{},"schema":{},"metadataColumns":[{"key":"input_file_name","value":"{replace_local_files}"},{"key":"input_file_block_length","value":"459"},{"key":"input_file_block_start","value":"0"}],"properties":{"fileSize":"459","modificationTime":"1735012863732"}}]})"; - constexpr std::string_view file{GLUTEN_DATA_DIR("/utils/extern-local-engine/tests/data/upper_case_col.parquet")}; - auto [_, local_executor] = local_engine::test::create_plan_and_executor(EMBEDDED_PLAN(_upper_col_parquet_), split_template, file, {}); + const std::string file{test::gtest_uri("upper_case_col.parquet")}; + auto [_, local_executor] = test::create_plan_and_executor(EMBEDDED_PLAN(_upper_col_parquet_), split_template, file, {}); EXPECT_TRUE(local_executor->hasNext()); const Block & block = *local_executor->nextColumnar(); diff --git a/cpp-ch/local-engine/tests/gtest_parser.cpp b/cpp-ch/local-engine/tests/gtest_parser.cpp index c85fef6525ae..25e216672918 100644 --- a/cpp-ch/local-engine/tests/gtest_parser.cpp +++ b/cpp-ch/local-engine/tests/gtest_parser.cpp @@ -40,7 +40,7 @@ TEST(LocalExecutor, ReadCSV) constexpr std::string_view split_template = R"({"items":[{"uriFile":"{replace_local_files}","length":"56","text":{"fieldDelimiter":",","maxBlockSize":"8192","header":"1"},"schema":{"names":["id","name","language"],"struct":{"types":[{"string":{"nullability":"NULLABILITY_NULLABLE"}},{"string":{"nullability":"NULLABILITY_NULLABLE"}},{"string":{"nullability":"NULLABILITY_NULLABLE"}}]}},"metadataColumns":[{}]}]})"; const std::string split = replaceLocalFilesWildcards( - split_template, GLUTEN_SOURCE_DIR("/backends-velox/src/test/resources/datasource/csv/student_option_schema.csv")); + split_template, GLUTEN_SOURCE_URI("/backends-velox/src/test/resources/datasource/csv/student_option_schema.csv")); auto plan = local_engine::JsonStringToMessage(EMBEDDED_PLAN(_readcsv_plan)); auto parser_context = ParserContext::build(QueryContext::globalContext(), plan); SerializedPlanParser parser(parser_context); diff --git a/cpp-ch/local-engine/tests/gtest_write_pipeline.cpp b/cpp-ch/local-engine/tests/gtest_write_pipeline.cpp index 9761d837d0e9..d48db92b14fa 100644 --- a/cpp-ch/local-engine/tests/gtest_write_pipeline.cpp +++ b/cpp-ch/local-engine/tests/gtest_write_pipeline.cpp @@ -127,7 +127,7 @@ TEST(WritePipeline, SubstraitFileSink) constexpr std::string_view split_template = R"({"items":[{"uriFile":"{replace_local_files}","length":"1399183","text":{"fieldDelimiter":"|","maxBlockSize":"8192"},"schema":{"names":["s_suppkey","s_name","s_address","s_nationkey","s_phone","s_acctbal","s_comment"],"struct":{"types":[{"i64":{"nullability":"NULLABILITY_NULLABLE"}},{"string":{"nullability":"NULLABILITY_NULLABLE"}},{"string":{"nullability":"NULLABILITY_NULLABLE"}},{"i64":{"nullability":"NULLABILITY_NULLABLE"}},{"string":{"nullability":"NULLABILITY_NULLABLE"}},{"decimal":{"scale":2,"precision":15,"nullability":"NULLABILITY_NULLABLE"}},{"string":{"nullability":"NULLABILITY_NULLABLE"}}]}},"metadataColumns":[{}]}]})"; - constexpr std::string_view file{GLUTEN_SOURCE_DIR("/backends-clickhouse/src/test/resources/csv-data/supplier.csv")}; + constexpr std::string_view file{GLUTEN_SOURCE_URI("/backends-clickhouse/src/test/resources/csv-data/supplier.csv")}; auto [plan, local_executor] = test::create_plan_and_executor(EMBEDDED_PLAN(native_write), split_template, file, context); EXPECT_EQ(1, plan.relations_size()); diff --git a/cpp-ch/local-engine/tests/gtest_write_pipeline_mergetree.cpp b/cpp-ch/local-engine/tests/gtest_write_pipeline_mergetree.cpp index 7bc39ccc8a2a..85825cdca7dc 100644 --- a/cpp-ch/local-engine/tests/gtest_write_pipeline_mergetree.cpp +++ b/cpp-ch/local-engine/tests/gtest_write_pipeline_mergetree.cpp @@ -219,7 +219,7 @@ TEST(MergeTree, SparkMergeTree) constexpr std::string_view split_template = R"({"items":[{"uriFile":"{replace_local_files}","length":"19230111","parquet":{},"schema":{},"metadataColumns":[{}],"properties":{"fileSize":"19230111","modificationTime":"1722330598029"}}]})"; - constexpr std::string_view file{GLUTEN_SOURCE_TPCH_DIR("lineitem/part-00000-d08071cb-0dfa-42dc-9198-83cb334ccda3-c000.snappy.parquet")}; + constexpr std::string_view file{GLUTEN_SOURCE_TPCH_URI("lineitem/part-00000-d08071cb-0dfa-42dc-9198-83cb334ccda3-c000.snappy.parquet")}; SparkMergeTreeWritePartitionSettings gm_write_settings{ .part_name_prefix{"this_is_prefix"}, @@ -293,8 +293,7 @@ TEST(MergeTree, Pipeline) writeMerge( EMBEDDED_PLAN(_3_mergetree_plan_), "tmp/lineitem_mergetree", - {{"min_insert_block_size_rows", 100000} - /*, {"optimize.minFileSize", 1024 * 1024 * 10}*/}, + {{"min_insert_block_size_rows", 100000} /*, {"optimize.minFileSize", 1024 * 1024 * 10}*/}, [&](const DB::Block & block) { EXPECT_EQ(1, block.rows()); diff --git a/cpp-ch/local-engine/tests/json/gtest_local_engine_config.json b/cpp-ch/local-engine/tests/json/gtest_local_engine_config.json index 5eaa47397265..42e57aacdff0 100644 --- a/cpp-ch/local-engine/tests/json/gtest_local_engine_config.json +++ b/cpp-ch/local-engine/tests/json/gtest_local_engine_config.json @@ -4,6 +4,7 @@ "spark.hadoop.fs.s3a.connection.ssl.enabled": "false", "spark.hadoop.dfs.client.log.severity": "INFO", "spark.gluten.sql.columnar.backend.ch.runtime_config.use_local_format": "true", + "spark.gluten.sql.columnar.backend.ch.runtime_config.dump_pipeline": "true", "spark.hadoop.fs.s3a.access.key": "", "spark.gluten.sql.columnar.backend.ch.runtime_config.hdfs.input_read_timeout": "180000", "spark.gluten.sql.columnar.backend.velox.IOThreads": "0", diff --git a/cpp-ch/local-engine/tests/json/parquet_metadata/input_filename.json b/cpp-ch/local-engine/tests/json/parquet_metadata/input_filename.json new file mode 100644 index 000000000000..3fd278a5eb03 --- /dev/null +++ b/cpp-ch/local-engine/tests/json/parquet_metadata/input_filename.json @@ -0,0 +1,194 @@ +{ + "extensions": [ + { + "extensionFunction": { + "name": "alias:str" + } + }, + { + "extensionFunction": { + "functionAnchor": 1, + "name": "alias:i64" + } + } + ], + "relations": [ + { + "root": { + "input": { + "project": { + "common": { + "emit": { + "outputMapping": [ + 4, + 5, + 6, + 7 + ] + } + }, + "input": { + "read": { + "common": { + "direct": {} + }, + "baseSchema": { + "names": [ + "a", + "input_file_name", + "input_file_block_length", + "input_file_block_start" + ], + "struct": { + "types": [ + { + "i32": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_REQUIRED" + } + }, + { + "i64": { + "nullability": "NULLABILITY_REQUIRED" + } + }, + { + "i64": { + "nullability": "NULLABILITY_REQUIRED" + } + } + ] + }, + "columnTypes": [ + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL" + ] + }, + "advancedExtension": { + "optimization": { + "@type": "type.googleapis.com/google.protobuf.StringValue", + "value": "isMergeTree=0\n" + } + } + } + }, + "expressions": [ + { + "selection": { + "directReference": { + "structField": {} + } + } + }, + { + "scalarFunction": { + "outputType": { + "string": { + "nullability": "NULLABILITY_REQUIRED" + } + }, + "arguments": [ + { + "value": { + "selection": { + "directReference": { + "structField": { + "field": 1 + } + } + } + } + } + ] + } + }, + { + "scalarFunction": { + "functionReference": 1, + "outputType": { + "i64": { + "nullability": "NULLABILITY_REQUIRED" + } + }, + "arguments": [ + { + "value": { + "selection": { + "directReference": { + "structField": { + "field": 3 + } + } + } + } + } + ] + } + }, + { + "scalarFunction": { + "functionReference": 1, + "outputType": { + "i64": { + "nullability": "NULLABILITY_REQUIRED" + } + }, + "arguments": [ + { + "value": { + "selection": { + "directReference": { + "structField": { + "field": 2 + } + } + } + } + } + ] + } + } + ] + } + }, + "names": [ + "a#23", + "input_file_name#37", + "input_file_block_start#38", + "input_file_block_length#39" + ], + "outputSchema": { + "types": [ + { + "i32": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_REQUIRED" + } + }, + { + "i64": { + "nullability": "NULLABILITY_REQUIRED" + } + }, + { + "i64": { + "nullability": "NULLABILITY_REQUIRED" + } + } + ], + "nullability": "NULLABILITY_REQUIRED" + } + } + } + ] +} \ No newline at end of file diff --git a/cpp-ch/local-engine/tests/json/parquet_metadata/input_filename_no_real_column.json b/cpp-ch/local-engine/tests/json/parquet_metadata/input_filename_no_real_column.json new file mode 100644 index 000000000000..264ab3aacca9 --- /dev/null +++ b/cpp-ch/local-engine/tests/json/parquet_metadata/input_filename_no_real_column.json @@ -0,0 +1,171 @@ +{ + "extensions": [ + { + "extensionFunction": { + "name": "alias:str" + } + }, + { + "extensionFunction": { + "functionAnchor": 1, + "name": "alias:i64" + } + } + ], + "relations": [ + { + "root": { + "input": { + "project": { + "common": { + "emit": { + "outputMapping": [ + 3, + 4, + 5 + ] + } + }, + "input": { + "read": { + "common": { + "direct": {} + }, + "baseSchema": { + "names": [ + "input_file_name", + "input_file_block_length", + "input_file_block_start" + ], + "struct": { + "types": [ + { + "string": { + "nullability": "NULLABILITY_REQUIRED" + } + }, + { + "i64": { + "nullability": "NULLABILITY_REQUIRED" + } + }, + { + "i64": { + "nullability": "NULLABILITY_REQUIRED" + } + } + ] + }, + "columnTypes": [ + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL" + ] + }, + "advancedExtension": { + "optimization": { + "@type": "type.googleapis.com/google.protobuf.StringValue", + "value": "isMergeTree=0\n" + } + } + } + }, + "expressions": [ + { + "scalarFunction": { + "outputType": { + "string": { + "nullability": "NULLABILITY_REQUIRED" + } + }, + "arguments": [ + { + "value": { + "selection": { + "directReference": { + "structField": {} + } + } + } + } + ] + } + }, + { + "scalarFunction": { + "functionReference": 1, + "outputType": { + "i64": { + "nullability": "NULLABILITY_REQUIRED" + } + }, + "arguments": [ + { + "value": { + "selection": { + "directReference": { + "structField": { + "field": 2 + } + } + } + } + } + ] + } + }, + { + "scalarFunction": { + "functionReference": 1, + "outputType": { + "i64": { + "nullability": "NULLABILITY_REQUIRED" + } + }, + "arguments": [ + { + "value": { + "selection": { + "directReference": { + "structField": { + "field": 1 + } + } + } + } + } + ] + } + } + ] + } + }, + "names": [ + "input_file_name#34", + "input_file_block_start#35", + "input_file_block_length#36" + ], + "outputSchema": { + "types": [ + { + "string": { + "nullability": "NULLABILITY_REQUIRED" + } + }, + { + "i64": { + "nullability": "NULLABILITY_REQUIRED" + } + }, + { + "i64": { + "nullability": "NULLABILITY_REQUIRED" + } + } + ], + "nullability": "NULLABILITY_REQUIRED" + } + } + } + ] +} \ No newline at end of file diff --git a/cpp-ch/local-engine/tests/json/parquet_metadata/read_metadata.all.json b/cpp-ch/local-engine/tests/json/parquet_metadata/read_metadata.all.json new file mode 100644 index 000000000000..4ad7fb019003 --- /dev/null +++ b/cpp-ch/local-engine/tests/json/parquet_metadata/read_metadata.all.json @@ -0,0 +1,790 @@ +{ + "extensions": [ + { + "extensionFunction": { + "functionAnchor": 5, + "name": "alias:ts" + } + }, + { + "extensionFunction": { + "name": "alias:struct" + } + }, + { + "extensionFunction": { + "functionAnchor": 2, + "name": "alias:str" + } + }, + { + "extensionFunction": { + "functionAnchor": 4, + "name": "alias:i64" + } + }, + { + "extensionFunction": { + "functionAnchor": 1, + "name": "named_struct:str_str_str_str_str_i64_str_i64_str_i64_str_ts" + } + }, + { + "extensionFunction": { + "functionAnchor": 3, + "name": "get_struct_field:struct" + } + } + ], + "relations": [ + { + "root": { + "input": { + "project": { + "common": { + "emit": { + "outputMapping": [ + 4, + 5, + 6, + 7, + 8, + 9, + 10, + 11, + 12 + ] + } + }, + "input": { + "project": { + "common": { + "emit": { + "outputMapping": [ + 9, + 10, + 11, + 12 + ] + } + }, + "input": { + "read": { + "common": { + "direct": {} + }, + "baseSchema": { + "names": [ + "name", + "age", + "info", + "id", + "university", + "file_path", + "file_name", + "file_size", + "file_block_start", + "file_block_length", + "file_modification_time" + ], + "struct": { + "types": [ + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "i32": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "struct": { + "types": [ + { + "i64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + } + ], + "nullability": "NULLABILITY_NULLABLE", + "names": [ + "id", + "university" + ] + } + }, + { + "string": { + "nullability": "NULLABILITY_REQUIRED" + } + }, + { + "string": { + "nullability": "NULLABILITY_REQUIRED" + } + }, + { + "i64": { + "nullability": "NULLABILITY_REQUIRED" + } + }, + { + "i64": { + "nullability": "NULLABILITY_REQUIRED" + } + }, + { + "i64": { + "nullability": "NULLABILITY_REQUIRED" + } + }, + { + "timestamp": { + "nullability": "NULLABILITY_REQUIRED" + } + } + ] + }, + "columnTypes": [ + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "METADATA_COL", + "METADATA_COL", + "METADATA_COL", + "METADATA_COL", + "METADATA_COL", + "METADATA_COL" + ] + }, + "advancedExtension": { + "optimization": { + "@type": "type.googleapis.com/google.protobuf.StringValue", + "value": "isMergeTree=0\n" + } + } + } + }, + "expressions": [ + { + "selection": { + "directReference": { + "structField": {} + } + } + }, + { + "selection": { + "directReference": { + "structField": { + "field": 1 + } + } + } + }, + { + "selection": { + "directReference": { + "structField": { + "field": 2 + } + } + } + }, + { + "scalarFunction": { + "outputType": { + "struct": { + "types": [ + { + "string": { + "nullability": "NULLABILITY_REQUIRED" + } + }, + { + "string": { + "nullability": "NULLABILITY_REQUIRED" + } + }, + { + "i64": { + "nullability": "NULLABILITY_REQUIRED" + } + }, + { + "i64": { + "nullability": "NULLABILITY_REQUIRED" + } + }, + { + "i64": { + "nullability": "NULLABILITY_REQUIRED" + } + }, + { + "timestamp": { + "nullability": "NULLABILITY_REQUIRED" + } + } + ], + "nullability": "NULLABILITY_REQUIRED", + "names": [ + "file_path", + "file_name", + "file_size", + "file_block_start", + "file_block_length", + "file_modification_time" + ] + } + }, + "arguments": [ + { + "value": { + "scalarFunction": { + "functionReference": 1, + "outputType": { + "struct": { + "types": [ + { + "string": { + "nullability": "NULLABILITY_REQUIRED" + } + }, + { + "string": { + "nullability": "NULLABILITY_REQUIRED" + } + }, + { + "i64": { + "nullability": "NULLABILITY_REQUIRED" + } + }, + { + "i64": { + "nullability": "NULLABILITY_REQUIRED" + } + }, + { + "i64": { + "nullability": "NULLABILITY_REQUIRED" + } + }, + { + "timestamp": { + "nullability": "NULLABILITY_REQUIRED" + } + } + ], + "nullability": "NULLABILITY_REQUIRED", + "names": [ + "file_path", + "file_name", + "file_size", + "file_block_start", + "file_block_length", + "file_modification_time" + ] + } + }, + "arguments": [ + { + "value": { + "literal": { + "string": "file_path" + } + } + }, + { + "value": { + "selection": { + "directReference": { + "structField": { + "field": 3 + } + } + } + } + }, + { + "value": { + "literal": { + "string": "file_name" + } + } + }, + { + "value": { + "selection": { + "directReference": { + "structField": { + "field": 4 + } + } + } + } + }, + { + "value": { + "literal": { + "string": "file_size" + } + } + }, + { + "value": { + "selection": { + "directReference": { + "structField": { + "field": 5 + } + } + } + } + }, + { + "value": { + "literal": { + "string": "file_block_start" + } + } + }, + { + "value": { + "selection": { + "directReference": { + "structField": { + "field": 6 + } + } + } + } + }, + { + "value": { + "literal": { + "string": "file_block_length" + } + } + }, + { + "value": { + "selection": { + "directReference": { + "structField": { + "field": 7 + } + } + } + } + }, + { + "value": { + "literal": { + "string": "file_modification_time" + } + } + }, + { + "value": { + "selection": { + "directReference": { + "structField": { + "field": 8 + } + } + } + } + } + ] + } + } + } + ] + } + } + ] + } + }, + "expressions": [ + { + "selection": { + "directReference": { + "structField": {} + } + } + }, + { + "selection": { + "directReference": { + "structField": { + "field": 1 + } + } + } + }, + { + "selection": { + "directReference": { + "structField": { + "field": 2 + } + } + } + }, + { + "scalarFunction": { + "functionReference": 2, + "outputType": { + "string": { + "nullability": "NULLABILITY_REQUIRED" + } + }, + "arguments": [ + { + "value": { + "scalarFunction": { + "functionReference": 3, + "outputType": { + "string": { + "nullability": "NULLABILITY_REQUIRED" + } + }, + "arguments": [ + { + "value": { + "selection": { + "directReference": { + "structField": { + "field": 3 + } + } + } + } + }, + { + "value": { + "literal": { + "i32": 1 + } + } + } + ] + } + } + } + ] + } + }, + { + "scalarFunction": { + "functionReference": 2, + "outputType": { + "string": { + "nullability": "NULLABILITY_REQUIRED" + } + }, + "arguments": [ + { + "value": { + "scalarFunction": { + "functionReference": 3, + "outputType": { + "string": { + "nullability": "NULLABILITY_REQUIRED" + } + }, + "arguments": [ + { + "value": { + "selection": { + "directReference": { + "structField": { + "field": 3 + } + } + } + } + }, + { + "value": { + "literal": { + "i32": 0 + } + } + } + ] + } + } + } + ] + } + }, + { + "scalarFunction": { + "functionReference": 4, + "outputType": { + "i64": { + "nullability": "NULLABILITY_REQUIRED" + } + }, + "arguments": [ + { + "value": { + "scalarFunction": { + "functionReference": 3, + "outputType": { + "i64": { + "nullability": "NULLABILITY_REQUIRED" + } + }, + "arguments": [ + { + "value": { + "selection": { + "directReference": { + "structField": { + "field": 3 + } + } + } + } + }, + { + "value": { + "literal": { + "i32": 2 + } + } + } + ] + } + } + } + ] + } + }, + { + "scalarFunction": { + "functionReference": 4, + "outputType": { + "i64": { + "nullability": "NULLABILITY_REQUIRED" + } + }, + "arguments": [ + { + "value": { + "scalarFunction": { + "functionReference": 3, + "outputType": { + "i64": { + "nullability": "NULLABILITY_REQUIRED" + } + }, + "arguments": [ + { + "value": { + "selection": { + "directReference": { + "structField": { + "field": 3 + } + } + } + } + }, + { + "value": { + "literal": { + "i32": 3 + } + } + } + ] + } + } + } + ] + } + }, + { + "scalarFunction": { + "functionReference": 4, + "outputType": { + "i64": { + "nullability": "NULLABILITY_REQUIRED" + } + }, + "arguments": [ + { + "value": { + "scalarFunction": { + "functionReference": 3, + "outputType": { + "i64": { + "nullability": "NULLABILITY_REQUIRED" + } + }, + "arguments": [ + { + "value": { + "selection": { + "directReference": { + "structField": { + "field": 3 + } + } + } + } + }, + { + "value": { + "literal": { + "i32": 4 + } + } + } + ] + } + } + } + ] + } + }, + { + "scalarFunction": { + "functionReference": 5, + "outputType": { + "timestamp": { + "nullability": "NULLABILITY_REQUIRED" + } + }, + "arguments": [ + { + "value": { + "scalarFunction": { + "functionReference": 3, + "outputType": { + "timestamp": { + "nullability": "NULLABILITY_REQUIRED" + } + }, + "arguments": [ + { + "value": { + "selection": { + "directReference": { + "structField": { + "field": 3 + } + } + } + } + }, + { + "value": { + "literal": { + "i32": 5 + } + } + } + ] + } + } + } + ] + } + } + ] + } + }, + "names": [ + "name#18", + "age#19", + "info#20", + "file_name#25", + "file_path#26", + "file_size#27", + "file_block_start#28", + "file_block_length#29", + "file_modification_time#30" + ], + "outputSchema": { + "types": [ + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "i32": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "struct": { + "types": [ + { + "i64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + } + ], + "nullability": "NULLABILITY_NULLABLE", + "names": [ + "id", + "university" + ] + } + }, + { + "string": { + "nullability": "NULLABILITY_REQUIRED" + } + }, + { + "string": { + "nullability": "NULLABILITY_REQUIRED" + } + }, + { + "i64": { + "nullability": "NULLABILITY_REQUIRED" + } + }, + { + "i64": { + "nullability": "NULLABILITY_REQUIRED" + } + }, + { + "i64": { + "nullability": "NULLABILITY_REQUIRED" + } + }, + { + "timestamp": { + "nullability": "NULLABILITY_REQUIRED" + } + } + ], + "nullability": "NULLABILITY_REQUIRED" + } + } + } + ] +} \ No newline at end of file diff --git a/cpp-ch/local-engine/tests/json/parquet_metadata/read_metadata.row_index.json b/cpp-ch/local-engine/tests/json/parquet_metadata/read_metadata.row_index.json new file mode 100644 index 000000000000..c5d839dc0364 --- /dev/null +++ b/cpp-ch/local-engine/tests/json/parquet_metadata/read_metadata.row_index.json @@ -0,0 +1,384 @@ +{ + "extensions": [ + { + "extensionFunction": { + "functionAnchor": 6, + "name": "get_struct_field:struct" + } + }, + { + "extensionFunction": { + "functionAnchor": 4, + "name": "not:bool" + } + }, + { + "extensionFunction": { + "functionAnchor": 2, + "name": "named_struct:str_i64" + } + }, + { + "extensionFunction": { + "name": "is_not_null:i64" + } + }, + { + "extensionFunction": { + "functionAnchor": 3, + "name": "and:bool_bool" + } + }, + { + "extensionFunction": { + "functionAnchor": 5, + "name": "equal:i64_i64" + } + }, + { + "extensionFunction": { + "functionAnchor": 1, + "name": "alias:struct" + } + }, + { + "extensionFunction": { + "functionAnchor": 7, + "name": "count:req_i32" + } + } + ], + "relations": [ + { + "root": { + "input": { + "aggregate": { + "common": { + "direct": {} + }, + "input": { + "project": { + "common": { + "emit": {} + }, + "input": { + "filter": { + "common": { + "direct": {} + }, + "input": { + "project": { + "common": { + "emit": { + "outputMapping": [ + 3, + 4, + 5 + ] + } + }, + "input": { + "read": { + "common": { + "direct": {} + }, + "baseSchema": { + "names": [ + "expected_row_idx", + "_tmp_metadata_row_index", + "pb" + ], + "struct": { + "types": [ + { + "i64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "i64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "i32": { + "nullability": "NULLABILITY_NULLABLE" + } + } + ] + }, + "columnTypes": [ + "NORMAL_COL", + "ROWINDEX_COL", + "PARTITION_COL" + ] + }, + "filter": { + "scalarFunction": { + "outputType": { + "bool": { + "nullability": "NULLABILITY_REQUIRED" + } + }, + "arguments": [ + { + "value": { + "selection": { + "directReference": { + "structField": {} + } + } + } + } + ] + } + }, + "advancedExtension": { + "optimization": { + "@type": "type.googleapis.com/google.protobuf.StringValue", + "value": "isMergeTree=0\n" + } + } + } + }, + "expressions": [ + { + "selection": { + "directReference": { + "structField": {} + } + } + }, + { + "selection": { + "directReference": { + "structField": { + "field": 2 + } + } + } + }, + { + "scalarFunction": { + "functionReference": 1, + "outputType": { + "struct": { + "types": [ + { + "i64": { + "nullability": "NULLABILITY_REQUIRED" + } + } + ], + "nullability": "NULLABILITY_REQUIRED", + "names": [ + "row_index" + ] + } + }, + "arguments": [ + { + "value": { + "scalarFunction": { + "functionReference": 2, + "outputType": { + "struct": { + "types": [ + { + "i64": { + "nullability": "NULLABILITY_REQUIRED" + } + } + ], + "nullability": "NULLABILITY_REQUIRED", + "names": [ + "row_index" + ] + } + }, + "arguments": [ + { + "value": { + "literal": { + "string": "row_index" + } + } + }, + { + "value": { + "selection": { + "directReference": { + "structField": { + "field": 1 + } + } + } + } + } + ] + } + } + } + ] + } + } + ] + } + }, + "condition": { + "scalarFunction": { + "functionReference": 3, + "outputType": { + "bool": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + "arguments": [ + { + "value": { + "scalarFunction": { + "outputType": { + "bool": { + "nullability": "NULLABILITY_REQUIRED" + } + }, + "arguments": [ + { + "value": { + "selection": { + "directReference": { + "structField": {} + } + } + } + } + ] + } + } + }, + { + "value": { + "scalarFunction": { + "functionReference": 4, + "outputType": { + "bool": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + "arguments": [ + { + "value": { + "scalarFunction": { + "functionReference": 5, + "outputType": { + "bool": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + "arguments": [ + { + "value": { + "selection": { + "directReference": { + "structField": {} + } + } + } + }, + { + "value": { + "scalarFunction": { + "functionReference": 6, + "outputType": { + "i64": { + "nullability": "NULLABILITY_REQUIRED" + } + }, + "arguments": [ + { + "value": { + "selection": { + "directReference": { + "structField": { + "field": 2 + } + } + } + } + }, + { + "value": { + "literal": { + "i32": 0 + } + } + } + ] + } + } + } + ] + } + } + } + ] + } + } + } + ] + } + } + } + } + } + }, + "groupings": [ + {} + ], + "measures": [ + { + "measure": { + "functionReference": 7, + "phase": "AGGREGATION_PHASE_INITIAL_TO_INTERMEDIATE", + "outputType": { + "i64": { + "nullability": "NULLABILITY_REQUIRED" + } + }, + "arguments": [ + { + "value": { + "literal": { + "i32": 1 + } + } + } + ] + } + } + ], + "advancedExtension": { + "optimization": { + "@type": "type.googleapis.com/google.protobuf.StringValue", + "value": "AggregateParams:hasPrePartialAggregate=false\nhasRequiredChildDistributionExpressions=false\n" + } + } + } + }, + "names": [ + "count#64" + ], + "outputSchema": { + "types": [ + { + "i64": { + "nullability": "NULLABILITY_REQUIRED" + } + } + ], + "nullability": "NULLABILITY_REQUIRED" + } + } + } + ] +} \ No newline at end of file diff --git a/cpp-ch/local-engine/tests/json/parquet_metadata/rowindex_in.json b/cpp-ch/local-engine/tests/json/parquet_metadata/rowindex_in.json new file mode 100644 index 000000000000..b513fb3e3362 --- /dev/null +++ b/cpp-ch/local-engine/tests/json/parquet_metadata/rowindex_in.json @@ -0,0 +1 @@ +{"extensions":[{"extensionFunction":{"name":"count:req_i32"}}],"relations":[{"root":{"input":{"aggregate":{"common":{"direct":{}},"input":{"project":{"common":{"emit":{}},"input":{"filter":{"common":{"direct":{}},"input":{"read":{"common":{"direct":{}},"baseSchema":{"names":["_tmp_metadata_row_index"],"struct":{"types":[{"i64":{"nullability":"NULLABILITY_NULLABLE"}}]},"columnTypes":["ROWINDEX_COL"]},"advancedExtension":{"optimization":{"@type":"type.googleapis.com/google.protobuf.StringValue","value":"isMergeTree=0\n"}}}},"condition":{"singularOrList":{"value":{"selection":{"directReference":{"structField":{}}}},"options":[{"literal":{"i64":"0"}},{"literal":{"i64":"1"}},{"literal":{"i64":"10"}},{"literal":{"i64":"100"}},{"literal":{"i64":"1000"}},{"literal":{"i64":"1001"}},{"literal":{"i64":"1002"}},{"literal":{"i64":"1003"}},{"literal":{"i64":"1004"}},{"literal":{"i64":"1005"}},{"literal":{"i64":"1006"}},{"literal":{"i64":"1007"}},{"literal":{"i64":"1008"}},{"literal":{"i64":"1009"}},{"literal":{"i64":"101"}},{"literal":{"i64":"1010"}},{"literal":{"i64":"1011"}},{"literal":{"i64":"1012"}},{"literal":{"i64":"1013"}},{"literal":{"i64":"1014"}},{"literal":{"i64":"1015"}},{"literal":{"i64":"1016"}},{"literal":{"i64":"1017"}},{"literal":{"i64":"1018"}},{"literal":{"i64":"1019"}},{"literal":{"i64":"102"}},{"literal":{"i64":"1020"}},{"literal":{"i64":"1021"}},{"literal":{"i64":"1022"}},{"literal":{"i64":"1023"}},{"literal":{"i64":"1024"}},{"literal":{"i64":"1025"}},{"literal":{"i64":"1026"}},{"literal":{"i64":"1027"}},{"literal":{"i64":"1028"}},{"literal":{"i64":"1029"}},{"literal":{"i64":"103"}},{"literal":{"i64":"1030"}},{"literal":{"i64":"1031"}},{"literal":{"i64":"1032"}},{"literal":{"i64":"1033"}},{"literal":{"i64":"1034"}},{"literal":{"i64":"1035"}},{"literal":{"i64":"1036"}},{"literal":{"i64":"1037"}},{"literal":{"i64":"1038"}},{"literal":{"i64":"1039"}},{"literal":{"i64":"104"}},{"literal":{"i64":"1040"}},{"literal":{"i64":"1041"}},{"literal":{"i64":"1042"}},{"literal":{"i64":"1043"}},{"literal":{"i64":"1044"}},{"literal":{"i64":"1045"}},{"literal":{"i64":"1046"}},{"literal":{"i64":"1047"}},{"literal":{"i64":"1048"}},{"literal":{"i64":"1049"}},{"literal":{"i64":"105"}},{"literal":{"i64":"1050"}},{"literal":{"i64":"1051"}},{"literal":{"i64":"1052"}},{"literal":{"i64":"1053"}},{"literal":{"i64":"1054"}},{"literal":{"i64":"1055"}},{"literal":{"i64":"1056"}},{"literal":{"i64":"1057"}},{"literal":{"i64":"1058"}},{"literal":{"i64":"1059"}},{"literal":{"i64":"106"}},{"literal":{"i64":"1060"}},{"literal":{"i64":"1061"}},{"literal":{"i64":"1062"}},{"literal":{"i64":"1063"}},{"literal":{"i64":"1064"}},{"literal":{"i64":"1065"}},{"literal":{"i64":"1066"}},{"literal":{"i64":"1067"}},{"literal":{"i64":"1068"}},{"literal":{"i64":"1069"}},{"literal":{"i64":"107"}},{"literal":{"i64":"1070"}},{"literal":{"i64":"1071"}},{"literal":{"i64":"1072"}},{"literal":{"i64":"1073"}},{"literal":{"i64":"1074"}},{"literal":{"i64":"1075"}},{"literal":{"i64":"1076"}},{"literal":{"i64":"1077"}},{"literal":{"i64":"1078"}},{"literal":{"i64":"1079"}},{"literal":{"i64":"108"}},{"literal":{"i64":"1080"}},{"literal":{"i64":"1081"}},{"literal":{"i64":"1082"}},{"literal":{"i64":"1083"}},{"literal":{"i64":"1084"}},{"literal":{"i64":"1085"}},{"literal":{"i64":"1086"}},{"literal":{"i64":"1087"}},{"literal":{"i64":"1088"}},{"literal":{"i64":"1089"}},{"literal":{"i64":"109"}},{"literal":{"i64":"1090"}},{"literal":{"i64":"1091"}},{"literal":{"i64":"1092"}},{"literal":{"i64":"1093"}},{"literal":{"i64":"1094"}},{"literal":{"i64":"1095"}},{"literal":{"i64":"1096"}},{"literal":{"i64":"1097"}},{"literal":{"i64":"1098"}},{"literal":{"i64":"1099"}},{"literal":{"i64":"11"}},{"literal":{"i64":"110"}},{"literal":{"i64":"1100"}},{"literal":{"i64":"1101"}},{"literal":{"i64":"1102"}},{"literal":{"i64":"1103"}},{"literal":{"i64":"1104"}},{"literal":{"i64":"1105"}},{"literal":{"i64":"1106"}},{"literal":{"i64":"1107"}},{"literal":{"i64":"1108"}},{"literal":{"i64":"1109"}},{"literal":{"i64":"111"}},{"literal":{"i64":"1110"}},{"literal":{"i64":"1111"}},{"literal":{"i64":"1112"}},{"literal":{"i64":"1113"}},{"literal":{"i64":"1114"}},{"literal":{"i64":"1115"}},{"literal":{"i64":"1116"}},{"literal":{"i64":"1117"}},{"literal":{"i64":"1118"}},{"literal":{"i64":"1119"}},{"literal":{"i64":"112"}},{"literal":{"i64":"1120"}},{"literal":{"i64":"1121"}},{"literal":{"i64":"1122"}},{"literal":{"i64":"1123"}},{"literal":{"i64":"1124"}},{"literal":{"i64":"1125"}},{"literal":{"i64":"1126"}},{"literal":{"i64":"1127"}},{"literal":{"i64":"1128"}},{"literal":{"i64":"1129"}},{"literal":{"i64":"113"}},{"literal":{"i64":"1130"}},{"literal":{"i64":"1131"}},{"literal":{"i64":"1132"}},{"literal":{"i64":"1133"}},{"literal":{"i64":"1134"}},{"literal":{"i64":"1135"}},{"literal":{"i64":"1136"}},{"literal":{"i64":"1137"}},{"literal":{"i64":"1138"}},{"literal":{"i64":"1139"}},{"literal":{"i64":"114"}},{"literal":{"i64":"1140"}},{"literal":{"i64":"1141"}},{"literal":{"i64":"1142"}},{"literal":{"i64":"1143"}},{"literal":{"i64":"1144"}},{"literal":{"i64":"1145"}},{"literal":{"i64":"1146"}},{"literal":{"i64":"1147"}},{"literal":{"i64":"1148"}},{"literal":{"i64":"1149"}},{"literal":{"i64":"115"}},{"literal":{"i64":"1150"}},{"literal":{"i64":"1151"}},{"literal":{"i64":"1152"}},{"literal":{"i64":"1153"}},{"literal":{"i64":"1154"}},{"literal":{"i64":"1155"}},{"literal":{"i64":"1156"}},{"literal":{"i64":"1157"}},{"literal":{"i64":"1158"}},{"literal":{"i64":"1159"}},{"literal":{"i64":"116"}},{"literal":{"i64":"1160"}},{"literal":{"i64":"1161"}},{"literal":{"i64":"1162"}},{"literal":{"i64":"1163"}},{"literal":{"i64":"1164"}},{"literal":{"i64":"1165"}},{"literal":{"i64":"1166"}},{"literal":{"i64":"1167"}},{"literal":{"i64":"1168"}},{"literal":{"i64":"1169"}},{"literal":{"i64":"117"}},{"literal":{"i64":"1170"}},{"literal":{"i64":"1171"}},{"literal":{"i64":"1172"}},{"literal":{"i64":"1173"}},{"literal":{"i64":"1174"}},{"literal":{"i64":"1175"}},{"literal":{"i64":"1176"}},{"literal":{"i64":"1177"}},{"literal":{"i64":"1178"}},{"literal":{"i64":"1179"}},{"literal":{"i64":"118"}},{"literal":{"i64":"1180"}},{"literal":{"i64":"1181"}},{"literal":{"i64":"1182"}},{"literal":{"i64":"1183"}},{"literal":{"i64":"1184"}},{"literal":{"i64":"1185"}},{"literal":{"i64":"1186"}},{"literal":{"i64":"1187"}},{"literal":{"i64":"1188"}},{"literal":{"i64":"1189"}},{"literal":{"i64":"119"}},{"literal":{"i64":"1190"}},{"literal":{"i64":"1191"}},{"literal":{"i64":"1192"}},{"literal":{"i64":"1193"}},{"literal":{"i64":"1194"}},{"literal":{"i64":"1195"}},{"literal":{"i64":"1196"}},{"literal":{"i64":"1197"}},{"literal":{"i64":"1198"}},{"literal":{"i64":"1199"}},{"literal":{"i64":"12"}},{"literal":{"i64":"120"}},{"literal":{"i64":"1200"}},{"literal":{"i64":"1201"}},{"literal":{"i64":"1202"}},{"literal":{"i64":"1203"}},{"literal":{"i64":"1204"}},{"literal":{"i64":"1205"}},{"literal":{"i64":"1206"}},{"literal":{"i64":"1207"}},{"literal":{"i64":"1208"}},{"literal":{"i64":"1209"}},{"literal":{"i64":"121"}},{"literal":{"i64":"1210"}},{"literal":{"i64":"1211"}},{"literal":{"i64":"1212"}},{"literal":{"i64":"1213"}},{"literal":{"i64":"1214"}},{"literal":{"i64":"1215"}},{"literal":{"i64":"1216"}},{"literal":{"i64":"1217"}},{"literal":{"i64":"1218"}},{"literal":{"i64":"1219"}},{"literal":{"i64":"122"}},{"literal":{"i64":"1220"}},{"literal":{"i64":"1221"}},{"literal":{"i64":"1222"}},{"literal":{"i64":"1223"}},{"literal":{"i64":"1224"}},{"literal":{"i64":"1225"}},{"literal":{"i64":"1226"}},{"literal":{"i64":"1227"}},{"literal":{"i64":"1228"}},{"literal":{"i64":"1229"}},{"literal":{"i64":"123"}},{"literal":{"i64":"1230"}},{"literal":{"i64":"1231"}},{"literal":{"i64":"1232"}},{"literal":{"i64":"1233"}},{"literal":{"i64":"1234"}},{"literal":{"i64":"1235"}},{"literal":{"i64":"1236"}},{"literal":{"i64":"1237"}},{"literal":{"i64":"1238"}},{"literal":{"i64":"1239"}},{"literal":{"i64":"124"}},{"literal":{"i64":"1240"}},{"literal":{"i64":"1241"}},{"literal":{"i64":"1242"}},{"literal":{"i64":"1243"}},{"literal":{"i64":"1244"}},{"literal":{"i64":"1245"}},{"literal":{"i64":"1246"}},{"literal":{"i64":"1247"}},{"literal":{"i64":"1248"}},{"literal":{"i64":"1249"}},{"literal":{"i64":"125"}},{"literal":{"i64":"1250"}},{"literal":{"i64":"1251"}},{"literal":{"i64":"1252"}},{"literal":{"i64":"1253"}},{"literal":{"i64":"1254"}},{"literal":{"i64":"1255"}},{"literal":{"i64":"1256"}},{"literal":{"i64":"1257"}},{"literal":{"i64":"1258"}},{"literal":{"i64":"1259"}},{"literal":{"i64":"126"}},{"literal":{"i64":"1260"}},{"literal":{"i64":"1261"}},{"literal":{"i64":"1262"}},{"literal":{"i64":"1263"}},{"literal":{"i64":"1264"}},{"literal":{"i64":"1265"}},{"literal":{"i64":"1266"}},{"literal":{"i64":"1267"}},{"literal":{"i64":"1268"}},{"literal":{"i64":"1269"}},{"literal":{"i64":"127"}},{"literal":{"i64":"1270"}},{"literal":{"i64":"1271"}},{"literal":{"i64":"1272"}},{"literal":{"i64":"1273"}},{"literal":{"i64":"1274"}},{"literal":{"i64":"1275"}},{"literal":{"i64":"1276"}},{"literal":{"i64":"1277"}},{"literal":{"i64":"1278"}},{"literal":{"i64":"1279"}},{"literal":{"i64":"128"}},{"literal":{"i64":"1280"}},{"literal":{"i64":"1281"}},{"literal":{"i64":"1282"}},{"literal":{"i64":"1283"}},{"literal":{"i64":"1284"}},{"literal":{"i64":"1285"}},{"literal":{"i64":"1286"}},{"literal":{"i64":"1287"}},{"literal":{"i64":"1288"}},{"literal":{"i64":"1289"}},{"literal":{"i64":"129"}},{"literal":{"i64":"1290"}},{"literal":{"i64":"1291"}},{"literal":{"i64":"1292"}},{"literal":{"i64":"1293"}},{"literal":{"i64":"1294"}},{"literal":{"i64":"1295"}},{"literal":{"i64":"1296"}},{"literal":{"i64":"1297"}},{"literal":{"i64":"1298"}},{"literal":{"i64":"1299"}},{"literal":{"i64":"13"}},{"literal":{"i64":"130"}},{"literal":{"i64":"1300"}},{"literal":{"i64":"1301"}},{"literal":{"i64":"1302"}},{"literal":{"i64":"1303"}},{"literal":{"i64":"1304"}},{"literal":{"i64":"1305"}},{"literal":{"i64":"1306"}},{"literal":{"i64":"1307"}},{"literal":{"i64":"1308"}},{"literal":{"i64":"1309"}},{"literal":{"i64":"131"}},{"literal":{"i64":"1310"}},{"literal":{"i64":"1311"}},{"literal":{"i64":"1312"}},{"literal":{"i64":"1313"}},{"literal":{"i64":"1314"}},{"literal":{"i64":"1315"}},{"literal":{"i64":"1316"}},{"literal":{"i64":"1317"}},{"literal":{"i64":"1318"}},{"literal":{"i64":"1319"}},{"literal":{"i64":"132"}},{"literal":{"i64":"1320"}},{"literal":{"i64":"1321"}},{"literal":{"i64":"1322"}},{"literal":{"i64":"1323"}},{"literal":{"i64":"1324"}},{"literal":{"i64":"1325"}},{"literal":{"i64":"1326"}},{"literal":{"i64":"1327"}},{"literal":{"i64":"1328"}},{"literal":{"i64":"1329"}},{"literal":{"i64":"133"}},{"literal":{"i64":"1330"}},{"literal":{"i64":"1331"}},{"literal":{"i64":"1332"}},{"literal":{"i64":"1333"}},{"literal":{"i64":"1334"}},{"literal":{"i64":"1335"}},{"literal":{"i64":"1336"}},{"literal":{"i64":"1337"}},{"literal":{"i64":"1338"}},{"literal":{"i64":"1339"}},{"literal":{"i64":"134"}},{"literal":{"i64":"1340"}},{"literal":{"i64":"1341"}},{"literal":{"i64":"1342"}},{"literal":{"i64":"1343"}},{"literal":{"i64":"1344"}},{"literal":{"i64":"1345"}},{"literal":{"i64":"1346"}},{"literal":{"i64":"1347"}},{"literal":{"i64":"1348"}},{"literal":{"i64":"1349"}},{"literal":{"i64":"135"}},{"literal":{"i64":"1350"}},{"literal":{"i64":"1351"}},{"literal":{"i64":"1352"}},{"literal":{"i64":"1353"}},{"literal":{"i64":"1354"}},{"literal":{"i64":"1355"}},{"literal":{"i64":"1356"}},{"literal":{"i64":"1357"}},{"literal":{"i64":"1358"}},{"literal":{"i64":"1359"}},{"literal":{"i64":"136"}},{"literal":{"i64":"1360"}},{"literal":{"i64":"1361"}},{"literal":{"i64":"1362"}},{"literal":{"i64":"1363"}},{"literal":{"i64":"1364"}},{"literal":{"i64":"1365"}},{"literal":{"i64":"1366"}},{"literal":{"i64":"1367"}},{"literal":{"i64":"1368"}},{"literal":{"i64":"1369"}},{"literal":{"i64":"137"}},{"literal":{"i64":"1370"}},{"literal":{"i64":"1371"}},{"literal":{"i64":"1372"}},{"literal":{"i64":"1373"}},{"literal":{"i64":"1374"}},{"literal":{"i64":"1375"}},{"literal":{"i64":"1376"}},{"literal":{"i64":"1377"}},{"literal":{"i64":"1378"}},{"literal":{"i64":"1379"}},{"literal":{"i64":"138"}},{"literal":{"i64":"1380"}},{"literal":{"i64":"1381"}},{"literal":{"i64":"1382"}},{"literal":{"i64":"1383"}},{"literal":{"i64":"1384"}},{"literal":{"i64":"1385"}},{"literal":{"i64":"1386"}},{"literal":{"i64":"1387"}},{"literal":{"i64":"1388"}},{"literal":{"i64":"1389"}},{"literal":{"i64":"139"}},{"literal":{"i64":"1390"}},{"literal":{"i64":"1391"}},{"literal":{"i64":"1392"}},{"literal":{"i64":"1393"}},{"literal":{"i64":"1394"}},{"literal":{"i64":"1395"}},{"literal":{"i64":"1396"}},{"literal":{"i64":"1397"}},{"literal":{"i64":"1398"}},{"literal":{"i64":"1399"}},{"literal":{"i64":"14"}},{"literal":{"i64":"140"}},{"literal":{"i64":"1400"}},{"literal":{"i64":"1401"}},{"literal":{"i64":"1402"}},{"literal":{"i64":"1403"}},{"literal":{"i64":"1404"}},{"literal":{"i64":"1405"}},{"literal":{"i64":"1406"}},{"literal":{"i64":"1407"}},{"literal":{"i64":"1408"}},{"literal":{"i64":"1409"}},{"literal":{"i64":"141"}},{"literal":{"i64":"1410"}},{"literal":{"i64":"1411"}},{"literal":{"i64":"1412"}},{"literal":{"i64":"1413"}},{"literal":{"i64":"1414"}},{"literal":{"i64":"1415"}},{"literal":{"i64":"1416"}},{"literal":{"i64":"1417"}},{"literal":{"i64":"1418"}},{"literal":{"i64":"1419"}},{"literal":{"i64":"142"}},{"literal":{"i64":"1420"}},{"literal":{"i64":"1421"}},{"literal":{"i64":"1422"}},{"literal":{"i64":"1423"}},{"literal":{"i64":"1424"}},{"literal":{"i64":"1425"}},{"literal":{"i64":"1426"}},{"literal":{"i64":"1427"}},{"literal":{"i64":"1428"}},{"literal":{"i64":"1429"}},{"literal":{"i64":"143"}},{"literal":{"i64":"1430"}},{"literal":{"i64":"1431"}},{"literal":{"i64":"1432"}},{"literal":{"i64":"1433"}},{"literal":{"i64":"1434"}},{"literal":{"i64":"1435"}},{"literal":{"i64":"1436"}},{"literal":{"i64":"1437"}},{"literal":{"i64":"1438"}},{"literal":{"i64":"1439"}},{"literal":{"i64":"144"}},{"literal":{"i64":"1440"}},{"literal":{"i64":"1441"}},{"literal":{"i64":"1442"}},{"literal":{"i64":"1443"}},{"literal":{"i64":"1444"}},{"literal":{"i64":"1445"}},{"literal":{"i64":"1446"}},{"literal":{"i64":"1447"}},{"literal":{"i64":"1448"}},{"literal":{"i64":"1449"}},{"literal":{"i64":"145"}},{"literal":{"i64":"1450"}},{"literal":{"i64":"1451"}},{"literal":{"i64":"1452"}},{"literal":{"i64":"1453"}},{"literal":{"i64":"1454"}},{"literal":{"i64":"1455"}},{"literal":{"i64":"1456"}},{"literal":{"i64":"1457"}},{"literal":{"i64":"1458"}},{"literal":{"i64":"1459"}},{"literal":{"i64":"146"}},{"literal":{"i64":"1460"}},{"literal":{"i64":"1461"}},{"literal":{"i64":"1462"}},{"literal":{"i64":"1463"}},{"literal":{"i64":"1464"}},{"literal":{"i64":"1465"}},{"literal":{"i64":"1466"}},{"literal":{"i64":"1467"}},{"literal":{"i64":"1468"}},{"literal":{"i64":"1469"}},{"literal":{"i64":"147"}},{"literal":{"i64":"1470"}},{"literal":{"i64":"1471"}},{"literal":{"i64":"1472"}},{"literal":{"i64":"1473"}},{"literal":{"i64":"1474"}},{"literal":{"i64":"1475"}},{"literal":{"i64":"1476"}},{"literal":{"i64":"1477"}},{"literal":{"i64":"1478"}},{"literal":{"i64":"1479"}},{"literal":{"i64":"148"}},{"literal":{"i64":"1480"}},{"literal":{"i64":"1481"}},{"literal":{"i64":"1482"}},{"literal":{"i64":"1483"}},{"literal":{"i64":"1484"}},{"literal":{"i64":"1485"}},{"literal":{"i64":"1486"}},{"literal":{"i64":"1487"}},{"literal":{"i64":"1488"}},{"literal":{"i64":"1489"}},{"literal":{"i64":"149"}},{"literal":{"i64":"1490"}},{"literal":{"i64":"1491"}},{"literal":{"i64":"1492"}},{"literal":{"i64":"1493"}},{"literal":{"i64":"1494"}},{"literal":{"i64":"1495"}},{"literal":{"i64":"1496"}},{"literal":{"i64":"1497"}},{"literal":{"i64":"1498"}},{"literal":{"i64":"1499"}},{"literal":{"i64":"15"}},{"literal":{"i64":"150"}},{"literal":{"i64":"1500"}},{"literal":{"i64":"1501"}},{"literal":{"i64":"1502"}},{"literal":{"i64":"1503"}},{"literal":{"i64":"1504"}},{"literal":{"i64":"1505"}},{"literal":{"i64":"1506"}},{"literal":{"i64":"1507"}},{"literal":{"i64":"1508"}},{"literal":{"i64":"1509"}},{"literal":{"i64":"151"}},{"literal":{"i64":"1510"}},{"literal":{"i64":"1511"}},{"literal":{"i64":"1512"}},{"literal":{"i64":"1513"}},{"literal":{"i64":"1514"}},{"literal":{"i64":"1515"}},{"literal":{"i64":"1516"}},{"literal":{"i64":"1517"}},{"literal":{"i64":"1518"}},{"literal":{"i64":"1519"}},{"literal":{"i64":"152"}},{"literal":{"i64":"1520"}},{"literal":{"i64":"1521"}},{"literal":{"i64":"1522"}},{"literal":{"i64":"1523"}},{"literal":{"i64":"1524"}},{"literal":{"i64":"1525"}},{"literal":{"i64":"1526"}},{"literal":{"i64":"1527"}},{"literal":{"i64":"1528"}},{"literal":{"i64":"1529"}},{"literal":{"i64":"153"}},{"literal":{"i64":"1530"}},{"literal":{"i64":"1531"}},{"literal":{"i64":"1532"}},{"literal":{"i64":"1533"}},{"literal":{"i64":"1534"}},{"literal":{"i64":"1535"}},{"literal":{"i64":"1536"}},{"literal":{"i64":"1537"}},{"literal":{"i64":"1538"}},{"literal":{"i64":"1539"}},{"literal":{"i64":"154"}},{"literal":{"i64":"1540"}},{"literal":{"i64":"1541"}},{"literal":{"i64":"1542"}},{"literal":{"i64":"1543"}},{"literal":{"i64":"1544"}},{"literal":{"i64":"1545"}},{"literal":{"i64":"1546"}},{"literal":{"i64":"1547"}},{"literal":{"i64":"1548"}},{"literal":{"i64":"1549"}},{"literal":{"i64":"155"}},{"literal":{"i64":"1550"}},{"literal":{"i64":"1551"}},{"literal":{"i64":"1552"}},{"literal":{"i64":"1553"}},{"literal":{"i64":"1554"}},{"literal":{"i64":"1555"}},{"literal":{"i64":"1556"}},{"literal":{"i64":"1557"}},{"literal":{"i64":"1558"}},{"literal":{"i64":"1559"}},{"literal":{"i64":"156"}},{"literal":{"i64":"1560"}},{"literal":{"i64":"1561"}},{"literal":{"i64":"1562"}},{"literal":{"i64":"1563"}},{"literal":{"i64":"1564"}},{"literal":{"i64":"1565"}},{"literal":{"i64":"1566"}},{"literal":{"i64":"1567"}},{"literal":{"i64":"1568"}},{"literal":{"i64":"1569"}},{"literal":{"i64":"157"}},{"literal":{"i64":"1570"}},{"literal":{"i64":"1571"}},{"literal":{"i64":"1572"}},{"literal":{"i64":"1573"}},{"literal":{"i64":"1574"}},{"literal":{"i64":"1575"}},{"literal":{"i64":"1576"}},{"literal":{"i64":"1577"}},{"literal":{"i64":"1578"}},{"literal":{"i64":"1579"}},{"literal":{"i64":"158"}},{"literal":{"i64":"1580"}},{"literal":{"i64":"1581"}},{"literal":{"i64":"1582"}},{"literal":{"i64":"1583"}},{"literal":{"i64":"1584"}},{"literal":{"i64":"1585"}},{"literal":{"i64":"1586"}},{"literal":{"i64":"1587"}},{"literal":{"i64":"1588"}},{"literal":{"i64":"1589"}},{"literal":{"i64":"159"}},{"literal":{"i64":"1590"}},{"literal":{"i64":"1591"}},{"literal":{"i64":"1592"}},{"literal":{"i64":"1593"}},{"literal":{"i64":"1594"}},{"literal":{"i64":"1595"}},{"literal":{"i64":"1596"}},{"literal":{"i64":"1597"}},{"literal":{"i64":"1598"}},{"literal":{"i64":"1599"}},{"literal":{"i64":"16"}},{"literal":{"i64":"160"}},{"literal":{"i64":"1600"}},{"literal":{"i64":"1601"}},{"literal":{"i64":"1602"}},{"literal":{"i64":"1603"}},{"literal":{"i64":"1604"}},{"literal":{"i64":"1605"}},{"literal":{"i64":"1606"}},{"literal":{"i64":"1607"}},{"literal":{"i64":"1608"}},{"literal":{"i64":"1609"}},{"literal":{"i64":"161"}},{"literal":{"i64":"1610"}},{"literal":{"i64":"1611"}},{"literal":{"i64":"1612"}},{"literal":{"i64":"1613"}},{"literal":{"i64":"1614"}},{"literal":{"i64":"1615"}},{"literal":{"i64":"1616"}},{"literal":{"i64":"1617"}},{"literal":{"i64":"1618"}},{"literal":{"i64":"1619"}},{"literal":{"i64":"162"}},{"literal":{"i64":"1620"}},{"literal":{"i64":"1621"}},{"literal":{"i64":"1622"}},{"literal":{"i64":"1623"}},{"literal":{"i64":"1624"}},{"literal":{"i64":"1625"}},{"literal":{"i64":"1626"}},{"literal":{"i64":"1627"}},{"literal":{"i64":"1628"}},{"literal":{"i64":"1629"}},{"literal":{"i64":"163"}},{"literal":{"i64":"1630"}},{"literal":{"i64":"1631"}},{"literal":{"i64":"1632"}},{"literal":{"i64":"1633"}},{"literal":{"i64":"1634"}},{"literal":{"i64":"1635"}},{"literal":{"i64":"1636"}},{"literal":{"i64":"1637"}},{"literal":{"i64":"1638"}},{"literal":{"i64":"1639"}},{"literal":{"i64":"164"}},{"literal":{"i64":"1640"}},{"literal":{"i64":"1641"}},{"literal":{"i64":"1642"}},{"literal":{"i64":"1643"}},{"literal":{"i64":"1644"}},{"literal":{"i64":"1645"}},{"literal":{"i64":"1646"}},{"literal":{"i64":"1647"}},{"literal":{"i64":"1648"}},{"literal":{"i64":"1649"}},{"literal":{"i64":"165"}},{"literal":{"i64":"1650"}},{"literal":{"i64":"1651"}},{"literal":{"i64":"1652"}},{"literal":{"i64":"1653"}},{"literal":{"i64":"1654"}},{"literal":{"i64":"1655"}},{"literal":{"i64":"1656"}},{"literal":{"i64":"1657"}},{"literal":{"i64":"1658"}},{"literal":{"i64":"1659"}},{"literal":{"i64":"166"}},{"literal":{"i64":"1660"}},{"literal":{"i64":"1661"}},{"literal":{"i64":"1662"}},{"literal":{"i64":"1663"}},{"literal":{"i64":"1664"}},{"literal":{"i64":"1665"}},{"literal":{"i64":"1666"}},{"literal":{"i64":"1667"}},{"literal":{"i64":"1668"}},{"literal":{"i64":"1669"}},{"literal":{"i64":"167"}},{"literal":{"i64":"1670"}},{"literal":{"i64":"1671"}},{"literal":{"i64":"1672"}},{"literal":{"i64":"1673"}},{"literal":{"i64":"1674"}},{"literal":{"i64":"1675"}},{"literal":{"i64":"1676"}},{"literal":{"i64":"1677"}},{"literal":{"i64":"1678"}},{"literal":{"i64":"1679"}},{"literal":{"i64":"168"}},{"literal":{"i64":"1680"}},{"literal":{"i64":"1681"}},{"literal":{"i64":"1682"}},{"literal":{"i64":"1683"}},{"literal":{"i64":"1684"}},{"literal":{"i64":"1685"}},{"literal":{"i64":"1686"}},{"literal":{"i64":"1687"}},{"literal":{"i64":"1688"}},{"literal":{"i64":"1689"}},{"literal":{"i64":"169"}},{"literal":{"i64":"1690"}},{"literal":{"i64":"1691"}},{"literal":{"i64":"1692"}},{"literal":{"i64":"1693"}},{"literal":{"i64":"1694"}},{"literal":{"i64":"1695"}},{"literal":{"i64":"1696"}},{"literal":{"i64":"1697"}},{"literal":{"i64":"1698"}},{"literal":{"i64":"1699"}},{"literal":{"i64":"17"}},{"literal":{"i64":"170"}},{"literal":{"i64":"1700"}},{"literal":{"i64":"1701"}},{"literal":{"i64":"1702"}},{"literal":{"i64":"1703"}},{"literal":{"i64":"1704"}},{"literal":{"i64":"1705"}},{"literal":{"i64":"1706"}},{"literal":{"i64":"1707"}},{"literal":{"i64":"1708"}},{"literal":{"i64":"1709"}},{"literal":{"i64":"171"}},{"literal":{"i64":"1710"}},{"literal":{"i64":"1711"}},{"literal":{"i64":"1712"}},{"literal":{"i64":"1713"}},{"literal":{"i64":"1714"}},{"literal":{"i64":"1715"}},{"literal":{"i64":"1716"}},{"literal":{"i64":"1717"}},{"literal":{"i64":"1718"}},{"literal":{"i64":"1719"}},{"literal":{"i64":"172"}},{"literal":{"i64":"1720"}},{"literal":{"i64":"1721"}},{"literal":{"i64":"1722"}},{"literal":{"i64":"1723"}},{"literal":{"i64":"1724"}},{"literal":{"i64":"1725"}},{"literal":{"i64":"1726"}},{"literal":{"i64":"1727"}},{"literal":{"i64":"1728"}},{"literal":{"i64":"1729"}},{"literal":{"i64":"173"}},{"literal":{"i64":"1730"}},{"literal":{"i64":"1731"}},{"literal":{"i64":"1732"}},{"literal":{"i64":"1733"}},{"literal":{"i64":"1734"}},{"literal":{"i64":"1735"}},{"literal":{"i64":"1736"}},{"literal":{"i64":"1737"}},{"literal":{"i64":"1738"}},{"literal":{"i64":"1739"}},{"literal":{"i64":"174"}},{"literal":{"i64":"1740"}},{"literal":{"i64":"1741"}},{"literal":{"i64":"1742"}},{"literal":{"i64":"1743"}},{"literal":{"i64":"1744"}},{"literal":{"i64":"1745"}},{"literal":{"i64":"1746"}},{"literal":{"i64":"1747"}},{"literal":{"i64":"1748"}},{"literal":{"i64":"1749"}},{"literal":{"i64":"175"}},{"literal":{"i64":"1750"}},{"literal":{"i64":"1751"}},{"literal":{"i64":"1752"}},{"literal":{"i64":"1753"}},{"literal":{"i64":"1754"}},{"literal":{"i64":"1755"}},{"literal":{"i64":"1756"}},{"literal":{"i64":"1757"}},{"literal":{"i64":"1758"}},{"literal":{"i64":"1759"}},{"literal":{"i64":"176"}},{"literal":{"i64":"1760"}},{"literal":{"i64":"1761"}},{"literal":{"i64":"1762"}},{"literal":{"i64":"1763"}},{"literal":{"i64":"1764"}},{"literal":{"i64":"1765"}},{"literal":{"i64":"1766"}},{"literal":{"i64":"1767"}},{"literal":{"i64":"1768"}},{"literal":{"i64":"1769"}},{"literal":{"i64":"177"}},{"literal":{"i64":"1770"}},{"literal":{"i64":"1771"}},{"literal":{"i64":"1772"}},{"literal":{"i64":"1773"}},{"literal":{"i64":"1774"}},{"literal":{"i64":"1775"}},{"literal":{"i64":"1776"}},{"literal":{"i64":"1777"}},{"literal":{"i64":"1778"}},{"literal":{"i64":"1779"}},{"literal":{"i64":"178"}},{"literal":{"i64":"1780"}},{"literal":{"i64":"1781"}},{"literal":{"i64":"1782"}},{"literal":{"i64":"1783"}},{"literal":{"i64":"1784"}},{"literal":{"i64":"1785"}},{"literal":{"i64":"1786"}},{"literal":{"i64":"1787"}},{"literal":{"i64":"1788"}},{"literal":{"i64":"1789"}},{"literal":{"i64":"179"}},{"literal":{"i64":"1790"}},{"literal":{"i64":"1791"}},{"literal":{"i64":"1792"}},{"literal":{"i64":"1793"}},{"literal":{"i64":"1794"}},{"literal":{"i64":"1795"}},{"literal":{"i64":"1796"}},{"literal":{"i64":"1797"}},{"literal":{"i64":"1798"}},{"literal":{"i64":"1799"}},{"literal":{"i64":"18"}},{"literal":{"i64":"180"}},{"literal":{"i64":"1800"}},{"literal":{"i64":"1801"}},{"literal":{"i64":"1802"}},{"literal":{"i64":"1803"}},{"literal":{"i64":"1804"}},{"literal":{"i64":"1805"}},{"literal":{"i64":"1806"}},{"literal":{"i64":"1807"}},{"literal":{"i64":"1808"}},{"literal":{"i64":"1809"}},{"literal":{"i64":"181"}},{"literal":{"i64":"1810"}},{"literal":{"i64":"1811"}},{"literal":{"i64":"1812"}},{"literal":{"i64":"1813"}},{"literal":{"i64":"1814"}},{"literal":{"i64":"1815"}},{"literal":{"i64":"1816"}},{"literal":{"i64":"1817"}},{"literal":{"i64":"1818"}},{"literal":{"i64":"1819"}},{"literal":{"i64":"182"}},{"literal":{"i64":"1820"}},{"literal":{"i64":"1821"}},{"literal":{"i64":"1822"}},{"literal":{"i64":"1823"}},{"literal":{"i64":"1824"}},{"literal":{"i64":"1825"}},{"literal":{"i64":"1826"}},{"literal":{"i64":"1827"}},{"literal":{"i64":"1828"}},{"literal":{"i64":"1829"}},{"literal":{"i64":"183"}},{"literal":{"i64":"1830"}},{"literal":{"i64":"1831"}},{"literal":{"i64":"1832"}},{"literal":{"i64":"1833"}},{"literal":{"i64":"1834"}},{"literal":{"i64":"1835"}},{"literal":{"i64":"1836"}},{"literal":{"i64":"1837"}},{"literal":{"i64":"1838"}},{"literal":{"i64":"1839"}},{"literal":{"i64":"184"}},{"literal":{"i64":"1840"}},{"literal":{"i64":"1841"}},{"literal":{"i64":"1842"}},{"literal":{"i64":"1843"}},{"literal":{"i64":"1844"}},{"literal":{"i64":"1845"}},{"literal":{"i64":"1846"}},{"literal":{"i64":"1847"}},{"literal":{"i64":"1848"}},{"literal":{"i64":"1849"}},{"literal":{"i64":"185"}},{"literal":{"i64":"1850"}},{"literal":{"i64":"1851"}},{"literal":{"i64":"1852"}},{"literal":{"i64":"1853"}},{"literal":{"i64":"1854"}},{"literal":{"i64":"1855"}},{"literal":{"i64":"1856"}},{"literal":{"i64":"1857"}},{"literal":{"i64":"1858"}},{"literal":{"i64":"1859"}},{"literal":{"i64":"186"}},{"literal":{"i64":"1860"}},{"literal":{"i64":"1861"}},{"literal":{"i64":"1862"}},{"literal":{"i64":"1863"}},{"literal":{"i64":"1864"}},{"literal":{"i64":"1865"}},{"literal":{"i64":"1866"}},{"literal":{"i64":"1867"}},{"literal":{"i64":"1868"}},{"literal":{"i64":"1869"}},{"literal":{"i64":"187"}},{"literal":{"i64":"1870"}},{"literal":{"i64":"1871"}},{"literal":{"i64":"1872"}},{"literal":{"i64":"1873"}},{"literal":{"i64":"1874"}},{"literal":{"i64":"1875"}},{"literal":{"i64":"1876"}},{"literal":{"i64":"1877"}},{"literal":{"i64":"1878"}},{"literal":{"i64":"1879"}},{"literal":{"i64":"188"}},{"literal":{"i64":"1880"}},{"literal":{"i64":"1881"}},{"literal":{"i64":"1882"}},{"literal":{"i64":"1883"}},{"literal":{"i64":"1884"}},{"literal":{"i64":"1885"}},{"literal":{"i64":"1886"}},{"literal":{"i64":"1887"}},{"literal":{"i64":"1888"}},{"literal":{"i64":"1889"}},{"literal":{"i64":"189"}},{"literal":{"i64":"1890"}},{"literal":{"i64":"1891"}},{"literal":{"i64":"1892"}},{"literal":{"i64":"1893"}},{"literal":{"i64":"1894"}},{"literal":{"i64":"1895"}},{"literal":{"i64":"1896"}},{"literal":{"i64":"1897"}},{"literal":{"i64":"1898"}},{"literal":{"i64":"1899"}},{"literal":{"i64":"19"}},{"literal":{"i64":"190"}},{"literal":{"i64":"1900"}},{"literal":{"i64":"1901"}},{"literal":{"i64":"1902"}},{"literal":{"i64":"1903"}},{"literal":{"i64":"1904"}},{"literal":{"i64":"1905"}},{"literal":{"i64":"1906"}},{"literal":{"i64":"1907"}},{"literal":{"i64":"1908"}},{"literal":{"i64":"1909"}},{"literal":{"i64":"191"}},{"literal":{"i64":"1910"}},{"literal":{"i64":"1911"}},{"literal":{"i64":"1912"}},{"literal":{"i64":"1913"}},{"literal":{"i64":"1914"}},{"literal":{"i64":"1915"}},{"literal":{"i64":"1916"}},{"literal":{"i64":"1917"}},{"literal":{"i64":"1918"}},{"literal":{"i64":"1919"}},{"literal":{"i64":"192"}},{"literal":{"i64":"1920"}},{"literal":{"i64":"1921"}},{"literal":{"i64":"1922"}},{"literal":{"i64":"1923"}},{"literal":{"i64":"1924"}},{"literal":{"i64":"1925"}},{"literal":{"i64":"1926"}},{"literal":{"i64":"1927"}},{"literal":{"i64":"1928"}},{"literal":{"i64":"1929"}},{"literal":{"i64":"193"}},{"literal":{"i64":"1930"}},{"literal":{"i64":"1931"}},{"literal":{"i64":"1932"}},{"literal":{"i64":"1933"}},{"literal":{"i64":"1934"}},{"literal":{"i64":"1935"}},{"literal":{"i64":"1936"}},{"literal":{"i64":"1937"}},{"literal":{"i64":"1938"}},{"literal":{"i64":"1939"}},{"literal":{"i64":"194"}},{"literal":{"i64":"1940"}},{"literal":{"i64":"1941"}},{"literal":{"i64":"1942"}},{"literal":{"i64":"1943"}},{"literal":{"i64":"1944"}},{"literal":{"i64":"1945"}},{"literal":{"i64":"1946"}},{"literal":{"i64":"1947"}},{"literal":{"i64":"1948"}},{"literal":{"i64":"1949"}},{"literal":{"i64":"195"}},{"literal":{"i64":"1950"}},{"literal":{"i64":"1951"}},{"literal":{"i64":"1952"}},{"literal":{"i64":"1953"}},{"literal":{"i64":"1954"}},{"literal":{"i64":"1955"}},{"literal":{"i64":"1956"}},{"literal":{"i64":"1957"}},{"literal":{"i64":"1958"}},{"literal":{"i64":"1959"}},{"literal":{"i64":"196"}},{"literal":{"i64":"1960"}},{"literal":{"i64":"1961"}},{"literal":{"i64":"1962"}},{"literal":{"i64":"1963"}},{"literal":{"i64":"1964"}},{"literal":{"i64":"1965"}},{"literal":{"i64":"1966"}},{"literal":{"i64":"1967"}},{"literal":{"i64":"1968"}},{"literal":{"i64":"1969"}},{"literal":{"i64":"197"}},{"literal":{"i64":"1970"}},{"literal":{"i64":"1971"}},{"literal":{"i64":"1972"}},{"literal":{"i64":"1973"}},{"literal":{"i64":"1974"}},{"literal":{"i64":"1975"}},{"literal":{"i64":"1976"}},{"literal":{"i64":"1977"}},{"literal":{"i64":"1978"}},{"literal":{"i64":"1979"}},{"literal":{"i64":"198"}},{"literal":{"i64":"1980"}},{"literal":{"i64":"1981"}},{"literal":{"i64":"1982"}},{"literal":{"i64":"1983"}},{"literal":{"i64":"1984"}},{"literal":{"i64":"1985"}},{"literal":{"i64":"1986"}},{"literal":{"i64":"1987"}},{"literal":{"i64":"1988"}},{"literal":{"i64":"1989"}},{"literal":{"i64":"199"}},{"literal":{"i64":"1990"}},{"literal":{"i64":"1991"}},{"literal":{"i64":"1992"}},{"literal":{"i64":"1993"}},{"literal":{"i64":"1994"}},{"literal":{"i64":"1995"}},{"literal":{"i64":"1996"}},{"literal":{"i64":"1997"}},{"literal":{"i64":"1998"}},{"literal":{"i64":"1999"}},{"literal":{"i64":"2"}},{"literal":{"i64":"20"}},{"literal":{"i64":"200"}},{"literal":{"i64":"2000"}},{"literal":{"i64":"2001"}},{"literal":{"i64":"2002"}},{"literal":{"i64":"2003"}},{"literal":{"i64":"2004"}},{"literal":{"i64":"2005"}},{"literal":{"i64":"2006"}},{"literal":{"i64":"2007"}},{"literal":{"i64":"2008"}},{"literal":{"i64":"2009"}},{"literal":{"i64":"201"}},{"literal":{"i64":"2010"}},{"literal":{"i64":"2011"}},{"literal":{"i64":"2012"}},{"literal":{"i64":"2013"}},{"literal":{"i64":"2014"}},{"literal":{"i64":"2015"}},{"literal":{"i64":"2016"}},{"literal":{"i64":"2017"}},{"literal":{"i64":"2018"}},{"literal":{"i64":"2019"}},{"literal":{"i64":"202"}},{"literal":{"i64":"2020"}},{"literal":{"i64":"2021"}},{"literal":{"i64":"2022"}},{"literal":{"i64":"2023"}},{"literal":{"i64":"2024"}},{"literal":{"i64":"2025"}},{"literal":{"i64":"2026"}},{"literal":{"i64":"2027"}},{"literal":{"i64":"2028"}},{"literal":{"i64":"2029"}},{"literal":{"i64":"203"}},{"literal":{"i64":"2030"}},{"literal":{"i64":"2031"}},{"literal":{"i64":"2032"}},{"literal":{"i64":"2033"}},{"literal":{"i64":"2034"}},{"literal":{"i64":"2035"}},{"literal":{"i64":"2036"}},{"literal":{"i64":"2037"}},{"literal":{"i64":"2038"}},{"literal":{"i64":"2039"}},{"literal":{"i64":"204"}},{"literal":{"i64":"2040"}},{"literal":{"i64":"2041"}},{"literal":{"i64":"2042"}},{"literal":{"i64":"2043"}},{"literal":{"i64":"2044"}},{"literal":{"i64":"2045"}},{"literal":{"i64":"2046"}},{"literal":{"i64":"2047"}},{"literal":{"i64":"2048"}},{"literal":{"i64":"2049"}},{"literal":{"i64":"205"}},{"literal":{"i64":"2050"}},{"literal":{"i64":"2051"}},{"literal":{"i64":"2052"}},{"literal":{"i64":"2053"}},{"literal":{"i64":"2054"}},{"literal":{"i64":"2055"}},{"literal":{"i64":"2056"}},{"literal":{"i64":"2057"}},{"literal":{"i64":"2058"}},{"literal":{"i64":"2059"}},{"literal":{"i64":"206"}},{"literal":{"i64":"2060"}},{"literal":{"i64":"2061"}},{"literal":{"i64":"2062"}},{"literal":{"i64":"2063"}},{"literal":{"i64":"2064"}},{"literal":{"i64":"2065"}},{"literal":{"i64":"2066"}},{"literal":{"i64":"2067"}},{"literal":{"i64":"2068"}},{"literal":{"i64":"2069"}},{"literal":{"i64":"207"}},{"literal":{"i64":"2070"}},{"literal":{"i64":"2071"}},{"literal":{"i64":"2072"}},{"literal":{"i64":"2073"}},{"literal":{"i64":"2074"}},{"literal":{"i64":"2075"}},{"literal":{"i64":"2076"}},{"literal":{"i64":"2077"}},{"literal":{"i64":"2078"}},{"literal":{"i64":"2079"}},{"literal":{"i64":"208"}},{"literal":{"i64":"2080"}},{"literal":{"i64":"2081"}},{"literal":{"i64":"2082"}},{"literal":{"i64":"2083"}},{"literal":{"i64":"2084"}},{"literal":{"i64":"2085"}},{"literal":{"i64":"2086"}},{"literal":{"i64":"2087"}},{"literal":{"i64":"2088"}},{"literal":{"i64":"2089"}},{"literal":{"i64":"209"}},{"literal":{"i64":"2090"}},{"literal":{"i64":"2091"}},{"literal":{"i64":"2092"}},{"literal":{"i64":"2093"}},{"literal":{"i64":"2094"}},{"literal":{"i64":"2095"}},{"literal":{"i64":"2096"}},{"literal":{"i64":"2097"}},{"literal":{"i64":"2098"}},{"literal":{"i64":"2099"}},{"literal":{"i64":"21"}},{"literal":{"i64":"210"}},{"literal":{"i64":"2100"}},{"literal":{"i64":"2101"}},{"literal":{"i64":"2102"}},{"literal":{"i64":"2103"}},{"literal":{"i64":"2104"}},{"literal":{"i64":"2105"}},{"literal":{"i64":"2106"}},{"literal":{"i64":"2107"}},{"literal":{"i64":"2108"}},{"literal":{"i64":"2109"}},{"literal":{"i64":"211"}},{"literal":{"i64":"2110"}},{"literal":{"i64":"2111"}},{"literal":{"i64":"2112"}},{"literal":{"i64":"2113"}},{"literal":{"i64":"2114"}},{"literal":{"i64":"2115"}},{"literal":{"i64":"2116"}},{"literal":{"i64":"2117"}},{"literal":{"i64":"2118"}},{"literal":{"i64":"2119"}},{"literal":{"i64":"212"}},{"literal":{"i64":"2120"}},{"literal":{"i64":"2121"}},{"literal":{"i64":"2122"}},{"literal":{"i64":"2123"}},{"literal":{"i64":"2124"}},{"literal":{"i64":"2125"}},{"literal":{"i64":"2126"}},{"literal":{"i64":"2127"}},{"literal":{"i64":"2128"}},{"literal":{"i64":"2129"}},{"literal":{"i64":"213"}},{"literal":{"i64":"2130"}},{"literal":{"i64":"2131"}},{"literal":{"i64":"2132"}},{"literal":{"i64":"2133"}},{"literal":{"i64":"2134"}},{"literal":{"i64":"2135"}},{"literal":{"i64":"2136"}},{"literal":{"i64":"2137"}},{"literal":{"i64":"2138"}},{"literal":{"i64":"2139"}},{"literal":{"i64":"214"}},{"literal":{"i64":"2140"}},{"literal":{"i64":"2141"}},{"literal":{"i64":"2142"}},{"literal":{"i64":"2143"}},{"literal":{"i64":"2144"}},{"literal":{"i64":"2145"}},{"literal":{"i64":"2146"}},{"literal":{"i64":"2147"}},{"literal":{"i64":"2148"}},{"literal":{"i64":"2149"}},{"literal":{"i64":"215"}},{"literal":{"i64":"2150"}},{"literal":{"i64":"2151"}},{"literal":{"i64":"2152"}},{"literal":{"i64":"2153"}},{"literal":{"i64":"2154"}},{"literal":{"i64":"2155"}},{"literal":{"i64":"2156"}},{"literal":{"i64":"2157"}},{"literal":{"i64":"2158"}},{"literal":{"i64":"2159"}},{"literal":{"i64":"216"}},{"literal":{"i64":"2160"}},{"literal":{"i64":"2161"}},{"literal":{"i64":"2162"}},{"literal":{"i64":"2163"}},{"literal":{"i64":"2164"}},{"literal":{"i64":"2165"}},{"literal":{"i64":"2166"}},{"literal":{"i64":"2167"}},{"literal":{"i64":"2168"}},{"literal":{"i64":"2169"}},{"literal":{"i64":"217"}},{"literal":{"i64":"2170"}},{"literal":{"i64":"2171"}},{"literal":{"i64":"2172"}},{"literal":{"i64":"2173"}},{"literal":{"i64":"2174"}},{"literal":{"i64":"2175"}},{"literal":{"i64":"2176"}},{"literal":{"i64":"2177"}},{"literal":{"i64":"2178"}},{"literal":{"i64":"2179"}},{"literal":{"i64":"218"}},{"literal":{"i64":"2180"}},{"literal":{"i64":"2181"}},{"literal":{"i64":"2182"}},{"literal":{"i64":"2183"}},{"literal":{"i64":"2184"}},{"literal":{"i64":"2185"}},{"literal":{"i64":"2186"}},{"literal":{"i64":"2187"}},{"literal":{"i64":"2188"}},{"literal":{"i64":"2189"}},{"literal":{"i64":"219"}},{"literal":{"i64":"2190"}},{"literal":{"i64":"2191"}},{"literal":{"i64":"2192"}},{"literal":{"i64":"2193"}},{"literal":{"i64":"2194"}},{"literal":{"i64":"2195"}},{"literal":{"i64":"2196"}},{"literal":{"i64":"2197"}},{"literal":{"i64":"2198"}},{"literal":{"i64":"2199"}},{"literal":{"i64":"22"}},{"literal":{"i64":"220"}},{"literal":{"i64":"2200"}},{"literal":{"i64":"2201"}},{"literal":{"i64":"2202"}},{"literal":{"i64":"2203"}},{"literal":{"i64":"2204"}},{"literal":{"i64":"2205"}},{"literal":{"i64":"2206"}},{"literal":{"i64":"2207"}},{"literal":{"i64":"2208"}},{"literal":{"i64":"2209"}},{"literal":{"i64":"221"}},{"literal":{"i64":"2210"}},{"literal":{"i64":"2211"}},{"literal":{"i64":"2212"}},{"literal":{"i64":"2213"}},{"literal":{"i64":"2214"}},{"literal":{"i64":"2215"}},{"literal":{"i64":"2216"}},{"literal":{"i64":"2217"}},{"literal":{"i64":"2218"}},{"literal":{"i64":"2219"}},{"literal":{"i64":"222"}},{"literal":{"i64":"2220"}},{"literal":{"i64":"2221"}},{"literal":{"i64":"2222"}},{"literal":{"i64":"2223"}},{"literal":{"i64":"2224"}},{"literal":{"i64":"2225"}},{"literal":{"i64":"2226"}},{"literal":{"i64":"2227"}},{"literal":{"i64":"2228"}},{"literal":{"i64":"2229"}},{"literal":{"i64":"223"}},{"literal":{"i64":"2230"}},{"literal":{"i64":"2231"}},{"literal":{"i64":"2232"}},{"literal":{"i64":"2233"}},{"literal":{"i64":"2234"}},{"literal":{"i64":"2235"}},{"literal":{"i64":"2236"}},{"literal":{"i64":"2237"}},{"literal":{"i64":"2238"}},{"literal":{"i64":"2239"}},{"literal":{"i64":"224"}},{"literal":{"i64":"2240"}},{"literal":{"i64":"2241"}},{"literal":{"i64":"2242"}},{"literal":{"i64":"2243"}},{"literal":{"i64":"2244"}},{"literal":{"i64":"2245"}},{"literal":{"i64":"2246"}},{"literal":{"i64":"2247"}},{"literal":{"i64":"2248"}},{"literal":{"i64":"2249"}},{"literal":{"i64":"225"}},{"literal":{"i64":"2250"}},{"literal":{"i64":"2251"}},{"literal":{"i64":"2252"}},{"literal":{"i64":"2253"}},{"literal":{"i64":"2254"}},{"literal":{"i64":"2255"}},{"literal":{"i64":"2256"}},{"literal":{"i64":"2257"}},{"literal":{"i64":"2258"}},{"literal":{"i64":"2259"}},{"literal":{"i64":"226"}},{"literal":{"i64":"2260"}},{"literal":{"i64":"2261"}},{"literal":{"i64":"2262"}},{"literal":{"i64":"2263"}},{"literal":{"i64":"2264"}},{"literal":{"i64":"2265"}},{"literal":{"i64":"2266"}},{"literal":{"i64":"2267"}},{"literal":{"i64":"2268"}},{"literal":{"i64":"2269"}},{"literal":{"i64":"227"}},{"literal":{"i64":"2270"}},{"literal":{"i64":"2271"}},{"literal":{"i64":"2272"}},{"literal":{"i64":"2273"}},{"literal":{"i64":"2274"}},{"literal":{"i64":"2275"}},{"literal":{"i64":"2276"}},{"literal":{"i64":"2277"}},{"literal":{"i64":"2278"}},{"literal":{"i64":"2279"}},{"literal":{"i64":"228"}},{"literal":{"i64":"2280"}},{"literal":{"i64":"2281"}},{"literal":{"i64":"2282"}},{"literal":{"i64":"2283"}},{"literal":{"i64":"2284"}},{"literal":{"i64":"2285"}},{"literal":{"i64":"2286"}},{"literal":{"i64":"2287"}},{"literal":{"i64":"2288"}},{"literal":{"i64":"2289"}},{"literal":{"i64":"229"}},{"literal":{"i64":"2290"}},{"literal":{"i64":"2291"}},{"literal":{"i64":"2292"}},{"literal":{"i64":"2293"}},{"literal":{"i64":"2294"}},{"literal":{"i64":"2295"}},{"literal":{"i64":"2296"}},{"literal":{"i64":"2297"}},{"literal":{"i64":"2298"}},{"literal":{"i64":"2299"}},{"literal":{"i64":"23"}},{"literal":{"i64":"230"}},{"literal":{"i64":"2300"}},{"literal":{"i64":"2301"}},{"literal":{"i64":"2302"}},{"literal":{"i64":"2303"}},{"literal":{"i64":"2304"}},{"literal":{"i64":"2305"}},{"literal":{"i64":"2306"}},{"literal":{"i64":"2307"}},{"literal":{"i64":"2308"}},{"literal":{"i64":"2309"}},{"literal":{"i64":"231"}},{"literal":{"i64":"2310"}},{"literal":{"i64":"2311"}},{"literal":{"i64":"2312"}},{"literal":{"i64":"2313"}},{"literal":{"i64":"2314"}},{"literal":{"i64":"2315"}},{"literal":{"i64":"2316"}},{"literal":{"i64":"2317"}},{"literal":{"i64":"2318"}},{"literal":{"i64":"2319"}},{"literal":{"i64":"232"}},{"literal":{"i64":"2320"}},{"literal":{"i64":"2321"}},{"literal":{"i64":"2322"}},{"literal":{"i64":"2323"}},{"literal":{"i64":"2324"}},{"literal":{"i64":"2325"}},{"literal":{"i64":"2326"}},{"literal":{"i64":"2327"}},{"literal":{"i64":"2328"}},{"literal":{"i64":"2329"}},{"literal":{"i64":"233"}},{"literal":{"i64":"2330"}},{"literal":{"i64":"2331"}},{"literal":{"i64":"2332"}},{"literal":{"i64":"2333"}},{"literal":{"i64":"2334"}},{"literal":{"i64":"2335"}},{"literal":{"i64":"2336"}},{"literal":{"i64":"2337"}},{"literal":{"i64":"2338"}},{"literal":{"i64":"2339"}},{"literal":{"i64":"234"}},{"literal":{"i64":"2340"}},{"literal":{"i64":"2341"}},{"literal":{"i64":"2342"}},{"literal":{"i64":"2343"}},{"literal":{"i64":"2344"}},{"literal":{"i64":"2345"}},{"literal":{"i64":"2346"}},{"literal":{"i64":"2347"}},{"literal":{"i64":"2348"}},{"literal":{"i64":"2349"}},{"literal":{"i64":"235"}},{"literal":{"i64":"2350"}},{"literal":{"i64":"2351"}},{"literal":{"i64":"2352"}},{"literal":{"i64":"2353"}},{"literal":{"i64":"2354"}},{"literal":{"i64":"2355"}},{"literal":{"i64":"2356"}},{"literal":{"i64":"2357"}},{"literal":{"i64":"2358"}},{"literal":{"i64":"2359"}},{"literal":{"i64":"236"}},{"literal":{"i64":"2360"}},{"literal":{"i64":"2361"}},{"literal":{"i64":"2362"}},{"literal":{"i64":"2363"}},{"literal":{"i64":"2364"}},{"literal":{"i64":"2365"}},{"literal":{"i64":"2366"}},{"literal":{"i64":"2367"}},{"literal":{"i64":"2368"}},{"literal":{"i64":"2369"}},{"literal":{"i64":"237"}},{"literal":{"i64":"2370"}},{"literal":{"i64":"2371"}},{"literal":{"i64":"2372"}},{"literal":{"i64":"2373"}},{"literal":{"i64":"2374"}},{"literal":{"i64":"2375"}},{"literal":{"i64":"2376"}},{"literal":{"i64":"2377"}},{"literal":{"i64":"2378"}},{"literal":{"i64":"2379"}},{"literal":{"i64":"238"}},{"literal":{"i64":"2380"}},{"literal":{"i64":"2381"}},{"literal":{"i64":"2382"}},{"literal":{"i64":"2383"}},{"literal":{"i64":"2384"}},{"literal":{"i64":"2385"}},{"literal":{"i64":"2386"}},{"literal":{"i64":"2387"}},{"literal":{"i64":"2388"}},{"literal":{"i64":"2389"}},{"literal":{"i64":"239"}},{"literal":{"i64":"2390"}},{"literal":{"i64":"2391"}},{"literal":{"i64":"2392"}},{"literal":{"i64":"2393"}},{"literal":{"i64":"2394"}},{"literal":{"i64":"2395"}},{"literal":{"i64":"2396"}},{"literal":{"i64":"2397"}},{"literal":{"i64":"2398"}},{"literal":{"i64":"2399"}},{"literal":{"i64":"24"}},{"literal":{"i64":"240"}},{"literal":{"i64":"2400"}},{"literal":{"i64":"2401"}},{"literal":{"i64":"2402"}},{"literal":{"i64":"2403"}},{"literal":{"i64":"2404"}},{"literal":{"i64":"2405"}},{"literal":{"i64":"2406"}},{"literal":{"i64":"2407"}},{"literal":{"i64":"2408"}},{"literal":{"i64":"2409"}},{"literal":{"i64":"241"}},{"literal":{"i64":"2410"}},{"literal":{"i64":"2411"}},{"literal":{"i64":"2412"}},{"literal":{"i64":"2413"}},{"literal":{"i64":"2414"}},{"literal":{"i64":"2415"}},{"literal":{"i64":"2416"}},{"literal":{"i64":"2417"}},{"literal":{"i64":"2418"}},{"literal":{"i64":"2419"}},{"literal":{"i64":"242"}},{"literal":{"i64":"2420"}},{"literal":{"i64":"2421"}},{"literal":{"i64":"2422"}},{"literal":{"i64":"2423"}},{"literal":{"i64":"2424"}},{"literal":{"i64":"2425"}},{"literal":{"i64":"2426"}},{"literal":{"i64":"2427"}},{"literal":{"i64":"2428"}},{"literal":{"i64":"2429"}},{"literal":{"i64":"243"}},{"literal":{"i64":"2430"}},{"literal":{"i64":"2431"}},{"literal":{"i64":"2432"}},{"literal":{"i64":"2433"}},{"literal":{"i64":"2434"}},{"literal":{"i64":"2435"}},{"literal":{"i64":"2436"}},{"literal":{"i64":"2437"}},{"literal":{"i64":"2438"}},{"literal":{"i64":"2439"}},{"literal":{"i64":"244"}},{"literal":{"i64":"2440"}},{"literal":{"i64":"2441"}},{"literal":{"i64":"2442"}},{"literal":{"i64":"2443"}},{"literal":{"i64":"2444"}},{"literal":{"i64":"2445"}},{"literal":{"i64":"2446"}},{"literal":{"i64":"2447"}},{"literal":{"i64":"2448"}},{"literal":{"i64":"2449"}},{"literal":{"i64":"245"}},{"literal":{"i64":"2450"}},{"literal":{"i64":"2451"}},{"literal":{"i64":"2452"}},{"literal":{"i64":"2453"}},{"literal":{"i64":"2454"}},{"literal":{"i64":"2455"}},{"literal":{"i64":"2456"}},{"literal":{"i64":"2457"}},{"literal":{"i64":"2458"}},{"literal":{"i64":"2459"}},{"literal":{"i64":"246"}},{"literal":{"i64":"2460"}},{"literal":{"i64":"2461"}},{"literal":{"i64":"2462"}},{"literal":{"i64":"2463"}},{"literal":{"i64":"2464"}},{"literal":{"i64":"2465"}},{"literal":{"i64":"2466"}},{"literal":{"i64":"2467"}},{"literal":{"i64":"2468"}},{"literal":{"i64":"2469"}},{"literal":{"i64":"247"}},{"literal":{"i64":"2470"}},{"literal":{"i64":"2471"}},{"literal":{"i64":"2472"}},{"literal":{"i64":"2473"}},{"literal":{"i64":"2474"}},{"literal":{"i64":"2475"}},{"literal":{"i64":"2476"}},{"literal":{"i64":"2477"}},{"literal":{"i64":"2478"}},{"literal":{"i64":"2479"}},{"literal":{"i64":"248"}},{"literal":{"i64":"2480"}},{"literal":{"i64":"2481"}},{"literal":{"i64":"2482"}},{"literal":{"i64":"2483"}},{"literal":{"i64":"2484"}},{"literal":{"i64":"2485"}},{"literal":{"i64":"2486"}},{"literal":{"i64":"2487"}},{"literal":{"i64":"2488"}},{"literal":{"i64":"2489"}},{"literal":{"i64":"249"}},{"literal":{"i64":"2490"}},{"literal":{"i64":"2491"}},{"literal":{"i64":"2492"}},{"literal":{"i64":"2493"}},{"literal":{"i64":"2494"}},{"literal":{"i64":"2495"}},{"literal":{"i64":"2496"}},{"literal":{"i64":"2497"}},{"literal":{"i64":"2498"}},{"literal":{"i64":"2499"}},{"literal":{"i64":"25"}},{"literal":{"i64":"250"}},{"literal":{"i64":"2500"}},{"literal":{"i64":"2501"}},{"literal":{"i64":"2502"}},{"literal":{"i64":"2503"}},{"literal":{"i64":"2504"}},{"literal":{"i64":"2505"}},{"literal":{"i64":"2506"}},{"literal":{"i64":"2507"}},{"literal":{"i64":"2508"}},{"literal":{"i64":"2509"}},{"literal":{"i64":"251"}},{"literal":{"i64":"2510"}},{"literal":{"i64":"2511"}},{"literal":{"i64":"2512"}},{"literal":{"i64":"2513"}},{"literal":{"i64":"2514"}},{"literal":{"i64":"2515"}},{"literal":{"i64":"2516"}},{"literal":{"i64":"2517"}},{"literal":{"i64":"2518"}},{"literal":{"i64":"2519"}},{"literal":{"i64":"252"}},{"literal":{"i64":"2520"}},{"literal":{"i64":"2521"}},{"literal":{"i64":"2522"}},{"literal":{"i64":"2523"}},{"literal":{"i64":"2524"}},{"literal":{"i64":"2525"}},{"literal":{"i64":"2526"}},{"literal":{"i64":"2527"}},{"literal":{"i64":"2528"}},{"literal":{"i64":"2529"}},{"literal":{"i64":"253"}},{"literal":{"i64":"2530"}},{"literal":{"i64":"2531"}},{"literal":{"i64":"2532"}},{"literal":{"i64":"2533"}},{"literal":{"i64":"2534"}},{"literal":{"i64":"2535"}},{"literal":{"i64":"2536"}},{"literal":{"i64":"2537"}},{"literal":{"i64":"2538"}},{"literal":{"i64":"2539"}},{"literal":{"i64":"254"}},{"literal":{"i64":"2540"}},{"literal":{"i64":"2541"}},{"literal":{"i64":"2542"}},{"literal":{"i64":"2543"}},{"literal":{"i64":"2544"}},{"literal":{"i64":"2545"}},{"literal":{"i64":"2546"}},{"literal":{"i64":"2547"}},{"literal":{"i64":"2548"}},{"literal":{"i64":"2549"}},{"literal":{"i64":"255"}},{"literal":{"i64":"2550"}},{"literal":{"i64":"2551"}},{"literal":{"i64":"2552"}},{"literal":{"i64":"2553"}},{"literal":{"i64":"2554"}},{"literal":{"i64":"2555"}},{"literal":{"i64":"2556"}},{"literal":{"i64":"2557"}},{"literal":{"i64":"2558"}},{"literal":{"i64":"2559"}},{"literal":{"i64":"256"}},{"literal":{"i64":"2560"}},{"literal":{"i64":"2561"}},{"literal":{"i64":"2562"}},{"literal":{"i64":"2563"}},{"literal":{"i64":"2564"}},{"literal":{"i64":"2565"}},{"literal":{"i64":"2566"}},{"literal":{"i64":"2567"}},{"literal":{"i64":"2568"}},{"literal":{"i64":"2569"}},{"literal":{"i64":"257"}},{"literal":{"i64":"2570"}},{"literal":{"i64":"2571"}},{"literal":{"i64":"2572"}},{"literal":{"i64":"2573"}},{"literal":{"i64":"2574"}},{"literal":{"i64":"2575"}},{"literal":{"i64":"2576"}},{"literal":{"i64":"2577"}},{"literal":{"i64":"2578"}},{"literal":{"i64":"2579"}},{"literal":{"i64":"258"}},{"literal":{"i64":"2580"}},{"literal":{"i64":"2581"}},{"literal":{"i64":"2582"}},{"literal":{"i64":"2583"}},{"literal":{"i64":"2584"}},{"literal":{"i64":"2585"}},{"literal":{"i64":"2586"}},{"literal":{"i64":"2587"}},{"literal":{"i64":"2588"}},{"literal":{"i64":"2589"}},{"literal":{"i64":"259"}},{"literal":{"i64":"2590"}},{"literal":{"i64":"2591"}},{"literal":{"i64":"2592"}},{"literal":{"i64":"2593"}},{"literal":{"i64":"2594"}},{"literal":{"i64":"2595"}},{"literal":{"i64":"2596"}},{"literal":{"i64":"2597"}},{"literal":{"i64":"2598"}},{"literal":{"i64":"2599"}},{"literal":{"i64":"26"}},{"literal":{"i64":"260"}},{"literal":{"i64":"2600"}},{"literal":{"i64":"2601"}},{"literal":{"i64":"2602"}},{"literal":{"i64":"2603"}},{"literal":{"i64":"2604"}},{"literal":{"i64":"2605"}},{"literal":{"i64":"2606"}},{"literal":{"i64":"2607"}},{"literal":{"i64":"2608"}},{"literal":{"i64":"2609"}},{"literal":{"i64":"261"}},{"literal":{"i64":"2610"}},{"literal":{"i64":"2611"}},{"literal":{"i64":"2612"}},{"literal":{"i64":"2613"}},{"literal":{"i64":"2614"}},{"literal":{"i64":"2615"}},{"literal":{"i64":"2616"}},{"literal":{"i64":"2617"}},{"literal":{"i64":"2618"}},{"literal":{"i64":"2619"}},{"literal":{"i64":"262"}},{"literal":{"i64":"2620"}},{"literal":{"i64":"2621"}},{"literal":{"i64":"2622"}},{"literal":{"i64":"2623"}},{"literal":{"i64":"2624"}},{"literal":{"i64":"2625"}},{"literal":{"i64":"2626"}},{"literal":{"i64":"2627"}},{"literal":{"i64":"2628"}},{"literal":{"i64":"2629"}},{"literal":{"i64":"263"}},{"literal":{"i64":"2630"}},{"literal":{"i64":"2631"}},{"literal":{"i64":"2632"}},{"literal":{"i64":"2633"}},{"literal":{"i64":"2634"}},{"literal":{"i64":"2635"}},{"literal":{"i64":"2636"}},{"literal":{"i64":"2637"}},{"literal":{"i64":"2638"}},{"literal":{"i64":"2639"}},{"literal":{"i64":"264"}},{"literal":{"i64":"2640"}},{"literal":{"i64":"2641"}},{"literal":{"i64":"2642"}},{"literal":{"i64":"2643"}},{"literal":{"i64":"2644"}},{"literal":{"i64":"2645"}},{"literal":{"i64":"2646"}},{"literal":{"i64":"2647"}},{"literal":{"i64":"2648"}},{"literal":{"i64":"2649"}},{"literal":{"i64":"265"}},{"literal":{"i64":"2650"}},{"literal":{"i64":"2651"}},{"literal":{"i64":"2652"}},{"literal":{"i64":"2653"}},{"literal":{"i64":"2654"}},{"literal":{"i64":"2655"}},{"literal":{"i64":"2656"}},{"literal":{"i64":"2657"}},{"literal":{"i64":"2658"}},{"literal":{"i64":"2659"}},{"literal":{"i64":"266"}},{"literal":{"i64":"2660"}},{"literal":{"i64":"2661"}},{"literal":{"i64":"2662"}},{"literal":{"i64":"2663"}},{"literal":{"i64":"2664"}},{"literal":{"i64":"2665"}},{"literal":{"i64":"2666"}},{"literal":{"i64":"2667"}},{"literal":{"i64":"2668"}},{"literal":{"i64":"2669"}},{"literal":{"i64":"267"}},{"literal":{"i64":"2670"}},{"literal":{"i64":"2671"}},{"literal":{"i64":"2672"}},{"literal":{"i64":"2673"}},{"literal":{"i64":"2674"}},{"literal":{"i64":"2675"}},{"literal":{"i64":"2676"}},{"literal":{"i64":"2677"}},{"literal":{"i64":"2678"}},{"literal":{"i64":"2679"}},{"literal":{"i64":"268"}},{"literal":{"i64":"2680"}},{"literal":{"i64":"2681"}},{"literal":{"i64":"2682"}},{"literal":{"i64":"2683"}},{"literal":{"i64":"2684"}},{"literal":{"i64":"2685"}},{"literal":{"i64":"2686"}},{"literal":{"i64":"2687"}},{"literal":{"i64":"2688"}},{"literal":{"i64":"2689"}},{"literal":{"i64":"269"}},{"literal":{"i64":"2690"}},{"literal":{"i64":"2691"}},{"literal":{"i64":"2692"}},{"literal":{"i64":"2693"}},{"literal":{"i64":"2694"}},{"literal":{"i64":"2695"}},{"literal":{"i64":"2696"}},{"literal":{"i64":"2697"}},{"literal":{"i64":"2698"}},{"literal":{"i64":"2699"}},{"literal":{"i64":"27"}},{"literal":{"i64":"270"}},{"literal":{"i64":"2700"}},{"literal":{"i64":"2701"}},{"literal":{"i64":"2702"}},{"literal":{"i64":"2703"}},{"literal":{"i64":"2704"}},{"literal":{"i64":"2705"}},{"literal":{"i64":"2706"}},{"literal":{"i64":"2707"}},{"literal":{"i64":"2708"}},{"literal":{"i64":"2709"}},{"literal":{"i64":"271"}},{"literal":{"i64":"2710"}},{"literal":{"i64":"2711"}},{"literal":{"i64":"2712"}},{"literal":{"i64":"2713"}},{"literal":{"i64":"2714"}},{"literal":{"i64":"2715"}},{"literal":{"i64":"2716"}},{"literal":{"i64":"2717"}},{"literal":{"i64":"2718"}},{"literal":{"i64":"2719"}},{"literal":{"i64":"272"}},{"literal":{"i64":"2720"}},{"literal":{"i64":"2721"}},{"literal":{"i64":"2722"}},{"literal":{"i64":"2723"}},{"literal":{"i64":"2724"}},{"literal":{"i64":"2725"}},{"literal":{"i64":"2726"}},{"literal":{"i64":"2727"}},{"literal":{"i64":"2728"}},{"literal":{"i64":"2729"}},{"literal":{"i64":"273"}},{"literal":{"i64":"2730"}},{"literal":{"i64":"2731"}},{"literal":{"i64":"2732"}},{"literal":{"i64":"2733"}},{"literal":{"i64":"2734"}},{"literal":{"i64":"2735"}},{"literal":{"i64":"2736"}},{"literal":{"i64":"2737"}},{"literal":{"i64":"2738"}},{"literal":{"i64":"2739"}},{"literal":{"i64":"274"}},{"literal":{"i64":"2740"}},{"literal":{"i64":"2741"}},{"literal":{"i64":"2742"}},{"literal":{"i64":"2743"}},{"literal":{"i64":"2744"}},{"literal":{"i64":"2745"}},{"literal":{"i64":"2746"}},{"literal":{"i64":"2747"}},{"literal":{"i64":"2748"}},{"literal":{"i64":"2749"}},{"literal":{"i64":"275"}},{"literal":{"i64":"2750"}},{"literal":{"i64":"2751"}},{"literal":{"i64":"2752"}},{"literal":{"i64":"2753"}},{"literal":{"i64":"2754"}},{"literal":{"i64":"2755"}},{"literal":{"i64":"2756"}},{"literal":{"i64":"2757"}},{"literal":{"i64":"2758"}},{"literal":{"i64":"2759"}},{"literal":{"i64":"276"}},{"literal":{"i64":"2760"}},{"literal":{"i64":"2761"}},{"literal":{"i64":"2762"}},{"literal":{"i64":"2763"}},{"literal":{"i64":"2764"}},{"literal":{"i64":"2765"}},{"literal":{"i64":"2766"}},{"literal":{"i64":"2767"}},{"literal":{"i64":"2768"}},{"literal":{"i64":"2769"}},{"literal":{"i64":"277"}},{"literal":{"i64":"2770"}},{"literal":{"i64":"2771"}},{"literal":{"i64":"2772"}},{"literal":{"i64":"2773"}},{"literal":{"i64":"2774"}},{"literal":{"i64":"2775"}},{"literal":{"i64":"2776"}},{"literal":{"i64":"2777"}},{"literal":{"i64":"2778"}},{"literal":{"i64":"2779"}},{"literal":{"i64":"278"}},{"literal":{"i64":"2780"}},{"literal":{"i64":"2781"}},{"literal":{"i64":"2782"}},{"literal":{"i64":"2783"}},{"literal":{"i64":"2784"}},{"literal":{"i64":"2785"}},{"literal":{"i64":"2786"}},{"literal":{"i64":"2787"}},{"literal":{"i64":"2788"}},{"literal":{"i64":"2789"}},{"literal":{"i64":"279"}},{"literal":{"i64":"2790"}},{"literal":{"i64":"2791"}},{"literal":{"i64":"2792"}},{"literal":{"i64":"2793"}},{"literal":{"i64":"2794"}},{"literal":{"i64":"2795"}},{"literal":{"i64":"2796"}},{"literal":{"i64":"2797"}},{"literal":{"i64":"2798"}},{"literal":{"i64":"2799"}},{"literal":{"i64":"28"}},{"literal":{"i64":"280"}},{"literal":{"i64":"2800"}},{"literal":{"i64":"2801"}},{"literal":{"i64":"2802"}},{"literal":{"i64":"2803"}},{"literal":{"i64":"2804"}},{"literal":{"i64":"2805"}},{"literal":{"i64":"2806"}},{"literal":{"i64":"2807"}},{"literal":{"i64":"2808"}},{"literal":{"i64":"2809"}},{"literal":{"i64":"281"}},{"literal":{"i64":"2810"}},{"literal":{"i64":"2811"}},{"literal":{"i64":"2812"}},{"literal":{"i64":"2813"}},{"literal":{"i64":"2814"}},{"literal":{"i64":"2815"}},{"literal":{"i64":"2816"}},{"literal":{"i64":"2817"}},{"literal":{"i64":"2818"}},{"literal":{"i64":"2819"}},{"literal":{"i64":"282"}},{"literal":{"i64":"2820"}},{"literal":{"i64":"2821"}},{"literal":{"i64":"2822"}},{"literal":{"i64":"2823"}},{"literal":{"i64":"2824"}},{"literal":{"i64":"2825"}},{"literal":{"i64":"2826"}},{"literal":{"i64":"2827"}},{"literal":{"i64":"2828"}},{"literal":{"i64":"2829"}},{"literal":{"i64":"283"}},{"literal":{"i64":"2830"}},{"literal":{"i64":"2831"}},{"literal":{"i64":"2832"}},{"literal":{"i64":"2833"}},{"literal":{"i64":"2834"}},{"literal":{"i64":"2835"}},{"literal":{"i64":"2836"}},{"literal":{"i64":"2837"}},{"literal":{"i64":"2838"}},{"literal":{"i64":"2839"}},{"literal":{"i64":"284"}},{"literal":{"i64":"2840"}},{"literal":{"i64":"2841"}},{"literal":{"i64":"2842"}},{"literal":{"i64":"2843"}},{"literal":{"i64":"2844"}},{"literal":{"i64":"2845"}},{"literal":{"i64":"2846"}},{"literal":{"i64":"2847"}},{"literal":{"i64":"2848"}},{"literal":{"i64":"2849"}},{"literal":{"i64":"285"}},{"literal":{"i64":"2850"}},{"literal":{"i64":"2851"}},{"literal":{"i64":"2852"}},{"literal":{"i64":"2853"}},{"literal":{"i64":"2854"}},{"literal":{"i64":"2855"}},{"literal":{"i64":"2856"}},{"literal":{"i64":"2857"}},{"literal":{"i64":"2858"}},{"literal":{"i64":"2859"}},{"literal":{"i64":"286"}},{"literal":{"i64":"2860"}},{"literal":{"i64":"2861"}},{"literal":{"i64":"2862"}},{"literal":{"i64":"2863"}},{"literal":{"i64":"2864"}},{"literal":{"i64":"2865"}},{"literal":{"i64":"2866"}},{"literal":{"i64":"2867"}},{"literal":{"i64":"2868"}},{"literal":{"i64":"2869"}},{"literal":{"i64":"287"}},{"literal":{"i64":"2870"}},{"literal":{"i64":"2871"}},{"literal":{"i64":"2872"}},{"literal":{"i64":"2873"}},{"literal":{"i64":"2874"}},{"literal":{"i64":"2875"}},{"literal":{"i64":"2876"}},{"literal":{"i64":"2877"}},{"literal":{"i64":"2878"}},{"literal":{"i64":"2879"}},{"literal":{"i64":"288"}},{"literal":{"i64":"2880"}},{"literal":{"i64":"2881"}},{"literal":{"i64":"2882"}},{"literal":{"i64":"2883"}},{"literal":{"i64":"2884"}},{"literal":{"i64":"2885"}},{"literal":{"i64":"2886"}},{"literal":{"i64":"2887"}},{"literal":{"i64":"2888"}},{"literal":{"i64":"2889"}},{"literal":{"i64":"289"}},{"literal":{"i64":"2890"}},{"literal":{"i64":"2891"}},{"literal":{"i64":"2892"}},{"literal":{"i64":"2893"}},{"literal":{"i64":"2894"}},{"literal":{"i64":"2895"}},{"literal":{"i64":"2896"}},{"literal":{"i64":"2897"}},{"literal":{"i64":"2898"}},{"literal":{"i64":"2899"}},{"literal":{"i64":"29"}},{"literal":{"i64":"290"}},{"literal":{"i64":"2900"}},{"literal":{"i64":"2901"}},{"literal":{"i64":"2902"}},{"literal":{"i64":"2903"}},{"literal":{"i64":"2904"}},{"literal":{"i64":"2905"}},{"literal":{"i64":"2906"}},{"literal":{"i64":"2907"}},{"literal":{"i64":"2908"}},{"literal":{"i64":"2909"}},{"literal":{"i64":"291"}},{"literal":{"i64":"2910"}},{"literal":{"i64":"2911"}},{"literal":{"i64":"2912"}},{"literal":{"i64":"2913"}},{"literal":{"i64":"2914"}},{"literal":{"i64":"2915"}},{"literal":{"i64":"2916"}},{"literal":{"i64":"2917"}},{"literal":{"i64":"2918"}},{"literal":{"i64":"2919"}},{"literal":{"i64":"292"}},{"literal":{"i64":"2920"}},{"literal":{"i64":"2921"}},{"literal":{"i64":"2922"}},{"literal":{"i64":"2923"}},{"literal":{"i64":"2924"}},{"literal":{"i64":"2925"}},{"literal":{"i64":"2926"}},{"literal":{"i64":"2927"}},{"literal":{"i64":"2928"}},{"literal":{"i64":"2929"}},{"literal":{"i64":"293"}},{"literal":{"i64":"2930"}},{"literal":{"i64":"2931"}},{"literal":{"i64":"2932"}},{"literal":{"i64":"2933"}},{"literal":{"i64":"2934"}},{"literal":{"i64":"2935"}},{"literal":{"i64":"2936"}},{"literal":{"i64":"2937"}},{"literal":{"i64":"2938"}},{"literal":{"i64":"2939"}},{"literal":{"i64":"294"}},{"literal":{"i64":"2940"}},{"literal":{"i64":"2941"}},{"literal":{"i64":"2942"}},{"literal":{"i64":"2943"}},{"literal":{"i64":"2944"}},{"literal":{"i64":"2945"}},{"literal":{"i64":"2946"}},{"literal":{"i64":"2947"}},{"literal":{"i64":"2948"}},{"literal":{"i64":"2949"}},{"literal":{"i64":"295"}},{"literal":{"i64":"2950"}},{"literal":{"i64":"2951"}},{"literal":{"i64":"2952"}},{"literal":{"i64":"2953"}},{"literal":{"i64":"2954"}},{"literal":{"i64":"2955"}},{"literal":{"i64":"2956"}},{"literal":{"i64":"2957"}},{"literal":{"i64":"2958"}},{"literal":{"i64":"2959"}},{"literal":{"i64":"296"}},{"literal":{"i64":"2960"}},{"literal":{"i64":"2961"}},{"literal":{"i64":"2962"}},{"literal":{"i64":"2963"}},{"literal":{"i64":"2964"}},{"literal":{"i64":"2965"}},{"literal":{"i64":"2966"}},{"literal":{"i64":"2967"}},{"literal":{"i64":"2968"}},{"literal":{"i64":"2969"}},{"literal":{"i64":"297"}},{"literal":{"i64":"2970"}},{"literal":{"i64":"2971"}},{"literal":{"i64":"2972"}},{"literal":{"i64":"2973"}},{"literal":{"i64":"2974"}},{"literal":{"i64":"2975"}},{"literal":{"i64":"2976"}},{"literal":{"i64":"2977"}},{"literal":{"i64":"2978"}},{"literal":{"i64":"2979"}},{"literal":{"i64":"298"}},{"literal":{"i64":"2980"}},{"literal":{"i64":"2981"}},{"literal":{"i64":"2982"}},{"literal":{"i64":"2983"}},{"literal":{"i64":"2984"}},{"literal":{"i64":"2985"}},{"literal":{"i64":"2986"}},{"literal":{"i64":"2987"}},{"literal":{"i64":"2988"}},{"literal":{"i64":"2989"}},{"literal":{"i64":"299"}},{"literal":{"i64":"2990"}},{"literal":{"i64":"2991"}},{"literal":{"i64":"2992"}},{"literal":{"i64":"2993"}},{"literal":{"i64":"2994"}},{"literal":{"i64":"2995"}},{"literal":{"i64":"2996"}},{"literal":{"i64":"2997"}},{"literal":{"i64":"2998"}},{"literal":{"i64":"2999"}},{"literal":{"i64":"3"}},{"literal":{"i64":"30"}},{"literal":{"i64":"300"}},{"literal":{"i64":"3000"}},{"literal":{"i64":"3001"}},{"literal":{"i64":"3002"}},{"literal":{"i64":"3003"}},{"literal":{"i64":"3004"}},{"literal":{"i64":"3005"}},{"literal":{"i64":"3006"}},{"literal":{"i64":"3007"}},{"literal":{"i64":"3008"}},{"literal":{"i64":"3009"}},{"literal":{"i64":"301"}},{"literal":{"i64":"3010"}},{"literal":{"i64":"3011"}},{"literal":{"i64":"3012"}},{"literal":{"i64":"3013"}},{"literal":{"i64":"3014"}},{"literal":{"i64":"3015"}},{"literal":{"i64":"3016"}},{"literal":{"i64":"3017"}},{"literal":{"i64":"3018"}},{"literal":{"i64":"3019"}},{"literal":{"i64":"302"}},{"literal":{"i64":"3020"}},{"literal":{"i64":"3021"}},{"literal":{"i64":"3022"}},{"literal":{"i64":"3023"}},{"literal":{"i64":"3024"}},{"literal":{"i64":"3025"}},{"literal":{"i64":"3026"}},{"literal":{"i64":"3027"}},{"literal":{"i64":"3028"}},{"literal":{"i64":"3029"}},{"literal":{"i64":"303"}},{"literal":{"i64":"3030"}},{"literal":{"i64":"3031"}},{"literal":{"i64":"3032"}},{"literal":{"i64":"3033"}},{"literal":{"i64":"3034"}},{"literal":{"i64":"3035"}},{"literal":{"i64":"3036"}},{"literal":{"i64":"3037"}},{"literal":{"i64":"3038"}},{"literal":{"i64":"3039"}},{"literal":{"i64":"304"}},{"literal":{"i64":"3040"}},{"literal":{"i64":"3041"}},{"literal":{"i64":"3042"}},{"literal":{"i64":"3043"}},{"literal":{"i64":"3044"}},{"literal":{"i64":"3045"}},{"literal":{"i64":"3046"}},{"literal":{"i64":"3047"}},{"literal":{"i64":"3048"}},{"literal":{"i64":"3049"}},{"literal":{"i64":"305"}},{"literal":{"i64":"3050"}},{"literal":{"i64":"3051"}},{"literal":{"i64":"3052"}},{"literal":{"i64":"3053"}},{"literal":{"i64":"3054"}},{"literal":{"i64":"3055"}},{"literal":{"i64":"3056"}},{"literal":{"i64":"3057"}},{"literal":{"i64":"3058"}},{"literal":{"i64":"3059"}},{"literal":{"i64":"306"}},{"literal":{"i64":"3060"}},{"literal":{"i64":"3061"}},{"literal":{"i64":"3062"}},{"literal":{"i64":"3063"}},{"literal":{"i64":"3064"}},{"literal":{"i64":"3065"}},{"literal":{"i64":"3066"}},{"literal":{"i64":"3067"}},{"literal":{"i64":"3068"}},{"literal":{"i64":"3069"}},{"literal":{"i64":"307"}},{"literal":{"i64":"3070"}},{"literal":{"i64":"3071"}},{"literal":{"i64":"3072"}},{"literal":{"i64":"3073"}},{"literal":{"i64":"3074"}},{"literal":{"i64":"3075"}},{"literal":{"i64":"3076"}},{"literal":{"i64":"3077"}},{"literal":{"i64":"3078"}},{"literal":{"i64":"3079"}},{"literal":{"i64":"308"}},{"literal":{"i64":"3080"}},{"literal":{"i64":"3081"}},{"literal":{"i64":"3082"}},{"literal":{"i64":"3083"}},{"literal":{"i64":"3084"}},{"literal":{"i64":"3085"}},{"literal":{"i64":"3086"}},{"literal":{"i64":"3087"}},{"literal":{"i64":"3088"}},{"literal":{"i64":"3089"}},{"literal":{"i64":"309"}},{"literal":{"i64":"3090"}},{"literal":{"i64":"3091"}},{"literal":{"i64":"3092"}},{"literal":{"i64":"3093"}},{"literal":{"i64":"3094"}},{"literal":{"i64":"3095"}},{"literal":{"i64":"3096"}},{"literal":{"i64":"3097"}},{"literal":{"i64":"3098"}},{"literal":{"i64":"3099"}},{"literal":{"i64":"31"}},{"literal":{"i64":"310"}},{"literal":{"i64":"3100"}},{"literal":{"i64":"3101"}},{"literal":{"i64":"3102"}},{"literal":{"i64":"3103"}},{"literal":{"i64":"3104"}},{"literal":{"i64":"3105"}},{"literal":{"i64":"3106"}},{"literal":{"i64":"3107"}},{"literal":{"i64":"3108"}},{"literal":{"i64":"3109"}},{"literal":{"i64":"311"}},{"literal":{"i64":"3110"}},{"literal":{"i64":"3111"}},{"literal":{"i64":"3112"}},{"literal":{"i64":"3113"}},{"literal":{"i64":"3114"}},{"literal":{"i64":"3115"}},{"literal":{"i64":"3116"}},{"literal":{"i64":"3117"}},{"literal":{"i64":"3118"}},{"literal":{"i64":"3119"}},{"literal":{"i64":"312"}},{"literal":{"i64":"3120"}},{"literal":{"i64":"3121"}},{"literal":{"i64":"3122"}},{"literal":{"i64":"3123"}},{"literal":{"i64":"3124"}},{"literal":{"i64":"3125"}},{"literal":{"i64":"3126"}},{"literal":{"i64":"3127"}},{"literal":{"i64":"3128"}},{"literal":{"i64":"3129"}},{"literal":{"i64":"313"}},{"literal":{"i64":"3130"}},{"literal":{"i64":"3131"}},{"literal":{"i64":"3132"}},{"literal":{"i64":"3133"}},{"literal":{"i64":"3134"}},{"literal":{"i64":"3135"}},{"literal":{"i64":"3136"}},{"literal":{"i64":"3137"}},{"literal":{"i64":"3138"}},{"literal":{"i64":"3139"}},{"literal":{"i64":"314"}},{"literal":{"i64":"3140"}},{"literal":{"i64":"3141"}},{"literal":{"i64":"3142"}},{"literal":{"i64":"3143"}},{"literal":{"i64":"3144"}},{"literal":{"i64":"3145"}},{"literal":{"i64":"3146"}},{"literal":{"i64":"3147"}},{"literal":{"i64":"3148"}},{"literal":{"i64":"3149"}},{"literal":{"i64":"315"}},{"literal":{"i64":"3150"}},{"literal":{"i64":"3151"}},{"literal":{"i64":"3152"}},{"literal":{"i64":"3153"}},{"literal":{"i64":"3154"}},{"literal":{"i64":"3155"}},{"literal":{"i64":"3156"}},{"literal":{"i64":"3157"}},{"literal":{"i64":"3158"}},{"literal":{"i64":"3159"}},{"literal":{"i64":"316"}},{"literal":{"i64":"3160"}},{"literal":{"i64":"3161"}},{"literal":{"i64":"3162"}},{"literal":{"i64":"3163"}},{"literal":{"i64":"3164"}},{"literal":{"i64":"3165"}},{"literal":{"i64":"3166"}},{"literal":{"i64":"3167"}},{"literal":{"i64":"3168"}},{"literal":{"i64":"3169"}},{"literal":{"i64":"317"}},{"literal":{"i64":"3170"}},{"literal":{"i64":"3171"}},{"literal":{"i64":"3172"}},{"literal":{"i64":"3173"}},{"literal":{"i64":"3174"}},{"literal":{"i64":"3175"}},{"literal":{"i64":"3176"}},{"literal":{"i64":"3177"}},{"literal":{"i64":"3178"}},{"literal":{"i64":"3179"}},{"literal":{"i64":"318"}},{"literal":{"i64":"3180"}},{"literal":{"i64":"3181"}},{"literal":{"i64":"3182"}},{"literal":{"i64":"3183"}},{"literal":{"i64":"3184"}},{"literal":{"i64":"3185"}},{"literal":{"i64":"3186"}},{"literal":{"i64":"3187"}},{"literal":{"i64":"3188"}},{"literal":{"i64":"3189"}},{"literal":{"i64":"319"}},{"literal":{"i64":"3190"}},{"literal":{"i64":"3191"}},{"literal":{"i64":"3192"}},{"literal":{"i64":"3193"}},{"literal":{"i64":"3194"}},{"literal":{"i64":"3195"}},{"literal":{"i64":"3196"}},{"literal":{"i64":"3197"}},{"literal":{"i64":"3198"}},{"literal":{"i64":"3199"}},{"literal":{"i64":"32"}},{"literal":{"i64":"320"}},{"literal":{"i64":"3200"}},{"literal":{"i64":"3201"}},{"literal":{"i64":"3202"}},{"literal":{"i64":"3203"}},{"literal":{"i64":"3204"}},{"literal":{"i64":"3205"}},{"literal":{"i64":"3206"}},{"literal":{"i64":"3207"}},{"literal":{"i64":"3208"}},{"literal":{"i64":"3209"}},{"literal":{"i64":"321"}},{"literal":{"i64":"3210"}},{"literal":{"i64":"3211"}},{"literal":{"i64":"3212"}},{"literal":{"i64":"3213"}},{"literal":{"i64":"3214"}},{"literal":{"i64":"3215"}},{"literal":{"i64":"3216"}},{"literal":{"i64":"3217"}},{"literal":{"i64":"3218"}},{"literal":{"i64":"3219"}},{"literal":{"i64":"322"}},{"literal":{"i64":"3220"}},{"literal":{"i64":"3221"}},{"literal":{"i64":"3222"}},{"literal":{"i64":"3223"}},{"literal":{"i64":"3224"}},{"literal":{"i64":"3225"}},{"literal":{"i64":"3226"}},{"literal":{"i64":"3227"}},{"literal":{"i64":"3228"}},{"literal":{"i64":"3229"}},{"literal":{"i64":"323"}},{"literal":{"i64":"3230"}},{"literal":{"i64":"3231"}},{"literal":{"i64":"3232"}},{"literal":{"i64":"3233"}},{"literal":{"i64":"3234"}},{"literal":{"i64":"3235"}},{"literal":{"i64":"3236"}},{"literal":{"i64":"3237"}},{"literal":{"i64":"3238"}},{"literal":{"i64":"3239"}},{"literal":{"i64":"324"}},{"literal":{"i64":"3240"}},{"literal":{"i64":"3241"}},{"literal":{"i64":"3242"}},{"literal":{"i64":"3243"}},{"literal":{"i64":"3244"}},{"literal":{"i64":"3245"}},{"literal":{"i64":"3246"}},{"literal":{"i64":"3247"}},{"literal":{"i64":"3248"}},{"literal":{"i64":"3249"}},{"literal":{"i64":"325"}},{"literal":{"i64":"3250"}},{"literal":{"i64":"3251"}},{"literal":{"i64":"3252"}},{"literal":{"i64":"3253"}},{"literal":{"i64":"3254"}},{"literal":{"i64":"3255"}},{"literal":{"i64":"3256"}},{"literal":{"i64":"3257"}},{"literal":{"i64":"3258"}},{"literal":{"i64":"3259"}},{"literal":{"i64":"326"}},{"literal":{"i64":"3260"}},{"literal":{"i64":"3261"}},{"literal":{"i64":"3262"}},{"literal":{"i64":"3263"}},{"literal":{"i64":"3264"}},{"literal":{"i64":"3265"}},{"literal":{"i64":"3266"}},{"literal":{"i64":"3267"}},{"literal":{"i64":"3268"}},{"literal":{"i64":"3269"}},{"literal":{"i64":"327"}},{"literal":{"i64":"3270"}},{"literal":{"i64":"3271"}},{"literal":{"i64":"3272"}},{"literal":{"i64":"3273"}},{"literal":{"i64":"3274"}},{"literal":{"i64":"3275"}},{"literal":{"i64":"3276"}},{"literal":{"i64":"3277"}},{"literal":{"i64":"3278"}},{"literal":{"i64":"3279"}},{"literal":{"i64":"328"}},{"literal":{"i64":"3280"}},{"literal":{"i64":"3281"}},{"literal":{"i64":"3282"}},{"literal":{"i64":"3283"}},{"literal":{"i64":"3284"}},{"literal":{"i64":"3285"}},{"literal":{"i64":"3286"}},{"literal":{"i64":"3287"}},{"literal":{"i64":"3288"}},{"literal":{"i64":"3289"}},{"literal":{"i64":"329"}},{"literal":{"i64":"3290"}},{"literal":{"i64":"3291"}},{"literal":{"i64":"3292"}},{"literal":{"i64":"3293"}},{"literal":{"i64":"3294"}},{"literal":{"i64":"3295"}},{"literal":{"i64":"3296"}},{"literal":{"i64":"3297"}},{"literal":{"i64":"3298"}},{"literal":{"i64":"3299"}},{"literal":{"i64":"33"}},{"literal":{"i64":"330"}},{"literal":{"i64":"3300"}},{"literal":{"i64":"3301"}},{"literal":{"i64":"3302"}},{"literal":{"i64":"3303"}},{"literal":{"i64":"3304"}},{"literal":{"i64":"3305"}},{"literal":{"i64":"3306"}},{"literal":{"i64":"3307"}},{"literal":{"i64":"3308"}},{"literal":{"i64":"3309"}},{"literal":{"i64":"331"}},{"literal":{"i64":"3310"}},{"literal":{"i64":"3311"}},{"literal":{"i64":"3312"}},{"literal":{"i64":"3313"}},{"literal":{"i64":"3314"}},{"literal":{"i64":"3315"}},{"literal":{"i64":"3316"}},{"literal":{"i64":"3317"}},{"literal":{"i64":"3318"}},{"literal":{"i64":"3319"}},{"literal":{"i64":"332"}},{"literal":{"i64":"3320"}},{"literal":{"i64":"3321"}},{"literal":{"i64":"3322"}},{"literal":{"i64":"3323"}},{"literal":{"i64":"3324"}},{"literal":{"i64":"3325"}},{"literal":{"i64":"3326"}},{"literal":{"i64":"3327"}},{"literal":{"i64":"3328"}},{"literal":{"i64":"3329"}},{"literal":{"i64":"333"}},{"literal":{"i64":"3330"}},{"literal":{"i64":"3331"}},{"literal":{"i64":"3332"}},{"literal":{"i64":"3333"}},{"literal":{"i64":"3334"}},{"literal":{"i64":"3335"}},{"literal":{"i64":"3336"}},{"literal":{"i64":"3337"}},{"literal":{"i64":"3338"}},{"literal":{"i64":"3339"}},{"literal":{"i64":"334"}},{"literal":{"i64":"3340"}},{"literal":{"i64":"3341"}},{"literal":{"i64":"3342"}},{"literal":{"i64":"3343"}},{"literal":{"i64":"3344"}},{"literal":{"i64":"3345"}},{"literal":{"i64":"3346"}},{"literal":{"i64":"3347"}},{"literal":{"i64":"3348"}},{"literal":{"i64":"3349"}},{"literal":{"i64":"335"}},{"literal":{"i64":"3350"}},{"literal":{"i64":"3351"}},{"literal":{"i64":"3352"}},{"literal":{"i64":"3353"}},{"literal":{"i64":"3354"}},{"literal":{"i64":"3355"}},{"literal":{"i64":"3356"}},{"literal":{"i64":"3357"}},{"literal":{"i64":"3358"}},{"literal":{"i64":"3359"}},{"literal":{"i64":"336"}},{"literal":{"i64":"3360"}},{"literal":{"i64":"3361"}},{"literal":{"i64":"3362"}},{"literal":{"i64":"3363"}},{"literal":{"i64":"3364"}},{"literal":{"i64":"3365"}},{"literal":{"i64":"3366"}},{"literal":{"i64":"3367"}},{"literal":{"i64":"3368"}},{"literal":{"i64":"3369"}},{"literal":{"i64":"337"}},{"literal":{"i64":"3370"}},{"literal":{"i64":"3371"}},{"literal":{"i64":"3372"}},{"literal":{"i64":"3373"}},{"literal":{"i64":"3374"}},{"literal":{"i64":"3375"}},{"literal":{"i64":"3376"}},{"literal":{"i64":"3377"}},{"literal":{"i64":"3378"}},{"literal":{"i64":"3379"}},{"literal":{"i64":"338"}},{"literal":{"i64":"3380"}},{"literal":{"i64":"3381"}},{"literal":{"i64":"3382"}},{"literal":{"i64":"3383"}},{"literal":{"i64":"3384"}},{"literal":{"i64":"3385"}},{"literal":{"i64":"3386"}},{"literal":{"i64":"3387"}},{"literal":{"i64":"3388"}},{"literal":{"i64":"3389"}},{"literal":{"i64":"339"}},{"literal":{"i64":"3390"}},{"literal":{"i64":"3391"}},{"literal":{"i64":"3392"}},{"literal":{"i64":"3393"}},{"literal":{"i64":"3394"}},{"literal":{"i64":"3395"}},{"literal":{"i64":"3396"}},{"literal":{"i64":"3397"}},{"literal":{"i64":"3398"}},{"literal":{"i64":"3399"}},{"literal":{"i64":"34"}},{"literal":{"i64":"340"}},{"literal":{"i64":"3400"}},{"literal":{"i64":"3401"}},{"literal":{"i64":"3402"}},{"literal":{"i64":"3403"}},{"literal":{"i64":"3404"}},{"literal":{"i64":"3405"}},{"literal":{"i64":"3406"}},{"literal":{"i64":"3407"}},{"literal":{"i64":"3408"}},{"literal":{"i64":"3409"}},{"literal":{"i64":"341"}},{"literal":{"i64":"3410"}},{"literal":{"i64":"3411"}},{"literal":{"i64":"3412"}},{"literal":{"i64":"3413"}},{"literal":{"i64":"3414"}},{"literal":{"i64":"3415"}},{"literal":{"i64":"3416"}},{"literal":{"i64":"3417"}},{"literal":{"i64":"3418"}},{"literal":{"i64":"3419"}},{"literal":{"i64":"342"}},{"literal":{"i64":"3420"}},{"literal":{"i64":"3421"}},{"literal":{"i64":"3422"}},{"literal":{"i64":"3423"}},{"literal":{"i64":"3424"}},{"literal":{"i64":"3425"}},{"literal":{"i64":"3426"}},{"literal":{"i64":"3427"}},{"literal":{"i64":"3428"}},{"literal":{"i64":"3429"}},{"literal":{"i64":"343"}},{"literal":{"i64":"3430"}},{"literal":{"i64":"3431"}},{"literal":{"i64":"3432"}},{"literal":{"i64":"3433"}},{"literal":{"i64":"3434"}},{"literal":{"i64":"3435"}},{"literal":{"i64":"3436"}},{"literal":{"i64":"3437"}},{"literal":{"i64":"3438"}},{"literal":{"i64":"3439"}},{"literal":{"i64":"344"}},{"literal":{"i64":"3440"}},{"literal":{"i64":"3441"}},{"literal":{"i64":"3442"}},{"literal":{"i64":"3443"}},{"literal":{"i64":"3444"}},{"literal":{"i64":"3445"}},{"literal":{"i64":"3446"}},{"literal":{"i64":"3447"}},{"literal":{"i64":"3448"}},{"literal":{"i64":"3449"}},{"literal":{"i64":"345"}},{"literal":{"i64":"3450"}},{"literal":{"i64":"3451"}},{"literal":{"i64":"3452"}},{"literal":{"i64":"3453"}},{"literal":{"i64":"3454"}},{"literal":{"i64":"3455"}},{"literal":{"i64":"3456"}},{"literal":{"i64":"3457"}},{"literal":{"i64":"3458"}},{"literal":{"i64":"3459"}},{"literal":{"i64":"346"}},{"literal":{"i64":"3460"}},{"literal":{"i64":"3461"}},{"literal":{"i64":"3462"}},{"literal":{"i64":"3463"}},{"literal":{"i64":"3464"}},{"literal":{"i64":"3465"}},{"literal":{"i64":"3466"}},{"literal":{"i64":"3467"}},{"literal":{"i64":"3468"}},{"literal":{"i64":"3469"}},{"literal":{"i64":"347"}},{"literal":{"i64":"3470"}},{"literal":{"i64":"3471"}},{"literal":{"i64":"3472"}},{"literal":{"i64":"3473"}},{"literal":{"i64":"3474"}},{"literal":{"i64":"3475"}},{"literal":{"i64":"3476"}},{"literal":{"i64":"3477"}},{"literal":{"i64":"3478"}},{"literal":{"i64":"3479"}},{"literal":{"i64":"348"}},{"literal":{"i64":"3480"}},{"literal":{"i64":"3481"}},{"literal":{"i64":"3482"}},{"literal":{"i64":"3483"}},{"literal":{"i64":"3484"}},{"literal":{"i64":"3485"}},{"literal":{"i64":"3486"}},{"literal":{"i64":"3487"}},{"literal":{"i64":"3488"}},{"literal":{"i64":"3489"}},{"literal":{"i64":"349"}},{"literal":{"i64":"3490"}},{"literal":{"i64":"3491"}},{"literal":{"i64":"3492"}},{"literal":{"i64":"3493"}},{"literal":{"i64":"3494"}},{"literal":{"i64":"3495"}},{"literal":{"i64":"3496"}},{"literal":{"i64":"3497"}},{"literal":{"i64":"3498"}},{"literal":{"i64":"3499"}},{"literal":{"i64":"35"}},{"literal":{"i64":"350"}},{"literal":{"i64":"3500"}},{"literal":{"i64":"3501"}},{"literal":{"i64":"3502"}},{"literal":{"i64":"3503"}},{"literal":{"i64":"3504"}},{"literal":{"i64":"3505"}},{"literal":{"i64":"3506"}},{"literal":{"i64":"3507"}},{"literal":{"i64":"3508"}},{"literal":{"i64":"3509"}},{"literal":{"i64":"351"}},{"literal":{"i64":"3510"}},{"literal":{"i64":"3511"}},{"literal":{"i64":"3512"}},{"literal":{"i64":"3513"}},{"literal":{"i64":"3514"}},{"literal":{"i64":"3515"}},{"literal":{"i64":"3516"}},{"literal":{"i64":"3517"}},{"literal":{"i64":"3518"}},{"literal":{"i64":"3519"}},{"literal":{"i64":"352"}},{"literal":{"i64":"3520"}},{"literal":{"i64":"3521"}},{"literal":{"i64":"3522"}},{"literal":{"i64":"3523"}},{"literal":{"i64":"3524"}},{"literal":{"i64":"3525"}},{"literal":{"i64":"3526"}},{"literal":{"i64":"3527"}},{"literal":{"i64":"3528"}},{"literal":{"i64":"3529"}},{"literal":{"i64":"353"}},{"literal":{"i64":"3530"}},{"literal":{"i64":"3531"}},{"literal":{"i64":"3532"}},{"literal":{"i64":"3533"}},{"literal":{"i64":"3534"}},{"literal":{"i64":"3535"}},{"literal":{"i64":"3536"}},{"literal":{"i64":"3537"}},{"literal":{"i64":"3538"}},{"literal":{"i64":"3539"}},{"literal":{"i64":"354"}},{"literal":{"i64":"3540"}},{"literal":{"i64":"3541"}},{"literal":{"i64":"3542"}},{"literal":{"i64":"3543"}},{"literal":{"i64":"3544"}},{"literal":{"i64":"3545"}},{"literal":{"i64":"3546"}},{"literal":{"i64":"3547"}},{"literal":{"i64":"3548"}},{"literal":{"i64":"3549"}},{"literal":{"i64":"355"}},{"literal":{"i64":"3550"}},{"literal":{"i64":"3551"}},{"literal":{"i64":"3552"}},{"literal":{"i64":"3553"}},{"literal":{"i64":"3554"}},{"literal":{"i64":"3555"}},{"literal":{"i64":"3556"}},{"literal":{"i64":"3557"}},{"literal":{"i64":"3558"}},{"literal":{"i64":"3559"}},{"literal":{"i64":"356"}},{"literal":{"i64":"3560"}},{"literal":{"i64":"3561"}},{"literal":{"i64":"3562"}},{"literal":{"i64":"3563"}},{"literal":{"i64":"3564"}},{"literal":{"i64":"3565"}},{"literal":{"i64":"3566"}},{"literal":{"i64":"3567"}},{"literal":{"i64":"3568"}},{"literal":{"i64":"3569"}},{"literal":{"i64":"357"}},{"literal":{"i64":"3570"}},{"literal":{"i64":"3571"}},{"literal":{"i64":"3572"}},{"literal":{"i64":"3573"}},{"literal":{"i64":"3574"}},{"literal":{"i64":"3575"}},{"literal":{"i64":"3576"}},{"literal":{"i64":"3577"}},{"literal":{"i64":"3578"}},{"literal":{"i64":"3579"}},{"literal":{"i64":"358"}},{"literal":{"i64":"3580"}},{"literal":{"i64":"3581"}},{"literal":{"i64":"3582"}},{"literal":{"i64":"3583"}},{"literal":{"i64":"3584"}},{"literal":{"i64":"3585"}},{"literal":{"i64":"3586"}},{"literal":{"i64":"3587"}},{"literal":{"i64":"3588"}},{"literal":{"i64":"3589"}},{"literal":{"i64":"359"}},{"literal":{"i64":"3590"}},{"literal":{"i64":"3591"}},{"literal":{"i64":"3592"}},{"literal":{"i64":"3593"}},{"literal":{"i64":"3594"}},{"literal":{"i64":"3595"}},{"literal":{"i64":"3596"}},{"literal":{"i64":"3597"}},{"literal":{"i64":"3598"}},{"literal":{"i64":"3599"}},{"literal":{"i64":"36"}},{"literal":{"i64":"360"}},{"literal":{"i64":"3600"}},{"literal":{"i64":"3601"}},{"literal":{"i64":"3602"}},{"literal":{"i64":"3603"}},{"literal":{"i64":"3604"}},{"literal":{"i64":"3605"}},{"literal":{"i64":"3606"}},{"literal":{"i64":"3607"}},{"literal":{"i64":"3608"}},{"literal":{"i64":"3609"}},{"literal":{"i64":"361"}},{"literal":{"i64":"3610"}},{"literal":{"i64":"3611"}},{"literal":{"i64":"3612"}},{"literal":{"i64":"3613"}},{"literal":{"i64":"3614"}},{"literal":{"i64":"3615"}},{"literal":{"i64":"3616"}},{"literal":{"i64":"3617"}},{"literal":{"i64":"3618"}},{"literal":{"i64":"3619"}},{"literal":{"i64":"362"}},{"literal":{"i64":"3620"}},{"literal":{"i64":"3621"}},{"literal":{"i64":"3622"}},{"literal":{"i64":"3623"}},{"literal":{"i64":"3624"}},{"literal":{"i64":"3625"}},{"literal":{"i64":"3626"}},{"literal":{"i64":"3627"}},{"literal":{"i64":"3628"}},{"literal":{"i64":"3629"}},{"literal":{"i64":"363"}},{"literal":{"i64":"3630"}},{"literal":{"i64":"3631"}},{"literal":{"i64":"3632"}},{"literal":{"i64":"3633"}},{"literal":{"i64":"3634"}},{"literal":{"i64":"3635"}},{"literal":{"i64":"3636"}},{"literal":{"i64":"3637"}},{"literal":{"i64":"3638"}},{"literal":{"i64":"3639"}},{"literal":{"i64":"364"}},{"literal":{"i64":"3640"}},{"literal":{"i64":"3641"}},{"literal":{"i64":"3642"}},{"literal":{"i64":"3643"}},{"literal":{"i64":"3644"}},{"literal":{"i64":"3645"}},{"literal":{"i64":"3646"}},{"literal":{"i64":"3647"}},{"literal":{"i64":"3648"}},{"literal":{"i64":"3649"}},{"literal":{"i64":"365"}},{"literal":{"i64":"3650"}},{"literal":{"i64":"3651"}},{"literal":{"i64":"3652"}},{"literal":{"i64":"3653"}},{"literal":{"i64":"3654"}},{"literal":{"i64":"3655"}},{"literal":{"i64":"3656"}},{"literal":{"i64":"3657"}},{"literal":{"i64":"3658"}},{"literal":{"i64":"3659"}},{"literal":{"i64":"366"}},{"literal":{"i64":"3660"}},{"literal":{"i64":"3661"}},{"literal":{"i64":"3662"}},{"literal":{"i64":"3663"}},{"literal":{"i64":"3664"}},{"literal":{"i64":"3665"}},{"literal":{"i64":"3666"}},{"literal":{"i64":"3667"}},{"literal":{"i64":"3668"}},{"literal":{"i64":"3669"}},{"literal":{"i64":"367"}},{"literal":{"i64":"3670"}},{"literal":{"i64":"3671"}},{"literal":{"i64":"3672"}},{"literal":{"i64":"3673"}},{"literal":{"i64":"3674"}},{"literal":{"i64":"3675"}},{"literal":{"i64":"3676"}},{"literal":{"i64":"3677"}},{"literal":{"i64":"3678"}},{"literal":{"i64":"3679"}},{"literal":{"i64":"368"}},{"literal":{"i64":"3680"}},{"literal":{"i64":"3681"}},{"literal":{"i64":"3682"}},{"literal":{"i64":"3683"}},{"literal":{"i64":"3684"}},{"literal":{"i64":"3685"}},{"literal":{"i64":"3686"}},{"literal":{"i64":"3687"}},{"literal":{"i64":"3688"}},{"literal":{"i64":"3689"}},{"literal":{"i64":"369"}},{"literal":{"i64":"3690"}},{"literal":{"i64":"3691"}},{"literal":{"i64":"3692"}},{"literal":{"i64":"3693"}},{"literal":{"i64":"3694"}},{"literal":{"i64":"3695"}},{"literal":{"i64":"3696"}},{"literal":{"i64":"3697"}},{"literal":{"i64":"3698"}},{"literal":{"i64":"3699"}},{"literal":{"i64":"37"}},{"literal":{"i64":"370"}},{"literal":{"i64":"3700"}},{"literal":{"i64":"3701"}},{"literal":{"i64":"3702"}},{"literal":{"i64":"3703"}},{"literal":{"i64":"3704"}},{"literal":{"i64":"3705"}},{"literal":{"i64":"3706"}},{"literal":{"i64":"3707"}},{"literal":{"i64":"3708"}},{"literal":{"i64":"3709"}},{"literal":{"i64":"371"}},{"literal":{"i64":"3710"}},{"literal":{"i64":"3711"}},{"literal":{"i64":"3712"}},{"literal":{"i64":"3713"}},{"literal":{"i64":"3714"}},{"literal":{"i64":"3715"}},{"literal":{"i64":"3716"}},{"literal":{"i64":"3717"}},{"literal":{"i64":"3718"}},{"literal":{"i64":"3719"}},{"literal":{"i64":"372"}},{"literal":{"i64":"3720"}},{"literal":{"i64":"3721"}},{"literal":{"i64":"3722"}},{"literal":{"i64":"3723"}},{"literal":{"i64":"3724"}},{"literal":{"i64":"3725"}},{"literal":{"i64":"3726"}},{"literal":{"i64":"3727"}},{"literal":{"i64":"3728"}},{"literal":{"i64":"3729"}},{"literal":{"i64":"373"}},{"literal":{"i64":"3730"}},{"literal":{"i64":"3731"}},{"literal":{"i64":"3732"}},{"literal":{"i64":"3733"}},{"literal":{"i64":"3734"}},{"literal":{"i64":"3735"}},{"literal":{"i64":"3736"}},{"literal":{"i64":"3737"}},{"literal":{"i64":"3738"}},{"literal":{"i64":"3739"}},{"literal":{"i64":"374"}},{"literal":{"i64":"3740"}},{"literal":{"i64":"3741"}},{"literal":{"i64":"3742"}},{"literal":{"i64":"3743"}},{"literal":{"i64":"3744"}},{"literal":{"i64":"3745"}},{"literal":{"i64":"3746"}},{"literal":{"i64":"3747"}},{"literal":{"i64":"3748"}},{"literal":{"i64":"3749"}},{"literal":{"i64":"375"}},{"literal":{"i64":"3750"}},{"literal":{"i64":"3751"}},{"literal":{"i64":"3752"}},{"literal":{"i64":"3753"}},{"literal":{"i64":"3754"}},{"literal":{"i64":"3755"}},{"literal":{"i64":"3756"}},{"literal":{"i64":"3757"}},{"literal":{"i64":"3758"}},{"literal":{"i64":"3759"}},{"literal":{"i64":"376"}},{"literal":{"i64":"3760"}},{"literal":{"i64":"3761"}},{"literal":{"i64":"3762"}},{"literal":{"i64":"3763"}},{"literal":{"i64":"3764"}},{"literal":{"i64":"3765"}},{"literal":{"i64":"3766"}},{"literal":{"i64":"3767"}},{"literal":{"i64":"3768"}},{"literal":{"i64":"3769"}},{"literal":{"i64":"377"}},{"literal":{"i64":"3770"}},{"literal":{"i64":"3771"}},{"literal":{"i64":"3772"}},{"literal":{"i64":"3773"}},{"literal":{"i64":"3774"}},{"literal":{"i64":"3775"}},{"literal":{"i64":"3776"}},{"literal":{"i64":"3777"}},{"literal":{"i64":"3778"}},{"literal":{"i64":"3779"}},{"literal":{"i64":"378"}},{"literal":{"i64":"3780"}},{"literal":{"i64":"3781"}},{"literal":{"i64":"3782"}},{"literal":{"i64":"3783"}},{"literal":{"i64":"3784"}},{"literal":{"i64":"3785"}},{"literal":{"i64":"3786"}},{"literal":{"i64":"3787"}},{"literal":{"i64":"3788"}},{"literal":{"i64":"3789"}},{"literal":{"i64":"379"}},{"literal":{"i64":"3790"}},{"literal":{"i64":"3791"}},{"literal":{"i64":"3792"}},{"literal":{"i64":"3793"}},{"literal":{"i64":"3794"}},{"literal":{"i64":"3795"}},{"literal":{"i64":"3796"}},{"literal":{"i64":"3797"}},{"literal":{"i64":"3798"}},{"literal":{"i64":"3799"}},{"literal":{"i64":"38"}},{"literal":{"i64":"380"}},{"literal":{"i64":"3800"}},{"literal":{"i64":"3801"}},{"literal":{"i64":"3802"}},{"literal":{"i64":"3803"}},{"literal":{"i64":"3804"}},{"literal":{"i64":"3805"}},{"literal":{"i64":"3806"}},{"literal":{"i64":"3807"}},{"literal":{"i64":"3808"}},{"literal":{"i64":"3809"}},{"literal":{"i64":"381"}},{"literal":{"i64":"3810"}},{"literal":{"i64":"3811"}},{"literal":{"i64":"3812"}},{"literal":{"i64":"3813"}},{"literal":{"i64":"3814"}},{"literal":{"i64":"3815"}},{"literal":{"i64":"3816"}},{"literal":{"i64":"3817"}},{"literal":{"i64":"3818"}},{"literal":{"i64":"3819"}},{"literal":{"i64":"382"}},{"literal":{"i64":"3820"}},{"literal":{"i64":"3821"}},{"literal":{"i64":"3822"}},{"literal":{"i64":"3823"}},{"literal":{"i64":"3824"}},{"literal":{"i64":"3825"}},{"literal":{"i64":"3826"}},{"literal":{"i64":"3827"}},{"literal":{"i64":"3828"}},{"literal":{"i64":"3829"}},{"literal":{"i64":"383"}},{"literal":{"i64":"3830"}},{"literal":{"i64":"3831"}},{"literal":{"i64":"3832"}},{"literal":{"i64":"3833"}},{"literal":{"i64":"3834"}},{"literal":{"i64":"3835"}},{"literal":{"i64":"3836"}},{"literal":{"i64":"3837"}},{"literal":{"i64":"3838"}},{"literal":{"i64":"3839"}},{"literal":{"i64":"384"}},{"literal":{"i64":"3840"}},{"literal":{"i64":"3841"}},{"literal":{"i64":"3842"}},{"literal":{"i64":"3843"}},{"literal":{"i64":"3844"}},{"literal":{"i64":"3845"}},{"literal":{"i64":"3846"}},{"literal":{"i64":"3847"}},{"literal":{"i64":"3848"}},{"literal":{"i64":"3849"}},{"literal":{"i64":"385"}},{"literal":{"i64":"3850"}},{"literal":{"i64":"3851"}},{"literal":{"i64":"3852"}},{"literal":{"i64":"3853"}},{"literal":{"i64":"3854"}},{"literal":{"i64":"3855"}},{"literal":{"i64":"3856"}},{"literal":{"i64":"3857"}},{"literal":{"i64":"3858"}},{"literal":{"i64":"3859"}},{"literal":{"i64":"386"}},{"literal":{"i64":"3860"}},{"literal":{"i64":"3861"}},{"literal":{"i64":"3862"}},{"literal":{"i64":"3863"}},{"literal":{"i64":"3864"}},{"literal":{"i64":"3865"}},{"literal":{"i64":"3866"}},{"literal":{"i64":"3867"}},{"literal":{"i64":"3868"}},{"literal":{"i64":"3869"}},{"literal":{"i64":"387"}},{"literal":{"i64":"3870"}},{"literal":{"i64":"3871"}},{"literal":{"i64":"3872"}},{"literal":{"i64":"3873"}},{"literal":{"i64":"3874"}},{"literal":{"i64":"3875"}},{"literal":{"i64":"3876"}},{"literal":{"i64":"3877"}},{"literal":{"i64":"3878"}},{"literal":{"i64":"3879"}},{"literal":{"i64":"388"}},{"literal":{"i64":"3880"}},{"literal":{"i64":"3881"}},{"literal":{"i64":"3882"}},{"literal":{"i64":"3883"}},{"literal":{"i64":"3884"}},{"literal":{"i64":"3885"}},{"literal":{"i64":"3886"}},{"literal":{"i64":"3887"}},{"literal":{"i64":"3888"}},{"literal":{"i64":"3889"}},{"literal":{"i64":"389"}},{"literal":{"i64":"3890"}},{"literal":{"i64":"3891"}},{"literal":{"i64":"3892"}},{"literal":{"i64":"3893"}},{"literal":{"i64":"3894"}},{"literal":{"i64":"3895"}},{"literal":{"i64":"3896"}},{"literal":{"i64":"3897"}},{"literal":{"i64":"3898"}},{"literal":{"i64":"3899"}},{"literal":{"i64":"39"}},{"literal":{"i64":"390"}},{"literal":{"i64":"3900"}},{"literal":{"i64":"3901"}},{"literal":{"i64":"3902"}},{"literal":{"i64":"3903"}},{"literal":{"i64":"3904"}},{"literal":{"i64":"3905"}},{"literal":{"i64":"3906"}},{"literal":{"i64":"3907"}},{"literal":{"i64":"3908"}},{"literal":{"i64":"3909"}},{"literal":{"i64":"391"}},{"literal":{"i64":"3910"}},{"literal":{"i64":"3911"}},{"literal":{"i64":"3912"}},{"literal":{"i64":"3913"}},{"literal":{"i64":"3914"}},{"literal":{"i64":"3915"}},{"literal":{"i64":"3916"}},{"literal":{"i64":"3917"}},{"literal":{"i64":"3918"}},{"literal":{"i64":"3919"}},{"literal":{"i64":"392"}},{"literal":{"i64":"3920"}},{"literal":{"i64":"3921"}},{"literal":{"i64":"3922"}},{"literal":{"i64":"3923"}},{"literal":{"i64":"3924"}},{"literal":{"i64":"3925"}},{"literal":{"i64":"3926"}},{"literal":{"i64":"3927"}},{"literal":{"i64":"3928"}},{"literal":{"i64":"3929"}},{"literal":{"i64":"393"}},{"literal":{"i64":"3930"}},{"literal":{"i64":"3931"}},{"literal":{"i64":"3932"}},{"literal":{"i64":"3933"}},{"literal":{"i64":"3934"}},{"literal":{"i64":"3935"}},{"literal":{"i64":"3936"}},{"literal":{"i64":"3937"}},{"literal":{"i64":"3938"}},{"literal":{"i64":"3939"}},{"literal":{"i64":"394"}},{"literal":{"i64":"3940"}},{"literal":{"i64":"3941"}},{"literal":{"i64":"3942"}},{"literal":{"i64":"3943"}},{"literal":{"i64":"3944"}},{"literal":{"i64":"3945"}},{"literal":{"i64":"3946"}},{"literal":{"i64":"3947"}},{"literal":{"i64":"3948"}},{"literal":{"i64":"3949"}},{"literal":{"i64":"395"}},{"literal":{"i64":"3950"}},{"literal":{"i64":"3951"}},{"literal":{"i64":"3952"}},{"literal":{"i64":"3953"}},{"literal":{"i64":"3954"}},{"literal":{"i64":"3955"}},{"literal":{"i64":"3956"}},{"literal":{"i64":"3957"}},{"literal":{"i64":"3958"}},{"literal":{"i64":"3959"}},{"literal":{"i64":"396"}},{"literal":{"i64":"3960"}},{"literal":{"i64":"3961"}},{"literal":{"i64":"3962"}},{"literal":{"i64":"3963"}},{"literal":{"i64":"3964"}},{"literal":{"i64":"3965"}},{"literal":{"i64":"3966"}},{"literal":{"i64":"3967"}},{"literal":{"i64":"3968"}},{"literal":{"i64":"3969"}},{"literal":{"i64":"397"}},{"literal":{"i64":"3970"}},{"literal":{"i64":"3971"}},{"literal":{"i64":"3972"}},{"literal":{"i64":"3973"}},{"literal":{"i64":"3974"}},{"literal":{"i64":"3975"}},{"literal":{"i64":"3976"}},{"literal":{"i64":"3977"}},{"literal":{"i64":"3978"}},{"literal":{"i64":"3979"}},{"literal":{"i64":"398"}},{"literal":{"i64":"3980"}},{"literal":{"i64":"3981"}},{"literal":{"i64":"3982"}},{"literal":{"i64":"3983"}},{"literal":{"i64":"3984"}},{"literal":{"i64":"3985"}},{"literal":{"i64":"3986"}},{"literal":{"i64":"3987"}},{"literal":{"i64":"3988"}},{"literal":{"i64":"3989"}},{"literal":{"i64":"399"}},{"literal":{"i64":"3990"}},{"literal":{"i64":"3991"}},{"literal":{"i64":"3992"}},{"literal":{"i64":"3993"}},{"literal":{"i64":"3994"}},{"literal":{"i64":"3995"}},{"literal":{"i64":"3996"}},{"literal":{"i64":"3997"}},{"literal":{"i64":"3998"}},{"literal":{"i64":"3999"}},{"literal":{"i64":"4"}},{"literal":{"i64":"40"}},{"literal":{"i64":"400"}},{"literal":{"i64":"4000"}},{"literal":{"i64":"4001"}},{"literal":{"i64":"4002"}},{"literal":{"i64":"4003"}},{"literal":{"i64":"4004"}},{"literal":{"i64":"4005"}},{"literal":{"i64":"4006"}},{"literal":{"i64":"4007"}},{"literal":{"i64":"4008"}},{"literal":{"i64":"4009"}},{"literal":{"i64":"401"}},{"literal":{"i64":"4010"}},{"literal":{"i64":"4011"}},{"literal":{"i64":"4012"}},{"literal":{"i64":"4013"}},{"literal":{"i64":"4014"}},{"literal":{"i64":"4015"}},{"literal":{"i64":"4016"}},{"literal":{"i64":"4017"}},{"literal":{"i64":"4018"}},{"literal":{"i64":"4019"}},{"literal":{"i64":"402"}},{"literal":{"i64":"4020"}},{"literal":{"i64":"4021"}},{"literal":{"i64":"4022"}},{"literal":{"i64":"4023"}},{"literal":{"i64":"4024"}},{"literal":{"i64":"4025"}},{"literal":{"i64":"4026"}},{"literal":{"i64":"4027"}},{"literal":{"i64":"4028"}},{"literal":{"i64":"4029"}},{"literal":{"i64":"403"}},{"literal":{"i64":"4030"}},{"literal":{"i64":"4031"}},{"literal":{"i64":"4032"}},{"literal":{"i64":"4033"}},{"literal":{"i64":"4034"}},{"literal":{"i64":"4035"}},{"literal":{"i64":"4036"}},{"literal":{"i64":"4037"}},{"literal":{"i64":"4038"}},{"literal":{"i64":"4039"}},{"literal":{"i64":"404"}},{"literal":{"i64":"4040"}},{"literal":{"i64":"4041"}},{"literal":{"i64":"4042"}},{"literal":{"i64":"4043"}},{"literal":{"i64":"4044"}},{"literal":{"i64":"4045"}},{"literal":{"i64":"4046"}},{"literal":{"i64":"4047"}},{"literal":{"i64":"4048"}},{"literal":{"i64":"4049"}},{"literal":{"i64":"405"}},{"literal":{"i64":"4050"}},{"literal":{"i64":"4051"}},{"literal":{"i64":"4052"}},{"literal":{"i64":"4053"}},{"literal":{"i64":"4054"}},{"literal":{"i64":"4055"}},{"literal":{"i64":"4056"}},{"literal":{"i64":"4057"}},{"literal":{"i64":"4058"}},{"literal":{"i64":"4059"}},{"literal":{"i64":"406"}},{"literal":{"i64":"4060"}},{"literal":{"i64":"4061"}},{"literal":{"i64":"4062"}},{"literal":{"i64":"4063"}},{"literal":{"i64":"4064"}},{"literal":{"i64":"4065"}},{"literal":{"i64":"4066"}},{"literal":{"i64":"4067"}},{"literal":{"i64":"4068"}},{"literal":{"i64":"4069"}},{"literal":{"i64":"407"}},{"literal":{"i64":"4070"}},{"literal":{"i64":"4071"}},{"literal":{"i64":"4072"}},{"literal":{"i64":"4073"}},{"literal":{"i64":"4074"}},{"literal":{"i64":"4075"}},{"literal":{"i64":"4076"}},{"literal":{"i64":"4077"}},{"literal":{"i64":"4078"}},{"literal":{"i64":"4079"}},{"literal":{"i64":"408"}},{"literal":{"i64":"4080"}},{"literal":{"i64":"4081"}},{"literal":{"i64":"4082"}},{"literal":{"i64":"4083"}},{"literal":{"i64":"4084"}},{"literal":{"i64":"4085"}},{"literal":{"i64":"4086"}},{"literal":{"i64":"4087"}},{"literal":{"i64":"4088"}},{"literal":{"i64":"4089"}},{"literal":{"i64":"409"}},{"literal":{"i64":"4090"}},{"literal":{"i64":"4091"}},{"literal":{"i64":"4092"}},{"literal":{"i64":"4093"}},{"literal":{"i64":"4094"}},{"literal":{"i64":"4095"}},{"literal":{"i64":"4096"}},{"literal":{"i64":"4097"}},{"literal":{"i64":"4098"}},{"literal":{"i64":"4099"}},{"literal":{"i64":"41"}},{"literal":{"i64":"410"}},{"literal":{"i64":"4100"}},{"literal":{"i64":"4101"}},{"literal":{"i64":"4102"}},{"literal":{"i64":"4103"}},{"literal":{"i64":"4104"}},{"literal":{"i64":"4105"}},{"literal":{"i64":"4106"}},{"literal":{"i64":"4107"}},{"literal":{"i64":"4108"}},{"literal":{"i64":"4109"}},{"literal":{"i64":"411"}},{"literal":{"i64":"4110"}},{"literal":{"i64":"4111"}},{"literal":{"i64":"4112"}},{"literal":{"i64":"4113"}},{"literal":{"i64":"4114"}},{"literal":{"i64":"4115"}},{"literal":{"i64":"4116"}},{"literal":{"i64":"4117"}},{"literal":{"i64":"4118"}},{"literal":{"i64":"4119"}},{"literal":{"i64":"412"}},{"literal":{"i64":"4120"}},{"literal":{"i64":"4121"}},{"literal":{"i64":"4122"}},{"literal":{"i64":"4123"}},{"literal":{"i64":"4124"}},{"literal":{"i64":"4125"}},{"literal":{"i64":"4126"}},{"literal":{"i64":"4127"}},{"literal":{"i64":"4128"}},{"literal":{"i64":"4129"}},{"literal":{"i64":"413"}},{"literal":{"i64":"4130"}},{"literal":{"i64":"4131"}},{"literal":{"i64":"4132"}},{"literal":{"i64":"4133"}},{"literal":{"i64":"4134"}},{"literal":{"i64":"4135"}},{"literal":{"i64":"4136"}},{"literal":{"i64":"4137"}},{"literal":{"i64":"4138"}},{"literal":{"i64":"4139"}},{"literal":{"i64":"414"}},{"literal":{"i64":"4140"}},{"literal":{"i64":"4141"}},{"literal":{"i64":"4142"}},{"literal":{"i64":"4143"}},{"literal":{"i64":"4144"}},{"literal":{"i64":"4145"}},{"literal":{"i64":"4146"}},{"literal":{"i64":"4147"}},{"literal":{"i64":"4148"}},{"literal":{"i64":"4149"}},{"literal":{"i64":"415"}},{"literal":{"i64":"4150"}},{"literal":{"i64":"4151"}},{"literal":{"i64":"4152"}},{"literal":{"i64":"4153"}},{"literal":{"i64":"4154"}},{"literal":{"i64":"4155"}},{"literal":{"i64":"4156"}},{"literal":{"i64":"4157"}},{"literal":{"i64":"4158"}},{"literal":{"i64":"4159"}},{"literal":{"i64":"416"}},{"literal":{"i64":"4160"}},{"literal":{"i64":"4161"}},{"literal":{"i64":"4162"}},{"literal":{"i64":"4163"}},{"literal":{"i64":"4164"}},{"literal":{"i64":"4165"}},{"literal":{"i64":"4166"}},{"literal":{"i64":"4167"}},{"literal":{"i64":"4168"}},{"literal":{"i64":"4169"}},{"literal":{"i64":"417"}},{"literal":{"i64":"4170"}},{"literal":{"i64":"4171"}},{"literal":{"i64":"4172"}},{"literal":{"i64":"4173"}},{"literal":{"i64":"4174"}},{"literal":{"i64":"4175"}},{"literal":{"i64":"4176"}},{"literal":{"i64":"4177"}},{"literal":{"i64":"4178"}},{"literal":{"i64":"4179"}},{"literal":{"i64":"418"}},{"literal":{"i64":"4180"}},{"literal":{"i64":"4181"}},{"literal":{"i64":"4182"}},{"literal":{"i64":"4183"}},{"literal":{"i64":"4184"}},{"literal":{"i64":"4185"}},{"literal":{"i64":"4186"}},{"literal":{"i64":"4187"}},{"literal":{"i64":"4188"}},{"literal":{"i64":"4189"}},{"literal":{"i64":"419"}},{"literal":{"i64":"4190"}},{"literal":{"i64":"4191"}},{"literal":{"i64":"4192"}},{"literal":{"i64":"4193"}},{"literal":{"i64":"4194"}},{"literal":{"i64":"4195"}},{"literal":{"i64":"4196"}},{"literal":{"i64":"4197"}},{"literal":{"i64":"4198"}},{"literal":{"i64":"4199"}},{"literal":{"i64":"42"}},{"literal":{"i64":"420"}},{"literal":{"i64":"4200"}},{"literal":{"i64":"4201"}},{"literal":{"i64":"4202"}},{"literal":{"i64":"4203"}},{"literal":{"i64":"4204"}},{"literal":{"i64":"4205"}},{"literal":{"i64":"4206"}},{"literal":{"i64":"4207"}},{"literal":{"i64":"4208"}},{"literal":{"i64":"4209"}},{"literal":{"i64":"421"}},{"literal":{"i64":"4210"}},{"literal":{"i64":"4211"}},{"literal":{"i64":"4212"}},{"literal":{"i64":"4213"}},{"literal":{"i64":"4214"}},{"literal":{"i64":"4215"}},{"literal":{"i64":"4216"}},{"literal":{"i64":"4217"}},{"literal":{"i64":"4218"}},{"literal":{"i64":"4219"}},{"literal":{"i64":"422"}},{"literal":{"i64":"4220"}},{"literal":{"i64":"4221"}},{"literal":{"i64":"4222"}},{"literal":{"i64":"4223"}},{"literal":{"i64":"4224"}},{"literal":{"i64":"4225"}},{"literal":{"i64":"4226"}},{"literal":{"i64":"4227"}},{"literal":{"i64":"4228"}},{"literal":{"i64":"4229"}},{"literal":{"i64":"423"}},{"literal":{"i64":"4230"}},{"literal":{"i64":"4231"}},{"literal":{"i64":"4232"}},{"literal":{"i64":"4233"}},{"literal":{"i64":"4234"}},{"literal":{"i64":"4235"}},{"literal":{"i64":"4236"}},{"literal":{"i64":"4237"}},{"literal":{"i64":"4238"}},{"literal":{"i64":"4239"}},{"literal":{"i64":"424"}},{"literal":{"i64":"4240"}},{"literal":{"i64":"4241"}},{"literal":{"i64":"4242"}},{"literal":{"i64":"4243"}},{"literal":{"i64":"4244"}},{"literal":{"i64":"4245"}},{"literal":{"i64":"4246"}},{"literal":{"i64":"4247"}},{"literal":{"i64":"4248"}},{"literal":{"i64":"4249"}},{"literal":{"i64":"425"}},{"literal":{"i64":"4250"}},{"literal":{"i64":"4251"}},{"literal":{"i64":"4252"}},{"literal":{"i64":"4253"}},{"literal":{"i64":"4254"}},{"literal":{"i64":"4255"}},{"literal":{"i64":"4256"}},{"literal":{"i64":"4257"}},{"literal":{"i64":"4258"}},{"literal":{"i64":"4259"}},{"literal":{"i64":"426"}},{"literal":{"i64":"4260"}},{"literal":{"i64":"4261"}},{"literal":{"i64":"4262"}},{"literal":{"i64":"4263"}},{"literal":{"i64":"4264"}},{"literal":{"i64":"4265"}},{"literal":{"i64":"4266"}},{"literal":{"i64":"4267"}},{"literal":{"i64":"4268"}},{"literal":{"i64":"4269"}},{"literal":{"i64":"427"}},{"literal":{"i64":"4270"}},{"literal":{"i64":"4271"}},{"literal":{"i64":"4272"}},{"literal":{"i64":"4273"}},{"literal":{"i64":"4274"}},{"literal":{"i64":"4275"}},{"literal":{"i64":"4276"}},{"literal":{"i64":"4277"}},{"literal":{"i64":"4278"}},{"literal":{"i64":"4279"}},{"literal":{"i64":"428"}},{"literal":{"i64":"4280"}},{"literal":{"i64":"4281"}},{"literal":{"i64":"4282"}},{"literal":{"i64":"4283"}},{"literal":{"i64":"4284"}},{"literal":{"i64":"4285"}},{"literal":{"i64":"4286"}},{"literal":{"i64":"4287"}},{"literal":{"i64":"4288"}},{"literal":{"i64":"4289"}},{"literal":{"i64":"429"}},{"literal":{"i64":"4290"}},{"literal":{"i64":"4291"}},{"literal":{"i64":"4292"}},{"literal":{"i64":"4293"}},{"literal":{"i64":"4294"}},{"literal":{"i64":"4295"}},{"literal":{"i64":"4296"}},{"literal":{"i64":"4297"}},{"literal":{"i64":"4298"}},{"literal":{"i64":"4299"}},{"literal":{"i64":"43"}},{"literal":{"i64":"430"}},{"literal":{"i64":"4300"}},{"literal":{"i64":"4301"}},{"literal":{"i64":"4302"}},{"literal":{"i64":"4303"}},{"literal":{"i64":"4304"}},{"literal":{"i64":"4305"}},{"literal":{"i64":"4306"}},{"literal":{"i64":"4307"}},{"literal":{"i64":"4308"}},{"literal":{"i64":"4309"}},{"literal":{"i64":"431"}},{"literal":{"i64":"4310"}},{"literal":{"i64":"4311"}},{"literal":{"i64":"4312"}},{"literal":{"i64":"4313"}},{"literal":{"i64":"4314"}},{"literal":{"i64":"4315"}},{"literal":{"i64":"4316"}},{"literal":{"i64":"4317"}},{"literal":{"i64":"4318"}},{"literal":{"i64":"4319"}},{"literal":{"i64":"432"}},{"literal":{"i64":"4320"}},{"literal":{"i64":"4321"}},{"literal":{"i64":"4322"}},{"literal":{"i64":"4323"}},{"literal":{"i64":"4324"}},{"literal":{"i64":"4325"}},{"literal":{"i64":"4326"}},{"literal":{"i64":"4327"}},{"literal":{"i64":"4328"}},{"literal":{"i64":"4329"}},{"literal":{"i64":"433"}},{"literal":{"i64":"4330"}},{"literal":{"i64":"4331"}},{"literal":{"i64":"4332"}},{"literal":{"i64":"4333"}},{"literal":{"i64":"4334"}},{"literal":{"i64":"4335"}},{"literal":{"i64":"4336"}},{"literal":{"i64":"4337"}},{"literal":{"i64":"4338"}},{"literal":{"i64":"4339"}},{"literal":{"i64":"434"}},{"literal":{"i64":"4340"}},{"literal":{"i64":"4341"}},{"literal":{"i64":"4342"}},{"literal":{"i64":"4343"}},{"literal":{"i64":"4344"}},{"literal":{"i64":"4345"}},{"literal":{"i64":"4346"}},{"literal":{"i64":"4347"}},{"literal":{"i64":"4348"}},{"literal":{"i64":"4349"}},{"literal":{"i64":"435"}},{"literal":{"i64":"4350"}},{"literal":{"i64":"4351"}},{"literal":{"i64":"4352"}},{"literal":{"i64":"4353"}},{"literal":{"i64":"4354"}},{"literal":{"i64":"4355"}},{"literal":{"i64":"4356"}},{"literal":{"i64":"4357"}},{"literal":{"i64":"4358"}},{"literal":{"i64":"4359"}},{"literal":{"i64":"436"}},{"literal":{"i64":"4360"}},{"literal":{"i64":"4361"}},{"literal":{"i64":"4362"}},{"literal":{"i64":"4363"}},{"literal":{"i64":"4364"}},{"literal":{"i64":"4365"}},{"literal":{"i64":"4366"}},{"literal":{"i64":"4367"}},{"literal":{"i64":"4368"}},{"literal":{"i64":"4369"}},{"literal":{"i64":"437"}},{"literal":{"i64":"4370"}},{"literal":{"i64":"4371"}},{"literal":{"i64":"4372"}},{"literal":{"i64":"4373"}},{"literal":{"i64":"4374"}},{"literal":{"i64":"4375"}},{"literal":{"i64":"4376"}},{"literal":{"i64":"4377"}},{"literal":{"i64":"4378"}},{"literal":{"i64":"4379"}},{"literal":{"i64":"438"}},{"literal":{"i64":"4380"}},{"literal":{"i64":"4381"}},{"literal":{"i64":"4382"}},{"literal":{"i64":"4383"}},{"literal":{"i64":"4384"}},{"literal":{"i64":"4385"}},{"literal":{"i64":"4386"}},{"literal":{"i64":"4387"}},{"literal":{"i64":"4388"}},{"literal":{"i64":"4389"}},{"literal":{"i64":"439"}},{"literal":{"i64":"4390"}},{"literal":{"i64":"4391"}},{"literal":{"i64":"4392"}},{"literal":{"i64":"4393"}},{"literal":{"i64":"4394"}},{"literal":{"i64":"4395"}},{"literal":{"i64":"4396"}},{"literal":{"i64":"4397"}},{"literal":{"i64":"4398"}},{"literal":{"i64":"4399"}},{"literal":{"i64":"44"}},{"literal":{"i64":"440"}},{"literal":{"i64":"4400"}},{"literal":{"i64":"4401"}},{"literal":{"i64":"4402"}},{"literal":{"i64":"4403"}},{"literal":{"i64":"4404"}},{"literal":{"i64":"4405"}},{"literal":{"i64":"4406"}},{"literal":{"i64":"4407"}},{"literal":{"i64":"4408"}},{"literal":{"i64":"4409"}},{"literal":{"i64":"441"}},{"literal":{"i64":"4410"}},{"literal":{"i64":"4411"}},{"literal":{"i64":"4412"}},{"literal":{"i64":"4413"}},{"literal":{"i64":"4414"}},{"literal":{"i64":"4415"}},{"literal":{"i64":"4416"}},{"literal":{"i64":"4417"}},{"literal":{"i64":"4418"}},{"literal":{"i64":"4419"}},{"literal":{"i64":"442"}},{"literal":{"i64":"4420"}},{"literal":{"i64":"4421"}},{"literal":{"i64":"4422"}},{"literal":{"i64":"4423"}},{"literal":{"i64":"4424"}},{"literal":{"i64":"4425"}},{"literal":{"i64":"4426"}},{"literal":{"i64":"4427"}},{"literal":{"i64":"4428"}},{"literal":{"i64":"4429"}},{"literal":{"i64":"443"}},{"literal":{"i64":"4430"}},{"literal":{"i64":"4431"}},{"literal":{"i64":"4432"}},{"literal":{"i64":"4433"}},{"literal":{"i64":"4434"}},{"literal":{"i64":"4435"}},{"literal":{"i64":"4436"}},{"literal":{"i64":"4437"}},{"literal":{"i64":"4438"}},{"literal":{"i64":"4439"}},{"literal":{"i64":"444"}},{"literal":{"i64":"4440"}},{"literal":{"i64":"4441"}},{"literal":{"i64":"4442"}},{"literal":{"i64":"4443"}},{"literal":{"i64":"4444"}},{"literal":{"i64":"4445"}},{"literal":{"i64":"4446"}},{"literal":{"i64":"4447"}},{"literal":{"i64":"4448"}},{"literal":{"i64":"4449"}},{"literal":{"i64":"445"}},{"literal":{"i64":"4450"}},{"literal":{"i64":"4451"}},{"literal":{"i64":"4452"}},{"literal":{"i64":"4453"}},{"literal":{"i64":"4454"}},{"literal":{"i64":"4455"}},{"literal":{"i64":"4456"}},{"literal":{"i64":"4457"}},{"literal":{"i64":"4458"}},{"literal":{"i64":"4459"}},{"literal":{"i64":"446"}},{"literal":{"i64":"4460"}},{"literal":{"i64":"4461"}},{"literal":{"i64":"4462"}},{"literal":{"i64":"4463"}},{"literal":{"i64":"4464"}},{"literal":{"i64":"4465"}},{"literal":{"i64":"4466"}},{"literal":{"i64":"4467"}},{"literal":{"i64":"4468"}},{"literal":{"i64":"4469"}},{"literal":{"i64":"447"}},{"literal":{"i64":"4470"}},{"literal":{"i64":"4471"}},{"literal":{"i64":"4472"}},{"literal":{"i64":"4473"}},{"literal":{"i64":"4474"}},{"literal":{"i64":"4475"}},{"literal":{"i64":"4476"}},{"literal":{"i64":"4477"}},{"literal":{"i64":"4478"}},{"literal":{"i64":"4479"}},{"literal":{"i64":"448"}},{"literal":{"i64":"4480"}},{"literal":{"i64":"4481"}},{"literal":{"i64":"4482"}},{"literal":{"i64":"4483"}},{"literal":{"i64":"4484"}},{"literal":{"i64":"4485"}},{"literal":{"i64":"4486"}},{"literal":{"i64":"4487"}},{"literal":{"i64":"4488"}},{"literal":{"i64":"4489"}},{"literal":{"i64":"449"}},{"literal":{"i64":"4490"}},{"literal":{"i64":"4491"}},{"literal":{"i64":"4492"}},{"literal":{"i64":"4493"}},{"literal":{"i64":"4494"}},{"literal":{"i64":"4495"}},{"literal":{"i64":"4496"}},{"literal":{"i64":"4497"}},{"literal":{"i64":"4498"}},{"literal":{"i64":"4499"}},{"literal":{"i64":"45"}},{"literal":{"i64":"450"}},{"literal":{"i64":"4500"}},{"literal":{"i64":"4501"}},{"literal":{"i64":"4502"}},{"literal":{"i64":"4503"}},{"literal":{"i64":"4504"}},{"literal":{"i64":"4505"}},{"literal":{"i64":"4506"}},{"literal":{"i64":"4507"}},{"literal":{"i64":"4508"}},{"literal":{"i64":"4509"}},{"literal":{"i64":"451"}},{"literal":{"i64":"4510"}},{"literal":{"i64":"4511"}},{"literal":{"i64":"4512"}},{"literal":{"i64":"4513"}},{"literal":{"i64":"4514"}},{"literal":{"i64":"4515"}},{"literal":{"i64":"4516"}},{"literal":{"i64":"4517"}},{"literal":{"i64":"4518"}},{"literal":{"i64":"4519"}},{"literal":{"i64":"452"}},{"literal":{"i64":"4520"}},{"literal":{"i64":"4521"}},{"literal":{"i64":"4522"}},{"literal":{"i64":"4523"}},{"literal":{"i64":"4524"}},{"literal":{"i64":"4525"}},{"literal":{"i64":"4526"}},{"literal":{"i64":"4527"}},{"literal":{"i64":"4528"}},{"literal":{"i64":"4529"}},{"literal":{"i64":"453"}},{"literal":{"i64":"4530"}},{"literal":{"i64":"4531"}},{"literal":{"i64":"4532"}},{"literal":{"i64":"4533"}},{"literal":{"i64":"4534"}},{"literal":{"i64":"4535"}},{"literal":{"i64":"4536"}},{"literal":{"i64":"4537"}},{"literal":{"i64":"4538"}},{"literal":{"i64":"4539"}},{"literal":{"i64":"454"}},{"literal":{"i64":"4540"}},{"literal":{"i64":"4541"}},{"literal":{"i64":"4542"}},{"literal":{"i64":"4543"}},{"literal":{"i64":"4544"}},{"literal":{"i64":"4545"}},{"literal":{"i64":"4546"}},{"literal":{"i64":"4547"}},{"literal":{"i64":"4548"}},{"literal":{"i64":"4549"}},{"literal":{"i64":"455"}},{"literal":{"i64":"4550"}},{"literal":{"i64":"4551"}},{"literal":{"i64":"4552"}},{"literal":{"i64":"4553"}},{"literal":{"i64":"4554"}},{"literal":{"i64":"4555"}},{"literal":{"i64":"4556"}},{"literal":{"i64":"4557"}},{"literal":{"i64":"4558"}},{"literal":{"i64":"4559"}},{"literal":{"i64":"456"}},{"literal":{"i64":"4560"}},{"literal":{"i64":"4561"}},{"literal":{"i64":"4562"}},{"literal":{"i64":"4563"}},{"literal":{"i64":"4564"}},{"literal":{"i64":"4565"}},{"literal":{"i64":"4566"}},{"literal":{"i64":"4567"}},{"literal":{"i64":"4568"}},{"literal":{"i64":"4569"}},{"literal":{"i64":"457"}},{"literal":{"i64":"4570"}},{"literal":{"i64":"4571"}},{"literal":{"i64":"4572"}},{"literal":{"i64":"4573"}},{"literal":{"i64":"4574"}},{"literal":{"i64":"4575"}},{"literal":{"i64":"4576"}},{"literal":{"i64":"4577"}},{"literal":{"i64":"4578"}},{"literal":{"i64":"4579"}},{"literal":{"i64":"458"}},{"literal":{"i64":"4580"}},{"literal":{"i64":"4581"}},{"literal":{"i64":"4582"}},{"literal":{"i64":"4583"}},{"literal":{"i64":"4584"}},{"literal":{"i64":"4585"}},{"literal":{"i64":"4586"}},{"literal":{"i64":"4587"}},{"literal":{"i64":"4588"}},{"literal":{"i64":"4589"}},{"literal":{"i64":"459"}},{"literal":{"i64":"4590"}},{"literal":{"i64":"4591"}},{"literal":{"i64":"4592"}},{"literal":{"i64":"4593"}},{"literal":{"i64":"4594"}},{"literal":{"i64":"4595"}},{"literal":{"i64":"4596"}},{"literal":{"i64":"4597"}},{"literal":{"i64":"4598"}},{"literal":{"i64":"4599"}},{"literal":{"i64":"46"}},{"literal":{"i64":"460"}},{"literal":{"i64":"4600"}},{"literal":{"i64":"4601"}},{"literal":{"i64":"4602"}},{"literal":{"i64":"4603"}},{"literal":{"i64":"4604"}},{"literal":{"i64":"4605"}},{"literal":{"i64":"4606"}},{"literal":{"i64":"4607"}},{"literal":{"i64":"4608"}},{"literal":{"i64":"4609"}},{"literal":{"i64":"461"}},{"literal":{"i64":"4610"}},{"literal":{"i64":"4611"}},{"literal":{"i64":"4612"}},{"literal":{"i64":"4613"}},{"literal":{"i64":"4614"}},{"literal":{"i64":"4615"}},{"literal":{"i64":"4616"}},{"literal":{"i64":"4617"}},{"literal":{"i64":"4618"}},{"literal":{"i64":"4619"}},{"literal":{"i64":"462"}},{"literal":{"i64":"4620"}},{"literal":{"i64":"4621"}},{"literal":{"i64":"4622"}},{"literal":{"i64":"4623"}},{"literal":{"i64":"4624"}},{"literal":{"i64":"4625"}},{"literal":{"i64":"4626"}},{"literal":{"i64":"4627"}},{"literal":{"i64":"4628"}},{"literal":{"i64":"4629"}},{"literal":{"i64":"463"}},{"literal":{"i64":"4630"}},{"literal":{"i64":"4631"}},{"literal":{"i64":"4632"}},{"literal":{"i64":"4633"}},{"literal":{"i64":"4634"}},{"literal":{"i64":"4635"}},{"literal":{"i64":"4636"}},{"literal":{"i64":"4637"}},{"literal":{"i64":"4638"}},{"literal":{"i64":"4639"}},{"literal":{"i64":"464"}},{"literal":{"i64":"4640"}},{"literal":{"i64":"4641"}},{"literal":{"i64":"4642"}},{"literal":{"i64":"4643"}},{"literal":{"i64":"4644"}},{"literal":{"i64":"4645"}},{"literal":{"i64":"4646"}},{"literal":{"i64":"4647"}},{"literal":{"i64":"4648"}},{"literal":{"i64":"4649"}},{"literal":{"i64":"465"}},{"literal":{"i64":"4650"}},{"literal":{"i64":"4651"}},{"literal":{"i64":"4652"}},{"literal":{"i64":"4653"}},{"literal":{"i64":"4654"}},{"literal":{"i64":"4655"}},{"literal":{"i64":"4656"}},{"literal":{"i64":"4657"}},{"literal":{"i64":"4658"}},{"literal":{"i64":"4659"}},{"literal":{"i64":"466"}},{"literal":{"i64":"4660"}},{"literal":{"i64":"4661"}},{"literal":{"i64":"4662"}},{"literal":{"i64":"4663"}},{"literal":{"i64":"4664"}},{"literal":{"i64":"4665"}},{"literal":{"i64":"4666"}},{"literal":{"i64":"4667"}},{"literal":{"i64":"4668"}},{"literal":{"i64":"4669"}},{"literal":{"i64":"467"}},{"literal":{"i64":"4670"}},{"literal":{"i64":"4671"}},{"literal":{"i64":"4672"}},{"literal":{"i64":"4673"}},{"literal":{"i64":"4674"}},{"literal":{"i64":"4675"}},{"literal":{"i64":"4676"}},{"literal":{"i64":"4677"}},{"literal":{"i64":"4678"}},{"literal":{"i64":"4679"}},{"literal":{"i64":"468"}},{"literal":{"i64":"4680"}},{"literal":{"i64":"4681"}},{"literal":{"i64":"4682"}},{"literal":{"i64":"4683"}},{"literal":{"i64":"4684"}},{"literal":{"i64":"4685"}},{"literal":{"i64":"4686"}},{"literal":{"i64":"4687"}},{"literal":{"i64":"4688"}},{"literal":{"i64":"4689"}},{"literal":{"i64":"469"}},{"literal":{"i64":"4690"}},{"literal":{"i64":"4691"}},{"literal":{"i64":"4692"}},{"literal":{"i64":"4693"}},{"literal":{"i64":"4694"}},{"literal":{"i64":"4695"}},{"literal":{"i64":"4696"}},{"literal":{"i64":"4697"}},{"literal":{"i64":"4698"}},{"literal":{"i64":"4699"}},{"literal":{"i64":"47"}},{"literal":{"i64":"470"}},{"literal":{"i64":"4700"}},{"literal":{"i64":"4701"}},{"literal":{"i64":"4702"}},{"literal":{"i64":"4703"}},{"literal":{"i64":"4704"}},{"literal":{"i64":"4705"}},{"literal":{"i64":"4706"}},{"literal":{"i64":"4707"}},{"literal":{"i64":"4708"}},{"literal":{"i64":"4709"}},{"literal":{"i64":"471"}},{"literal":{"i64":"4710"}},{"literal":{"i64":"4711"}},{"literal":{"i64":"4712"}},{"literal":{"i64":"4713"}},{"literal":{"i64":"4714"}},{"literal":{"i64":"4715"}},{"literal":{"i64":"4716"}},{"literal":{"i64":"4717"}},{"literal":{"i64":"4718"}},{"literal":{"i64":"4719"}},{"literal":{"i64":"472"}},{"literal":{"i64":"4720"}},{"literal":{"i64":"4721"}},{"literal":{"i64":"4722"}},{"literal":{"i64":"4723"}},{"literal":{"i64":"4724"}},{"literal":{"i64":"4725"}},{"literal":{"i64":"4726"}},{"literal":{"i64":"4727"}},{"literal":{"i64":"4728"}},{"literal":{"i64":"4729"}},{"literal":{"i64":"473"}},{"literal":{"i64":"4730"}},{"literal":{"i64":"4731"}},{"literal":{"i64":"4732"}},{"literal":{"i64":"4733"}},{"literal":{"i64":"4734"}},{"literal":{"i64":"4735"}},{"literal":{"i64":"4736"}},{"literal":{"i64":"4737"}},{"literal":{"i64":"4738"}},{"literal":{"i64":"4739"}},{"literal":{"i64":"474"}},{"literal":{"i64":"4740"}},{"literal":{"i64":"4741"}},{"literal":{"i64":"4742"}},{"literal":{"i64":"4743"}},{"literal":{"i64":"4744"}},{"literal":{"i64":"4745"}},{"literal":{"i64":"4746"}},{"literal":{"i64":"4747"}},{"literal":{"i64":"4748"}},{"literal":{"i64":"4749"}},{"literal":{"i64":"475"}},{"literal":{"i64":"4750"}},{"literal":{"i64":"4751"}},{"literal":{"i64":"4752"}},{"literal":{"i64":"4753"}},{"literal":{"i64":"4754"}},{"literal":{"i64":"4755"}},{"literal":{"i64":"4756"}},{"literal":{"i64":"4757"}},{"literal":{"i64":"4758"}},{"literal":{"i64":"4759"}},{"literal":{"i64":"476"}},{"literal":{"i64":"4760"}},{"literal":{"i64":"4761"}},{"literal":{"i64":"4762"}},{"literal":{"i64":"4763"}},{"literal":{"i64":"4764"}},{"literal":{"i64":"4765"}},{"literal":{"i64":"4766"}},{"literal":{"i64":"4767"}},{"literal":{"i64":"4768"}},{"literal":{"i64":"4769"}},{"literal":{"i64":"477"}},{"literal":{"i64":"4770"}},{"literal":{"i64":"4771"}},{"literal":{"i64":"4772"}},{"literal":{"i64":"4773"}},{"literal":{"i64":"4774"}},{"literal":{"i64":"4775"}},{"literal":{"i64":"4776"}},{"literal":{"i64":"4777"}},{"literal":{"i64":"4778"}},{"literal":{"i64":"4779"}},{"literal":{"i64":"478"}},{"literal":{"i64":"4780"}},{"literal":{"i64":"4781"}},{"literal":{"i64":"4782"}},{"literal":{"i64":"4783"}},{"literal":{"i64":"4784"}},{"literal":{"i64":"4785"}},{"literal":{"i64":"4786"}},{"literal":{"i64":"4787"}},{"literal":{"i64":"4788"}},{"literal":{"i64":"4789"}},{"literal":{"i64":"479"}},{"literal":{"i64":"4790"}},{"literal":{"i64":"4791"}},{"literal":{"i64":"4792"}},{"literal":{"i64":"4793"}},{"literal":{"i64":"4794"}},{"literal":{"i64":"4795"}},{"literal":{"i64":"4796"}},{"literal":{"i64":"4797"}},{"literal":{"i64":"4798"}},{"literal":{"i64":"4799"}},{"literal":{"i64":"48"}},{"literal":{"i64":"480"}},{"literal":{"i64":"4800"}},{"literal":{"i64":"4801"}},{"literal":{"i64":"4802"}},{"literal":{"i64":"4803"}},{"literal":{"i64":"4804"}},{"literal":{"i64":"4805"}},{"literal":{"i64":"4806"}},{"literal":{"i64":"4807"}},{"literal":{"i64":"4808"}},{"literal":{"i64":"4809"}},{"literal":{"i64":"481"}},{"literal":{"i64":"4810"}},{"literal":{"i64":"4811"}},{"literal":{"i64":"4812"}},{"literal":{"i64":"4813"}},{"literal":{"i64":"4814"}},{"literal":{"i64":"4815"}},{"literal":{"i64":"4816"}},{"literal":{"i64":"4817"}},{"literal":{"i64":"4818"}},{"literal":{"i64":"4819"}},{"literal":{"i64":"482"}},{"literal":{"i64":"4820"}},{"literal":{"i64":"4821"}},{"literal":{"i64":"4822"}},{"literal":{"i64":"4823"}},{"literal":{"i64":"4824"}},{"literal":{"i64":"4825"}},{"literal":{"i64":"4826"}},{"literal":{"i64":"4827"}},{"literal":{"i64":"4828"}},{"literal":{"i64":"4829"}},{"literal":{"i64":"483"}},{"literal":{"i64":"4830"}},{"literal":{"i64":"4831"}},{"literal":{"i64":"4832"}},{"literal":{"i64":"4833"}},{"literal":{"i64":"4834"}},{"literal":{"i64":"4835"}},{"literal":{"i64":"4836"}},{"literal":{"i64":"4837"}},{"literal":{"i64":"4838"}},{"literal":{"i64":"4839"}},{"literal":{"i64":"484"}},{"literal":{"i64":"4840"}},{"literal":{"i64":"4841"}},{"literal":{"i64":"4842"}},{"literal":{"i64":"4843"}},{"literal":{"i64":"4844"}},{"literal":{"i64":"4845"}},{"literal":{"i64":"4846"}},{"literal":{"i64":"4847"}},{"literal":{"i64":"4848"}},{"literal":{"i64":"4849"}},{"literal":{"i64":"485"}},{"literal":{"i64":"4850"}},{"literal":{"i64":"4851"}},{"literal":{"i64":"4852"}},{"literal":{"i64":"4853"}},{"literal":{"i64":"4854"}},{"literal":{"i64":"4855"}},{"literal":{"i64":"4856"}},{"literal":{"i64":"4857"}},{"literal":{"i64":"4858"}},{"literal":{"i64":"4859"}},{"literal":{"i64":"486"}},{"literal":{"i64":"4860"}},{"literal":{"i64":"4861"}},{"literal":{"i64":"4862"}},{"literal":{"i64":"4863"}},{"literal":{"i64":"4864"}},{"literal":{"i64":"4865"}},{"literal":{"i64":"4866"}},{"literal":{"i64":"4867"}},{"literal":{"i64":"4868"}},{"literal":{"i64":"4869"}},{"literal":{"i64":"487"}},{"literal":{"i64":"4870"}},{"literal":{"i64":"4871"}},{"literal":{"i64":"4872"}},{"literal":{"i64":"4873"}},{"literal":{"i64":"4874"}},{"literal":{"i64":"4875"}},{"literal":{"i64":"4876"}},{"literal":{"i64":"4877"}},{"literal":{"i64":"4878"}},{"literal":{"i64":"4879"}},{"literal":{"i64":"488"}},{"literal":{"i64":"4880"}},{"literal":{"i64":"4881"}},{"literal":{"i64":"4882"}},{"literal":{"i64":"4883"}},{"literal":{"i64":"4884"}},{"literal":{"i64":"4885"}},{"literal":{"i64":"4886"}},{"literal":{"i64":"4887"}},{"literal":{"i64":"4888"}},{"literal":{"i64":"4889"}},{"literal":{"i64":"489"}},{"literal":{"i64":"4890"}},{"literal":{"i64":"4891"}},{"literal":{"i64":"4892"}},{"literal":{"i64":"4893"}},{"literal":{"i64":"4894"}},{"literal":{"i64":"4895"}},{"literal":{"i64":"4896"}},{"literal":{"i64":"4897"}},{"literal":{"i64":"4898"}},{"literal":{"i64":"4899"}},{"literal":{"i64":"49"}},{"literal":{"i64":"490"}},{"literal":{"i64":"4900"}},{"literal":{"i64":"4901"}},{"literal":{"i64":"4902"}},{"literal":{"i64":"4903"}},{"literal":{"i64":"4904"}},{"literal":{"i64":"4905"}},{"literal":{"i64":"4906"}},{"literal":{"i64":"4907"}},{"literal":{"i64":"4908"}},{"literal":{"i64":"4909"}},{"literal":{"i64":"491"}},{"literal":{"i64":"4910"}},{"literal":{"i64":"4911"}},{"literal":{"i64":"4912"}},{"literal":{"i64":"4913"}},{"literal":{"i64":"4914"}},{"literal":{"i64":"4915"}},{"literal":{"i64":"4916"}},{"literal":{"i64":"4917"}},{"literal":{"i64":"4918"}},{"literal":{"i64":"4919"}},{"literal":{"i64":"492"}},{"literal":{"i64":"4920"}},{"literal":{"i64":"4921"}},{"literal":{"i64":"4922"}},{"literal":{"i64":"4923"}},{"literal":{"i64":"4924"}},{"literal":{"i64":"4925"}},{"literal":{"i64":"4926"}},{"literal":{"i64":"4927"}},{"literal":{"i64":"4928"}},{"literal":{"i64":"4929"}},{"literal":{"i64":"493"}},{"literal":{"i64":"4930"}},{"literal":{"i64":"4931"}},{"literal":{"i64":"4932"}},{"literal":{"i64":"4933"}},{"literal":{"i64":"4934"}},{"literal":{"i64":"4935"}},{"literal":{"i64":"4936"}},{"literal":{"i64":"4937"}},{"literal":{"i64":"4938"}},{"literal":{"i64":"4939"}},{"literal":{"i64":"494"}},{"literal":{"i64":"4940"}},{"literal":{"i64":"4941"}},{"literal":{"i64":"4942"}},{"literal":{"i64":"4943"}},{"literal":{"i64":"4944"}},{"literal":{"i64":"4945"}},{"literal":{"i64":"4946"}},{"literal":{"i64":"4947"}},{"literal":{"i64":"4948"}},{"literal":{"i64":"4949"}},{"literal":{"i64":"495"}},{"literal":{"i64":"4950"}},{"literal":{"i64":"4951"}},{"literal":{"i64":"4952"}},{"literal":{"i64":"4953"}},{"literal":{"i64":"4954"}},{"literal":{"i64":"4955"}},{"literal":{"i64":"4956"}},{"literal":{"i64":"4957"}},{"literal":{"i64":"4958"}},{"literal":{"i64":"4959"}},{"literal":{"i64":"496"}},{"literal":{"i64":"4960"}},{"literal":{"i64":"4961"}},{"literal":{"i64":"4962"}},{"literal":{"i64":"4963"}},{"literal":{"i64":"4964"}},{"literal":{"i64":"4965"}},{"literal":{"i64":"4966"}},{"literal":{"i64":"4967"}},{"literal":{"i64":"4968"}},{"literal":{"i64":"4969"}},{"literal":{"i64":"497"}},{"literal":{"i64":"4970"}},{"literal":{"i64":"4971"}},{"literal":{"i64":"4972"}},{"literal":{"i64":"4973"}},{"literal":{"i64":"4974"}},{"literal":{"i64":"4975"}},{"literal":{"i64":"4976"}},{"literal":{"i64":"4977"}},{"literal":{"i64":"4978"}},{"literal":{"i64":"4979"}},{"literal":{"i64":"498"}},{"literal":{"i64":"4980"}},{"literal":{"i64":"4981"}},{"literal":{"i64":"4982"}},{"literal":{"i64":"4983"}},{"literal":{"i64":"4984"}},{"literal":{"i64":"4985"}},{"literal":{"i64":"4986"}},{"literal":{"i64":"4987"}},{"literal":{"i64":"4988"}},{"literal":{"i64":"4989"}},{"literal":{"i64":"499"}},{"literal":{"i64":"4990"}},{"literal":{"i64":"4991"}},{"literal":{"i64":"4992"}},{"literal":{"i64":"4993"}},{"literal":{"i64":"4994"}},{"literal":{"i64":"4995"}},{"literal":{"i64":"4996"}},{"literal":{"i64":"4997"}},{"literal":{"i64":"4998"}},{"literal":{"i64":"4999"}},{"literal":{"i64":"5"}},{"literal":{"i64":"50"}},{"literal":{"i64":"500"}},{"literal":{"i64":"5000"}},{"literal":{"i64":"5001"}},{"literal":{"i64":"5002"}},{"literal":{"i64":"5003"}},{"literal":{"i64":"5004"}},{"literal":{"i64":"5005"}},{"literal":{"i64":"5006"}},{"literal":{"i64":"5007"}},{"literal":{"i64":"5008"}},{"literal":{"i64":"5009"}},{"literal":{"i64":"501"}},{"literal":{"i64":"5010"}},{"literal":{"i64":"5011"}},{"literal":{"i64":"5012"}},{"literal":{"i64":"5013"}},{"literal":{"i64":"5014"}},{"literal":{"i64":"5015"}},{"literal":{"i64":"5016"}},{"literal":{"i64":"5017"}},{"literal":{"i64":"5018"}},{"literal":{"i64":"5019"}},{"literal":{"i64":"502"}},{"literal":{"i64":"5020"}},{"literal":{"i64":"5021"}},{"literal":{"i64":"5022"}},{"literal":{"i64":"5023"}},{"literal":{"i64":"5024"}},{"literal":{"i64":"5025"}},{"literal":{"i64":"5026"}},{"literal":{"i64":"5027"}},{"literal":{"i64":"5028"}},{"literal":{"i64":"5029"}},{"literal":{"i64":"503"}},{"literal":{"i64":"5030"}},{"literal":{"i64":"5031"}},{"literal":{"i64":"5032"}},{"literal":{"i64":"5033"}},{"literal":{"i64":"5034"}},{"literal":{"i64":"5035"}},{"literal":{"i64":"5036"}},{"literal":{"i64":"5037"}},{"literal":{"i64":"5038"}},{"literal":{"i64":"5039"}},{"literal":{"i64":"504"}},{"literal":{"i64":"5040"}},{"literal":{"i64":"5041"}},{"literal":{"i64":"5042"}},{"literal":{"i64":"5043"}},{"literal":{"i64":"5044"}},{"literal":{"i64":"5045"}},{"literal":{"i64":"5046"}},{"literal":{"i64":"5047"}},{"literal":{"i64":"5048"}},{"literal":{"i64":"5049"}},{"literal":{"i64":"505"}},{"literal":{"i64":"5050"}},{"literal":{"i64":"5051"}},{"literal":{"i64":"5052"}},{"literal":{"i64":"5053"}},{"literal":{"i64":"5054"}},{"literal":{"i64":"5055"}},{"literal":{"i64":"5056"}},{"literal":{"i64":"5057"}},{"literal":{"i64":"5058"}},{"literal":{"i64":"5059"}},{"literal":{"i64":"506"}},{"literal":{"i64":"5060"}},{"literal":{"i64":"5061"}},{"literal":{"i64":"5062"}},{"literal":{"i64":"5063"}},{"literal":{"i64":"5064"}},{"literal":{"i64":"5065"}},{"literal":{"i64":"5066"}},{"literal":{"i64":"5067"}},{"literal":{"i64":"5068"}},{"literal":{"i64":"5069"}},{"literal":{"i64":"507"}},{"literal":{"i64":"5070"}},{"literal":{"i64":"5071"}},{"literal":{"i64":"5072"}},{"literal":{"i64":"5073"}},{"literal":{"i64":"5074"}},{"literal":{"i64":"5075"}},{"literal":{"i64":"5076"}},{"literal":{"i64":"5077"}},{"literal":{"i64":"5078"}},{"literal":{"i64":"5079"}},{"literal":{"i64":"508"}},{"literal":{"i64":"5080"}},{"literal":{"i64":"5081"}},{"literal":{"i64":"5082"}},{"literal":{"i64":"5083"}},{"literal":{"i64":"5084"}},{"literal":{"i64":"5085"}},{"literal":{"i64":"5086"}},{"literal":{"i64":"5087"}},{"literal":{"i64":"5088"}},{"literal":{"i64":"5089"}},{"literal":{"i64":"509"}},{"literal":{"i64":"5090"}},{"literal":{"i64":"5091"}},{"literal":{"i64":"5092"}},{"literal":{"i64":"5093"}},{"literal":{"i64":"5094"}},{"literal":{"i64":"5095"}},{"literal":{"i64":"5096"}},{"literal":{"i64":"5097"}},{"literal":{"i64":"5098"}},{"literal":{"i64":"5099"}},{"literal":{"i64":"51"}},{"literal":{"i64":"510"}},{"literal":{"i64":"5100"}},{"literal":{"i64":"5101"}},{"literal":{"i64":"5102"}},{"literal":{"i64":"5103"}},{"literal":{"i64":"5104"}},{"literal":{"i64":"5105"}},{"literal":{"i64":"5106"}},{"literal":{"i64":"5107"}},{"literal":{"i64":"5108"}},{"literal":{"i64":"5109"}},{"literal":{"i64":"511"}},{"literal":{"i64":"5110"}},{"literal":{"i64":"5111"}},{"literal":{"i64":"5112"}},{"literal":{"i64":"5113"}},{"literal":{"i64":"5114"}},{"literal":{"i64":"5115"}},{"literal":{"i64":"5116"}},{"literal":{"i64":"5117"}},{"literal":{"i64":"5118"}},{"literal":{"i64":"5119"}},{"literal":{"i64":"512"}},{"literal":{"i64":"5120"}},{"literal":{"i64":"5121"}},{"literal":{"i64":"5122"}},{"literal":{"i64":"5123"}},{"literal":{"i64":"5124"}},{"literal":{"i64":"5125"}},{"literal":{"i64":"5126"}},{"literal":{"i64":"5127"}},{"literal":{"i64":"5128"}},{"literal":{"i64":"5129"}},{"literal":{"i64":"513"}},{"literal":{"i64":"5130"}},{"literal":{"i64":"5131"}},{"literal":{"i64":"5132"}},{"literal":{"i64":"5133"}},{"literal":{"i64":"5134"}},{"literal":{"i64":"5135"}},{"literal":{"i64":"5136"}},{"literal":{"i64":"5137"}},{"literal":{"i64":"5138"}},{"literal":{"i64":"5139"}},{"literal":{"i64":"514"}},{"literal":{"i64":"5140"}},{"literal":{"i64":"5141"}},{"literal":{"i64":"5142"}},{"literal":{"i64":"5143"}},{"literal":{"i64":"5144"}},{"literal":{"i64":"5145"}},{"literal":{"i64":"5146"}},{"literal":{"i64":"5147"}},{"literal":{"i64":"5148"}},{"literal":{"i64":"5149"}},{"literal":{"i64":"515"}},{"literal":{"i64":"5150"}},{"literal":{"i64":"5151"}},{"literal":{"i64":"5152"}},{"literal":{"i64":"5153"}},{"literal":{"i64":"5154"}},{"literal":{"i64":"5155"}},{"literal":{"i64":"5156"}},{"literal":{"i64":"5157"}},{"literal":{"i64":"5158"}},{"literal":{"i64":"5159"}},{"literal":{"i64":"516"}},{"literal":{"i64":"5160"}},{"literal":{"i64":"5161"}},{"literal":{"i64":"5162"}},{"literal":{"i64":"5163"}},{"literal":{"i64":"5164"}},{"literal":{"i64":"5165"}},{"literal":{"i64":"5166"}},{"literal":{"i64":"5167"}},{"literal":{"i64":"5168"}},{"literal":{"i64":"5169"}},{"literal":{"i64":"517"}},{"literal":{"i64":"5170"}},{"literal":{"i64":"5171"}},{"literal":{"i64":"5172"}},{"literal":{"i64":"5173"}},{"literal":{"i64":"5174"}},{"literal":{"i64":"5175"}},{"literal":{"i64":"5176"}},{"literal":{"i64":"5177"}},{"literal":{"i64":"5178"}},{"literal":{"i64":"5179"}},{"literal":{"i64":"518"}},{"literal":{"i64":"5180"}},{"literal":{"i64":"5181"}},{"literal":{"i64":"5182"}},{"literal":{"i64":"5183"}},{"literal":{"i64":"5184"}},{"literal":{"i64":"5185"}},{"literal":{"i64":"5186"}},{"literal":{"i64":"5187"}},{"literal":{"i64":"5188"}},{"literal":{"i64":"5189"}},{"literal":{"i64":"519"}},{"literal":{"i64":"5190"}},{"literal":{"i64":"5191"}},{"literal":{"i64":"5192"}},{"literal":{"i64":"5193"}},{"literal":{"i64":"5194"}},{"literal":{"i64":"5195"}},{"literal":{"i64":"5196"}},{"literal":{"i64":"5197"}},{"literal":{"i64":"5198"}},{"literal":{"i64":"5199"}},{"literal":{"i64":"52"}},{"literal":{"i64":"520"}},{"literal":{"i64":"5200"}},{"literal":{"i64":"5201"}},{"literal":{"i64":"5202"}},{"literal":{"i64":"5203"}},{"literal":{"i64":"5204"}},{"literal":{"i64":"5205"}},{"literal":{"i64":"5206"}},{"literal":{"i64":"5207"}},{"literal":{"i64":"5208"}},{"literal":{"i64":"5209"}},{"literal":{"i64":"521"}},{"literal":{"i64":"5210"}},{"literal":{"i64":"5211"}},{"literal":{"i64":"5212"}},{"literal":{"i64":"5213"}},{"literal":{"i64":"5214"}},{"literal":{"i64":"5215"}},{"literal":{"i64":"5216"}},{"literal":{"i64":"5217"}},{"literal":{"i64":"5218"}},{"literal":{"i64":"5219"}},{"literal":{"i64":"522"}},{"literal":{"i64":"5220"}},{"literal":{"i64":"5221"}},{"literal":{"i64":"5222"}},{"literal":{"i64":"5223"}},{"literal":{"i64":"5224"}},{"literal":{"i64":"5225"}},{"literal":{"i64":"5226"}},{"literal":{"i64":"5227"}},{"literal":{"i64":"5228"}},{"literal":{"i64":"5229"}},{"literal":{"i64":"523"}},{"literal":{"i64":"5230"}},{"literal":{"i64":"5231"}},{"literal":{"i64":"5232"}},{"literal":{"i64":"5233"}},{"literal":{"i64":"5234"}},{"literal":{"i64":"5235"}},{"literal":{"i64":"5236"}},{"literal":{"i64":"5237"}},{"literal":{"i64":"5238"}},{"literal":{"i64":"5239"}},{"literal":{"i64":"524"}},{"literal":{"i64":"5240"}},{"literal":{"i64":"5241"}},{"literal":{"i64":"5242"}},{"literal":{"i64":"5243"}},{"literal":{"i64":"5244"}},{"literal":{"i64":"5245"}},{"literal":{"i64":"5246"}},{"literal":{"i64":"5247"}},{"literal":{"i64":"5248"}},{"literal":{"i64":"5249"}},{"literal":{"i64":"525"}},{"literal":{"i64":"5250"}},{"literal":{"i64":"5251"}},{"literal":{"i64":"5252"}},{"literal":{"i64":"5253"}},{"literal":{"i64":"5254"}},{"literal":{"i64":"5255"}},{"literal":{"i64":"5256"}},{"literal":{"i64":"5257"}},{"literal":{"i64":"5258"}},{"literal":{"i64":"5259"}},{"literal":{"i64":"526"}},{"literal":{"i64":"5260"}},{"literal":{"i64":"5261"}},{"literal":{"i64":"5262"}},{"literal":{"i64":"5263"}},{"literal":{"i64":"5264"}},{"literal":{"i64":"5265"}},{"literal":{"i64":"5266"}},{"literal":{"i64":"5267"}},{"literal":{"i64":"5268"}},{"literal":{"i64":"5269"}},{"literal":{"i64":"527"}},{"literal":{"i64":"5270"}},{"literal":{"i64":"5271"}},{"literal":{"i64":"5272"}},{"literal":{"i64":"5273"}},{"literal":{"i64":"5274"}},{"literal":{"i64":"5275"}},{"literal":{"i64":"5276"}},{"literal":{"i64":"5277"}},{"literal":{"i64":"5278"}},{"literal":{"i64":"5279"}},{"literal":{"i64":"528"}},{"literal":{"i64":"5280"}},{"literal":{"i64":"5281"}},{"literal":{"i64":"5282"}},{"literal":{"i64":"5283"}},{"literal":{"i64":"5284"}},{"literal":{"i64":"5285"}},{"literal":{"i64":"5286"}},{"literal":{"i64":"5287"}},{"literal":{"i64":"5288"}},{"literal":{"i64":"5289"}},{"literal":{"i64":"529"}},{"literal":{"i64":"5290"}},{"literal":{"i64":"5291"}},{"literal":{"i64":"5292"}},{"literal":{"i64":"5293"}},{"literal":{"i64":"5294"}},{"literal":{"i64":"5295"}},{"literal":{"i64":"5296"}},{"literal":{"i64":"5297"}},{"literal":{"i64":"5298"}},{"literal":{"i64":"5299"}},{"literal":{"i64":"53"}},{"literal":{"i64":"530"}},{"literal":{"i64":"5300"}},{"literal":{"i64":"5301"}},{"literal":{"i64":"5302"}},{"literal":{"i64":"5303"}},{"literal":{"i64":"5304"}},{"literal":{"i64":"5305"}},{"literal":{"i64":"5306"}},{"literal":{"i64":"5307"}},{"literal":{"i64":"5308"}},{"literal":{"i64":"5309"}},{"literal":{"i64":"531"}},{"literal":{"i64":"5310"}},{"literal":{"i64":"5311"}},{"literal":{"i64":"5312"}},{"literal":{"i64":"5313"}},{"literal":{"i64":"5314"}},{"literal":{"i64":"5315"}},{"literal":{"i64":"5316"}},{"literal":{"i64":"5317"}},{"literal":{"i64":"5318"}},{"literal":{"i64":"5319"}},{"literal":{"i64":"532"}},{"literal":{"i64":"5320"}},{"literal":{"i64":"5321"}},{"literal":{"i64":"5322"}},{"literal":{"i64":"5323"}},{"literal":{"i64":"5324"}},{"literal":{"i64":"5325"}},{"literal":{"i64":"5326"}},{"literal":{"i64":"5327"}},{"literal":{"i64":"5328"}},{"literal":{"i64":"5329"}},{"literal":{"i64":"533"}},{"literal":{"i64":"5330"}},{"literal":{"i64":"5331"}},{"literal":{"i64":"5332"}},{"literal":{"i64":"5333"}},{"literal":{"i64":"5334"}},{"literal":{"i64":"5335"}},{"literal":{"i64":"5336"}},{"literal":{"i64":"5337"}},{"literal":{"i64":"5338"}},{"literal":{"i64":"5339"}},{"literal":{"i64":"534"}},{"literal":{"i64":"5340"}},{"literal":{"i64":"5341"}},{"literal":{"i64":"5342"}},{"literal":{"i64":"5343"}},{"literal":{"i64":"5344"}},{"literal":{"i64":"5345"}},{"literal":{"i64":"5346"}},{"literal":{"i64":"5347"}},{"literal":{"i64":"5348"}},{"literal":{"i64":"5349"}},{"literal":{"i64":"535"}},{"literal":{"i64":"5350"}},{"literal":{"i64":"5351"}},{"literal":{"i64":"5352"}},{"literal":{"i64":"5353"}},{"literal":{"i64":"5354"}},{"literal":{"i64":"5355"}},{"literal":{"i64":"5356"}},{"literal":{"i64":"5357"}},{"literal":{"i64":"5358"}},{"literal":{"i64":"5359"}},{"literal":{"i64":"536"}},{"literal":{"i64":"5360"}},{"literal":{"i64":"5361"}},{"literal":{"i64":"5362"}},{"literal":{"i64":"5363"}},{"literal":{"i64":"5364"}},{"literal":{"i64":"5365"}},{"literal":{"i64":"5366"}},{"literal":{"i64":"5367"}},{"literal":{"i64":"5368"}},{"literal":{"i64":"5369"}},{"literal":{"i64":"537"}},{"literal":{"i64":"5370"}},{"literal":{"i64":"5371"}},{"literal":{"i64":"5372"}},{"literal":{"i64":"5373"}},{"literal":{"i64":"5374"}},{"literal":{"i64":"5375"}},{"literal":{"i64":"5376"}},{"literal":{"i64":"5377"}},{"literal":{"i64":"5378"}},{"literal":{"i64":"5379"}},{"literal":{"i64":"538"}},{"literal":{"i64":"5380"}},{"literal":{"i64":"5381"}},{"literal":{"i64":"5382"}},{"literal":{"i64":"5383"}},{"literal":{"i64":"5384"}},{"literal":{"i64":"5385"}},{"literal":{"i64":"5386"}},{"literal":{"i64":"5387"}},{"literal":{"i64":"5388"}},{"literal":{"i64":"5389"}},{"literal":{"i64":"539"}},{"literal":{"i64":"5390"}},{"literal":{"i64":"5391"}},{"literal":{"i64":"5392"}},{"literal":{"i64":"5393"}},{"literal":{"i64":"5394"}},{"literal":{"i64":"5395"}},{"literal":{"i64":"5396"}},{"literal":{"i64":"5397"}},{"literal":{"i64":"5398"}},{"literal":{"i64":"5399"}},{"literal":{"i64":"54"}},{"literal":{"i64":"540"}},{"literal":{"i64":"5400"}},{"literal":{"i64":"5401"}},{"literal":{"i64":"5402"}},{"literal":{"i64":"5403"}},{"literal":{"i64":"5404"}},{"literal":{"i64":"5405"}},{"literal":{"i64":"5406"}},{"literal":{"i64":"5407"}},{"literal":{"i64":"5408"}},{"literal":{"i64":"5409"}},{"literal":{"i64":"541"}},{"literal":{"i64":"5410"}},{"literal":{"i64":"5411"}},{"literal":{"i64":"5412"}},{"literal":{"i64":"5413"}},{"literal":{"i64":"5414"}},{"literal":{"i64":"5415"}},{"literal":{"i64":"5416"}},{"literal":{"i64":"5417"}},{"literal":{"i64":"5418"}},{"literal":{"i64":"5419"}},{"literal":{"i64":"542"}},{"literal":{"i64":"5420"}},{"literal":{"i64":"5421"}},{"literal":{"i64":"5422"}},{"literal":{"i64":"5423"}},{"literal":{"i64":"5424"}},{"literal":{"i64":"5425"}},{"literal":{"i64":"5426"}},{"literal":{"i64":"5427"}},{"literal":{"i64":"5428"}},{"literal":{"i64":"5429"}},{"literal":{"i64":"543"}},{"literal":{"i64":"5430"}},{"literal":{"i64":"5431"}},{"literal":{"i64":"5432"}},{"literal":{"i64":"5433"}},{"literal":{"i64":"5434"}},{"literal":{"i64":"5435"}},{"literal":{"i64":"5436"}},{"literal":{"i64":"5437"}},{"literal":{"i64":"5438"}},{"literal":{"i64":"5439"}},{"literal":{"i64":"544"}},{"literal":{"i64":"5440"}},{"literal":{"i64":"5441"}},{"literal":{"i64":"5442"}},{"literal":{"i64":"5443"}},{"literal":{"i64":"5444"}},{"literal":{"i64":"5445"}},{"literal":{"i64":"5446"}},{"literal":{"i64":"5447"}},{"literal":{"i64":"5448"}},{"literal":{"i64":"5449"}},{"literal":{"i64":"545"}},{"literal":{"i64":"5450"}},{"literal":{"i64":"5451"}},{"literal":{"i64":"5452"}},{"literal":{"i64":"5453"}},{"literal":{"i64":"5454"}},{"literal":{"i64":"5455"}},{"literal":{"i64":"5456"}},{"literal":{"i64":"5457"}},{"literal":{"i64":"5458"}},{"literal":{"i64":"5459"}},{"literal":{"i64":"546"}},{"literal":{"i64":"5460"}},{"literal":{"i64":"5461"}},{"literal":{"i64":"5462"}},{"literal":{"i64":"5463"}},{"literal":{"i64":"5464"}},{"literal":{"i64":"5465"}},{"literal":{"i64":"5466"}},{"literal":{"i64":"5467"}},{"literal":{"i64":"5468"}},{"literal":{"i64":"5469"}},{"literal":{"i64":"547"}},{"literal":{"i64":"5470"}},{"literal":{"i64":"5471"}},{"literal":{"i64":"5472"}},{"literal":{"i64":"5473"}},{"literal":{"i64":"5474"}},{"literal":{"i64":"5475"}},{"literal":{"i64":"5476"}},{"literal":{"i64":"5477"}},{"literal":{"i64":"5478"}},{"literal":{"i64":"5479"}},{"literal":{"i64":"548"}},{"literal":{"i64":"5480"}},{"literal":{"i64":"5481"}},{"literal":{"i64":"5482"}},{"literal":{"i64":"5483"}},{"literal":{"i64":"5484"}},{"literal":{"i64":"5485"}},{"literal":{"i64":"5486"}},{"literal":{"i64":"5487"}},{"literal":{"i64":"5488"}},{"literal":{"i64":"5489"}},{"literal":{"i64":"549"}},{"literal":{"i64":"5490"}},{"literal":{"i64":"5491"}},{"literal":{"i64":"5492"}},{"literal":{"i64":"5493"}},{"literal":{"i64":"5494"}},{"literal":{"i64":"5495"}},{"literal":{"i64":"5496"}},{"literal":{"i64":"5497"}},{"literal":{"i64":"5498"}},{"literal":{"i64":"5499"}},{"literal":{"i64":"55"}},{"literal":{"i64":"550"}},{"literal":{"i64":"5500"}},{"literal":{"i64":"5501"}},{"literal":{"i64":"5502"}},{"literal":{"i64":"5503"}},{"literal":{"i64":"5504"}},{"literal":{"i64":"5505"}},{"literal":{"i64":"5506"}},{"literal":{"i64":"5507"}},{"literal":{"i64":"5508"}},{"literal":{"i64":"5509"}},{"literal":{"i64":"551"}},{"literal":{"i64":"5510"}},{"literal":{"i64":"5511"}},{"literal":{"i64":"5512"}},{"literal":{"i64":"5513"}},{"literal":{"i64":"5514"}},{"literal":{"i64":"5515"}},{"literal":{"i64":"5516"}},{"literal":{"i64":"5517"}},{"literal":{"i64":"5518"}},{"literal":{"i64":"5519"}},{"literal":{"i64":"552"}},{"literal":{"i64":"5520"}},{"literal":{"i64":"5521"}},{"literal":{"i64":"5522"}},{"literal":{"i64":"5523"}},{"literal":{"i64":"5524"}},{"literal":{"i64":"5525"}},{"literal":{"i64":"5526"}},{"literal":{"i64":"5527"}},{"literal":{"i64":"5528"}},{"literal":{"i64":"5529"}},{"literal":{"i64":"553"}},{"literal":{"i64":"5530"}},{"literal":{"i64":"5531"}},{"literal":{"i64":"5532"}},{"literal":{"i64":"5533"}},{"literal":{"i64":"5534"}},{"literal":{"i64":"5535"}},{"literal":{"i64":"5536"}},{"literal":{"i64":"5537"}},{"literal":{"i64":"5538"}},{"literal":{"i64":"5539"}},{"literal":{"i64":"554"}},{"literal":{"i64":"5540"}},{"literal":{"i64":"5541"}},{"literal":{"i64":"5542"}},{"literal":{"i64":"5543"}},{"literal":{"i64":"5544"}},{"literal":{"i64":"5545"}},{"literal":{"i64":"5546"}},{"literal":{"i64":"5547"}},{"literal":{"i64":"5548"}},{"literal":{"i64":"5549"}},{"literal":{"i64":"555"}},{"literal":{"i64":"5550"}},{"literal":{"i64":"5551"}},{"literal":{"i64":"5552"}},{"literal":{"i64":"5553"}},{"literal":{"i64":"5554"}},{"literal":{"i64":"5555"}},{"literal":{"i64":"5556"}},{"literal":{"i64":"5557"}},{"literal":{"i64":"5558"}},{"literal":{"i64":"5559"}},{"literal":{"i64":"556"}},{"literal":{"i64":"5560"}},{"literal":{"i64":"5561"}},{"literal":{"i64":"5562"}},{"literal":{"i64":"5563"}},{"literal":{"i64":"5564"}},{"literal":{"i64":"5565"}},{"literal":{"i64":"5566"}},{"literal":{"i64":"5567"}},{"literal":{"i64":"5568"}},{"literal":{"i64":"5569"}},{"literal":{"i64":"557"}},{"literal":{"i64":"5570"}},{"literal":{"i64":"5571"}},{"literal":{"i64":"5572"}},{"literal":{"i64":"5573"}},{"literal":{"i64":"5574"}},{"literal":{"i64":"5575"}},{"literal":{"i64":"5576"}},{"literal":{"i64":"5577"}},{"literal":{"i64":"5578"}},{"literal":{"i64":"5579"}},{"literal":{"i64":"558"}},{"literal":{"i64":"5580"}},{"literal":{"i64":"5581"}},{"literal":{"i64":"5582"}},{"literal":{"i64":"5583"}},{"literal":{"i64":"5584"}},{"literal":{"i64":"5585"}},{"literal":{"i64":"5586"}},{"literal":{"i64":"5587"}},{"literal":{"i64":"5588"}},{"literal":{"i64":"5589"}},{"literal":{"i64":"559"}},{"literal":{"i64":"5590"}},{"literal":{"i64":"5591"}},{"literal":{"i64":"5592"}},{"literal":{"i64":"5593"}},{"literal":{"i64":"5594"}},{"literal":{"i64":"5595"}},{"literal":{"i64":"5596"}},{"literal":{"i64":"5597"}},{"literal":{"i64":"5598"}},{"literal":{"i64":"5599"}},{"literal":{"i64":"56"}},{"literal":{"i64":"560"}},{"literal":{"i64":"5600"}},{"literal":{"i64":"5601"}},{"literal":{"i64":"5602"}},{"literal":{"i64":"5603"}},{"literal":{"i64":"5604"}},{"literal":{"i64":"5605"}},{"literal":{"i64":"5606"}},{"literal":{"i64":"5607"}},{"literal":{"i64":"5608"}},{"literal":{"i64":"5609"}},{"literal":{"i64":"561"}},{"literal":{"i64":"5610"}},{"literal":{"i64":"5611"}},{"literal":{"i64":"5612"}},{"literal":{"i64":"5613"}},{"literal":{"i64":"5614"}},{"literal":{"i64":"5615"}},{"literal":{"i64":"5616"}},{"literal":{"i64":"5617"}},{"literal":{"i64":"5618"}},{"literal":{"i64":"5619"}},{"literal":{"i64":"562"}},{"literal":{"i64":"5620"}},{"literal":{"i64":"5621"}},{"literal":{"i64":"5622"}},{"literal":{"i64":"5623"}},{"literal":{"i64":"5624"}},{"literal":{"i64":"5625"}},{"literal":{"i64":"5626"}},{"literal":{"i64":"5627"}},{"literal":{"i64":"5628"}},{"literal":{"i64":"5629"}},{"literal":{"i64":"563"}},{"literal":{"i64":"5630"}},{"literal":{"i64":"5631"}},{"literal":{"i64":"5632"}},{"literal":{"i64":"5633"}},{"literal":{"i64":"5634"}},{"literal":{"i64":"5635"}},{"literal":{"i64":"5636"}},{"literal":{"i64":"5637"}},{"literal":{"i64":"5638"}},{"literal":{"i64":"5639"}},{"literal":{"i64":"564"}},{"literal":{"i64":"5640"}},{"literal":{"i64":"5641"}},{"literal":{"i64":"5642"}},{"literal":{"i64":"5643"}},{"literal":{"i64":"5644"}},{"literal":{"i64":"5645"}},{"literal":{"i64":"5646"}},{"literal":{"i64":"5647"}},{"literal":{"i64":"5648"}},{"literal":{"i64":"5649"}},{"literal":{"i64":"565"}},{"literal":{"i64":"5650"}},{"literal":{"i64":"5651"}},{"literal":{"i64":"5652"}},{"literal":{"i64":"5653"}},{"literal":{"i64":"5654"}},{"literal":{"i64":"5655"}},{"literal":{"i64":"5656"}},{"literal":{"i64":"5657"}},{"literal":{"i64":"5658"}},{"literal":{"i64":"5659"}},{"literal":{"i64":"566"}},{"literal":{"i64":"5660"}},{"literal":{"i64":"5661"}},{"literal":{"i64":"5662"}},{"literal":{"i64":"5663"}},{"literal":{"i64":"5664"}},{"literal":{"i64":"5665"}},{"literal":{"i64":"5666"}},{"literal":{"i64":"5667"}},{"literal":{"i64":"5668"}},{"literal":{"i64":"5669"}},{"literal":{"i64":"567"}},{"literal":{"i64":"5670"}},{"literal":{"i64":"5671"}},{"literal":{"i64":"5672"}},{"literal":{"i64":"5673"}},{"literal":{"i64":"5674"}},{"literal":{"i64":"5675"}},{"literal":{"i64":"5676"}},{"literal":{"i64":"5677"}},{"literal":{"i64":"5678"}},{"literal":{"i64":"5679"}},{"literal":{"i64":"568"}},{"literal":{"i64":"5680"}},{"literal":{"i64":"5681"}},{"literal":{"i64":"5682"}},{"literal":{"i64":"5683"}},{"literal":{"i64":"5684"}},{"literal":{"i64":"5685"}},{"literal":{"i64":"5686"}},{"literal":{"i64":"5687"}},{"literal":{"i64":"5688"}},{"literal":{"i64":"5689"}},{"literal":{"i64":"569"}},{"literal":{"i64":"5690"}},{"literal":{"i64":"5691"}},{"literal":{"i64":"5692"}},{"literal":{"i64":"5693"}},{"literal":{"i64":"5694"}},{"literal":{"i64":"5695"}},{"literal":{"i64":"5696"}},{"literal":{"i64":"5697"}},{"literal":{"i64":"5698"}},{"literal":{"i64":"5699"}},{"literal":{"i64":"57"}},{"literal":{"i64":"570"}},{"literal":{"i64":"5700"}},{"literal":{"i64":"5701"}},{"literal":{"i64":"5702"}},{"literal":{"i64":"5703"}},{"literal":{"i64":"5704"}},{"literal":{"i64":"5705"}},{"literal":{"i64":"5706"}},{"literal":{"i64":"5707"}},{"literal":{"i64":"5708"}},{"literal":{"i64":"5709"}},{"literal":{"i64":"571"}},{"literal":{"i64":"5710"}},{"literal":{"i64":"5711"}},{"literal":{"i64":"5712"}},{"literal":{"i64":"5713"}},{"literal":{"i64":"5714"}},{"literal":{"i64":"5715"}},{"literal":{"i64":"5716"}},{"literal":{"i64":"5717"}},{"literal":{"i64":"5718"}},{"literal":{"i64":"5719"}},{"literal":{"i64":"572"}},{"literal":{"i64":"5720"}},{"literal":{"i64":"5721"}},{"literal":{"i64":"5722"}},{"literal":{"i64":"5723"}},{"literal":{"i64":"5724"}},{"literal":{"i64":"5725"}},{"literal":{"i64":"5726"}},{"literal":{"i64":"5727"}},{"literal":{"i64":"5728"}},{"literal":{"i64":"5729"}},{"literal":{"i64":"573"}},{"literal":{"i64":"5730"}},{"literal":{"i64":"5731"}},{"literal":{"i64":"5732"}},{"literal":{"i64":"5733"}},{"literal":{"i64":"5734"}},{"literal":{"i64":"5735"}},{"literal":{"i64":"5736"}},{"literal":{"i64":"5737"}},{"literal":{"i64":"5738"}},{"literal":{"i64":"5739"}},{"literal":{"i64":"574"}},{"literal":{"i64":"5740"}},{"literal":{"i64":"5741"}},{"literal":{"i64":"5742"}},{"literal":{"i64":"5743"}},{"literal":{"i64":"5744"}},{"literal":{"i64":"5745"}},{"literal":{"i64":"5746"}},{"literal":{"i64":"5747"}},{"literal":{"i64":"5748"}},{"literal":{"i64":"5749"}},{"literal":{"i64":"575"}},{"literal":{"i64":"5750"}},{"literal":{"i64":"5751"}},{"literal":{"i64":"5752"}},{"literal":{"i64":"5753"}},{"literal":{"i64":"5754"}},{"literal":{"i64":"5755"}},{"literal":{"i64":"5756"}},{"literal":{"i64":"5757"}},{"literal":{"i64":"5758"}},{"literal":{"i64":"5759"}},{"literal":{"i64":"576"}},{"literal":{"i64":"5760"}},{"literal":{"i64":"5761"}},{"literal":{"i64":"5762"}},{"literal":{"i64":"5763"}},{"literal":{"i64":"5764"}},{"literal":{"i64":"5765"}},{"literal":{"i64":"5766"}},{"literal":{"i64":"5767"}},{"literal":{"i64":"5768"}},{"literal":{"i64":"5769"}},{"literal":{"i64":"577"}},{"literal":{"i64":"5770"}},{"literal":{"i64":"5771"}},{"literal":{"i64":"5772"}},{"literal":{"i64":"5773"}},{"literal":{"i64":"5774"}},{"literal":{"i64":"5775"}},{"literal":{"i64":"5776"}},{"literal":{"i64":"5777"}},{"literal":{"i64":"5778"}},{"literal":{"i64":"5779"}},{"literal":{"i64":"578"}},{"literal":{"i64":"5780"}},{"literal":{"i64":"5781"}},{"literal":{"i64":"5782"}},{"literal":{"i64":"5783"}},{"literal":{"i64":"5784"}},{"literal":{"i64":"5785"}},{"literal":{"i64":"5786"}},{"literal":{"i64":"5787"}},{"literal":{"i64":"5788"}},{"literal":{"i64":"5789"}},{"literal":{"i64":"579"}},{"literal":{"i64":"5790"}},{"literal":{"i64":"5791"}},{"literal":{"i64":"5792"}},{"literal":{"i64":"5793"}},{"literal":{"i64":"5794"}},{"literal":{"i64":"5795"}},{"literal":{"i64":"5796"}},{"literal":{"i64":"5797"}},{"literal":{"i64":"5798"}},{"literal":{"i64":"5799"}},{"literal":{"i64":"58"}},{"literal":{"i64":"580"}},{"literal":{"i64":"5800"}},{"literal":{"i64":"5801"}},{"literal":{"i64":"5802"}},{"literal":{"i64":"5803"}},{"literal":{"i64":"5804"}},{"literal":{"i64":"5805"}},{"literal":{"i64":"5806"}},{"literal":{"i64":"5807"}},{"literal":{"i64":"5808"}},{"literal":{"i64":"5809"}},{"literal":{"i64":"581"}},{"literal":{"i64":"5810"}},{"literal":{"i64":"5811"}},{"literal":{"i64":"5812"}},{"literal":{"i64":"5813"}},{"literal":{"i64":"5814"}},{"literal":{"i64":"5815"}},{"literal":{"i64":"5816"}},{"literal":{"i64":"5817"}},{"literal":{"i64":"5818"}},{"literal":{"i64":"5819"}},{"literal":{"i64":"582"}},{"literal":{"i64":"5820"}},{"literal":{"i64":"5821"}},{"literal":{"i64":"5822"}},{"literal":{"i64":"5823"}},{"literal":{"i64":"5824"}},{"literal":{"i64":"5825"}},{"literal":{"i64":"5826"}},{"literal":{"i64":"5827"}},{"literal":{"i64":"5828"}},{"literal":{"i64":"5829"}},{"literal":{"i64":"583"}},{"literal":{"i64":"5830"}},{"literal":{"i64":"5831"}},{"literal":{"i64":"5832"}},{"literal":{"i64":"5833"}},{"literal":{"i64":"5834"}},{"literal":{"i64":"5835"}},{"literal":{"i64":"5836"}},{"literal":{"i64":"5837"}},{"literal":{"i64":"5838"}},{"literal":{"i64":"5839"}},{"literal":{"i64":"584"}},{"literal":{"i64":"5840"}},{"literal":{"i64":"5841"}},{"literal":{"i64":"5842"}},{"literal":{"i64":"5843"}},{"literal":{"i64":"5844"}},{"literal":{"i64":"5845"}},{"literal":{"i64":"5846"}},{"literal":{"i64":"5847"}},{"literal":{"i64":"5848"}},{"literal":{"i64":"5849"}},{"literal":{"i64":"585"}},{"literal":{"i64":"5850"}},{"literal":{"i64":"5851"}},{"literal":{"i64":"5852"}},{"literal":{"i64":"5853"}},{"literal":{"i64":"5854"}},{"literal":{"i64":"5855"}},{"literal":{"i64":"5856"}},{"literal":{"i64":"5857"}},{"literal":{"i64":"5858"}},{"literal":{"i64":"5859"}},{"literal":{"i64":"586"}},{"literal":{"i64":"5860"}},{"literal":{"i64":"5861"}},{"literal":{"i64":"5862"}},{"literal":{"i64":"5863"}},{"literal":{"i64":"5864"}},{"literal":{"i64":"5865"}},{"literal":{"i64":"5866"}},{"literal":{"i64":"5867"}},{"literal":{"i64":"5868"}},{"literal":{"i64":"5869"}},{"literal":{"i64":"587"}},{"literal":{"i64":"5870"}},{"literal":{"i64":"5871"}},{"literal":{"i64":"5872"}},{"literal":{"i64":"5873"}},{"literal":{"i64":"5874"}},{"literal":{"i64":"5875"}},{"literal":{"i64":"5876"}},{"literal":{"i64":"5877"}},{"literal":{"i64":"5878"}},{"literal":{"i64":"5879"}},{"literal":{"i64":"588"}},{"literal":{"i64":"5880"}},{"literal":{"i64":"5881"}},{"literal":{"i64":"5882"}},{"literal":{"i64":"5883"}},{"literal":{"i64":"5884"}},{"literal":{"i64":"5885"}},{"literal":{"i64":"5886"}},{"literal":{"i64":"5887"}},{"literal":{"i64":"5888"}},{"literal":{"i64":"5889"}},{"literal":{"i64":"589"}},{"literal":{"i64":"5890"}},{"literal":{"i64":"5891"}},{"literal":{"i64":"5892"}},{"literal":{"i64":"5893"}},{"literal":{"i64":"5894"}},{"literal":{"i64":"5895"}},{"literal":{"i64":"5896"}},{"literal":{"i64":"5897"}},{"literal":{"i64":"5898"}},{"literal":{"i64":"5899"}},{"literal":{"i64":"59"}},{"literal":{"i64":"590"}},{"literal":{"i64":"5900"}},{"literal":{"i64":"5901"}},{"literal":{"i64":"5902"}},{"literal":{"i64":"5903"}},{"literal":{"i64":"5904"}},{"literal":{"i64":"5905"}},{"literal":{"i64":"5906"}},{"literal":{"i64":"5907"}},{"literal":{"i64":"5908"}},{"literal":{"i64":"5909"}},{"literal":{"i64":"591"}},{"literal":{"i64":"5910"}},{"literal":{"i64":"5911"}},{"literal":{"i64":"5912"}},{"literal":{"i64":"5913"}},{"literal":{"i64":"5914"}},{"literal":{"i64":"5915"}},{"literal":{"i64":"5916"}},{"literal":{"i64":"5917"}},{"literal":{"i64":"5918"}},{"literal":{"i64":"5919"}},{"literal":{"i64":"592"}},{"literal":{"i64":"5920"}},{"literal":{"i64":"5921"}},{"literal":{"i64":"5922"}},{"literal":{"i64":"5923"}},{"literal":{"i64":"5924"}},{"literal":{"i64":"5925"}},{"literal":{"i64":"5926"}},{"literal":{"i64":"5927"}},{"literal":{"i64":"5928"}},{"literal":{"i64":"5929"}},{"literal":{"i64":"593"}},{"literal":{"i64":"5930"}},{"literal":{"i64":"5931"}},{"literal":{"i64":"5932"}},{"literal":{"i64":"5933"}},{"literal":{"i64":"5934"}},{"literal":{"i64":"5935"}},{"literal":{"i64":"5936"}},{"literal":{"i64":"5937"}},{"literal":{"i64":"5938"}},{"literal":{"i64":"5939"}},{"literal":{"i64":"594"}},{"literal":{"i64":"5940"}},{"literal":{"i64":"5941"}},{"literal":{"i64":"5942"}},{"literal":{"i64":"5943"}},{"literal":{"i64":"5944"}},{"literal":{"i64":"5945"}},{"literal":{"i64":"5946"}},{"literal":{"i64":"5947"}},{"literal":{"i64":"5948"}},{"literal":{"i64":"5949"}},{"literal":{"i64":"595"}},{"literal":{"i64":"5950"}},{"literal":{"i64":"5951"}},{"literal":{"i64":"5952"}},{"literal":{"i64":"5953"}},{"literal":{"i64":"5954"}},{"literal":{"i64":"5955"}},{"literal":{"i64":"5956"}},{"literal":{"i64":"5957"}},{"literal":{"i64":"5958"}},{"literal":{"i64":"5959"}},{"literal":{"i64":"596"}},{"literal":{"i64":"5960"}},{"literal":{"i64":"5961"}},{"literal":{"i64":"5962"}},{"literal":{"i64":"5963"}},{"literal":{"i64":"5964"}},{"literal":{"i64":"5965"}},{"literal":{"i64":"5966"}},{"literal":{"i64":"5967"}},{"literal":{"i64":"5968"}},{"literal":{"i64":"5969"}},{"literal":{"i64":"597"}},{"literal":{"i64":"5970"}},{"literal":{"i64":"5971"}},{"literal":{"i64":"5972"}},{"literal":{"i64":"5973"}},{"literal":{"i64":"5974"}},{"literal":{"i64":"5975"}},{"literal":{"i64":"5976"}},{"literal":{"i64":"5977"}},{"literal":{"i64":"5978"}},{"literal":{"i64":"5979"}},{"literal":{"i64":"598"}},{"literal":{"i64":"5980"}},{"literal":{"i64":"5981"}},{"literal":{"i64":"5982"}},{"literal":{"i64":"5983"}},{"literal":{"i64":"5984"}},{"literal":{"i64":"5985"}},{"literal":{"i64":"5986"}},{"literal":{"i64":"5987"}},{"literal":{"i64":"5988"}},{"literal":{"i64":"5989"}},{"literal":{"i64":"599"}},{"literal":{"i64":"5990"}},{"literal":{"i64":"5991"}},{"literal":{"i64":"5992"}},{"literal":{"i64":"5993"}},{"literal":{"i64":"5994"}},{"literal":{"i64":"5995"}},{"literal":{"i64":"5996"}},{"literal":{"i64":"5997"}},{"literal":{"i64":"5998"}},{"literal":{"i64":"5999"}},{"literal":{"i64":"6"}},{"literal":{"i64":"60"}},{"literal":{"i64":"600"}},{"literal":{"i64":"6000"}},{"literal":{"i64":"6001"}},{"literal":{"i64":"6002"}},{"literal":{"i64":"6003"}},{"literal":{"i64":"6004"}},{"literal":{"i64":"6005"}},{"literal":{"i64":"6006"}},{"literal":{"i64":"6007"}},{"literal":{"i64":"6008"}},{"literal":{"i64":"6009"}},{"literal":{"i64":"601"}},{"literal":{"i64":"6010"}},{"literal":{"i64":"6011"}},{"literal":{"i64":"6012"}},{"literal":{"i64":"6013"}},{"literal":{"i64":"6014"}},{"literal":{"i64":"6015"}},{"literal":{"i64":"6016"}},{"literal":{"i64":"6017"}},{"literal":{"i64":"6018"}},{"literal":{"i64":"6019"}},{"literal":{"i64":"602"}},{"literal":{"i64":"6020"}},{"literal":{"i64":"6021"}},{"literal":{"i64":"6022"}},{"literal":{"i64":"6023"}},{"literal":{"i64":"6024"}},{"literal":{"i64":"6025"}},{"literal":{"i64":"6026"}},{"literal":{"i64":"6027"}},{"literal":{"i64":"6028"}},{"literal":{"i64":"6029"}},{"literal":{"i64":"603"}},{"literal":{"i64":"6030"}},{"literal":{"i64":"6031"}},{"literal":{"i64":"6032"}},{"literal":{"i64":"6033"}},{"literal":{"i64":"6034"}},{"literal":{"i64":"6035"}},{"literal":{"i64":"6036"}},{"literal":{"i64":"6037"}},{"literal":{"i64":"6038"}},{"literal":{"i64":"6039"}},{"literal":{"i64":"604"}},{"literal":{"i64":"6040"}},{"literal":{"i64":"6041"}},{"literal":{"i64":"6042"}},{"literal":{"i64":"6043"}},{"literal":{"i64":"6044"}},{"literal":{"i64":"6045"}},{"literal":{"i64":"6046"}},{"literal":{"i64":"6047"}},{"literal":{"i64":"6048"}},{"literal":{"i64":"6049"}},{"literal":{"i64":"605"}},{"literal":{"i64":"6050"}},{"literal":{"i64":"6051"}},{"literal":{"i64":"6052"}},{"literal":{"i64":"6053"}},{"literal":{"i64":"6054"}},{"literal":{"i64":"6055"}},{"literal":{"i64":"6056"}},{"literal":{"i64":"6057"}},{"literal":{"i64":"6058"}},{"literal":{"i64":"6059"}},{"literal":{"i64":"606"}},{"literal":{"i64":"6060"}},{"literal":{"i64":"6061"}},{"literal":{"i64":"6062"}},{"literal":{"i64":"6063"}},{"literal":{"i64":"6064"}},{"literal":{"i64":"6065"}},{"literal":{"i64":"6066"}},{"literal":{"i64":"6067"}},{"literal":{"i64":"6068"}},{"literal":{"i64":"6069"}},{"literal":{"i64":"607"}},{"literal":{"i64":"6070"}},{"literal":{"i64":"6071"}},{"literal":{"i64":"6072"}},{"literal":{"i64":"6073"}},{"literal":{"i64":"6074"}},{"literal":{"i64":"6075"}},{"literal":{"i64":"6076"}},{"literal":{"i64":"6077"}},{"literal":{"i64":"6078"}},{"literal":{"i64":"6079"}},{"literal":{"i64":"608"}},{"literal":{"i64":"6080"}},{"literal":{"i64":"6081"}},{"literal":{"i64":"6082"}},{"literal":{"i64":"6083"}},{"literal":{"i64":"6084"}},{"literal":{"i64":"6085"}},{"literal":{"i64":"6086"}},{"literal":{"i64":"6087"}},{"literal":{"i64":"6088"}},{"literal":{"i64":"6089"}},{"literal":{"i64":"609"}},{"literal":{"i64":"6090"}},{"literal":{"i64":"6091"}},{"literal":{"i64":"6092"}},{"literal":{"i64":"6093"}},{"literal":{"i64":"6094"}},{"literal":{"i64":"6095"}},{"literal":{"i64":"6096"}},{"literal":{"i64":"6097"}},{"literal":{"i64":"6098"}},{"literal":{"i64":"6099"}},{"literal":{"i64":"61"}},{"literal":{"i64":"610"}},{"literal":{"i64":"6100"}},{"literal":{"i64":"6101"}},{"literal":{"i64":"6102"}},{"literal":{"i64":"6103"}},{"literal":{"i64":"6104"}},{"literal":{"i64":"6105"}},{"literal":{"i64":"6106"}},{"literal":{"i64":"6107"}},{"literal":{"i64":"6108"}},{"literal":{"i64":"6109"}},{"literal":{"i64":"611"}},{"literal":{"i64":"6110"}},{"literal":{"i64":"6111"}},{"literal":{"i64":"6112"}},{"literal":{"i64":"6113"}},{"literal":{"i64":"6114"}},{"literal":{"i64":"6115"}},{"literal":{"i64":"6116"}},{"literal":{"i64":"6117"}},{"literal":{"i64":"6118"}},{"literal":{"i64":"6119"}},{"literal":{"i64":"612"}},{"literal":{"i64":"6120"}},{"literal":{"i64":"6121"}},{"literal":{"i64":"6122"}},{"literal":{"i64":"6123"}},{"literal":{"i64":"6124"}},{"literal":{"i64":"6125"}},{"literal":{"i64":"6126"}},{"literal":{"i64":"6127"}},{"literal":{"i64":"6128"}},{"literal":{"i64":"6129"}},{"literal":{"i64":"613"}},{"literal":{"i64":"6130"}},{"literal":{"i64":"6131"}},{"literal":{"i64":"6132"}},{"literal":{"i64":"6133"}},{"literal":{"i64":"6134"}},{"literal":{"i64":"6135"}},{"literal":{"i64":"6136"}},{"literal":{"i64":"6137"}},{"literal":{"i64":"6138"}},{"literal":{"i64":"6139"}},{"literal":{"i64":"614"}},{"literal":{"i64":"6140"}},{"literal":{"i64":"6141"}},{"literal":{"i64":"6142"}},{"literal":{"i64":"6143"}},{"literal":{"i64":"6144"}},{"literal":{"i64":"6145"}},{"literal":{"i64":"6146"}},{"literal":{"i64":"6147"}},{"literal":{"i64":"6148"}},{"literal":{"i64":"6149"}},{"literal":{"i64":"615"}},{"literal":{"i64":"6150"}},{"literal":{"i64":"6151"}},{"literal":{"i64":"6152"}},{"literal":{"i64":"6153"}},{"literal":{"i64":"6154"}},{"literal":{"i64":"6155"}},{"literal":{"i64":"6156"}},{"literal":{"i64":"6157"}},{"literal":{"i64":"6158"}},{"literal":{"i64":"6159"}},{"literal":{"i64":"616"}},{"literal":{"i64":"6160"}},{"literal":{"i64":"6161"}},{"literal":{"i64":"6162"}},{"literal":{"i64":"6163"}},{"literal":{"i64":"6164"}},{"literal":{"i64":"6165"}},{"literal":{"i64":"6166"}},{"literal":{"i64":"6167"}},{"literal":{"i64":"6168"}},{"literal":{"i64":"6169"}},{"literal":{"i64":"617"}},{"literal":{"i64":"6170"}},{"literal":{"i64":"6171"}},{"literal":{"i64":"6172"}},{"literal":{"i64":"6173"}},{"literal":{"i64":"6174"}},{"literal":{"i64":"6175"}},{"literal":{"i64":"6176"}},{"literal":{"i64":"6177"}},{"literal":{"i64":"6178"}},{"literal":{"i64":"6179"}},{"literal":{"i64":"618"}},{"literal":{"i64":"6180"}},{"literal":{"i64":"6181"}},{"literal":{"i64":"6182"}},{"literal":{"i64":"6183"}},{"literal":{"i64":"6184"}},{"literal":{"i64":"6185"}},{"literal":{"i64":"6186"}},{"literal":{"i64":"6187"}},{"literal":{"i64":"6188"}},{"literal":{"i64":"6189"}},{"literal":{"i64":"619"}},{"literal":{"i64":"6190"}},{"literal":{"i64":"6191"}},{"literal":{"i64":"6192"}},{"literal":{"i64":"6193"}},{"literal":{"i64":"6194"}},{"literal":{"i64":"6195"}},{"literal":{"i64":"6196"}},{"literal":{"i64":"6197"}},{"literal":{"i64":"6198"}},{"literal":{"i64":"6199"}},{"literal":{"i64":"62"}},{"literal":{"i64":"620"}},{"literal":{"i64":"6200"}},{"literal":{"i64":"6201"}},{"literal":{"i64":"6202"}},{"literal":{"i64":"6203"}},{"literal":{"i64":"6204"}},{"literal":{"i64":"6205"}},{"literal":{"i64":"6206"}},{"literal":{"i64":"6207"}},{"literal":{"i64":"6208"}},{"literal":{"i64":"6209"}},{"literal":{"i64":"621"}},{"literal":{"i64":"6210"}},{"literal":{"i64":"6211"}},{"literal":{"i64":"6212"}},{"literal":{"i64":"6213"}},{"literal":{"i64":"6214"}},{"literal":{"i64":"6215"}},{"literal":{"i64":"6216"}},{"literal":{"i64":"6217"}},{"literal":{"i64":"6218"}},{"literal":{"i64":"6219"}},{"literal":{"i64":"622"}},{"literal":{"i64":"6220"}},{"literal":{"i64":"6221"}},{"literal":{"i64":"6222"}},{"literal":{"i64":"6223"}},{"literal":{"i64":"6224"}},{"literal":{"i64":"6225"}},{"literal":{"i64":"6226"}},{"literal":{"i64":"6227"}},{"literal":{"i64":"6228"}},{"literal":{"i64":"6229"}},{"literal":{"i64":"623"}},{"literal":{"i64":"6230"}},{"literal":{"i64":"6231"}},{"literal":{"i64":"6232"}},{"literal":{"i64":"6233"}},{"literal":{"i64":"6234"}},{"literal":{"i64":"6235"}},{"literal":{"i64":"6236"}},{"literal":{"i64":"6237"}},{"literal":{"i64":"6238"}},{"literal":{"i64":"6239"}},{"literal":{"i64":"624"}},{"literal":{"i64":"6240"}},{"literal":{"i64":"6241"}},{"literal":{"i64":"6242"}},{"literal":{"i64":"6243"}},{"literal":{"i64":"6244"}},{"literal":{"i64":"6245"}},{"literal":{"i64":"6246"}},{"literal":{"i64":"6247"}},{"literal":{"i64":"6248"}},{"literal":{"i64":"6249"}},{"literal":{"i64":"625"}},{"literal":{"i64":"6250"}},{"literal":{"i64":"6251"}},{"literal":{"i64":"6252"}},{"literal":{"i64":"6253"}},{"literal":{"i64":"6254"}},{"literal":{"i64":"6255"}},{"literal":{"i64":"6256"}},{"literal":{"i64":"6257"}},{"literal":{"i64":"6258"}},{"literal":{"i64":"6259"}},{"literal":{"i64":"626"}},{"literal":{"i64":"6260"}},{"literal":{"i64":"6261"}},{"literal":{"i64":"6262"}},{"literal":{"i64":"6263"}},{"literal":{"i64":"6264"}},{"literal":{"i64":"6265"}},{"literal":{"i64":"6266"}},{"literal":{"i64":"6267"}},{"literal":{"i64":"6268"}},{"literal":{"i64":"6269"}},{"literal":{"i64":"627"}},{"literal":{"i64":"6270"}},{"literal":{"i64":"6271"}},{"literal":{"i64":"6272"}},{"literal":{"i64":"6273"}},{"literal":{"i64":"6274"}},{"literal":{"i64":"6275"}},{"literal":{"i64":"6276"}},{"literal":{"i64":"6277"}},{"literal":{"i64":"6278"}},{"literal":{"i64":"6279"}},{"literal":{"i64":"628"}},{"literal":{"i64":"6280"}},{"literal":{"i64":"6281"}},{"literal":{"i64":"6282"}},{"literal":{"i64":"6283"}},{"literal":{"i64":"6284"}},{"literal":{"i64":"6285"}},{"literal":{"i64":"6286"}},{"literal":{"i64":"6287"}},{"literal":{"i64":"6288"}},{"literal":{"i64":"6289"}},{"literal":{"i64":"629"}},{"literal":{"i64":"6290"}},{"literal":{"i64":"6291"}},{"literal":{"i64":"6292"}},{"literal":{"i64":"6293"}},{"literal":{"i64":"6294"}},{"literal":{"i64":"6295"}},{"literal":{"i64":"6296"}},{"literal":{"i64":"6297"}},{"literal":{"i64":"6298"}},{"literal":{"i64":"6299"}},{"literal":{"i64":"63"}},{"literal":{"i64":"630"}},{"literal":{"i64":"6300"}},{"literal":{"i64":"6301"}},{"literal":{"i64":"6302"}},{"literal":{"i64":"6303"}},{"literal":{"i64":"6304"}},{"literal":{"i64":"6305"}},{"literal":{"i64":"6306"}},{"literal":{"i64":"6307"}},{"literal":{"i64":"6308"}},{"literal":{"i64":"6309"}},{"literal":{"i64":"631"}},{"literal":{"i64":"6310"}},{"literal":{"i64":"6311"}},{"literal":{"i64":"6312"}},{"literal":{"i64":"6313"}},{"literal":{"i64":"6314"}},{"literal":{"i64":"6315"}},{"literal":{"i64":"6316"}},{"literal":{"i64":"6317"}},{"literal":{"i64":"6318"}},{"literal":{"i64":"6319"}},{"literal":{"i64":"632"}},{"literal":{"i64":"6320"}},{"literal":{"i64":"6321"}},{"literal":{"i64":"6322"}},{"literal":{"i64":"6323"}},{"literal":{"i64":"6324"}},{"literal":{"i64":"6325"}},{"literal":{"i64":"6326"}},{"literal":{"i64":"6327"}},{"literal":{"i64":"6328"}},{"literal":{"i64":"6329"}},{"literal":{"i64":"633"}},{"literal":{"i64":"6330"}},{"literal":{"i64":"6331"}},{"literal":{"i64":"6332"}},{"literal":{"i64":"6333"}},{"literal":{"i64":"6334"}},{"literal":{"i64":"6335"}},{"literal":{"i64":"6336"}},{"literal":{"i64":"6337"}},{"literal":{"i64":"6338"}},{"literal":{"i64":"6339"}},{"literal":{"i64":"634"}},{"literal":{"i64":"6340"}},{"literal":{"i64":"6341"}},{"literal":{"i64":"6342"}},{"literal":{"i64":"6343"}},{"literal":{"i64":"6344"}},{"literal":{"i64":"6345"}},{"literal":{"i64":"6346"}},{"literal":{"i64":"6347"}},{"literal":{"i64":"6348"}},{"literal":{"i64":"6349"}},{"literal":{"i64":"635"}},{"literal":{"i64":"6350"}},{"literal":{"i64":"6351"}},{"literal":{"i64":"6352"}},{"literal":{"i64":"6353"}},{"literal":{"i64":"6354"}},{"literal":{"i64":"6355"}},{"literal":{"i64":"6356"}},{"literal":{"i64":"6357"}},{"literal":{"i64":"6358"}},{"literal":{"i64":"6359"}},{"literal":{"i64":"636"}},{"literal":{"i64":"6360"}},{"literal":{"i64":"6361"}},{"literal":{"i64":"6362"}},{"literal":{"i64":"6363"}},{"literal":{"i64":"6364"}},{"literal":{"i64":"6365"}},{"literal":{"i64":"6366"}},{"literal":{"i64":"6367"}},{"literal":{"i64":"6368"}},{"literal":{"i64":"6369"}},{"literal":{"i64":"637"}},{"literal":{"i64":"6370"}},{"literal":{"i64":"6371"}},{"literal":{"i64":"6372"}},{"literal":{"i64":"6373"}},{"literal":{"i64":"6374"}},{"literal":{"i64":"6375"}},{"literal":{"i64":"6376"}},{"literal":{"i64":"6377"}},{"literal":{"i64":"6378"}},{"literal":{"i64":"6379"}},{"literal":{"i64":"638"}},{"literal":{"i64":"6380"}},{"literal":{"i64":"6381"}},{"literal":{"i64":"6382"}},{"literal":{"i64":"6383"}},{"literal":{"i64":"6384"}},{"literal":{"i64":"6385"}},{"literal":{"i64":"6386"}},{"literal":{"i64":"6387"}},{"literal":{"i64":"6388"}},{"literal":{"i64":"6389"}},{"literal":{"i64":"639"}},{"literal":{"i64":"6390"}},{"literal":{"i64":"6391"}},{"literal":{"i64":"6392"}},{"literal":{"i64":"6393"}},{"literal":{"i64":"6394"}},{"literal":{"i64":"6395"}},{"literal":{"i64":"6396"}},{"literal":{"i64":"6397"}},{"literal":{"i64":"6398"}},{"literal":{"i64":"6399"}},{"literal":{"i64":"64"}},{"literal":{"i64":"640"}},{"literal":{"i64":"6400"}},{"literal":{"i64":"6401"}},{"literal":{"i64":"6402"}},{"literal":{"i64":"6403"}},{"literal":{"i64":"6404"}},{"literal":{"i64":"6405"}},{"literal":{"i64":"6406"}},{"literal":{"i64":"6407"}},{"literal":{"i64":"6408"}},{"literal":{"i64":"6409"}},{"literal":{"i64":"641"}},{"literal":{"i64":"6410"}},{"literal":{"i64":"6411"}},{"literal":{"i64":"6412"}},{"literal":{"i64":"6413"}},{"literal":{"i64":"6414"}},{"literal":{"i64":"6415"}},{"literal":{"i64":"6416"}},{"literal":{"i64":"6417"}},{"literal":{"i64":"6418"}},{"literal":{"i64":"6419"}},{"literal":{"i64":"642"}},{"literal":{"i64":"6420"}},{"literal":{"i64":"6421"}},{"literal":{"i64":"6422"}},{"literal":{"i64":"6423"}},{"literal":{"i64":"6424"}},{"literal":{"i64":"6425"}},{"literal":{"i64":"6426"}},{"literal":{"i64":"6427"}},{"literal":{"i64":"6428"}},{"literal":{"i64":"6429"}},{"literal":{"i64":"643"}},{"literal":{"i64":"6430"}},{"literal":{"i64":"6431"}},{"literal":{"i64":"6432"}},{"literal":{"i64":"6433"}},{"literal":{"i64":"6434"}},{"literal":{"i64":"6435"}},{"literal":{"i64":"6436"}},{"literal":{"i64":"6437"}},{"literal":{"i64":"6438"}},{"literal":{"i64":"6439"}},{"literal":{"i64":"644"}},{"literal":{"i64":"6440"}},{"literal":{"i64":"6441"}},{"literal":{"i64":"6442"}},{"literal":{"i64":"6443"}},{"literal":{"i64":"6444"}},{"literal":{"i64":"6445"}},{"literal":{"i64":"6446"}},{"literal":{"i64":"6447"}},{"literal":{"i64":"6448"}},{"literal":{"i64":"6449"}},{"literal":{"i64":"645"}},{"literal":{"i64":"6450"}},{"literal":{"i64":"6451"}},{"literal":{"i64":"6452"}},{"literal":{"i64":"6453"}},{"literal":{"i64":"6454"}},{"literal":{"i64":"6455"}},{"literal":{"i64":"6456"}},{"literal":{"i64":"6457"}},{"literal":{"i64":"6458"}},{"literal":{"i64":"6459"}},{"literal":{"i64":"646"}},{"literal":{"i64":"6460"}},{"literal":{"i64":"6461"}},{"literal":{"i64":"6462"}},{"literal":{"i64":"6463"}},{"literal":{"i64":"6464"}},{"literal":{"i64":"6465"}},{"literal":{"i64":"6466"}},{"literal":{"i64":"6467"}},{"literal":{"i64":"6468"}},{"literal":{"i64":"6469"}},{"literal":{"i64":"647"}},{"literal":{"i64":"6470"}},{"literal":{"i64":"6471"}},{"literal":{"i64":"6472"}},{"literal":{"i64":"6473"}},{"literal":{"i64":"6474"}},{"literal":{"i64":"6475"}},{"literal":{"i64":"6476"}},{"literal":{"i64":"6477"}},{"literal":{"i64":"6478"}},{"literal":{"i64":"6479"}},{"literal":{"i64":"648"}},{"literal":{"i64":"6480"}},{"literal":{"i64":"6481"}},{"literal":{"i64":"6482"}},{"literal":{"i64":"6483"}},{"literal":{"i64":"6484"}},{"literal":{"i64":"6485"}},{"literal":{"i64":"6486"}},{"literal":{"i64":"6487"}},{"literal":{"i64":"6488"}},{"literal":{"i64":"6489"}},{"literal":{"i64":"649"}},{"literal":{"i64":"6490"}},{"literal":{"i64":"6491"}},{"literal":{"i64":"6492"}},{"literal":{"i64":"6493"}},{"literal":{"i64":"6494"}},{"literal":{"i64":"6495"}},{"literal":{"i64":"6496"}},{"literal":{"i64":"6497"}},{"literal":{"i64":"6498"}},{"literal":{"i64":"6499"}},{"literal":{"i64":"65"}},{"literal":{"i64":"650"}},{"literal":{"i64":"6500"}},{"literal":{"i64":"6501"}},{"literal":{"i64":"6502"}},{"literal":{"i64":"6503"}},{"literal":{"i64":"6504"}},{"literal":{"i64":"6505"}},{"literal":{"i64":"6506"}},{"literal":{"i64":"6507"}},{"literal":{"i64":"6508"}},{"literal":{"i64":"6509"}},{"literal":{"i64":"651"}},{"literal":{"i64":"6510"}},{"literal":{"i64":"6511"}},{"literal":{"i64":"6512"}},{"literal":{"i64":"6513"}},{"literal":{"i64":"6514"}},{"literal":{"i64":"6515"}},{"literal":{"i64":"6516"}},{"literal":{"i64":"6517"}},{"literal":{"i64":"6518"}},{"literal":{"i64":"6519"}},{"literal":{"i64":"652"}},{"literal":{"i64":"6520"}},{"literal":{"i64":"6521"}},{"literal":{"i64":"6522"}},{"literal":{"i64":"6523"}},{"literal":{"i64":"6524"}},{"literal":{"i64":"6525"}},{"literal":{"i64":"6526"}},{"literal":{"i64":"6527"}},{"literal":{"i64":"6528"}},{"literal":{"i64":"6529"}},{"literal":{"i64":"653"}},{"literal":{"i64":"6530"}},{"literal":{"i64":"6531"}},{"literal":{"i64":"6532"}},{"literal":{"i64":"6533"}},{"literal":{"i64":"6534"}},{"literal":{"i64":"6535"}},{"literal":{"i64":"6536"}},{"literal":{"i64":"6537"}},{"literal":{"i64":"6538"}},{"literal":{"i64":"6539"}},{"literal":{"i64":"654"}},{"literal":{"i64":"6540"}},{"literal":{"i64":"6541"}},{"literal":{"i64":"6542"}},{"literal":{"i64":"6543"}},{"literal":{"i64":"6544"}},{"literal":{"i64":"6545"}},{"literal":{"i64":"6546"}},{"literal":{"i64":"6547"}},{"literal":{"i64":"6548"}},{"literal":{"i64":"6549"}},{"literal":{"i64":"655"}},{"literal":{"i64":"6550"}},{"literal":{"i64":"6551"}},{"literal":{"i64":"6552"}},{"literal":{"i64":"6553"}},{"literal":{"i64":"6554"}},{"literal":{"i64":"6555"}},{"literal":{"i64":"6556"}},{"literal":{"i64":"6557"}},{"literal":{"i64":"6558"}},{"literal":{"i64":"6559"}},{"literal":{"i64":"656"}},{"literal":{"i64":"6560"}},{"literal":{"i64":"6561"}},{"literal":{"i64":"6562"}},{"literal":{"i64":"6563"}},{"literal":{"i64":"6564"}},{"literal":{"i64":"6565"}},{"literal":{"i64":"6566"}},{"literal":{"i64":"6567"}},{"literal":{"i64":"6568"}},{"literal":{"i64":"6569"}},{"literal":{"i64":"657"}},{"literal":{"i64":"6570"}},{"literal":{"i64":"6571"}},{"literal":{"i64":"6572"}},{"literal":{"i64":"6573"}},{"literal":{"i64":"6574"}},{"literal":{"i64":"6575"}},{"literal":{"i64":"6576"}},{"literal":{"i64":"6577"}},{"literal":{"i64":"6578"}},{"literal":{"i64":"6579"}},{"literal":{"i64":"658"}},{"literal":{"i64":"6580"}},{"literal":{"i64":"6581"}},{"literal":{"i64":"6582"}},{"literal":{"i64":"6583"}},{"literal":{"i64":"6584"}},{"literal":{"i64":"6585"}},{"literal":{"i64":"6586"}},{"literal":{"i64":"6587"}},{"literal":{"i64":"6588"}},{"literal":{"i64":"6589"}},{"literal":{"i64":"659"}},{"literal":{"i64":"6590"}},{"literal":{"i64":"6591"}},{"literal":{"i64":"6592"}},{"literal":{"i64":"6593"}},{"literal":{"i64":"6594"}},{"literal":{"i64":"6595"}},{"literal":{"i64":"6596"}},{"literal":{"i64":"6597"}},{"literal":{"i64":"6598"}},{"literal":{"i64":"6599"}},{"literal":{"i64":"66"}},{"literal":{"i64":"660"}},{"literal":{"i64":"6600"}},{"literal":{"i64":"6601"}},{"literal":{"i64":"6602"}},{"literal":{"i64":"6603"}},{"literal":{"i64":"6604"}},{"literal":{"i64":"6605"}},{"literal":{"i64":"6606"}},{"literal":{"i64":"6607"}},{"literal":{"i64":"6608"}},{"literal":{"i64":"6609"}},{"literal":{"i64":"661"}},{"literal":{"i64":"6610"}},{"literal":{"i64":"6611"}},{"literal":{"i64":"6612"}},{"literal":{"i64":"6613"}},{"literal":{"i64":"6614"}},{"literal":{"i64":"6615"}},{"literal":{"i64":"6616"}},{"literal":{"i64":"6617"}},{"literal":{"i64":"6618"}},{"literal":{"i64":"6619"}},{"literal":{"i64":"662"}},{"literal":{"i64":"6620"}},{"literal":{"i64":"6621"}},{"literal":{"i64":"6622"}},{"literal":{"i64":"6623"}},{"literal":{"i64":"6624"}},{"literal":{"i64":"6625"}},{"literal":{"i64":"6626"}},{"literal":{"i64":"6627"}},{"literal":{"i64":"6628"}},{"literal":{"i64":"6629"}},{"literal":{"i64":"663"}},{"literal":{"i64":"6630"}},{"literal":{"i64":"6631"}},{"literal":{"i64":"6632"}},{"literal":{"i64":"6633"}},{"literal":{"i64":"6634"}},{"literal":{"i64":"6635"}},{"literal":{"i64":"6636"}},{"literal":{"i64":"6637"}},{"literal":{"i64":"6638"}},{"literal":{"i64":"6639"}},{"literal":{"i64":"664"}},{"literal":{"i64":"6640"}},{"literal":{"i64":"6641"}},{"literal":{"i64":"6642"}},{"literal":{"i64":"6643"}},{"literal":{"i64":"6644"}},{"literal":{"i64":"6645"}},{"literal":{"i64":"6646"}},{"literal":{"i64":"6647"}},{"literal":{"i64":"6648"}},{"literal":{"i64":"6649"}},{"literal":{"i64":"665"}},{"literal":{"i64":"6650"}},{"literal":{"i64":"6651"}},{"literal":{"i64":"6652"}},{"literal":{"i64":"6653"}},{"literal":{"i64":"6654"}},{"literal":{"i64":"6655"}},{"literal":{"i64":"6656"}},{"literal":{"i64":"6657"}},{"literal":{"i64":"6658"}},{"literal":{"i64":"6659"}},{"literal":{"i64":"666"}},{"literal":{"i64":"6660"}},{"literal":{"i64":"6661"}},{"literal":{"i64":"6662"}},{"literal":{"i64":"6663"}},{"literal":{"i64":"6664"}},{"literal":{"i64":"6665"}},{"literal":{"i64":"6666"}},{"literal":{"i64":"6667"}},{"literal":{"i64":"6668"}},{"literal":{"i64":"6669"}},{"literal":{"i64":"667"}},{"literal":{"i64":"6670"}},{"literal":{"i64":"6671"}},{"literal":{"i64":"6672"}},{"literal":{"i64":"6673"}},{"literal":{"i64":"6674"}},{"literal":{"i64":"6675"}},{"literal":{"i64":"6676"}},{"literal":{"i64":"6677"}},{"literal":{"i64":"6678"}},{"literal":{"i64":"6679"}},{"literal":{"i64":"668"}},{"literal":{"i64":"6680"}},{"literal":{"i64":"6681"}},{"literal":{"i64":"6682"}},{"literal":{"i64":"6683"}},{"literal":{"i64":"6684"}},{"literal":{"i64":"6685"}},{"literal":{"i64":"6686"}},{"literal":{"i64":"6687"}},{"literal":{"i64":"6688"}},{"literal":{"i64":"6689"}},{"literal":{"i64":"669"}},{"literal":{"i64":"6690"}},{"literal":{"i64":"6691"}},{"literal":{"i64":"6692"}},{"literal":{"i64":"6693"}},{"literal":{"i64":"6694"}},{"literal":{"i64":"6695"}},{"literal":{"i64":"6696"}},{"literal":{"i64":"6697"}},{"literal":{"i64":"6698"}},{"literal":{"i64":"6699"}},{"literal":{"i64":"67"}},{"literal":{"i64":"670"}},{"literal":{"i64":"6700"}},{"literal":{"i64":"6701"}},{"literal":{"i64":"6702"}},{"literal":{"i64":"6703"}},{"literal":{"i64":"6704"}},{"literal":{"i64":"6705"}},{"literal":{"i64":"6706"}},{"literal":{"i64":"6707"}},{"literal":{"i64":"6708"}},{"literal":{"i64":"6709"}},{"literal":{"i64":"671"}},{"literal":{"i64":"6710"}},{"literal":{"i64":"6711"}},{"literal":{"i64":"6712"}},{"literal":{"i64":"6713"}},{"literal":{"i64":"6714"}},{"literal":{"i64":"6715"}},{"literal":{"i64":"6716"}},{"literal":{"i64":"6717"}},{"literal":{"i64":"6718"}},{"literal":{"i64":"6719"}},{"literal":{"i64":"672"}},{"literal":{"i64":"6720"}},{"literal":{"i64":"6721"}},{"literal":{"i64":"6722"}},{"literal":{"i64":"6723"}},{"literal":{"i64":"6724"}},{"literal":{"i64":"6725"}},{"literal":{"i64":"6726"}},{"literal":{"i64":"6727"}},{"literal":{"i64":"6728"}},{"literal":{"i64":"6729"}},{"literal":{"i64":"673"}},{"literal":{"i64":"6730"}},{"literal":{"i64":"6731"}},{"literal":{"i64":"6732"}},{"literal":{"i64":"6733"}},{"literal":{"i64":"6734"}},{"literal":{"i64":"6735"}},{"literal":{"i64":"6736"}},{"literal":{"i64":"6737"}},{"literal":{"i64":"6738"}},{"literal":{"i64":"6739"}},{"literal":{"i64":"674"}},{"literal":{"i64":"6740"}},{"literal":{"i64":"6741"}},{"literal":{"i64":"6742"}},{"literal":{"i64":"6743"}},{"literal":{"i64":"6744"}},{"literal":{"i64":"6745"}},{"literal":{"i64":"6746"}},{"literal":{"i64":"6747"}},{"literal":{"i64":"6748"}},{"literal":{"i64":"6749"}},{"literal":{"i64":"675"}},{"literal":{"i64":"6750"}},{"literal":{"i64":"6751"}},{"literal":{"i64":"6752"}},{"literal":{"i64":"6753"}},{"literal":{"i64":"6754"}},{"literal":{"i64":"6755"}},{"literal":{"i64":"6756"}},{"literal":{"i64":"6757"}},{"literal":{"i64":"6758"}},{"literal":{"i64":"6759"}},{"literal":{"i64":"676"}},{"literal":{"i64":"6760"}},{"literal":{"i64":"6761"}},{"literal":{"i64":"6762"}},{"literal":{"i64":"6763"}},{"literal":{"i64":"6764"}},{"literal":{"i64":"6765"}},{"literal":{"i64":"6766"}},{"literal":{"i64":"6767"}},{"literal":{"i64":"6768"}},{"literal":{"i64":"6769"}},{"literal":{"i64":"677"}},{"literal":{"i64":"6770"}},{"literal":{"i64":"6771"}},{"literal":{"i64":"6772"}},{"literal":{"i64":"6773"}},{"literal":{"i64":"6774"}},{"literal":{"i64":"6775"}},{"literal":{"i64":"6776"}},{"literal":{"i64":"6777"}},{"literal":{"i64":"6778"}},{"literal":{"i64":"6779"}},{"literal":{"i64":"678"}},{"literal":{"i64":"6780"}},{"literal":{"i64":"6781"}},{"literal":{"i64":"6782"}},{"literal":{"i64":"6783"}},{"literal":{"i64":"6784"}},{"literal":{"i64":"6785"}},{"literal":{"i64":"6786"}},{"literal":{"i64":"6787"}},{"literal":{"i64":"6788"}},{"literal":{"i64":"6789"}},{"literal":{"i64":"679"}},{"literal":{"i64":"6790"}},{"literal":{"i64":"6791"}},{"literal":{"i64":"6792"}},{"literal":{"i64":"6793"}},{"literal":{"i64":"6794"}},{"literal":{"i64":"6795"}},{"literal":{"i64":"6796"}},{"literal":{"i64":"6797"}},{"literal":{"i64":"6798"}},{"literal":{"i64":"6799"}},{"literal":{"i64":"68"}},{"literal":{"i64":"680"}},{"literal":{"i64":"6800"}},{"literal":{"i64":"6801"}},{"literal":{"i64":"6802"}},{"literal":{"i64":"6803"}},{"literal":{"i64":"6804"}},{"literal":{"i64":"6805"}},{"literal":{"i64":"6806"}},{"literal":{"i64":"6807"}},{"literal":{"i64":"6808"}},{"literal":{"i64":"6809"}},{"literal":{"i64":"681"}},{"literal":{"i64":"6810"}},{"literal":{"i64":"6811"}},{"literal":{"i64":"6812"}},{"literal":{"i64":"6813"}},{"literal":{"i64":"6814"}},{"literal":{"i64":"6815"}},{"literal":{"i64":"6816"}},{"literal":{"i64":"6817"}},{"literal":{"i64":"6818"}},{"literal":{"i64":"6819"}},{"literal":{"i64":"682"}},{"literal":{"i64":"6820"}},{"literal":{"i64":"6821"}},{"literal":{"i64":"6822"}},{"literal":{"i64":"6823"}},{"literal":{"i64":"6824"}},{"literal":{"i64":"6825"}},{"literal":{"i64":"6826"}},{"literal":{"i64":"6827"}},{"literal":{"i64":"6828"}},{"literal":{"i64":"6829"}},{"literal":{"i64":"683"}},{"literal":{"i64":"6830"}},{"literal":{"i64":"6831"}},{"literal":{"i64":"6832"}},{"literal":{"i64":"6833"}},{"literal":{"i64":"6834"}},{"literal":{"i64":"6835"}},{"literal":{"i64":"6836"}},{"literal":{"i64":"6837"}},{"literal":{"i64":"6838"}},{"literal":{"i64":"6839"}},{"literal":{"i64":"684"}},{"literal":{"i64":"6840"}},{"literal":{"i64":"6841"}},{"literal":{"i64":"6842"}},{"literal":{"i64":"6843"}},{"literal":{"i64":"6844"}},{"literal":{"i64":"6845"}},{"literal":{"i64":"6846"}},{"literal":{"i64":"6847"}},{"literal":{"i64":"6848"}},{"literal":{"i64":"6849"}},{"literal":{"i64":"685"}},{"literal":{"i64":"6850"}},{"literal":{"i64":"6851"}},{"literal":{"i64":"6852"}},{"literal":{"i64":"6853"}},{"literal":{"i64":"6854"}},{"literal":{"i64":"6855"}},{"literal":{"i64":"6856"}},{"literal":{"i64":"6857"}},{"literal":{"i64":"6858"}},{"literal":{"i64":"6859"}},{"literal":{"i64":"686"}},{"literal":{"i64":"6860"}},{"literal":{"i64":"6861"}},{"literal":{"i64":"6862"}},{"literal":{"i64":"6863"}},{"literal":{"i64":"6864"}},{"literal":{"i64":"6865"}},{"literal":{"i64":"6866"}},{"literal":{"i64":"6867"}},{"literal":{"i64":"6868"}},{"literal":{"i64":"6869"}},{"literal":{"i64":"687"}},{"literal":{"i64":"6870"}},{"literal":{"i64":"6871"}},{"literal":{"i64":"6872"}},{"literal":{"i64":"6873"}},{"literal":{"i64":"6874"}},{"literal":{"i64":"6875"}},{"literal":{"i64":"6876"}},{"literal":{"i64":"6877"}},{"literal":{"i64":"6878"}},{"literal":{"i64":"6879"}},{"literal":{"i64":"688"}},{"literal":{"i64":"6880"}},{"literal":{"i64":"6881"}},{"literal":{"i64":"6882"}},{"literal":{"i64":"6883"}},{"literal":{"i64":"6884"}},{"literal":{"i64":"6885"}},{"literal":{"i64":"6886"}},{"literal":{"i64":"6887"}},{"literal":{"i64":"6888"}},{"literal":{"i64":"6889"}},{"literal":{"i64":"689"}},{"literal":{"i64":"6890"}},{"literal":{"i64":"6891"}},{"literal":{"i64":"6892"}},{"literal":{"i64":"6893"}},{"literal":{"i64":"6894"}},{"literal":{"i64":"6895"}},{"literal":{"i64":"6896"}},{"literal":{"i64":"6897"}},{"literal":{"i64":"6898"}},{"literal":{"i64":"6899"}},{"literal":{"i64":"69"}},{"literal":{"i64":"690"}},{"literal":{"i64":"6900"}},{"literal":{"i64":"6901"}},{"literal":{"i64":"6902"}},{"literal":{"i64":"6903"}},{"literal":{"i64":"6904"}},{"literal":{"i64":"6905"}},{"literal":{"i64":"6906"}},{"literal":{"i64":"6907"}},{"literal":{"i64":"6908"}},{"literal":{"i64":"6909"}},{"literal":{"i64":"691"}},{"literal":{"i64":"6910"}},{"literal":{"i64":"6911"}},{"literal":{"i64":"6912"}},{"literal":{"i64":"6913"}},{"literal":{"i64":"6914"}},{"literal":{"i64":"6915"}},{"literal":{"i64":"6916"}},{"literal":{"i64":"6917"}},{"literal":{"i64":"6918"}},{"literal":{"i64":"6919"}},{"literal":{"i64":"692"}},{"literal":{"i64":"6920"}},{"literal":{"i64":"6921"}},{"literal":{"i64":"6922"}},{"literal":{"i64":"6923"}},{"literal":{"i64":"6924"}},{"literal":{"i64":"6925"}},{"literal":{"i64":"6926"}},{"literal":{"i64":"6927"}},{"literal":{"i64":"6928"}},{"literal":{"i64":"6929"}},{"literal":{"i64":"693"}},{"literal":{"i64":"6930"}},{"literal":{"i64":"6931"}},{"literal":{"i64":"6932"}},{"literal":{"i64":"6933"}},{"literal":{"i64":"6934"}},{"literal":{"i64":"6935"}},{"literal":{"i64":"6936"}},{"literal":{"i64":"6937"}},{"literal":{"i64":"6938"}},{"literal":{"i64":"6939"}},{"literal":{"i64":"694"}},{"literal":{"i64":"6940"}},{"literal":{"i64":"6941"}},{"literal":{"i64":"6942"}},{"literal":{"i64":"6943"}},{"literal":{"i64":"6944"}},{"literal":{"i64":"6945"}},{"literal":{"i64":"6946"}},{"literal":{"i64":"6947"}},{"literal":{"i64":"6948"}},{"literal":{"i64":"6949"}},{"literal":{"i64":"695"}},{"literal":{"i64":"6950"}},{"literal":{"i64":"6951"}},{"literal":{"i64":"6952"}},{"literal":{"i64":"6953"}},{"literal":{"i64":"6954"}},{"literal":{"i64":"6955"}},{"literal":{"i64":"6956"}},{"literal":{"i64":"6957"}},{"literal":{"i64":"6958"}},{"literal":{"i64":"6959"}},{"literal":{"i64":"696"}},{"literal":{"i64":"6960"}},{"literal":{"i64":"6961"}},{"literal":{"i64":"6962"}},{"literal":{"i64":"6963"}},{"literal":{"i64":"6964"}},{"literal":{"i64":"6965"}},{"literal":{"i64":"6966"}},{"literal":{"i64":"6967"}},{"literal":{"i64":"6968"}},{"literal":{"i64":"6969"}},{"literal":{"i64":"697"}},{"literal":{"i64":"6970"}},{"literal":{"i64":"6971"}},{"literal":{"i64":"6972"}},{"literal":{"i64":"6973"}},{"literal":{"i64":"6974"}},{"literal":{"i64":"6975"}},{"literal":{"i64":"6976"}},{"literal":{"i64":"6977"}},{"literal":{"i64":"6978"}},{"literal":{"i64":"6979"}},{"literal":{"i64":"698"}},{"literal":{"i64":"6980"}},{"literal":{"i64":"6981"}},{"literal":{"i64":"6982"}},{"literal":{"i64":"6983"}},{"literal":{"i64":"6984"}},{"literal":{"i64":"6985"}},{"literal":{"i64":"6986"}},{"literal":{"i64":"6987"}},{"literal":{"i64":"6988"}},{"literal":{"i64":"6989"}},{"literal":{"i64":"699"}},{"literal":{"i64":"6990"}},{"literal":{"i64":"6991"}},{"literal":{"i64":"6992"}},{"literal":{"i64":"6993"}},{"literal":{"i64":"6994"}},{"literal":{"i64":"6995"}},{"literal":{"i64":"6996"}},{"literal":{"i64":"6997"}},{"literal":{"i64":"6998"}},{"literal":{"i64":"6999"}},{"literal":{"i64":"7"}},{"literal":{"i64":"70"}},{"literal":{"i64":"700"}},{"literal":{"i64":"7000"}},{"literal":{"i64":"7001"}},{"literal":{"i64":"7002"}},{"literal":{"i64":"7003"}},{"literal":{"i64":"7004"}},{"literal":{"i64":"7005"}},{"literal":{"i64":"7006"}},{"literal":{"i64":"7007"}},{"literal":{"i64":"7008"}},{"literal":{"i64":"7009"}},{"literal":{"i64":"701"}},{"literal":{"i64":"7010"}},{"literal":{"i64":"7011"}},{"literal":{"i64":"7012"}},{"literal":{"i64":"7013"}},{"literal":{"i64":"7014"}},{"literal":{"i64":"7015"}},{"literal":{"i64":"7016"}},{"literal":{"i64":"7017"}},{"literal":{"i64":"7018"}},{"literal":{"i64":"7019"}},{"literal":{"i64":"702"}},{"literal":{"i64":"7020"}},{"literal":{"i64":"7021"}},{"literal":{"i64":"7022"}},{"literal":{"i64":"7023"}},{"literal":{"i64":"7024"}},{"literal":{"i64":"7025"}},{"literal":{"i64":"7026"}},{"literal":{"i64":"7027"}},{"literal":{"i64":"7028"}},{"literal":{"i64":"7029"}},{"literal":{"i64":"703"}},{"literal":{"i64":"7030"}},{"literal":{"i64":"7031"}},{"literal":{"i64":"7032"}},{"literal":{"i64":"7033"}},{"literal":{"i64":"7034"}},{"literal":{"i64":"7035"}},{"literal":{"i64":"7036"}},{"literal":{"i64":"7037"}},{"literal":{"i64":"7038"}},{"literal":{"i64":"7039"}},{"literal":{"i64":"704"}},{"literal":{"i64":"7040"}},{"literal":{"i64":"7041"}},{"literal":{"i64":"7042"}},{"literal":{"i64":"7043"}},{"literal":{"i64":"7044"}},{"literal":{"i64":"7045"}},{"literal":{"i64":"7046"}},{"literal":{"i64":"7047"}},{"literal":{"i64":"7048"}},{"literal":{"i64":"7049"}},{"literal":{"i64":"705"}},{"literal":{"i64":"7050"}},{"literal":{"i64":"7051"}},{"literal":{"i64":"7052"}},{"literal":{"i64":"7053"}},{"literal":{"i64":"7054"}},{"literal":{"i64":"7055"}},{"literal":{"i64":"7056"}},{"literal":{"i64":"7057"}},{"literal":{"i64":"7058"}},{"literal":{"i64":"7059"}},{"literal":{"i64":"706"}},{"literal":{"i64":"7060"}},{"literal":{"i64":"7061"}},{"literal":{"i64":"7062"}},{"literal":{"i64":"7063"}},{"literal":{"i64":"7064"}},{"literal":{"i64":"7065"}},{"literal":{"i64":"7066"}},{"literal":{"i64":"7067"}},{"literal":{"i64":"7068"}},{"literal":{"i64":"7069"}},{"literal":{"i64":"707"}},{"literal":{"i64":"7070"}},{"literal":{"i64":"7071"}},{"literal":{"i64":"7072"}},{"literal":{"i64":"7073"}},{"literal":{"i64":"7074"}},{"literal":{"i64":"7075"}},{"literal":{"i64":"7076"}},{"literal":{"i64":"7077"}},{"literal":{"i64":"7078"}},{"literal":{"i64":"7079"}},{"literal":{"i64":"708"}},{"literal":{"i64":"7080"}},{"literal":{"i64":"7081"}},{"literal":{"i64":"7082"}},{"literal":{"i64":"7083"}},{"literal":{"i64":"7084"}},{"literal":{"i64":"7085"}},{"literal":{"i64":"7086"}},{"literal":{"i64":"7087"}},{"literal":{"i64":"7088"}},{"literal":{"i64":"7089"}},{"literal":{"i64":"709"}},{"literal":{"i64":"7090"}},{"literal":{"i64":"7091"}},{"literal":{"i64":"7092"}},{"literal":{"i64":"7093"}},{"literal":{"i64":"7094"}},{"literal":{"i64":"7095"}},{"literal":{"i64":"7096"}},{"literal":{"i64":"7097"}},{"literal":{"i64":"7098"}},{"literal":{"i64":"7099"}},{"literal":{"i64":"71"}},{"literal":{"i64":"710"}},{"literal":{"i64":"7100"}},{"literal":{"i64":"7101"}},{"literal":{"i64":"7102"}},{"literal":{"i64":"7103"}},{"literal":{"i64":"7104"}},{"literal":{"i64":"7105"}},{"literal":{"i64":"7106"}},{"literal":{"i64":"7107"}},{"literal":{"i64":"7108"}},{"literal":{"i64":"7109"}},{"literal":{"i64":"711"}},{"literal":{"i64":"7110"}},{"literal":{"i64":"7111"}},{"literal":{"i64":"7112"}},{"literal":{"i64":"7113"}},{"literal":{"i64":"7114"}},{"literal":{"i64":"7115"}},{"literal":{"i64":"7116"}},{"literal":{"i64":"7117"}},{"literal":{"i64":"7118"}},{"literal":{"i64":"7119"}},{"literal":{"i64":"712"}},{"literal":{"i64":"7120"}},{"literal":{"i64":"7121"}},{"literal":{"i64":"7122"}},{"literal":{"i64":"7123"}},{"literal":{"i64":"7124"}},{"literal":{"i64":"7125"}},{"literal":{"i64":"7126"}},{"literal":{"i64":"7127"}},{"literal":{"i64":"7128"}},{"literal":{"i64":"7129"}},{"literal":{"i64":"713"}},{"literal":{"i64":"7130"}},{"literal":{"i64":"7131"}},{"literal":{"i64":"7132"}},{"literal":{"i64":"7133"}},{"literal":{"i64":"7134"}},{"literal":{"i64":"7135"}},{"literal":{"i64":"7136"}},{"literal":{"i64":"7137"}},{"literal":{"i64":"7138"}},{"literal":{"i64":"7139"}},{"literal":{"i64":"714"}},{"literal":{"i64":"7140"}},{"literal":{"i64":"7141"}},{"literal":{"i64":"7142"}},{"literal":{"i64":"7143"}},{"literal":{"i64":"7144"}},{"literal":{"i64":"7145"}},{"literal":{"i64":"7146"}},{"literal":{"i64":"7147"}},{"literal":{"i64":"7148"}},{"literal":{"i64":"7149"}},{"literal":{"i64":"715"}},{"literal":{"i64":"7150"}},{"literal":{"i64":"7151"}},{"literal":{"i64":"7152"}},{"literal":{"i64":"7153"}},{"literal":{"i64":"7154"}},{"literal":{"i64":"7155"}},{"literal":{"i64":"7156"}},{"literal":{"i64":"7157"}},{"literal":{"i64":"7158"}},{"literal":{"i64":"7159"}},{"literal":{"i64":"716"}},{"literal":{"i64":"7160"}},{"literal":{"i64":"7161"}},{"literal":{"i64":"7162"}},{"literal":{"i64":"7163"}},{"literal":{"i64":"7164"}},{"literal":{"i64":"7165"}},{"literal":{"i64":"7166"}},{"literal":{"i64":"7167"}},{"literal":{"i64":"7168"}},{"literal":{"i64":"7169"}},{"literal":{"i64":"717"}},{"literal":{"i64":"7170"}},{"literal":{"i64":"7171"}},{"literal":{"i64":"7172"}},{"literal":{"i64":"7173"}},{"literal":{"i64":"7174"}},{"literal":{"i64":"7175"}},{"literal":{"i64":"7176"}},{"literal":{"i64":"7177"}},{"literal":{"i64":"7178"}},{"literal":{"i64":"7179"}},{"literal":{"i64":"718"}},{"literal":{"i64":"7180"}},{"literal":{"i64":"7181"}},{"literal":{"i64":"7182"}},{"literal":{"i64":"7183"}},{"literal":{"i64":"7184"}},{"literal":{"i64":"7185"}},{"literal":{"i64":"7186"}},{"literal":{"i64":"7187"}},{"literal":{"i64":"7188"}},{"literal":{"i64":"7189"}},{"literal":{"i64":"719"}},{"literal":{"i64":"7190"}},{"literal":{"i64":"7191"}},{"literal":{"i64":"7192"}},{"literal":{"i64":"7193"}},{"literal":{"i64":"7194"}},{"literal":{"i64":"7195"}},{"literal":{"i64":"7196"}},{"literal":{"i64":"7197"}},{"literal":{"i64":"7198"}},{"literal":{"i64":"7199"}},{"literal":{"i64":"72"}},{"literal":{"i64":"720"}},{"literal":{"i64":"7200"}},{"literal":{"i64":"7201"}},{"literal":{"i64":"7202"}},{"literal":{"i64":"7203"}},{"literal":{"i64":"7204"}},{"literal":{"i64":"7205"}},{"literal":{"i64":"7206"}},{"literal":{"i64":"7207"}},{"literal":{"i64":"7208"}},{"literal":{"i64":"7209"}},{"literal":{"i64":"721"}},{"literal":{"i64":"7210"}},{"literal":{"i64":"7211"}},{"literal":{"i64":"7212"}},{"literal":{"i64":"7213"}},{"literal":{"i64":"7214"}},{"literal":{"i64":"7215"}},{"literal":{"i64":"7216"}},{"literal":{"i64":"7217"}},{"literal":{"i64":"7218"}},{"literal":{"i64":"7219"}},{"literal":{"i64":"722"}},{"literal":{"i64":"7220"}},{"literal":{"i64":"7221"}},{"literal":{"i64":"7222"}},{"literal":{"i64":"7223"}},{"literal":{"i64":"7224"}},{"literal":{"i64":"7225"}},{"literal":{"i64":"7226"}},{"literal":{"i64":"7227"}},{"literal":{"i64":"7228"}},{"literal":{"i64":"7229"}},{"literal":{"i64":"723"}},{"literal":{"i64":"7230"}},{"literal":{"i64":"7231"}},{"literal":{"i64":"7232"}},{"literal":{"i64":"7233"}},{"literal":{"i64":"7234"}},{"literal":{"i64":"7235"}},{"literal":{"i64":"7236"}},{"literal":{"i64":"7237"}},{"literal":{"i64":"7238"}},{"literal":{"i64":"7239"}},{"literal":{"i64":"724"}},{"literal":{"i64":"7240"}},{"literal":{"i64":"7241"}},{"literal":{"i64":"7242"}},{"literal":{"i64":"7243"}},{"literal":{"i64":"7244"}},{"literal":{"i64":"7245"}},{"literal":{"i64":"7246"}},{"literal":{"i64":"7247"}},{"literal":{"i64":"7248"}},{"literal":{"i64":"7249"}},{"literal":{"i64":"725"}},{"literal":{"i64":"7250"}},{"literal":{"i64":"7251"}},{"literal":{"i64":"7252"}},{"literal":{"i64":"7253"}},{"literal":{"i64":"7254"}},{"literal":{"i64":"7255"}},{"literal":{"i64":"7256"}},{"literal":{"i64":"7257"}},{"literal":{"i64":"7258"}},{"literal":{"i64":"7259"}},{"literal":{"i64":"726"}},{"literal":{"i64":"7260"}},{"literal":{"i64":"7261"}},{"literal":{"i64":"7262"}},{"literal":{"i64":"7263"}},{"literal":{"i64":"7264"}},{"literal":{"i64":"7265"}},{"literal":{"i64":"7266"}},{"literal":{"i64":"7267"}},{"literal":{"i64":"7268"}},{"literal":{"i64":"7269"}},{"literal":{"i64":"727"}},{"literal":{"i64":"7270"}},{"literal":{"i64":"7271"}},{"literal":{"i64":"7272"}},{"literal":{"i64":"7273"}},{"literal":{"i64":"7274"}},{"literal":{"i64":"7275"}},{"literal":{"i64":"7276"}},{"literal":{"i64":"7277"}},{"literal":{"i64":"7278"}},{"literal":{"i64":"7279"}},{"literal":{"i64":"728"}},{"literal":{"i64":"7280"}},{"literal":{"i64":"7281"}},{"literal":{"i64":"7282"}},{"literal":{"i64":"7283"}},{"literal":{"i64":"7284"}},{"literal":{"i64":"7285"}},{"literal":{"i64":"7286"}},{"literal":{"i64":"7287"}},{"literal":{"i64":"7288"}},{"literal":{"i64":"7289"}},{"literal":{"i64":"729"}},{"literal":{"i64":"7290"}},{"literal":{"i64":"7291"}},{"literal":{"i64":"7292"}},{"literal":{"i64":"7293"}},{"literal":{"i64":"7294"}},{"literal":{"i64":"7295"}},{"literal":{"i64":"7296"}},{"literal":{"i64":"7297"}},{"literal":{"i64":"7298"}},{"literal":{"i64":"7299"}},{"literal":{"i64":"73"}},{"literal":{"i64":"730"}},{"literal":{"i64":"7300"}},{"literal":{"i64":"7301"}},{"literal":{"i64":"7302"}},{"literal":{"i64":"7303"}},{"literal":{"i64":"7304"}},{"literal":{"i64":"7305"}},{"literal":{"i64":"7306"}},{"literal":{"i64":"7307"}},{"literal":{"i64":"7308"}},{"literal":{"i64":"7309"}},{"literal":{"i64":"731"}},{"literal":{"i64":"7310"}},{"literal":{"i64":"7311"}},{"literal":{"i64":"7312"}},{"literal":{"i64":"7313"}},{"literal":{"i64":"7314"}},{"literal":{"i64":"7315"}},{"literal":{"i64":"7316"}},{"literal":{"i64":"7317"}},{"literal":{"i64":"7318"}},{"literal":{"i64":"7319"}},{"literal":{"i64":"732"}},{"literal":{"i64":"7320"}},{"literal":{"i64":"7321"}},{"literal":{"i64":"7322"}},{"literal":{"i64":"7323"}},{"literal":{"i64":"7324"}},{"literal":{"i64":"7325"}},{"literal":{"i64":"7326"}},{"literal":{"i64":"7327"}},{"literal":{"i64":"7328"}},{"literal":{"i64":"7329"}},{"literal":{"i64":"733"}},{"literal":{"i64":"7330"}},{"literal":{"i64":"7331"}},{"literal":{"i64":"7332"}},{"literal":{"i64":"7333"}},{"literal":{"i64":"7334"}},{"literal":{"i64":"7335"}},{"literal":{"i64":"7336"}},{"literal":{"i64":"7337"}},{"literal":{"i64":"7338"}},{"literal":{"i64":"7339"}},{"literal":{"i64":"734"}},{"literal":{"i64":"7340"}},{"literal":{"i64":"7341"}},{"literal":{"i64":"7342"}},{"literal":{"i64":"7343"}},{"literal":{"i64":"7344"}},{"literal":{"i64":"7345"}},{"literal":{"i64":"7346"}},{"literal":{"i64":"7347"}},{"literal":{"i64":"7348"}},{"literal":{"i64":"7349"}},{"literal":{"i64":"735"}},{"literal":{"i64":"7350"}},{"literal":{"i64":"7351"}},{"literal":{"i64":"7352"}},{"literal":{"i64":"7353"}},{"literal":{"i64":"7354"}},{"literal":{"i64":"7355"}},{"literal":{"i64":"7356"}},{"literal":{"i64":"7357"}},{"literal":{"i64":"7358"}},{"literal":{"i64":"7359"}},{"literal":{"i64":"736"}},{"literal":{"i64":"7360"}},{"literal":{"i64":"7361"}},{"literal":{"i64":"7362"}},{"literal":{"i64":"7363"}},{"literal":{"i64":"7364"}},{"literal":{"i64":"7365"}},{"literal":{"i64":"7366"}},{"literal":{"i64":"7367"}},{"literal":{"i64":"7368"}},{"literal":{"i64":"7369"}},{"literal":{"i64":"737"}},{"literal":{"i64":"7370"}},{"literal":{"i64":"7371"}},{"literal":{"i64":"7372"}},{"literal":{"i64":"7373"}},{"literal":{"i64":"7374"}},{"literal":{"i64":"7375"}},{"literal":{"i64":"7376"}},{"literal":{"i64":"7377"}},{"literal":{"i64":"7378"}},{"literal":{"i64":"7379"}},{"literal":{"i64":"738"}},{"literal":{"i64":"7380"}},{"literal":{"i64":"7381"}},{"literal":{"i64":"7382"}},{"literal":{"i64":"7383"}},{"literal":{"i64":"7384"}},{"literal":{"i64":"7385"}},{"literal":{"i64":"7386"}},{"literal":{"i64":"7387"}},{"literal":{"i64":"7388"}},{"literal":{"i64":"7389"}},{"literal":{"i64":"739"}},{"literal":{"i64":"7390"}},{"literal":{"i64":"7391"}},{"literal":{"i64":"7392"}},{"literal":{"i64":"7393"}},{"literal":{"i64":"7394"}},{"literal":{"i64":"7395"}},{"literal":{"i64":"7396"}},{"literal":{"i64":"7397"}},{"literal":{"i64":"7398"}},{"literal":{"i64":"7399"}},{"literal":{"i64":"74"}},{"literal":{"i64":"740"}},{"literal":{"i64":"7400"}},{"literal":{"i64":"7401"}},{"literal":{"i64":"7402"}},{"literal":{"i64":"7403"}},{"literal":{"i64":"7404"}},{"literal":{"i64":"7405"}},{"literal":{"i64":"7406"}},{"literal":{"i64":"7407"}},{"literal":{"i64":"7408"}},{"literal":{"i64":"7409"}},{"literal":{"i64":"741"}},{"literal":{"i64":"7410"}},{"literal":{"i64":"7411"}},{"literal":{"i64":"7412"}},{"literal":{"i64":"7413"}},{"literal":{"i64":"7414"}},{"literal":{"i64":"7415"}},{"literal":{"i64":"7416"}},{"literal":{"i64":"7417"}},{"literal":{"i64":"7418"}},{"literal":{"i64":"7419"}},{"literal":{"i64":"742"}},{"literal":{"i64":"7420"}},{"literal":{"i64":"7421"}},{"literal":{"i64":"7422"}},{"literal":{"i64":"7423"}},{"literal":{"i64":"7424"}},{"literal":{"i64":"7425"}},{"literal":{"i64":"7426"}},{"literal":{"i64":"7427"}},{"literal":{"i64":"7428"}},{"literal":{"i64":"7429"}},{"literal":{"i64":"743"}},{"literal":{"i64":"7430"}},{"literal":{"i64":"7431"}},{"literal":{"i64":"7432"}},{"literal":{"i64":"7433"}},{"literal":{"i64":"7434"}},{"literal":{"i64":"7435"}},{"literal":{"i64":"7436"}},{"literal":{"i64":"7437"}},{"literal":{"i64":"7438"}},{"literal":{"i64":"7439"}},{"literal":{"i64":"744"}},{"literal":{"i64":"7440"}},{"literal":{"i64":"7441"}},{"literal":{"i64":"7442"}},{"literal":{"i64":"7443"}},{"literal":{"i64":"7444"}},{"literal":{"i64":"7445"}},{"literal":{"i64":"7446"}},{"literal":{"i64":"7447"}},{"literal":{"i64":"7448"}},{"literal":{"i64":"7449"}},{"literal":{"i64":"745"}},{"literal":{"i64":"7450"}},{"literal":{"i64":"7451"}},{"literal":{"i64":"7452"}},{"literal":{"i64":"7453"}},{"literal":{"i64":"7454"}},{"literal":{"i64":"7455"}},{"literal":{"i64":"7456"}},{"literal":{"i64":"7457"}},{"literal":{"i64":"7458"}},{"literal":{"i64":"7459"}},{"literal":{"i64":"746"}},{"literal":{"i64":"7460"}},{"literal":{"i64":"7461"}},{"literal":{"i64":"7462"}},{"literal":{"i64":"7463"}},{"literal":{"i64":"7464"}},{"literal":{"i64":"7465"}},{"literal":{"i64":"7466"}},{"literal":{"i64":"7467"}},{"literal":{"i64":"7468"}},{"literal":{"i64":"7469"}},{"literal":{"i64":"747"}},{"literal":{"i64":"7470"}},{"literal":{"i64":"7471"}},{"literal":{"i64":"7472"}},{"literal":{"i64":"7473"}},{"literal":{"i64":"7474"}},{"literal":{"i64":"7475"}},{"literal":{"i64":"7476"}},{"literal":{"i64":"7477"}},{"literal":{"i64":"7478"}},{"literal":{"i64":"7479"}},{"literal":{"i64":"748"}},{"literal":{"i64":"7480"}},{"literal":{"i64":"7481"}},{"literal":{"i64":"7482"}},{"literal":{"i64":"7483"}},{"literal":{"i64":"7484"}},{"literal":{"i64":"7485"}},{"literal":{"i64":"7486"}},{"literal":{"i64":"7487"}},{"literal":{"i64":"7488"}},{"literal":{"i64":"7489"}},{"literal":{"i64":"749"}},{"literal":{"i64":"7490"}},{"literal":{"i64":"7491"}},{"literal":{"i64":"7492"}},{"literal":{"i64":"7493"}},{"literal":{"i64":"7494"}},{"literal":{"i64":"7495"}},{"literal":{"i64":"7496"}},{"literal":{"i64":"7497"}},{"literal":{"i64":"7498"}},{"literal":{"i64":"7499"}},{"literal":{"i64":"75"}},{"literal":{"i64":"750"}},{"literal":{"i64":"7500"}},{"literal":{"i64":"7501"}},{"literal":{"i64":"7502"}},{"literal":{"i64":"7503"}},{"literal":{"i64":"7504"}},{"literal":{"i64":"7505"}},{"literal":{"i64":"7506"}},{"literal":{"i64":"7507"}},{"literal":{"i64":"7508"}},{"literal":{"i64":"7509"}},{"literal":{"i64":"751"}},{"literal":{"i64":"7510"}},{"literal":{"i64":"7511"}},{"literal":{"i64":"7512"}},{"literal":{"i64":"7513"}},{"literal":{"i64":"7514"}},{"literal":{"i64":"7515"}},{"literal":{"i64":"7516"}},{"literal":{"i64":"7517"}},{"literal":{"i64":"7518"}},{"literal":{"i64":"7519"}},{"literal":{"i64":"752"}},{"literal":{"i64":"7520"}},{"literal":{"i64":"7521"}},{"literal":{"i64":"7522"}},{"literal":{"i64":"7523"}},{"literal":{"i64":"7524"}},{"literal":{"i64":"7525"}},{"literal":{"i64":"7526"}},{"literal":{"i64":"7527"}},{"literal":{"i64":"7528"}},{"literal":{"i64":"7529"}},{"literal":{"i64":"753"}},{"literal":{"i64":"7530"}},{"literal":{"i64":"7531"}},{"literal":{"i64":"7532"}},{"literal":{"i64":"7533"}},{"literal":{"i64":"7534"}},{"literal":{"i64":"7535"}},{"literal":{"i64":"7536"}},{"literal":{"i64":"7537"}},{"literal":{"i64":"7538"}},{"literal":{"i64":"7539"}},{"literal":{"i64":"754"}},{"literal":{"i64":"7540"}},{"literal":{"i64":"7541"}},{"literal":{"i64":"7542"}},{"literal":{"i64":"7543"}},{"literal":{"i64":"7544"}},{"literal":{"i64":"7545"}},{"literal":{"i64":"7546"}},{"literal":{"i64":"7547"}},{"literal":{"i64":"7548"}},{"literal":{"i64":"7549"}},{"literal":{"i64":"755"}},{"literal":{"i64":"7550"}},{"literal":{"i64":"7551"}},{"literal":{"i64":"7552"}},{"literal":{"i64":"7553"}},{"literal":{"i64":"7554"}},{"literal":{"i64":"7555"}},{"literal":{"i64":"7556"}},{"literal":{"i64":"7557"}},{"literal":{"i64":"7558"}},{"literal":{"i64":"7559"}},{"literal":{"i64":"756"}},{"literal":{"i64":"7560"}},{"literal":{"i64":"7561"}},{"literal":{"i64":"7562"}},{"literal":{"i64":"7563"}},{"literal":{"i64":"7564"}},{"literal":{"i64":"7565"}},{"literal":{"i64":"7566"}},{"literal":{"i64":"7567"}},{"literal":{"i64":"7568"}},{"literal":{"i64":"7569"}},{"literal":{"i64":"757"}},{"literal":{"i64":"7570"}},{"literal":{"i64":"7571"}},{"literal":{"i64":"7572"}},{"literal":{"i64":"7573"}},{"literal":{"i64":"7574"}},{"literal":{"i64":"7575"}},{"literal":{"i64":"7576"}},{"literal":{"i64":"7577"}},{"literal":{"i64":"7578"}},{"literal":{"i64":"7579"}},{"literal":{"i64":"758"}},{"literal":{"i64":"7580"}},{"literal":{"i64":"7581"}},{"literal":{"i64":"7582"}},{"literal":{"i64":"7583"}},{"literal":{"i64":"7584"}},{"literal":{"i64":"7585"}},{"literal":{"i64":"7586"}},{"literal":{"i64":"7587"}},{"literal":{"i64":"7588"}},{"literal":{"i64":"7589"}},{"literal":{"i64":"759"}},{"literal":{"i64":"7590"}},{"literal":{"i64":"7591"}},{"literal":{"i64":"7592"}},{"literal":{"i64":"7593"}},{"literal":{"i64":"7594"}},{"literal":{"i64":"7595"}},{"literal":{"i64":"7596"}},{"literal":{"i64":"7597"}},{"literal":{"i64":"7598"}},{"literal":{"i64":"7599"}},{"literal":{"i64":"76"}},{"literal":{"i64":"760"}},{"literal":{"i64":"7600"}},{"literal":{"i64":"7601"}},{"literal":{"i64":"7602"}},{"literal":{"i64":"7603"}},{"literal":{"i64":"7604"}},{"literal":{"i64":"7605"}},{"literal":{"i64":"7606"}},{"literal":{"i64":"7607"}},{"literal":{"i64":"7608"}},{"literal":{"i64":"7609"}},{"literal":{"i64":"761"}},{"literal":{"i64":"7610"}},{"literal":{"i64":"7611"}},{"literal":{"i64":"7612"}},{"literal":{"i64":"7613"}},{"literal":{"i64":"7614"}},{"literal":{"i64":"7615"}},{"literal":{"i64":"7616"}},{"literal":{"i64":"7617"}},{"literal":{"i64":"7618"}},{"literal":{"i64":"7619"}},{"literal":{"i64":"762"}},{"literal":{"i64":"7620"}},{"literal":{"i64":"7621"}},{"literal":{"i64":"7622"}},{"literal":{"i64":"7623"}},{"literal":{"i64":"7624"}},{"literal":{"i64":"7625"}},{"literal":{"i64":"7626"}},{"literal":{"i64":"7627"}},{"literal":{"i64":"7628"}},{"literal":{"i64":"7629"}},{"literal":{"i64":"763"}},{"literal":{"i64":"7630"}},{"literal":{"i64":"7631"}},{"literal":{"i64":"7632"}},{"literal":{"i64":"7633"}},{"literal":{"i64":"7634"}},{"literal":{"i64":"7635"}},{"literal":{"i64":"7636"}},{"literal":{"i64":"7637"}},{"literal":{"i64":"7638"}},{"literal":{"i64":"7639"}},{"literal":{"i64":"764"}},{"literal":{"i64":"7640"}},{"literal":{"i64":"7641"}},{"literal":{"i64":"7642"}},{"literal":{"i64":"7643"}},{"literal":{"i64":"7644"}},{"literal":{"i64":"7645"}},{"literal":{"i64":"7646"}},{"literal":{"i64":"7647"}},{"literal":{"i64":"7648"}},{"literal":{"i64":"7649"}},{"literal":{"i64":"765"}},{"literal":{"i64":"7650"}},{"literal":{"i64":"7651"}},{"literal":{"i64":"7652"}},{"literal":{"i64":"7653"}},{"literal":{"i64":"7654"}},{"literal":{"i64":"7655"}},{"literal":{"i64":"7656"}},{"literal":{"i64":"7657"}},{"literal":{"i64":"7658"}},{"literal":{"i64":"7659"}},{"literal":{"i64":"766"}},{"literal":{"i64":"7660"}},{"literal":{"i64":"7661"}},{"literal":{"i64":"7662"}},{"literal":{"i64":"7663"}},{"literal":{"i64":"7664"}},{"literal":{"i64":"7665"}},{"literal":{"i64":"7666"}},{"literal":{"i64":"7667"}},{"literal":{"i64":"7668"}},{"literal":{"i64":"7669"}},{"literal":{"i64":"767"}},{"literal":{"i64":"7670"}},{"literal":{"i64":"7671"}},{"literal":{"i64":"7672"}},{"literal":{"i64":"7673"}},{"literal":{"i64":"7674"}},{"literal":{"i64":"7675"}},{"literal":{"i64":"7676"}},{"literal":{"i64":"7677"}},{"literal":{"i64":"7678"}},{"literal":{"i64":"7679"}},{"literal":{"i64":"768"}},{"literal":{"i64":"7680"}},{"literal":{"i64":"7681"}},{"literal":{"i64":"7682"}},{"literal":{"i64":"7683"}},{"literal":{"i64":"7684"}},{"literal":{"i64":"7685"}},{"literal":{"i64":"7686"}},{"literal":{"i64":"7687"}},{"literal":{"i64":"7688"}},{"literal":{"i64":"7689"}},{"literal":{"i64":"769"}},{"literal":{"i64":"7690"}},{"literal":{"i64":"7691"}},{"literal":{"i64":"7692"}},{"literal":{"i64":"7693"}},{"literal":{"i64":"7694"}},{"literal":{"i64":"7695"}},{"literal":{"i64":"7696"}},{"literal":{"i64":"7697"}},{"literal":{"i64":"7698"}},{"literal":{"i64":"7699"}},{"literal":{"i64":"77"}},{"literal":{"i64":"770"}},{"literal":{"i64":"7700"}},{"literal":{"i64":"7701"}},{"literal":{"i64":"7702"}},{"literal":{"i64":"7703"}},{"literal":{"i64":"7704"}},{"literal":{"i64":"7705"}},{"literal":{"i64":"7706"}},{"literal":{"i64":"7707"}},{"literal":{"i64":"7708"}},{"literal":{"i64":"7709"}},{"literal":{"i64":"771"}},{"literal":{"i64":"7710"}},{"literal":{"i64":"7711"}},{"literal":{"i64":"7712"}},{"literal":{"i64":"7713"}},{"literal":{"i64":"7714"}},{"literal":{"i64":"7715"}},{"literal":{"i64":"7716"}},{"literal":{"i64":"7717"}},{"literal":{"i64":"7718"}},{"literal":{"i64":"7719"}},{"literal":{"i64":"772"}},{"literal":{"i64":"7720"}},{"literal":{"i64":"7721"}},{"literal":{"i64":"7722"}},{"literal":{"i64":"7723"}},{"literal":{"i64":"7724"}},{"literal":{"i64":"7725"}},{"literal":{"i64":"7726"}},{"literal":{"i64":"7727"}},{"literal":{"i64":"7728"}},{"literal":{"i64":"7729"}},{"literal":{"i64":"773"}},{"literal":{"i64":"7730"}},{"literal":{"i64":"7731"}},{"literal":{"i64":"7732"}},{"literal":{"i64":"7733"}},{"literal":{"i64":"7734"}},{"literal":{"i64":"7735"}},{"literal":{"i64":"7736"}},{"literal":{"i64":"7737"}},{"literal":{"i64":"7738"}},{"literal":{"i64":"7739"}},{"literal":{"i64":"774"}},{"literal":{"i64":"7740"}},{"literal":{"i64":"7741"}},{"literal":{"i64":"7742"}},{"literal":{"i64":"7743"}},{"literal":{"i64":"7744"}},{"literal":{"i64":"7745"}},{"literal":{"i64":"7746"}},{"literal":{"i64":"7747"}},{"literal":{"i64":"7748"}},{"literal":{"i64":"7749"}},{"literal":{"i64":"775"}},{"literal":{"i64":"7750"}},{"literal":{"i64":"7751"}},{"literal":{"i64":"7752"}},{"literal":{"i64":"7753"}},{"literal":{"i64":"7754"}},{"literal":{"i64":"7755"}},{"literal":{"i64":"7756"}},{"literal":{"i64":"7757"}},{"literal":{"i64":"7758"}},{"literal":{"i64":"7759"}},{"literal":{"i64":"776"}},{"literal":{"i64":"7760"}},{"literal":{"i64":"7761"}},{"literal":{"i64":"7762"}},{"literal":{"i64":"7763"}},{"literal":{"i64":"7764"}},{"literal":{"i64":"7765"}},{"literal":{"i64":"7766"}},{"literal":{"i64":"7767"}},{"literal":{"i64":"7768"}},{"literal":{"i64":"7769"}},{"literal":{"i64":"777"}},{"literal":{"i64":"7770"}},{"literal":{"i64":"7771"}},{"literal":{"i64":"7772"}},{"literal":{"i64":"7773"}},{"literal":{"i64":"7774"}},{"literal":{"i64":"7775"}},{"literal":{"i64":"7776"}},{"literal":{"i64":"7777"}},{"literal":{"i64":"7778"}},{"literal":{"i64":"7779"}},{"literal":{"i64":"778"}},{"literal":{"i64":"7780"}},{"literal":{"i64":"7781"}},{"literal":{"i64":"7782"}},{"literal":{"i64":"7783"}},{"literal":{"i64":"7784"}},{"literal":{"i64":"7785"}},{"literal":{"i64":"7786"}},{"literal":{"i64":"7787"}},{"literal":{"i64":"7788"}},{"literal":{"i64":"7789"}},{"literal":{"i64":"779"}},{"literal":{"i64":"7790"}},{"literal":{"i64":"7791"}},{"literal":{"i64":"7792"}},{"literal":{"i64":"7793"}},{"literal":{"i64":"7794"}},{"literal":{"i64":"7795"}},{"literal":{"i64":"7796"}},{"literal":{"i64":"7797"}},{"literal":{"i64":"7798"}},{"literal":{"i64":"7799"}},{"literal":{"i64":"78"}},{"literal":{"i64":"780"}},{"literal":{"i64":"7800"}},{"literal":{"i64":"7801"}},{"literal":{"i64":"7802"}},{"literal":{"i64":"7803"}},{"literal":{"i64":"7804"}},{"literal":{"i64":"7805"}},{"literal":{"i64":"7806"}},{"literal":{"i64":"7807"}},{"literal":{"i64":"7808"}},{"literal":{"i64":"7809"}},{"literal":{"i64":"781"}},{"literal":{"i64":"7810"}},{"literal":{"i64":"7811"}},{"literal":{"i64":"7812"}},{"literal":{"i64":"7813"}},{"literal":{"i64":"7814"}},{"literal":{"i64":"7815"}},{"literal":{"i64":"7816"}},{"literal":{"i64":"7817"}},{"literal":{"i64":"7818"}},{"literal":{"i64":"7819"}},{"literal":{"i64":"782"}},{"literal":{"i64":"7820"}},{"literal":{"i64":"7821"}},{"literal":{"i64":"7822"}},{"literal":{"i64":"7823"}},{"literal":{"i64":"7824"}},{"literal":{"i64":"7825"}},{"literal":{"i64":"7826"}},{"literal":{"i64":"7827"}},{"literal":{"i64":"7828"}},{"literal":{"i64":"7829"}},{"literal":{"i64":"783"}},{"literal":{"i64":"7830"}},{"literal":{"i64":"7831"}},{"literal":{"i64":"7832"}},{"literal":{"i64":"7833"}},{"literal":{"i64":"7834"}},{"literal":{"i64":"7835"}},{"literal":{"i64":"7836"}},{"literal":{"i64":"7837"}},{"literal":{"i64":"7838"}},{"literal":{"i64":"7839"}},{"literal":{"i64":"784"}},{"literal":{"i64":"7840"}},{"literal":{"i64":"7841"}},{"literal":{"i64":"7842"}},{"literal":{"i64":"7843"}},{"literal":{"i64":"7844"}},{"literal":{"i64":"7845"}},{"literal":{"i64":"7846"}},{"literal":{"i64":"7847"}},{"literal":{"i64":"7848"}},{"literal":{"i64":"7849"}},{"literal":{"i64":"785"}},{"literal":{"i64":"7850"}},{"literal":{"i64":"7851"}},{"literal":{"i64":"7852"}},{"literal":{"i64":"7853"}},{"literal":{"i64":"7854"}},{"literal":{"i64":"7855"}},{"literal":{"i64":"7856"}},{"literal":{"i64":"7857"}},{"literal":{"i64":"7858"}},{"literal":{"i64":"7859"}},{"literal":{"i64":"786"}},{"literal":{"i64":"7860"}},{"literal":{"i64":"7861"}},{"literal":{"i64":"7862"}},{"literal":{"i64":"7863"}},{"literal":{"i64":"7864"}},{"literal":{"i64":"7865"}},{"literal":{"i64":"7866"}},{"literal":{"i64":"7867"}},{"literal":{"i64":"7868"}},{"literal":{"i64":"7869"}},{"literal":{"i64":"787"}},{"literal":{"i64":"7870"}},{"literal":{"i64":"7871"}},{"literal":{"i64":"7872"}},{"literal":{"i64":"7873"}},{"literal":{"i64":"7874"}},{"literal":{"i64":"7875"}},{"literal":{"i64":"7876"}},{"literal":{"i64":"7877"}},{"literal":{"i64":"7878"}},{"literal":{"i64":"7879"}},{"literal":{"i64":"788"}},{"literal":{"i64":"7880"}},{"literal":{"i64":"7881"}},{"literal":{"i64":"7882"}},{"literal":{"i64":"7883"}},{"literal":{"i64":"7884"}},{"literal":{"i64":"7885"}},{"literal":{"i64":"7886"}},{"literal":{"i64":"7887"}},{"literal":{"i64":"7888"}},{"literal":{"i64":"7889"}},{"literal":{"i64":"789"}},{"literal":{"i64":"7890"}},{"literal":{"i64":"7891"}},{"literal":{"i64":"7892"}},{"literal":{"i64":"7893"}},{"literal":{"i64":"7894"}},{"literal":{"i64":"7895"}},{"literal":{"i64":"7896"}},{"literal":{"i64":"7897"}},{"literal":{"i64":"7898"}},{"literal":{"i64":"7899"}},{"literal":{"i64":"79"}},{"literal":{"i64":"790"}},{"literal":{"i64":"7900"}},{"literal":{"i64":"7901"}},{"literal":{"i64":"7902"}},{"literal":{"i64":"7903"}},{"literal":{"i64":"7904"}},{"literal":{"i64":"7905"}},{"literal":{"i64":"7906"}},{"literal":{"i64":"7907"}},{"literal":{"i64":"7908"}},{"literal":{"i64":"7909"}},{"literal":{"i64":"791"}},{"literal":{"i64":"7910"}},{"literal":{"i64":"7911"}},{"literal":{"i64":"7912"}},{"literal":{"i64":"7913"}},{"literal":{"i64":"7914"}},{"literal":{"i64":"7915"}},{"literal":{"i64":"7916"}},{"literal":{"i64":"7917"}},{"literal":{"i64":"7918"}},{"literal":{"i64":"7919"}},{"literal":{"i64":"792"}},{"literal":{"i64":"7920"}},{"literal":{"i64":"7921"}},{"literal":{"i64":"7922"}},{"literal":{"i64":"7923"}},{"literal":{"i64":"7924"}},{"literal":{"i64":"7925"}},{"literal":{"i64":"7926"}},{"literal":{"i64":"7927"}},{"literal":{"i64":"7928"}},{"literal":{"i64":"7929"}},{"literal":{"i64":"793"}},{"literal":{"i64":"7930"}},{"literal":{"i64":"7931"}},{"literal":{"i64":"7932"}},{"literal":{"i64":"7933"}},{"literal":{"i64":"7934"}},{"literal":{"i64":"7935"}},{"literal":{"i64":"7936"}},{"literal":{"i64":"7937"}},{"literal":{"i64":"7938"}},{"literal":{"i64":"7939"}},{"literal":{"i64":"794"}},{"literal":{"i64":"7940"}},{"literal":{"i64":"7941"}},{"literal":{"i64":"7942"}},{"literal":{"i64":"7943"}},{"literal":{"i64":"7944"}},{"literal":{"i64":"7945"}},{"literal":{"i64":"7946"}},{"literal":{"i64":"7947"}},{"literal":{"i64":"7948"}},{"literal":{"i64":"7949"}},{"literal":{"i64":"795"}},{"literal":{"i64":"7950"}},{"literal":{"i64":"7951"}},{"literal":{"i64":"7952"}},{"literal":{"i64":"7953"}},{"literal":{"i64":"7954"}},{"literal":{"i64":"7955"}},{"literal":{"i64":"7956"}},{"literal":{"i64":"7957"}},{"literal":{"i64":"7958"}},{"literal":{"i64":"7959"}},{"literal":{"i64":"796"}},{"literal":{"i64":"7960"}},{"literal":{"i64":"7961"}},{"literal":{"i64":"7962"}},{"literal":{"i64":"7963"}},{"literal":{"i64":"7964"}},{"literal":{"i64":"7965"}},{"literal":{"i64":"7966"}},{"literal":{"i64":"7967"}},{"literal":{"i64":"7968"}},{"literal":{"i64":"7969"}},{"literal":{"i64":"797"}},{"literal":{"i64":"7970"}},{"literal":{"i64":"7971"}},{"literal":{"i64":"7972"}},{"literal":{"i64":"7973"}},{"literal":{"i64":"7974"}},{"literal":{"i64":"7975"}},{"literal":{"i64":"7976"}},{"literal":{"i64":"7977"}},{"literal":{"i64":"7978"}},{"literal":{"i64":"7979"}},{"literal":{"i64":"798"}},{"literal":{"i64":"7980"}},{"literal":{"i64":"7981"}},{"literal":{"i64":"7982"}},{"literal":{"i64":"7983"}},{"literal":{"i64":"7984"}},{"literal":{"i64":"7985"}},{"literal":{"i64":"7986"}},{"literal":{"i64":"7987"}},{"literal":{"i64":"7988"}},{"literal":{"i64":"7989"}},{"literal":{"i64":"799"}},{"literal":{"i64":"7990"}},{"literal":{"i64":"7991"}},{"literal":{"i64":"7992"}},{"literal":{"i64":"7993"}},{"literal":{"i64":"7994"}},{"literal":{"i64":"7995"}},{"literal":{"i64":"7996"}},{"literal":{"i64":"7997"}},{"literal":{"i64":"7998"}},{"literal":{"i64":"7999"}},{"literal":{"i64":"8"}},{"literal":{"i64":"80"}},{"literal":{"i64":"800"}},{"literal":{"i64":"8000"}},{"literal":{"i64":"8001"}},{"literal":{"i64":"8002"}},{"literal":{"i64":"8003"}},{"literal":{"i64":"8004"}},{"literal":{"i64":"8005"}},{"literal":{"i64":"8006"}},{"literal":{"i64":"8007"}},{"literal":{"i64":"8008"}},{"literal":{"i64":"8009"}},{"literal":{"i64":"801"}},{"literal":{"i64":"8010"}},{"literal":{"i64":"8011"}},{"literal":{"i64":"8012"}},{"literal":{"i64":"8013"}},{"literal":{"i64":"8014"}},{"literal":{"i64":"8015"}},{"literal":{"i64":"8016"}},{"literal":{"i64":"8017"}},{"literal":{"i64":"8018"}},{"literal":{"i64":"8019"}},{"literal":{"i64":"802"}},{"literal":{"i64":"8020"}},{"literal":{"i64":"8021"}},{"literal":{"i64":"8022"}},{"literal":{"i64":"8023"}},{"literal":{"i64":"8024"}},{"literal":{"i64":"8025"}},{"literal":{"i64":"8026"}},{"literal":{"i64":"8027"}},{"literal":{"i64":"8028"}},{"literal":{"i64":"8029"}},{"literal":{"i64":"803"}},{"literal":{"i64":"8030"}},{"literal":{"i64":"8031"}},{"literal":{"i64":"8032"}},{"literal":{"i64":"8033"}},{"literal":{"i64":"8034"}},{"literal":{"i64":"8035"}},{"literal":{"i64":"8036"}},{"literal":{"i64":"8037"}},{"literal":{"i64":"8038"}},{"literal":{"i64":"8039"}},{"literal":{"i64":"804"}},{"literal":{"i64":"8040"}},{"literal":{"i64":"8041"}},{"literal":{"i64":"8042"}},{"literal":{"i64":"8043"}},{"literal":{"i64":"8044"}},{"literal":{"i64":"8045"}},{"literal":{"i64":"8046"}},{"literal":{"i64":"8047"}},{"literal":{"i64":"8048"}},{"literal":{"i64":"8049"}},{"literal":{"i64":"805"}},{"literal":{"i64":"8050"}},{"literal":{"i64":"8051"}},{"literal":{"i64":"8052"}},{"literal":{"i64":"8053"}},{"literal":{"i64":"8054"}},{"literal":{"i64":"8055"}},{"literal":{"i64":"8056"}},{"literal":{"i64":"8057"}},{"literal":{"i64":"8058"}},{"literal":{"i64":"8059"}},{"literal":{"i64":"806"}},{"literal":{"i64":"8060"}},{"literal":{"i64":"8061"}},{"literal":{"i64":"8062"}},{"literal":{"i64":"8063"}},{"literal":{"i64":"8064"}},{"literal":{"i64":"8065"}},{"literal":{"i64":"8066"}},{"literal":{"i64":"8067"}},{"literal":{"i64":"8068"}},{"literal":{"i64":"8069"}},{"literal":{"i64":"807"}},{"literal":{"i64":"8070"}},{"literal":{"i64":"8071"}},{"literal":{"i64":"8072"}},{"literal":{"i64":"8073"}},{"literal":{"i64":"8074"}},{"literal":{"i64":"8075"}},{"literal":{"i64":"8076"}},{"literal":{"i64":"8077"}},{"literal":{"i64":"8078"}},{"literal":{"i64":"8079"}},{"literal":{"i64":"808"}},{"literal":{"i64":"8080"}},{"literal":{"i64":"8081"}},{"literal":{"i64":"8082"}},{"literal":{"i64":"8083"}},{"literal":{"i64":"8084"}},{"literal":{"i64":"8085"}},{"literal":{"i64":"8086"}},{"literal":{"i64":"8087"}},{"literal":{"i64":"8088"}},{"literal":{"i64":"8089"}},{"literal":{"i64":"809"}},{"literal":{"i64":"8090"}},{"literal":{"i64":"8091"}},{"literal":{"i64":"8092"}},{"literal":{"i64":"8093"}},{"literal":{"i64":"8094"}},{"literal":{"i64":"8095"}},{"literal":{"i64":"8096"}},{"literal":{"i64":"8097"}},{"literal":{"i64":"8098"}},{"literal":{"i64":"8099"}},{"literal":{"i64":"81"}},{"literal":{"i64":"810"}},{"literal":{"i64":"8100"}},{"literal":{"i64":"8101"}},{"literal":{"i64":"8102"}},{"literal":{"i64":"8103"}},{"literal":{"i64":"8104"}},{"literal":{"i64":"8105"}},{"literal":{"i64":"8106"}},{"literal":{"i64":"8107"}},{"literal":{"i64":"8108"}},{"literal":{"i64":"8109"}},{"literal":{"i64":"811"}},{"literal":{"i64":"8110"}},{"literal":{"i64":"8111"}},{"literal":{"i64":"8112"}},{"literal":{"i64":"8113"}},{"literal":{"i64":"8114"}},{"literal":{"i64":"8115"}},{"literal":{"i64":"8116"}},{"literal":{"i64":"8117"}},{"literal":{"i64":"8118"}},{"literal":{"i64":"8119"}},{"literal":{"i64":"812"}},{"literal":{"i64":"8120"}},{"literal":{"i64":"8121"}},{"literal":{"i64":"8122"}},{"literal":{"i64":"8123"}},{"literal":{"i64":"8124"}},{"literal":{"i64":"8125"}},{"literal":{"i64":"8126"}},{"literal":{"i64":"8127"}},{"literal":{"i64":"8128"}},{"literal":{"i64":"8129"}},{"literal":{"i64":"813"}},{"literal":{"i64":"8130"}},{"literal":{"i64":"8131"}},{"literal":{"i64":"8132"}},{"literal":{"i64":"8133"}},{"literal":{"i64":"8134"}},{"literal":{"i64":"8135"}},{"literal":{"i64":"8136"}},{"literal":{"i64":"8137"}},{"literal":{"i64":"8138"}},{"literal":{"i64":"8139"}},{"literal":{"i64":"814"}},{"literal":{"i64":"8140"}},{"literal":{"i64":"8141"}},{"literal":{"i64":"8142"}},{"literal":{"i64":"8143"}},{"literal":{"i64":"8144"}},{"literal":{"i64":"8145"}},{"literal":{"i64":"8146"}},{"literal":{"i64":"8147"}},{"literal":{"i64":"8148"}},{"literal":{"i64":"8149"}},{"literal":{"i64":"815"}},{"literal":{"i64":"8150"}},{"literal":{"i64":"8151"}},{"literal":{"i64":"8152"}},{"literal":{"i64":"8153"}},{"literal":{"i64":"8154"}},{"literal":{"i64":"8155"}},{"literal":{"i64":"8156"}},{"literal":{"i64":"8157"}},{"literal":{"i64":"8158"}},{"literal":{"i64":"8159"}},{"literal":{"i64":"816"}},{"literal":{"i64":"8160"}},{"literal":{"i64":"8161"}},{"literal":{"i64":"8162"}},{"literal":{"i64":"8163"}},{"literal":{"i64":"8164"}},{"literal":{"i64":"8165"}},{"literal":{"i64":"8166"}},{"literal":{"i64":"8167"}},{"literal":{"i64":"8168"}},{"literal":{"i64":"8169"}},{"literal":{"i64":"817"}},{"literal":{"i64":"8170"}},{"literal":{"i64":"8171"}},{"literal":{"i64":"8172"}},{"literal":{"i64":"8173"}},{"literal":{"i64":"8174"}},{"literal":{"i64":"8175"}},{"literal":{"i64":"8176"}},{"literal":{"i64":"8177"}},{"literal":{"i64":"8178"}},{"literal":{"i64":"8179"}},{"literal":{"i64":"818"}},{"literal":{"i64":"8180"}},{"literal":{"i64":"8181"}},{"literal":{"i64":"8182"}},{"literal":{"i64":"8183"}},{"literal":{"i64":"8184"}},{"literal":{"i64":"8185"}},{"literal":{"i64":"8186"}},{"literal":{"i64":"8187"}},{"literal":{"i64":"8188"}},{"literal":{"i64":"8189"}},{"literal":{"i64":"819"}},{"literal":{"i64":"8190"}},{"literal":{"i64":"8191"}},{"literal":{"i64":"8192"}},{"literal":{"i64":"8193"}},{"literal":{"i64":"8194"}},{"literal":{"i64":"8195"}},{"literal":{"i64":"8196"}},{"literal":{"i64":"8197"}},{"literal":{"i64":"8198"}},{"literal":{"i64":"8199"}},{"literal":{"i64":"82"}},{"literal":{"i64":"820"}},{"literal":{"i64":"8200"}},{"literal":{"i64":"8201"}},{"literal":{"i64":"8202"}},{"literal":{"i64":"8203"}},{"literal":{"i64":"8204"}},{"literal":{"i64":"8205"}},{"literal":{"i64":"8206"}},{"literal":{"i64":"8207"}},{"literal":{"i64":"8208"}},{"literal":{"i64":"8209"}},{"literal":{"i64":"821"}},{"literal":{"i64":"8210"}},{"literal":{"i64":"8211"}},{"literal":{"i64":"8212"}},{"literal":{"i64":"8213"}},{"literal":{"i64":"8214"}},{"literal":{"i64":"8215"}},{"literal":{"i64":"8216"}},{"literal":{"i64":"8217"}},{"literal":{"i64":"8218"}},{"literal":{"i64":"8219"}},{"literal":{"i64":"822"}},{"literal":{"i64":"8220"}},{"literal":{"i64":"8221"}},{"literal":{"i64":"8222"}},{"literal":{"i64":"8223"}},{"literal":{"i64":"8224"}},{"literal":{"i64":"8225"}},{"literal":{"i64":"8226"}},{"literal":{"i64":"8227"}},{"literal":{"i64":"8228"}},{"literal":{"i64":"8229"}},{"literal":{"i64":"823"}},{"literal":{"i64":"8230"}},{"literal":{"i64":"8231"}},{"literal":{"i64":"8232"}},{"literal":{"i64":"8233"}},{"literal":{"i64":"8234"}},{"literal":{"i64":"8235"}},{"literal":{"i64":"8236"}},{"literal":{"i64":"8237"}},{"literal":{"i64":"8238"}},{"literal":{"i64":"8239"}},{"literal":{"i64":"824"}},{"literal":{"i64":"8240"}},{"literal":{"i64":"8241"}},{"literal":{"i64":"8242"}},{"literal":{"i64":"8243"}},{"literal":{"i64":"8244"}},{"literal":{"i64":"8245"}},{"literal":{"i64":"8246"}},{"literal":{"i64":"8247"}},{"literal":{"i64":"8248"}},{"literal":{"i64":"8249"}},{"literal":{"i64":"825"}},{"literal":{"i64":"8250"}},{"literal":{"i64":"8251"}},{"literal":{"i64":"8252"}},{"literal":{"i64":"8253"}},{"literal":{"i64":"8254"}},{"literal":{"i64":"8255"}},{"literal":{"i64":"8256"}},{"literal":{"i64":"8257"}},{"literal":{"i64":"8258"}},{"literal":{"i64":"8259"}},{"literal":{"i64":"826"}},{"literal":{"i64":"8260"}},{"literal":{"i64":"8261"}},{"literal":{"i64":"8262"}},{"literal":{"i64":"8263"}},{"literal":{"i64":"8264"}},{"literal":{"i64":"8265"}},{"literal":{"i64":"8266"}},{"literal":{"i64":"8267"}},{"literal":{"i64":"8268"}},{"literal":{"i64":"8269"}},{"literal":{"i64":"827"}},{"literal":{"i64":"8270"}},{"literal":{"i64":"8271"}},{"literal":{"i64":"8272"}},{"literal":{"i64":"8273"}},{"literal":{"i64":"8274"}},{"literal":{"i64":"8275"}},{"literal":{"i64":"8276"}},{"literal":{"i64":"8277"}},{"literal":{"i64":"8278"}},{"literal":{"i64":"8279"}},{"literal":{"i64":"828"}},{"literal":{"i64":"8280"}},{"literal":{"i64":"8281"}},{"literal":{"i64":"8282"}},{"literal":{"i64":"8283"}},{"literal":{"i64":"8284"}},{"literal":{"i64":"8285"}},{"literal":{"i64":"8286"}},{"literal":{"i64":"8287"}},{"literal":{"i64":"8288"}},{"literal":{"i64":"8289"}},{"literal":{"i64":"829"}},{"literal":{"i64":"8290"}},{"literal":{"i64":"8291"}},{"literal":{"i64":"8292"}},{"literal":{"i64":"8293"}},{"literal":{"i64":"8294"}},{"literal":{"i64":"8295"}},{"literal":{"i64":"8296"}},{"literal":{"i64":"8297"}},{"literal":{"i64":"8298"}},{"literal":{"i64":"8299"}},{"literal":{"i64":"83"}},{"literal":{"i64":"830"}},{"literal":{"i64":"8300"}},{"literal":{"i64":"8301"}},{"literal":{"i64":"8302"}},{"literal":{"i64":"8303"}},{"literal":{"i64":"8304"}},{"literal":{"i64":"8305"}},{"literal":{"i64":"8306"}},{"literal":{"i64":"8307"}},{"literal":{"i64":"8308"}},{"literal":{"i64":"8309"}},{"literal":{"i64":"831"}},{"literal":{"i64":"8310"}},{"literal":{"i64":"8311"}},{"literal":{"i64":"8312"}},{"literal":{"i64":"8313"}},{"literal":{"i64":"8314"}},{"literal":{"i64":"8315"}},{"literal":{"i64":"8316"}},{"literal":{"i64":"8317"}},{"literal":{"i64":"8318"}},{"literal":{"i64":"8319"}},{"literal":{"i64":"832"}},{"literal":{"i64":"8320"}},{"literal":{"i64":"8321"}},{"literal":{"i64":"8322"}},{"literal":{"i64":"8323"}},{"literal":{"i64":"8324"}},{"literal":{"i64":"8325"}},{"literal":{"i64":"8326"}},{"literal":{"i64":"8327"}},{"literal":{"i64":"8328"}},{"literal":{"i64":"8329"}},{"literal":{"i64":"833"}},{"literal":{"i64":"8330"}},{"literal":{"i64":"8331"}},{"literal":{"i64":"8332"}},{"literal":{"i64":"8333"}},{"literal":{"i64":"8334"}},{"literal":{"i64":"8335"}},{"literal":{"i64":"8336"}},{"literal":{"i64":"8337"}},{"literal":{"i64":"8338"}},{"literal":{"i64":"8339"}},{"literal":{"i64":"834"}},{"literal":{"i64":"8340"}},{"literal":{"i64":"8341"}},{"literal":{"i64":"8342"}},{"literal":{"i64":"8343"}},{"literal":{"i64":"8344"}},{"literal":{"i64":"8345"}},{"literal":{"i64":"8346"}},{"literal":{"i64":"8347"}},{"literal":{"i64":"8348"}},{"literal":{"i64":"8349"}},{"literal":{"i64":"835"}},{"literal":{"i64":"8350"}},{"literal":{"i64":"8351"}},{"literal":{"i64":"8352"}},{"literal":{"i64":"8353"}},{"literal":{"i64":"8354"}},{"literal":{"i64":"8355"}},{"literal":{"i64":"8356"}},{"literal":{"i64":"8357"}},{"literal":{"i64":"8358"}},{"literal":{"i64":"8359"}},{"literal":{"i64":"836"}},{"literal":{"i64":"8360"}},{"literal":{"i64":"8361"}},{"literal":{"i64":"8362"}},{"literal":{"i64":"8363"}},{"literal":{"i64":"8364"}},{"literal":{"i64":"8365"}},{"literal":{"i64":"8366"}},{"literal":{"i64":"8367"}},{"literal":{"i64":"8368"}},{"literal":{"i64":"8369"}},{"literal":{"i64":"837"}},{"literal":{"i64":"8370"}},{"literal":{"i64":"8371"}},{"literal":{"i64":"8372"}},{"literal":{"i64":"8373"}},{"literal":{"i64":"8374"}},{"literal":{"i64":"8375"}},{"literal":{"i64":"8376"}},{"literal":{"i64":"8377"}},{"literal":{"i64":"8378"}},{"literal":{"i64":"8379"}},{"literal":{"i64":"838"}},{"literal":{"i64":"8380"}},{"literal":{"i64":"8381"}},{"literal":{"i64":"8382"}},{"literal":{"i64":"8383"}},{"literal":{"i64":"8384"}},{"literal":{"i64":"8385"}},{"literal":{"i64":"8386"}},{"literal":{"i64":"8387"}},{"literal":{"i64":"8388"}},{"literal":{"i64":"8389"}},{"literal":{"i64":"839"}},{"literal":{"i64":"8390"}},{"literal":{"i64":"8391"}},{"literal":{"i64":"8392"}},{"literal":{"i64":"8393"}},{"literal":{"i64":"8394"}},{"literal":{"i64":"8395"}},{"literal":{"i64":"8396"}},{"literal":{"i64":"8397"}},{"literal":{"i64":"8398"}},{"literal":{"i64":"8399"}},{"literal":{"i64":"84"}},{"literal":{"i64":"840"}},{"literal":{"i64":"8400"}},{"literal":{"i64":"8401"}},{"literal":{"i64":"8402"}},{"literal":{"i64":"8403"}},{"literal":{"i64":"8404"}},{"literal":{"i64":"8405"}},{"literal":{"i64":"8406"}},{"literal":{"i64":"8407"}},{"literal":{"i64":"8408"}},{"literal":{"i64":"8409"}},{"literal":{"i64":"841"}},{"literal":{"i64":"8410"}},{"literal":{"i64":"8411"}},{"literal":{"i64":"8412"}},{"literal":{"i64":"8413"}},{"literal":{"i64":"8414"}},{"literal":{"i64":"8415"}},{"literal":{"i64":"8416"}},{"literal":{"i64":"8417"}},{"literal":{"i64":"8418"}},{"literal":{"i64":"8419"}},{"literal":{"i64":"842"}},{"literal":{"i64":"8420"}},{"literal":{"i64":"8421"}},{"literal":{"i64":"8422"}},{"literal":{"i64":"8423"}},{"literal":{"i64":"8424"}},{"literal":{"i64":"8425"}},{"literal":{"i64":"8426"}},{"literal":{"i64":"8427"}},{"literal":{"i64":"8428"}},{"literal":{"i64":"8429"}},{"literal":{"i64":"843"}},{"literal":{"i64":"8430"}},{"literal":{"i64":"8431"}},{"literal":{"i64":"8432"}},{"literal":{"i64":"8433"}},{"literal":{"i64":"8434"}},{"literal":{"i64":"8435"}},{"literal":{"i64":"8436"}},{"literal":{"i64":"8437"}},{"literal":{"i64":"8438"}},{"literal":{"i64":"8439"}},{"literal":{"i64":"844"}},{"literal":{"i64":"8440"}},{"literal":{"i64":"8441"}},{"literal":{"i64":"8442"}},{"literal":{"i64":"8443"}},{"literal":{"i64":"8444"}},{"literal":{"i64":"8445"}},{"literal":{"i64":"8446"}},{"literal":{"i64":"8447"}},{"literal":{"i64":"8448"}},{"literal":{"i64":"8449"}},{"literal":{"i64":"845"}},{"literal":{"i64":"8450"}},{"literal":{"i64":"8451"}},{"literal":{"i64":"8452"}},{"literal":{"i64":"8453"}},{"literal":{"i64":"8454"}},{"literal":{"i64":"8455"}},{"literal":{"i64":"8456"}},{"literal":{"i64":"8457"}},{"literal":{"i64":"8458"}},{"literal":{"i64":"8459"}},{"literal":{"i64":"846"}},{"literal":{"i64":"8460"}},{"literal":{"i64":"8461"}},{"literal":{"i64":"8462"}},{"literal":{"i64":"8463"}},{"literal":{"i64":"8464"}},{"literal":{"i64":"8465"}},{"literal":{"i64":"8466"}},{"literal":{"i64":"8467"}},{"literal":{"i64":"8468"}},{"literal":{"i64":"8469"}},{"literal":{"i64":"847"}},{"literal":{"i64":"8470"}},{"literal":{"i64":"8471"}},{"literal":{"i64":"8472"}},{"literal":{"i64":"8473"}},{"literal":{"i64":"8474"}},{"literal":{"i64":"8475"}},{"literal":{"i64":"8476"}},{"literal":{"i64":"8477"}},{"literal":{"i64":"8478"}},{"literal":{"i64":"8479"}},{"literal":{"i64":"848"}},{"literal":{"i64":"8480"}},{"literal":{"i64":"8481"}},{"literal":{"i64":"8482"}},{"literal":{"i64":"8483"}},{"literal":{"i64":"8484"}},{"literal":{"i64":"8485"}},{"literal":{"i64":"8486"}},{"literal":{"i64":"8487"}},{"literal":{"i64":"8488"}},{"literal":{"i64":"8489"}},{"literal":{"i64":"849"}},{"literal":{"i64":"8490"}},{"literal":{"i64":"8491"}},{"literal":{"i64":"8492"}},{"literal":{"i64":"8493"}},{"literal":{"i64":"8494"}},{"literal":{"i64":"8495"}},{"literal":{"i64":"8496"}},{"literal":{"i64":"8497"}},{"literal":{"i64":"8498"}},{"literal":{"i64":"8499"}},{"literal":{"i64":"85"}},{"literal":{"i64":"850"}},{"literal":{"i64":"8500"}},{"literal":{"i64":"8501"}},{"literal":{"i64":"8502"}},{"literal":{"i64":"8503"}},{"literal":{"i64":"8504"}},{"literal":{"i64":"8505"}},{"literal":{"i64":"8506"}},{"literal":{"i64":"8507"}},{"literal":{"i64":"8508"}},{"literal":{"i64":"8509"}},{"literal":{"i64":"851"}},{"literal":{"i64":"8510"}},{"literal":{"i64":"8511"}},{"literal":{"i64":"8512"}},{"literal":{"i64":"8513"}},{"literal":{"i64":"8514"}},{"literal":{"i64":"8515"}},{"literal":{"i64":"8516"}},{"literal":{"i64":"8517"}},{"literal":{"i64":"8518"}},{"literal":{"i64":"8519"}},{"literal":{"i64":"852"}},{"literal":{"i64":"8520"}},{"literal":{"i64":"8521"}},{"literal":{"i64":"8522"}},{"literal":{"i64":"8523"}},{"literal":{"i64":"8524"}},{"literal":{"i64":"8525"}},{"literal":{"i64":"8526"}},{"literal":{"i64":"8527"}},{"literal":{"i64":"8528"}},{"literal":{"i64":"8529"}},{"literal":{"i64":"853"}},{"literal":{"i64":"8530"}},{"literal":{"i64":"8531"}},{"literal":{"i64":"8532"}},{"literal":{"i64":"8533"}},{"literal":{"i64":"8534"}},{"literal":{"i64":"8535"}},{"literal":{"i64":"8536"}},{"literal":{"i64":"8537"}},{"literal":{"i64":"8538"}},{"literal":{"i64":"8539"}},{"literal":{"i64":"854"}},{"literal":{"i64":"8540"}},{"literal":{"i64":"8541"}},{"literal":{"i64":"8542"}},{"literal":{"i64":"8543"}},{"literal":{"i64":"8544"}},{"literal":{"i64":"8545"}},{"literal":{"i64":"8546"}},{"literal":{"i64":"8547"}},{"literal":{"i64":"8548"}},{"literal":{"i64":"8549"}},{"literal":{"i64":"855"}},{"literal":{"i64":"8550"}},{"literal":{"i64":"8551"}},{"literal":{"i64":"8552"}},{"literal":{"i64":"8553"}},{"literal":{"i64":"8554"}},{"literal":{"i64":"8555"}},{"literal":{"i64":"8556"}},{"literal":{"i64":"8557"}},{"literal":{"i64":"8558"}},{"literal":{"i64":"8559"}},{"literal":{"i64":"856"}},{"literal":{"i64":"8560"}},{"literal":{"i64":"8561"}},{"literal":{"i64":"8562"}},{"literal":{"i64":"8563"}},{"literal":{"i64":"8564"}},{"literal":{"i64":"8565"}},{"literal":{"i64":"8566"}},{"literal":{"i64":"8567"}},{"literal":{"i64":"8568"}},{"literal":{"i64":"8569"}},{"literal":{"i64":"857"}},{"literal":{"i64":"8570"}},{"literal":{"i64":"8571"}},{"literal":{"i64":"8572"}},{"literal":{"i64":"8573"}},{"literal":{"i64":"8574"}},{"literal":{"i64":"8575"}},{"literal":{"i64":"8576"}},{"literal":{"i64":"8577"}},{"literal":{"i64":"8578"}},{"literal":{"i64":"8579"}},{"literal":{"i64":"858"}},{"literal":{"i64":"8580"}},{"literal":{"i64":"8581"}},{"literal":{"i64":"8582"}},{"literal":{"i64":"8583"}},{"literal":{"i64":"8584"}},{"literal":{"i64":"8585"}},{"literal":{"i64":"8586"}},{"literal":{"i64":"8587"}},{"literal":{"i64":"8588"}},{"literal":{"i64":"8589"}},{"literal":{"i64":"859"}},{"literal":{"i64":"8590"}},{"literal":{"i64":"8591"}},{"literal":{"i64":"8592"}},{"literal":{"i64":"8593"}},{"literal":{"i64":"8594"}},{"literal":{"i64":"8595"}},{"literal":{"i64":"8596"}},{"literal":{"i64":"8597"}},{"literal":{"i64":"8598"}},{"literal":{"i64":"8599"}},{"literal":{"i64":"86"}},{"literal":{"i64":"860"}},{"literal":{"i64":"8600"}},{"literal":{"i64":"8601"}},{"literal":{"i64":"8602"}},{"literal":{"i64":"8603"}},{"literal":{"i64":"8604"}},{"literal":{"i64":"8605"}},{"literal":{"i64":"8606"}},{"literal":{"i64":"8607"}},{"literal":{"i64":"8608"}},{"literal":{"i64":"8609"}},{"literal":{"i64":"861"}},{"literal":{"i64":"8610"}},{"literal":{"i64":"8611"}},{"literal":{"i64":"8612"}},{"literal":{"i64":"8613"}},{"literal":{"i64":"8614"}},{"literal":{"i64":"8615"}},{"literal":{"i64":"8616"}},{"literal":{"i64":"8617"}},{"literal":{"i64":"8618"}},{"literal":{"i64":"8619"}},{"literal":{"i64":"862"}},{"literal":{"i64":"8620"}},{"literal":{"i64":"8621"}},{"literal":{"i64":"8622"}},{"literal":{"i64":"8623"}},{"literal":{"i64":"8624"}},{"literal":{"i64":"8625"}},{"literal":{"i64":"8626"}},{"literal":{"i64":"8627"}},{"literal":{"i64":"8628"}},{"literal":{"i64":"8629"}},{"literal":{"i64":"863"}},{"literal":{"i64":"8630"}},{"literal":{"i64":"8631"}},{"literal":{"i64":"8632"}},{"literal":{"i64":"8633"}},{"literal":{"i64":"8634"}},{"literal":{"i64":"8635"}},{"literal":{"i64":"8636"}},{"literal":{"i64":"8637"}},{"literal":{"i64":"8638"}},{"literal":{"i64":"8639"}},{"literal":{"i64":"864"}},{"literal":{"i64":"8640"}},{"literal":{"i64":"8641"}},{"literal":{"i64":"8642"}},{"literal":{"i64":"8643"}},{"literal":{"i64":"8644"}},{"literal":{"i64":"8645"}},{"literal":{"i64":"8646"}},{"literal":{"i64":"8647"}},{"literal":{"i64":"8648"}},{"literal":{"i64":"8649"}},{"literal":{"i64":"865"}},{"literal":{"i64":"8650"}},{"literal":{"i64":"8651"}},{"literal":{"i64":"8652"}},{"literal":{"i64":"8653"}},{"literal":{"i64":"8654"}},{"literal":{"i64":"8655"}},{"literal":{"i64":"8656"}},{"literal":{"i64":"8657"}},{"literal":{"i64":"8658"}},{"literal":{"i64":"8659"}},{"literal":{"i64":"866"}},{"literal":{"i64":"8660"}},{"literal":{"i64":"8661"}},{"literal":{"i64":"8662"}},{"literal":{"i64":"8663"}},{"literal":{"i64":"8664"}},{"literal":{"i64":"8665"}},{"literal":{"i64":"8666"}},{"literal":{"i64":"8667"}},{"literal":{"i64":"8668"}},{"literal":{"i64":"8669"}},{"literal":{"i64":"867"}},{"literal":{"i64":"8670"}},{"literal":{"i64":"8671"}},{"literal":{"i64":"8672"}},{"literal":{"i64":"8673"}},{"literal":{"i64":"8674"}},{"literal":{"i64":"8675"}},{"literal":{"i64":"8676"}},{"literal":{"i64":"8677"}},{"literal":{"i64":"8678"}},{"literal":{"i64":"8679"}},{"literal":{"i64":"868"}},{"literal":{"i64":"8680"}},{"literal":{"i64":"8681"}},{"literal":{"i64":"8682"}},{"literal":{"i64":"8683"}},{"literal":{"i64":"8684"}},{"literal":{"i64":"8685"}},{"literal":{"i64":"8686"}},{"literal":{"i64":"8687"}},{"literal":{"i64":"8688"}},{"literal":{"i64":"8689"}},{"literal":{"i64":"869"}},{"literal":{"i64":"8690"}},{"literal":{"i64":"8691"}},{"literal":{"i64":"8692"}},{"literal":{"i64":"8693"}},{"literal":{"i64":"8694"}},{"literal":{"i64":"8695"}},{"literal":{"i64":"8696"}},{"literal":{"i64":"8697"}},{"literal":{"i64":"8698"}},{"literal":{"i64":"8699"}},{"literal":{"i64":"87"}},{"literal":{"i64":"870"}},{"literal":{"i64":"8700"}},{"literal":{"i64":"8701"}},{"literal":{"i64":"8702"}},{"literal":{"i64":"8703"}},{"literal":{"i64":"8704"}},{"literal":{"i64":"8705"}},{"literal":{"i64":"8706"}},{"literal":{"i64":"8707"}},{"literal":{"i64":"8708"}},{"literal":{"i64":"8709"}},{"literal":{"i64":"871"}},{"literal":{"i64":"8710"}},{"literal":{"i64":"8711"}},{"literal":{"i64":"8712"}},{"literal":{"i64":"8713"}},{"literal":{"i64":"8714"}},{"literal":{"i64":"8715"}},{"literal":{"i64":"8716"}},{"literal":{"i64":"8717"}},{"literal":{"i64":"8718"}},{"literal":{"i64":"8719"}},{"literal":{"i64":"872"}},{"literal":{"i64":"8720"}},{"literal":{"i64":"8721"}},{"literal":{"i64":"8722"}},{"literal":{"i64":"8723"}},{"literal":{"i64":"8724"}},{"literal":{"i64":"8725"}},{"literal":{"i64":"8726"}},{"literal":{"i64":"8727"}},{"literal":{"i64":"8728"}},{"literal":{"i64":"8729"}},{"literal":{"i64":"873"}},{"literal":{"i64":"8730"}},{"literal":{"i64":"8731"}},{"literal":{"i64":"8732"}},{"literal":{"i64":"8733"}},{"literal":{"i64":"8734"}},{"literal":{"i64":"8735"}},{"literal":{"i64":"8736"}},{"literal":{"i64":"8737"}},{"literal":{"i64":"8738"}},{"literal":{"i64":"8739"}},{"literal":{"i64":"874"}},{"literal":{"i64":"8740"}},{"literal":{"i64":"8741"}},{"literal":{"i64":"8742"}},{"literal":{"i64":"8743"}},{"literal":{"i64":"8744"}},{"literal":{"i64":"8745"}},{"literal":{"i64":"8746"}},{"literal":{"i64":"8747"}},{"literal":{"i64":"8748"}},{"literal":{"i64":"8749"}},{"literal":{"i64":"875"}},{"literal":{"i64":"8750"}},{"literal":{"i64":"8751"}},{"literal":{"i64":"8752"}},{"literal":{"i64":"8753"}},{"literal":{"i64":"8754"}},{"literal":{"i64":"8755"}},{"literal":{"i64":"8756"}},{"literal":{"i64":"8757"}},{"literal":{"i64":"8758"}},{"literal":{"i64":"8759"}},{"literal":{"i64":"876"}},{"literal":{"i64":"8760"}},{"literal":{"i64":"8761"}},{"literal":{"i64":"8762"}},{"literal":{"i64":"8763"}},{"literal":{"i64":"8764"}},{"literal":{"i64":"8765"}},{"literal":{"i64":"8766"}},{"literal":{"i64":"8767"}},{"literal":{"i64":"8768"}},{"literal":{"i64":"8769"}},{"literal":{"i64":"877"}},{"literal":{"i64":"8770"}},{"literal":{"i64":"8771"}},{"literal":{"i64":"8772"}},{"literal":{"i64":"8773"}},{"literal":{"i64":"8774"}},{"literal":{"i64":"8775"}},{"literal":{"i64":"8776"}},{"literal":{"i64":"8777"}},{"literal":{"i64":"8778"}},{"literal":{"i64":"8779"}},{"literal":{"i64":"878"}},{"literal":{"i64":"8780"}},{"literal":{"i64":"8781"}},{"literal":{"i64":"8782"}},{"literal":{"i64":"8783"}},{"literal":{"i64":"8784"}},{"literal":{"i64":"8785"}},{"literal":{"i64":"8786"}},{"literal":{"i64":"8787"}},{"literal":{"i64":"8788"}},{"literal":{"i64":"8789"}},{"literal":{"i64":"879"}},{"literal":{"i64":"8790"}},{"literal":{"i64":"8791"}},{"literal":{"i64":"8792"}},{"literal":{"i64":"8793"}},{"literal":{"i64":"8794"}},{"literal":{"i64":"8795"}},{"literal":{"i64":"8796"}},{"literal":{"i64":"8797"}},{"literal":{"i64":"8798"}},{"literal":{"i64":"8799"}},{"literal":{"i64":"88"}},{"literal":{"i64":"880"}},{"literal":{"i64":"8800"}},{"literal":{"i64":"8801"}},{"literal":{"i64":"8802"}},{"literal":{"i64":"8803"}},{"literal":{"i64":"8804"}},{"literal":{"i64":"8805"}},{"literal":{"i64":"8806"}},{"literal":{"i64":"8807"}},{"literal":{"i64":"8808"}},{"literal":{"i64":"8809"}},{"literal":{"i64":"881"}},{"literal":{"i64":"8810"}},{"literal":{"i64":"8811"}},{"literal":{"i64":"8812"}},{"literal":{"i64":"8813"}},{"literal":{"i64":"8814"}},{"literal":{"i64":"8815"}},{"literal":{"i64":"8816"}},{"literal":{"i64":"8817"}},{"literal":{"i64":"8818"}},{"literal":{"i64":"8819"}},{"literal":{"i64":"882"}},{"literal":{"i64":"8820"}},{"literal":{"i64":"8821"}},{"literal":{"i64":"8822"}},{"literal":{"i64":"8823"}},{"literal":{"i64":"8824"}},{"literal":{"i64":"8825"}},{"literal":{"i64":"8826"}},{"literal":{"i64":"8827"}},{"literal":{"i64":"8828"}},{"literal":{"i64":"8829"}},{"literal":{"i64":"883"}},{"literal":{"i64":"8830"}},{"literal":{"i64":"8831"}},{"literal":{"i64":"8832"}},{"literal":{"i64":"8833"}},{"literal":{"i64":"8834"}},{"literal":{"i64":"8835"}},{"literal":{"i64":"8836"}},{"literal":{"i64":"8837"}},{"literal":{"i64":"8838"}},{"literal":{"i64":"8839"}},{"literal":{"i64":"884"}},{"literal":{"i64":"8840"}},{"literal":{"i64":"8841"}},{"literal":{"i64":"8842"}},{"literal":{"i64":"8843"}},{"literal":{"i64":"8844"}},{"literal":{"i64":"8845"}},{"literal":{"i64":"8846"}},{"literal":{"i64":"8847"}},{"literal":{"i64":"8848"}},{"literal":{"i64":"8849"}},{"literal":{"i64":"885"}},{"literal":{"i64":"8850"}},{"literal":{"i64":"8851"}},{"literal":{"i64":"8852"}},{"literal":{"i64":"8853"}},{"literal":{"i64":"8854"}},{"literal":{"i64":"8855"}},{"literal":{"i64":"8856"}},{"literal":{"i64":"8857"}},{"literal":{"i64":"8858"}},{"literal":{"i64":"8859"}},{"literal":{"i64":"886"}},{"literal":{"i64":"8860"}},{"literal":{"i64":"8861"}},{"literal":{"i64":"8862"}},{"literal":{"i64":"8863"}},{"literal":{"i64":"8864"}},{"literal":{"i64":"8865"}},{"literal":{"i64":"8866"}},{"literal":{"i64":"8867"}},{"literal":{"i64":"8868"}},{"literal":{"i64":"8869"}},{"literal":{"i64":"887"}},{"literal":{"i64":"8870"}},{"literal":{"i64":"8871"}},{"literal":{"i64":"8872"}},{"literal":{"i64":"8873"}},{"literal":{"i64":"8874"}},{"literal":{"i64":"8875"}},{"literal":{"i64":"8876"}},{"literal":{"i64":"8877"}},{"literal":{"i64":"8878"}},{"literal":{"i64":"8879"}},{"literal":{"i64":"888"}},{"literal":{"i64":"8880"}},{"literal":{"i64":"8881"}},{"literal":{"i64":"8882"}},{"literal":{"i64":"8883"}},{"literal":{"i64":"8884"}},{"literal":{"i64":"8885"}},{"literal":{"i64":"8886"}},{"literal":{"i64":"8887"}},{"literal":{"i64":"8888"}},{"literal":{"i64":"8889"}},{"literal":{"i64":"889"}},{"literal":{"i64":"8890"}},{"literal":{"i64":"8891"}},{"literal":{"i64":"8892"}},{"literal":{"i64":"8893"}},{"literal":{"i64":"8894"}},{"literal":{"i64":"8895"}},{"literal":{"i64":"8896"}},{"literal":{"i64":"8897"}},{"literal":{"i64":"8898"}},{"literal":{"i64":"8899"}},{"literal":{"i64":"89"}},{"literal":{"i64":"890"}},{"literal":{"i64":"8900"}},{"literal":{"i64":"8901"}},{"literal":{"i64":"8902"}},{"literal":{"i64":"8903"}},{"literal":{"i64":"8904"}},{"literal":{"i64":"8905"}},{"literal":{"i64":"8906"}},{"literal":{"i64":"8907"}},{"literal":{"i64":"8908"}},{"literal":{"i64":"8909"}},{"literal":{"i64":"891"}},{"literal":{"i64":"8910"}},{"literal":{"i64":"8911"}},{"literal":{"i64":"8912"}},{"literal":{"i64":"8913"}},{"literal":{"i64":"8914"}},{"literal":{"i64":"8915"}},{"literal":{"i64":"8916"}},{"literal":{"i64":"8917"}},{"literal":{"i64":"8918"}},{"literal":{"i64":"8919"}},{"literal":{"i64":"892"}},{"literal":{"i64":"8920"}},{"literal":{"i64":"8921"}},{"literal":{"i64":"8922"}},{"literal":{"i64":"8923"}},{"literal":{"i64":"8924"}},{"literal":{"i64":"8925"}},{"literal":{"i64":"8926"}},{"literal":{"i64":"8927"}},{"literal":{"i64":"8928"}},{"literal":{"i64":"8929"}},{"literal":{"i64":"893"}},{"literal":{"i64":"8930"}},{"literal":{"i64":"8931"}},{"literal":{"i64":"8932"}},{"literal":{"i64":"8933"}},{"literal":{"i64":"8934"}},{"literal":{"i64":"8935"}},{"literal":{"i64":"8936"}},{"literal":{"i64":"8937"}},{"literal":{"i64":"8938"}},{"literal":{"i64":"8939"}},{"literal":{"i64":"894"}},{"literal":{"i64":"8940"}},{"literal":{"i64":"8941"}},{"literal":{"i64":"8942"}},{"literal":{"i64":"8943"}},{"literal":{"i64":"8944"}},{"literal":{"i64":"8945"}},{"literal":{"i64":"8946"}},{"literal":{"i64":"8947"}},{"literal":{"i64":"8948"}},{"literal":{"i64":"8949"}},{"literal":{"i64":"895"}},{"literal":{"i64":"8950"}},{"literal":{"i64":"8951"}},{"literal":{"i64":"8952"}},{"literal":{"i64":"8953"}},{"literal":{"i64":"8954"}},{"literal":{"i64":"8955"}},{"literal":{"i64":"8956"}},{"literal":{"i64":"8957"}},{"literal":{"i64":"8958"}},{"literal":{"i64":"8959"}},{"literal":{"i64":"896"}},{"literal":{"i64":"8960"}},{"literal":{"i64":"8961"}},{"literal":{"i64":"8962"}},{"literal":{"i64":"8963"}},{"literal":{"i64":"8964"}},{"literal":{"i64":"8965"}},{"literal":{"i64":"8966"}},{"literal":{"i64":"8967"}},{"literal":{"i64":"8968"}},{"literal":{"i64":"8969"}},{"literal":{"i64":"897"}},{"literal":{"i64":"8970"}},{"literal":{"i64":"8971"}},{"literal":{"i64":"8972"}},{"literal":{"i64":"8973"}},{"literal":{"i64":"8974"}},{"literal":{"i64":"8975"}},{"literal":{"i64":"8976"}},{"literal":{"i64":"8977"}},{"literal":{"i64":"8978"}},{"literal":{"i64":"8979"}},{"literal":{"i64":"898"}},{"literal":{"i64":"8980"}},{"literal":{"i64":"8981"}},{"literal":{"i64":"8982"}},{"literal":{"i64":"8983"}},{"literal":{"i64":"8984"}},{"literal":{"i64":"8985"}},{"literal":{"i64":"8986"}},{"literal":{"i64":"8987"}},{"literal":{"i64":"8988"}},{"literal":{"i64":"8989"}},{"literal":{"i64":"899"}},{"literal":{"i64":"8990"}},{"literal":{"i64":"8991"}},{"literal":{"i64":"8992"}},{"literal":{"i64":"8993"}},{"literal":{"i64":"8994"}},{"literal":{"i64":"8995"}},{"literal":{"i64":"8996"}},{"literal":{"i64":"8997"}},{"literal":{"i64":"8998"}},{"literal":{"i64":"8999"}},{"literal":{"i64":"9"}},{"literal":{"i64":"90"}},{"literal":{"i64":"900"}},{"literal":{"i64":"9000"}},{"literal":{"i64":"9001"}},{"literal":{"i64":"9002"}},{"literal":{"i64":"9003"}},{"literal":{"i64":"9004"}},{"literal":{"i64":"9005"}},{"literal":{"i64":"9006"}},{"literal":{"i64":"9007"}},{"literal":{"i64":"9008"}},{"literal":{"i64":"9009"}},{"literal":{"i64":"901"}},{"literal":{"i64":"9010"}},{"literal":{"i64":"9011"}},{"literal":{"i64":"9012"}},{"literal":{"i64":"9013"}},{"literal":{"i64":"9014"}},{"literal":{"i64":"9015"}},{"literal":{"i64":"9016"}},{"literal":{"i64":"9017"}},{"literal":{"i64":"9018"}},{"literal":{"i64":"9019"}},{"literal":{"i64":"902"}},{"literal":{"i64":"9020"}},{"literal":{"i64":"9021"}},{"literal":{"i64":"9022"}},{"literal":{"i64":"9023"}},{"literal":{"i64":"9024"}},{"literal":{"i64":"9025"}},{"literal":{"i64":"9026"}},{"literal":{"i64":"9027"}},{"literal":{"i64":"9028"}},{"literal":{"i64":"9029"}},{"literal":{"i64":"903"}},{"literal":{"i64":"9030"}},{"literal":{"i64":"9031"}},{"literal":{"i64":"9032"}},{"literal":{"i64":"9033"}},{"literal":{"i64":"9034"}},{"literal":{"i64":"9035"}},{"literal":{"i64":"9036"}},{"literal":{"i64":"9037"}},{"literal":{"i64":"9038"}},{"literal":{"i64":"9039"}},{"literal":{"i64":"904"}},{"literal":{"i64":"9040"}},{"literal":{"i64":"9041"}},{"literal":{"i64":"9042"}},{"literal":{"i64":"9043"}},{"literal":{"i64":"9044"}},{"literal":{"i64":"9045"}},{"literal":{"i64":"9046"}},{"literal":{"i64":"9047"}},{"literal":{"i64":"9048"}},{"literal":{"i64":"9049"}},{"literal":{"i64":"905"}},{"literal":{"i64":"9050"}},{"literal":{"i64":"9051"}},{"literal":{"i64":"9052"}},{"literal":{"i64":"9053"}},{"literal":{"i64":"9054"}},{"literal":{"i64":"9055"}},{"literal":{"i64":"9056"}},{"literal":{"i64":"9057"}},{"literal":{"i64":"9058"}},{"literal":{"i64":"9059"}},{"literal":{"i64":"906"}},{"literal":{"i64":"9060"}},{"literal":{"i64":"9061"}},{"literal":{"i64":"9062"}},{"literal":{"i64":"9063"}},{"literal":{"i64":"9064"}},{"literal":{"i64":"9065"}},{"literal":{"i64":"9066"}},{"literal":{"i64":"9067"}},{"literal":{"i64":"9068"}},{"literal":{"i64":"9069"}},{"literal":{"i64":"907"}},{"literal":{"i64":"9070"}},{"literal":{"i64":"9071"}},{"literal":{"i64":"9072"}},{"literal":{"i64":"9073"}},{"literal":{"i64":"9074"}},{"literal":{"i64":"9075"}},{"literal":{"i64":"9076"}},{"literal":{"i64":"9077"}},{"literal":{"i64":"9078"}},{"literal":{"i64":"9079"}},{"literal":{"i64":"908"}},{"literal":{"i64":"9080"}},{"literal":{"i64":"9081"}},{"literal":{"i64":"9082"}},{"literal":{"i64":"9083"}},{"literal":{"i64":"9084"}},{"literal":{"i64":"9085"}},{"literal":{"i64":"9086"}},{"literal":{"i64":"9087"}},{"literal":{"i64":"9088"}},{"literal":{"i64":"9089"}},{"literal":{"i64":"909"}},{"literal":{"i64":"9090"}},{"literal":{"i64":"9091"}},{"literal":{"i64":"9092"}},{"literal":{"i64":"9093"}},{"literal":{"i64":"9094"}},{"literal":{"i64":"9095"}},{"literal":{"i64":"9096"}},{"literal":{"i64":"9097"}},{"literal":{"i64":"9098"}},{"literal":{"i64":"9099"}},{"literal":{"i64":"91"}},{"literal":{"i64":"910"}},{"literal":{"i64":"9100"}},{"literal":{"i64":"9101"}},{"literal":{"i64":"9102"}},{"literal":{"i64":"9103"}},{"literal":{"i64":"9104"}},{"literal":{"i64":"9105"}},{"literal":{"i64":"9106"}},{"literal":{"i64":"9107"}},{"literal":{"i64":"9108"}},{"literal":{"i64":"9109"}},{"literal":{"i64":"911"}},{"literal":{"i64":"9110"}},{"literal":{"i64":"9111"}},{"literal":{"i64":"9112"}},{"literal":{"i64":"9113"}},{"literal":{"i64":"9114"}},{"literal":{"i64":"9115"}},{"literal":{"i64":"9116"}},{"literal":{"i64":"9117"}},{"literal":{"i64":"9118"}},{"literal":{"i64":"9119"}},{"literal":{"i64":"912"}},{"literal":{"i64":"9120"}},{"literal":{"i64":"9121"}},{"literal":{"i64":"9122"}},{"literal":{"i64":"9123"}},{"literal":{"i64":"9124"}},{"literal":{"i64":"9125"}},{"literal":{"i64":"9126"}},{"literal":{"i64":"9127"}},{"literal":{"i64":"9128"}},{"literal":{"i64":"9129"}},{"literal":{"i64":"913"}},{"literal":{"i64":"9130"}},{"literal":{"i64":"9131"}},{"literal":{"i64":"9132"}},{"literal":{"i64":"9133"}},{"literal":{"i64":"9134"}},{"literal":{"i64":"9135"}},{"literal":{"i64":"9136"}},{"literal":{"i64":"9137"}},{"literal":{"i64":"9138"}},{"literal":{"i64":"9139"}},{"literal":{"i64":"914"}},{"literal":{"i64":"9140"}},{"literal":{"i64":"9141"}},{"literal":{"i64":"9142"}},{"literal":{"i64":"9143"}},{"literal":{"i64":"9144"}},{"literal":{"i64":"9145"}},{"literal":{"i64":"9146"}},{"literal":{"i64":"9147"}},{"literal":{"i64":"9148"}},{"literal":{"i64":"9149"}},{"literal":{"i64":"915"}},{"literal":{"i64":"9150"}},{"literal":{"i64":"9151"}},{"literal":{"i64":"9152"}},{"literal":{"i64":"9153"}},{"literal":{"i64":"9154"}},{"literal":{"i64":"9155"}},{"literal":{"i64":"9156"}},{"literal":{"i64":"9157"}},{"literal":{"i64":"9158"}},{"literal":{"i64":"9159"}},{"literal":{"i64":"916"}},{"literal":{"i64":"9160"}},{"literal":{"i64":"9161"}},{"literal":{"i64":"9162"}},{"literal":{"i64":"9163"}},{"literal":{"i64":"9164"}},{"literal":{"i64":"9165"}},{"literal":{"i64":"9166"}},{"literal":{"i64":"9167"}},{"literal":{"i64":"9168"}},{"literal":{"i64":"9169"}},{"literal":{"i64":"917"}},{"literal":{"i64":"9170"}},{"literal":{"i64":"9171"}},{"literal":{"i64":"9172"}},{"literal":{"i64":"9173"}},{"literal":{"i64":"9174"}},{"literal":{"i64":"9175"}},{"literal":{"i64":"9176"}},{"literal":{"i64":"9177"}},{"literal":{"i64":"9178"}},{"literal":{"i64":"9179"}},{"literal":{"i64":"918"}},{"literal":{"i64":"9180"}},{"literal":{"i64":"9181"}},{"literal":{"i64":"9182"}},{"literal":{"i64":"9183"}},{"literal":{"i64":"9184"}},{"literal":{"i64":"9185"}},{"literal":{"i64":"9186"}},{"literal":{"i64":"9187"}},{"literal":{"i64":"9188"}},{"literal":{"i64":"9189"}},{"literal":{"i64":"919"}},{"literal":{"i64":"9190"}},{"literal":{"i64":"9191"}},{"literal":{"i64":"9192"}},{"literal":{"i64":"9193"}},{"literal":{"i64":"9194"}},{"literal":{"i64":"9195"}},{"literal":{"i64":"9196"}},{"literal":{"i64":"9197"}},{"literal":{"i64":"9198"}},{"literal":{"i64":"9199"}},{"literal":{"i64":"92"}},{"literal":{"i64":"920"}},{"literal":{"i64":"9200"}},{"literal":{"i64":"9201"}},{"literal":{"i64":"9202"}},{"literal":{"i64":"9203"}},{"literal":{"i64":"9204"}},{"literal":{"i64":"9205"}},{"literal":{"i64":"9206"}},{"literal":{"i64":"9207"}},{"literal":{"i64":"9208"}},{"literal":{"i64":"9209"}},{"literal":{"i64":"921"}},{"literal":{"i64":"9210"}},{"literal":{"i64":"9211"}},{"literal":{"i64":"9212"}},{"literal":{"i64":"9213"}},{"literal":{"i64":"9214"}},{"literal":{"i64":"9215"}},{"literal":{"i64":"9216"}},{"literal":{"i64":"9217"}},{"literal":{"i64":"9218"}},{"literal":{"i64":"9219"}},{"literal":{"i64":"922"}},{"literal":{"i64":"9220"}},{"literal":{"i64":"9221"}},{"literal":{"i64":"9222"}},{"literal":{"i64":"9223"}},{"literal":{"i64":"9224"}},{"literal":{"i64":"9225"}},{"literal":{"i64":"9226"}},{"literal":{"i64":"9227"}},{"literal":{"i64":"9228"}},{"literal":{"i64":"9229"}},{"literal":{"i64":"923"}},{"literal":{"i64":"9230"}},{"literal":{"i64":"9231"}},{"literal":{"i64":"9232"}},{"literal":{"i64":"9233"}},{"literal":{"i64":"9234"}},{"literal":{"i64":"9235"}},{"literal":{"i64":"9236"}},{"literal":{"i64":"9237"}},{"literal":{"i64":"9238"}},{"literal":{"i64":"9239"}},{"literal":{"i64":"924"}},{"literal":{"i64":"9240"}},{"literal":{"i64":"9241"}},{"literal":{"i64":"9242"}},{"literal":{"i64":"9243"}},{"literal":{"i64":"9244"}},{"literal":{"i64":"9245"}},{"literal":{"i64":"9246"}},{"literal":{"i64":"9247"}},{"literal":{"i64":"9248"}},{"literal":{"i64":"9249"}},{"literal":{"i64":"925"}},{"literal":{"i64":"9250"}},{"literal":{"i64":"9251"}},{"literal":{"i64":"9252"}},{"literal":{"i64":"9253"}},{"literal":{"i64":"9254"}},{"literal":{"i64":"9255"}},{"literal":{"i64":"9256"}},{"literal":{"i64":"9257"}},{"literal":{"i64":"9258"}},{"literal":{"i64":"9259"}},{"literal":{"i64":"926"}},{"literal":{"i64":"9260"}},{"literal":{"i64":"9261"}},{"literal":{"i64":"9262"}},{"literal":{"i64":"9263"}},{"literal":{"i64":"9264"}},{"literal":{"i64":"9265"}},{"literal":{"i64":"9266"}},{"literal":{"i64":"9267"}},{"literal":{"i64":"9268"}},{"literal":{"i64":"9269"}},{"literal":{"i64":"927"}},{"literal":{"i64":"9270"}},{"literal":{"i64":"9271"}},{"literal":{"i64":"9272"}},{"literal":{"i64":"9273"}},{"literal":{"i64":"9274"}},{"literal":{"i64":"9275"}},{"literal":{"i64":"9276"}},{"literal":{"i64":"9277"}},{"literal":{"i64":"9278"}},{"literal":{"i64":"9279"}},{"literal":{"i64":"928"}},{"literal":{"i64":"9280"}},{"literal":{"i64":"9281"}},{"literal":{"i64":"9282"}},{"literal":{"i64":"9283"}},{"literal":{"i64":"9284"}},{"literal":{"i64":"9285"}},{"literal":{"i64":"9286"}},{"literal":{"i64":"9287"}},{"literal":{"i64":"9288"}},{"literal":{"i64":"9289"}},{"literal":{"i64":"929"}},{"literal":{"i64":"9290"}},{"literal":{"i64":"9291"}},{"literal":{"i64":"9292"}},{"literal":{"i64":"9293"}},{"literal":{"i64":"9294"}},{"literal":{"i64":"9295"}},{"literal":{"i64":"9296"}},{"literal":{"i64":"9297"}},{"literal":{"i64":"9298"}},{"literal":{"i64":"9299"}},{"literal":{"i64":"93"}},{"literal":{"i64":"930"}},{"literal":{"i64":"9300"}},{"literal":{"i64":"9301"}},{"literal":{"i64":"9302"}},{"literal":{"i64":"9303"}},{"literal":{"i64":"9304"}},{"literal":{"i64":"9305"}},{"literal":{"i64":"9306"}},{"literal":{"i64":"9307"}},{"literal":{"i64":"9308"}},{"literal":{"i64":"9309"}},{"literal":{"i64":"931"}},{"literal":{"i64":"9310"}},{"literal":{"i64":"9311"}},{"literal":{"i64":"9312"}},{"literal":{"i64":"9313"}},{"literal":{"i64":"9314"}},{"literal":{"i64":"9315"}},{"literal":{"i64":"9316"}},{"literal":{"i64":"9317"}},{"literal":{"i64":"9318"}},{"literal":{"i64":"9319"}},{"literal":{"i64":"932"}},{"literal":{"i64":"9320"}},{"literal":{"i64":"9321"}},{"literal":{"i64":"9322"}},{"literal":{"i64":"9323"}},{"literal":{"i64":"9324"}},{"literal":{"i64":"9325"}},{"literal":{"i64":"9326"}},{"literal":{"i64":"9327"}},{"literal":{"i64":"9328"}},{"literal":{"i64":"9329"}},{"literal":{"i64":"933"}},{"literal":{"i64":"9330"}},{"literal":{"i64":"9331"}},{"literal":{"i64":"9332"}},{"literal":{"i64":"9333"}},{"literal":{"i64":"9334"}},{"literal":{"i64":"9335"}},{"literal":{"i64":"9336"}},{"literal":{"i64":"9337"}},{"literal":{"i64":"9338"}},{"literal":{"i64":"9339"}},{"literal":{"i64":"934"}},{"literal":{"i64":"9340"}},{"literal":{"i64":"9341"}},{"literal":{"i64":"9342"}},{"literal":{"i64":"9343"}},{"literal":{"i64":"9344"}},{"literal":{"i64":"9345"}},{"literal":{"i64":"9346"}},{"literal":{"i64":"9347"}},{"literal":{"i64":"9348"}},{"literal":{"i64":"9349"}},{"literal":{"i64":"935"}},{"literal":{"i64":"9350"}},{"literal":{"i64":"9351"}},{"literal":{"i64":"9352"}},{"literal":{"i64":"9353"}},{"literal":{"i64":"9354"}},{"literal":{"i64":"9355"}},{"literal":{"i64":"9356"}},{"literal":{"i64":"9357"}},{"literal":{"i64":"9358"}},{"literal":{"i64":"9359"}},{"literal":{"i64":"936"}},{"literal":{"i64":"9360"}},{"literal":{"i64":"9361"}},{"literal":{"i64":"9362"}},{"literal":{"i64":"9363"}},{"literal":{"i64":"9364"}},{"literal":{"i64":"9365"}},{"literal":{"i64":"9366"}},{"literal":{"i64":"9367"}},{"literal":{"i64":"9368"}},{"literal":{"i64":"9369"}},{"literal":{"i64":"937"}},{"literal":{"i64":"9370"}},{"literal":{"i64":"9371"}},{"literal":{"i64":"9372"}},{"literal":{"i64":"9373"}},{"literal":{"i64":"9374"}},{"literal":{"i64":"9375"}},{"literal":{"i64":"9376"}},{"literal":{"i64":"9377"}},{"literal":{"i64":"9378"}},{"literal":{"i64":"9379"}},{"literal":{"i64":"938"}},{"literal":{"i64":"9380"}},{"literal":{"i64":"9381"}},{"literal":{"i64":"9382"}},{"literal":{"i64":"9383"}},{"literal":{"i64":"9384"}},{"literal":{"i64":"9385"}},{"literal":{"i64":"9386"}},{"literal":{"i64":"9387"}},{"literal":{"i64":"9388"}},{"literal":{"i64":"9389"}},{"literal":{"i64":"939"}},{"literal":{"i64":"9390"}},{"literal":{"i64":"9391"}},{"literal":{"i64":"9392"}},{"literal":{"i64":"9393"}},{"literal":{"i64":"9394"}},{"literal":{"i64":"9395"}},{"literal":{"i64":"9396"}},{"literal":{"i64":"9397"}},{"literal":{"i64":"9398"}},{"literal":{"i64":"9399"}},{"literal":{"i64":"94"}},{"literal":{"i64":"940"}},{"literal":{"i64":"9400"}},{"literal":{"i64":"9401"}},{"literal":{"i64":"9402"}},{"literal":{"i64":"9403"}},{"literal":{"i64":"9404"}},{"literal":{"i64":"9405"}},{"literal":{"i64":"9406"}},{"literal":{"i64":"9407"}},{"literal":{"i64":"9408"}},{"literal":{"i64":"9409"}},{"literal":{"i64":"941"}},{"literal":{"i64":"9410"}},{"literal":{"i64":"9411"}},{"literal":{"i64":"9412"}},{"literal":{"i64":"9413"}},{"literal":{"i64":"9414"}},{"literal":{"i64":"9415"}},{"literal":{"i64":"9416"}},{"literal":{"i64":"9417"}},{"literal":{"i64":"9418"}},{"literal":{"i64":"9419"}},{"literal":{"i64":"942"}},{"literal":{"i64":"9420"}},{"literal":{"i64":"9421"}},{"literal":{"i64":"9422"}},{"literal":{"i64":"9423"}},{"literal":{"i64":"9424"}},{"literal":{"i64":"9425"}},{"literal":{"i64":"9426"}},{"literal":{"i64":"9427"}},{"literal":{"i64":"9428"}},{"literal":{"i64":"9429"}},{"literal":{"i64":"943"}},{"literal":{"i64":"9430"}},{"literal":{"i64":"9431"}},{"literal":{"i64":"9432"}},{"literal":{"i64":"9433"}},{"literal":{"i64":"9434"}},{"literal":{"i64":"9435"}},{"literal":{"i64":"9436"}},{"literal":{"i64":"9437"}},{"literal":{"i64":"9438"}},{"literal":{"i64":"9439"}},{"literal":{"i64":"944"}},{"literal":{"i64":"9440"}},{"literal":{"i64":"9441"}},{"literal":{"i64":"9442"}},{"literal":{"i64":"9443"}},{"literal":{"i64":"9444"}},{"literal":{"i64":"9445"}},{"literal":{"i64":"9446"}},{"literal":{"i64":"9447"}},{"literal":{"i64":"9448"}},{"literal":{"i64":"9449"}},{"literal":{"i64":"945"}},{"literal":{"i64":"9450"}},{"literal":{"i64":"9451"}},{"literal":{"i64":"9452"}},{"literal":{"i64":"9453"}},{"literal":{"i64":"9454"}},{"literal":{"i64":"9455"}},{"literal":{"i64":"9456"}},{"literal":{"i64":"9457"}},{"literal":{"i64":"9458"}},{"literal":{"i64":"9459"}},{"literal":{"i64":"946"}},{"literal":{"i64":"9460"}},{"literal":{"i64":"9461"}},{"literal":{"i64":"9462"}},{"literal":{"i64":"9463"}},{"literal":{"i64":"9464"}},{"literal":{"i64":"9465"}},{"literal":{"i64":"9466"}},{"literal":{"i64":"9467"}},{"literal":{"i64":"9468"}},{"literal":{"i64":"9469"}},{"literal":{"i64":"947"}},{"literal":{"i64":"9470"}},{"literal":{"i64":"9471"}},{"literal":{"i64":"9472"}},{"literal":{"i64":"9473"}},{"literal":{"i64":"9474"}},{"literal":{"i64":"9475"}},{"literal":{"i64":"9476"}},{"literal":{"i64":"9477"}},{"literal":{"i64":"9478"}},{"literal":{"i64":"9479"}},{"literal":{"i64":"948"}},{"literal":{"i64":"9480"}},{"literal":{"i64":"9481"}},{"literal":{"i64":"9482"}},{"literal":{"i64":"9483"}},{"literal":{"i64":"9484"}},{"literal":{"i64":"9485"}},{"literal":{"i64":"9486"}},{"literal":{"i64":"9487"}},{"literal":{"i64":"9488"}},{"literal":{"i64":"9489"}},{"literal":{"i64":"949"}},{"literal":{"i64":"9490"}},{"literal":{"i64":"9491"}},{"literal":{"i64":"9492"}},{"literal":{"i64":"9493"}},{"literal":{"i64":"9494"}},{"literal":{"i64":"9495"}},{"literal":{"i64":"9496"}},{"literal":{"i64":"9497"}},{"literal":{"i64":"9498"}},{"literal":{"i64":"9499"}},{"literal":{"i64":"95"}},{"literal":{"i64":"950"}},{"literal":{"i64":"9500"}},{"literal":{"i64":"9501"}},{"literal":{"i64":"9502"}},{"literal":{"i64":"9503"}},{"literal":{"i64":"9504"}},{"literal":{"i64":"9505"}},{"literal":{"i64":"9506"}},{"literal":{"i64":"9507"}},{"literal":{"i64":"9508"}},{"literal":{"i64":"9509"}},{"literal":{"i64":"951"}},{"literal":{"i64":"9510"}},{"literal":{"i64":"9511"}},{"literal":{"i64":"9512"}},{"literal":{"i64":"9513"}},{"literal":{"i64":"9514"}},{"literal":{"i64":"9515"}},{"literal":{"i64":"9516"}},{"literal":{"i64":"9517"}},{"literal":{"i64":"9518"}},{"literal":{"i64":"9519"}},{"literal":{"i64":"952"}},{"literal":{"i64":"9520"}},{"literal":{"i64":"9521"}},{"literal":{"i64":"9522"}},{"literal":{"i64":"9523"}},{"literal":{"i64":"9524"}},{"literal":{"i64":"9525"}},{"literal":{"i64":"9526"}},{"literal":{"i64":"9527"}},{"literal":{"i64":"9528"}},{"literal":{"i64":"9529"}},{"literal":{"i64":"953"}},{"literal":{"i64":"9530"}},{"literal":{"i64":"9531"}},{"literal":{"i64":"9532"}},{"literal":{"i64":"9533"}},{"literal":{"i64":"9534"}},{"literal":{"i64":"9535"}},{"literal":{"i64":"9536"}},{"literal":{"i64":"9537"}},{"literal":{"i64":"9538"}},{"literal":{"i64":"9539"}},{"literal":{"i64":"954"}},{"literal":{"i64":"9540"}},{"literal":{"i64":"9541"}},{"literal":{"i64":"9542"}},{"literal":{"i64":"9543"}},{"literal":{"i64":"9544"}},{"literal":{"i64":"9545"}},{"literal":{"i64":"9546"}},{"literal":{"i64":"9547"}},{"literal":{"i64":"9548"}},{"literal":{"i64":"9549"}},{"literal":{"i64":"955"}},{"literal":{"i64":"9550"}},{"literal":{"i64":"9551"}},{"literal":{"i64":"9552"}},{"literal":{"i64":"9553"}},{"literal":{"i64":"9554"}},{"literal":{"i64":"9555"}},{"literal":{"i64":"9556"}},{"literal":{"i64":"9557"}},{"literal":{"i64":"9558"}},{"literal":{"i64":"9559"}},{"literal":{"i64":"956"}},{"literal":{"i64":"9560"}},{"literal":{"i64":"9561"}},{"literal":{"i64":"9562"}},{"literal":{"i64":"9563"}},{"literal":{"i64":"9564"}},{"literal":{"i64":"9565"}},{"literal":{"i64":"9566"}},{"literal":{"i64":"9567"}},{"literal":{"i64":"9568"}},{"literal":{"i64":"9569"}},{"literal":{"i64":"957"}},{"literal":{"i64":"9570"}},{"literal":{"i64":"9571"}},{"literal":{"i64":"9572"}},{"literal":{"i64":"9573"}},{"literal":{"i64":"9574"}},{"literal":{"i64":"9575"}},{"literal":{"i64":"9576"}},{"literal":{"i64":"9577"}},{"literal":{"i64":"9578"}},{"literal":{"i64":"9579"}},{"literal":{"i64":"958"}},{"literal":{"i64":"9580"}},{"literal":{"i64":"9581"}},{"literal":{"i64":"9582"}},{"literal":{"i64":"9583"}},{"literal":{"i64":"9584"}},{"literal":{"i64":"9585"}},{"literal":{"i64":"9586"}},{"literal":{"i64":"9587"}},{"literal":{"i64":"9588"}},{"literal":{"i64":"9589"}},{"literal":{"i64":"959"}},{"literal":{"i64":"9590"}},{"literal":{"i64":"9591"}},{"literal":{"i64":"9592"}},{"literal":{"i64":"9593"}},{"literal":{"i64":"9594"}},{"literal":{"i64":"9595"}},{"literal":{"i64":"9596"}},{"literal":{"i64":"9597"}},{"literal":{"i64":"9598"}},{"literal":{"i64":"9599"}},{"literal":{"i64":"96"}},{"literal":{"i64":"960"}},{"literal":{"i64":"9600"}},{"literal":{"i64":"9601"}},{"literal":{"i64":"9602"}},{"literal":{"i64":"9603"}},{"literal":{"i64":"9604"}},{"literal":{"i64":"9605"}},{"literal":{"i64":"9606"}},{"literal":{"i64":"9607"}},{"literal":{"i64":"9608"}},{"literal":{"i64":"9609"}},{"literal":{"i64":"961"}},{"literal":{"i64":"9610"}},{"literal":{"i64":"9611"}},{"literal":{"i64":"9612"}},{"literal":{"i64":"9613"}},{"literal":{"i64":"9614"}},{"literal":{"i64":"9615"}},{"literal":{"i64":"9616"}},{"literal":{"i64":"9617"}},{"literal":{"i64":"9618"}},{"literal":{"i64":"9619"}},{"literal":{"i64":"962"}},{"literal":{"i64":"9620"}},{"literal":{"i64":"9621"}},{"literal":{"i64":"9622"}},{"literal":{"i64":"9623"}},{"literal":{"i64":"9624"}},{"literal":{"i64":"9625"}},{"literal":{"i64":"9626"}},{"literal":{"i64":"9627"}},{"literal":{"i64":"9628"}},{"literal":{"i64":"9629"}},{"literal":{"i64":"963"}},{"literal":{"i64":"9630"}},{"literal":{"i64":"9631"}},{"literal":{"i64":"9632"}},{"literal":{"i64":"9633"}},{"literal":{"i64":"9634"}},{"literal":{"i64":"9635"}},{"literal":{"i64":"9636"}},{"literal":{"i64":"9637"}},{"literal":{"i64":"9638"}},{"literal":{"i64":"9639"}},{"literal":{"i64":"964"}},{"literal":{"i64":"9640"}},{"literal":{"i64":"9641"}},{"literal":{"i64":"9642"}},{"literal":{"i64":"9643"}},{"literal":{"i64":"9644"}},{"literal":{"i64":"9645"}},{"literal":{"i64":"9646"}},{"literal":{"i64":"9647"}},{"literal":{"i64":"9648"}},{"literal":{"i64":"9649"}},{"literal":{"i64":"965"}},{"literal":{"i64":"9650"}},{"literal":{"i64":"9651"}},{"literal":{"i64":"9652"}},{"literal":{"i64":"9653"}},{"literal":{"i64":"9654"}},{"literal":{"i64":"9655"}},{"literal":{"i64":"9656"}},{"literal":{"i64":"9657"}},{"literal":{"i64":"9658"}},{"literal":{"i64":"9659"}},{"literal":{"i64":"966"}},{"literal":{"i64":"9660"}},{"literal":{"i64":"9661"}},{"literal":{"i64":"9662"}},{"literal":{"i64":"9663"}},{"literal":{"i64":"9664"}},{"literal":{"i64":"9665"}},{"literal":{"i64":"9666"}},{"literal":{"i64":"9667"}},{"literal":{"i64":"9668"}},{"literal":{"i64":"9669"}},{"literal":{"i64":"967"}},{"literal":{"i64":"9670"}},{"literal":{"i64":"9671"}},{"literal":{"i64":"9672"}},{"literal":{"i64":"9673"}},{"literal":{"i64":"9674"}},{"literal":{"i64":"9675"}},{"literal":{"i64":"9676"}},{"literal":{"i64":"9677"}},{"literal":{"i64":"9678"}},{"literal":{"i64":"9679"}},{"literal":{"i64":"968"}},{"literal":{"i64":"9680"}},{"literal":{"i64":"9681"}},{"literal":{"i64":"9682"}},{"literal":{"i64":"9683"}},{"literal":{"i64":"9684"}},{"literal":{"i64":"9685"}},{"literal":{"i64":"9686"}},{"literal":{"i64":"9687"}},{"literal":{"i64":"9688"}},{"literal":{"i64":"9689"}},{"literal":{"i64":"969"}},{"literal":{"i64":"9690"}},{"literal":{"i64":"9691"}},{"literal":{"i64":"9692"}},{"literal":{"i64":"9693"}},{"literal":{"i64":"9694"}},{"literal":{"i64":"9695"}},{"literal":{"i64":"9696"}},{"literal":{"i64":"9697"}},{"literal":{"i64":"9698"}},{"literal":{"i64":"9699"}},{"literal":{"i64":"97"}},{"literal":{"i64":"970"}},{"literal":{"i64":"9700"}},{"literal":{"i64":"9701"}},{"literal":{"i64":"9702"}},{"literal":{"i64":"9703"}},{"literal":{"i64":"9704"}},{"literal":{"i64":"9705"}},{"literal":{"i64":"9706"}},{"literal":{"i64":"9707"}},{"literal":{"i64":"9708"}},{"literal":{"i64":"9709"}},{"literal":{"i64":"971"}},{"literal":{"i64":"9710"}},{"literal":{"i64":"9711"}},{"literal":{"i64":"9712"}},{"literal":{"i64":"9713"}},{"literal":{"i64":"9714"}},{"literal":{"i64":"9715"}},{"literal":{"i64":"9716"}},{"literal":{"i64":"9717"}},{"literal":{"i64":"9718"}},{"literal":{"i64":"9719"}},{"literal":{"i64":"972"}},{"literal":{"i64":"9720"}},{"literal":{"i64":"9721"}},{"literal":{"i64":"9722"}},{"literal":{"i64":"9723"}},{"literal":{"i64":"9724"}},{"literal":{"i64":"9725"}},{"literal":{"i64":"9726"}},{"literal":{"i64":"9727"}},{"literal":{"i64":"9728"}},{"literal":{"i64":"9729"}},{"literal":{"i64":"973"}},{"literal":{"i64":"9730"}},{"literal":{"i64":"9731"}},{"literal":{"i64":"9732"}},{"literal":{"i64":"9733"}},{"literal":{"i64":"9734"}},{"literal":{"i64":"9735"}},{"literal":{"i64":"9736"}},{"literal":{"i64":"9737"}},{"literal":{"i64":"9738"}},{"literal":{"i64":"9739"}},{"literal":{"i64":"974"}},{"literal":{"i64":"9740"}},{"literal":{"i64":"9741"}},{"literal":{"i64":"9742"}},{"literal":{"i64":"9743"}},{"literal":{"i64":"9744"}},{"literal":{"i64":"9745"}},{"literal":{"i64":"9746"}},{"literal":{"i64":"9747"}},{"literal":{"i64":"9748"}},{"literal":{"i64":"9749"}},{"literal":{"i64":"975"}},{"literal":{"i64":"9750"}},{"literal":{"i64":"9751"}},{"literal":{"i64":"9752"}},{"literal":{"i64":"9753"}},{"literal":{"i64":"9754"}},{"literal":{"i64":"9755"}},{"literal":{"i64":"9756"}},{"literal":{"i64":"9757"}},{"literal":{"i64":"9758"}},{"literal":{"i64":"9759"}},{"literal":{"i64":"976"}},{"literal":{"i64":"9760"}},{"literal":{"i64":"9761"}},{"literal":{"i64":"9762"}},{"literal":{"i64":"9763"}},{"literal":{"i64":"9764"}},{"literal":{"i64":"9765"}},{"literal":{"i64":"9766"}},{"literal":{"i64":"9767"}},{"literal":{"i64":"9768"}},{"literal":{"i64":"9769"}},{"literal":{"i64":"977"}},{"literal":{"i64":"9770"}},{"literal":{"i64":"9771"}},{"literal":{"i64":"9772"}},{"literal":{"i64":"9773"}},{"literal":{"i64":"9774"}},{"literal":{"i64":"9775"}},{"literal":{"i64":"9776"}},{"literal":{"i64":"9777"}},{"literal":{"i64":"9778"}},{"literal":{"i64":"9779"}},{"literal":{"i64":"978"}},{"literal":{"i64":"9780"}},{"literal":{"i64":"9781"}},{"literal":{"i64":"9782"}},{"literal":{"i64":"9783"}},{"literal":{"i64":"9784"}},{"literal":{"i64":"9785"}},{"literal":{"i64":"9786"}},{"literal":{"i64":"9787"}},{"literal":{"i64":"9788"}},{"literal":{"i64":"9789"}},{"literal":{"i64":"979"}},{"literal":{"i64":"9790"}},{"literal":{"i64":"9791"}},{"literal":{"i64":"9792"}},{"literal":{"i64":"9793"}},{"literal":{"i64":"9794"}},{"literal":{"i64":"9795"}},{"literal":{"i64":"9796"}},{"literal":{"i64":"9797"}},{"literal":{"i64":"9798"}},{"literal":{"i64":"9799"}},{"literal":{"i64":"98"}},{"literal":{"i64":"980"}},{"literal":{"i64":"9800"}},{"literal":{"i64":"9801"}},{"literal":{"i64":"9802"}},{"literal":{"i64":"9803"}},{"literal":{"i64":"9804"}},{"literal":{"i64":"9805"}},{"literal":{"i64":"9806"}},{"literal":{"i64":"9807"}},{"literal":{"i64":"9808"}},{"literal":{"i64":"9809"}},{"literal":{"i64":"981"}},{"literal":{"i64":"9810"}},{"literal":{"i64":"9811"}},{"literal":{"i64":"9812"}},{"literal":{"i64":"9813"}},{"literal":{"i64":"9814"}},{"literal":{"i64":"9815"}},{"literal":{"i64":"9816"}},{"literal":{"i64":"9817"}},{"literal":{"i64":"9818"}},{"literal":{"i64":"9819"}},{"literal":{"i64":"982"}},{"literal":{"i64":"9820"}},{"literal":{"i64":"9821"}},{"literal":{"i64":"9822"}},{"literal":{"i64":"9823"}},{"literal":{"i64":"9824"}},{"literal":{"i64":"9825"}},{"literal":{"i64":"9826"}},{"literal":{"i64":"9827"}},{"literal":{"i64":"9828"}},{"literal":{"i64":"9829"}},{"literal":{"i64":"983"}},{"literal":{"i64":"9830"}},{"literal":{"i64":"9831"}},{"literal":{"i64":"9832"}},{"literal":{"i64":"9833"}},{"literal":{"i64":"9834"}},{"literal":{"i64":"9835"}},{"literal":{"i64":"9836"}},{"literal":{"i64":"9837"}},{"literal":{"i64":"9838"}},{"literal":{"i64":"9839"}},{"literal":{"i64":"984"}},{"literal":{"i64":"9840"}},{"literal":{"i64":"9841"}},{"literal":{"i64":"9842"}},{"literal":{"i64":"9843"}},{"literal":{"i64":"9844"}},{"literal":{"i64":"9845"}},{"literal":{"i64":"9846"}},{"literal":{"i64":"9847"}},{"literal":{"i64":"9848"}},{"literal":{"i64":"9849"}},{"literal":{"i64":"985"}},{"literal":{"i64":"9850"}},{"literal":{"i64":"9851"}},{"literal":{"i64":"9852"}},{"literal":{"i64":"9853"}},{"literal":{"i64":"9854"}},{"literal":{"i64":"9855"}},{"literal":{"i64":"9856"}},{"literal":{"i64":"9857"}},{"literal":{"i64":"9858"}},{"literal":{"i64":"9859"}},{"literal":{"i64":"986"}},{"literal":{"i64":"9860"}},{"literal":{"i64":"9861"}},{"literal":{"i64":"9862"}},{"literal":{"i64":"9863"}},{"literal":{"i64":"9864"}},{"literal":{"i64":"9865"}},{"literal":{"i64":"9866"}},{"literal":{"i64":"9867"}},{"literal":{"i64":"9868"}},{"literal":{"i64":"9869"}},{"literal":{"i64":"987"}},{"literal":{"i64":"9870"}},{"literal":{"i64":"9871"}},{"literal":{"i64":"9872"}},{"literal":{"i64":"9873"}},{"literal":{"i64":"9874"}},{"literal":{"i64":"9875"}},{"literal":{"i64":"9876"}},{"literal":{"i64":"9877"}},{"literal":{"i64":"9878"}},{"literal":{"i64":"9879"}},{"literal":{"i64":"988"}},{"literal":{"i64":"9880"}},{"literal":{"i64":"9881"}},{"literal":{"i64":"9882"}},{"literal":{"i64":"9883"}},{"literal":{"i64":"9884"}},{"literal":{"i64":"9885"}},{"literal":{"i64":"9886"}},{"literal":{"i64":"9887"}},{"literal":{"i64":"9888"}},{"literal":{"i64":"9889"}},{"literal":{"i64":"989"}},{"literal":{"i64":"9890"}},{"literal":{"i64":"9891"}},{"literal":{"i64":"9892"}},{"literal":{"i64":"9893"}},{"literal":{"i64":"9894"}},{"literal":{"i64":"9895"}},{"literal":{"i64":"9896"}},{"literal":{"i64":"9897"}},{"literal":{"i64":"9898"}},{"literal":{"i64":"9899"}},{"literal":{"i64":"99"}},{"literal":{"i64":"990"}},{"literal":{"i64":"9900"}},{"literal":{"i64":"9901"}},{"literal":{"i64":"9902"}},{"literal":{"i64":"9903"}},{"literal":{"i64":"9904"}},{"literal":{"i64":"9905"}},{"literal":{"i64":"9906"}},{"literal":{"i64":"9907"}},{"literal":{"i64":"9908"}},{"literal":{"i64":"9909"}},{"literal":{"i64":"991"}},{"literal":{"i64":"9910"}},{"literal":{"i64":"9911"}},{"literal":{"i64":"9912"}},{"literal":{"i64":"9913"}},{"literal":{"i64":"9914"}},{"literal":{"i64":"9915"}},{"literal":{"i64":"9916"}},{"literal":{"i64":"9917"}},{"literal":{"i64":"9918"}},{"literal":{"i64":"9919"}},{"literal":{"i64":"992"}},{"literal":{"i64":"9920"}},{"literal":{"i64":"9921"}},{"literal":{"i64":"9922"}},{"literal":{"i64":"9923"}},{"literal":{"i64":"9924"}},{"literal":{"i64":"9925"}},{"literal":{"i64":"9926"}},{"literal":{"i64":"9927"}},{"literal":{"i64":"9928"}},{"literal":{"i64":"9929"}},{"literal":{"i64":"993"}},{"literal":{"i64":"9930"}},{"literal":{"i64":"9931"}},{"literal":{"i64":"9932"}},{"literal":{"i64":"9933"}},{"literal":{"i64":"9934"}},{"literal":{"i64":"9935"}},{"literal":{"i64":"9936"}},{"literal":{"i64":"9937"}},{"literal":{"i64":"9938"}},{"literal":{"i64":"9939"}},{"literal":{"i64":"994"}},{"literal":{"i64":"9940"}},{"literal":{"i64":"9941"}},{"literal":{"i64":"9942"}},{"literal":{"i64":"9943"}},{"literal":{"i64":"9944"}},{"literal":{"i64":"9945"}},{"literal":{"i64":"9946"}},{"literal":{"i64":"9947"}},{"literal":{"i64":"9948"}},{"literal":{"i64":"9949"}},{"literal":{"i64":"995"}},{"literal":{"i64":"9950"}},{"literal":{"i64":"9951"}},{"literal":{"i64":"9952"}},{"literal":{"i64":"9953"}},{"literal":{"i64":"9954"}},{"literal":{"i64":"9955"}},{"literal":{"i64":"9956"}},{"literal":{"i64":"9957"}},{"literal":{"i64":"9958"}},{"literal":{"i64":"9959"}},{"literal":{"i64":"996"}},{"literal":{"i64":"9960"}},{"literal":{"i64":"9961"}},{"literal":{"i64":"9962"}},{"literal":{"i64":"9963"}},{"literal":{"i64":"9964"}},{"literal":{"i64":"9965"}},{"literal":{"i64":"9966"}},{"literal":{"i64":"9967"}},{"literal":{"i64":"9968"}},{"literal":{"i64":"9969"}},{"literal":{"i64":"997"}},{"literal":{"i64":"9970"}},{"literal":{"i64":"9971"}},{"literal":{"i64":"9972"}},{"literal":{"i64":"9973"}},{"literal":{"i64":"9974"}},{"literal":{"i64":"9975"}},{"literal":{"i64":"9976"}},{"literal":{"i64":"9977"}},{"literal":{"i64":"9978"}},{"literal":{"i64":"9979"}},{"literal":{"i64":"998"}},{"literal":{"i64":"9980"}},{"literal":{"i64":"9981"}},{"literal":{"i64":"9982"}},{"literal":{"i64":"9983"}},{"literal":{"i64":"9984"}},{"literal":{"i64":"9985"}},{"literal":{"i64":"9986"}},{"literal":{"i64":"9987"}},{"literal":{"i64":"9988"}},{"literal":{"i64":"9989"}},{"literal":{"i64":"999"}},{"literal":{"i64":"9990"}},{"literal":{"i64":"9991"}},{"literal":{"i64":"9992"}},{"literal":{"i64":"9993"}},{"literal":{"i64":"9994"}},{"literal":{"i64":"9995"}},{"literal":{"i64":"9996"}},{"literal":{"i64":"9997"}},{"literal":{"i64":"9998"}},{"literal":{"i64":"9999"}}]}}}}}},"groupings":[{}],"measures":[{"measure":{"phase":"AGGREGATION_PHASE_INITIAL_TO_INTERMEDIATE","outputType":{"i64":{"nullability":"NULLABILITY_REQUIRED"}},"arguments":[{"value":{"literal":{"i32":1}}}]}}],"advancedExtension":{"optimization":{"@type":"type.googleapis.com/google.protobuf.StringValue","value":"AggregateParams:hasPrePartialAggregate=false\nhasRequiredChildDistributionExpressions=false\n"}}}},"names":["count#58"],"outputSchema":{"types":[{"i64":{"nullability":"NULLABILITY_REQUIRED"}}],"nullability":"NULLABILITY_REQUIRED"}}}]} diff --git a/cpp-ch/local-engine/tests/testConfig.h.in b/cpp-ch/local-engine/tests/testConfig.h.in index 98740e48e930..0a7d34bc6a40 100644 --- a/cpp-ch/local-engine/tests/testConfig.h.in +++ b/cpp-ch/local-engine/tests/testConfig.h.in @@ -4,6 +4,10 @@ #define PARQUET_DATA(file) "file://@PARQUET_DATA_DIR@"#file #define MERGETREE_DATA(file) "@MERGETREE_DATA_DIR@"#file -#define GLUTEN_SOURCE_DIR_ "file://@GLUTEN_REAL_PATH@" +#define GLUTEN_SOURCE_URI_ "file://@GLUTEN_REAL_PATH@" +#define GLUTEN_SOURCE_URI(file) GLUTEN_SOURCE_URI_ file +#define GLUTEN_SOURCE_TPCH_URI(file) GLUTEN_SOURCE_URI("/gluten-core/src/test/resources/tpch-data/" file) + +#define GLUTEN_SOURCE_DIR_ "@GLUTEN_REAL_PATH@" #define GLUTEN_SOURCE_DIR(file) GLUTEN_SOURCE_DIR_ file -#define GLUTEN_SOURCE_TPCH_DIR(file) GLUTEN_SOURCE_DIR("/gluten-core/src/test/resources/tpch-data/" file) \ No newline at end of file +#define GLUTEN_SOURCE_TPCH_DIR(file) GLUTEN_SOURCE_DIR("/gluten-core/src/test/resources/tpch-data/" file) diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/BackendSettingsApi.scala b/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/BackendSettingsApi.scala index d7ee6f7f7681..87f4ac771290 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/BackendSettingsApi.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/BackendSettingsApi.scala @@ -57,9 +57,9 @@ trait BackendSettingsApi { def supportNativeWrite(fields: Array[StructField]): Boolean = true - def supportNativeMetadataColumns(): Boolean = false + def supportNativeMetadataColumns(): Boolean = true - def supportNativeRowIndexColumn(): Boolean = false + def supportNativeRowIndexColumn(): Boolean = true def supportExpandExec(): Boolean = false diff --git a/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala b/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala index 308c56f40548..e9cb3aea85c7 100644 --- a/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala +++ b/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala @@ -1713,6 +1713,10 @@ class ClickHouseTestSettings extends BackendTestSettings { enableSuite[GlutenSparkSessionExtensionSuite] enableSuite[GlutenHiveSQLQueryCHSuite] enableSuite[GlutenPercentileSuite] + enableSuite[GlutenParquetFileMetadataStructRowIndexSuite] + enableSuite[GlutenParquetRowIndexSuite] + .excludeByPrefix("row index generation") + .excludeByPrefix("invalid row index column type") override def getSQLQueryTestSettings: SQLQueryTestSettings = ClickHouseSQLQueryTestSettings } From 04a571106348799af5ee8d62bd9c9e553b00593d Mon Sep 17 00:00:00 2001 From: Gluten Performance Bot <137994563+GlutenPerfBot@users.noreply.github.com> Date: Wed, 5 Feb 2025 15:44:34 +0800 Subject: [PATCH 06/22] [GLUTEN-6887][VL] Daily Update Velox Version (2025_02_05) (#8661) Upstream Velox's New Commits: 5d13c130d by Krishna Pai, build(pyvelox) : Remove VELOX_BUILD_MINIMAL_WITH_DWIO (12256) a785f7ed7 by aditi-pandit, refactor(fuzzer): Move setupReferenceQueryRunner to FuzzerUtil (12232) f73b7b194 by Deepak Majeti, refactor: Remove unused header in functions/Registerer.h (12231) 95a5a4f0e by Yenda Li, feat: Support cancellation in HashProbe::getOutput [3/n] (12237) 48d1bbf2f by Yenda Li, fix: Avoid invalid access into random region in memory in OperatorUtils (12253) b2cb03e4d by Kevin Wilfong, feat: Optimize PrestoBatchVectorSerializer [1/7]: Serialize FlatVectors (12060) 688465f3b by Natasha Sehgal, Add JSON to Map Union Sum (12208) 8bc88970e by Yenda Li, refactor: Compute joinProjectedVarColumnsSize by taking in a vector instead of by row [2/n] (12235) ccb509421 by Deepak Majeti, refactor(hive): Remove Parquet dependency from HiveConnectorTestBase (12228) 2bb7aab29 by Pramod Satya, feat: Add companionFunction to function metadata (9250) f0a96f166 by Yiyang Chen, Add merge() method to ApproxMostFrequentStreamSummary as an optimization over mergeSerialized() --- ep/build-velox/src/get_velox.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ep/build-velox/src/get_velox.sh b/ep/build-velox/src/get_velox.sh index 124ba6155004..4978f4a7453e 100755 --- a/ep/build-velox/src/get_velox.sh +++ b/ep/build-velox/src/get_velox.sh @@ -17,7 +17,7 @@ set -exu VELOX_REPO=https://github.com/oap-project/velox.git -VELOX_BRANCH=2025_02_04 +VELOX_BRANCH=2025_02_05 VELOX_HOME="" OS=`uname -s` From f52f6ca5556ce02ca981498022f66f1e3eff42a7 Mon Sep 17 00:00:00 2001 From: Nicholas Jiang Date: Wed, 5 Feb 2025 16:03:07 +0800 Subject: [PATCH 07/22] [GLUTEN-8631][UNIFFLE] Bump Uniffle to 0.9.2 (#8632) --- .github/workflows/velox_backend.yml | 4 +++- dev/docker/Dockerfile.centos8-dynamic-build | 2 +- docs/get-started/Velox.md | 2 +- pom.xml | 2 +- tools/gluten-it/pom.xml | 2 +- 5 files changed, 7 insertions(+), 5 deletions(-) diff --git a/.github/workflows/velox_backend.yml b/.github/workflows/velox_backend.yml index bc823cc2fb36..8735d74e0bad 100644 --- a/.github/workflows/velox_backend.yml +++ b/.github/workflows/velox_backend.yml @@ -507,7 +507,7 @@ jobs: fail-fast: false matrix: spark: [ "spark-3.2" ] - uniffle: [ "0.9.1" ] + uniffle: [ "0.9.2" ] hadoop: [ "2.8.5" ] runs-on: ubuntu-20.04 container: apache/gluten:centos-8 @@ -527,7 +527,9 @@ jobs: run: | export JAVA_HOME=/usr/lib/jvm/java-1.8.0-openjdk && \ cd /opt && \ + ${WGET_CMD} https://archive.apache.org/dist/incubator/uniffle/${{ matrix.uniffle }}/apache-uniffle-${{ matrix.uniffle }}-incubating-bin.tar.gz && \ mkdir /opt/uniffle && tar xzf apache-uniffle-${{ matrix.uniffle }}-incubating-bin.tar.gz -C /opt/uniffle --strip-components=1 && \ + ${WGET_CMD} https://archive.apache.org/dist/hadoop/common/hadoop-${{ matrix.hadoop }}/hadoop-${{ matrix.hadoop }}.tar.gz && \ tar xzf hadoop-${{ matrix.hadoop }}.tar.gz -C /opt/ && \ cd /opt/uniffle && mkdir shuffle_data && \ bash -c "echo -e 'XMX_SIZE=16g\nHADOOP_HOME=/opt/hadoop-${{ matrix.hadoop }}' > ./bin/rss-env.sh" && \ diff --git a/dev/docker/Dockerfile.centos8-dynamic-build b/dev/docker/Dockerfile.centos8-dynamic-build index 341ebff8808e..fc848ed6ef1a 100644 --- a/dev/docker/Dockerfile.centos8-dynamic-build +++ b/dev/docker/Dockerfile.centos8-dynamic-build @@ -19,7 +19,7 @@ RUN wget -nv https://archive.apache.org/dist/celeborn/celeborn-0.3.2-incubating/ RUN wget -nv https://archive.apache.org/dist/celeborn/celeborn-0.4.3/apache-celeborn-0.4.3-bin.tgz -P /opt/ RUN wget -nv https://archive.apache.org/dist/celeborn/celeborn-0.5.3/apache-celeborn-0.5.3-bin.tgz -P /opt/ -RUN wget -nv https://archive.apache.org/dist/incubator/uniffle/0.9.1/apache-uniffle-0.9.1-incubating-bin.tar.gz -P /opt/ +RUN wget -nv https://archive.apache.org/dist/incubator/uniffle/0.9.2/apache-uniffle-0.9.2-incubating-bin.tar.gz -P /opt/ RUN wget -nv https://archive.apache.org/dist/hadoop/common/hadoop-2.8.5/hadoop-2.8.5.tar.gz -P /opt/ RUN git clone --depth=1 https://github.com/apache/incubator-gluten /opt/gluten diff --git a/docs/get-started/Velox.md b/docs/get-started/Velox.md index 0371ea6168a5..1989922a937c 100644 --- a/docs/get-started/Velox.md +++ b/docs/get-started/Velox.md @@ -286,7 +286,7 @@ spark.dynamicAllocation.enabled false ## Uniffle support -Uniffle with velox backend supports [Uniffle](https://github.com/apache/incubator-uniffle) as remote shuffle service. Currently, the supported Uniffle versions are `0.9.1`. +Uniffle with velox backend supports [Uniffle](https://github.com/apache/incubator-uniffle) as remote shuffle service. Currently, the supported Uniffle versions are `0.9.2`. First refer to this URL(https://uniffle.apache.org/docs/intro) to get start with uniffle. diff --git a/pom.xml b/pom.xml index 07cd0d5f3ad2..41dfc803814c 100644 --- a/pom.xml +++ b/pom.xml @@ -66,7 +66,7 @@ 2.4.0 24 0.5.3 - 0.9.1 + 0.9.2 15.0.0 15.0.0-gluten arrow-memory-unsafe diff --git a/tools/gluten-it/pom.xml b/tools/gluten-it/pom.xml index 22256bd281ec..93af9b9d778c 100644 --- a/tools/gluten-it/pom.xml +++ b/tools/gluten-it/pom.xml @@ -22,7 +22,7 @@ 2.12 3 0.3.2-incubating - 0.9.1 + 0.9.2 1.4.0-SNAPSHOT 32.0.1-jre 1.1 From 28f9cfd364856eb6b76d9b4be753baaf402bfbef Mon Sep 17 00:00:00 2001 From: Chang chen Date: Wed, 5 Feb 2025 16:54:29 +0800 Subject: [PATCH 08/22] [GLUTEN-8655][CH] Refactor: remove clickhouse.lib.path (#8656) * remove SystemParameters.getClickHouseLibPath * remove SystemParameters.getClickHouseLibPath - 3.2 * remove SystemParameters.getClickHouseLibPath - 3.3 * remove UTSystemParameters.clickHouseLibPath * remove SystemParameters * remvoe clickhouse.lib.path * remove tpcds.data.path * Fix * Revert "Fix" This reverts commit cd006817e40f47497833f99f1de96f80d56861f9. * fix spark34 * Simplify the testing directory * fix doc * remove unnecessary test data --- backends-clickhouse/pom.xml | 6 --- .../execution/GlutenClickHouseJoinSuite.scala | 3 -- ...GlutenClickHouseNativeExceptionSuite.scala | 10 +---- .../GlutenClickHouseNativeLibSuite.scala | 2 - .../GlutenClickHouseSyntheticDataSuite.scala | 2 - .../GlutenClickHouseTPCDSAbstractSuite.scala | 1 - .../GlutenClickHouseTPCHAbstractSuite.scala | 2 - ...ClickHouseWholeStageTransformerSuite.scala | 2 - .../GlutenFunctionValidateSuite.scala | 4 -- .../GlutenClickhouseFunctionSuite.scala | 2 - .../hive/GlutenClickHouseHiveTableSuite.scala | 3 -- ...lutenClickHouseNativeWriteTableSuite.scala | 2 - .../gluten/utils/UTSystemParameters.scala | 30 +------------- .../benchmarks/CHSqlBasedBenchmark.scala | 6 +-- .../tests/data/all_meta/data/f0/._SUCCESS.crc | Bin 8 -> 0 bytes ...-8bdd-a6911d0d465a-c000.snappy.parquet.crc | Bin 20 -> 0 bytes .../tests/data/all_meta/data/f0/_SUCCESS | 0 ...4f9b-8bdd-a6911d0d465a-c000.snappy.parquet | Bin 1282 -> 0 bytes .../tests/data/all_meta/data/f1/._SUCCESS.crc | Bin 8 -> 0 bytes ...-9cf1-aa914dc7cd57-c000.snappy.parquet.crc | Bin 20 -> 0 bytes .../tests/data/all_meta/data/f1/_SUCCESS | 0 ...495b-9cf1-aa914dc7cd57-c000.snappy.parquet | Bin 1282 -> 0 bytes docs/developers/clickhouse-backend-debug.md | 2 +- gluten-ut/common/pom.xml | 6 --- .../gluten/utils/SystemParameters.scala | 38 ------------------ .../spark/sql/DummyFilterColmnarHelper.scala | 4 +- .../spark/sql/GlutenSQLTestsBaseTrait.scala | 4 +- .../apache/spark/sql/GlutenTestsTrait.scala | 3 +- gluten-ut/spark32/pom.xml | 9 ----- .../spark/sql/GlutenSQLQueryTestSuite.scala | 4 +- .../benchmarks/ParquetReadBenchmark.scala | 7 +--- .../joins/GlutenBroadcastJoinSuite.scala | 4 +- .../execution/GlutenHiveSQLQueryCHSuite.scala | 4 -- .../statistics/SparkFunctionStatistics.scala | 4 +- gluten-ut/spark33/pom.xml | 9 ----- .../spark/sql/GlutenSQLQueryTestSuite.scala | 4 +- .../benchmarks/ParquetReadBenchmark.scala | 7 +--- .../joins/GlutenBroadcastJoinSuite.scala | 4 +- .../execution/GlutenHiveSQLQueryCHSuite.scala | 4 -- .../statistics/SparkFunctionStatistics.scala | 4 +- gluten-ut/spark34/pom.xml | 7 ---- .../spark/sql/GlutenSQLQueryTestSuite.scala | 4 +- .../benchmarks/ParquetReadBenchmark.scala | 7 +--- .../joins/GlutenBroadcastJoinSuite.scala | 4 +- .../execution/GlutenHiveSQLQueryCHSuite.scala | 4 -- .../statistics/SparkFunctionStatistics.scala | 4 +- gluten-ut/spark35/pom.xml | 7 ---- .../spark/sql/GlutenSQLQueryTestSuite.scala | 4 +- .../benchmarks/ParquetReadBenchmark.scala | 7 +--- .../joins/GlutenBroadcastJoinSuite.scala | 4 +- .../execution/GlutenHiveSQLQueryCHSuite.scala | 4 -- .../statistics/SparkFunctionStatistics.scala | 4 +- gluten-ut/test/pom.xml | 9 ----- ...MergeTwoPhasesHashBaseAggregateSuite.scala | 4 +- .../GlutenExpressionMappingSuite.scala | 3 +- .../GlutenExtensionRewriteRuleSuite.scala | 4 +- .../org/apache/gluten/sql/SQLQuerySuite.scala | 2 - .../GlutenExpressionDataTypesValidation.scala | 4 +- .../GlutenNoopWriterRuleSuite.scala | 9 +---- pom.xml | 2 - 60 files changed, 33 insertions(+), 260 deletions(-) delete mode 100644 cpp-ch/local-engine/tests/data/all_meta/data/f0/._SUCCESS.crc delete mode 100644 cpp-ch/local-engine/tests/data/all_meta/data/f0/.part-00000-92bb25d0-7446-4f9b-8bdd-a6911d0d465a-c000.snappy.parquet.crc delete mode 100644 cpp-ch/local-engine/tests/data/all_meta/data/f0/_SUCCESS delete mode 100644 cpp-ch/local-engine/tests/data/all_meta/data/f0/part-00000-92bb25d0-7446-4f9b-8bdd-a6911d0d465a-c000.snappy.parquet delete mode 100644 cpp-ch/local-engine/tests/data/all_meta/data/f1/._SUCCESS.crc delete mode 100644 cpp-ch/local-engine/tests/data/all_meta/data/f1/.part-00000-963c0854-a1e2-495b-9cf1-aa914dc7cd57-c000.snappy.parquet.crc delete mode 100644 cpp-ch/local-engine/tests/data/all_meta/data/f1/_SUCCESS delete mode 100644 cpp-ch/local-engine/tests/data/all_meta/data/f1/part-00000-963c0854-a1e2-495b-9cf1-aa914dc7cd57-c000.snappy.parquet delete mode 100644 gluten-ut/common/src/test/scala/org/apache/gluten/utils/SystemParameters.scala diff --git a/backends-clickhouse/pom.xml b/backends-clickhouse/pom.xml index 52c88354a195..9b954c2ccf6e 100644 --- a/backends-clickhouse/pom.xml +++ b/backends-clickhouse/pom.xml @@ -490,12 +490,6 @@ test - - - ${clickhouse.lib.path} - ${tpcds.data.path} - - diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseJoinSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseJoinSuite.scala index 28aad5537903..efde239f4357 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseJoinSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseJoinSuite.scala @@ -17,8 +17,6 @@ package org.apache.gluten.execution import org.apache.gluten.backendsapi.clickhouse.CHConf -import org.apache.gluten.config.GlutenConfig -import org.apache.gluten.utils.UTSystemParameters import org.apache.spark.SparkConf import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskEnd} @@ -43,7 +41,6 @@ class GlutenClickHouseJoinSuite extends GlutenClickHouseWholeStageTransformerSui .set("spark.sql.adaptive.enabled", "false") .set("spark.sql.files.minPartitionNum", "1") .set(ClickHouseConfig.CLICKHOUSE_WORKER_ID, "1") - .set(GlutenConfig.GLUTEN_LIB_PATH.key, UTSystemParameters.clickHouseLibPath) .set("spark.gluten.sql.columnar.iterator", "true") .set("spark.gluten.sql.columnar.hashagg.enablefinal", "true") .set("spark.gluten.sql.enable.native.validation", "false") diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseNativeExceptionSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseNativeExceptionSuite.scala index 9b617b998c6c..394895fdc8aa 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseNativeExceptionSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseNativeExceptionSuite.scala @@ -16,18 +16,10 @@ */ package org.apache.gluten.execution -import org.apache.gluten.config.GlutenConfig -import org.apache.gluten.utils.{TestExceptionUtils, UTSystemParameters} - -import org.apache.spark.SparkConf +import org.apache.gluten.utils.TestExceptionUtils class GlutenClickHouseNativeExceptionSuite extends GlutenClickHouseWholeStageTransformerSuite { - override protected def sparkConf: SparkConf = { - super.sparkConf - .set(GlutenConfig.GLUTEN_LIB_PATH.key, UTSystemParameters.clickHouseLibPath) - } - test("native exception caught by jvm") { try { TestExceptionUtils.generateNativeException() diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseNativeLibSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseNativeLibSuite.scala index 98d8fb9ee06b..f91e84194259 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseNativeLibSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseNativeLibSuite.scala @@ -18,7 +18,6 @@ package org.apache.gluten.execution import org.apache.gluten.config.GlutenConfig import org.apache.gluten.exception.GlutenException -import org.apache.gluten.utils.UTSystemParameters import org.apache.spark.SparkConf import org.apache.spark.sql.SparkSession @@ -65,7 +64,6 @@ class GlutenClickHouseNativeLibSuite extends PlanTest { .builder() .master("local[1]") .config(baseSparkConf) - .config(GlutenConfig.GLUTEN_LIB_PATH.key, UTSystemParameters.clickHouseLibPath) .config(GlutenConfig.GLUTEN_EXECUTOR_LIB_PATH.key, "/path/not/exist/libch.so") .getOrCreate() spark.sql("select 1").show() diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseSyntheticDataSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseSyntheticDataSuite.scala index 6fffe30f05d8..25244c58618e 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseSyntheticDataSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseSyntheticDataSuite.scala @@ -17,7 +17,6 @@ package org.apache.gluten.execution import org.apache.gluten.config.GlutenConfig -import org.apache.gluten.utils.UTSystemParameters import org.apache.spark.SparkConf import org.apache.spark.internal.Logging @@ -52,7 +51,6 @@ class GlutenClickHouseSyntheticDataSuite .set("spark.databricks.delta.properties.defaults.checkpointInterval", "5") .set("spark.databricks.delta.stalenessLimit", "3600000") .set(ClickHouseConfig.CLICKHOUSE_WORKER_ID, "1") - .set(GlutenConfig.GLUTEN_LIB_PATH.key, UTSystemParameters.clickHouseLibPath) .set("spark.gluten.sql.columnar.iterator", "true") .set("spark.gluten.sql.columnar.hashagg.enablefinal", "true") .set("spark.gluten.sql.enable.native.validation", "false") diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCDSAbstractSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCDSAbstractSuite.scala index f1f93692ecb0..d15ac1334cfb 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCDSAbstractSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCDSAbstractSuite.scala @@ -132,7 +132,6 @@ abstract class GlutenClickHouseTPCDSAbstractSuite .set("spark.databricks.delta.properties.defaults.checkpointInterval", "5") .set("spark.databricks.delta.stalenessLimit", "3600000") .set(ClickHouseConfig.CLICKHOUSE_WORKER_ID, "1") - .set(GlutenConfig.GLUTEN_LIB_PATH.key, UTSystemParameters.clickHouseLibPath) .set("spark.gluten.sql.columnar.iterator", "true") .set("spark.gluten.sql.columnar.hashagg.enablefinal", "true") .set("spark.gluten.sql.enable.native.validation", "false") diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHAbstractSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHAbstractSuite.scala index 4c748fcb1d06..6a7b35337ddc 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHAbstractSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHAbstractSuite.scala @@ -17,7 +17,6 @@ package org.apache.gluten.execution import org.apache.gluten.config.GlutenConfig -import org.apache.gluten.utils.UTSystemParameters import org.apache.spark.SparkConf import org.apache.spark.internal.Logging @@ -568,7 +567,6 @@ abstract class GlutenClickHouseTPCHAbstractSuite .set("spark.databricks.delta.stalenessLimit", "3600000") .set("spark.sql.extensions", "io.delta.sql.DeltaSparkSessionExtension") .set(ClickHouseConfig.CLICKHOUSE_WORKER_ID, "1") - .set(GlutenConfig.GLUTEN_LIB_PATH.key, UTSystemParameters.clickHouseLibPath) .set("spark.gluten.sql.columnar.iterator", "true") .set("spark.gluten.sql.columnar.hashagg.enablefinal", "true") .set("spark.gluten.sql.enable.native.validation", "false") diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseWholeStageTransformerSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseWholeStageTransformerSuite.scala index db5fede0d5ae..24efab534319 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseWholeStageTransformerSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseWholeStageTransformerSuite.scala @@ -17,7 +17,6 @@ package org.apache.gluten.execution import org.apache.gluten.backendsapi.clickhouse.RuntimeConfig -import org.apache.gluten.config.GlutenConfig import org.apache.gluten.utils.UTSystemParameters import org.apache.spark.{SPARK_VERSION_SHORT, SparkConf} @@ -79,7 +78,6 @@ class GlutenClickHouseWholeStageTransformerSuite extends WholeStageTransformerSu import org.apache.gluten.backendsapi.clickhouse.CHConf._ val conf = super.sparkConf - .set(GlutenConfig.GLUTEN_LIB_PATH.key, UTSystemParameters.clickHouseLibPath) .set("spark.gluten.sql.enable.native.validation", "false") .set("spark.sql.warehouse.dir", warehouse) .setCHConfig("user_defined_path", "/tmp/user_defined") diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenFunctionValidateSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenFunctionValidateSuite.scala index 5923f1484ae5..e14666976e6e 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenFunctionValidateSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenFunctionValidateSuite.scala @@ -16,9 +16,6 @@ */ package org.apache.gluten.execution -import org.apache.gluten.config.GlutenConfig -import org.apache.gluten.utils.UTSystemParameters - import org.apache.spark.SparkConf import org.apache.spark.sql.{DataFrame, Row, TestUtils} import org.apache.spark.sql.catalyst.expressions.{Expression, GetJsonObject, Literal} @@ -57,7 +54,6 @@ class GlutenFunctionValidateSuite extends GlutenClickHouseWholeStageTransformerS .set("spark.databricks.delta.properties.defaults.checkpointInterval", "5") .set("spark.databricks.delta.stalenessLimit", "3600000") .set(ClickHouseConfig.CLICKHOUSE_WORKER_ID, "1") - .set(GlutenConfig.GLUTEN_LIB_PATH.key, UTSystemParameters.clickHouseLibPath) .set("spark.gluten.sql.columnar.iterator", "true") .set("spark.gluten.sql.columnar.hashagg.enablefinal", "true") .set("spark.gluten.sql.enable.native.validation", "false") diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/compatibility/GlutenClickhouseFunctionSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/compatibility/GlutenClickhouseFunctionSuite.scala index a8ffc90dc33e..faccfa105c2a 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/compatibility/GlutenClickhouseFunctionSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/compatibility/GlutenClickhouseFunctionSuite.scala @@ -18,7 +18,6 @@ package org.apache.gluten.execution.compatibility import org.apache.gluten.config.GlutenConfig import org.apache.gluten.execution.{GlutenClickHouseTPCHAbstractSuite, ProjectExecTransformer} -import org.apache.gluten.utils.UTSystemParameters import org.apache.spark.SparkConf import org.apache.spark.sql.execution.datasources.v2.clickhouse.ClickHouseConfig @@ -51,7 +50,6 @@ class GlutenClickhouseFunctionSuite extends GlutenClickHouseTPCHAbstractSuite { .set("spark.databricks.delta.properties.defaults.checkpointInterval", "5") .set("spark.databricks.delta.stalenessLimit", "3600000") .set(ClickHouseConfig.CLICKHOUSE_WORKER_ID, "1") - .set(GlutenConfig.GLUTEN_LIB_PATH.key, UTSystemParameters.clickHouseLibPath) .set("spark.gluten.sql.columnar.iterator", "true") .set("spark.gluten.sql.columnar.hashagg.enablefinal", "true") .set("spark.gluten.sql.enable.native.validation", "false") diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseHiveTableSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseHiveTableSuite.scala index 917317bcc65d..dddeb43d6a38 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseHiveTableSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseHiveTableSuite.scala @@ -16,10 +16,8 @@ */ package org.apache.gluten.execution.hive -import org.apache.gluten.config.GlutenConfig import org.apache.gluten.execution.{FileSourceScanExecTransformer, GlutenClickHouseWholeStageTransformerSuite, ProjectExecTransformer, TransformSupport} import org.apache.gluten.test.AllDataTypesWithComplexType -import org.apache.gluten.utils.UTSystemParameters import org.apache.spark.SparkConf import org.apache.spark.sql.{DataFrame, SaveMode} @@ -56,7 +54,6 @@ class GlutenClickHouseHiveTableSuite .set("spark.sql.adaptive.enabled", "false") .set("spark.sql.files.minPartitionNum", "1") .set(ClickHouseConfig.CLICKHOUSE_WORKER_ID, "1") - .set(GlutenConfig.GLUTEN_LIB_PATH.key, UTSystemParameters.clickHouseLibPath) .set("spark.gluten.sql.columnar.iterator", "true") .set("spark.gluten.sql.columnar.hashagg.enablefinal", "true") .set("spark.gluten.sql.enable.native.validation", "false") diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseNativeWriteTableSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseNativeWriteTableSuite.scala index b6067f138d07..1ee0b18b1194 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseNativeWriteTableSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseNativeWriteTableSuite.scala @@ -20,7 +20,6 @@ import org.apache.gluten.backendsapi.clickhouse.RuntimeConfig import org.apache.gluten.config.GlutenConfig import org.apache.gluten.execution.GlutenClickHouseWholeStageTransformerSuite import org.apache.gluten.test.AllDataTypesWithComplexType.genTestData -import org.apache.gluten.utils.UTSystemParameters import org.apache.spark.SparkConf import org.apache.spark.gluten.NativeWriteChecker @@ -57,7 +56,6 @@ class GlutenClickHouseNativeWriteTableSuite .set("spark.databricks.delta.properties.defaults.checkpointInterval", "5") .set("spark.databricks.delta.stalenessLimit", "3600000") .set(ClickHouseConfig.CLICKHOUSE_WORKER_ID, "1") - .set(GlutenConfig.GLUTEN_LIB_PATH.key, UTSystemParameters.clickHouseLibPath) .set("spark.gluten.sql.columnar.iterator", "true") .set("spark.gluten.sql.columnar.hashagg.enablefinal", "true") .set("spark.gluten.sql.enable.native.validation", "false") diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/utils/UTSystemParameters.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/utils/UTSystemParameters.scala index 334f9fb4bec2..31b5514cceac 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/utils/UTSystemParameters.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/utils/UTSystemParameters.scala @@ -18,15 +18,6 @@ package org.apache.gluten.utils object UTSystemParameters { - private val CLICKHOUSE_LIB_PATH_KEY = "clickhouse.lib.path" - private val CLICKHOUSE_LIB_PATH_DEFAULT_VALUE = "/usr/local/clickhouse/lib/libch.so" - - def clickHouseLibPath: String = { - System.getProperty( - UTSystemParameters.CLICKHOUSE_LIB_PATH_KEY, - UTSystemParameters.CLICKHOUSE_LIB_PATH_DEFAULT_VALUE) - } - private val TEST_DATA_PATH_KEY = "gluten.test.data.path" private val TEST_DATA_PATH_DEFAULT_VALUE = "/data" @@ -43,29 +34,12 @@ object UTSystemParameters { UTSystemParameters.TEST_DATA_DISK_OUTPUT_KEY, UTSystemParameters.TEST_DATA_DISK_OUTPUT_DEFAULT_VALUE) } - - private val TPCDS_DATA_PATH_KEY = "tpcds.data.path" - private val TPCDS_DATA_DECIMAL_PATH_KEY = "tpcds.data.decimal.path" private val TPCDS_RELATIVE_DATA_PATH = "tpcds-data-sf1" private val TPCDS_DECIMAL_RELATIVE_DATA_PATH = "tpcds-data-sf1-decimal" - def tpcdsDataPath: String = { - val result = System.getProperty(UTSystemParameters.TPCDS_DATA_PATH_KEY, null) - if (result == null) { - s"$testDataPath/$TPCDS_RELATIVE_DATA_PATH" - } else { - result - } - } + def tpcdsDataPath: String = s"$testDataPath/$TPCDS_RELATIVE_DATA_PATH" - def tpcdsDecimalDataPath: String = { - val result = System.getProperty(UTSystemParameters.TPCDS_DATA_DECIMAL_PATH_KEY, null) - if (result == null) { - s"$testDataPath/$TPCDS_DECIMAL_RELATIVE_DATA_PATH" - } else { - result - } - } + def tpcdsDecimalDataPath: String = s"$testDataPath/$TPCDS_DECIMAL_RELATIVE_DATA_PATH" private val TEST_MERGETREE_ON_OBJECT_STORAGE = "gluten.ch.test.mergetree.object.storage" private val TEST_MERGETREE_ON_OBJECT_STORAGE_DEFAULT_VALUE = "true" diff --git a/backends-clickhouse/src/test/scala/org/apache/spark/sql/execution/benchmarks/CHSqlBasedBenchmark.scala b/backends-clickhouse/src/test/scala/org/apache/spark/sql/execution/benchmarks/CHSqlBasedBenchmark.scala index 7b067643f68d..2a9a2b233a1f 100644 --- a/backends-clickhouse/src/test/scala/org/apache/spark/sql/execution/benchmarks/CHSqlBasedBenchmark.scala +++ b/backends-clickhouse/src/test/scala/org/apache/spark/sql/execution/benchmarks/CHSqlBasedBenchmark.scala @@ -18,7 +18,6 @@ package org.apache.spark.sql.execution.benchmarks import org.apache.gluten.config.GlutenConfig import org.apache.gluten.jni.JniLibLoader -import org.apache.gluten.utils.UTSystemParameters import org.apache.spark.SparkConf import org.apache.spark.sql.delta.DeltaLog @@ -32,7 +31,6 @@ trait CHSqlBasedBenchmark extends SqlBasedBenchmark { def getSparkConf: SparkConf = { val conf = new SparkConf() .setAppName(appName) - .setIfMissing(GlutenConfig.GLUTEN_LIB_PATH.key, UTSystemParameters.clickHouseLibPath) .setIfMissing("spark.master", s"local[$thrdNum]") .set("spark.plugins", "org.apache.gluten.GlutenPlugin") .set( @@ -57,9 +55,7 @@ trait CHSqlBasedBenchmark extends SqlBasedBenchmark { override def afterAll(): Unit = { DeltaLog.clearCache() - val libPath = - spark.conf.get(GlutenConfig.GLUTEN_LIB_PATH.key, UTSystemParameters.clickHouseLibPath) - JniLibLoader.unloadFromPath(libPath) + JniLibLoader.unloadFromPath(spark.conf.get(GlutenConfig.GLUTEN_LIB_PATH.key)) // Wait for Ctrl+C, convenient for seeing Spark UI // Thread.sleep(600000) super.afterAll() diff --git a/cpp-ch/local-engine/tests/data/all_meta/data/f0/._SUCCESS.crc b/cpp-ch/local-engine/tests/data/all_meta/data/f0/._SUCCESS.crc deleted file mode 100644 index 3b7b044936a890cd8d651d349a752d819d71d22c..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8 PcmYc;N@ieSU}69O2$TUk diff --git a/cpp-ch/local-engine/tests/data/all_meta/data/f0/.part-00000-92bb25d0-7446-4f9b-8bdd-a6911d0d465a-c000.snappy.parquet.crc b/cpp-ch/local-engine/tests/data/all_meta/data/f0/.part-00000-92bb25d0-7446-4f9b-8bdd-a6911d0d465a-c000.snappy.parquet.crc deleted file mode 100644 index a4db2dd16fbfd36d990c93aab7bb43e3a7834945..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 20 bcmYc;N@ieSU}E?y{70`$(_*XATQgGtJrf3d diff --git a/cpp-ch/local-engine/tests/data/all_meta/data/f0/_SUCCESS b/cpp-ch/local-engine/tests/data/all_meta/data/f0/_SUCCESS deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/cpp-ch/local-engine/tests/data/all_meta/data/f0/part-00000-92bb25d0-7446-4f9b-8bdd-a6911d0d465a-c000.snappy.parquet b/cpp-ch/local-engine/tests/data/all_meta/data/f0/part-00000-92bb25d0-7446-4f9b-8bdd-a6911d0d465a-c000.snappy.parquet deleted file mode 100644 index 1e702dce838679b7bba4478846f6ffc9bc3556bb..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1282 zcmb7E-)qxQ6u!NQC5+M`>|7EEQi2Pc)X>=0bz=znFl3BTeCUe|Nt?SGm!?~qDr58? z5Pb2$r#<;F@JaAdMDQ;VVZMrQP8j$fi039*67a!U%KdTfcfRkQ?$lU24ivX#Pzb5EC?P~D$v8f7274G^02O}yIC)zwz~-(eFco14xdsM2Kl%BwT!74X zB*8?$Bw?Y4;aD~gD;E-)nk@@i zl3-QEmbVJgZW?$`c{uT+!!(IeP1m+eMTd&kR!AAE5|lt0 zWt(ii*?Q8(t7T5zCz;6{XAL}IFW&|R}$PI<~a(ag(m_9 zhcXY&QAo5c9*{m;7fC+|(o+d;5cV3)GZprZE?{fxsyYsbmb33*#@fRp zL-Y=h;;CK$dchEZ!PNJizK;k@<#pqN)mS^4&FWb;HVwh4iWfwD$ivHLiw7@DzszMl zH-j7g_EhS#4{|FVqyPW_ diff --git a/cpp-ch/local-engine/tests/data/all_meta/data/f1/._SUCCESS.crc b/cpp-ch/local-engine/tests/data/all_meta/data/f1/._SUCCESS.crc deleted file mode 100644 index 3b7b044936a890cd8d651d349a752d819d71d22c..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8 PcmYc;N@ieSU}69O2$TUk diff --git a/cpp-ch/local-engine/tests/data/all_meta/data/f1/.part-00000-963c0854-a1e2-495b-9cf1-aa914dc7cd57-c000.snappy.parquet.crc b/cpp-ch/local-engine/tests/data/all_meta/data/f1/.part-00000-963c0854-a1e2-495b-9cf1-aa914dc7cd57-c000.snappy.parquet.crc deleted file mode 100644 index 98fb6204411d2f24b0b80d5b57792a05b3e32f47..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 20 bcmYc;N@ieSU}BhA^-I7k@}9iXTQgGtJM;!# diff --git a/cpp-ch/local-engine/tests/data/all_meta/data/f1/_SUCCESS b/cpp-ch/local-engine/tests/data/all_meta/data/f1/_SUCCESS deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/cpp-ch/local-engine/tests/data/all_meta/data/f1/part-00000-963c0854-a1e2-495b-9cf1-aa914dc7cd57-c000.snappy.parquet b/cpp-ch/local-engine/tests/data/all_meta/data/f1/part-00000-963c0854-a1e2-495b-9cf1-aa914dc7cd57-c000.snappy.parquet deleted file mode 100644 index 4cdaff7168a77f7aef33ac16cd9f9897b7faab44..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1282 zcmb7EO>5Lp6n*m&Ybd26^ga>@Is_Y98N7u{DMe)8QO#&{g!|*=tJNKM-&rQ;~bKeF6I#l7= z_m^)k>OgTj0}3JYO-cxnDl(3~=T9)c04n_a{o^?;!Dg?eu&W3|$Q3Z)`0JO`GUQTQ z5=@lXM>yjzQ~dbr^&6!`HQ&r|Vv2iR?K0I=S!ibyP>m6ihJ_x66WJoHoJnYEzAR-) zf>jk`7Sx#n+sH{Yo$bIzwmPgpCtIFW(sRW7_vXeUCw%;H+& zwscV##en4WIced67wio|CLm*wdTt7zjRWrq5650K$&wh=bZyH-C+4)4LT0ckK?#&m zw#l|lWj!?^9T{0lEJ=5GZ!{Bi31O!aEU-@$EMo4Us~MNLO+s~-ZJ0YqB^*mfK~?O^ zR4-(tI!>q#D>}$2*HTguR+8{-v5t9ISucDpR!E$?Uqs6S?`2@@l?1nmd4d92;i*8$ zq1=NL6cTMq2Bgo|Wzx@r^hkp1guO%aOoe@*3)tG4st&`x<%}F`oR##@N?I@s)C&!( zW!)s|@?VJYzHec>_hTMe;qa+7;F04xk)s~d10$M@xUpu8qcHA8M$OpsxbKdQwFd`= z=p7y_H_f<6L+*!P{Tj|fcVHRFucSUZ?bYk4*{4Z)d;7eu_z!}Dj02QN=Q&t*L~ zha3Lze=b@)%>@Ixa*kqmr~hzccS_XFtOGX&VKr-6)wb-0WmlKFvFE#0x7)Pa4ZFVT rwz?~gChxUY8+Et6Qt!DAw{O|JS#P^;*ICxY8=C*gH~h$S@ay*n^|c}{ diff --git a/docs/developers/clickhouse-backend-debug.md b/docs/developers/clickhouse-backend-debug.md index 6f27b25a09d6..8fee0eab5d14 100644 --- a/docs/developers/clickhouse-backend-debug.md +++ b/docs/developers/clickhouse-backend-debug.md @@ -31,7 +31,7 @@ parent: /developer-overview/ ![gluten-debug-idea-config.png](../image/ClickHouse/gluten-debug-idea-config.png) VM Options: - `-Dtpcds.data.path=/data/tpcds-data-sf1 -Dclickhouse.lib.path=/path/to/gluten/cpp-ch/build/utils/extern-local-engine/libch.so -Dspark.test.home=/path/to/spark33` + `-Dgluten.test.data.path=/data -Dspark.gluten.sql.columnar.libpath=/path/to/gluten/cpp-ch/build/utils/extern-local-engine/libch.so -Dspark.test.home=/path/to/spark33` > Download tpcds-data in https://gluten-nginx.kyligence.com/dataset/ > Download spark33 using `git clone --depth 1 --branch v3.3.1 https://github.com/apache/spark.git /tmp/spark33` diff --git a/gluten-ut/common/pom.xml b/gluten-ut/common/pom.xml index 11caaa4dd4ff..5d6dd94b43b6 100644 --- a/gluten-ut/common/pom.xml +++ b/gluten-ut/common/pom.xml @@ -69,12 +69,6 @@ test - - - ${clickhouse.lib.path} - ${tpcds.data.path} - - diff --git a/gluten-ut/common/src/test/scala/org/apache/gluten/utils/SystemParameters.scala b/gluten-ut/common/src/test/scala/org/apache/gluten/utils/SystemParameters.scala deleted file mode 100644 index 71fc26227baa..000000000000 --- a/gluten-ut/common/src/test/scala/org/apache/gluten/utils/SystemParameters.scala +++ /dev/null @@ -1,38 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.gluten.utils - -object SystemParameters { - - val CLICKHOUSE_LIB_PATH_KEY = "clickhouse.lib.path" - val CLICKHOUSE_LIB_PATH_DEFAULT_VALUE = "/usr/local/clickhouse/lib/libch.so" - - val TPCDS_DATA_PATH_KEY = "tpcds.data.path" - val TPCDS_DATA_PATH_DEFAULT_VALUE = "/data/tpcds-data-sf1" - - def getClickHouseLibPath: String = { - System.getProperty( - SystemParameters.CLICKHOUSE_LIB_PATH_KEY, - SystemParameters.CLICKHOUSE_LIB_PATH_DEFAULT_VALUE) - } - - def getTpcdsDataPath: String = { - System.getProperty( - SystemParameters.TPCDS_DATA_PATH_KEY, - SystemParameters.TPCDS_DATA_PATH_DEFAULT_VALUE) - } -} diff --git a/gluten-ut/common/src/test/scala/org/apache/spark/sql/DummyFilterColmnarHelper.scala b/gluten-ut/common/src/test/scala/org/apache/spark/sql/DummyFilterColmnarHelper.scala index ab4c6de441db..41c652fc60d7 100644 --- a/gluten-ut/common/src/test/scala/org/apache/spark/sql/DummyFilterColmnarHelper.scala +++ b/gluten-ut/common/src/test/scala/org/apache/spark/sql/DummyFilterColmnarHelper.scala @@ -16,8 +16,7 @@ */ package org.apache.spark.sql -import org.apache.gluten.config.GlutenConfig -import org.apache.gluten.utils.{BackendTestUtils, SystemParameters} +import org.apache.gluten.utils.BackendTestUtils import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow @@ -86,7 +85,6 @@ object DummyFilterColmnarHelper { .config("spark.memory.offHeap.size", "1024MB") .config("spark.plugins", "org.apache.gluten.GlutenPlugin") .config("spark.shuffle.manager", "org.apache.spark.shuffle.sort.ColumnarShuffleManager") - .config(GlutenConfig.GLUTEN_LIB_PATH.key, SystemParameters.getClickHouseLibPath) .config("spark.io.compression.codec", "LZ4") .config("spark.gluten.sql.enable.native.validation", "false") } else { diff --git a/gluten-ut/common/src/test/scala/org/apache/spark/sql/GlutenSQLTestsBaseTrait.scala b/gluten-ut/common/src/test/scala/org/apache/spark/sql/GlutenSQLTestsBaseTrait.scala index 128ce144f2ea..b6ece98e4af7 100644 --- a/gluten-ut/common/src/test/scala/org/apache/spark/sql/GlutenSQLTestsBaseTrait.scala +++ b/gluten-ut/common/src/test/scala/org/apache/spark/sql/GlutenSQLTestsBaseTrait.scala @@ -16,8 +16,7 @@ */ package org.apache.spark.sql -import org.apache.gluten.config.GlutenConfig -import org.apache.gluten.utils.{BackendTestUtils, SystemParameters} +import org.apache.gluten.utils.BackendTestUtils import org.apache.spark.SparkConf import org.apache.spark.sql.execution.SparkPlan @@ -105,7 +104,6 @@ object GlutenSQLTestsBaseTrait { .set("spark.io.compression.codec", "LZ4") .set("spark.gluten.sql.columnar.backend.ch.worker.id", "1") .set("spark.gluten.sql.enable.native.validation", "false") - .set(GlutenConfig.GLUTEN_LIB_PATH.key, SystemParameters.getClickHouseLibPath) .set("spark.sql.files.openCostInBytes", "134217728") .set("spark.unsafe.exceptionOnMemoryLeak", "true") } else { diff --git a/gluten-ut/common/src/test/scala/org/apache/spark/sql/GlutenTestsTrait.scala b/gluten-ut/common/src/test/scala/org/apache/spark/sql/GlutenTestsTrait.scala index 330bf3b7ac89..5e0686095c80 100644 --- a/gluten-ut/common/src/test/scala/org/apache/spark/sql/GlutenTestsTrait.scala +++ b/gluten-ut/common/src/test/scala/org/apache/spark/sql/GlutenTestsTrait.scala @@ -20,7 +20,7 @@ import org.apache.gluten.backendsapi.BackendsApiManager import org.apache.gluten.config.GlutenConfig import org.apache.gluten.execution.ProjectExecTransformer import org.apache.gluten.test.TestStats -import org.apache.gluten.utils.{BackendTestUtils, SystemParameters} +import org.apache.gluten.utils.BackendTestUtils import org.apache.spark.sql.GlutenQueryTestUtil.isNaNOrInf import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} @@ -113,7 +113,6 @@ trait GlutenTestsTrait extends GlutenTestsCommonTrait { .config("spark.gluten.sql.columnar.backend.ch.worker.id", "1") .config("spark.gluten.sql.enable.native.validation", "false") .config("spark.sql.files.openCostInBytes", "134217728") - .config(GlutenConfig.GLUTEN_LIB_PATH.key, SystemParameters.getClickHouseLibPath) .config("spark.unsafe.exceptionOnMemoryLeak", "true") .config(GlutenConfig.UT_STATISTIC.key, "true") .getOrCreate() diff --git a/gluten-ut/spark32/pom.xml b/gluten-ut/spark32/pom.xml index 8d0d0f4fcae2..15a56e009fed 100644 --- a/gluten-ut/spark32/pom.xml +++ b/gluten-ut/spark32/pom.xml @@ -58,9 +58,6 @@ test - - - @@ -105,12 +102,6 @@ test - - - ${clickhouse.lib.path} - ${tpcds.data.path} - - diff --git a/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/GlutenSQLQueryTestSuite.scala b/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/GlutenSQLQueryTestSuite.scala index 739446111bd9..7a4b1c781674 100644 --- a/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/GlutenSQLQueryTestSuite.scala +++ b/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/GlutenSQLQueryTestSuite.scala @@ -16,9 +16,8 @@ */ package org.apache.spark.sql -import org.apache.gluten.config.GlutenConfig import org.apache.gluten.exception.GlutenException -import org.apache.gluten.utils.{BackendTestSettings, BackendTestUtils, SystemParameters} +import org.apache.gluten.utils.{BackendTestSettings, BackendTestUtils} import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator @@ -195,7 +194,6 @@ class GlutenSQLQueryTestSuite .set("spark.io.compression.codec", "LZ4") .set("spark.gluten.sql.columnar.backend.ch.worker.id", "1") .set("spark.gluten.sql.enable.native.validation", "false") - .set(GlutenConfig.GLUTEN_LIB_PATH.key, SystemParameters.getClickHouseLibPath) .set("spark.sql.files.openCostInBytes", "134217728") .set("spark.unsafe.exceptionOnMemoryLeak", "true") } else { diff --git a/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/execution/benchmarks/ParquetReadBenchmark.scala b/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/execution/benchmarks/ParquetReadBenchmark.scala index c55f4bf7f0da..aa561f17f0c1 100644 --- a/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/execution/benchmarks/ParquetReadBenchmark.scala +++ b/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/execution/benchmarks/ParquetReadBenchmark.scala @@ -20,7 +20,7 @@ import org.apache.gluten.config.GlutenConfig import org.apache.gluten.execution.{FileSourceScanExecTransformer, WholeStageTransformer} import org.apache.gluten.extension.columnar.transition.Transitions import org.apache.gluten.jni.JniLibLoader -import org.apache.gluten.utils.{BackendTestUtils, SystemParameters} +import org.apache.gluten.utils.BackendTestUtils import org.apache.spark.SparkConf import org.apache.spark.benchmark.Benchmark @@ -82,7 +82,6 @@ object ParquetReadBenchmark extends SqlBasedBenchmark { .set("spark.gluten.sql.enable.native.validation", "false") .set("spark.gluten.sql.columnar.backend.ch.worker.id", "1") .set("spark.gluten.sql.columnar.separate.scan.rdd.for.ch", "false") - .setIfMissing(GlutenConfig.GLUTEN_LIB_PATH.key, SystemParameters.getClickHouseLibPath) .set( "spark.sql.catalog.spark_catalog", "org.apache.spark.sql.execution.datasources.v2.clickhouse.ClickHouseSparkCatalog") @@ -225,9 +224,7 @@ object ParquetReadBenchmark extends SqlBasedBenchmark { override def afterAll(): Unit = { if (BackendTestUtils.isCHBackendLoaded()) { - val libPath = - spark.conf.get(GlutenConfig.GLUTEN_LIB_PATH.key, SystemParameters.getClickHouseLibPath) - JniLibLoader.unloadFromPath(libPath) + JniLibLoader.unloadFromPath(spark.conf.get(GlutenConfig.GLUTEN_LIB_PATH.key)) } super.afterAll() } diff --git a/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/execution/joins/GlutenBroadcastJoinSuite.scala b/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/execution/joins/GlutenBroadcastJoinSuite.scala index 838d30e7ca67..7d83405ebaf4 100644 --- a/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/execution/joins/GlutenBroadcastJoinSuite.scala +++ b/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/execution/joins/GlutenBroadcastJoinSuite.scala @@ -16,9 +16,8 @@ */ package org.apache.spark.sql.execution.joins -import org.apache.gluten.config.GlutenConfig import org.apache.gluten.execution.{BroadcastHashJoinExecTransformerBase, BroadcastNestedLoopJoinExecTransformer, ColumnarToRowExecBase, WholeStageTransformer} -import org.apache.gluten.utils.{BackendTestUtils, SystemParameters} +import org.apache.gluten.utils.BackendTestUtils import org.apache.spark.sql.{GlutenTestsCommonTrait, SparkSession} import org.apache.spark.sql.catalyst.optimizer._ @@ -89,7 +88,6 @@ class GlutenBroadcastJoinSuite extends BroadcastJoinSuite with GlutenTestsCommon .config("spark.gluten.sql.columnar.backend.ch.worker.id", "1") .config("spark.gluten.sql.enable.native.validation", "false") .config("spark.sql.files.openCostInBytes", "134217728") - .config(GlutenConfig.GLUTEN_LIB_PATH.key, SystemParameters.getClickHouseLibPath) .config("spark.unsafe.exceptionOnMemoryLeak", "true") .getOrCreate() } else { diff --git a/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSQLQueryCHSuite.scala b/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSQLQueryCHSuite.scala index 0b14467aba80..859e6df48459 100644 --- a/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSQLQueryCHSuite.scala +++ b/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSQLQueryCHSuite.scala @@ -16,9 +16,6 @@ */ package org.apache.spark.sql.hive.execution -import org.apache.gluten.config.GlutenConfig -import org.apache.gluten.utils.SystemParameters - import org.apache.spark.{DebugFilesystem, SparkConf} import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.TableIdentifier @@ -28,7 +25,6 @@ class GlutenHiveSQLQueryCHSuite extends GlutenHiveSQLQuerySuiteBase { override def sparkConf: SparkConf = { defaultSparkConf .set("spark.plugins", "org.apache.gluten.GlutenPlugin") - .set(GlutenConfig.GLUTEN_LIB_PATH.key, SystemParameters.getClickHouseLibPath) .set("spark.gluten.sql.enable.native.validation", "false") .set("spark.gluten.sql.native.writer.enabled", "true") .set("spark.sql.storeAssignmentPolicy", "legacy") diff --git a/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/statistics/SparkFunctionStatistics.scala b/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/statistics/SparkFunctionStatistics.scala index be9063c83966..925d60df8eab 100644 --- a/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/statistics/SparkFunctionStatistics.scala +++ b/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/statistics/SparkFunctionStatistics.scala @@ -16,9 +16,8 @@ */ package org.apache.spark.sql.statistics -import org.apache.gluten.config.GlutenConfig import org.apache.gluten.execution.GlutenPlan -import org.apache.gluten.utils.{BackendTestUtils, SystemParameters} +import org.apache.gluten.utils.BackendTestUtils import org.apache.spark.sql.{GlutenTestConstants, QueryTest, SparkSession} import org.apache.spark.sql.catalyst.analysis.FunctionRegistry @@ -64,7 +63,6 @@ class SparkFunctionStatistics extends QueryTest { .config("spark.gluten.sql.columnar.backend.ch.worker.id", "1") .config("spark.gluten.sql.enable.native.validation", "false") .config("spark.sql.files.openCostInBytes", "134217728") - .config(GlutenConfig.GLUTEN_LIB_PATH.key, SystemParameters.getClickHouseLibPath) .config("spark.unsafe.exceptionOnMemoryLeak", "true") .getOrCreate() } else { diff --git a/gluten-ut/spark33/pom.xml b/gluten-ut/spark33/pom.xml index f8377195eec9..93683220d399 100644 --- a/gluten-ut/spark33/pom.xml +++ b/gluten-ut/spark33/pom.xml @@ -87,9 +87,6 @@ test - - - @@ -133,12 +130,6 @@ test - - - ${clickhouse.lib.path} - ${tpcds.data.path} - - diff --git a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/GlutenSQLQueryTestSuite.scala b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/GlutenSQLQueryTestSuite.scala index 84d677ef596f..e336ca88a788 100644 --- a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/GlutenSQLQueryTestSuite.scala +++ b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/GlutenSQLQueryTestSuite.scala @@ -16,9 +16,8 @@ */ package org.apache.spark.sql -import org.apache.gluten.config.GlutenConfig import org.apache.gluten.exception.GlutenException -import org.apache.gluten.utils.{BackendTestSettings, BackendTestUtils, SystemParameters} +import org.apache.gluten.utils.{BackendTestSettings, BackendTestUtils} import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator @@ -195,7 +194,6 @@ class GlutenSQLQueryTestSuite .set("spark.io.compression.codec", "LZ4") .set("spark.gluten.sql.columnar.backend.ch.worker.id", "1") .set("spark.gluten.sql.enable.native.validation", "false") - .set(GlutenConfig.GLUTEN_LIB_PATH.key, SystemParameters.getClickHouseLibPath) .set("spark.sql.files.openCostInBytes", "134217728") .set("spark.unsafe.exceptionOnMemoryLeak", "true") } else { diff --git a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/benchmarks/ParquetReadBenchmark.scala b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/benchmarks/ParquetReadBenchmark.scala index aa5ee0a881ec..517f17bba927 100644 --- a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/benchmarks/ParquetReadBenchmark.scala +++ b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/benchmarks/ParquetReadBenchmark.scala @@ -20,7 +20,7 @@ import org.apache.gluten.config.GlutenConfig import org.apache.gluten.execution.{FileSourceScanExecTransformer, WholeStageTransformer} import org.apache.gluten.extension.columnar.transition.Transitions import org.apache.gluten.jni.JniLibLoader -import org.apache.gluten.utils.{BackendTestUtils, SystemParameters} +import org.apache.gluten.utils.BackendTestUtils import org.apache.spark.SparkConf import org.apache.spark.benchmark.Benchmark @@ -82,7 +82,6 @@ object ParquetReadBenchmark extends SqlBasedBenchmark { .set("spark.gluten.sql.enable.native.validation", "false") .set("spark.gluten.sql.columnar.backend.ch.worker.id", "1") .set("spark.gluten.sql.columnar.separate.scan.rdd.for.ch", "false") - .setIfMissing(GlutenConfig.GLUTEN_LIB_PATH.key, SystemParameters.getClickHouseLibPath) .set( "spark.sql.catalog.spark_catalog", "org.apache.spark.sql.execution.datasources.v2.clickhouse.ClickHouseSparkCatalog") @@ -225,9 +224,7 @@ object ParquetReadBenchmark extends SqlBasedBenchmark { override def afterAll(): Unit = { if (BackendTestUtils.isCHBackendLoaded()) { - val libPath = - spark.conf.get(GlutenConfig.GLUTEN_LIB_PATH.key, SystemParameters.getClickHouseLibPath) - JniLibLoader.unloadFromPath(libPath) + JniLibLoader.unloadFromPath(spark.conf.get(GlutenConfig.GLUTEN_LIB_PATH.key)) } super.afterAll() } diff --git a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/joins/GlutenBroadcastJoinSuite.scala b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/joins/GlutenBroadcastJoinSuite.scala index f5fd73079619..c8b371f48fb8 100644 --- a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/joins/GlutenBroadcastJoinSuite.scala +++ b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/joins/GlutenBroadcastJoinSuite.scala @@ -16,8 +16,7 @@ */ package org.apache.spark.sql.execution.joins -import org.apache.gluten.config.GlutenConfig -import org.apache.gluten.utils.{BackendTestUtils, SystemParameters} +import org.apache.gluten.utils.BackendTestUtils import org.apache.spark.sql.{GlutenTestsCommonTrait, SparkSession} import org.apache.spark.sql.catalyst.optimizer.{ConstantFolding, ConvertToLocalRelation, NullPropagation} @@ -66,7 +65,6 @@ class GlutenBroadcastJoinSuite extends BroadcastJoinSuite with GlutenTestsCommon .config("spark.gluten.sql.columnar.backend.ch.worker.id", "1") .config("spark.gluten.sql.enable.native.validation", "false") .config("spark.sql.files.openCostInBytes", "134217728") - .config(GlutenConfig.GLUTEN_LIB_PATH.key, SystemParameters.getClickHouseLibPath) .config("spark.unsafe.exceptionOnMemoryLeak", "true") .getOrCreate() } else { diff --git a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSQLQueryCHSuite.scala b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSQLQueryCHSuite.scala index 0b14467aba80..859e6df48459 100644 --- a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSQLQueryCHSuite.scala +++ b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSQLQueryCHSuite.scala @@ -16,9 +16,6 @@ */ package org.apache.spark.sql.hive.execution -import org.apache.gluten.config.GlutenConfig -import org.apache.gluten.utils.SystemParameters - import org.apache.spark.{DebugFilesystem, SparkConf} import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.TableIdentifier @@ -28,7 +25,6 @@ class GlutenHiveSQLQueryCHSuite extends GlutenHiveSQLQuerySuiteBase { override def sparkConf: SparkConf = { defaultSparkConf .set("spark.plugins", "org.apache.gluten.GlutenPlugin") - .set(GlutenConfig.GLUTEN_LIB_PATH.key, SystemParameters.getClickHouseLibPath) .set("spark.gluten.sql.enable.native.validation", "false") .set("spark.gluten.sql.native.writer.enabled", "true") .set("spark.sql.storeAssignmentPolicy", "legacy") diff --git a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/statistics/SparkFunctionStatistics.scala b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/statistics/SparkFunctionStatistics.scala index 69969e948e8c..c349d9ca9184 100644 --- a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/statistics/SparkFunctionStatistics.scala +++ b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/statistics/SparkFunctionStatistics.scala @@ -16,9 +16,8 @@ */ package org.apache.spark.sql.statistics -import org.apache.gluten.config.GlutenConfig import org.apache.gluten.execution.GlutenPlan -import org.apache.gluten.utils.{BackendTestUtils, SystemParameters} +import org.apache.gluten.utils.BackendTestUtils import org.apache.spark.sql.{GlutenTestConstants, QueryTest, SparkSession} import org.apache.spark.sql.catalyst.analysis.FunctionRegistry @@ -64,7 +63,6 @@ class SparkFunctionStatistics extends QueryTest { .config("spark.gluten.sql.columnar.backend.ch.worker.id", "1") .config("spark.gluten.sql.enable.native.validation", "false") .config("spark.sql.files.openCostInBytes", "134217728") - .config(GlutenConfig.GLUTEN_LIB_PATH.key, SystemParameters.getClickHouseLibPath) .config("spark.unsafe.exceptionOnMemoryLeak", "true") .getOrCreate() } else { diff --git a/gluten-ut/spark34/pom.xml b/gluten-ut/spark34/pom.xml index e61142b79df8..e0e55c23e8c3 100644 --- a/gluten-ut/spark34/pom.xml +++ b/gluten-ut/spark34/pom.xml @@ -87,7 +87,6 @@ 2.0.6 2.19.0 - @@ -132,12 +131,6 @@ test - - - ${clickhouse.lib.path} - ${tpcds.data.path} - - diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenSQLQueryTestSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenSQLQueryTestSuite.scala index 7c0e9801c2de..39a36898af6d 100644 --- a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenSQLQueryTestSuite.scala +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenSQLQueryTestSuite.scala @@ -16,9 +16,8 @@ */ package org.apache.spark.sql -import org.apache.gluten.config.GlutenConfig import org.apache.gluten.exception.GlutenException -import org.apache.gluten.utils.{BackendTestSettings, BackendTestUtils, SystemParameters} +import org.apache.gluten.utils.{BackendTestSettings, BackendTestUtils} import org.apache.spark.{SparkConf, SparkException, SparkThrowable} import org.apache.spark.ErrorMessageFormat.MINIMAL @@ -199,7 +198,6 @@ class GlutenSQLQueryTestSuite .set("spark.io.compression.codec", "LZ4") .set("spark.gluten.sql.columnar.backend.ch.worker.id", "1") .set("spark.gluten.sql.enable.native.validation", "false") - .set(GlutenConfig.GLUTEN_LIB_PATH.key, SystemParameters.getClickHouseLibPath) .set("spark.sql.files.openCostInBytes", "134217728") .set("spark.unsafe.exceptionOnMemoryLeak", "true") } else { diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/benchmarks/ParquetReadBenchmark.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/benchmarks/ParquetReadBenchmark.scala index d065495b35d6..62eb3ec8a36e 100644 --- a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/benchmarks/ParquetReadBenchmark.scala +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/benchmarks/ParquetReadBenchmark.scala @@ -21,7 +21,7 @@ import org.apache.gluten.config.GlutenConfig import org.apache.gluten.execution.{FileSourceScanExecTransformer, WholeStageTransformer} import org.apache.gluten.extension.columnar.transition.Transitions import org.apache.gluten.jni.JniLibLoader -import org.apache.gluten.utils.{BackendTestUtils, SystemParameters} +import org.apache.gluten.utils.BackendTestUtils import org.apache.spark.SparkConf import org.apache.spark.benchmark.Benchmark @@ -83,7 +83,6 @@ object ParquetReadBenchmark extends SqlBasedBenchmark { .set("spark.gluten.sql.enable.native.validation", "false") .set("spark.gluten.sql.columnar.backend.ch.worker.id", "1") .set("spark.gluten.sql.columnar.separate.scan.rdd.for.ch", "false") - .setIfMissing(GlutenConfig.GLUTEN_LIB_PATH.key, SystemParameters.getClickHouseLibPath) .set( "spark.sql.catalog.spark_catalog", "org.apache.spark.sql.execution.datasources.v2.clickhouse.ClickHouseSparkCatalog") @@ -227,9 +226,7 @@ object ParquetReadBenchmark extends SqlBasedBenchmark { override def afterAll(): Unit = { if (BackendTestUtils.isCHBackendLoaded()) { - val libPath = - spark.conf.get(GlutenConfig.GLUTEN_LIB_PATH.key, SystemParameters.getClickHouseLibPath) - JniLibLoader.unloadFromPath(libPath) + JniLibLoader.unloadFromPath(spark.conf.get(GlutenConfig.GLUTEN_LIB_PATH.key)) } super.afterAll() } diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/joins/GlutenBroadcastJoinSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/joins/GlutenBroadcastJoinSuite.scala index e9d5f9bbdc37..6543160f1ac9 100644 --- a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/joins/GlutenBroadcastJoinSuite.scala +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/joins/GlutenBroadcastJoinSuite.scala @@ -16,8 +16,7 @@ */ package org.apache.spark.sql.execution.joins -import org.apache.gluten.config.GlutenConfig -import org.apache.gluten.utils.{BackendTestUtils, SystemParameters} +import org.apache.gluten.utils.BackendTestUtils import org.apache.spark.sql.{GlutenTestsCommonTrait, SparkSession} import org.apache.spark.sql.catalyst.optimizer.{ConstantFolding, ConvertToLocalRelation, NullPropagation} @@ -64,7 +63,6 @@ class GlutenBroadcastJoinSuite extends BroadcastJoinSuite with GlutenTestsCommon .config("spark.gluten.sql.columnar.backend.ch.worker.id", "1") .config("spark.gluten.sql.enable.native.validation", "false") .config("spark.sql.files.openCostInBytes", "134217728") - .config(GlutenConfig.GLUTEN_LIB_PATH.key, SystemParameters.getClickHouseLibPath) .config("spark.unsafe.exceptionOnMemoryLeak", "true") .getOrCreate() } else { diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSQLQueryCHSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSQLQueryCHSuite.scala index 0b14467aba80..859e6df48459 100644 --- a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSQLQueryCHSuite.scala +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSQLQueryCHSuite.scala @@ -16,9 +16,6 @@ */ package org.apache.spark.sql.hive.execution -import org.apache.gluten.config.GlutenConfig -import org.apache.gluten.utils.SystemParameters - import org.apache.spark.{DebugFilesystem, SparkConf} import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.TableIdentifier @@ -28,7 +25,6 @@ class GlutenHiveSQLQueryCHSuite extends GlutenHiveSQLQuerySuiteBase { override def sparkConf: SparkConf = { defaultSparkConf .set("spark.plugins", "org.apache.gluten.GlutenPlugin") - .set(GlutenConfig.GLUTEN_LIB_PATH.key, SystemParameters.getClickHouseLibPath) .set("spark.gluten.sql.enable.native.validation", "false") .set("spark.gluten.sql.native.writer.enabled", "true") .set("spark.sql.storeAssignmentPolicy", "legacy") diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/statistics/SparkFunctionStatistics.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/statistics/SparkFunctionStatistics.scala index f483d714f12b..f45b5c659c5f 100644 --- a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/statistics/SparkFunctionStatistics.scala +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/statistics/SparkFunctionStatistics.scala @@ -16,9 +16,8 @@ */ package org.apache.spark.sql.statistics -import org.apache.gluten.config.GlutenConfig import org.apache.gluten.execution.GlutenPlan -import org.apache.gluten.utils.{BackendTestUtils, SystemParameters} +import org.apache.gluten.utils.BackendTestUtils import org.apache.spark.sql.{GlutenTestConstants, QueryTest, SparkSession} import org.apache.spark.sql.catalyst.optimizer.{ConstantFolding, ConvertToLocalRelation, NullPropagation} @@ -63,7 +62,6 @@ class SparkFunctionStatistics extends QueryTest { .config("spark.gluten.sql.columnar.backend.ch.worker.id", "1") .config("spark.gluten.sql.enable.native.validation", "false") .config("spark.sql.files.openCostInBytes", "134217728") - .config(GlutenConfig.GLUTEN_LIB_PATH.key, SystemParameters.getClickHouseLibPath) .config("spark.unsafe.exceptionOnMemoryLeak", "true") .getOrCreate() } else { diff --git a/gluten-ut/spark35/pom.xml b/gluten-ut/spark35/pom.xml index d95e4aeeee05..1ee63b95dc5b 100644 --- a/gluten-ut/spark35/pom.xml +++ b/gluten-ut/spark35/pom.xml @@ -148,7 +148,6 @@ 2.0.6 2.19.0 - @@ -215,12 +214,6 @@ test - - - ${clickhouse.lib.path} - ${tpcds.data.path} - - diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/GlutenSQLQueryTestSuite.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/GlutenSQLQueryTestSuite.scala index 3eebda83a0d1..01d36635f357 100644 --- a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/GlutenSQLQueryTestSuite.scala +++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/GlutenSQLQueryTestSuite.scala @@ -16,9 +16,8 @@ */ package org.apache.spark.sql -import org.apache.gluten.config.GlutenConfig import org.apache.gluten.exception.GlutenException -import org.apache.gluten.utils.{BackendTestSettings, BackendTestUtils, SystemParameters} +import org.apache.gluten.utils.{BackendTestSettings, BackendTestUtils} import org.apache.spark.{SparkConf, SparkException, SparkThrowable} import org.apache.spark.ErrorMessageFormat.MINIMAL @@ -197,7 +196,6 @@ class GlutenSQLQueryTestSuite .set("spark.io.compression.codec", "LZ4") .set("spark.gluten.sql.columnar.backend.ch.worker.id", "1") .set("spark.gluten.sql.enable.native.validation", "false") - .set(GlutenConfig.GLUTEN_LIB_PATH.key, SystemParameters.getClickHouseLibPath) .set("spark.sql.files.openCostInBytes", "134217728") .set("spark.unsafe.exceptionOnMemoryLeak", "true") } else { diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/benchmarks/ParquetReadBenchmark.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/benchmarks/ParquetReadBenchmark.scala index aa5ee0a881ec..517f17bba927 100644 --- a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/benchmarks/ParquetReadBenchmark.scala +++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/benchmarks/ParquetReadBenchmark.scala @@ -20,7 +20,7 @@ import org.apache.gluten.config.GlutenConfig import org.apache.gluten.execution.{FileSourceScanExecTransformer, WholeStageTransformer} import org.apache.gluten.extension.columnar.transition.Transitions import org.apache.gluten.jni.JniLibLoader -import org.apache.gluten.utils.{BackendTestUtils, SystemParameters} +import org.apache.gluten.utils.BackendTestUtils import org.apache.spark.SparkConf import org.apache.spark.benchmark.Benchmark @@ -82,7 +82,6 @@ object ParquetReadBenchmark extends SqlBasedBenchmark { .set("spark.gluten.sql.enable.native.validation", "false") .set("spark.gluten.sql.columnar.backend.ch.worker.id", "1") .set("spark.gluten.sql.columnar.separate.scan.rdd.for.ch", "false") - .setIfMissing(GlutenConfig.GLUTEN_LIB_PATH.key, SystemParameters.getClickHouseLibPath) .set( "spark.sql.catalog.spark_catalog", "org.apache.spark.sql.execution.datasources.v2.clickhouse.ClickHouseSparkCatalog") @@ -225,9 +224,7 @@ object ParquetReadBenchmark extends SqlBasedBenchmark { override def afterAll(): Unit = { if (BackendTestUtils.isCHBackendLoaded()) { - val libPath = - spark.conf.get(GlutenConfig.GLUTEN_LIB_PATH.key, SystemParameters.getClickHouseLibPath) - JniLibLoader.unloadFromPath(libPath) + JniLibLoader.unloadFromPath(spark.conf.get(GlutenConfig.GLUTEN_LIB_PATH.key)) } super.afterAll() } diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/joins/GlutenBroadcastJoinSuite.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/joins/GlutenBroadcastJoinSuite.scala index f5fd73079619..c8b371f48fb8 100644 --- a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/joins/GlutenBroadcastJoinSuite.scala +++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/joins/GlutenBroadcastJoinSuite.scala @@ -16,8 +16,7 @@ */ package org.apache.spark.sql.execution.joins -import org.apache.gluten.config.GlutenConfig -import org.apache.gluten.utils.{BackendTestUtils, SystemParameters} +import org.apache.gluten.utils.BackendTestUtils import org.apache.spark.sql.{GlutenTestsCommonTrait, SparkSession} import org.apache.spark.sql.catalyst.optimizer.{ConstantFolding, ConvertToLocalRelation, NullPropagation} @@ -66,7 +65,6 @@ class GlutenBroadcastJoinSuite extends BroadcastJoinSuite with GlutenTestsCommon .config("spark.gluten.sql.columnar.backend.ch.worker.id", "1") .config("spark.gluten.sql.enable.native.validation", "false") .config("spark.sql.files.openCostInBytes", "134217728") - .config(GlutenConfig.GLUTEN_LIB_PATH.key, SystemParameters.getClickHouseLibPath) .config("spark.unsafe.exceptionOnMemoryLeak", "true") .getOrCreate() } else { diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSQLQueryCHSuite.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSQLQueryCHSuite.scala index 0b14467aba80..859e6df48459 100644 --- a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSQLQueryCHSuite.scala +++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSQLQueryCHSuite.scala @@ -16,9 +16,6 @@ */ package org.apache.spark.sql.hive.execution -import org.apache.gluten.config.GlutenConfig -import org.apache.gluten.utils.SystemParameters - import org.apache.spark.{DebugFilesystem, SparkConf} import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.TableIdentifier @@ -28,7 +25,6 @@ class GlutenHiveSQLQueryCHSuite extends GlutenHiveSQLQuerySuiteBase { override def sparkConf: SparkConf = { defaultSparkConf .set("spark.plugins", "org.apache.gluten.GlutenPlugin") - .set(GlutenConfig.GLUTEN_LIB_PATH.key, SystemParameters.getClickHouseLibPath) .set("spark.gluten.sql.enable.native.validation", "false") .set("spark.gluten.sql.native.writer.enabled", "true") .set("spark.sql.storeAssignmentPolicy", "legacy") diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/statistics/SparkFunctionStatistics.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/statistics/SparkFunctionStatistics.scala index f483d714f12b..f45b5c659c5f 100644 --- a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/statistics/SparkFunctionStatistics.scala +++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/statistics/SparkFunctionStatistics.scala @@ -16,9 +16,8 @@ */ package org.apache.spark.sql.statistics -import org.apache.gluten.config.GlutenConfig import org.apache.gluten.execution.GlutenPlan -import org.apache.gluten.utils.{BackendTestUtils, SystemParameters} +import org.apache.gluten.utils.BackendTestUtils import org.apache.spark.sql.{GlutenTestConstants, QueryTest, SparkSession} import org.apache.spark.sql.catalyst.optimizer.{ConstantFolding, ConvertToLocalRelation, NullPropagation} @@ -63,7 +62,6 @@ class SparkFunctionStatistics extends QueryTest { .config("spark.gluten.sql.columnar.backend.ch.worker.id", "1") .config("spark.gluten.sql.enable.native.validation", "false") .config("spark.sql.files.openCostInBytes", "134217728") - .config(GlutenConfig.GLUTEN_LIB_PATH.key, SystemParameters.getClickHouseLibPath) .config("spark.unsafe.exceptionOnMemoryLeak", "true") .getOrCreate() } else { diff --git a/gluten-ut/test/pom.xml b/gluten-ut/test/pom.xml index a641a19b2faf..d7d04ed5ed08 100644 --- a/gluten-ut/test/pom.xml +++ b/gluten-ut/test/pom.xml @@ -97,9 +97,6 @@ test - - - @@ -143,12 +140,6 @@ test - - - ${clickhouse.lib.path} - ${tpcds.data.path} - - diff --git a/gluten-ut/test/src/test/scala/org/apache/gluten/execution/MergeTwoPhasesHashBaseAggregateSuite.scala b/gluten-ut/test/src/test/scala/org/apache/gluten/execution/MergeTwoPhasesHashBaseAggregateSuite.scala index 32eb10a32fad..061f5660e375 100644 --- a/gluten-ut/test/src/test/scala/org/apache/gluten/execution/MergeTwoPhasesHashBaseAggregateSuite.scala +++ b/gluten-ut/test/src/test/scala/org/apache/gluten/execution/MergeTwoPhasesHashBaseAggregateSuite.scala @@ -16,8 +16,7 @@ */ package org.apache.gluten.execution -import org.apache.gluten.config.GlutenConfig -import org.apache.gluten.utils.{BackendTestUtils, SystemParameters} +import org.apache.gluten.utils.BackendTestUtils import org.apache.spark.SparkConf import org.apache.spark.sql.DataFrame @@ -57,7 +56,6 @@ abstract class BaseMergeTwoPhasesHashBaseAggregateSuite extends WholeStageTransf if (BackendTestUtils.isCHBackendLoaded()) { conf .set("spark.gluten.sql.enable.native.validation", "false") - .set(GlutenConfig.GLUTEN_LIB_PATH.key, SystemParameters.getClickHouseLibPath) } conf } diff --git a/gluten-ut/test/src/test/scala/org/apache/gluten/expressions/GlutenExpressionMappingSuite.scala b/gluten-ut/test/src/test/scala/org/apache/gluten/expressions/GlutenExpressionMappingSuite.scala index 14a07aea5bf7..b2fbd11c0abc 100644 --- a/gluten-ut/test/src/test/scala/org/apache/gluten/expressions/GlutenExpressionMappingSuite.scala +++ b/gluten-ut/test/src/test/scala/org/apache/gluten/expressions/GlutenExpressionMappingSuite.scala @@ -19,7 +19,7 @@ package org.apache.gluten.expressions import org.apache.gluten.config.GlutenConfig import org.apache.gluten.execution.ProjectExecTransformer import org.apache.gluten.expression.ExpressionMappings -import org.apache.gluten.utils.{BackendTestUtils, SystemParameters} +import org.apache.gluten.utils.BackendTestUtils import org.apache.spark.SparkConf import org.apache.spark.sql.{GlutenQueryTest, Row} @@ -43,7 +43,6 @@ class GlutenExpressionMappingSuite if (BackendTestUtils.isCHBackendLoaded()) { conf .set("spark.gluten.sql.enable.native.validation", "false") - .set(GlutenConfig.GLUTEN_LIB_PATH.key, SystemParameters.getClickHouseLibPath) } conf } diff --git a/gluten-ut/test/src/test/scala/org/apache/gluten/extension/GlutenExtensionRewriteRuleSuite.scala b/gluten-ut/test/src/test/scala/org/apache/gluten/extension/GlutenExtensionRewriteRuleSuite.scala index 0068e83a6df3..a29550815041 100644 --- a/gluten-ut/test/src/test/scala/org/apache/gluten/extension/GlutenExtensionRewriteRuleSuite.scala +++ b/gluten-ut/test/src/test/scala/org/apache/gluten/extension/GlutenExtensionRewriteRuleSuite.scala @@ -16,9 +16,8 @@ */ package org.apache.gluten.extension -import org.apache.gluten.config.GlutenConfig import org.apache.gluten.execution.{ProjectExecTransformer, WholeStageTransformerSuite} -import org.apache.gluten.utils.{BackendTestUtils, SystemParameters} +import org.apache.gluten.utils.BackendTestUtils import org.apache.spark.SparkConf @@ -33,7 +32,6 @@ class GlutenExtensionRewriteRuleSuite extends WholeStageTransformerSuite { if (BackendTestUtils.isCHBackendLoaded()) { conf .set("spark.gluten.sql.enable.native.validation", "false") - .set(GlutenConfig.GLUTEN_LIB_PATH.key, SystemParameters.getClickHouseLibPath) } conf } diff --git a/gluten-ut/test/src/test/scala/org/apache/gluten/sql/SQLQuerySuite.scala b/gluten-ut/test/src/test/scala/org/apache/gluten/sql/SQLQuerySuite.scala index 74d0e44db72e..c584852b9110 100644 --- a/gluten-ut/test/src/test/scala/org/apache/gluten/sql/SQLQuerySuite.scala +++ b/gluten-ut/test/src/test/scala/org/apache/gluten/sql/SQLQuerySuite.scala @@ -19,7 +19,6 @@ package org.apache.gluten.sql import org.apache.gluten.config.GlutenConfig import org.apache.gluten.execution.WholeStageTransformerSuite import org.apache.gluten.utils.BackendTestUtils -import org.apache.gluten.utils.SystemParameters import org.apache.spark.SparkConf import org.apache.spark.sql.Row @@ -41,7 +40,6 @@ class SQLQuerySuite extends WholeStageTransformerSuite { if (BackendTestUtils.isCHBackendLoaded()) { conf .set("spark.gluten.sql.enable.native.validation", "false") - .set(GlutenConfig.GLUTEN_LIB_PATH.key, SystemParameters.getClickHouseLibPath) } conf } diff --git a/gluten-ut/test/src/test/scala/org/apache/spark/sql/GlutenExpressionDataTypesValidation.scala b/gluten-ut/test/src/test/scala/org/apache/spark/sql/GlutenExpressionDataTypesValidation.scala index adf72a3f6331..9bea13703793 100644 --- a/gluten-ut/test/src/test/scala/org/apache/spark/sql/GlutenExpressionDataTypesValidation.scala +++ b/gluten-ut/test/src/test/scala/org/apache/spark/sql/GlutenExpressionDataTypesValidation.scala @@ -16,9 +16,8 @@ */ package org.apache.spark.sql -import org.apache.gluten.config.GlutenConfig import org.apache.gluten.execution.{ProjectExecTransformer, TransformSupport, WholeStageTransformerSuite} -import org.apache.gluten.utils.{BackendTestUtils, SystemParameters} +import org.apache.gluten.utils.BackendTestUtils import org.apache.spark.SparkConf import org.apache.spark.rdd.RDD @@ -46,7 +45,6 @@ class GlutenExpressionDataTypesValidation extends WholeStageTransformerSuite { if (BackendTestUtils.isCHBackendLoaded()) { conf .set("spark.gluten.sql.enable.native.validation", "false") - .set(GlutenConfig.GLUTEN_LIB_PATH.key, SystemParameters.getClickHouseLibPath) } conf } diff --git a/gluten-ut/test/src/test/scala/org/apache/spark/sql/datasources/GlutenNoopWriterRuleSuite.scala b/gluten-ut/test/src/test/scala/org/apache/spark/sql/datasources/GlutenNoopWriterRuleSuite.scala index b01b543006e8..b2a3a2ba8ae6 100644 --- a/gluten-ut/test/src/test/scala/org/apache/spark/sql/datasources/GlutenNoopWriterRuleSuite.scala +++ b/gluten-ut/test/src/test/scala/org/apache/spark/sql/datasources/GlutenNoopWriterRuleSuite.scala @@ -16,9 +16,6 @@ */ package org.apache.spark.sql.datasources -import org.apache.gluten.config.GlutenConfig -import org.apache.gluten.utils.{BackendTestUtils, SystemParameters} - import org.apache.spark.SparkConf import org.apache.spark.sql.{GlutenQueryTest, SaveMode} import org.apache.spark.sql.execution.QueryExecution @@ -29,17 +26,13 @@ import org.apache.spark.sql.util.QueryExecutionListener class GlutenNoopWriterRuleSuite extends GlutenQueryTest with SharedSparkSession { override def sparkConf: SparkConf = { - val conf = super.sparkConf + super.sparkConf .set("spark.plugins", "org.apache.gluten.GlutenPlugin") .set("spark.default.parallelism", "1") .set("spark.memory.offHeap.enabled", "true") .set("spark.memory.offHeap.size", "1024MB") .set("spark.ui.enabled", "false") .set("spark.gluten.ui.enabled", "false") - if (BackendTestUtils.isCHBackendLoaded()) { - conf.set(GlutenConfig.GLUTEN_LIB_PATH.key, SystemParameters.getClickHouseLibPath) - } - conf } class WriterColumnarListener extends QueryExecutionListener { diff --git a/pom.xml b/pom.xml index 41dfc803814c..3dd3e7089706 100644 --- a/pom.xml +++ b/pom.xml @@ -82,8 +82,6 @@ all - /usr/local/clickhouse/lib/libch.so - /data/tpcds-data-sf1 2.13.5 4.13.1 From f7a8d6653942947f3b05b06fdd06c4be5abcecdb Mon Sep 17 00:00:00 2001 From: Kyligence Git Date: Wed, 5 Feb 2025 09:21:16 -0600 Subject: [PATCH 09/22] [GLUTEN-1632][CH]Daily Update Clickhouse Version (20250205) (#8660) * [GLUTEN-1632][CH]Daily Update Clickhouse Version (20250205) * Fix build due to https://github.com/ClickHouse/ClickHouse/pull/75333 --------- Co-authored-by: kyligence-git Co-authored-by: Chang Chen --- cpp-ch/clickhouse.version | 4 ++-- .../Disks/ObjectStorages/MetadataStorageFromRocksDB.h | 3 ++- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/cpp-ch/clickhouse.version b/cpp-ch/clickhouse.version index ad7aa8b85332..45a2eb06c300 100644 --- a/cpp-ch/clickhouse.version +++ b/cpp-ch/clickhouse.version @@ -1,3 +1,3 @@ CH_ORG=Kyligence -CH_BRANCH=rebase_ch/20250204 -CH_COMMIT=69b2d923f37 +CH_BRANCH=rebase_ch/20250205 +CH_COMMIT=c1451130e6d diff --git a/cpp-ch/local-engine/Disks/ObjectStorages/MetadataStorageFromRocksDB.h b/cpp-ch/local-engine/Disks/ObjectStorages/MetadataStorageFromRocksDB.h index 8d5273da0e23..744acf4a8f9b 100644 --- a/cpp-ch/local-engine/Disks/ObjectStorages/MetadataStorageFromRocksDB.h +++ b/cpp-ch/local-engine/Disks/ObjectStorages/MetadataStorageFromRocksDB.h @@ -17,12 +17,13 @@ #pragma once #include #if USE_ROCKSDB +#include +#include #include #include #include #include #include -#include namespace local_engine { From 3156a8252322e660f3509a3e67473f752359b96d Mon Sep 17 00:00:00 2001 From: Yuan Date: Thu, 6 Feb 2025 09:30:26 +0800 Subject: [PATCH 10/22] [GLUTEN-8574][VL]CI: adding Spark-344 unit tests on JDK8 and adding Spark-352 unit tests on JDK17 (#8591) adding Spark-344 unit tests on JDK8 and adding Spark-352 unit tests on JDK17 also removed the spark-344 source download step as we already installed in the docker image --- .github/workflows/velox_backend.yml | 132 ++++++++++++++++++++++++++-- 1 file changed, 123 insertions(+), 9 deletions(-) diff --git a/.github/workflows/velox_backend.yml b/.github/workflows/velox_backend.yml index 8735d74e0bad..cea185964597 100644 --- a/.github/workflows/velox_backend.yml +++ b/.github/workflows/velox_backend.yml @@ -759,8 +759,6 @@ jobs: path: /root/.m2/repository/org/apache/arrow/ - name: Prepare spark.test.home for Spark 3.4.4 (other tests) run: | - rm -rf /opt/shims/spark34 - bash .github/workflows/util/install_spark_resources.sh 3.4 dnf module -y install python39 && \ alternatives --set python3 /usr/bin/python3.9 && \ pip3 install setuptools && \ @@ -780,7 +778,7 @@ jobs: if: always() uses: actions/upload-artifact@v4 with: - name: test-report-spark34 + name: test-report-spark34-jdk17 path: '**/surefire-reports/TEST-*.xml' - name: Upload golden files if: failure() @@ -789,6 +787,51 @@ jobs: name: golden-files-spark34 path: /tmp/tpch-approved-plan/** + run-spark-test-spark34-jdk8: + needs: build-native-lib-centos-7 + runs-on: ubuntu-20.04 + container: apache/gluten:centos-8 + steps: + - uses: actions/checkout@v2 + - name: Download All Artifacts + uses: actions/download-artifact@v3 + with: + name: velox-native-lib-centos-7-${{github.sha}} + path: ./cpp/build/releases + - name: Download Arrow Jars + uses: actions/download-artifact@v3 + with: + name: arrow-jars-centos-7-${{github.sha}} + path: /root/.m2/repository/org/apache/arrow/ + - name: Prepare spark.test.home for Spark 3.4.4 (other tests) + run: | + dnf module -y install python39 && \ + alternatives --set python3 /usr/bin/python3.9 && \ + pip3 install setuptools && \ + pip3 install pyspark==3.4.4 cython && \ + pip3 install pandas pyarrow + - name: Build and Run unit test for Spark 3.4.4 (other tests) + run: | + cd $GITHUB_WORKSPACE/ + export SPARK_SCALA_VERSION=2.12 + export JAVA_HOME=/usr/lib/jvm/java-1.8.0-openjdk + export SPARK_HOME=/opt/shims/spark34/spark_home/ + ls -l /opt/shims/spark34/spark_home/ + $MVN_CMD clean test -Pspark-3.4 -Pjava-8 -Pbackends-velox -Pceleborn -Pdelta -Phudi -Pspark-ut \ + -DtagsToExclude=org.apache.spark.tags.ExtendedSQLTest,org.apache.gluten.tags.UDFTest,org.apache.gluten.tags.SkipTestTags \ + -DargLine="-Dspark.test.home=/opt/shims/spark34/spark_home/" + - name: Upload test report + if: always() + uses: actions/upload-artifact@v4 + with: + name: test-report-spark34 + path: '**/surefire-reports/TEST-*.xml' + - name: Upload golden files + if: failure() + uses: actions/upload-artifact@v4 + with: + name: golden-files-spark34-jdk8 + path: /tmp/tpch-approved-plan/** run-spark-test-spark34-slow: needs: build-native-lib-centos-7 @@ -806,14 +849,9 @@ jobs: with: name: arrow-jars-centos-7-${{github.sha}} path: /root/.m2/repository/org/apache/arrow/ - - name: Prepare - run: | - rm -rf /opt/shims/spark34 - bash .github/workflows/util/install_spark_resources.sh 3.4 - name: Build and Run unit test for Spark 3.4.4 (slow tests) run: | cd $GITHUB_WORKSPACE/ - export JAVA_HOME=/usr/lib/jvm/java-17-openjdk export SPARK_HOME=/opt/shims/spark34/spark_home/ ls -l /opt/shims/spark34/spark_home/ $MVN_CMD clean test -Pspark-3.4 -Pjava-17 -Pbackends-velox -Pceleborn -Piceberg -Pdelta -Pspark-ut -Phudi \ @@ -823,7 +861,39 @@ jobs: if: always() uses: actions/upload-artifact@v4 with: - name: test-report-spark34-slow + name: test-report-spark34-slow-jdk17 + path: '**/surefire-reports/TEST-*.xml' + + run-spark-test-spark34-slow-jdk8: + needs: build-native-lib-centos-7 + runs-on: ubuntu-20.04 + container: apache/gluten:centos-8 + steps: + - uses: actions/checkout@v2 + - name: Download All Artifacts + uses: actions/download-artifact@v3 + with: + name: velox-native-lib-centos-7-${{github.sha}} + path: ./cpp/build/releases + - name: Download Arrow Jars + uses: actions/download-artifact@v3 + with: + name: arrow-jars-centos-7-${{github.sha}} + path: /root/.m2/repository/org/apache/arrow/ + - name: Build and Run unit test for Spark 3.4.4 (slow tests) + run: | + cd $GITHUB_WORKSPACE/ + export JAVA_HOME=/usr/lib/jvm/java-1.8.0-openjdk + export SPARK_HOME=/opt/shims/spark34/spark_home/ + ls -l /opt/shims/spark34/spark_home/ + $MVN_CMD clean test -Pspark-3.4 -Pjava-8 -Pbackends-velox -Pceleborn -Pdelta -Pspark-ut -Phudi \ + -DtagsToInclude=org.apache.spark.tags.ExtendedSQLTest \ + -DargLine="-Dspark.test.home=/opt/shims/spark34/spark_home/" + - name: Upload test report + if: always() + uses: actions/upload-artifact@v4 + with: + name: test-report-spark34-slow-jdk8 path: '**/surefire-reports/TEST-*.xml' run-spark-test-spark35: @@ -869,6 +939,50 @@ jobs: name: golden-files-spark35 path: /tmp/tpch-approved-plan/** + run-spark-test-spark35-jdk17: + needs: build-native-lib-centos-7 + runs-on: ubuntu-20.04 + container: apache/gluten:centos-8-jdk17 + steps: + - uses: actions/checkout@v2 + - name: Download All Artifacts + uses: actions/download-artifact@v3 + with: + name: velox-native-lib-centos-7-${{github.sha}} + path: ./cpp/build/releases + - name: Download Arrow Jars + uses: actions/download-artifact@v3 + with: + name: arrow-jars-centos-7-${{github.sha}} + path: /root/.m2/repository/org/apache/arrow/ + - name: Prepare + run: | + dnf module -y install python39 && \ + alternatives --set python3 /usr/bin/python3.9 && \ + pip3 install setuptools && \ + pip3 install pyspark==3.5.2 cython && \ + pip3 install pandas pyarrow + - name: Build and Run unit test for Spark 3.5.2 (other tests) + run: | + cd $GITHUB_WORKSPACE/ + export SPARK_SCALA_VERSION=2.12 + export JAVA_HOME=/usr/lib/jvm/java-17-openjdk + $MVN_CMD clean test -Pspark-3.5 -Pjava-17 -Pbackends-velox -Pceleborn -Piceberg -Pdelta -Phudi -Pspark-ut \ + -DargLine="-Dspark.test.home=/opt/shims/spark35/spark_home/ ${EXTRA_FLAGS}" \ + -DtagsToExclude=org.apache.spark.tags.ExtendedSQLTest,org.apache.gluten.tags.UDFTest,org.apache.gluten.tags.SkipTestTags + - name: Upload test report + if: always() + uses: actions/upload-artifact@v4 + with: + name: test-report-spark35-jdk17 + path: '**/surefire-reports/TEST-*.xml' + - name: Upload golden files + if: failure() + uses: actions/upload-artifact@v4 + with: + name: golden-files-spark35 + path: /tmp/tpch-approved-plan/** + run-spark-test-spark35-scala213: needs: build-native-lib-centos-7 runs-on: ubuntu-20.04 From 9aa21250fc85d65fef39de341a3f5500fcddf32d Mon Sep 17 00:00:00 2001 From: Yuan Date: Thu, 6 Feb 2025 15:19:59 +0800 Subject: [PATCH 11/22] [VL] Bump GHA upload/restore action (#8672) Signed-off-by: Yuan Zhou --- .github/workflows/velox_backend.yml | 20 ++++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/.github/workflows/velox_backend.yml b/.github/workflows/velox_backend.yml index cea185964597..71a79d98eda5 100644 --- a/.github/workflows/velox_backend.yml +++ b/.github/workflows/velox_backend.yml @@ -75,7 +75,7 @@ jobs: steps: - uses: actions/checkout@v4 - name: Get Ccache - uses: actions/cache/restore@v3 + uses: actions/cache/restore@v4 with: path: '${{ env.CCACHE_DIR }}' key: ccache-centos7-release-default-${{github.sha}} @@ -96,7 +96,7 @@ jobs: " - name: "Save ccache" - uses: actions/cache/save@v3 + uses: actions/cache/save@v4 id: ccache with: path: '${{ env.CCACHE_DIR }}' @@ -794,12 +794,12 @@ jobs: steps: - uses: actions/checkout@v2 - name: Download All Artifacts - uses: actions/download-artifact@v3 + uses: actions/download-artifact@v4 with: name: velox-native-lib-centos-7-${{github.sha}} path: ./cpp/build/releases - name: Download Arrow Jars - uses: actions/download-artifact@v3 + uses: actions/download-artifact@v4 with: name: arrow-jars-centos-7-${{github.sha}} path: /root/.m2/repository/org/apache/arrow/ @@ -871,12 +871,12 @@ jobs: steps: - uses: actions/checkout@v2 - name: Download All Artifacts - uses: actions/download-artifact@v3 + uses: actions/download-artifact@v4 with: name: velox-native-lib-centos-7-${{github.sha}} path: ./cpp/build/releases - name: Download Arrow Jars - uses: actions/download-artifact@v3 + uses: actions/download-artifact@v4 with: name: arrow-jars-centos-7-${{github.sha}} path: /root/.m2/repository/org/apache/arrow/ @@ -946,12 +946,12 @@ jobs: steps: - uses: actions/checkout@v2 - name: Download All Artifacts - uses: actions/download-artifact@v3 + uses: actions/download-artifact@v4 with: name: velox-native-lib-centos-7-${{github.sha}} path: ./cpp/build/releases - name: Download Arrow Jars - uses: actions/download-artifact@v3 + uses: actions/download-artifact@v4 with: name: arrow-jars-centos-7-${{github.sha}} path: /root/.m2/repository/org/apache/arrow/ @@ -1183,7 +1183,7 @@ jobs: steps: - uses: actions/checkout@v2 - name: Get Ccache - uses: actions/cache/restore@v3 + uses: actions/cache/restore@v4 with: path: '${{ env.CCACHE_DIR }}' key: ccache-centos8-release-default-${{github.sha}} @@ -1195,7 +1195,7 @@ jobs: bash dev/ci-velox-buildshared-centos-8.sh ccache -s # - name: "Save ccache" - # uses: actions/cache/save@v3 + # uses: actions/cache/save@v4 # id: ccache # with: # path: '${{ env.CCACHE_DIR }}' From 852340219f87fb99cba3b0b9801967c6fb80d4c0 Mon Sep 17 00:00:00 2001 From: Hongze Zhang Date: Fri, 7 Feb 2025 09:14:08 +0800 Subject: [PATCH 12/22] [VL] nit: Remove shadowed variables in SubstraitToVeloxPlan.cc (#8677) gluten::kHiveConnectorId was defined as const so no need to define kHiveConnectorId locally in the function bodies. --- cpp/velox/substrait/SubstraitToVeloxPlan.cc | 5 ----- 1 file changed, 5 deletions(-) diff --git a/cpp/velox/substrait/SubstraitToVeloxPlan.cc b/cpp/velox/substrait/SubstraitToVeloxPlan.cc index b7fdc59dc4b6..3661b7608487 100644 --- a/cpp/velox/substrait/SubstraitToVeloxPlan.cc +++ b/cpp/velox/substrait/SubstraitToVeloxPlan.cc @@ -667,8 +667,6 @@ core::PlanNodePtr SubstraitToVeloxPlanConverter::toVeloxPlan(const ::substrait:: } } - // Do not hard-code connector ID and allow for connectors other than Hive. - static const std::string kHiveConnectorId = "test-hive"; // Currently only support parquet format. dwio::common::FileFormat fileFormat = dwio::common::FileFormat::PARQUET; @@ -1273,9 +1271,6 @@ core::PlanNodePtr SubstraitToVeloxPlanConverter::toVeloxPlan(const ::substrait:: SubstraitParser::parseColumnTypes(baseSchema, columnTypes); } - // Do not hard-code connector ID and allow for connectors other than Hive. - static const std::string kHiveConnectorId = "test-hive"; - // Velox requires Filter Pushdown must being enabled. bool filterPushdownEnabled = true; std::shared_ptr tableHandle; From 6cb0b566d66ba96093a472176f90b11314b097a4 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Fri, 7 Feb 2025 09:15:10 +0800 Subject: [PATCH 13/22] Bump junit:junit from 4.12 to 4.13.1 in /tools/qualification-tool (#8667) Bumps [junit:junit](https://github.com/junit-team/junit4) from 4.12 to 4.13.1. - [Release notes](https://github.com/junit-team/junit4/releases) - [Changelog](https://github.com/junit-team/junit4/blob/main/doc/ReleaseNotes4.12.md) - [Commits](https://github.com/junit-team/junit4/compare/r4.12...r4.13.1) --- updated-dependencies: - dependency-name: junit:junit dependency-type: direct:development ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- tools/qualification-tool/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tools/qualification-tool/pom.xml b/tools/qualification-tool/pom.xml index 0b49fc723f7b..d59c4a6c2d28 100644 --- a/tools/qualification-tool/pom.xml +++ b/tools/qualification-tool/pom.xml @@ -69,7 +69,7 @@ junit junit - 4.12 + 4.13.1 test From 9d23b461f45eef198ef296fb1b6a5aa5ee65d6e6 Mon Sep 17 00:00:00 2001 From: Kyligence Git Date: Fri, 7 Feb 2025 03:43:43 -0600 Subject: [PATCH 14/22] [GLUTEN-1632][CH]Daily Update Clickhouse Version (20250207) (#8681) * [GLUTEN-1632][CH]Daily Update Clickhouse Version (20250207) * Fix build due to https://github.com/ClickHouse/ClickHouse/pull/75299 * Fix build due to https://github.com/ClickHouse/ClickHouse/pull/75636 --------- Co-authored-by: kyligence-git Co-authored-by: Chang Chen --- cpp-ch/clickhouse.version | 4 ++-- cpp-ch/local-engine/Common/ChunkBuffer.cpp | 2 ++ .../Functions/FunctionsBloomFilter.h | 2 +- .../local-engine/Functions/SparkArrayFlatten.cpp | 9 +++++---- .../Functions/SparkFunctionArraySort.cpp | 10 +++++----- .../SparkFunctionCheckDecimalOverflow.cpp | 3 ++- .../local-engine/Functions/SparkFunctionFloor.h | 1 + .../Functions/SparkFunctionMakeDecimal.cpp | 2 ++ .../Functions/SparkFunctionMapToString.h | 3 ++- .../Functions/SparkFunctionToDateTime.h | 1 + .../Functions/SparkFunctionTupleElement.cpp | 3 ++- .../Functions/SparkFunctionUnscaleValue.cpp | 1 + .../local-engine/Operator/AdvancedExpandStep.cpp | 8 +++----- .../Operator/PartitionColumnFillingTransform.cpp | 3 ++- .../local-engine/Operator/ReplicateRowsStep.cpp | 3 +-- .../Operator/WindowGroupLimitStep.cpp | 1 + .../Parser/AggregateFunctionParser.cpp | 1 + .../Parser/RelParsers/ExpandRelParser.cpp | 16 +++++----------- .../Parser/RelParsers/ReadRelParser.cpp | 4 +++- cpp-ch/local-engine/Shuffle/PartitionWriter.cpp | 1 + cpp-ch/local-engine/Shuffle/SelectorBuilder.h | 4 +--- .../Storages/Output/NormalFileWriter.cpp | 2 ++ .../Storages/Output/NormalFileWriter.h | 1 + .../Serializations/ExcelDecimalSerialization.h | 8 ++++++++ .../Storages/Serializations/ExcelSerialization.h | 1 + .../Storages/SubstraitSource/FormatFile.cpp | 1 + cpp-ch/local-engine/tests/gtest_transformer.cpp | 1 + 27 files changed, 58 insertions(+), 38 deletions(-) diff --git a/cpp-ch/clickhouse.version b/cpp-ch/clickhouse.version index 45a2eb06c300..72d0685e152a 100644 --- a/cpp-ch/clickhouse.version +++ b/cpp-ch/clickhouse.version @@ -1,3 +1,3 @@ CH_ORG=Kyligence -CH_BRANCH=rebase_ch/20250205 -CH_COMMIT=c1451130e6d +CH_BRANCH=rebase_ch/20250207 +CH_COMMIT=b6c11f40aed diff --git a/cpp-ch/local-engine/Common/ChunkBuffer.cpp b/cpp-ch/local-engine/Common/ChunkBuffer.cpp index 9477a61a8647..629b83c059a3 100644 --- a/cpp-ch/local-engine/Common/ChunkBuffer.cpp +++ b/cpp-ch/local-engine/Common/ChunkBuffer.cpp @@ -16,6 +16,8 @@ */ #include "ChunkBuffer.h" +#include + namespace local_engine { void ChunkBuffer::add(DB::Chunk & columns, int start, int end) diff --git a/cpp-ch/local-engine/Functions/FunctionsBloomFilter.h b/cpp-ch/local-engine/Functions/FunctionsBloomFilter.h index d677a3311c8d..2546030db505 100644 --- a/cpp-ch/local-engine/Functions/FunctionsBloomFilter.h +++ b/cpp-ch/local-engine/Functions/FunctionsBloomFilter.h @@ -18,7 +18,6 @@ #include #include -#include #include #include #include @@ -31,6 +30,7 @@ #include #include #include +#include #include #include #include diff --git a/cpp-ch/local-engine/Functions/SparkArrayFlatten.cpp b/cpp-ch/local-engine/Functions/SparkArrayFlatten.cpp index d39bca5ea104..96faa9d1dc1d 100644 --- a/cpp-ch/local-engine/Functions/SparkArrayFlatten.cpp +++ b/cpp-ch/local-engine/Functions/SparkArrayFlatten.cpp @@ -14,12 +14,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -#include -#include -#include -#include #include #include +#include +#include +#include +#include +#include namespace DB diff --git a/cpp-ch/local-engine/Functions/SparkFunctionArraySort.cpp b/cpp-ch/local-engine/Functions/SparkFunctionArraySort.cpp index cf9d67f1696b..c53d9dbd319b 100644 --- a/cpp-ch/local-engine/Functions/SparkFunctionArraySort.cpp +++ b/cpp-ch/local-engine/Functions/SparkFunctionArraySort.cpp @@ -14,18 +14,18 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -#include -#include #include #include #include -#include #include #include #include -#include -#include +#include +#include +#include #include +#include +#include namespace DB::ErrorCodes { diff --git a/cpp-ch/local-engine/Functions/SparkFunctionCheckDecimalOverflow.cpp b/cpp-ch/local-engine/Functions/SparkFunctionCheckDecimalOverflow.cpp index 73fc7b5d4e40..95853b187a42 100644 --- a/cpp-ch/local-engine/Functions/SparkFunctionCheckDecimalOverflow.cpp +++ b/cpp-ch/local-engine/Functions/SparkFunctionCheckDecimalOverflow.cpp @@ -16,16 +16,17 @@ */ #include "SparkFunctionCheckDecimalOverflow.h" +#include #include #include #include #include +#include #include #include #include #include #include -#include namespace DB { diff --git a/cpp-ch/local-engine/Functions/SparkFunctionFloor.h b/cpp-ch/local-engine/Functions/SparkFunctionFloor.h index a2db1083e1df..a56c33baddcf 100644 --- a/cpp-ch/local-engine/Functions/SparkFunctionFloor.h +++ b/cpp-ch/local-engine/Functions/SparkFunctionFloor.h @@ -18,6 +18,7 @@ #include #include +#include #include #include #include diff --git a/cpp-ch/local-engine/Functions/SparkFunctionMakeDecimal.cpp b/cpp-ch/local-engine/Functions/SparkFunctionMakeDecimal.cpp index f136f587c539..7ffb390bc899 100644 --- a/cpp-ch/local-engine/Functions/SparkFunctionMakeDecimal.cpp +++ b/cpp-ch/local-engine/Functions/SparkFunctionMakeDecimal.cpp @@ -16,6 +16,8 @@ */ #include #include +#include +#include #include #include #include diff --git a/cpp-ch/local-engine/Functions/SparkFunctionMapToString.h b/cpp-ch/local-engine/Functions/SparkFunctionMapToString.h index 739f9536d6f8..6c49d3c18da8 100644 --- a/cpp-ch/local-engine/Functions/SparkFunctionMapToString.h +++ b/cpp-ch/local-engine/Functions/SparkFunctionMapToString.h @@ -15,10 +15,11 @@ * limitations under the License. */ #pragma once -#include + #include #include #include +#include #include #include #include diff --git a/cpp-ch/local-engine/Functions/SparkFunctionToDateTime.h b/cpp-ch/local-engine/Functions/SparkFunctionToDateTime.h index 15c23041d944..aae0a359f59e 100644 --- a/cpp-ch/local-engine/Functions/SparkFunctionToDateTime.h +++ b/cpp-ch/local-engine/Functions/SparkFunctionToDateTime.h @@ -19,6 +19,7 @@ #include #include #include +#include #include #include #include diff --git a/cpp-ch/local-engine/Functions/SparkFunctionTupleElement.cpp b/cpp-ch/local-engine/Functions/SparkFunctionTupleElement.cpp index 53c48b97c7bc..d52abde2ccfd 100644 --- a/cpp-ch/local-engine/Functions/SparkFunctionTupleElement.cpp +++ b/cpp-ch/local-engine/Functions/SparkFunctionTupleElement.cpp @@ -16,17 +16,18 @@ */ #include #include +#include #include #include #include #include +#include #include #include #include #include #include #include -#include "Columns/ColumnNullable.h" namespace DB diff --git a/cpp-ch/local-engine/Functions/SparkFunctionUnscaleValue.cpp b/cpp-ch/local-engine/Functions/SparkFunctionUnscaleValue.cpp index 33ec2e308a3e..c964482c897e 100644 --- a/cpp-ch/local-engine/Functions/SparkFunctionUnscaleValue.cpp +++ b/cpp-ch/local-engine/Functions/SparkFunctionUnscaleValue.cpp @@ -15,6 +15,7 @@ * limitations under the License. */ #include +#include #include #include #include diff --git a/cpp-ch/local-engine/Operator/AdvancedExpandStep.cpp b/cpp-ch/local-engine/Operator/AdvancedExpandStep.cpp index 72721ce85921..458e8d5a7c23 100644 --- a/cpp-ch/local-engine/Operator/AdvancedExpandStep.cpp +++ b/cpp-ch/local-engine/Operator/AdvancedExpandStep.cpp @@ -16,8 +16,9 @@ */ #include "AdvancedExpandStep.h" + +#include #include -#include #include #include #include @@ -27,14 +28,11 @@ #include #include #include -#include #include #include +#include #include #include -#include - -#include #include namespace local_engine diff --git a/cpp-ch/local-engine/Operator/PartitionColumnFillingTransform.cpp b/cpp-ch/local-engine/Operator/PartitionColumnFillingTransform.cpp index ab87e1b39b79..d0e2a4b3b620 100644 --- a/cpp-ch/local-engine/Operator/PartitionColumnFillingTransform.cpp +++ b/cpp-ch/local-engine/Operator/PartitionColumnFillingTransform.cpp @@ -15,8 +15,9 @@ * limitations under the License. */ #include "PartitionColumnFillingTransform.h" -#include + #include +#include #include #include #include diff --git a/cpp-ch/local-engine/Operator/ReplicateRowsStep.cpp b/cpp-ch/local-engine/Operator/ReplicateRowsStep.cpp index 705e4136751f..5281ce4a5f8a 100644 --- a/cpp-ch/local-engine/Operator/ReplicateRowsStep.cpp +++ b/cpp-ch/local-engine/Operator/ReplicateRowsStep.cpp @@ -16,8 +16,7 @@ */ #include "ReplicateRowsStep.h" -#include - +#include #include namespace DB diff --git a/cpp-ch/local-engine/Operator/WindowGroupLimitStep.cpp b/cpp-ch/local-engine/Operator/WindowGroupLimitStep.cpp index d2264e24dc13..a0b623b7cc62 100644 --- a/cpp-ch/local-engine/Operator/WindowGroupLimitStep.cpp +++ b/cpp-ch/local-engine/Operator/WindowGroupLimitStep.cpp @@ -17,6 +17,7 @@ #include "WindowGroupLimitStep.h" +#include #include #include #include diff --git a/cpp-ch/local-engine/Parser/AggregateFunctionParser.cpp b/cpp-ch/local-engine/Parser/AggregateFunctionParser.cpp index e02af53d3709..696f5352f200 100644 --- a/cpp-ch/local-engine/Parser/AggregateFunctionParser.cpp +++ b/cpp-ch/local-engine/Parser/AggregateFunctionParser.cpp @@ -18,6 +18,7 @@ #include #include +#include #include #include #include diff --git a/cpp-ch/local-engine/Parser/RelParsers/ExpandRelParser.cpp b/cpp-ch/local-engine/Parser/RelParsers/ExpandRelParser.cpp index f3d8b4ab11ec..7d31fcf2bca1 100644 --- a/cpp-ch/local-engine/Parser/RelParsers/ExpandRelParser.cpp +++ b/cpp-ch/local-engine/Parser/RelParsers/ExpandRelParser.cpp @@ -15,8 +15,9 @@ * limitations under the License. */ #include "ExpandRelParser.h" -#include + #include +#include #include #include #include @@ -41,9 +42,7 @@ ExpandRelParser::ExpandRelParser(ParserContextPtr parser_context_) : RelParser(p void updateType(DB::DataTypePtr & type, const DB::DataTypePtr & new_type) { if (type == nullptr || (!type->isNullable() && new_type->isNullable())) - { type = new_type; - } } DB::QueryPlanPtr @@ -91,7 +90,8 @@ ExpandField ExpandRelParser::buildExpandField(const DB::Block & header, const su fields.push_back(field); if (field >= header.columns()) { - throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Field index out of range: {}, header: {}", field, header.dumpStructure()); + throw DB::Exception( + DB::ErrorCodes::LOGICAL_ERROR, "Field index out of range: {}, header: {}", field, header.dumpStructure()); } updateType(types[i], header.getByPosition(field).type); const auto & name = header.getByPosition(field).name; @@ -128,10 +128,8 @@ ExpandField ExpandRelParser::buildExpandField(const DB::Block & header, const su } for (int i = 0; i < names.size(); ++i) - { if (names[i].empty()) names[i] = getUniqueName("expand_" + std::to_string(i)); - } ExpandField expand_field(names, types, expand_kinds, expand_fields); return expand_field; @@ -144,10 +142,8 @@ bool ExpandRelParser::isLazyAggregateExpand(const substrait::ExpandRel & expand_ return false; const auto & aggregate_rel = input_rel.aggregate(); for (const auto & measure : aggregate_rel.measures()) - { if (measure.measure().phase() != substrait::AggregationPhase::AGGREGATION_PHASE_INITIAL_TO_INTERMEDIATE) return false; - } return true; } @@ -172,7 +168,7 @@ DB::QueryPlanPtr ExpandRelParser::lazyAggregateExpandParse( auto aggregate_rel = rel.expand().input().aggregate(); auto aggregate_descriptions = buildAggregations(input_header, expand_field, aggregate_rel); - size_t grouping_keys = aggregate_rel.groupings(0).grouping_expressions_size(); + size_t grouping_keys = aggregate_rel.groupings(0).grouping_expressions_size(); auto expand_step = std::make_unique(getContext(), input_header, grouping_keys, aggregate_descriptions, expand_field); @@ -189,10 +185,8 @@ DB::AggregateDescriptions ExpandRelParser::buildAggregations( DB::AggregateDescriptions descriptions; DB::ColumnsWithTypeAndName aggregate_columns; for (const auto & col : header.getColumnsWithTypeAndName()) - { if (typeid_cast(col.column.get())) aggregate_columns.push_back(col); - } for (size_t i = 0; i < aggregate_rel.measures_size(); ++i) { diff --git a/cpp-ch/local-engine/Parser/RelParsers/ReadRelParser.cpp b/cpp-ch/local-engine/Parser/RelParsers/ReadRelParser.cpp index 0a663a88a6cf..8056d26b2eb6 100644 --- a/cpp-ch/local-engine/Parser/RelParsers/ReadRelParser.cpp +++ b/cpp-ch/local-engine/Parser/RelParsers/ReadRelParser.cpp @@ -19,6 +19,7 @@ #include #include #include +#include #include #include #include @@ -47,7 +48,8 @@ extern const int LOGICAL_ERROR; namespace local_engine { using namespace DB; -DB::QueryPlanPtr ReadRelParser::parse(DB::QueryPlanPtr query_plan, const substrait::Rel & rel, std::list & rel_stack) +DB::QueryPlanPtr +ReadRelParser::parse(DB::QueryPlanPtr query_plan, const substrait::Rel & rel, std::list & rel_stack) { if (query_plan) throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Source node's input plan should be null"); diff --git a/cpp-ch/local-engine/Shuffle/PartitionWriter.cpp b/cpp-ch/local-engine/Shuffle/PartitionWriter.cpp index 158f1a9cfa0e..ebda09a4e68b 100644 --- a/cpp-ch/local-engine/Shuffle/PartitionWriter.cpp +++ b/cpp-ch/local-engine/Shuffle/PartitionWriter.cpp @@ -31,6 +31,7 @@ #include #include #include +#include namespace DB { diff --git a/cpp-ch/local-engine/Shuffle/SelectorBuilder.h b/cpp-ch/local-engine/Shuffle/SelectorBuilder.h index 7349849f538e..b972e0a516c4 100644 --- a/cpp-ch/local-engine/Shuffle/SelectorBuilder.h +++ b/cpp-ch/local-engine/Shuffle/SelectorBuilder.h @@ -17,17 +17,15 @@ #pragma once #include #include +#include #include -#include #include #include -#include #include #include #include #include #include -#include #include namespace local_engine diff --git a/cpp-ch/local-engine/Storages/Output/NormalFileWriter.cpp b/cpp-ch/local-engine/Storages/Output/NormalFileWriter.cpp index 1f3d7f0b2598..546a29be711b 100644 --- a/cpp-ch/local-engine/Storages/Output/NormalFileWriter.cpp +++ b/cpp-ch/local-engine/Storages/Output/NormalFileWriter.cpp @@ -19,6 +19,8 @@ #include #include #include +#include +#include #include #include #include diff --git a/cpp-ch/local-engine/Storages/Output/NormalFileWriter.h b/cpp-ch/local-engine/Storages/Output/NormalFileWriter.h index d12ccaae6062..c337a50f75d3 100644 --- a/cpp-ch/local-engine/Storages/Output/NormalFileWriter.h +++ b/cpp-ch/local-engine/Storages/Output/NormalFileWriter.h @@ -16,6 +16,7 @@ */ #pragma once +#include #include #include #include diff --git a/cpp-ch/local-engine/Storages/Serializations/ExcelDecimalSerialization.h b/cpp-ch/local-engine/Storages/Serializations/ExcelDecimalSerialization.h index 351e4458ab57..29bafe0a9059 100644 --- a/cpp-ch/local-engine/Storages/Serializations/ExcelDecimalSerialization.h +++ b/cpp-ch/local-engine/Storages/Serializations/ExcelDecimalSerialization.h @@ -19,6 +19,14 @@ #include #include +namespace DB +{ +namespace ErrorCodes +{ +extern const int NOT_IMPLEMENTED; +} +} + namespace local_engine { diff --git a/cpp-ch/local-engine/Storages/Serializations/ExcelSerialization.h b/cpp-ch/local-engine/Storages/Serializations/ExcelSerialization.h index a7215b3490eb..f55d7df9cbf6 100644 --- a/cpp-ch/local-engine/Storages/Serializations/ExcelSerialization.h +++ b/cpp-ch/local-engine/Storages/Serializations/ExcelSerialization.h @@ -26,6 +26,7 @@ namespace DB namespace ErrorCodes { extern const int ATTEMPT_TO_READ_AFTER_EOF; + extern const int NOT_IMPLEMENTED; } } diff --git a/cpp-ch/local-engine/Storages/SubstraitSource/FormatFile.cpp b/cpp-ch/local-engine/Storages/SubstraitSource/FormatFile.cpp index 0fb5294e6073..cc6366785677 100644 --- a/cpp-ch/local-engine/Storages/SubstraitSource/FormatFile.cpp +++ b/cpp-ch/local-engine/Storages/SubstraitSource/FormatFile.cpp @@ -17,6 +17,7 @@ #include "FormatFile.h" #include #include +#include #include #include #include diff --git a/cpp-ch/local-engine/tests/gtest_transformer.cpp b/cpp-ch/local-engine/tests/gtest_transformer.cpp index a4d62d5681a3..d6ed2d8ef8d4 100644 --- a/cpp-ch/local-engine/tests/gtest_transformer.cpp +++ b/cpp-ch/local-engine/tests/gtest_transformer.cpp @@ -14,6 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ +#include #include #include #include From e0092089453286b6822447a875666b41ab6f0b5c Mon Sep 17 00:00:00 2001 From: Zhen Li Date: Fri, 7 Feb 2025 19:09:53 +0800 Subject: [PATCH 15/22] [VL] Enable make_date function (#8683) --- .../scala/org/apache/gluten/utils/CHExpressionUtil.scala | 3 ++- .../gluten/execution/ScalarFunctionsValidateSuite.scala | 7 +++++++ .../org/apache/gluten/expression/ExpressionMappings.scala | 1 + .../org/apache/gluten/expression/ExpressionNames.scala | 1 + 4 files changed, 11 insertions(+), 1 deletion(-) diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/utils/CHExpressionUtil.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/utils/CHExpressionUtil.scala index 1dd815b6d78d..dc2d8716b2f8 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/utils/CHExpressionUtil.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/utils/CHExpressionUtil.scala @@ -204,6 +204,7 @@ object CHExpressionUtil { FROM_UTC_TIMESTAMP -> UtcTimestampValidator(), STACK -> DefaultValidator(), RAISE_ERROR -> DefaultValidator(), - WIDTH_BUCKET -> DefaultValidator() + WIDTH_BUCKET -> DefaultValidator(), + MAKE_DATE -> DefaultValidator() ) } diff --git a/backends-velox/src/test/scala/org/apache/gluten/execution/ScalarFunctionsValidateSuite.scala b/backends-velox/src/test/scala/org/apache/gluten/execution/ScalarFunctionsValidateSuite.scala index 94c69abd8d32..5a4af560b2c5 100644 --- a/backends-velox/src/test/scala/org/apache/gluten/execution/ScalarFunctionsValidateSuite.scala +++ b/backends-velox/src/test/scala/org/apache/gluten/execution/ScalarFunctionsValidateSuite.scala @@ -1530,4 +1530,11 @@ abstract class ScalarFunctionsValidateSuite extends FunctionsValidateSuite { } } } + + test("make_date") { + runQueryAndCompare( + "select make_date(2025, 2, 7), make_date(2024, 11, null), make_date(2024, 11, 50)") { + checkGlutenOperatorMatch[ProjectExecTransformer] + } + } } diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/expression/ExpressionMappings.scala b/gluten-substrait/src/main/scala/org/apache/gluten/expression/ExpressionMappings.scala index f9b5a2446171..14f912cb4770 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/expression/ExpressionMappings.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/expression/ExpressionMappings.scala @@ -192,6 +192,7 @@ object ExpressionMappings { Sig[MonthsBetween](MONTHS_BETWEEN), Sig[DateFromUnixDate](DATE_FROM_UNIX_DATE), Sig[UnixDate](UNIX_DATE), + Sig[MakeDate](MAKE_DATE), Sig[MakeTimestamp](MAKE_TIMESTAMP), Sig[MakeYMInterval](MAKE_YM_INTERVAL), Sig[ToUTCTimestamp](TO_UTC_TIMESTAMP), diff --git a/shims/common/src/main/scala/org/apache/gluten/expression/ExpressionNames.scala b/shims/common/src/main/scala/org/apache/gluten/expression/ExpressionNames.scala index 713a6ba0dcca..9e974120fa5f 100644 --- a/shims/common/src/main/scala/org/apache/gluten/expression/ExpressionNames.scala +++ b/shims/common/src/main/scala/org/apache/gluten/expression/ExpressionNames.scala @@ -217,6 +217,7 @@ object ExpressionNames { final val MONTHS_BETWEEN = "months_between" final val DATE_FROM_UNIX_DATE = "date_from_unix_date" final val UNIX_DATE = "unix_date" + final val MAKE_DATE = "make_date" final val MAKE_TIMESTAMP = "make_timestamp" final val MAKE_YM_INTERVAL = "make_ym_interval" final val TO_UTC_TIMESTAMP = "to_utc_timestamp" From 91800b43fa1703517a92cf19150b7261891ced8b Mon Sep 17 00:00:00 2001 From: Arnav Balyan <60175178+ArnavBalyan@users.noreply.github.com> Date: Fri, 7 Feb 2025 18:19:29 +0530 Subject: [PATCH 16/22] [GLUTEN-8616] [VL] Make filescan limit for encrypted fallback as configurable (#8621) --- .../gluten/backendsapi/velox/VeloxBackend.scala | 7 ++++++- .../apache/gluten/utils/ParquetMetadataUtils.scala | 5 +++-- .../org/apache/gluten/config/GlutenConfig.scala | 12 ++++++++++++ 3 files changed, 21 insertions(+), 3 deletions(-) diff --git a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxBackend.scala b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxBackend.scala index 61ed9a2de17f..56d6fb2e65c2 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxBackend.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxBackend.scala @@ -200,8 +200,13 @@ object VeloxBackendSettings extends BackendSettingsApi { return None } + val fileLimit = GlutenConfig.get.parquetEncryptionValidationFileLimit val encryptionResult = - ParquetMetadataUtils.validateEncryption(format, rootPaths, serializableHadoopConf) + ParquetMetadataUtils.validateEncryption( + format, + rootPaths, + serializableHadoopConf, + fileLimit) if (encryptionResult.ok()) { None } else { diff --git a/backends-velox/src/main/scala/org/apache/gluten/utils/ParquetMetadataUtils.scala b/backends-velox/src/main/scala/org/apache/gluten/utils/ParquetMetadataUtils.scala index 9f43575cf90c..48d0629268da 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/utils/ParquetMetadataUtils.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/utils/ParquetMetadataUtils.scala @@ -46,7 +46,8 @@ object ParquetMetadataUtils { def validateEncryption( format: ReadFileFormat, rootPaths: Seq[String], - serializableHadoopConf: Option[SerializableConfiguration] + serializableHadoopConf: Option[SerializableConfiguration], + fileLimit: Int ): ValidationResult = { if (format != ParquetReadFormat || rootPaths.isEmpty) { return ValidationResult.succeeded @@ -59,7 +60,7 @@ object ParquetMetadataUtils { val fs = new Path(rootPath).getFileSystem(conf) try { val encryptionDetected = - checkForEncryptionWithLimit(fs, new Path(rootPath), conf, fileLimit = 10) + checkForEncryptionWithLimit(fs, new Path(rootPath), conf, fileLimit = fileLimit) if (encryptionDetected) { return ValidationResult.failed("Encrypted Parquet file detected.") } diff --git a/shims/common/src/main/scala/org/apache/gluten/config/GlutenConfig.scala b/shims/common/src/main/scala/org/apache/gluten/config/GlutenConfig.scala index 1cdc3d552af1..e9e2dbac54a0 100644 --- a/shims/common/src/main/scala/org/apache/gluten/config/GlutenConfig.scala +++ b/shims/common/src/main/scala/org/apache/gluten/config/GlutenConfig.scala @@ -503,6 +503,8 @@ class GlutenConfig(conf: SQLConf) extends Logging { def autoAdjustStageFallenNodeThreshold: Double = getConf(AUTO_ADJUST_STAGE_RESOURCES_FALLEN_NODE_RATIO_THRESHOLD) + + def parquetEncryptionValidationFileLimit: Int = getConf(ENCRYPTED_PARQUET_FALLBACK_FILE_LIMIT) } object GlutenConfig { @@ -2310,4 +2312,14 @@ object GlutenConfig { "count exceeds the total node count ratio.") .doubleConf .createWithDefault(0.5d) + + val ENCRYPTED_PARQUET_FALLBACK_FILE_LIMIT = + buildConf("spark.gluten.sql.fallbackEncryptedParquet.limit") + .internal() + .doc("If supplied, `limit` number of files will be checked to determine encryption " + + "and falling back java scan") + .intConf + .checkValue(_ > 0, s"must be positive.") + .createWithDefault(10) + } From 547cc36c6aa899dcf431b79ea94a81dcafda8c41 Mon Sep 17 00:00:00 2001 From: Gluten Performance Bot <137994563+GlutenPerfBot@users.noreply.github.com> Date: Sat, 8 Feb 2025 02:25:55 +0800 Subject: [PATCH 17/22] [GLUTEN-6887][VL] Daily Update Velox Version (2025_02_06) (#8664) Upstream Velox's New Commits: 1fe767ba2 by Kevin Wilfong, refactor(trace): Only register Parquet if it's enabled (12267) ad4291b14 by Yenda Li, fix: Address incompatibility of ipaddress (12134) 79b563249 by Kevin Wilfong, fix: Fix crash in MemoryArbitrationFuzzer (12266) d3e7af8a5 by Gaurav Mogre, refactor: Changed RowType::getChildIdx to take a std::string_view instead of string ref (12245) a6644b9e2 by Ke, fix: Fix merge storageStats_ in IoStatistics (12263) be35478ff by Jimmy Lu, fix: Avoid redownload coalesced region gap twice in buffered inputs (12018) 93facc8e8 by Orri Erling, refactor(wave): Cleanup and Codegen Benchmarks (12205) 162a27c92 by Pedro Eugenio Rocha Pedreira, feat(cursor): Enable TaskCursor to take a memory pool (12261) ddc20e668 by Pradeep Vaka, feat: Add combine_hash internal operator (12244) f6fe8a3bf by Natasha Sehgal, Prestissimo ApproxMostFrequent JSON (12189) --- ep/build-velox/src/get_velox.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ep/build-velox/src/get_velox.sh b/ep/build-velox/src/get_velox.sh index 4978f4a7453e..1b29c7dc1068 100755 --- a/ep/build-velox/src/get_velox.sh +++ b/ep/build-velox/src/get_velox.sh @@ -17,7 +17,7 @@ set -exu VELOX_REPO=https://github.com/oap-project/velox.git -VELOX_BRANCH=2025_02_05 +VELOX_BRANCH=2025_02_06 VELOX_HOME="" OS=`uname -s` From fcd83db265153b06166551fd2dedca3145101a1a Mon Sep 17 00:00:00 2001 From: Gluten Performance Bot <137994563+GlutenPerfBot@users.noreply.github.com> Date: Sat, 8 Feb 2025 03:20:37 +0800 Subject: [PATCH 18/22] [GLUTEN-6887][VL] Daily Update Velox Version (2025_02_07) (#8680) Upstream Velox's New Commits: 1e455347a by Pedro Eugenio Rocha Pedreira, feat(python): TableWriter and Hive connector (12279) 3375085bf by Kevin Wilfong, fix: Fixe handling of user errors in json_parse (12272) 4caf5d182 by Natasha Sehgal, Add TDigest type and varbinary conversion (12270) 1c32192ce by Jimmy Lu, test: Add serialization backward compatibility test for ApproxMostFrequentStreamSummary (12242) 1eb79878d by Yenda Li, Add ip_subnet_max/min [5/n] (11515) a7faee2a8 by Pedro Eugenio Rocha Pedreira, feat(python): Support equality and .child_at in PyVector (12260) bd248793f by Bikramjeet Vig, fix: Ensure lambda functions fail on oversized array inputs (12271) 34cec0243 by Yenda Li, fix: Address ipprefix type incompatibility mismatch (12145) c0d8954c4 by Krishna Pai, fix(build): Change to 32-core-ubuntu (12265) 589d657fb by Pedro Eugenio Rocha Pedreira, feat(python): Add missing .pyi symbols (12269) 8e6e2ccee by Jimmy Lu, feat: Add DecodedVector::sharedBase() (12249) --- ep/build-velox/src/get_velox.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ep/build-velox/src/get_velox.sh b/ep/build-velox/src/get_velox.sh index 1b29c7dc1068..68b6c720e866 100755 --- a/ep/build-velox/src/get_velox.sh +++ b/ep/build-velox/src/get_velox.sh @@ -17,7 +17,7 @@ set -exu VELOX_REPO=https://github.com/oap-project/velox.git -VELOX_BRANCH=2025_02_06 +VELOX_BRANCH=2025_02_07 VELOX_HOME="" OS=`uname -s` From 9539ebf4262e298697188eebf51a924126c0fd3f Mon Sep 17 00:00:00 2001 From: Rong Ma Date: Sat, 8 Feb 2025 15:55:18 +0000 Subject: [PATCH 19/22] [GLUTEN-8678] Fix jar name on macos (#8679) --- package/pom.xml | 2 +- pom.xml | 9 +++++++++ 2 files changed, 10 insertions(+), 1 deletion(-) diff --git a/package/pom.xml b/package/pom.xml index a1df1edc4eb8..2917cb923da5 100644 --- a/package/pom.xml +++ b/package/pom.xml @@ -224,7 +224,7 @@ target/gluten-package-${project.version}.jar - target/${jar.assembly.name.prefix}-${backend_type}-bundle-spark${sparkbundle.version}_${scala.binary.version}-${os.detected.release}_${os.detected.release.version}_${os.detected.arch}-${project.version}.jar + target/${jar.assembly.name.prefix}-${backend_type}-bundle-spark${sparkbundle.version}_${scala.binary.version}-${os.full.name}-${project.version}.jar diff --git a/pom.xml b/pom.xml index 3dd3e7089706..be08b7d19d46 100644 --- a/pom.xml +++ b/pom.xml @@ -79,6 +79,11 @@ spark-sql-columnar gluten 1.5.0.Final + + unknown all @@ -952,6 +957,7 @@ darwin x86_64 + ${platform}_${arch} @@ -965,6 +971,7 @@ darwin aarch64 + ${platform}_${arch} @@ -978,6 +985,7 @@ linux amd64 + ${platform}_${arch} @@ -991,6 +999,7 @@ linux aarch64 + ${platform}_${arch} From 944c681eb8f0c15dedda1951cfebd513ddc4cf01 Mon Sep 17 00:00:00 2001 From: Kyligence Git Date: Sun, 9 Feb 2025 03:19:26 -0600 Subject: [PATCH 20/22] [GLUTEN-1632][CH]Daily Update Clickhouse Version (20250208) (#8688) * [GLUTEN-1632][CH]Daily Update Clickhouse Version (20250208) * Fix Build due to https://github.com/ClickHouse/ClickHouse/pull/75659 * latest version --------- Co-authored-by: kyligence-git Co-authored-by: Chang Chen --- cpp-ch/clickhouse.version | 4 ++-- cpp-ch/local-engine/Operator/StreamingAggregatingStep.cpp | 1 + cpp-ch/local-engine/Parser/LocalExecutor.cpp | 1 + .../local-engine/Storages/MergeTree/SparkMergeTreeWriter.cpp | 2 +- 4 files changed, 5 insertions(+), 3 deletions(-) diff --git a/cpp-ch/clickhouse.version b/cpp-ch/clickhouse.version index 72d0685e152a..f528a7b1e3e2 100644 --- a/cpp-ch/clickhouse.version +++ b/cpp-ch/clickhouse.version @@ -1,3 +1,3 @@ CH_ORG=Kyligence -CH_BRANCH=rebase_ch/20250207 -CH_COMMIT=b6c11f40aed +CH_BRANCH=rebase_ch/20250208 +CH_COMMIT=3a115ba3dfc diff --git a/cpp-ch/local-engine/Operator/StreamingAggregatingStep.cpp b/cpp-ch/local-engine/Operator/StreamingAggregatingStep.cpp index d19c9717d466..66aba9257510 100644 --- a/cpp-ch/local-engine/Operator/StreamingAggregatingStep.cpp +++ b/cpp-ch/local-engine/Operator/StreamingAggregatingStep.cpp @@ -23,6 +23,7 @@ #include #include #include +#include namespace DB { diff --git a/cpp-ch/local-engine/Parser/LocalExecutor.cpp b/cpp-ch/local-engine/Parser/LocalExecutor.cpp index 6ef860aff92d..54e03983886c 100644 --- a/cpp-ch/local-engine/Parser/LocalExecutor.cpp +++ b/cpp-ch/local-engine/Parser/LocalExecutor.cpp @@ -22,6 +22,7 @@ #include #include #include +#include namespace DB::Setting { diff --git a/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeWriter.cpp b/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeWriter.cpp index f412bfac0a6b..488b46318df3 100644 --- a/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeWriter.cpp +++ b/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeWriter.cpp @@ -16,8 +16,8 @@ */ #include "SparkMergeTreeWriter.h" -#include #include +#include #include #include #include From bdb68de256d6aee36f6a1e27241b3f7b299e6723 Mon Sep 17 00:00:00 2001 From: Gluten Performance Bot <137994563+GlutenPerfBot@users.noreply.github.com> Date: Mon, 10 Feb 2025 07:27:40 +0800 Subject: [PATCH 21/22] [GLUTEN-6887][VL] Daily Update Velox Version (2025_02_08) (#8687) Upstream Velox's New Commits: 5b053cc12 by Kevin Wilfong, feat: Optimize PrestoBatchVectorSerializer [2/7]: Serialize RowVectors (12072) 62aa025c5 by rui-mo, fix(parquet): Get sessionTimezone from QueryConfig (11949) 8e9e50d31 by Kevin Wilfong, test: Deflake MultiFragmentTest.maxBytes (12284) 179cef72e by Shakyan Kushwaha, build: Allow options to wget curl cmake during setup (12201) cf4806548 by Deepak Majeti, feat(S3): Add payload-signing-policy config (12197) 788555cc8 by David Reveman, feat: Remove unnecessary PTX usage in WarpScan (12264) 8ad54c52b by Ke Wang, feat: Add exchange http transaction create delay counters (12276) 1ff6df282 by joey.ljy, feat: Extend the simple UDAF interface with function-level variables (12067) 50c525af6 by Yenda Li, feat: Add setting/getting properties on directories. (12247) 4f946c7d3 by Yenda Li, fix: IPPrefix serialization with null case (12280) --- cpp/velox/udf/examples/MyUDAF.cc | 10 +++++----- ep/build-velox/src/get_velox.sh | 2 +- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/cpp/velox/udf/examples/MyUDAF.cc b/cpp/velox/udf/examples/MyUDAF.cc index 816880d2e71b..e6c8290e709e 100644 --- a/cpp/velox/udf/examples/MyUDAF.cc +++ b/cpp/velox/udf/examples/MyUDAF.cc @@ -66,7 +66,7 @@ class AverageAggregate { AccumulatorType() = delete; // Constructor used in initializeNewGroups(). - explicit AccumulatorType(HashStringAllocator* /*allocator*/) { + explicit AccumulatorType(HashStringAllocator* /*allocator*/, AverageAggregate* /* fn*/) { sum_ = 0; count_ = 0; } @@ -145,19 +145,19 @@ class MyAvgRegisterer final : public gluten::UdafRegisterer { if (exec::isRawInput(step)) { switch (inputType->kind()) { case TypeKind::REAL: - return std::make_unique>>(resultType); + return std::make_unique>>(step, argTypes, resultType); case TypeKind::DOUBLE: - return std::make_unique>>(resultType); + return std::make_unique>>(step, argTypes, resultType); default: VELOX_FAIL("Unknown input type for {} aggregation {}", name_, inputType->kindName()); } } else { switch (resultType->kind()) { case TypeKind::REAL: - return std::make_unique>>(resultType); + return std::make_unique>>(step, argTypes, resultType); case TypeKind::DOUBLE: case TypeKind::ROW: - return std::make_unique>>(resultType); + return std::make_unique>>(step, argTypes, resultType); default: VELOX_FAIL("Unsupported result type for final aggregation: {}", resultType->kindName()); } diff --git a/ep/build-velox/src/get_velox.sh b/ep/build-velox/src/get_velox.sh index 68b6c720e866..62848fb5138c 100755 --- a/ep/build-velox/src/get_velox.sh +++ b/ep/build-velox/src/get_velox.sh @@ -17,7 +17,7 @@ set -exu VELOX_REPO=https://github.com/oap-project/velox.git -VELOX_BRANCH=2025_02_07 +VELOX_BRANCH=2025_02_08 VELOX_HOME="" OS=`uname -s` From c90f17088a26897500a08e4aeddbbe3cda09d6ac Mon Sep 17 00:00:00 2001 From: Rong Ma Date: Sun, 9 Feb 2025 23:30:11 +0000 Subject: [PATCH 22/22] [GLUTEN-8689][VL] Enable some test cases in GlutenSQLQueryTestSuite (#8690) * enable some test cases * update --- .../apache/gluten/utils/velox/VeloxSQLQueryTestSettings.scala | 3 +++ .../scala/org/apache/spark/sql/GlutenSQLQueryTestSuite.scala | 3 --- .../apache/gluten/utils/velox/VeloxSQLQueryTestSettings.scala | 3 +++ .../scala/org/apache/spark/sql/GlutenSQLQueryTestSuite.scala | 3 --- .../apache/gluten/utils/velox/VeloxSQLQueryTestSettings.scala | 3 +++ .../scala/org/apache/spark/sql/GlutenSQLQueryTestSuite.scala | 4 +--- .../apache/gluten/utils/velox/VeloxSQLQueryTestSettings.scala | 3 +++ .../scala/org/apache/spark/sql/GlutenSQLQueryTestSuite.scala | 4 +--- 8 files changed, 14 insertions(+), 12 deletions(-) diff --git a/gluten-ut/spark32/src/test/scala/org/apache/gluten/utils/velox/VeloxSQLQueryTestSettings.scala b/gluten-ut/spark32/src/test/scala/org/apache/gluten/utils/velox/VeloxSQLQueryTestSettings.scala index ae5c78c56594..739cb44e3b1d 100644 --- a/gluten-ut/spark32/src/test/scala/org/apache/gluten/utils/velox/VeloxSQLQueryTestSettings.scala +++ b/gluten-ut/spark32/src/test/scala/org/apache/gluten/utils/velox/VeloxSQLQueryTestSettings.scala @@ -26,6 +26,7 @@ object VeloxSQLQueryTestSettings extends SQLQueryTestSettings { // Put relative path to "/path/to/spark/sql/core/src/test/resources/sql-tests/inputs" in this list val SUPPORTED_SQL_QUERY_LIST: Set[String] = Set( + "array.sql", "bitwise.sql", "cast.sql", "change-column.sql", @@ -63,10 +64,12 @@ object VeloxSQLQueryTestSettings extends SQLQueryTestSettings { "except-all.sql", "except.sql", "extract.sql", + "group-analytics.sql", "group-by-filter.sql", "group-by-ordinal.sql", "grouping_set.sql", "having.sql", + "higher-order-functions.sql", "ignored.sql", "inline-table.sql", "inner-join.sql", diff --git a/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/GlutenSQLQueryTestSuite.scala b/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/GlutenSQLQueryTestSuite.scala index 7a4b1c781674..de6c9e7ff3a8 100644 --- a/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/GlutenSQLQueryTestSuite.scala +++ b/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/GlutenSQLQueryTestSuite.scala @@ -213,9 +213,6 @@ class GlutenSQLQueryTestSuite "explain-aqe.sql", // explain plan is different "explain-cbo.sql", // explain "explain.sql", // explain - "group-analytics.sql", // wait velox to fix issue 3357 - "array.sql", // blocked by VELOX-5768 - "higher-order-functions.sql", // blocked by VELOX-5768 "udf/udf-window.sql" // Local window fixes are not added. ) ++ otherIgnoreList diff --git a/gluten-ut/spark33/src/test/scala/org/apache/gluten/utils/velox/VeloxSQLQueryTestSettings.scala b/gluten-ut/spark33/src/test/scala/org/apache/gluten/utils/velox/VeloxSQLQueryTestSettings.scala index 00a45d713e6c..d4004b6424f9 100644 --- a/gluten-ut/spark33/src/test/scala/org/apache/gluten/utils/velox/VeloxSQLQueryTestSettings.scala +++ b/gluten-ut/spark33/src/test/scala/org/apache/gluten/utils/velox/VeloxSQLQueryTestSettings.scala @@ -25,6 +25,7 @@ object VeloxSQLQueryTestSettings extends SQLQueryTestSettings { // Put relative path to "/path/to/spark/sql/core/src/test/resources/sql-tests/inputs" in this list private val SUPPORTED_SQL_QUERY_LIST: Set[String] = Set( + "array.sql", "bitwise.sql", "cast.sql", "change-column.sql", @@ -61,10 +62,12 @@ object VeloxSQLQueryTestSettings extends SQLQueryTestSettings { "except-all.sql", "except.sql", "extract.sql", + "group-analytics.sql", "group-by-filter.sql", "group-by-ordinal.sql", "grouping_set.sql", "having.sql", + "higher-order-functions.sql", "ignored.sql", "ilike-all.sql", "ilike-any.sql", diff --git a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/GlutenSQLQueryTestSuite.scala b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/GlutenSQLQueryTestSuite.scala index e336ca88a788..9c1bf9cbe1fe 100644 --- a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/GlutenSQLQueryTestSuite.scala +++ b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/GlutenSQLQueryTestSuite.scala @@ -213,9 +213,6 @@ class GlutenSQLQueryTestSuite "explain-aqe.sql", // explain plan is different "explain-cbo.sql", // explain "explain.sql", // explain - "group-analytics.sql", // wait velox to fix issue 3357 - "array.sql", // blocked by VELOX-5768 - "higher-order-functions.sql", // blocked by VELOX-5768 "udf/udf-window.sql", // Local window fixes are not added. "window.sql" // Local window fixes are not added. ) ++ otherIgnoreList diff --git a/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/velox/VeloxSQLQueryTestSettings.scala b/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/velox/VeloxSQLQueryTestSettings.scala index 345971e9ffc0..d49e9ccc663f 100644 --- a/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/velox/VeloxSQLQueryTestSettings.scala +++ b/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/velox/VeloxSQLQueryTestSettings.scala @@ -25,6 +25,7 @@ object VeloxSQLQueryTestSettings extends SQLQueryTestSettings { // Put relative path to "/path/to/spark/sql/core/src/test/resources/sql-tests/inputs" in this list val SUPPORTED_SQL_QUERY_LIST: Set[String] = Set( + "array.sql", "bitwise.sql", "cast.sql", "change-column.sql", @@ -61,10 +62,12 @@ object VeloxSQLQueryTestSettings extends SQLQueryTestSettings { "except-all.sql", "except.sql", "extract.sql", + "group-analytics.sql", "group-by-filter.sql", "group-by-ordinal.sql", "grouping_set.sql", "having.sql", + "higher-order-functions.sql", "ignored.sql", "ilike-all.sql", "ilike-any.sql", diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenSQLQueryTestSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenSQLQueryTestSuite.scala index 39a36898af6d..ce5d50b17532 100644 --- a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenSQLQueryTestSuite.scala +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenSQLQueryTestSuite.scala @@ -213,6 +213,7 @@ class GlutenSQLQueryTestSuite // 3.4 inadvertently enabled with "group-by.sql" and "group-by-ordinal.sql" private val udafIgnoreList = Set( + "udaf/udaf-group-analytics.sql", "udaf/udaf-group-by-ordinal.sql", "udaf/udaf-group-by.sql" ) @@ -223,9 +224,6 @@ class GlutenSQLQueryTestSuite "explain-aqe.sql", // explain plan is different "explain-cbo.sql", // explain "explain.sql", // explain - "group-analytics.sql", // wait velox to fix issue 3357 - "array.sql", // blocked by VELOX-5768 - "higher-order-functions.sql", // blocked by VELOX-5768 "udf/udf-window.sql", // Local window fixes are not added. "window.sql" // Local window fixes are not added. ) ++ otherIgnoreList ++ udafIgnoreList diff --git a/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/velox/VeloxSQLQueryTestSettings.scala b/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/velox/VeloxSQLQueryTestSettings.scala index 345971e9ffc0..d49e9ccc663f 100644 --- a/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/velox/VeloxSQLQueryTestSettings.scala +++ b/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/velox/VeloxSQLQueryTestSettings.scala @@ -25,6 +25,7 @@ object VeloxSQLQueryTestSettings extends SQLQueryTestSettings { // Put relative path to "/path/to/spark/sql/core/src/test/resources/sql-tests/inputs" in this list val SUPPORTED_SQL_QUERY_LIST: Set[String] = Set( + "array.sql", "bitwise.sql", "cast.sql", "change-column.sql", @@ -61,10 +62,12 @@ object VeloxSQLQueryTestSettings extends SQLQueryTestSettings { "except-all.sql", "except.sql", "extract.sql", + "group-analytics.sql", "group-by-filter.sql", "group-by-ordinal.sql", "grouping_set.sql", "having.sql", + "higher-order-functions.sql", "ignored.sql", "ilike-all.sql", "ilike-any.sql", diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/GlutenSQLQueryTestSuite.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/GlutenSQLQueryTestSuite.scala index 01d36635f357..5810cc5abe76 100644 --- a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/GlutenSQLQueryTestSuite.scala +++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/GlutenSQLQueryTestSuite.scala @@ -211,6 +211,7 @@ class GlutenSQLQueryTestSuite // 3.4 inadvertently enabled with "group-by.sql" and "group-by-ordinal.sql" private val udafIgnoreList = Set( + "udaf/udaf-group-analytics.sql", "udaf/udaf-group-by-ordinal.sql", "udaf/udaf-group-by.sql" ) @@ -221,9 +222,6 @@ class GlutenSQLQueryTestSuite "explain-aqe.sql", // explain plan is different "explain-cbo.sql", // explain "explain.sql", // explain - "group-analytics.sql", // wait velox to fix issue 3357 - "array.sql", // blocked by VELOX-5768 - "higher-order-functions.sql", // blocked by VELOX-5768 "udf/udf-window.sql", // Local window fixes are not added. "window.sql", // Local window fixes are not added. // Disable for Spark 3.