From 5d3fc80067d607ae898d7465e3e82faed3871467 Mon Sep 17 00:00:00 2001 From: morningman Date: Wed, 31 Oct 2018 14:46:22 +0800 Subject: [PATCH] Added: * Add streaming load feature. You can execute 'help stream load;' to see more information. Changed: * Loading phase of a certain table can be parallelized, to reduce the load job execution time when multi load jobs to a single table. * Using RocksDB to save the header info of tablets in Backends, to reduce the IO operations and increate speeding of restarting. Fixed: * A lot of bugs fixed. --- be/src/aes/CMakeLists.txt | 26 + be/src/aes/my_aes.cpp | 57 + be/src/aes/my_aes.h | 139 ++ be/src/aes/my_aes_impl.h | 37 + be/src/aes/my_aes_openssl.cpp | 216 +++ be/src/exec/olap_table_info.cpp | 329 +++++ be/src/exec/olap_table_info.h | 235 ++++ be/src/exec/olap_table_sink.cpp | 754 +++++++++++ be/src/exec/olap_table_sink.h | 248 ++++ be/src/exprs/es_functions.cpp | 33 + be/src/exprs/es_functions.h | 40 + be/src/http/action/meta_action.cpp | 78 ++ be/src/http/action/meta_action.h | 48 + be/src/http/action/restore_tablet_action.cpp | 294 +++++ be/src/http/action/restore_tablet_action.h | 62 + be/src/http/action/stream_load.cpp | 606 +++++++++ be/src/http/action/stream_load.h | 63 + be/src/http/http_common.h | 42 + be/src/http/message_body_sink.cpp | 75 ++ be/src/http/message_body_sink.h | 58 + be/src/http/utils.cpp | 70 + be/src/http/utils.h | 44 + be/src/olap/delta_writer.cpp | 226 ++++ be/src/olap/delta_writer.h | 83 ++ be/src/olap/memtable.cpp | 161 +++ be/src/olap/memtable.h | 64 + be/src/olap/new_status.cpp | 134 ++ be/src/olap/new_status.h | 196 +++ be/src/olap/olap_header_manager.cpp | 175 +++ be/src/olap/olap_header_manager.h | 54 + be/src/olap/olap_meta.cpp | 152 +++ be/src/olap/olap_meta.h | 56 + be/src/olap/options.cpp | 69 + be/src/olap/options.h | 40 + be/src/olap/rowset.cpp | 635 +++++++++ be/src/olap/rowset.h | 285 ++++ be/src/olap/rowset_builder.h | 63 + be/src/olap/schema.h | 165 +++ be/src/olap/skiplist.h | 397 ++++++ be/src/olap/store.cpp | 557 ++++++++ be/src/olap/store.h | 126 ++ be/src/runtime/kafka_consumer_pipe.cpp | 21 + be/src/runtime/kafka_consumer_pipe.h | 48 + be/src/runtime/load_stream_mgr.h | 60 + be/src/runtime/stream_load_pipe.h | 173 +++ be/src/runtime/tablet_writer_mgr.cpp | 327 +++++ be/src/runtime/tablet_writer_mgr.h | 97 ++ be/src/tools/CMakeLists.txt | 36 + be/src/tools/meta_tool.cpp | 167 +++ be/src/util/arena.cc | 68 + be/src/util/arena.h | 69 + be/src/util/byte_buffer.h | 68 + be/src/util/json_util.cpp | 46 + be/src/util/random.h | 64 + be/src/util/ref_count_closure.h | 53 + be/src/util/string_util.h | 65 + be/src/util/uid_util.cpp | 25 + be/test/exec/olap_table_info_test.cpp | 439 ++++++ be/test/exec/olap_table_sink_test.cpp | 927 +++++++++++++ be/test/http/http_utils_test.cpp | 91 ++ be/test/http/message_body_sink_test.cpp | 61 + be/test/http/stream_load_test.cpp | 231 ++++ be/test/olap/delta_writer_test.cpp | 666 ++++++++++ be/test/olap/olap_header_manager_test.cpp | 118 ++ be/test/olap/olap_meta_test.cpp | 121 ++ be/test/olap/serialize_test.cpp | 242 ++++ be/test/olap/skiplist_test.cpp | 387 ++++++ be/test/olap/test_data/header.txt | 152 +++ be/test/runtime/stream_load_pipe_test.cpp | 260 ++++ be/test/runtime/tablet_writer_mgr_test.cpp | 678 ++++++++++ be/test/util/arena_test.cpp | 68 + be/test/util/byte_buffer_test2.cpp | 56 + be/test/util/descriptor_helper.h | 155 +++ be/test/util/json_util_test.cpp | 70 + be/test/util/string_util_test.cpp | 78 ++ be/test/util/uid_util_test.cpp | 98 ++ fe/build.xml.deprecated | 247 ---- .../AdminShowReplicaDistributionStmt.java | 93 ++ .../analysis/AdminShowReplicaStatusStmt.java | 161 +++ .../baidu/palo/analysis/ImportColumnDesc.java | 30 + .../palo/analysis/ImportColumnsStmt.java | 23 + .../baidu/palo/analysis/ImportWhereStmt.java | 21 + .../java/com/baidu/palo/catalog/EsTable.java | 225 ++++ .../baidu/palo/catalog/MetadataViewer.java | 245 ++++ .../com/baidu/palo/catalog/TabletStatMgr.java | 141 ++ .../palo/common/AuthenticationException.java | 34 + .../com/baidu/palo/common/UserException.java | 43 + .../palo/common/proc/EsPartitionsProcDir.java | 131 ++ .../palo/common/proc/EsShardProcDir.java | 108 ++ .../palo/common/proc/TransDbProcDir.java | 92 ++ .../common/proc/TransPartitionProcNode.java | 75 ++ .../baidu/palo/common/proc/TransProcDir.java | 98 ++ .../palo/common/proc/TransTablesProcDir.java | 113 ++ .../com/baidu/palo/external/EsIndexState.java | 145 ++ .../baidu/palo/external/EsShardRouting.java | 64 + .../com/baidu/palo/external/EsStateStore.java | 221 ++++ .../com/baidu/palo/external/EsTableState.java | 85 ++ .../java/com/baidu/palo/external/EsUtil.java | 91 ++ .../external/ExternalDataSourceException.java | 27 + .../palo/http/rest/CancelStreamLoad.java | 87 ++ .../palo/http/rest/GetStreamLoadState.java | 89 ++ .../com/baidu/palo/planner/EsScanNode.java | 258 ++++ .../com/baidu/palo/planner/OlapTableSink.java | 303 +++++ .../baidu/palo/planner/StreamLoadPlanner.java | 142 ++ .../palo/planner/StreamLoadScanNode.java | 374 ++++++ .../com/baidu/palo/task/ClearAlterTask.java | 34 + .../baidu/palo/task/ClearTransactionTask.java | 40 + .../baidu/palo/task/PublishVersionTask.java | 62 + .../baidu/palo/task/RecoverTabletTask.java | 42 + .../BeginTransactionException.java | 16 + .../transaction/GlobalTransactionMgr.java | 1173 +++++++++++++++++ .../IllegalTransactionParameterException.java | 16 + .../LabelAlreadyExistsException.java | 36 + .../palo/transaction/PartitionCommitInfo.java | 77 ++ .../transaction/PublishVersionDaemon.java | 195 +++ .../palo/transaction/TableCommitInfo.java | 88 ++ .../palo/transaction/TabletCommitInfo.java | 66 + .../TransactionCommitFailedException.java | 31 + .../transaction/TransactionIDGenerator.java | 73 + .../palo/transaction/TransactionState.java | 322 +++++ .../palo/transaction/TransactionStatus.java | 69 + .../com/baidu/palo/alter/RollupJobTest.java | 329 +++++ .../baidu/palo/alter/SchemaChangeJobTest.java | 256 ++++ .../palo/analysis/AdminShowReplicaTest.java | 77 ++ .../baidu/palo/catalog/CatalogTestUtil.java | 294 +++++ .../com/baidu/palo/catalog/FakeCatalog.java | 31 + .../com/baidu/palo/catalog/FakeEditLog.java | 74 ++ .../palo/catalog/MetadataViewerTest.java | 107 ++ .../com/baidu/palo/es/EsStateStoreTest.java | 206 +++ .../java/com/baidu/palo/es/EsUtilTest.java | 69 + .../baidu/palo/planner/OlapTableSinkTest.java | 174 +++ .../palo/planner/StreamLoadPlannerTest.java | 74 ++ .../palo/planner/StreamLoadScanNodeTest.java | 522 ++++++++ .../FakeTransactionIDGenerator.java | 45 + .../transaction/GlobalTransactionMgrTest.java | 448 +++++++ .../test/resources/data/es/clusterstate1.json | 750 +++++++++++ .../test/resources/data/es/clusterstate2.json | 720 ++++++++++ .../test/resources/data/es/clusterstate3.json | 745 +++++++++++ .../apache_hdfs_broker/build.xml.deprecated | 95 -- fs_brokers/apache_hdfs_broker/pom.xml | 458 +++++++ gensrc/proto/descriptors.proto | 59 + .../PaloExternalDataSourceService.thrift | 221 ++++ 142 files changed, 24995 insertions(+), 342 deletions(-) create mode 100644 be/src/aes/CMakeLists.txt create mode 100644 be/src/aes/my_aes.cpp create mode 100644 be/src/aes/my_aes.h create mode 100644 be/src/aes/my_aes_impl.h create mode 100644 be/src/aes/my_aes_openssl.cpp create mode 100644 be/src/exec/olap_table_info.cpp create mode 100644 be/src/exec/olap_table_info.h create mode 100644 be/src/exec/olap_table_sink.cpp create mode 100644 be/src/exec/olap_table_sink.h create mode 100644 be/src/exprs/es_functions.cpp create mode 100644 be/src/exprs/es_functions.h create mode 100644 be/src/http/action/meta_action.cpp create mode 100644 be/src/http/action/meta_action.h create mode 100644 be/src/http/action/restore_tablet_action.cpp create mode 100644 be/src/http/action/restore_tablet_action.h create mode 100644 be/src/http/action/stream_load.cpp create mode 100644 be/src/http/action/stream_load.h create mode 100644 be/src/http/http_common.h create mode 100644 be/src/http/message_body_sink.cpp create mode 100644 be/src/http/message_body_sink.h create mode 100644 be/src/http/utils.cpp create mode 100644 be/src/http/utils.h create mode 100644 be/src/olap/delta_writer.cpp create mode 100644 be/src/olap/delta_writer.h create mode 100644 be/src/olap/memtable.cpp create mode 100644 be/src/olap/memtable.h create mode 100644 be/src/olap/new_status.cpp create mode 100644 be/src/olap/new_status.h create mode 100644 be/src/olap/olap_header_manager.cpp create mode 100644 be/src/olap/olap_header_manager.h create mode 100644 be/src/olap/olap_meta.cpp create mode 100644 be/src/olap/olap_meta.h create mode 100644 be/src/olap/options.cpp create mode 100644 be/src/olap/options.h create mode 100644 be/src/olap/rowset.cpp create mode 100644 be/src/olap/rowset.h create mode 100644 be/src/olap/rowset_builder.h create mode 100644 be/src/olap/schema.h create mode 100644 be/src/olap/skiplist.h create mode 100644 be/src/olap/store.cpp create mode 100644 be/src/olap/store.h create mode 100644 be/src/runtime/kafka_consumer_pipe.cpp create mode 100644 be/src/runtime/kafka_consumer_pipe.h create mode 100644 be/src/runtime/load_stream_mgr.h create mode 100644 be/src/runtime/stream_load_pipe.h create mode 100644 be/src/runtime/tablet_writer_mgr.cpp create mode 100644 be/src/runtime/tablet_writer_mgr.h create mode 100644 be/src/tools/CMakeLists.txt create mode 100644 be/src/tools/meta_tool.cpp create mode 100644 be/src/util/arena.cc create mode 100644 be/src/util/arena.h create mode 100644 be/src/util/byte_buffer.h create mode 100644 be/src/util/json_util.cpp create mode 100644 be/src/util/random.h create mode 100644 be/src/util/ref_count_closure.h create mode 100644 be/src/util/string_util.h create mode 100644 be/src/util/uid_util.cpp create mode 100644 be/test/exec/olap_table_info_test.cpp create mode 100644 be/test/exec/olap_table_sink_test.cpp create mode 100644 be/test/http/http_utils_test.cpp create mode 100644 be/test/http/message_body_sink_test.cpp create mode 100644 be/test/http/stream_load_test.cpp create mode 100644 be/test/olap/delta_writer_test.cpp create mode 100644 be/test/olap/olap_header_manager_test.cpp create mode 100644 be/test/olap/olap_meta_test.cpp create mode 100644 be/test/olap/serialize_test.cpp create mode 100644 be/test/olap/skiplist_test.cpp create mode 100644 be/test/olap/test_data/header.txt create mode 100644 be/test/runtime/stream_load_pipe_test.cpp create mode 100644 be/test/runtime/tablet_writer_mgr_test.cpp create mode 100644 be/test/util/arena_test.cpp create mode 100644 be/test/util/byte_buffer_test2.cpp create mode 100644 be/test/util/descriptor_helper.h create mode 100644 be/test/util/json_util_test.cpp create mode 100644 be/test/util/string_util_test.cpp create mode 100644 be/test/util/uid_util_test.cpp delete mode 100644 fe/build.xml.deprecated create mode 100644 fe/src/main/java/com/baidu/palo/analysis/AdminShowReplicaDistributionStmt.java create mode 100644 fe/src/main/java/com/baidu/palo/analysis/AdminShowReplicaStatusStmt.java create mode 100644 fe/src/main/java/com/baidu/palo/analysis/ImportColumnDesc.java create mode 100644 fe/src/main/java/com/baidu/palo/analysis/ImportColumnsStmt.java create mode 100644 fe/src/main/java/com/baidu/palo/analysis/ImportWhereStmt.java create mode 100644 fe/src/main/java/com/baidu/palo/catalog/EsTable.java create mode 100644 fe/src/main/java/com/baidu/palo/catalog/MetadataViewer.java create mode 100644 fe/src/main/java/com/baidu/palo/catalog/TabletStatMgr.java create mode 100644 fe/src/main/java/com/baidu/palo/common/AuthenticationException.java create mode 100644 fe/src/main/java/com/baidu/palo/common/UserException.java create mode 100644 fe/src/main/java/com/baidu/palo/common/proc/EsPartitionsProcDir.java create mode 100644 fe/src/main/java/com/baidu/palo/common/proc/EsShardProcDir.java create mode 100644 fe/src/main/java/com/baidu/palo/common/proc/TransDbProcDir.java create mode 100644 fe/src/main/java/com/baidu/palo/common/proc/TransPartitionProcNode.java create mode 100644 fe/src/main/java/com/baidu/palo/common/proc/TransProcDir.java create mode 100644 fe/src/main/java/com/baidu/palo/common/proc/TransTablesProcDir.java create mode 100644 fe/src/main/java/com/baidu/palo/external/EsIndexState.java create mode 100644 fe/src/main/java/com/baidu/palo/external/EsShardRouting.java create mode 100644 fe/src/main/java/com/baidu/palo/external/EsStateStore.java create mode 100644 fe/src/main/java/com/baidu/palo/external/EsTableState.java create mode 100644 fe/src/main/java/com/baidu/palo/external/EsUtil.java create mode 100644 fe/src/main/java/com/baidu/palo/external/ExternalDataSourceException.java create mode 100644 fe/src/main/java/com/baidu/palo/http/rest/CancelStreamLoad.java create mode 100644 fe/src/main/java/com/baidu/palo/http/rest/GetStreamLoadState.java create mode 100644 fe/src/main/java/com/baidu/palo/planner/EsScanNode.java create mode 100644 fe/src/main/java/com/baidu/palo/planner/OlapTableSink.java create mode 100644 fe/src/main/java/com/baidu/palo/planner/StreamLoadPlanner.java create mode 100644 fe/src/main/java/com/baidu/palo/planner/StreamLoadScanNode.java create mode 100644 fe/src/main/java/com/baidu/palo/task/ClearAlterTask.java create mode 100644 fe/src/main/java/com/baidu/palo/task/ClearTransactionTask.java create mode 100644 fe/src/main/java/com/baidu/palo/task/PublishVersionTask.java create mode 100644 fe/src/main/java/com/baidu/palo/task/RecoverTabletTask.java create mode 100644 fe/src/main/java/com/baidu/palo/transaction/BeginTransactionException.java create mode 100644 fe/src/main/java/com/baidu/palo/transaction/GlobalTransactionMgr.java create mode 100644 fe/src/main/java/com/baidu/palo/transaction/IllegalTransactionParameterException.java create mode 100644 fe/src/main/java/com/baidu/palo/transaction/LabelAlreadyExistsException.java create mode 100644 fe/src/main/java/com/baidu/palo/transaction/PartitionCommitInfo.java create mode 100644 fe/src/main/java/com/baidu/palo/transaction/PublishVersionDaemon.java create mode 100644 fe/src/main/java/com/baidu/palo/transaction/TableCommitInfo.java create mode 100644 fe/src/main/java/com/baidu/palo/transaction/TabletCommitInfo.java create mode 100644 fe/src/main/java/com/baidu/palo/transaction/TransactionCommitFailedException.java create mode 100644 fe/src/main/java/com/baidu/palo/transaction/TransactionIDGenerator.java create mode 100644 fe/src/main/java/com/baidu/palo/transaction/TransactionState.java create mode 100644 fe/src/main/java/com/baidu/palo/transaction/TransactionStatus.java create mode 100644 fe/src/test/java/com/baidu/palo/alter/RollupJobTest.java create mode 100644 fe/src/test/java/com/baidu/palo/alter/SchemaChangeJobTest.java create mode 100644 fe/src/test/java/com/baidu/palo/analysis/AdminShowReplicaTest.java create mode 100644 fe/src/test/java/com/baidu/palo/catalog/CatalogTestUtil.java create mode 100644 fe/src/test/java/com/baidu/palo/catalog/FakeCatalog.java create mode 100644 fe/src/test/java/com/baidu/palo/catalog/FakeEditLog.java create mode 100644 fe/src/test/java/com/baidu/palo/catalog/MetadataViewerTest.java create mode 100644 fe/src/test/java/com/baidu/palo/es/EsStateStoreTest.java create mode 100644 fe/src/test/java/com/baidu/palo/es/EsUtilTest.java create mode 100644 fe/src/test/java/com/baidu/palo/planner/OlapTableSinkTest.java create mode 100644 fe/src/test/java/com/baidu/palo/planner/StreamLoadPlannerTest.java create mode 100644 fe/src/test/java/com/baidu/palo/planner/StreamLoadScanNodeTest.java create mode 100644 fe/src/test/java/com/baidu/palo/transaction/FakeTransactionIDGenerator.java create mode 100644 fe/src/test/java/com/baidu/palo/transaction/GlobalTransactionMgrTest.java create mode 100644 fe/src/test/resources/data/es/clusterstate1.json create mode 100644 fe/src/test/resources/data/es/clusterstate2.json create mode 100644 fe/src/test/resources/data/es/clusterstate3.json delete mode 100644 fs_brokers/apache_hdfs_broker/build.xml.deprecated create mode 100644 fs_brokers/apache_hdfs_broker/pom.xml create mode 100644 gensrc/proto/descriptors.proto create mode 100644 gensrc/thrift/PaloExternalDataSourceService.thrift diff --git a/be/src/aes/CMakeLists.txt b/be/src/aes/CMakeLists.txt new file mode 100644 index 0000000000..0e6177c5e6 --- /dev/null +++ b/be/src/aes/CMakeLists.txt @@ -0,0 +1,26 @@ +# Copyright (c) 2017, Baidu.com, Inc. All Rights Reserved + +# Licensed 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. +# under the License. + +# where to put generated libraries +set(LIBRARY_OUTPUT_PATH "${BUILD_DIR}/src/aes") + +# where to put generated binaries +set(EXECUTABLE_OUTPUT_PATH "${BUILD_DIR}/src/aes") + +add_library(AES STATIC + my_aes.cpp + my_aes_openssl.cpp +) diff --git a/be/src/aes/my_aes.cpp b/be/src/aes/my_aes.cpp new file mode 100644 index 0000000000..4cae2f8e02 --- /dev/null +++ b/be/src/aes/my_aes.cpp @@ -0,0 +1,57 @@ +// Copyright (c) 2017, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 "my_aes.h" +#include "my_aes_impl.h" +#include + +/** + Transforms an arbitrary long key into a fixed length AES key + + AES keys are of fixed length. This routine takes an arbitrary long key + iterates over it in AES key length increment and XORs the bytes with the + AES key buffer being prepared. + The bytes from the last incomplete iteration are XORed to the start + of the key until their depletion. + Needed since crypto function routines expect a fixed length key. + + @param key [in] Key to use for real key creation + @param key_length [in] Length of the key + @param rkey [out] Real key (used by OpenSSL/YaSSL) + @param opmode [out] encryption mode +*/ +namespace palo { +void my_aes_create_key(const unsigned char *key, uint key_length, + uint8 *rkey, enum my_aes_opmode opmode) +{ + const uint key_size= my_aes_opmode_key_sizes[opmode] / 8; + uint8 *rkey_end; /* Real key boundary */ + uint8 *ptr; /* Start of the real key*/ + uint8 *sptr; /* Start of the working key */ + uint8 *key_end= ((uint8 *)key) + key_length; /* Working key boundary*/ + + rkey_end= rkey + key_size; + + memset(rkey, 0, key_size); /* Set initial key */ + + for (ptr= rkey, sptr= (uint8 *)key; sptr < key_end; ptr++, sptr++) + { + if (ptr == rkey_end) + /* Just loop over tmp_key until we used all key */ + ptr= rkey; + *ptr^= *sptr; + } +} +} diff --git a/be/src/aes/my_aes.h b/be/src/aes/my_aes.h new file mode 100644 index 0000000000..d65987e51d --- /dev/null +++ b/be/src/aes/my_aes.h @@ -0,0 +1,139 @@ +// Copyright (c) 2017, Baidu.com, Inc. All Rights Reserved + +// Licensed 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. + +#ifndef MY_AES_INCLUDED +#define MY_AES_INCLUDED + +/* Header file for my_aes.c */ +/* Wrapper to give simple interface for MySQL to AES standard encryption */ + +//C_MODE_START +#include + +/** AES IV size is 16 bytes for all supported ciphers except ECB */ +#define MY_AES_IV_SIZE 16 + +/** AES block size is fixed to be 128 bits for CBC and ECB */ +#define MY_AES_BLOCK_SIZE 16 +typedef uint32_t uint32; +typedef bool my_bool; +typedef uint32_t uint; + + +/** Supported AES cipher/block mode combos */ +enum my_aes_opmode +{ + my_aes_128_ecb, + my_aes_192_ecb, + my_aes_256_ecb, + my_aes_128_cbc, + my_aes_192_cbc, + my_aes_256_cbc +#ifndef HAVE_YASSL + ,my_aes_128_cfb1, + my_aes_192_cfb1, + my_aes_256_cfb1, + my_aes_128_cfb8, + my_aes_192_cfb8, + my_aes_256_cfb8, + my_aes_128_cfb128, + my_aes_192_cfb128, + my_aes_256_cfb128, + my_aes_128_ofb, + my_aes_192_ofb, + my_aes_256_ofb +#endif +}; + +#define MY_AES_BEGIN my_aes_128_ecb +#ifdef HAVE_YASSL +#define MY_AES_END my_aes_256_cbc +#else +#define MY_AES_END my_aes_256_ofb +#endif + +/* If bad data discovered during decoding */ +#define MY_AES_BAD_DATA -1 + +/** String representations of the supported AES modes. Keep in sync with my_aes_opmode */ +extern const char *my_aes_opmode_names[]; +namespace palo { +/** + Encrypt a buffer using AES + + @param source [in] Pointer to data for encryption + @param source_length [in] Size of encryption data + @param dest [out] Buffer to place encrypted data (must be large enough) + @param key [in] Key to be used for encryption + @param key_length [in] Length of the key. Will handle keys of any length + @param mode [in] encryption mode + @param iv [in] 16 bytes initialization vector if needed. Otherwise NULL + @param padding [in] if padding needed. + @return size of encrypted data, or negative in case of error +*/ + +int my_aes_encrypt(const unsigned char *source, uint32 source_length, + unsigned char *dest, + const unsigned char *key, uint32 key_length, + enum my_aes_opmode mode, const unsigned char *iv, + bool padding = true); + +/** + Decrypt an AES encrypted buffer + + @param source Pointer to data for decryption + @param source_length size of encrypted data + @param dest buffer to place decrypted data (must be large enough) + @param key Key to be used for decryption + @param key_length Length of the key. Will handle keys of any length + @param mode encryption mode + @param iv 16 bytes initialization vector if needed. Otherwise NULL + @param padding if padding needed. + @return size of original data. +*/ + + +int my_aes_decrypt(const unsigned char *source, uint32 source_length, + unsigned char *dest, + const unsigned char *key, uint32 key_length, + enum my_aes_opmode mode, const unsigned char *iv, + bool padding = true); + +/** + Calculate the size of a buffer large enough for encrypted data + + @param source_length length of data to be encrypted + @param mode encryption mode + @return size of buffer required to store encrypted data +*/ + +int my_aes_get_size(uint32 source_length, enum my_aes_opmode mode); + +/** + Return true if the AES cipher and block mode requires an IV + + SYNOPSIS + my_aes_needs_iv() + @param mode encryption mode + + @retval TRUE IV needed + @retval FALSE IV not needed +*/ + +my_bool my_aes_needs_iv(my_aes_opmode opmode); +} +//C_MODE_END + +#endif /* MY_AES_INCLUDED */ diff --git a/be/src/aes/my_aes_impl.h b/be/src/aes/my_aes_impl.h new file mode 100644 index 0000000000..82c710c5e2 --- /dev/null +++ b/be/src/aes/my_aes_impl.h @@ -0,0 +1,37 @@ +// Copyright (c) 2017, Baidu.com, Inc. All Rights Reserved + +// Licensed 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. + +#ifndef BDG_PALO_BE_EXPRS_MY_AES_IMPL_H +#define BDG_PALO_BE_EXPRS_MY_AES_IMPL_H + +/** Maximum supported key kength */ +const int MAX_AES_KEY_LENGTH = 256; + +/* TODO: remove in a future version */ +/* Guard against using an old export control restriction #define */ +#ifdef AES_USE_KEY_BITS +#error AES_USE_KEY_BITS not supported +#endif +typedef uint32_t uint; +typedef uint8_t uint8; + +namespace palo { + +extern uint *my_aes_opmode_key_sizes; +void my_aes_create_key(const unsigned char *key, uint key_length, + uint8 *rkey, enum my_aes_opmode opmode); +} + +#endif diff --git a/be/src/aes/my_aes_openssl.cpp b/be/src/aes/my_aes_openssl.cpp new file mode 100644 index 0000000000..72f3eb6447 --- /dev/null +++ b/be/src/aes/my_aes_openssl.cpp @@ -0,0 +1,216 @@ +// Copyright (c) 2017, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 "my_aes.h" +#include "my_aes_impl.h" +#include +#include + +#include +#include +#include + +#define DBUG_ASSERT(A) assert(A) +#define TRUE true +#define FALSE false +namespace palo { +/* keep in sync with enum my_aes_opmode in my_aes.h */ +const char *my_aes_opmode_names[]= +{ + "aes-128-ecb", + "aes-192-ecb", + "aes-256-ecb", + "aes-128-cbc", + "aes-192-cbc", + "aes-256-cbc", + "aes-128-cfb1", + "aes-192-cfb1", + "aes-256-cfb1", + "aes-128-cfb8", + "aes-192-cfb8", + "aes-256-cfb8", + "aes-128-cfb128", + "aes-192-cfb128", + "aes-256-cfb128", + "aes-128-ofb", + "aes-192-ofb", + "aes-256-ofb", + NULL /* needed for the type enumeration */ +}; + + +/* keep in sync with enum my_aes_opmode in my_aes.h */ +static uint my_aes_opmode_key_sizes_impl[]= +{ + 128 /* aes-128-ecb */, + 192 /* aes-192-ecb */, + 256 /* aes-256-ecb */, + 128 /* aes-128-cbc */, + 192 /* aes-192-cbc */, + 256 /* aes-256-cbc */, + 128 /* aes-128-cfb1 */, + 192 /* aes-192-cfb1 */, + 256 /* aes-256-cfb1 */, + 128 /* aes-128-cfb8 */, + 192 /* aes-192-cfb8 */, + 256 /* aes-256-cfb8 */, + 128 /* aes-128-cfb128 */, + 192 /* aes-192-cfb128 */, + 256 /* aes-256-cfb128 */, + 128 /* aes-128-ofb */, + 192 /* aes-192-ofb */, + 256 /* aes-256-ofb */ +}; + +uint *my_aes_opmode_key_sizes= my_aes_opmode_key_sizes_impl; + + + +static const EVP_CIPHER * +aes_evp_type(const my_aes_opmode mode) +{ + switch (mode) + { + case my_aes_128_ecb: return EVP_aes_128_ecb(); + case my_aes_128_cbc: return EVP_aes_128_cbc(); + case my_aes_128_cfb1: return EVP_aes_128_cfb1(); + case my_aes_128_cfb8: return EVP_aes_128_cfb8(); + case my_aes_128_cfb128: return EVP_aes_128_cfb128(); + case my_aes_128_ofb: return EVP_aes_128_ofb(); + case my_aes_192_ecb: return EVP_aes_192_ecb(); + case my_aes_192_cbc: return EVP_aes_192_cbc(); + case my_aes_192_cfb1: return EVP_aes_192_cfb1(); + case my_aes_192_cfb8: return EVP_aes_192_cfb8(); + case my_aes_192_cfb128: return EVP_aes_192_cfb128(); + case my_aes_192_ofb: return EVP_aes_192_ofb(); + case my_aes_256_ecb: return EVP_aes_256_ecb(); + case my_aes_256_cbc: return EVP_aes_256_cbc(); + case my_aes_256_cfb1: return EVP_aes_256_cfb1(); + case my_aes_256_cfb8: return EVP_aes_256_cfb8(); + case my_aes_256_cfb128: return EVP_aes_256_cfb128(); + case my_aes_256_ofb: return EVP_aes_256_ofb(); + default: return NULL; + } +} + + +int my_aes_encrypt(const unsigned char *source, uint32 source_length, + unsigned char *dest, + const unsigned char *key, uint32 key_length, + enum my_aes_opmode mode, const unsigned char *iv, + bool padding) +{ + EVP_CIPHER_CTX ctx; + const EVP_CIPHER *cipher= aes_evp_type(mode); + int u_len, f_len; + /* The real key to be used for encryption */ + unsigned char rkey[MAX_AES_KEY_LENGTH / 8]; + my_aes_create_key(key, key_length, rkey, mode); + + if (!cipher || (EVP_CIPHER_iv_length(cipher) > 0 && !iv)) + return MY_AES_BAD_DATA; + + if (!EVP_EncryptInit(&ctx, cipher, rkey, iv)) + goto aes_error; /* Error */ + if (!EVP_CIPHER_CTX_set_padding(&ctx, padding)) + goto aes_error; /* Error */ + if (!EVP_EncryptUpdate(&ctx, dest, &u_len, source, source_length)) + goto aes_error; /* Error */ + + if (!EVP_EncryptFinal(&ctx, dest + u_len, &f_len)) + goto aes_error; /* Error */ + + EVP_CIPHER_CTX_cleanup(&ctx); + return u_len + f_len; + +aes_error: + /* need to explicitly clean up the error if we want to ignore it */ + ERR_clear_error(); + EVP_CIPHER_CTX_cleanup(&ctx); + return MY_AES_BAD_DATA; +} + +int my_aes_decrypt(const unsigned char *source, uint32 source_length, + unsigned char *dest, + const unsigned char *key, uint32 key_length, + enum my_aes_opmode mode, const unsigned char *iv, + bool padding) +{ + + EVP_CIPHER_CTX ctx; + const EVP_CIPHER *cipher= aes_evp_type(mode); + int u_len, f_len; + + /* The real key to be used for decryption */ + unsigned char rkey[MAX_AES_KEY_LENGTH / 8]; + + my_aes_create_key(key, key_length, rkey, mode); + if (!cipher || (EVP_CIPHER_iv_length(cipher) > 0 && !iv)) + return MY_AES_BAD_DATA; + + EVP_CIPHER_CTX_init(&ctx); + + if (!EVP_DecryptInit(&ctx, aes_evp_type(mode), rkey, iv)) + goto aes_error; /* Error */ + if (!EVP_CIPHER_CTX_set_padding(&ctx, padding)) + goto aes_error; /* Error */ + if (!EVP_DecryptUpdate(&ctx, dest, &u_len, source, source_length)) + goto aes_error; /* Error */ + if (!EVP_DecryptFinal_ex(&ctx, dest + u_len, &f_len)) + goto aes_error; /* Error */ + + EVP_CIPHER_CTX_cleanup(&ctx); + return u_len + f_len; + +aes_error: + /* need to explicitly clean up the error if we want to ignore it */ + ERR_clear_error(); + EVP_CIPHER_CTX_cleanup(&ctx); + return MY_AES_BAD_DATA; +} + +int my_aes_get_size(uint32 source_length, my_aes_opmode opmode) +{ + const EVP_CIPHER *cipher= aes_evp_type(opmode); + size_t block_size; + + block_size= EVP_CIPHER_block_size(cipher); + + return block_size > 1 ? + block_size * (source_length / block_size) + block_size : + source_length; +} + +/** + Return true if the AES cipher and block mode requires an IV + + SYNOPSIS + my_aes_needs_iv() + @param mode encryption mode + + @retval TRUE IV needed + @retval FALSE IV not needed +*/ + +my_bool my_aes_needs_iv(my_aes_opmode opmode) +{ + const EVP_CIPHER *cipher= aes_evp_type(opmode); + int iv_length; + + iv_length= EVP_CIPHER_iv_length(cipher); + DBUG_ASSERT(iv_length == 0 || iv_length == MY_AES_IV_SIZE); + return iv_length != 0 ? TRUE : FALSE; +} +} diff --git a/be/src/exec/olap_table_info.cpp b/be/src/exec/olap_table_info.cpp new file mode 100644 index 0000000000..6406cb85d6 --- /dev/null +++ b/be/src/exec/olap_table_info.cpp @@ -0,0 +1,329 @@ +// Copyright (c) 2017, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 "exec/olap_table_info.h" + +#include "runtime/mem_pool.h" +#include "runtime/mem_tracker.h" +#include "runtime/row_batch.h" +#include "runtime/tuple_row.h" +#include "util/debug_util.h" +#include "util/string_parser.hpp" + +namespace palo { + +void OlapTableIndexSchema::to_protobuf(POlapTableIndexSchema* pindex) const { + pindex->set_id(index_id); + pindex->set_schema_hash(schema_hash); + for (auto slot : slots) { + pindex->add_columns(slot->col_name()); + } +} + +Status OlapTableSchemaParam::init(const POlapTableSchemaParam& pschema) { + _db_id = pschema.db_id(); + _table_id = pschema.table_id(); + _version = pschema.version(); + std::map slots_map; + _tuple_desc = _obj_pool.add(new TupleDescriptor(pschema.tuple_desc())); + for (auto& p_slot_desc : pschema.slot_descs()) { + auto slot_desc = _obj_pool.add(new SlotDescriptor(p_slot_desc)); + _tuple_desc->add_slot(slot_desc); + slots_map.emplace(slot_desc->col_name(), slot_desc); + } + for (auto& p_index : pschema.indexes()) { + auto index = _obj_pool.add(new OlapTableIndexSchema()); + index->index_id = p_index.id(); + index->schema_hash = p_index.schema_hash(); + for (auto& col : p_index.columns()) { + auto it = slots_map.find(col); + if (it == std::end(slots_map)) { + std::stringstream ss; + ss << "unknown index column, column=" << col; + return Status(ss.str()); + } + index->slots.emplace_back(it->second); + } + _indexes.emplace_back(index); + } + + std::sort(_indexes.begin(), _indexes.end(), + [] (const OlapTableIndexSchema* lhs, const OlapTableIndexSchema* rhs) { + return lhs->index_id < rhs->index_id; + }); + return Status::OK; +} + +Status OlapTableSchemaParam::init(const TOlapTableSchemaParam& tschema) { + _db_id = tschema.db_id; + _table_id = tschema.table_id; + _version = tschema.version; + std::map slots_map; + _tuple_desc = _obj_pool.add(new TupleDescriptor(tschema.tuple_desc)); + for (auto& t_slot_desc : tschema.slot_descs) { + auto slot_desc = _obj_pool.add(new SlotDescriptor(t_slot_desc)); + _tuple_desc->add_slot(slot_desc); + slots_map.emplace(slot_desc->col_name(), slot_desc); + } + for (auto& t_index : tschema.indexes) { + auto index = _obj_pool.add(new OlapTableIndexSchema()); + index->index_id = t_index.id; + index->schema_hash = t_index.schema_hash; + for (auto& col : t_index.columns) { + auto it = slots_map.find(col); + if (it == std::end(slots_map)) { + std::stringstream ss; + ss << "unknown index column, column=" << col; + return Status(ss.str()); + } + index->slots.emplace_back(it->second); + } + _indexes.emplace_back(index); + } + + std::sort(_indexes.begin(), _indexes.end(), + [] (const OlapTableIndexSchema* lhs, const OlapTableIndexSchema* rhs) { + return lhs->index_id < rhs->index_id; + }); + return Status::OK; +} + +void OlapTableSchemaParam::to_protobuf(POlapTableSchemaParam* pschema) const { + pschema->set_db_id(_db_id); + pschema->set_table_id(_table_id); + pschema->set_version(_version); + _tuple_desc->to_protobuf(pschema->mutable_tuple_desc()); + for (auto slot : _tuple_desc->slots()) { + slot->to_protobuf(pschema->add_slot_descs()); + } + for (auto index : _indexes) { + index->to_protobuf(pschema->add_indexes()); + } +} + +std::string OlapTableSchemaParam::debug_string() const { + std::stringstream ss; + ss << "tuple_desc=" << _tuple_desc->debug_string(); + return ss.str(); +} + +std::string OlapTablePartition::debug_string(TupleDescriptor* tuple_desc) const { + std::stringstream ss; + ss << "(id=" << id + << ",start_key=" << print_tuple(start_key, *tuple_desc) + << ",end_key=" << print_tuple(end_key, *tuple_desc) + << ",num_buckets=" << num_buckets + << ",indexes=["; + int idx = 0; + for (auto& index : indexes) { + if (idx++ > 0) { + ss << ","; + } + ss << "(id=" << index.index_id << ",tablets=["; + int jdx = 0; + for (auto id : index.tablets) { + if (jdx++ > 0) { + ss << ","; + } + ss << id; + } + ss << "])"; + } + ss << "])"; + return ss.str(); +} + +OlapTablePartitionParam::OlapTablePartitionParam( + std::shared_ptr schema, + const TOlapTablePartitionParam& t_param) + : _schema(schema), _t_param(t_param), + _partition_slot_desc(nullptr), + _mem_tracker(new MemTracker()), + _mem_pool(new MemPool(_mem_tracker.get())) { +} + +OlapTablePartitionParam::~OlapTablePartitionParam() { +} + +Status OlapTablePartitionParam::init() { + std::map slots_map; + for (auto slot_desc : _schema->tuple_desc()->slots()) { + slots_map.emplace(slot_desc->col_name(), slot_desc); + } + if (_t_param.__isset.partition_column) { + auto it = slots_map.find(_t_param.partition_column); + if (it == std::end(slots_map)) { + std::stringstream ss; + ss << "partition column not found, column=" << _t_param.partition_column; + return Status(ss.str()); + } + _partition_slot_desc = it->second; + } + _partitions_map.reset( + new std::map( + OlapTablePartKeyComparator(_partition_slot_desc))); + if (_t_param.__isset.distributed_columns) { + for (auto& col : _t_param.distributed_columns) { + auto it = slots_map.find(col); + if (it == std::end(slots_map)) { + std::stringstream ss; + ss << "distributed column not found, columns=" << col; + return Status(ss.str()); + } + _distributed_slot_descs.emplace_back(it->second); + } + } + // initial partitions + for (int i = 0; i < _t_param.partitions.size(); ++i) { + const TOlapTablePartition& t_part = _t_param.partitions[i]; + OlapTablePartition* part = _obj_pool.add(new OlapTablePartition()); + part->id = t_part.id; + if (t_part.__isset.start_key) { + RETURN_IF_ERROR(_create_partition_key(t_part.start_key, &part->start_key)); + } + if (t_part.__isset.end_key) { + RETURN_IF_ERROR(_create_partition_key(t_part.end_key, &part->end_key)); + } + part->num_buckets = t_part.num_buckets; + auto num_indexes = _schema->indexes().size(); + if (t_part.indexes.size() != num_indexes) { + std::stringstream ss; + ss << "number of partition's index is not equal with schema's" + << ", num_part_indexes=" << t_part.indexes.size() + << ", num_schema_indexes=" << num_indexes; + return Status(ss.str()); + } + part->indexes = t_part.indexes; + std::sort(part->indexes.begin(), part->indexes.end(), + [] (const OlapTableIndexTablets& lhs, const OlapTableIndexTablets& rhs) { + return lhs.index_id < rhs.index_id; + }); + // check index + for (int j = 0; j < num_indexes; ++j) { + if (part->indexes[j].index_id != _schema->indexes()[j]->index_id) { + std::stringstream ss; + ss << "partition's index is not equal with schema's" + << ", part_index=" << part->indexes[j].index_id + << ", schema_index=" << _schema->indexes()[j]->index_id; + return Status(ss.str()); + } + } + _partitions.emplace_back(part); + _partitions_map->emplace(part->end_key, part); + } + return Status::OK; +} + +bool OlapTablePartitionParam::find_tablet(Tuple* tuple, + const OlapTablePartition** partition, + uint32_t* dist_hashes) const { + auto it = _partitions_map->upper_bound(tuple); + if (it == _partitions_map->end()) { + return false; + } + if (_part_contains(it->second, tuple)) { + *partition = it->second; + *dist_hashes = _compute_dist_hash(tuple); + return true; + } + return false; +} + +Status OlapTablePartitionParam::_create_partition_key(const TExprNode& t_expr, Tuple** part_key) { + Tuple* tuple = (Tuple*)_mem_pool->allocate(_schema->tuple_desc()->byte_size()); + void* slot = tuple->get_slot(_partition_slot_desc->tuple_offset()); + tuple->set_not_null(_partition_slot_desc->null_indicator_offset()); + switch (t_expr.node_type) { + case TExprNodeType::DATE_LITERAL: { + if (!reinterpret_cast(slot)->from_date_str( + t_expr.date_literal.value.c_str(), t_expr.date_literal.value.size())) { + std::stringstream ss; + ss << "invalid date literal in partition column, date=" << t_expr.date_literal; + return Status(ss.str()); + } + break; + } + case TExprNodeType::INT_LITERAL: { + switch (t_expr.type.types[0].scalar_type.type) { + case TPrimitiveType::TINYINT: + *reinterpret_cast(slot) = t_expr.int_literal.value; + break; + case TPrimitiveType::SMALLINT: + *reinterpret_cast(slot) = t_expr.int_literal.value; + break; + case TPrimitiveType::INT: + *reinterpret_cast(slot) = t_expr.int_literal.value; + break; + case TPrimitiveType::BIGINT: + *reinterpret_cast(slot) = t_expr.int_literal.value; + break; + default: + DCHECK(false) << "unsupport int literal type, type=" << t_expr.type.types[0].type; + break; + } + break; + } + case TExprNodeType::LARGE_INT_LITERAL: { + StringParser::ParseResult parse_result = StringParser::PARSE_SUCCESS; + __int128 val = StringParser::string_to_int<__int128>( + t_expr.large_int_literal.value.c_str(), t_expr.large_int_literal.value.size(), + &parse_result); + if (parse_result != StringParser::PARSE_SUCCESS) { + val = MAX_INT128; + } + memcpy(slot, &val, sizeof(val)); + break; + } + default: { + std::stringstream ss; + ss << "unsupported partition column node type, type=" << t_expr.node_type; + return Status(ss.str()); + } + } + *part_key = tuple; + return Status::OK; +} + +std::string OlapTablePartitionParam::debug_string() const { + std::stringstream ss; + ss << "partitions=["; + int idx = 0; + for (auto part : _partitions) { + if (idx++ > 0) { + ss << ","; + } + ss << part->debug_string(_schema->tuple_desc()); + } + ss << "]"; + return ss.str(); +} + +uint32_t OlapTablePartitionParam::_compute_dist_hash(Tuple* key) const { + uint32_t hash_val = 0; + for (auto slot_desc : _distributed_slot_descs) { + auto slot = key->get_slot(slot_desc->tuple_offset()); + if (slot != nullptr) { + hash_val = RawValue::zlib_crc32(slot, slot_desc->type(), hash_val); + } else { + //NULL is treat as 0 when hash + static const int INT_VALUE = 0; + static const TypeDescriptor INT_TYPE(TYPE_INT); + hash_val = RawValue::zlib_crc32(&INT_VALUE, INT_TYPE, hash_val); + } + } + return hash_val; +} + +} diff --git a/be/src/exec/olap_table_info.h b/be/src/exec/olap_table_info.h new file mode 100644 index 0000000000..3db0d88270 --- /dev/null +++ b/be/src/exec/olap_table_info.h @@ -0,0 +1,235 @@ +// Copyright (c) 2017, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 "common/object_pool.h" +#include "common/status.h" +#include "gen_cpp/Descriptors_types.h" +#include "gen_cpp/descriptors.pb.h" +#include "runtime/descriptors.h" +#include "runtime/tuple.h" +#include "runtime/raw_value.h" + +namespace palo { + +class MemPool; +class MemTracker; +class RowBatch; + +struct OlapTableIndexSchema { + int64_t index_id; + std::vector slots; + int32_t schema_hash; + + void to_protobuf(POlapTableIndexSchema* pindex) const; +}; + +class OlapTableSchemaParam { +public: + OlapTableSchemaParam() { } + ~OlapTableSchemaParam() noexcept { } + + Status init(const TOlapTableSchemaParam& tschema); + Status init(const POlapTableSchemaParam& pschema); + + int64_t db_id() const { return _db_id; } + int64_t table_id() const { return _table_id; } + int64_t version() const { return _version; } + + TupleDescriptor* tuple_desc() const { return _tuple_desc; } + const std::vector& indexes() const { + return _indexes; + } + + void to_protobuf(POlapTableSchemaParam* pschema) const; + + // NOTE: this function is not thread-safe. + POlapTableSchemaParam* to_protobuf() const { + if (_proto_schema == nullptr) { + _proto_schema = _obj_pool.add(new POlapTableSchemaParam()); + to_protobuf(_proto_schema); + } + return _proto_schema; + } + + std::string debug_string() const; + +private: + int64_t _db_id; + int64_t _table_id; + int64_t _version; + + TupleDescriptor* _tuple_desc = nullptr; + mutable POlapTableSchemaParam* _proto_schema = nullptr; + std::vector _indexes; + mutable ObjectPool _obj_pool; +}; + +using OlapTableIndexTablets = TOlapTableIndexTablets; +// struct TOlapTableIndexTablets { +// 1: required i64 index_id +// 2: required list tablets +// } + +struct OlapTablePartition { + int64_t id = 0; + Tuple* start_key = nullptr; + Tuple* end_key = nullptr; + int64_t num_buckets = 0; + std::vector indexes; + + std::string debug_string(TupleDescriptor* tuple_desc) const; +}; + +class OlapTablePartKeyComparator { +public: + OlapTablePartKeyComparator(SlotDescriptor* slot_desc) : _slot_desc(slot_desc) { } + bool operator()(const Tuple* lhs, const Tuple* rhs) const { + if (lhs == nullptr) { + return false; + } else if (rhs == nullptr) { + return true; + } + auto lhs_value = lhs->get_slot(_slot_desc->tuple_offset()); + auto rhs_value = rhs->get_slot(_slot_desc->tuple_offset()); + return RawValue::lt(lhs_value, rhs_value, _slot_desc->type()); + } +private: + SlotDescriptor* _slot_desc; +}; + +// store an olap table's tablet information +class OlapTablePartitionParam { +public: + OlapTablePartitionParam( + std::shared_ptr schema, + const TOlapTablePartitionParam& param); + ~OlapTablePartitionParam(); + + Status init(); + + int64_t db_id() const { return _t_param.db_id; } + int64_t table_id() const { return _t_param.table_id; } + int64_t version() const { return _t_param.version; } + + // return true if we found this tuple in partition + bool find_tablet(Tuple* tuple, + const OlapTablePartition** partitions, + uint32_t* dist_hash) const; + + const std::vector& get_partitions() const { + return _partitions; + } + std::string debug_string() const; +private: + Status _create_partition_key(const TExprNode& t_expr, Tuple** part_key); + + uint32_t _compute_dist_hash(Tuple* key) const; + + // check if this partition contain this key + bool _part_contains(OlapTablePartition* part, Tuple* key) const { + if (part->start_key == nullptr) { + return true; + } + OlapTablePartKeyComparator comparator(_partition_slot_desc); + return !comparator(key, part->start_key); + } +private: + // this partition only valid in this schema + std::shared_ptr _schema; + TOlapTablePartitionParam _t_param; + + SlotDescriptor* _partition_slot_desc; + std::vector _distributed_slot_descs; + + ObjectPool _obj_pool; + std::unique_ptr _mem_tracker; + std::unique_ptr _mem_pool; + std::vector _partitions; + std::unique_ptr< + std::map> _partitions_map; +}; + +using TabletLocation = TTabletLocation; +// struct TTabletLocation { +// 1: required i64 tablet_id +// 2: required list node_ids +// } + +class OlapTableLocationParam { +public: + OlapTableLocationParam(const TOlapTableLocationParam& t_param) : _t_param(t_param) { + for (auto& location : _t_param.tablets) { + _tablets.emplace(location.tablet_id, &location); + } + } + + int64_t db_id() const { return _t_param.db_id; } + int64_t table_id() const { return _t_param.table_id; } + int64_t version() const { return _t_param.version; } + + TabletLocation* find_tablet(int64_t id) const { + auto it = _tablets.find(id); + if (it != std::end(_tablets)) { + return it->second; + } + return nullptr; + } +private: + TOlapTableLocationParam _t_param; + + std::unordered_map _tablets; +}; + +struct NodeInfo { + int64_t id; + int64_t option; + std::string host; + int32_t brpc_port; + + NodeInfo(const TNodeInfo& tnode) + : id(tnode.id), + option(tnode.option), + host(tnode.host), + brpc_port(tnode.async_internal_port) { + } +}; + +class PaloNodesInfo { +public: + PaloNodesInfo(const TPaloNodesInfo& t_nodes) { + for (auto& node : t_nodes.nodes) { + _nodes.emplace(node.id, node); + } + } + const NodeInfo* find_node(int64_t id) const { + auto it = _nodes.find(id); + if (it != std::end(_nodes)) { + return &it->second; + } + return nullptr; + } +private: + std::unordered_map _nodes; +}; + +} diff --git a/be/src/exec/olap_table_sink.cpp b/be/src/exec/olap_table_sink.cpp new file mode 100644 index 0000000000..d3130e66e8 --- /dev/null +++ b/be/src/exec/olap_table_sink.cpp @@ -0,0 +1,754 @@ +// Copyright (c) 2017, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 "exec/olap_table_sink.h" + +#include + +#include "exprs/expr.h" +#include "runtime/exec_env.h" +#include "runtime/row_batch.h" +#include "runtime/runtime_state.h" +#include "runtime/tuple_row.h" +#include "util/debug_util.h" + +#include "util/brpc_stub_cache.h" +#include "util/uid_util.h" +#include "service/brpc.h" + +namespace palo { +namespace stream_load { + +NodeChannel::NodeChannel(OlapTableSink* parent, int64_t index_id, + int64_t node_id, int32_t schema_hash) + : _parent(parent), _index_id(index_id), + _node_id(node_id), _schema_hash(schema_hash) { +} + +NodeChannel::~NodeChannel() { + if (_open_closure != nullptr) { + if (_open_closure->unref()) { + delete _open_closure; + } + _open_closure = nullptr; + } + if (_add_batch_closure != nullptr) { + if (_add_batch_closure->unref()) { + delete _add_batch_closure; + } + _add_batch_closure = nullptr; + } + _add_batch_request.release_id(); +} + +Status NodeChannel::init(RuntimeState* state) { + _tuple_desc = _parent->_output_tuple_desc; + _node_info = _parent->_nodes_info->find_node(_node_id); + if (_node_info == nullptr) { + std::stringstream ss; + ss << "unknown node id, id=" << _node_id; + return Status(ss.str()); + } + RowDescriptor row_desc(_tuple_desc, false); + _batch.reset(new RowBatch(row_desc, state->batch_size(), _parent->_mem_tracker)); + + _stub = state->exec_env()->brpc_stub_cache()->get_stub( + _node_info->host, _node_info->brpc_port); + if (_stub == nullptr) { + LOG(WARNING) << "Get rpc stub failed, host=" << _node_info->host + << ", port=" << _node_info->brpc_port; + return Status("get rpc stub failed"); + } + + // Initialize _add_batch_request + _add_batch_request.set_allocated_id(&_parent->_load_id); + _add_batch_request.set_index_id(_index_id); + _add_batch_request.set_sender_id(_parent->_sender_id); + + return Status::OK; +} + +void NodeChannel::open() { + PTabletWriterOpenRequest request; + request.set_allocated_id(&_parent->_load_id); + request.set_index_id(_index_id); + request.set_txn_id(_parent->_txn_id); + request.set_allocated_schema(_parent->_schema->to_protobuf()); + for (auto& tablet : _all_tablets) { + auto ptablet = request.add_tablets(); + ptablet->set_partition_id(tablet.partition_id); + ptablet->set_tablet_id(tablet.tablet_id); + } + request.set_num_senders(_parent->_num_senders); + request.set_need_gen_rollup(_parent->_need_gen_rollup); + + _open_closure = new RefCountClosure(); + _open_closure->ref(); + + // This ref is for RPC's reference + _open_closure->ref(); + _open_closure->cntl.set_timeout_ms(_rpc_timeout_ms); + _stub->tablet_writer_open(&_open_closure->cntl, + &request, + &_open_closure->result, + _open_closure); + request.release_id(); + request.release_schema(); +} + +Status NodeChannel::open_wait() { + _open_closure->join(); + if (_open_closure->cntl.Failed()) { + LOG(WARNING) << "failed to open tablet writer, error=" + << berror(_open_closure->cntl.ErrorCode()) + << ", error_text=" << _open_closure->cntl.ErrorText(); + return Status("failed to open tablet writer"); + } + Status status(_open_closure->result.status()); + if (_open_closure->unref()) { + delete _open_closure; + } + _open_closure = nullptr; + + // add batch closure + _add_batch_closure = new RefCountClosure(); + _add_batch_closure->ref(); + + return status; +} + +Status NodeChannel::add_row(Tuple* input_tuple, int64_t tablet_id) { + auto row_no = _batch->add_row(); + if (row_no == RowBatch::INVALID_ROW_INDEX) { + RETURN_IF_ERROR(_send_cur_batch()); + row_no = _batch->add_row(); + } + DCHECK_NE(row_no, RowBatch::INVALID_ROW_INDEX); + auto tuple = input_tuple->deep_copy(*_tuple_desc, _batch->tuple_data_pool()); + _batch->get_row(row_no)->set_tuple(0, tuple); + _batch->commit_last_row(); + _add_batch_request.add_tablet_ids(tablet_id); + return Status::OK; +} + +Status NodeChannel::close(RuntimeState* state) { + auto st = _close(state); + _batch.reset(); + return st; +} + +Status NodeChannel::_close(RuntimeState* state) { + RETURN_IF_ERROR(_wait_in_flight_packet()); + return _send_cur_batch(true); +} + +Status NodeChannel::close_wait(RuntimeState* state) { + RETURN_IF_ERROR(_wait_in_flight_packet()); + Status status(_add_batch_closure->result.status()); + if (status.ok()) { + for (auto& tablet : _add_batch_closure->result.tablet_vec()) { + TTabletCommitInfo commit_info; + commit_info.tabletId = tablet.tablet_id(); + commit_info.backendId = _node_id; + state->tablet_commit_infos().emplace_back(std::move(commit_info)); + } + } + // clear batch after sendt + _batch.reset(); + return status; +} + +void NodeChannel::cancel() { + // Do we need to wait last rpc finished??? + PTabletWriterCancelRequest request; + request.set_allocated_id(&_parent->_load_id); + request.set_index_id(_index_id); + request.set_sender_id(_parent->_sender_id); + + auto closure = new RefCountClosure(); + + closure->ref(); + closure->cntl.set_timeout_ms(_rpc_timeout_ms); + _stub->tablet_writer_cancel(&closure->cntl, + &request, + &closure->result, + closure); + request.release_id(); + + // reset batch + _batch.reset(); +} + +Status NodeChannel::_wait_in_flight_packet() { + if (!_has_in_flight_packet) { + return Status::OK; + } + _add_batch_closure->join(); + _has_in_flight_packet = false; + if (_add_batch_closure->cntl.Failed()) { + LOG(WARNING) << "failed to send batch, error=" + << berror(_add_batch_closure->cntl.ErrorCode()) + << ", error_text=" << _add_batch_closure->cntl.ErrorText(); + return Status("failed to send batch"); + } + return {_add_batch_closure->result.status()}; +} + +Status NodeChannel::_send_cur_batch(bool eos) { + RETURN_IF_ERROR(_wait_in_flight_packet()); + + // tablet_ids has already set when add row + _add_batch_request.set_eos(eos); + _add_batch_request.set_packet_seq(_next_packet_seq); + if (_batch->num_rows() > 0) { + _batch->serialize(_add_batch_request.mutable_row_batch()); + } + + _add_batch_closure->ref(); + _add_batch_closure->cntl.Reset(); + _add_batch_closure->cntl.set_timeout_ms(_rpc_timeout_ms); + + if (eos) { + for (auto pid : _parent->_partition_ids) { + _add_batch_request.add_partition_ids(pid); + } + } + + _stub->tablet_writer_add_batch(&_add_batch_closure->cntl, + &_add_batch_request, + &_add_batch_closure->result, + _add_batch_closure); + _add_batch_request.clear_tablet_ids(); + _add_batch_request.clear_row_batch(); + _add_batch_request.clear_partition_ids(); + + _has_in_flight_packet = true; + _next_packet_seq++; + + _batch->reset(); + return Status::OK; +} + +IndexChannel::~IndexChannel() { +} + +Status IndexChannel::init(RuntimeState* state, + const std::vector& tablets) { + // nodeId -> tabletIds + std::map> tablets_by_node; + for (auto& tablet : tablets) { + auto location = _parent->_location->find_tablet(tablet.tablet_id); + if (location == nullptr) { + LOG(WARNING) << "unknow tablet, tablet_id=" << tablet.tablet_id; + return Status("unknown tablet"); + } + std::vector channels; + for (auto& node_id : location->node_ids) { + NodeChannel* channel = nullptr; + auto it = _node_channels.find(node_id); + if (it == std::end(_node_channels)) { + channel = _parent->_pool->add( + new NodeChannel(_parent, _index_id, node_id, _schema_hash)); + _node_channels.emplace(node_id, channel); + } else { + channel = it->second; + } + channel->add_tablet(tablet); + channels.push_back(channel); + } + _channels_by_tablet.emplace(tablet.tablet_id, std::move(channels)); + } + for (auto& it : _node_channels) { + RETURN_IF_ERROR(it.second->init(state)); + } + return Status::OK; +} + +Status IndexChannel::open() { + for (auto& it : _node_channels) { + it.second->open(); + } + for (auto& it : _node_channels) { + auto channel = it.second; + auto st = channel->open_wait(); + if (!st.ok()) { + LOG(WARNING) << "tablet open failed, load_id=" << _parent->_load_id + << ", node=" << channel->node_info()->host + << ":" << channel->node_info()->brpc_port + << ", errmsg=" << st.get_error_msg(); + if (_handle_failed_node(channel)) { + LOG(WARNING) << "open failed, load_id=" << _parent->_load_id; + return st; + } + } + } + return Status::OK; +} + +Status IndexChannel::add_row(Tuple* tuple, int64_t tablet_id) { + auto it = _channels_by_tablet.find(tablet_id); + DCHECK(it != std::end(_channels_by_tablet)) << "unknown tablet, tablet_id=" << tablet_id; + for (auto channel : it->second) { + if (channel->already_failed()) { + continue; + } + auto st = channel->add_row(tuple, tablet_id); + if (!st.ok()) { + LOG(WARNING) << "NodeChannel add row failed, load_id=" << _parent->_load_id + << ", tablet_id=" << tablet_id + << ", node=" << channel->node_info()->host + << ":" << channel->node_info()->brpc_port + << ", errmsg=" << st.get_error_msg(); + if (_handle_failed_node(channel)) { + LOG(WARNING) << "add row failed, load_id=" << _parent->_load_id; + return st; + } + } + } + return Status::OK; +} + +Status IndexChannel::close(RuntimeState* state) { + std::vector need_wait_channels; + need_wait_channels.reserve(_node_channels.size()); + + Status close_status; + for (auto& it : _node_channels) { + auto channel = it.second; + if (channel->already_failed() || !close_status.ok()) { + channel->cancel(); + continue; + } + auto st = channel->close(state); + if (st.ok()) { + need_wait_channels.push_back(channel); + } else { + LOG(WARNING) << "close node channel failed, load_id=" << _parent->_load_id + << ", node=" << channel->node_info()->host + << ":" << channel->node_info()->brpc_port + << ", errmsg=" << st.get_error_msg(); + if (_handle_failed_node(channel)) { + LOG(WARNING) << "close failed, load_id=" << _parent->_load_id; + close_status = st; + } + } + } + + if (close_status.ok()) { + for (auto channel : need_wait_channels) { + auto st = channel->close_wait(state); + if (!st.ok()) { + LOG(WARNING) << "close_wait node channel failed, load_id=" << _parent->_load_id + << ", node=" << channel->node_info()->host + << ":" << channel->node_info()->brpc_port + << ", errmsg=" << st.get_error_msg(); + if (_handle_failed_node(channel)) { + LOG(WARNING) << "close_wait failed, load_id=" << _parent->_load_id; + return st; + } + } + } + } + return close_status; +} + +void IndexChannel::cancel() { + for (auto& it : _node_channels) { + it.second->cancel(); + } +} + +bool IndexChannel::_handle_failed_node(NodeChannel* channel) { + DCHECK(!channel->already_failed()); + channel->set_failed(); + _num_failed_channels++; + return _num_failed_channels >= ((_parent->_num_repicas + 1) / 2); +} + +OlapTableSink::OlapTableSink(ObjectPool* pool, + const RowDescriptor& row_desc, + const std::vector& texprs, + Status* status) + : _pool(pool), _input_row_desc(row_desc), _filter_bitmap(1024) { + if (!texprs.empty()) { + *status = Expr::create_expr_trees(_pool, texprs, &_output_expr_ctxs); + } +} + +OlapTableSink::~OlapTableSink() { +} + +Status OlapTableSink::init(const TDataSink& t_sink) { + DCHECK(t_sink.__isset.olap_table_sink); + auto& table_sink = t_sink.olap_table_sink; + _load_id.set_hi(table_sink.load_id.hi); + _load_id.set_lo(table_sink.load_id.lo); + _txn_id = table_sink.txn_id; + _db_id = table_sink.db_id; + _table_id = table_sink.table_id; + _num_repicas = table_sink.num_replicas; + _need_gen_rollup = table_sink.need_gen_rollup; + _db_name = table_sink.db_name; + _table_name = table_sink.table_name; + _tuple_desc_id = table_sink.tuple_id; + _schema.reset(new OlapTableSchemaParam()); + RETURN_IF_ERROR(_schema->init(table_sink.schema)); + _partition = _pool->add(new OlapTablePartitionParam(_schema, table_sink.partition)); + RETURN_IF_ERROR(_partition->init()); + _location = _pool->add(new OlapTableLocationParam(table_sink.location)); + _nodes_info = _pool->add(new PaloNodesInfo(table_sink.nodes_info)); + + return Status::OK; +} + +Status OlapTableSink::prepare(RuntimeState* state) { + RETURN_IF_ERROR(DataSink::prepare(state)); + + _sender_id = state->per_fragment_instance_idx(); + _num_senders = state->num_per_fragment_instances(); + + // profile must add to state's object pool + _profile = state->obj_pool()->add(new RuntimeProfile(_pool, "OlapTableSink")); + _mem_tracker = _pool->add( + new MemTracker(-1, "OlapTableSink", state->instance_mem_tracker())); + + SCOPED_TIMER(_profile->total_time_counter()); + + // Prepare the exprs to run. + RETURN_IF_ERROR(Expr::prepare(_output_expr_ctxs, state, + _input_row_desc, _expr_mem_tracker.get())); + + // get table's tuple descriptor + _output_tuple_desc = state->desc_tbl().get_tuple_descriptor(_tuple_desc_id); + if (_output_tuple_desc == nullptr) { + LOG(WARNING) << "unknown destination tuple descriptor, id=" << _tuple_desc_id; + return Status("unknown destination tuple descriptor"); + } + if (!_output_expr_ctxs.empty()) { + if (_output_expr_ctxs.size() != _output_tuple_desc->slots().size()) { + LOG(WARNING) << "number of exprs is not same with slots, num_exprs=" + << _output_expr_ctxs.size() + << ", num_slots=" << _output_tuple_desc->slots().size(); + return Status("number of exprs is not same with slots"); + } + for (int i = 0; i < _output_expr_ctxs.size(); ++i) { + if (!is_type_compatible(_output_expr_ctxs[i]->root()->type().type, + _output_tuple_desc->slots()[i]->type().type)) { + LOG(WARNING) << "type of exprs is not match slot's, expr_type=" + << _output_expr_ctxs[i]->root()->type().type + << ", slot_type=" << _output_tuple_desc->slots()[i]->type().type + << ", slot_name=" << _output_tuple_desc->slots()[i]->col_name(); + return Status("expr's type is not same with slot's"); + } + } + } + + _output_row_desc = _pool->add(new RowDescriptor(_output_tuple_desc, false)); + _output_batch.reset(new RowBatch(*_output_row_desc, state->batch_size(), _mem_tracker)); + + _max_decimal_val.resize(_output_tuple_desc->slots().size()); + _min_decimal_val.resize(_output_tuple_desc->slots().size()); + // check if need validate batch + for (int i = 0; i < _output_tuple_desc->slots().size(); ++i) { + auto slot = _output_tuple_desc->slots()[i]; + switch (slot->type().type) { + case TYPE_DECIMAL: + _max_decimal_val[i].to_max_decimal(slot->type().precision, slot->type().scale); + _min_decimal_val[i].to_min_decimal(slot->type().precision, slot->type().scale); + _need_validate_data = true; + break; + case TYPE_CHAR: + case TYPE_VARCHAR: + case TYPE_DATE: + case TYPE_DATETIME: + _need_validate_data = true; + break; + default: + break; + } + } + + // add all counter + _input_rows_counter = ADD_COUNTER(_profile, "RowsRead", TUnit::UNIT); + _output_rows_counter = ADD_COUNTER(_profile, "RowsReturned", TUnit::UNIT); + _filtered_rows_counter = ADD_COUNTER(_profile, "RowsFiltered", TUnit::UNIT); + _send_data_timer = ADD_TIMER(_profile, "SendDataTime"); + _convert_batch_timer = ADD_TIMER(_profile, "ConvertBatchTime"); + _validate_data_timer = ADD_TIMER(_profile, "ValidateDataTime"); + _open_timer = ADD_TIMER(_profile, "OpenTime"); + _close_timer = ADD_TIMER(_profile, "CloseTime"); + + // open all channels + auto& partitions = _partition->get_partitions(); + for (int i = 0; i < _schema->indexes().size(); ++i) { + // collect all tablets belong to this rollup + std::vector tablets; + auto index = _schema->indexes()[i]; + for (auto part : partitions) { + for (auto tablet : part->indexes[i].tablets) { + TTabletWithPartition tablet_with_partition; + tablet_with_partition.partition_id = part->id; + tablet_with_partition.tablet_id = tablet; + tablets.emplace_back(std::move(tablet_with_partition)); + } + } + auto channel = _pool->add(new IndexChannel(this, index->index_id, index->schema_hash)); + RETURN_IF_ERROR(channel->init(state, tablets)); + _channels.emplace_back(channel); + } + + return Status::OK; +} + +Status OlapTableSink::open(RuntimeState* state) { + SCOPED_TIMER(_profile->total_time_counter()); + SCOPED_TIMER(_open_timer); + // Prepare the exprs to run. + RETURN_IF_ERROR(Expr::open(_output_expr_ctxs, state)); + + for (auto channel : _channels) { + RETURN_IF_ERROR(channel->open()); + } + return Status::OK; +} + +Status OlapTableSink::send(RuntimeState* state, RowBatch* input_batch) { + SCOPED_TIMER(_profile->total_time_counter()); + _number_input_rows += input_batch->num_rows(); + RowBatch* batch = input_batch; + if (!_output_expr_ctxs.empty()) { + SCOPED_RAW_TIMER(&_convert_batch_ns); + _output_batch->reset(); + _convert_batch(state, input_batch, _output_batch.get()); + batch = _output_batch.get(); + } + + int num_invalid_rows = 0; + if (_need_validate_data) { + SCOPED_RAW_TIMER(&_validate_data_ns); + _filter_bitmap.Reset(batch->num_rows()); + num_invalid_rows = _validate_data(state, batch, &_filter_bitmap); + _number_filtered_rows += num_invalid_rows; + } + SCOPED_RAW_TIMER(&_send_data_ns); + for (int i = 0; i < batch->num_rows(); ++i) { + Tuple* tuple = batch->get_row(i)->get_tuple(0); + if (num_invalid_rows > 0 && _filter_bitmap.Get(i)) { + continue; + } + const OlapTablePartition* partition = nullptr; + uint32_t dist_hash = 0; + if (!_partition->find_tablet(tuple, &partition, &dist_hash)) { + std::stringstream ss; + ss << "no partition for this tuple. tuple=" << print_tuple(tuple, *_output_tuple_desc); +#if BE_TEST + LOG(INFO) << ss.str(); +#else + state->append_error_msg_to_file("", ss.str()); +#endif + _number_filtered_rows++; + continue; + } + _partition_ids.emplace(partition->id); + uint32_t tablet_index = dist_hash % partition->num_buckets; + for (int j = 0; j < partition->indexes.size(); ++j) { + int64_t tablet_id = partition->indexes[j].tablets[tablet_index]; + RETURN_IF_ERROR(_channels[j]->add_row(tuple, tablet_id)); + _number_output_rows++; + } + } + return Status::OK; +} + +Status OlapTableSink::close(RuntimeState* state, Status close_status) { + SCOPED_TIMER(_profile->total_time_counter()); + Status status = close_status; + if (status.ok()) { + { + SCOPED_TIMER(_close_timer); + for (auto channel : _channels) { + status = channel->close(state); + if (!status.ok()) { + LOG(WARNING) << "close channel failed, load_id=" << _load_id + << ", txn_id=" << _txn_id; + } + } + } + COUNTER_SET(_input_rows_counter, _number_input_rows); + COUNTER_SET(_output_rows_counter, _number_output_rows); + COUNTER_SET(_filtered_rows_counter, _number_filtered_rows); + COUNTER_SET(_send_data_timer, _send_data_ns); + COUNTER_SET(_convert_batch_timer, _convert_batch_ns); + COUNTER_SET(_validate_data_timer, _validate_data_ns); + state->update_num_rows_load_filtered(_number_filtered_rows); + } else { + for (auto channel : _channels) { + channel->cancel(); + } + } + Expr::close(_output_expr_ctxs, state); + _output_batch.reset(); + return status; +} + +void OlapTableSink::_convert_batch(RuntimeState* state, RowBatch* input_batch, RowBatch* output_batch) { + DCHECK_GE(output_batch->capacity(), input_batch->num_rows()); + output_batch->add_rows(input_batch->num_rows()); + for (int i = 0; i < input_batch->num_rows(); ++i) { + auto src_row = input_batch->get_row(i); + Tuple* dst_tuple = (Tuple*)output_batch->tuple_data_pool()->allocate( + _output_tuple_desc->byte_size()); + output_batch->get_row(i)->set_tuple(0, dst_tuple); + for (int j = 0; j < _output_expr_ctxs.size(); ++j) { + auto src_val = _output_expr_ctxs[j]->get_value(src_row); + auto slot_desc = _output_tuple_desc->slots()[j]; + if (slot_desc->is_nullable()) { + if (src_val == nullptr) { + dst_tuple->set_null(slot_desc->null_indicator_offset()); + continue; + } else { + dst_tuple->set_not_null(slot_desc->null_indicator_offset()); + } + } else { + if (src_val == nullptr) { + std::stringstream ss; + ss << "null value for not null column, column=" << slot_desc->col_name(); +#if BE_TEST + LOG(INFO) << ss.str(); +#else + state->append_error_msg_to_file("", ss.str()); +#endif + continue; + } + } + void* slot = dst_tuple->get_slot(slot_desc->tuple_offset()); + RawValue::write(src_val, slot, slot_desc->type(), _output_batch->tuple_data_pool()); + } + } + output_batch->commit_rows(input_batch->num_rows()); +} + +int OlapTableSink::_validate_data(RuntimeState* state, RowBatch* batch, Bitmap* filter_bitmap) { + int filtered_rows = 0; + for (int row_no = 0; row_no < batch->num_rows(); ++row_no) { + Tuple* tuple = batch->get_row(row_no)->get_tuple(0); + bool row_valid = true; + for (int i = 0; row_valid && i < _output_tuple_desc->slots().size(); ++i) { + SlotDescriptor* desc = _output_tuple_desc->slots()[i]; + if (tuple->is_null(desc->null_indicator_offset())) { + continue; + } + void* slot = tuple->get_slot(desc->tuple_offset()); + switch (desc->type().type) { + case TYPE_CHAR: + case TYPE_VARCHAR: { + // Fixed length string + StringValue* str_val = (StringValue*)slot; + if (str_val->len > desc->type().len) { + std::stringstream ss; + ss << "the length of input is too long than schema. " + << "input_str: [" << std::string(str_val->ptr, str_val->len) << "] " + << "schema length: " << desc->type().len << "; " + << "actual length: " << str_val->len << "; "; +#if BE_TEST + LOG(INFO) << ss.str(); +#else + state->append_error_msg_to_file("", ss.str()); +#endif + + filtered_rows++; + row_valid = false; + filter_bitmap->Set(row_no, true); + continue; + } + // padding 0 to CHAR field + if (desc->type().type == TYPE_CHAR + && str_val->len < desc->type().len) { + auto new_ptr = (char*)batch->tuple_data_pool()->allocate(desc->type().len); + memcpy(new_ptr, str_val->ptr, str_val->len); + memset(new_ptr + str_val->len, 0, desc->type().len - str_val->len); + + str_val->ptr = new_ptr; + str_val->len = desc->type().len; + } + break; + } + case TYPE_DECIMAL: { + DecimalValue* dec_val = (DecimalValue*)slot; + if (dec_val->scale() > desc->type().scale) { + int code = dec_val->round(dec_val, desc->type().scale, HALF_UP); + if (code != E_DEC_OK) { + std::stringstream ss; + ss << "round one decimal failed.value=" << dec_val->to_string(); +#if BE_TEST + LOG(INFO) << ss.str(); +#else + state->append_error_msg_to_file("", ss.str()); +#endif + + filtered_rows++; + row_valid = false; + filter_bitmap->Set(row_no, true); + continue; + } + } + if (*dec_val > _max_decimal_val[i] || *dec_val < _min_decimal_val[i]) { + std::stringstream ss; + ss << "decimal value is not valid for defination, column=" << desc->col_name() + << ", value=" << dec_val->to_string() + << ", precision=" << desc->type().precision + << ", scale=" << desc->type().scale; +#if BE_TEST + LOG(INFO) << ss.str(); +#else + state->append_error_msg_to_file("", ss.str()); +#endif + filtered_rows++; + row_valid = false; + filter_bitmap->Set(row_no, true); + continue; + } + break; + } + case TYPE_DATE: + case TYPE_DATETIME: { + static DateTimeValue s_min_value = DateTimeValue(19000101000000UL); + // static DateTimeValue s_max_value = DateTimeValue(99991231235959UL); + DateTimeValue* date_val = (DateTimeValue*)slot; + if (*date_val < s_min_value) { + std::stringstream ss; + ss << "datetime value is not valid, column=" << desc->col_name() + << ", value=" << date_val->debug_string(); +#if BE_TEST + LOG(INFO) << ss.str(); +#else + state->append_error_msg_to_file("", ss.str()); +#endif + filtered_rows++; + row_valid = false; + filter_bitmap->Set(row_no, true); + continue; + } + } + default: + break; + } + } + } + return filtered_rows; +} + +} +} diff --git a/be/src/exec/olap_table_sink.h b/be/src/exec/olap_table_sink.h new file mode 100644 index 0000000000..16339b74e3 --- /dev/null +++ b/be/src/exec/olap_table_sink.h @@ -0,0 +1,248 @@ +// Copyright (c) 2017, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 "common/status.h" +#include "common/object_pool.h" +#include "exec/data_sink.h" +#include "exec/olap_table_info.h" +#include "gen_cpp/Types_types.h" +#include "gen_cpp/internal_service.pb.h" +#include "util/bitmap.h" +#include "util/thrift_util.h" +#include "util/ref_count_closure.h" + +namespace palo { + +class Bitmap; +class MemTracker; +class RuntimeProfile; +class RowDescriptor; +class Tuple; +class TupleDescriptor; +class ExprContext; +class TExpr; + +namespace stream_load { + +class OlapTableSink; + +class NodeChannel { +public: + NodeChannel(OlapTableSink* parent, int64_t index_id, int64_t node_id, int32_t schema_hash); + ~NodeChannel() noexcept; + + // called before open, used to add tablet loacted in this backend + void add_tablet(const TTabletWithPartition& tablet) { + _all_tablets.emplace_back(tablet); + } + + Status init(RuntimeState* state); + + // we use open/open_wait to parallel + void open(); + Status open_wait(); + + Status add_row(Tuple* tuple, int64_t tablet_id); + + Status close(RuntimeState* state); + Status close_wait(RuntimeState* state); + + void cancel(); + + int64_t node_id() const { return _node_id; } + + void set_failed() { _already_failed = true; } + bool already_failed() const { return _already_failed; } + const NodeInfo* node_info() const { return _node_info; } + +private: + Status _send_cur_batch(bool eos = false); + // wait inflight packet finish, return error if inflight packet return failed + Status _wait_in_flight_packet(); + + Status _close(RuntimeState* state); + +private: + OlapTableSink* _parent = nullptr; + int64_t _index_id = -1; + int64_t _node_id = -1; + int32_t _schema_hash = 0; + + TupleDescriptor* _tuple_desc = nullptr; + const NodeInfo* _node_info = nullptr; + + bool _already_failed = false; + bool _has_in_flight_packet = false; + int _rpc_timeout_ms = 50000; + int64_t _next_packet_seq = 0; + + std::unique_ptr _batch; + PInternalService_Stub* _stub = nullptr; + RefCountClosure* _open_closure = nullptr; + RefCountClosure* _add_batch_closure = nullptr; + + std::vector _all_tablets; + PTabletWriterAddBatchRequest _add_batch_request; +}; + +class IndexChannel { +public: + IndexChannel(OlapTableSink* parent, int64_t index_id, int32_t schema_hash) + : _parent(parent), _index_id(index_id), + _schema_hash(schema_hash) { + } + ~IndexChannel(); + + Status init(RuntimeState* state, + const std::vector& tablets); + Status open(); + Status add_row(Tuple* tuple, int64_t tablet_id); + + Status close(RuntimeState* state); + + void cancel(); + +private: + // return true if this load can't success. + bool _handle_failed_node(NodeChannel* channel); + +private: + OlapTableSink* _parent; + int64_t _index_id; + int32_t _schema_hash; + int _num_failed_channels = 0; + + // BeId -> channel + std::unordered_map _node_channels; + // from tablet_id to backend channel + std::unordered_map> _channels_by_tablet; +}; + +// write data to Olap Table. +// this class distributed data according to +class OlapTableSink : public DataSink { +public: + // Construct from thrift struct which is generated by FE. + OlapTableSink(ObjectPool* pool, + const RowDescriptor& row_desc, + const std::vector& texprs, + Status* status); + ~OlapTableSink() override; + + Status init(const TDataSink& sink) override; + + Status prepare(RuntimeState* state) override; + + Status open(RuntimeState* state) override; + + Status send(RuntimeState* state, RowBatch* batch) override; + + Status close(RuntimeState* state, Status close_status) override; + + // Returns the runtime profile for the sink. + RuntimeProfile* profile() override { + return _profile; + } + +private: + // convert input batch to output batch which will be loaded into OLAP table. + // this is only used in insert statement. + void _convert_batch(RuntimeState* state, RowBatch* input_batch, RowBatch* output_batch); + + // make input data valid for OLAP table + // return number of invalid/filtered rows. + // invalid row number is set in Bitmap + int _validate_data(RuntimeState* state, RowBatch* batch, Bitmap* filter_bitmap); + +private: + friend class NodeChannel; + friend class IndexChannel; + + ObjectPool* _pool; + const RowDescriptor& _input_row_desc; + + // unique load id + PUniqueId _load_id; + int64_t _txn_id = -1; + int64_t _db_id = -1; + int64_t _table_id = -1; + int _num_repicas = -1; + bool _need_gen_rollup = true; + std::string _db_name; + std::string _table_name; + int _tuple_desc_id = -1; + + // this is tuple descriptor of destination OLAP table + TupleDescriptor* _output_tuple_desc = nullptr; + RowDescriptor* _output_row_desc = nullptr; + std::vector _output_expr_ctxs; + std::unique_ptr _output_batch; + + bool _need_validate_data = false; + + // number of senders used to insert into OlapTable, if we only support single node insert, + // all data from select should collectted and then send to OlapTable. + // To support multiple senders, we maintain a channel for each sender. + int _sender_id = -1; + int _num_senders = -1; + + // TODO(zc): think about cache this data + std::shared_ptr _schema; + OlapTablePartitionParam* _partition = nullptr; + OlapTableLocationParam* _location = nullptr; + PaloNodesInfo* _nodes_info = nullptr; + + RuntimeProfile* _profile = nullptr; + MemTracker* _mem_tracker = nullptr; + + std::set _partition_ids; + + Bitmap _filter_bitmap; + + // index_channel + std::vector _channels; + + std::vector _max_decimal_val; + std::vector _min_decimal_val; + + // Stats for this + int64_t _convert_batch_ns = 0; + int64_t _validate_data_ns = 0; + int64_t _send_data_ns = 0; + int64_t _number_input_rows = 0; + int64_t _number_output_rows = 0; + int64_t _number_filtered_rows = 0; + + RuntimeProfile::Counter* _input_rows_counter = nullptr; + RuntimeProfile::Counter* _output_rows_counter = nullptr; + RuntimeProfile::Counter* _filtered_rows_counter = nullptr; + RuntimeProfile::Counter* _send_data_timer = nullptr; + RuntimeProfile::Counter* _convert_batch_timer = nullptr; + RuntimeProfile::Counter* _validate_data_timer = nullptr; + RuntimeProfile::Counter* _open_timer = nullptr; + RuntimeProfile::Counter* _close_timer = nullptr; +}; + +} +} diff --git a/be/src/exprs/es_functions.cpp b/be/src/exprs/es_functions.cpp new file mode 100644 index 0000000000..79df9c6b3d --- /dev/null +++ b/be/src/exprs/es_functions.cpp @@ -0,0 +1,33 @@ +// Copyright (c) 2017, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 "exprs/es_functions.h" + +#include "exprs/expr.h" +#include "exprs/anyval_util.h" +#include "util/debug_util.h" +#include "runtime/tuple_row.h" + +namespace palo { + +void ESFunctions::init() { +} + +BooleanVal ESFunctions::match(FunctionContext* ctx, const StringVal& col, + const StringVal& condition) { + return BooleanVal(true); +} + +} // palo diff --git a/be/src/exprs/es_functions.h b/be/src/exprs/es_functions.h new file mode 100644 index 0000000000..19a18bf434 --- /dev/null +++ b/be/src/exprs/es_functions.h @@ -0,0 +1,40 @@ +// Copyright (c) 2017, Baidu.com, Inc. All Rights Reserved + +// Licensed 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. + +#ifndef BDG_PALO_BE_SRC_QUERY_EXPRS_ES_FUNCTIONS_H +#define BDG_PALO_BE_SRC_QUERY_EXPRS_ES_FUNCTIONS_H + +#include "udf/udf.h" + +namespace palo { + +class Expr; +class OpcodeRegistry; +class TupleRow; + +class ESFunctions { +public: + static void init(); + + // used to push down query conditions to es. + static palo_udf::BooleanVal match( + palo_udf::FunctionContext* ctx, const palo_udf::StringVal& col, + const palo_udf::StringVal& condition); + +}; + +} + +#endif diff --git a/be/src/http/action/meta_action.cpp b/be/src/http/action/meta_action.cpp new file mode 100644 index 0000000000..bdd4b5d589 --- /dev/null +++ b/be/src/http/action/meta_action.cpp @@ -0,0 +1,78 @@ +// Copyright (c) 2018, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 "http/action/meta_action.h" + +#include +#include + +#include "http/http_channel.h" +#include "http/http_request.h" +#include "http/http_response.h" +#include "http/http_headers.h" +#include "http/http_status.h" + +#include "olap/olap_header_manager.h" +#include "olap/olap_engine.h" +#include "olap/olap_define.h" +#include "olap/olap_header.h" +#include "olap/olap_table.h" +#include "common/logging.h" +#include "util/json_util.h" + +namespace palo { + +const static std::string HEADER_JSON = "application/json"; + +Status MetaAction::_handle_header(HttpRequest *req, std::string* json_header) { + req->add_output_header(HttpHeaders::CONTENT_TYPE, HEADER_JSON.c_str()); + std::string req_tablet_id = req->param(TABLET_ID_KEY); + std::string req_schema_hash = req->param(TABLET_SCHEMA_HASH_KEY); + if (req_tablet_id == "" || req_schema_hash == "") { + LOG(WARNING) << "invalid argument.tablet_id:" << req_tablet_id + << ", schema_hash:" << req_schema_hash; + return Status("invalid arguments"); + } + uint64_t tablet_id = std::stoull(req_tablet_id); + uint32_t schema_hash = std::stoul(req_schema_hash); + OLAPTablePtr olap_table = OLAPEngine::get_instance()->get_table(tablet_id, schema_hash); + if (olap_table == nullptr) { + LOG(WARNING) << "no tablet for tablet_id:" << tablet_id << " schema hash:" << schema_hash; + return Status("no tablet exist"); + } + OLAPStatus s = OlapHeaderManager::get_json_header(olap_table->store(), tablet_id, schema_hash, json_header); + if (s == OLAP_ERR_META_KEY_NOT_FOUND) { + return Status("no header exist"); + } else if (s != OLAP_SUCCESS) { + return Status("backend error"); + } + return Status::OK; +} + +void MetaAction::handle(HttpRequest *req) { + if (_meta_type == META_TYPE::HEADER) { + std::string json_header; + Status status = _handle_header(req, &json_header); + std::string status_result = to_json(status); + LOG(INFO) << "handle request result:" << status_result; + if (status.ok()) { + HttpChannel::send_reply(req, HttpStatus::OK, json_header); + } else { + HttpChannel::send_reply(req, HttpStatus::INTERNAL_SERVER_ERROR, status_result); + } + } +} + +} // end namespace palo diff --git a/be/src/http/action/meta_action.h b/be/src/http/action/meta_action.h new file mode 100644 index 0000000000..4b497d3324 --- /dev/null +++ b/be/src/http/action/meta_action.h @@ -0,0 +1,48 @@ +// Copyright (c) 2018, Baidu.com, Inc. All Rights Reserved + +// Licensed 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. + +#ifndef BDG_PALO_BE_SRC_HTTP_ACTION_META_ACTION_H +#define BDG_PALO_BE_SRC_HTTP_ACTION_META_ACTION_H + +#include "http/http_handler.h" +#include "common/status.h" + +namespace palo { + +class ExecEnv; + +enum META_TYPE { + HEADER = 1, +}; + +// Get Meta Info +class MetaAction : public HttpHandler { +public: + MetaAction(META_TYPE meta_type) : _meta_type(meta_type) {} + + virtual ~MetaAction() {} + + void handle(HttpRequest *req) override; + +private: + Status _handle_header(HttpRequest *req, std::string* json_header); + +private: + META_TYPE _meta_type; +}; + +} // end namespace palo + +#endif // BDG_PALO_BE_SRC_HTTP_ACTION_META_ACTION_H diff --git a/be/src/http/action/restore_tablet_action.cpp b/be/src/http/action/restore_tablet_action.cpp new file mode 100644 index 0000000000..5890685107 --- /dev/null +++ b/be/src/http/action/restore_tablet_action.cpp @@ -0,0 +1,294 @@ +// Copyright (c) 2018, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 "http/action/restore_tablet_action.h" + +#include +#include +#include + +#include "boost/lexical_cast.hpp" + +#include "agent/cgroups_mgr.h" +#include "http/http_channel.h" +#include "http/http_headers.h" +#include "http/http_request.h" +#include "http/http_response.h" +#include "http/http_status.h" +#include "util/file_utils.h" +#include "olap/utils.h" +#include "olap/olap_header.h" +#include "util/json_util.h" +#include "olap/olap_define.h" +#include "olap/olap_engine.h" +#include "olap/store.h" +#include "runtime/exec_env.h" + +using boost::filesystem::path; + +namespace palo { + +const std::string TABLET_ID = "tablet_id"; +const std::string SCHEMA_HASH = "schema_hash"; + +RestoreTabletAction::RestoreTabletAction(ExecEnv* exec_env) : _exec_env(exec_env) { +} + +void RestoreTabletAction::handle(HttpRequest *req) { + LOG(INFO) << "accept one request " << req->debug_string(); + // add tid to cgroup in order to limit read bandwidth + CgroupsMgr::apply_system_cgroup(); + Status status = _handle(req); + std::string result = to_json(status); + LOG(INFO) << "handle request result:" << result; + if (status.ok()) { + HttpChannel::send_reply(req, HttpStatus::OK, result); + } else { + HttpChannel::send_reply(req, HttpStatus::INTERNAL_SERVER_ERROR, result); + } +} + +Status RestoreTabletAction::_handle(HttpRequest *req) { + // Get tablet id + const std::string& tablet_id_str = req->param(TABLET_ID); + if (tablet_id_str.empty()) { + std::string error_msg = std::string( + "parameter " + TABLET_ID + " not specified in url."); + return Status(error_msg); + } + + // Get schema hash + const std::string& schema_hash_str = req->param(SCHEMA_HASH); + if (schema_hash_str.empty()) { + std::string error_msg = std::string( + "parameter " + SCHEMA_HASH + " not specified in url."); + return Status(error_msg); + } + + // valid str format + int64_t tablet_id = std::atol(tablet_id_str.c_str()); + int32_t schema_hash = std::atoi(schema_hash_str.c_str()); + LOG(INFO) << "get restore tablet action request: " << tablet_id << "-" << schema_hash; + + OLAPTablePtr tablet = + OLAPEngine::get_instance()->get_table(tablet_id, schema_hash); + if (tablet.get() != nullptr) { + LOG(WARNING) << "find tablet. tablet_id=" << tablet_id << " schema_hash=" << schema_hash; + return Status("tablet already exists, can not restore."); + } + std::string key = std::to_string(tablet_id) + "_" + std::to_string(schema_hash); + { + // check tablet_id + schema_hash already is restoring + std::lock_guard l(_tablet_restore_lock); + if (_tablet_path_map.find(key) != _tablet_path_map.end()) { + LOG(INFO) << "tablet_id:" << tablet_id << " schema_hash:" << schema_hash << " is restoring."; + return Status("tablet is already restoring"); + } else { + // set key in map and initialize value as "" + _tablet_path_map[key] = ""; + LOG(INFO) << "start to restore tablet_id:" << tablet_id << " schema_hash:" << schema_hash; + } + } + Status status = _restore(key, tablet_id, schema_hash); + _clear_key(key); + LOG(INFO) << "deal with restore tablet request finished! tablet id: " << tablet_id << "-" << schema_hash; + return status; +} + +Status RestoreTabletAction::_reload_tablet( + const std::string& key, const std::string& shard_path, int64_t tablet_id, int32_t schema_hash) { + TCloneReq clone_req; + clone_req.__set_tablet_id(tablet_id); + clone_req.__set_schema_hash(schema_hash); + OLAPStatus res = OLAPStatus::OLAP_SUCCESS; + res = _exec_env->olap_engine()->load_header(shard_path, clone_req); + if (res != OLAPStatus::OLAP_SUCCESS) { + LOG(WARNING) << "load header failed. status: " << res + << ", signature: " << tablet_id; + // remove tablet data path in data path + // path: /roo_path/data/shard/tablet_id + std::string tablet_path = shard_path + "/" + std::to_string(tablet_id); + LOG(INFO) << "remove tablet_path:" << tablet_path; + Status s = FileUtils::remove_all(tablet_path); + if (!s.ok()) { + LOG(WARNING) << "remove invalid tablet schema hash path:" << tablet_path << " failed"; + } + return Status("command executor load header failed"); + } else { + LOG(INFO) << "load header success. status: " << res + << ", signature: " << tablet_id; + // remove tablet data path in trash + // path: /root_path/trash/time_label, because only one tablet path under time_label + std::string trash_tablet_schema_hash_dir = ""; + + { + // get tablet path in trash + std::lock_guard l(_tablet_restore_lock); + trash_tablet_schema_hash_dir = _tablet_path_map[key]; + } + + boost::filesystem::path trash_tablet_schema_hash_path(trash_tablet_schema_hash_dir); + boost::filesystem::path time_label_path = trash_tablet_schema_hash_path.parent_path().parent_path(); + LOG(INFO) << "remove time label path:" << time_label_path.string(); + Status s = FileUtils::remove_all(time_label_path.string()); + if (!s.ok()) { + LOG(WARNING) << "remove time label path:" << time_label_path.string() << " failed"; + } + return Status::OK; + } +} + +Status RestoreTabletAction::_restore(const std::string& key, int64_t tablet_id, int32_t schema_hash) { + // get latest tablet path in trash + std::string latest_tablet_path; + bool ret = _get_latest_tablet_path_from_trash(tablet_id, schema_hash, &latest_tablet_path); + if (!ret) { + LOG(WARNING) << "can not find tablet:" << tablet_id + << ", schema hash:" << schema_hash; + return Status("can find tablet path in trash"); + } + LOG(INFO) << "tablet path in trash:" << latest_tablet_path; + std::string original_header_path = latest_tablet_path + "/" + std::to_string(tablet_id) +".hdr"; + OLAPHeader header(original_header_path); + OLAPStatus load_status = header.load_and_init(); + if (load_status != OLAP_SUCCESS) { + LOG(WARNING) << "header load and init error, header path:" << original_header_path; + return Status("load header failed"); + } + // latest_tablet_path: /root_path/trash/time_label/tablet_id/schema_hash + { + // update _tablet_path_map to save tablet path in trash for delete when succeed + std::lock_guard l(_tablet_restore_lock); + _tablet_path_map[key] = latest_tablet_path; + } + + std::string root_path = OlapStore::get_root_path_from_schema_hash_path_in_trash(latest_tablet_path); + OlapStore* store = OLAPEngine::get_instance()->get_store(root_path); + std::string restore_schema_hash_path = store->get_tablet_schema_hash_path_from_header(&header); + Status s = FileUtils::create_dir(restore_schema_hash_path); + if (!s.ok()) { + LOG(WARNING) << "create tablet path failed:" << restore_schema_hash_path; + return s; + } + // create hard link for files in /root_path/data/shard/tablet_id/schema_hash + std::vector files; + s = FileUtils::scan_dir(latest_tablet_path, &files); + if (!s.ok()) { + LOG(WARNING) << "scan dir failed:" << latest_tablet_path; + return s; + } + for (auto& file : files) { + std::string from = latest_tablet_path + "/" + file; + std::string to = restore_schema_hash_path + "/" + file; + int link_ret = link(from.c_str(), to.c_str()); + if (link_ret != 0) { + LOG(WARNING) << "link from:" << from + << " to:" << to << " failed, link ret:" << link_ret; + std::string restore_tablet_path = store->get_tablet_path_from_header(&header); + LOG(WARNING) << "remove tablet_path:" << restore_tablet_path; + Status s = FileUtils::remove_all(restore_tablet_path); + if (!s.ok()) { + LOG(WARNING) << "remove invalid tablet path:" << restore_tablet_path << " failed"; + } + return Status("create link path failed"); + } + } + std::string restore_shard_path = store->get_shard_path_from_header(std::to_string(header.shard())); + Status status = _reload_tablet(key, restore_shard_path, tablet_id, schema_hash); + return status; +} + +bool RestoreTabletAction::_get_latest_tablet_path_from_trash( + int64_t tablet_id, int32_t schema_hash, std::string* path) { + std::vector tablet_paths; + std::vector stores = OLAPEngine::get_instance()->get_stores(); + for (auto& store : stores) { + store->find_tablet_in_trash(tablet_id, &tablet_paths); + } + if (tablet_paths.empty()) { + LOG(WARNING) << "can not find tablet_id:" << tablet_id << ", schema_hash:" << schema_hash; + return false; + } + std::vector schema_hash_paths; + for (auto& tablet_path : tablet_paths) { + std::string schema_hash_path = tablet_path + "/" + std::to_string(schema_hash); + bool exist = FileUtils::check_exist(schema_hash_path); + if (exist) { + schema_hash_paths.emplace_back(std::move(schema_hash_path)); + } + } + if (schema_hash_paths.size() == 0) { + LOG(WARNING) << "can not find tablet_id:" << tablet_id + << ", schema_hash:" << schema_hash;; + return false; + } else if (schema_hash_paths.size() == 1) { + *path = schema_hash_paths[0]; + return true; + } else { + int start_index = 0; + uint64_t max_timestamp = 0; + uint64_t max_counter = 0; + *path = schema_hash_paths[start_index]; + if (!_get_timestamp_and_count_from_schema_hash_path( + schema_hash_paths[start_index], &max_timestamp, &max_counter)) { + LOG(WARNING) << "schema hash paths are invalid, path:" << schema_hash_paths[start_index]; + return false; + } + // find latest path + for (int i = start_index + 1; i < schema_hash_paths.size(); i++) { + uint64_t current_timestamp = 0; + uint64_t current_counter = 0; + if (!_get_timestamp_and_count_from_schema_hash_path( + schema_hash_paths[i], ¤t_timestamp, ¤t_counter)) { + LOG(WARNING) << "schema hash path:" << schema_hash_paths[i] << " is invalid"; + continue; + } + if (current_timestamp > max_timestamp) { + *path = schema_hash_paths[i]; + max_timestamp = current_timestamp; + max_counter = current_counter; + } else if (current_timestamp == max_timestamp) { + if (current_counter > max_counter) { + *path = schema_hash_paths[i]; + max_counter = current_counter; + } + } + } + return true; + } +} + +bool RestoreTabletAction::_get_timestamp_and_count_from_schema_hash_path( + const std::string& schema_hash_dir, uint64_t* timestamp, uint64_t* counter) { + path schema_hash_path(schema_hash_dir); + path time_label_path = schema_hash_path.parent_path().parent_path(); + std::string time_label = time_label_path.filename().string(); + std::vector parts; + palo::split_string(time_label, '.', &parts); + if (parts.size() != 2) { + LOG(WARNING) << "invalid time label:" << time_label; + return false; + } + *timestamp = std::stoul(parts[0]); + *counter = std::stoul(parts[1]); + return true; +} + +void RestoreTabletAction::_clear_key(const std::string& key) { + std::lock_guard l(_tablet_restore_lock); + _tablet_path_map.erase(key); +} + +} // end namespace palo diff --git a/be/src/http/action/restore_tablet_action.h b/be/src/http/action/restore_tablet_action.h new file mode 100644 index 0000000000..e997ef45bd --- /dev/null +++ b/be/src/http/action/restore_tablet_action.h @@ -0,0 +1,62 @@ +// Copyright (c) 2018, Baidu.com, Inc. All Rights Reserved + +// Licensed 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. + +#ifndef BDG_PALO_BE_SRC_HTTP_RESTORE_TABLET_ACTION_H +#define BDG_PALO_BE_SRC_HTTP_RESTORE_TABLET_ACTION_H + +#include +#include +#include + +#include "http/http_handler.h" +#include "gen_cpp/AgentService_types.h" +#include "common/status.h" + +namespace palo { + +class ExecEnv; + +class RestoreTabletAction : public HttpHandler { +public: + RestoreTabletAction(ExecEnv* exec_env); + + virtual ~RestoreTabletAction() { } + + void handle(HttpRequest *req) override; +private: + Status _handle(HttpRequest *req); + + Status _restore(const std::string& key, int64_t tablet_id, int32_t schema_hash); + + Status _reload_tablet(const std::string& key, const std::string& shard_path, int64_t tablet_id, int32_t schema_hash); + + bool _get_latest_tablet_path_from_trash(int64_t tablet_id, int32_t schema_hash, std::string* path); + + bool _get_timestamp_and_count_from_schema_hash_path( + const std::string& time_label, uint64_t* timestamp, uint64_t* counter); + + void _clear_key(const std::string& key); + +private: + ExecEnv* _exec_env; + std::mutex _tablet_restore_lock; + // store all current restoring tablet_id + schema_hash + // key: tablet_id + schema_hash + // value: "" or tablet path in trash + std::map _tablet_path_map; +}; // end class RestoreTabletAction + +} // end namespace palo +#endif // BDG_PALO_BE_SRC_HTTP_RESTORE_TABLET_ACTION_H diff --git a/be/src/http/action/stream_load.cpp b/be/src/http/action/stream_load.cpp new file mode 100644 index 0000000000..2ee1d25c6d --- /dev/null +++ b/be/src/http/action/stream_load.cpp @@ -0,0 +1,606 @@ +// Copyright (c) 2017, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 "http/action/stream_load.h" + +#include +#include +#include + +// use string iequal +#include +#include +#include +#include +#include + +#include "common/logging.h" +#include "exec/schema_scanner/frontend_helper.h" +#include "gen_cpp/FrontendService.h" +#include "gen_cpp/FrontendService_types.h" +#include "gen_cpp/HeartbeatService_types.h" +#include "http/http_channel.h" +#include "http/http_common.h" +#include "http/http_request.h" +#include "http/http_response.h" +#include "http/http_headers.h" +#include "http/utils.h" +#include "runtime/client_cache.h" +#include "runtime/exec_env.h" +#include "runtime/fragment_mgr.h" +#include "runtime/load_path_mgr.h" +#include "runtime/plan_fragment_executor.h" +#include "runtime/stream_load_pipe.h" +#include "runtime/load_stream_mgr.h" +#include "util/byte_buffer.h" +#include "util/debug_util.h" +#include "util/json_util.h" +#include "util/metrics.h" +#include "util/palo_metrics.h" +#include "util/time.h" +#include "util/uid_util.h" + +namespace palo { + +IntCounter k_streaming_load_requests_total; +IntCounter k_streaming_load_bytes; +IntCounter k_streaming_load_duration_ms; +static IntGauge k_streaming_load_current_processing; + +#ifdef BE_TEST +TLoadTxnBeginResult k_stream_load_begin_result; +TLoadTxnCommitResult k_stream_load_commit_result; +TLoadTxnRollbackResult k_stream_load_rollback_result; +TStreamLoadPutResult k_stream_load_put_result; +Status k_stream_load_plan_status; +#endif + +static TFileFormatType::type parse_format(const std::string& format_str) { + if (boost::iequals(format_str, "CSV")) { + return TFileFormatType::FORMAT_CSV_PLAIN; + } + return TFileFormatType::FORMAT_UNKNOWN; +} + +static bool is_format_support_streaming(TFileFormatType::type format) { + switch (format) { + case TFileFormatType::FORMAT_CSV_PLAIN: + return true; + default: + return false; + } +} + +// stream load context +struct StreamLoadContext { + StreamLoadContext(StreamLoadAction* action_) : action(action_), _refs(0) { + start_nanos = MonotonicNanos(); + } + + ~StreamLoadContext(); + + StreamLoadAction* action; + // id for each load + UniqueId id; + + std::string db; + std::string table; + // load label, used to identify + std::string label; + + std::string user_ip; + + HttpAuthInfo auth; + + // only used to check if we receive whole body + size_t body_bytes = 0; + size_t receive_bytes = 0; + + int64_t txn_id = -1; + + bool need_rollback = false; + // when use_streaming is true, we use stream_pipe to send source data, + // otherwise we save source data to file first, then process it. + bool use_streaming = false; + TFileFormatType::type format = TFileFormatType::FORMAT_CSV_PLAIN; + + std::shared_ptr body_sink; + + TStreamLoadPutResult put_result; + double max_filter_ratio = 0.0; + std::vector commit_infos; + + std::promise promise; + std::future future = promise.get_future(); + + Status status; + + int64_t number_loaded_rows = 0; + int64_t number_filtered_rows = 0; + int64_t start_nanos = 0; + int64_t load_cost_nanos = 0; + std::string error_url; + + std::string to_json() const; + + void ref() { _refs.fetch_add(1); } + // If unref() returns true, this object should be delete + bool unref() { return _refs.fetch_sub(1) == 1; } +private: + std::atomic _refs; +}; + +StreamLoadContext::~StreamLoadContext() { + if (need_rollback) { + action->rollback(this); + need_rollback = false; + } +} + +std::string StreamLoadContext::to_json() const { + rapidjson::StringBuffer s; + rapidjson::PrettyWriter writer(s); + + writer.StartObject(); + // status + writer.Key("Status"); + switch (status.code()) { + case TStatusCode::OK: + writer.String("Success"); + break; + case TStatusCode::PUBLISH_TIMEOUT: + writer.String("Publish Timeout"); + break; + case TStatusCode::LABEL_ALREADY_EXISTS: + writer.String("Label Already Exists"); + break; + default: + writer.String("Fail"); + break; + } + // msg + writer.Key("Message"); + if (status.ok()) { + writer.String("OK"); + } else { + writer.String(status.get_error_msg().c_str()); + } + // number_load_rows + writer.Key("NumberLoadedRows"); + writer.Int64(number_loaded_rows); + writer.Key("NumberFilteredRows"); + writer.Int64(number_filtered_rows); + writer.Key("LoadBytes"); + writer.Int64(receive_bytes); + writer.Key("LoadTimeMs"); + writer.Int64(load_cost_nanos / 1000000); + if (!error_url.empty()) { + writer.Key("ErrorURL"); + writer.String(error_url.c_str()); + } + writer.EndObject(); + return s.GetString(); +} + +StreamLoadAction::StreamLoadAction(ExecEnv* exec_env) : _exec_env(exec_env) { + PaloMetrics::metrics()->register_metric("streaming_load_requests_total", + &k_streaming_load_requests_total); + PaloMetrics::metrics()->register_metric("streaming_load_bytes", + &k_streaming_load_bytes); + PaloMetrics::metrics()->register_metric("streaming_load_duration_ms", + &k_streaming_load_duration_ms); + PaloMetrics::metrics()->register_metric("streaming_load_current_processing", + &k_streaming_load_current_processing); +} + +StreamLoadAction::~StreamLoadAction() { +} + +void StreamLoadAction::handle(HttpRequest* req) { + StreamLoadContext* ctx = (StreamLoadContext*) req->handler_ctx(); + if (ctx == nullptr) { + return; + } + + // status already set to fail + if (ctx->status.ok()) { + ctx->status = _handle(ctx); + if (!ctx->status.ok()) { + LOG(WARNING) << "handle streaming load failed, id=" << ctx->id + << ", errmsg=" << ctx->status.get_error_msg(); + } + } + ctx->load_cost_nanos = MonotonicNanos() - ctx->start_nanos; + + if (!ctx->status.ok()) { + if (ctx->need_rollback) { + rollback(ctx); + ctx->need_rollback = false; + } + if (ctx->body_sink.get() != nullptr) { + ctx->body_sink->cancel(); + } + } + + auto str = ctx->to_json(); + HttpChannel::send_reply(req, str); + + // update statstics + k_streaming_load_requests_total.increment(1); + k_streaming_load_duration_ms.increment(ctx->load_cost_nanos / 1000000); + k_streaming_load_bytes.increment(ctx->receive_bytes); + k_streaming_load_current_processing.increment(-1); +} + +Status StreamLoadAction::_handle(StreamLoadContext* ctx) { + if (ctx->body_bytes > 0 && ctx->receive_bytes != ctx->body_bytes) { + LOG(WARNING) << "recevie body don't equal with body bytes, body_bytes=" + << ctx->body_bytes << ", receive_bytes=" << ctx->receive_bytes + << ", id=" << ctx->id; + return Status("receive body dont't equal with body bytes"); + } + if (!ctx->use_streaming) { + // if we use non-streaming, we need to close file first, + // then _execute_plan_fragment here + // this will close file + ctx->body_sink.reset(); + RETURN_IF_ERROR(_execute_plan_fragment(ctx)); + } else { + RETURN_IF_ERROR(ctx->body_sink->finish()); + } + + // wait stream load finish + RETURN_IF_ERROR(ctx->future.get()); + + // If put file succeess we need commit this load + TLoadTxnCommitRequest request; + set_http_auth(&request, ctx->auth); + request.db = ctx->db; + request.tbl = ctx->table; + request.txnId = ctx->txn_id; + request.sync = true; + request.commitInfos = std::move(ctx->commit_infos); + request.__isset.commitInfos = true; + + TNetworkAddress master_addr = _exec_env->master_info()->network_address; + TLoadTxnCommitResult result; +#ifndef BE_TEST + RETURN_IF_ERROR(FrontendHelper::rpc( + master_addr.hostname, master_addr.port, + [&request, &result] (FrontendServiceConnection& client) { + client->loadTxnCommit(result, request); + }, 10000)); +#else + result = k_stream_load_commit_result; +#endif + // Return if this transaction is committed successful; otherwise, we need try to + // rollback this transaction + Status status(result.status); + if (!status.ok()) { + LOG(WARNING) << "commit transaction failed, id=" << ctx->id + << ", errmsg=" << status.get_error_msg(); + return status; + } + // commit success, set need_rollback to false + ctx->need_rollback = false; + + return Status::OK; +} + +int StreamLoadAction::on_header(HttpRequest* req) { + k_streaming_load_current_processing.increment(1); + + StreamLoadContext* ctx = new StreamLoadContext(this); + ctx->ref(); + req->set_handler_ctx(ctx); + + ctx->db = req->param(HTTP_DB_KEY); + ctx->table = req->param(HTTP_TABLE_KEY); + ctx->label = req->header(HTTP_LABEL_KEY); + if (ctx->label.empty()) { + ctx->label = generate_uuid_string(); + } + + LOG(INFO) << "new income streaming load request, id=" << ctx->id + << ", db=" << ctx->db << ", table=" << ctx->table << ", label=" << ctx->label; + + auto st = _on_header(req, ctx); + if (!st.ok()) { + ctx->status = st; + if (ctx->need_rollback) { + rollback(ctx); + ctx->need_rollback = false; + } + if (ctx->body_sink.get() != nullptr) { + ctx->body_sink->cancel(); + } + auto str = ctx->to_json(); + HttpChannel::send_reply(req, str); + return -1; + } + return 0; +} + +Status StreamLoadAction::_on_header(HttpRequest* http_req, StreamLoadContext* ctx) { + // auth information + if (!parse_basic_auth(*http_req, &ctx->auth)) { + LOG(WARNING) << "parse basic authorization failed, id=" << ctx->id; + return Status("no valid Basic authorization"); + } + // check content length + ctx->body_bytes = 0; + size_t max_body_bytes = config::streaming_load_max_mb * 1024 * 1024; + if (!http_req->header(HttpHeaders::CONTENT_LENGTH).empty()) { + ctx->body_bytes = std::stol(http_req->header(HttpHeaders::CONTENT_LENGTH)); + if (ctx->body_bytes > max_body_bytes) { + LOG(WARNING) << "body exceed max size, id=" << ctx->id; + + std::stringstream ss; + ss << "body exceed max size, max_body_bytes=" << max_body_bytes; + return Status(ss.str()); + } + } else { +#ifndef BE_TEST + evhttp_connection_set_max_body_size( + evhttp_request_get_connection(http_req->get_evhttp_request()), + max_body_bytes); +#endif + } + // get format of this put + if (http_req->header(HTTP_FORMAT_KEY).empty()) { + ctx->format = TFileFormatType::FORMAT_CSV_PLAIN; + } else { + ctx->format = parse_format(http_req->header(HTTP_FORMAT_KEY)); + if (ctx->format == TFileFormatType::FORMAT_UNKNOWN) { + LOG(WARNING) << "unknown data format, id=" << ctx->id; + std::stringstream ss; + ss << "unknown data format, format=" << http_req->header(HTTP_FORMAT_KEY); + return Status(ss.str()); + } + } + + TNetworkAddress master_addr = _exec_env->master_info()->network_address; + + // begin transaction + { + TLoadTxnBeginRequest request; + set_http_auth(&request, ctx->auth); + request.db = ctx->db; + request.tbl = ctx->table; + request.label = ctx->label; + + TLoadTxnBeginResult result; +#ifndef BE_TEST + RETURN_IF_ERROR(FrontendHelper::rpc( + master_addr.hostname, master_addr.port, + [&request, &result] (FrontendServiceConnection& client) { + client->loadTxnBegin(result, request); + })); +#else + result = k_stream_load_begin_result; +#endif + Status status(result.status); + if (!status.ok()) { + LOG(WARNING) << "begin transaction failed, id=" << ctx->id + << "errmsg=" << status.get_error_msg(); + return status; + } + ctx->txn_id = result.txnId; + ctx->need_rollback = true; + } + + // process put file + return _process_put(http_req, ctx); +} + +void StreamLoadAction::on_chunk_data(HttpRequest* req) { + StreamLoadContext* ctx = (StreamLoadContext*)req->handler_ctx(); + if (ctx == nullptr || !ctx->status.ok()) { + return; + } + + struct evhttp_request* ev_req = req->get_evhttp_request(); + auto evbuf = evhttp_request_get_input_buffer(ev_req); + + while (evbuffer_get_length(evbuf) > 0) { + auto bb = ByteBuffer::allocate(4096); + auto remove_bytes = evbuffer_remove(evbuf, bb->ptr, bb->capacity); + bb->pos = remove_bytes; + bb->flip(); + auto st = ctx->body_sink->append(bb); + if (!st.ok()) { + LOG(WARNING) << "append body content failed, id=" << ctx->id + << ", errmsg=" << st.get_error_msg(); + ctx->status = st; + return; + } + ctx->receive_bytes += remove_bytes; + } +} + +void StreamLoadAction::free_handler_ctx(void* param) { + StreamLoadContext* ctx = (StreamLoadContext*)param; + if (ctx == nullptr) { + return; + } + // sender is going, make receiver know it + if (ctx->body_sink != nullptr) { + ctx->body_sink->cancel(); + } + if (ctx->unref()) { + delete ctx; + } +} + +Status StreamLoadAction::_process_put(HttpRequest* http_req, StreamLoadContext* ctx) { + // Now we use stream + ctx->use_streaming = is_format_support_streaming(ctx->format); + + // put request + TStreamLoadPutRequest request; + set_http_auth(&request, ctx->auth); + request.db = ctx->db; + request.tbl = ctx->table; + request.txnId = ctx->txn_id; + request.formatType = ctx->format; + request.__set_loadId(ctx->id.to_thrift()); + if (ctx->use_streaming) { + auto pipe = std::make_shared(); + RETURN_IF_ERROR(_exec_env->load_stream_mgr()->put(ctx->id, pipe)); + request.fileType = TFileType::FILE_STREAM; + ctx->body_sink = pipe; + } else { + RETURN_IF_ERROR(_data_saved_path(http_req, &request.path)); + auto file_sink = std::make_shared(request.path); + RETURN_IF_ERROR(file_sink->open()); + request.__isset.path = true; + request.fileType = TFileType::FILE_LOCAL; + ctx->body_sink = file_sink; + } + if (!http_req->header(HTTP_COLUMNS).empty()) { + request.__set_columns(http_req->header(HTTP_COLUMNS)); + } + if (!http_req->header(HTTP_WHERE).empty()) { + request.__set_where(http_req->header(HTTP_WHERE)); + } + if (!http_req->header(HTTP_COLUMN_SEPARATOR).empty()) { + request.__set_columnSeparator(http_req->header(HTTP_COLUMN_SEPARATOR)); + } + if (!http_req->header(HTTP_PARTITIONS).empty()) { + request.__set_partitions(http_req->header(HTTP_PARTITIONS)); + } + + // plan this load + TNetworkAddress master_addr = _exec_env->master_info()->network_address; +#ifndef BE_TEST + if (!http_req->header(HTTP_MAX_FILTER_RATIO).empty()) { + ctx->max_filter_ratio = strtod(http_req->header(HTTP_MAX_FILTER_RATIO).c_str(), nullptr); + } + + RETURN_IF_ERROR(FrontendHelper::rpc( + master_addr.hostname, master_addr.port, + [&request, ctx] (FrontendServiceConnection& client) { + client->streamLoadPut(ctx->put_result, request); + })); +#else + ctx->put_result = k_stream_load_put_result; +#endif + Status plan_status(ctx->put_result.status); + if (!plan_status.ok()) { + LOG(WARNING) << "plan streaming load failed, id=" << ctx->id + << ", txn_id=" << ctx->txn_id + << ", errmsg=" << plan_status.get_error_msg(); + return plan_status; + } + VLOG(3) << "params is " << apache::thrift::ThriftDebugString(ctx->put_result.params); + // if we not use streaming, we must download total content before we begin + // to process this load + if (!ctx->use_streaming) { + return Status::OK; + } + return _execute_plan_fragment(ctx); +} + +Status StreamLoadAction::_execute_plan_fragment(StreamLoadContext* ctx) { + // submit this params +#ifndef BE_TEST + ctx->ref(); + auto st = _exec_env->fragment_mgr()->exec_plan_fragment( + ctx->put_result.params, + [ctx] (PlanFragmentExecutor* executor) { + ctx->commit_infos = std::move(executor->runtime_state()->tablet_commit_infos()); + Status status = executor->status(); + if (status.ok()) { + ctx->number_loaded_rows = executor->runtime_state()->num_rows_load_success(); + ctx->number_filtered_rows = executor->runtime_state()->num_rows_load_filtered(); + int64_t num_total_rows = + ctx->number_loaded_rows + ctx->number_filtered_rows; + if ((0.0 + ctx->number_filtered_rows) / num_total_rows > ctx->max_filter_ratio) { + status = Status("too many filtered rows"); + } + if (ctx->number_filtered_rows > 0 && + !executor->runtime_state()->get_error_log_file_path().empty()) { + ctx->error_url = to_load_error_http_path( + executor->runtime_state()->get_error_log_file_path()); + } + } else { + LOG(WARNING) << "fragment execute failed, load_id=" << ctx->id + << ", txn_id=" << ctx->txn_id + << ", query_id=" << UniqueId(ctx->put_result.params.params.query_id) + << ", errmsg=" << status.get_error_msg(); + // cancel body_sink, make sender known it + if (ctx->body_sink != nullptr) { + ctx->body_sink->cancel(); + } + } + ctx->promise.set_value(status); + if (ctx->unref()) { + delete ctx; + } + }); + if (!st.ok()) { + // no need to check unref's return value + ctx->unref(); + return st; + } +#else + ctx->promise.set_value(k_stream_load_plan_status); +#endif + return Status::OK; +} + +void StreamLoadAction::rollback(StreamLoadContext* ctx) { + TNetworkAddress master_addr = _exec_env->master_info()->network_address; + TLoadTxnRollbackRequest request; + set_http_auth(&request, ctx->auth); + request.db = ctx->db; + request.tbl = ctx->table; + request.txnId = ctx->txn_id; + request.__set_reason(ctx->status.get_error_msg()); + TLoadTxnRollbackResult result; +#ifndef BE_TEST + auto rpc_st = FrontendHelper::rpc( + master_addr.hostname, master_addr.port, + [&request, &result] (FrontendServiceConnection& client) { + client->loadTxnRollback(result, request); + }); + if (!rpc_st.ok()) { + LOG(WARNING) << "transaction rollback failed, id=" << ctx->id + << ", txn_id=" << ctx->txn_id + << ", errmsg=" << rpc_st.get_error_msg(); + } +#else + result = k_stream_load_rollback_result; +#endif +} + +Status StreamLoadAction::_data_saved_path(HttpRequest* req, std::string* file_path) { + std::string prefix; + RETURN_IF_ERROR(_exec_env->load_path_mgr()->allocate_dir(req->param(HTTP_DB_KEY), "", &prefix)); + timeval tv; + gettimeofday(&tv, nullptr); + struct tm tm; + time_t cur_sec = tv.tv_sec; + localtime_r(&cur_sec, &tm); + char buf[64]; + strftime(buf, 64, "%Y%m%d%H%M%S", &tm); + std::stringstream ss; + ss << prefix << "/" << req->param(HTTP_TABLE_KEY) << "." << buf << "." << tv.tv_usec; + *file_path = ss.str(); + return Status::OK; +} + +} + diff --git a/be/src/http/action/stream_load.h b/be/src/http/action/stream_load.h new file mode 100644 index 0000000000..7823852d1f --- /dev/null +++ b/be/src/http/action/stream_load.h @@ -0,0 +1,63 @@ +// Copyright (c) 2017, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 "common/status.h" +#include "gen_cpp/PlanNodes_types.h" +#include "http/http_handler.h" +#include "http/message_body_sink.h" +#include "runtime/client_cache.h" + +namespace palo { + +class ExecEnv; +class TStreamLoadPutRequest; +class TStreamLoadPutResult; +class HttpAuthInfo; +class TTabletCommitInfo; +class StreamLoadContext; + +class StreamLoadAction : public HttpHandler { +public: + StreamLoadAction(ExecEnv* exec_env); + ~StreamLoadAction() override; + + void handle(HttpRequest *req) override; + + bool request_will_be_read_progressively() override { return true; } + + int on_header(HttpRequest* req) override; + + void on_chunk_data(HttpRequest* req) override; + void free_handler_ctx(void* ctx) override; + + // called by deconstructor + void rollback(StreamLoadContext* ctx); + +private: + Status _on_header(HttpRequest* http_req, StreamLoadContext* ctx); + Status _handle(StreamLoadContext* ctx); + Status _data_saved_path(HttpRequest* req, std::string* file_path); + Status _execute_plan_fragment(StreamLoadContext* ctx); + Status _process_put(HttpRequest* http_req, StreamLoadContext* ctx); + +private: + ExecEnv* _exec_env; +}; + +} diff --git a/be/src/http/http_common.h b/be/src/http/http_common.h new file mode 100644 index 0000000000..f03dec0f00 --- /dev/null +++ b/be/src/http/http_common.h @@ -0,0 +1,42 @@ +// Copyright (c) 2017, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 + +namespace palo { + +struct HttpAuthInfo { + std::string user; + std::string passwd; + std::string cluster; + std::string user_ip; +}; + +static const std::string HTTP_DB_KEY = "db"; +static const std::string HTTP_TABLE_KEY = "table"; +static const std::string HTTP_LABEL_KEY = "label"; +static const std::string HTTP_FORMAT_KEY = "format"; +static const std::string HTTP_COLUMNS = "columns"; +static const std::string HTTP_WHERE = "where"; +static const std::string HTTP_COLUMN_SEPARATOR = "column_separator"; +static const std::string HTTP_MAX_FILTER_RATIO = "max_filter_ratio"; +static const std::string HTTP_TIMEOUT = "timeout"; +static const std::string HTTP_PARTITIONS = "partitions"; + +static const std::string HTTP_100_CONTINUE = "100-continue"; + +} diff --git a/be/src/http/message_body_sink.cpp b/be/src/http/message_body_sink.cpp new file mode 100644 index 0000000000..b6b8dfe653 --- /dev/null +++ b/be/src/http/message_body_sink.cpp @@ -0,0 +1,75 @@ +// Copyright (c) 2017, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 "http/message_body_sink.h" + +#include +#include +#include +#include + +#include "http/http_channel.h" +#include "http/http_parser.h" + +#include "util/runtime_profile.h" + +namespace palo { + +MessageBodyFileSink::~MessageBodyFileSink() { + if (_fd >= 0) { + close(_fd); + } +} + +Status MessageBodyFileSink::open() { + _fd = ::open(_path.data(), O_RDWR|O_CREAT|O_TRUNC, S_IRUSR | S_IWUSR | S_IRGRP | S_IWGRP); + if (_fd < 0) { + char errmsg[64]; + LOG(WARNING) << "fail to open file, file=" << _path + << ", errmsg=" << strerror_r(errno, errmsg, 64); + return Status("fail to open file"); + } + return Status::OK; +} + +Status MessageBodyFileSink::append(const char* data, size_t size) { + auto written = ::write(_fd, data, size); + if (written == size) { + return Status::OK; + } + char errmsg[64]; + LOG(WARNING) << "fail to write, file=" << _path + << ", error=" << strerror_r(errno, errmsg, 64); + return Status("fail to write file"); +} + +Status MessageBodyFileSink::finish() { + if (::close(_fd) < 0) { + std::stringstream ss; + char errmsg[64]; + LOG(WARNING) << "fail to write, file=" << _path + << ", error=" << strerror_r(errno, errmsg, 64); + _fd = -1; + return Status("fail to close file"); + } + _fd = -1; + return Status::OK; +} + +void MessageBodyFileSink::cancel() { + unlink(_path.data()); +} + +} diff --git a/be/src/http/message_body_sink.h b/be/src/http/message_body_sink.h new file mode 100644 index 0000000000..ec2079f5ce --- /dev/null +++ b/be/src/http/message_body_sink.h @@ -0,0 +1,58 @@ +// Copyright (c) 2017, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 "common/status.h" + +#include "util/byte_buffer.h" + +namespace palo { + +class HttpChannel; +class BodySink; + +class MessageBodySink { +public: + virtual ~MessageBodySink() { } + virtual Status append(const char* data, size_t size) = 0; + virtual Status append(const ByteBufferPtr& buf) { + return append(buf->ptr, buf->remaining()); + } + // called when all data has been append + virtual Status finish() { + return Status::OK; + } + // called when read HTTP failed + virtual void cancel() { } +}; + +// write HTTP request's message-body to a local file +class MessageBodyFileSink : public MessageBodySink { +public: + MessageBodyFileSink(const std::string& path) : _path(path) { } + virtual ~MessageBodyFileSink(); + + Status open(); + + Status append(const char* data, size_t size) override; + Status finish() override; + void cancel() override; +private: + std::string _path; + int _fd = -1; +}; + +} diff --git a/be/src/http/utils.cpp b/be/src/http/utils.cpp new file mode 100644 index 0000000000..01dfb6291f --- /dev/null +++ b/be/src/http/utils.cpp @@ -0,0 +1,70 @@ +// Copyright (c) 2018, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 + +#include "common/logging.h" +#include "http/http_common.h" +#include "http/http_headers.h" +#include "http/http_request.h" +#include "util/url_coding.h" + +namespace palo { + +bool parse_basic_auth(const HttpRequest& req, std::string* user, std::string* passwd) { + const char k_basic[] = "Basic "; + auto& auth = req.header(HttpHeaders::AUTHORIZATION); + if (auth.compare(0, sizeof(k_basic) - 1, k_basic, sizeof(k_basic) - 1) != 0) { + return false; + } + std::string encoded_str = auth.substr(sizeof(k_basic) - 1); + std::string decoded_auth; + if (!base64_decode(encoded_str, &decoded_auth)) { + return false; + } + auto pos = decoded_auth.find(':'); + if (pos == std::string::npos) { + return false; + } + user->assign(decoded_auth.c_str(), pos); + passwd->assign(decoded_auth.c_str() + pos + 1); + + return true; +} + +bool parse_basic_auth(const HttpRequest& req, HttpAuthInfo* auth) { + std::string full_user; + if (!parse_basic_auth(req, &full_user, &auth->passwd)) { + return false; + } + auto pos = full_user.find('@'); + if (pos != std::string::npos) { + auth->user.assign(full_user.data(), pos); + auth->cluster.assign(full_user.data() + pos + 1); + } else { + auth->user = full_user; + } + + // set user ip + if (req.remote_host() != nullptr) { + auth->user_ip.assign(req.remote_host()); + } else { + auth->user_ip.assign(""); + } + + return true; +} + +} diff --git a/be/src/http/utils.h b/be/src/http/utils.h new file mode 100644 index 0000000000..3896625fdc --- /dev/null +++ b/be/src/http/utils.h @@ -0,0 +1,44 @@ +// Copyright (c) 2018, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 "http/http_common.h" + +namespace palo { + +class HttpRequest; +class HttpAuthInfo; + +// parse Basic authorization +// return true, if request contain valid basic authorization. +// Otherwise return fasle +bool parse_basic_auth(const HttpRequest& req, std::string* user, std::string* passwd); + +bool parse_basic_auth(const HttpRequest& req, HttpAuthInfo* auth); + +template +void set_http_auth(T* req, const HttpAuthInfo& auth) { + req->user = auth.user; + req->passwd = auth.passwd; + if (!auth.cluster.empty()) { + req->__set_cluster(auth.cluster); + } + req->__set_user_ip(auth.user_ip); +} + +} diff --git a/be/src/olap/delta_writer.cpp b/be/src/olap/delta_writer.cpp new file mode 100644 index 0000000000..7f869859a1 --- /dev/null +++ b/be/src/olap/delta_writer.cpp @@ -0,0 +1,226 @@ +// Copyright (c) 2018, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 "olap/delta_writer.h" + +#include "olap/schema.h" +#include "olap/rowset.h" + +namespace palo { + +OLAPStatus DeltaWriter::open(WriteRequest* req, DeltaWriter** writer) { + *writer = new DeltaWriter(req); + return OLAP_SUCCESS; +} + +DeltaWriter::DeltaWriter(WriteRequest* req) + : _req(*req), _table(nullptr), + _cur_rowset(nullptr), _new_table(nullptr), + _writer(nullptr), _mem_table(nullptr), + _schema(nullptr), _field_infos(nullptr), + _rowset_id(-1), _delta_written_success(false) {} + +DeltaWriter::~DeltaWriter() { + if (!_delta_written_success) { + _garbage_collection(); + } + SAFE_DELETE(_writer); + SAFE_DELETE(_mem_table); + SAFE_DELETE(_schema); +} + +void DeltaWriter::_garbage_collection() { + OLAPEngine::get_instance()->delete_transaction(_req.partition_id, _req.transaction_id, + _req.tablet_id, _req.schema_hash); + for (Rowset* rowset : _rowset_vec) { + rowset->release(); + OLAPEngine::get_instance()->add_unused_index(rowset); + } + if (_new_table != nullptr) { + OLAPEngine::get_instance()->delete_transaction(_req.partition_id, _req.transaction_id, + _new_table->tablet_id(), _new_table->schema_hash()); + for (Rowset* rowset : _new_rowset_vec) { + rowset->release(); + OLAPEngine::get_instance()->add_unused_index(rowset); + } + } +} + +OLAPStatus DeltaWriter::init() { + _table = OLAPEngine::get_instance()->get_table(_req.tablet_id, _req.schema_hash); + if (_table == nullptr) { + LOG(WARNING) << "tablet_id: " << _req.tablet_id << ", " + << "schema_hash: " << _req.schema_hash << " not found"; + return OLAP_ERR_TABLE_NOT_FOUND; + } + + { + MutexLock push_lock(_table->get_push_lock()); + RETURN_NOT_OK(OLAPEngine::get_instance()->add_transaction( + _req.partition_id, _req.transaction_id, + _req.tablet_id, _req.schema_hash, _req.load_id)); + //_rowset_id = _table->current_pending_rowset_id(_req.transaction_id); + if (_req.need_gen_rollup) { + TTabletId new_tablet_id; + TSchemaHash new_schema_hash; + _table->obtain_header_rdlock(); + bool is_schema_changing = + _table->get_schema_change_request(&new_tablet_id, &new_schema_hash, nullptr, nullptr); + _table->release_header_lock(); + + if (is_schema_changing) { + LOG(INFO) << "load with schema change." << "old_tablet_id: " << _table->tablet_id() << ", " + << "old_schema_hash: " << _table->schema_hash() << ", " + << "new_tablet_id: " << new_tablet_id << ", " + << "new_schema_hash: " << new_schema_hash << ", " + << "transaction_id: " << _req.transaction_id; + _new_table = OLAPEngine::get_instance()->get_table(new_tablet_id, new_schema_hash); + OLAPEngine::get_instance()->add_transaction( + _req.partition_id, _req.transaction_id, + new_tablet_id, new_schema_hash, _req.load_id); + } + } + + // create pending data dir + std::string dir_path = _table->construct_pending_data_dir_path(); + if (!check_dir_existed(dir_path)) { + RETURN_NOT_OK(create_dirs(dir_path)); + } + } + + ++_rowset_id; + _cur_rowset = new Rowset(_table.get(), false, _rowset_id, 0, true, + _req.partition_id, _req.transaction_id); + DCHECK(_cur_rowset != nullptr) << "failed to malloc Rowset"; + _cur_rowset->acquire(); + _cur_rowset->set_load_id(_req.load_id); + _rowset_vec.push_back(_cur_rowset); + + // New Writer to write data into Rowset + VLOG(3) << "init writer. table=" << _table->full_name() << ", " + << "block_row_size=" << _table->num_rows_per_row_block(); + _writer = IWriter::create(_table, _cur_rowset, true); + DCHECK(_writer != nullptr) << "memory error occur when creating writer"; + + const std::vector& slots = _req.tuple_desc->slots(); + for (auto& field_info : _table->tablet_schema()) { + for (size_t i = 0; i < slots.size(); ++i) { + if (slots[i]->col_name() == field_info.name) { + _col_ids.push_back(i); + } + } + } + _field_infos = &(_table->tablet_schema()); + _schema = new Schema(*_field_infos), + _mem_table = new MemTable(_schema, _field_infos, &_col_ids, + _req.tuple_desc, _table->keys_type()); + _is_init = true; + return OLAP_SUCCESS; +} + +OLAPStatus DeltaWriter::write(Tuple* tuple) { + if (!_is_init) { + auto st = init(); + if (st != OLAP_SUCCESS) { + return st; + } + } + + _mem_table->insert(tuple); + if (_mem_table->memory_usage() >= config::write_buffer_size) { + RETURN_NOT_OK(_mem_table->flush(_writer)); + + ++_rowset_id; + _cur_rowset = new Rowset(_table.get(), false, _rowset_id, 0, true, + _req.partition_id, _req.transaction_id); + DCHECK(_cur_rowset != nullptr) << "failed to malloc Rowset"; + _cur_rowset->acquire(); + _cur_rowset->set_load_id(_req.load_id); + _rowset_vec.push_back(_cur_rowset); + + SAFE_DELETE(_writer); + _writer = IWriter::create(_table, _cur_rowset, true); + DCHECK(_writer != nullptr) << "memory error occur when creating writer"; + + SAFE_DELETE(_mem_table); + _mem_table = new MemTable(_schema, _field_infos, &_col_ids, + _req.tuple_desc, _table->keys_type()); + } + return OLAP_SUCCESS; +} + +OLAPStatus DeltaWriter::close(google::protobuf::RepeatedPtrField* tablet_vec) { + if (!_is_init) { + auto st = init(); + if (st != OLAP_SUCCESS) { + return st; + } + } + RETURN_NOT_OK(_mem_table->close(_writer)); + + OLAPStatus res = OLAP_SUCCESS; + //add pending data to tablet + RETURN_NOT_OK(_table->add_pending_version(_req.partition_id, _req.transaction_id, nullptr)); + for (Rowset* rowset : _rowset_vec) { + RETURN_NOT_OK(_table->add_pending_rowset(rowset)); + RETURN_NOT_OK(rowset->load()); + } + if (_new_table != nullptr) { + LOG(INFO) << "convert version for schema change"; + { + MutexLock push_lock(_new_table->get_push_lock()); + // create pending data dir + std::string dir_path = _new_table->construct_pending_data_dir_path(); + if (!check_dir_existed(dir_path)) { + RETURN_NOT_OK(create_dirs(dir_path)); + } + } + SchemaChangeHandler schema_change; + res = schema_change.schema_version_convert( + _table, _new_table, &_rowset_vec, &_new_rowset_vec); + if (res != OLAP_SUCCESS) { + LOG(WARNING) << "failed to convert delta for new table in schema change." + << "res: " << res << ", " << "new_table: " << _new_table->full_name(); + return res; + } + + RETURN_NOT_OK(_new_table->add_pending_version(_req.partition_id, _req.transaction_id, nullptr)); + for (Rowset* rowset : _new_rowset_vec) { + RETURN_NOT_OK(_new_table->add_pending_rowset(rowset)); + RETURN_NOT_OK(rowset->load()); + } + } + +#ifndef BE_TEST + PTabletInfo* tablet_info = tablet_vec->Add(); + tablet_info->set_tablet_id(_table->tablet_id()); + tablet_info->set_schema_hash(_table->schema_hash()); + if (_new_table != nullptr) { + tablet_info = tablet_vec->Add(); + tablet_info->set_tablet_id(_new_table->tablet_id()); + tablet_info->set_schema_hash(_new_table->schema_hash()); + } +#endif + + _delta_written_success = true; + return OLAP_SUCCESS; +} + +OLAPStatus DeltaWriter::cancel() { + DCHECK(!_is_init); + return OLAP_SUCCESS; +} + +} // namespace palo diff --git a/be/src/olap/delta_writer.h b/be/src/olap/delta_writer.h new file mode 100644 index 0000000000..6bd508bca3 --- /dev/null +++ b/be/src/olap/delta_writer.h @@ -0,0 +1,83 @@ +// Copyright (c) 2018, Baidu.com, Inc. All Rights Reserved + +// Licensed 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. + +#ifndef BDG_PALO_BE_SRC_DELTA_WRITER_H +#define BDG_PALO_BE_SRC_DELTA_WRITER_H + +#include "olap/memtable.h" +#include "olap/olap_engine.h" +#include "olap/olap_table.h" +#include "olap/schema_change.h" +#include "olap/writer.h" +#include "runtime/descriptors.h" +#include "runtime/tuple.h" +#include "gen_cpp/internal_service.pb.h" + +namespace palo { + +class Rowset; + +enum WriteType { + LOAD = 1, + LOAD_DELETE = 2, + DELETE = 3 +}; + +struct WriteRequest { + int64_t tablet_id; + int32_t schema_hash; + WriteType write_type; + int64_t transaction_id; + int64_t partition_id; + PUniqueId load_id; + bool need_gen_rollup; + TupleDescriptor* tuple_desc; +}; + +class DeltaWriter { +public: + static OLAPStatus open(WriteRequest* req, DeltaWriter** writer); + OLAPStatus init(); + DeltaWriter(WriteRequest* req); + ~DeltaWriter(); + OLAPStatus write(Tuple* tuple); + OLAPStatus close(google::protobuf::RepeatedPtrField* tablet_vec); + + OLAPStatus cancel(); + + int64_t partition_id() const { return _req.partition_id; } +private: + void _garbage_collection(); + + bool _is_init = false; + WriteRequest _req; + OLAPTablePtr _table; + Rowset* _cur_rowset; + std::vector _rowset_vec; + std::vector _new_rowset_vec; + OLAPTablePtr _new_table; + IWriter* _writer; + MemTable* _mem_table; + Schema* _schema; + std::vector* _field_infos; + std::vector _col_ids; + + int32_t _rowset_id; + bool _delta_written_success; +}; + +} // namespace palo + +#endif // BDG_PALO_BE_SRC_OLAP_DELTA_WRITER_H diff --git a/be/src/olap/memtable.cpp b/be/src/olap/memtable.cpp new file mode 100644 index 0000000000..d4aae47842 --- /dev/null +++ b/be/src/olap/memtable.cpp @@ -0,0 +1,161 @@ +// Copyright (c) 2018, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 "olap/memtable.h" + +#include "olap/hll.h" +#include "olap/writer.h" +#include "olap/row_cursor.h" +#include "util/runtime_profile.h" +#include "util/debug_util.h" + +namespace palo { + +MemTable::MemTable(Schema* schema, std::vector* field_infos, + std::vector* col_ids, TupleDescriptor* tuple_desc, + KeysType keys_type) + : _schema(schema), + _field_infos(field_infos), + _tuple_desc(tuple_desc), + _col_ids(col_ids), + _keys_type(keys_type), + _row_comparator(_schema) { + _schema_size = _schema->schema_size(); + _tuple_buf = _arena.Allocate(_schema_size); + _skip_list = new Table(_row_comparator, &_arena); +} + +MemTable::~MemTable() { + delete _skip_list; +} + +MemTable::RowCursorComparator::RowCursorComparator(const Schema* schema) + : _schema(schema) {} + +int MemTable::RowCursorComparator::operator() + (const char* left, const char* right) const { + return _schema->compare(left, right); +} + +size_t MemTable::memory_usage() { + return _arena.MemoryUsage(); +} + +void MemTable::insert(Tuple* tuple) { + const std::vector& slots = _tuple_desc->slots(); + size_t offset = 0; + for (size_t i = 0; i < _col_ids->size(); ++i) { + const SlotDescriptor* slot = slots[(*_col_ids)[i]]; + _schema->set_not_null(i, _tuple_buf); + if (tuple->is_null(slot->null_indicator_offset())) { + _schema->set_null(i, _tuple_buf); + offset += _schema->get_col_size(i) + 1; + continue; + } + offset += 1; + TypeDescriptor type = slot->type(); + switch (type.type) { + case TYPE_CHAR: { + const StringValue* src = tuple->get_string_slot(slot->tuple_offset()); + StringSlice* dest = (StringSlice*)(_tuple_buf + offset); + dest->size = (*_field_infos)[i].length; + dest->data = _arena.Allocate(dest->size); + memcpy(dest->data, src->ptr, src->len); + memset(dest->data + src->len, 0, dest->size - src->len); + break; + } + case TYPE_VARCHAR: { + const StringValue* src = tuple->get_string_slot(slot->tuple_offset()); + StringSlice* dest = (StringSlice*)(_tuple_buf + offset); + dest->size = src->len; + dest->data = _arena.Allocate(dest->size); + memcpy(dest->data, src->ptr, dest->size); + break; + } + case TYPE_HLL: { + const StringValue* src = tuple->get_string_slot(slot->tuple_offset()); + StringSlice* dest = (StringSlice*)(_tuple_buf + offset); + dest->size = src->len; + bool exist = _skip_list->Contains(_tuple_buf); + if (exist) { + dest->data = _arena.Allocate(dest->size); + memcpy(dest->data, src->ptr, dest->size); + } else { + dest->data = src->ptr; + char* mem = _arena.Allocate(sizeof(HllContext)); + HllContext* context = new (mem) HllContext; + HllSetHelper::init_context(context); + HllSetHelper::fill_set(reinterpret_cast(dest), context); + context->has_value = true; + char* variable_ptr = _arena.Allocate(sizeof(HllContext*) + HLL_COLUMN_DEFAULT_LEN); + *(size_t*)(variable_ptr) = (size_t)(context); + variable_ptr += sizeof(HllContext*); + dest->data = variable_ptr; + dest->size = HLL_COLUMN_DEFAULT_LEN; + } + break; + } + case TYPE_DECIMAL: { + DecimalValue* decimal_value = tuple->get_decimal_slot(slot->tuple_offset()); + decimal12_t* storage_decimal_value = reinterpret_cast(_tuple_buf + offset); + storage_decimal_value->integer = decimal_value->int_value(); + storage_decimal_value->fraction = decimal_value->frac_value(); + break; + } + case TYPE_DATETIME: { + DateTimeValue* datetime_value = tuple->get_datetime_slot(slot->tuple_offset()); + uint64_t* storage_datetime_value = reinterpret_cast(_tuple_buf + offset); + *storage_datetime_value = datetime_value->to_olap_datetime(); + break; + } + case TYPE_DATE: { + DateTimeValue* date_value = tuple->get_datetime_slot(slot->tuple_offset()); + uint24_t* storage_date_value = reinterpret_cast(_tuple_buf + offset); + *storage_date_value = static_cast(date_value->to_olap_date()); + break; + } + default: { + memcpy(_tuple_buf + offset, tuple->get_slot(slot->tuple_offset()), _schema->get_col_size(i)); + break; + } + } + offset = offset + _schema->get_col_size(i); + } + + bool overwritten = false; + _skip_list->Insert(_tuple_buf, &overwritten, _keys_type); + if (!overwritten) { + _tuple_buf = _arena.Allocate(_schema_size); + } +} + +OLAPStatus MemTable::flush(IWriter* writer) { + Table::Iterator it(_skip_list); + for (it.SeekToFirst(); it.Valid(); it.Next()) { + const char* row = it.key(); + _schema->finalize(row); + RETURN_NOT_OK(writer->write(row)); + writer->next(row, _schema); + } + + RETURN_NOT_OK(writer->finalize()); + return OLAP_SUCCESS; +} + +OLAPStatus MemTable::close(IWriter* writer) { + return flush(writer); +} + +} // namespace palo diff --git a/be/src/olap/memtable.h b/be/src/olap/memtable.h new file mode 100644 index 0000000000..994f5c7e3b --- /dev/null +++ b/be/src/olap/memtable.h @@ -0,0 +1,64 @@ +// Copyright (c) 2018, Baidu.com, Inc. All Rights Reserved + +// Licensed 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. + +#ifndef BDG_PALO_BE_SRC_OLAP_MEMTABLE_H +#define BDG_PALO_BE_SRC_OLAP_MEMTABLE_H + +#include + +#include "olap/schema.h" +#include "olap/skiplist.h" +#include "runtime/tuple.h" + +namespace palo { + +class IWriter; +class RowCursor; + +class MemTable { +public: + MemTable(Schema* schema, std::vector* field_infos, + std::vector* col_ids, TupleDescriptor* tuple_desc, + KeysType keys_type); + ~MemTable(); + size_t memory_usage(); + void insert(Tuple* tuple); + OLAPStatus flush(IWriter* writer); + OLAPStatus close(IWriter* writer); +private: + Schema* _schema; + std::vector* _field_infos; + TupleDescriptor* _tuple_desc; + std::vector* _col_ids; + KeysType _keys_type; + + struct RowCursorComparator { + const Schema* _schema; + RowCursorComparator(const Schema* schema); + int operator()(const char* left, const char* right) const; + }; + + RowCursorComparator _row_comparator; + Arena _arena; + + typedef SkipList Table; + char* _tuple_buf; + size_t _schema_size; + Table* _skip_list; +}; // class MemTable + +} // namespace palo + +#endif // BDG_PALO_BE_SRC_OLAP_MEMTABLE_H diff --git a/be/src/olap/new_status.cpp b/be/src/olap/new_status.cpp new file mode 100644 index 0000000000..006cae9182 --- /dev/null +++ b/be/src/olap/new_status.cpp @@ -0,0 +1,134 @@ +// Copyright (c) 2011 The LevelDB Authors. All rights reserved. +// Use of this source code is governed by a BSD-style license that can be +// found in the LICENSE file. See the AUTHORS file for names of contributors. + +#include "olap/new_status.h" + +#include +#include "common/logging.h" + +namespace palo { + +std::string ErrnoToString(int err) { + char errmsg[128]; + char* ret = strerror_r(err, errmsg, 128); + if (ret != errmsg) { + strncpy(errmsg, ret, 128); + errmsg[127] = '\0'; + } + return std::string(errmsg); +} + +NewStatus IOError(const std::string& context, int err) { + switch (err) { + case ENOENT: + return NewStatus::NotFound(context, ErrnoToString(err), err); + case EEXIST: + return NewStatus::AlreadyExist(context, ErrnoToString(err), err); + case EOPNOTSUPP: + return NewStatus::NotSupported(context, ErrnoToString(err), err); + case EIO: + return NewStatus::DiskFailure(context, ErrnoToString(err), err); + case ENODEV: + return NewStatus::DiskFailure(context, ErrnoToString(err), err); + case ENXIO: + return NewStatus::DiskFailure(context, ErrnoToString(err), err); + case EROFS: + return NewStatus::DiskFailure(context, ErrnoToString(err), err); + } + return NewStatus::IOError(context, ErrnoToString(err), err); +} + +NewStatus::NewStatus(Code code, const StringSlice& msg, const StringSlice& msg2, int32_t posix_code) + : _code(code), _posix_code(posix_code) { + DCHECK(code != kOk); + const uint32_t len1 = msg.size; + const uint32_t len2 = msg2.size; + const uint32_t size = len1 + (len2 ? (2 + len2) : 0); + char* result = new char[size + 4]; + memcpy(result, &size, sizeof(size)); + memcpy(result + 4, msg.data, len1); + if (len2) { + result[len1 + 4] = ':'; + result[len1 + 5] = ' '; + memcpy(result + len1 + 6, msg2.data, len2); + } + _state = result; +} + +const char* NewStatus::CopyState(const char* state) { + uint32_t size; + memcpy(&size, state, sizeof(size)); + char* result = new char[size + 4]; + memcpy(result, state, size + 4); + return result; +} + +std::string NewStatus::CodeAsString() const { + const char* type = nullptr; + switch (_code) { + case kOk: + type = "OK"; + break; + case kNotFound: + type = "NotFound"; + break; + case kCorruption: + type = "Corruption"; + break; + case kNotSupported: + type = "Not implemented"; + break; + case kInvalidArgument: + type = "Invalid argument"; + break; + case kAlreadyExist: + type = "Already Exist"; + break; + case kNoSpace: + type = "No Space"; + break; + case kEndOfFile: + type = "End Of File"; + break; + case kDiskFailure: + type = "Disk Failure"; + break; + case kIOError: + type = "IO error"; + break; + case kTimedOut: + type = "Timed Out"; + break; + case kMemoryLimitExceeded: + type = "Memory Limit Exceeded"; + break; + case kDeadLock: + type = "Dead Lock"; + break; + } + + return std::string(type); +} + +std::string NewStatus::ToString() const { + std::string result(CodeAsString()); + if (code() == kOk) { + return result; + } + + result.append(": "); + uint32_t length; + memcpy(&length, _state, sizeof(length)); + result.append(_state + 4, length); + + if (_posix_code != -1) { + char buf[64]; + snprintf(buf, sizeof(buf), ", errno=%d", _posix_code); + result.append(buf); + } + + return result; +} + +} // namespace palo diff --git a/be/src/olap/new_status.h b/be/src/olap/new_status.h new file mode 100644 index 0000000000..162919c8b4 --- /dev/null +++ b/be/src/olap/new_status.h @@ -0,0 +1,196 @@ +// Copyright (c) 2011 The LevelDB Authors. All rights reserved. +// Use of this source code is governed by a BSD-style license that can be +// found in the LICENSE file. See the AUTHORS file for names of contributors. +// +// A NewStatus encapsulates the result of an operation. It may indicate success, +// or it may indicate an error with an associated error message. +// +// Multiple threads can invoke const methods on a NewStatus without +// external synchronization, but if any of the threads may call a +// non-const method, all threads accessing the same NewStatus must use +// external synchronization. + +#ifndef BDG_PALO_BE_SRC_OLAP_STATUS_H +#define BDG_PALO_BE_SRC_OLAP_STATUS_H + +#include +#include "olap/string_slice.h" + +namespace palo { + +#define WARN_AND_RETURN(status) do { \ + const NewStatus& s = (status); \ + LOG(WARNING) << s.ToString(); \ + return s; \ +} while (0); + +class NewStatus { +public: + // Create a success status. + NewStatus() : _code(kOk), _posix_code(0), _state(nullptr) {} + ~NewStatus() { delete[] _state; } + + // Copy the specified status. + NewStatus(const NewStatus& s); + void operator=(const NewStatus& s); + + // Return a success status. + static NewStatus OK() { return NewStatus(); } + + // Return error status of an appropriate type. + static NewStatus NotFound(const StringSlice& msg, const StringSlice& msg2 = StringSlice(), + int32_t posix_code = -1) { + return NewStatus(kNotFound, msg, msg2, posix_code); + } + static NewStatus Corruption(const StringSlice& msg, const StringSlice& msg2 = StringSlice(), + int32_t posix_code = -1) { + return NewStatus(kCorruption, msg, msg2, posix_code); + } + static NewStatus NotSupported(const StringSlice& msg, const StringSlice& msg2 = StringSlice(), + int32_t posix_code = -1) { + return NewStatus(kNotSupported, msg, msg2, posix_code); + } + static NewStatus InvalidArgument(const StringSlice& msg, const StringSlice& msg2 = StringSlice(), + int32_t posix_code = -1) { + return NewStatus(kInvalidArgument, msg, msg2, posix_code); + } + static NewStatus AlreadyExist(const StringSlice& msg, const StringSlice& msg2 = StringSlice(), + int32_t posix_code = -1) { + return NewStatus(kAlreadyExist, msg, msg2, posix_code); + } + static NewStatus NoSpace(const StringSlice& msg, const StringSlice& msg2 = StringSlice(), + int32_t posix_code = -1) { + return NewStatus(kNoSpace, msg, msg2, posix_code); + } + static NewStatus EndOfFile(const StringSlice& msg, const StringSlice& msg2 = StringSlice(), + int32_t posix_code = -1) { + return NewStatus(kEndOfFile, msg, msg2, posix_code); + } + static NewStatus DiskFailure(const StringSlice& msg, const StringSlice& msg2 = StringSlice(), + int32_t posix_code = -1) { + return NewStatus(kDiskFailure, msg, msg2, posix_code); + } + static NewStatus IOError(const StringSlice& msg, const StringSlice& msg2 = StringSlice(), + int32_t posix_code = -1) { + return NewStatus(kIOError, msg, msg2, posix_code); + } + static NewStatus TimedOut(const StringSlice& msg, const StringSlice& msg2 = StringSlice(), + int32_t posix_code = -1) { + return NewStatus(kTimedOut, msg, msg2, posix_code); + } + static NewStatus MemoryLimitExceeded(const StringSlice& msg, const StringSlice& msg2 = StringSlice(), + int32_t posix_code = -1) { + return NewStatus(kMemoryLimitExceeded, msg, msg2, posix_code); + } + static NewStatus DeadLock(const StringSlice& msg, const StringSlice& msg2 = StringSlice(), + int32_t posix_code = -1) { + return NewStatus(kDeadLock, msg, msg2, posix_code); + } + + // Returns true iff the status indicates success. + bool ok() const { return code() == kOk; } + + // Returns true iff the status indicates a NotFound error. + bool IsNotFound() const { return code() == kNotFound; } + + // Returns true iff the status indicates a Corruption error. + bool IsCorruption() const { return code() == kCorruption; } + + // Returns true iff the status indicates a NotSupportedError. + bool IsNotSupported() const { return code() == kNotSupported; } + + // Returns true iff the status indicates an InvalidArgument. + bool IsInvalidArgument() const { return code() == kInvalidArgument; } + + // Returns true iff the status indicates an AlreadyExist. + bool IsAlreadyExist() const { return code() == kAlreadyExist; } + + // Returns true iff the status indicates an NoSpace Error. + bool IsNoSpace() const { return code() == kNoSpace; } + + // Returns true iff the status indicates an end of file. + bool IsEndOfFile() const { return code() == kEndOfFile; } + + // Returns true iff the status indicates an IOError. + bool IsDiskFailure() const { return code() == kDiskFailure; } + + // Returns true iff the status indicates an IOError. + bool IsIOError() const { return code() == kIOError; } + + // Returns true iff the status indicates timed out. + bool IsTimedOut() const { return code() == kTimedOut; } + + // Returns true iff the status indicates a memory limit error. + bool IsMemoryLimitExceeded() const { return code() == kMemoryLimitExceeded; } + + // Returns true iff the status indicates a DeadLock. + bool IsDeadLock() const { return code() == kDeadLock; } + + // Return a string representation of this status suitable for printing. + // Returns the string "OK" for success. + std::string ToString() const; + + // return A string representation of the status code, without the message + // text or POSIX code information + std::string CodeAsString() const; + + // return The Posix code associated with this Status Object + inline int32_t posix_code() const { return _posix_code; } + +private: + enum Code { + kOk = 0, + kNotFound = 1, + kCorruption = 2, + kNotSupported = 3, + kInvalidArgument = 4, + kAlreadyExist = 5, + kNoSpace = 6, + kEndOfFile = 7, + kIOError = 8, + kDiskFailure = 9, + kTimedOut = 10, + kMemoryLimitExceeded = 11, + kDeadLock = 12 + }; + + Code _code; + Code code() const { return _code; } + + // The POSIX code accociated with the NewStatus object, + // if _posix_code == -1, indicates no posix_code. + int32_t _posix_code; + + // OK status has a nullptr _state. Otherwise, _state is a new[] array + // of the following form: + // _state[0..3] == length of message + // _state[4..] == message + const char* _state; + + NewStatus(Code code, const StringSlice& msg, const StringSlice& msg2, int32_t posix_code); + static const char* CopyState(const char* s); +}; + +inline NewStatus::NewStatus(const NewStatus& s) + : _code(s._code), _posix_code(s._posix_code) { + _state = (s._state == nullptr) ? nullptr : CopyState(s._state); +} + +inline void NewStatus::operator=(const NewStatus& s) { + // The following condition catches both aliasing (when this == &s), + // and the common case where both s and *this are ok. + if (_state != s._state) { + _code = s._code; + _posix_code = s._posix_code; + delete[] _state; + _state = (s._state == nullptr) ? nullptr : CopyState(s._state); + } +} + +std::string ErrnoToString(int err); +NewStatus IOError(const std::string& context, int err); + + +} // namespace palo + +#endif // BDG_PALO_BE_SRC_OLAP_STATUS_H diff --git a/be/src/olap/olap_header_manager.cpp b/be/src/olap/olap_header_manager.cpp new file mode 100644 index 0000000000..a65352a37c --- /dev/null +++ b/be/src/olap/olap_header_manager.cpp @@ -0,0 +1,175 @@ +// Copyright (c) 2018, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 "olap/olap_header_manager.h" + +#include +#include +#include +#include +#include + +#include "olap/olap_define.h" +#include "olap/olap_engine.h" +#include "olap/olap_meta.h" +#include "common/logging.h" +#include "json2pb/json_to_pb.h" +#include "json2pb/pb_to_json.h" + +using rocksdb::DB; +using rocksdb::DBOptions; +using rocksdb::ColumnFamilyDescriptor; +using rocksdb::ColumnFamilyHandle; +using rocksdb::ColumnFamilyOptions; +using rocksdb::ReadOptions; +using rocksdb::WriteOptions; +using rocksdb::Slice; +using rocksdb::Iterator; +using rocksdb::Status; +using rocksdb::kDefaultColumnFamilyName; + +namespace palo { + +const std::string HEADER_PREFIX = "hdr_"; + +OLAPStatus OlapHeaderManager::get_header(OlapStore* store, + TTabletId tablet_id, TSchemaHash schema_hash, OLAPHeader* header) { + OlapMeta* meta = store->get_meta(); + std::stringstream key_stream; + key_stream << HEADER_PREFIX << tablet_id << "_" << schema_hash; + std::string key = key_stream.str(); + std::string value; + OLAPStatus s = meta->get(META_COLUMN_FAMILY_INDEX, key, value); + if (s == OLAP_ERR_META_KEY_NOT_FOUND) { + LOG(WARNING) << "tablet_id:" << tablet_id << ", schema_hash:" << schema_hash << " not found."; + return OLAP_ERR_META_KEY_NOT_FOUND; + } else if (s != OLAP_SUCCESS) { + LOG(WARNING) << "load tablet_id:" << tablet_id << ", schema_hash:" << schema_hash << " failed."; + return s; + } + header->ParseFromString(value); + return header->init(); +} + +OLAPStatus OlapHeaderManager::get_json_header(OlapStore* store, + TTabletId tablet_id, TSchemaHash schema_hash, std::string* json_header) { + OLAPHeader header; + OLAPStatus s = get_header(store, tablet_id, schema_hash, &header); + if (s != OLAP_SUCCESS) { + return s; + } + json2pb::Pb2JsonOptions json_options; + json_options.pretty_json = true; + json2pb::ProtoMessageToJson(header, json_header, json_options); + return OLAP_SUCCESS; +} + + +OLAPStatus OlapHeaderManager::save(OlapStore* store, + TTabletId tablet_id, TSchemaHash schema_hash, const OLAPHeader* header) { + std::stringstream key_stream; + key_stream << HEADER_PREFIX << tablet_id << "_" << schema_hash; + std::string key = key_stream.str(); + std::string value; + header->SerializeToString(&value); + OlapMeta* meta = store->get_meta(); + OLAPStatus s = meta->put(META_COLUMN_FAMILY_INDEX, key, value); + return s; +} + +OLAPStatus OlapHeaderManager::remove(OlapStore* store, TTabletId tablet_id, TSchemaHash schema_hash) { + std::stringstream key_stream; + key_stream << HEADER_PREFIX << tablet_id << "_" << schema_hash; + std::string key = key_stream.str(); + OlapMeta* meta = store->get_meta(); + LOG(INFO) << "start to remove header, key:" << key; + OLAPStatus res = meta->remove(META_COLUMN_FAMILY_INDEX, key); + LOG(INFO) << "remove header, key:" << key << ", res:" << res; + return res; +} + +OLAPStatus OlapHeaderManager::get_header_converted(OlapStore* store, bool& flag) { + // get is_header_converted flag + std::string value; + std::string key = IS_HEADER_CONVERTED; + OlapMeta* meta = store->get_meta(); + OLAPStatus s = meta->get(DEFAULT_COLUMN_FAMILY_INDEX, key, value); + if (s == OLAP_ERR_META_KEY_NOT_FOUND || value == "false") { + flag = false; + } else if (value == "true") { + flag = true; + } else { + LOG(WARNING) << "invalid _is_header_converted. _is_header_converted=" << value; + return OLAP_ERR_HEADER_INVALID_FLAG; + } + return OLAP_SUCCESS; +} + +OLAPStatus OlapHeaderManager::set_converted_flag(OlapStore* store) { + OlapMeta* meta = store->get_meta(); + OLAPStatus s = meta->put(DEFAULT_COLUMN_FAMILY_INDEX, IS_HEADER_CONVERTED, CONVERTED_FLAG); + return s; +} + +OLAPStatus OlapHeaderManager::traverse_headers(OlapMeta* meta, + std::function const& func) { + auto traverse_header_func = [&func](const std::string& key, const std::string& value) -> bool { + std::vector parts; + // key format: "hdr_" + tablet_id + "_" + schema_hash + split_string(key, '_', &parts); + if (parts.size() != 3) { + LOG(WARNING) << "invalid header key:" << key << ", splitted size:" << parts.size(); + return true; + } + TTabletId tablet_id = std::stol(parts[1].c_str(), NULL, 10); + TSchemaHash schema_hash = std::stol(parts[2].c_str(), NULL, 10); + return func(tablet_id, schema_hash, value); + }; + OLAPStatus status = meta->iterate(META_COLUMN_FAMILY_INDEX, HEADER_PREFIX, traverse_header_func); + return status; +} + +OLAPStatus OlapHeaderManager::load_json_header(OlapStore* store, const std::string& header_path) { + std::ifstream infile(header_path); + char buffer[1024]; + std::string json_header; + while (!infile.eof()) { + infile.getline(buffer, 1024); + json_header = json_header + buffer; + } + boost::algorithm::trim(json_header); + OLAPHeader header; + bool ret = json2pb::JsonToProtoMessage(json_header, &header); + if (!ret) { + return OLAP_ERR_HEADER_LOAD_JSON_HEADER; + } + TTabletId tablet_id = header.tablet_id(); + TSchemaHash schema_hash = header.schema_hash(); + OLAPStatus s = save(store, tablet_id, schema_hash, &header); + return s; +} + +OLAPStatus OlapHeaderManager::dump_header(OlapStore* store, TTabletId tablet_id, + TSchemaHash schema_hash, const std::string& dump_path) { + OLAPHeader header; + OLAPStatus res = OlapHeaderManager::get_header(store, tablet_id, schema_hash, &header); + if (res != OLAP_SUCCESS) { + return res; + } + res = header.save(dump_path); + return res; +} + +} diff --git a/be/src/olap/olap_header_manager.h b/be/src/olap/olap_header_manager.h new file mode 100644 index 0000000000..bc3b9a513b --- /dev/null +++ b/be/src/olap/olap_header_manager.h @@ -0,0 +1,54 @@ +// Copyright (c) 2018, Baidu.com, Inc. All Rights Reserved + +// Licensed 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. + +#ifndef BDG_PALO_BE_SRC_OLAP_OLAP_HEADER_MANAGER_H +#define BDG_PALO_BE_SRC_OLAP_OLAP_HEADER_MANAGER_H + +#include + +#include "olap/olap_header.h" +#include "olap/olap_define.h" +#include "olap/store.h" + +namespace palo { + +// Helper Class for managing olap table headers of one root path. +class OlapHeaderManager { +public: + static OLAPStatus get_header(OlapStore* store, TTabletId tablet_id, TSchemaHash schema_hash, OLAPHeader* header); + + static OLAPStatus get_json_header(OlapStore* store, TTabletId tablet_id, + TSchemaHash schema_hash, std::string* json_header); + + static OLAPStatus save(OlapStore* store, TTabletId tablet_id, TSchemaHash schema_hash, const OLAPHeader* header); + + static OLAPStatus remove(OlapStore* store, TTabletId tablet_id, TSchemaHash schema_hash); + + static OLAPStatus traverse_headers(OlapMeta* meta, + std::function const& func); + + static OLAPStatus get_header_converted(OlapStore* store, bool& flag); + + static OLAPStatus set_converted_flag(OlapStore* store); + + static OLAPStatus load_json_header(OlapStore* store, const std::string& header_path); + + static OLAPStatus dump_header(OlapStore* store, TTabletId tablet_id, + TSchemaHash schema_hash, const std::string& path); +}; + +} + +#endif // BDG_PALO_BE_SRC_OLAP_OLAP_HEADER_MANAGER_H diff --git a/be/src/olap/olap_meta.cpp b/be/src/olap/olap_meta.cpp new file mode 100644 index 0000000000..9be07832ef --- /dev/null +++ b/be/src/olap/olap_meta.cpp @@ -0,0 +1,152 @@ +// Copyright (c) 2018, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 "olap/olap_meta.h" + +#include +#include + +#include "olap/olap_define.h" +#include "rocksdb/db.h" +#include "rocksdb/slice.h" +#include "rocksdb/options.h" +#include "rocksdb/slice_transform.h" +#include "common/logging.h" + +using rocksdb::DB; +using rocksdb::DBOptions; +using rocksdb::ColumnFamilyDescriptor; +using rocksdb::ColumnFamilyHandle; +using rocksdb::ColumnFamilyOptions; +using rocksdb::ReadOptions; +using rocksdb::WriteOptions; +using rocksdb::Slice; +using rocksdb::Iterator; +using rocksdb::Status; +using rocksdb::kDefaultColumnFamilyName; +using rocksdb::NewFixedPrefixTransform; + +namespace palo { +const std::string META_POSTFIX = "/meta"; +const size_t PREFIX_LENGTH = 4; + +OlapMeta::OlapMeta(const std::string& root_path) + : _root_path(root_path), + _db(nullptr) { +} + +OlapMeta::~OlapMeta() { + for (auto handle : _handles) { + delete handle; + } + if (_db != NULL) { + _db->Close(); + delete _db; + _db= NULL; + } +} + +OLAPStatus OlapMeta::init() { + // init db + DBOptions options; + options.IncreaseParallelism(); + options.create_if_missing = true; + options.create_missing_column_families = true; + std::string db_path = _root_path + META_POSTFIX; + std::vector column_families; + // default column family is required + column_families.emplace_back(DEFAULT_COLUMN_FAMILY, ColumnFamilyOptions()); + column_families.emplace_back(DORIS_COLUMN_FAMILY, ColumnFamilyOptions()); + + // meta column family add prefix extrator to improve performance and ensure correctness + ColumnFamilyOptions meta_column_family; + meta_column_family.prefix_extractor.reset(NewFixedPrefixTransform(PREFIX_LENGTH)); + column_families.emplace_back(META_COLUMN_FAMILY, meta_column_family); + Status s = DB::Open(options, db_path, column_families, &_handles, &_db); + if (!s.ok() || _db == NULL) { + LOG(WARNING) << "rocks db open failed, reason:" << s.ToString(); + return OLAP_ERR_META_OPEN_DB; + } + return OLAP_SUCCESS; +} + +OLAPStatus OlapMeta::get(const int column_family_index, const std::string& key, std::string& value) { + rocksdb::ColumnFamilyHandle* handle = _handles[column_family_index]; + Status s = _db->Get(ReadOptions(), handle, Slice(key), &value); + if (s.IsNotFound()) { + LOG(WARNING) << "rocks db key not found:" << key; + return OLAP_ERR_META_KEY_NOT_FOUND; + } else if (!s.ok()) { + LOG(WARNING) << "rocks db get key:" << key << " failed, reason:" << s.ToString(); + return OLAP_ERR_META_GET; + } + return OLAP_SUCCESS; +} + +OLAPStatus OlapMeta::put(const int column_family_index, const std::string& key, const std::string& value) { + rocksdb::ColumnFamilyHandle* handle = _handles[column_family_index]; + Status s = _db->Put(WriteOptions(), handle, Slice(key), Slice(value)); + if (!s.ok()) { + LOG(WARNING) << "rocks db put key:" << key << " failed, reason:" << s.ToString(); + return OLAP_ERR_META_PUT; + } + return OLAP_SUCCESS; +} + +OLAPStatus OlapMeta::remove(const int column_family_index, const std::string& key) { + rocksdb::ColumnFamilyHandle* handle = _handles[column_family_index]; + Status s = _db->Delete(WriteOptions(), handle, Slice(key)); + if (!s.ok()) { + LOG(WARNING) << "rocks db delete key:" << key << " failed, reason:" << s.ToString(); + return OLAP_ERR_META_DELETE; + } + return OLAP_SUCCESS; +} + +OLAPStatus OlapMeta::iterate(const int column_family_index, const std::string& prefix, + std::function const& func) { + rocksdb::ColumnFamilyHandle* handle = _handles[column_family_index]; + std::unique_ptr it(_db->NewIterator(ReadOptions(), handle)); + if (prefix == "") { + it->SeekToFirst(); + } else { + it->Seek(prefix); + } + Status status = it->status(); + if (!status.ok()) { + LOG(WARNING) << "rocksdb seek failed. reason:" << status.ToString(); + return OLAP_ERR_META_ITERATOR; + } + for (; it->Valid(); it->Next()) { + if (prefix != "") { + if (!it->key().starts_with(prefix)) { + return OLAP_SUCCESS; + } + } + std::string key = it->key().ToString(); + std::string value = it->value().ToString(); + bool ret = func(key, value); + if (!ret) { + break; + } + } + return OLAP_SUCCESS; +} + +std::string OlapMeta::get_root_path() { + return _root_path; +} + +} diff --git a/be/src/olap/olap_meta.h b/be/src/olap/olap_meta.h new file mode 100644 index 0000000000..b1bdac4e09 --- /dev/null +++ b/be/src/olap/olap_meta.h @@ -0,0 +1,56 @@ +// Copyright (c) 2018, Baidu.com, Inc. All Rights Reserved + +// Licensed 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. + +#ifndef BDG_PALO_BE_SRC_OLAP_OLAP_OLAP_META_H +#define BDG_PALO_BE_SRC_OLAP_OLAP_OLAP_META_H + +#include +#include +#include + +#include "olap/olap_header.h" +#include "olap/olap_define.h" +#include "rocksdb/db.h" + +namespace palo { + +class OlapMeta { +public: + OlapMeta(const std::string& root_path); + + virtual ~OlapMeta(); + + OLAPStatus init(); + + OLAPStatus get(const int column_family_index, const std::string& key, std::string& value); + + OLAPStatus put(const int column_family_index, const std::string& key, const std::string& value); + + OLAPStatus remove(const int column_family_index, const std::string& key); + + OLAPStatus iterate(const int column_family_index, const std::string& prefix, + std::function const& func); + + std::string get_root_path(); + +private: + std::string _root_path; + rocksdb::DB* _db; + std::vector _handles; +}; + +} + +#endif // BDG_PALO_BE_SRC_OLAP_OLAP_OLAP_META_H diff --git a/be/src/olap/options.cpp b/be/src/olap/options.cpp new file mode 100644 index 0000000000..107188ba9e --- /dev/null +++ b/be/src/olap/options.cpp @@ -0,0 +1,69 @@ +// Copyright (c) 2017, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 "olap/options.h" + +#include +#include +#include + +#include "common/logging.h" + +#include "olap/utils.h" + +namespace palo { + +// compatible with old multi path configuration: +// /path1,2014;/path2,2048 +OLAPStatus parse_conf_store_paths( + const std::string& config_path, + std::vector* paths) { + try { + std::vector item_vec; + boost::split(item_vec, config_path, boost::is_any_of(";"), boost::token_compress_on); + for (auto& item : item_vec) { + std::vector tmp_vec; + boost::split(tmp_vec, item, boost::is_any_of(","), boost::token_compress_on); + + // parse root path name + boost::trim(tmp_vec[0]); + tmp_vec[0].erase(tmp_vec[0].find_last_not_of("/") + 1); + if (tmp_vec[0].empty() || tmp_vec[0][0] != '/') { + LOG(WARNING) << "invalid store path. path=" << tmp_vec[0]; + return OLAP_ERR_INPUT_PARAMETER_ERROR; + } + + // parse root path capacity + int64_t capacity_bytes = -1; + if (tmp_vec.size() > 1) { + if (!valid_signed_number(tmp_vec[1]) + || strtol(tmp_vec[1].c_str(), NULL, 10) < 0) { + LOG(WARNING) << "invalid capacity of store path, capacity=" << tmp_vec[1]; + return OLAP_ERR_INPUT_PARAMETER_ERROR; + } + capacity_bytes = strtol(tmp_vec[1].c_str(), NULL, 10) * GB_EXCHANGE_BYTE; + } + + paths->emplace_back(tmp_vec[0], capacity_bytes); + } + } catch (...) { + LOG(WARNING) << "get config store path failed. path=" << config_path; + return OLAP_ERR_INPUT_PARAMETER_ERROR; + } + + return OLAP_SUCCESS; +} + +} diff --git a/be/src/olap/options.h b/be/src/olap/options.h new file mode 100644 index 0000000000..151c41202c --- /dev/null +++ b/be/src/olap/options.h @@ -0,0 +1,40 @@ +// Copyright (c) 2017, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 "olap/olap_define.h" + +namespace palo { + +struct StorePath { + StorePath() : capacity_bytes(-1) { } + StorePath(const std::string& path_, int64_t capacity_bytes_) + : path(path_), capacity_bytes(capacity_bytes_) { } + std::string path; + int64_t capacity_bytes; +}; + +OLAPStatus parse_conf_store_paths(const std::string& config_path, std::vector* path); + +struct EngineOptions { + // list paths that tablet will be put into. + std::vector store_paths; +}; + +} diff --git a/be/src/olap/rowset.cpp b/be/src/olap/rowset.cpp new file mode 100644 index 0000000000..454a88f3a6 --- /dev/null +++ b/be/src/olap/rowset.cpp @@ -0,0 +1,635 @@ +// Copyright (c) 2017, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 "olap/rowset.h" + +#include +#include +#include +#include + +#include "olap/olap_data.h" +#include "olap/olap_table.h" +#include "olap/row_block.h" +#include "olap/row_cursor.h" +#include "olap/utils.h" +#include "olap/wrapper_field.h" + +using std::ifstream; +using std::string; +using std::vector; + +namespace palo { + +#define TABLE_PARAM_VALIDATE() \ + do { \ + if (!_index_loaded) { \ + OLAP_LOG_WARNING("fail to find, index is not loaded. [table=%ld schema_hash=%d]", \ + _table->tablet_id(), \ + _table->schema_hash()); \ + return OLAP_ERR_NOT_INITED; \ + } \ + } while (0); + +#define POS_PARAM_VALIDATE(pos) \ + do { \ + if (NULL == pos) { \ + OLAP_LOG_WARNING("fail to find, NULL position parameter."); \ + return OLAP_ERR_INPUT_PARAMETER_ERROR; \ + } \ + } while (0); + +#define SLICE_PARAM_VALIDATE(slice) \ + do { \ + if (NULL == slice) { \ + OLAP_LOG_WARNING("fail to find, NULL slice parameter."); \ + return OLAP_ERR_INPUT_PARAMETER_ERROR; \ + } \ + } while (0); + +Rowset::Rowset(OLAPTable* table, Version version, VersionHash version_hash, + bool delete_flag, int32_t rowset_id, int32_t num_segments) + : _table(table), + _version(version), + _version_hash(version_hash), + _delete_flag(delete_flag), + _rowset_id(rowset_id), + _num_segments(num_segments) { + _index_loaded = false; + _ref_count = 0; + _is_pending = false; + _partition_id = 0; + _transaction_id = 0; + _short_key_length = 0; + _new_short_key_length = 0; + _short_key_buf = nullptr; + _file_created = false; + _new_segment_created = false; + _empty = false; + + const RowFields& tablet_schema = _table->tablet_schema(); + for (size_t i = 0; i < _table->num_short_key_fields(); ++i) { + _short_key_info_list.push_back(tablet_schema[i]); + _short_key_length += tablet_schema[i].index_length + 1;// 1 for null byte + if (tablet_schema[i].type == OLAP_FIELD_TYPE_CHAR || + tablet_schema[i].type == OLAP_FIELD_TYPE_VARCHAR) { + _new_short_key_length += sizeof(StringSlice) + 1; + } else { + _new_short_key_length += tablet_schema[i].index_length + 1; + } + } +} + +Rowset::Rowset(OLAPTable* table, bool delete_flag, + int32_t rowset_id, int32_t num_segments, bool is_pending, + TPartitionId partition_id, TTransactionId transaction_id) + : _table(table), _delete_flag(delete_flag), + _rowset_id(rowset_id), _num_segments(num_segments), + _is_pending(is_pending), _partition_id(partition_id), + _transaction_id(transaction_id) +{ + _version = {-1, -1}; + _version_hash = 0; + _load_id.set_hi(0); + _load_id.set_lo(0); + _index_loaded = false; + _ref_count = 0; + _short_key_length = 0; + _new_short_key_length = 0; + _short_key_buf = NULL; + _file_created = false; + _new_segment_created = false; + _empty = false; + + const RowFields& tablet_schema = _table->tablet_schema(); + for (size_t i = 0; i < _table->num_short_key_fields(); ++i) { + _short_key_info_list.push_back(tablet_schema[i]); + _short_key_length += tablet_schema[i].index_length + 1;// 1 for null byte + if (tablet_schema[i].type == OLAP_FIELD_TYPE_CHAR || + tablet_schema[i].type == OLAP_FIELD_TYPE_VARCHAR) { + _new_short_key_length += sizeof(StringSlice) + 1; + } else { + _new_short_key_length += tablet_schema[i].index_length + 1; + } + } +} + +Rowset::~Rowset() { + delete [] _short_key_buf; + _current_file_handler.close(); + + for (size_t i = 0; i < _column_statistics.size(); ++i) { + SAFE_DELETE(_column_statistics[i].first); + SAFE_DELETE(_column_statistics[i].second); + } + _seg_pb_map.clear(); +} + +string Rowset::construct_index_file_path(int32_t rowset_id, int32_t segment) const { + if (_is_pending) { + return _table->construct_pending_index_file_path(_transaction_id, _rowset_id, segment); + } else { + return _table->construct_index_file_path(_version, _version_hash, _rowset_id, segment); + } +} + +string Rowset::construct_data_file_path(int32_t rowset_id, int32_t segment) const { + if (_is_pending) { + return _table->construct_pending_data_file_path(_transaction_id, rowset_id, segment); + } else { + return _table->construct_data_file_path(_version, _version_hash, rowset_id, segment); + } +} + +void Rowset::publish_version(Version version, VersionHash version_hash) { + _version = version; + _version_hash = version_hash; +} + +void Rowset::acquire() { + atomic_inc(&_ref_count); +} + +int64_t Rowset::ref_count() { + return _ref_count; +} + +void Rowset::release() { + atomic_dec(&_ref_count); +} + +bool Rowset::is_in_use() { + return _ref_count > 0; +} + +// you can not use Rowset after delete_all_files(), or else unknown behavior occurs. +void Rowset::delete_all_files() { + if (!_file_created) { return; } + for (uint32_t seg_id = 0; seg_id < _num_segments; ++seg_id) { + // get full path for one segment + string index_path = construct_index_file_path(_rowset_id, seg_id); + string data_path = construct_data_file_path(_rowset_id, seg_id); + + if (remove(index_path.c_str()) != 0) { + char errmsg[64]; + LOG(WARNING) << "fail to delete index file. [err='" << strerror_r(errno, errmsg, 64) + << "' path='" << index_path << "']"; + } + + if (remove(data_path.c_str()) != 0) { + char errmsg[64]; + LOG(WARNING) << "fail to delete data file. [err='" << strerror_r(errno, errmsg, 64) + << "' path='" << data_path << "']"; + } + } +} + +OLAPStatus Rowset::add_column_statistics( + const std::vector>& column_statistic_fields) { + DCHECK(column_statistic_fields.size() == _table->num_key_fields()); + for (size_t i = 0; i < column_statistic_fields.size(); ++i) { + WrapperField* first = WrapperField::create(_table->tablet_schema()[i]); + DCHECK(first != NULL) << "failed to allocate memory for field: " << i; + first->copy(column_statistic_fields[i].first); + + WrapperField* second = WrapperField::create(_table->tablet_schema()[i]); + DCHECK(second != NULL) << "failed to allocate memory for field: " << i; + second->copy(column_statistic_fields[i].second); + + _column_statistics.push_back(std::make_pair(first, second)); + } + return OLAP_SUCCESS; +} + +OLAPStatus Rowset::add_column_statistics( + std::vector > &column_statistic_strings, + std::vector &null_vec) { + DCHECK(column_statistic_strings.size() == _table->num_key_fields()); + std::vector> column_statistics; + for (size_t i = 0; i < column_statistic_strings.size(); ++i) { + WrapperField* first = WrapperField::create(_table->tablet_schema()[i]); + DCHECK(first != NULL) << "failed to allocate memory for field: " << i ; + RETURN_NOT_OK(first->from_string(column_statistic_strings[i].first)); + if (null_vec[i]) { + //[min, max] -> [NULL, max] + first->set_null(); + } + WrapperField* second = WrapperField::create(_table->tablet_schema()[i]); + DCHECK(first != NULL) << "failed to allocate memory for field: " << i ; + RETURN_NOT_OK(second->from_string(column_statistic_strings[i].second)); + _column_statistics.push_back(std::make_pair(first, second)); + } + return OLAP_SUCCESS; +} + +OLAPStatus Rowset::load() { + if (_empty) { + return OLAP_SUCCESS; + } + OLAPStatus res = OLAP_ERR_INDEX_LOAD_ERROR; + boost::lock_guard guard(_index_load_lock); + + if (_index_loaded) { + return OLAP_SUCCESS; + } + + if (_num_segments == 0) { + OLAP_LOG_WARNING("fail to load index, segments number is 0."); + return res; + } + + if (_index.init(_short_key_length, _new_short_key_length, + _table->num_short_key_fields(), &_short_key_info_list) != OLAP_SUCCESS) { + OLAP_LOG_WARNING("fail to create MemIndex. [num_segment=%d]", _num_segments); + return res; + } + + // for each segment + for (uint32_t seg_id = 0; seg_id < _num_segments; ++seg_id) { + if (COLUMN_ORIENTED_FILE == _table->data_file_type()) { + string seg_path = construct_data_file_path(_rowset_id, seg_id); + if (OLAP_SUCCESS != (res = load_pb(seg_path.c_str(), seg_id))) { + LOG(WARNING) << "failed to load pb structures. [seg_path='" << seg_path << "']"; + _check_io_error(res); + return res; + } + } + + // get full path for one segment + string path = construct_index_file_path(_rowset_id, seg_id); + if ((res = _index.load_segment(path.c_str(), &_current_num_rows_per_row_block)) + != OLAP_SUCCESS) { + LOG(WARNING) << "fail to load segment. [path='" << path << "']"; + _check_io_error(res); + return res; + } + } + + _delete_flag = _index.delete_flag(); + _index_loaded = true; + _file_created = true; + + return OLAP_SUCCESS; +} + +OLAPStatus Rowset::load_pb(const char* file, uint32_t seg_id) { + OLAPStatus res = OLAP_SUCCESS; + + FileHeader seg_file_header; + FileHandler seg_file_handler; + res = seg_file_handler.open(file, O_RDONLY); + if (OLAP_SUCCESS != res) { + OLAP_LOG_WARNING("failed to open segment file. [err=%d, file=%s]", res, file); + return res; + } + + res = seg_file_header.unserialize(&seg_file_handler); + if (OLAP_SUCCESS != res) { + seg_file_handler.close(); + OLAP_LOG_WARNING("fail to unserialize header. [err=%d, path='%s']", res, file); + return res; + } + + _seg_pb_map[seg_id] = seg_file_header; + seg_file_handler.close(); + return OLAP_SUCCESS; +} + +bool Rowset::index_loaded() { + return _index_loaded; +} + +OLAPStatus Rowset::validate() { + if (_empty) { + return OLAP_SUCCESS; + } + + OLAPStatus res = OLAP_SUCCESS; + for (uint32_t seg_id = 0; seg_id < _num_segments; ++seg_id) { + FileHeader index_file_header; + FileHeader data_file_header; + + // get full path for one segment + string index_path = construct_index_file_path(_rowset_id, seg_id); + string data_path = construct_data_file_path(_rowset_id, seg_id); + + // 检查index文件头 + if ((res = index_file_header.validate(index_path)) != OLAP_SUCCESS) { + LOG(WARNING) << "validate index file error. [file='" << index_path << "']"; + _check_io_error(res); + return res; + } + + // 检查data文件头 + if ((res = data_file_header.validate(data_path)) != OLAP_SUCCESS) { + LOG(WARNING) << "validate data file error. [file='" << data_path << "']"; + _check_io_error(res); + return res; + } + } + + return OLAP_SUCCESS; +} + +OLAPStatus Rowset::find_row_block(const RowCursor& key, + RowCursor* helper_cursor, + bool find_last, + RowBlockPosition* pos) const { + TABLE_PARAM_VALIDATE(); + POS_PARAM_VALIDATE(pos); + + // 将这部分逻辑从memindex移出来,这样可以复用find。 + OLAPIndexOffset offset = _index.find(key, helper_cursor, find_last); + if (offset.offset > 0) { + offset.offset = offset.offset - 1; + } else { + offset.offset = 0; + } + + if (find_last) { + OLAPIndexOffset next_offset = _index.next(offset); + if (!(next_offset == _index.end())) { + offset = next_offset; + } + } + + return _index.get_row_block_position(offset, pos); +} + +OLAPStatus Rowset::find_short_key(const RowCursor& key, + RowCursor* helper_cursor, + bool find_last, + RowBlockPosition* pos) const { + TABLE_PARAM_VALIDATE(); + POS_PARAM_VALIDATE(pos); + + // 由于find会从前一个segment找起,如果前一个segment中恰好没有该key, + // 就用前移后移来移动segment的位置. + OLAPIndexOffset offset = _index.find(key, helper_cursor, find_last); + if (offset.offset > 0) { + offset.offset = offset.offset - 1; + + OLAPIndexOffset next_offset = _index.next(offset); + if (!(next_offset == _index.end())) { + offset = next_offset; + } + } + + OLAP_LOG_DEBUG("[seg='%d', offset='%d']", offset.segment, offset.offset); + return _index.get_row_block_position(offset, pos); +} + +OLAPStatus Rowset::get_row_block_entry(const RowBlockPosition& pos, EntrySlice* entry) const { + TABLE_PARAM_VALIDATE(); + SLICE_PARAM_VALIDATE(entry); + + return _index.get_entry(_index.get_offset(pos), entry); +} + +OLAPStatus Rowset::find_first_row_block(RowBlockPosition* position) const { + TABLE_PARAM_VALIDATE(); + POS_PARAM_VALIDATE(position); + + return _index.get_row_block_position(_index.find_first(), position); +} + +OLAPStatus Rowset::find_last_row_block(RowBlockPosition* position) const { + TABLE_PARAM_VALIDATE(); + POS_PARAM_VALIDATE(position); + + return _index.get_row_block_position(_index.find_last(), position); +} + +OLAPStatus Rowset::find_next_row_block(RowBlockPosition* pos, bool* eof) const { + TABLE_PARAM_VALIDATE(); + POS_PARAM_VALIDATE(pos); + POS_PARAM_VALIDATE(eof); + + OLAPIndexOffset current = _index.get_offset(*pos); + *eof = false; + + OLAPIndexOffset next = _index.next(current); + if (next == _index.end()) { + *eof = true; + return OLAP_ERR_INDEX_EOF; + } + + return _index.get_row_block_position(next, pos); +} + +OLAPStatus Rowset::find_mid_point(const RowBlockPosition& low, + const RowBlockPosition& high, + RowBlockPosition* output, + uint32_t* dis) const { + *dis = compute_distance(low, high); + if (*dis >= _index.count()) { + return OLAP_ERR_INDEX_EOF; + } else { + *output = low; + if (advance_row_block(*dis / 2, output) != OLAP_SUCCESS) { + return OLAP_ERR_INDEX_EOF; + } + + return OLAP_SUCCESS; + } +} + +OLAPStatus Rowset::find_prev_point( + const RowBlockPosition& current, RowBlockPosition* prev) const { + OLAPIndexOffset current_offset = _index.get_offset(current); + OLAPIndexOffset prev_offset = _index.prev(current_offset); + + return _index.get_row_block_position(prev_offset, prev); +} + +OLAPStatus Rowset::advance_row_block(int64_t num_row_blocks, RowBlockPosition* position) const { + TABLE_PARAM_VALIDATE(); + POS_PARAM_VALIDATE(position); + + OLAPIndexOffset off = _index.get_offset(*position); + iterator_offset_t absolute_offset = _index.get_absolute_offset(off) + num_row_blocks; + if (absolute_offset >= _index.count()) { + return OLAP_ERR_INDEX_EOF; + } + + return _index.get_row_block_position(_index.get_relative_offset(absolute_offset), position); +} + +// PRECONDITION position1 < position2 +uint32_t Rowset::compute_distance(const RowBlockPosition& position1, + const RowBlockPosition& position2) const { + iterator_offset_t offset1 = _index.get_absolute_offset(_index.get_offset(position1)); + iterator_offset_t offset2 = _index.get_absolute_offset(_index.get_offset(position2)); + + return offset2 > offset1 ? offset2 - offset1 : 0; +} + +OLAPStatus Rowset::add_segment() { + // 打开文件 + ++_num_segments; + + OLAPIndexHeaderMessage* index_header = NULL; + // 构造Proto格式的Header + index_header = _file_header.mutable_message(); + index_header->set_start_version(_version.first); + index_header->set_end_version(_version.second); + index_header->set_cumulative_version_hash(_version_hash); + index_header->set_segment(_num_segments - 1); + index_header->set_num_rows_per_block(_table->num_rows_per_row_block()); + index_header->set_delete_flag(_delete_flag); + index_header->set_null_supported(true); + + // 分配一段存储short key的内存, 初始化index_row + if (_short_key_buf == NULL) { + _short_key_buf = new(std::nothrow) char[_short_key_length]; + if (_short_key_buf == NULL) { + OLAP_LOG_WARNING("malloc short_key_buf error."); + return OLAP_ERR_MALLOC_ERROR; + } + + if (_current_index_row.init(_table->tablet_schema()) != OLAP_SUCCESS) { + OLAP_LOG_WARNING("init _current_index_row fail."); + return OLAP_ERR_INIT_FAILED; + } + } + + // 初始化checksum + _checksum = ADLER32_INIT; + return OLAP_SUCCESS; +} + +OLAPStatus Rowset::add_row_block(const RowBlock& row_block, const uint32_t data_offset) { + // get first row of the row_block to distill index item. + row_block.get_row(0, &_current_index_row); + return add_short_key(_current_index_row, data_offset); +} + +OLAPStatus Rowset::add_short_key(const RowCursor& short_key, const uint32_t data_offset) { + OLAPStatus res = OLAP_SUCCESS; + if (!_new_segment_created) { + string file_path = construct_index_file_path(_rowset_id, _num_segments - 1); + res = _current_file_handler.open_with_mode( + file_path.c_str(), O_CREAT | O_EXCL | O_WRONLY, S_IRUSR | S_IWUSR); + if (res != OLAP_SUCCESS) { + char errmsg[64]; + LOG(WARNING) << "can not create file. [file_path='" << file_path + << "' err='" << strerror_r(errno, errmsg, 64) << "']"; + _check_io_error(res); + return res; + } + _new_segment_created = true; + + // 准备FileHeader + if ((res = _file_header.prepare(&_current_file_handler)) != OLAP_SUCCESS) { + OLAP_LOG_WARNING("write file header error. [err=%m]"); + _check_io_error(res); + return res; + } + + // 跳过FileHeader + if (_current_file_handler.seek(_file_header.size(), SEEK_SET) == -1) { + OLAP_LOG_WARNING("lseek header file error. [err=%m]"); + res = OLAP_ERR_IO_ERROR; + _check_io_error(res); + return res; + } + } + + // 将short key的内容写入_short_key_buf + size_t offset = 0; + + //short_key.write_null_array(_short_key_buf); + //offset += short_key.get_num_null_byte(); + for (size_t i = 0; i < _short_key_info_list.size(); i++) { + short_key.write_index_by_index(i, _short_key_buf + offset); + offset += short_key.get_index_size(i) + 1; + } + + // 写入Short Key对应的数据 + if ((res = _current_file_handler.write(_short_key_buf, _short_key_length)) != OLAP_SUCCESS) { + OLAP_LOG_WARNING("write short key failed. [err=%m]"); + _check_io_error(res); + return res; + } + + // 写入对应的数据文件偏移量 + if ((res = _current_file_handler.write(&data_offset, sizeof(data_offset))) != OLAP_SUCCESS) { + OLAP_LOG_WARNING("write data_offset failed. [err=%m]"); + _check_io_error(res); + return res; + } + + _checksum = olap_adler32(_checksum, _short_key_buf, _short_key_length); + _checksum = olap_adler32(_checksum, + reinterpret_cast(&data_offset), + sizeof(data_offset)); + return OLAP_SUCCESS; +} + +OLAPStatus Rowset::finalize_segment(uint32_t data_segment_size, int64_t num_rows) { + // 准备FileHeader + OLAPStatus res = OLAP_SUCCESS; + + int file_length = _current_file_handler.tell(); + if (file_length == -1) { + OLAP_LOG_WARNING("get file_length error. [err=%m]"); + _check_io_error(res); + return OLAP_ERR_IO_ERROR; + } + + _file_header.set_file_length(file_length); + _file_header.set_checksum(_checksum); + _file_header.mutable_extra()->data_length = data_segment_size; + _file_header.mutable_extra()->num_rows = num_rows; + + // 写入更新之后的FileHeader + if ((res = _file_header.serialize(&_current_file_handler)) != OLAP_SUCCESS) { + OLAP_LOG_WARNING("write file header error. [err=%m]"); + _check_io_error(res); + return res; + } + + OLAP_LOG_DEBUG("finalize_segment. [file_name='%s' file_length=%d]", + _current_file_handler.file_name().c_str(), + file_length); + + if ((res = _current_file_handler.close()) != OLAP_SUCCESS) { + OLAP_LOG_WARNING("close file error. [err=%m]"); + _check_io_error(res); + return res; + } + + _new_segment_created = false; + return OLAP_SUCCESS; +} + +void Rowset::sync() { + if (_current_file_handler.sync() == -1) { + OLAP_LOG_WARNING("fail to sync file.[err=%m]"); + _table->set_io_error(); + } +} + +void Rowset::_check_io_error(OLAPStatus res) { + if (is_io_error(res)) { + _table->set_io_error(); + } +} + +uint64_t Rowset::num_index_entries() const { + return _index.count(); +} + +} diff --git a/be/src/olap/rowset.h b/be/src/olap/rowset.h new file mode 100644 index 0000000000..2fae93b28e --- /dev/null +++ b/be/src/olap/rowset.h @@ -0,0 +1,285 @@ +// Copyright (c) 2017, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 "gen_cpp/olap_file.pb.h" +#include "gen_cpp/column_data_file.pb.h" +#include "olap/atomic.h" +#include "olap/field.h" +#include "olap/file_helper.h" +#include "olap/olap_common.h" +#include "olap/olap_define.h" +#include "olap/olap_table.h" +#include "olap/row_cursor.h" +#include "olap/olap_index.h" +#include "olap/utils.h" + +namespace palo { + +// Class for managing OLAP table indices +// For fast key lookup, we maintain a sparse index for every data file. The +// index is sparse because we only have one pointer per row block. Each +// index entry contains the short key for the first row of the +// corresponding row block +class Rowset { + friend class MemIndex; +public: + Rowset(OLAPTable* table, Version version, VersionHash version_hash, + bool delete_flag, int rowset_id, int32_t num_segments); + + Rowset(OLAPTable* table, bool delete_flag, int32_t rowset_id, + int32_t num_segments, bool is_pending, + TPartitionId partition_id, TTransactionId transaction_id); + + virtual ~Rowset(); + + // Load the index into memory. + OLAPStatus load(); + bool index_loaded(); + OLAPStatus load_pb(const char* file, uint32_t seg_id); + + bool has_column_statistics() { + return _column_statistics.size() != 0; + } + + OLAPStatus add_column_statistics( + const std::vector>& column_statistic_fields); + + OLAPStatus add_column_statistics( + std::vector> &column_statistic_strings, + std::vector &null_vec); + + const std::vector>& get_column_statistics() { + return _column_statistics; + } + + // 检查index文件和data文件的有效性 + OLAPStatus validate(); + + // Finds position of the first (or last if find_last is set) row + // block that may contain the smallest key equal to or greater than + // 'key'. Returns true on success. If find_last is set, note that + // the position is the last block that can possibly contain the + // given key. + OLAPStatus find_row_block(const RowCursor& key, + RowCursor* helper_cursor, + bool find_last, + RowBlockPosition* position) const; + + // Finds position of first row block contain the smallest key equal + // to or greater than 'key'. Returns true on success. + OLAPStatus find_short_key(const RowCursor& key, + RowCursor* helper_cursor, + bool find_last, + RowBlockPosition* position) const; + + // Returns position of the first row block in the index. + OLAPStatus find_first_row_block(RowBlockPosition* position) const; + + // Returns position of the last row block in the index. + OLAPStatus find_last_row_block(RowBlockPosition* position) const; + + // Given the position of a row block, finds position of the next block. + // Sets eof to tru if there are no more blocks to go through, and + // returns false. Returns true on success. + OLAPStatus find_next_row_block(RowBlockPosition* position, bool* eof) const; + + // Given two positions in an index, low and high, set output to be + // the midpoint between those two positions. Returns the distance + // between low and high as computed by ComputeDistance. + OLAPStatus find_mid_point(const RowBlockPosition& low, + const RowBlockPosition& high, + RowBlockPosition* output, + uint32_t* dis) const; + + OLAPStatus find_prev_point(const RowBlockPosition& current, RowBlockPosition* prev) const; + + OLAPStatus get_row_block_entry(const RowBlockPosition& pos, EntrySlice* entry) const; + + // Given a starting row block position, advances the position by + // num_row_blocks, then stores back the new position through the + // pointer. Returns true on success, false on attempt to seek past + // the last block. + OLAPStatus advance_row_block(int64_t num_row_blocks, RowBlockPosition* position) const; + + // Computes the distance between two positions, in row blocks. + uint32_t compute_distance(const RowBlockPosition& position1, + const RowBlockPosition& position2) const; + + // The following four functions are used for creating new index + // files. AddSegment() and FinalizeSegment() start and end a new + // segment respectively, while IndexRowBlock() and IndexShortKey() + // add a new index entry to the current segment. + OLAPStatus add_segment(); + OLAPStatus add_short_key(const RowCursor& short_key, const uint32_t data_offset); + OLAPStatus add_row_block(const RowBlock& row_block, const uint32_t data_offset); + OLAPStatus finalize_segment(uint32_t data_segment_size, int64_t num_rows); + void sync(); + + // reference count + void acquire(); + void release(); + bool is_in_use(); + int64_t ref_count(); + + // delete all files (*.idx; *.dat) + void delete_all_files(); + + // getters and setters. + // get associated OLAPTable pointer + inline OLAPTable* table() const { return _table; } + inline void set_table(OLAPTable* table) { _table = table; } + + inline Version version() const { return _version; } + inline VersionHash version_hash() const { return _version_hash; } + + inline bool is_pending() const { return _is_pending; } + inline void set_pending_finished() { _is_pending = false; } + + inline TPartitionId partition_id() const { return _partition_id; } + inline TTransactionId transaction_id() const { return _transaction_id; } + + inline bool delete_flag() const { return _delete_flag; } + + inline int32_t rowset_id() const { return _rowset_id; } + inline void set_rowset_id(int32_t rowset_id) { _rowset_id = rowset_id; } + + inline PUniqueId load_id() const { return _load_id; } + inline void set_load_id(const PUniqueId& load_id) { _load_id = load_id; } + + inline int32_t num_segments() const { return _num_segments; } + inline void set_num_segments(int32_t num_segments) { _num_segments = num_segments; } + + size_t index_size() const { + return _index.index_size(); + } + + size_t data_size() const { + return _index.data_size(); + } + + int64_t num_rows() const { + return _index.num_rows(); + } + + const size_t short_key_length() const { + return _short_key_length; + } + + const size_t new_short_key_length() const { + return _new_short_key_length; + } + + const RowFields& short_key_fields() const { + return _short_key_info_list; + } + + bool empty() const { + return _empty; + } + + bool zero_num_rows() const { + // previous version may has non-sense file in disk. + // to be compatible, it should be handled. + return _index.zero_num_rows(); + } + + void set_empty(bool empty) { + _empty = empty; + } + + // return count of entries in MemIndex + uint64_t num_index_entries() const; + + size_t current_num_rows_per_row_block() const { + return _current_num_rows_per_row_block; + } + + OLAPStatus get_row_block_position(const OLAPIndexOffset& pos, RowBlockPosition* rbp) const { + return _index.get_row_block_position(pos, rbp); + } + + inline const FileHeader* get_seg_pb(uint32_t seg_id) const { + return &(_seg_pb_map.at(seg_id)); + } + + inline bool get_null_supported(uint32_t seg_id) { + return _index.get_null_supported(seg_id); + } + + std::string construct_index_file_path(int32_t rowset_id, int32_t segment) const; + std::string construct_data_file_path(int32_t rowset_id, int32_t segment) const; + void publish_version(Version version, VersionHash version_hash); + +private: + void _check_io_error(OLAPStatus res); + + OLAPTable* _table; // table definition for this index + Version _version; // version of associated data file + VersionHash _version_hash; // version hash for this index + bool _delete_flag; + int32_t _rowset_id; // rowset id of olapindex + PUniqueId _load_id; // load id for rowset + int32_t _num_segments; // number of segments in this index + bool _index_loaded; // whether the index has been read + atomic_t _ref_count; // reference count + MemIndex _index; + bool _is_pending; + TPartitionId _partition_id; + TTransactionId _transaction_id; + + // short key对应的field_info数组 + RowFields _short_key_info_list; + // short key对应的总长度 + size_t _short_key_length; + size_t _new_short_key_length; + // 当前写入的short_key的buf + char* _short_key_buf; + // 当前写入的segment的checksum + uint32_t _checksum; + // 当前写入时用作索引项的RowCursor + RowCursor _current_index_row; + + // 以下是写入流程时需要的一些中间状态 + // 当前写入文件的FileHandler + FileHandler _current_file_handler; + bool _file_created; + bool _new_segment_created; + // 当前写入的FileHeader + FileHeader _file_header; + + bool _empty; + + // Lock held while loading the index. + mutable boost::mutex _index_load_lock; + + size_t _current_num_rows_per_row_block; + + std::vector> _column_statistics; + std::unordered_map > _seg_pb_map; + + DISALLOW_COPY_AND_ASSIGN(Rowset); +}; + +} diff --git a/be/src/olap/rowset_builder.h b/be/src/olap/rowset_builder.h new file mode 100644 index 0000000000..bd95a21f74 --- /dev/null +++ b/be/src/olap/rowset_builder.h @@ -0,0 +1,63 @@ +// Copyright (c) 2017, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 "olap/writer.h" + +namespace palo { + +class Rowset; + +class RowsetBuilder : public IWriter { +public: + RowsetBuil(OLAPTablePtr table, Rowset* rowset, IWriter* writer, bool is_push_write) + : IWriter(is_push_write, table), + _rowset(rowset), + _writer(write) { + } + + virtual ~RowSetBuilder() { + } + + OLAPStatus init() override { + return _writer->init(); + } + + OLAPStatus attached_by(RowCursor* row_cursor) override { + return _writer->attached_by(row_cursor); + } + OLAPStatus write(const char* row) override { + return _writer->write(row); + } + OLAPStatus finalize() override { + return _writer->finalize(); + } + uint64_t written_bytes() override { + return _writer->written_bytes(); + } + MemPool* mem_pool() override { + return _writer->mem_pool(); + } + + Rowset* rowset() { return _rowset; } + IWriter* writer() { return _writer; } + +private: + Rowset* _rowset; + IWriter* _writer; +}; + +} diff --git a/be/src/olap/schema.h b/be/src/olap/schema.h new file mode 100644 index 0000000000..aff6a86434 --- /dev/null +++ b/be/src/olap/schema.h @@ -0,0 +1,165 @@ +// Copyright (c) 2018, Baidu.com, Inc. All Rights Reserved + +// Licensed 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. + +#ifndef BDG_PALO_BE_SRC_OLAP_SCHEMA_H +#define BDG_PALO_BE_SRC_OLAP_SCHEMA_H + +#include + +#include "olap/aggregate_func.h" +#include "olap/types.h" +#include "runtime/descriptors.h" + +namespace palo { + +class ColumnSchema { +public: + ColumnSchema(const FieldAggregationMethod& agg, const FieldType& type) { + _type_info = get_type_info(type); + _aggregate_func = get_aggregate_func(agg, type); + _finalize_func = get_finalize_func(agg, type); + _size = _type_info->size(); + _col_offset = 0; + } + + bool is_null(const char* row) const { + bool is_null = *reinterpret_cast(row + _col_offset); + return is_null; + } + + void set_null(char* row) const { + *reinterpret_cast(row + _col_offset) = true; + } + + void set_not_null(char* row) const { + *reinterpret_cast(row + _col_offset) = false; + } + + void set_col_offset(int offset) { + _col_offset = offset; + } + + int get_col_offset() const { + return _col_offset; + } + + int compare(const char* left, const char* right) const { + bool l_null = *reinterpret_cast(left + _col_offset); + bool r_null = *reinterpret_cast(right + _col_offset); + if (l_null != r_null) { + return l_null ? -1 : 1; + } else { + return l_null ? 0 : (_type_info->cmp(left + _col_offset + 1, right + _col_offset + 1)); + } + } + + void aggregate(char* left, const char* right, Arena* arena) const { + _aggregate_func(left + _col_offset, right + _col_offset, arena); + } + + void finalize(char* data) const { + // data of Hyperloglog type will call this function. + _finalize_func(data + _col_offset + 1); + } + + int size() const { + return _size; + } +private: + FieldType _type; + TypeInfo* _type_info; + AggregateFunc _aggregate_func; + FinalizeFunc _finalize_func; + int _size; + int _col_offset; +}; + +class Schema { +public: + Schema(const std::vector& field_infos) { + int offset = 0; + _num_key_columns = 0; + for (int i = 0; i < field_infos.size(); ++i) { + FieldInfo field_info = field_infos[i]; + ColumnSchema col_schema(field_info.aggregation, field_info.type); + col_schema.set_col_offset(offset); + offset += col_schema.size() + 1; // 1 for null byte + if (field_info.is_key) { + _num_key_columns++; + } + if (field_info.type == OLAP_FIELD_TYPE_HLL) { + _hll_col_ids.push_back(i); + } + _cols.push_back(col_schema); + } + } + + int compare(const char* left , const char* right) const { + for (size_t i = 0; i < _num_key_columns; ++i) { + int comp = _cols[i].compare(left, right); + if (comp != 0) { + return comp; + } + } + return 0; + } + + void aggregate(const char* left, const char* right, Arena* arena) const { + for (size_t i = _num_key_columns; i < _cols.size(); ++i) { + _cols[i].aggregate(const_cast(left), right, arena); + } + } + + void finalize(const char* data) const { + for (int col_id : _hll_col_ids) { + _cols[col_id].finalize(const_cast(data)); + } + } + + int get_col_offset(int index) const { + return _cols[index].get_col_offset(); + } + size_t get_col_size(int index) const { + return _cols[index].size(); + } + + bool is_null(int index, const char* row) const { + return _cols[index].is_null(row); + } + + void set_null(int index, char*row) { + _cols[index].set_null(row); + } + + void set_not_null(int index, char*row) { + _cols[index].set_not_null(row); + } + + size_t schema_size() { + size_t size = _cols.size(); + for (auto col : _cols) { + size += col.size(); + } + return size; + } +private: + std::vector _cols; + size_t _num_key_columns; + std::vector _hll_col_ids; +}; + +} // namespace palo + +#endif // BDG_PALO_BE_SRC_OLAP_SCHEMA_H diff --git a/be/src/olap/skiplist.h b/be/src/olap/skiplist.h new file mode 100644 index 0000000000..4087e2b52c --- /dev/null +++ b/be/src/olap/skiplist.h @@ -0,0 +1,397 @@ +// Copyright (c) 2011 The LevelDB Authors. All rights reserved. +// Use of this source code is governed by a BSD-style license that can be +// found in the LICENSE file. See the AUTHORS file for names of contributors. + +#ifndef BDG_PALO_BE_SRC_OLAP_SKIPLIST_H +#define BDG_PALO_BE_SRC_OLAP_SKIPLIST_H + +// Thread safety +// ------------- +// +// Writes require external synchronization, most likely a mutex. +// Reads require a guarantee that the SkipList will not be destroyed +// while the read is in progress. Apart from that, reads progress +// without any internal locking or synchronization. +// +// Invariants: +// +// (1) Allocated nodes are never deleted until the SkipList is +// destroyed. This is trivially guaranteed by the code since we +// never delete any skip list nodes. +// +// (2) The contents of a Node except for the next/prev pointers are +// immutable after the Node has been linked into the SkipList. +// Only Insert() modifies the list, and it is careful to initialize +// a node and use release-stores to publish the nodes in one or +// more lists. +// +// ... prev vs. next pointer ordering ... + +#include + +#include "common/logging.h" +#include "gen_cpp/olap_file.pb.h" +#include "util/arena.h" +#include "util/random.h" + +namespace palo { + +class Arena; + +template +class SkipList { +private: + struct Node; + +public: + // Create a new SkipList object that will use "cmp" for comparing keys, + // and will allocate memory using "*arena". Objects allocated in the arena + // must remain allocated for the lifetime of the skiplist object. + explicit SkipList(Comparator cmp, Arena* arena); + + // Insert key into the list. + // REQUIRES: nothing that compares equal to key is currently in the list. + void Insert(const Key& key, bool* overwritten, KeysType keys_type); + void Aggregate(const Key& k1, const Key& k2); + + // Returns true iff an entry that compares equal to key is in the list. + bool Contains(const Key& key) const; + + // Iteration over the contents of a skip list + class Iterator { + public: + // Initialize an iterator over the specified list. + // The returned iterator is not valid. + explicit Iterator(const SkipList* list); + + // Returns true iff the iterator is positioned at a valid node. + bool Valid() const; + + // Returns the key at the current position. + // REQUIRES: Valid() + const Key& key() const; + + // Advances to the next position. + // REQUIRES: Valid() + void Next(); + + // Advances to the previous position. + // REQUIRES: Valid() + void Prev(); + + // Advance to the first entry with a key >= target + void Seek(const Key& target); + + // Position at the first entry in list. + // Final state of iterator is Valid() iff list is not empty. + void SeekToFirst(); + + // Position at the last entry in list. + // Final state of iterator is Valid() iff list is not empty. + void SeekToLast(); + + private: + const SkipList* list_; + Node* node_; + // Intentionally copyable + }; + +private: + enum { kMaxHeight = 12 }; + + // Immutable after construction + Comparator const compare_; + Arena* const arena_; // Arena used for allocations of nodes + + Node* const head_; + + // Modified only by Insert(). Read racily by readers, but stale + // values are ok. + std::atomic max_height_; // Height of the entire list + + inline int GetMaxHeight() const { + return max_height_.load(std::memory_order_relaxed); + } + + // Read/written only by Insert(). + Random rnd_; + + Node* NewNode(const Key& key, int height); + int RandomHeight(); + bool Equal(const Key& a, const Key& b) const { return (compare_(a, b) == 0); } + + // Return true if key is greater than the data stored in "n" + bool KeyIsAfterNode(const Key& key, Node* n) const; + + // Return the earliest node that comes at or after key. + // Return NULL if there is no such node. + // + // If prev is non-NULL, fills prev[level] with pointer to previous + // node at "level" for every level in [0..max_height_-1]. + Node* FindGreaterOrEqual(const Key& key, Node** prev) const; + + // Return the latest node with a key < key. + // Return head_ if there is no such node. + Node* FindLessThan(const Key& key) const; + + // Return the last node in the list. + // Return head_ if list is empty. + Node* FindLast() const; + + // No copying allowed + SkipList(const SkipList&); + void operator=(const SkipList&); +}; + +// Implementation details follow +template +struct SkipList::Node { + explicit Node(const Key& k) : key(k) { } + + Key const key; + + // Accessors/mutators for links. Wrapped in methods so we can + // add the appropriate barriers as necessary. + Node* Next(int n) { + DCHECK(n >= 0); + // Use an 'acquire load' so that we observe a fully initialized + // version of the returned Node. + return (next_[n].load(std::memory_order_acquire)); + } + void SetNext(int n, Node* x) { + DCHECK(n >= 0); + // Use a 'release store' so that anybody who reads through this + // pointer observes a fully initialized version of the inserted node. + next_[n].store(x, std::memory_order_release); + } + + // No-barrier variants that can be safely used in a few locations. + Node* NoBarrier_Next(int n) { + DCHECK(n >= 0); + return next_[n].load(std::memory_order_relaxed); + } + void NoBarrier_SetNext(int n, Node* x) { + DCHECK(n >= 0); + next_[n].store(x, std::memory_order_relaxed); + } + + private: + // Array of length equal to the node height. next_[0] is lowest level link. + std::atomic next_[1]; +}; + +template +typename SkipList::Node* +SkipList::NewNode(const Key& key, int height) { + char* mem = arena_->AllocateAligned( + sizeof(Node) + sizeof(std::atomic) * (height - 1)); + return new (mem) Node(key); +} + +template +inline SkipList::Iterator::Iterator(const SkipList* list) { + list_ = list; + node_ = NULL; +} + +template +inline bool SkipList::Iterator::Valid() const { + return node_ != NULL; +} + +template +inline const Key& SkipList::Iterator::key() const { + DCHECK(Valid()); + return node_->key; +} + +template +inline void SkipList::Iterator::Next() { + DCHECK(Valid()); + node_ = node_->Next(0); +} + +template +inline void SkipList::Iterator::Prev() { + // Instead of using explicit "prev" links, we just search for the + // last node that falls before key. + DCHECK(Valid()); + node_ = list_->FindLessThan(node_->key); + if (node_ == list_->head_) { + node_ = NULL; + } +} + +template +inline void SkipList::Iterator::Seek(const Key& target) { + node_ = list_->FindGreaterOrEqual(target, NULL); +} + +template +inline void SkipList::Iterator::SeekToFirst() { + node_ = list_->head_->Next(0); +} + +template +inline void SkipList::Iterator::SeekToLast() { + node_ = list_->FindLast(); + if (node_ == list_->head_) { + node_ = NULL; + } +} + +template +int SkipList::RandomHeight() { + // Increase height with probability 1 in kBranching + static const unsigned int kBranching = 4; + int height = 1; + while (height < kMaxHeight && ((rnd_.Next() % kBranching) == 0)) { + height++; + } + DCHECK(height > 0); + DCHECK(height <= kMaxHeight); + return height; +} + +template +bool SkipList::KeyIsAfterNode(const Key& key, Node* n) const { + // NULL n is considered infinite + return (n != NULL) && (compare_(n->key, key) < 0); +} + +template +typename SkipList::Node* +SkipList::FindGreaterOrEqual(const Key& key, Node** prev) const { + Node* x = head_; + int level = GetMaxHeight() - 1; + while (true) { + Node* next = x->Next(level); + if (KeyIsAfterNode(key, next)) { + // Keep searching in this list + x = next; + } else { + if (prev != NULL) prev[level] = x; + if (level == 0) { + return next; + } else { + // Switch to next list + level--; + } + } + } +} + +template +typename SkipList::Node* +SkipList::FindLessThan(const Key& key) const { + Node* x = head_; + int level = GetMaxHeight() - 1; + while (true) { + DCHECK(x == head_ || compare_(x->key, key) < 0); + Node* next = x->Next(level); + if (next == NULL || compare_(next->key, key) >= 0) { + if (level == 0) { + return x; + } else { + // Switch to next list + level--; + } + } else { + x = next; + } + } +} + +template +typename SkipList::Node* +SkipList::FindLast() const { + Node* x = head_; + int level = GetMaxHeight() - 1; + while (true) { + Node* next = x->Next(level); + if (next == NULL) { + if (level == 0) { + return x; + } else { + // Switch to next list + level--; + } + } else { + x = next; + } + } +} + +template +SkipList::SkipList(Comparator cmp, Arena* arena) + : compare_(cmp), + arena_(arena), + head_(NewNode(0 /* any key will do */, kMaxHeight)), + max_height_(1), + rnd_(0xdeadbeef) { + for (int i = 0; i < kMaxHeight; i++) { + head_->SetNext(i, NULL); + } + } + +template +void SkipList::Aggregate(const Key& k1, const Key& k2) { + compare_._schema->aggregate(k1, k2, arena_); +} + +template +void SkipList::Insert(const Key& key, bool* overwritten, KeysType keys_type) { + // TODO(opt): We can use a barrier-free variant of FindGreaterOrEqual() + // here since Insert() is externally synchronized. + Node* prev[kMaxHeight]; + Node* x = FindGreaterOrEqual(key, prev); + +#ifndef BE_TEST + if (x != nullptr && keys_type != KeysType::DUP_KEYS && Equal(key, x->key)) { + Aggregate(x->key, key); + *overwritten = true; + return; + } +#endif + + *overwritten = false; + // Our data structure does not allow duplicate insertion + int height = RandomHeight(); + if (height > GetMaxHeight()) { + for (int i = GetMaxHeight(); i < height; i++) { + prev[i] = head_; + } + //fprintf(stderr, "Change height from %d to %d\n", max_height_, height); + + // It is ok to mutate max_height_ without any synchronization + // with concurrent readers. A concurrent reader that observes + // the new value of max_height_ will see either the old value of + // new level pointers from head_ (NULL), or a new value set in + // the loop below. In the former case the reader will + // immediately drop to the next level since NULL sorts after all + // keys. In the latter case the reader will use the new node. + max_height_.store(height, std::memory_order_relaxed); + } + + x = NewNode(key, height); + for (int i = 0; i < height; i++) { + // NoBarrier_SetNext() suffices since we will add a barrier when + // we publish a pointer to "x" in prev[i]. + x->NoBarrier_SetNext(i, prev[i]->NoBarrier_Next(i)); + prev[i]->SetNext(i, x); + } +} + +template +bool SkipList::Contains(const Key& key) const { + Node* x = FindGreaterOrEqual(key, NULL); + if (x != NULL && Equal(key, x->key)) { + return true; + } else { + return false; + } +} + +} // namespace palo + +#endif // BDG_PALO_BE_SRC_OLAP_SKIPLIST_H diff --git a/be/src/olap/store.cpp b/be/src/olap/store.cpp new file mode 100644 index 0000000000..7684f256a8 --- /dev/null +++ b/be/src/olap/store.cpp @@ -0,0 +1,557 @@ +// Copyright (c) 2017, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 "olap/store.h" + +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include + +#include +#include +#include +#include +#include +#include + +#include "olap/file_helper.h" +#include "olap/olap_define.h" +#include "olap/utils.h" // for check_dir_existed +#include "util/file_utils.h" +#include "olap/olap_header_manager.h" + +namespace palo { + +static const char* const kMtabPath = "/etc/mtab"; +static const char* const kTestFilePath = "/.testfile"; + +OlapStore::OlapStore(const std::string& path, int64_t capacity_bytes) + : _path(path), + _cluster_id(-1), + _capacity_bytes(capacity_bytes), + _available_bytes(0), + _used_bytes(0), + _current_shard(0), + _is_used(false), + _to_be_deleted(false), + _test_file_read_buf(nullptr), + _test_file_write_buf(nullptr), + _meta((nullptr)) { +} + +OlapStore::~OlapStore() { + free(_test_file_read_buf); + free(_test_file_write_buf); + if (_meta != nullptr) { + delete _meta; + } +} + +Status OlapStore::load() { + _rand_seed = static_cast(time(NULL)); + if (posix_memalign((void**)&_test_file_write_buf, + DIRECT_IO_ALIGNMENT, + TEST_FILE_BUF_SIZE) != 0) { + LOG(WARNING) << "fail to allocate memory. size=" << TEST_FILE_BUF_SIZE; + return Status("No memory"); + } + if (posix_memalign((void**)&_test_file_read_buf, + DIRECT_IO_ALIGNMENT, + TEST_FILE_BUF_SIZE) != 0) { + LOG(WARNING) << "fail to allocate memory. size=" << TEST_FILE_BUF_SIZE; + return Status("No memory"); + } + RETURN_IF_ERROR(_check_path_exist()); + std::string align_tag_path = _path + ALIGN_TAG_PREFIX; + if (access(align_tag_path.c_str(), F_OK) == 0) { + LOG(WARNING) << "align tag was found, path=" << _path; + return Status("invalid root path: "); + } + + RETURN_IF_ERROR(_init_cluster_id()); + RETURN_IF_ERROR(_init_extension_and_capacity()); + RETURN_IF_ERROR(_init_file_system()); + + RETURN_IF_ERROR(_init_meta()); + + _is_used = true; + return Status::OK; +} + +Status OlapStore::_check_path_exist() { + DIR* dirp = opendir(_path.c_str()); + if (dirp == nullptr) { + char buf[64]; + LOG(WARNING) << "opendir failed, path=" << _path + << ", errno=" << errno << ", errmsg=" << strerror_r(errno, buf, 64); + return Status("opendir failed"); + } + struct dirent dirent; + struct dirent* result = nullptr; + if (readdir_r(dirp, &dirent, &result) != 0) { + char buf[64]; + LOG(WARNING) << "readdir failed, path=" << _path + << ", errno=" << errno << ", errmsg=" << strerror_r(errno, buf, 64); + closedir(dirp); + return Status("readdir failed"); + } + return Status::OK; +} + +Status OlapStore::_init_cluster_id() { + std::string cluster_id_path = _path + CLUSTER_ID_PREFIX; + if (access(cluster_id_path.c_str(), F_OK) != 0) { + int fd = open(cluster_id_path.c_str(), O_RDWR | O_CREAT, S_IRUSR | S_IWUSR | S_IRGRP | S_IWGRP); + if (fd < 0 || close(fd) < 0) { + char errmsg[64]; + LOG(WARNING) << "fail to create file. [path='" << cluster_id_path + << "' err='" << strerror_r(errno, errmsg, 64) << "']"; + return Status("invalid store path: create cluster id failed"); + } + } + + // obtain lock of all cluster id paths + FILE* fp = NULL; + fp = fopen(cluster_id_path.c_str(), "r+b"); + if (fp == NULL) { + LOG(WARNING) << "fail to open cluster id path. path=" << cluster_id_path; + return Status("invalid store path: open cluster id failed"); + } + + int lock_res = flock(fp->_fileno, LOCK_EX | LOCK_NB); + if (lock_res < 0) { + LOG(WARNING) << "fail to lock file descriptor. path=" << cluster_id_path; + fclose(fp); + fp = NULL; + return Status("invalid store path: flock cluster id failed"); + } + + // obtain cluster id of all root paths + auto st = _read_cluster_id(cluster_id_path, &_cluster_id); + fclose(fp); + return st; +} + +Status OlapStore::_read_cluster_id(const std::string& path, int32_t* cluster_id) { + int32_t tmp_cluster_id = -1; + + std::fstream fs(path.c_str(), std::fstream::in); + if (!fs.is_open()) { + LOG(WARNING) << "fail to open cluster id path. [path='" << path << "']"; + return Status("open file failed"); + } + + fs >> tmp_cluster_id; + fs.close(); + + if (tmp_cluster_id == -1 && (fs.rdstate() & std::fstream::eofbit) != 0) { + *cluster_id = -1; + } else if (tmp_cluster_id >= 0 && (fs.rdstate() & std::fstream::eofbit) != 0) { + *cluster_id = tmp_cluster_id; + } else { + OLAP_LOG_WARNING("fail to read cluster id from file. " + "[id=%d eofbit=%d failbit=%d badbit=%d]", + tmp_cluster_id, + fs.rdstate() & std::fstream::eofbit, + fs.rdstate() & std::fstream::failbit, + fs.rdstate() & std::fstream::badbit); + return Status("cluster id file corrupt"); + } + return Status::OK; +} + +Status OlapStore::_init_extension_and_capacity() { + boost::filesystem::path boost_path = _path; + std::string extension = boost::filesystem::canonical(boost_path).extension().string(); + if (extension != "") { + if (boost::iequals(extension, ".ssd")) { + _storage_medium = TStorageMedium::SSD; + } else if (boost::iequals(extension, ".hdd")) { + _storage_medium = TStorageMedium::HDD; + } else { + LOG(WARNING) << "store path has wrong extension. path=" << _path; + return Status("invalid sotre path: invalid extension"); + } + } else { + _storage_medium = TStorageMedium::HDD; + } + + int64_t disk_capacity = boost::filesystem::space(boost_path).capacity; + if (_capacity_bytes == -1) { + _capacity_bytes = disk_capacity; + } else if (_capacity_bytes > disk_capacity) { + LOG(WARNING) << "root path capacity should not larger than disk capacity. " + << "path=" << _path + << ", capacity_bytes=" << _capacity_bytes + << ", disk_capacity=" << disk_capacity; + return Status("invalid store path: invalid capacity"); + } + + std::string data_path = _path + DATA_PREFIX; + if (!check_dir_existed(data_path) && create_dir(data_path) != OLAP_SUCCESS) { + LOG(WARNING) << "failed to create data root path. path=" << data_path; + return Status("invalid store path: failed to create data directory"); + } + + return Status::OK; +} + +Status OlapStore::_init_file_system() { + struct stat s; + if (stat(_path.c_str(), &s) != 0) { + char errmsg[64]; + LOG(WARNING) << "stat failed, path=" << _path + << ", errno=" << errno << ", errmsg=" << strerror_r(errno, errmsg, 64); + return Status("invalid store path: stat failed"); + } + + dev_t mount_device; + if ((s.st_mode & S_IFMT) == S_IFBLK) { + mount_device = s.st_rdev; + } else { + mount_device = s.st_dev; + } + + FILE* mount_table = nullptr; + if ((mount_table = setmntent(kMtabPath, "r")) == NULL) { + char errmsg[64]; + LOG(WARNING) << "setmntent failed, path=" << kMtabPath + << ", errno=" << errno << ", errmsg=" << strerror_r(errno, errmsg, 64); + return Status("invalid store path: setmntent failed"); + } + + bool is_find = false; + struct mntent* mount_entry = NULL; + while ((mount_entry = getmntent(mount_table)) != NULL) { + if (strcmp(_path.c_str(), mount_entry->mnt_dir) == 0 + || strcmp(_path.c_str(), mount_entry->mnt_fsname) == 0) { + is_find = true; + break; + } + + if (stat(mount_entry->mnt_fsname, &s) == 0 && s.st_rdev == mount_device) { + is_find = true; + break; + } + + if (stat(mount_entry->mnt_dir, &s) == 0 && s.st_dev == mount_device) { + is_find = true; + break; + } + } + + endmntent(mount_table); + + if (!is_find) { + LOG(WARNING) << "fail to find file system, path=" << _path; + return Status("invalid store path: find file system failed"); + } + + _file_system = mount_entry->mnt_fsname; + + return Status::OK; +} + +Status OlapStore::_init_meta() { + // init meta + _meta = new(std::nothrow) OlapMeta(_path); + if (_meta == nullptr) { + LOG(WARNING) << "new olap meta failed"; + return Status("new olap meta failed"); + } + OLAPStatus res = _meta->init(); + if (res != OLAP_SUCCESS) { + LOG(WARNING) << "init meta failed"; + return Status("init meta failed"); + } + return Status::OK; +} + +Status OlapStore::set_cluster_id(int32_t cluster_id) { + if (_cluster_id != -1) { + if (_cluster_id == cluster_id) { + return Status::OK; + } + LOG(ERROR) << "going to set cluster id to already assigned store, cluster_id=" + << _cluster_id << ", new_cluster_id=" << cluster_id; + return Status("going to set cluster id to already assigned store"); + } + return _write_cluster_id_to_path(_cluster_id_path(), cluster_id); +} + +Status OlapStore::_write_cluster_id_to_path(const std::string& path, int32_t cluster_id) { + std::fstream fs(path.c_str(), std::fstream::out); + if (!fs.is_open()) { + LOG(WARNING) << "fail to open cluster id path. path=" << path; + return Status("IO Error"); + } + fs << cluster_id; + fs.close(); + return Status::OK; +} + +void OlapStore::health_check() { + // check disk + if (_is_used) { + OLAPStatus res = OLAP_SUCCESS; + if ((res = _read_and_write_test_file()) != OLAP_SUCCESS) { + LOG(WARNING) << "store read/write test file occur IO Error. path=" << _path; + if (is_io_error(res)) { + _is_used = false; + } + } + } +} + +OLAPStatus OlapStore::_read_and_write_test_file() { + std::string test_file = _path + kTestFilePath; + + if (access(test_file.c_str(), F_OK) == 0) { + if (remove(test_file.c_str()) != 0) { + char errmsg[64]; + LOG(WARNING) << "fail to delete test file. " + << "path=" << test_file + << ", errno=" << errno << ", err=" << strerror_r(errno, errmsg, 64); + return OLAP_ERR_IO_ERROR; + } + } else { + if (errno != ENOENT) { + char errmsg[64]; + LOG(WARNING) << "fail to access test file. " + << "path=" << test_file + << ", errno=" << errno << ", err=" << strerror_r(errno, errmsg, 64); + return OLAP_ERR_IO_ERROR; + } + } + + OLAPStatus res = OLAP_SUCCESS; + FileHandler file_handler; + if ((res = file_handler.open_with_mode(test_file.c_str(), + O_RDWR | O_CREAT | O_DIRECT, + S_IRUSR | S_IWUSR)) != OLAP_SUCCESS) { + LOG(WARNING) << "fail to create test file. path=" << test_file; + return res; + } + + for (size_t i = 0; i < TEST_FILE_BUF_SIZE; ++i) { + int32_t tmp_value = rand_r(&_rand_seed); + _test_file_write_buf[i] = static_cast(tmp_value); + } + + if ((res = file_handler.pwrite(_test_file_write_buf, TEST_FILE_BUF_SIZE, SEEK_SET)) != OLAP_SUCCESS) { + LOG(WARNING) << "fail to write test file. [file_name=" << test_file << "]"; + return res; + } + + if ((res = file_handler.pread(_test_file_read_buf, TEST_FILE_BUF_SIZE, SEEK_SET)) != OLAP_SUCCESS) { + LOG(WARNING) << "fail to read test file. [file_name=" << test_file << "]"; + return res; + } + + if (memcmp(_test_file_write_buf, _test_file_read_buf, TEST_FILE_BUF_SIZE) != 0) { + OLAP_LOG_WARNING("the test file write_buf and read_buf not equal."); + return OLAP_ERR_TEST_FILE_ERROR; + } + + if ((res = file_handler.close()) != OLAP_SUCCESS) { + LOG(WARNING) << "fail to close test file. [file_name=" << test_file << "]"; + return res; + } + + if (remove(test_file.c_str()) != 0) { + char errmsg[64]; + VLOG(3) << "fail to delete test file. [err='" << strerror_r(errno, errmsg, 64) + << "' path='" << test_file << "']"; + return OLAP_ERR_IO_ERROR; + } + + return res; +} + +OLAPStatus OlapStore::get_shard(uint64_t* shard) { + OLAPStatus res = OLAP_SUCCESS; + std::lock_guard l(_mutex); + + std::stringstream shard_path_stream; + uint32_t next_shard = _current_shard; + _current_shard = (_current_shard + 1) % MAX_SHARD_NUM; + shard_path_stream << _path << DATA_PREFIX << "/" << next_shard; + std::string shard_path = shard_path_stream.str(); + if (!check_dir_existed(shard_path)) { + res = create_dir(shard_path); + if (res != OLAP_SUCCESS) { + LOG(WARNING) << "fail to create path. [path='" << shard_path << "']"; + return res; + } + } + + *shard = next_shard; + return OLAP_SUCCESS; +} + +OlapMeta* OlapStore::get_meta() { + return _meta; +} + +OLAPStatus OlapStore::register_table(OLAPTable* table) { + std::lock_guard l(_mutex); + + TabletInfo tablet_info(table->tablet_id(), table->schema_hash()); + _tablet_set.insert(tablet_info); + return OLAP_SUCCESS; +} + +OLAPStatus OlapStore::deregister_table(OLAPTable* table) { + std::lock_guard l(_mutex); + + TabletInfo tablet_info(table->tablet_id(), table->schema_hash()); + _tablet_set.erase(tablet_info); + return OLAP_SUCCESS; +} + +std::string OlapStore::get_shard_path_from_header(const std::string& shard_string) { + return _path + DATA_PREFIX + "/" + shard_string; +} + +std::string OlapStore::get_tablet_schema_hash_path_from_header(OLAPHeader* header) { + return _path + DATA_PREFIX + "/" + std::to_string(header->shard()) + + "/" + std::to_string(header->tablet_id()) + "/" + std::to_string(header->schema_hash()); +} + +std::string OlapStore::get_tablet_path_from_header(OLAPHeader* header) { + return _path + DATA_PREFIX + "/" + std::to_string(header->shard()) + + "/" + std::to_string(header->tablet_id()); +} + +void OlapStore::find_tablet_in_trash(int64_t tablet_id, std::vector* paths) { + // path: /root_path/trash/time_label/tablet_id/schema_hash + std::string trash_path = _path + TRASH_PREFIX; + std::vector sub_dirs; + FileUtils::scan_dir(trash_path, &sub_dirs); + for (auto& sub_dir : sub_dirs) { + // sub dir is time_label + std::string sub_path = trash_path + "/" + sub_dir; + if (!FileUtils::is_dir(sub_path)) { + continue; + } + std::string tablet_path = sub_path + "/" + std::to_string(tablet_id); + bool exist = FileUtils::check_exist(tablet_path); + if (exist) { + paths->emplace_back(std::move(tablet_path)); + } + } +} + +std::string OlapStore::get_root_path_from_schema_hash_path_in_trash( + const std::string& schema_hash_dir_in_trash) { + boost::filesystem::path schema_hash_path_in_trash(schema_hash_dir_in_trash); + return schema_hash_path_in_trash.parent_path().parent_path().parent_path().parent_path().string(); +} + +OLAPStatus OlapStore::_load_table_from_header(OLAPEngine* engine, TTabletId tablet_id, + TSchemaHash schema_hash, const std::string& header) { + std::unique_ptr olap_header(new OLAPHeader()); + bool parsed = olap_header->ParseFromString(header); + if (!parsed) { + LOG(WARNING) << "parse header string failed for tablet_id:" << tablet_id << " schema_hash:" << schema_hash; + return OLAP_ERR_HEADER_PB_PARSE_FAILED; + } + OLAPStatus res = OLAP_SUCCESS; + if (olap_header->file_version_size() != 0) { + olap_header->change_file_version_to_delta(); + res = OlapHeaderManager::save(this, tablet_id, schema_hash, olap_header.get()); + } + if (res != OLAP_SUCCESS) { + LOG(FATAL) << "fail to save header, tablet_id:" << tablet_id + << ", schema_hash:" << schema_hash << " to path:" << path(); + return OLAP_ERR_HEADER_PUT; + } + // init must be called + res = olap_header->init(); + if (res != OLAP_SUCCESS) { + LOG(WARNING) << "fail to init header, tablet_id:" << tablet_id << ", schema_hash:" << schema_hash; + res = OlapHeaderManager::remove(this, tablet_id, schema_hash); + if (res != OLAP_SUCCESS) { + LOG(WARNING) << "remove header failed. tablet_id:" << tablet_id + << "schema_hash:" << schema_hash + << "store path:" << path(); + } + return OLAP_ERR_HEADER_INIT_FAILED; + } + OLAPTablePtr olap_table = + OLAPTable::create_from_header(olap_header.release(), this); + if (olap_table == nullptr) { + LOG(WARNING) << "fail to new table. tablet_id=" << tablet_id << ", schema_hash:" << schema_hash; + return OLAP_ERR_TABLE_CREATE_FROM_HEADER_ERROR; + } + + if (olap_table->lastest_version() == nullptr && !olap_table->is_schema_changing()) { + LOG(WARNING) << "tablet not in schema change state without delta is invalid. tablet:" + << olap_table->full_name(); + // tablet state is invalid, drop tablet + olap_table->mark_dropped(); + return OLAP_ERR_TABLE_INDEX_VALIDATE_ERROR; + } + + res = engine->add_table(tablet_id, schema_hash, olap_table); + if (res != OLAP_SUCCESS) { + // insert existed tablet return OLAP_SUCCESS + if (res == OLAP_ERR_ENGINE_INSERT_EXISTS_TABLE) { + LOG(WARNING) << "add duplicate table. table=" << olap_table->full_name(); + } + + LOG(WARNING) << "failed to add table. table=" << olap_table->full_name(); + return res; + } + res = engine->register_table_into_root_path(olap_table.get()); + if (res != OLAP_SUCCESS) { + LOG(WARNING) << "fail to register table into root path. root_path=" << olap_table->storage_root_path_name(); + + if (engine->drop_table(tablet_id, schema_hash) != OLAP_SUCCESS) { + LOG(WARNING) << "fail to drop table when create table failed. " + <<"tablet=" << tablet_id << " schema_hash=" << schema_hash; + } + + return res; + } + // load pending data (for realtime push), will add transaction relationship into engine + olap_table->load_pending_data(); + + return OLAP_SUCCESS; +} + +OLAPStatus OlapStore::load_tables(OLAPEngine* engine) { + auto load_table_func = [this, engine](long tablet_id, + long schema_hash, const std::string& value) -> bool { + OLAPStatus status = _load_table_from_header(engine, tablet_id, schema_hash, value); + if (status != OLAP_SUCCESS) { + LOG(WARNING) << "load table from header failed.tablet_id:" << tablet_id + << ", schema_hash:" << schema_hash << ", status:" << status; + }; + return true; + }; + OLAPStatus status = OlapHeaderManager::traverse_headers(_meta, load_table_func); + return status; +} + +} diff --git a/be/src/olap/store.h b/be/src/olap/store.h new file mode 100644 index 0000000000..5471b65734 --- /dev/null +++ b/be/src/olap/store.h @@ -0,0 +1,126 @@ +// Copyright (c) 2017, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 "common/status.h" +#include "gen_cpp/Types_types.h" +#include "olap/olap_common.h" +#include "olap/olap_engine.h" + +namespace palo { + +class OLAPRootPath; +class OLAPEngine; + +// A OlapStore used to manange data in same path. +// Now, After OlapStore was created, it will never be deleted for easy implementation. +class OlapStore { +public: + OlapStore(const std::string& path, int64_t capacity_bytes = -1); + ~OlapStore(); + + Status load(); + + const std::string& path() const { return _path; } + bool is_used() const { return _is_used; } + void set_is_used(bool is_used) { _is_used = is_used; } + int32_t cluster_id() const { return _cluster_id; } + RootPathInfo to_root_path_info() { + RootPathInfo info; + info.path = _path; + info.is_used = _is_used; + info.capacity = _capacity_bytes; + return info; + } + + Status set_cluster_id(int32_t cluster_id); + void health_check(); + + OLAPStatus get_shard(uint64_t* shard); + + OlapMeta* get_meta(); + + bool is_ssd_disk() const { + return _storage_medium == TStorageMedium::SSD; + } + TStorageMedium::type storage_medium() const { return _storage_medium; } + + OLAPStatus register_table(OLAPTable* table); + OLAPStatus deregister_table(OLAPTable* table); + + std::string get_tablet_schema_hash_path_from_header(OLAPHeader* header); + + std::string get_tablet_path_from_header(OLAPHeader* header); + + std::string get_shard_path_from_header(const std::string& shard_string); + + void find_tablet_in_trash(int64_t tablet_id, std::vector* paths); + + static std::string get_root_path_from_schema_hash_path_in_trash(const std::string& schema_hash_dir_in_trash); + + OLAPStatus load_tables(OLAPEngine* engine); + +private: + std::string _cluster_id_path() const { return _path + CLUSTER_ID_PREFIX; } + Status _init_cluster_id(); + Status _check_path_exist(); + Status _init_extension_and_capacity(); + Status _init_file_system(); + Status _init_meta(); + + Status _check_disk(); + OLAPStatus _read_and_write_test_file(); + Status _read_cluster_id(const std::string& path, int32_t* cluster_id); + Status _write_cluster_id_to_path(const std::string& path, int32_t cluster_id); + + OLAPStatus _load_table_from_header(OLAPEngine* engine, TTabletId tablet_id, + TSchemaHash schema_hash, const std::string& header); + +private: + friend class OLAPRootPath; + friend class OLAPEngine; + + std::string _path; + int32_t _cluster_id; + uint32_t _rand_seed; + + std::string _file_system; + int64_t _capacity_bytes; + int64_t _available_bytes; + int64_t _used_bytes; + uint64_t _current_shard; + bool _is_used; + // This flag will be set true if this store was not in root path when reloading + bool _to_be_deleted; + + std::mutex _mutex; + TStorageMedium::type _storage_medium; // 存储介质类型:SSD|HDD + std::set _tablet_set; + + static const size_t TEST_FILE_BUF_SIZE = 4096; + static const size_t DIRECT_IO_ALIGNMENT = 512; + static const uint32_t MAX_SHARD_NUM = 1024; + char* _test_file_read_buf; + char* _test_file_write_buf; + OlapMeta* _meta; +}; + +} diff --git a/be/src/runtime/kafka_consumer_pipe.cpp b/be/src/runtime/kafka_consumer_pipe.cpp new file mode 100644 index 0000000000..a2f3a322c1 --- /dev/null +++ b/be/src/runtime/kafka_consumer_pipe.cpp @@ -0,0 +1,21 @@ +// Copyright (c) 2018, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 "runtime/kafka_consumer_pipe.h" + +namespace palo { + + +} // end namespace palo diff --git a/be/src/runtime/kafka_consumer_pipe.h b/be/src/runtime/kafka_consumer_pipe.h new file mode 100644 index 0000000000..7a7b448e30 --- /dev/null +++ b/be/src/runtime/kafka_consumer_pipe.h @@ -0,0 +1,48 @@ +// Copyright (c) 2018, Baidu.com, Inc. All Rights Reserved + +// Licensed 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. + +#ifndef BDG_PALO_BE_SRC_RUNTIME_KAFKA_COMSUMER_PIPE_H +#define BDG_PALO_BE_SRC_RUNTIME_KAFKA_COMSUMER_PIPE_H + +#include + +#include +#include +#include + +#include "librdkafka/rdkafka.h" + +#include "exec/file_reader.h" +#include "http/message_body_sink.h" + +namespace palo { + +class KafkaConsumerPipe : public MessageBodySink, public FileReader { +public: + KafkaConsumerPipe(); + ~KafkaConsumerPipe(); + + +private: + // this is only for testing librdkafka.a + void test_kafka_lib() { + //rd_kafka_conf_t *conf = rd_kafka_conf_new(); + //rd_kafka_topic_conf_t *topic_conf = rd_kafka_topic_conf_new(); + } +}; + +} // end namespace palo + +#endif // BDG_PALO_BE_SRC_RUNTIME_KAFKA_COMSUMER_PIPE_H diff --git a/be/src/runtime/load_stream_mgr.h b/be/src/runtime/load_stream_mgr.h new file mode 100644 index 0000000000..63a3e57159 --- /dev/null +++ b/be/src/runtime/load_stream_mgr.h @@ -0,0 +1,60 @@ +// Copyright (c) 2017, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 "runtime/stream_load_pipe.h" // for StreamLoadPipe +#include "util/uid_util.h" // for std::hash for UniqueId + +namespace palo { + +// used to register all streams in process so that other module can get this stream +class LoadStreamMgr { +public: + LoadStreamMgr() { } + ~LoadStreamMgr() { } + + Status put(const UniqueId& id, + std::shared_ptr stream) { + std::lock_guard l(_lock); + auto it = _stream_map.find(id); + if (it != std::end(_stream_map)) { + return Status("id already exist"); + } + _stream_map.emplace(id, stream); + return Status::OK; + } + + std::shared_ptr get(const UniqueId& id) { + std::lock_guard l(_lock); + auto it = _stream_map.find(id); + if (it == std::end(_stream_map)) { + return nullptr; + } + auto stream = it->second; + _stream_map.erase(it); + return stream; + } + +private: + std::mutex _lock; + std::unordered_map> _stream_map; +}; + +} diff --git a/be/src/runtime/stream_load_pipe.h b/be/src/runtime/stream_load_pipe.h new file mode 100644 index 0000000000..da2fd540c7 --- /dev/null +++ b/be/src/runtime/stream_load_pipe.h @@ -0,0 +1,173 @@ +// Copyright (c) 2017, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 "exec/file_reader.h" +#include "http/message_body_sink.h" +#include "util/bit_util.h" +#include "util/byte_buffer.h" + +namespace palo { + +// StreamLoadPipe use to transfer data from producer to consumer +// Data in pip is stored in chunks. +class StreamLoadPipe : public MessageBodySink, public FileReader { +public: + StreamLoadPipe(size_t max_buffered_bytes = 1024 * 1024, + size_t min_chunk_size = 64 * 1024) + : _buffered_bytes(0), + _max_buffered_bytes(max_buffered_bytes), + _min_chunk_size(min_chunk_size), + _finished(false), _cancelled(false) { + } + virtual ~StreamLoadPipe() { } + + Status append(const char* data, size_t size) override { + size_t pos = 0; + if (_write_buf != nullptr) { + if (size < _write_buf->remaining()) { + _write_buf->put_bytes(data, size); + return Status::OK; + } else { + pos = _write_buf->remaining(); + _write_buf->put_bytes(data, pos); + + _write_buf->flip(); + RETURN_IF_ERROR(_append(_write_buf)); + _write_buf.reset(); + } + } + // need to allocate a new chunk + size_t chunk_size = std::max(_min_chunk_size, size - pos); + chunk_size = BitUtil::RoundUpToPowerOfTwo(chunk_size); + _write_buf = ByteBuffer::allocate(chunk_size); + _write_buf->put_bytes(data + pos, size - pos); + return Status::OK; + } + + Status append(const ByteBufferPtr& buf) override { + if (_write_buf != nullptr) { + _write_buf->flip(); + RETURN_IF_ERROR(_append(_write_buf)); + _write_buf.reset(); + } + return _append(buf); + } + + Status read(uint8_t* data, size_t* data_size, bool* eof) override { + size_t bytes_read = 0; + while (bytes_read < *data_size) { + std::unique_lock l(_lock); + while (!_cancelled && !_finished && _buf_queue.empty()) { + _get_cond.wait(l); + } + // cancelled + if (_cancelled) { + return Status("cancelled"); + } + // finished + if (_buf_queue.empty()) { + DCHECK(_finished); + *data_size = bytes_read; + *eof = (bytes_read == 0); + return Status::OK; + } + auto buf = _buf_queue.front(); + size_t copy_size = std::min(*data_size - bytes_read, buf->remaining()); + buf->get_bytes((char*)data + bytes_read, copy_size); + bytes_read += copy_size; + if (!buf->has_remaining()) { + _buf_queue.pop_front(); + _buffered_bytes -= buf->limit; + _put_cond.notify_one(); + } + } + DCHECK(bytes_read == *data_size) + << "bytes_read=" << bytes_read << ", *data_size=" << *data_size; + *eof = false; + return Status::OK; + } + + // called when comsumer finished + void close() override { + cancel(); + } + + // called when producer finished + Status finish() override { + if (_write_buf != nullptr) { + _write_buf->flip(); + _append(_write_buf); + _write_buf.reset(); + } + { + std::lock_guard l(_lock); + _finished = true; + } + _get_cond.notify_all(); + return Status::OK; + } + + // called when producer/comsumer failed + void cancel() override { + { + std::lock_guard l(_lock); + _cancelled = true; + } + _get_cond.notify_all(); + _put_cond.notify_all(); + } + +private: + Status _append(const ByteBufferPtr& buf) { + { + std::unique_lock l(_lock); + // if _buf_queue is empty, we append this buf without size check + while (!_cancelled && + !_buf_queue.empty() && + _buffered_bytes + buf->remaining() > _max_buffered_bytes) { + _put_cond.wait(l); + } + if (_cancelled) { + return Status("cancelled"); + } + _buf_queue.push_back(buf); + _buffered_bytes += buf->remaining(); + } + _get_cond.notify_one(); + return Status::OK; + } + + // Blocking queue + std::mutex _lock; + size_t _buffered_bytes; + size_t _max_buffered_bytes; + size_t _min_chunk_size; + std::deque _buf_queue; + std::condition_variable _put_cond; + std::condition_variable _get_cond; + + bool _finished; + bool _cancelled; + + ByteBufferPtr _write_buf; +}; + +} diff --git a/be/src/runtime/tablet_writer_mgr.cpp b/be/src/runtime/tablet_writer_mgr.cpp new file mode 100644 index 0000000000..1f01d01daf --- /dev/null +++ b/be/src/runtime/tablet_writer_mgr.cpp @@ -0,0 +1,327 @@ +// Copyright (c) 2017, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 "runtime/tablet_writer_mgr.h" + +#include +#include +#include + +#include "common/object_pool.h" +#include "exec/olap_table_info.h" +#include "runtime/descriptors.h" +#include "runtime/mem_tracker.h" +#include "runtime/row_batch.h" +#include "runtime/tuple_row.h" +#include "util/bitmap.h" +#include "olap/delta_writer.h" +#include "olap/lru_cache.h" + +namespace palo { + +// channel that process all data for this load +class TabletsChannel { +public: + TabletsChannel(const TabletsChannelKey& key) : _key(key), _closed_senders(64) { } + ~TabletsChannel(); + + Status open(const PTabletWriterOpenRequest& params); + + Status add_batch(const PTabletWriterAddBatchRequest& batch); + + Status close(int sender_id, bool* finished, + const google::protobuf::RepeatedField& partition_ids, + google::protobuf::RepeatedPtrField* tablet_vec); + +private: + // open all writer + Status _open_all_writers(const PTabletWriterOpenRequest& params); + +private: + // id of this load channel, just for + TabletsChannelKey _key; + + // make execute sequece + std::mutex _lock; + + // initialized in open function + int64_t _txn_id = -1; + int64_t _index_id = -1; + OlapTableSchemaParam* _schema = nullptr; + TupleDescriptor* _tuple_desc = nullptr; + // row_desc used to construct + RowDescriptor* _row_desc = nullptr; + bool _opened = false; + + // next sequence we expect + int _num_remaining_senders = 0; + std::vector _next_seqs; + Bitmap _closed_senders; + Status _close_status; + + // tablet_id -> TabletChannel + std::unordered_map _tablet_writers; + + std::unordered_set _partition_ids; + + // TODO(zc): to add this tracker to somewhere + MemTracker _mem_tracker; +}; + +TabletsChannel::~TabletsChannel() { + for (auto& it : _tablet_writers) { + delete it.second; + } + delete _row_desc; + delete _schema; +} + +Status TabletsChannel::open(const PTabletWriterOpenRequest& params) { + std::lock_guard l(_lock); + if (_opened) { + // Normal case, already open by other sender + return Status::OK; + } + _txn_id = params.txn_id(); + _index_id = params.index_id(); + _schema = new OlapTableSchemaParam(); + RETURN_IF_ERROR(_schema->init(params.schema())); + _tuple_desc = _schema->tuple_desc(); + _row_desc = new RowDescriptor(_tuple_desc, false); + + _num_remaining_senders = params.num_senders(); + _next_seqs.resize(_num_remaining_senders, 0); + _closed_senders.Reset(_num_remaining_senders); + + RETURN_IF_ERROR(_open_all_writers(params)); + + _opened = true; + return Status::OK; +} + +Status TabletsChannel::add_batch(const PTabletWriterAddBatchRequest& params) { + DCHECK(params.tablet_ids_size() == params.row_batch().num_rows()); + std::lock_guard l(_lock); + DCHECK(_opened); + auto next_seq = _next_seqs[params.sender_id()]; + // check packet + if (params.packet_seq() < next_seq) { + LOG(INFO) << "packet has already recept before, expect_seq=" << next_seq + << ", recept_seq=" << params.packet_seq(); + return Status::OK; + } else if (params.packet_seq() > next_seq) { + LOG(WARNING) << "lost data packet, expect_seq=" << next_seq + << ", recept_seq=" << params.packet_seq(); + return Status("lost data packet"); + } + + RowBatch row_batch(*_row_desc, params.row_batch(), &_mem_tracker); + + // iterator all data + for (int i = 0; i < params.tablet_ids_size(); ++i) { + auto tablet_id = params.tablet_ids(i); + auto it = _tablet_writers.find(tablet_id); + if (it == std::end(_tablet_writers)) { + std::stringstream ss; + ss << "unknown tablet to append data, tablet=" << tablet_id; + return Status(ss.str()); + } + auto st = it->second->write(row_batch.get_row(i)->get_tuple(0)); + if (st != OLAP_SUCCESS) { + LOG(WARNING) << "tablet writer writer failed, tablet_id=" << it->first + << ", transaction_id=" << _txn_id; + return Status("tablet writer write failed"); + } + } + _next_seqs[params.sender_id()]++; + return Status::OK; +} + +Status TabletsChannel::close(int sender_id, bool* finished, + const google::protobuf::RepeatedField& partition_ids, + google::protobuf::RepeatedPtrField* tablet_vec) { + std::lock_guard l(_lock); + if (_closed_senders.Get(sender_id)) { + // Dobule close from one sender, just return OK + *finished = (_num_remaining_senders == 0); + return _close_status; + } + for (auto pid : partition_ids) { + _partition_ids.emplace(pid); + } + _closed_senders.Set(sender_id, true); + _num_remaining_senders--; + *finished = (_num_remaining_senders == 0); + if (*finished) { + // All senders are closed + for (auto& it : _tablet_writers) { + if (_partition_ids.count(it.second->partition_id()) > 0) { + auto st = it.second->close(tablet_vec); + if (st != OLAP_SUCCESS) { + LOG(WARNING) << "close tablet writer failed, tablet_id=" << it.first + << ", transaction_id=" << _txn_id; + _close_status = Status("close tablet writer failed"); + return _close_status; + } + } else { + auto st = it.second->cancel(); + if (st != OLAP_SUCCESS) { + LOG(WARNING) << "cancel tablet writer failed, tablet_id=" << it.first + << ", transaction_id=" << _txn_id; + } + } + } + } + return Status::OK; +} + +Status TabletsChannel::_open_all_writers(const PTabletWriterOpenRequest& params) { + std::vector* columns = nullptr; + int32_t schema_hash = 0; + for (auto& index : _schema->indexes()) { + if (index->index_id == _index_id) { + columns = &index->slots; + schema_hash = index->schema_hash; + break; + } + } + if (columns == nullptr) { + std::stringstream ss; + ss << "unknown index id, key=" << _key; + return Status(ss.str()); + } + for (auto& tablet : params.tablets()) { + WriteRequest request; + request.tablet_id = tablet.tablet_id(); + request.schema_hash = schema_hash; + request.write_type = LOAD; + request.transaction_id = _txn_id; + request.partition_id = tablet.partition_id(); + request.load_id = params.id(); + request.need_gen_rollup = params.need_gen_rollup(); + request.tuple_desc = _tuple_desc; + + DeltaWriter* writer = nullptr; + auto st = DeltaWriter::open(&request, &writer); + if (st != OLAP_SUCCESS) { + LOG(WARNING) << "open delta writer failed, tablet_id=" << tablet.tablet_id() + << ", transaction_id=" << _txn_id + << ", partition_id=" << tablet.partition_id() + << ", status=" << st; + return Status("open tablet writer failed"); + } + _tablet_writers.emplace(tablet.tablet_id(), writer); + } + DCHECK(_tablet_writers.size() == params.tablets_size()); + return Status::OK; +} + +TabletWriterMgr::TabletWriterMgr(ExecEnv* exec_env) :_exec_env(exec_env) { + _tablets_channels.init(2011); + _lastest_success_channel = new_lru_cache(1024); +} + +TabletWriterMgr::~TabletWriterMgr() { + delete _lastest_success_channel; +} + +Status TabletWriterMgr::open(const PTabletWriterOpenRequest& params) { + TabletsChannelKey key(params.id(), params.index_id()); + std::shared_ptr channel; + { + std::lock_guard l(_lock); + auto val = _tablets_channels.seek(key); + if (val != nullptr) { + channel = *val; + } else { + // create a new + channel.reset(new TabletsChannel(key)); + _tablets_channels.insert(key, channel); + } + } + RETURN_IF_ERROR(channel->open(params)); + return Status::OK; +} + +static void dummy_deleter(const CacheKey& key, void* value) { +} + +Status TabletWriterMgr::add_batch( + const PTabletWriterAddBatchRequest& request, + google::protobuf::RepeatedPtrField* tablet_vec) { + TabletsChannelKey key(request.id(), request.index_id()); + std::shared_ptr channel; + { + std::lock_guard l(_lock); + auto value = _tablets_channels.seek(key); + if (value == nullptr) { + auto handle = _lastest_success_channel->lookup(key.to_string()); + // success only when eos be true + if (handle != nullptr && request.has_eos() && request.eos()) { + _lastest_success_channel->release(handle); + return Status::OK; + } + std::stringstream ss; + ss << "TabletWriter add batch with unknown id, key=" << key; + return Status(ss.str()); + } + channel = *value; + } + if (request.has_row_batch()) { + RETURN_IF_ERROR(channel->add_batch(request)); + } + Status st; + if (request.has_eos() && request.eos()) { + bool finished = false; + st = channel->close(request.sender_id(), &finished, request.partition_ids(), tablet_vec); + if (!st.ok()) { + LOG(WARNING) << "channle close failed, key=" << key + << ", sender_id=" << request.sender_id() + << ", err_msg=" << st.get_error_msg(); + } + if (finished) { + std::lock_guard l(_lock); + _tablets_channels.erase(key); + if (st.ok()) { + auto handle = _lastest_success_channel->insert( + key.to_string(), nullptr, 1, dummy_deleter); + _lastest_success_channel->release(handle); + } + } + } + return st; +} + +Status TabletWriterMgr::cancel(const PTabletWriterCancelRequest& params) { + TabletsChannelKey key(params.id(), params.index_id()); + { + std::lock_guard l(_lock); + _tablets_channels.erase(key); + } + return Status::OK; +} + +std::string TabletsChannelKey::to_string() const { + std::stringstream ss; + ss << *this; + return ss.str(); +} + +std::ostream& operator<<(std::ostream& os, const TabletsChannelKey& key) { + os << "(id=" << key.id << ",index_id=" << key.index_id << ")"; + return os; +} + +} diff --git a/be/src/runtime/tablet_writer_mgr.h b/be/src/runtime/tablet_writer_mgr.h new file mode 100644 index 0000000000..d6ccb81a14 --- /dev/null +++ b/be/src/runtime/tablet_writer_mgr.h @@ -0,0 +1,97 @@ +// Copyright (c) 2017, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 "common/status.h" +#include "gen_cpp/Types_types.h" +#include "gen_cpp/PaloInternalService_types.h" +#include "gen_cpp/internal_service.pb.h" +#include "util/hash_util.hpp" +#include "util/uid_util.h" + +#include "service/brpc.h" + +namespace palo { + +class ExecEnv; +class TabletsChannel; + +struct TabletsChannelKey { + UniqueId id; + int64_t index_id; + + TabletsChannelKey(const PUniqueId& pid, int64_t index_id_) + : id(pid), index_id(index_id_) { } + ~TabletsChannelKey() noexcept { } + + bool operator==(const TabletsChannelKey& rhs) const noexcept { + return index_id == rhs.index_id && id == rhs.id; + } + + std::string to_string() const; +}; + +struct TabletsChannelKeyHasher { + std::size_t operator()(const TabletsChannelKey& key) const { + size_t seed = key.id.hash(); + return palo::HashUtil::hash(&key.index_id, sizeof(key.index_id), seed); + } +}; + +class Cache; + +// Mgr -> load -> tablet +// All dispached load data for this backend is routed from this class +class TabletWriterMgr { +public: + TabletWriterMgr(ExecEnv* exec_env); + ~TabletWriterMgr(); + + // open a new backend + Status open(const PTabletWriterOpenRequest& request); + + // this batch must belong to a index in one transaction + // when batch. + Status add_batch(const PTabletWriterAddBatchRequest& request, + google::protobuf::RepeatedPtrField* tablet_vec); + + // cancel all tablet stream for 'load_id' load + // id: stream load's id + Status cancel(const PTabletWriterCancelRequest& request); + +private: + ExecEnv* _exec_env; + // lock protect the channel map + std::mutex _lock; + + // A map from load_id|index_id to load channel + butil::FlatMap< + TabletsChannelKey, + std::shared_ptr, + TabletsChannelKeyHasher> _tablets_channels; + + Cache* _lastest_success_channel = nullptr; +}; + +std::ostream& operator<<(std::ostream& os, const TabletsChannelKey&); + +} diff --git a/be/src/tools/CMakeLists.txt b/be/src/tools/CMakeLists.txt new file mode 100644 index 0000000000..89de3af1f7 --- /dev/null +++ b/be/src/tools/CMakeLists.txt @@ -0,0 +1,36 @@ +# Copyright (c) 2018, Baidu.com, Inc. All Rights Reserved + +# Licensed 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. + +# where to put generated libraries +set(LIBRARY_OUTPUT_PATH "${BUILD_DIR}/src/tools") + +# where to put generated binaries +set(EXECUTABLE_OUTPUT_PATH "${BUILD_DIR}/src/tools") + +add_executable(meta_tool + meta_tool.cpp +) + +# This permits libraries loaded by dlopen to link to the symbols in the program. +# set_target_properties(palo_be PROPERTIES LINK_FLAGS -pthread) + +target_link_libraries(meta_tool + ${DORIS_LINK_LIBS} +) + +install(DIRECTORY DESTINATION ${OUTPUT_DIR}/lib/) + +install(TARGETS meta_tool + DESTINATION ${OUTPUT_DIR}/lib/) diff --git a/be/src/tools/meta_tool.cpp b/be/src/tools/meta_tool.cpp new file mode 100644 index 0000000000..eeb048b2f1 --- /dev/null +++ b/be/src/tools/meta_tool.cpp @@ -0,0 +1,167 @@ +// Copyright (c) 2018, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 +#include +#include +#include +#include +#include +#include + +#include "common/status.h" +#include "olap/store.h" +#include "olap/olap_header_manager.h" +#include "olap/olap_define.h" +#include "olap/olap_header.h" +#include "olap/olap_meta.h" +#include "olap/utils.h" +#include "json2pb/pb_to_json.h" + +using palo::OlapStore; +using palo::OlapMeta; +using palo::OlapHeaderManager; +using palo::OLAPHeader; +using palo::OLAPStatus; +using palo::OLAP_SUCCESS; +using palo::Status; + +const std::string HEADER_PREFIX = "hdr_"; + +DEFINE_string(root_path, "./", "storage root path"); +DEFINE_string(operation, "get_header", + "valid operation: get_header, flag, load_header, delete_header, rollback, show_header"); +DEFINE_int64(tablet_id, 0, "tablet_id for header operation"); +DEFINE_int32(schema_hash, 0, "schema_hash for header operation"); +DEFINE_string(json_header_path, "", "json header file path"); +DEFINE_string(pb_header_path, "", "pb header file path"); + +void print_usage(std::string progname) { + std::cout << progname << " is the Doris File tool." << std::endl; + std::cout << "Usage:" << std::endl; + std::cout << "./meta_tool --operation=get_header --tablet_id=tabletid --schema_hash=schemahash" << std::endl; + std::cout << "./meta_tool --operation=flag" << std::endl; + std::cout << "./meta_tool --operation=load_header --json_header_path=path" << std::endl; + std::cout << "./meta_tool --operation=delete_header --tablet_id=tabletid --schema_hash=schemahash" << std::endl; + std::cout << "./meta_tool --root_path=rootpath --operation=rollback" << std::endl; + std::cout << "./meta_tool --operation=show_header --pb_header_path=path" << std::endl; +} + +int main(int argc, char** argv) { + google::ParseCommandLineFlags(&argc, &argv, true); + + std::string root_path = FLAGS_root_path; + if (FLAGS_root_path == "") { + std::cout << "empty root path" << std::endl; + print_usage(argv[0]); + return -1; + } else if (FLAGS_root_path.find("/") != 0) { + // relative path + char dir[PATH_MAX] = {0}; + readlink("/proc/self/exe", dir, PATH_MAX); + std::string path_prefix(dir); + path_prefix = path_prefix.substr(0, path_prefix.rfind("/") + 1); + std::string root_path_postfix = FLAGS_root_path; + // trim tailing / + if (root_path_postfix.rfind("/") == (root_path_postfix.size() -1)) { + root_path_postfix = root_path_postfix.substr(0, root_path_postfix.size() -1); + } + + root_path = path_prefix + root_path_postfix; + } + std::unique_ptr store(new(std::nothrow) OlapStore(root_path)); + if (store.get() == NULL) { + std::cout << "new store failed" << std::endl; + return -1; + } + Status st = store->load(); + if (!st.ok()) { + std::cout << "store load failed" << std::endl; + return -1; + } + + if (FLAGS_operation == "get_header") { + std::string value; + OLAPStatus s = OlapHeaderManager::get_json_header(store.get(), FLAGS_tablet_id, FLAGS_schema_hash, &value); + if (s == palo::OLAP_ERR_META_KEY_NOT_FOUND) { + std::cout << "no header for tablet_id:" << FLAGS_tablet_id + << " schema_hash:" << FLAGS_schema_hash; + return 0; + } + std::cout << value << std::endl; + } else if (FLAGS_operation == "flag") { + bool converted = false; + OLAPStatus s = OlapHeaderManager::get_header_converted(store.get(), converted); + if (s != OLAP_SUCCESS) { + std::cout << "get header converted flag failed" << std::endl; + return -1; + } + std::cout << "is_header_converted is " << converted << std::endl; + } else if (FLAGS_operation == "load_header") { + OLAPStatus s = OlapHeaderManager::load_json_header(store.get(), FLAGS_json_header_path); + if (s != OLAP_SUCCESS) { + std::cout << "load header failed" << std::endl; + return -1; + } + std::cout << "load header successfully" << std::endl; + } else if (FLAGS_operation == "delete_header") { + OLAPStatus s = OlapHeaderManager::remove(store.get(), FLAGS_tablet_id, FLAGS_schema_hash); + if (s != OLAP_SUCCESS) { + std::cout << "delete header failed for tablet_id:" << FLAGS_tablet_id + << " schema_hash:" << FLAGS_schema_hash << std::endl; + return -1; + } + std::cout << "delete header successfully" << std::endl; + } else if (FLAGS_operation == "rollback") { + auto rollback_func = [&root_path](long tablet_id, + long schema_hash, const std::string& value) -> bool { + OLAPHeader olap_header; + bool parsed = olap_header.ParseFromString(value); + if (!parsed) { + std::cout << "parse header failed"; + return true; + } + std::string tablet_id_str = std::to_string(tablet_id); + std::string schema_hash_path = root_path + "/data/" + std::to_string(olap_header.shard()) + + "/" + tablet_id_str + "/" + std::to_string(schema_hash); + std::string header_file_path = schema_hash_path + "/" + tablet_id_str + ".hdr"; + std::cout << "save header to path:" << header_file_path << std::endl; + OLAPStatus s = olap_header.save(header_file_path); + if (s != OLAP_SUCCESS) { + std::cout << "save header file to path:" << header_file_path << " failed" << std::endl; + } + return true; + }; + OlapHeaderManager::traverse_headers(store->get_meta(), rollback_func); + } else if (FLAGS_operation == "show_header") { + OLAPHeader header(FLAGS_pb_header_path); + OLAPStatus s = header.load_and_init(); + if (s != OLAP_SUCCESS) { + std::cout << "load pb header file:" << FLAGS_pb_header_path << " failed" << std::endl; + return -1; + } + std::string json_header; + json2pb::Pb2JsonOptions json_options; + json_options.pretty_json = true; + json2pb::ProtoMessageToJson(header, &json_header, json_options); + std::cout << "header:" << std::endl; + std::cout << json_header << std::endl; + } else { + std::cout << "invalid operation:" << FLAGS_operation << std::endl; + print_usage(argv[0]); + return -1; + } + return 0; +} diff --git a/be/src/util/arena.cc b/be/src/util/arena.cc new file mode 100644 index 0000000000..2503275edb --- /dev/null +++ b/be/src/util/arena.cc @@ -0,0 +1,68 @@ +// Copyright (c) 2011 The LevelDB Authors. All rights reserved. +// Use of this source code is governed by a BSD-style license that can be +// found in the LICENSE file. See the AUTHORS file for names of contributors. + +#include "util/arena.h" +#include + +namespace palo { + +static const int kBlockSize = 4096; + +Arena::Arena() : memory_usage_(0) { + alloc_ptr_ = NULL; // First allocation will allocate a block + alloc_bytes_remaining_ = 0; +} + +Arena::~Arena() { + for (size_t i = 0; i < blocks_.size(); i++) { + delete[] blocks_[i]; + } +} + +char* Arena::AllocateFallback(size_t bytes) { + if (bytes > kBlockSize / 4) { + // Object is more than a quarter of our block size. Allocate it separately + // to avoid wasting too much space in leftover bytes. + char* result = AllocateNewBlock(bytes); + return result; + } + + // We waste the remaining space in the current block. + alloc_ptr_ = AllocateNewBlock(kBlockSize); + alloc_bytes_remaining_ = kBlockSize; + + char* result = alloc_ptr_; + alloc_ptr_ += bytes; + alloc_bytes_remaining_ -= bytes; + return result; +} + +char* Arena::AllocateAligned(size_t bytes) { + const int align = (sizeof(void*) > 8) ? sizeof(void*) : 8; + assert((align & (align-1)) == 0); // Pointer size should be a power of 2 + size_t current_mod = reinterpret_cast(alloc_ptr_) & (align-1); + size_t slop = (current_mod == 0 ? 0 : align - current_mod); + size_t needed = bytes + slop; + char* result; + if (needed <= alloc_bytes_remaining_) { + result = alloc_ptr_ + slop; + alloc_ptr_ += needed; + alloc_bytes_remaining_ -= needed; + } else { + // AllocateFallback always returned aligned memory + result = AllocateFallback(bytes); + } + assert((reinterpret_cast(result) & (align-1)) == 0); + return result; +} + +char* Arena::AllocateNewBlock(size_t block_bytes) { + char* result = new char[block_bytes]; + blocks_.push_back(result); + memory_usage_.store(MemoryUsage() + block_bytes + sizeof(char*), + std::memory_order_relaxed); + return result; +} + +} // namespace palo diff --git a/be/src/util/arena.h b/be/src/util/arena.h new file mode 100644 index 0000000000..8b291fa5da --- /dev/null +++ b/be/src/util/arena.h @@ -0,0 +1,69 @@ +// Copyright (c) 2011 The LevelDB Authors. All rights reserved. +// Use of this source code is governed by a BSD-style license that can be +// found in the LICENSE file. See the AUTHORS file for names of contributors. + +#ifndef BDG_PALO_BE_SRC_COMMON_UTIL_ARENA_H +#define BDG_PALO_BE_SRC_COMMON_UTIL_ARENA_H + +#include +#include +#include + +#include +#include + +#include "common/compiler_util.h" + +namespace palo { + +class Arena { +public: + Arena(); + ~Arena(); + + // Return a pointer to a newly allocated memory block of "bytes" bytes. + char* Allocate(size_t bytes); + + // Allocate memory with the normal alignment guarantees provided by malloc + char* AllocateAligned(size_t bytes); + + // Returns an estimate of the total memory usage of data allocated + // by the arena. + size_t MemoryUsage() const { + //return reinterpret_cast(memory_usage_.NoBarrier_Load()); + return memory_usage_.load(std::memory_order_relaxed); + } + +private: + char* AllocateFallback(size_t bytes); + char* AllocateNewBlock(size_t block_bytes); + + // Allocation state + char* alloc_ptr_; + size_t alloc_bytes_remaining_; + + // Array of new[] allocated memory blocks + std::vector blocks_; + + // Total memory usage of the arena. + std::atomic memory_usage_; + + // No copying allowed + Arena(const Arena&); + void operator=(const Arena&); +}; + +inline char* Arena::Allocate(size_t bytes) { + if (UNLIKELY(bytes == 0)) { return nullptr; } + if (bytes <= alloc_bytes_remaining_) { + char* result = alloc_ptr_; + alloc_ptr_ += bytes; + alloc_bytes_remaining_ -= bytes; + return result; + } + return AllocateFallback(bytes); +} + +} // namespace palo + +#endif // BDG_PALO_BE_SRC_COMMON_UTIL_ARENA_H diff --git a/be/src/util/byte_buffer.h b/be/src/util/byte_buffer.h new file mode 100644 index 0000000000..d3e3104692 --- /dev/null +++ b/be/src/util/byte_buffer.h @@ -0,0 +1,68 @@ +// Copyright (c) 2017, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 "common/logging.h" + +namespace palo { + +struct ByteBuffer; +using ByteBufferPtr = std::shared_ptr; + +struct ByteBuffer { + static ByteBufferPtr allocate(size_t size) { + ByteBufferPtr ptr(new ByteBuffer(size)); + return ptr; + } + + ~ByteBuffer() { delete[] ptr; } + + void put_bytes(const char* data, size_t size) { + memcpy(ptr + pos , data, size); + pos += size; + } + + void get_bytes(char* data, size_t size) { + memcpy(data, ptr + pos, size); + pos += size; + DCHECK(pos <= limit); + } + + void flip() { + limit = pos; + pos = 0; + } + + size_t remaining() const { return limit - pos; } + bool has_remaining() const { return limit > pos; } + + char* const ptr; + size_t pos; + size_t limit; + size_t capacity; +private: + ByteBuffer(size_t capacity_) + : ptr(new char[capacity_]), pos(0), + limit(capacity_), capacity(capacity_) { + } +}; + +} diff --git a/be/src/util/json_util.cpp b/be/src/util/json_util.cpp new file mode 100644 index 0000000000..69edcf94a5 --- /dev/null +++ b/be/src/util/json_util.cpp @@ -0,0 +1,46 @@ +// Copyright (c) 2017, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 "util/json_util.h" + +#include +#include + +namespace palo { + +std::string to_json(const Status& status) { + rapidjson::StringBuffer s; + rapidjson::PrettyWriter writer(s); + + writer.StartObject(); + // status + writer.Key("status"); + if (status.ok()) { + writer.String("Success"); + } else { + writer.String("Fail"); + } + // msg + writer.Key("msg"); + if (status.ok()) { + writer.String("OK"); + } else { + writer.String(status.get_error_msg().c_str()); + } + writer.EndObject(); + return s.GetString(); +} + +} diff --git a/be/src/util/random.h b/be/src/util/random.h new file mode 100644 index 0000000000..a91b1b0e6d --- /dev/null +++ b/be/src/util/random.h @@ -0,0 +1,64 @@ +// Copyright (c) 2011 The LevelDB Authors. All rights reserved. +// Use of this source code is governed by a BSD-style license that can be +// found in the LICENSE file. See the AUTHORS file for names of contributors. + +#ifndef BDG_PALO_BE_SRC_COMMON_UTIL_RANDOM_H +#define BDG_PALO_BE_SRC_COMMON_UTIL_RANDOM_H + +#include + +namespace palo { + +// A very simple random number generator. Not especially good at +// generating truly random bits, but good enough for our needs in this +// package. +class Random { +private: + uint32_t seed_; +public: + explicit Random(uint32_t s) : seed_(s & 0x7fffffffu) { + // Avoid bad seeds. + if (seed_ == 0 || seed_ == 2147483647L) { + seed_ = 1; + } + } + uint32_t Next() { + static const uint32_t M = 2147483647L; // 2^31-1 + static const uint64_t A = 16807; // bits 14, 8, 7, 5, 2, 1, 0 + // We are computing + // seed_ = (seed_ * A) % M, where M = 2^31-1 + // + // seed_ must not be zero or M, or else all subsequent computed values + // will be zero or M respectively. For all other values, seed_ will end + // up cycling through every number in [1,M-1] + uint64_t product = seed_ * A; + + // Compute (product % M) using the fact that ((x << 31) % M) == x. + seed_ = static_cast((product >> 31) + (product & M)); + // The first reduction may overflow by 1 bit, so we may need to + // repeat. mod == M is not possible; using > allows the faster + // sign-bit-based test. + if (seed_ > M) { + seed_ -= M; + } + return seed_; + } + // Returns a uniformly distributed value in the range [0..n-1] + // REQUIRES: n > 0 + uint32_t Uniform(int n) { return Next() % n; } + + // Randomly returns true ~"1/n" of the time, and false otherwise. + // REQUIRES: n > 0 + bool OneIn(int n) { return (Next() % n) == 0; } + + // Skewed: pick "base" uniformly from range [0,max_log] and then + // return "base" random bits. The effect is to pick a number in the + // range [0,2^max_log-1] with exponential bias towards smaller numbers. + uint32_t Skewed(int max_log) { + return Uniform(1 << Uniform(max_log + 1)); + } +}; + +} // namespace palo + +#endif //BDG_PALO_BE_SRC_COMMON_UTIL_RANDOM_H diff --git a/be/src/util/ref_count_closure.h b/be/src/util/ref_count_closure.h new file mode 100644 index 0000000000..f34ebb0ae0 --- /dev/null +++ b/be/src/util/ref_count_closure.h @@ -0,0 +1,53 @@ +// Copyright (c) 2018, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 "service/brpc.h" + +namespace palo { + +template +class RefCountClosure : public google::protobuf::Closure { +public: + RefCountClosure() : _refs(0) { } + ~RefCountClosure() { } + + void ref() { _refs.fetch_add(1); } + + // If unref() returns true, this object should be delete + bool unref() { return _refs.fetch_sub(1) == 1; } + + void Run() override { + if (unref()) { + delete this; + } + } + + void join() { + brpc::Join(cntl.call_id()); + } + + brpc::Controller cntl; + T result; +private: + std::atomic _refs; +}; + +} diff --git a/be/src/util/string_util.h b/be/src/util/string_util.h new file mode 100644 index 0000000000..c78671ab7e --- /dev/null +++ b/be/src/util/string_util.h @@ -0,0 +1,65 @@ +// Copyright (c) 2017, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 // to_lower_copy + +namespace palo { + +struct StringCaseHasher { +public: + std::size_t operator()(const std::string& value) const { + std::string lower_value = boost::algorithm::to_lower_copy(value); + return std::hash()(lower_value); + } +}; + +struct StringCaseEqual { +public: + bool operator()(const std::string& lhs, const std::string& rhs) const { + if (lhs.size() != rhs.size()) { + return false; + } + return strncasecmp(lhs.c_str(), rhs.c_str(), 0) == 0; + } +}; + +struct StringCaseLess { +public: + bool operator()(const std::string& lhs, const std::string& rhs) const { + size_t common_size = std::min(lhs.size(), rhs.size()); + auto cmp = strncasecmp(lhs.c_str(), rhs.c_str(), common_size); + if (cmp == 0) { + return lhs.size() < rhs.size(); + } + return cmp < 0; + } +}; + +using StringCaseSet = std::set; +using StringCaseUnorderedSet = std::unordered_set; +template +using StringCaseMap = std::map; +template +using StringCaseUnorderedMap = std::unordered_map; + +} diff --git a/be/src/util/uid_util.cpp b/be/src/util/uid_util.cpp new file mode 100644 index 0000000000..2a926f0ffa --- /dev/null +++ b/be/src/util/uid_util.cpp @@ -0,0 +1,25 @@ +// Copyright (c) 2018, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 "util/uid_util.h" + +namespace palo { + +std::ostream& operator<<(std::ostream& os, const UniqueId& uid) { + os << uid.to_string(); + return os; +} + +} diff --git a/be/test/exec/olap_table_info_test.cpp b/be/test/exec/olap_table_info_test.cpp new file mode 100644 index 0000000000..50f9e941d1 --- /dev/null +++ b/be/test/exec/olap_table_info_test.cpp @@ -0,0 +1,439 @@ +// Copyright (c) 2017, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 "exec/olap_table_info.h" + +#include + +#include "util/descriptor_helper.h" +#include "runtime/mem_tracker.h" +#include "runtime/row_batch.h" +#include "runtime/tuple_row.h" + +namespace palo { + +class OlapTablePartitionParamTest : public testing::Test { +public: + OlapTablePartitionParamTest() { } + virtual ~OlapTablePartitionParamTest() { } + void SetUp() override { } +private: +}; + +TOlapTableSchemaParam get_schema(TDescriptorTable* desc_tbl) { + TOlapTableSchemaParam t_schema_param; + t_schema_param.db_id = 1; + t_schema_param.table_id = 2; + t_schema_param.version = 0; + + // descriptor + { + TDescriptorTableBuilder dtb; + TTupleDescriptorBuilder tuple_builder; + + tuple_builder.add_slot( + TSlotDescriptorBuilder().type(TYPE_INT).column_name("c1").column_pos(1).build()); + tuple_builder.add_slot( + TSlotDescriptorBuilder().type(TYPE_BIGINT).column_name("c2").column_pos(2).build()); + tuple_builder.add_slot( + TSlotDescriptorBuilder().string_type(20).column_name("c3").column_pos(3).build()); + + tuple_builder.build(&dtb); + + *desc_tbl = dtb.desc_tbl(); + t_schema_param.slot_descs = desc_tbl->slotDescriptors; + t_schema_param.tuple_desc = desc_tbl->tupleDescriptors[0]; + } + // index + t_schema_param.indexes.resize(2); + t_schema_param.indexes[0].id = 4; + t_schema_param.indexes[0].columns = {"c1", "c2", "c3"}; + t_schema_param.indexes[1].id = 5; + t_schema_param.indexes[1].columns = {"c1", "c3"}; + + return t_schema_param; +} + +TEST_F(OlapTablePartitionParamTest, normal) { + TDescriptorTable t_desc_tbl; + auto t_schema = get_schema(&t_desc_tbl); + std::shared_ptr schema(new OlapTableSchemaParam()); + auto st = schema->init(t_schema); + ASSERT_TRUE(st.ok()); + LOG(INFO) << schema->debug_string(); + + // (-oo, 10] | [10.50) | [60, +oo) + TOlapTablePartitionParam t_partition_param; + t_partition_param.db_id = 1; + t_partition_param.table_id = 2; + t_partition_param.version = 0; + t_partition_param.__set_partition_column("c2"); + t_partition_param.__set_distributed_columns({"c1", "c3"}); + t_partition_param.partitions.resize(3); + t_partition_param.partitions[0].id = 10; + t_partition_param.partitions[0].__isset.end_key = true; + t_partition_param.partitions[0].end_key.node_type = TExprNodeType::INT_LITERAL; + t_partition_param.partitions[0].end_key.type = t_desc_tbl.slotDescriptors[1].slotType; + t_partition_param.partitions[0].end_key.num_children = 0; + t_partition_param.partitions[0].end_key.__isset.int_literal = true; + t_partition_param.partitions[0].end_key.int_literal.value = 10; + t_partition_param.partitions[0].num_buckets = 1; + t_partition_param.partitions[0].indexes.resize(2); + t_partition_param.partitions[0].indexes[0].index_id = 4; + t_partition_param.partitions[0].indexes[0].tablets = {21}; + t_partition_param.partitions[0].indexes[1].index_id = 5; + t_partition_param.partitions[0].indexes[1].tablets = {22}; + + t_partition_param.partitions[1].id = 11; + t_partition_param.partitions[1].__isset.start_key = true; + t_partition_param.partitions[1].start_key.node_type = TExprNodeType::INT_LITERAL; + t_partition_param.partitions[1].start_key.type = t_desc_tbl.slotDescriptors[1].slotType; + t_partition_param.partitions[1].start_key.num_children = 0; + t_partition_param.partitions[1].start_key.__isset.int_literal = true; + t_partition_param.partitions[1].start_key.int_literal.value = 10; + t_partition_param.partitions[1].__isset.end_key = true; + t_partition_param.partitions[1].end_key.node_type = TExprNodeType::INT_LITERAL; + t_partition_param.partitions[1].end_key.type = t_desc_tbl.slotDescriptors[1].slotType; + t_partition_param.partitions[1].end_key.num_children = 0; + t_partition_param.partitions[1].end_key.__isset.int_literal = true; + t_partition_param.partitions[1].end_key.int_literal.value = 50; + t_partition_param.partitions[1].num_buckets = 2; + t_partition_param.partitions[1].indexes.resize(2); + t_partition_param.partitions[1].indexes[0].index_id = 4; + t_partition_param.partitions[1].indexes[0].tablets = {31, 32}; + t_partition_param.partitions[1].indexes[1].index_id = 5; + t_partition_param.partitions[1].indexes[1].tablets = {33, 34}; + + t_partition_param.partitions[2].id = 12; + t_partition_param.partitions[2].__isset.start_key = true; + t_partition_param.partitions[2].start_key.node_type = TExprNodeType::INT_LITERAL; + t_partition_param.partitions[2].start_key.type = t_desc_tbl.slotDescriptors[1].slotType; + t_partition_param.partitions[2].start_key.num_children = 0; + t_partition_param.partitions[2].start_key.__isset.int_literal = true; + t_partition_param.partitions[2].start_key.int_literal.value = 60; + t_partition_param.partitions[2].num_buckets = 4; + t_partition_param.partitions[2].indexes.resize(2); + t_partition_param.partitions[2].indexes[0].index_id = 4; + t_partition_param.partitions[2].indexes[0].tablets = {41, 42, 43, 44}; + t_partition_param.partitions[2].indexes[1].index_id = 5; + t_partition_param.partitions[2].indexes[1].tablets = {45, 46, 47, 48}; + + OlapTablePartitionParam part(schema, t_partition_param); + st = part.init(); + ASSERT_TRUE(st.ok()); + LOG(INFO) << part.debug_string(); + + ObjectPool pool; + DescriptorTbl* desc_tbl = nullptr; + st = DescriptorTbl::create(&pool, t_desc_tbl, &desc_tbl); + ASSERT_TRUE(st.ok()); + RowDescriptor row_desc(*desc_tbl, {0}, {false}); + TupleDescriptor* tuple_desc = desc_tbl->get_tuple_descriptor(0); + MemTracker tracker; + RowBatch batch(row_desc, 1024, &tracker); + // 12, 9, "abc" + { + Tuple* tuple = (Tuple*)batch.tuple_data_pool()->allocate(tuple_desc->byte_size()); + memset(tuple, 0, tuple_desc->byte_size()); + + *reinterpret_cast(tuple->get_slot(4)) = 12; + *reinterpret_cast(tuple->get_slot(8)) = 9; + StringValue* str_val = reinterpret_cast(tuple->get_slot(16)); + str_val->ptr = (char*)batch.tuple_data_pool()->allocate(10); + str_val->len = 3; + memcpy(str_val->ptr, "abc", str_val->len); + + // 9: + uint32_t dist_hash = 0; + const OlapTablePartition* partition = nullptr; + auto found = part.find_tablet(tuple, &partition, &dist_hash); + ASSERT_TRUE(found); + ASSERT_EQ(10, partition->id); + } + // 13, 25, "abcd" + { + Tuple* tuple = (Tuple*)batch.tuple_data_pool()->allocate(tuple_desc->byte_size()); + memset(tuple, 0, tuple_desc->byte_size()); + + *reinterpret_cast(tuple->get_slot(4)) = 13; + *reinterpret_cast(tuple->get_slot(8)) = 25; + StringValue* str_val = reinterpret_cast(tuple->get_slot(16)); + str_val->ptr = (char*)batch.tuple_data_pool()->allocate(10); + str_val->len = 4; + memcpy(str_val->ptr, "abcd", str_val->len); + + // 25: + uint32_t dist_hash = 0; + const OlapTablePartition* partition = nullptr; + auto found = part.find_tablet(tuple, &partition, &dist_hash); + ASSERT_TRUE(found); + ASSERT_EQ(11, partition->id); + } + // 14, 50, "abcde" + { + Tuple* tuple = (Tuple*)batch.tuple_data_pool()->allocate(tuple_desc->byte_size()); + memset(tuple, 0, tuple_desc->byte_size()); + + *reinterpret_cast(tuple->get_slot(4)) = 14; + *reinterpret_cast(tuple->get_slot(8)) = 50; + StringValue* str_val = reinterpret_cast(tuple->get_slot(16)); + str_val->ptr = reinterpret_cast(batch.tuple_data_pool()->allocate(10)); + str_val->len = 5; + memcpy(str_val->ptr, "abcde", str_val->len); + + // 50: + uint32_t dist_hash = 0; + const OlapTablePartition* partition = nullptr; + auto found = part.find_tablet(tuple, &partition, &dist_hash); + ASSERT_FALSE(found); + } + + // 15, 60, "abcdef" + { + Tuple* tuple = (Tuple*)batch.tuple_data_pool()->allocate(tuple_desc->byte_size()); + memset(tuple, 0, tuple_desc->byte_size()); + + *reinterpret_cast(tuple->get_slot(4)) = 15; + *reinterpret_cast(tuple->get_slot(8)) = 60; + StringValue* str_val = reinterpret_cast(tuple->get_slot(16)); + str_val->ptr = reinterpret_cast(batch.tuple_data_pool()->allocate(10)); + str_val->len = 6; + memcpy(str_val->ptr, "abcdef", str_val->len); + + // 60: + uint32_t dist_hash = 0; + const OlapTablePartition* partition = nullptr; + auto found = part.find_tablet(tuple, &partition, &dist_hash); + ASSERT_TRUE(found); + ASSERT_EQ(12, partition->id); + } +} + +TEST_F(OlapTablePartitionParamTest, to_protobuf) { + TDescriptorTable t_desc_tbl; + auto t_schema = get_schema(&t_desc_tbl); + std::shared_ptr schema(new OlapTableSchemaParam()); + auto st = schema->init(t_schema); + ASSERT_TRUE(st.ok()); + POlapTableSchemaParam pschema; + schema->to_protobuf(&pschema); + { + std::shared_ptr schema2(new OlapTableSchemaParam()); + auto st = schema2->init(pschema); + ASSERT_TRUE(st.ok()); + + ASSERT_STREQ(schema->debug_string().c_str(), schema2->debug_string().c_str()); + } +} + +TEST_F(OlapTablePartitionParamTest, unknown_index_column) { + TDescriptorTable t_desc_tbl; + auto tschema = get_schema(&t_desc_tbl); + std::shared_ptr schema(new OlapTableSchemaParam()); + tschema.indexes[0].columns.push_back("unknown_col"); + auto st = schema->init(tschema); + ASSERT_FALSE(st.ok()); +} + +TEST_F(OlapTablePartitionParamTest, unpartitioned) { + TDescriptorTable t_desc_tbl; + auto t_schema = get_schema(&t_desc_tbl); + std::shared_ptr schema(new OlapTableSchemaParam()); + auto st = schema->init(t_schema); + ASSERT_TRUE(st.ok()); + + // (-oo, 10] | [10.50) | [60, +oo) + TOlapTablePartitionParam t_partition_param; + t_partition_param.db_id = 1; + t_partition_param.table_id = 2; + t_partition_param.version = 0; + t_partition_param.__set_distributed_columns({"c1", "c3"}); + t_partition_param.partitions.resize(1); + t_partition_param.partitions[0].id = 10; + t_partition_param.partitions[0].num_buckets = 1; + t_partition_param.partitions[0].indexes.resize(2); + t_partition_param.partitions[0].indexes[0].index_id = 4; + t_partition_param.partitions[0].indexes[0].tablets = {21}; + t_partition_param.partitions[0].indexes[1].index_id = 5; + + OlapTablePartitionParam part(schema, t_partition_param); + st = part.init(); + ASSERT_TRUE(st.ok()); + + ObjectPool pool; + DescriptorTbl* desc_tbl = nullptr; + st = DescriptorTbl::create(&pool, t_desc_tbl, &desc_tbl); + ASSERT_TRUE(st.ok()); + RowDescriptor row_desc(*desc_tbl, {0}, {false}); + TupleDescriptor* tuple_desc = desc_tbl->get_tuple_descriptor(0); + MemTracker tracker; + RowBatch batch(row_desc, 1024, &tracker); + // 12, 9, "abc" + { + Tuple* tuple = (Tuple*)batch.tuple_data_pool()->allocate(tuple_desc->byte_size()); + memset(tuple, 0, tuple_desc->byte_size()); + + *reinterpret_cast(tuple->get_slot(4)) = 12; + *reinterpret_cast(tuple->get_slot(8)) = 9; + StringValue* str_val = reinterpret_cast(tuple->get_slot(16)); + str_val->ptr = (char*)batch.tuple_data_pool()->allocate(10); + str_val->len = 3; + memcpy(str_val->ptr, "abc", str_val->len); + + // 9: + uint32_t dist_hash = 0; + const OlapTablePartition* partition = nullptr; + auto found = part.find_tablet(tuple, &partition, &dist_hash); + ASSERT_TRUE(found); + ASSERT_EQ(10, partition->id); + } +} + +TEST_F(OlapTablePartitionParamTest, unknown_partition_column) { + TDescriptorTable t_desc_tbl; + auto t_schema = get_schema(&t_desc_tbl); + std::shared_ptr schema(new OlapTableSchemaParam()); + auto st = schema->init(t_schema); + ASSERT_TRUE(st.ok()); + + // (-oo, 10] | [10.50) | [60, +oo) + TOlapTablePartitionParam t_partition_param; + t_partition_param.db_id = 1; + t_partition_param.table_id = 2; + t_partition_param.version = 0; + t_partition_param.__set_partition_column("c4"); + t_partition_param.__set_distributed_columns({"c1", "c3"}); + t_partition_param.partitions.resize(1); + t_partition_param.partitions[0].id = 10; + t_partition_param.partitions[0].num_buckets = 1; + t_partition_param.partitions[0].indexes.resize(2); + t_partition_param.partitions[0].indexes[0].index_id = 4; + t_partition_param.partitions[0].indexes[0].tablets = {21}; + t_partition_param.partitions[0].indexes[1].index_id = 5; + + OlapTablePartitionParam part(schema, t_partition_param); + st = part.init(); + ASSERT_FALSE(st.ok()); +} + +TEST_F(OlapTablePartitionParamTest, unknown_distributed_col) { + TDescriptorTable t_desc_tbl; + auto t_schema = get_schema(&t_desc_tbl); + std::shared_ptr schema(new OlapTableSchemaParam()); + auto st = schema->init(t_schema); + ASSERT_TRUE(st.ok()); + + // (-oo, 10] | [10.50) | [60, +oo) + TOlapTablePartitionParam t_partition_param; + t_partition_param.db_id = 1; + t_partition_param.table_id = 2; + t_partition_param.version = 0; + t_partition_param.__set_distributed_columns({"c4"}); + t_partition_param.partitions.resize(1); + t_partition_param.partitions[0].id = 10; + t_partition_param.partitions[0].num_buckets = 1; + t_partition_param.partitions[0].indexes.resize(2); + t_partition_param.partitions[0].indexes[0].index_id = 4; + t_partition_param.partitions[0].indexes[0].tablets = {21}; + t_partition_param.partitions[0].indexes[1].index_id = 5; + + OlapTablePartitionParam part(schema, t_partition_param); + st = part.init(); + ASSERT_FALSE(st.ok()); +} + +TEST_F(OlapTablePartitionParamTest, bad_index) { + TDescriptorTable t_desc_tbl; + auto t_schema = get_schema(&t_desc_tbl); + std::shared_ptr schema(new OlapTableSchemaParam()); + auto st = schema->init(t_schema); + ASSERT_TRUE(st.ok()); + + { + // (-oo, 10] | [10.50) | [60, +oo) + TOlapTablePartitionParam t_partition_param; + t_partition_param.db_id = 1; + t_partition_param.table_id = 2; + t_partition_param.version = 0; + t_partition_param.__set_distributed_columns({"c1", "c3"}); + t_partition_param.partitions.resize(1); + t_partition_param.partitions[0].id = 10; + t_partition_param.partitions[0].num_buckets = 1; + t_partition_param.partitions[0].indexes.resize(1); + t_partition_param.partitions[0].indexes[0].index_id = 4; + t_partition_param.partitions[0].indexes[0].tablets = {21}; + + OlapTablePartitionParam part(schema, t_partition_param); + st = part.init(); + ASSERT_FALSE(st.ok()); + } + { + // (-oo, 10] | [10.50) | [60, +oo) + TOlapTablePartitionParam t_partition_param; + t_partition_param.db_id = 1; + t_partition_param.table_id = 2; + t_partition_param.version = 0; + t_partition_param.__set_partition_column("c4"); + t_partition_param.__set_distributed_columns({"c1", "c3"}); + t_partition_param.partitions.resize(1); + t_partition_param.partitions[0].id = 10; + t_partition_param.partitions[0].num_buckets = 1; + t_partition_param.partitions[0].indexes.resize(2); + t_partition_param.partitions[0].indexes[0].index_id = 4; + t_partition_param.partitions[0].indexes[0].tablets = {21}; + t_partition_param.partitions[0].indexes[1].index_id = 6; + + OlapTablePartitionParam part(schema, t_partition_param); + st = part.init(); + ASSERT_FALSE(st.ok()); + } +} + +TEST_F(OlapTablePartitionParamTest, tableLoacation) { + TOlapTableLocationParam tparam; + tparam.tablets.resize(1); + tparam.tablets[0].tablet_id = 1; + OlapTableLocationParam location(tparam); + { + auto loc = location.find_tablet(1); + ASSERT_TRUE(loc != nullptr); + } + { + auto loc = location.find_tablet(2); + ASSERT_TRUE(loc == nullptr); + } +} + +TEST_F(OlapTablePartitionParamTest, NodesInfo) { + TPaloNodesInfo tinfo; + tinfo.nodes.resize(1); + tinfo.nodes[0].id = 1; + PaloNodesInfo nodes(tinfo); + { + auto node = nodes.find_node(1); + ASSERT_TRUE(node != nullptr); + } + { + auto node = nodes.find_node(2); + ASSERT_TRUE(node == nullptr); + } +} + +} + +int main(int argc, char* argv[]) { + ::testing::InitGoogleTest(&argc, argv); + return RUN_ALL_TESTS(); +} diff --git a/be/test/exec/olap_table_sink_test.cpp b/be/test/exec/olap_table_sink_test.cpp new file mode 100644 index 0000000000..d208058c3d --- /dev/null +++ b/be/test/exec/olap_table_sink_test.cpp @@ -0,0 +1,927 @@ +// Copyright (c) 2017, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 "exec/olap_table_sink.h" + +#include + +#include "gen_cpp/internal_service.pb.h" +#include "runtime/exec_env.h" +#include "runtime/row_batch.h" +#include "runtime/tuple_row.h" +#include "runtime/runtime_state.h" +#include "runtime/decimal_value.h" +#include "service/brpc.h" +#include "util/descriptor_helper.h" +#include "util/cpu_info.h" +#include "util/debug_util.h" + +namespace palo { +namespace stream_load { + +Status k_add_batch_status; + +class OlapTableSinkTest : public testing::Test { +public: + OlapTableSinkTest() { } + virtual ~OlapTableSinkTest() { } + void SetUp() override { + k_add_batch_status = Status::OK; + } +private: +}; + +TDataSink get_data_sink(TDescriptorTable* desc_tbl) { + int64_t db_id = 1; + int64_t table_id = 2; + int64_t partition_id = 3; + int64_t index1_id = 4; + int64_t tablet1_id = 6; + int64_t tablet2_id = 7; + + TDataSink data_sink; + data_sink.type = TDataSinkType::OLAP_TABLE_SINK; + data_sink.__isset.olap_table_sink = true; + + TOlapTableSink& tsink = data_sink.olap_table_sink; + tsink.load_id.hi = 123; + tsink.load_id.lo = 456; + tsink.txn_id = 789; + tsink.db_id = 1; + tsink.table_id = 2; + tsink.tuple_id = 0; + tsink.num_replicas = 3; + tsink.db_name = "testDb"; + tsink.table_name = "testTable"; + + // cosntruct schema + TOlapTableSchemaParam& tschema = tsink.schema; + tschema.db_id = 1; + tschema.table_id = 2; + tschema.version = 0; + + // descriptor + { + TDescriptorTableBuilder dtb; + { + TTupleDescriptorBuilder tuple_builder; + + tuple_builder.add_slot( + TSlotDescriptorBuilder().type(TYPE_INT).column_name("c1").column_pos(1).build()); + tuple_builder.add_slot( + TSlotDescriptorBuilder().type(TYPE_BIGINT).column_name("c2").column_pos(2).build()); + tuple_builder.add_slot( + TSlotDescriptorBuilder().string_type(10).column_name("c3").column_pos(3).build()); + + tuple_builder.build(&dtb); + } + { + TTupleDescriptorBuilder tuple_builder; + + tuple_builder.add_slot( + TSlotDescriptorBuilder().type(TYPE_INT).column_name("c1").column_pos(1).build()); + tuple_builder.add_slot( + TSlotDescriptorBuilder().type(TYPE_BIGINT).column_name("c2").column_pos(2).build()); + tuple_builder.add_slot( + TSlotDescriptorBuilder().string_type(20).column_name("c3").column_pos(3).build()); + + tuple_builder.build(&dtb); + } + + *desc_tbl = dtb.desc_tbl(); + tschema.slot_descs = desc_tbl->slotDescriptors; + tschema.tuple_desc = desc_tbl->tupleDescriptors[0]; + } + // index + tschema.indexes.resize(1); + tschema.indexes[0].id = index1_id; + tschema.indexes[0].columns = {"c1", "c2", "c3"}; + // tschema.indexes[1].id = 5; + // tschema.indexes[1].columns = {"c1", "c3"}; + // partition + TOlapTablePartitionParam& tpartition = tsink.partition; + tpartition.db_id = db_id; + tpartition.table_id = table_id; + tpartition.version = table_id; + tpartition.__set_partition_column("c2"); + tpartition.__set_distributed_columns({"c1", "c3"}); + tpartition.partitions.resize(1); + tpartition.partitions[0].id = partition_id; + tpartition.partitions[0].num_buckets = 2; + tpartition.partitions[0].indexes.resize(1); + tpartition.partitions[0].indexes[0].index_id = index1_id; + tpartition.partitions[0].indexes[0].tablets = {tablet1_id, tablet2_id}; + // location + TOlapTableLocationParam& location = tsink.location; + location.db_id = db_id; + location.table_id = table_id; + location.version = 0; + location.tablets.resize(2); + location.tablets[0].tablet_id = tablet1_id; + location.tablets[0].node_ids = {0, 1, 2}; + location.tablets[1].tablet_id = tablet2_id; + location.tablets[1].node_ids = {0, 1, 2}; + // location + TPaloNodesInfo& nodes_info = tsink.nodes_info; + nodes_info.nodes.resize(3); + nodes_info.nodes[0].id = 0; + nodes_info.nodes[0].host = "127.0.0.1"; + nodes_info.nodes[0].async_internal_port = 4356; + nodes_info.nodes[1].id = 1; + nodes_info.nodes[1].host = "127.0.0.1"; + nodes_info.nodes[1].async_internal_port = 4356; + nodes_info.nodes[2].id = 2; + nodes_info.nodes[2].host = "127.0.0.1"; + nodes_info.nodes[2].async_internal_port = 4357; + + return data_sink; +} + +TDataSink get_decimal_sink(TDescriptorTable* desc_tbl) { + int64_t db_id = 1; + int64_t table_id = 2; + int64_t partition_id = 3; + int64_t index1_id = 4; + int64_t tablet1_id = 6; + int64_t tablet2_id = 7; + + TDataSink data_sink; + data_sink.type = TDataSinkType::OLAP_TABLE_SINK; + data_sink.__isset.olap_table_sink = true; + + TOlapTableSink& tsink = data_sink.olap_table_sink; + tsink.load_id.hi = 123; + tsink.load_id.lo = 456; + tsink.txn_id = 789; + tsink.db_id = 1; + tsink.table_id = 2; + tsink.tuple_id = 0; + tsink.num_replicas = 3; + tsink.db_name = "testDb"; + tsink.table_name = "testTable"; + + // cosntruct schema + TOlapTableSchemaParam& tschema = tsink.schema; + tschema.db_id = 1; + tschema.table_id = 2; + tschema.version = 0; + + // descriptor + { + TDescriptorTableBuilder dtb; + { + TTupleDescriptorBuilder tuple_builder; + + tuple_builder.add_slot( + TSlotDescriptorBuilder().type(TYPE_INT).column_name("c1").column_pos(1).build()); + tuple_builder.add_slot( + TSlotDescriptorBuilder().decimal_type(5, 2).column_name("c2").column_pos(2).build()); + + tuple_builder.build(&dtb); + } + + *desc_tbl = dtb.desc_tbl(); + tschema.slot_descs = desc_tbl->slotDescriptors; + tschema.tuple_desc = desc_tbl->tupleDescriptors[0]; + } + // index + tschema.indexes.resize(1); + tschema.indexes[0].id = index1_id; + tschema.indexes[0].columns = {"c1", "c2"}; + // tschema.indexes[1].id = 5; + // tschema.indexes[1].columns = {"c1", "c3"}; + // partition + TOlapTablePartitionParam& tpartition = tsink.partition; + tpartition.db_id = db_id; + tpartition.table_id = table_id; + tpartition.version = table_id; + tpartition.__set_partition_column("c1"); + tpartition.__set_distributed_columns({"c2"}); + tpartition.partitions.resize(1); + tpartition.partitions[0].id = partition_id; + tpartition.partitions[0].num_buckets = 2; + tpartition.partitions[0].indexes.resize(1); + tpartition.partitions[0].indexes[0].index_id = index1_id; + tpartition.partitions[0].indexes[0].tablets = {tablet1_id, tablet2_id}; + // location + TOlapTableLocationParam& location = tsink.location; + location.db_id = db_id; + location.table_id = table_id; + location.version = 0; + location.tablets.resize(2); + location.tablets[0].tablet_id = tablet1_id; + location.tablets[0].node_ids = {0, 1, 2}; + location.tablets[1].tablet_id = tablet2_id; + location.tablets[1].node_ids = {0, 1, 2}; + // location + TPaloNodesInfo& nodes_info = tsink.nodes_info; + nodes_info.nodes.resize(3); + nodes_info.nodes[0].id = 0; + nodes_info.nodes[0].host = "127.0.0.1"; + nodes_info.nodes[0].async_internal_port = 4356; + nodes_info.nodes[1].id = 1; + nodes_info.nodes[1].host = "127.0.0.1"; + nodes_info.nodes[1].async_internal_port = 4356; + nodes_info.nodes[2].id = 2; + nodes_info.nodes[2].host = "127.0.0.1"; + nodes_info.nodes[2].async_internal_port = 4357; + + return data_sink; +} + +class TestInternalService : public PInternalService { +public: + TestInternalService() { } + virtual ~TestInternalService() { } + + void transmit_data(::google::protobuf::RpcController* controller, + const ::palo::PTransmitDataParams* request, + ::palo::PTransmitDataResult* response, + ::google::protobuf::Closure* done) override { + done->Run(); + } + + + void tablet_writer_open(google::protobuf::RpcController* controller, + const PTabletWriterOpenRequest* request, + PTabletWriterOpenResult* response, + google::protobuf::Closure* done) override { + Status status; + status.to_protobuf(response->mutable_status()); + done->Run(); + } + + void tablet_writer_add_batch(google::protobuf::RpcController* controller, + const PTabletWriterAddBatchRequest* request, + PTabletWriterAddBatchResult* response, + google::protobuf::Closure* done) override { + { + std::lock_guard l(_lock); + row_counters += request->tablet_ids_size(); + if (request->eos()) { + eof_counters++; + } + k_add_batch_status.to_protobuf(response->mutable_status()); + + if (request->has_row_batch() && _row_desc != nullptr) { + MemTracker tracker; + RowBatch batch(*_row_desc, request->row_batch(), &tracker); + for (int i = 0; i < batch.num_rows(); ++i){ + LOG(INFO) << print_row(batch.get_row(i), *_row_desc); + _output_set->emplace(print_row(batch.get_row(i), *_row_desc)); + } + } + } + done->Run(); + } + void tablet_writer_cancel(google::protobuf::RpcController* controller, + const PTabletWriterCancelRequest* request, + PTabletWriterCancelResult* response, + google::protobuf::Closure* done) override { + done->Run(); + } + + std::mutex _lock; + int64_t eof_counters = 0; + int64_t row_counters = 0; + RowDescriptor* _row_desc = nullptr; + std::set* _output_set; +}; + +TEST_F(OlapTableSinkTest, normal) { + // start brpc service first + auto server = new brpc::Server(); + auto service = new TestInternalService(); + server->AddService(service, brpc::SERVER_OWNS_SERVICE); + brpc::ServerOptions options; + server->Start(4356, &options); + + ExecEnv env; + TUniqueId fragment_id; + TQueryOptions query_options; + query_options.batch_size = 1; + RuntimeState state(fragment_id, query_options, "2018-05-25 12:14:15", &env); + state._instance_mem_tracker.reset(new MemTracker()); + + ObjectPool obj_pool; + TDescriptorTable tdesc_tbl; + auto t_data_sink = get_data_sink(&tdesc_tbl); + + // crate desc_tabl + DescriptorTbl* desc_tbl = nullptr; + auto st = DescriptorTbl::create(&obj_pool, tdesc_tbl, &desc_tbl); + ASSERT_TRUE(st.ok()); + state._desc_tbl = desc_tbl; + + TupleDescriptor* tuple_desc = desc_tbl->get_tuple_descriptor(0); + LOG(INFO) << "tuple_desc=" << tuple_desc->debug_string(); + + RowDescriptor row_desc(*desc_tbl, {0}, {false}); + + OlapTableSink sink(&obj_pool, row_desc, {}, &st); + ASSERT_TRUE(st.ok()); + + // init + st = sink.init(t_data_sink); + ASSERT_TRUE(st.ok()); + // prepare + st = sink.prepare(&state); + ASSERT_TRUE(st.ok()); + // open + st = sink.open(&state); + ASSERT_TRUE(st.ok()); + // send + MemTracker tracker; + RowBatch batch(row_desc, 1024, &tracker); + // 12, 9, "abc" + { + Tuple* tuple = (Tuple*)batch.tuple_data_pool()->allocate(tuple_desc->byte_size()); + batch.get_row(batch.add_row())->set_tuple(0, tuple); + memset(tuple, 0, tuple_desc->byte_size()); + + *reinterpret_cast(tuple->get_slot(4)) = 12; + *reinterpret_cast(tuple->get_slot(8)) = 9; + StringValue* str_val = reinterpret_cast(tuple->get_slot(16)); + str_val->ptr = (char*)batch.tuple_data_pool()->allocate(10); + str_val->len = 3; + memcpy(str_val->ptr, "abc", str_val->len); + batch.commit_last_row(); + } + // 13, 25, "abcd" + { + Tuple* tuple = (Tuple*)batch.tuple_data_pool()->allocate(tuple_desc->byte_size()); + batch.get_row(batch.add_row())->set_tuple(0, tuple); + memset(tuple, 0, tuple_desc->byte_size()); + + *reinterpret_cast(tuple->get_slot(4)) = 13; + *reinterpret_cast(tuple->get_slot(8)) = 25; + StringValue* str_val = reinterpret_cast(tuple->get_slot(16)); + str_val->ptr = (char*)batch.tuple_data_pool()->allocate(10); + str_val->len = 4; + memcpy(str_val->ptr, "abcd", str_val->len); + + batch.commit_last_row(); + } + // 14, 50, "abcde" + { + Tuple* tuple = (Tuple*)batch.tuple_data_pool()->allocate(tuple_desc->byte_size()); + batch.get_row(batch.add_row())->set_tuple(0, tuple); + memset(tuple, 0, tuple_desc->byte_size()); + + *reinterpret_cast(tuple->get_slot(4)) = 14; + *reinterpret_cast(tuple->get_slot(8)) = 50; + StringValue* str_val = reinterpret_cast(tuple->get_slot(16)); + str_val->ptr = reinterpret_cast(batch.tuple_data_pool()->allocate(10)); + str_val->len = 15; + memcpy(str_val->ptr, "abcde1234567890", str_val->len); + + batch.commit_last_row(); + } + st = sink.send(&state, &batch); + ASSERT_TRUE(st.ok()); + // close + st = sink.close(&state, Status::OK); + ASSERT_TRUE(st.ok()); + + // each node has a eof + ASSERT_EQ(2, service->eof_counters); + ASSERT_EQ(2 * 2, service->row_counters); + + // 2node * 2 + ASSERT_EQ(1, state.num_rows_load_filtered()); + + server->Stop(100); + server->Join(); + delete server; +} + +TEST_F(OlapTableSinkTest, convert) { + // start brpc service first + auto server = new brpc::Server(); + auto service = new TestInternalService(); + server->AddService(service, brpc::SERVER_OWNS_SERVICE); + brpc::ServerOptions options; + server->Start(4356, &options); + + ExecEnv env; + TUniqueId fragment_id; + TQueryOptions query_options; + query_options.batch_size = 1024; + RuntimeState state(fragment_id, query_options, "2018-05-25 12:14:15", &env); + state._instance_mem_tracker.reset(new MemTracker()); + + ObjectPool obj_pool; + TDescriptorTable tdesc_tbl; + auto t_data_sink = get_data_sink(&tdesc_tbl); + + // crate desc_tabl + DescriptorTbl* desc_tbl = nullptr; + auto st = DescriptorTbl::create(&obj_pool, tdesc_tbl, &desc_tbl); + ASSERT_TRUE(st.ok()); + state._desc_tbl = desc_tbl; + + TupleDescriptor* tuple_desc = desc_tbl->get_tuple_descriptor(0); + + RowDescriptor row_desc(*desc_tbl, {0}, {false}); + + // expr + std::vector exprs; + exprs.resize(3); + exprs[0].nodes.resize(1); + exprs[0].nodes[0].node_type = TExprNodeType::SLOT_REF; + exprs[0].nodes[0].type = tdesc_tbl.slotDescriptors[3].slotType; + exprs[0].nodes[0].num_children = 0; + exprs[0].nodes[0].__isset.slot_ref = true; + exprs[0].nodes[0].slot_ref.slot_id = 0; + exprs[0].nodes[0].slot_ref.tuple_id = 1; + + exprs[1].nodes.resize(1); + exprs[1].nodes[0].node_type = TExprNodeType::SLOT_REF; + exprs[1].nodes[0].type = tdesc_tbl.slotDescriptors[4].slotType; + exprs[1].nodes[0].num_children = 0; + exprs[1].nodes[0].__isset.slot_ref = true; + exprs[1].nodes[0].slot_ref.slot_id = 1; + exprs[1].nodes[0].slot_ref.tuple_id = 1; + + exprs[2].nodes.resize(1); + exprs[2].nodes[0].node_type = TExprNodeType::SLOT_REF; + exprs[2].nodes[0].type = tdesc_tbl.slotDescriptors[5].slotType; + exprs[2].nodes[0].num_children = 0; + exprs[2].nodes[0].__isset.slot_ref = true; + exprs[2].nodes[0].slot_ref.slot_id = 2; + exprs[2].nodes[0].slot_ref.tuple_id = 1; + + OlapTableSink sink(&obj_pool, row_desc, exprs, &st); + ASSERT_TRUE(st.ok()); + + // set output tuple_id + t_data_sink.olap_table_sink.tuple_id = 1; + // init + st = sink.init(t_data_sink); + ASSERT_TRUE(st.ok()); + // prepare + st = sink.prepare(&state); + ASSERT_TRUE(st.ok()); + // open + st = sink.open(&state); + ASSERT_TRUE(st.ok()); + // send + MemTracker tracker; + RowBatch batch(row_desc, 1024, &tracker); + // 12, 9, "abc" + { + Tuple* tuple = (Tuple*)batch.tuple_data_pool()->allocate(tuple_desc->byte_size()); + batch.get_row(batch.add_row())->set_tuple(0, tuple); + memset(tuple, 0, tuple_desc->byte_size()); + + *reinterpret_cast(tuple->get_slot(4)) = 12; + *reinterpret_cast(tuple->get_slot(8)) = 9; + StringValue* str_val = reinterpret_cast(tuple->get_slot(16)); + str_val->ptr = (char*)batch.tuple_data_pool()->allocate(10); + str_val->len = 3; + memcpy(str_val->ptr, "abc", str_val->len); + batch.commit_last_row(); + } + // 13, 25, "abcd" + { + Tuple* tuple = (Tuple*)batch.tuple_data_pool()->allocate(tuple_desc->byte_size()); + batch.get_row(batch.add_row())->set_tuple(0, tuple); + memset(tuple, 0, tuple_desc->byte_size()); + + *reinterpret_cast(tuple->get_slot(4)) = 13; + *reinterpret_cast(tuple->get_slot(8)) = 25; + StringValue* str_val = reinterpret_cast(tuple->get_slot(16)); + str_val->ptr = (char*)batch.tuple_data_pool()->allocate(10); + str_val->len = 4; + memcpy(str_val->ptr, "abcd", str_val->len); + + batch.commit_last_row(); + } + // 14, 50, "abcde" + { + Tuple* tuple = (Tuple*)batch.tuple_data_pool()->allocate(tuple_desc->byte_size()); + batch.get_row(batch.add_row())->set_tuple(0, tuple); + memset(tuple, 0, tuple_desc->byte_size()); + + *reinterpret_cast(tuple->get_slot(4)) = 14; + *reinterpret_cast(tuple->get_slot(8)) = 50; + StringValue* str_val = reinterpret_cast(tuple->get_slot(16)); + str_val->ptr = reinterpret_cast(batch.tuple_data_pool()->allocate(10)); + str_val->len = 5; + memcpy(str_val->ptr, "abcde", str_val->len); + + batch.commit_last_row(); + } + st = sink.send(&state, &batch); + ASSERT_TRUE(st.ok()); + // close + st = sink.close(&state, Status::OK); + ASSERT_TRUE(st.ok()); + + // each node has a eof + ASSERT_EQ(2, service->eof_counters); + ASSERT_EQ(2 * 3, service->row_counters); + + // 2node * 2 + ASSERT_EQ(0, state.num_rows_load_filtered()); + + server->Stop(100); + server->Join(); + delete server; +} + +TEST_F(OlapTableSinkTest, init_fail1) { + ExecEnv env; + TUniqueId fragment_id; + TQueryOptions query_options; + query_options.batch_size = 1; + RuntimeState state(fragment_id, query_options, "2018-05-25 12:14:15", &env); + state._instance_mem_tracker.reset(new MemTracker()); + + ObjectPool obj_pool; + TDescriptorTable tdesc_tbl; + auto t_data_sink = get_data_sink(&tdesc_tbl); + + // crate desc_tabl + DescriptorTbl* desc_tbl = nullptr; + auto st = DescriptorTbl::create(&obj_pool, tdesc_tbl, &desc_tbl); + ASSERT_TRUE(st.ok()); + state._desc_tbl = desc_tbl; + + RowDescriptor row_desc(*desc_tbl, {0}, {false}); + + // expr + std::vector exprs; + exprs.resize(1); + exprs[0].nodes.resize(1); + exprs[0].nodes[0].node_type = TExprNodeType::SLOT_REF; + exprs[0].nodes[0].type = tdesc_tbl.slotDescriptors[3].slotType; + exprs[0].nodes[0].num_children = 0; + exprs[0].nodes[0].__isset.slot_ref = true; + exprs[0].nodes[0].slot_ref.slot_id = 0; + exprs[0].nodes[0].slot_ref.tuple_id = 1; + + { + OlapTableSink sink(&obj_pool, row_desc, exprs, &st); + ASSERT_TRUE(st.ok()); + + // set output tuple_id + t_data_sink.olap_table_sink.tuple_id = 5; + // init + st = sink.init(t_data_sink); + ASSERT_TRUE(st.ok()); + st = sink.prepare(&state); + EXPECT_FALSE(st.ok()); + sink.close(&state, st); + } + { + OlapTableSink sink(&obj_pool, row_desc, exprs, &st); + ASSERT_TRUE(st.ok()); + + // set output tuple_id + t_data_sink.olap_table_sink.tuple_id = 1; + // init + st = sink.init(t_data_sink); + ASSERT_TRUE(st.ok()); + st = sink.prepare(&state); + EXPECT_FALSE(st.ok()); + sink.close(&state, st); + } +} + +TEST_F(OlapTableSinkTest, init_fail3) { + ExecEnv env; + TUniqueId fragment_id; + TQueryOptions query_options; + query_options.batch_size = 1; + RuntimeState state(fragment_id, query_options, "2018-05-25 12:14:15", &env); + state._instance_mem_tracker.reset(new MemTracker()); + + ObjectPool obj_pool; + TDescriptorTable tdesc_tbl; + auto t_data_sink = get_data_sink(&tdesc_tbl); + + // crate desc_tabl + DescriptorTbl* desc_tbl = nullptr; + auto st = DescriptorTbl::create(&obj_pool, tdesc_tbl, &desc_tbl); + ASSERT_TRUE(st.ok()); + state._desc_tbl = desc_tbl; + + RowDescriptor row_desc(*desc_tbl, {0}, {false}); + + // expr + std::vector exprs; + exprs.resize(3); + exprs[0].nodes.resize(1); + exprs[0].nodes[0].node_type = TExprNodeType::SLOT_REF; + exprs[0].nodes[0].type = tdesc_tbl.slotDescriptors[3].slotType; + exprs[0].nodes[0].num_children = 0; + exprs[0].nodes[0].__isset.slot_ref = true; + exprs[0].nodes[0].slot_ref.slot_id = 0; + exprs[0].nodes[0].slot_ref.tuple_id = 1; + + exprs[1].nodes.resize(1); + exprs[1].nodes[0].node_type = TExprNodeType::SLOT_REF; + exprs[1].nodes[0].type = tdesc_tbl.slotDescriptors[3].slotType; + exprs[1].nodes[0].num_children = 0; + exprs[1].nodes[0].__isset.slot_ref = true; + exprs[1].nodes[0].slot_ref.slot_id = 1; + exprs[1].nodes[0].slot_ref.tuple_id = 1; + + exprs[2].nodes.resize(1); + exprs[2].nodes[0].node_type = TExprNodeType::SLOT_REF; + exprs[2].nodes[0].type = tdesc_tbl.slotDescriptors[5].slotType; + exprs[2].nodes[0].num_children = 0; + exprs[2].nodes[0].__isset.slot_ref = true; + exprs[2].nodes[0].slot_ref.slot_id = 2; + exprs[2].nodes[0].slot_ref.tuple_id = 1; + + OlapTableSink sink(&obj_pool, row_desc, exprs, &st); + ASSERT_TRUE(st.ok()); + + // set output tuple_id + t_data_sink.olap_table_sink.tuple_id = 1; + // init + st = sink.init(t_data_sink); + ASSERT_TRUE(st.ok()); + st = sink.prepare(&state); + EXPECT_FALSE(st.ok()); + sink.close(&state, st); +} + +TEST_F(OlapTableSinkTest, init_fail4) { + ExecEnv env; + TUniqueId fragment_id; + TQueryOptions query_options; + query_options.batch_size = 1; + RuntimeState state(fragment_id, query_options, "2018-05-25 12:14:15", &env); + state._instance_mem_tracker.reset(new MemTracker()); + + ObjectPool obj_pool; + TDescriptorTable tdesc_tbl; + auto t_data_sink = get_data_sink(&tdesc_tbl); + + // crate desc_tabl + DescriptorTbl* desc_tbl = nullptr; + auto st = DescriptorTbl::create(&obj_pool, tdesc_tbl, &desc_tbl); + ASSERT_TRUE(st.ok()); + state._desc_tbl = desc_tbl; + + RowDescriptor row_desc(*desc_tbl, {0}, {false}); + + // expr + std::vector exprs; + exprs.resize(3); + exprs[0].nodes.resize(1); + exprs[0].nodes[0].node_type = TExprNodeType::SLOT_REF; + exprs[0].nodes[0].type = tdesc_tbl.slotDescriptors[3].slotType; + exprs[0].nodes[0].num_children = 0; + exprs[0].nodes[0].__isset.slot_ref = true; + exprs[0].nodes[0].slot_ref.slot_id = 0; + exprs[0].nodes[0].slot_ref.tuple_id = 1; + + exprs[1].nodes.resize(1); + exprs[1].nodes[0].node_type = TExprNodeType::SLOT_REF; + exprs[1].nodes[0].type = tdesc_tbl.slotDescriptors[4].slotType; + exprs[1].nodes[0].num_children = 0; + exprs[1].nodes[0].__isset.slot_ref = true; + exprs[1].nodes[0].slot_ref.slot_id = 1; + exprs[1].nodes[0].slot_ref.tuple_id = 1; + + exprs[2].nodes.resize(1); + exprs[2].nodes[0].node_type = TExprNodeType::SLOT_REF; + exprs[2].nodes[0].type = tdesc_tbl.slotDescriptors[5].slotType; + exprs[2].nodes[0].num_children = 0; + exprs[2].nodes[0].__isset.slot_ref = true; + exprs[2].nodes[0].slot_ref.slot_id = 2; + exprs[2].nodes[0].slot_ref.tuple_id = 1; + + OlapTableSink sink(&obj_pool, row_desc, exprs, &st); + ASSERT_TRUE(st.ok()); + + // set output tuple_id + t_data_sink.olap_table_sink.tuple_id = 1; + // init + t_data_sink.olap_table_sink.partition.partitions[0].indexes[0].tablets = {101, 102}; + st = sink.init(t_data_sink); + ASSERT_TRUE(st.ok()); + st = sink.prepare(&state); + EXPECT_FALSE(st.ok()); + sink.close(&state, st); +} + +TEST_F(OlapTableSinkTest, add_batch_failed) { + // start brpc service first + auto server = new brpc::Server(); + auto service = new TestInternalService(); + server->AddService(service, brpc::SERVER_OWNS_SERVICE); + brpc::ServerOptions options; + server->Start(4356, &options); + + ExecEnv env; + TUniqueId fragment_id; + TQueryOptions query_options; + query_options.batch_size = 1; + RuntimeState state(fragment_id, query_options, "2018-05-25 12:14:15", &env); + state._instance_mem_tracker.reset(new MemTracker()); + + ObjectPool obj_pool; + TDescriptorTable tdesc_tbl; + auto t_data_sink = get_data_sink(&tdesc_tbl); + + // crate desc_tabl + DescriptorTbl* desc_tbl = nullptr; + auto st = DescriptorTbl::create(&obj_pool, tdesc_tbl, &desc_tbl); + ASSERT_TRUE(st.ok()); + state._desc_tbl = desc_tbl; + + RowDescriptor row_desc(*desc_tbl, {0}, {false}); + + // expr + std::vector exprs; + exprs.resize(3); + exprs[0].nodes.resize(1); + exprs[0].nodes[0].node_type = TExprNodeType::SLOT_REF; + exprs[0].nodes[0].type = tdesc_tbl.slotDescriptors[3].slotType; + exprs[0].nodes[0].num_children = 0; + exprs[0].nodes[0].__isset.slot_ref = true; + exprs[0].nodes[0].slot_ref.slot_id = 0; + exprs[0].nodes[0].slot_ref.tuple_id = 1; + + exprs[1].nodes.resize(1); + exprs[1].nodes[0].node_type = TExprNodeType::SLOT_REF; + exprs[1].nodes[0].type = tdesc_tbl.slotDescriptors[4].slotType; + exprs[1].nodes[0].num_children = 0; + exprs[1].nodes[0].__isset.slot_ref = true; + exprs[1].nodes[0].slot_ref.slot_id = 1; + exprs[1].nodes[0].slot_ref.tuple_id = 1; + + exprs[2].nodes.resize(1); + exprs[2].nodes[0].node_type = TExprNodeType::SLOT_REF; + exprs[2].nodes[0].type = tdesc_tbl.slotDescriptors[5].slotType; + exprs[2].nodes[0].num_children = 0; + exprs[2].nodes[0].__isset.slot_ref = true; + exprs[2].nodes[0].slot_ref.slot_id = 2; + exprs[2].nodes[0].slot_ref.tuple_id = 1; + + OlapTableSink sink(&obj_pool, row_desc, exprs, &st); + ASSERT_TRUE(st.ok()); + + // set output tuple_id + t_data_sink.olap_table_sink.tuple_id = 1; + // init + st = sink.init(t_data_sink); + ASSERT_TRUE(st.ok()); + st = sink.prepare(&state); + ASSERT_TRUE(st.ok()); + st = sink.open(&state); + ASSERT_TRUE(st.ok()); + // send + MemTracker tracker; + RowBatch batch(row_desc, 1024, &tracker); + TupleDescriptor* tuple_desc = desc_tbl->get_tuple_descriptor(0); + // 12, 9, "abc" + { + Tuple* tuple = (Tuple*)batch.tuple_data_pool()->allocate(tuple_desc->byte_size()); + batch.get_row(batch.add_row())->set_tuple(0, tuple); + memset(tuple, 0, tuple_desc->byte_size()); + + *reinterpret_cast(tuple->get_slot(4)) = 12; + *reinterpret_cast(tuple->get_slot(8)) = 9; + StringValue* str_val = reinterpret_cast(tuple->get_slot(16)); + str_val->ptr = (char*)batch.tuple_data_pool()->allocate(10); + str_val->len = 3; + memcpy(str_val->ptr, "abc", str_val->len); + batch.commit_last_row(); + } + k_add_batch_status = Status("dummy failed"); + st = sink.send(&state, &batch); + ASSERT_TRUE(st.ok()); + // close + st = sink.close(&state, Status::OK); + ASSERT_FALSE(st.ok()); + + server->Stop(100); + server->Join(); + delete server; +} + +TEST_F(OlapTableSinkTest, decimal) { + // start brpc service first + auto server = new brpc::Server(); + auto service = new TestInternalService(); + server->AddService(service, brpc::SERVER_OWNS_SERVICE); + brpc::ServerOptions options; + server->Start(4356, &options); + + ExecEnv env; + TUniqueId fragment_id; + TQueryOptions query_options; + query_options.batch_size = 1; + RuntimeState state(fragment_id, query_options, "2018-05-25 12:14:15", &env); + state._instance_mem_tracker.reset(new MemTracker()); + + ObjectPool obj_pool; + TDescriptorTable tdesc_tbl; + auto t_data_sink = get_decimal_sink(&tdesc_tbl); + + // crate desc_tabl + DescriptorTbl* desc_tbl = nullptr; + auto st = DescriptorTbl::create(&obj_pool, tdesc_tbl, &desc_tbl); + ASSERT_TRUE(st.ok()); + state._desc_tbl = desc_tbl; + + TupleDescriptor* tuple_desc = desc_tbl->get_tuple_descriptor(0); + LOG(INFO) << "tuple_desc=" << tuple_desc->debug_string(); + + RowDescriptor row_desc(*desc_tbl, {0}, {false}); + service->_row_desc = &row_desc; + std::set output_set; + service->_output_set = &output_set; + + OlapTableSink sink(&obj_pool, row_desc, {}, &st); + ASSERT_TRUE(st.ok()); + + // init + st = sink.init(t_data_sink); + ASSERT_TRUE(st.ok()); + // prepare + st = sink.prepare(&state); + ASSERT_TRUE(st.ok()); + // open + st = sink.open(&state); + ASSERT_TRUE(st.ok()); + // send + MemTracker tracker; + RowBatch batch(row_desc, 1024, &tracker); + // 12, 12.3 + { + Tuple* tuple = (Tuple*)batch.tuple_data_pool()->allocate(tuple_desc->byte_size()); + batch.get_row(batch.add_row())->set_tuple(0, tuple); + memset(tuple, 0, tuple_desc->byte_size()); + + *reinterpret_cast(tuple->get_slot(4)) = 12; + DecimalValue* dec_val = reinterpret_cast(tuple->get_slot(16)); + *dec_val = DecimalValue("12.3"); + batch.commit_last_row(); + } + // 13, 123.123456789 + { + Tuple* tuple = (Tuple*)batch.tuple_data_pool()->allocate(tuple_desc->byte_size()); + batch.get_row(batch.add_row())->set_tuple(0, tuple); + memset(tuple, 0, tuple_desc->byte_size()); + + *reinterpret_cast(tuple->get_slot(4)) = 13; + DecimalValue* dec_val = reinterpret_cast(tuple->get_slot(16)); + *dec_val = DecimalValue("123.123456789"); + + batch.commit_last_row(); + } + // 14, 123456789123.1234 + { + Tuple* tuple = (Tuple*)batch.tuple_data_pool()->allocate(tuple_desc->byte_size()); + batch.get_row(batch.add_row())->set_tuple(0, tuple); + memset(tuple, 0, tuple_desc->byte_size()); + + *reinterpret_cast(tuple->get_slot(4)) = 14; + DecimalValue* dec_val = reinterpret_cast(tuple->get_slot(16)); + *dec_val = DecimalValue("123456789123.1234"); + + batch.commit_last_row(); + } + st = sink.send(&state, &batch); + ASSERT_TRUE(st.ok()); + // close + st = sink.close(&state, Status::OK); + ASSERT_TRUE(st.ok()); + + ASSERT_EQ(2, output_set.size()); + ASSERT_TRUE(output_set.count("[(12 12.3)]") > 0); + ASSERT_TRUE(output_set.count("[(13 123.12)]") > 0); + // ASSERT_TRUE(output_set.count("[(14 999.99)]") > 0); + + server->Stop(100); + server->Join(); + delete server; +} + +} +} + +int main(int argc, char* argv[]) { + palo::CpuInfo::init(); + ::testing::InitGoogleTest(&argc, argv); + return RUN_ALL_TESTS(); +} diff --git a/be/test/http/http_utils_test.cpp b/be/test/http/http_utils_test.cpp new file mode 100644 index 0000000000..3fdcc6b436 --- /dev/null +++ b/be/test/http/http_utils_test.cpp @@ -0,0 +1,91 @@ +// Copyright (c) 2018, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 "http/utils.h" + +#include + +#include "common/logging.h" +#include "http/http_headers.h" +#include "http/http_request.h" +#include "util/url_coding.h" + +namespace palo { + +class HttpUtilsTest : public testing::Test { +public: + HttpUtilsTest() { } + virtual ~HttpUtilsTest() { + } +}; + +TEST_F(HttpUtilsTest, parse_basic_auth) { + { + HttpRequest req; + std::string auth = "Basic "; + std::string encoded_str; + base64_encode("palo:passwd", &encoded_str); + auth += encoded_str; + req._headers.emplace(HttpHeaders::AUTHORIZATION, auth); + std::string user; + std::string passwd; + auto res = parse_basic_auth(req, &user, &passwd); + ASSERT_TRUE(res); + ASSERT_STREQ("palo", user.data()); + ASSERT_STREQ("passwd", passwd.data()); + } + { + HttpRequest req; + std::string auth = "Basic "; + std::string encoded_str = "palo:passwd"; + auth += encoded_str; + req._headers.emplace(HttpHeaders::AUTHORIZATION, auth); + std::string user; + std::string passwd; + auto res = parse_basic_auth(req, &user, &passwd); + ASSERT_FALSE(res); + } + { + HttpRequest req; + std::string auth = "Basic "; + std::string encoded_str; + base64_encode("palopasswd", &encoded_str); + auth += encoded_str; + req._headers.emplace(HttpHeaders::AUTHORIZATION, auth); + std::string user; + std::string passwd; + auto res = parse_basic_auth(req, &user, &passwd); + ASSERT_FALSE(res); + } + { + HttpRequest req; + std::string auth = "Basic"; + std::string encoded_str; + base64_encode("palo:passwd", &encoded_str); + auth += encoded_str; + req._headers.emplace(HttpHeaders::AUTHORIZATION, auth); + std::string user; + std::string passwd; + auto res = parse_basic_auth(req, &user, &passwd); + ASSERT_FALSE(res); + } +} + +} + +int main(int argc, char** argv) { + ::testing::InitGoogleTest(&argc, argv); + return RUN_ALL_TESTS(); +} diff --git a/be/test/http/message_body_sink_test.cpp b/be/test/http/message_body_sink_test.cpp new file mode 100644 index 0000000000..9e33db9d17 --- /dev/null +++ b/be/test/http/message_body_sink_test.cpp @@ -0,0 +1,61 @@ +// Copyright (c) 2017, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 "http/message_body_sink.h" + +#include + +#include +#include +#include +#include + +namespace palo { + +class MessageBodySinkTest : public testing::Test { +public: + MessageBodySinkTest() { } + virtual ~MessageBodySinkTest() { } + + void SetUp() override { + } +private: +}; + +TEST_F(MessageBodySinkTest, file_sink) { + char data[] = "hello world"; + + MessageBodyFileSink sink("./body_sink_test_file_sink"); + ASSERT_TRUE(sink.open().ok()); + ASSERT_TRUE(sink.append(data, sizeof(data)).ok()); + ASSERT_TRUE(sink.finish().ok()); + + { + char buf[256]; + memset(buf, 0, 256); + int fd = open("././body_sink_test_file_sink", O_RDONLY); + read(fd, buf, 256); + close(fd); + ASSERT_STREQ("hello world", buf); + unlink("././body_sink_test_file_sink"); + } +} + +} + +int main(int argc, char* argv[]) { + ::testing::InitGoogleTest(&argc, argv); + return RUN_ALL_TESTS(); +} diff --git a/be/test/http/stream_load_test.cpp b/be/test/http/stream_load_test.cpp new file mode 100644 index 0000000000..ca7dbb9afb --- /dev/null +++ b/be/test/http/stream_load_test.cpp @@ -0,0 +1,231 @@ +// Copyright (c) 2018, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 "http/action/stream_load.h" + +#include +#include + +#include "exec/schema_scanner/frontend_helper.h" +#include "http/http_channel.h" +#include "http/http_request.h" +#include "runtime/exec_env.h" +#include "util/palo_metrics.h" + +class mg_connection; + +namespace palo { + +std::string k_response_str; + +// Send Unauthorized status with basic challenge +void HttpChannel::send_basic_challenge(HttpRequest* req, const std::string& realm) { +} + +void HttpChannel::send_error(HttpRequest* request, HttpStatus status) { +} + +void HttpChannel::send_reply(HttpRequest* request, HttpStatus status) { +} + +void HttpChannel::send_reply( + HttpRequest* request, HttpStatus status, const std::string& content) { + k_response_str = content; +} + +void HttpChannel::send_file(HttpRequest* request, int fd, size_t off, size_t size) { +} + +extern TLoadTxnBeginResult k_stream_load_begin_result; +extern TLoadTxnCommitResult k_stream_load_commit_result; +extern TLoadTxnRollbackResult k_stream_load_rollback_result; +extern TStreamLoadPutResult k_stream_load_put_result; +extern Status k_stream_load_plan_status; + +class StreamLoadActionTest : public testing::Test { +public: + StreamLoadActionTest() { } + virtual ~StreamLoadActionTest() { } + void SetUp() override { + k_stream_load_begin_result = TLoadTxnBeginResult(); + k_stream_load_commit_result = TLoadTxnCommitResult(); + k_stream_load_rollback_result = TLoadTxnRollbackResult(); + k_stream_load_put_result = TStreamLoadPutResult(); + k_stream_load_plan_status = Status::OK; + k_response_str = ""; + config::streaming_load_max_mb = 1; + } +private: +}; + +TEST_F(StreamLoadActionTest, no_auth) { + PaloMetrics::instance()->initialize("StreamLoadActionTest"); + ExecEnv env; + StreamLoadAction action(&env); + + HttpRequest request; + action.on_header(&request); + action.handle(&request); + + rapidjson::Document doc; + doc.Parse(k_response_str.c_str()); + ASSERT_STREQ("Fail", doc["Status"].GetString()); +} + +#if 0 +TEST_F(StreamLoadActionTest, no_content_length) { + PaloMetrics::instance()->initialize("StreamLoadActionTest"); + ExecEnv env; + StreamLoadAction action(&env); + + HttpRequest request; + request._headers.emplace(HttpHeaders::AUTHORIZATION, "Basic cm9vdDo="); + action.on_header(&request); + action.handle(&request); + + rapidjson::Document doc; + doc.Parse(k_response_str.c_str()); + ASSERT_STREQ("Fail", doc["Status"].GetString()); +} + +TEST_F(StreamLoadActionTest, unknown_encoding) { + PaloMetrics::instance()->initialize("StreamLoadActionTest"); + ExecEnv env; + StreamLoadAction action(&env); + + HttpRequest request; + request._headers.emplace(HttpHeaders::AUTHORIZATION, "Basic cm9vdDo="); + request._headers.emplace(HttpHeaders::TRANSFER_ENCODING, "chunked111"); + action.on_header(&request); + action.handle(&request); + + rapidjson::Document doc; + doc.Parse(k_response_str.c_str()); + ASSERT_STREQ("Fail", doc["Status"].GetString()); +} +#endif + +TEST_F(StreamLoadActionTest, normal) { + PaloMetrics::instance()->initialize("StreamLoadActionTest"); + ExecEnv env; + StreamLoadAction action(&env); + + HttpRequest request; + request._headers.emplace(HttpHeaders::AUTHORIZATION, "Basic cm9vdDo="); + request._headers.emplace(HttpHeaders::CONTENT_LENGTH, "0"); + action.on_header(&request); + action.handle(&request); + + rapidjson::Document doc; + doc.Parse(k_response_str.c_str()); + ASSERT_STREQ("Success", doc["Status"].GetString()); +} + +TEST_F(StreamLoadActionTest, put_fail) { + PaloMetrics::instance()->initialize("StreamLoadActionTest"); + ExecEnv env; + StreamLoadAction action(&env); + + HttpRequest request; + request._headers.emplace(HttpHeaders::AUTHORIZATION, "Basic cm9vdDo="); + request._headers.emplace(HttpHeaders::CONTENT_LENGTH, "16"); + Status status("TestFail"); + status.to_thrift(&k_stream_load_put_result.status); + action.on_header(&request); + action.handle(&request); + + rapidjson::Document doc; + doc.Parse(k_response_str.c_str()); + ASSERT_STREQ("Fail", doc["Status"].GetString()); +} + +TEST_F(StreamLoadActionTest, commit_fail) { + PaloMetrics::instance()->initialize("StreamLoadActionTest"); + ExecEnv env; + StreamLoadAction action(&env); + + HttpRequest request; + request._headers.emplace(HttpHeaders::AUTHORIZATION, "Basic cm9vdDo="); + request._headers.emplace(HttpHeaders::CONTENT_LENGTH, "16"); + Status status("TestFail"); + status.to_thrift(&k_stream_load_commit_result.status); + action.on_header(&request); + action.handle(&request); + + rapidjson::Document doc; + doc.Parse(k_response_str.c_str()); + ASSERT_STREQ("Fail", doc["Status"].GetString()); +} + +TEST_F(StreamLoadActionTest, begin_fail) { + PaloMetrics::instance()->initialize("StreamLoadActionTest"); + ExecEnv env; + StreamLoadAction action(&env); + + HttpRequest request; + request._headers.emplace(HttpHeaders::AUTHORIZATION, "Basic cm9vdDo="); + request._headers.emplace(HttpHeaders::CONTENT_LENGTH, "16"); + Status status("TestFail"); + status.to_thrift(&k_stream_load_begin_result.status); + action.on_header(&request); + action.handle(&request); + + rapidjson::Document doc; + doc.Parse(k_response_str.c_str()); + ASSERT_STREQ("Fail", doc["Status"].GetString()); +} + +#if 0 +TEST_F(StreamLoadActionTest, receive_failed) { + PaloMetrics::instance()->initialize("StreamLoadActionTest"); + ExecEnv env; + StreamLoadAction action(&env); + + HttpRequest request; + request._headers.emplace(HttpHeaders::AUTHORIZATION, "Basic cm9vdDo="); + request._headers.emplace(HttpHeaders::TRANSFER_ENCODING, "chunked"); + action.on_header(&request); + action.handle(&request); + + rapidjson::Document doc; + doc.Parse(k_response_str.c_str()); + ASSERT_STREQ("Fail", doc["Status"].GetString()); +} +#endif + +TEST_F(StreamLoadActionTest, plan_fail) { + PaloMetrics::instance()->initialize("StreamLoadActionTest"); + ExecEnv env; + StreamLoadAction action(&env); + + HttpRequest request; + request._headers.emplace(HttpHeaders::AUTHORIZATION, "Basic cm9vdDo="); + request._headers.emplace(HttpHeaders::CONTENT_LENGTH, "16"); + k_stream_load_plan_status = Status("TestFail"); + action.on_header(&request); + action.handle(&request); + + rapidjson::Document doc; + doc.Parse(k_response_str.c_str()); + ASSERT_STREQ("Fail", doc["Status"].GetString()); +} + +} + +int main(int argc, char* argv[]) { + ::testing::InitGoogleTest(&argc, argv); + palo::CpuInfo::init(); + return RUN_ALL_TESTS(); +} diff --git a/be/test/olap/delta_writer_test.cpp b/be/test/olap/delta_writer_test.cpp new file mode 100644 index 0000000000..c0db7f0ab9 --- /dev/null +++ b/be/test/olap/delta_writer_test.cpp @@ -0,0 +1,666 @@ +// Copyright (c) 2018, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 "olap/delta_writer.h" + +#include +#include +#include + +#include "gen_cpp/Descriptors_types.h" +#include "gen_cpp/PaloInternalService_types.h" +#include "gen_cpp/Types_types.h" +#include "olap/field.h" +#include "olap/olap_engine.h" +#include "olap/olap_table.h" +#include "olap/utils.h" +#include "runtime/tuple.h" +#include "util/descriptor_helper.h" +#include "util/logging.h" +#include "olap/options.h" +#include "olap/olap_header_manager.h" + +namespace palo { + +// This is DeltaWriter unit test which used by streaming load. +// And also it should take schema change into account after streaming load. + +static const uint32_t MAX_RETRY_TIMES = 10; +static const uint32_t MAX_PATH_LEN = 1024; + +OLAPEngine* k_engine = nullptr; + +void set_up() { + char buffer[MAX_PATH_LEN]; + getcwd(buffer, MAX_PATH_LEN); + config::storage_root_path = std::string(buffer) + "/data_test"; + remove_all_dir(config::storage_root_path); + create_dir(config::storage_root_path); + std::vector paths; + paths.emplace_back(config::storage_root_path, -1); + + palo::EngineOptions options; + options.store_paths = paths; + palo::OLAPEngine::open(options, &k_engine); +} + +void tear_down() { + system("rm -rf ./data_test"); + remove_all_dir(std::string(getenv("DORIS_HOME")) + UNUSED_PREFIX); +} + +void create_table_request(TCreateTabletReq* request) { + request->tablet_id = 10003; + request->__set_version(1); + request->__set_version_hash(0); + request->tablet_schema.schema_hash = 270068375; + request->tablet_schema.short_key_column_count = 6; + request->tablet_schema.keys_type = TKeysType::AGG_KEYS; + request->tablet_schema.storage_type = TStorageType::COLUMN; + + TColumn k1; + k1.column_name = "k1"; + k1.__set_is_key(true); + k1.column_type.type = TPrimitiveType::TINYINT; + request->tablet_schema.columns.push_back(k1); + + TColumn k2; + k2.column_name = "k2"; + k2.__set_is_key(true); + k2.column_type.type = TPrimitiveType::SMALLINT; + request->tablet_schema.columns.push_back(k2); + + TColumn k3; + k3.column_name = "k3"; + k3.__set_is_key(true); + k3.column_type.type = TPrimitiveType::INT; + request->tablet_schema.columns.push_back(k3); + + TColumn k4; + k4.column_name = "k4"; + k4.__set_is_key(true); + k4.column_type.type = TPrimitiveType::BIGINT; + request->tablet_schema.columns.push_back(k4); + + TColumn k5; + k5.column_name = "k5"; + k5.__set_is_key(true); + k5.column_type.type = TPrimitiveType::LARGEINT; + request->tablet_schema.columns.push_back(k5); + + TColumn k6; + k6.column_name = "k6"; + k6.__set_is_key(true); + k6.column_type.type = TPrimitiveType::DATE; + request->tablet_schema.columns.push_back(k6); + + TColumn k7; + k7.column_name = "k7"; + k7.__set_is_key(true); + k7.column_type.type = TPrimitiveType::DATETIME; + request->tablet_schema.columns.push_back(k7); + + TColumn k8; + k8.column_name = "k8"; + k8.__set_is_key(true); + k8.column_type.type = TPrimitiveType::CHAR; + k8.column_type.__set_len(4); + request->tablet_schema.columns.push_back(k8); + + TColumn k9; + k9.column_name = "k9"; + k9.__set_is_key(true); + k9.column_type.type = TPrimitiveType::VARCHAR; + k9.column_type.__set_len(65); + request->tablet_schema.columns.push_back(k9); + + TColumn k10; + k10.column_name = "k10"; + k10.__set_is_key(true); + k10.column_type.type = TPrimitiveType::DECIMAL; + k10.column_type.__set_precision(6); + k10.column_type.__set_scale(3); + request->tablet_schema.columns.push_back(k10); + + TColumn v1; + v1.column_name = "v1"; + v1.__set_is_key(false); + v1.column_type.type = TPrimitiveType::TINYINT; + v1.__set_aggregation_type(TAggregationType::SUM); + request->tablet_schema.columns.push_back(v1); + + TColumn v2; + v2.column_name = "v2"; + v2.__set_is_key(false); + v2.column_type.type = TPrimitiveType::SMALLINT; + v2.__set_aggregation_type(TAggregationType::SUM); + request->tablet_schema.columns.push_back(v2); + + TColumn v3; + v3.column_name = "v3"; + v3.__set_is_key(false); + v3.column_type.type = TPrimitiveType::INT; + v3.__set_aggregation_type(TAggregationType::SUM); + request->tablet_schema.columns.push_back(v3); + + TColumn v4; + v4.column_name = "v4"; + v4.__set_is_key(false); + v4.column_type.type = TPrimitiveType::BIGINT; + v4.__set_aggregation_type(TAggregationType::SUM); + request->tablet_schema.columns.push_back(v4); + + TColumn v5; + v5.column_name = "v5"; + v5.__set_is_key(false); + v5.column_type.type = TPrimitiveType::LARGEINT; + v5.__set_aggregation_type(TAggregationType::SUM); + request->tablet_schema.columns.push_back(v5); + + TColumn v6; + v6.column_name = "v6"; + v6.__set_is_key(false); + v6.column_type.type = TPrimitiveType::DATE; + v6.__set_aggregation_type(TAggregationType::REPLACE); + request->tablet_schema.columns.push_back(v6); + + TColumn v7; + v7.column_name = "v7"; + v7.__set_is_key(false); + v7.column_type.type = TPrimitiveType::DATETIME; + v7.__set_aggregation_type(TAggregationType::REPLACE); + request->tablet_schema.columns.push_back(v7); + + TColumn v8; + v8.column_name = "v8"; + v8.__set_is_key(false); + v8.column_type.type = TPrimitiveType::CHAR; + v8.column_type.__set_len(4); + v8.__set_aggregation_type(TAggregationType::REPLACE); + request->tablet_schema.columns.push_back(v8); + + TColumn v9; + v9.column_name = "v9"; + v9.__set_is_key(false); + v9.column_type.type = TPrimitiveType::VARCHAR; + v9.column_type.__set_len(65); + v9.__set_aggregation_type(TAggregationType::REPLACE); + request->tablet_schema.columns.push_back(v9); + + TColumn v10; + v10.column_name = "v10"; + v10.__set_is_key(false); + v10.column_type.type = TPrimitiveType::DECIMAL; + v10.column_type.__set_precision(6); + v10.column_type.__set_scale(3); + v10.__set_aggregation_type(TAggregationType::SUM); + request->tablet_schema.columns.push_back(v10); +} + +TDescriptorTable create_descriptor_table() { + TDescriptorTableBuilder dtb; + TTupleDescriptorBuilder tuple_builder; + + tuple_builder.add_slot( + TSlotDescriptorBuilder().type(TYPE_TINYINT).column_name("k1").column_pos(0).build()); + tuple_builder.add_slot( + TSlotDescriptorBuilder().type(TYPE_SMALLINT).column_name("k2").column_pos(1).build()); + tuple_builder.add_slot( + TSlotDescriptorBuilder().type(TYPE_INT).column_name("k3").column_pos(2).build()); + tuple_builder.add_slot( + TSlotDescriptorBuilder().type(TYPE_BIGINT).column_name("k4").column_pos(3).build()); + tuple_builder.add_slot( + TSlotDescriptorBuilder().type(TYPE_LARGEINT).column_name("k5").column_pos(4).build()); + tuple_builder.add_slot( + TSlotDescriptorBuilder().type(TYPE_DATE).column_name("k6").column_pos(5).build()); + tuple_builder.add_slot( + TSlotDescriptorBuilder().type(TYPE_DATETIME).column_name("k7").column_pos(6).build()); + tuple_builder.add_slot( + TSlotDescriptorBuilder().string_type(4).column_name("k8").column_pos(7).build()); + tuple_builder.add_slot( + TSlotDescriptorBuilder().string_type(65).column_name("k9").column_pos(8).build()); + tuple_builder.add_slot( + TSlotDescriptorBuilder().decimal_type(6, 3).column_name("k10").column_pos(9).build()); + + tuple_builder.add_slot( + TSlotDescriptorBuilder().type(TYPE_TINYINT).column_name("v1").column_pos(10).build()); + tuple_builder.add_slot( + TSlotDescriptorBuilder().type(TYPE_SMALLINT).column_name("v2").column_pos(11).build()); + tuple_builder.add_slot( + TSlotDescriptorBuilder().type(TYPE_INT).column_name("v3").column_pos(12).build()); + tuple_builder.add_slot( + TSlotDescriptorBuilder().type(TYPE_BIGINT).column_name("v4").column_pos(13).build()); + tuple_builder.add_slot( + TSlotDescriptorBuilder().type(TYPE_LARGEINT).column_name("v5").column_pos(14).build()); + tuple_builder.add_slot( + TSlotDescriptorBuilder().type(TYPE_DATE).column_name("v6").column_pos(15).build()); + tuple_builder.add_slot( + TSlotDescriptorBuilder().type(TYPE_DATETIME).column_name("v7").column_pos(16).build()); + tuple_builder.add_slot( + TSlotDescriptorBuilder().string_type(4).column_name("v8").column_pos(17).build()); + tuple_builder.add_slot( + TSlotDescriptorBuilder().string_type(65).column_name("v9").column_pos(18).build()); + tuple_builder.add_slot( + TSlotDescriptorBuilder().decimal_type(6, 3).column_name("v10").column_pos(19).build()); + tuple_builder.build(&dtb); + + return dtb.desc_tbl(); +} + +class TestDeltaWriter : public ::testing::Test { +public: + TestDeltaWriter() { } + ~TestDeltaWriter() { } + + void SetUp() { + // Create local data dir for OLAPEngine. + char buffer[MAX_PATH_LEN]; + getcwd(buffer, MAX_PATH_LEN); + config::storage_root_path = std::string(buffer) + "/data_push"; + remove_all_dir(config::storage_root_path); + ASSERT_EQ(create_dir(config::storage_root_path), OLAP_SUCCESS); + + // Initialize all singleton object. + // OLAPRootPath::get_instance()->reload_root_paths(config::storage_root_path.c_str()); + } + + void TearDown(){ + // Remove all dir. + ASSERT_EQ(OLAP_SUCCESS, remove_all_dir(config::storage_root_path)); + } +}; + +TEST_F(TestDeltaWriter, open) { + TCreateTabletReq request; + create_table_request(&request); + OLAPStatus res = k_engine->create_table(request); + ASSERT_EQ(OLAP_SUCCESS, res); + + TDescriptorTable tdesc_tbl = create_descriptor_table(); + ObjectPool obj_pool; + DescriptorTbl* desc_tbl = nullptr; + DescriptorTbl::create(&obj_pool, tdesc_tbl, &desc_tbl); + TupleDescriptor* tuple_desc = desc_tbl->get_tuple_descriptor(0); + + PUniqueId load_id; + load_id.set_hi(0); + load_id.set_lo(0); + WriteRequest write_req = {10003, 270068375, WriteType::LOAD, + 20001, 30001, load_id, false, tuple_desc}; + DeltaWriter* delta_writer = nullptr; + DeltaWriter::open(&write_req, &delta_writer); + ASSERT_NE(delta_writer, nullptr); + res = delta_writer->close(nullptr); + ASSERT_EQ(OLAP_SUCCESS, res); + SAFE_DELETE(delta_writer); + + TDropTabletReq drop_request; + auto tablet_id = 10003; + auto schema_hash = 270068375; + res = k_engine->drop_table(tablet_id, schema_hash); + ASSERT_EQ(OLAP_SUCCESS, res); +} + +TEST_F(TestDeltaWriter, write) { + TCreateTabletReq request; + create_table_request(&request); + OLAPStatus res = k_engine->create_table(request); + ASSERT_EQ(OLAP_SUCCESS, res); + + TDescriptorTable tdesc_tbl = create_descriptor_table(); + ObjectPool obj_pool; + DescriptorTbl* desc_tbl = nullptr; + DescriptorTbl::create(&obj_pool, tdesc_tbl, &desc_tbl); + TupleDescriptor* tuple_desc = desc_tbl->get_tuple_descriptor(0); + + PUniqueId load_id; + load_id.set_hi(0); + load_id.set_lo(0); + WriteRequest write_req = {10003, 270068375, WriteType::LOAD, + 20001, 30001, load_id, false, tuple_desc}; + DeltaWriter* delta_writer = nullptr; + DeltaWriter::open(&write_req, &delta_writer); + ASSERT_NE(delta_writer, nullptr); + + const std::vector& slots = tuple_desc->slots(); + Arena arena; + // Tuple 1 + { + Tuple* tuple = reinterpret_cast(arena.Allocate(tuple_desc->byte_size())); + memset(tuple, 0, tuple_desc->byte_size()); + *(int8_t*)(tuple->get_slot(slots[0]->tuple_offset())) = -127; + *(int16_t*)(tuple->get_slot(slots[1]->tuple_offset())) = -32767; + *(int32_t*)(tuple->get_slot(slots[2]->tuple_offset())) = -2147483647; + *(int64_t*)(tuple->get_slot(slots[3]->tuple_offset())) = -9223372036854775807L; + + int128_t large_int_value = -90000; + memcpy(tuple->get_slot(slots[4]->tuple_offset()), &large_int_value, sizeof(int128_t)); + + ((DateTimeValue*)(tuple->get_slot(slots[5]->tuple_offset())))->from_date_str("2048-11-10", 10); + ((DateTimeValue*)(tuple->get_slot(slots[6]->tuple_offset())))->from_date_str("2636-08-16 19:39:43", 19); + + StringValue* char_ptr = (StringValue*)(tuple->get_slot(slots[7]->tuple_offset())); + char_ptr->ptr = arena.Allocate(4); + memcpy(char_ptr->ptr, "abcd", 4); + char_ptr->len = 4; + + StringValue* var_ptr = (StringValue*)(tuple->get_slot(slots[8]->tuple_offset())); + var_ptr->ptr = arena.Allocate(5); + memcpy(var_ptr->ptr, "abcde", 5); + var_ptr->len = 5; + + DecimalValue decimal_value(1.1); + *(DecimalValue*)(tuple->get_slot(slots[9]->tuple_offset())) = decimal_value; + + *(int8_t*)(tuple->get_slot(slots[10]->tuple_offset())) = -127; + *(int16_t*)(tuple->get_slot(slots[11]->tuple_offset())) = -32767; + *(int32_t*)(tuple->get_slot(slots[12]->tuple_offset())) = -2147483647; + *(int64_t*)(tuple->get_slot(slots[13]->tuple_offset())) = -9223372036854775807L; + + memcpy(tuple->get_slot(slots[14]->tuple_offset()), &large_int_value, sizeof(int128_t)); + + ((DateTimeValue*)(tuple->get_slot(slots[15]->tuple_offset())))->from_date_str("2048-11-10", 10); + ((DateTimeValue*)(tuple->get_slot(slots[16]->tuple_offset())))->from_date_str("2636-08-16 19:39:43", 19); + + char_ptr = (StringValue*)(tuple->get_slot(slots[17]->tuple_offset())); + char_ptr->ptr = arena.Allocate(4); + memcpy(char_ptr->ptr, "abcd", 4); + char_ptr->len = 4; + + var_ptr = (StringValue*)(tuple->get_slot(slots[18]->tuple_offset())); + var_ptr->ptr = arena.Allocate(5); + memcpy(var_ptr->ptr, "abcde", 5); + var_ptr->len = 5; + + DecimalValue val_decimal(1.1); + *(DecimalValue*)(tuple->get_slot(slots[19]->tuple_offset())) = val_decimal; + + res = delta_writer->write(tuple); + ASSERT_EQ(OLAP_SUCCESS, res); + } + + res = delta_writer->close(nullptr); + ASSERT_EQ(res, OLAP_SUCCESS); + + // publish version success + OLAPTablePtr table = OLAPEngine::get_instance()->get_table(write_req.tablet_id, write_req.schema_hash); + TPublishVersionRequest publish_req; + publish_req.transaction_id = write_req.transaction_id; + TPartitionVersionInfo info; + info.partition_id = write_req.partition_id; + info.version = table->lastest_version()->end_version() + 1; + info.version_hash = table->lastest_version()->version_hash() + 1; + std::vector partition_version_infos; + partition_version_infos.push_back(info); + publish_req.partition_version_infos = partition_version_infos; + std::vector error_tablet_ids; + res = k_engine->publish_version(publish_req, &error_tablet_ids); + + ASSERT_EQ(1, table->get_num_rows()); + + auto tablet_id = 10003; + auto schema_hash = 270068375; + res = k_engine->drop_table(tablet_id, schema_hash); + ASSERT_EQ(OLAP_SUCCESS, res); +} + +// ######################### ALTER TABLE TEST BEGIN ######################### + +void schema_change_request(const TCreateTabletReq& base_request, TCreateTabletReq* request) { + //linked schema change, add a value column + request->tablet_id = base_request.tablet_id + 1; + request->__set_version(base_request.version); + request->__set_version_hash(base_request.version_hash); + request->tablet_schema.schema_hash = base_request.tablet_schema.schema_hash + 1; + request->tablet_schema.short_key_column_count = 3; + request->tablet_schema.storage_type = TStorageType::COLUMN; + + request->tablet_schema.columns.push_back(base_request.tablet_schema.columns[2]); + request->tablet_schema.columns.push_back(base_request.tablet_schema.columns[3]); + request->tablet_schema.columns.push_back(base_request.tablet_schema.columns[4]); + request->tablet_schema.columns.push_back(base_request.tablet_schema.columns[5]); + request->tablet_schema.columns.push_back(base_request.tablet_schema.columns[6]); + request->tablet_schema.columns.push_back(base_request.tablet_schema.columns[7]); + request->tablet_schema.columns.push_back(base_request.tablet_schema.columns[8]); + request->tablet_schema.columns.push_back(base_request.tablet_schema.columns[9]); + + TColumn v0; + v0.column_name = "v0"; + v0.column_type.type = TPrimitiveType::BIGINT; + v0.__set_is_key(false); + v0.__set_default_value("0"); + v0.__set_aggregation_type(TAggregationType::SUM); + request->tablet_schema.columns.push_back(v0); + + request->tablet_schema.columns.push_back(base_request.tablet_schema.columns[10]); + request->tablet_schema.columns.push_back(base_request.tablet_schema.columns[11]); + request->tablet_schema.columns.push_back(base_request.tablet_schema.columns[12]); + request->tablet_schema.columns.push_back(base_request.tablet_schema.columns[13]); + request->tablet_schema.columns.push_back(base_request.tablet_schema.columns[14]); + request->tablet_schema.columns.push_back(base_request.tablet_schema.columns[15]); + request->tablet_schema.columns.push_back(base_request.tablet_schema.columns[16]); + request->tablet_schema.columns.push_back(base_request.tablet_schema.columns[17]); + + TColumn v9; + v9.column_name = "v9"; + v9.__set_is_key(false); + v9.column_type.type = TPrimitiveType::VARCHAR; + v9.column_type.__set_len(130); + v9.__set_aggregation_type(TAggregationType::REPLACE); + request->tablet_schema.columns.push_back(v9); + + request->tablet_schema.columns.push_back(base_request.tablet_schema.columns[19]); +} + +AlterTableStatus show_alter_table_status(const TAlterTabletReq& request) { + AlterTableStatus status = ALTER_TABLE_RUNNING; + uint32_t max_retry = MAX_RETRY_TIMES; + while (max_retry > 0) { + status = k_engine->show_alter_table_status( + request.base_tablet_id, request.base_schema_hash); + if (status != ALTER_TABLE_RUNNING) { break; } + LOG(INFO) << "doing alter table......"; + --max_retry; + sleep(1); + } + return status; +} + +class TestSchemaChange : public ::testing::Test { +public: + TestSchemaChange() { } + ~TestSchemaChange() { } + + void SetUp() { + // Create local data dir for OLAPEngine. + char buffer[MAX_PATH_LEN]; + getcwd(buffer, MAX_PATH_LEN); + config::storage_root_path = std::string(buffer) + "/data_schema_change"; + remove_all_dir(config::storage_root_path); + ASSERT_EQ(create_dir(config::storage_root_path), OLAP_SUCCESS); + + // Initialize all singleton object. + // OLAPRootPath::get_instance()->reload_root_paths(config::storage_root_path.c_str()); + } + + void TearDown(){ + // Remove all dir. + ASSERT_EQ(OLAP_SUCCESS, remove_all_dir(config::storage_root_path)); + } +}; + +TEST_F(TestSchemaChange, schema_change) { + OLAPStatus res = OLAP_SUCCESS; + AlterTableStatus status = ALTER_TABLE_WAITING; + + // 1. Prepare for schema change. + // create base table + TCreateTabletReq create_base_tablet; + create_table_request(&create_base_tablet); + res = k_engine->create_table(create_base_tablet); + ASSERT_EQ(OLAP_SUCCESS, res); + + TDescriptorTable tdesc_tbl = create_descriptor_table(); + ObjectPool obj_pool; + DescriptorTbl* desc_tbl = nullptr; + DescriptorTbl::create(&obj_pool, tdesc_tbl, &desc_tbl); + TupleDescriptor* tuple_desc = desc_tbl->get_tuple_descriptor(0); + + PUniqueId load_id; + load_id.set_hi(0); + load_id.set_lo(0); + WriteRequest write_req = {10003, 270068375, WriteType::LOAD, + 20001, 30001, load_id, false, tuple_desc}; + DeltaWriter* delta_writer = nullptr; + DeltaWriter::open(&write_req, &delta_writer); + ASSERT_NE(delta_writer, nullptr); + + const std::vector& slots = tuple_desc->slots(); + Arena arena; + // streaming load data + { + Tuple* tuple = reinterpret_cast(arena.Allocate(tuple_desc->byte_size())); + memset(tuple, 0, tuple_desc->byte_size()); + *(int8_t*)(tuple->get_slot(slots[0]->tuple_offset())) = -127; + *(int16_t*)(tuple->get_slot(slots[1]->tuple_offset())) = -32767; + *(int32_t*)(tuple->get_slot(slots[2]->tuple_offset())) = -2147483647; + *(int64_t*)(tuple->get_slot(slots[3]->tuple_offset())) = -9223372036854775807L; + + int128_t large_int_value = -90000; + memcpy(tuple->get_slot(slots[4]->tuple_offset()), &large_int_value, sizeof(int128_t)); + + ((DateTimeValue*)(tuple->get_slot(slots[5]->tuple_offset())))->from_date_str("2048-11-10", 10); + ((DateTimeValue*)(tuple->get_slot(slots[6]->tuple_offset())))->from_date_str("2636-08-16 19:39:43", 19); + + StringValue* char_ptr = (StringValue*)(tuple->get_slot(slots[7]->tuple_offset())); + char_ptr->ptr = arena.Allocate(4); + memcpy(char_ptr->ptr, "abcd", 4); + char_ptr->len = 4; + + StringValue* var_ptr = (StringValue*)(tuple->get_slot(slots[8]->tuple_offset())); + var_ptr->ptr = arena.Allocate(5); + memcpy(var_ptr->ptr, "abcde", 5); + var_ptr->len = 5; + + DecimalValue decimal_value(1.1); + *(DecimalValue*)(tuple->get_slot(slots[9]->tuple_offset())) = decimal_value; + + *(int8_t*)(tuple->get_slot(slots[10]->tuple_offset())) = -127; + *(int16_t*)(tuple->get_slot(slots[11]->tuple_offset())) = -32767; + *(int32_t*)(tuple->get_slot(slots[12]->tuple_offset())) = -2147483647; + *(int64_t*)(tuple->get_slot(slots[13]->tuple_offset())) = -9223372036854775807L; + + memcpy(tuple->get_slot(slots[14]->tuple_offset()), &large_int_value, sizeof(int128_t)); + + ((DateTimeValue*)(tuple->get_slot(slots[15]->tuple_offset())))->from_date_str("2048-11-10", 10); + ((DateTimeValue*)(tuple->get_slot(slots[16]->tuple_offset())))->from_date_str("2636-08-16 19:39:43", 19); + + char_ptr = (StringValue*)(tuple->get_slot(slots[17]->tuple_offset())); + char_ptr->ptr = arena.Allocate(4); + memcpy(char_ptr->ptr, "abcd", 4); + char_ptr->len = 4; + + var_ptr = (StringValue*)(tuple->get_slot(slots[18]->tuple_offset())); + var_ptr->ptr = arena.Allocate(5); + memcpy(var_ptr->ptr, "abcde", 5); + var_ptr->len = 5; + + DecimalValue val_decimal(1.1); + *(DecimalValue*)(tuple->get_slot(slots[19]->tuple_offset())) = val_decimal; + + res = delta_writer->write(tuple); + ASSERT_EQ(OLAP_SUCCESS, res); + } + + // publish version + res = delta_writer->close(nullptr); + ASSERT_EQ(res, OLAP_SUCCESS); + + // publish version success + OLAPTablePtr table = OLAPEngine::get_instance()->get_table(write_req.tablet_id, write_req.schema_hash); + TPublishVersionRequest publish_req; + publish_req.transaction_id = write_req.transaction_id; + TPartitionVersionInfo info; + info.partition_id = write_req.partition_id; + info.version = table->lastest_version()->end_version() + 1; + info.version_hash = table->lastest_version()->version_hash() + 1; + std::vector partition_version_infos; + partition_version_infos.push_back(info); + publish_req.partition_version_infos = partition_version_infos; + std::vector error_tablet_ids; + res = k_engine->publish_version(publish_req, &error_tablet_ids); + ASSERT_EQ(res, OLAP_SUCCESS); + + // 1. set add column request + TCreateTabletReq create_new_tablet; + schema_change_request(create_base_tablet, &create_new_tablet); + TAlterTabletReq request; + request.__set_base_tablet_id(create_base_tablet.tablet_id); + request.__set_base_schema_hash(create_base_tablet.tablet_schema.schema_hash); + request.__set_new_tablet_req(create_new_tablet); + + // 2. Submit schema change + request.base_schema_hash = create_base_tablet.tablet_schema.schema_hash; + res = k_engine->schema_change(request); + ASSERT_EQ(OLAP_SUCCESS, res); + + // 3. Verify schema change result. + // show schema change status + status = show_alter_table_status(request); + ASSERT_EQ(ALTER_TABLE_FINISHED, status); + + // check new tablet information + TTabletInfo tablet_info; + tablet_info.tablet_id = create_new_tablet.tablet_id; + tablet_info.schema_hash = create_new_tablet.tablet_schema.schema_hash; + res = k_engine->report_tablet_info(&tablet_info); + ASSERT_EQ(OLAP_SUCCESS, res); + ASSERT_EQ(info.version, tablet_info.version); + ASSERT_EQ(info.version_hash, tablet_info.version_hash); + ASSERT_EQ(1, tablet_info.row_count); + + // 4. Retry the same schema change request. + res = k_engine->schema_change(request); + ASSERT_EQ(OLAP_SUCCESS, res); + status = k_engine->show_alter_table_status( + request.base_tablet_id, request.base_schema_hash); + ASSERT_EQ(ALTER_TABLE_FINISHED, status); + + auto tablet_id = create_new_tablet.tablet_id; + auto schema_hash = create_new_tablet.tablet_schema.schema_hash; + res = k_engine->drop_table(tablet_id, schema_hash); + ASSERT_EQ(OLAP_SUCCESS, res); +} + +} // namespace palo + +int main(int argc, char** argv) { + std::string conffile = std::string(getenv("DORIS_HOME")) + "/conf/be.conf"; + if (!palo::config::init(conffile.c_str(), false)) { + fprintf(stderr, "error read config file. \n"); + return -1; + } + palo::init_glog("be-test"); + int ret = palo::OLAP_SUCCESS; + testing::InitGoogleTest(&argc, argv); + palo::CpuInfo::init(); + + palo::set_up(); + ret = RUN_ALL_TESTS(); + palo::tear_down(); + + google::protobuf::ShutdownProtobufLibrary(); + return ret; +} diff --git a/be/test/olap/olap_header_manager_test.cpp b/be/test/olap/olap_header_manager_test.cpp new file mode 100644 index 0000000000..84d5e45a09 --- /dev/null +++ b/be/test/olap/olap_header_manager_test.cpp @@ -0,0 +1,118 @@ +// Copyright (c) 2018, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 +#include +#include + +#include "gtest/gtest.h" +#include "gmock/gmock.h" +#include "olap/store.h" +#include "olap/olap_header_manager.h" +#include "olap/olap_define.h" +#include "boost/filesystem.hpp" +#include "json2pb/json_to_pb.h" + +#ifndef BE_TEST +#define BE_TEST +#endif + +using ::testing::_; +using ::testing::Return; +using ::testing::SetArgPointee; +using std::string; + +namespace palo { + +const std::string header_path = "./be/test/olap/test_data/header.txt"; + +class OlapHeaderManagerTest : public testing::Test { +public: + virtual void SetUp() { + std::string root_path = "./store"; + ASSERT_TRUE(boost::filesystem::create_directory(root_path)); + _store = new(std::nothrow) OlapStore(root_path); + ASSERT_NE(nullptr, _store); + Status st = _store->load(); + ASSERT_TRUE(st.ok()); + ASSERT_TRUE(boost::filesystem::exists("./store/meta")); + + std::ifstream infile(header_path); + char buffer[1024]; + while (!infile.eof()) { + infile.getline(buffer, 1024); + _json_header = _json_header + buffer + "\n"; + } + _json_header = _json_header.substr(0, _json_header.size() - 1); + _json_header = _json_header.substr(0, _json_header.size() - 1); + std::cout << "set up finish" << std::endl; + } + + virtual void TearDown() { + delete _store; + ASSERT_TRUE(boost::filesystem::remove_all("./store")); + } + +private: + OlapStore* _store; + std::string _json_header; +}; + +TEST_F(OlapHeaderManagerTest, TestConvertedFlag) { + bool converted_flag; + OLAPStatus s = OlapHeaderManager::get_header_converted(_store, converted_flag); + ASSERT_EQ(false, converted_flag); + s = OlapHeaderManager::set_converted_flag(_store); + ASSERT_EQ(OLAP_SUCCESS, s); + s = OlapHeaderManager::get_header_converted(_store, converted_flag); + ASSERT_EQ(true, converted_flag); +} + +TEST_F(OlapHeaderManagerTest, TestSaveAndGetAndRemove) { + const TTabletId tablet_id = 20487; + const TSchemaHash schema_hash = 1520686811; + OLAPHeader header; + bool ret = json2pb::JsonToProtoMessage(_json_header, &header); + ASSERT_TRUE(ret); + OLAPStatus s = OlapHeaderManager::save(_store, tablet_id, schema_hash, &header); + ASSERT_EQ(OLAP_SUCCESS, s); + std::string json_header_read; + s = OlapHeaderManager::get_json_header(_store, tablet_id, schema_hash, &json_header_read); + ASSERT_EQ(OLAP_SUCCESS, s); + ASSERT_EQ(_json_header, json_header_read); + s = OlapHeaderManager::remove(_store, tablet_id, schema_hash); + ASSERT_EQ(OLAP_SUCCESS, s); + OLAPHeader header_read; + s = OlapHeaderManager::get_header(_store, tablet_id, schema_hash, &header_read); + ASSERT_EQ(OLAP_ERR_META_KEY_NOT_FOUND, s); +} + +TEST_F(OlapHeaderManagerTest, TestLoad) { + const TTabletId tablet_id = 20487; + const TSchemaHash schema_hash = 1520686811; + OLAPStatus s = OlapHeaderManager::load_json_header(_store, header_path); + ASSERT_EQ(OLAP_SUCCESS, s); + std::string json_header_read; + s = OlapHeaderManager::get_json_header(_store, tablet_id, schema_hash, &json_header_read); + ASSERT_EQ(OLAP_SUCCESS, s); + ASSERT_EQ(_json_header, json_header_read); +} + +} // namespace palo + +int main(int argc, char **argv) { + ::testing::InitGoogleTest(&argc, argv); + return RUN_ALL_TESTS(); +} diff --git a/be/test/olap/olap_meta_test.cpp b/be/test/olap/olap_meta_test.cpp new file mode 100644 index 0000000000..c23fa44b33 --- /dev/null +++ b/be/test/olap/olap_meta_test.cpp @@ -0,0 +1,121 @@ +// Copyright (c) 2018, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 +#include + +#include "gtest/gtest.h" +#include "gmock/gmock.h" +#include "olap/olap_meta.h" +#include "olap/olap_define.h" +#include "boost/filesystem.hpp" + +#ifndef BE_TEST +#define BE_TEST +#endif + +using ::testing::_; +using ::testing::Return; +using ::testing::SetArgPointee; +using std::string; + +namespace palo { + +class OlapMetaTest : public testing::Test { +public: + virtual void SetUp() { + std::string root_path = "./"; + _meta = new OlapMeta(root_path); + OLAPStatus s = _meta->init(); + ASSERT_EQ(OLAP_SUCCESS, s); + ASSERT_TRUE(boost::filesystem::exists("./meta")); + } + + virtual void TearDown() { + delete _meta; + ASSERT_TRUE(boost::filesystem::remove_all("./meta")); + } + +private: + OlapMeta* _meta; +}; + +TEST_F(OlapMetaTest, TestGetRootPath) { + std::string root_path = _meta->get_root_path(); + ASSERT_EQ("./", root_path); +} + +TEST_F(OlapMetaTest, TestPutAndGet) { + // normal cases + std::string key = "key"; + std::string value = "value"; + OLAPStatus s = _meta->put(META_COLUMN_FAMILY_INDEX, key, value); + ASSERT_EQ(OLAP_SUCCESS, s); + std::string value_get; + s = _meta->get(META_COLUMN_FAMILY_INDEX, key, value_get); + ASSERT_EQ(OLAP_SUCCESS, s); + ASSERT_EQ(value, value_get); + + // abnormal cases + s = _meta->get(META_COLUMN_FAMILY_INDEX, "key_not_exist", value_get); + ASSERT_EQ(OLAP_ERR_META_KEY_NOT_FOUND, s); +} + +TEST_F(OlapMetaTest, TestRemove) { + // normal cases + std::string key = "key"; + std::string value = "value"; + OLAPStatus s = _meta->put(META_COLUMN_FAMILY_INDEX, key, value); + ASSERT_EQ(OLAP_SUCCESS, s); + std::string value_get; + s = _meta->get(META_COLUMN_FAMILY_INDEX, key, value_get); + ASSERT_EQ(OLAP_SUCCESS, s); + ASSERT_EQ(value, value_get); + s = _meta->remove(META_COLUMN_FAMILY_INDEX, key); + ASSERT_EQ(OLAP_SUCCESS, s); + s = _meta->remove(META_COLUMN_FAMILY_INDEX, "key_not_exist"); + ASSERT_EQ(OLAP_SUCCESS, s); +} + +TEST_F(OlapMetaTest, TestIterate) { + // normal cases + std::string key = "hdr_key"; + std::string value = "value"; + OLAPStatus s = OLAP_SUCCESS; + for (int i = 0; i < 10; i++) { + std::stringstream ss; + ss << key << "_" << i; + s = _meta->put(META_COLUMN_FAMILY_INDEX, ss.str(), value); + ASSERT_EQ(OLAP_SUCCESS, s); + } + bool error_flag = false; + s = _meta->iterate(META_COLUMN_FAMILY_INDEX, "hdr_", + [&error_flag](const std::string& key, const std::string& value) -> bool { + size_t pos = key.find_first_of("hdr_"); + if (pos != 0) { + error_flag = true; + } + return true; + }); + ASSERT_EQ(false, error_flag); + ASSERT_EQ(OLAP_SUCCESS, s); +} + +} // namespace palo + +int main(int argc, char **argv) { + ::testing::InitGoogleTest(&argc, argv); + return RUN_ALL_TESTS(); +} diff --git a/be/test/olap/serialize_test.cpp b/be/test/olap/serialize_test.cpp new file mode 100644 index 0000000000..ca3314a32a --- /dev/null +++ b/be/test/olap/serialize_test.cpp @@ -0,0 +1,242 @@ +// Copyright (c) 2017, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 "olap/column_file/serialize.h" + +#include + +namespace palo { +namespace column_file { +namespace ser { + +class SerializeTest : public testing::Test { +public: + SerializeTest() { } + virtual ~SerializeTest() { + } +}; + +TEST_F(SerializeTest, get_closet_fixed_bits) { + ASSERT_EQ(1, get_closet_fixed_bits(0)); + for (int i = 1; i <= 24; ++i) { + ASSERT_EQ(i, get_closet_fixed_bits(i)); + } + for (int i = 25; i <= 26; ++i) { + ASSERT_EQ(26, get_closet_fixed_bits(i)); + } + for (int i = 27; i <= 28; ++i) { + ASSERT_EQ(28, get_closet_fixed_bits(i)); + } + for (int i = 29; i <= 30; ++i) { + ASSERT_EQ(30, get_closet_fixed_bits(i)); + } + for (int i = 31; i <= 32; ++i) { + ASSERT_EQ(32, get_closet_fixed_bits(i)); + } + for (int i = 33; i <= 40; ++i) { + ASSERT_EQ(40, get_closet_fixed_bits(i)); + } + for (int i = 41; i <= 48; ++i) { + ASSERT_EQ(48, get_closet_fixed_bits(i)); + } + for (int i = 49; i <= 56; ++i) { + ASSERT_EQ(56, get_closet_fixed_bits(i)); + } + for (int i = 57; i <= 64; ++i) { + ASSERT_EQ(64, get_closet_fixed_bits(i)); + } +} + +TEST_F(SerializeTest, find_closet_num_bits) { + ASSERT_EQ(1, find_closet_num_bits(0)); + for (int i = 1; i <= 24; ++i) { + uint64_t val = (1l << (i - 1)); + ASSERT_EQ(i, find_closet_num_bits(val)); + } + for (int i = 25; i <= 26; ++i) { + uint64_t val = (1l << (i - 1)); + ASSERT_EQ(26, find_closet_num_bits(val)); + } + for (int i = 27; i <= 28; ++i) { + uint64_t val = (1l << (i - 1)); + ASSERT_EQ(28, find_closet_num_bits(val)); + } + for (int i = 29; i <= 30; ++i) { + uint64_t val = (1l << (i - 1)); + ASSERT_EQ(30, find_closet_num_bits(val)); + } + for (int i = 31; i <= 32; ++i) { + uint64_t val = (1l << (i - 1)); + ASSERT_EQ(32, find_closet_num_bits(val)); + } + for (int i = 33; i <= 40; ++i) { + uint64_t val = (1l << (i - 1)); + ASSERT_EQ(40, find_closet_num_bits(val)); + } + for (int i = 41; i <= 48; ++i) { + uint64_t val = (1l << (i - 1)); + ASSERT_EQ(48, find_closet_num_bits(val)); + } + for (int i = 49; i <= 56; ++i) { + uint64_t val = (1l << (i - 1)); + ASSERT_EQ(56, find_closet_num_bits(val)); + } + for (int i = 57; i <= 64; ++i) { + uint64_t val = (1l << (i - 1)); + ASSERT_EQ(64, find_closet_num_bits(val)); + } +} + +TEST_F(SerializeTest, encode_bit_width) { + ASSERT_EQ(ONE, encode_bit_width(0)); + for (int i = 1; i <= 24; ++i) { + ASSERT_EQ(i - 1, encode_bit_width(i)); + } + for (int i = 25; i <= 26; ++i) { + ASSERT_EQ(TWENTYSIX, encode_bit_width(i)); + } + for (int i = 27; i <= 28; ++i) { + ASSERT_EQ(TWENTYEIGHT, encode_bit_width(i)); + } + for (int i = 29; i <= 30; ++i) { + ASSERT_EQ(THIRTY, encode_bit_width(i)); + } + for (int i = 31; i <= 32; ++i) { + ASSERT_EQ(THIRTYTWO, encode_bit_width(i)); + } + for (int i = 33; i <= 40; ++i) { + ASSERT_EQ(FORTY, encode_bit_width(i)); + } + for (int i = 41; i <= 48; ++i) { + ASSERT_EQ(FORTYEIGHT, encode_bit_width(i)); + } + for (int i = 49; i <= 56; ++i) { + ASSERT_EQ(FIFTYSIX, encode_bit_width(i)); + } + for (int i = 57; i <= 64; ++i) { + ASSERT_EQ(SIXTYFOUR, encode_bit_width(i)); + } +} + +TEST_F(SerializeTest, decode_bit_width) { + for (int i = 0; i <= TWENTYFOUR; ++i) { + ASSERT_EQ(i + 1, decode_bit_width(i)); + } + ASSERT_EQ(26, decode_bit_width(TWENTYSIX)); + ASSERT_EQ(28, decode_bit_width(TWENTYEIGHT)); + ASSERT_EQ(30, decode_bit_width(THIRTY)); + ASSERT_EQ(32, decode_bit_width(THIRTYTWO)); + ASSERT_EQ(40, decode_bit_width(FORTY)); + ASSERT_EQ(48, decode_bit_width(FORTYEIGHT)); + ASSERT_EQ(56, decode_bit_width(FIFTYSIX)); + ASSERT_EQ(64, decode_bit_width(SIXTYFOUR)); +} + +TEST_F(SerializeTest, percentile_bits) { + int64_t data[100]; + + { + for (int i = 0; i < 5; ++i) { + data[i] = (1l << 58); + } + for (int i = 5; i < 100; ++i) { + data[i] = 1; + } + ASSERT_EQ(0, percentile_bits(data, 100, 0.0)); + ASSERT_EQ(1, percentile_bits(data, 100, 0.95)); + ASSERT_EQ(64, percentile_bits(data, 100, 0.99)); + ASSERT_EQ(64, percentile_bits(data, 100, 1.0)); + } + { + for (int i = 0; i < 11; ++i) { + data[i] = (1l << 26); + } + for (int i = 11; i < 100; ++i) { + data[i] = 1; + } + ASSERT_EQ(0, percentile_bits(data, 100, 0.0)); + ASSERT_EQ(1, percentile_bits(data, 100, 0.8)); + ASSERT_EQ(28, percentile_bits(data, 100, 0.9)); + } + { + for (int i = 0; i < 11; ++i) { + data[i] = (1l << 26); + } + for (int i = 11; i < 100; ++i) { + data[i] = 0; + } + ASSERT_EQ(0, percentile_bits(data, 100, 0.0)); + ASSERT_EQ(1, percentile_bits(data, 100, 0.1)); + ASSERT_EQ(1, percentile_bits(data, 100, 0.8)); + ASSERT_EQ(28, percentile_bits(data, 100, 0.9)); + } +} + +TEST_F(SerializeTest, new_percentile_bits) { + int64_t data[100]; + + { + for (int i = 0; i < 5; ++i) { + data[i] = (1l << 58); + } + for (int i = 5; i < 100; ++i) { + data[i] = 1; + } + uint16_t hists[65]; + compute_hists(data, 100, hists); + ASSERT_EQ(0, percentile_bits_with_hist(hists, 100, 0.0)); + ASSERT_EQ(1, percentile_bits_with_hist(hists, 100, 0.95)); + ASSERT_EQ(64, percentile_bits_with_hist(hists, 100, 0.99)); + ASSERT_EQ(64, percentile_bits_with_hist(hists, 100, 1.0)); + } + { + for (int i = 0; i < 11; ++i) { + data[i] = (1l << 26); + } + for (int i = 11; i < 100; ++i) { + data[i] = 1; + } + uint16_t hists[65]; + compute_hists(data, 100, hists); + ASSERT_EQ(0, percentile_bits_with_hist(hists, 100, 0.0)); + ASSERT_EQ(1, percentile_bits_with_hist(hists, 100, 0.8)); + ASSERT_EQ(28, percentile_bits_with_hist(hists, 100, 0.9)); + } + { + for (int i = 0; i < 11; ++i) { + data[i] = (1l << 26); + } + for (int i = 11; i < 100; ++i) { + data[i] = 0; + } + uint16_t hists[65]; + compute_hists(data, 100, hists); + ASSERT_EQ(0, percentile_bits_with_hist(hists, 100, 0.0)); + ASSERT_EQ(1, percentile_bits_with_hist(hists, 100, 0.1)); + ASSERT_EQ(1, percentile_bits_with_hist(hists, 100, 0.8)); + ASSERT_EQ(28, percentile_bits_with_hist(hists, 100, 0.9)); + } +} + + +} +} +} + +int main(int argc, char** argv) { + ::testing::InitGoogleTest(&argc, argv); + return RUN_ALL_TESTS(); +} + diff --git a/be/test/olap/skiplist_test.cpp b/be/test/olap/skiplist_test.cpp new file mode 100644 index 0000000000..428734926d --- /dev/null +++ b/be/test/olap/skiplist_test.cpp @@ -0,0 +1,387 @@ +// Copyright (c) 2011 The LevelDB Authors. All rights reserved. +// Use of this source code is governed by a BSD-style license that can be +// found in the LICENSE file. See the AUTHORS file for names of contributors. + +#include "olap/skiplist.h" + +#include +#include + +#include +#include "olap/schema.h" +#include "olap/utils.h" +#include "util/arena.h" +#include "util/hash_util.hpp" +#include "util/random.h" +#include "util/thread_pool.hpp" + +namespace palo { + +typedef uint64_t Key; +const int random_seed = 301; + +struct TestComparator { + int operator()(const Key& a, const Key& b) const { + if (a < b) { + return -1; + } else if (a > b) { + return +1; + } else { + return 0; + } + } +}; + +class SkipTest : public testing::Test {}; + +TEST_F(SkipTest, Empty) { + Arena arena; + TestComparator cmp; + SkipList list(cmp, &arena); + ASSERT_TRUE(!list.Contains(10)); + + SkipList::Iterator iter(&list); + ASSERT_TRUE(!iter.Valid()); + iter.SeekToFirst(); + ASSERT_TRUE(!iter.Valid()); + iter.Seek(100); + ASSERT_TRUE(!iter.Valid()); + iter.SeekToLast(); + ASSERT_TRUE(!iter.Valid()); +} + +TEST_F(SkipTest, InsertAndLookup) { + const int N = 2000; + const int R = 5000; + Random rnd(1000); + std::set keys; + Arena arena; + TestComparator cmp; + SkipList list(cmp, &arena); + for (int i = 0; i < N; i++) { + Key key = rnd.Next() % R; + if (keys.insert(key).second) { + bool overwritten = false; + list.Insert(key, &overwritten, KeysType::AGG_KEYS); + } + } + + for (int i = 0; i < R; i++) { + if (list.Contains(i)) { + ASSERT_EQ(keys.count(i), 1); + } else { + ASSERT_EQ(keys.count(i), 0); + } + } + + // Simple iterator tests + { + SkipList::Iterator iter(&list); + ASSERT_TRUE(!iter.Valid()); + + iter.Seek(0); + ASSERT_TRUE(iter.Valid()); + ASSERT_EQ(*(keys.begin()), iter.key()); + + iter.SeekToFirst(); + ASSERT_TRUE(iter.Valid()); + ASSERT_EQ(*(keys.begin()), iter.key()); + + iter.SeekToLast(); + ASSERT_TRUE(iter.Valid()); + ASSERT_EQ(*(keys.rbegin()), iter.key()); + } + + // Forward iteration test + for (int i = 0; i < R; i++) { + SkipList::Iterator iter(&list); + iter.Seek(i); + + // Compare against model iterator + std::set::iterator model_iter = keys.lower_bound(i); + for (int j = 0; j < 3; j++) { + if (model_iter == keys.end()) { + ASSERT_TRUE(!iter.Valid()); + break; + } else { + ASSERT_TRUE(iter.Valid()); + ASSERT_EQ(*model_iter, iter.key()); + ++model_iter; + iter.Next(); + } + } + } + + // Backward iteration test + { + SkipList::Iterator iter(&list); + iter.SeekToLast(); + + // Compare against model iterator + for (std::set::reverse_iterator model_iter = keys.rbegin(); + model_iter != keys.rend(); + ++model_iter) { + ASSERT_TRUE(iter.Valid()); + ASSERT_EQ(*model_iter, iter.key()); + iter.Prev(); + } + ASSERT_TRUE(!iter.Valid()); + } +} + +// We want to make sure that with a single writer and multiple +// concurrent readers (with no synchronization other than when a +// reader's iterator is created), the reader always observes all the +// data that was present in the skip list when the iterator was +// constructor. Because insertions are happening concurrently, we may +// also observe new values that were inserted since the iterator was +// constructed, but we should never miss any values that were present +// at iterator construction time. +// +// We generate multi-part keys: +// +// where: +// key is in range [0..K-1] +// gen is a generation number for key +// hash is hash(key,gen) +// +// The insertion code picks a random key, sets gen to be 1 + the last +// generation number inserted for that key, and sets hash to Hash(key,gen). +// +// At the beginning of a read, we snapshot the last inserted +// generation number for each key. We then iterate, including random +// calls to Next() and Seek(). For every key we encounter, we +// check that it is either expected given the initial snapshot or has +// been concurrently added since the iterator started. +class ConcurrentTest { +private: + static const uint32_t K = 4; + + static uint64_t key(Key key) { return (key >> 40); } + static uint64_t gen(Key key) { return (key >> 8) & 0xffffffffu; } + static uint64_t hash(Key key) { return key & 0xff; } + + static uint64_t hash_numbers(uint64_t k, uint64_t g) { + uint64_t data[2] = { k, g }; + return HashUtil::hash(reinterpret_cast(data), sizeof(data), 0); + } + + static Key make_key(uint64_t k, uint64_t g) { + assert(sizeof(Key) == sizeof(uint64_t)); + assert(k <= K); // We sometimes pass K to seek to the end of the skiplist + assert(g <= 0xffffffffu); + return ((k << 40) | (g << 8) | (hash_numbers(k, g) & 0xff)); + } + + static bool is_valid_key(Key k) { + return hash(k) == (hash_numbers(key(k), gen(k)) & 0xff); + } + + static Key random_target(Random* rnd) { + switch (rnd->Next() % 10) { + case 0: + // Seek to beginning + return make_key(0, 0); + case 1: + // Seek to end + return make_key(K, 0); + default: + // Seek to middle + return make_key(rnd->Next() % K, 0); + } + } + + // Per-key generation + struct State { + std::atomic generation[K]; + void set(int k, int v) { + generation[k].store(v, std::memory_order_release); + } + int get(int k) { return generation[k].load(std::memory_order_acquire); } + + State() { + for (int k = 0; k < K; k++) { + set(k, 0); + } + } + }; + + // Current state of the test + State _current; + + Arena _arena; + + // SkipList is not protected by _mu. We just use a single writer + // thread to modify it. + SkipList _list; + +public: + ConcurrentTest() : _list(TestComparator(), &_arena) {} + + // REQUIRES: External synchronization + void write_step(Random* rnd) { + const uint32_t k = rnd->Next() % K; + const int g = _current.get(k) + 1; + const Key new_key = make_key(k, g); + bool overwritten = false; + _list.Insert(new_key, &overwritten, KeysType::AGG_KEYS); + _current.set(k, g); + } + + void read_step(Random* rnd) { + // Remember the initial committed state of the skiplist. + State initial_state; + for (int k = 0; k < K; k++) { + initial_state.set(k, _current.get(k)); + } + + Key pos = random_target(rnd); + SkipList::Iterator iter(&_list); + iter.Seek(pos); + while (true) { + Key current; + if (!iter.Valid()) { + current = make_key(K, 0); + } else { + current = iter.key(); + ASSERT_TRUE(is_valid_key(current)) << current; + } + ASSERT_LE(pos, current) << "should not go backwards"; + + // Verify that everything in [pos,current) was not present in + // initial_state. + while (pos < current) { + ASSERT_LT(key(pos), K) << pos; + + // Note that generation 0 is never inserted, so it is ok if + // <*,0,*> is missing. + ASSERT_TRUE((gen(pos) == 0) || + (gen(pos) > static_cast(initial_state.get(key(pos)))) + ) << "key: " << key(pos) + << "; gen: " << gen(pos) + << "; initgen: " + << initial_state.get(key(pos)); + + // Advance to next key in the valid key space + if (key(pos) < key(current)) { + pos = make_key(key(pos) + 1, 0); + } else { + pos = make_key(key(pos), gen(pos) + 1); + } + } + + if (!iter.Valid()) { + break; + } + + if (rnd->Next() % 2) { + iter.Next(); + pos = make_key(key(pos), gen(pos) + 1); + } else { + Key new_target = random_target(rnd); + if (new_target > pos) { + pos = new_target; + iter.Seek(new_target); + } + } + } + } +}; +const uint32_t ConcurrentTest::K; + +// Simple test that does single-threaded testing of the ConcurrentTest +// scaffolding. +TEST_F(SkipTest, ConcurrentWithoutThreads) { + ConcurrentTest test; + Random rnd(random_seed); + for (int i = 0; i < 10000; i++) { + test.read_step(&rnd); + test.write_step(&rnd); + } +} + +class TestState { +public: + ConcurrentTest _t; + int _seed; + std::atomic _quit_flag; + + enum ReaderState { + STARTING, + RUNNING, + DONE + }; + + explicit TestState(int s) + : _seed(s), + _quit_flag(NULL), + _state(STARTING), + _cv_state(_mu) {} + + void wait(ReaderState s) { + _mu.lock(); + while (_state != s) { + _cv_state.wait(); + } + _mu.unlock(); + } + + void change(ReaderState s) { + _mu.lock(); + _state = s; + _cv_state.notify(); + _mu.unlock(); + } + +private: + Mutex _mu; + ReaderState _state; + Condition _cv_state; +}; + +static void concurrent_reader(void* arg) { + TestState* state = reinterpret_cast(arg); + Random rnd(state->_seed); + int64_t reads = 0; + state->change(TestState::RUNNING); + while (!state->_quit_flag.load(std::memory_order_acquire)) { + state->_t.read_step(&rnd); + ++reads; + } + state->change(TestState::DONE); +} + +static void run_concurrent(int run) { + const int seed = random_seed + (run * 100); + Random rnd(seed); + const int N = 1000; + const int kSize = 1000; + ThreadPool thread_pool(10, 100); + for (int i = 0; i < N; i++) { + if ((i % 100) == 0) { + fprintf(stderr, "Run %d of %d\n", i, N); + } + TestState state(seed + 1); + thread_pool.offer(std::bind(concurrent_reader, &state)); + state.wait(TestState::RUNNING); + for (int i = 0; i < kSize; i++) { + state._t.write_step(&rnd); + } + state._quit_flag.store(true, std::memory_order_release); // Any non-NULL arg will do + state.wait(TestState::DONE); + } +} + +TEST_F(SkipTest, Concurrent1) { run_concurrent(1); } +TEST_F(SkipTest, Concurrent2) { run_concurrent(2); } +TEST_F(SkipTest, Concurrent3) { run_concurrent(3); } +TEST_F(SkipTest, Concurrent4) { run_concurrent(4); } +TEST_F(SkipTest, Concurrent5) { run_concurrent(5); } + +} // namespace palo + +int main(int argc, char** argv) { + ::testing::InitGoogleTest(&argc, argv); + palo::CpuInfo::init(); + return RUN_ALL_TESTS(); +} diff --git a/be/test/olap/test_data/header.txt b/be/test/olap/test_data/header.txt new file mode 100644 index 0000000000..a657c2c40b --- /dev/null +++ b/be/test/olap/test_data/header.txt @@ -0,0 +1,152 @@ +{ + "num_rows_per_data_block": 1024, + "file_version": [ + { + "num_segments": 1, + "start_version": 0, + "end_version": 1, + "version_hash": 0, + "max_timestamp": 0, + "index_size": 67, + "data_size": 477, + "num_rows": 0, + "creation_time": 1534750461, + "delta_pruning": { + "column_pruning": [ + { + "min": "OTk5OS0xMi0zMQ==", + "max": "MC0wMC0wMA==", + "null_flag": false + }, + { + "min": "MjE0NzQ4MzY0Nw==", + "max": "MA==", + "null_flag": false + }, + { + "min": "MzI3Njc=", + "max": "MA==", + "null_flag": false + }, + { + "min": "/w==", + "max": "AAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA=", + "null_flag": false + } + ] + } + }, + { + "num_segments": 1, + "start_version": 2, + "end_version": 2, + "version_hash": 0, + "max_timestamp": 0, + "index_size": 67, + "data_size": 477, + "num_rows": 0, + "creation_time": 1534750461, + "delta_pruning": { + "column_pruning": [ + { + "min": "OTk5OS0xMi0zMQ==", + "max": "MC0wMC0wMA==", + "null_flag": false + }, + { + "min": "MjE0NzQ4MzY0Nw==", + "max": "MA==", + "null_flag": false + }, + { + "min": "MzI3Njc=", + "max": "MA==", + "null_flag": false + }, + { + "min": "/w==", + "max": "AAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA=", + "null_flag": false + } + ] + } + } + ], + "cumulative_layer_point": 2, + "num_short_key_fields": 4, + "column": [ + { + "name": "event_day", + "type": "DATE", + "aggregation": "NONE", + "length": 3, + "is_key": true, + "index_length": 3, + "is_allow_null": true, + "unique_id": 0, + "is_root_column": true + }, + { + "name": "siteid", + "type": "INT", + "aggregation": "NONE", + "length": 4, + "is_key": true, + "default_value": "10", + "index_length": 4, + "is_allow_null": true, + "unique_id": 1, + "is_root_column": true + }, + { + "name": "citycode", + "type": "SMALLINT", + "aggregation": "NONE", + "length": 2, + "is_key": true, + "index_length": 2, + "is_allow_null": true, + "unique_id": 2, + "is_root_column": true + }, + { + "name": "username", + "type": "VARCHAR", + "aggregation": "NONE", + "length": 34, + "is_key": true, + "default_value": "", + "index_length": 20, + "is_allow_null": true, + "unique_id": 3, + "is_root_column": true + }, + { + "name": "pv", + "type": "BIGINT", + "aggregation": "SUM", + "length": 8, + "is_key": false, + "default_value": "0", + "index_length": 8, + "is_allow_null": true, + "unique_id": 4, + "is_root_column": true + } + ], + "creation_time": 1534750461, + "selectivity": [ + 1, + 1, + 1, + 1 + ], + "data_file_type": "COLUMN_ORIENTED_FILE", + "next_column_unique_id": 5, + "compress_kind": "COMPRESS_LZ4", + "segment_size": 268435456, + "keys_type": "AGG_KEYS", + "tablet_id": 20487, + "schema_hash": 1520686811, + "shard": 0 +} diff --git a/be/test/runtime/stream_load_pipe_test.cpp b/be/test/runtime/stream_load_pipe_test.cpp new file mode 100644 index 0000000000..88c3346ce4 --- /dev/null +++ b/be/test/runtime/stream_load_pipe_test.cpp @@ -0,0 +1,260 @@ +// Copyright (c) 2017, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 "runtime/stream_load_pipe.h" + +#include + +#include + +namespace palo { + +class StreamLoadPipeTest : public testing::Test { +public: + StreamLoadPipeTest() { } + virtual ~StreamLoadPipeTest() { } + void SetUp() override { } +}; + +TEST_F(StreamLoadPipeTest, append_buffer) { + StreamLoadPipe pipe(66, 64); + + auto appender = [&pipe] { + int k = 0; + for (int i = 0; i < 2; ++i) { + auto byte_buf = ByteBuffer::allocate(64); + char buf[64]; + for (int j = 0; j < 64; ++j) { + buf[j] = '0' + (k++ % 10); + } + byte_buf->put_bytes(buf, 64); + byte_buf->flip(); + pipe.append(byte_buf); + } + pipe.finish(); + }; + std::thread t1(appender); + + char buf[256]; + size_t buf_len = 256; + bool eof = false; + auto st = pipe.read((uint8_t*)buf, &buf_len, &eof); + ASSERT_TRUE(st.ok()); + ASSERT_EQ(128, buf_len); + ASSERT_FALSE(eof); + for (int i = 0; i < 128; ++i) { + ASSERT_EQ('0' + (i % 10), buf[i]); + } + st = pipe.read((uint8_t*)buf, &buf_len, &eof); + ASSERT_TRUE(st.ok()); + ASSERT_EQ(0, buf_len); + ASSERT_TRUE(eof); + + t1.join(); +} + +TEST_F(StreamLoadPipeTest, append_bytes) { + StreamLoadPipe pipe(66, 64); + + auto appender = [&pipe] { + for (int i = 0; i < 128; ++i) { + char buf = '0' + (i % 10); + pipe.append(&buf, 1); + } + pipe.finish(); + }; + std::thread t1(appender); + + char buf[256]; + size_t buf_len = 256; + bool eof = false; + auto st = pipe.read((uint8_t*)buf, &buf_len, &eof); + ASSERT_TRUE(st.ok()); + ASSERT_EQ(128, buf_len); + ASSERT_FALSE(eof); + for (int i = 0; i < 128; ++i) { + ASSERT_EQ('0' + (i % 10), buf[i]); + } + st = pipe.read((uint8_t*)buf, &buf_len, &eof); + ASSERT_TRUE(st.ok()); + ASSERT_EQ(0, buf_len); + ASSERT_TRUE(eof); + + t1.join(); +} + +TEST_F(StreamLoadPipeTest, append_bytes2) { + StreamLoadPipe pipe(66, 64); + + auto appender = [&pipe] { + for (int i = 0; i < 128; ++i) { + char buf = '0' + (i % 10); + pipe.append(&buf, 1); + } + pipe.finish(); + }; + std::thread t1(appender); + + char buf[128]; + size_t buf_len = 62; + bool eof = false; + auto st = pipe.read((uint8_t*)buf, &buf_len, &eof); + ASSERT_TRUE(st.ok()); + ASSERT_EQ(62, buf_len); + ASSERT_FALSE(eof); + for (int i = 0; i < 62; ++i) { + ASSERT_EQ('0' + (i % 10), buf[i]); + } + for (int i = 62; i < 128; ++i) { + char ch; + buf_len = 1; + auto st = pipe.read((uint8_t*)&ch, &buf_len, &eof); + ASSERT_TRUE(st.ok()); + ASSERT_EQ(1, buf_len); + ASSERT_FALSE(eof); + ASSERT_EQ('0' + (i % 10), ch); + } + st = pipe.read((uint8_t*)buf, &buf_len, &eof); + ASSERT_TRUE(st.ok()); + ASSERT_EQ(0, buf_len); + ASSERT_TRUE(eof); + + t1.join(); +} + + +TEST_F(StreamLoadPipeTest, append_mix) { + StreamLoadPipe pipe(66, 64); + + auto appender = [&pipe] { + // 10 + int k = 0; + for (int i = 0; i < 10; ++i) { + char buf = '0' + (k++ % 10); + pipe.append(&buf, 1); + } + // 60 + { + auto byte_buf = ByteBuffer::allocate(60); + char buf[60]; + for (int j = 0; j < 60; ++j) { + buf[j] = '0' + (k++ % 10); + } + byte_buf->put_bytes(buf, 60); + byte_buf->flip(); + pipe.append(byte_buf); + } + // 8 + for (int i = 0; i < 8; ++i) { + char buf = '0' + (k++ % 10); + pipe.append(&buf, 1); + } + // 50 + { + auto byte_buf = ByteBuffer::allocate(50); + char buf[50]; + for (int j = 0; j < 50; ++j) { + buf[j] = '0' + (k++ % 10); + } + byte_buf->put_bytes(buf, 50); + byte_buf->flip(); + pipe.append(byte_buf); + } + pipe.finish(); + }; + std::thread t1(appender); + + char buf[128]; + size_t buf_len = 128; + bool eof = false; + auto st = pipe.read((uint8_t*)buf, &buf_len, &eof); + ASSERT_TRUE(st.ok()); + ASSERT_EQ(128, buf_len); + ASSERT_FALSE(eof); + for (int i = 0; i < 128; ++i) { + ASSERT_EQ('0' + (i % 10), buf[i]); + } + st = pipe.read((uint8_t*)buf, &buf_len, &eof); + ASSERT_TRUE(st.ok()); + ASSERT_EQ(0, buf_len); + ASSERT_TRUE(eof); + + t1.join(); +} + +TEST_F(StreamLoadPipeTest, cancel) { + StreamLoadPipe pipe(66, 64); + + auto appender = [&pipe] { + int k = 0; + for (int i = 0; i < 10; ++i) { + char buf = '0' + (k++ % 10); + pipe.append(&buf, 1); + } + usleep(100000); + pipe.cancel(); + }; + std::thread t1(appender); + + char buf[128]; + size_t buf_len = 128; + bool eof = false; + auto st = pipe.read((uint8_t*)buf, &buf_len, &eof); + ASSERT_FALSE(st.ok()); + t1.join(); +} + +TEST_F(StreamLoadPipeTest, close) { + StreamLoadPipe pipe(66, 64); + + auto appender = [&pipe] { + int k = 0; + { + auto byte_buf = ByteBuffer::allocate(64); + char buf[64]; + for (int j = 0; j < 64; ++j) { + buf[j] = '0' + (k++ % 10); + } + byte_buf->put_bytes(buf, 64); + byte_buf->flip(); + pipe.append(byte_buf); + } + { + auto byte_buf = ByteBuffer::allocate(64); + char buf[64]; + for (int j = 0; j < 64; ++j) { + buf[j] = '0' + (k++ % 10); + } + byte_buf->put_bytes(buf, 64); + byte_buf->flip(); + auto st = pipe.append(byte_buf); + ASSERT_FALSE(st.ok()); + } + }; + std::thread t1(appender); + + usleep(10000); + + pipe.close(); + + t1.join(); +} + +} + +int main(int argc, char* argv[]) { + ::testing::InitGoogleTest(&argc, argv); + return RUN_ALL_TESTS(); +} diff --git a/be/test/runtime/tablet_writer_mgr_test.cpp b/be/test/runtime/tablet_writer_mgr_test.cpp new file mode 100644 index 0000000000..78c485c629 --- /dev/null +++ b/be/test/runtime/tablet_writer_mgr_test.cpp @@ -0,0 +1,678 @@ +// Copyright (c) 2017, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 "runtime/tablet_writer_mgr.h" + +#include + +#include "common/object_pool.h" +#include "gen_cpp/Descriptors_types.h" +#include "gen_cpp/PaloInternalService_types.h" +#include "gen_cpp/Types_types.h" +#include "runtime/exec_env.h" +#include "runtime/descriptors.h" +#include "runtime/primitive_type.h" +#include "runtime/mem_tracker.h" +#include "runtime/row_batch.h" +#include "runtime/tuple_row.h" +#include "util/descriptor_helper.h" +#include "util/thrift_util.h" +#include "olap/delta_writer.h" + +namespace palo { + +std::unordered_map _k_tablet_recorder; +OLAPStatus open_status; +OLAPStatus add_status; +OLAPStatus close_status; + +// mock +DeltaWriter::DeltaWriter(WriteRequest* req) : _req(*req) { +} + +DeltaWriter::~DeltaWriter() { +} + +OLAPStatus DeltaWriter::init() { + return OLAP_SUCCESS; +} + +OLAPStatus DeltaWriter::open(WriteRequest* req, DeltaWriter** writer) { + if (open_status != OLAP_SUCCESS) { + return open_status; + } + *writer = new DeltaWriter(req); + return open_status; +} + +OLAPStatus DeltaWriter::write(Tuple* tuple) { + if (_k_tablet_recorder.find(_req.tablet_id) == std::end(_k_tablet_recorder)) { + _k_tablet_recorder[_req.tablet_id] = 1; + } else { + _k_tablet_recorder[_req.tablet_id]++; + } + return add_status; +} + +OLAPStatus DeltaWriter::close(google::protobuf::RepeatedPtrField* tablet_vec) { + return close_status; +} + +OLAPStatus DeltaWriter::cancel() { + return OLAP_SUCCESS; +} + +class TabletWriterMgrTest : public testing::Test { +public: + TabletWriterMgrTest() { } + virtual ~TabletWriterMgrTest() { } + void SetUp() override { + _k_tablet_recorder.clear(); + open_status = OLAP_SUCCESS; + add_status = OLAP_SUCCESS; + close_status = OLAP_SUCCESS; + } +private: +}; + +TEST_F(TabletWriterMgrTest, check_builder) { + TDescriptorTableBuilder table_builder; + { + TTupleDescriptorBuilder tuple; + tuple.add_slot( + TSlotDescriptorBuilder().type(TYPE_INT).column_name("c1").column_pos(0).build()); + tuple.add_slot( + TSlotDescriptorBuilder().type(TYPE_BIGINT).column_name("c2").column_pos(1).build()); + tuple.add_slot( + TSlotDescriptorBuilder().string_type(64).column_name("c3").column_pos(2).build()); + tuple.build(&table_builder); + } + DescriptorTbl* desc_tbl = nullptr; + ObjectPool obj_pool; + DescriptorTbl::create(&obj_pool, table_builder.desc_tbl(), &desc_tbl); + auto tuple = desc_tbl->get_tuple_descriptor(0); + ASSERT_EQ(32, tuple->byte_size()); + ASSERT_EQ(4, tuple->slots()[0]->tuple_offset()); + ASSERT_EQ(8, tuple->slots()[1]->tuple_offset()); + ASSERT_EQ(16, tuple->slots()[2]->tuple_offset()); +} + +TDescriptorTable create_descriptor_table() { + TDescriptorTableBuilder dtb; + TTupleDescriptorBuilder tuple_builder; + + tuple_builder.add_slot( + TSlotDescriptorBuilder().type(TYPE_INT).column_name("c1").column_pos(0).build()); + tuple_builder.add_slot( + TSlotDescriptorBuilder().type(TYPE_BIGINT).column_name("c2").column_pos(1).build()); + tuple_builder.build(&dtb); + + return dtb.desc_tbl(); +} + +// dbId=1, tableId=2, indexId=4, columns={c1, c2} +void create_schema(DescriptorTbl* desc_tbl, POlapTableSchemaParam* pschema) { + pschema->set_db_id(1); + pschema->set_table_id(2); + pschema->set_version(0); + + auto tuple_desc = desc_tbl->get_tuple_descriptor(0); + tuple_desc->to_protobuf(pschema->mutable_tuple_desc()); + for (auto slot : tuple_desc->slots()) { + slot->to_protobuf(pschema->add_slot_descs()); + } + + // index schema + auto indexes = pschema->add_indexes(); + indexes->set_id(4); + indexes->add_columns("c1"); + indexes->add_columns("c2"); + indexes->set_schema_hash(123); +} + +TEST_F(TabletWriterMgrTest, normal) { + ExecEnv env; + TabletWriterMgr mgr(&env); + + auto tdesc_tbl = create_descriptor_table(); + ObjectPool obj_pool; + DescriptorTbl* desc_tbl = nullptr; + DescriptorTbl::create(&obj_pool, tdesc_tbl, &desc_tbl); + auto tuple_desc = desc_tbl->get_tuple_descriptor(0); + RowDescriptor row_desc(*desc_tbl, {0}, {false}); + MemTracker tracker; + PUniqueId load_id; + load_id.set_hi(2); + load_id.set_lo(3); + { + PTabletWriterOpenRequest request; + request.set_allocated_id(&load_id); + request.set_index_id(4); + request.set_txn_id(1); + create_schema(desc_tbl, request.mutable_schema()); + for (int i = 0; i < 2; ++i) { + auto tablet = request.add_tablets(); + tablet->set_partition_id(10 + i); + tablet->set_tablet_id(20 + i); + } + request.set_num_senders(1); + request.set_need_gen_rollup(false); + auto st = mgr.open(request); + request.release_id(); + ASSERT_TRUE(st.ok()); + } + + // add a batch + { + PTabletWriterAddBatchRequest request; + request.set_allocated_id(&load_id); + request.set_index_id(4); + request.set_sender_id(0); + request.set_eos(true); + request.set_packet_seq(0); + + request.add_tablet_ids(20); + request.add_tablet_ids(21); + request.add_tablet_ids(20); + + RowBatch row_batch(row_desc, 1024, &tracker); + + // row1 + { + auto id = row_batch.add_row(); + auto tuple = (Tuple*)row_batch.tuple_data_pool()->allocate(tuple_desc->byte_size()); + row_batch.get_row(id)->set_tuple(0, tuple); + memset(tuple, 0, tuple_desc->byte_size()); + *(int*)tuple->get_slot(tuple_desc->slots()[0]->tuple_offset()) = 987654; + *(int64_t*)tuple->get_slot(tuple_desc->slots()[1]->tuple_offset()) = 1234567899876; + row_batch.commit_last_row(); + } + // row2 + { + auto id = row_batch.add_row(); + auto tuple = (Tuple*)row_batch.tuple_data_pool()->allocate(tuple_desc->byte_size()); + row_batch.get_row(id)->set_tuple(0, tuple); + memset(tuple, 0, tuple_desc->byte_size()); + *(int*)tuple->get_slot(tuple_desc->slots()[0]->tuple_offset()) = 12345678; + *(int64_t*)tuple->get_slot(tuple_desc->slots()[1]->tuple_offset()) = 9876567899876; + row_batch.commit_last_row(); + } + // row3 + { + auto id = row_batch.add_row(); + auto tuple = (Tuple*)row_batch.tuple_data_pool()->allocate(tuple_desc->byte_size()); + row_batch.get_row(id)->set_tuple(0, tuple); + memset(tuple, 0, tuple_desc->byte_size()); + *(int*)tuple->get_slot(tuple_desc->slots()[0]->tuple_offset()) = 876545678; + *(int64_t*)tuple->get_slot(tuple_desc->slots()[1]->tuple_offset()) = 76543234567; + row_batch.commit_last_row(); + } + row_batch.serialize(request.mutable_row_batch()); + google::protobuf::RepeatedPtrField tablet_vec; + auto st = mgr.add_batch(request, &tablet_vec); + request.release_id(); + ASSERT_TRUE(st.ok()); + } + // check content + ASSERT_EQ(_k_tablet_recorder[20], 2); + ASSERT_EQ(_k_tablet_recorder[21], 1); +} + +TEST_F(TabletWriterMgrTest, cancel) { + ExecEnv env; + TabletWriterMgr mgr(&env); + + auto tdesc_tbl = create_descriptor_table(); + ObjectPool obj_pool; + DescriptorTbl* desc_tbl = nullptr; + DescriptorTbl::create(&obj_pool, tdesc_tbl, &desc_tbl); + RowDescriptor row_desc(*desc_tbl, {0}, {false}); + MemTracker tracker; + PUniqueId load_id; + load_id.set_hi(2); + load_id.set_lo(3); + { + PTabletWriterOpenRequest request; + request.set_allocated_id(&load_id); + request.set_index_id(4); + request.set_txn_id(1); + create_schema(desc_tbl, request.mutable_schema()); + for (int i = 0; i < 2; ++i) { + auto tablet = request.add_tablets(); + tablet->set_partition_id(10 + i); + tablet->set_tablet_id(20 + i); + } + request.set_num_senders(1); + request.set_need_gen_rollup(false); + auto st = mgr.open(request); + request.release_id(); + ASSERT_TRUE(st.ok()); + } + + // add a batch + { + PTabletWriterCancelRequest request; + request.set_allocated_id(&load_id); + request.set_index_id(4); + auto st = mgr.cancel(request); + request.release_id(); + ASSERT_TRUE(st.ok()); + } +} + +TEST_F(TabletWriterMgrTest, open_failed) { + ExecEnv env; + TabletWriterMgr mgr(&env); + + auto tdesc_tbl = create_descriptor_table(); + ObjectPool obj_pool; + DescriptorTbl* desc_tbl = nullptr; + DescriptorTbl::create(&obj_pool, tdesc_tbl, &desc_tbl); + RowDescriptor row_desc(*desc_tbl, {0}, {false}); + MemTracker tracker; + PUniqueId load_id; + load_id.set_hi(2); + load_id.set_lo(3); + { + PTabletWriterOpenRequest request; + request.set_allocated_id(&load_id); + request.set_index_id(4); + request.set_txn_id(1); + create_schema(desc_tbl, request.mutable_schema()); + for (int i = 0; i < 2; ++i) { + auto tablet = request.add_tablets(); + tablet->set_partition_id(10 + i); + tablet->set_tablet_id(20 + i); + } + request.set_num_senders(1); + request.set_need_gen_rollup(false); + open_status = OLAP_ERR_TABLE_NOT_FOUND; + auto st = mgr.open(request); + request.release_id(); + ASSERT_FALSE(st.ok()); + } +} + +TEST_F(TabletWriterMgrTest, add_failed) { + ExecEnv env; + TabletWriterMgr mgr(&env); + + auto tdesc_tbl = create_descriptor_table(); + ObjectPool obj_pool; + DescriptorTbl* desc_tbl = nullptr; + DescriptorTbl::create(&obj_pool, tdesc_tbl, &desc_tbl); + auto tuple_desc = desc_tbl->get_tuple_descriptor(0); + RowDescriptor row_desc(*desc_tbl, {0}, {false}); + MemTracker tracker; + PUniqueId load_id; + load_id.set_hi(2); + load_id.set_lo(3); + { + PTabletWriterOpenRequest request; + request.set_allocated_id(&load_id); + request.set_index_id(4); + request.set_txn_id(1); + create_schema(desc_tbl, request.mutable_schema()); + for (int i = 0; i < 2; ++i) { + auto tablet = request.add_tablets(); + tablet->set_partition_id(10 + i); + tablet->set_tablet_id(20 + i); + } + request.set_num_senders(1); + request.set_need_gen_rollup(false); + auto st = mgr.open(request); + request.release_id(); + ASSERT_TRUE(st.ok()); + } + + // add a batch + { + PTabletWriterAddBatchRequest request; + request.set_allocated_id(&load_id); + request.set_index_id(4); + request.set_sender_id(0); + request.set_eos(true); + request.set_packet_seq(0); + + request.add_tablet_ids(20); + request.add_tablet_ids(21); + request.add_tablet_ids(20); + + RowBatch row_batch(row_desc, 1024, &tracker); + + // row1 + { + auto id = row_batch.add_row(); + auto tuple = (Tuple*)row_batch.tuple_data_pool()->allocate(tuple_desc->byte_size()); + row_batch.get_row(id)->set_tuple(0, tuple); + memset(tuple, 0, tuple_desc->byte_size()); + *(int*)tuple->get_slot(tuple_desc->slots()[0]->tuple_offset()) = 987654; + *(int64_t*)tuple->get_slot(tuple_desc->slots()[1]->tuple_offset()) = 1234567899876; + row_batch.commit_last_row(); + } + // row2 + { + auto id = row_batch.add_row(); + auto tuple = (Tuple*)row_batch.tuple_data_pool()->allocate(tuple_desc->byte_size()); + row_batch.get_row(id)->set_tuple(0, tuple); + memset(tuple, 0, tuple_desc->byte_size()); + *(int*)tuple->get_slot(tuple_desc->slots()[0]->tuple_offset()) = 12345678; + *(int64_t*)tuple->get_slot(tuple_desc->slots()[1]->tuple_offset()) = 9876567899876; + row_batch.commit_last_row(); + } + // row3 + { + auto id = row_batch.add_row(); + auto tuple = (Tuple*)row_batch.tuple_data_pool()->allocate(tuple_desc->byte_size()); + row_batch.get_row(id)->set_tuple(0, tuple); + memset(tuple, 0, tuple_desc->byte_size()); + *(int*)tuple->get_slot(tuple_desc->slots()[0]->tuple_offset()) = 876545678; + *(int64_t*)tuple->get_slot(tuple_desc->slots()[1]->tuple_offset()) = 76543234567; + row_batch.commit_last_row(); + } + row_batch.serialize(request.mutable_row_batch()); + add_status = OLAP_ERR_TABLE_NOT_FOUND; + google::protobuf::RepeatedPtrField tablet_vec; + auto st = mgr.add_batch(request, &tablet_vec); + request.release_id(); + ASSERT_FALSE(st.ok()); + } +} + +TEST_F(TabletWriterMgrTest, close_failed) { + ExecEnv env; + TabletWriterMgr mgr(&env); + + auto tdesc_tbl = create_descriptor_table(); + ObjectPool obj_pool; + DescriptorTbl* desc_tbl = nullptr; + DescriptorTbl::create(&obj_pool, tdesc_tbl, &desc_tbl); + auto tuple_desc = desc_tbl->get_tuple_descriptor(0); + RowDescriptor row_desc(*desc_tbl, {0}, {false}); + MemTracker tracker; + PUniqueId load_id; + load_id.set_hi(2); + load_id.set_lo(3); + { + PTabletWriterOpenRequest request; + request.set_allocated_id(&load_id); + request.set_index_id(4); + request.set_txn_id(1); + create_schema(desc_tbl, request.mutable_schema()); + for (int i = 0; i < 2; ++i) { + auto tablet = request.add_tablets(); + tablet->set_partition_id(10 + i); + tablet->set_tablet_id(20 + i); + } + request.set_num_senders(1); + request.set_need_gen_rollup(false); + auto st = mgr.open(request); + request.release_id(); + ASSERT_TRUE(st.ok()); + } + + // add a batch + { + PTabletWriterAddBatchRequest request; + request.set_allocated_id(&load_id); + request.set_index_id(4); + request.set_sender_id(0); + request.set_eos(true); + request.set_packet_seq(0); + + request.add_tablet_ids(20); + request.add_tablet_ids(21); + request.add_tablet_ids(20); + + request.add_partition_ids(10); + request.add_partition_ids(11); + + RowBatch row_batch(row_desc, 1024, &tracker); + + // row1 + { + auto id = row_batch.add_row(); + auto tuple = (Tuple*)row_batch.tuple_data_pool()->allocate(tuple_desc->byte_size()); + row_batch.get_row(id)->set_tuple(0, tuple); + memset(tuple, 0, tuple_desc->byte_size()); + *(int*)tuple->get_slot(tuple_desc->slots()[0]->tuple_offset()) = 987654; + *(int64_t*)tuple->get_slot(tuple_desc->slots()[1]->tuple_offset()) = 1234567899876; + row_batch.commit_last_row(); + } + // row2 + { + auto id = row_batch.add_row(); + auto tuple = (Tuple*)row_batch.tuple_data_pool()->allocate(tuple_desc->byte_size()); + row_batch.get_row(id)->set_tuple(0, tuple); + memset(tuple, 0, tuple_desc->byte_size()); + *(int*)tuple->get_slot(tuple_desc->slots()[0]->tuple_offset()) = 12345678; + *(int64_t*)tuple->get_slot(tuple_desc->slots()[1]->tuple_offset()) = 9876567899876; + row_batch.commit_last_row(); + } + // row3 + { + auto id = row_batch.add_row(); + auto tuple = (Tuple*)row_batch.tuple_data_pool()->allocate(tuple_desc->byte_size()); + row_batch.get_row(id)->set_tuple(0, tuple); + memset(tuple, 0, tuple_desc->byte_size()); + *(int*)tuple->get_slot(tuple_desc->slots()[0]->tuple_offset()) = 876545678; + *(int64_t*)tuple->get_slot(tuple_desc->slots()[1]->tuple_offset()) = 76543234567; + row_batch.commit_last_row(); + } + row_batch.serialize(request.mutable_row_batch()); + close_status = OLAP_ERR_TABLE_NOT_FOUND; + google::protobuf::RepeatedPtrField tablet_vec; + auto st = mgr.add_batch(request, &tablet_vec); + request.release_id(); + ASSERT_FALSE(st.ok()); + } +} + +TEST_F(TabletWriterMgrTest, unknown_tablet) { + ExecEnv env; + TabletWriterMgr mgr(&env); + + auto tdesc_tbl = create_descriptor_table(); + ObjectPool obj_pool; + DescriptorTbl* desc_tbl = nullptr; + DescriptorTbl::create(&obj_pool, tdesc_tbl, &desc_tbl); + auto tuple_desc = desc_tbl->get_tuple_descriptor(0); + RowDescriptor row_desc(*desc_tbl, {0}, {false}); + MemTracker tracker; + PUniqueId load_id; + load_id.set_hi(2); + load_id.set_lo(3); + { + PTabletWriterOpenRequest request; + request.set_allocated_id(&load_id); + request.set_index_id(4); + request.set_txn_id(1); + create_schema(desc_tbl, request.mutable_schema()); + for (int i = 0; i < 2; ++i) { + auto tablet = request.add_tablets(); + tablet->set_partition_id(10 + i); + tablet->set_tablet_id(20 + i); + } + request.set_num_senders(1); + request.set_need_gen_rollup(false); + auto st = mgr.open(request); + request.release_id(); + ASSERT_TRUE(st.ok()); + } + + // add a batch + { + PTabletWriterAddBatchRequest request; + request.set_allocated_id(&load_id); + request.set_index_id(4); + request.set_sender_id(0); + request.set_eos(true); + request.set_packet_seq(0); + + request.add_tablet_ids(20); + request.add_tablet_ids(22); + request.add_tablet_ids(20); + + RowBatch row_batch(row_desc, 1024, &tracker); + + // row1 + { + auto id = row_batch.add_row(); + auto tuple = (Tuple*)row_batch.tuple_data_pool()->allocate(tuple_desc->byte_size()); + row_batch.get_row(id)->set_tuple(0, tuple); + memset(tuple, 0, tuple_desc->byte_size()); + *(int*)tuple->get_slot(tuple_desc->slots()[0]->tuple_offset()) = 987654; + *(int64_t*)tuple->get_slot(tuple_desc->slots()[1]->tuple_offset()) = 1234567899876; + row_batch.commit_last_row(); + } + // row2 + { + auto id = row_batch.add_row(); + auto tuple = (Tuple*)row_batch.tuple_data_pool()->allocate(tuple_desc->byte_size()); + row_batch.get_row(id)->set_tuple(0, tuple); + memset(tuple, 0, tuple_desc->byte_size()); + *(int*)tuple->get_slot(tuple_desc->slots()[0]->tuple_offset()) = 12345678; + *(int64_t*)tuple->get_slot(tuple_desc->slots()[1]->tuple_offset()) = 9876567899876; + row_batch.commit_last_row(); + } + // row3 + { + auto id = row_batch.add_row(); + auto tuple = (Tuple*)row_batch.tuple_data_pool()->allocate(tuple_desc->byte_size()); + row_batch.get_row(id)->set_tuple(0, tuple); + memset(tuple, 0, tuple_desc->byte_size()); + *(int*)tuple->get_slot(tuple_desc->slots()[0]->tuple_offset()) = 876545678; + *(int64_t*)tuple->get_slot(tuple_desc->slots()[1]->tuple_offset()) = 76543234567; + row_batch.commit_last_row(); + } + row_batch.serialize(request.mutable_row_batch()); + google::protobuf::RepeatedPtrField tablet_vec; + auto st = mgr.add_batch(request, &tablet_vec); + request.release_id(); + ASSERT_FALSE(st.ok()); + } +} + +TEST_F(TabletWriterMgrTest, duplicate_packet) { + ExecEnv env; + TabletWriterMgr mgr(&env); + + auto tdesc_tbl = create_descriptor_table(); + ObjectPool obj_pool; + DescriptorTbl* desc_tbl = nullptr; + DescriptorTbl::create(&obj_pool, tdesc_tbl, &desc_tbl); + auto tuple_desc = desc_tbl->get_tuple_descriptor(0); + RowDescriptor row_desc(*desc_tbl, {0}, {false}); + MemTracker tracker; + PUniqueId load_id; + load_id.set_hi(2); + load_id.set_lo(3); + { + PTabletWriterOpenRequest request; + request.set_allocated_id(&load_id); + request.set_index_id(4); + request.set_txn_id(1); + create_schema(desc_tbl, request.mutable_schema()); + for (int i = 0; i < 2; ++i) { + auto tablet = request.add_tablets(); + tablet->set_partition_id(10 + i); + tablet->set_tablet_id(20 + i); + } + request.set_num_senders(1); + request.set_need_gen_rollup(false); + auto st = mgr.open(request); + request.release_id(); + ASSERT_TRUE(st.ok()); + } + + // add a batch + { + PTabletWriterAddBatchRequest request; + request.set_allocated_id(&load_id); + request.set_index_id(4); + request.set_sender_id(0); + request.set_eos(false); + request.set_packet_seq(0); + + request.add_tablet_ids(20); + request.add_tablet_ids(21); + request.add_tablet_ids(20); + + RowBatch row_batch(row_desc, 1024, &tracker); + + // row1 + { + auto id = row_batch.add_row(); + auto tuple = (Tuple*)row_batch.tuple_data_pool()->allocate(tuple_desc->byte_size()); + row_batch.get_row(id)->set_tuple(0, tuple); + memset(tuple, 0, tuple_desc->byte_size()); + *(int*)tuple->get_slot(tuple_desc->slots()[0]->tuple_offset()) = 987654; + *(int64_t*)tuple->get_slot(tuple_desc->slots()[1]->tuple_offset()) = 1234567899876; + row_batch.commit_last_row(); + } + // row2 + { + auto id = row_batch.add_row(); + auto tuple = (Tuple*)row_batch.tuple_data_pool()->allocate(tuple_desc->byte_size()); + row_batch.get_row(id)->set_tuple(0, tuple); + memset(tuple, 0, tuple_desc->byte_size()); + *(int*)tuple->get_slot(tuple_desc->slots()[0]->tuple_offset()) = 12345678; + *(int64_t*)tuple->get_slot(tuple_desc->slots()[1]->tuple_offset()) = 9876567899876; + row_batch.commit_last_row(); + } + // row3 + { + auto id = row_batch.add_row(); + auto tuple = (Tuple*)row_batch.tuple_data_pool()->allocate(tuple_desc->byte_size()); + row_batch.get_row(id)->set_tuple(0, tuple); + memset(tuple, 0, tuple_desc->byte_size()); + *(int*)tuple->get_slot(tuple_desc->slots()[0]->tuple_offset()) = 876545678; + *(int64_t*)tuple->get_slot(tuple_desc->slots()[1]->tuple_offset()) = 76543234567; + row_batch.commit_last_row(); + } + row_batch.serialize(request.mutable_row_batch()); + google::protobuf::RepeatedPtrField tablet_vec1; + auto st = mgr.add_batch(request, &tablet_vec1); + ASSERT_TRUE(st.ok()); + google::protobuf::RepeatedPtrField tablet_vec2; + st = mgr.add_batch(request, &tablet_vec2); + request.release_id(); + ASSERT_TRUE(st.ok()); + } + // close + { + PTabletWriterAddBatchRequest request; + request.set_allocated_id(&load_id); + request.set_index_id(4); + request.set_sender_id(0); + request.set_eos(true); + request.set_packet_seq(0); + google::protobuf::RepeatedPtrField tablet_vec; + auto st = mgr.add_batch(request, &tablet_vec); + request.release_id(); + ASSERT_TRUE(st.ok()); + } + // check content + ASSERT_EQ(_k_tablet_recorder[20], 2); + ASSERT_EQ(_k_tablet_recorder[21], 1); +} + +} + +int main(int argc, char* argv[]) { + ::testing::InitGoogleTest(&argc, argv); + palo::CpuInfo::init(); + return RUN_ALL_TESTS(); +} diff --git a/be/test/util/arena_test.cpp b/be/test/util/arena_test.cpp new file mode 100644 index 0000000000..60b58f8414 --- /dev/null +++ b/be/test/util/arena_test.cpp @@ -0,0 +1,68 @@ +// Copyright (c) 2011 The LevelDB Authors. All rights reserved. +// Use of this source code is governed by a BSD-style license that can be +// found in the LICENSE file. See the AUTHORS file for names of contributors. + +#include +#include "util/arena.h" +#include "util/random.h" + +namespace palo { + +class ArenaTest : public testing::Test { }; + +TEST_F(ArenaTest, Empty) { + Arena arena; +} + +TEST_F(ArenaTest, Simple) { + std::vector > allocated; + Arena arena; + const int N = 100000; + size_t bytes = 0; + Random rnd(301); + for (int i = 0; i < N; i++) { + size_t s; + if (i % (N / 10) == 0) { + s = i; + } else { + s = rnd.OneIn(4000) ? rnd.Uniform(6000) : + (rnd.OneIn(10) ? rnd.Uniform(100) : rnd.Uniform(20)); + } + if (s == 0) { + // Our arena disallows size 0 allocations. + s = 1; + } + char* r; + if (rnd.OneIn(10)) { + r = arena.AllocateAligned(s); + } else { + r = arena.Allocate(s); + } + + for (size_t b = 0; b < s; b++) { + // Fill the "i"th allocation with a known bit pattern + r[b] = i % 256; + } + bytes += s; + allocated.push_back(std::make_pair(s, r)); + ASSERT_GE(arena.MemoryUsage(), bytes); + if (i > N / 10) { + ASSERT_LE(arena.MemoryUsage(), bytes * 1.10); + } + } + for (size_t i = 0; i < allocated.size(); i++) { + size_t num_bytes = allocated[i].first; + const char* p = allocated[i].second; + for (size_t b = 0; b < num_bytes; b++) { + // Check the "i"th allocation for the known bit pattern + ASSERT_EQ(int(p[b]) & 0xff, i % 256); + } + } +} + +} // namespace palo + +int main(int argc, char** argv) { + ::testing::InitGoogleTest(&argc, argv); + return RUN_ALL_TESTS(); +} diff --git a/be/test/util/byte_buffer_test2.cpp b/be/test/util/byte_buffer_test2.cpp new file mode 100644 index 0000000000..843d4713e7 --- /dev/null +++ b/be/test/util/byte_buffer_test2.cpp @@ -0,0 +1,56 @@ +// Copyright (c) 2017, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 "util/byte_buffer.h" + +#include + +#include "common/logging.h" + +namespace palo { + +class ByteBufferTest : public testing::Test { +public: + ByteBufferTest() { } + virtual ~ByteBufferTest() { } +}; + +TEST_F(ByteBufferTest, normal) { + auto buf = ByteBuffer::allocate(4); + ASSERT_EQ(0, buf->pos); + ASSERT_EQ(4, buf->limit); + ASSERT_EQ(4, buf->capacity); + + char test[] = {1, 2, 3}; + buf->put_bytes(test, 3); + + ASSERT_EQ(3, buf->pos); + ASSERT_EQ(4, buf->limit); + ASSERT_EQ(4, buf->capacity); + + ASSERT_EQ(1, buf->remaining()); + buf->flip(); + ASSERT_EQ(0, buf->pos); + ASSERT_EQ(3, buf->limit); + ASSERT_EQ(4, buf->capacity); + ASSERT_EQ(3, buf->remaining()); +} + +} + +int main(int argc, char* argv[]) { + ::testing::InitGoogleTest(&argc, argv); + return RUN_ALL_TESTS(); +} diff --git a/be/test/util/descriptor_helper.h b/be/test/util/descriptor_helper.h new file mode 100644 index 0000000000..d0374ca726 --- /dev/null +++ b/be/test/util/descriptor_helper.h @@ -0,0 +1,155 @@ +// Copyright (c) 2017, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 "gen_cpp/Descriptors_types.h" +#include "gen_cpp/Types_types.h" + +namespace palo { + +class TDescriptorTableBuilder { +public: + TSlotId next_slot_id() { return _next_slot_id++; } + TTupleId next_tuple_id() { return _next_tuple_id++; } + + void add_slots(const std::vector& slots) { + _desc_tbl.__isset.slotDescriptors = true; + _desc_tbl.slotDescriptors.insert( + _desc_tbl.slotDescriptors.end(), slots.begin(), slots.end()); + } + void add_tuple(const TTupleDescriptor& tuple) { + _desc_tbl.tupleDescriptors.push_back(tuple); + } + + TDescriptorTable desc_tbl() { return _desc_tbl; } +private: + TSlotId _next_slot_id = 0; + TTupleId _next_tuple_id = 0; + TDescriptorTable _desc_tbl; +}; + +class TTupleDescriptorBuilder; +class TSlotDescriptorBuilder { +public: + TTypeDesc get_common_type(TPrimitiveType::type type) { + TTypeNode node; + node.type = TTypeNodeType::SCALAR; + node.__isset.scalar_type = true; + node.scalar_type.type = type; + + TTypeDesc type_desc; + type_desc.types.push_back(node); + + return type_desc; + } + + TSlotDescriptorBuilder() { _slot_desc.isMaterialized = true; } + TSlotDescriptorBuilder& type(PrimitiveType type) { + _slot_desc.slotType = get_common_type(to_thrift(type)); + return *this; + } + TSlotDescriptorBuilder& string_type(int len) { + _slot_desc.slotType = get_common_type(to_thrift(TYPE_VARCHAR)); + _slot_desc.slotType.types[0].scalar_type.__set_len(len); + return *this; + } + TSlotDescriptorBuilder& decimal_type(int precision, int scale) { + _slot_desc.slotType = get_common_type(to_thrift(TYPE_DECIMAL)); + _slot_desc.slotType.types[0].scalar_type.__set_precision(precision); + _slot_desc.slotType.types[0].scalar_type.__set_scale(scale); + return *this; + } + TSlotDescriptorBuilder& nullable(bool nullable) { + _slot_desc.nullIndicatorByte = (nullable) ? 0 : -1; + return *this; + } + TSlotDescriptorBuilder& is_materialized(bool is_materialized) { + _slot_desc.isMaterialized = is_materialized; + return *this; + } + TSlotDescriptorBuilder& column_name(const std::string& name) { + _slot_desc.colName = name; + return *this; + } + TSlotDescriptorBuilder& column_pos(int column_pos) { + _slot_desc.columnPos = column_pos; + return *this; + } + TSlotDescriptor build() { + return _slot_desc; + } +private: + friend TTupleDescriptorBuilder; + TSlotDescriptor _slot_desc; +}; + +class TTupleDescriptorBuilder { +public: + TTupleDescriptorBuilder& add_slot(const TSlotDescriptor& slot_desc) { + _slot_descs.push_back(slot_desc); + return *this; + } + void build(TDescriptorTableBuilder* tb) { + // build slot desc + _tuple_id = tb->next_tuple_id(); + int num_nullables = 0; + for (int i = 0; i < _slot_descs.size(); ++i) { + auto& slot_desc = _slot_descs[i]; + if (slot_desc.nullIndicatorByte >= 0) { + num_nullables++; + } + } + int null_byetes = (num_nullables + 7) / 8; + int offset = null_byetes; + int null_offset = 0; + for (int i = 0; i < _slot_descs.size(); ++i) { + auto& slot_desc = _slot_descs[i]; + int size = get_slot_size( + thrift_to_type(slot_desc.slotType.types[0].scalar_type.type)); + int align = (size > 16) ? 16 : size; + offset = ((offset + align - 1) / align) * align; + slot_desc.id = tb->next_slot_id(); + slot_desc.parent = _tuple_id; + slot_desc.byteOffset = offset; + offset += size; + if (slot_desc.nullIndicatorByte >= 0) { + slot_desc.nullIndicatorBit = null_offset % 8; + slot_desc.nullIndicatorByte = null_offset / 8; + null_offset++; + } else { + slot_desc.nullIndicatorByte = 0; + slot_desc.nullIndicatorBit = -1; + } + slot_desc.slotIdx = i; + } + + _tuple_desc.id = _tuple_id; + _tuple_desc.byteSize = offset; + _tuple_desc.numNullBytes = null_byetes; + _tuple_desc.numNullSlots = _slot_descs.size(); + + tb->add_slots(_slot_descs); + tb->add_tuple(_tuple_desc); + } +private: + TTupleId _tuple_id; + std::vector _slot_descs; + TTupleDescriptor _tuple_desc; +}; + +} diff --git a/be/test/util/json_util_test.cpp b/be/test/util/json_util_test.cpp new file mode 100644 index 0000000000..d95f89314e --- /dev/null +++ b/be/test/util/json_util_test.cpp @@ -0,0 +1,70 @@ +// Copyright (c) 2017, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 "util/json_util.h" + +#include + +#include "common/logging.h" + +namespace palo { + +class JsonUtilTest : public testing::Test { +public: + JsonUtilTest() { } + virtual ~JsonUtilTest() { } +}; + +TEST_F(JsonUtilTest, success) { + Status status; + + auto str = to_json(status); + + const char* result = "{\n" + " \"status\": \"Success\",\n" + " \"msg\": \"OK\"\n}"; + ASSERT_STREQ(result, str.c_str()); +} + +TEST_F(JsonUtilTest, normal_fail) { + Status status("so bad"); + + auto str = to_json(status); + + const char* result = "{\n" + " \"status\": \"Fail\",\n" + " \"msg\": \"so bad\"\n}"; + ASSERT_STREQ(result, str.c_str()); +} + +TEST_F(JsonUtilTest, normal_fail_str) { + Status status("\"so bad\""); + + auto str = to_json(status); + + // "msg": "\"so bad\"" + const char* result = "{\n" + " \"status\": \"Fail\",\n" + " \"msg\": \"\\\"so bad\\\"\"\n}"; + LOG(INFO) << "str: " << str; + ASSERT_STREQ(result, str.c_str()); +} + +} + +int main(int argc, char* argv[]) { + ::testing::InitGoogleTest(&argc, argv); + return RUN_ALL_TESTS(); +} diff --git a/be/test/util/string_util_test.cpp b/be/test/util/string_util_test.cpp new file mode 100644 index 0000000000..6586d5b178 --- /dev/null +++ b/be/test/util/string_util_test.cpp @@ -0,0 +1,78 @@ +// Copyright (c) 2017, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 "util/string_util.h" + +#include + +#include "util/cpu_info.h" + +namespace palo { + +class StringUtilTest : public testing::Test { +public: + StringUtilTest() { } + virtual ~StringUtilTest() { + } +}; + +TEST_F(StringUtilTest, normal) { + { + StringCaseSet test_set; + test_set.emplace("AbC"); + test_set.emplace("AbCD"); + test_set.emplace("AbCE"); + ASSERT_EQ(1, test_set.count("abc")); + ASSERT_EQ(1, test_set.count("abcd")); + ASSERT_EQ(1, test_set.count("abce")); + ASSERT_EQ(0, test_set.count("ab")); + } + { + StringCaseUnorderedSet test_set; + test_set.emplace("AbC"); + test_set.emplace("AbCD"); + test_set.emplace("AbCE"); + ASSERT_EQ(1, test_set.count("abc")); + ASSERT_EQ(0, test_set.count("ab")); + } + { + StringCaseMap test_map; + test_map.emplace("AbC", 123); + test_map.emplace("AbCD", 234); + test_map.emplace("AbCE", 345); + ASSERT_EQ(123, test_map["abc"]); + ASSERT_EQ(234, test_map["aBcD"]); + ASSERT_EQ(345, test_map["abcE"]); + ASSERT_EQ(0, test_map.count("ab")); + } + { + StringCaseUnorderedMap test_map; + test_map.emplace("AbC", 123); + test_map.emplace("AbCD", 234); + test_map.emplace("AbCE", 345); + ASSERT_EQ(123, test_map["abc"]); + ASSERT_EQ(234, test_map["aBcD"]); + ASSERT_EQ(345, test_map["abcE"]); + ASSERT_EQ(0, test_map.count("ab")); + } +} + +} + +int main(int argc, char** argv) { + ::testing::InitGoogleTest(&argc, argv); + palo::CpuInfo::init(); + return RUN_ALL_TESTS(); +} diff --git a/be/test/util/uid_util_test.cpp b/be/test/util/uid_util_test.cpp new file mode 100644 index 0000000000..7c82c26e86 --- /dev/null +++ b/be/test/util/uid_util_test.cpp @@ -0,0 +1,98 @@ +// Modifications copyright (C) 2017, Baidu.com, Inc. +// Copyright 2017 The Apache Software Foundation + +// 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 "util/uid_util.h" + +#include +#include + +namespace palo { +class UidUtilTest : public testing::Test { +public: + UidUtilTest() { } + virtual ~UidUtilTest() { + } +}; + +TEST_F(UidUtilTest, UniqueId) { + { + UniqueId id; + std::string hex_str = id.to_string(); + ASSERT_STRNE("0000000000000000:0000000000000000", hex_str.c_str()); + } + { + UniqueId id(123456789, 987654321); + std::string hex_str = id.to_string(); + ASSERT_STREQ("00000000075BCD15:000000003ADE68B1", hex_str.c_str()); + } + { + PUniqueId puid; + puid.set_hi(12345678987654321); + puid.set_lo(98765432123456789); + UniqueId id(puid); + std::string hex_str = id.to_string(); + ASSERT_STREQ("002BDC546291F4B1:015EE2A321CE7D15", hex_str.c_str()); + } + { + TUniqueId tuid; + tuid.__set_hi(12345678987654321); + tuid.__set_lo(98765432123456789); + UniqueId id(tuid); + std::string hex_str = id.to_string(); + ASSERT_STREQ("002BDC546291F4B1:015EE2A321CE7D15", hex_str.c_str()); + } + { + TUniqueId tuid; + tuid.__set_hi(12345678987654321); + tuid.__set_lo(98765432123456789); + std::stringstream ss; + ss << UniqueId(tuid); + ASSERT_STREQ("002BDC546291F4B1:015EE2A321CE7D15", ss.str().c_str()); + } +} + +TEST_F(UidUtilTest, Hash) { + std::hash hasher; + UniqueId uid(1, 2); + { + TUniqueId tuid; + tuid.__set_hi(1); + tuid.__set_lo(2); + + ASSERT_EQ(hasher(uid), hasher(tuid)); + ASSERT_TRUE(uid == UniqueId(tuid)); + } + { + TUniqueId tuid; + tuid.__set_hi(1); + tuid.__set_lo(1); + + ASSERT_NE(hasher(uid), hasher(tuid)); + } +} + +} + +int main(int argc, char** argv) { + ::testing::InitGoogleTest(&argc, argv); + palo::CpuInfo::init(); + return RUN_ALL_TESTS(); +} + diff --git a/fe/build.xml.deprecated b/fe/build.xml.deprecated deleted file mode 100644 index c156590768..0000000000 --- a/fe/build.xml.deprecated +++ /dev/null @@ -1,247 +0,0 @@ - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - diff --git a/fe/src/main/java/com/baidu/palo/analysis/AdminShowReplicaDistributionStmt.java b/fe/src/main/java/com/baidu/palo/analysis/AdminShowReplicaDistributionStmt.java new file mode 100644 index 0000000000..6f5c78d5b6 --- /dev/null +++ b/fe/src/main/java/com/baidu/palo/analysis/AdminShowReplicaDistributionStmt.java @@ -0,0 +1,93 @@ +// Copyright (c) 2018, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 com.baidu.palo.analysis; + +import com.baidu.palo.catalog.Catalog; +import com.baidu.palo.catalog.Column; +import com.baidu.palo.catalog.ColumnType; +import com.baidu.palo.cluster.ClusterNamespace; +import com.baidu.palo.common.AnalysisException; +import com.baidu.palo.common.ErrorCode; +import com.baidu.palo.common.ErrorReport; +import com.baidu.palo.common.UserException; +import com.baidu.palo.mysql.privilege.PrivPredicate; +import com.baidu.palo.qe.ConnectContext; +import com.baidu.palo.qe.ShowResultSetMetaData; + +import com.google.common.base.Strings; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; + +import java.util.List; + +public class AdminShowReplicaDistributionStmt extends ShowStmt { + public static final ImmutableList TITLE_NAMES = new ImmutableList.Builder() + .add("BackendId").add("ReplicaNum").add("Graph").add("Percent").build(); + + private TableRef tblRef; + private List partitions = Lists.newArrayList(); + + public AdminShowReplicaDistributionStmt(TableRef tblRef) { + this.tblRef = tblRef; + } + + @Override + public void analyze(Analyzer analyzer) throws AnalysisException, UserException { + super.analyze(analyzer); + + // check auth + if (!Catalog.getCurrentCatalog().getAuth().checkGlobalPriv(ConnectContext.get(), PrivPredicate.ADMIN)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, "ADMIN"); + } + + String dbName = null; + if (Strings.isNullOrEmpty(tblRef.getName().getDb())) { + dbName = analyzer.getDefaultDb(); + if (Strings.isNullOrEmpty(dbName)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_NO_DB_ERROR); + } + } else { + dbName = ClusterNamespace.getFullName(getClusterName(), tblRef.getName().getDb()); + } + + tblRef.getName().setDb(dbName); + + if (tblRef.getPartitions() != null && !tblRef.getPartitions().isEmpty()) { + partitions.addAll(tblRef.getPartitions()); + } + } + + public String getDbName() { + return tblRef.getName().getDb(); + } + + public String getTblName() { + return tblRef.getName().getTbl(); + } + + public List getPartitions() { + return partitions; + } + + @Override + public ShowResultSetMetaData getMetaData() { + ShowResultSetMetaData.Builder builder = ShowResultSetMetaData.builder(); + for (String title : TITLE_NAMES) { + builder.addColumn(new Column(title, ColumnType.createVarchar(30))); + } + return builder.build(); + } +} diff --git a/fe/src/main/java/com/baidu/palo/analysis/AdminShowReplicaStatusStmt.java b/fe/src/main/java/com/baidu/palo/analysis/AdminShowReplicaStatusStmt.java new file mode 100644 index 0000000000..2fded5b2a1 --- /dev/null +++ b/fe/src/main/java/com/baidu/palo/analysis/AdminShowReplicaStatusStmt.java @@ -0,0 +1,161 @@ +// Copyright (c) 2018, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 com.baidu.palo.analysis; + +import com.baidu.palo.analysis.BinaryPredicate.Operator; +import com.baidu.palo.catalog.Catalog; +import com.baidu.palo.catalog.Column; +import com.baidu.palo.catalog.ColumnType; +import com.baidu.palo.catalog.Replica.ReplicaStatus; +import com.baidu.palo.cluster.ClusterNamespace; +import com.baidu.palo.common.AnalysisException; +import com.baidu.palo.common.ErrorCode; +import com.baidu.palo.common.ErrorReport; +import com.baidu.palo.common.UserException; +import com.baidu.palo.mysql.privilege.PrivPredicate; +import com.baidu.palo.qe.ConnectContext; +import com.baidu.palo.qe.ShowResultSetMetaData; + +import com.google.common.base.Strings; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; + +import java.util.List; + +public class AdminShowReplicaStatusStmt extends ShowStmt { + public static final ImmutableList TITLE_NAMES = new ImmutableList.Builder() + .add("TabletId").add("ReplicaId").add("BackendId").add("Version").add("LastFailedVersion") + .add("LastSuccessVersion").add("CommittedVersion").add("VersionNum") + .add("State").add("Status") + .build(); + + private TableRef tblRef; + private Expr where; + private List partitions = Lists.newArrayList(); + + private Operator op; + private ReplicaStatus statusFilter; + + public AdminShowReplicaStatusStmt(TableRef tblRef, Expr where) { + this.tblRef = tblRef; + this.where = where; + } + + @Override + public void analyze(Analyzer analyzer) throws AnalysisException, UserException { + super.analyze(analyzer); + + // check auth + if (!Catalog.getCurrentCatalog().getAuth().checkGlobalPriv(ConnectContext.get(), PrivPredicate.ADMIN)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, "ADMIN"); + } + + String dbName = null; + if (Strings.isNullOrEmpty(tblRef.getName().getDb())) { + dbName = analyzer.getDefaultDb(); + if (Strings.isNullOrEmpty(dbName)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_NO_DB_ERROR); + } + } else { + dbName = ClusterNamespace.getFullName(getClusterName(), tblRef.getName().getDb()); + } + + tblRef.getName().setDb(dbName); + + if (tblRef.getPartitions() != null && !tblRef.getPartitions().isEmpty()) { + partitions.addAll(tblRef.getPartitions()); + } + + if (!analyzeWhere()) { + throw new AnalysisException( + "Where clause should looks like: status =/!= 'OK/DEAD/VERSION_ERROR/MISSING'"); + } + } + + private boolean analyzeWhere() throws AnalysisException { + // analyze where clause if not null + if (where == null) { + return true; + } + + if (!(where instanceof BinaryPredicate)) { + return false; + } + + BinaryPredicate binaryPredicate = (BinaryPredicate) where; + op = binaryPredicate.getOp(); + if (op != Operator.EQ && op != Operator.NE) { + return false; + } + + Expr leftChild = binaryPredicate.getChild(0); + if (!(leftChild instanceof SlotRef)) { + return false; + } + + String leftKey = ((SlotRef) leftChild).getColumnName(); + if (!leftKey.equalsIgnoreCase("status")) { + return false; + } + + Expr rightChild = binaryPredicate.getChild(1); + if (!(rightChild instanceof StringLiteral)) { + return false; + } + + try { + statusFilter = ReplicaStatus.valueOf(((StringLiteral) rightChild).getStringValue().toUpperCase()); + } catch (Exception e) { + return false; + } + + if (statusFilter == null) { + return false; + } + + return true; + } + + public String getDbName() { + return tblRef.getName().getDb(); + } + + public String getTblName() { + return tblRef.getName().getTbl(); + } + + public List getPartitions() { + return partitions; + } + + public Operator getOp() { + return op; + } + + public ReplicaStatus getStatusFilter() { + return statusFilter; + } + + @Override + public ShowResultSetMetaData getMetaData() { + ShowResultSetMetaData.Builder builder = ShowResultSetMetaData.builder(); + for (String title : TITLE_NAMES) { + builder.addColumn(new Column(title, ColumnType.createVarchar(30))); + } + return builder.build(); + } + +} diff --git a/fe/src/main/java/com/baidu/palo/analysis/ImportColumnDesc.java b/fe/src/main/java/com/baidu/palo/analysis/ImportColumnDesc.java new file mode 100644 index 0000000000..48fc185507 --- /dev/null +++ b/fe/src/main/java/com/baidu/palo/analysis/ImportColumnDesc.java @@ -0,0 +1,30 @@ +package com.baidu.palo.analysis; + +/** + * Created by zhaochun on 2018/4/23. + */ +public class ImportColumnDesc { + private String column; + private Expr expr; + + public ImportColumnDesc(String column) { + this.column = column; + } + + public ImportColumnDesc(String column, Expr expr) { + this.column = column; + this.expr = expr; + } + + public String getColumn() { + return column; + } + + public Expr getExpr() { + return expr; + } + + public boolean isColumn() { + return expr == null; + } +} diff --git a/fe/src/main/java/com/baidu/palo/analysis/ImportColumnsStmt.java b/fe/src/main/java/com/baidu/palo/analysis/ImportColumnsStmt.java new file mode 100644 index 0000000000..b6ecd15553 --- /dev/null +++ b/fe/src/main/java/com/baidu/palo/analysis/ImportColumnsStmt.java @@ -0,0 +1,23 @@ +package com.baidu.palo.analysis; + +import java.util.List; + +/** + * Created by zhaochun on 2018/4/23. + */ +public class ImportColumnsStmt extends StatementBase { + private List columns; + + public ImportColumnsStmt(List columns) { + this.columns = columns; + } + + public List getColumns() { + return columns; + } + + @Override + public RedirectStatus getRedirectStatus() { + return null; + } +} diff --git a/fe/src/main/java/com/baidu/palo/analysis/ImportWhereStmt.java b/fe/src/main/java/com/baidu/palo/analysis/ImportWhereStmt.java new file mode 100644 index 0000000000..a2b11abf82 --- /dev/null +++ b/fe/src/main/java/com/baidu/palo/analysis/ImportWhereStmt.java @@ -0,0 +1,21 @@ +package com.baidu.palo.analysis; + +/** + * Created by zhaochun on 2018/4/24. + */ +public class ImportWhereStmt extends StatementBase { + private Expr expr; + + public ImportWhereStmt(Expr expr) { + this.expr = expr; + } + + public Expr getExpr() { + return expr; + } + + @Override + public RedirectStatus getRedirectStatus() { + return null; + } +} diff --git a/fe/src/main/java/com/baidu/palo/catalog/EsTable.java b/fe/src/main/java/com/baidu/palo/catalog/EsTable.java new file mode 100644 index 0000000000..3f2661a20b --- /dev/null +++ b/fe/src/main/java/com/baidu/palo/catalog/EsTable.java @@ -0,0 +1,225 @@ +// Copyright (c) 2017, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 com.baidu.palo.catalog; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.io.UnsupportedEncodingException; +import java.util.List; +import java.util.Map; +import java.util.zip.Adler32; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import com.baidu.palo.analysis.CreateTableStmt; +import com.baidu.palo.analysis.TableName; +import com.baidu.palo.common.DdlException; +import com.baidu.palo.common.io.Text; +import com.baidu.palo.external.EsTableState; +import com.baidu.palo.thrift.TEsTable; +import com.baidu.palo.thrift.TTableDescriptor; +import com.baidu.palo.thrift.TTableType; +import com.google.common.base.Strings; +import com.google.common.collect.Maps; + +public class EsTable extends Table { + private static final Logger LOG = LogManager.getLogger(EsTable.class); + + public static final String HOSTS = "hosts"; + public static final String USER = "user"; + public static final String PASSWORD = "password"; + public static final String INDEX = "index"; + public static final String TYPE = "type"; + + private String hosts; + private String[] seeds; + private String userName; + private String passwd; + private String indexName; + private String mappingType = "doc"; + // only save the partition definition, save the partition key, + // partition list is got from es cluster dynamically and is saved in esTableState + private PartitionInfo partitionInfo; + private EsTableState esTableState; + + public EsTable() { + super(TableType.ELASTICSEARCH); + } + + public EsTable(long id, String name, List schema, + Map properties, PartitionInfo partitionInfo) + throws DdlException { + super(id, name, TableType.ELASTICSEARCH, schema); + this.partitionInfo = partitionInfo; + validate(properties); + } + + private void validate(Map properties) throws DdlException { + if (properties == null) { + throw new DdlException("Please set properties of elasticsearch table, " + + "they are: hosts, thrift_port, http_port, user, password, index"); + } + + hosts = properties.get(HOSTS); + if (Strings.isNullOrEmpty(hosts)) { + throw new DdlException("Hosts of ES table is null. " + + "Please add properties('hosts'='xxx.xxx.xxx.xxx,xxx.xxx.xxx.xxx') when create table"); + } + seeds = hosts.split(","); + // TODO(ygl) validate the seeds? + + userName = properties.get(USER); + if (Strings.isNullOrEmpty(userName)) { + userName = ""; + } + + passwd = properties.get(PASSWORD); + if (passwd == null) { + passwd = ""; + } + + indexName = properties.get(INDEX); + if (Strings.isNullOrEmpty(indexName)) { + throw new DdlException("Index of ES table is null. " + + "Please add properties('index'='xxxx') when create table"); + } + + mappingType = properties.get(TYPE); + if (Strings.isNullOrEmpty(mappingType)) { + mappingType = "docs"; + } + } + + public TTableDescriptor toThrift() { + TEsTable tEsTable = new TEsTable(); + TTableDescriptor tTableDescriptor = new TTableDescriptor(getId(), TTableType.ES_TABLE, + baseSchema.size(), 0, getName(), ""); + tTableDescriptor.setEsTable(tEsTable); + return tTableDescriptor; + } + + @Override + public CreateTableStmt toCreateTableStmt(String dbName) { + Map properties = Maps.newHashMap(); + properties.put(HOSTS, hosts); + properties.put(USER, userName); + properties.put(PASSWORD, passwd); + properties.put(INDEX, indexName); + properties.put(TYPE, mappingType); + + CreateTableStmt stmt = new CreateTableStmt(false, true, new TableName(dbName, name), baseSchema, + type.name(), null, null, null, properties, null); + return stmt; + } + + @Override + public int getSignature(int signatureVersion) { + Adler32 adler32 = new Adler32(); + adler32.update(signatureVersion); + String charsetName = "UTF-8"; + + try { + // name + adler32.update(name.getBytes(charsetName)); + // type + adler32.update(type.name().getBytes(charsetName)); + // host + adler32.update(hosts.getBytes(charsetName)); + // username + adler32.update(userName.getBytes(charsetName)); + // passwd + adler32.update(passwd.getBytes(charsetName)); + // mysql db + adler32.update(indexName.getBytes(charsetName)); + // mysql table + adler32.update(mappingType.getBytes(charsetName)); + + } catch (UnsupportedEncodingException e) { + LOG.error("encoding error", e); + return -1; + } + + return Math.abs((int) adler32.getValue()); + } + + @Override + public void write(DataOutput out) throws IOException { + super.write(out); + Text.writeString(out, hosts); + Text.writeString(out, userName); + Text.writeString(out, passwd); + Text.writeString(out, indexName); + Text.writeString(out, mappingType); + Text.writeString(out, partitionInfo.getType().name()); + partitionInfo.write(out); + } + + @Override + public void readFields(DataInput in) throws IOException { + super.readFields(in); + hosts = Text.readString(in); + userName = Text.readString(in); + passwd = Text.readString(in); + indexName = Text.readString(in); + mappingType = Text.readString(in); + PartitionType partType = PartitionType.valueOf(Text.readString(in)); + if (partType == PartitionType.UNPARTITIONED) { + partitionInfo = SinglePartitionInfo.read(in); + } else if (partType == PartitionType.RANGE) { + partitionInfo = RangePartitionInfo.read(in); + } else { + throw new IOException("invalid partition type: " + partType); + } + } + + public String getHosts() { + return hosts; + } + + public String[] getSeeds() { + return seeds; + } + + public String getUserName() { + return userName; + } + + public String getPasswd() { + return passwd; + } + + public String getIndexName() { + return indexName; + } + + public String getMappingType() { + return mappingType; + } + + public PartitionInfo getPartitionInfo() { + return partitionInfo; + } + + public EsTableState getEsTableState() { + return esTableState; + } + + public void setEsTableState(EsTableState esTableState) { + this.esTableState = esTableState; + } +} diff --git a/fe/src/main/java/com/baidu/palo/catalog/MetadataViewer.java b/fe/src/main/java/com/baidu/palo/catalog/MetadataViewer.java new file mode 100644 index 0000000000..eef45d240c --- /dev/null +++ b/fe/src/main/java/com/baidu/palo/catalog/MetadataViewer.java @@ -0,0 +1,245 @@ +// Copyright (c) 2018, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 com.baidu.palo.catalog; + +import com.baidu.palo.analysis.AdminShowReplicaDistributionStmt; +import com.baidu.palo.analysis.AdminShowReplicaStatusStmt; +import com.baidu.palo.analysis.BinaryPredicate.Operator; +import com.baidu.palo.catalog.Replica.ReplicaStatus; +import com.baidu.palo.catalog.Table.TableType; +import com.baidu.palo.common.DdlException; +import com.baidu.palo.system.Backend; +import com.baidu.palo.system.SystemInfoService; + +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; + +import java.text.DecimalFormat; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +public class MetadataViewer { + + public static List> getTabletStatus(AdminShowReplicaStatusStmt stmt) throws DdlException { + return getTabletStatus(stmt.getDbName(), stmt.getTblName(), stmt.getPartitions(), + stmt.getStatusFilter(), stmt.getOp()); + } + + private static List> getTabletStatus(String dbName, String tblName, List partitions, + ReplicaStatus statusFilter, Operator op) throws DdlException { + List> result = Lists.newArrayList(); + + Catalog catalog = Catalog.getCurrentCatalog(); + SystemInfoService infoService = Catalog.getCurrentSystemInfo(); + + Database db = catalog.getDb(dbName); + if (db == null) { + throw new DdlException("Database " + dbName + " does not exsit"); + } + + db.readLock(); + try { + Table tbl = db.getTable(tblName); + if (tbl == null || tbl.getType() != TableType.OLAP) { + throw new DdlException("Table does not exist or is not OLAP table: " + tblName); + } + + OlapTable olapTable = (OlapTable) tbl; + + if (partitions.isEmpty()) { + partitions.addAll(olapTable.getPartitionNames()); + } else { + // check partition + for (String partName : partitions) { + Partition partition = olapTable.getPartition(partName); + if (partition == null) { + throw new DdlException("Partition does not exist: " + partName); + } + } + } + + for (String partName : partitions) { + Partition partition = olapTable.getPartition(partName); + long committedVersion = partition.getCommittedVersion(); + short replicationNum = olapTable.getPartitionInfo().getReplicationNum(partition.getId()); + + for (MaterializedIndex index : partition.getMaterializedIndices()) { + for (Tablet tablet : index.getTablets()) { + long tabletId = tablet.getId(); + int count = replicationNum; + for (Replica replica : tablet.getReplicas()) { + --count; + List row = Lists.newArrayList(); + + ReplicaStatus status = ReplicaStatus.OK; + Backend be = infoService.getBackend(replica.getBackendId()); + if (be == null || !be.isAvailable()) { + status = ReplicaStatus.DEAD; + } else { + if (replica.getVersion() < committedVersion + || replica.getLastFailedVersion() > 0) { + status = ReplicaStatus.VERSION_ERROR; + } + } + + if (filterReplica(status, statusFilter, op)) { + continue; + } + + row.add(String.valueOf(tabletId)); + row.add(String.valueOf(replica.getId())); + row.add(String.valueOf(replica.getBackendId())); + row.add(String.valueOf(replica.getVersion())); + row.add(String.valueOf(replica.getLastFailedVersion())); + row.add(String.valueOf(replica.getLastSuccessVersion())); + row.add(String.valueOf(committedVersion)); + row.add(String.valueOf(replica.getVersionCount())); + row.add(replica.getState().name()); + row.add(status.name()); + result.add(row); + } + + if (filterReplica(ReplicaStatus.MISSING, statusFilter, op)) { + continue; + } + + // get missing replicas + for (int i = 0; i < count; ++i) { + List row = Lists.newArrayList(); + row.add(String.valueOf(tabletId)); + row.add("-1"); + row.add("-1"); + row.add("-1"); + row.add("-1"); + row.add("-1"); + row.add("-1"); + row.add("-1"); + row.add("N/A"); + row.add(ReplicaStatus.MISSING.name()); + result.add(row); + } + } + } + } + } finally { + db.readUnlock(); + } + + return result; + } + + private static boolean filterReplica(ReplicaStatus status, ReplicaStatus statusFilter, Operator op) { + if (statusFilter == null) { + return false; + } + if (op == Operator.EQ) { + return status != statusFilter; + } else { + return status == statusFilter; + } + } + + public static List> getTabletDistribution(AdminShowReplicaDistributionStmt stmt) throws DdlException { + return getTabletDistribution(stmt.getDbName(), stmt.getTblName(), stmt.getPartitions()); + } + + private static List> getTabletDistribution(String dbName, String tblName, List partitions) + throws DdlException { + DecimalFormat df = new DecimalFormat("##.00 %"); + + List> result = Lists.newArrayList(); + + Catalog catalog = Catalog.getCurrentCatalog(); + SystemInfoService infoService = Catalog.getCurrentSystemInfo(); + + Database db = catalog.getDb(dbName); + if (db == null) { + throw new DdlException("Database " + dbName + " does not exsit"); + } + + db.readLock(); + try { + Table tbl = db.getTable(tblName); + if (tbl == null || tbl.getType() != TableType.OLAP) { + throw new DdlException("Table does not exist or is not OLAP table: " + tblName); + } + + OlapTable olapTable = (OlapTable) tbl; + + if (partitions.isEmpty()) { + partitions.addAll(olapTable.getPartitionNames()); + } else { + // check partition + for (String partName : partitions) { + Partition partition = olapTable.getPartition(partName); + if (partition == null) { + throw new DdlException("Partition does not exist: " + partName); + } + } + } + + // backend id -> replica count + Map countMap = Maps.newHashMap(); + // init map + List beIds = infoService.getBackendIds(false); + for (long beId : beIds) { + countMap.put(beId, 0); + } + + int totalReplicaNum = 0; + for (String partName : partitions) { + Partition partition = olapTable.getPartition(partName); + for (MaterializedIndex index : partition.getMaterializedIndices()) { + for (Tablet tablet : index.getTablets()) { + for (Replica replica : tablet.getReplicas()) { + if (!countMap.containsKey(replica.getBackendId())) { + continue; + } + countMap.put(replica.getBackendId(), countMap.get(replica.getBackendId()) + 1); + totalReplicaNum++; + } + } + } + } + + // graph + Collections.sort(beIds); + for (Long beId : beIds) { + List row = Lists.newArrayList(); + row.add(String.valueOf(beId)); + row.add(String.valueOf(countMap.get(beId))); + row.add(graph(countMap.get(beId), totalReplicaNum, beIds.size())); + row.add(df.format((double) countMap.get(beId) / totalReplicaNum)); + result.add(row); + } + + } finally { + db.readUnlock(); + } + + return result; + } + + private static String graph(int num, int totalNum, int mod) { + StringBuilder sb = new StringBuilder(); + int normalized = (int) Math.ceil(num * mod / totalNum); + for (int i = 0; i < normalized; ++i) { + sb.append(">"); + } + return sb.toString(); + } +} diff --git a/fe/src/main/java/com/baidu/palo/catalog/TabletStatMgr.java b/fe/src/main/java/com/baidu/palo/catalog/TabletStatMgr.java new file mode 100644 index 0000000000..8141a430e5 --- /dev/null +++ b/fe/src/main/java/com/baidu/palo/catalog/TabletStatMgr.java @@ -0,0 +1,141 @@ +package com.baidu.palo.catalog; + +import com.baidu.palo.catalog.Table.TableType; +import com.baidu.palo.common.ClientPool; +import com.baidu.palo.common.Config; +import com.baidu.palo.common.util.Daemon; +import com.baidu.palo.system.Backend; +import com.baidu.palo.thrift.BackendService; +import com.baidu.palo.thrift.TNetworkAddress; +import com.baidu.palo.thrift.TTabletStat; +import com.baidu.palo.thrift.TTabletStatResult; + +import com.google.common.collect.ImmutableMap; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicBoolean; + +public class TabletStatMgr extends Daemon { + private static final Logger LOG = LogManager.getLogger(TabletStatMgr.class); + + private AtomicBoolean isStart = new AtomicBoolean(false); + + public TabletStatMgr() { + super("tablet stat mgr", Config.tablet_stat_update_interval_second * 1000); + } + + @Override + public synchronized void start() { + if (isStart.compareAndSet(false, true)) { + super.start(); + } + } + + @Override + protected void runOneCycle() { + // We should wait Frontend finished replaying logs, then begin to get tablet status + while (!Catalog.getInstance().canRead()) { + LOG.info("Frontend's canRead flag is false, waiting..."); + try { + // sleep here, not return. because if we return, we have to wait until next round, which may + // take a long time(default is tablet_stat_update_interval_second: 5 min) + Thread.sleep(10000); + } catch (InterruptedException e) { + LOG.info("get interrupted exception when sleep: ", e); + continue; + } + } + + ImmutableMap backends = Catalog.getCurrentSystemInfo().getIdToBackend(); + + long start = System.currentTimeMillis(); + for (Backend backend : backends.values()) { + BackendService.Client client = null; + TNetworkAddress address = null; + boolean ok = false; + try { + address = new TNetworkAddress(backend.getHost(), backend.getBePort()); + client = ClientPool.backendPool.borrowObject(address); + TTabletStatResult result = client.get_tablet_stat(); + + LOG.info("get tablet stat from backend: {}, num: {}", backend.getId(), result.getTablets_statsSize()); + // LOG.debug("get tablet stat from backend: {}, stat: {}", backend.getId(), result.getTablets_stats()); + updateTabletStat(backend.getId(), result); + + ok = true; + } catch (Exception e) { + LOG.warn("task exec error. backend[{}]", backend.getId(), e); + } finally { + if (ok) { + ClientPool.backendPool.returnObject(address, client); + } else { + ClientPool.backendPool.invalidateObject(address, client); + } + } + } + LOG.info("finished to get tablet stat of all backends. cost: {} ms", + (System.currentTimeMillis() - start)); + + // after update replica in all backends, update index row num + start = System.currentTimeMillis(); + List dbIds = Catalog.getCurrentCatalog().getDbIds(); + for (Long dbId : dbIds) { + Database db = Catalog.getCurrentCatalog().getDb(dbId); + if (db == null) { + continue; + } + db.writeLock(); + try { + for (Table table : db.getTables()) { + if (table.getType() != TableType.OLAP) { + continue; + } + + OlapTable olapTable = (OlapTable) table; + for (Partition partition : olapTable.getPartitions()) { + long version = partition.getCommittedVersion(); + long versionHash = partition.getCommittedVersionHash(); + for (MaterializedIndex index : partition.getMaterializedIndices()) { + long indexRowCount = 0L; + for (Tablet tablet : index.getTablets()) { + long tabletRowCount = 0L; + for (Replica replica : tablet.getReplicas()) { + if (replica.checkVersionCatchUp(version, versionHash) + && replica.getRowCount() > tabletRowCount) { + tabletRowCount = replica.getRowCount(); + } + } + indexRowCount += tabletRowCount; + } // end for tablets + index.setRowCount(indexRowCount); + } // end for indices + } // end for partitions + LOG.info("finished to set row num for table: {} in database: {}", + table.getName(), db.getFullName()); + } + } finally { + db.writeUnlock(); + } + } + LOG.info("finished to update index row num of all databases. cost: {} ms", + (System.currentTimeMillis() - start)); + } + + private void updateTabletStat(Long beId, TTabletStatResult result) { + TabletInvertedIndex invertedIndex = Catalog.getCurrentInvertedIndex(); + + for (Map.Entry entry : result.getTablets_stats().entrySet()) { + Replica replica = invertedIndex.getReplica(entry.getKey(), beId); + if (replica == null) { + // replica may be deleted from catalog + continue; + } + // TODO(cmy) no db lock protected. I think it is ok even we get wrong row num + replica.updateStat(entry.getValue().getData_size(), entry.getValue().getRow_num()); + } + } +} diff --git a/fe/src/main/java/com/baidu/palo/common/AuthenticationException.java b/fe/src/main/java/com/baidu/palo/common/AuthenticationException.java new file mode 100644 index 0000000000..17e718a2da --- /dev/null +++ b/fe/src/main/java/com/baidu/palo/common/AuthenticationException.java @@ -0,0 +1,34 @@ +// Modifications copyright (C) 2017, Baidu.com, Inc. +// Copyright 2017 The Apache Software Foundation + +// 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 com.baidu.palo.common; + +/** + * Thrown for authorization errors encountered when accessing Catalog objects. + */ +public class AuthenticationException extends UserException { + public AuthenticationException(String msg, Throwable cause) { + super(msg, cause); + } + + public AuthenticationException(String msg) { + super(msg); + } +} diff --git a/fe/src/main/java/com/baidu/palo/common/UserException.java b/fe/src/main/java/com/baidu/palo/common/UserException.java new file mode 100644 index 0000000000..60bcea1073 --- /dev/null +++ b/fe/src/main/java/com/baidu/palo/common/UserException.java @@ -0,0 +1,43 @@ +// Modifications copyright (C) 2017, Baidu.com, Inc. +// Copyright 2017 The Apache Software Foundation + +// 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 com.baidu.palo.common; + +/** + * Thrown for internal server errors. + */ +public class UserException extends Exception { + public UserException(String msg, Throwable cause) { + super(msg, cause); + } + + public UserException(Throwable cause) { + super(cause); + } + + public UserException(String message, Throwable cause, boolean enableSuppression, boolean writableStackTrace) { + super(message, cause, enableSuppression, writableStackTrace); + } + + public UserException(String msg) { + super(msg); + } + +} diff --git a/fe/src/main/java/com/baidu/palo/common/proc/EsPartitionsProcDir.java b/fe/src/main/java/com/baidu/palo/common/proc/EsPartitionsProcDir.java new file mode 100644 index 0000000000..bcb9245218 --- /dev/null +++ b/fe/src/main/java/com/baidu/palo/common/proc/EsPartitionsProcDir.java @@ -0,0 +1,131 @@ +// Copyright (c) 2017, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 com.baidu.palo.common.proc; + +import com.baidu.palo.catalog.Column; +import com.baidu.palo.catalog.Database; +import com.baidu.palo.catalog.PartitionType; +import com.baidu.palo.catalog.RangePartitionInfo; +import com.baidu.palo.catalog.EsTable; +import com.baidu.palo.catalog.Table.TableType; +import com.baidu.palo.common.AnalysisException; +import com.baidu.palo.external.EsIndexState; +import com.google.common.base.Joiner; +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +/* + * SHOW PROC /dbs/dbId/tableId/espartitions + * show partitions' detail info within a table + */ +public class EsPartitionsProcDir implements ProcDirInterface { + public static final ImmutableList TITLE_NAMES = new ImmutableList.Builder() + .add("IndexName").add("PartitionKey").add("Range").add("DistributionKey") + .add("Shards").add("ReplicationNum") + .build(); + + public static final int PARTITION_NAME_INDEX = 1; + + private Database db; + private EsTable esTable; + + public EsPartitionsProcDir(Database db, EsTable esTable) { + this.db = db; + this.esTable = esTable; + } + + @Override + public ProcResult fetchResult() throws AnalysisException { + Preconditions.checkNotNull(db); + Preconditions.checkNotNull(esTable); + Preconditions.checkState(esTable.getType() == TableType.ELASTICSEARCH); + + // get info + List> partitionInfos = new ArrayList>(); + db.readLock(); + try { + RangePartitionInfo rangePartitionInfo = null; + if (esTable.getPartitionInfo().getType() == PartitionType.RANGE) { + rangePartitionInfo = (RangePartitionInfo) esTable.getEsTableState().getPartitionInfo(); + } + Joiner joiner = Joiner.on(", "); + Map unPartitionedIndices = esTable.getEsTableState().getUnPartitionedIndexStates(); + Map partitionedIndices = esTable.getEsTableState().getPartitionedIndexStates(); + for (EsIndexState esIndexState : unPartitionedIndices.values()) { + List partitionInfo = new ArrayList(); + partitionInfo.add(esIndexState.getIndexName()); + partitionInfo.add("-"); // partition key + partitionInfo.add("-"); // range + partitionInfo.add("-"); // dis + partitionInfo.add(esIndexState.getShardRoutings().size()); // shards + partitionInfo.add(1); // replica num + partitionInfos.add(partitionInfo); + } + for (EsIndexState esIndexState : partitionedIndices.values()) { + List partitionInfo = new ArrayList(); + partitionInfo.add(esIndexState.getIndexName()); + List partitionColumns = rangePartitionInfo.getPartitionColumns(); + List colNames = new ArrayList(); + for (Column column : partitionColumns) { + colNames.add(column.getName()); + } + partitionInfo.add(joiner.join(colNames)); // partition key + partitionInfo.add(rangePartitionInfo.getIdToRange().get( + esIndexState.getPartitionId()).toString()); // range + partitionInfo.add("-"); // dis + partitionInfo.add(esIndexState.getShardRoutings().size()); // shards + partitionInfo.add(1); // replica num + partitionInfos.add(partitionInfo); + } + } finally { + db.readUnlock(); + } + + // set result + BaseProcResult result = new BaseProcResult(); + result.setNames(TITLE_NAMES); + for (List info : partitionInfos) { + List row = new ArrayList(info.size()); + for (Comparable comparable : info) { + row.add(comparable.toString()); + } + result.addRow(row); + } + + return result; + } + + @Override + public boolean register(String name, ProcNodeInterface node) { + return false; + } + + @Override + public ProcNodeInterface lookup(String indexName) throws AnalysisException { + + db.readLock(); + try { + return new EsShardProcDir(db, esTable, indexName); + } finally { + db.readUnlock(); + } + } + +} diff --git a/fe/src/main/java/com/baidu/palo/common/proc/EsShardProcDir.java b/fe/src/main/java/com/baidu/palo/common/proc/EsShardProcDir.java new file mode 100644 index 0000000000..e00a0e9d3d --- /dev/null +++ b/fe/src/main/java/com/baidu/palo/common/proc/EsShardProcDir.java @@ -0,0 +1,108 @@ +// Copyright (c) 2017, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 com.baidu.palo.common.proc; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +import com.baidu.palo.catalog.Database; +import com.baidu.palo.catalog.EsTable; +import com.baidu.palo.common.AnalysisException; +import com.baidu.palo.common.util.ListComparator; +import com.baidu.palo.external.EsIndexState; +import com.baidu.palo.external.EsShardRouting; +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; + +public class EsShardProcDir implements ProcDirInterface { + public static final ImmutableList TITLE_NAMES = new ImmutableList.Builder() + .add("ShardId").add("Host").add("IsPrimary") + .build(); + + private Database db; + private EsTable esTable; + private String indexName; + + public EsShardProcDir(Database db, EsTable esTable, String indexName) { + this.db = db; + this.esTable = esTable; + this.indexName = indexName; + } + + @Override + public ProcResult fetchResult() { + Preconditions.checkNotNull(db); + Preconditions.checkNotNull(esTable); + Preconditions.checkNotNull(indexName); + + List> shardInfos = new ArrayList>(); + db.readLock(); + try { + // get infos + EsIndexState esIndexState = esTable.getEsTableState().getIndexState(indexName); + for (int shardId : esIndexState.getShardRoutings().keySet()) { + List shardRoutings = esIndexState.getShardRoutings().get(shardId); + if (shardRoutings != null && shardRoutings.size() > 0) { + for (EsShardRouting esShardRouting : shardRoutings) { + List shardInfo = new ArrayList(); + shardInfo.add(shardId); + shardInfo.add(esShardRouting.getAddress().toString()); + shardInfo.add(esShardRouting.isPrimary()); + shardInfos.add(shardInfo); + } + } else { + List shardInfo = new ArrayList(); + shardInfo.add(shardId); + shardInfo.add(""); + shardInfo.add(false); + shardInfos.add(shardInfo); + } + } + } finally { + db.readUnlock(); + } + + // sort by tabletId, replicaId + ListComparator> comparator = new ListComparator>(0, 1); + Collections.sort(shardInfos, comparator); + + // set result + BaseProcResult result = new BaseProcResult(); + result.setNames(TITLE_NAMES); + + for (int i = 0; i < shardInfos.size(); i++) { + List info = shardInfos.get(i); + List row = new ArrayList(info.size()); + for (int j = 0; j < info.size(); j++) { + row.add(info.get(j).toString()); + } + result.addRow(row); + } + return result; + } + + @Override + public boolean register(String name, ProcNodeInterface node) { + return false; + } + + @Override + public ProcNodeInterface lookup(String shardIdStr) throws AnalysisException { + return null; + } + +} diff --git a/fe/src/main/java/com/baidu/palo/common/proc/TransDbProcDir.java b/fe/src/main/java/com/baidu/palo/common/proc/TransDbProcDir.java new file mode 100644 index 0000000000..98f3d00fcd --- /dev/null +++ b/fe/src/main/java/com/baidu/palo/common/proc/TransDbProcDir.java @@ -0,0 +1,92 @@ +// Copyright (c) 2017, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 com.baidu.palo.common.proc; + +import com.baidu.palo.catalog.Catalog; +import com.baidu.palo.catalog.Database; +import com.baidu.palo.transaction.GlobalTransactionMgr; +import com.baidu.palo.common.AnalysisException; +import com.baidu.palo.common.util.ListComparator; +import com.google.common.base.Preconditions; +import com.google.common.base.Strings; +import com.google.common.collect.ImmutableList; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +/** + * author: wuyunfeng + * date: 18/1/5 10:43 + * project: palo2 + */ +public class TransDbProcDir implements ProcDirInterface { + public static final ImmutableList TITLE_NAMES = new ImmutableList.Builder() + .add("DbId") + .add("DbName") + .build(); + private Catalog catalog; + + public TransDbProcDir(Catalog catalog) { + this.catalog = catalog; + } + + @Override + public ProcResult fetchResult() throws AnalysisException { + Preconditions.checkNotNull(catalog); + BaseProcResult result = new BaseProcResult(); + result.setNames(TITLE_NAMES); + GlobalTransactionMgr transactionMgr = catalog.getCurrentGlobalTransactionMgr(); + List> infos = transactionMgr.getDbInfo(); + // order by dbId, asc + ListComparator> comparator = new ListComparator>(0); + Collections.sort(infos, comparator); + for (List info : infos) { + List row = new ArrayList(info.size()); + for (Comparable comparable : info) { + row.add(comparable.toString()); + } + result.addRow(row); + } + return result; + + } + + @Override + public boolean register(String name, ProcNodeInterface node) { + return false; + } + + @Override + public ProcNodeInterface lookup(String dbIdStr) throws AnalysisException { + if (catalog == null || Strings.isNullOrEmpty(dbIdStr)) { + throw new AnalysisException("Db id is null"); + } + long dbId = -1L; + try { + dbId = Long.valueOf(dbIdStr); + } catch (NumberFormatException e) { + throw new AnalysisException("Invalid db id format: " + dbIdStr); + } + + Database db = catalog.getDb(dbId); + if (db == null) { + throw new AnalysisException("Database[" + dbId + "] does not exist."); + } + + return new TransProcDir(dbId); + } +} diff --git a/fe/src/main/java/com/baidu/palo/common/proc/TransPartitionProcNode.java b/fe/src/main/java/com/baidu/palo/common/proc/TransPartitionProcNode.java new file mode 100644 index 0000000000..2576fb6d98 --- /dev/null +++ b/fe/src/main/java/com/baidu/palo/common/proc/TransPartitionProcNode.java @@ -0,0 +1,75 @@ +// Copyright (c) 2017, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 com.baidu.palo.common.proc; + +import com.baidu.palo.catalog.Catalog; +import com.baidu.palo.catalog.Database; +import com.baidu.palo.catalog.OlapTable; +import com.baidu.palo.catalog.Table; +import com.baidu.palo.transaction.GlobalTransactionMgr; +import com.baidu.palo.common.AnalysisException; +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; + +import java.util.ArrayList; +import java.util.List; + +/** + * author: wuyunfeng + * date: 18/1/5 10:58 + * project: palo2 + */ +public class TransPartitionProcNode implements ProcNodeInterface { + public static final ImmutableList TITLE_NAMES = new ImmutableList.Builder() + .add("PartitionId") + .add("PartitionName") + .add("CommittedVersion") + .add("CommittedVersionHash") + .add("State") + .build(); + + private long tid; + private Database db; + private OlapTable olapTable; + + public TransPartitionProcNode(long tid, Database db, OlapTable olapTable) { + this.tid = tid; + this.db = db; + this.olapTable = olapTable; + } + + @Override + public ProcResult fetchResult() throws AnalysisException { + Preconditions.checkNotNull(db); + Preconditions.checkNotNull(olapTable); + Preconditions.checkState(olapTable.getType() == Table.TableType.OLAP); + Catalog catalog = Catalog.getInstance(); + GlobalTransactionMgr transactionMgr = catalog.getCurrentGlobalTransactionMgr(); + List> partitionInfos = transactionMgr.getPartitionTransInfo(tid, db, olapTable); + // set result + BaseProcResult result = new BaseProcResult(); + result.setNames(TITLE_NAMES); + for (List info : partitionInfos) { + List row = new ArrayList(info.size()); + for (Comparable comparable : info) { + row.add(comparable.toString()); + } + result.addRow(row); + } + + return result; + } +} diff --git a/fe/src/main/java/com/baidu/palo/common/proc/TransProcDir.java b/fe/src/main/java/com/baidu/palo/common/proc/TransProcDir.java new file mode 100644 index 0000000000..1439e4a0dc --- /dev/null +++ b/fe/src/main/java/com/baidu/palo/common/proc/TransProcDir.java @@ -0,0 +1,98 @@ +// Copyright (c) 2017, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 com.baidu.palo.common.proc; + +import com.baidu.palo.catalog.Catalog; +import com.baidu.palo.catalog.Database; +import com.baidu.palo.transaction.GlobalTransactionMgr; +import com.baidu.palo.common.AnalysisException; +import com.baidu.palo.common.util.ListComparator; +import com.google.common.base.Strings; +import com.google.common.collect.ImmutableList; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +/** + * author: wuyunfeng + * date: 18/1/5 10:43 + * project: palo2 + */ +public class TransProcDir implements ProcDirInterface { + public static final ImmutableList TITLE_NAMES = new ImmutableList.Builder() + .add("TransactionId") + .add("Label") + .add("Coordinator") + .add("TransactionStatus") + .add("LoadJobSourceType") + .add("PrepareTime") + .add("CommitTime") + .add("FinishTime") + .add("Reason") + .add("ErrorReplicasCount") + .build(); + private long dbId; + + public TransProcDir(long dbId) { + this.dbId = dbId; + } + + @Override + public ProcResult fetchResult() throws AnalysisException { + BaseProcResult result = new BaseProcResult(); + result.setNames(TITLE_NAMES); + Catalog catalog = Catalog.getInstance(); + GlobalTransactionMgr transactionMgr = Catalog.getCurrentGlobalTransactionMgr(); + List> infos = transactionMgr.getDbTransInfo(dbId); + // order by transactionId, asc + ListComparator> comparator = new ListComparator>(0); + Collections.sort(infos, comparator); + for (List info : infos) { + List row = new ArrayList(info.size()); + for (Comparable comparable : info) { + row.add(comparable.toString()); + } + result.addRow(row); + } + return result; + } + + @Override + public boolean register(String name, ProcNodeInterface node) { + return false; + } + + @Override + public ProcNodeInterface lookup(String tidStr) throws AnalysisException { + if (tidStr == null || Strings.isNullOrEmpty(tidStr)) { + throw new AnalysisException("Table id is null"); + } + + long tid = -1L; + try { + tid = Long.valueOf(tidStr); + } catch (NumberFormatException e) { + throw new AnalysisException("Invalid transaction id format: " + tid); + } + Database db = Catalog.getInstance().getDb(dbId); + if (db == null) { + throw new AnalysisException("Database[" + dbId + "] does not exist."); + } + + return new TransTablesProcDir(db, tid); + } +} diff --git a/fe/src/main/java/com/baidu/palo/common/proc/TransTablesProcDir.java b/fe/src/main/java/com/baidu/palo/common/proc/TransTablesProcDir.java new file mode 100644 index 0000000000..0e2aacaa42 --- /dev/null +++ b/fe/src/main/java/com/baidu/palo/common/proc/TransTablesProcDir.java @@ -0,0 +1,113 @@ +// Copyright (c) 2017, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 com.baidu.palo.common.proc; + +import com.baidu.palo.catalog.Catalog; +import com.baidu.palo.catalog.Database; +import com.baidu.palo.catalog.OlapTable; +import com.baidu.palo.catalog.Table; +import com.baidu.palo.transaction.GlobalTransactionMgr; +import com.baidu.palo.common.AnalysisException; +import com.baidu.palo.common.util.ListComparator; +import com.google.common.base.Preconditions; +import com.google.common.base.Strings; +import com.google.common.collect.ImmutableList; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +/** + * author: wuyunfeng + * date: 18/1/5 19:15 + * project: palo2 + */ +public class TransTablesProcDir implements ProcDirInterface { + public static final ImmutableList TITLE_NAMES = new ImmutableList.Builder() + .add("TableId") + .add("TableName") + .add("PartitionNum") + .add("State") + .build(); + + private Database db; + private long tid; + + public TransTablesProcDir(Database db, long tid) { + this.db = db; + this.tid = tid; + } + + + @Override + public boolean register(String name, ProcNodeInterface node) { + return false; + } + + @Override + public ProcNodeInterface lookup(String tableIdStr) throws AnalysisException { + Preconditions.checkNotNull(db); + if (Strings.isNullOrEmpty(tableIdStr)) { + throw new AnalysisException("TableIdStr is null"); + } + + long tableId = -1L; + try { + tableId = Long.valueOf(tableIdStr); + } catch (NumberFormatException e) { + throw new AnalysisException("Invalid table id format: " + tableIdStr); + } + + Table table = null; + db.readLock(); + try { + table = db.getTable(tableId); + } finally { + db.readUnlock(); + } + if (table == null) { + throw new AnalysisException("Table[" + tableId + "] does not exist"); + } + + return new TransPartitionProcNode(tid, db, (OlapTable) table); + } + + @Override + public ProcResult fetchResult() throws AnalysisException { + Preconditions.checkNotNull(db); + + // get info + GlobalTransactionMgr transactionMgr = Catalog.getCurrentGlobalTransactionMgr(); + List> tableInfos = transactionMgr.getTableTransInfo(tid, db); + // sort by table id + ListComparator> comparator = new ListComparator>(0); + Collections.sort(tableInfos, comparator); + + // set result + BaseProcResult result = new BaseProcResult(); + result.setNames(TITLE_NAMES); + + for (List info : tableInfos) { + List row = new ArrayList(info.size()); + for (Comparable comparable : info) { + row.add(comparable.toString()); + } + result.addRow(row); + } + + return result; + } +} diff --git a/fe/src/main/java/com/baidu/palo/external/EsIndexState.java b/fe/src/main/java/com/baidu/palo/external/EsIndexState.java new file mode 100644 index 0000000000..a7993d88cd --- /dev/null +++ b/fe/src/main/java/com/baidu/palo/external/EsIndexState.java @@ -0,0 +1,145 @@ +// Copyright (c) 2017, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 com.baidu.palo.external; + +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.json.JSONArray; +import org.json.JSONObject; +import org.mortbay.util.ajax.JSON; + +import com.baidu.palo.analysis.PartitionKeyDesc; +import com.baidu.palo.analysis.SingleRangePartitionDesc; +import com.baidu.palo.catalog.PartitionInfo; +import com.baidu.palo.catalog.PartitionKey; +import com.baidu.palo.catalog.RangePartitionInfo; +import com.baidu.palo.common.AnalysisException; +import com.baidu.palo.thrift.TNetworkAddress; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; + +public class EsIndexState { + + private static final Logger LOG = LogManager.getLogger(EsIndexState.class); + + private final String indexName; + // shardid -> host1, host2, host3 + private Map> shardRoutings; + private SingleRangePartitionDesc partitionDesc; + private PartitionKey partitionKey; + private long partitionId = -1; + + public EsIndexState(String indexName) { + this.indexName = indexName; + this.shardRoutings = Maps.newHashMap(); + this.partitionDesc = null; + this.partitionKey = null; + } + + public static EsIndexState parseIndexStateV55(String indexName, JSONObject indicesRoutingMap, + JSONObject nodesMap, + JSONObject indicesMetaMap, PartitionInfo partitionInfo) throws AnalysisException { + EsIndexState indexState = new EsIndexState(indexName); + JSONObject shardRoutings = indicesRoutingMap.getJSONObject(indexName).getJSONObject("shards"); + for (String shardKey : shardRoutings.keySet()) { + List singleShardRouting = Lists.newArrayList(); + JSONArray shardRouting = shardRoutings.getJSONArray(shardKey); + for (int i = 0; i < shardRouting.length(); ++i) { + JSONObject shard = shardRouting.getJSONObject(i); + String shardState = shard.getString("state"); + if ("STARTED".equalsIgnoreCase(shardState)) { + singleShardRouting.add(EsShardRouting.parseShardRoutingV55(shardState, + shardKey, shard, nodesMap)); + } + } + if (singleShardRouting.isEmpty()) { + LOG.warn("could not find a healthy allocation for [{}][{}]", indexName, shardKey); + } + indexState.addShardRouting(Integer.valueOf(shardKey), singleShardRouting); + } + + // get some meta info from es, could be used to prune index when query + // index.bpack.partition.upperbound: stu_age + if (partitionInfo instanceof RangePartitionInfo) { + JSONObject indexMeta = indicesMetaMap.getJSONObject(indexName); + JSONObject partitionSetting = EsUtil.getJsonObject(indexMeta, "settings.index.bpack.partition", 0); + LOG.debug("index {} range partition setting is {}", indexName, + partitionSetting == null ? "" : partitionSetting.toString()); + if (partitionSetting != null && partitionSetting.has("upperbound")) { + String upperBound = partitionSetting.getString("upperbound"); + List upperValues = Stream.of(upperBound).collect(Collectors.toList()); + PartitionKeyDesc partitionKeyDesc = new PartitionKeyDesc(upperValues); + // use index name as partition name + SingleRangePartitionDesc desc = new SingleRangePartitionDesc(false, + indexName, partitionKeyDesc, null); + PartitionKey partitionKey = PartitionKey.createPartitionKey( + desc.getPartitionKeyDesc().getUpperValues(), + ((RangePartitionInfo) partitionInfo).getPartitionColumns()); + desc.analyze(((RangePartitionInfo) partitionInfo).getPartitionColumns().size(), null); + indexState.setPartitionDesc(desc); + indexState.setPartitionKey(partitionKey); + } + } + return indexState; + } + + public void addShardRouting(int shardId, List singleShardRouting) { + shardRoutings.put(shardId, singleShardRouting); + } + + public String getIndexName() { + return indexName; + } + + public Map> getShardRoutings() { + return shardRoutings; + } + + public SingleRangePartitionDesc getPartitionDesc() { + return partitionDesc; + } + + public void setPartitionDesc(SingleRangePartitionDesc partitionDesc) { + this.partitionDesc = partitionDesc; + } + + public PartitionKey getPartitionKey() { + return partitionKey; + } + + public void setPartitionKey(PartitionKey partitionKey) { + this.partitionKey = partitionKey; + } + + public long getPartitionId() { + return partitionId; + } + + public void setPartitionId(long partitionId) { + this.partitionId = partitionId; + } + + @Override + public String toString() { + return "EsIndexState [indexName=" + indexName + ", partitionDesc=" + partitionDesc + ", partitionKey=" + + partitionKey + "]"; + } +} diff --git a/fe/src/main/java/com/baidu/palo/external/EsShardRouting.java b/fe/src/main/java/com/baidu/palo/external/EsShardRouting.java new file mode 100644 index 0000000000..519f9a5135 --- /dev/null +++ b/fe/src/main/java/com/baidu/palo/external/EsShardRouting.java @@ -0,0 +1,64 @@ +// Copyright (c) 2017, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 com.baidu.palo.external; + + +import org.json.JSONObject; + +import com.baidu.palo.thrift.TNetworkAddress; + +public class EsShardRouting { + + private final String indexName; + private final int shardId; + private final boolean isPrimary; + private final TNetworkAddress address; + + public EsShardRouting(String indexName, int shardId, boolean isPrimary, TNetworkAddress address) { + this.indexName = indexName; + this.shardId = shardId; + this.isPrimary = isPrimary; + this.address = address; + } + + public static EsShardRouting parseShardRoutingV55(String indexName, String shardKey, + JSONObject shardInfo, JSONObject nodesMap) { + String nodeId = shardInfo.getString("node"); + JSONObject nodeInfo = nodesMap.getJSONObject(nodeId); + String[] transportAddr = nodeInfo.getString("transport_address").split(":"); + // TODO(ygl) should get thrift port from node info + TNetworkAddress addr = new TNetworkAddress(transportAddr[0], Integer.valueOf(transportAddr[1])); + boolean isPrimary = shardInfo.getBoolean("primary"); + return new EsShardRouting(indexName, Integer.valueOf(shardKey), + isPrimary, addr); + } + + public int getShardId() { + return shardId; + } + + public boolean isPrimary() { + return isPrimary; + } + + public TNetworkAddress getAddress() { + return address; + } + + public String getIndexName() { + return indexName; + } +} diff --git a/fe/src/main/java/com/baidu/palo/external/EsStateStore.java b/fe/src/main/java/com/baidu/palo/external/EsStateStore.java new file mode 100644 index 0000000000..50e9042eef --- /dev/null +++ b/fe/src/main/java/com/baidu/palo/external/EsStateStore.java @@ -0,0 +1,221 @@ +// Copyright (c) 2017, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 com.baidu.palo.external; + +import java.io.IOException; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.json.JSONObject; + +import com.baidu.palo.catalog.Catalog; +import com.baidu.palo.catalog.Column; +import com.baidu.palo.catalog.Database; +import com.baidu.palo.catalog.EsTable; +import com.baidu.palo.catalog.PartitionInfo; +import com.baidu.palo.catalog.PartitionKey; +import com.baidu.palo.catalog.RangePartitionInfo; +import com.baidu.palo.catalog.SinglePartitionInfo; +import com.baidu.palo.catalog.Table; +import com.baidu.palo.catalog.Table.TableType; +import com.baidu.palo.common.AnalysisException; +import com.baidu.palo.common.Config; +import com.baidu.palo.common.DdlException; +import com.baidu.palo.common.util.Daemon; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.Maps; +import com.google.common.collect.Range; + +import okhttp3.Authenticator; +import okhttp3.Call; +import okhttp3.Credentials; +import okhttp3.OkHttpClient; +import okhttp3.Request; +import okhttp3.Response; +import okhttp3.Route; + +/** + * it is used to call es api to get shard allocation state + * @author yiguolei + * + */ +public class EsStateStore extends Daemon { + + private static final Logger LOG = LogManager.getLogger(EsStateStore.class); + + private Map esTables; + + public EsStateStore() { + super(Config.es_state_sync_interval_secs * 1000); + esTables = Maps.newConcurrentMap(); + } + + public void registerTable(EsTable esTable) { + if (Catalog.isCheckpointThread()) { + return; + } + esTables.put(esTable.getId(), esTable); + LOG.info("register a new table [{}] to sync list", esTable.toString()); + } + + public void deRegisterTable(long tableId) { + esTables.remove(tableId); + LOG.info("deregister table [{}] from sync list", tableId); + } + + protected void runOneCycle() { + for (EsTable esTable : esTables.values()) { + EsTableState esTableState = loadEsIndexMetadataV55(esTable); + if (esTableState != null) { + esTable.setEsTableState(esTableState); + } + } + } + + // when fe is start to load image, should call this method to init the state store + public void loadTableFromCatalog() { + List dbIds = Catalog.getCurrentCatalog().getDbIds(); + for(Long dbId : dbIds) { + Database database = Catalog.getInstance().getDb(dbId); + List tables = database.getTables(); + for (Table table : tables) { + if (table.getType() == TableType.ELASTICSEARCH) { + esTables.put(table.getId(), (EsTable) table); + } + } + } + } + + private EsTableState loadEsIndexMetadataV55(final EsTable esTable) { + OkHttpClient.Builder clientBuilder = new OkHttpClient.Builder(); + clientBuilder.authenticator(new Authenticator() { + @Override + public Request authenticate(Route route, Response response) throws IOException { + String credential = Credentials.basic(esTable.getUserName(), esTable.getPasswd()); + return response.request().newBuilder().header("Authorization", credential).build(); + } + }); + String[] seeds = esTable.getSeeds(); + for (String seed : seeds) { + String url = seed + "/_cluster/state?indices=" + + esTable.getIndexName() + + "&metric=routing_table,nodes,metadata&expand_wildcards=open"; + String basicAuth = ""; + try { + Request request = new Request.Builder() + .get() + .url(url) + .addHeader("Authorization", basicAuth) + .build(); + Call call = clientBuilder.build().newCall(request); + Response response = call.execute(); + String responseStr = response.body().string(); + if (response.isSuccessful()) { + try { + EsTableState esTableState = parseClusterState55(responseStr, esTable); + if (esTableState != null) { + return esTableState; + } + } + catch (Exception e) { + LOG.warn("errors while parse response msg {}", responseStr, e); + } + } else { + LOG.info("errors while call es [{}] to get state info {}", url, responseStr); + } + } catch (Exception e) { + LOG.warn("errors while call es [{}]", url, e); + } + } + return null; + } + + @VisibleForTesting + public EsTableState parseClusterState55(String responseStr, EsTable esTable) + throws DdlException, AnalysisException, ExternalDataSourceException { + JSONObject jsonObject = new JSONObject(responseStr); + String clusterName = jsonObject.getString("cluster_name"); + JSONObject nodesMap = jsonObject.getJSONObject("nodes"); + + JSONObject indicesMetaMap = jsonObject.getJSONObject("metadata").getJSONObject("indices"); + JSONObject indicesRoutingMap = jsonObject.getJSONObject("routing_table").getJSONObject("indices"); + EsTableState esTableState = new EsTableState(); + PartitionInfo partitionInfo = null; + if (esTable.getPartitionInfo() != null) { + if (esTable.getPartitionInfo() instanceof RangePartitionInfo) { + RangePartitionInfo rangePartitionInfo = (RangePartitionInfo) esTable.getPartitionInfo(); + partitionInfo = new RangePartitionInfo(rangePartitionInfo.getPartitionColumns()); + esTableState.setPartitionInfo(partitionInfo); + if (LOG.isDebugEnabled()) { + StringBuilder sb = new StringBuilder(); + int idx = 0; + for (Column column : rangePartitionInfo.getPartitionColumns()) { + if (idx != 0) { + sb.append(", "); + } + sb.append("`").append(column.getName()).append("`"); + idx++; + } + sb.append(")"); + LOG.debug("begin to parse es table [{}] state from cluster state," + + " with partition info [{}]", esTable.getName(), sb.toString()); + } + } else if (esTable.getPartitionInfo() instanceof SinglePartitionInfo) { + LOG.debug("begin to parse es table [{}] state from cluster state, " + + "with no partition info", esTable.getName()); + } else { + throw new ExternalDataSourceException("es table only support range partition, " + + "but current partition type is " + + esTable.getPartitionInfo().getType()); + } + } + for (String indexName : indicesRoutingMap.keySet()) { + EsIndexState indexState = EsIndexState.parseIndexStateV55(indexName, + indicesRoutingMap, nodesMap, + indicesMetaMap, partitionInfo); + esTableState.addIndexState(indexName, indexState); + LOG.debug("add index {} to es table {}", indexState, esTable.getName()); + } + + if (partitionInfo instanceof RangePartitionInfo) { + // sort the index state according to partition key and then add to range map + List esIndexStates = esTableState.getPartitionedIndexStates().values() + .stream().collect(Collectors.toList()); + Collections.sort(esIndexStates, new Comparator() { + @Override + public int compare(EsIndexState o1, EsIndexState o2) { + return o1.getPartitionKey().compareTo(o2.getPartitionKey()); + } + }); + long partitionId = 0; + for (EsIndexState esIndexState : esIndexStates) { + Range range = ((RangePartitionInfo) partitionInfo).handleNewSinglePartitionDesc( + esIndexState.getPartitionDesc(), + partitionId); + esTableState.addPartition(esIndexState.getIndexName(), partitionId); + esIndexState.setPartitionId(partitionId); + ++ partitionId; + LOG.debug("add parition to es table [{}] with range [{}]", esTable.getName(), range); + } + } + return esTableState; + } +} diff --git a/fe/src/main/java/com/baidu/palo/external/EsTableState.java b/fe/src/main/java/com/baidu/palo/external/EsTableState.java new file mode 100644 index 0000000000..04bce8458a --- /dev/null +++ b/fe/src/main/java/com/baidu/palo/external/EsTableState.java @@ -0,0 +1,85 @@ +// Copyright (c) 2017, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 com.baidu.palo.external; + +import java.util.Map; + +import com.baidu.palo.catalog.PartitionInfo; +import com.google.common.collect.Maps; + +/** + * save the dynamic info parsed from es cluster state such as shard routing, partition info + */ +public class EsTableState { + + private PartitionInfo partitionInfo; + private Map partitionIdToIndices; + private Map partitionedIndexStates; + private Map unPartitionedIndexStates; + + public EsTableState() { + partitionInfo = null; + partitionIdToIndices = Maps.newHashMap(); + partitionedIndexStates = Maps.newHashMap(); + unPartitionedIndexStates = Maps.newHashMap(); + } + + public PartitionInfo getPartitionInfo() { + return partitionInfo; + } + + public void setPartitionInfo(PartitionInfo partitionInfo) { + this.partitionInfo = partitionInfo; + } + + public Map getPartitionIdToIndices() { + return partitionIdToIndices; + } + + public void addPartition(String indexName, long partitionId) { + partitionIdToIndices.put(partitionId, indexName); + } + + public void addIndexState(String indexName, EsIndexState indexState) { + if (indexState.getPartitionDesc() != null) { + partitionedIndexStates.put(indexName, indexState); + } else { + unPartitionedIndexStates.put(indexName, indexState); + } + } + + public Map getPartitionedIndexStates() { + return partitionedIndexStates; + } + + public Map getUnPartitionedIndexStates() { + return unPartitionedIndexStates; + } + + public EsIndexState getIndexState(long partitionId) { + if (partitionIdToIndices.containsKey(partitionId)) { + return partitionedIndexStates.get(partitionIdToIndices.get(partitionId)); + } + return null; + } + + public EsIndexState getIndexState(String indexName) { + if (partitionedIndexStates.containsKey(indexName)) { + return partitionedIndexStates.get(indexName); + } + return unPartitionedIndexStates.get(indexName); + } +} diff --git a/fe/src/main/java/com/baidu/palo/external/EsUtil.java b/fe/src/main/java/com/baidu/palo/external/EsUtil.java new file mode 100644 index 0000000000..278bf459f7 --- /dev/null +++ b/fe/src/main/java/com/baidu/palo/external/EsUtil.java @@ -0,0 +1,91 @@ +// Copyright (c) 2017, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 com.baidu.palo.external; + +import org.json.JSONObject; + +import com.baidu.palo.analysis.DistributionDesc; +import com.baidu.palo.analysis.PartitionDesc; +import com.baidu.palo.analysis.RangePartitionDesc; +import com.baidu.palo.common.AnalysisException; + +public class EsUtil { + + public static void analyzePartitionAndDistributionDesc(PartitionDesc partitionDesc, + DistributionDesc distributionDesc) + throws AnalysisException { + if (partitionDesc == null && distributionDesc == null) { + return; + } + + if (partitionDesc != null) { + if (!(partitionDesc instanceof RangePartitionDesc)) { + throw new AnalysisException("Elasticsearch table only permit range partition"); + } + + RangePartitionDesc rangePartitionDesc = (RangePartitionDesc) partitionDesc; + analyzePartitionDesc(rangePartitionDesc); + } + + if (distributionDesc != null) { + throw new AnalysisException("could not support distribution clause"); + + // if (!(distributionDesc instanceof HashDistributionDesc)) { + // throw new AnalysisException("Kudu table only permit hash distribution"); + // } + + // HashDistributionDesc hashDistributionDesc = (HashDistributionDesc) distributionDesc; + // analyzeDistributionDesc(hashDistributionDesc); + } + } + + private static void analyzePartitionDesc(RangePartitionDesc partDesc) + throws AnalysisException { + if (partDesc.getPartitionColNames() == null || partDesc.getPartitionColNames().isEmpty()) { + throw new AnalysisException("No partition columns."); + } + + if (partDesc.getPartitionColNames().size() > 1) { + throw new AnalysisException("Elasticsearch table's parition column could only be a single column"); + } + } + + + + /** + * get the json object from specified jsonObject + * @param jsonObject + * @param key + * @return + */ + public static JSONObject getJsonObject(JSONObject jsonObject, String key, int fromIndex) { + int firstOccr = key.indexOf('.', fromIndex); + if (firstOccr == -1) { + String token = key.substring(key.lastIndexOf('.') + 1); + if (jsonObject.has(token)) { + return jsonObject.getJSONObject(token); + } else { + return null; + } + } + String fieldName = key.substring(fromIndex, firstOccr); + if (jsonObject.has(fieldName)) { + return getJsonObject(jsonObject.getJSONObject(fieldName), key, firstOccr + 1); + } else { + return null; + } + } +} diff --git a/fe/src/main/java/com/baidu/palo/external/ExternalDataSourceException.java b/fe/src/main/java/com/baidu/palo/external/ExternalDataSourceException.java new file mode 100644 index 0000000000..9375458f98 --- /dev/null +++ b/fe/src/main/java/com/baidu/palo/external/ExternalDataSourceException.java @@ -0,0 +1,27 @@ +// Copyright (c) 2017, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 com.baidu.palo.external; + +import com.baidu.palo.common.UserException; + +public class ExternalDataSourceException extends UserException { + + private static final long serialVersionUID = 7912833584319374692L; + + public ExternalDataSourceException(String msg) { + super(msg); + } +} diff --git a/fe/src/main/java/com/baidu/palo/http/rest/CancelStreamLoad.java b/fe/src/main/java/com/baidu/palo/http/rest/CancelStreamLoad.java new file mode 100644 index 0000000000..e664085ba4 --- /dev/null +++ b/fe/src/main/java/com/baidu/palo/http/rest/CancelStreamLoad.java @@ -0,0 +1,87 @@ +// Copyright (c) 2017, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 com.baidu.palo.http.rest; + +import com.baidu.palo.catalog.Catalog; +import com.baidu.palo.catalog.Database; +import com.baidu.palo.cluster.ClusterNamespace; +import com.baidu.palo.common.DdlException; +import com.baidu.palo.common.UserException; +import com.baidu.palo.http.ActionController; +import com.baidu.palo.http.BaseRequest; +import com.baidu.palo.http.BaseResponse; +import com.baidu.palo.http.IllegalArgException; + +import com.google.common.base.Strings; + +import io.netty.handler.codec.http.HttpMethod; + +public class CancelStreamLoad extends RestBaseAction { + private static final String DB_KEY = "db"; + private static final String LABEL_KEY = "label"; + + public CancelStreamLoad(ActionController controller) { + super(controller); + } + + public static void registerAction(ActionController controller) + throws IllegalArgException { + CancelStreamLoad action = new CancelStreamLoad(controller); + controller.registerHandler(HttpMethod.POST, "/api/{" + DB_KEY + "}/{" + LABEL_KEY + "}/_cancel", action); + } + + @Override + public void executeWithoutPassword(AuthorizationInfo authInfo, BaseRequest request, BaseResponse response) + throws DdlException { + + if (redirectToMaster(request, response)) { + return; + } + + final String clusterName = authInfo.cluster; + if (Strings.isNullOrEmpty(clusterName)) { + throw new DdlException("No cluster selected."); + } + + String dbName = request.getSingleParameter(DB_KEY); + if (Strings.isNullOrEmpty(dbName)) { + throw new DdlException("No database selected."); + } + + String fullDbName = ClusterNamespace.getFullName(clusterName, dbName); + + String label = request.getSingleParameter(LABEL_KEY); + if (Strings.isNullOrEmpty(label)) { + throw new DdlException("No label selected."); + } + + // FIXME(cmy) + // checkWritePriv(authInfo.fullUserName, fullDbName); + + Database db = Catalog.getInstance().getDb(fullDbName); + if (db == null) { + throw new DdlException("unknown database, database=" + dbName); + } + + try { + Catalog.getCurrentGlobalTransactionMgr().abortTransaction(db.getId(), label, "user cancel"); + } catch (UserException e) { + throw new DdlException(e.getMessage()); + } + + sendResult(request, response, new RestBaseResult()); + } +} diff --git a/fe/src/main/java/com/baidu/palo/http/rest/GetStreamLoadState.java b/fe/src/main/java/com/baidu/palo/http/rest/GetStreamLoadState.java new file mode 100644 index 0000000000..2637071ddd --- /dev/null +++ b/fe/src/main/java/com/baidu/palo/http/rest/GetStreamLoadState.java @@ -0,0 +1,89 @@ +// Copyright (c) 2017, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 com.baidu.palo.http.rest; + +import com.baidu.palo.catalog.Catalog; +import com.baidu.palo.catalog.Database; +import com.baidu.palo.cluster.ClusterNamespace; +import com.baidu.palo.common.DdlException; +import com.baidu.palo.http.ActionController; +import com.baidu.palo.http.BaseRequest; +import com.baidu.palo.http.BaseResponse; +import com.baidu.palo.http.IllegalArgException; + +import com.google.common.base.Strings; + +import io.netty.handler.codec.http.HttpMethod; + +public class GetStreamLoadState extends RestBaseAction { + private static final String DB_KEY = "db"; + private static final String LABEL_KEY = "label"; + + public GetStreamLoadState(ActionController controller) { + super(controller); + } + + public static void registerAction(ActionController controller) + throws IllegalArgException { + GetStreamLoadState action = new GetStreamLoadState(controller); + controller.registerHandler(HttpMethod.GET, "/api/{" + DB_KEY + "}/{" + LABEL_KEY + "}/_state", action); + } + + @Override + public void executeWithoutPassword(AuthorizationInfo authInfo, BaseRequest request, BaseResponse response) + throws DdlException { + + if (redirectToMaster(request, response)) { + return; + } + + final String clusterName = authInfo.cluster; + if (Strings.isNullOrEmpty(clusterName)) { + throw new DdlException("No cluster selected."); + } + + String dbName = request.getSingleParameter(DB_KEY); + if (Strings.isNullOrEmpty(dbName)) { + throw new DdlException("No database selected."); + } + + String fullDbName = ClusterNamespace.getFullName(clusterName, dbName); + + String label = request.getSingleParameter(LABEL_KEY); + if (Strings.isNullOrEmpty(label)) { + throw new DdlException("No label selected."); + } + + // FIXME(cmy) + // checkReadPriv(authInfo.fullUserName, fullDbName); + + Database db = Catalog.getInstance().getDb(fullDbName); + if (db == null) { + throw new DdlException("unknown database, database=" + dbName); + } + + String state = Catalog.getCurrentGlobalTransactionMgr().getLabelState(db.getId(), label).toString(); + + sendResult(request, response, new Result(state)); + } + + private static class Result extends RestBaseResult { + private String state; + public Result(String state) { + this.state = state; + } + } +} diff --git a/fe/src/main/java/com/baidu/palo/planner/EsScanNode.java b/fe/src/main/java/com/baidu/palo/planner/EsScanNode.java new file mode 100644 index 0000000000..9ad4729fa6 --- /dev/null +++ b/fe/src/main/java/com/baidu/palo/planner/EsScanNode.java @@ -0,0 +1,258 @@ +// Copyright (c) 2017, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 com.baidu.palo.planner; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.Set; +import java.util.stream.Collectors; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import com.baidu.palo.analysis.Analyzer; +import com.baidu.palo.analysis.BinaryPredicate; +import com.baidu.palo.analysis.CompoundPredicate; +import com.baidu.palo.analysis.Expr; +import com.baidu.palo.analysis.LiteralExpr; +import com.baidu.palo.analysis.SlotRef; +import com.baidu.palo.analysis.TupleDescriptor; +import com.baidu.palo.catalog.Catalog; +import com.baidu.palo.catalog.EsTable; +import com.baidu.palo.catalog.PartitionInfo; +import com.baidu.palo.catalog.PartitionKey; +import com.baidu.palo.catalog.RangePartitionInfo; +import com.baidu.palo.common.AnalysisException; +import com.baidu.palo.common.UserException; +import com.baidu.palo.external.EsIndexState; +import com.baidu.palo.external.EsShardRouting; +import com.baidu.palo.external.EsTableState; +import com.baidu.palo.system.Backend; +import com.baidu.palo.thrift.TEsScanNode; +import com.baidu.palo.thrift.TEsScanRange; +import com.baidu.palo.thrift.TExpr; +import com.baidu.palo.thrift.TExprNode; +import com.baidu.palo.thrift.TExprNodeType; +import com.baidu.palo.thrift.TExprOpcode; +import com.baidu.palo.thrift.TExtBinaryPredicate; +import com.baidu.palo.thrift.TExtColumnDesc; +import com.baidu.palo.thrift.TExtLiteral; +import com.baidu.palo.thrift.TExtPredicate; +import com.baidu.palo.thrift.TExtPrepareParams; +import com.baidu.palo.thrift.TExtPrepareResult; +import com.baidu.palo.thrift.TNetworkAddress; +import com.baidu.palo.thrift.TPlanNode; +import com.baidu.palo.thrift.TPlanNodeType; +import com.baidu.palo.thrift.TScanRange; +import com.baidu.palo.thrift.TScanRangeLocation; +import com.baidu.palo.thrift.TScanRangeLocations; +import com.baidu.palo.thrift.TSlotDescriptor; +import com.baidu.palo.thrift.TStatus; +import com.google.common.collect.HashMultimap; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.collect.Multimap; +import com.google.common.collect.Range; +import com.google.common.collect.Sets; + +public class EsScanNode extends ScanNode { + + private static final Logger LOG = LogManager.getLogger(EsScanNode.class); + + private final Random random = new Random(System.currentTimeMillis()); + private Multimap backendMap; + private List backendList; + private EsTableState esTableState; + private List shardScanRanges = Lists.newArrayList(); + private EsTable table; + + boolean isFinalized = false; + + public EsScanNode(PlanNodeId id, TupleDescriptor desc, String planNodeName) { + super(id, desc, planNodeName); + table = (EsTable) (desc.getTable()); + esTableState = table.getEsTableState(); + } + + @Override + public void init(Analyzer analyzer) throws UserException { + super.init(analyzer); + + assignBackends(); + } + + @Override + public List getScanRangeLocations(long maxScanRangeLength) { + try { + return getShardLocations(); + } catch (UserException e) { + LOG.error("errors while get es shard locations", e); + } + return null; + } + + @Override + public void finalize(Analyzer analyzer) throws UserException { + if (isFinalized) { + return; + } + + try { + shardScanRanges = getShardLocations(); + } catch (AnalysisException e) { + throw new UserException(e.getMessage()); + } + + isFinalized = true; + } + + @Override + protected void toThrift(TPlanNode msg) { + msg.node_type = TPlanNodeType.ES_SCAN_NODE; + Map properties = Maps.newHashMap(); + properties.put(EsTable.USER, table.getUserName()); + properties.put(EsTable.PASSWORD, table.getPasswd()); + TEsScanNode esScanNode = new TEsScanNode(desc.getId().asInt()); + esScanNode.setProperties(properties); + msg.es_scan_node = esScanNode; + } + + // TODO(ygl) assign backend that belong to the same cluster + private void assignBackends() throws UserException { + backendMap = HashMultimap.create(); + backendList = Lists.newArrayList(); + for (Backend be : Catalog.getCurrentSystemInfo().getIdToBackend().values()) { + if (be.isAlive()) { + backendMap.put(be.getHost(), be); + backendList.add(be); + } + } + if (backendMap.isEmpty()) { + throw new UserException("No Alive backends"); + } + } + + // only do partition(es index level) prune + // TODO (ygl) should not get all shards, prune unrelated shard + private List getShardLocations() throws UserException { + // has to get partition info from es state not from table because the partition info is generated from es cluster state dynamically + Collection partitionIds = partitionPrune(esTableState.getPartitionInfo()); + List selectedIndex = Lists.newArrayList(); + ArrayList unPartitionedIndices = Lists.newArrayList(); + ArrayList partitionedIndices = Lists.newArrayList(); + for (EsIndexState esIndexState : esTableState.getUnPartitionedIndexStates().values()) { + selectedIndex.add(esIndexState); + unPartitionedIndices.add(esIndexState.getIndexName()); + } + if (partitionIds != null) { + for (Long partitionId : partitionIds) { + EsIndexState indexState = esTableState.getIndexState(partitionId); + selectedIndex.add(indexState); + partitionedIndices.add(indexState.getIndexName()); + } + } + if (LOG.isDebugEnabled()) { + LOG.debug("partition prune finished, unpartitioned index [{}], " + + "partitioned index [{}]", + String.join(",", unPartitionedIndices), + String.join(",", partitionedIndices)); + } + int beIndex = random.nextInt(backendList.size()); + List result = Lists.newArrayList(); + for (EsIndexState indexState : selectedIndex) { + for (List shardRouting : indexState.getShardRoutings().values()) { + // get backends + Set colocatedBes = Sets.newHashSet(); + int numBe = Math.min(3, backendMap.size()); + List shardAllocations = shardRouting.stream().map(e -> e.getAddress()) + .collect(Collectors.toList()); + Collections.shuffle(shardAllocations, random); + for (TNetworkAddress address : shardAllocations) { + colocatedBes.addAll(backendMap.get(address.getHostname())); + } + boolean usingRandomBackend = colocatedBes.size() == 0; + List candidateBeList = Lists.newArrayList(); + if (usingRandomBackend) { + for (int i = 0; i < numBe; ++i) { + candidateBeList.add(backendList.get(beIndex++ % numBe)); + } + } else { + candidateBeList.addAll(colocatedBes); + Collections.shuffle(candidateBeList); + } + + // Locations + TScanRangeLocations locations = new TScanRangeLocations(); + for (int i = 0; i < numBe && i < candidateBeList.size(); ++i) { + TScanRangeLocation location = new TScanRangeLocation(); + Backend be = candidateBeList.get(i); + location.setBackend_id(be.getId()); + location.setServer(new TNetworkAddress(be.getHost(), be.getBePort())); + locations.addToLocations(location); + } + + // Generate on es scan range + TEsScanRange esScanRange = new TEsScanRange(); + esScanRange.setEs_hosts(shardAllocations); + esScanRange.setIndex(indexState.getIndexName()); + esScanRange.setType(table.getMappingType()); + esScanRange.setShard_id(shardRouting.get(0).getShardId()); + // Scan range + TScanRange scanRange = new TScanRange(); + scanRange.setEs_scan_range(esScanRange); + locations.setScan_range(scanRange); + // result + result.add(locations); + } + + } + LOG.debug("generate [{}] scan ranges to scan node [{}]", result.size(), result.get(0)); + return result; + } + + /** + * if the index name is an alias or index pattern, then the es table is related + * with one or more indices some indices could be pruned by using partition info + * in index settings currently only support range partition setting + * + * @param partitionInfo + * @return + * @throws AnalysisException + */ + private Collection partitionPrune(PartitionInfo partitionInfo) throws AnalysisException { + PartitionPruner partitionPruner = null; + switch (partitionInfo.getType()) { + case RANGE: { + RangePartitionInfo rangePartitionInfo = (RangePartitionInfo) partitionInfo; + Map> keyRangeById = rangePartitionInfo.getIdToRange(); + partitionPruner = new RangePartitionPruner(keyRangeById, rangePartitionInfo.getPartitionColumns(), + columnFilters); + return partitionPruner.prune(); + } + case UNPARTITIONED: { + return null; + } + default: { + return null; + } + } + } +} diff --git a/fe/src/main/java/com/baidu/palo/planner/OlapTableSink.java b/fe/src/main/java/com/baidu/palo/planner/OlapTableSink.java new file mode 100644 index 0000000000..6f7fd8a086 --- /dev/null +++ b/fe/src/main/java/com/baidu/palo/planner/OlapTableSink.java @@ -0,0 +1,303 @@ +// Copyright (c) 2017, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 com.baidu.palo.planner; + +import com.baidu.palo.analysis.SlotDescriptor; +import com.baidu.palo.analysis.TupleDescriptor; +import com.baidu.palo.catalog.Catalog; +import com.baidu.palo.catalog.Column; +import com.baidu.palo.catalog.DistributionInfo; +import com.baidu.palo.catalog.HashDistributionInfo; +import com.baidu.palo.catalog.MaterializedIndex; +import com.baidu.palo.catalog.OlapTable; +import com.baidu.palo.catalog.Partition; +import com.baidu.palo.catalog.PartitionKey; +import com.baidu.palo.catalog.PartitionType; +import com.baidu.palo.catalog.RangePartitionInfo; +import com.baidu.palo.catalog.Tablet; +import com.baidu.palo.common.AnalysisException; +import com.baidu.palo.common.ErrorCode; +import com.baidu.palo.common.ErrorReport; +import com.baidu.palo.common.UserException; +import com.baidu.palo.system.Backend; +import com.baidu.palo.system.SystemInfoService; +import com.baidu.palo.thrift.TDataSink; +import com.baidu.palo.thrift.TDataSinkType; +import com.baidu.palo.thrift.TExplainLevel; +import com.baidu.palo.thrift.TNodeInfo; +import com.baidu.palo.thrift.TOlapTableIndexSchema; +import com.baidu.palo.thrift.TOlapTableIndexTablets; +import com.baidu.palo.thrift.TOlapTableLocationParam; +import com.baidu.palo.thrift.TOlapTablePartition; +import com.baidu.palo.thrift.TOlapTablePartitionParam; +import com.baidu.palo.thrift.TOlapTableSchemaParam; +import com.baidu.palo.thrift.TOlapTableSink; +import com.baidu.palo.thrift.TPaloNodesInfo; +import com.baidu.palo.thrift.TTabletLocation; +import com.baidu.palo.thrift.TUniqueId; + +import com.google.common.base.Preconditions; +import com.google.common.base.Strings; +import com.google.common.collect.Lists; +import com.google.common.collect.Range; +import com.google.common.collect.Sets; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +public class OlapTableSink extends DataSink { + private static final Logger LOG = LogManager.getLogger(OlapTableSink.class); + + // input variables + private OlapTable dstTable; + private TupleDescriptor tupleDescriptor; + private String partitions; + private Set partitionSet; + + // set after init called + private TDataSink tDataSink; + + public OlapTableSink(OlapTable dstTable, TupleDescriptor tupleDescriptor) { + this.dstTable = dstTable; + this.tupleDescriptor = tupleDescriptor; + } + + public OlapTableSink(OlapTable dstTable, TupleDescriptor tupleDescriptor, String partitions) { + this.dstTable = dstTable; + this.tupleDescriptor = tupleDescriptor; + this.partitions = partitions; + } + + public void init(TUniqueId loadId, long txnId, long dbId) throws AnalysisException { + TOlapTableSink tSink = new TOlapTableSink(); + tSink.setLoad_id(loadId); + tSink.setTxn_id(txnId); + tSink.setDb_id(dbId); + tDataSink = new TDataSink(TDataSinkType.DATA_SPLIT_SINK); + tDataSink.setType(TDataSinkType.OLAP_TABLE_SINK); + tDataSink.setOlap_table_sink(tSink); + + // check partition + if (partitions != null) { + if (dstTable.getPartitionInfo().getType() == PartitionType.UNPARTITIONED) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_PARTITION_CLAUSE_NO_ALLOWED); + } + partitionSet = Sets.newHashSet(); + String[] partNames = partitions.trim().split("\\s*,\\s*"); + for (String partName : partNames) { + Partition part = dstTable.getPartition(partName); + if (part == null) { + ErrorReport.reportAnalysisException( + ErrorCode.ERR_UNKNOWN_PARTITION, partName, dstTable.getName()); + } + partitionSet.add(partName); + } + if (partitionSet.isEmpty()) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_PARTITION_CLAUSE_ON_NONPARTITIONED); + } + } + } + + // must called after tupleDescriptor is computed + public void finalize() throws UserException { + TOlapTableSink tSink = tDataSink.getOlap_table_sink(); + + tSink.setTable_id(dstTable.getId()); + tSink.setTuple_id(tupleDescriptor.getId().asInt()); + int numReplicas = 1; + for (Partition partition : dstTable.getPartitions()) { + numReplicas = dstTable.getPartitionInfo().getReplicationNum(partition.getId()); + break; + } + tSink.setNum_replicas(numReplicas); + tSink.setNeed_gen_rollup(dstTable.shouldLoadToNewRollup()); + tSink.setSchema(createSchema(tSink.getDb_id(), dstTable)); + tSink.setPartition(createPartition(tSink.getDb_id(), dstTable)); + tSink.setLocation(createLocation(dstTable)); + tSink.setNodes_info(createPaloNodesInfo()); + } + + @Override + public String getExplainString(String prefix, TExplainLevel explainLevel) { + StringBuilder strBuilder = new StringBuilder(); + strBuilder.append(prefix + "OLAP TABLE SINK\n"); + strBuilder.append(prefix + " TUPLE ID: " + tupleDescriptor.getId() + "\n"); + strBuilder.append(prefix + " " + DataPartition.RANDOM.getExplainString(explainLevel)); + return strBuilder.toString(); + } + + @Override + public PlanNodeId getExchNodeId() { + return null; + } + + @Override + public DataPartition getOutputPartition() { + return DataPartition.RANDOM; + } + + @Override + protected TDataSink toThrift() { + return tDataSink; + } + + private TOlapTableSchemaParam createSchema(long dbId, OlapTable table) { + TOlapTableSchemaParam schemaParam = new TOlapTableSchemaParam(); + schemaParam.setDb_id(dbId); + schemaParam.setTable_id(table.getId()); + schemaParam.setVersion(0); + + schemaParam.tuple_desc = tupleDescriptor.toThrift(); + for (SlotDescriptor slotDesc : tupleDescriptor.getSlots()) { + schemaParam.addToSlot_descs(slotDesc.toThrift()); + } + + for (Map.Entry> pair : table.getIndexIdToSchema().entrySet()) { + List columns = Lists.newArrayList(); + columns.addAll(pair.getValue().stream().map(Column::getName).collect(Collectors.toList())); + schemaParam.addToIndexes(new TOlapTableIndexSchema(pair.getKey(), columns, + table.getSchemaHashByIndexId(pair.getKey()))); + } + return schemaParam; + } + + private List getDistColumns(DistributionInfo distInfo, OlapTable table) throws UserException { + List distColumns = Lists.newArrayList(); + switch (distInfo.getType()) { + case HASH: { + HashDistributionInfo hashDistributionInfo = (HashDistributionInfo) distInfo; + for (Column column : hashDistributionInfo.getDistributionColumns()) { + distColumns.add(column.getName()); + } + break; + } + case RANDOM: { + for (Column column : table.getBaseSchema()) { + distColumns.add(column.getName()); + } + break; + } + default: + throw new UserException("unsupported distributed type, type=" + distInfo.getType()); + } + return distColumns; + } + + private TOlapTablePartitionParam createPartition(long dbId, OlapTable table) throws UserException { + TOlapTablePartitionParam partitionParam = new TOlapTablePartitionParam(); + partitionParam.setDb_id(dbId); + partitionParam.setTable_id(table.getId()); + partitionParam.setVersion(0); + + PartitionType partType = table.getPartitionInfo().getType(); + switch (partType) { + case RANGE: { + RangePartitionInfo rangePartitionInfo = (RangePartitionInfo) table.getPartitionInfo(); + // range partition's only has one column + Preconditions.checkArgument(rangePartitionInfo.getPartitionColumns().size() == 1, + "number columns of range partition is not 1, number_columns=" + + rangePartitionInfo.getPartitionColumns().size()); + partitionParam.setPartition_column(rangePartitionInfo.getPartitionColumns().get(0).getName()); + + DistributionInfo selectedDistInfo = null; + for (Partition partition : table.getPartitions()) { + if (partitionSet != null && !partitionSet.contains(partition.getName())) { + continue; + } + TOlapTablePartition tPartition = new TOlapTablePartition(); + tPartition.setId(partition.getId()); + Range range = rangePartitionInfo.getRange(partition.getId()); + if (range.hasLowerBound()) { + tPartition.setStart_key(range.lowerEndpoint().getKeys().get(0).treeToThrift().getNodes().get(0)); + } + if (range.hasUpperBound() && !range.upperEndpoint().isMaxValue()) { + tPartition.setEnd_key(range.upperEndpoint().getKeys().get(0).treeToThrift().getNodes().get(0)); + } + for (MaterializedIndex index : partition.getMaterializedIndices()) { + tPartition.addToIndexes(new TOlapTableIndexTablets(index.getId(), Lists.newArrayList( + index.getTablets().stream().map(Tablet::getId).collect(Collectors.toList())))); + tPartition.setNum_buckets(index.getTablets().size()); + } + partitionParam.addToPartitions(tPartition); + + DistributionInfo distInfo = partition.getDistributionInfo(); + if (selectedDistInfo == null) { + partitionParam.setDistributed_columns(getDistColumns(distInfo, table)); + selectedDistInfo = distInfo; + } else { + if (selectedDistInfo.getType() != distInfo.getType()) { + throw new UserException("different distribute types in two different partitions, type1=" + + selectedDistInfo.getType() + ", type2=" + distInfo.getType()); + } + } + } + break; + } + case UNPARTITIONED: { + // there is no partition columns for single partition + Preconditions.checkArgument(table.getPartitions().size() == 1, + "Number of table partitions is not 1 for unpartitioned table, partitionNum=" + + table.getPartitions().size()); + Partition partition = table.getPartitions().iterator().next(); + + TOlapTablePartition tPartition = new TOlapTablePartition(); + tPartition.setId(partition.getId()); + // No lowerBound and upperBound for this range + for (MaterializedIndex index : partition.getMaterializedIndices()) { + tPartition.addToIndexes(new TOlapTableIndexTablets(index.getId(), Lists.newArrayList( + index.getTablets().stream().map(Tablet::getId).collect(Collectors.toList())))); + tPartition.setNum_buckets(index.getTablets().size()); + } + partitionParam.addToPartitions(tPartition); + partitionParam.setDistributed_columns( + getDistColumns(partition.getDistributionInfo(), table)); + break; + } + default: { + throw new UserException("unsupported partition for OlapTable, partition=" + partType); + } + } + return partitionParam; + } + + private TOlapTableLocationParam createLocation(OlapTable table) { + TOlapTableLocationParam locationParam = new TOlapTableLocationParam(); + for (Partition partition : table.getPartitions()) { + for (MaterializedIndex index : partition.getMaterializedIndices()) { + for (Tablet tablet : index.getTablets()) { + locationParam.addToTablets( + new TTabletLocation(tablet.getId(), Lists.newArrayList(tablet.getBackendIds()))); + } + } + } + return locationParam; + } + + private TPaloNodesInfo createPaloNodesInfo() { + TPaloNodesInfo nodesInfo = new TPaloNodesInfo(); + SystemInfoService systemInfoService = Catalog.getCurrentSystemInfo(); + for (Long id : systemInfoService.getBackendIds(false)) { + Backend backend = systemInfoService.getBackend(id); + nodesInfo.addToNodes(new TNodeInfo(backend.getId(), 0, backend.getHost(), backend.getBrpcPort())); + } + return nodesInfo; + } + +} diff --git a/fe/src/main/java/com/baidu/palo/planner/StreamLoadPlanner.java b/fe/src/main/java/com/baidu/palo/planner/StreamLoadPlanner.java new file mode 100644 index 0000000000..74a49cd706 --- /dev/null +++ b/fe/src/main/java/com/baidu/palo/planner/StreamLoadPlanner.java @@ -0,0 +1,142 @@ +// Copyright (c) 2017, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 com.baidu.palo.planner; + +import com.baidu.palo.analysis.Analyzer; +import com.baidu.palo.analysis.DescriptorTable; +import com.baidu.palo.analysis.SlotDescriptor; +import com.baidu.palo.analysis.TupleDescriptor; +import com.baidu.palo.catalog.Catalog; +import com.baidu.palo.catalog.Column; +import com.baidu.palo.catalog.Database; +import com.baidu.palo.catalog.OlapTable; +import com.baidu.palo.common.UserException; +import com.baidu.palo.thrift.PaloInternalServiceVersion; +import com.baidu.palo.thrift.TExecPlanFragmentParams; +import com.baidu.palo.thrift.TPlanFragmentExecParams; +import com.baidu.palo.thrift.TQueryGlobals; +import com.baidu.palo.thrift.TQueryOptions; +import com.baidu.palo.thrift.TQueryType; +import com.baidu.palo.thrift.TScanRangeLocations; +import com.baidu.palo.thrift.TScanRangeParams; +import com.baidu.palo.thrift.TStreamLoadPutRequest; +import com.baidu.palo.thrift.TUniqueId; + +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.text.DateFormat; +import java.text.SimpleDateFormat; +import java.util.Date; +import java.util.List; +import java.util.Map; +import java.util.UUID; + +// Used to generate a plan fragment for a streaming load. +// we only support OlapTable now. +// TODO(zc): support other type table +public class StreamLoadPlanner { + private static final Logger LOG = LogManager.getLogger(StreamLoadPlanner.class); + private static final DateFormat DATE_FORMAT = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"); + + // destination Db and table get from request + // Data will load to this table + private Database db; + private OlapTable destTable; + private TStreamLoadPutRequest request; + + private Analyzer analyzer; + private DescriptorTable descTable; + + public StreamLoadPlanner(Database db, OlapTable destTable, TStreamLoadPutRequest request) { + this.db = db; + this.destTable = destTable; + this.request = request; + + analyzer = new Analyzer(Catalog.getInstance(), null); + descTable = analyzer.getDescTbl(); + } + + public TExecPlanFragmentParams plan() throws UserException { + // construct tuple descriptor, used for scanNode and dataSink + TupleDescriptor tupleDesc = descTable.createTupleDescriptor("DstTableTuple"); + for (Column col : destTable.getBaseSchema()) { + SlotDescriptor slotDesc = descTable.addSlotDescriptor(tupleDesc); + slotDesc.setIsMaterialized(true); + slotDesc.setColumn(col); + if (col.isAllowNull()) { + slotDesc.setIsNullable(true); + } else { + slotDesc.setIsNullable(false); + } + } + + // create scan node + StreamLoadScanNode scanNode = new StreamLoadScanNode(new PlanNodeId(0), tupleDesc, destTable, request); + scanNode.init(analyzer); + descTable.computeMemLayout(); + scanNode.finalize(analyzer); + + // create dest sink + OlapTableSink olapTableSink = new OlapTableSink(destTable, tupleDesc, request.getPartitions()); + olapTableSink.init(request.getLoadId(), request.getTxnId(), db.getId()); + olapTableSink.finalize(); + + // for stream load, we only need one fragment, ScanNode -> DataSink. + // OlapTableSink can dispatch data to corresponding node. + PlanFragment fragment = new PlanFragment(new PlanFragmentId(0), scanNode, DataPartition.UNPARTITIONED); + fragment.setSink(olapTableSink); + + fragment.finalize(null, false); + + TExecPlanFragmentParams params = new TExecPlanFragmentParams(); + params.setProtocol_version(PaloInternalServiceVersion.V1); + params.setFragment(fragment.toThrift()); + + params.setDesc_tbl(analyzer.getDescTbl().toThrift()); + + TPlanFragmentExecParams execParams = new TPlanFragmentExecParams(); + // Only use fragment id + UUID uuid = UUID.randomUUID(); + TUniqueId queryId = new TUniqueId(uuid.getMostSignificantBits(), uuid.getLeastSignificantBits()); + execParams.setQuery_id(queryId); + execParams.setFragment_instance_id(new TUniqueId(queryId.hi, queryId.lo + 1)); + execParams.per_exch_num_senders = Maps.newHashMap(); + execParams.destinations = Lists.newArrayList(); + Map> perNodeScanRange = Maps.newHashMap(); + List scanRangeParams = Lists.newArrayList(); + for (TScanRangeLocations locations : scanNode.getScanRangeLocations(0)) { + scanRangeParams.add(new TScanRangeParams(locations.getScan_range())); + } + // For stream load, only one sender + execParams.setSender_id(0); + execParams.setNum_senders(1); + perNodeScanRange.put(scanNode.getId().asInt(), scanRangeParams); + execParams.setPer_node_scan_ranges(perNodeScanRange); + params.setParams(execParams); + TQueryOptions queryOptions = new TQueryOptions(); + queryOptions.setQuery_type(TQueryType.LOAD); + params.setQuery_options(queryOptions); + TQueryGlobals queryGlobals = new TQueryGlobals(); + queryGlobals.setNow_string(DATE_FORMAT.format(new Date())); + params.setQuery_globals(queryGlobals); + + LOG.info("params is {}", params); + return params; + } +} diff --git a/fe/src/main/java/com/baidu/palo/planner/StreamLoadScanNode.java b/fe/src/main/java/com/baidu/palo/planner/StreamLoadScanNode.java new file mode 100644 index 0000000000..59969b88dc --- /dev/null +++ b/fe/src/main/java/com/baidu/palo/planner/StreamLoadScanNode.java @@ -0,0 +1,374 @@ +// Copyright (c) 2017, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 com.baidu.palo.planner; + +import com.baidu.palo.analysis.Analyzer; +import com.baidu.palo.analysis.CastExpr; +import com.baidu.palo.analysis.ColumnSeparator; +import com.baidu.palo.analysis.Expr; +import com.baidu.palo.analysis.ExprSubstitutionMap; +import com.baidu.palo.analysis.FunctionCallExpr; +import com.baidu.palo.analysis.ImportColumnDesc; +import com.baidu.palo.analysis.ImportColumnsStmt; +import com.baidu.palo.analysis.ImportWhereStmt; +import com.baidu.palo.analysis.NullLiteral; +import com.baidu.palo.analysis.SlotDescriptor; +import com.baidu.palo.analysis.SlotRef; +import com.baidu.palo.analysis.SqlParser; +import com.baidu.palo.analysis.SqlScanner; +import com.baidu.palo.analysis.StringLiteral; +import com.baidu.palo.analysis.TupleDescriptor; +import com.baidu.palo.catalog.Column; +import com.baidu.palo.catalog.PrimitiveType; +import com.baidu.palo.catalog.ScalarType; +import com.baidu.palo.catalog.Table; +import com.baidu.palo.catalog.Type; +import com.baidu.palo.common.AnalysisException; +import com.baidu.palo.common.UserException; +import com.baidu.palo.thrift.TBrokerRangeDesc; +import com.baidu.palo.thrift.TBrokerScanNode; +import com.baidu.palo.thrift.TBrokerScanRange; +import com.baidu.palo.thrift.TBrokerScanRangeParams; +import com.baidu.palo.thrift.TExplainLevel; +import com.baidu.palo.thrift.TPlanNode; +import com.baidu.palo.thrift.TPlanNodeType; +import com.baidu.palo.thrift.TScanRange; +import com.baidu.palo.thrift.TScanRangeLocations; +import com.baidu.palo.thrift.TStreamLoadPutRequest; + +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.io.StringReader; +import java.nio.charset.Charset; +import java.util.List; +import java.util.Map; + +/** + * used to scan from stream + */ +public class StreamLoadScanNode extends ScanNode { + private static final Logger LOG = LogManager.getLogger(StreamLoadScanNode.class); + + // TODO(zc): now we use scanRange + // input parameter + private Table dstTable; + private TStreamLoadPutRequest request; + + // helper + private Analyzer analyzer; + private TupleDescriptor srcTupleDesc; + private TBrokerScanRange brokerScanRange; + + private Map slotDescByName = Maps.newHashMap(); + private Map exprsByName = Maps.newHashMap(); + + // used to construct for streaming loading + public StreamLoadScanNode( + PlanNodeId id, TupleDescriptor tupleDesc, Table dstTable, TStreamLoadPutRequest request) { + super(id, tupleDesc, "StreamLoadScanNode"); + this.dstTable = dstTable; + this.request = request; + } + + @Override + public void init(Analyzer analyzer) throws UserException { + // can't call super.init(), because after super.init, conjuncts would be null + assignConjuncts(analyzer); + + this.analyzer = analyzer; + brokerScanRange = new TBrokerScanRange(); + + TBrokerRangeDesc rangeDesc = new TBrokerRangeDesc(); + rangeDesc.file_type = request.getFileType(); + rangeDesc.format_type = request.getFormatType(); + rangeDesc.splittable = false; + switch (request.getFileType()) { + case FILE_LOCAL: + rangeDesc.path = request.getPath(); + break; + case FILE_STREAM: + rangeDesc.path = "Invalid Path"; + rangeDesc.load_id = request.getLoadId(); + break; + default: + throw new UserException("unsupported file type, type=" + request.getFileType()); + } + rangeDesc.start_offset = 0; + rangeDesc.size = -1; + brokerScanRange.addToRanges(rangeDesc); + + srcTupleDesc = analyzer.getDescTbl().createTupleDescriptor("StreamLoadScanNode"); + + TBrokerScanRangeParams params = new TBrokerScanRangeParams(); + + // parse columns header. this contain map from input column to column of destination table + // columns: k1, k2, v1, v2=k1 + k2 + // this means that there are three columns(k1, k2, v1) in source file, + // and v2 is derived from (k1 + k2) + if (request.isSetColumns()) { + String columnsSQL = new String("COLUMNS " + request.getColumns()); + SqlParser parser = new SqlParser(new SqlScanner(new StringReader(columnsSQL))); + ImportColumnsStmt columnsStmt; + try { + columnsStmt = (ImportColumnsStmt) parser.parse().value; + } catch (Error e) { + LOG.warn("error happens when parsing columns, sql={}", columnsSQL, e); + throw new AnalysisException("failed to parsing columns' header, maybe contain unsupported character"); + } catch (AnalysisException e) { + LOG.warn("analyze columns' statement failed, sql={}, error={}", + columnsSQL, parser.getErrorMsg(columnsSQL), e); + String errorMessage = parser.getErrorMsg(columnsSQL); + if (errorMessage == null) { + throw e; + } else { + throw new AnalysisException(errorMessage, e); + } + } catch (Exception e) { + LOG.warn("failed to parse columns header, sql={}", columnsSQL, e); + throw new UserException("parse columns header failed", e); + } + + for (ImportColumnDesc columnDesc : columnsStmt.getColumns()) { + if (columnDesc.getExpr() != null) { + exprsByName.put(columnDesc.getColumn(), columnDesc.getExpr()); + } else { + SlotDescriptor slotDesc = analyzer.getDescTbl().addSlotDescriptor(srcTupleDesc); + slotDesc.setType(ScalarType.createType(PrimitiveType.VARCHAR)); + slotDesc.setIsMaterialized(true); + slotDesc.setIsNullable(false); + params.addToSrc_slot_ids(slotDesc.getId().asInt()); + slotDescByName.put(columnDesc.getColumn(), slotDesc); + } + } + + // analyze all exprs + for (Map.Entry entry : exprsByName.entrySet()) { + ExprSubstitutionMap smap = new ExprSubstitutionMap(); + List slots = Lists.newArrayList(); + entry.getValue().collect(SlotRef.class, slots); + for (SlotRef slot : slots) { + SlotDescriptor slotDesc = slotDescByName.get(slot.getColumnName()); + if (slotDesc == null) { + throw new UserException("unknown reference column, column=" + entry.getKey() + + ", reference=" + slot.getColumnName()); + } + smap.getLhs().add(slot); + smap.getRhs().add(new SlotRef(slotDesc)); + } + Expr expr = entry.getValue().clone(smap); + expr.analyze(analyzer); + + // check if contain aggregation + List funcs = Lists.newArrayList(); + expr.collect(FunctionCallExpr.class, funcs); + for (FunctionCallExpr fn : funcs) { + if (fn.isAggregateFunction()) { + throw new AnalysisException("Don't support aggregation function in load expression"); + } + } + + exprsByName.put(entry.getKey(), expr); + } + } else { + for (Column column : dstTable.getBaseSchema()) { + SlotDescriptor slotDesc = analyzer.getDescTbl().addSlotDescriptor(srcTupleDesc); + slotDesc.setType(ScalarType.createType(PrimitiveType.VARCHAR)); + slotDesc.setIsMaterialized(true); + slotDesc.setIsNullable(false); + params.addToSrc_slot_ids(slotDesc.getId().asInt()); + + slotDescByName.put(column.getName(), slotDesc); + } + } + + // analyze where statement + if (request.isSetWhere()) { + Map dstDescMap = Maps.newHashMap(); + for (SlotDescriptor slotDescriptor : desc.getSlots()) { + dstDescMap.put(slotDescriptor.getColumn().getName(), slotDescriptor); + } + + String whereSQL = new String("WHERE " + request.getWhere()); + SqlParser parser = new SqlParser(new SqlScanner(new StringReader(whereSQL))); + ImportWhereStmt whereStmt; + try { + whereStmt = (ImportWhereStmt) parser.parse().value; + } catch (Error e) { + LOG.warn("error happens when parsing where header, sql={}", whereSQL, e); + throw new AnalysisException("failed to parsing where header, maybe contain unsupported character"); + } catch (AnalysisException e) { + LOG.warn("analyze where statement failed, sql={}, error={}", + whereSQL, parser.getErrorMsg(whereSQL), e); + String errorMessage = parser.getErrorMsg(whereSQL); + if (errorMessage == null) { + throw e; + } else { + throw new AnalysisException(errorMessage, e); + } + } catch (Exception e) { + LOG.warn("failed to parse where header, sql={}", whereSQL, e); + throw new UserException("parse columns header failed", e); + } + + // substitute SlotRef in filter expression + Expr whereExpr = whereStmt.getExpr(); + + List slots = Lists.newArrayList(); + whereExpr.collect(SlotRef.class, slots); + + ExprSubstitutionMap smap = new ExprSubstitutionMap(); + for (SlotRef slot : slots) { + SlotDescriptor slotDesc = dstDescMap.get(slot.getColumnName()); + if (slotDesc == null) { + throw new UserException("unknown column reference in where statement, reference=" + + slot.getColumnName()); + } + smap.getLhs().add(slot); + smap.getRhs().add(new SlotRef(slotDesc)); + } + whereExpr= whereExpr.clone(smap); + whereExpr.analyze(analyzer); + if (whereExpr.getType() != Type.BOOLEAN) { + throw new UserException("where statement is not a valid statement return bool"); + } + addConjuncts(whereExpr.getConjuncts()); + } + + computeStats(analyzer); + createDefaultSmap(analyzer); + + if (request.isSetColumnSeparator()) { + String sep = ColumnSeparator.convertSeparator(request.getColumnSeparator()); + params.setColumn_separator(sep.getBytes(Charset.forName("UTF-8"))[0]); + } else { + params.setColumn_separator((byte) '\t'); + } + params.setLine_delimiter((byte) '\n'); + params.setSrc_tuple_id(srcTupleDesc.getId().asInt()); + params.setDest_tuple_id(desc.getId().asInt()); + brokerScanRange.setParams(params); + + brokerScanRange.setBroker_addresses(Lists.newArrayList()); + } + + @Override + public void finalize(Analyzer analyzer) throws UserException, UserException { + finalizeParams(); + } + + private void finalizeParams() throws UserException { + for (SlotDescriptor dstSlotDesc : desc.getSlots()) { + if (!dstSlotDesc.isMaterialized()) { + continue; + } + Expr expr = null; + if (exprsByName != null) { + expr = exprsByName.get(dstSlotDesc.getColumn().getName()); + } + if (expr == null) { + SlotDescriptor srcSlotDesc = slotDescByName.get(dstSlotDesc.getColumn().getName()); + if (srcSlotDesc != null) { + // If dest is allow null, we set source to nullable + if (dstSlotDesc.getColumn().isAllowNull()) { + srcSlotDesc.setIsNullable(true); + } + expr = new SlotRef(srcSlotDesc); + } else { + Column column = dstSlotDesc.getColumn(); + if (column.getDefaultValue() != null) { + expr = new StringLiteral(dstSlotDesc.getColumn().getDefaultValue()); + } else { + if (column.isAllowNull()) { + expr = NullLiteral.create(column.getType()); + } else { + throw new AnalysisException("column has no source field, column=" + column.getName()); + } + } + } + } + // check hll_hash + if (dstSlotDesc.getType().getPrimitiveType() == PrimitiveType.HLL) { + if (!(expr instanceof FunctionCallExpr)) { + throw new AnalysisException("HLL column must use hll_hash function, like " + + dstSlotDesc.getColumn().getName() + "=hll_hash(xxx)"); + } + FunctionCallExpr fn = (FunctionCallExpr) expr; + if (!fn.getFnName().getFunction().equalsIgnoreCase("hll_hash")) { + throw new AnalysisException("HLL column must use hll_hash function, like " + + dstSlotDesc.getColumn().getName() + "=hll_hash(xxx)"); + } + expr.setType(Type.HLL); + } + expr = castToSlot(dstSlotDesc, expr); + brokerScanRange.params.putToExpr_of_dest_slot(dstSlotDesc.getId().asInt(), expr.treeToThrift()); + } + brokerScanRange.params.setDest_tuple_id(desc.getId().asInt()); + LOG.info("brokerScanRange is {}", brokerScanRange); + + // Need re compute memory layout after set some slot descriptor to nullable + srcTupleDesc.computeMemLayout(); + } + + private Expr castToSlot(SlotDescriptor slotDesc, Expr expr) throws UserException { + PrimitiveType dstType = slotDesc.getType().getPrimitiveType(); + PrimitiveType srcType = expr.getType().getPrimitiveType(); + + if (dstType.isStringType()) { + if (srcType.isStringType()) { + return expr; + } else { + CastExpr castExpr = new CastExpr(Type.VARCHAR, expr, true); + castExpr.analyze(analyzer); + return castExpr; + } + } else if (dstType != srcType) { + CastExpr castExpr = new CastExpr(slotDesc.getType(), expr, true); + castExpr.analyze(analyzer); + return castExpr; + } + + return expr; + } + + @Override + protected void toThrift(TPlanNode planNode) { + planNode.setNode_type(TPlanNodeType.BROKER_SCAN_NODE); + TBrokerScanNode brokerScanNode = new TBrokerScanNode(desc.getId().asInt()); + planNode.setBroker_scan_node(brokerScanNode); + } + + @Override + public List getScanRangeLocations(long maxScanRangeLength) { + TScanRangeLocations locations = new TScanRangeLocations(); + TScanRange scanRange = new TScanRange(); + scanRange.setBroker_scan_range(brokerScanRange); + locations.setScan_range(scanRange); + locations.setLocations(Lists.newArrayList()); + return Lists.newArrayList(locations); + } + + @Override + public int getNumInstances() { return 1; } + + @Override + protected String getNodeExplainString(String prefix, TExplainLevel detailLevel) { + return "StreamLoadScanNode"; + } +} diff --git a/fe/src/main/java/com/baidu/palo/task/ClearAlterTask.java b/fe/src/main/java/com/baidu/palo/task/ClearAlterTask.java new file mode 100644 index 0000000000..285ba6ad0c --- /dev/null +++ b/fe/src/main/java/com/baidu/palo/task/ClearAlterTask.java @@ -0,0 +1,34 @@ +package com.baidu.palo.task; + +import com.baidu.palo.thrift.TClearAlterTaskRequest; +import com.baidu.palo.thrift.TTaskType; + +public class ClearAlterTask extends AgentTask { + private int schemaHash; + private boolean isFinished; + + public ClearAlterTask(long backendId, long dbId, long tableId, long partitionId, long indexId, + long tabletId, int schemaHash) { + super(null, backendId, TTaskType.CLEAR_ALTER_TASK, dbId, tableId, partitionId, indexId, tabletId); + + this.schemaHash = schemaHash; + this.isFinished = false; + } + + public TClearAlterTaskRequest toThrift() { + TClearAlterTaskRequest request = new TClearAlterTaskRequest(tabletId, schemaHash); + return request; + } + + public int getSchemaHash() { + return schemaHash; + } + + public void setFinished() { + this.isFinished = true; + } + + public boolean isFinished() { + return isFinished; + } +} diff --git a/fe/src/main/java/com/baidu/palo/task/ClearTransactionTask.java b/fe/src/main/java/com/baidu/palo/task/ClearTransactionTask.java new file mode 100644 index 0000000000..5f59b814dd --- /dev/null +++ b/fe/src/main/java/com/baidu/palo/task/ClearTransactionTask.java @@ -0,0 +1,40 @@ +package com.baidu.palo.task; + +import java.util.List; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import com.baidu.palo.thrift.TClearTransactionTaskRequest; +import com.baidu.palo.thrift.TTaskType; + +public class ClearTransactionTask extends AgentTask { + + private static final Logger LOG = LogManager.getLogger(ClearTransactionTask.class); + + private long transactionId; + private List partitionIds; + private boolean isFinished; + + public ClearTransactionTask(long backendId, long transactionId, + List partitionIds) { + super(null, backendId, TTaskType.CLEAR_TRANSACTION_TASK, -1L, -1L, -1L, -1L, -1L, transactionId); + this.transactionId = transactionId; + this.partitionIds = partitionIds; + this.isFinished = false; + } + + public TClearTransactionTaskRequest toThrift() { + TClearTransactionTaskRequest clearTransactionTaskRequest = new TClearTransactionTaskRequest(transactionId, + partitionIds); + return clearTransactionTaskRequest; + } + + public void setFinished() { + this.isFinished = true; + } + + public boolean isFinished() { + return this.isFinished; + } +} diff --git a/fe/src/main/java/com/baidu/palo/task/PublishVersionTask.java b/fe/src/main/java/com/baidu/palo/task/PublishVersionTask.java new file mode 100644 index 0000000000..4373434f9e --- /dev/null +++ b/fe/src/main/java/com/baidu/palo/task/PublishVersionTask.java @@ -0,0 +1,62 @@ +package com.baidu.palo.task; + +import java.util.ArrayList; +import java.util.List; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import com.baidu.palo.thrift.TPartitionVersionInfo; +import com.baidu.palo.thrift.TPublishVersionRequest; +import com.baidu.palo.thrift.TTaskType; + +public class PublishVersionTask extends AgentTask { + private static final Logger LOG = LogManager.getLogger(PublishVersionTask.class); + + private long transactionId; + private List partitionVersionInfos; + private List errorTablets; + private boolean isFinished; + + public PublishVersionTask(long backendId, long transactionId, + List partitionVersionInfos) { + super(null, backendId, TTaskType.PUBLISH_VERSION, -1L, -1L, -1L, -1L, -1L, transactionId); + this.transactionId = transactionId; + this.partitionVersionInfos = partitionVersionInfos; + this.errorTablets = new ArrayList(); + this.isFinished = false; + } + + public TPublishVersionRequest toThrift() { + TPublishVersionRequest publishVersionRequest = new TPublishVersionRequest(transactionId, + partitionVersionInfos); + return publishVersionRequest; + } + + public long getTransactionId() { + return transactionId; + } + + public List getPartitionVersionInfos() { + return partitionVersionInfos; + } + + public List getErrorTablets() { + return errorTablets; + } + + public void addErrorTablets(List errorTablets) { + if (errorTablets == null) { + return; + } + this.errorTablets.addAll(errorTablets); + } + + public void setIsFinished(boolean isFinished) { + this.isFinished = isFinished; + } + + public boolean isFinished() { + return isFinished; + } +} diff --git a/fe/src/main/java/com/baidu/palo/task/RecoverTabletTask.java b/fe/src/main/java/com/baidu/palo/task/RecoverTabletTask.java new file mode 100644 index 0000000000..cdaae33e9b --- /dev/null +++ b/fe/src/main/java/com/baidu/palo/task/RecoverTabletTask.java @@ -0,0 +1,42 @@ +// Copyright (c) 2017, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 com.baidu.palo.task; + +import com.baidu.palo.thrift.TRecoverTabletReq; +import com.baidu.palo.thrift.TTaskType; + +public class RecoverTabletTask extends AgentTask { + + private long version; + private long versionHash; + private int schemaHash; + + public RecoverTabletTask(long backendId, long tabletId, long version, long versionHash, int schemaHash) { + super(null, backendId, TTaskType.RECOVER_TABLET, -1L, -1L, -1L, -1L, tabletId, tabletId); + this.version = version; + this.versionHash = versionHash; + this.schemaHash = schemaHash; + } + + public TRecoverTabletReq toThrift() { + TRecoverTabletReq recoverTabletReq = new TRecoverTabletReq(); + recoverTabletReq.setTablet_id(tabletId); + recoverTabletReq.setVersion(version); + recoverTabletReq.setVersion_hash(versionHash); + recoverTabletReq.setSchema_hash(schemaHash); + return recoverTabletReq; + } +} \ No newline at end of file diff --git a/fe/src/main/java/com/baidu/palo/transaction/BeginTransactionException.java b/fe/src/main/java/com/baidu/palo/transaction/BeginTransactionException.java new file mode 100644 index 0000000000..0be0e06718 --- /dev/null +++ b/fe/src/main/java/com/baidu/palo/transaction/BeginTransactionException.java @@ -0,0 +1,16 @@ +package com.baidu.palo.transaction; + +import com.baidu.palo.common.UserException; + +public class BeginTransactionException extends UserException { + + private static final long serialVersionUID = 1L; + + public BeginTransactionException(String msg) { + super(msg); + } + + public BeginTransactionException(String msg, Throwable e) { + super(msg, e); + } +} diff --git a/fe/src/main/java/com/baidu/palo/transaction/GlobalTransactionMgr.java b/fe/src/main/java/com/baidu/palo/transaction/GlobalTransactionMgr.java new file mode 100644 index 0000000000..235fc0de3d --- /dev/null +++ b/fe/src/main/java/com/baidu/palo/transaction/GlobalTransactionMgr.java @@ -0,0 +1,1173 @@ +// Copyright (c) 2017, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 com.baidu.palo.transaction; + +import com.baidu.palo.alter.RollupJob; +import com.baidu.palo.catalog.Catalog; +import com.baidu.palo.catalog.Database; +import com.baidu.palo.catalog.MaterializedIndex; +import com.baidu.palo.catalog.MaterializedIndex.IndexState; +import com.baidu.palo.catalog.OlapTable; +import com.baidu.palo.catalog.OlapTable.OlapTableState; +import com.baidu.palo.catalog.Partition; +import com.baidu.palo.catalog.PartitionInfo; +import com.baidu.palo.catalog.Replica; +import com.baidu.palo.catalog.Table; +import com.baidu.palo.catalog.Tablet; +import com.baidu.palo.catalog.TabletInvertedIndex; +import com.baidu.palo.common.AnalysisException; +import com.baidu.palo.common.Config; +import com.baidu.palo.common.FeNameFormat; +import com.baidu.palo.common.MetaNotFoundException; +import com.baidu.palo.common.UserException; +import com.baidu.palo.common.util.TimeUtils; +import com.baidu.palo.common.util.Util; +import com.baidu.palo.load.Load; +import com.baidu.palo.persist.EditLog; +import com.baidu.palo.task.AgentTaskQueue; +import com.baidu.palo.task.PublishVersionTask; +import com.baidu.palo.thrift.TTaskType; +import com.baidu.palo.transaction.TransactionState.LoadJobSourceType; + +import com.google.common.base.Preconditions; +import com.google.common.collect.HashBasedTable; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.locks.ReentrantReadWriteLock; +import java.util.stream.Collectors; + +/** + * Transaction Manager + * 1. begin + * 2. commit + * 3. abort + * + * Attention: all api in txn manager should get db lock or load lock first, then get txn manager's lock, or there will be dead lock + */ +public class GlobalTransactionMgr { + private static final Logger LOG = LogManager.getLogger(GlobalTransactionMgr.class); + + // the lock is used to control the access to transaction states + // no other locks should be inside this lock + private ReentrantReadWriteLock transactionLock = new ReentrantReadWriteLock(true); + private EditLog editLog; + + // transactionId -> TransactionState + private Map idToTransactionState; + private com.google.common.collect.Table dbIdToTxnLabels; + private Map runningTxnNums; + private TransactionIDGenerator idGenerator; + + private Catalog catalog; + + public GlobalTransactionMgr(Catalog catalog) { + idToTransactionState = new HashMap<>(); + dbIdToTxnLabels = HashBasedTable.create(); + runningTxnNums = Maps.newHashMap(); + this.catalog = catalog; + this.idGenerator = new TransactionIDGenerator(); + } + + /** + * the app could specify the transactionid and + * + * @param coordinator + * @throws BeginTransactionException + * @throws IllegalTransactionParameterException + */ + public long beginTransaction(long dbId, String label, String coordinator, LoadJobSourceType sourceType) + throws AnalysisException, LabelAlreadyExistsException, BeginTransactionException { + + if (Config.disable_load_job) { + throw new BeginTransactionException("disable_load_job is set to true, all load job is prevented"); + } + + writeLock(); + try { + Preconditions.checkNotNull(coordinator); + Preconditions.checkNotNull(label); + FeNameFormat.checkLabel(label); + Map txnLabels = dbIdToTxnLabels.row(dbId); + if (txnLabels != null && txnLabels.containsKey(label)) { + throw new LabelAlreadyExistsException("label already exists, label=" + label); + } + if (runningTxnNums.get(dbId) != null + && runningTxnNums.get(dbId) > Config.max_running_txn_num_per_db) { + throw new BeginTransactionException("current running txns on db " + dbId + " is " + + runningTxnNums.get(dbId) + ", larger than limit " + Config.max_running_txn_num_per_db); + } + long tid = idGenerator.getNextTransactionId(); + LOG.debug("beginTransaction: tid {} with label {} from coordinator {}", tid, label, coordinator); + TransactionState transactionState = new TransactionState(dbId, tid, label, sourceType, coordinator); + transactionState.setPrepareTime(System.currentTimeMillis()); + unprotectUpsertTransactionState(transactionState); + return tid; + } finally { + writeUnlock(); + } + } + + public TransactionStatus getLabelState(long dbId, String label) { + readLock(); + try { + Map txnLabels = dbIdToTxnLabels.row(dbId); + if (txnLabels == null) { + return TransactionStatus.UNKNOWN; + } + Long transactionId = txnLabels.get(label); + if (transactionId == null) { + return TransactionStatus.UNKNOWN; + } + return idToTransactionState.get(transactionId).getTransactionStatus(); + } finally { + readUnlock(); + } + } + + public void deleteTransaction(long transactionId) { + writeLock(); + try { + TransactionState state = idToTransactionState.get(transactionId); + if (state == null) { + return; + } + editLog.logDeleteTransactionState(state); + replayDeleteTransactionState(state); + } finally { + writeUnlock(); + } + } + + /** + * commit transaction process as follows: + * 1. validate whether `Load` is cancelled + * 2. validate whether `Table` is deleted + * 3. validate replicas consistency + * 4. update transaction state version + * 5. persistent transactionState + * 6. update nextVersion because of the failure of persistent transaction resulting in error version + * + * @param transactionId + * @param tabletCommitInfos + * @return + * @throws MetaNotFoundException + * @throws TransactionCommitFailedException + * @note it is necessary to optimize the `lock` mechanism and `lock` scope resulting from wait lock long time + * @note callers should get db.write lock before call this api + */ + public void commitTransaction(long dbId, long transactionId, List tabletCommitInfos) throws MetaNotFoundException, TransactionCommitFailedException { + + if (Config.disable_load_job) { + throw new TransactionCommitFailedException("disable_load_job is set to true, all load job is prevented"); + } + + LOG.debug("try to commit transaction:[{}]", transactionId); + if (tabletCommitInfos == null || tabletCommitInfos.isEmpty()) { + throw new TransactionCommitFailedException("all partitions have no load data"); + } + // 1. check status + // the caller method already own db lock, we not obtain db lock here + Database db = catalog.getDb(dbId); + if (null == db) { + throw new MetaNotFoundException("could not find db [" + dbId + "]"); + } + TransactionState transactionState = idToTransactionState.get(transactionId); + if (transactionState == null + || transactionState.getTransactionStatus() == TransactionStatus.ABORTED) { + throw new TransactionCommitFailedException("Transaction has already been cancelled"); + } + if (transactionState.getTransactionStatus() == TransactionStatus.VISIBLE) { + return; + } + if (transactionState.getTransactionStatus() == TransactionStatus.COMMITTED) { + return; + } + + TabletInvertedIndex tabletInvertedIndex = catalog.getTabletInvertedIndex(); + Map> tabletToBackends = new HashMap<>(); + Map> tableToPartition = new HashMap<>(); + // 2. validate potential exists problem: db->table->partition + // guarantee exist exception during a transaction + // if table or partition is dropped during load, the job is fail + // if index is dropped, it does not matter + for (TabletCommitInfo tabletCommitInfo : tabletCommitInfos) { + long tabletId = tabletCommitInfo.getTabletId(); + long tableId = tabletInvertedIndex.getTableId(tabletId); + if (TabletInvertedIndex.NOT_EXIST_VALUE == tableId) { + throw new MetaNotFoundException("could not find table for tablet [" + tabletId + "]"); + } + + long partitionId = tabletInvertedIndex.getPartitionId(tabletId); + if (TabletInvertedIndex.NOT_EXIST_VALUE == partitionId) { + throw new MetaNotFoundException("could not find partition for tablet [" + tabletId + "]"); + } + + if (!tableToPartition.containsKey(tableId)) { + tableToPartition.put(tableId, new HashSet<>()); + } + tableToPartition.get(tableId).add(partitionId); + if (!tabletToBackends.containsKey(tabletId)) { + tabletToBackends.put(tabletId, new HashSet<>()); + } + tabletToBackends.get(tabletId).add(tabletCommitInfo.getBackendId()); + } + + Set errorReplicaIds = Sets.newHashSet(); + Set totalInvolvedBackends = Sets.newHashSet(); + for (long tableId : tableToPartition.keySet()) { + OlapTable table = (OlapTable) db.getTable(tableId); + for (Partition partition : table.getPartitions()) { + if (!tableToPartition.get(tableId).contains(partition.getId())) { + continue; + } + List allIndices = new ArrayList<>(); + allIndices.addAll(partition.getMaterializedIndices()); + MaterializedIndex rollingUpIndex = null; + RollupJob rollupJob = null; + if (table.getState() == OlapTableState.ROLLUP) { + rollupJob = (RollupJob) catalog.getRollupHandler().getAlterJob(tableId); + rollingUpIndex = rollupJob.getRollupIndex(partition.getId()); + } + // the rolling up index should also be taken care + // if the rollup index failed during load, then set its last failed version + // if rollup task finished, it should compare version and last failed version, if version < last failed version, then the replica is failed + if (rollingUpIndex != null) { + allIndices.add(rollingUpIndex); + } + MaterializedIndex baseIndex = partition.getBaseIndex(); + int quorumReplicaNum = table.getPartitionInfo().getReplicationNum(partition.getId()) / 2 + 1; + for (MaterializedIndex index : allIndices) { + for (Tablet tablet : index.getTablets()) { + int successReplicaNum = 0; + long tabletId = tablet.getId(); + Set tabletBackends = tablet.getBackendIds(); + totalInvolvedBackends.addAll(tabletBackends); + Set commitBackends = tabletToBackends.get(tabletId); + for (long tabletBackend : tabletBackends) { + Replica replica = tabletInvertedIndex.getReplica(tabletId, tabletBackend); + if (replica == null) { + throw new TransactionCommitFailedException("could not find replica for tablet [" + + tabletId + "], backend [" + + tabletBackend + "]"); + } + // if the tablet have no replica's to commit or the tablet is a rolling up tablet, the commit backends maybe null + // if the commit backends is null, set all replicas as error replicas + if (commitBackends != null && commitBackends.contains(tabletBackend)) { + // if the backend load success but the backend has some errors previously, then it is not a normal replica + // ignore it but not log it + // for example, a replica is in clone state + if (replica.getLastFailedVersion() < 0) { + ++ successReplicaNum; + } else { + // if this error replica is a base replica and it is under rollup + // then remove the rollup task and rollup job will remove the rollup replica automatically + // should remove here, because the error replicas not contains this base replica, but it have errors in the past + if (index.getId() == baseIndex.getId() && rollupJob != null) { + LOG.info("the base replica [{}] has error, remove the related rollup replica from rollupjob [{}]", + replica, rollupJob); + rollupJob.removeReplicaRelatedTask(partition.getId(), + tabletId, replica.getId(), replica.getBackendId()); + } + } + } else { + errorReplicaIds.add(replica.getId()); + // not remove rollup task here, because the commit maybe failed + // remove rollup task when commit successfully + } + } + if (index.getState() != IndexState.ROLLUP && successReplicaNum < quorumReplicaNum) { + // not throw exception here, wait the upper application retry + LOG.info("Index [{}] success replica num is {} < quorum replica num {}", + index, successReplicaNum, quorumReplicaNum); + return; + } + } + } + } + } + + writeLock(); + try { + // transaction state is modified during check if the transaction could committed + if (transactionState.getTransactionStatus() != TransactionStatus.PREPARE) { + return; + } + // 4. update transaction state version + transactionState.setCommitTime(System.currentTimeMillis()); + transactionState.setTransactionStatus(TransactionStatus.COMMITTED); + transactionState.setErrorReplicas(errorReplicaIds); + for (long tableId : tableToPartition.keySet()) { + TableCommitInfo tableCommitInfo = new TableCommitInfo(tableId); + for (long partitionId : tableToPartition.get(tableId)) { + OlapTable table = (OlapTable) db.getTable(tableId); + Partition partition = table.getPartition(partitionId); + PartitionCommitInfo partitionCommitInfo = new PartitionCommitInfo(partitionId, + partition.getNextVersion(), + partition.getNextVersionHash()); + tableCommitInfo.addPartitionCommitInfo(partitionCommitInfo); + } + transactionState.putIdToTableCommitInfo(tableId, tableCommitInfo); + } + // 5. persistent transactionState + unprotectUpsertTransactionState(transactionState); + for (long backendId : totalInvolvedBackends) { + transactionState.addPublishVersionTask(backendId, null); + } + } finally { + writeUnlock(); + } + // 6. update nextVersion because of the failure of persistent transaction resulting in error version + updateCatalogAfterCommitted(transactionState, db); + LOG.info("transaction:[{}] successfully committed", transactionState); + } + + public boolean commitAndPublishTransaction(Database db, long transactionId, + List tabletCommitInfos, long timeoutMillis) + throws MetaNotFoundException, TransactionCommitFailedException { + db.writeLock(); + try { + commitTransaction(db.getId(), transactionId, tabletCommitInfos); + } finally { + db.writeUnlock(); + } + + TransactionState transactionState = idToTransactionState.get(transactionId); + switch (transactionState.getTransactionStatus()) { + case COMMITTED: + case VISIBLE: + break; + default: + LOG.warn("transaction commit failed, db={}, txn={}", db.getFullName(), transactionId); + throw new TransactionCommitFailedException("transaction commit failed"); + } + + long currentTimeMillis = System.currentTimeMillis(); + long timeoutTimeMillis = currentTimeMillis + timeoutMillis; + while (currentTimeMillis < timeoutTimeMillis && + transactionState.getTransactionStatus() == TransactionStatus.COMMITTED) { + try { + transactionState.waitTransactionVisible(timeoutMillis); + } catch (InterruptedException e) { + } + currentTimeMillis = System.currentTimeMillis(); + } + return transactionState.getTransactionStatus() == TransactionStatus.VISIBLE; + } + + public void abortTransaction(long transactionId, String reason) throws UserException { + + if (transactionId < 0) { + LOG.info("transaction id is {}, less than 0, maybe this is an old type load job, ignore abort operation", transactionId); + return; + } + writeLock(); + try { + unprotectAbortTransaction(transactionId, reason); + } catch (Exception exception) { + LOG.info("transaction:[{}] reason:[{}] abort failure exception:{}", transactionId, reason, exception); + throw exception; + } finally { + writeUnlock(); + } + return; + } + + public void abortTransaction(Long dbId, String label, String reason) throws UserException { + Preconditions.checkNotNull(label); + writeLock(); + try { + Map dbTxns = dbIdToTxnLabels.row(dbId); + if (dbTxns == null) { + throw new UserException("transaction not found, label=" + label); + } + Long transactionId = dbTxns.get(label); + if (transactionId == null) { + throw new UserException("transaction not found, label=" + label); + } + unprotectAbortTransaction(transactionId, reason); + } finally { + writeUnlock(); + } + } + + public List getReadyToPublishTransactions() { + List readyPublishTransactionState = new ArrayList<>(); + List allCommittedTransactionState = null; + writeLock(); + try { + // only send task to committed transaction + allCommittedTransactionState = idToTransactionState.values().stream() + .filter(transactionState -> (transactionState.getTransactionStatus() == TransactionStatus.COMMITTED)) + .collect(Collectors.toList()); + for (TransactionState transactionState : allCommittedTransactionState) { + long dbId = transactionState.getDbId(); + Database db = catalog.getDb(dbId); + if (null == db) { + transactionState.setTransactionStatus(TransactionStatus.ABORTED); + unprotectUpsertTransactionState(transactionState); + continue; + } + } + } finally { + writeUnlock(); + } + for (TransactionState transactionState : allCommittedTransactionState) { + boolean meetPublishPredicate = true; + long dbId = transactionState.getDbId(); + Database db = catalog.getDb(dbId); + if (null == db) { + continue; + } + db.readLock(); + writeLock(); + try { + for (TableCommitInfo tableCommitInfo : transactionState.getIdToTableCommitInfos().values()) { + OlapTable table = (OlapTable) db.getTable(tableCommitInfo.getTableId()); + if (null == table) { + LOG.warn("table {} is dropped after commit, ignore this table", tableCommitInfo.getTableId()); + continue; + } + for (PartitionCommitInfo partitionCommitInfo : tableCommitInfo.getIdToPartitionCommitInfo().values()) { + Partition partition = table.getPartition(partitionCommitInfo.getPartitionId()); + if (null == partition) { + LOG.warn("partition {} is dropped after commit, ignore this partition", partitionCommitInfo.getPartitionId()); + continue; + } + if (partitionCommitInfo.getVersion() != partition.getCommittedVersion() + 1) { + meetPublishPredicate = false; + break; + } + } + if (!meetPublishPredicate) { + break; + } + } + if (meetPublishPredicate) { + LOG.debug("transaction [{}] is ready to publish", transactionState); + readyPublishTransactionState.add(transactionState); + } + } finally { + writeUnlock(); + db.readUnlock(); + } + } + return readyPublishTransactionState; + } + + /** + * if the table is deleted between commit and publish version, then should ignore the partition + * if a tablet is not find in + * + * @param transactionId + * @param errorReplicaIds + * @return + */ + public void finishTransaction(long transactionId, Set errorReplicaIds) { + TransactionState transactionState = idToTransactionState.get(transactionId); + // add all commit errors and publish errors to a single set + if (errorReplicaIds == null) { + errorReplicaIds = Sets.newHashSet(); + } + Set originalErrorReplicas = transactionState.getErrorReplicas(); + if (originalErrorReplicas != null) { + errorReplicaIds.addAll(originalErrorReplicas); + } + + Database db = catalog.getDb(transactionState.getDbId()); + if (db == null) { + writeLock(); + try { + transactionState.setTransactionStatus(TransactionStatus.ABORTED); + transactionState.setReason("db is dropped"); + LOG.warn("db is dropped during transaction, abort transaction {}", transactionState); + unprotectUpsertTransactionState(transactionState); + return; + } finally { + writeUnlock(); + } + } + db.writeLock(); + try { + boolean hasError = false; + for (TableCommitInfo tableCommitInfo : transactionState.getIdToTableCommitInfos().values()) { + long tableId = tableCommitInfo.getTableId(); + OlapTable table = (OlapTable) db.getTable(tableId); + // table maybe dropped between commit and publish, ignore this error + if (table == null) { + transactionState.removeTable(tableId); + LOG.warn("table {} is dropped, skip version check and remove it from transaction state {}", + tableId, + transactionState); + continue; + } + PartitionInfo partitionInfo = table.getPartitionInfo(); + for (PartitionCommitInfo partitionCommitInfo : tableCommitInfo.getIdToPartitionCommitInfo().values()) { + long partitionId = partitionCommitInfo.getPartitionId(); + Partition partition = table.getPartition(partitionId); + // partition maybe dropped between commit and publish version, ignore this error + if (partition == null) { + tableCommitInfo.removePartition(partitionId); + LOG.warn("partition {} is dropped, skip version check and remove it from transaction state {}", + partitionId, + transactionState); + continue; + } + int quorumReplicaNum = partitionInfo.getReplicationNum(partitionId) / 2 + 1; + MaterializedIndex baseIndex = partition.getBaseIndex(); + MaterializedIndex rollingUpIndex = null; + RollupJob rollupJob = null; + if (table.getState() == OlapTableState.ROLLUP) { + rollupJob = (RollupJob) catalog.getRollupHandler().getAlterJob(tableId); + rollingUpIndex = rollupJob.getRollupIndex(partitionId); + } + List allInices = new ArrayList<>(); + allInices.addAll(partition.getMaterializedIndices()); + if (rollingUpIndex != null) { + allInices.add(rollingUpIndex); + } + for (MaterializedIndex index : allInices) { + for (Tablet tablet : index.getTablets()) { + int healthReplicaNum = 0; + for (Replica replica : tablet.getReplicas()) { + // this means the replica is a healthy replica, it is health in the past and does not have error in current load + if (!errorReplicaIds.contains(replica.getId()) + && replica.getLastFailedVersion() < 0) { + if (replica.getVersion() == partition.getCommittedVersion() && replica.getVersionHash() == partition.getCommittedVersionHash() + || replica.getVersion() >= partitionCommitInfo.getVersion()) { + // during rollup the rollup replica's last failed version < 0, it maybe treated as a normal replica + // the replica is not failed during commit or publish + // during upgrade, one replica's last version maybe invalid, has to compare version hash + // if a,b,c commit 10 transactions, and then b,c crashed, we add new b',c' it has to recover, we improve a's version one by one and b' c' will recover + // from a one by one + replica.updateInfo(partitionCommitInfo.getVersion(), partitionCommitInfo.getVersionHash(), + replica.getDataSize(), replica.getRowCount()); + ++ healthReplicaNum; + } else { + // this means the replica has error in the past, but we did not observe it + // during upgrade, one job maybe in quorum finished state, for example, A,B,C 3 replica + // A,B 's verison is 10, C's version is 10 but C' 10 is abnormal should be rollback + // then we will detect this and set C's last failed version to 10 and last success version to 11 + // this logic has to be replayed in checkpoint thread + replica.updateVersionInfo(replica.getVersion(), replica.getVersionHash(), + partition.getCommittedVersion(), partition.getCommittedVersionHash(), + partitionCommitInfo.getVersion(), partitionCommitInfo.getVersionHash()); + LOG.warn("transaction state {} has error, the replica [{}] not appeared in error replica list " + + " and its version not equal to partition commit version or commit version - 1" + + " if its not a upgrate stage, its a fatal error. "); + } + } else if (replica.getVersion() == partitionCommitInfo.getVersion() + && replica.getVersionHash() == partitionCommitInfo.getVersionHash()) { + // the replica's version and versionhash is equal to current transaction partition's version and version hash + // the replica is normal, then remove it from error replica ids + errorReplicaIds.remove(replica.getId()); + ++ healthReplicaNum; + } + if (replica.getLastFailedVersion() > 0) { + // if this error replica is a base replica and it is under rollup + // then remove the rollup task and rollup job will remove the rollup replica automatically + if (index.getId() == baseIndex.getId() && rollupJob != null) { + LOG.info("base replica [{}] has errors during load, remove rollup task on related replica", replica); + rollupJob.removeReplicaRelatedTask(partition.getId(), + tablet.getId(), replica.getId(), replica.getBackendId()); + } + } + } + if (index.getState() != IndexState.ROLLUP && healthReplicaNum < quorumReplicaNum) { + LOG.info("publish version failed for transaction {} on tablet {}, with only {} replicas less than quorum {}", + transactionState, tablet, healthReplicaNum, quorumReplicaNum); + hasError = true; + } + } + } + } + } + if (hasError) { + return; + } + writeLock(); + try { + transactionState.setErrorReplicas(errorReplicaIds); + transactionState.setFinishTime(System.currentTimeMillis()); + transactionState.setTransactionStatus(TransactionStatus.VISIBLE); + unprotectUpsertTransactionState(transactionState); + } finally { + writeUnlock(); + } + updateCatalogAfterVisible(transactionState, db); + } finally { + db.writeUnlock(); + } + LOG.info("finish transaction {} successfully", transactionState); + return; + } + + // check if there exists a load job before the endTransactionId have all finished + // load job maybe started but could not know the affected tableid, so that we not check by table + public boolean hasPreviousTransactionsFinished(long endTransactionId, long dbId) { + readLock(); + try { + for (Map.Entry entry : idToTransactionState.entrySet()) { + if (entry.getValue().getDbId() != dbId || !entry.getValue().isRunning()) { + continue; + } + if (entry.getKey() <= endTransactionId) { + return false; + } + } + } finally { + readUnlock(); + } + return true; + } + + /** + * in this method should get db lock or load lock first then get txn manager lock , or there will be dead lock + */ + public void removeOldTransactions() { + long currentMillis = System.currentTimeMillis(); + + // to avoid dead lock (transaction lock and load lock), we do this in 3 phases + // 1. get all related db ids of txn in idToTransactionState + Set dbIds = Sets.newHashSet(); + readLock(); + try { + for (TransactionState transactionState : idToTransactionState.values()) { + if (transactionState.getTransactionStatus() == TransactionStatus.ABORTED + || transactionState.getTransactionStatus() == TransactionStatus.VISIBLE) { + if ((currentMillis - transactionState.getFinishTime()) / 1000 > Config.label_keep_max_second) { + dbIds.add(transactionState.getDbId()); + } + } else { + // check if job is also deleted + // streaming insert stmt not add to fe load job, should use this method to + // recycle the timeout insert stmt load job + if (transactionState.getTransactionStatus() == TransactionStatus.PREPARE + && (currentMillis - transactionState.getPrepareTime()) + / 1000 > Config.stream_load_default_timeout_second) { + dbIds.add(transactionState.getDbId()); + } + } + } + } finally { + readUnlock(); + } + + // 2. get all load jobs' txn id of these databases + Map> dbIdToTxnIds = Maps.newHashMap(); + Load loadInstance = Catalog.getCurrentCatalog().getLoadInstance(); + for (Long dbId : dbIds) { + Set txnIds = loadInstance.getTxnIdsByDb(dbId); + dbIdToTxnIds.put(dbId, txnIds); + } + + // 3. use dbIdToTxnIds to remove old transactions, without holding load locks again + writeLock(); + try { + List transactionsToDelete = Lists.newArrayList(); + for (TransactionState transactionState : idToTransactionState.values()) { + if (transactionState.getTransactionStatus() == TransactionStatus.ABORTED + || transactionState.getTransactionStatus() == TransactionStatus.VISIBLE) { + if ((currentMillis - transactionState.getFinishTime()) / 1000 > Config.label_keep_max_second) { + // if this txn is not from front end then delete it immediately + // if this txn is from front end but could not find in job list, then delete it immediately + if (transactionState.getSourceType() != LoadJobSourceType.FRONTEND + || !checkTxnHasRelatedJob(transactionState, dbIdToTxnIds)) { + transactionsToDelete.add(transactionState.getTransactionId()); + } + } + } else { + // check if job is also deleted + // streaming insert stmt not add to fe load job, should use this method to + // recycle the timeout insert stmt load job + if (transactionState.getTransactionStatus() == TransactionStatus.PREPARE + && (currentMillis - transactionState.getPrepareTime()) / 1000 > Config.stream_load_default_timeout_second) { + if (transactionState.getSourceType() != LoadJobSourceType.FRONTEND + || !checkTxnHasRelatedJob(transactionState, dbIdToTxnIds)) { + transactionState.setFinishTime(System.currentTimeMillis()); + transactionState.setTransactionStatus(TransactionStatus.ABORTED); + transactionState.setReason("transaction is timeout and is cancelled automatically"); + unprotectUpsertTransactionState(transactionState); + } + } + } + } + + for (Long transId : transactionsToDelete) { + deleteTransaction(transId); + LOG.info("transaction [" + transId + "] is expired, remove it from transaction table"); + } + } finally { + writeUnlock(); + } + } + + private boolean checkTxnHasRelatedJob(TransactionState txnState, Map> dbIdToTxnIds) { + Set txnIds = dbIdToTxnIds.get(txnState.getDbId()); + if (txnIds == null) { + // We can't find the related load job of this database. + // But dbIdToTxnIds is not a up-to-date results. + // So we return true to assume that we find a related load job, to avoid mistaken delete + return true; + } + + return txnIds.contains(txnState.getTransactionId()); + } + + public TransactionState getTransactionState(long transactionId) { + readLock(); + try { + return idToTransactionState.get(transactionId); + } finally { + readUnlock(); + } + } + + public void setEditLog(EditLog editLog) { + this.editLog = editLog; + this.idGenerator.setEditLog(editLog); + } + + private void readLock() { + this.transactionLock.readLock().lock(); + } + + private void readUnlock() { + this.transactionLock.readLock().unlock(); + } + + private void writeLock() { + this.transactionLock.writeLock().lock(); + } + + private void writeUnlock() { + this.transactionLock.writeLock().unlock(); + } + + // for add/update/delete TransactionState + private void unprotectUpsertTransactionState(TransactionState transactionState) { + editLog.logInsertTransactionState(transactionState); + idToTransactionState.put(transactionState.getTransactionId(), transactionState); + updateTxnLabels(transactionState); + updateDBRunningTxnNum(transactionState.getPreStatus(), transactionState); + } + + private void unprotectAbortTransaction(long transactionId, String reason) throws UserException { + TransactionState transactionState = idToTransactionState.get(transactionId); + if (transactionState == null) { + throw new UserException("transaction not found"); + } + if (transactionState.getTransactionStatus() == TransactionStatus.ABORTED) { + return; + } + if (transactionState.getTransactionStatus() == TransactionStatus.COMMITTED + || transactionState.getTransactionStatus() == TransactionStatus.VISIBLE) { + throw new UserException("transaction's state is already committed or visible, could not abort"); + } + transactionState.setFinishTime(System.currentTimeMillis()); + transactionState.setReason(reason); + transactionState.setTransactionStatus(TransactionStatus.ABORTED); + unprotectUpsertTransactionState(transactionState); + for (PublishVersionTask task : transactionState.getPublishVersionTasks().values()) { + AgentTaskQueue.removeTask(task.getBackendId(), TTaskType.PUBLISH_VERSION, task.getSignature()); + } + } + + // for replay idToTransactionState + // check point also run transaction cleaner, the cleaner maybe concurrently modify id to + public void replayUpsertTransactionState(TransactionState transactionState) { + writeLock(); + try { + Database db = catalog.getDb(transactionState.getDbId()); + if (transactionState.getTransactionStatus() == TransactionStatus.COMMITTED) { + LOG.debug("replay a committed transaction {}", transactionState); + updateCatalogAfterCommitted(transactionState, db); + } else if (transactionState.getTransactionStatus() == TransactionStatus.VISIBLE) { + LOG.debug("replay a visible transaction {}", transactionState); + updateCatalogAfterVisible(transactionState, db); + } + TransactionState preTxnState = idToTransactionState.get(transactionState.getTransactionId()); + idToTransactionState.put(transactionState.getTransactionId(), transactionState); + updateTxnLabels(transactionState); + updateDBRunningTxnNum(preTxnState == null ? null : preTxnState.getTransactionStatus(), + transactionState); + } finally { + writeUnlock(); + } + } + + public void replayDeleteTransactionState(TransactionState transactionState) { + writeLock(); + try { + idToTransactionState.remove(transactionState.getTransactionId()); + dbIdToTxnLabels.remove(transactionState.getDbId(), transactionState.getLabel()); + } finally { + writeUnlock(); + } + } + + private void updateCatalogAfterCommitted(TransactionState transactionState, Database db) { + Set errorReplicaIds = transactionState.getErrorReplicas(); + for (TableCommitInfo tableCommitInfo : transactionState.getIdToTableCommitInfos().values()) { + long tableId = tableCommitInfo.getTableId(); + OlapTable table = (OlapTable) db.getTable(tableId); + for (PartitionCommitInfo partitionCommitInfo : tableCommitInfo.getIdToPartitionCommitInfo().values()) { + long partitionId = partitionCommitInfo.getPartitionId(); + Partition partition = table.getPartition(partitionId); + List allIndices = new ArrayList<>(); + allIndices.addAll(partition.getMaterializedIndices()); + MaterializedIndex baseIndex = partition.getBaseIndex(); + MaterializedIndex rollingUpIndex = null; + RollupJob rollupJob = null; + if (table.getState() == OlapTableState.ROLLUP) { + rollupJob = (RollupJob) catalog.getRollupHandler().getAlterJob(tableId); + rollingUpIndex = rollupJob.getRollupIndex(partition.getId()); + } + if (rollingUpIndex != null) { + allIndices.add(rollingUpIndex); + } + for (MaterializedIndex index : allIndices) { + List tablets = index.getTablets(); + for (Tablet tablet : tablets) { + for (Replica replica : tablet.getReplicas()) { + if (errorReplicaIds.contains(replica.getId())) { + // should not use partition.getNextVersion and partition.getNextVersionHash because partition's next version hash is generated locally + // should get from transaction state + replica.updateLastFailedVersion(partitionCommitInfo.getVersion(), + partitionCommitInfo.getVersionHash()); + // if this error replica is a base replica and it is under rollup + // then remove the rollup task and rollup job will remove the rollup replica automatically + if (index.getId() == baseIndex.getId() && rollupJob != null) { + LOG.debug("the base replica [{}] has error, remove the related rollup replica from rollupjob [{}]", + replica, rollupJob); + rollupJob.removeReplicaRelatedTask(partition.getId(), + tablet.getId(), replica.getId(), replica.getBackendId()); + } + } + } + } + } + partition.setNextVersion(partition.getNextVersion() + 1); + // the partition's current version hash should be set from partition commit info + // for example, fe master's partition current version hash is 123123, fe followers partition current version hash is 3333 + // they are different, fe master changed, the follower is master now, but its current version hash is 333, if clone happened, + // clone finished but its finished version hash != partition's current version hash, then clone is failed + // because clone depend on partition's current version to clone + partition.setNextVersionHash(Util.generateVersionHash(), partitionCommitInfo.getVersionHash()); + } + } + } + + private boolean updateCatalogAfterVisible(TransactionState transactionState, Database db) { + Set errorReplicaIds = transactionState.getErrorReplicas(); + for (TableCommitInfo tableCommitInfo : transactionState.getIdToTableCommitInfos().values()) { + long tableId = tableCommitInfo.getTableId(); + OlapTable table = (OlapTable) db.getTable(tableId); + for (PartitionCommitInfo partitionCommitInfo : tableCommitInfo.getIdToPartitionCommitInfo().values()) { + long partitionId = partitionCommitInfo.getPartitionId(); + long newCommitVersion = partitionCommitInfo.getVersion(); + long newCommitVersionHash = partitionCommitInfo.getVersionHash(); + Partition partition = table.getPartition(partitionId); + MaterializedIndex baseIndex = partition.getBaseIndex(); + MaterializedIndex rollingUpIndex = null; + RollupJob rollupJob = null; + if (table.getState() == OlapTableState.ROLLUP) { + rollupJob = (RollupJob) catalog.getRollupHandler().getAlterJob(tableId); + rollingUpIndex = rollupJob.getRollupIndex(partitionId); + } + List allInices = new ArrayList<>(); + allInices.addAll(partition.getMaterializedIndices()); + if (rollingUpIndex != null) { + allInices.add(rollingUpIndex); + } + for (MaterializedIndex index : allInices) { + for (Tablet tablet : index.getTablets()) { + for (Replica replica : tablet.getReplicas()) { + long lastFailedVersion = replica.getLastFailedVersion(); + long lastFailedVersionHash = replica.getLastFailedVersionHash(); + long newVersion = newCommitVersion; + long newVersionHash = newCommitVersionHash; + long lastSucessVersion = replica.getLastSuccessVersion(); + long lastSuccessVersionHash = replica.getLastSuccessVersionHash(); + if (!errorReplicaIds.contains(replica.getId())) { + if (replica.getLastFailedVersion() > 0) { + // if the replica is a failed replica, then not change version and version hash + newVersion = replica.getVersion(); + newVersionHash = replica.getVersionHash(); + } else { + if (replica.getVersion() == partition.getCommittedVersion() && replica.getVersionHash() == partition.getCommittedVersionHash() + || replica.getVersion() >= partitionCommitInfo.getVersion()) { + // during rollup the rollup replica's last failed version < 0, it maybe treated as a normal replica + // the replica is not failed during commit or publish + // during upgrade, one replica's last version maybe invalid, has to compare version hash + // if a,b,c commit 10 transactions, and then b,c crashed, we add new b',c' it has to recover, we improve a's version one by one and b' c' will recover + // from a one by one + // DO NOTHING + } else { + // this means the replica has error in the past, but we did not observe it + // during upgrade, one job maybe in quorum finished state, for example, A,B,C 3 replica + // A,B 's version is 10, C's version is 10 but C' 10 is abnormal should be rollback + // then we will detect this and set C's last failed version to 10 and last success version to 11 + // this logic has to be replayed in checkpoint thread + lastFailedVersion = partition.getCommittedVersion(); + lastFailedVersionHash = partition.getCommittedVersionHash(); + newVersion = replica.getVersion(); + newVersionHash = replica.getVersionHash(); + } + } + // success version always move forward + lastSucessVersion = newCommitVersion; + lastSuccessVersionHash = newCommitVersionHash; + } else { + // for example, A,B,C 3 replicas, B,C failed during publish version, then B C will be set abnormal + // all loading will failed, B,C will have to recovery by clone, it is very inefficient and maybe lost data + // Using this method, B,C will publish failed, and fe will publish again, not update their last failed version + // if B is publish successfully in next turn, then B is normal and C will be set abnormal so that quorum is maintained + // and loading will go on. + newVersion = replica.getVersion(); + newVersionHash = replica.getVersionHash(); + if (newCommitVersion > lastFailedVersion) { + lastFailedVersion = newCommitVersion; + lastFailedVersionHash = newCommitVersionHash; + } + } + replica.updateVersionInfo(newVersion, newVersionHash, lastFailedVersion, lastFailedVersionHash, lastSucessVersion, lastSuccessVersionHash); + // if this error replica is a base replica and it is under rollup + // then remove the rollup task and rollup job will remove the rollup replica automatically + if (index.getId() == baseIndex.getId() + && replica.getLastFailedVersion() > 0 + && rollupJob != null) { + LOG.debug("base replica [{}] has errors during load, remove rollup task on related replica", replica); + rollupJob.removeReplicaRelatedTask(partition.getId(), + tablet.getId(), replica.getId(), replica.getBackendId()); + } + } + } + } + long version = partitionCommitInfo.getVersion(); + long versionHash = partitionCommitInfo.getVersionHash(); + partition.updateCommitVersionAndVersionHash(version, versionHash); + if (LOG.isDebugEnabled()) { + LOG.debug("transaction state {} set partition's version to [{}] and version hash to [{}]", + transactionState, version, versionHash); + } + } + } + return true; + } + + private void updateTxnLabels(TransactionState transactionState) { + // if the transaction is aborted, then its label could be reused + if (transactionState.getTransactionStatus() == TransactionStatus.ABORTED) { + dbIdToTxnLabels.remove(transactionState.getDbId(), transactionState.getLabel()); + } else { + dbIdToTxnLabels.put(transactionState.getDbId(), transactionState.getLabel(), + transactionState.getTransactionId()); + } + } + + private void updateDBRunningTxnNum(TransactionStatus preStatus, TransactionState curTxnState) { + int dbRunningTxnNum = 0; + if (runningTxnNums.get(curTxnState.getDbId()) != null) { + dbRunningTxnNum = runningTxnNums.get(curTxnState.getDbId()); + } + if (preStatus == null + && (curTxnState.getTransactionStatus() == TransactionStatus.PREPARE + || curTxnState.getTransactionStatus() == TransactionStatus.COMMITTED)) { + ++ dbRunningTxnNum; + runningTxnNums.put(curTxnState.getDbId(), dbRunningTxnNum); + } else if (preStatus != null + && (preStatus == TransactionStatus.PREPARE + || preStatus == TransactionStatus.COMMITTED) + && (curTxnState.getTransactionStatus() == TransactionStatus.VISIBLE + || curTxnState.getTransactionStatus() == TransactionStatus.ABORTED)) { + -- dbRunningTxnNum; + if (dbRunningTxnNum < 1) { + runningTxnNums.remove(curTxnState.getDbId()); + } else { + runningTxnNums.put(curTxnState.getDbId(), dbRunningTxnNum); + } + } + } + + public List> getDbInfo() { + List> infos = new ArrayList>(); + readLock(); + try { + Set dbIds = new HashSet<>(); + for (TransactionState transactionState : idToTransactionState.values()) { + dbIds.add(transactionState.getDbId()); + } + for (long dbId : dbIds) { + List info = new ArrayList(); + info.add(dbId); + Database db = Catalog.getInstance().getDb(dbId); + if (db == null) { + continue; + } + info.add(db.getFullName()); + infos.add(info); + } + } finally { + readUnlock(); + } + return infos; + } + + public List> getDbTransInfo(long dbId) throws AnalysisException { + List> infos = new ArrayList>(); + readLock(); + try { + Database db = Catalog.getInstance().getDb(dbId); + if (db == null) { + throw new AnalysisException("Database[" + dbId + "] does not exist"); + } + idToTransactionState.values().stream() + .filter(t -> t.getDbId() == dbId) + .forEach(t -> { + List info = new ArrayList(); + info.add(t.getTransactionId()); + info.add(t.getLabel()); + info.add(t.getCoordinator()); + info.add(t.getTransactionStatus()); + info.add(t.getSourceType()); + info.add(TimeUtils.longToTimeString(t.getPrepareTime())); + info.add(TimeUtils.longToTimeString(t.getCommitTime())); + info.add(TimeUtils.longToTimeString(t.getFinishTime())); + info.add(t.getReason()); + info.add(t.getErrorReplicas().size()); + infos.add(info); + }); + } finally { + readUnlock(); + } + return infos; + } + + public List> getTableTransInfo(long tid, Database db) throws AnalysisException { + List> tableInfos = new ArrayList>(); + readLock(); + try { + TransactionState transactionState = idToTransactionState.get(tid); + if (null == transactionState) { + throw new AnalysisException("Transaction[" + tid + "] does not exist."); + } + db.readLock(); + try { + for (long tableId : transactionState.getIdToTableCommitInfos().keySet()) { + List tableInfo = new ArrayList(); + Table table = db.getTable(tableId); + if (null == table) { + throw new AnalysisException("Table[" + tableId + "] does not exist."); + } + int partitionNum = 1; + if (table.getType() == Table.TableType.OLAP) { + OlapTable olapTable = (OlapTable) table; + tableInfo.add(table.getId()); + tableInfo.add(table.getName()); + tableInfo.add(partitionNum); + tableInfo.add(olapTable.getState()); + tableInfos.add(tableInfo); + } + } + } finally { + db.readUnlock(); + } + } finally { + readUnlock(); + } + return tableInfos; + } + + public List> getPartitionTransInfo(long tid, Database db, OlapTable olapTable) + throws AnalysisException { + List> partitionInfos = new ArrayList>(); + readLock(); + try { + TransactionState transactionState = idToTransactionState.get(tid); + if (null == transactionState) { + throw new AnalysisException("Transaction[" + tid + "] does not exist."); + } + TableCommitInfo tableCommitInfo = transactionState.getIdToTableCommitInfos().get(olapTable.getId()); + db.readLock(); + try { + Map idToPartitionCommitInfo = tableCommitInfo.getIdToPartitionCommitInfo(); + for (long partitionId : idToPartitionCommitInfo.keySet()) { + Partition partition = olapTable.getPartition(partitionId); + List partitionInfo = new ArrayList(); + String partitionName = partition.getName(); + partitionInfo.add(partitionId); + partitionInfo.add(partitionName); + PartitionCommitInfo partitionCommitInfo = idToPartitionCommitInfo.get(partitionId); + partitionInfo.add(partitionCommitInfo.getVersion()); + partitionInfo.add(partitionCommitInfo.getVersionHash()); + partitionInfo.add(partition.getState()); + partitionInfos.add(partitionInfo); + } + } finally { + db.readUnlock(); + } + } finally { + readUnlock(); + } + return partitionInfos; + } + + public int getTransactionNum() { + return this.idToTransactionState.size(); + } + + public TransactionIDGenerator getTransactionIDGenerator() { + return this.idGenerator; + } + + // this two function used to read snapshot or write snapshot + public void write(DataOutput out) throws IOException { + int numTransactions = idToTransactionState.size(); + out.writeInt(numTransactions); + for (Map.Entry entry : idToTransactionState.entrySet()) { + entry.getValue().write(out); + } + idGenerator.write(out); + } + + public void readFields(DataInput in) throws IOException { + int numTransactions = in.readInt(); + for (int i = 0; i < numTransactions; ++i) { + TransactionState transactionState = new TransactionState(); + transactionState.readFields(in); + TransactionState preTxnState = idToTransactionState.get(transactionState.getTransactionId()); + idToTransactionState.put(transactionState.getTransactionId(), transactionState); + updateTxnLabels(transactionState); + updateDBRunningTxnNum(preTxnState == null ? null : preTxnState.getTransactionStatus(), + transactionState); + } + idGenerator.readFields(in); + } +} diff --git a/fe/src/main/java/com/baidu/palo/transaction/IllegalTransactionParameterException.java b/fe/src/main/java/com/baidu/palo/transaction/IllegalTransactionParameterException.java new file mode 100644 index 0000000000..504c3d7281 --- /dev/null +++ b/fe/src/main/java/com/baidu/palo/transaction/IllegalTransactionParameterException.java @@ -0,0 +1,16 @@ +package com.baidu.palo.transaction; + +import com.baidu.palo.common.UserException; + +public class IllegalTransactionParameterException extends UserException { + + private static final long serialVersionUID = 1L; + + public IllegalTransactionParameterException(String msg) { + super(msg); + } + + public IllegalTransactionParameterException(String msg, Throwable e) { + super(msg, e); + } +} \ No newline at end of file diff --git a/fe/src/main/java/com/baidu/palo/transaction/LabelAlreadyExistsException.java b/fe/src/main/java/com/baidu/palo/transaction/LabelAlreadyExistsException.java new file mode 100644 index 0000000000..45fd60189b --- /dev/null +++ b/fe/src/main/java/com/baidu/palo/transaction/LabelAlreadyExistsException.java @@ -0,0 +1,36 @@ +// Copyright (c) 2017, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 com.baidu.palo.transaction; + +import com.baidu.palo.common.UserException; + +public class LabelAlreadyExistsException extends UserException { + public LabelAlreadyExistsException(String msg, Throwable cause) { + super(msg, cause); + } + + public LabelAlreadyExistsException(Throwable cause) { + super(cause); + } + + public LabelAlreadyExistsException(String message, Throwable cause, boolean enableSuppression, boolean writableStackTrace) { + super(message, cause, enableSuppression, writableStackTrace); + } + + public LabelAlreadyExistsException(String msg) { + super(msg); + } +} diff --git a/fe/src/main/java/com/baidu/palo/transaction/PartitionCommitInfo.java b/fe/src/main/java/com/baidu/palo/transaction/PartitionCommitInfo.java new file mode 100644 index 0000000000..1260163df1 --- /dev/null +++ b/fe/src/main/java/com/baidu/palo/transaction/PartitionCommitInfo.java @@ -0,0 +1,77 @@ +// Copyright (c) 2017, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 com.baidu.palo.transaction; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; + +import com.baidu.palo.common.io.Writable; + +public class PartitionCommitInfo implements Writable { + + private long partitionId; + private long version; + private long versionHash; + + public PartitionCommitInfo() { + + } + + public PartitionCommitInfo(long partitionId, long version, long versionHash) { + super(); + this.partitionId = partitionId; + this.version = version; + this.versionHash = versionHash; + } + + @Override + public void write(DataOutput out) throws IOException { + out.writeLong(partitionId); + out.writeLong(version); + out.writeLong(versionHash); + } + + @Override + public void readFields(DataInput in) throws IOException { + partitionId = in.readLong(); + version = in.readLong(); + versionHash = in.readLong(); + } + + public long getPartitionId() { + return partitionId; + } + + public long getVersion() { + return version; + } + + public long getVersionHash() { + return versionHash; + } + + @Override + public String toString() { + StringBuffer strBuffer = new StringBuffer("partitionid="); + strBuffer.append(partitionId); + strBuffer.append(", version="); + strBuffer.append(version); + strBuffer.append(", versionHash="); + strBuffer.append(versionHash); + return strBuffer.toString(); + } +} diff --git a/fe/src/main/java/com/baidu/palo/transaction/PublishVersionDaemon.java b/fe/src/main/java/com/baidu/palo/transaction/PublishVersionDaemon.java new file mode 100644 index 0000000000..f4b9692848 --- /dev/null +++ b/fe/src/main/java/com/baidu/palo/transaction/PublishVersionDaemon.java @@ -0,0 +1,195 @@ +// Copyright (c) 2017, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 com.baidu.palo.transaction; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import com.baidu.palo.catalog.Catalog; +import com.baidu.palo.catalog.Replica; +import com.baidu.palo.catalog.Replica.ReplicaState; +import com.baidu.palo.catalog.TabletInvertedIndex; +import com.baidu.palo.common.Config; +import com.baidu.palo.common.util.Daemon; +import com.baidu.palo.task.AgentBatchTask; +import com.baidu.palo.task.AgentTaskExecutor; +import com.baidu.palo.task.AgentTaskQueue; +import com.baidu.palo.task.PublishVersionTask; +import com.baidu.palo.thrift.TPartitionVersionInfo; +import com.baidu.palo.thrift.TTaskType; +import com.google.common.collect.Sets; + +public class PublishVersionDaemon extends Daemon { + + private static final Logger LOG = LogManager.getLogger(PublishVersionDaemon.class); + + public PublishVersionDaemon() { + super("PUBLISH_VERSION"); + setInterval(Config.publish_version_interval_millis); + } + + protected void runOneCycle() { + try { + publishVersion(); + } catch (Throwable t) { + LOG.error("errors while publish version to all backends, {}", t); + } + } + + private void publishVersion() { + GlobalTransactionMgr globalTransactionMgr = Catalog.getCurrentGlobalTransactionMgr(); + List readyTransactionStates = globalTransactionMgr.getReadyToPublishTransactions(); + if (readyTransactionStates == null || readyTransactionStates.isEmpty()) { + return; + } + // TODO yiguolei: could publish transaction state according to multi-tenant cluster info + // but should do more work. for example, if a table is migrate from one cluster to another cluster + // should pulish to two clusters. + // attention here, we publish transaction state to all backends including dead backend, if not publish to dead backend + // then transaction manager will treat it as success + List allBackends = Catalog.getCurrentSystemInfo().getBackendIds(false); + if (allBackends == null || allBackends.size() == 0) { + LOG.warn("some transaction state need to publish, but no alive backends!!!"); + return; + } + // every backend-transaction identified a single task + AgentBatchTask batchTask = new AgentBatchTask(); + // traverse all ready transactions and dispatch the publish version task to all backends + for (TransactionState transactionState : readyTransactionStates) { + if (transactionState.hasSendTask()) { + continue; + } + List partitionCommitInfos = new ArrayList<>(); + for (TableCommitInfo tableCommitInfo : transactionState.getIdToTableCommitInfos().values()) { + partitionCommitInfos.addAll(tableCommitInfo.getIdToPartitionCommitInfo().values()); + } + List partitionVersionInfos = new ArrayList<>(partitionCommitInfos.size()); + for (PartitionCommitInfo commitInfo : partitionCommitInfos) { + TPartitionVersionInfo versionInfo = new TPartitionVersionInfo(commitInfo.getPartitionId(), + commitInfo.getVersion(), + commitInfo.getVersionHash()); + partitionVersionInfos.add(versionInfo); + if (LOG.isDebugEnabled()) { + LOG.debug("try to publish version info partitionid [{}], version [{}], version hash [{}]", + commitInfo.getPartitionId(), + commitInfo.getVersion(), + commitInfo.getVersionHash()); + } + } + Set publishBackends = transactionState.getPublishVersionTasks().keySet(); + if (publishBackends.isEmpty()) { + // could not just add to it, should new a new object, or the back map will destroyed + publishBackends = Sets.newHashSet(); + // this is useful if fe master transfer to another master, because publish version task is not + // persistent to edit log, then it should publish to all backends + publishBackends.addAll(allBackends); + } + for (long backendId : publishBackends) { + PublishVersionTask task = new PublishVersionTask(backendId, + transactionState.getTransactionId(), + partitionVersionInfos); + // add to AgentTaskQueue for handling finish report. + // not check return value, because the add will success + AgentTaskQueue.addTask(task); + batchTask.addTask(task); + transactionState.addPublishVersionTask(backendId, task); + } + transactionState.setHasSendTask(true); + } + if (!batchTask.getAllTasks().isEmpty()) { + AgentTaskExecutor.submit(batchTask); + } + + TabletInvertedIndex tabletInvertedIndex = Catalog.getCurrentInvertedIndex(); + // try to finish the transaction, if failed just retry in next loop + for (TransactionState transactionState : readyTransactionStates) { + Map transTasks = transactionState.getPublishVersionTasks(); + Set transErrorReplicas = Sets.newHashSet(); + for (PublishVersionTask publishVersionTask : transTasks.values()) { + if (publishVersionTask.isFinished()) { + // sometimes backend finish publish version task, but it maybe failed to change transactionid to version for some tablets + // and it will upload the failed tabletinfo to fe and fe will deal with them + List errorTablets = publishVersionTask.getErrorTablets(); + if (errorTablets == null || errorTablets.size() == 0) { + continue; + } else { + for (long tabletId : errorTablets) { + // tablet inverted index also contains rollingup index + Replica replica = tabletInvertedIndex.getReplica(tabletId, publishVersionTask.getBackendId()); + transErrorReplicas.add(replica); + } + } + } else { + // if task is not finished in time, then set all replica in the backend to error state + List versionInfos = publishVersionTask.getPartitionVersionInfos(); + Set errorPartitionIds = Sets.newHashSet(); + for (TPartitionVersionInfo versionInfo : versionInfos) { + errorPartitionIds.add(versionInfo.getPartition_id()); + } + if (errorPartitionIds.isEmpty()) { + continue; + } + List tabletIds = tabletInvertedIndex.getTabletIdsByBackendId(publishVersionTask.getBackendId()); + for (long tabletId : tabletIds) { + long partitionId = tabletInvertedIndex.getPartitionId(tabletId); + if (errorPartitionIds.contains(partitionId)) { + Replica replica = tabletInvertedIndex.getReplica(tabletId, publishVersionTask.getBackendId()); + transErrorReplicas.add(replica); + } + } + } + } + // the timeout value is related with backend num + long timeoutMillis = Math.min(Config.publish_version_timeout_second * transTasks.size() * 1000, 10000); + // the minimal internal should be 3s + timeoutMillis = Math.max(timeoutMillis, 3000); + + // should not wait clone replica or replica's that with last failed version > 0 + // if wait for them, the publish process will be very slow + int normalReplicasNotRespond = 0; + Set allErrorReplicas = Sets.newHashSet(); + for (Replica replica : transErrorReplicas) { + allErrorReplicas.add(replica.getId()); + if (replica.getState() != ReplicaState.CLONE + && replica.getLastFailedVersion() < 1) { + ++ normalReplicasNotRespond; + } + } + if (normalReplicasNotRespond == 0 + || System.currentTimeMillis() - transactionState.getPublishVersionTime() > timeoutMillis) { + LOG.debug("transTask num {}, error replica id num {}", transTasks.size(), transErrorReplicas.size()); + globalTransactionMgr.finishTransaction(transactionState.getTransactionId(), allErrorReplicas); + if (transactionState.getTransactionStatus() != TransactionStatus.VISIBLE) { + // if finish transaction state failed, then update publish version time, should check + // to finish after some interval + transactionState.updateSendTaskTime(); + LOG.debug("publish version for transation {} failed, has {} error replicas during publish", + transactionState, transErrorReplicas.size()); + } + } + if (transactionState.getTransactionStatus() == TransactionStatus.VISIBLE) { + for (PublishVersionTask task : transactionState.getPublishVersionTasks().values()) { + AgentTaskQueue.removeTask(task.getBackendId(), TTaskType.PUBLISH_VERSION, task.getSignature()); + } + } + } + } +} diff --git a/fe/src/main/java/com/baidu/palo/transaction/TableCommitInfo.java b/fe/src/main/java/com/baidu/palo/transaction/TableCommitInfo.java new file mode 100644 index 0000000000..d77ca537a5 --- /dev/null +++ b/fe/src/main/java/com/baidu/palo/transaction/TableCommitInfo.java @@ -0,0 +1,88 @@ +// Copyright (c) 2017, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 com.baidu.palo.transaction; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.util.Map; + +import com.baidu.palo.common.io.Writable; +import com.google.common.collect.Maps; + +public class TableCommitInfo implements Writable { + + private long tableId; + private Map idToPartitionCommitInfo; + + public TableCommitInfo() { + + } + + public TableCommitInfo(long tableId) { + this.tableId = tableId; + idToPartitionCommitInfo = Maps.newHashMap(); + } + + @Override + public void write(DataOutput out) throws IOException { + out.writeLong(tableId); + if (idToPartitionCommitInfo == null) { + out.writeBoolean(false); + } else { + out.writeBoolean(true); + out.writeInt(idToPartitionCommitInfo.size()); + for (PartitionCommitInfo partitionCommitInfo : idToPartitionCommitInfo.values()) { + partitionCommitInfo.write(out); + } + } + } + + @Override + public void readFields(DataInput in) throws IOException { + tableId = in.readLong(); + boolean hasPartitionInfo = in.readBoolean(); + idToPartitionCommitInfo = Maps.newHashMap(); + if (hasPartitionInfo) { + int elementNum = in.readInt(); + for (int i = 0; i < elementNum; ++i) { + PartitionCommitInfo partitionCommitInfo = new PartitionCommitInfo(); + partitionCommitInfo.readFields(in); + idToPartitionCommitInfo.put(partitionCommitInfo.getPartitionId(), partitionCommitInfo); + } + } + } + + public long getTableId() { + return tableId; + } + + public Map getIdToPartitionCommitInfo() { + return idToPartitionCommitInfo; + } + + public void addPartitionCommitInfo(PartitionCommitInfo info) { + this.idToPartitionCommitInfo.put(info.getPartitionId(), info); + } + + public void removePartition(long partitionId) { + this.idToPartitionCommitInfo.remove(partitionId); + } + + public PartitionCommitInfo getPartitionCommitInfo(long partitionId) { + return this.idToPartitionCommitInfo.get(partitionId); + } +} diff --git a/fe/src/main/java/com/baidu/palo/transaction/TabletCommitInfo.java b/fe/src/main/java/com/baidu/palo/transaction/TabletCommitInfo.java new file mode 100644 index 0000000000..aeb043f80e --- /dev/null +++ b/fe/src/main/java/com/baidu/palo/transaction/TabletCommitInfo.java @@ -0,0 +1,66 @@ +// Copyright (c) 2017, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 com.baidu.palo.transaction; + +import com.baidu.palo.common.io.Writable; +import com.baidu.palo.thrift.TTabletCommitInfo; + +import com.google.common.collect.Lists; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.util.List; + +public class TabletCommitInfo implements Writable { + + private long tabletId; + private long backendId; + + public TabletCommitInfo(long tabletId, long backendId) { + super(); + this.tabletId = tabletId; + this.backendId = backendId; + } + + public long getTabletId() { + return tabletId; + } + + public long getBackendId() { + return backendId; + } + + public static List fromThrift(List tTabletCommitInfos) { + List commitInfos = Lists.newArrayList(); + for (TTabletCommitInfo tTabletCommitInfo : tTabletCommitInfos) { + commitInfos.add(new TabletCommitInfo(tTabletCommitInfo.getTabletId(), tTabletCommitInfo.getBackendId())); + } + return commitInfos; + } + + @Override + public void write(DataOutput out) throws IOException { + out.writeLong(tabletId); + out.writeLong(backendId); + } + + @Override + public void readFields(DataInput in) throws IOException { + tabletId = in.readLong(); + backendId = in.readLong(); + } +} diff --git a/fe/src/main/java/com/baidu/palo/transaction/TransactionCommitFailedException.java b/fe/src/main/java/com/baidu/palo/transaction/TransactionCommitFailedException.java new file mode 100644 index 0000000000..72d3ef362a --- /dev/null +++ b/fe/src/main/java/com/baidu/palo/transaction/TransactionCommitFailedException.java @@ -0,0 +1,31 @@ +// Copyright (c) 2017, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 com.baidu.palo.transaction; + +import com.baidu.palo.common.UserException; + +public class TransactionCommitFailedException extends UserException { + + private static final long serialVersionUID = -2528170792631761535L; + + public TransactionCommitFailedException(String msg) { + super(msg); + } + + public TransactionCommitFailedException(String msg, Throwable e) { + super(msg, e); + } +} diff --git a/fe/src/main/java/com/baidu/palo/transaction/TransactionIDGenerator.java b/fe/src/main/java/com/baidu/palo/transaction/TransactionIDGenerator.java new file mode 100644 index 0000000000..bb53335cf6 --- /dev/null +++ b/fe/src/main/java/com/baidu/palo/transaction/TransactionIDGenerator.java @@ -0,0 +1,73 @@ +// Copyright (c) 2017, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 com.baidu.palo.transaction; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; + +import com.baidu.palo.persist.EditLog; + +public class TransactionIDGenerator { + + public static final long NEXT_ID_INIT_VALUE = 1; + private static final int BATCH_ID_INTERVAL = 1000; + + private long nextId = NEXT_ID_INIT_VALUE; + // has to set it to an invalid value, then it will be logged when id is firstly increment + private long batchEndId = -1; + + private EditLog editLog; + + public TransactionIDGenerator() { + } + + public void setEditLog(EditLog editLog) { + this.editLog = editLog; + } + + // performance is more quickly + public synchronized long getNextTransactionId() { + if (nextId < batchEndId) { + ++ nextId; + return nextId; + } else { + batchEndId = batchEndId + BATCH_ID_INTERVAL; + editLog.logSaveTransactionId(batchEndId); + ++ nextId; + return nextId; + } + } + + public synchronized void initTransactionId(long id) { + if (id > batchEndId) { + batchEndId = id; + nextId = id; + } + } + + // this two function used to read snapshot or write snapshot + public void write(DataOutput out) throws IOException { + out.writeLong(batchEndId); + } + + public void readFields(DataInput in) throws IOException { + long endId = in.readLong(); + batchEndId = endId; + // maybe a little rough + nextId = batchEndId; + } +} diff --git a/fe/src/main/java/com/baidu/palo/transaction/TransactionState.java b/fe/src/main/java/com/baidu/palo/transaction/TransactionState.java new file mode 100644 index 0000000000..e2911b9874 --- /dev/null +++ b/fe/src/main/java/com/baidu/palo/transaction/TransactionState.java @@ -0,0 +1,322 @@ +// Copyright (c) 2017, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 com.baidu.palo.transaction; + +import com.baidu.palo.common.io.Text; +import com.baidu.palo.common.io.Writable; +import com.baidu.palo.task.PublishVersionTask; + +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; + +public class TransactionState implements Writable { + + public enum LoadJobSourceType { + FRONTEND(1), // old dpp load, mini load, insert stmt(not streaming type) use this type + BACKEND_STREAMING(2), // streaming load use this type + INSERT_STREAMING(3); // insert stmt (streaming type) use this type + + private final int flag; + + private LoadJobSourceType(int flag) { + this.flag = flag; + } + + public int value() { + return flag; + } + + public static LoadJobSourceType valueOf(int flag) { + switch (flag) { + case 1: + return FRONTEND; + case 2: + return BACKEND_STREAMING; + case 3: + return INSERT_STREAMING; + default: + return null; + } + } + + @Override + public String toString() { + switch (this) { + case FRONTEND: + return "frontend"; + case BACKEND_STREAMING: + return "backend_streaming"; + case INSERT_STREAMING: + return "insert_streaming"; + default: + return null; + } + } + } + + private long dbId; + private long transactionId; + private String label; + private Map idToTableCommitInfos; + private String coordinator; + private TransactionStatus transactionStatus; + private LoadJobSourceType sourceType; + private long prepareTime; + private long commitTime; + private long finishTime; + private String reason; + private Set errorReplicas; + private CountDownLatch latch; + + // this state need not to be serialized + private Map publishVersionTasks; + private boolean hasSendTask; + private long publishVersionTime; + private TransactionStatus preStatus = null; + + public TransactionState() { + this.dbId = -1; + this.transactionId = -1; + this.label = ""; + this.idToTableCommitInfos = Maps.newHashMap(); + this.coordinator = ""; + this.transactionStatus = TransactionStatus.PREPARE; + this.sourceType = LoadJobSourceType.FRONTEND; + this.prepareTime = -1; + this.commitTime = -1; + this.finishTime = -1; + this.reason = ""; + this.errorReplicas = Sets.newHashSet(); + this.publishVersionTasks = Maps.newHashMap(); + this.hasSendTask = false; + this.latch = new CountDownLatch(1); + } + + public TransactionState(long dbId, long transactionId, String label, LoadJobSourceType sourceType, String coordinator) { + this.dbId = dbId; + this.transactionId = transactionId; + this.label = label; + this.idToTableCommitInfos = Maps.newHashMap(); + this.coordinator = coordinator; + this.transactionStatus = TransactionStatus.PREPARE; + this.sourceType = sourceType; + this.prepareTime = -1; + this.commitTime = -1; + this.finishTime = -1; + this.reason = ""; + this.errorReplicas = Sets.newHashSet(); + this.publishVersionTasks = Maps.newHashMap(); + this.hasSendTask = false; + this.latch = new CountDownLatch(1); + } + + public void setErrorReplicas(Set newErrorReplicas) { + this.errorReplicas = newErrorReplicas; + } + + public boolean isRunning() { + if (transactionStatus == TransactionStatus.PREPARE + || transactionStatus == TransactionStatus.COMMITTED) { + return true; + } + return false; + } + + public void addPublishVersionTask(Long backendId, PublishVersionTask task) { + this.publishVersionTasks.put(backendId, task); + } + + public void setHasSendTask(boolean hasSendTask) { + this.hasSendTask = hasSendTask; + this.publishVersionTime = System.currentTimeMillis(); + } + + public void updateSendTaskTime() { + this.publishVersionTime = System.currentTimeMillis(); + } + + public long getPublishVersionTime() { + return this.publishVersionTime; + } + + public boolean hasSendTask() { + return this.hasSendTask; + } + + @Override + public void write(DataOutput out) throws IOException { + out.writeLong(transactionId); + Text.writeString(out, label); + out.writeLong(dbId); + out.writeInt(idToTableCommitInfos.size()); + for (TableCommitInfo info : idToTableCommitInfos.values()) { + info.write(out); + } + Text.writeString(out, coordinator); + out.writeInt(transactionStatus.value()); + out.writeInt(sourceType.value()); + out.writeLong(prepareTime); + out.writeLong(commitTime); + out.writeLong(finishTime); + Text.writeString(out, reason); + out.writeInt(errorReplicas.size()); + for (long errorReplciaId : errorReplicas) { + out.writeLong(errorReplciaId); + } + } + + @Override + public void readFields(DataInput in) throws IOException { + transactionId = in.readLong(); + label = Text.readString(in); + dbId = in.readLong(); + int size = in.readInt(); + for (int i = 0; i < size; i++) { + TableCommitInfo info = new TableCommitInfo(); + info.readFields(in); + idToTableCommitInfos.put(info.getTableId(), info); + } + coordinator = Text.readString(in); + transactionStatus = TransactionStatus.valueOf(in.readInt()); + sourceType = LoadJobSourceType.valueOf(in.readInt()); + prepareTime = in.readLong(); + commitTime = in.readLong(); + finishTime = in.readLong(); + reason = Text.readString(in); + int errorReplicaNum = in.readInt(); + for (int i = 0; i < errorReplicaNum; ++i) { + errorReplicas.add(in.readLong()); + } + } + + public long getTransactionId() { + return transactionId; + } + + public String getLabel() { + return this.label; + } + + public String getCoordinator() { + return coordinator; + } + + public TransactionStatus getTransactionStatus() { + return transactionStatus; + } + + public long getPrepareTime() { + return prepareTime; + } + + public long getCommitTime() { + return commitTime; + } + + public long getFinishTime() { + return finishTime; + } + + public String getReason() { + return reason; + } + + public TransactionStatus getPreStatus() { + return this.preStatus; + } + + public void setTransactionStatus(TransactionStatus transactionStatus) { + this.preStatus = this.transactionStatus; + this.transactionStatus = transactionStatus; + if (transactionStatus == TransactionStatus.VISIBLE) { + this.latch.countDown(); + } + } + + public void waitTransactionVisible(long timeoutMillis) throws InterruptedException { + this.latch.await(timeoutMillis, TimeUnit.MILLISECONDS); + } + + public void setPrepareTime(long prepareTime) { + this.prepareTime = prepareTime; + } + + public void setCommitTime(long commitTime) { + this.commitTime = commitTime; + } + + public void setFinishTime(long finishTime) { + this.finishTime = finishTime; + } + + public void setReason(String reason) { + this.reason = reason; + } + + public Set getErrorReplicas() { + return this.errorReplicas; + } + + public long getDbId() { + return dbId; + } + + public Map getIdToTableCommitInfos() { + return idToTableCommitInfos; + } + + public void putIdToTableCommitInfo(long tableId, TableCommitInfo tableCommitInfo) { + idToTableCommitInfos.put(tableId, tableCommitInfo); + } + + public TableCommitInfo getTableCommitInfo(long tableId) { + return this.idToTableCommitInfos.get(tableId); + } + + public void removeTable(long tableId) { + this.idToTableCommitInfos.remove(tableId); + } + + @Override + public String toString() { + return "TransactionState [transactionId=" + transactionId + + ", label=" + label + + ", dbId=" + dbId + + ", coordinator=" + coordinator + + ", loadjobsource=" + sourceType + + ", transactionStatus=" + transactionStatus + + ", errorReplicas=" + errorReplicas + + ", prepareTime=" + + prepareTime + ", commitTime=" + commitTime + ", finishTime=" + + finishTime + ", reason=" + reason + "]"; + } + + public LoadJobSourceType getSourceType() { + return sourceType; + } + + public Map getPublishVersionTasks() { + return publishVersionTasks; + } +} diff --git a/fe/src/main/java/com/baidu/palo/transaction/TransactionStatus.java b/fe/src/main/java/com/baidu/palo/transaction/TransactionStatus.java new file mode 100644 index 0000000000..88c60dce67 --- /dev/null +++ b/fe/src/main/java/com/baidu/palo/transaction/TransactionStatus.java @@ -0,0 +1,69 @@ +// Copyright (c) 2017, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 com.baidu.palo.transaction; + +public enum TransactionStatus { + UNKNOWN(0), + PREPARE(1), + COMMITTED(2), + VISIBLE(3), + ABORTED(4); + + private final int flag; + + private TransactionStatus(int flag) { + this.flag = flag; + } + + public int value() { + return flag; + } + + public static TransactionStatus valueOf(int flag) { + switch (flag) { + case 0: + return UNKNOWN; + case 1: + return PREPARE; + case 2: + return COMMITTED; + case 3: + return VISIBLE; + case 4: + return ABORTED; + default: + return null; + } + } + + @Override + public String toString() { + switch (this) { + case UNKNOWN: + return "UNKNOWN"; + case PREPARE: + return "PREPARE"; + case COMMITTED: + return "COMMITTED"; + case VISIBLE: + return "VISIBLE"; + case ABORTED: + return "ABORTED"; + default: + return "UNKNOWN"; + } + } +} diff --git a/fe/src/test/java/com/baidu/palo/alter/RollupJobTest.java b/fe/src/test/java/com/baidu/palo/alter/RollupJobTest.java new file mode 100644 index 0000000000..60399ba508 --- /dev/null +++ b/fe/src/test/java/com/baidu/palo/alter/RollupJobTest.java @@ -0,0 +1,329 @@ +package com.baidu.palo.alter; + +import static org.junit.Assert.assertEquals; + +import com.baidu.palo.alter.AlterJob.JobState; +import com.baidu.palo.analysis.AccessTestUtil; +import com.baidu.palo.analysis.AddRollupClause; +import com.baidu.palo.analysis.AlterClause; +import com.baidu.palo.analysis.Analyzer; +import com.baidu.palo.catalog.Catalog; +import com.baidu.palo.catalog.CatalogTestUtil; +import com.baidu.palo.catalog.Database; +import com.baidu.palo.catalog.FakeCatalog; +import com.baidu.palo.catalog.FakeEditLog; +import com.baidu.palo.catalog.MaterializedIndex; +import com.baidu.palo.catalog.MaterializedIndex.IndexState; +import com.baidu.palo.catalog.OlapTable; +import com.baidu.palo.catalog.OlapTable.OlapTableState; +import com.baidu.palo.catalog.Partition; +import com.baidu.palo.catalog.Partition.PartitionState; +import com.baidu.palo.catalog.Replica; +import com.baidu.palo.catalog.Tablet; +import com.baidu.palo.common.AnalysisException; +import com.baidu.palo.common.FeMetaVersion; +import com.baidu.palo.task.AgentTask; +import com.baidu.palo.task.AgentTaskQueue; +import com.baidu.palo.thrift.TTabletInfo; +import com.baidu.palo.thrift.TTaskType; +import com.baidu.palo.transaction.FakeTransactionIDGenerator; +import com.baidu.palo.transaction.GlobalTransactionMgr; +import com.baidu.palo.transaction.TabletCommitInfo; +import com.baidu.palo.transaction.TransactionState; +import com.baidu.palo.transaction.TransactionState.LoadJobSourceType; +import com.baidu.palo.transaction.TransactionStatus; + +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; + +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.lang.reflect.InvocationTargetException; +import java.util.ArrayList; +import java.util.List; +import java.util.Set; + +import mockit.internal.startup.Startup; + +public class RollupJobTest { + + private static FakeEditLog fakeEditLog; + private static FakeCatalog fakeCatalog; + private static FakeTransactionIDGenerator fakeTransactionIDGenerator; + private static GlobalTransactionMgr masterTransMgr; + private static GlobalTransactionMgr slaveTransMgr; + private static Catalog masterCatalog; + private static Catalog slaveCatalog; + + private String transactionSource = "localfe"; + private static Analyzer analyzer; + private static AddRollupClause clause; + + static { + Startup.initializeIfPossible(); + } + + @Before + public void setUp() throws InstantiationException, IllegalAccessException, IllegalArgumentException, + InvocationTargetException, NoSuchMethodException, SecurityException, AnalysisException { + fakeEditLog = new FakeEditLog(); + fakeCatalog = new FakeCatalog(); + fakeTransactionIDGenerator = new FakeTransactionIDGenerator(); + masterCatalog = CatalogTestUtil.createTestCatalog(); + slaveCatalog = CatalogTestUtil.createTestCatalog(); + masterCatalog.setJournalVersion(FeMetaVersion.VERSION_40); + slaveCatalog.setJournalVersion(FeMetaVersion.VERSION_40); + masterTransMgr = masterCatalog.getGlobalTransactionMgr(); + masterTransMgr.setEditLog(masterCatalog.getEditLog()); + + slaveTransMgr = slaveCatalog.getGlobalTransactionMgr(); + slaveTransMgr.setEditLog(slaveCatalog.getEditLog()); + analyzer = AccessTestUtil.fetchAdminAnalyzer(false); + clause = new AddRollupClause(CatalogTestUtil.testRollupIndex2, Lists.newArrayList("k1", "v"), null, + CatalogTestUtil.testIndex1, null); + clause.analyze(analyzer); + } + + @Test + public void testAddRollup() throws Exception { + FakeCatalog.setCatalog(masterCatalog); + RollupHandler rollupHandler = Catalog.getInstance().getRollupHandler(); + ArrayList alterClauses = new ArrayList<>(); + alterClauses.add(clause); + Database db = masterCatalog.getDb(CatalogTestUtil.testDbId1); + OlapTable olapTable = (OlapTable) db.getTable(CatalogTestUtil.testTableId1); + rollupHandler.process(alterClauses, db, olapTable, false); + RollupJob rollupJob = (RollupJob) rollupHandler.getAlterJob(CatalogTestUtil.testTableId1); + Assert.assertEquals(CatalogTestUtil.testIndexId1, rollupJob.getBaseIndexId()); + Assert.assertEquals(CatalogTestUtil.testRollupIndex2, rollupJob.getRollupIndexName()); + } + + // start a rollup, then finished + @Test + public void testRollup1() throws Exception { + FakeCatalog.setCatalog(masterCatalog); + RollupHandler rollupHandler = Catalog.getInstance().getRollupHandler(); + + // add a rollup job + ArrayList alterClauses = new ArrayList<>(); + alterClauses.add(clause); + Database db = masterCatalog.getDb(CatalogTestUtil.testDbId1); + OlapTable olapTable = (OlapTable) db.getTable(CatalogTestUtil.testTableId1); + Partition testPartition = olapTable.getPartition(CatalogTestUtil.testPartitionId1); + rollupHandler.process(alterClauses, db, olapTable, false); + RollupJob rollupJob = (RollupJob) rollupHandler.getAlterJob(CatalogTestUtil.testTableId1); + Assert.assertEquals(CatalogTestUtil.testIndexId1, rollupJob.getBaseIndexId()); + Assert.assertEquals(CatalogTestUtil.testRollupIndex2, rollupJob.getRollupIndexName()); + MaterializedIndex rollupIndex = rollupJob.getRollupIndex(CatalogTestUtil.testPartitionId1); + MaterializedIndex baseIndex = testPartition.getBaseIndex(); + assertEquals(IndexState.ROLLUP, rollupIndex.getState()); + assertEquals(IndexState.NORMAL, baseIndex.getState()); + assertEquals(OlapTableState.ROLLUP, olapTable.getState()); + assertEquals(PartitionState.ROLLUP, testPartition.getState()); + Tablet rollupTablet = rollupIndex.getTablets().get(0); + List replicas = rollupTablet.getReplicas(); + Replica rollupReplica1 = replicas.get(0); + Replica rollupReplica2 = replicas.get(1); + Replica rollupReplica3 = replicas.get(2); + + assertEquals(-1, rollupReplica1.getVersion()); + assertEquals(-1, rollupReplica2.getVersion()); + assertEquals(-1, rollupReplica3.getVersion()); + assertEquals(CatalogTestUtil.testStartVersion, rollupReplica1.getLastFailedVersion()); + assertEquals(CatalogTestUtil.testStartVersion, rollupReplica2.getLastFailedVersion()); + assertEquals(CatalogTestUtil.testStartVersion, rollupReplica3.getLastFailedVersion()); + assertEquals(-1, rollupReplica1.getLastSuccessVersion()); + assertEquals(-1, rollupReplica2.getLastSuccessVersion()); + assertEquals(-1, rollupReplica3.getLastSuccessVersion()); + + // rollup handler run one cycle, agent task is generated and send tasks + rollupHandler.runOneCycle(); + AgentTask task1 = AgentTaskQueue.getTask(rollupReplica1.getBackendId(), TTaskType.ROLLUP, rollupTablet.getId()); + AgentTask task2 = AgentTaskQueue.getTask(rollupReplica2.getBackendId(), TTaskType.ROLLUP, rollupTablet.getId()); + AgentTask task3 = AgentTaskQueue.getTask(rollupReplica3.getBackendId(), TTaskType.ROLLUP, rollupTablet.getId()); + + // be report finishe rollup success + TTabletInfo tTabletInfo = new TTabletInfo(rollupTablet.getId(), CatalogTestUtil.testSchemaHash1, + CatalogTestUtil.testStartVersion, CatalogTestUtil.testStartVersionHash, 0, 0); + rollupHandler.handleFinishedReplica(task1, tTabletInfo, -1); + rollupHandler.handleFinishedReplica(task2, tTabletInfo, -1); + rollupHandler.handleFinishedReplica(task3, tTabletInfo, -1); + + // rollup hander run one cycle again, the rollup job is finishing + rollupHandler.runOneCycle(); + Assert.assertEquals(JobState.FINISHING, rollupJob.getState()); + assertEquals(CatalogTestUtil.testStartVersion, rollupReplica1.getVersion()); + assertEquals(CatalogTestUtil.testStartVersion, rollupReplica2.getVersion()); + assertEquals(CatalogTestUtil.testStartVersion, rollupReplica3.getVersion()); + assertEquals(-1, rollupReplica1.getLastFailedVersion()); + assertEquals(-1, rollupReplica2.getLastFailedVersion()); + assertEquals(-1, rollupReplica3.getLastFailedVersion()); + assertEquals(CatalogTestUtil.testStartVersion, rollupReplica1.getLastSuccessVersion()); + assertEquals(CatalogTestUtil.testStartVersion, rollupReplica1.getLastSuccessVersion()); + assertEquals(CatalogTestUtil.testStartVersion, rollupReplica1.getLastSuccessVersion()); + } + + // load some data and one replica has errors + // start a rollup and then load data + // load finished and rollup finished + @Test + public void testRollup2() throws Exception { + FakeCatalog.setCatalog(masterCatalog); + // load one transaction with backend 2 has errors + long transactionId = masterTransMgr.beginTransaction(CatalogTestUtil.testDbId1, + CatalogTestUtil.testTxnLable1, transactionSource, + LoadJobSourceType.FRONTEND); + // commit a transaction, backend 2 has errors + TabletCommitInfo tabletCommitInfo1 = new TabletCommitInfo(CatalogTestUtil.testTabletId1, + CatalogTestUtil.testBackendId1); + // TabletCommitInfo tabletCommitInfo2 = new + // TabletCommitInfo(CatalogTestUtil.testTabletId1, + // CatalogTestUtil.testBackendId2); + TabletCommitInfo tabletCommitInfo3 = new TabletCommitInfo(CatalogTestUtil.testTabletId1, + CatalogTestUtil.testBackendId3); + List transTablets = Lists.newArrayList(); + transTablets.add(tabletCommitInfo1); + // transTablets.add(tabletCommitInfo2); + transTablets.add(tabletCommitInfo3); + masterTransMgr.commitTransaction(CatalogTestUtil.testDbId1, transactionId, transTablets); + TransactionState transactionState = fakeEditLog.getTransaction(transactionId); + assertEquals(TransactionStatus.COMMITTED, transactionState.getTransactionStatus()); + Set errorReplicaIds = Sets.newHashSet(); + errorReplicaIds.add(CatalogTestUtil.testReplicaId2); + masterTransMgr.finishTransaction(transactionId, errorReplicaIds); + transactionState = fakeEditLog.getTransaction(transactionId); + assertEquals(TransactionStatus.VISIBLE, transactionState.getTransactionStatus()); + + // start a rollup + RollupHandler rollupHandler = Catalog.getInstance().getRollupHandler(); + // add a rollup job + ArrayList alterClauses = new ArrayList<>(); + alterClauses.add(clause); + Database db = masterCatalog.getDb(CatalogTestUtil.testDbId1); + OlapTable olapTable = (OlapTable) db.getTable(CatalogTestUtil.testTableId1); + Partition testPartition = olapTable.getPartition(CatalogTestUtil.testPartitionId1); + rollupHandler.process(alterClauses, db, olapTable, false); + RollupJob rollupJob = (RollupJob) rollupHandler.getAlterJob(CatalogTestUtil.testTableId1); + Assert.assertEquals(CatalogTestUtil.testIndexId1, rollupJob.getBaseIndexId()); + Assert.assertEquals(CatalogTestUtil.testRollupIndex2, rollupJob.getRollupIndexName()); + MaterializedIndex rollupIndex = rollupJob.getRollupIndex(CatalogTestUtil.testPartitionId1); + MaterializedIndex baseIndex = testPartition.getBaseIndex(); + assertEquals(IndexState.ROLLUP, rollupIndex.getState()); + assertEquals(IndexState.NORMAL, baseIndex.getState()); + assertEquals(OlapTableState.ROLLUP, olapTable.getState()); + assertEquals(PartitionState.ROLLUP, testPartition.getState()); + Tablet rollupTablet = rollupIndex.getTablets().get(0); + List replicas = rollupTablet.getReplicas(); + Replica rollupReplica1 = replicas.get(0); + Replica rollupReplica3 = replicas.get(1); + assertEquals(2, rollupTablet.getReplicas().size()); + + assertEquals(-1, rollupReplica1.getVersion()); + assertEquals(-1, rollupReplica3.getVersion()); + assertEquals(CatalogTestUtil.testStartVersion + 1, rollupReplica1.getLastFailedVersion()); + assertEquals(CatalogTestUtil.testStartVersion + 1, rollupReplica3.getLastFailedVersion()); + assertEquals(-1, rollupReplica1.getLastSuccessVersion()); + assertEquals(-1, rollupReplica3.getLastSuccessVersion()); + + // rollup handler run one cycle, agent task is generated and send tasks + rollupHandler.runOneCycle(); + AgentTask task1 = AgentTaskQueue.getTask(rollupReplica1.getBackendId(), TTaskType.ROLLUP, rollupTablet.getId()); + AgentTask task3 = AgentTaskQueue.getTask(rollupReplica3.getBackendId(), TTaskType.ROLLUP, rollupTablet.getId()); + + // be report finishe rollup success + TTabletInfo tTabletInfo = new TTabletInfo(rollupTablet.getId(), CatalogTestUtil.testSchemaHash1, + CatalogTestUtil.testStartVersion + 1, CatalogTestUtil.testPartitionNextVersionHash, 0, 0); + rollupHandler.handleFinishedReplica(task1, tTabletInfo, -1); + rollupHandler.handleFinishedReplica(task3, tTabletInfo, -1); + + // rollup hander run one cycle again, the rollup job is finishing + rollupHandler.runOneCycle(); + Assert.assertEquals(JobState.FINISHING, rollupJob.getState()); + assertEquals(CatalogTestUtil.testStartVersion + 1, rollupReplica1.getVersion()); + assertEquals(CatalogTestUtil.testStartVersion + 1, rollupReplica3.getVersion()); + assertEquals(-1, rollupReplica1.getLastFailedVersion()); + assertEquals(-1, rollupReplica3.getLastFailedVersion()); + assertEquals(CatalogTestUtil.testStartVersion + 1, rollupReplica1.getLastSuccessVersion()); + assertEquals(CatalogTestUtil.testStartVersion + 1, rollupReplica3.getLastSuccessVersion()); + } + + // start a rollup and then load data + // but load to rolluping index failed, then rollup is cancelled + @Test + public void testRollup3() throws Exception { + FakeCatalog.setCatalog(masterCatalog); + RollupHandler rollupHandler = Catalog.getInstance().getRollupHandler(); + + // add a rollup job + ArrayList alterClauses = new ArrayList<>(); + alterClauses.add(clause); + Database db = masterCatalog.getDb(CatalogTestUtil.testDbId1); + OlapTable olapTable = (OlapTable) db.getTable(CatalogTestUtil.testTableId1); + Partition testPartition = olapTable.getPartition(CatalogTestUtil.testPartitionId1); + rollupHandler.process(alterClauses, db, olapTable, false); + RollupJob rollupJob = (RollupJob) rollupHandler.getAlterJob(CatalogTestUtil.testTableId1); + Assert.assertEquals(CatalogTestUtil.testIndexId1, rollupJob.getBaseIndexId()); + Assert.assertEquals(CatalogTestUtil.testRollupIndex2, rollupJob.getRollupIndexName()); + MaterializedIndex rollupIndex = rollupJob.getRollupIndex(CatalogTestUtil.testPartitionId1); + MaterializedIndex baseIndex = testPartition.getBaseIndex(); + assertEquals(IndexState.ROLLUP, rollupIndex.getState()); + assertEquals(IndexState.NORMAL, baseIndex.getState()); + assertEquals(OlapTableState.ROLLUP, olapTable.getState()); + assertEquals(PartitionState.ROLLUP, testPartition.getState()); + Tablet rollupTablet = rollupIndex.getTablets().get(0); + List replicas = rollupTablet.getReplicas(); + Replica rollupReplica1 = replicas.get(0); + Replica rollupReplica2 = replicas.get(1); + Replica rollupReplica3 = replicas.get(2); + + // rollup handler run one cycle, agent task is generated and send tasks + rollupHandler.runOneCycle(); + AgentTask task1 = AgentTaskQueue.getTask(rollupReplica1.getBackendId(), TTaskType.ROLLUP, rollupTablet.getId()); + AgentTask task2 = AgentTaskQueue.getTask(rollupReplica2.getBackendId(), TTaskType.ROLLUP, rollupTablet.getId()); + AgentTask task3 = AgentTaskQueue.getTask(rollupReplica3.getBackendId(), TTaskType.ROLLUP, rollupTablet.getId()); + + // load a transaction, but rollup tablet failed, then the rollup job should be + // cancelled + long transactionId = masterTransMgr.beginTransaction(CatalogTestUtil.testDbId1, + CatalogTestUtil.testTxnLable1, + transactionSource, + LoadJobSourceType.FRONTEND); + // commit a transaction, backend 2 has errors + TabletCommitInfo tabletCommitInfo1 = new TabletCommitInfo(CatalogTestUtil.testTabletId1, + CatalogTestUtil.testBackendId1); + TabletCommitInfo tabletCommitInfo2 = new TabletCommitInfo(CatalogTestUtil.testTabletId1, + CatalogTestUtil.testBackendId2); + TabletCommitInfo tabletCommitInfo3 = new TabletCommitInfo(CatalogTestUtil.testTabletId1, + CatalogTestUtil.testBackendId3); + List transTablets = Lists.newArrayList(); + transTablets.add(tabletCommitInfo1); + transTablets.add(tabletCommitInfo2); + transTablets.add(tabletCommitInfo3); + masterTransMgr.commitTransaction(CatalogTestUtil.testDbId1, transactionId, transTablets); + TransactionState transactionState = fakeEditLog.getTransaction(transactionId); + assertEquals(TransactionStatus.COMMITTED, transactionState.getTransactionStatus()); + Set errorReplicaIds = Sets.newHashSet(); + errorReplicaIds.add(CatalogTestUtil.testReplicaId2); + masterTransMgr.finishTransaction(transactionId, errorReplicaIds); + transactionState = fakeEditLog.getTransaction(transactionId); + + // rollup replca's last failed version should change to 13 + assertEquals(CatalogTestUtil.testStartVersion + 1, rollupReplica1.getLastFailedVersion()); + assertEquals(CatalogTestUtil.testStartVersion + 1, rollupReplica2.getLastFailedVersion()); + assertEquals(CatalogTestUtil.testStartVersion + 1, rollupReplica3.getLastFailedVersion()); + + // be report finishe rollup success + TTabletInfo tTabletInfo = new TTabletInfo(rollupTablet.getId(), CatalogTestUtil.testSchemaHash1, + CatalogTestUtil.testStartVersion, CatalogTestUtil.testStartVersionHash, 0, 0); + rollupHandler.handleFinishedReplica(task1, tTabletInfo, -1); + rollupHandler.handleFinishedReplica(task2, tTabletInfo, -1); + rollupHandler.handleFinishedReplica(task3, tTabletInfo, -1); + + // rollup hander run one cycle again, the rollup job is finishing + rollupHandler.runOneCycle(); + Assert.assertEquals(JobState.CANCELLED, rollupJob.getState()); + assertEquals(1, testPartition.getMaterializedIndices().size()); + } +} diff --git a/fe/src/test/java/com/baidu/palo/alter/SchemaChangeJobTest.java b/fe/src/test/java/com/baidu/palo/alter/SchemaChangeJobTest.java new file mode 100644 index 0000000000..71d74f7b64 --- /dev/null +++ b/fe/src/test/java/com/baidu/palo/alter/SchemaChangeJobTest.java @@ -0,0 +1,256 @@ +package com.baidu.palo.alter; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; + +import java.lang.reflect.InvocationTargetException; +import java.util.ArrayList; +import java.util.List; +import java.util.Set; + +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import com.baidu.palo.alter.AlterJob.JobState; +import com.baidu.palo.analysis.AccessTestUtil; +import com.baidu.palo.analysis.AddColumnClause; +import com.baidu.palo.analysis.AlterClause; +import com.baidu.palo.analysis.Analyzer; +import com.baidu.palo.analysis.ColumnPosition; +import com.baidu.palo.catalog.AggregateType; +import com.baidu.palo.catalog.Catalog; +import com.baidu.palo.catalog.CatalogTestUtil; +import com.baidu.palo.catalog.Column; +import com.baidu.palo.catalog.ColumnType; +import com.baidu.palo.catalog.Database; +import com.baidu.palo.catalog.FakeCatalog; +import com.baidu.palo.catalog.FakeEditLog; +import com.baidu.palo.catalog.MaterializedIndex; +import com.baidu.palo.catalog.OlapTable; +import com.baidu.palo.catalog.Partition; +import com.baidu.palo.catalog.PrimitiveType; +import com.baidu.palo.catalog.Replica; +import com.baidu.palo.catalog.Tablet; +import com.baidu.palo.catalog.MaterializedIndex.IndexState; +import com.baidu.palo.catalog.OlapTable.OlapTableState; +import com.baidu.palo.catalog.Partition.PartitionState; +import com.baidu.palo.catalog.Replica.ReplicaState; +import com.baidu.palo.common.AnalysisException; +import com.baidu.palo.common.FeMetaVersion; +import com.baidu.palo.task.AgentTask; +import com.baidu.palo.task.AgentTaskQueue; +import com.baidu.palo.thrift.TTabletInfo; +import com.baidu.palo.thrift.TTaskType; +import com.baidu.palo.transaction.FakeTransactionIDGenerator; +import com.baidu.palo.transaction.GlobalTransactionMgr; +import com.baidu.palo.transaction.TabletCommitInfo; +import com.baidu.palo.transaction.TransactionState; +import com.baidu.palo.transaction.TransactionStatus; +import com.baidu.palo.transaction.TransactionState.LoadJobSourceType; +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; + +public class SchemaChangeJobTest { + + private static FakeEditLog fakeEditLog; + private static FakeCatalog fakeCatalog; + private static FakeTransactionIDGenerator fakeTransactionIDGenerator; + private static GlobalTransactionMgr masterTransMgr; + private static GlobalTransactionMgr slaveTransMgr; + private static Catalog masterCatalog; + private static Catalog slaveCatalog; + + private String transactionSource = "localfe"; + private static Analyzer analyzer; + private static Column newCol = new Column("add_v", new ColumnType(PrimitiveType.INT), false, AggregateType.MAX, + false, "1", ""); + private static AddColumnClause addColumnClause = new AddColumnClause(newCol, new ColumnPosition("v"), null, null); + + @Before + public void setUp() throws InstantiationException, IllegalAccessException, IllegalArgumentException, + InvocationTargetException, NoSuchMethodException, SecurityException, AnalysisException { + fakeEditLog = new FakeEditLog(); + fakeCatalog = new FakeCatalog(); + fakeTransactionIDGenerator = new FakeTransactionIDGenerator(); + masterCatalog = CatalogTestUtil.createTestCatalog(); + slaveCatalog = CatalogTestUtil.createTestCatalog(); + masterCatalog.setJournalVersion(FeMetaVersion.VERSION_40); + slaveCatalog.setJournalVersion(FeMetaVersion.VERSION_40); + masterTransMgr = masterCatalog.getGlobalTransactionMgr(); + masterTransMgr.setEditLog(masterCatalog.getEditLog()); + slaveTransMgr = slaveCatalog.getGlobalTransactionMgr(); + slaveTransMgr.setEditLog(slaveCatalog.getEditLog()); + analyzer = AccessTestUtil.fetchAdminAnalyzer(false); + addColumnClause.analyze(analyzer); + } + + @Test + public void testAddSchemaChange() throws Exception { + FakeCatalog.setCatalog(masterCatalog); + SchemaChangeHandler schemaChangeHandler = Catalog.getInstance().getSchemaChangeHandler(); + ArrayList alterClauses = new ArrayList<>(); + alterClauses.add(addColumnClause); + Database db = masterCatalog.getDb(CatalogTestUtil.testDbId1); + OlapTable olapTable = (OlapTable) db.getTable(CatalogTestUtil.testTableId1); + schemaChangeHandler.process(alterClauses, "default", db, olapTable); + SchemaChangeJob schemaChangeJob = (SchemaChangeJob) schemaChangeHandler + .getAlterJob(CatalogTestUtil.testTableId1); + Assert.assertEquals(OlapTableState.SCHEMA_CHANGE, olapTable.getState()); + } + + // start a schema change, then finished + @Test + public void testSchemaChange1() throws Exception { + FakeCatalog.setCatalog(masterCatalog); + SchemaChangeHandler schemaChangeHandler = Catalog.getInstance().getSchemaChangeHandler(); + + // add a schema change job + ArrayList alterClauses = new ArrayList<>(); + alterClauses.add(addColumnClause); + Database db = masterCatalog.getDb(CatalogTestUtil.testDbId1); + OlapTable olapTable = (OlapTable) db.getTable(CatalogTestUtil.testTableId1); + Partition testPartition = olapTable.getPartition(CatalogTestUtil.testPartitionId1); + schemaChangeHandler.process(alterClauses, "default", db, olapTable); + SchemaChangeJob schemaChangeJob = (SchemaChangeJob) schemaChangeHandler + .getAlterJob(CatalogTestUtil.testTableId1); + MaterializedIndex baseIndex = testPartition.getBaseIndex(); + assertEquals(IndexState.SCHEMA_CHANGE, baseIndex.getState()); + assertEquals(OlapTableState.SCHEMA_CHANGE, olapTable.getState()); + assertEquals(PartitionState.SCHEMA_CHANGE, testPartition.getState()); + Tablet baseTablet = baseIndex.getTablets().get(0); + List replicas = baseTablet.getReplicas(); + Replica replica1 = replicas.get(0); + Replica replica2 = replicas.get(1); + Replica replica3 = replicas.get(2); + + assertEquals(CatalogTestUtil.testStartVersion, replica1.getVersion()); + assertEquals(CatalogTestUtil.testStartVersion, replica2.getVersion()); + assertEquals(CatalogTestUtil.testStartVersion, replica3.getVersion()); + assertEquals(-1, replica1.getLastFailedVersion()); + assertEquals(-1, replica2.getLastFailedVersion()); + assertEquals(-1, replica3.getLastFailedVersion()); + assertEquals(CatalogTestUtil.testStartVersion, replica1.getLastSuccessVersion()); + assertEquals(CatalogTestUtil.testStartVersion, replica2.getLastSuccessVersion()); + assertEquals(CatalogTestUtil.testStartVersion, replica3.getLastSuccessVersion()); + + // schemachange handler run one cycle, agent task is generated and send tasks + schemaChangeHandler.runOneCycle(); + AgentTask task1 = AgentTaskQueue.getTask(replica1.getBackendId(), TTaskType.SCHEMA_CHANGE, baseTablet.getId()); + AgentTask task2 = AgentTaskQueue.getTask(replica2.getBackendId(), TTaskType.SCHEMA_CHANGE, baseTablet.getId()); + AgentTask task3 = AgentTaskQueue.getTask(replica3.getBackendId(), TTaskType.SCHEMA_CHANGE, baseTablet.getId()); + + // be report finishe schema change success, report the new schema hash + TTabletInfo tTabletInfo = new TTabletInfo(baseTablet.getId(), + schemaChangeJob.getSchemaHashByIndexId(CatalogTestUtil.testIndexId1), CatalogTestUtil.testStartVersion, + CatalogTestUtil.testStartVersionHash, 0, 0); + schemaChangeHandler.handleFinishedReplica(task1, tTabletInfo, -1); + schemaChangeHandler.handleFinishedReplica(task2, tTabletInfo, -1); + schemaChangeHandler.handleFinishedReplica(task3, tTabletInfo, -1); + + // schema change hander run one cycle again, the rollup job is finishing + schemaChangeHandler.runOneCycle(); + Assert.assertEquals(JobState.FINISHING, schemaChangeJob.getState()); + assertEquals(CatalogTestUtil.testStartVersion, replica1.getVersion()); + assertEquals(CatalogTestUtil.testStartVersion, replica2.getVersion()); + assertEquals(CatalogTestUtil.testStartVersion, replica3.getVersion()); + assertEquals(-1, replica1.getLastFailedVersion()); + assertEquals(-1, replica2.getLastFailedVersion()); + assertEquals(-1, replica3.getLastFailedVersion()); + assertEquals(CatalogTestUtil.testStartVersion, replica1.getLastSuccessVersion()); + assertEquals(CatalogTestUtil.testStartVersion, replica2.getLastSuccessVersion()); + assertEquals(CatalogTestUtil.testStartVersion, replica3.getLastSuccessVersion()); + } + + // load some data and one replica has errors + // start a schema change and then load data + // load finished and schema change finished + @Test + public void testSchemaChange2() throws Exception { + FakeCatalog.setCatalog(masterCatalog); + SchemaChangeHandler schemaChangeHandler = Catalog.getInstance().getSchemaChangeHandler(); + // load one transaction with backend 2 has errors + long transactionId = masterTransMgr.beginTransaction(CatalogTestUtil.testDbId1, + CatalogTestUtil.testTxnLable1, + transactionSource, + LoadJobSourceType.FRONTEND); + // commit a transaction, backend 2 has errors + TabletCommitInfo tabletCommitInfo1 = new TabletCommitInfo(CatalogTestUtil.testTabletId1, + CatalogTestUtil.testBackendId1); + // TabletCommitInfo tabletCommitInfo2 = new + // TabletCommitInfo(CatalogTestUtil.testTabletId1, + // CatalogTestUtil.testBackendId2); + TabletCommitInfo tabletCommitInfo3 = new TabletCommitInfo(CatalogTestUtil.testTabletId1, + CatalogTestUtil.testBackendId3); + List transTablets = Lists.newArrayList(); + transTablets.add(tabletCommitInfo1); + // transTablets.add(tabletCommitInfo2); + transTablets.add(tabletCommitInfo3); + masterTransMgr.commitTransaction(CatalogTestUtil.testDbId1, transactionId, transTablets); + TransactionState transactionState = fakeEditLog.getTransaction(transactionId); + assertEquals(TransactionStatus.COMMITTED, transactionState.getTransactionStatus()); + Set errorReplicaIds = Sets.newHashSet(); + errorReplicaIds.add(CatalogTestUtil.testReplicaId2); + masterTransMgr.finishTransaction(transactionId, errorReplicaIds); + transactionState = fakeEditLog.getTransaction(transactionId); + assertEquals(TransactionStatus.VISIBLE, transactionState.getTransactionStatus()); + + // start a schema change + ArrayList alterClauses = new ArrayList<>(); + alterClauses.add(addColumnClause); + Database db = masterCatalog.getDb(CatalogTestUtil.testDbId1); + OlapTable olapTable = (OlapTable) db.getTable(CatalogTestUtil.testTableId1); + Partition testPartition = olapTable.getPartition(CatalogTestUtil.testPartitionId1); + schemaChangeHandler.process(alterClauses, "default", db, olapTable); + SchemaChangeJob schemaChangeJob = (SchemaChangeJob) schemaChangeHandler + .getAlterJob(CatalogTestUtil.testTableId1); + MaterializedIndex baseIndex = testPartition.getBaseIndex(); + assertEquals(IndexState.SCHEMA_CHANGE, baseIndex.getState()); + assertEquals(OlapTableState.SCHEMA_CHANGE, olapTable.getState()); + assertEquals(PartitionState.SCHEMA_CHANGE, testPartition.getState()); + Tablet baseTablet = baseIndex.getTablets().get(0); + List replicas = baseTablet.getReplicas(); + Replica replica1 = replicas.get(0); + Replica replica2 = replicas.get(1); + Replica replica3 = replicas.get(2); + assertEquals(3, baseTablet.getReplicas().size()); + + assertEquals(ReplicaState.SCHEMA_CHANGE, replica1.getState()); + assertEquals(ReplicaState.NORMAL, replica2.getState()); + assertEquals(ReplicaState.SCHEMA_CHANGE, replica3.getState()); + + assertEquals(CatalogTestUtil.testStartVersion + 1, replica1.getVersion()); + assertEquals(CatalogTestUtil.testStartVersion, replica2.getVersion()); + assertEquals(CatalogTestUtil.testStartVersion + 1, replica3.getVersion()); + assertEquals(-1, replica1.getLastFailedVersion()); + assertEquals(CatalogTestUtil.testStartVersion + 1, replica2.getLastFailedVersion()); + assertEquals(-1, replica3.getLastFailedVersion()); + assertEquals(CatalogTestUtil.testStartVersion + 1, replica1.getLastSuccessVersion()); + assertEquals(CatalogTestUtil.testStartVersion, replica2.getLastSuccessVersion()); + assertEquals(CatalogTestUtil.testStartVersion + 1, replica3.getLastSuccessVersion()); + + // schemachange handler run one cycle, agent task is generated and send tasks + schemaChangeHandler.runOneCycle(); + AgentTask task1 = AgentTaskQueue.getTask(replica1.getBackendId(), TTaskType.SCHEMA_CHANGE, baseTablet.getId()); + AgentTask task2 = AgentTaskQueue.getTask(replica2.getBackendId(), TTaskType.SCHEMA_CHANGE, baseTablet.getId()); + AgentTask task3 = AgentTaskQueue.getTask(replica3.getBackendId(), TTaskType.SCHEMA_CHANGE, baseTablet.getId()); + assertNull(task2); + + // be report finish schema change success, report the new schema hash + TTabletInfo tTabletInfo = new TTabletInfo(baseTablet.getId(), + schemaChangeJob.getSchemaHashByIndexId(CatalogTestUtil.testIndexId1), CatalogTestUtil.testStartVersion, + CatalogTestUtil.testStartVersionHash, 0, 0); + schemaChangeHandler.handleFinishedReplica(task1, tTabletInfo, -1); + schemaChangeHandler.handleFinishedReplica(task3, tTabletInfo, -1); + + // rollup hander run one cycle again, the rollup job is finishing + schemaChangeHandler.runOneCycle(); + Assert.assertEquals(JobState.FINISHING, schemaChangeJob.getState()); + assertEquals(CatalogTestUtil.testStartVersion + 1, replica1.getVersion()); + assertEquals(CatalogTestUtil.testStartVersion + 1, replica3.getVersion()); + assertEquals(-1, replica1.getLastFailedVersion()); + assertEquals(-1, replica3.getLastFailedVersion()); + assertEquals(CatalogTestUtil.testStartVersion + 1, replica1.getLastSuccessVersion()); + assertEquals(CatalogTestUtil.testStartVersion + 1, replica3.getLastSuccessVersion()); + } +} diff --git a/fe/src/test/java/com/baidu/palo/analysis/AdminShowReplicaTest.java b/fe/src/test/java/com/baidu/palo/analysis/AdminShowReplicaTest.java new file mode 100644 index 0000000000..0bcdc3d40a --- /dev/null +++ b/fe/src/test/java/com/baidu/palo/analysis/AdminShowReplicaTest.java @@ -0,0 +1,77 @@ +package com.baidu.palo.analysis; + +import org.junit.Assert; +import org.junit.Test; + +import java.io.StringReader; +import java.lang.reflect.Method; + +public class AdminShowReplicaTest { + + @Test + public void testShowReplicaStatus() { + String stmt = new String("ADMIN SHOW REPLICA STATUS FROM db.tbl1 WHERE status = 'ok'"); + testAnalyzeWhere(stmt, true); + + stmt = new String("ADMIN SHOW REPLICA STATUS FROM db.tbl1 WHERE status != 'ok'"); + testAnalyzeWhere(stmt, true); + + stmt = new String("ADMIN SHOW REPLICA STATUS FROM db.tbl1 WHERE status = 'dead'"); + testAnalyzeWhere(stmt, true); + + stmt = new String("ADMIN SHOW REPLICA STATUS FROM db.tbl1 WHERE status != 'VERSION_ERROR'"); + testAnalyzeWhere(stmt, true); + + stmt = new String("ADMIN SHOW REPLICA STATUS FROM db.tbl1 WHERE status = 'MISSING'"); + testAnalyzeWhere(stmt, true); + + stmt = new String("ADMIN SHOW REPLICA STATUS FROM db.tbl1 WHERE status = 'missing'"); + testAnalyzeWhere(stmt, true); + + stmt = new String("ADMIN SHOW REPLICA STATUS FROM db.tbl1 WHERE status != 'what'"); + testAnalyzeWhere(stmt, false); + + stmt = new String("ADMIN SHOW REPLICA STATUS FROM db.tbl1 WHERE status = 'how'"); + testAnalyzeWhere(stmt, false); + } + + private void testAnalyzeWhere(String stmt, boolean correct) { + SqlParser parser = new SqlParser(new SqlScanner(new StringReader(stmt))); + AdminShowReplicaStatusStmt showStmt = null; + try { + showStmt = (AdminShowReplicaStatusStmt) parser.parse().value; + } catch (Error e) { + Assert.fail(e.getMessage()); + } catch (Exception e) { + Assert.fail(e.getMessage()); + } + + try { + Method method = AdminShowReplicaStatusStmt.class.getDeclaredMethod("analyzeWhere"); + method.setAccessible(true); + if (!(Boolean) method.invoke(showStmt)) { + if (correct) { + Assert.fail(); + } + return; + } + } catch (Exception e) { + if (tryAssert(correct, e)) { + return; + } + } + if (!correct) { + Assert.fail(); + } + } + + private boolean tryAssert(boolean correct, Exception e) { + if (correct) { + e.printStackTrace(); + Assert.fail(e.getMessage()); + } + return true; + } + + +} diff --git a/fe/src/test/java/com/baidu/palo/catalog/CatalogTestUtil.java b/fe/src/test/java/com/baidu/palo/catalog/CatalogTestUtil.java new file mode 100644 index 0000000000..d7cc54a0db --- /dev/null +++ b/fe/src/test/java/com/baidu/palo/catalog/CatalogTestUtil.java @@ -0,0 +1,294 @@ +package com.baidu.palo.catalog; + +import com.baidu.palo.analysis.PartitionKeyDesc; +import com.baidu.palo.analysis.SingleRangePartitionDesc; +import com.baidu.palo.catalog.MaterializedIndex.IndexState; +import com.baidu.palo.catalog.Replica.ReplicaState; +import com.baidu.palo.common.DdlException; +import com.baidu.palo.persist.EditLog; +import com.baidu.palo.system.Backend; +import com.baidu.palo.system.SystemInfoService; +import com.baidu.palo.thrift.TDisk; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; + +import java.lang.reflect.Constructor; +import java.lang.reflect.InvocationTargetException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class CatalogTestUtil { + + public static String testDb1 = "testDb1"; + public static long testDbId1 = 1; + public static String testTable1 = "testTable1"; + public static long testTableId1 = 2; + public static String testPartition1 = "testPartition1"; + public static long testPartitionId1 = 3; + public static String testIndex1 = "testIndex1"; + public static long testIndexId1 = 2; // the base indexid == tableid + public static int testSchemaHash1 = 93423942; + public static long testBackendId1 = 5; + public static long testBackendId2 = 6; + public static long testBackendId3 = 7; + public static long testReplicaId1 = 8; + public static long testReplicaId2 = 9; + public static long testReplicaId3 = 10; + public static long testTabletId1 = 11; + public static long testStartVersion = 12; + public static long testStartVersionHash = 12312; + public static long testPartitionCurrentVersionHash = 12312; + public static long testPartitionNextVersionHash = 123123123; + public static long testRollupIndexId2 = 13; + public static String testRollupIndex2 = "newRollupIndex"; + public static String testTxnLable1 = "testTxnLable1"; + public static String testTxnLable2 = "testTxnLable2"; + public static String testTxnLable3 = "testTxnLable3"; + public static String testTxnLable4 = "testTxnLable4"; + public static String testTxnLable5 = "testTxnLable5"; + public static String testTxnLable6 = "testTxnLable6"; + public static String testTxnLable7 = "testTxnLable7"; + public static String testTxnLable8 = "testTxnLable8"; + public static String testTxnLable9 = "testTxnLable9"; + public static String testTxnLable10 = "testTxnLable10"; + public static String testTxnLable11 = "testTxnLable11"; + public static String testTxnLable12 = "testTxnLable12"; + public static String testPartitionedEsTable1 = "partitionedEsTable1"; + public static long testPartitionedEsTableId1 = 14; + public static String testUnPartitionedEsTable1 = "unpartitionedEsTable1"; + public static long testUnPartitionedEsTableId1 = 15; + + public static Catalog createTestCatalog() throws InstantiationException, IllegalAccessException, + IllegalArgumentException, InvocationTargetException, NoSuchMethodException, SecurityException { + Constructor constructor = Catalog.class.getDeclaredConstructor(); + constructor.setAccessible(true); + Catalog catalog = constructor.newInstance(); + catalog.setEditLog(new EditLog("name")); + FakeCatalog.setCatalog(catalog); + Backend backend1 = createBackend(testBackendId1, "host1", 123, 124, 125); + Backend backend2 = createBackend(testBackendId2, "host1", 123, 124, 125); + Backend backend3 = createBackend(testBackendId3, "host1", 123, 124, 125); + catalog.getCurrentSystemInfo().addBackend(backend1); + catalog.getCurrentSystemInfo().addBackend(backend2); + catalog.getCurrentSystemInfo().addBackend(backend3); + catalog.initDefaultCluster(); + Database db = createSimpleDb(testDbId1, testTableId1, testPartitionId1, testIndexId1, testTabletId1, + testStartVersion, testStartVersionHash); + catalog.unprotectCreateDb(db); + return catalog; + } + + public static boolean compareCatalog(Catalog masterCatalog, Catalog slaveCatalog) { + Database masterDb = masterCatalog.getDb(testDb1); + Database slaveDb = slaveCatalog.getDb(testDb1); + List
tables = masterDb.getTables(); + for (Table table : tables) { + Table slaveTable = slaveDb.getTable(table.getId()); + if (slaveTable == null) { + return false; + } + Partition masterPartition = table.getPartition(testPartition1); + Partition slavePartition = slaveTable.getPartition(testPartition1); + if (masterPartition == null && slavePartition == null) { + return true; + } + if (masterPartition.getId() != slavePartition.getId()) { + return false; + } + if (masterPartition.getCommittedVersion() != slavePartition.getCommittedVersion() + || masterPartition.getCommittedVersionHash() != slavePartition.getCommittedVersionHash() + || masterPartition.getNextVersion() != slavePartition.getNextVersion() + || masterPartition.getCurrentVersionHash() != slavePartition.getCurrentVersionHash()) { + return false; + } + List allMaterializedIndices = masterPartition.getMaterializedIndices(); + for (MaterializedIndex masterIndex : allMaterializedIndices) { + MaterializedIndex slaveIndex = slavePartition.getIndex(masterIndex.getId()); + if (slaveIndex == null) { + return false; + } + List allTablets = masterIndex.getTablets(); + for (Tablet masterTablet : allTablets) { + Tablet slaveTablet = slaveIndex.getTablet(masterTablet.getId()); + if (slaveTablet == null) { + return false; + } + List allReplicas = masterTablet.getReplicas(); + for (Replica masterReplica : allReplicas) { + Replica slaveReplica = slaveTablet.getReplicaById(masterReplica.getId()); + if (slaveReplica.getBackendId() != masterReplica.getBackendId() + || slaveReplica.getVersion() != masterReplica.getVersion() + || slaveReplica.getVersionHash() != masterReplica.getVersionHash() + || slaveReplica.getLastFailedVersion() != masterReplica.getLastFailedVersion() + || slaveReplica.getLastFailedVersionHash() != masterReplica.getLastFailedVersionHash() + || slaveReplica.getLastSuccessVersion() != slaveReplica.getLastSuccessVersion() + || slaveReplica.getLastSuccessVersionHash() != slaveReplica + .getLastSuccessVersionHash()) { + return false; + } + } + } + } + } + return true; + } + + public static Database createSimpleDb(long dbId, long tableId, long partitionId, long indexId, long tabletId, + long version, long versionHash) { + Catalog.getCurrentInvertedIndex().clear(); + + // replica + long replicaId = 0; + Replica replica1 = new Replica(testReplicaId1, testBackendId1, version, versionHash, 0L, 0L, + ReplicaState.NORMAL, -1, 0, 0, 0); + Replica replica2 = new Replica(testReplicaId2, testBackendId2, version, versionHash, 0L, 0L, + ReplicaState.NORMAL, -1, 0, 0, 0); + Replica replica3 = new Replica(testReplicaId3, testBackendId3, version, versionHash, 0L, 0L, + ReplicaState.NORMAL, -1, 0, 0, 0); + + // tablet + Tablet tablet = new Tablet(tabletId); + + // index + MaterializedIndex index = new MaterializedIndex(indexId, IndexState.NORMAL); + TabletMeta tabletMeta = new TabletMeta(dbId, tableId, partitionId, indexId, 0); + index.addTablet(tablet, tabletMeta); + + tablet.addReplica(replica1); + tablet.addReplica(replica2); + tablet.addReplica(replica3); + + // partition + RandomDistributionInfo distributionInfo = new RandomDistributionInfo(10); + Partition partition = new Partition(partitionId, testPartition1, index, distributionInfo); + partition.updateCommitVersionAndVersionHash(testStartVersion, testStartVersionHash); + partition.setNextVersion(testStartVersion + 1); + partition.setNextVersionHash(testPartitionNextVersionHash, testPartitionCurrentVersionHash); + + // columns + List columns = new ArrayList(); + Column temp = new Column("k1", PrimitiveType.INT); + temp.setIsKey(true); + columns.add(temp); + temp = new Column("k2", PrimitiveType.INT); + temp.setIsKey(true); + columns.add(temp); + columns.add(new Column("v", new ColumnType(PrimitiveType.DOUBLE), false, AggregateType.SUM, "0", "")); + + List keysColumn = new ArrayList(); + temp = new Column("k1", PrimitiveType.INT); + temp.setIsKey(true); + keysColumn.add(temp); + temp = new Column("k2", PrimitiveType.INT); + temp.setIsKey(true); + keysColumn.add(temp); + + // table + PartitionInfo partitionInfo = new SinglePartitionInfo(); + partitionInfo.setDataProperty(partitionId, DataProperty.DEFAULT_HDD_DATA_PROPERTY); + partitionInfo.setReplicationNum(partitionId, (short) 3); + OlapTable table = new OlapTable(tableId, testTable1, columns, KeysType.AGG_KEYS, partitionInfo, + distributionInfo); + table.addPartition(partition); + table.setIndexSchemaInfo(indexId, testIndex1, columns, 0, testSchemaHash1, (short) 1); + // db + Database db = new Database(dbId, testDb1); + db.createTable(table); + db.setClusterName(SystemInfoService.DEFAULT_CLUSTER); + + // add a es table to catalog + try { + createPartitionedEsTable(db); + createUnPartitionedEsTable(db); + } catch (DdlException e) { + // TODO Auto-generated catch block + // e.printStackTrace(); + } + return db; + } + + public static void createPartitionedEsTable(Database db) throws DdlException { + // columns + List columns = new ArrayList(); + Column k1 = new Column("k1", PrimitiveType.DATE); + k1.setIsKey(true); + columns.add(k1); + Column k2 = new Column("k2", PrimitiveType.INT); + k2.setIsKey(true); + columns.add(k2); + columns.add(new Column("v", new ColumnType(PrimitiveType.DOUBLE), false, AggregateType.SUM, "0", "")); + + // table + List partitionColumns = Lists.newArrayList(); + List singleRangePartitionDescs = Lists.newArrayList(); + partitionColumns.add(k1); + + singleRangePartitionDescs.add(new SingleRangePartitionDesc(false, "p1", + new PartitionKeyDesc(Lists + .newArrayList("100")), + null)); + + RangePartitionInfo partitionInfo = new RangePartitionInfo(partitionColumns); + Map properties = Maps.newHashMap(); + properties.put(EsTable.HOSTS, "xxx"); + properties.put(EsTable.INDEX, "indexa"); + properties.put(EsTable.PASSWORD, ""); + properties.put(EsTable.USER, "root"); + EsTable esTable = new EsTable(testPartitionedEsTableId1, testPartitionedEsTable1, + columns, properties, partitionInfo); + db.createTable(esTable); + } + + public static void createUnPartitionedEsTable(Database db) throws DdlException { + // columns + List columns = new ArrayList(); + Column k1 = new Column("k1", PrimitiveType.DATE); + k1.setIsKey(true); + columns.add(k1); + Column k2 = new Column("k2", PrimitiveType.INT); + k2.setIsKey(true); + columns.add(k2); + columns.add(new Column("v", new ColumnType(PrimitiveType.DOUBLE), false, AggregateType.SUM, "0", "")); + + // table + List partitionColumns = Lists.newArrayList(); + List singleRangePartitionDescs = Lists.newArrayList(); + partitionColumns.add(k1); + + singleRangePartitionDescs.add(new SingleRangePartitionDesc(false, "p1", + new PartitionKeyDesc(Lists + .newArrayList("100")), + null)); + + RangePartitionInfo partitionInfo = new RangePartitionInfo(partitionColumns); + Map properties = Maps.newHashMap(); + properties.put(EsTable.HOSTS, "xxx"); + properties.put(EsTable.INDEX, "indexa"); + properties.put(EsTable.PASSWORD, ""); + properties.put(EsTable.USER, "root"); + EsTable esTable = new EsTable(testUnPartitionedEsTableId1, testUnPartitionedEsTable1, + columns, properties, partitionInfo); + db.createTable(esTable); + } + + public static Backend createBackend(long id, String host, int heartPort, int bePort, int httpPort) { + Backend backend = new Backend(id, host, heartPort); + // backend.updateOnce(bePort, httpPort, 10000); + backend.setAlive(true); + return backend; + } + + public static Backend createBackend(long id, String host, int heartPort, int bePort, int httpPort, + long totalCapacityB, long availableCapacityB) { + Backend backend = createBackend(id, host, heartPort, bePort, httpPort); + Map backendDisks = new HashMap(); + String rootPath = "root_path"; + TDisk disk = new TDisk(rootPath, totalCapacityB, availableCapacityB, true); + backendDisks.put(rootPath, disk); + backend.updateDisks(backendDisks); + backend.setAlive(true); + return backend; + } +} diff --git a/fe/src/test/java/com/baidu/palo/catalog/FakeCatalog.java b/fe/src/test/java/com/baidu/palo/catalog/FakeCatalog.java new file mode 100644 index 0000000000..df718e9a38 --- /dev/null +++ b/fe/src/test/java/com/baidu/palo/catalog/FakeCatalog.java @@ -0,0 +1,31 @@ +package com.baidu.palo.catalog; + +import com.baidu.palo.common.FeMetaVersion; + +import mockit.Mock; +import mockit.MockUp; + +public class FakeCatalog extends MockUp { + + private static Catalog catalog; + + public static void setCatalog(Catalog catalog) { + FakeCatalog.catalog = catalog; + } + + @Mock + public int getJournalVersion() { + return FeMetaVersion.VERSION_45; + } + + @Mock + private static Catalog getCurrentCatalog() { + System.out.println("fake get current catalog is called"); + return catalog; + } + + @Mock + public static Catalog getInstance() { + return catalog; + } +} \ No newline at end of file diff --git a/fe/src/test/java/com/baidu/palo/catalog/FakeEditLog.java b/fe/src/test/java/com/baidu/palo/catalog/FakeEditLog.java new file mode 100644 index 0000000000..a7723eb7d4 --- /dev/null +++ b/fe/src/test/java/com/baidu/palo/catalog/FakeEditLog.java @@ -0,0 +1,74 @@ +package com.baidu.palo.catalog; + +import com.baidu.palo.alter.RollupJob; +import com.baidu.palo.alter.SchemaChangeJob; +import com.baidu.palo.cluster.Cluster; +import com.baidu.palo.persist.EditLog; +import com.baidu.palo.transaction.TransactionState; + +import java.util.HashMap; +import java.util.Map; + +import mockit.Mock; +import mockit.MockUp; + +public class FakeEditLog extends MockUp { + + private Map allTransactionState = new HashMap<>(); + + @Mock + public void $init(String nodeName) { + // do nothing + System.out.println("abc"); + } + + @Mock + public void logInsertTransactionState(TransactionState transactionState) { + // do nothing + System.out.println("insert transaction manager is called"); + allTransactionState.put(transactionState.getTransactionId(), transactionState); + } + + @Mock + public void logDeleteTransactionState(TransactionState transactionState) { + // do nothing + System.out.println("delete transaction state is deleted"); + allTransactionState.remove(transactionState.getTransactionId()); + } + + @Mock + public void logSaveNextId(long nextId) { + // do nothing + } + + @Mock + public void logCreateCluster(Cluster cluster) { + // do nothing + } + + @Mock + public void logStartRollup(RollupJob rollupJob) { + + } + + @Mock + public void logFinishingRollup(RollupJob rollupJob) { + + } + + @Mock + public void logCancelRollup(RollupJob rollupJob) { + } + + @Mock + public void logStartSchemaChange(SchemaChangeJob schemaChangeJob) { + } + + @Mock + public void logFinishingSchemaChange(SchemaChangeJob schemaChangeJob) { + } + + public TransactionState getTransaction(long transactionId) { + return allTransactionState.get(transactionId); + } +} diff --git a/fe/src/test/java/com/baidu/palo/catalog/MetadataViewerTest.java b/fe/src/test/java/com/baidu/palo/catalog/MetadataViewerTest.java new file mode 100644 index 0000000000..443b995840 --- /dev/null +++ b/fe/src/test/java/com/baidu/palo/catalog/MetadataViewerTest.java @@ -0,0 +1,107 @@ +package com.baidu.palo.catalog; + +import com.baidu.palo.analysis.BinaryPredicate.Operator; +import com.baidu.palo.backup.CatalogMocker; +import com.baidu.palo.catalog.Replica.ReplicaStatus; +import com.baidu.palo.common.AnalysisException; +import com.baidu.palo.system.SystemInfoService; + +import com.google.common.collect.Lists; + +import org.junit.Assert; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; + +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.util.List; + +import mockit.Mocked; +import mockit.NonStrictExpectations; + +public class MetadataViewerTest { + + private static Method getTabletStatusMethod; + private static Method getTabletDistributionMethod; + + @Mocked + private Catalog catalog; + + @Mocked + private SystemInfoService infoService; + + private static Database db; + + @BeforeClass + public static void setUp() throws NoSuchMethodException, SecurityException, InstantiationException, + IllegalAccessException, IllegalArgumentException, InvocationTargetException, AnalysisException { + Class[] argTypes = new Class[] { String.class, String.class, List.class, ReplicaStatus.class, Operator.class }; + getTabletStatusMethod = MetadataViewer.class.getDeclaredMethod("getTabletStatus", argTypes); + getTabletStatusMethod.setAccessible(true); + + argTypes = new Class[] { String.class, String.class, List.class }; + getTabletDistributionMethod = MetadataViewer.class.getDeclaredMethod("getTabletDistribution", argTypes); + getTabletDistributionMethod.setAccessible(true); + + db = CatalogMocker.mockDb(); + } + + @Before + public void before() { + + new NonStrictExpectations() { + { + Catalog.getCurrentCatalog(); + minTimes = 0; + result = catalog; + + catalog.getDb(anyString); + result = db; + } + }; + + new NonStrictExpectations() { + { + Catalog.getCurrentSystemInfo(); + minTimes = 0; + result = infoService; + + infoService.getBackendIds(anyBoolean); + result = Lists.newArrayList(10000L, 10001L, 10002L); + } + }; + } + + @Test + public void testGetTabletStatus() + throws IllegalAccessException, IllegalArgumentException, InvocationTargetException { + List partitions = Lists.newArrayList(); + Object[] args = new Object[] { CatalogMocker.TEST_DB_NAME, CatalogMocker.TEST_TBL_NAME, partitions, null, + null }; + List> result = (List>) getTabletStatusMethod.invoke(null, args); + Assert.assertEquals(3, result.size()); + System.out.println(result); + + args = new Object[] { CatalogMocker.TEST_DB_NAME, CatalogMocker.TEST_TBL_NAME, partitions, ReplicaStatus.DEAD, + Operator.EQ }; + result = (List>) getTabletStatusMethod.invoke(null, args); + Assert.assertEquals(3, result.size()); + + args = new Object[] { CatalogMocker.TEST_DB_NAME, CatalogMocker.TEST_TBL_NAME, partitions, ReplicaStatus.DEAD, + Operator.NE }; + result = (List>) getTabletStatusMethod.invoke(null, args); + Assert.assertEquals(0, result.size()); + } + + @Test + public void testGetTabletDistribution() + throws IllegalAccessException, IllegalArgumentException, InvocationTargetException { + List partitions = Lists.newArrayList(); + Object[] args = new Object[] { CatalogMocker.TEST_DB_NAME, CatalogMocker.TEST_TBL_NAME, partitions }; + List> result = (List>) getTabletDistributionMethod.invoke(null, args); + Assert.assertEquals(3, result.size()); + System.out.println(result); + } + +} diff --git a/fe/src/test/java/com/baidu/palo/es/EsStateStoreTest.java b/fe/src/test/java/com/baidu/palo/es/EsStateStoreTest.java new file mode 100644 index 0000000000..abbb9a28f4 --- /dev/null +++ b/fe/src/test/java/com/baidu/palo/es/EsStateStoreTest.java @@ -0,0 +1,206 @@ +// Copyright (c) 2017, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 com.baidu.palo.es; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + +import java.io.BufferedReader; +import java.io.File; +import java.io.FileInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.lang.reflect.InvocationTargetException; +import java.net.URISyntaxException; +import java.util.Map; + +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; + +import com.baidu.palo.catalog.Catalog; +import com.baidu.palo.catalog.CatalogTestUtil; +import com.baidu.palo.catalog.EsTable; +import com.baidu.palo.catalog.FakeCatalog; +import com.baidu.palo.catalog.FakeEditLog; +import com.baidu.palo.catalog.PartitionKey; +import com.baidu.palo.catalog.RangePartitionInfo; +import com.baidu.palo.common.AnalysisException; +import com.baidu.palo.common.FeMetaVersion; +import com.baidu.palo.external.EsIndexState; +import com.baidu.palo.external.EsStateStore; +import com.baidu.palo.external.EsTableState; +import com.google.common.collect.Lists; +import com.google.common.collect.Range; + +public class EsStateStoreTest { + + private static FakeEditLog fakeEditLog; + private static FakeCatalog fakeCatalog; + private static Catalog masterCatalog; + private static String clusterStateStr1 = ""; + private static String clusterStateStr2 = ""; + private static String clusterStateStr3 = ""; + private EsStateStore esStateStore; + + @BeforeClass + public static void init() throws IOException, InstantiationException, IllegalAccessException, + IllegalArgumentException, InvocationTargetException, NoSuchMethodException, SecurityException, + URISyntaxException { + fakeEditLog = new FakeEditLog(); + fakeCatalog = new FakeCatalog(); + masterCatalog = CatalogTestUtil.createTestCatalog(); + masterCatalog.setJournalVersion(FeMetaVersion.VERSION_40); + FakeCatalog.setCatalog(masterCatalog); + clusterStateStr1 = loadJsonFromFile("data/es/clusterstate1.json"); + clusterStateStr2 = loadJsonFromFile("data/es/clusterstate2.json"); + clusterStateStr3 = loadJsonFromFile("data/es/clusterstate3.json"); + } + + @Before + public void setUp() { + esStateStore = new EsStateStore(); + } + + /** + * partitioned es table schema: k1(date), k2(int), v(double) + * @throws AnalysisException + */ + @Test + public void testParsePartitionedClusterState() throws AnalysisException { + EsTable esTable = (EsTable) Catalog.getCurrentCatalog() + .getDb(CatalogTestUtil.testDb1) + .getTable(CatalogTestUtil.testPartitionedEsTable1); + boolean hasException = false; + EsTableState esTableState = null; + try { + esTableState = esStateStore.parseClusterState55(clusterStateStr1, esTable); + } catch (Exception e) { + e.printStackTrace(); + hasException = true; + } + assertFalse(hasException); + assertNotNull(esTableState); + assertEquals(2, esTableState.getPartitionedIndexStates().size()); + RangePartitionInfo definedPartInfo = (RangePartitionInfo) esTable.getPartitionInfo(); + RangePartitionInfo rangePartitionInfo = (RangePartitionInfo) esTableState.getPartitionInfo(); + Map> rangeMap = rangePartitionInfo.getIdToRange(); + assertEquals(2, rangeMap.size()); + Range part0 = rangeMap.get(new Long(0)); + EsIndexState esIndexState1 = esTableState.getIndexState(0); + assertEquals(5, esIndexState1.getShardRoutings().size()); + assertEquals("index1", esIndexState1.getIndexName()); + PartitionKey lowKey = PartitionKey.createInfinityPartitionKey(definedPartInfo.getPartitionColumns(), false); + PartitionKey upperKey = PartitionKey.createPartitionKey(Lists.newArrayList("2018-10-01"), + definedPartInfo.getPartitionColumns()); + Range newRange = Range.closedOpen(lowKey, upperKey); + assertEquals(newRange, part0); + Range part1 = rangeMap.get(new Long(1)); + EsIndexState esIndexState2 = esTableState.getIndexState(1); + assertEquals("index2", esIndexState2.getIndexName()); + lowKey = PartitionKey.createPartitionKey(Lists.newArrayList("2018-10-01"), + definedPartInfo.getPartitionColumns()); + upperKey = PartitionKey.createPartitionKey(Lists.newArrayList("2018-10-02"), + definedPartInfo.getPartitionColumns()); + newRange = Range.closedOpen(lowKey, upperKey); + assertEquals(newRange, part1); + assertEquals(6, esIndexState2.getShardRoutings().size()); + } + + /** + * partitioned es table schema: k1(date), k2(int), v(double) + * scenario desc: + * 2 indices, one with partition desc, the other does not contains partition desc + * @throws AnalysisException + */ + @Test + public void testParsePartitionedClusterStateTwoIndices() throws AnalysisException { + EsTable esTable = (EsTable) Catalog.getCurrentCatalog() + .getDb(CatalogTestUtil.testDb1) + .getTable(CatalogTestUtil.testPartitionedEsTable1); + boolean hasException = false; + EsTableState esTableState = null; + try { + esTableState = esStateStore.parseClusterState55(clusterStateStr3, esTable); + } catch (Exception e) { + e.printStackTrace(); + hasException = true; + } + assertFalse(hasException); + assertNotNull(esTableState); + + // check + assertEquals(1, esTableState.getPartitionedIndexStates().size()); + assertEquals(1, esTableState.getUnPartitionedIndexStates().size()); + + // check partition info + RangePartitionInfo definedPartInfo = (RangePartitionInfo) esTable.getPartitionInfo(); + RangePartitionInfo rangePartitionInfo = (RangePartitionInfo) esTableState.getPartitionInfo(); + Map> rangeMap = rangePartitionInfo.getIdToRange(); + assertEquals(1, rangeMap.size()); + Range part0 = rangeMap.get(new Long(0)); + EsIndexState esIndexState1 = esTableState.getIndexState(0); + assertEquals(5, esIndexState1.getShardRoutings().size()); + assertEquals("index1", esIndexState1.getIndexName()); + PartitionKey lowKey = PartitionKey.createInfinityPartitionKey(definedPartInfo.getPartitionColumns(), false); + PartitionKey upperKey = PartitionKey.createPartitionKey(Lists.newArrayList("2018-10-01"), + definedPartInfo.getPartitionColumns()); + Range newRange = Range.closedOpen(lowKey, upperKey); + assertEquals(newRange, part0); + + // check index with no partition desc + EsIndexState esIndexState2 = esTableState.getUnPartitionedIndexStates().get("index2"); + assertEquals("index2", esIndexState2.getIndexName()); + assertEquals(6, esIndexState2.getShardRoutings().size()); + } + + /** + * partitioned es table schema: k1(date), k2(int), v(double) + * "upperbound": "2018" is not a valid date value, so parsing procedure will fail + */ + @Test + public void testParseInvalidUpperbound() { + EsTable esTable = (EsTable) Catalog.getCurrentCatalog() + .getDb(CatalogTestUtil.testDb1) + .getTable(CatalogTestUtil.testPartitionedEsTable1); + boolean hasException = false; + EsTableState esTableState = null; + try { + esTableState = esStateStore.parseClusterState55(clusterStateStr2, esTable); + } catch (Exception e) { + hasException = true; + } + assertTrue(hasException); + assertTrue(esTableState == null); + } + + private static String loadJsonFromFile(String fileName) throws IOException, URISyntaxException { + File file = new File(EsStateStoreTest.class.getClassLoader().getResource(fileName).toURI()); + InputStream is = new FileInputStream(file); + BufferedReader br = new BufferedReader(new InputStreamReader(is)); + StringBuilder jsonStr = new StringBuilder(); + String line = ""; + while ((line = br.readLine()) != null) { + jsonStr.append(line); + } + br.close(); + is.close(); + return jsonStr.toString(); + } +} diff --git a/fe/src/test/java/com/baidu/palo/es/EsUtilTest.java b/fe/src/test/java/com/baidu/palo/es/EsUtilTest.java new file mode 100644 index 0000000000..1703904b9e --- /dev/null +++ b/fe/src/test/java/com/baidu/palo/es/EsUtilTest.java @@ -0,0 +1,69 @@ +// Copyright (c) 2017, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 com.baidu.palo.es; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; + +import org.json.JSONException; +import org.json.JSONObject; +import org.junit.Test; + +import com.baidu.palo.external.EsUtil; + +public class EsUtilTest { + + private String jsonStr = "{\"settings\": {\n" + + " \"index\": {\n" + + " \"bpack\": {\n" + + " \"partition\": {\n" + + " \"upperbound\": \"12\"\n" + + " }\n" + + " },\n" + + " \"number_of_shards\": \"5\",\n" + + " \"provided_name\": \"indexa\",\n" + + " \"creation_date\": \"1539328532060\",\n" + + " \"number_of_replicas\": \"1\",\n" + + " \"uuid\": \"plNNtKiiQ9-n6NpNskFzhQ\",\n" + + " \"version\": {\n" + + " \"created\": \"5050099\"\n" + + " }\n" + + " }\n" + + " }}"; + + @Test + public void testGetJsonObject() { + JSONObject json = new JSONObject(jsonStr); + JSONObject upperBoundSetting = EsUtil.getJsonObject(json, "settings.index.bpack.partition", 0); + assertTrue(upperBoundSetting.has("upperbound")); + assertEquals("12", upperBoundSetting.getString("upperbound")); + + JSONObject unExistKey = EsUtil.getJsonObject(json, "set", 0); + assertNull(unExistKey); + + JSONObject singleKey = EsUtil.getJsonObject(json, "settings", 0); + assertTrue(singleKey.has("index")); + } + + @Test(expected = JSONException.class) + public void testGetJsonObjectWithException() { + JSONObject json = new JSONObject(jsonStr); + // only support json object could not get string value directly from this api, exception will be threw + EsUtil.getJsonObject(json, "settings.index.bpack.partition.upperbound", 0); + } + +} diff --git a/fe/src/test/java/com/baidu/palo/planner/OlapTableSinkTest.java b/fe/src/test/java/com/baidu/palo/planner/OlapTableSinkTest.java new file mode 100644 index 0000000000..270cb21e13 --- /dev/null +++ b/fe/src/test/java/com/baidu/palo/planner/OlapTableSinkTest.java @@ -0,0 +1,174 @@ +// Copyright (c) 2017, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 com.baidu.palo.planner; + +import com.baidu.palo.analysis.DescriptorTable; +import com.baidu.palo.analysis.SlotDescriptor; +import com.baidu.palo.analysis.TupleDescriptor; +import com.baidu.palo.catalog.Column; +import com.baidu.palo.catalog.ColumnType; +import com.baidu.palo.catalog.HashDistributionInfo; +import com.baidu.palo.catalog.MaterializedIndex; +import com.baidu.palo.catalog.OlapTable; +import com.baidu.palo.catalog.Partition; +import com.baidu.palo.catalog.PartitionKey; +import com.baidu.palo.catalog.PartitionType; +import com.baidu.palo.catalog.PrimitiveType; +import com.baidu.palo.catalog.RangePartitionInfo; +import com.baidu.palo.catalog.SinglePartitionInfo; +import com.baidu.palo.catalog.Tablet; +import com.baidu.palo.common.UserException; +import com.baidu.palo.system.Backend; +import com.baidu.palo.system.SystemInfoService; +import com.baidu.palo.thrift.TExplainLevel; +import com.baidu.palo.thrift.TUniqueId; + +import com.google.common.collect.Lists; +import com.google.common.collect.Range; +import mockit.Expectations; +import mockit.Injectable; +import mockit.Mocked; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.junit.Test; + +public class OlapTableSinkTest { + private static final Logger LOG = LogManager.getLogger(OlapTableSinkTest.class); + + @Injectable + OlapTable dstTable; + + @Mocked + SystemInfoService systemInfoService; + + private TupleDescriptor getTuple() { + DescriptorTable descTable = new DescriptorTable(); + TupleDescriptor tuple = descTable.createTupleDescriptor("DstTable"); + // k1 + SlotDescriptor k1 = descTable.addSlotDescriptor(tuple); + k1.setColumn(new Column("k1", PrimitiveType.BIGINT)); + k1.setIsMaterialized(true); + + // k2 + SlotDescriptor k2 = descTable.addSlotDescriptor(tuple); + k2.setColumn(new Column("k2", new ColumnType(PrimitiveType.VARCHAR, 25, 12, 1))); + k2.setIsMaterialized(true); + // v1 + SlotDescriptor v1 = descTable.addSlotDescriptor(tuple); + v1.setColumn(new Column("v1", new ColumnType(PrimitiveType.VARCHAR, 25, 12, 1))); + v1.setIsMaterialized(true); + // v2 + SlotDescriptor v2 = descTable.addSlotDescriptor(tuple); + v2.setColumn(new Column("v2", PrimitiveType.BIGINT)); + v2.setIsMaterialized(true); + + return tuple; + } + + @Test + public void testSinglePartition() throws UserException { + TupleDescriptor tuple = getTuple(); + SinglePartitionInfo partInfo = new SinglePartitionInfo(); + partInfo.setReplicationNum(2, (short) 3); + MaterializedIndex index = new MaterializedIndex(2, MaterializedIndex.IndexState.NORMAL); + HashDistributionInfo distInfo = new HashDistributionInfo( + 2, Lists.newArrayList(new Column("k1", PrimitiveType.BIGINT))); + Partition partition = new Partition(2, "p1", index, distInfo); + + new Expectations() {{ + dstTable.getId(); result = 1; + dstTable.getPartitionInfo(); result = partInfo; + dstTable.getPartitions(); result = Lists.newArrayList(partition); + }}; + + OlapTableSink sink = new OlapTableSink(dstTable, tuple); + sink.init(new TUniqueId(1, 2), 3, 4); + sink.finalize(); + LOG.info("sink is {}", sink.toThrift()); + LOG.info("{}", sink.getExplainString("", TExplainLevel.NORMAL)); + } + + @Test + public void testRangePartition( + @Injectable RangePartitionInfo partInfo, + @Injectable MaterializedIndex index) throws UserException { + TupleDescriptor tuple = getTuple(); + + HashDistributionInfo distInfo = new HashDistributionInfo( + 2, Lists.newArrayList(new Column("k1", PrimitiveType.BIGINT))); + + Column partKey = new Column("k2", PrimitiveType.VARCHAR); + PartitionKey key = PartitionKey.createPartitionKey(Lists.newArrayList("123"), Lists.newArrayList(partKey)); + Partition p1 = new Partition(1, "p1", index, distInfo); + Partition p2 = new Partition(2, "p2", index, distInfo); + + new Expectations() {{ + dstTable.getId(); result = 1; + dstTable.getPartitionInfo(); result = partInfo; + partInfo.getType(); result = PartitionType.RANGE; + partInfo.getPartitionColumns(); result = Lists.newArrayList(partKey); + partInfo.getRange(1); result = Range.lessThan(key); + // partInfo.getRange(2); result = Range.atLeast(key); + dstTable.getPartitions(); result = Lists.newArrayList(p1, p2); + dstTable.getPartition("p1"); result = p1; + + index.getTablets(); result = Lists.newArrayList(new Tablet(1)); + systemInfoService.getBackendIds(anyBoolean); result = Lists.newArrayList(new Long(1)); + systemInfoService.getBackend(new Long(1)); result = new Backend(1, "abc", 1234); + }}; + + OlapTableSink sink = new OlapTableSink(dstTable, tuple, "p1"); + sink.init(new TUniqueId(1, 2), 3, 4); + sink.finalize(); + LOG.info("sink is {}", sink.toThrift()); + LOG.info("{}", sink.getExplainString("", TExplainLevel.NORMAL)); + } + + @Test(expected = UserException.class) + public void testRangeUnknownPartition( + @Injectable RangePartitionInfo partInfo, + @Injectable MaterializedIndex index) throws UserException { + TupleDescriptor tuple = getTuple(); + + new Expectations() {{ + partInfo.getType(); result = PartitionType.RANGE; + dstTable.getPartition("p3"); result = null; + }}; + + OlapTableSink sink = new OlapTableSink(dstTable, tuple, "p3"); + sink.init(new TUniqueId(1, 2), 3, 4); + sink.finalize(); + LOG.info("sink is {}", sink.toThrift()); + LOG.info("{}", sink.getExplainString("", TExplainLevel.NORMAL)); + } + + @Test(expected = UserException.class) + public void testUnpartFail( + @Injectable RangePartitionInfo partInfo, + @Injectable MaterializedIndex index) throws UserException { + TupleDescriptor tuple = getTuple(); + + new Expectations() {{ + partInfo.getType(); result = PartitionType.UNPARTITIONED; + }}; + + OlapTableSink sink = new OlapTableSink(dstTable, tuple, "p1"); + sink.init(new TUniqueId(1, 2), 3, 4); + sink.finalize(); + LOG.info("sink is {}", sink.toThrift()); + LOG.info("{}", sink.getExplainString("", TExplainLevel.NORMAL)); + } +} diff --git a/fe/src/test/java/com/baidu/palo/planner/StreamLoadPlannerTest.java b/fe/src/test/java/com/baidu/palo/planner/StreamLoadPlannerTest.java new file mode 100644 index 0000000000..5d96dbd74e --- /dev/null +++ b/fe/src/test/java/com/baidu/palo/planner/StreamLoadPlannerTest.java @@ -0,0 +1,74 @@ +// Copyright (c) 2017, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 com.baidu.palo.planner; + +import com.baidu.palo.analysis.Analyzer; +import com.baidu.palo.catalog.Column; +import com.baidu.palo.catalog.Database; +import com.baidu.palo.catalog.OlapTable; +import com.baidu.palo.catalog.PrimitiveType; +import com.baidu.palo.common.UserException; +import com.baidu.palo.thrift.TStreamLoadPutRequest; +import com.baidu.palo.thrift.TUniqueId; + +import com.google.common.collect.Lists; + +import org.junit.Test; + +import java.util.List; + +import mockit.Injectable; +import mockit.Mocked; +import mockit.NonStrictExpectations; + +public class StreamLoadPlannerTest { + @Injectable + Database db; + + @Injectable + OlapTable destTable; + + @Mocked + StreamLoadScanNode scanNode; + + @Mocked + OlapTableSink sink; + + @Test + public void testNormalPlan() throws UserException { + List columns = Lists.newArrayList(); + Column c1 = new Column("c1", PrimitiveType.BIGINT, false); + columns.add(c1); + Column c2 = new Column("c2", PrimitiveType.BIGINT, true); + columns.add(c2); + new NonStrictExpectations() { + { + destTable.getBaseSchema(); + result = columns; + scanNode.init((Analyzer) any); + scanNode.getChildren(); + result = Lists.newArrayList(); + scanNode.getId(); + result = new PlanNodeId(5); + } + }; + TStreamLoadPutRequest request = new TStreamLoadPutRequest(); + request.setTxnId(1); + request.setLoadId(new TUniqueId(2, 3)); + StreamLoadPlanner planner = new StreamLoadPlanner(db, destTable, request); + planner.plan(); + } +} \ No newline at end of file diff --git a/fe/src/test/java/com/baidu/palo/planner/StreamLoadScanNodeTest.java b/fe/src/test/java/com/baidu/palo/planner/StreamLoadScanNodeTest.java new file mode 100644 index 0000000000..18cf02ae7e --- /dev/null +++ b/fe/src/test/java/com/baidu/palo/planner/StreamLoadScanNodeTest.java @@ -0,0 +1,522 @@ +// Copyright (c) 2017, Baidu.com, Inc. All Rights Reserved + +// Licensed 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 com.baidu.palo.planner; + +import com.baidu.palo.analysis.Analyzer; +import com.baidu.palo.analysis.CastExpr; +import com.baidu.palo.analysis.DescriptorTable; +import com.baidu.palo.analysis.FunctionName; +import com.baidu.palo.analysis.SlotDescriptor; +import com.baidu.palo.analysis.TupleDescriptor; +import com.baidu.palo.catalog.AggregateType; +import com.baidu.palo.catalog.Catalog; +import com.baidu.palo.catalog.Column; +import com.baidu.palo.catalog.ColumnType; +import com.baidu.palo.catalog.Function; +import com.baidu.palo.catalog.OlapTable; +import com.baidu.palo.catalog.PrimitiveType; +import com.baidu.palo.catalog.ScalarFunction; +import com.baidu.palo.catalog.Type; +import com.baidu.palo.common.AnalysisException; +import com.baidu.palo.common.UserException; +import com.baidu.palo.qe.ConnectContext; +import com.baidu.palo.thrift.TExplainLevel; +import com.baidu.palo.thrift.TFileType; +import com.baidu.palo.thrift.TPlanNode; +import com.baidu.palo.thrift.TStreamLoadPutRequest; + +import com.google.common.collect.Lists; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.junit.Assert; +import org.junit.Test; + +import java.util.List; + +import mockit.Expectations; +import mockit.Injectable; +import mockit.Mocked; + +public class StreamLoadScanNodeTest { + private static final Logger LOG = LogManager.getLogger(StreamLoadScanNodeTest.class); + + @Mocked + Catalog catalog; + + @Injectable + ConnectContext connectContext; + + @Injectable + OlapTable dstTable; + + @Mocked + CastExpr castExpr; + + TStreamLoadPutRequest getBaseRequest() { + TStreamLoadPutRequest request = new TStreamLoadPutRequest(); + request.setFileType(TFileType.FILE_STREAM); + return request; + } + + List getBaseSchema() { + List columns = Lists.newArrayList(); + + Column k1 = new Column("k1", PrimitiveType.BIGINT); + k1.setIsKey(true); + k1.setIsAllowNull(false); + columns.add(k1); + + Column k2 = new Column("k2", new ColumnType(PrimitiveType.VARCHAR, 25, 10, 5)); + k2.setIsKey(true); + k2.setIsAllowNull(true); + columns.add(k2); + + Column v1 = new Column("v1", PrimitiveType.BIGINT); + v1.setIsKey(false); + v1.setIsAllowNull(true); + v1.setAggregationType(AggregateType.SUM, false); + + columns.add(v1); + + Column v2 = new Column("v2", new ColumnType(PrimitiveType.VARCHAR, 25, 10, 5)); + v2.setIsKey(false); + v2.setAggregationType(AggregateType.REPLACE, false); + v2.setIsAllowNull(false); + columns.add(v2); + + return columns; + } + + List getHllSchema() { + List columns = Lists.newArrayList(); + + Column k1 = new Column("k1", PrimitiveType.BIGINT); + k1.setIsKey(true); + k1.setIsAllowNull(false); + columns.add(k1); + + Column v1 = new Column("v1", PrimitiveType.HLL); + v1.setIsKey(false); + v1.setIsAllowNull(true); + v1.setAggregationType(AggregateType.HLL_UNION, false); + + columns.add(v1); + + return columns; + } + + @Test + public void testNormal() throws UserException { + Analyzer analyzer = new Analyzer(catalog, connectContext); + DescriptorTable descTbl = analyzer.getDescTbl(); + + List columns = getBaseSchema(); + TupleDescriptor dstDesc = descTbl.createTupleDescriptor("DstTableDesc"); + for (Column column : columns) { + SlotDescriptor slot = descTbl.addSlotDescriptor(dstDesc); + slot.setColumn(column); + slot.setIsMaterialized(true); + if (column.isAllowNull()) { + slot.setIsNullable(true); + } else { + slot.setIsNullable(false); + } + } + + TStreamLoadPutRequest request = getBaseRequest(); + StreamLoadScanNode scanNode = new StreamLoadScanNode(new PlanNodeId(1), dstDesc, dstTable, request); + new Expectations() {{ + dstTable.getBaseSchema(); result = columns; + castExpr.analyze((Analyzer) any); + }}; + scanNode.init(analyzer); + scanNode.finalize(analyzer); + scanNode.getNodeExplainString("", TExplainLevel.NORMAL); + TPlanNode planNode = new TPlanNode(); + scanNode.toThrift(planNode); + + Assert.assertEquals(1, scanNode.getNumInstances()); + Assert.assertEquals(1, scanNode.getScanRangeLocations(0).size()); + } + + @Test(expected = AnalysisException.class) + public void testLostV2() throws UserException { + Analyzer analyzer = new Analyzer(catalog, connectContext); + DescriptorTable descTbl = analyzer.getDescTbl(); + + List columns = getBaseSchema(); + TupleDescriptor dstDesc = descTbl.createTupleDescriptor("DstTableDesc"); + for (Column column : columns) { + SlotDescriptor slot = descTbl.addSlotDescriptor(dstDesc); + slot.setColumn(column); + slot.setIsMaterialized(true); + if (column.isAllowNull()) { + slot.setIsNullable(true); + } else { + slot.setIsNullable(false); + } + } + + TStreamLoadPutRequest request = getBaseRequest(); + request.setColumns("k1, k2, v1"); + StreamLoadScanNode scanNode = new StreamLoadScanNode(new PlanNodeId(1), dstDesc, dstTable, request); + + scanNode.init(analyzer); + scanNode.finalize(analyzer); + scanNode.getNodeExplainString("", TExplainLevel.NORMAL); + TPlanNode planNode = new TPlanNode(); + scanNode.toThrift(planNode); + } + + @Test(expected = AnalysisException.class) + public void testBadColumns() throws UserException, UserException { + Analyzer analyzer = new Analyzer(catalog, connectContext); + DescriptorTable descTbl = analyzer.getDescTbl(); + + List columns = getBaseSchema(); + TupleDescriptor dstDesc = descTbl.createTupleDescriptor("DstTableDesc"); + for (Column column : columns) { + SlotDescriptor slot = descTbl.addSlotDescriptor(dstDesc); + slot.setColumn(column); + slot.setIsMaterialized(true); + if (column.isAllowNull()) { + slot.setIsNullable(true); + } else { + slot.setIsNullable(false); + } + } + + TStreamLoadPutRequest request = getBaseRequest(); + request.setColumns("k1 k2 v1"); + StreamLoadScanNode scanNode = new StreamLoadScanNode(new PlanNodeId(1), dstDesc, dstTable, request); + + scanNode.init(analyzer); + scanNode.finalize(analyzer); + scanNode.getNodeExplainString("", TExplainLevel.NORMAL); + TPlanNode planNode = new TPlanNode(); + scanNode.toThrift(planNode); + } + + @Test + public void testColumnsNormal() throws UserException, UserException { + Analyzer analyzer = new Analyzer(catalog, connectContext); + DescriptorTable descTbl = analyzer.getDescTbl(); + + List columns = getBaseSchema(); + TupleDescriptor dstDesc = descTbl.createTupleDescriptor("DstTableDesc"); + for (Column column : columns) { + SlotDescriptor slot = descTbl.addSlotDescriptor(dstDesc); + slot.setColumn(column); + slot.setIsMaterialized(true); + if (column.isAllowNull()) { + slot.setIsNullable(true); + } else { + slot.setIsNullable(false); + } + } + + TStreamLoadPutRequest request = getBaseRequest(); + request.setFileType(TFileType.FILE_LOCAL); + request.setColumns("k1,k2,v1, v2=k2"); + StreamLoadScanNode scanNode = new StreamLoadScanNode(new PlanNodeId(1), dstDesc, dstTable, request); + + scanNode.init(analyzer); + scanNode.finalize(analyzer); + scanNode.getNodeExplainString("", TExplainLevel.NORMAL); + TPlanNode planNode = new TPlanNode(); + scanNode.toThrift(planNode); + } + + @Test + public void testHllColumnsNormal() throws UserException { + Analyzer analyzer = new Analyzer(catalog, connectContext); + DescriptorTable descTbl = analyzer.getDescTbl(); + + List columns = getHllSchema(); + TupleDescriptor dstDesc = descTbl.createTupleDescriptor("DstTableDesc"); + for (Column column : columns) { + SlotDescriptor slot = descTbl.addSlotDescriptor(dstDesc); + slot.setColumn(column); + slot.setIsMaterialized(true); + if (column.isAllowNull()) { + slot.setIsNullable(true); + } else { + slot.setIsNullable(false); + } + } + + new Expectations() {{ + catalog.getFunction((Function) any, (Function.CompareMode) any); + result = new ScalarFunction(new FunctionName("hll_hash"), Lists.newArrayList(), Type.BIGINT, false); + }}; + + TStreamLoadPutRequest request = getBaseRequest(); + request.setFileType(TFileType.FILE_LOCAL); + request.setColumns("k1,k2, v1=hll_hash(k2)"); + StreamLoadScanNode scanNode = new StreamLoadScanNode(new PlanNodeId(1), dstDesc, dstTable, request); + + scanNode.init(analyzer); + scanNode.finalize(analyzer); + scanNode.getNodeExplainString("", TExplainLevel.NORMAL); + TPlanNode planNode = new TPlanNode(); + scanNode.toThrift(planNode); + } + + @Test(expected = UserException.class) + public void testHllColumnsNoHllHash() throws UserException { + Analyzer analyzer = new Analyzer(catalog, connectContext); + DescriptorTable descTbl = analyzer.getDescTbl(); + + List columns = getHllSchema(); + TupleDescriptor dstDesc = descTbl.createTupleDescriptor("DstTableDesc"); + for (Column column : columns) { + SlotDescriptor slot = descTbl.addSlotDescriptor(dstDesc); + slot.setColumn(column); + slot.setIsMaterialized(true); + if (column.isAllowNull()) { + slot.setIsNullable(true); + } else { + slot.setIsNullable(false); + } + } + + new Expectations() {{ + catalog.getFunction((Function) any, (Function.CompareMode) any); + result = new ScalarFunction(new FunctionName("hll_hash1"), Lists.newArrayList(), Type.BIGINT, false); + }}; + + TStreamLoadPutRequest request = getBaseRequest(); + request.setFileType(TFileType.FILE_LOCAL); + request.setColumns("k1,k2, v1=hll_hash1(k2)"); + StreamLoadScanNode scanNode = new StreamLoadScanNode(new PlanNodeId(1), dstDesc, dstTable, request); + + scanNode.init(analyzer); + scanNode.finalize(analyzer); + scanNode.getNodeExplainString("", TExplainLevel.NORMAL); + TPlanNode planNode = new TPlanNode(); + scanNode.toThrift(planNode); + } + + @Test(expected = UserException.class) + public void testHllColumnsFail() throws UserException { + Analyzer analyzer = new Analyzer(catalog, connectContext); + DescriptorTable descTbl = analyzer.getDescTbl(); + + List columns = getHllSchema(); + TupleDescriptor dstDesc = descTbl.createTupleDescriptor("DstTableDesc"); + for (Column column : columns) { + SlotDescriptor slot = descTbl.addSlotDescriptor(dstDesc); + slot.setColumn(column); + slot.setIsMaterialized(true); + if (column.isAllowNull()) { + slot.setIsNullable(true); + } else { + slot.setIsNullable(false); + } + } + + TStreamLoadPutRequest request = getBaseRequest(); + request.setFileType(TFileType.FILE_LOCAL); + request.setColumns("k1,k2, v1=k2"); + StreamLoadScanNode scanNode = new StreamLoadScanNode(new PlanNodeId(1), dstDesc, dstTable, request); + + scanNode.init(analyzer); + scanNode.finalize(analyzer); + scanNode.getNodeExplainString("", TExplainLevel.NORMAL); + TPlanNode planNode = new TPlanNode(); + scanNode.toThrift(planNode); + } + + @Test(expected = UserException.class) + public void testUnsupportedFType() throws UserException, UserException { + Analyzer analyzer = new Analyzer(catalog, connectContext); + DescriptorTable descTbl = analyzer.getDescTbl(); + + List columns = getBaseSchema(); + TupleDescriptor dstDesc = descTbl.createTupleDescriptor("DstTableDesc"); + for (Column column : columns) { + SlotDescriptor slot = descTbl.addSlotDescriptor(dstDesc); + slot.setColumn(column); + slot.setIsMaterialized(true); + if (column.isAllowNull()) { + slot.setIsNullable(true); + } else { + slot.setIsNullable(false); + } + } + + TStreamLoadPutRequest request = getBaseRequest(); + request.setFileType(TFileType.FILE_BROKER); + request.setColumns("k1,k2,v1, v2=k2"); + StreamLoadScanNode scanNode = new StreamLoadScanNode(new PlanNodeId(1), dstDesc, dstTable, request); + + scanNode.init(analyzer); + scanNode.finalize(analyzer); + scanNode.getNodeExplainString("", TExplainLevel.NORMAL); + TPlanNode planNode = new TPlanNode(); + scanNode.toThrift(planNode); + } + + @Test(expected = UserException.class) + public void testColumnsUnknownRef() throws UserException, UserException { + Analyzer analyzer = new Analyzer(catalog, connectContext); + DescriptorTable descTbl = analyzer.getDescTbl(); + + List columns = getBaseSchema(); + TupleDescriptor dstDesc = descTbl.createTupleDescriptor("DstTableDesc"); + for (Column column : columns) { + SlotDescriptor slot = descTbl.addSlotDescriptor(dstDesc); + slot.setColumn(column); + slot.setIsMaterialized(true); + if (column.isAllowNull()) { + slot.setIsNullable(true); + } else { + slot.setIsNullable(false); + } + } + + TStreamLoadPutRequest request = getBaseRequest(); + request.setColumns("k1,k2,v1, v2=k3"); + StreamLoadScanNode scanNode = new StreamLoadScanNode(new PlanNodeId(1), dstDesc, dstTable, request); + + scanNode.init(analyzer); + scanNode.finalize(analyzer); + scanNode.getNodeExplainString("", TExplainLevel.NORMAL); + TPlanNode planNode = new TPlanNode(); + scanNode.toThrift(planNode); + } + + @Test + public void testWhereNormal() throws UserException, UserException { + Analyzer analyzer = new Analyzer(catalog, connectContext); + DescriptorTable descTbl = analyzer.getDescTbl(); + + List columns = getBaseSchema(); + TupleDescriptor dstDesc = descTbl.createTupleDescriptor("DstTableDesc"); + for (Column column : columns) { + SlotDescriptor slot = descTbl.addSlotDescriptor(dstDesc); + slot.setColumn(column); + slot.setIsMaterialized(true); + if (column.isAllowNull()) { + slot.setIsNullable(true); + } else { + slot.setIsNullable(false); + } + } + + TStreamLoadPutRequest request = getBaseRequest(); + request.setColumns("k1,k2,v1, v2=k1"); + request.setWhere("k1 = 1"); + StreamLoadScanNode scanNode = new StreamLoadScanNode(new PlanNodeId(1), dstDesc, dstTable, request); + + scanNode.init(analyzer); + scanNode.finalize(analyzer); + scanNode.getNodeExplainString("", TExplainLevel.NORMAL); + TPlanNode planNode = new TPlanNode(); + scanNode.toThrift(planNode); + } + + @Test(expected = AnalysisException.class) + public void testWhereBad() throws UserException, UserException { + Analyzer analyzer = new Analyzer(catalog, connectContext); + DescriptorTable descTbl = analyzer.getDescTbl(); + + List columns = getBaseSchema(); + TupleDescriptor dstDesc = descTbl.createTupleDescriptor("DstTableDesc"); + for (Column column : columns) { + SlotDescriptor slot = descTbl.addSlotDescriptor(dstDesc); + slot.setColumn(column); + slot.setIsMaterialized(true); + if (column.isAllowNull()) { + slot.setIsNullable(true); + } else { + slot.setIsNullable(false); + } + } + + TStreamLoadPutRequest request = getBaseRequest(); + request.setColumns("k1,k2,v1, v2=k2"); + request.setWhere("k1 1"); + StreamLoadScanNode scanNode = new StreamLoadScanNode(new PlanNodeId(1), dstDesc, dstTable, request); + + scanNode.init(analyzer); + scanNode.finalize(analyzer); + scanNode.getNodeExplainString("", TExplainLevel.NORMAL); + TPlanNode planNode = new TPlanNode(); + scanNode.toThrift(planNode); + } + + @Test(expected = UserException.class) + public void testWhereUnknownRef() throws UserException, UserException { + Analyzer analyzer = new Analyzer(catalog, connectContext); + DescriptorTable descTbl = analyzer.getDescTbl(); + + List columns = getBaseSchema(); + TupleDescriptor dstDesc = descTbl.createTupleDescriptor("DstTableDesc"); + for (Column column : columns) { + SlotDescriptor slot = descTbl.addSlotDescriptor(dstDesc); + slot.setColumn(column); + slot.setIsMaterialized(true); + if (column.isAllowNull()) { + slot.setIsNullable(true); + } else { + slot.setIsNullable(false); + } + } + + TStreamLoadPutRequest request = getBaseRequest(); + request.setColumns("k1,k2,v1, v2=k1"); + request.setWhere("k5 = 1"); + StreamLoadScanNode scanNode = new StreamLoadScanNode(new PlanNodeId(1), dstDesc, dstTable, request); + + scanNode.init(analyzer); + scanNode.finalize(analyzer); + scanNode.getNodeExplainString("", TExplainLevel.NORMAL); + TPlanNode planNode = new TPlanNode(); + scanNode.toThrift(planNode); + } + + @Test(expected = UserException.class) + public void testWhereNotBool() throws UserException, UserException { + Analyzer analyzer = new Analyzer(catalog, connectContext); + DescriptorTable descTbl = analyzer.getDescTbl(); + + List columns = getBaseSchema(); + TupleDescriptor dstDesc = descTbl.createTupleDescriptor("DstTableDesc"); + for (Column column : columns) { + SlotDescriptor slot = descTbl.addSlotDescriptor(dstDesc); + slot.setColumn(column); + slot.setIsMaterialized(true); + if (column.isAllowNull()) { + slot.setIsNullable(true); + } else { + slot.setIsNullable(false); + } + } + + TStreamLoadPutRequest request = getBaseRequest(); + request.setColumns("k1,k2,v1, v2=k1"); + request.setWhere("k1 + v2"); + StreamLoadScanNode scanNode = new StreamLoadScanNode(new PlanNodeId(1), dstDesc, dstTable, request); + + scanNode.init(analyzer); + scanNode.finalize(analyzer); + scanNode.getNodeExplainString("", TExplainLevel.NORMAL); + TPlanNode planNode = new TPlanNode(); + scanNode.toThrift(planNode); + } +} \ No newline at end of file diff --git a/fe/src/test/java/com/baidu/palo/transaction/FakeTransactionIDGenerator.java b/fe/src/test/java/com/baidu/palo/transaction/FakeTransactionIDGenerator.java new file mode 100644 index 0000000000..09c3d71e0b --- /dev/null +++ b/fe/src/test/java/com/baidu/palo/transaction/FakeTransactionIDGenerator.java @@ -0,0 +1,45 @@ +package com.baidu.palo.transaction; + +import com.baidu.palo.persist.EditLog; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; + +import mockit.Mock; +import mockit.MockUp; + +public final class FakeTransactionIDGenerator extends MockUp { + + private long currentId = 1000L; + + @Mock + public void $init() { + // do nothing + } + + @Mock + public void setEditLog(EditLog editLog) { + // do nothing + } + + @Mock + public synchronized long getNextTransactionId() { + System.out.println("getNextTransactionId is called"); + return currentId++; + } + + @Mock + public void write(DataOutput out) throws IOException { + // do nothing + } + + @Mock + public void readFields(DataInput in) throws IOException { + // do nothing + } + + public void setCurrentId(long newId) { + this.currentId = newId; + } +} diff --git a/fe/src/test/java/com/baidu/palo/transaction/GlobalTransactionMgrTest.java b/fe/src/test/java/com/baidu/palo/transaction/GlobalTransactionMgrTest.java new file mode 100644 index 0000000000..32c26b3a72 --- /dev/null +++ b/fe/src/test/java/com/baidu/palo/transaction/GlobalTransactionMgrTest.java @@ -0,0 +1,448 @@ +package com.baidu.palo.transaction; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; + +import com.baidu.palo.catalog.Catalog; +import com.baidu.palo.catalog.CatalogTestUtil; +import com.baidu.palo.catalog.FakeCatalog; +import com.baidu.palo.catalog.FakeEditLog; +import com.baidu.palo.catalog.Partition; +import com.baidu.palo.catalog.Replica; +import com.baidu.palo.catalog.Tablet; +import com.baidu.palo.common.AnalysisException; +import com.baidu.palo.common.FeMetaVersion; +import com.baidu.palo.common.MetaNotFoundException; +import com.baidu.palo.transaction.TransactionState.LoadJobSourceType; + +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; + +import org.junit.Before; +import org.junit.Test; + +import java.lang.reflect.InvocationTargetException; +import java.util.List; +import java.util.Set; + +public class GlobalTransactionMgrTest { + + private static FakeEditLog fakeEditLog; + private static FakeCatalog fakeCatalog; + private static FakeTransactionIDGenerator fakeTransactionIDGenerator; + private static GlobalTransactionMgr masterTransMgr; + private static GlobalTransactionMgr slaveTransMgr; + private static Catalog masterCatalog; + private static Catalog slaveCatalog; + + private String transactionSource = "localfe"; + + + @Before + public void setUp() throws InstantiationException, IllegalAccessException, IllegalArgumentException, + InvocationTargetException, NoSuchMethodException, SecurityException { + fakeEditLog = new FakeEditLog(); + fakeCatalog = new FakeCatalog(); + fakeTransactionIDGenerator = new FakeTransactionIDGenerator(); + masterCatalog = CatalogTestUtil.createTestCatalog(); + slaveCatalog = CatalogTestUtil.createTestCatalog(); + masterCatalog.setJournalVersion(FeMetaVersion.VERSION_40); + slaveCatalog.setJournalVersion(FeMetaVersion.VERSION_40); + masterTransMgr = masterCatalog.getGlobalTransactionMgr(); + masterTransMgr.setEditLog(masterCatalog.getEditLog()); + + slaveTransMgr = slaveCatalog.getGlobalTransactionMgr(); + slaveTransMgr.setEditLog(slaveCatalog.getEditLog()); + } + + @Test + public void testBeginTransaction() throws LabelAlreadyExistsException, AnalysisException, + BeginTransactionException { + FakeCatalog.setCatalog(masterCatalog); + long transactionId = masterTransMgr.beginTransaction(CatalogTestUtil.testDbId1, + CatalogTestUtil.testTxnLable1, + transactionSource, + LoadJobSourceType.FRONTEND); + TransactionState transactionState = fakeEditLog.getTransaction(transactionId); + assertNotNull(transactionState); + assertEquals(transactionId, transactionState.getTransactionId()); + assertEquals(TransactionStatus.PREPARE, transactionState.getTransactionStatus()); + assertEquals(CatalogTestUtil.testDbId1, transactionState.getDbId()); + assertEquals(transactionSource, transactionState.getCoordinator()); + } + + @Test + public void testBeginTransactionWithSameLabel() throws LabelAlreadyExistsException, AnalysisException, + BeginTransactionException { + FakeCatalog.setCatalog(masterCatalog); + long transactionId = 0; + Throwable throwable = null; + try { + transactionId = masterTransMgr.beginTransaction(CatalogTestUtil.testDbId1, + CatalogTestUtil.testTxnLable1, + transactionSource, + LoadJobSourceType.FRONTEND); + } catch (AnalysisException e) { + e.printStackTrace(); + } catch (LabelAlreadyExistsException e) { + e.printStackTrace(); + } + TransactionState transactionState = fakeEditLog.getTransaction(transactionId); + assertNotNull(transactionState); + assertEquals(transactionId, transactionState.getTransactionId()); + assertEquals(TransactionStatus.PREPARE, transactionState.getTransactionStatus()); + assertEquals(CatalogTestUtil.testDbId1, transactionState.getDbId()); + assertEquals(transactionSource, transactionState.getCoordinator()); + + try { + transactionId = masterTransMgr.beginTransaction(CatalogTestUtil.testDbId1, + CatalogTestUtil.testTxnLable1, + transactionSource, + LoadJobSourceType.FRONTEND); + } catch (Exception e) { + // TODO: handle exception + } + } + + // all replica committed success + @Test + public void testCommitTransaction1() throws MetaNotFoundException, + TransactionCommitFailedException, + IllegalTransactionParameterException, LabelAlreadyExistsException, + AnalysisException, BeginTransactionException { + FakeCatalog.setCatalog(masterCatalog); + long transactionId = masterTransMgr.beginTransaction(CatalogTestUtil.testDbId1, + CatalogTestUtil.testTxnLable1, + transactionSource, + LoadJobSourceType.FRONTEND); + // commit a transaction + TabletCommitInfo tabletCommitInfo1 = new TabletCommitInfo(CatalogTestUtil.testTabletId1, + CatalogTestUtil.testBackendId1); + TabletCommitInfo tabletCommitInfo2 = new TabletCommitInfo(CatalogTestUtil.testTabletId1, + CatalogTestUtil.testBackendId2); + TabletCommitInfo tabletCommitInfo3 = new TabletCommitInfo(CatalogTestUtil.testTabletId1, + CatalogTestUtil.testBackendId3); + List transTablets = Lists.newArrayList(); + transTablets.add(tabletCommitInfo1); + transTablets.add(tabletCommitInfo2); + transTablets.add(tabletCommitInfo3); + masterTransMgr.commitTransaction(CatalogTestUtil.testDbId1, transactionId, transTablets); + TransactionState transactionState = fakeEditLog.getTransaction(transactionId); + // check status is committed + assertEquals(TransactionStatus.COMMITTED, transactionState.getTransactionStatus()); + // check replica version + Partition testPartition = masterCatalog.getDb(CatalogTestUtil.testDbId1).getTable(CatalogTestUtil.testTableId1) + .getPartition(CatalogTestUtil.testPartition1); + // check partition version + assertEquals(CatalogTestUtil.testStartVersion, testPartition.getCommittedVersion()); + assertEquals(CatalogTestUtil.testStartVersion + 2, testPartition.getNextVersion()); + // check partition next version + Tablet tablet = testPartition.getIndex(CatalogTestUtil.testIndexId1).getTablet(CatalogTestUtil.testTabletId1); + for (Replica replica : tablet.getReplicas()) { + assertEquals(CatalogTestUtil.testStartVersion, replica.getVersion()); + } + // slave replay new state and compare catalog + FakeCatalog.setCatalog(slaveCatalog); + slaveTransMgr.replayUpsertTransactionState(transactionState); + assertTrue(CatalogTestUtil.compareCatalog(masterCatalog, slaveCatalog)); + } + + // commit with only two replicas + @Test + public void testCommitTransactionWithOneFailed() throws MetaNotFoundException, + TransactionCommitFailedException, + IllegalTransactionParameterException, LabelAlreadyExistsException, + AnalysisException, BeginTransactionException { + TransactionState transactionState = null; + FakeCatalog.setCatalog(masterCatalog); + long transactionId = masterTransMgr.beginTransaction(CatalogTestUtil.testDbId1, + CatalogTestUtil.testTxnLable1, + transactionSource, + LoadJobSourceType.FRONTEND); + // commit a transaction with 1,2 success + TabletCommitInfo tabletCommitInfo1 = new TabletCommitInfo(CatalogTestUtil.testTabletId1, + CatalogTestUtil.testBackendId1); + TabletCommitInfo tabletCommitInfo2 = new TabletCommitInfo(CatalogTestUtil.testTabletId1, + CatalogTestUtil.testBackendId2); + List transTablets = Lists.newArrayList(); + transTablets.add(tabletCommitInfo1); + transTablets.add(tabletCommitInfo2); + masterTransMgr.commitTransaction(CatalogTestUtil.testDbId1, transactionId, transTablets); + + // follower catalog replay the transaction + transactionState = fakeEditLog.getTransaction(transactionId); + FakeCatalog.setCatalog(slaveCatalog); + slaveTransMgr.replayUpsertTransactionState(transactionState); + assertTrue(CatalogTestUtil.compareCatalog(masterCatalog, slaveCatalog)); + + FakeCatalog.setCatalog(masterCatalog); + // commit another transaction with 1,3 success + long transactionId2 = masterTransMgr.beginTransaction(CatalogTestUtil.testDbId1, + CatalogTestUtil.testTxnLable2, + transactionSource, + LoadJobSourceType.FRONTEND); + tabletCommitInfo1 = new TabletCommitInfo(CatalogTestUtil.testTabletId1, CatalogTestUtil.testBackendId1); + TabletCommitInfo tabletCommitInfo3 = new TabletCommitInfo(CatalogTestUtil.testTabletId1, + CatalogTestUtil.testBackendId3); + transTablets = Lists.newArrayList(); + transTablets.add(tabletCommitInfo1); + transTablets.add(tabletCommitInfo3); + masterTransMgr.commitTransaction(CatalogTestUtil.testDbId1, transactionId2, transTablets); + transactionState = fakeEditLog.getTransaction(transactionId2); + // check status is prepare, because the commit failed + assertEquals(TransactionStatus.PREPARE, transactionState.getTransactionStatus()); + // check replica version + Partition testPartition = masterCatalog.getDb(CatalogTestUtil.testDbId1).getTable(CatalogTestUtil.testTableId1) + .getPartition(CatalogTestUtil.testPartition1); + // check partition version + assertEquals(CatalogTestUtil.testStartVersion, testPartition.getCommittedVersion()); + assertEquals(CatalogTestUtil.testStartVersion + 2, testPartition.getNextVersion()); + // check partition next version + Tablet tablet = testPartition.getIndex(CatalogTestUtil.testIndexId1).getTablet(CatalogTestUtil.testTabletId1); + for (Replica replica : tablet.getReplicas()) { + assertEquals(CatalogTestUtil.testStartVersion, replica.getVersion()); + } + // the transaction not committed, so that catalog should be equal + assertTrue(CatalogTestUtil.compareCatalog(masterCatalog, slaveCatalog)); + + // commit the second transaction with 1,2,3 success + tabletCommitInfo1 = new TabletCommitInfo(CatalogTestUtil.testTabletId1, CatalogTestUtil.testBackendId1); + tabletCommitInfo2 = new TabletCommitInfo(CatalogTestUtil.testTabletId1, CatalogTestUtil.testBackendId2); + tabletCommitInfo3 = new TabletCommitInfo(CatalogTestUtil.testTabletId1, CatalogTestUtil.testBackendId3); + transTablets = Lists.newArrayList(); + transTablets.add(tabletCommitInfo1); + transTablets.add(tabletCommitInfo2); + transTablets.add(tabletCommitInfo3); + masterTransMgr.commitTransaction(CatalogTestUtil.testDbId1, transactionId2, transTablets); + transactionState = fakeEditLog.getTransaction(transactionId2); + // check status is commit + assertEquals(TransactionStatus.COMMITTED, transactionState.getTransactionStatus()); + // check replica version + testPartition = masterCatalog.getDb(CatalogTestUtil.testDbId1).getTable(CatalogTestUtil.testTableId1) + .getPartition(CatalogTestUtil.testPartition1); + // check partition version + assertEquals(CatalogTestUtil.testStartVersion, testPartition.getCommittedVersion()); + assertEquals(CatalogTestUtil.testStartVersion + 3, testPartition.getNextVersion()); + // check partition next version + tablet = testPartition.getIndex(CatalogTestUtil.testIndexId1).getTablet(CatalogTestUtil.testTabletId1); + for (Replica replica : tablet.getReplicas()) { + assertEquals(CatalogTestUtil.testStartVersion, replica.getVersion()); + } + Replica replcia1 = tablet.getReplicaById(CatalogTestUtil.testReplicaId1); + Replica replcia2 = tablet.getReplicaById(CatalogTestUtil.testReplicaId2); + Replica replcia3 = tablet.getReplicaById(CatalogTestUtil.testReplicaId3); + assertEquals(CatalogTestUtil.testStartVersion, replcia1.getVersion()); + assertEquals(CatalogTestUtil.testStartVersion, replcia2.getVersion()); + assertEquals(CatalogTestUtil.testStartVersion, replcia3.getVersion()); + assertEquals(-1, replcia1.getLastFailedVersion()); + assertEquals(-1, replcia2.getLastFailedVersion()); + assertEquals(CatalogTestUtil.testStartVersion + 1, replcia3.getLastFailedVersion()); + + // last success version not change, because not published + assertEquals(CatalogTestUtil.testStartVersion, replcia1.getLastSuccessVersion()); + assertEquals(CatalogTestUtil.testStartVersion, replcia2.getLastSuccessVersion()); + assertEquals(CatalogTestUtil.testStartVersion, replcia3.getLastSuccessVersion()); + // check partition version + assertEquals(CatalogTestUtil.testStartVersion, testPartition.getCommittedVersion()); + assertEquals(CatalogTestUtil.testStartVersion + 3, testPartition.getNextVersion()); + + transactionState = fakeEditLog.getTransaction(transactionId2); + FakeCatalog.setCatalog(slaveCatalog); + slaveTransMgr.replayUpsertTransactionState(transactionState); + assertTrue(CatalogTestUtil.compareCatalog(masterCatalog, slaveCatalog)); + } + + public void testFinishTransaction() throws MetaNotFoundException, TransactionCommitFailedException, + IllegalTransactionParameterException, LabelAlreadyExistsException, + AnalysisException, BeginTransactionException { + long transactionId = masterTransMgr.beginTransaction(CatalogTestUtil.testDbId1, + CatalogTestUtil.testTxnLable1, + transactionSource, + LoadJobSourceType.FRONTEND); + // commit a transaction + TabletCommitInfo tabletCommitInfo1 = new TabletCommitInfo(CatalogTestUtil.testTabletId1, + CatalogTestUtil.testBackendId1); + TabletCommitInfo tabletCommitInfo2 = new TabletCommitInfo(CatalogTestUtil.testTabletId1, + CatalogTestUtil.testBackendId2); + TabletCommitInfo tabletCommitInfo3 = new TabletCommitInfo(CatalogTestUtil.testTabletId1, + CatalogTestUtil.testBackendId3); + List transTablets = Lists.newArrayList(); + transTablets.add(tabletCommitInfo1); + transTablets.add(tabletCommitInfo2); + transTablets.add(tabletCommitInfo3); + masterTransMgr.commitTransaction(CatalogTestUtil.testDbId1, transactionId, transTablets); + TransactionState transactionState = fakeEditLog.getTransaction(transactionId); + assertEquals(TransactionStatus.COMMITTED, transactionState.getTransactionStatus()); + Set errorReplicaIds = Sets.newHashSet(); + errorReplicaIds.add(CatalogTestUtil.testReplicaId1); + masterTransMgr.finishTransaction(transactionId, errorReplicaIds); + transactionState = fakeEditLog.getTransaction(transactionId); + assertEquals(TransactionStatus.VISIBLE, transactionState.getTransactionStatus()); + // check replica version + Partition testPartition = masterCatalog.getDb(CatalogTestUtil.testDbId1).getTable(CatalogTestUtil.testTableId1) + .getPartition(CatalogTestUtil.testPartition1); + // check partition version + assertEquals(CatalogTestUtil.testStartVersion + 1, testPartition.getCommittedVersion()); + assertEquals(CatalogTestUtil.testStartVersion + 2, testPartition.getNextVersion()); + // check partition next version + Tablet tablet = testPartition.getIndex(CatalogTestUtil.testIndexId1).getTablet(CatalogTestUtil.testTabletId1); + for (Replica replica : tablet.getReplicas()) { + assertEquals(CatalogTestUtil.testStartVersion + 1, replica.getVersion()); + } + // slave replay new state and compare catalog + slaveTransMgr.replayUpsertTransactionState(transactionState); + assertTrue(CatalogTestUtil.compareCatalog(masterCatalog, slaveCatalog)); + } + + @Test + public void testFinishTransactionWithOneFailed() throws MetaNotFoundException, + TransactionCommitFailedException, + IllegalTransactionParameterException, LabelAlreadyExistsException, + AnalysisException, BeginTransactionException { + TransactionState transactionState = null; + Partition testPartition = masterCatalog.getDb(CatalogTestUtil.testDbId1).getTable(CatalogTestUtil.testTableId1) + .getPartition(CatalogTestUtil.testPartition1); + Tablet tablet = testPartition.getIndex(CatalogTestUtil.testIndexId1).getTablet(CatalogTestUtil.testTabletId1); + FakeCatalog.setCatalog(masterCatalog); + long transactionId = masterTransMgr.beginTransaction(CatalogTestUtil.testDbId1, + CatalogTestUtil.testTxnLable1, + transactionSource, + LoadJobSourceType.FRONTEND); + // commit a transaction with 1,2 success + TabletCommitInfo tabletCommitInfo1 = new TabletCommitInfo(CatalogTestUtil.testTabletId1, + CatalogTestUtil.testBackendId1); + TabletCommitInfo tabletCommitInfo2 = new TabletCommitInfo(CatalogTestUtil.testTabletId1, + CatalogTestUtil.testBackendId2); + List transTablets = Lists.newArrayList(); + transTablets.add(tabletCommitInfo1); + transTablets.add(tabletCommitInfo2); + masterTransMgr.commitTransaction(CatalogTestUtil.testDbId1, transactionId, transTablets); + + // follower catalog replay the transaction + transactionState = fakeEditLog.getTransaction(transactionId); + FakeCatalog.setCatalog(slaveCatalog); + slaveTransMgr.replayUpsertTransactionState(transactionState); + assertTrue(CatalogTestUtil.compareCatalog(masterCatalog, slaveCatalog)); + + // master finish the transaction failed + FakeCatalog.setCatalog(masterCatalog); + Set errorReplicaIds = Sets.newHashSet(); + errorReplicaIds.add(CatalogTestUtil.testReplicaId2); + masterTransMgr.finishTransaction(transactionId, errorReplicaIds); + assertEquals(TransactionStatus.COMMITTED, transactionState.getTransactionStatus()); + Replica replcia1 = tablet.getReplicaById(CatalogTestUtil.testReplicaId1); + Replica replcia2 = tablet.getReplicaById(CatalogTestUtil.testReplicaId2); + Replica replcia3 = tablet.getReplicaById(CatalogTestUtil.testReplicaId3); + assertEquals(CatalogTestUtil.testStartVersion + 1, replcia1.getVersion()); + assertEquals(CatalogTestUtil.testStartVersion, replcia2.getVersion()); + assertEquals(CatalogTestUtil.testStartVersion, replcia3.getVersion()); + assertEquals(-1, replcia1.getLastFailedVersion()); + assertEquals(-1, replcia2.getLastFailedVersion()); + assertEquals(CatalogTestUtil.testStartVersion + 1, replcia3.getLastFailedVersion()); + + errorReplicaIds = Sets.newHashSet(); + masterTransMgr.finishTransaction(transactionId, errorReplicaIds); + assertEquals(TransactionStatus.VISIBLE, transactionState.getTransactionStatus()); + assertEquals(CatalogTestUtil.testStartVersion + 1, replcia1.getVersion()); + assertEquals(CatalogTestUtil.testStartVersion + 1, replcia2.getVersion()); + assertEquals(CatalogTestUtil.testStartVersion, replcia3.getVersion()); + assertEquals(-1, replcia1.getLastFailedVersion()); + assertEquals(-1, replcia2.getLastFailedVersion()); + assertEquals(CatalogTestUtil.testStartVersion + 1, replcia3.getLastFailedVersion()); + + // follower catalog replay the transaction + transactionState = fakeEditLog.getTransaction(transactionId); + FakeCatalog.setCatalog(slaveCatalog); + slaveTransMgr.replayUpsertTransactionState(transactionState); + assertTrue(CatalogTestUtil.compareCatalog(masterCatalog, slaveCatalog)); + + FakeCatalog.setCatalog(masterCatalog); + // commit another transaction with 1,3 success + long transactionId2 = masterTransMgr.beginTransaction(CatalogTestUtil.testDbId1, + CatalogTestUtil.testTxnLable2, + transactionSource, + LoadJobSourceType.FRONTEND); + tabletCommitInfo1 = new TabletCommitInfo(CatalogTestUtil.testTabletId1, CatalogTestUtil.testBackendId1); + TabletCommitInfo tabletCommitInfo3 = new TabletCommitInfo(CatalogTestUtil.testTabletId1, + CatalogTestUtil.testBackendId3); + transTablets = Lists.newArrayList(); + transTablets.add(tabletCommitInfo1); + transTablets.add(tabletCommitInfo3); + masterTransMgr.commitTransaction(CatalogTestUtil.testDbId1, transactionId2, transTablets); + transactionState = fakeEditLog.getTransaction(transactionId2); + // check status is prepare, because the commit failed + assertEquals(TransactionStatus.PREPARE, transactionState.getTransactionStatus()); + + // commit the second transaction with 1,2,3 success + tabletCommitInfo1 = new TabletCommitInfo(CatalogTestUtil.testTabletId1, CatalogTestUtil.testBackendId1); + tabletCommitInfo2 = new TabletCommitInfo(CatalogTestUtil.testTabletId1, CatalogTestUtil.testBackendId2); + tabletCommitInfo3 = new TabletCommitInfo(CatalogTestUtil.testTabletId1, CatalogTestUtil.testBackendId3); + transTablets = Lists.newArrayList(); + transTablets.add(tabletCommitInfo1); + transTablets.add(tabletCommitInfo2); + transTablets.add(tabletCommitInfo3); + masterTransMgr.commitTransaction(CatalogTestUtil.testDbId1, transactionId2, transTablets); + transactionState = fakeEditLog.getTransaction(transactionId2); + // check status is commit + assertEquals(TransactionStatus.COMMITTED, transactionState.getTransactionStatus()); + // check replica version + testPartition = masterCatalog.getDb(CatalogTestUtil.testDbId1).getTable(CatalogTestUtil.testTableId1) + .getPartition(CatalogTestUtil.testPartition1); + // check partition version + assertEquals(CatalogTestUtil.testStartVersion + 1, testPartition.getCommittedVersion()); + assertEquals(CatalogTestUtil.testStartVersion + 3, testPartition.getNextVersion()); + + // follower catalog replay the transaction + transactionState = fakeEditLog.getTransaction(transactionId2); + FakeCatalog.setCatalog(slaveCatalog); + slaveTransMgr.replayUpsertTransactionState(transactionState); + assertTrue(CatalogTestUtil.compareCatalog(masterCatalog, slaveCatalog)); + + // master finish the transaction2 + errorReplicaIds = Sets.newHashSet(); + masterTransMgr.finishTransaction(transactionId2, errorReplicaIds); + assertEquals(TransactionStatus.VISIBLE, transactionState.getTransactionStatus()); + assertEquals(CatalogTestUtil.testStartVersion + 2, replcia1.getVersion()); + assertEquals(CatalogTestUtil.testStartVersion + 2, replcia2.getVersion()); + assertEquals(CatalogTestUtil.testStartVersion, replcia3.getVersion()); + assertEquals(-1, replcia1.getLastFailedVersion()); + assertEquals(-1, replcia2.getLastFailedVersion()); + assertEquals(CatalogTestUtil.testStartVersion + 1, replcia3.getLastFailedVersion()); + + assertEquals(CatalogTestUtil.testStartVersion + 2, replcia1.getLastSuccessVersion()); + assertEquals(CatalogTestUtil.testStartVersion + 2, replcia2.getLastSuccessVersion()); + assertEquals(CatalogTestUtil.testStartVersion + 2, replcia3.getLastSuccessVersion()); + // check partition version + assertEquals(CatalogTestUtil.testStartVersion + 2, testPartition.getCommittedVersion()); + assertEquals(CatalogTestUtil.testStartVersion + 3, testPartition.getNextVersion()); + + transactionState = fakeEditLog.getTransaction(transactionId2); + FakeCatalog.setCatalog(slaveCatalog); + slaveTransMgr.replayUpsertTransactionState(transactionState); + assertTrue(CatalogTestUtil.compareCatalog(masterCatalog, slaveCatalog)); + } + + @Test + public void testDeleteTransaction() throws LabelAlreadyExistsException, + AnalysisException, BeginTransactionException { + + long transactionId = masterTransMgr.beginTransaction(CatalogTestUtil.testDbId1, + CatalogTestUtil.testTxnLable1, + transactionSource, + LoadJobSourceType.FRONTEND); + TransactionState transactionState = fakeEditLog.getTransaction(transactionId); + assertNotNull(transactionState); + assertEquals(transactionId, transactionState.getTransactionId()); + assertEquals(TransactionStatus.PREPARE, transactionState.getTransactionStatus()); + assertEquals(CatalogTestUtil.testDbId1, transactionState.getDbId()); + assertEquals(transactionSource, transactionState.getCoordinator()); + + masterTransMgr.deleteTransaction(transactionId); + transactionState = fakeEditLog.getTransaction(transactionId); + assertNull(transactionState); + transactionState = masterTransMgr.getTransactionState(transactionId); + assertNull(transactionState); + } +} diff --git a/fe/src/test/resources/data/es/clusterstate1.json b/fe/src/test/resources/data/es/clusterstate1.json new file mode 100644 index 0000000000..333119f117 --- /dev/null +++ b/fe/src/test/resources/data/es/clusterstate1.json @@ -0,0 +1,750 @@ +{ + "cluster_name": "elasticsearch", + "version": 28, + "state_uuid": "C6WNazFPSPyZcQlE-cNw1g", + "master_node": "ejy2E2sMTg6nqnjhF9KfsQ", + "blocks": {}, + "nodes": { + "ejy2E2sMTg6nqnjhF9KfsQ": { + "name": "ejy2E2s", + "ephemeral_id": "HO-_F6BLSqedHuv7-yhkNQ", + "transport_address": "192.168.0.1:9209", + "attributes": {} + } + }, + "metadata": { + "cluster_uuid": "_na_", + "templates": {}, + "indices": { + "index2": { + "state": "open", + "settings": { + "index": { + "bpack": { + "partition": { + "upperbound": "2018-10-02" + } + }, + "number_of_shards": "5", + "provided_name": "index2", + "creation_date": "1539592090322", + "number_of_replicas": "1", + "uuid": "T-Kg83WaTOSIXmiO0ZANzg", + "version": { + "created": "5050099" + } + } + }, + "mappings": {}, + "aliases": [ + "indexa" + ], + "primary_terms": { + "0": 1, + "1": 1, + "2": 1, + "3": 1, + "4": 1 + }, + "in_sync_allocations": { + "0": [ + "gbBnb3KFQOyXHl3eaVV6nA" + ], + "1": [ + "sNsOyQBnSdKQuFETNtRYng" + ], + "2": [ + "4UZfdxQeT7CMUxI9Fl5tYA" + ], + "3": [ + "70TzfLRxQ_KL-tT81-QO4w" + ], + "4": [ + "GhBM4mIdTAG-IVjGYFazkQ" + ] + } + }, + "index1": { + "state": "open", + "settings": { + "index": { + "bpack": { + "partition": { + "upperbound": "2018-10-01" + } + }, + "number_of_shards": "5", + "provided_name": "index1", + "creation_date": "1539592085059", + "number_of_replicas": "1", + "uuid": "hjbl6RxaTyCYFfAJwaSjCA", + "version": { + "created": "5050099" + } + } + }, + "mappings": {}, + "aliases": [ + "indexa" + ], + "primary_terms": { + "0": 1, + "1": 1, + "2": 1, + "3": 1, + "4": 1 + }, + "in_sync_allocations": { + "0": [ + "WFVdvk1eQrmOfemq6UoBIw" + ], + "1": [ + "_ASJYn1bRO2MnCN0HKH5BA" + ], + "2": [ + "o5kBd295ReKi2g4g1bpjyA" + ], + "3": [ + "orpHABU0S8eJ0Nd82U_SQA" + ], + "4": [ + "6I6OQ1QfTgCvHigkIdmuPA" + ] + } + } + }, + "index-graveyard": { + "tombstones": [] + } + }, + "routing_table": { + "indices": { + "index2": { + "shards": { + "0": [ + { + "state": "STARTED", + "primary": true, + "node": "ejy2E2sMTg6nqnjhF9KfsQ", + "relocating_node": null, + "shard": 0, + "index": "index2", + "allocation_id": { + "id": "gbBnb3KFQOyXHl3eaVV6nA" + } + }, + { + "state": "UNASSIGNED", + "primary": false, + "node": null, + "relocating_node": null, + "shard": 0, + "index": "index2", + "recovery_source": { + "type": "PEER" + }, + "unassigned_info": { + "reason": "INDEX_CREATED", + "at": "2018-10-15T08:28:10.325Z", + "delayed": false, + "allocation_status": "no_attempt" + } + } + ], + "1": [ + { + "state": "STARTED", + "primary": true, + "node": "ejy2E2sMTg6nqnjhF9KfsQ", + "relocating_node": null, + "shard": 1, + "index": "index2", + "allocation_id": { + "id": "sNsOyQBnSdKQuFETNtRYng" + } + }, + { + "state": "UNASSIGNED", + "primary": false, + "node": null, + "relocating_node": null, + "shard": 1, + "index": "index2", + "recovery_source": { + "type": "PEER" + }, + "unassigned_info": { + "reason": "INDEX_CREATED", + "at": "2018-10-15T08:28:10.325Z", + "delayed": false, + "allocation_status": "no_attempt" + } + } + ], + "2": [ + { + "state": "STARTED", + "primary": true, + "node": "ejy2E2sMTg6nqnjhF9KfsQ", + "relocating_node": null, + "shard": 2, + "index": "index2", + "allocation_id": { + "id": "4UZfdxQeT7CMUxI9Fl5tYA" + } + }, + { + "state": "UNASSIGNED", + "primary": false, + "node": null, + "relocating_node": null, + "shard": 2, + "index": "index2", + "recovery_source": { + "type": "PEER" + }, + "unassigned_info": { + "reason": "INDEX_CREATED", + "at": "2018-10-15T08:28:10.325Z", + "delayed": false, + "allocation_status": "no_attempt" + } + } + ], + "3": [ + { + "state": "STARTED", + "primary": true, + "node": "ejy2E2sMTg6nqnjhF9KfsQ", + "relocating_node": null, + "shard": 3, + "index": "index2", + "allocation_id": { + "id": "70TzfLRxQ_KL-tT81-QO4w" + } + }, + { + "state": "UNASSIGNED", + "primary": false, + "node": null, + "relocating_node": null, + "shard": 3, + "index": "index2", + "recovery_source": { + "type": "PEER" + }, + "unassigned_info": { + "reason": "INDEX_CREATED", + "at": "2018-10-15T08:28:10.325Z", + "delayed": false, + "allocation_status": "no_attempt" + } + } + ], + "4": [ + { + "state": "STARTED", + "primary": true, + "node": "ejy2E2sMTg6nqnjhF9KfsQ", + "relocating_node": null, + "shard": 4, + "index": "index2", + "allocation_id": { + "id": "GhBM4mIdTAG-IVjGYFazkQ" + } + }, + { + "state": "UNASSIGNED", + "primary": false, + "node": null, + "relocating_node": null, + "shard": 4, + "index": "index2", + "recovery_source": { + "type": "PEER" + }, + "unassigned_info": { + "reason": "INDEX_CREATED", + "at": "2018-10-15T08:28:10.325Z", + "delayed": false, + "allocation_status": "no_attempt" + } + } + ], + "5": [ + { + "state": "STARTED", + "primary": true, + "node": "ejy2E2sMTg6nqnjhF9KfsQ", + "relocating_node": null, + "shard": 4, + "index": "index2", + "allocation_id": { + "id": "GhBM4mIdTAG-IVjGYFazkQ" + } + }, + { + "state": "UNASSIGNED", + "primary": false, + "node": null, + "relocating_node": null, + "shard": 4, + "index": "index2", + "recovery_source": { + "type": "PEER" + }, + "unassigned_info": { + "reason": "INDEX_CREATED", + "at": "2018-10-15T08:28:10.325Z", + "delayed": false, + "allocation_status": "no_attempt" + } + } + ] + } + }, + "index1": { + "shards": { + "0": [ + { + "state": "STARTED", + "primary": true, + "node": "ejy2E2sMTg6nqnjhF9KfsQ", + "relocating_node": null, + "shard": 0, + "index": "index1", + "allocation_id": { + "id": "WFVdvk1eQrmOfemq6UoBIw" + } + }, + { + "state": "UNASSIGNED", + "primary": false, + "node": null, + "relocating_node": null, + "shard": 0, + "index": "index1", + "recovery_source": { + "type": "PEER" + }, + "unassigned_info": { + "reason": "INDEX_CREATED", + "at": "2018-10-15T08:28:05.063Z", + "delayed": false, + "allocation_status": "no_attempt" + } + } + ], + "1": [ + { + "state": "STARTED", + "primary": true, + "node": "ejy2E2sMTg6nqnjhF9KfsQ", + "relocating_node": null, + "shard": 1, + "index": "index1", + "allocation_id": { + "id": "_ASJYn1bRO2MnCN0HKH5BA" + } + }, + { + "state": "UNASSIGNED", + "primary": false, + "node": null, + "relocating_node": null, + "shard": 1, + "index": "index1", + "recovery_source": { + "type": "PEER" + }, + "unassigned_info": { + "reason": "INDEX_CREATED", + "at": "2018-10-15T08:28:05.063Z", + "delayed": false, + "allocation_status": "no_attempt" + } + } + ], + "2": [ + { + "state": "STARTED", + "primary": true, + "node": "ejy2E2sMTg6nqnjhF9KfsQ", + "relocating_node": null, + "shard": 2, + "index": "index1", + "allocation_id": { + "id": "o5kBd295ReKi2g4g1bpjyA" + } + }, + { + "state": "UNASSIGNED", + "primary": false, + "node": null, + "relocating_node": null, + "shard": 2, + "index": "index1", + "recovery_source": { + "type": "PEER" + }, + "unassigned_info": { + "reason": "INDEX_CREATED", + "at": "2018-10-15T08:28:05.063Z", + "delayed": false, + "allocation_status": "no_attempt" + } + } + ], + "3": [ + { + "state": "STARTED", + "primary": true, + "node": "ejy2E2sMTg6nqnjhF9KfsQ", + "relocating_node": null, + "shard": 3, + "index": "index1", + "allocation_id": { + "id": "orpHABU0S8eJ0Nd82U_SQA" + } + }, + { + "state": "UNASSIGNED", + "primary": false, + "node": null, + "relocating_node": null, + "shard": 3, + "index": "index1", + "recovery_source": { + "type": "PEER" + }, + "unassigned_info": { + "reason": "INDEX_CREATED", + "at": "2018-10-15T08:28:05.063Z", + "delayed": false, + "allocation_status": "no_attempt" + } + } + ], + "4": [ + { + "state": "STARTED", + "primary": true, + "node": "ejy2E2sMTg6nqnjhF9KfsQ", + "relocating_node": null, + "shard": 4, + "index": "index1", + "allocation_id": { + "id": "6I6OQ1QfTgCvHigkIdmuPA" + } + }, + { + "state": "UNASSIGNED", + "primary": false, + "node": null, + "relocating_node": null, + "shard": 4, + "index": "index1", + "recovery_source": { + "type": "PEER" + }, + "unassigned_info": { + "reason": "INDEX_CREATED", + "at": "2018-10-15T08:28:05.063Z", + "delayed": false, + "allocation_status": "no_attempt" + } + } + ] + } + } + } + }, + "routing_nodes": { + "unassigned": [ + { + "state": "UNASSIGNED", + "primary": false, + "node": null, + "relocating_node": null, + "shard": 2, + "index": "index2", + "recovery_source": { + "type": "PEER" + }, + "unassigned_info": { + "reason": "INDEX_CREATED", + "at": "2018-10-15T08:28:10.325Z", + "delayed": false, + "allocation_status": "no_attempt" + } + }, + { + "state": "UNASSIGNED", + "primary": false, + "node": null, + "relocating_node": null, + "shard": 1, + "index": "index2", + "recovery_source": { + "type": "PEER" + }, + "unassigned_info": { + "reason": "INDEX_CREATED", + "at": "2018-10-15T08:28:10.325Z", + "delayed": false, + "allocation_status": "no_attempt" + } + }, + { + "state": "UNASSIGNED", + "primary": false, + "node": null, + "relocating_node": null, + "shard": 4, + "index": "index2", + "recovery_source": { + "type": "PEER" + }, + "unassigned_info": { + "reason": "INDEX_CREATED", + "at": "2018-10-15T08:28:10.325Z", + "delayed": false, + "allocation_status": "no_attempt" + } + }, + { + "state": "UNASSIGNED", + "primary": false, + "node": null, + "relocating_node": null, + "shard": 3, + "index": "index2", + "recovery_source": { + "type": "PEER" + }, + "unassigned_info": { + "reason": "INDEX_CREATED", + "at": "2018-10-15T08:28:10.325Z", + "delayed": false, + "allocation_status": "no_attempt" + } + }, + { + "state": "UNASSIGNED", + "primary": false, + "node": null, + "relocating_node": null, + "shard": 0, + "index": "index2", + "recovery_source": { + "type": "PEER" + }, + "unassigned_info": { + "reason": "INDEX_CREATED", + "at": "2018-10-15T08:28:10.325Z", + "delayed": false, + "allocation_status": "no_attempt" + } + }, + { + "state": "UNASSIGNED", + "primary": false, + "node": null, + "relocating_node": null, + "shard": 2, + "index": "index1", + "recovery_source": { + "type": "PEER" + }, + "unassigned_info": { + "reason": "INDEX_CREATED", + "at": "2018-10-15T08:28:05.063Z", + "delayed": false, + "allocation_status": "no_attempt" + } + }, + { + "state": "UNASSIGNED", + "primary": false, + "node": null, + "relocating_node": null, + "shard": 1, + "index": "index1", + "recovery_source": { + "type": "PEER" + }, + "unassigned_info": { + "reason": "INDEX_CREATED", + "at": "2018-10-15T08:28:05.063Z", + "delayed": false, + "allocation_status": "no_attempt" + } + }, + { + "state": "UNASSIGNED", + "primary": false, + "node": null, + "relocating_node": null, + "shard": 4, + "index": "index1", + "recovery_source": { + "type": "PEER" + }, + "unassigned_info": { + "reason": "INDEX_CREATED", + "at": "2018-10-15T08:28:05.063Z", + "delayed": false, + "allocation_status": "no_attempt" + } + }, + { + "state": "UNASSIGNED", + "primary": false, + "node": null, + "relocating_node": null, + "shard": 3, + "index": "index1", + "recovery_source": { + "type": "PEER" + }, + "unassigned_info": { + "reason": "INDEX_CREATED", + "at": "2018-10-15T08:28:05.063Z", + "delayed": false, + "allocation_status": "no_attempt" + } + }, + { + "state": "UNASSIGNED", + "primary": false, + "node": null, + "relocating_node": null, + "shard": 0, + "index": "index1", + "recovery_source": { + "type": "PEER" + }, + "unassigned_info": { + "reason": "INDEX_CREATED", + "at": "2018-10-15T08:28:05.063Z", + "delayed": false, + "allocation_status": "no_attempt" + } + } + ], + "nodes": { + "ejy2E2sMTg6nqnjhF9KfsQ": [ + { + "state": "STARTED", + "primary": true, + "node": "ejy2E2sMTg6nqnjhF9KfsQ", + "relocating_node": null, + "shard": 2, + "index": "index2", + "allocation_id": { + "id": "4UZfdxQeT7CMUxI9Fl5tYA" + } + }, + { + "state": "STARTED", + "primary": true, + "node": "ejy2E2sMTg6nqnjhF9KfsQ", + "relocating_node": null, + "shard": 1, + "index": "index2", + "allocation_id": { + "id": "sNsOyQBnSdKQuFETNtRYng" + } + }, + { + "state": "STARTED", + "primary": true, + "node": "ejy2E2sMTg6nqnjhF9KfsQ", + "relocating_node": null, + "shard": 4, + "index": "index2", + "allocation_id": { + "id": "GhBM4mIdTAG-IVjGYFazkQ" + } + }, + { + "state": "STARTED", + "primary": true, + "node": "ejy2E2sMTg6nqnjhF9KfsQ", + "relocating_node": null, + "shard": 3, + "index": "index2", + "allocation_id": { + "id": "70TzfLRxQ_KL-tT81-QO4w" + } + }, + { + "state": "STARTED", + "primary": true, + "node": "ejy2E2sMTg6nqnjhF9KfsQ", + "relocating_node": null, + "shard": 0, + "index": "index2", + "allocation_id": { + "id": "gbBnb3KFQOyXHl3eaVV6nA" + } + }, + { + "state": "STARTED", + "primary": true, + "node": "ejy2E2sMTg6nqnjhF9KfsQ", + "relocating_node": null, + "shard": 2, + "index": "index1", + "allocation_id": { + "id": "o5kBd295ReKi2g4g1bpjyA" + } + }, + { + "state": "STARTED", + "primary": true, + "node": "ejy2E2sMTg6nqnjhF9KfsQ", + "relocating_node": null, + "shard": 1, + "index": "index1", + "allocation_id": { + "id": "_ASJYn1bRO2MnCN0HKH5BA" + } + }, + { + "state": "STARTED", + "primary": true, + "node": "ejy2E2sMTg6nqnjhF9KfsQ", + "relocating_node": null, + "shard": 4, + "index": "index1", + "allocation_id": { + "id": "6I6OQ1QfTgCvHigkIdmuPA" + } + }, + { + "state": "STARTED", + "primary": true, + "node": "ejy2E2sMTg6nqnjhF9KfsQ", + "relocating_node": null, + "shard": 3, + "index": "index1", + "allocation_id": { + "id": "orpHABU0S8eJ0Nd82U_SQA" + } + }, + { + "state": "STARTED", + "primary": true, + "node": "ejy2E2sMTg6nqnjhF9KfsQ", + "relocating_node": null, + "shard": 0, + "index": "index1", + "allocation_id": { + "id": "WFVdvk1eQrmOfemq6UoBIw" + } + } + ] + } + } + } \ No newline at end of file diff --git a/fe/src/test/resources/data/es/clusterstate2.json b/fe/src/test/resources/data/es/clusterstate2.json new file mode 100644 index 0000000000..804486562d --- /dev/null +++ b/fe/src/test/resources/data/es/clusterstate2.json @@ -0,0 +1,720 @@ +{ + "cluster_name": "elasticsearch", + "version": 28, + "state_uuid": "C6WNazFPSPyZcQlE-cNw1g", + "master_node": "ejy2E2sMTg6nqnjhF9KfsQ", + "blocks": {}, + "nodes": { + "ejy2E2sMTg6nqnjhF9KfsQ": { + "name": "ejy2E2s", + "ephemeral_id": "HO-_F6BLSqedHuv7-yhkNQ", + "transport_address": "192.168.0.1:9209", + "attributes": {} + } + }, + "metadata": { + "cluster_uuid": "_na_", + "templates": {}, + "indices": { + "index2": { + "state": "open", + "settings": { + "index": { + "bpack": { + "partition": { + "upperbound": "2018" + } + }, + "number_of_shards": "5", + "provided_name": "index2", + "creation_date": "1539592090322", + "number_of_replicas": "1", + "uuid": "T-Kg83WaTOSIXmiO0ZANzg", + "version": { + "created": "5050099" + } + } + }, + "mappings": {}, + "aliases": [ + "indexa" + ], + "primary_terms": { + "0": 1, + "1": 1, + "2": 1, + "3": 1, + "4": 1 + }, + "in_sync_allocations": { + "0": [ + "gbBnb3KFQOyXHl3eaVV6nA" + ], + "1": [ + "sNsOyQBnSdKQuFETNtRYng" + ], + "2": [ + "4UZfdxQeT7CMUxI9Fl5tYA" + ], + "3": [ + "70TzfLRxQ_KL-tT81-QO4w" + ], + "4": [ + "GhBM4mIdTAG-IVjGYFazkQ" + ] + } + }, + "index1": { + "state": "open", + "settings": { + "index": { + "bpack": { + "partition": { + "upperbound": "2018-10-01" + } + }, + "number_of_shards": "5", + "provided_name": "index1", + "creation_date": "1539592085059", + "number_of_replicas": "1", + "uuid": "hjbl6RxaTyCYFfAJwaSjCA", + "version": { + "created": "5050099" + } + } + }, + "mappings": {}, + "aliases": [ + "indexa" + ], + "primary_terms": { + "0": 1, + "1": 1, + "2": 1, + "3": 1, + "4": 1 + }, + "in_sync_allocations": { + "0": [ + "WFVdvk1eQrmOfemq6UoBIw" + ], + "1": [ + "_ASJYn1bRO2MnCN0HKH5BA" + ], + "2": [ + "o5kBd295ReKi2g4g1bpjyA" + ], + "3": [ + "orpHABU0S8eJ0Nd82U_SQA" + ], + "4": [ + "6I6OQ1QfTgCvHigkIdmuPA" + ] + } + } + }, + "index-graveyard": { + "tombstones": [] + } + }, + "routing_table": { + "indices": { + "index2": { + "shards": { + "0": [ + { + "state": "STARTED", + "primary": true, + "node": "ejy2E2sMTg6nqnjhF9KfsQ", + "relocating_node": null, + "shard": 0, + "index": "index2", + "allocation_id": { + "id": "gbBnb3KFQOyXHl3eaVV6nA" + } + }, + { + "state": "UNASSIGNED", + "primary": false, + "node": null, + "relocating_node": null, + "shard": 0, + "index": "index2", + "recovery_source": { + "type": "PEER" + }, + "unassigned_info": { + "reason": "INDEX_CREATED", + "at": "2018-10-15T08:28:10.325Z", + "delayed": false, + "allocation_status": "no_attempt" + } + } + ], + "1": [ + { + "state": "STARTED", + "primary": true, + "node": "ejy2E2sMTg6nqnjhF9KfsQ", + "relocating_node": null, + "shard": 1, + "index": "index2", + "allocation_id": { + "id": "sNsOyQBnSdKQuFETNtRYng" + } + }, + { + "state": "UNASSIGNED", + "primary": false, + "node": null, + "relocating_node": null, + "shard": 1, + "index": "index2", + "recovery_source": { + "type": "PEER" + }, + "unassigned_info": { + "reason": "INDEX_CREATED", + "at": "2018-10-15T08:28:10.325Z", + "delayed": false, + "allocation_status": "no_attempt" + } + } + ], + "2": [ + { + "state": "STARTED", + "primary": true, + "node": "ejy2E2sMTg6nqnjhF9KfsQ", + "relocating_node": null, + "shard": 2, + "index": "index2", + "allocation_id": { + "id": "4UZfdxQeT7CMUxI9Fl5tYA" + } + }, + { + "state": "UNASSIGNED", + "primary": false, + "node": null, + "relocating_node": null, + "shard": 2, + "index": "index2", + "recovery_source": { + "type": "PEER" + }, + "unassigned_info": { + "reason": "INDEX_CREATED", + "at": "2018-10-15T08:28:10.325Z", + "delayed": false, + "allocation_status": "no_attempt" + } + } + ], + "3": [ + { + "state": "STARTED", + "primary": true, + "node": "ejy2E2sMTg6nqnjhF9KfsQ", + "relocating_node": null, + "shard": 3, + "index": "index2", + "allocation_id": { + "id": "70TzfLRxQ_KL-tT81-QO4w" + } + }, + { + "state": "UNASSIGNED", + "primary": false, + "node": null, + "relocating_node": null, + "shard": 3, + "index": "index2", + "recovery_source": { + "type": "PEER" + }, + "unassigned_info": { + "reason": "INDEX_CREATED", + "at": "2018-10-15T08:28:10.325Z", + "delayed": false, + "allocation_status": "no_attempt" + } + } + ], + "4": [ + { + "state": "STARTED", + "primary": true, + "node": "ejy2E2sMTg6nqnjhF9KfsQ", + "relocating_node": null, + "shard": 4, + "index": "index2", + "allocation_id": { + "id": "GhBM4mIdTAG-IVjGYFazkQ" + } + }, + { + "state": "UNASSIGNED", + "primary": false, + "node": null, + "relocating_node": null, + "shard": 4, + "index": "index2", + "recovery_source": { + "type": "PEER" + }, + "unassigned_info": { + "reason": "INDEX_CREATED", + "at": "2018-10-15T08:28:10.325Z", + "delayed": false, + "allocation_status": "no_attempt" + } + } + ] + } + }, + "index1": { + "shards": { + "0": [ + { + "state": "STARTED", + "primary": true, + "node": "ejy2E2sMTg6nqnjhF9KfsQ", + "relocating_node": null, + "shard": 0, + "index": "index1", + "allocation_id": { + "id": "WFVdvk1eQrmOfemq6UoBIw" + } + }, + { + "state": "UNASSIGNED", + "primary": false, + "node": null, + "relocating_node": null, + "shard": 0, + "index": "index1", + "recovery_source": { + "type": "PEER" + }, + "unassigned_info": { + "reason": "INDEX_CREATED", + "at": "2018-10-15T08:28:05.063Z", + "delayed": false, + "allocation_status": "no_attempt" + } + } + ], + "1": [ + { + "state": "STARTED", + "primary": true, + "node": "ejy2E2sMTg6nqnjhF9KfsQ", + "relocating_node": null, + "shard": 1, + "index": "index1", + "allocation_id": { + "id": "_ASJYn1bRO2MnCN0HKH5BA" + } + }, + { + "state": "UNASSIGNED", + "primary": false, + "node": null, + "relocating_node": null, + "shard": 1, + "index": "index1", + "recovery_source": { + "type": "PEER" + }, + "unassigned_info": { + "reason": "INDEX_CREATED", + "at": "2018-10-15T08:28:05.063Z", + "delayed": false, + "allocation_status": "no_attempt" + } + } + ], + "2": [ + { + "state": "STARTED", + "primary": true, + "node": "ejy2E2sMTg6nqnjhF9KfsQ", + "relocating_node": null, + "shard": 2, + "index": "index1", + "allocation_id": { + "id": "o5kBd295ReKi2g4g1bpjyA" + } + }, + { + "state": "UNASSIGNED", + "primary": false, + "node": null, + "relocating_node": null, + "shard": 2, + "index": "index1", + "recovery_source": { + "type": "PEER" + }, + "unassigned_info": { + "reason": "INDEX_CREATED", + "at": "2018-10-15T08:28:05.063Z", + "delayed": false, + "allocation_status": "no_attempt" + } + } + ], + "3": [ + { + "state": "STARTED", + "primary": true, + "node": "ejy2E2sMTg6nqnjhF9KfsQ", + "relocating_node": null, + "shard": 3, + "index": "index1", + "allocation_id": { + "id": "orpHABU0S8eJ0Nd82U_SQA" + } + }, + { + "state": "UNASSIGNED", + "primary": false, + "node": null, + "relocating_node": null, + "shard": 3, + "index": "index1", + "recovery_source": { + "type": "PEER" + }, + "unassigned_info": { + "reason": "INDEX_CREATED", + "at": "2018-10-15T08:28:05.063Z", + "delayed": false, + "allocation_status": "no_attempt" + } + } + ], + "4": [ + { + "state": "STARTED", + "primary": true, + "node": "ejy2E2sMTg6nqnjhF9KfsQ", + "relocating_node": null, + "shard": 4, + "index": "index1", + "allocation_id": { + "id": "6I6OQ1QfTgCvHigkIdmuPA" + } + }, + { + "state": "UNASSIGNED", + "primary": false, + "node": null, + "relocating_node": null, + "shard": 4, + "index": "index1", + "recovery_source": { + "type": "PEER" + }, + "unassigned_info": { + "reason": "INDEX_CREATED", + "at": "2018-10-15T08:28:05.063Z", + "delayed": false, + "allocation_status": "no_attempt" + } + } + ] + } + } + } + }, + "routing_nodes": { + "unassigned": [ + { + "state": "UNASSIGNED", + "primary": false, + "node": null, + "relocating_node": null, + "shard": 2, + "index": "index2", + "recovery_source": { + "type": "PEER" + }, + "unassigned_info": { + "reason": "INDEX_CREATED", + "at": "2018-10-15T08:28:10.325Z", + "delayed": false, + "allocation_status": "no_attempt" + } + }, + { + "state": "UNASSIGNED", + "primary": false, + "node": null, + "relocating_node": null, + "shard": 1, + "index": "index2", + "recovery_source": { + "type": "PEER" + }, + "unassigned_info": { + "reason": "INDEX_CREATED", + "at": "2018-10-15T08:28:10.325Z", + "delayed": false, + "allocation_status": "no_attempt" + } + }, + { + "state": "UNASSIGNED", + "primary": false, + "node": null, + "relocating_node": null, + "shard": 4, + "index": "index2", + "recovery_source": { + "type": "PEER" + }, + "unassigned_info": { + "reason": "INDEX_CREATED", + "at": "2018-10-15T08:28:10.325Z", + "delayed": false, + "allocation_status": "no_attempt" + } + }, + { + "state": "UNASSIGNED", + "primary": false, + "node": null, + "relocating_node": null, + "shard": 3, + "index": "index2", + "recovery_source": { + "type": "PEER" + }, + "unassigned_info": { + "reason": "INDEX_CREATED", + "at": "2018-10-15T08:28:10.325Z", + "delayed": false, + "allocation_status": "no_attempt" + } + }, + { + "state": "UNASSIGNED", + "primary": false, + "node": null, + "relocating_node": null, + "shard": 0, + "index": "index2", + "recovery_source": { + "type": "PEER" + }, + "unassigned_info": { + "reason": "INDEX_CREATED", + "at": "2018-10-15T08:28:10.325Z", + "delayed": false, + "allocation_status": "no_attempt" + } + }, + { + "state": "UNASSIGNED", + "primary": false, + "node": null, + "relocating_node": null, + "shard": 2, + "index": "index1", + "recovery_source": { + "type": "PEER" + }, + "unassigned_info": { + "reason": "INDEX_CREATED", + "at": "2018-10-15T08:28:05.063Z", + "delayed": false, + "allocation_status": "no_attempt" + } + }, + { + "state": "UNASSIGNED", + "primary": false, + "node": null, + "relocating_node": null, + "shard": 1, + "index": "index1", + "recovery_source": { + "type": "PEER" + }, + "unassigned_info": { + "reason": "INDEX_CREATED", + "at": "2018-10-15T08:28:05.063Z", + "delayed": false, + "allocation_status": "no_attempt" + } + }, + { + "state": "UNASSIGNED", + "primary": false, + "node": null, + "relocating_node": null, + "shard": 4, + "index": "index1", + "recovery_source": { + "type": "PEER" + }, + "unassigned_info": { + "reason": "INDEX_CREATED", + "at": "2018-10-15T08:28:05.063Z", + "delayed": false, + "allocation_status": "no_attempt" + } + }, + { + "state": "UNASSIGNED", + "primary": false, + "node": null, + "relocating_node": null, + "shard": 3, + "index": "index1", + "recovery_source": { + "type": "PEER" + }, + "unassigned_info": { + "reason": "INDEX_CREATED", + "at": "2018-10-15T08:28:05.063Z", + "delayed": false, + "allocation_status": "no_attempt" + } + }, + { + "state": "UNASSIGNED", + "primary": false, + "node": null, + "relocating_node": null, + "shard": 0, + "index": "index1", + "recovery_source": { + "type": "PEER" + }, + "unassigned_info": { + "reason": "INDEX_CREATED", + "at": "2018-10-15T08:28:05.063Z", + "delayed": false, + "allocation_status": "no_attempt" + } + } + ], + "nodes": { + "ejy2E2sMTg6nqnjhF9KfsQ": [ + { + "state": "STARTED", + "primary": true, + "node": "ejy2E2sMTg6nqnjhF9KfsQ", + "relocating_node": null, + "shard": 2, + "index": "index2", + "allocation_id": { + "id": "4UZfdxQeT7CMUxI9Fl5tYA" + } + }, + { + "state": "STARTED", + "primary": true, + "node": "ejy2E2sMTg6nqnjhF9KfsQ", + "relocating_node": null, + "shard": 1, + "index": "index2", + "allocation_id": { + "id": "sNsOyQBnSdKQuFETNtRYng" + } + }, + { + "state": "STARTED", + "primary": true, + "node": "ejy2E2sMTg6nqnjhF9KfsQ", + "relocating_node": null, + "shard": 4, + "index": "index2", + "allocation_id": { + "id": "GhBM4mIdTAG-IVjGYFazkQ" + } + }, + { + "state": "STARTED", + "primary": true, + "node": "ejy2E2sMTg6nqnjhF9KfsQ", + "relocating_node": null, + "shard": 3, + "index": "index2", + "allocation_id": { + "id": "70TzfLRxQ_KL-tT81-QO4w" + } + }, + { + "state": "STARTED", + "primary": true, + "node": "ejy2E2sMTg6nqnjhF9KfsQ", + "relocating_node": null, + "shard": 0, + "index": "index2", + "allocation_id": { + "id": "gbBnb3KFQOyXHl3eaVV6nA" + } + }, + { + "state": "STARTED", + "primary": true, + "node": "ejy2E2sMTg6nqnjhF9KfsQ", + "relocating_node": null, + "shard": 2, + "index": "index1", + "allocation_id": { + "id": "o5kBd295ReKi2g4g1bpjyA" + } + }, + { + "state": "STARTED", + "primary": true, + "node": "ejy2E2sMTg6nqnjhF9KfsQ", + "relocating_node": null, + "shard": 1, + "index": "index1", + "allocation_id": { + "id": "_ASJYn1bRO2MnCN0HKH5BA" + } + }, + { + "state": "STARTED", + "primary": true, + "node": "ejy2E2sMTg6nqnjhF9KfsQ", + "relocating_node": null, + "shard": 4, + "index": "index1", + "allocation_id": { + "id": "6I6OQ1QfTgCvHigkIdmuPA" + } + }, + { + "state": "STARTED", + "primary": true, + "node": "ejy2E2sMTg6nqnjhF9KfsQ", + "relocating_node": null, + "shard": 3, + "index": "index1", + "allocation_id": { + "id": "orpHABU0S8eJ0Nd82U_SQA" + } + }, + { + "state": "STARTED", + "primary": true, + "node": "ejy2E2sMTg6nqnjhF9KfsQ", + "relocating_node": null, + "shard": 0, + "index": "index1", + "allocation_id": { + "id": "WFVdvk1eQrmOfemq6UoBIw" + } + } + ] + } + } + } \ No newline at end of file diff --git a/fe/src/test/resources/data/es/clusterstate3.json b/fe/src/test/resources/data/es/clusterstate3.json new file mode 100644 index 0000000000..121149ad07 --- /dev/null +++ b/fe/src/test/resources/data/es/clusterstate3.json @@ -0,0 +1,745 @@ +{ + "cluster_name": "elasticsearch", + "version": 28, + "state_uuid": "C6WNazFPSPyZcQlE-cNw1g", + "master_node": "ejy2E2sMTg6nqnjhF9KfsQ", + "blocks": {}, + "nodes": { + "ejy2E2sMTg6nqnjhF9KfsQ": { + "name": "ejy2E2s", + "ephemeral_id": "HO-_F6BLSqedHuv7-yhkNQ", + "transport_address": "192.168.0.1:9209", + "attributes": {} + } + }, + "metadata": { + "cluster_uuid": "_na_", + "templates": {}, + "indices": { + "index2": { + "state": "open", + "settings": { + "index": { + "number_of_shards": "5", + "provided_name": "index2", + "creation_date": "1539592090322", + "number_of_replicas": "1", + "uuid": "T-Kg83WaTOSIXmiO0ZANzg", + "version": { + "created": "5050099" + } + } + }, + "mappings": {}, + "aliases": [ + "indexa" + ], + "primary_terms": { + "0": 1, + "1": 1, + "2": 1, + "3": 1, + "4": 1 + }, + "in_sync_allocations": { + "0": [ + "gbBnb3KFQOyXHl3eaVV6nA" + ], + "1": [ + "sNsOyQBnSdKQuFETNtRYng" + ], + "2": [ + "4UZfdxQeT7CMUxI9Fl5tYA" + ], + "3": [ + "70TzfLRxQ_KL-tT81-QO4w" + ], + "4": [ + "GhBM4mIdTAG-IVjGYFazkQ" + ] + } + }, + "index1": { + "state": "open", + "settings": { + "index": { + "bpack": { + "partition": { + "upperbound": "2018-10-01" + } + }, + "number_of_shards": "5", + "provided_name": "index1", + "creation_date": "1539592085059", + "number_of_replicas": "1", + "uuid": "hjbl6RxaTyCYFfAJwaSjCA", + "version": { + "created": "5050099" + } + } + }, + "mappings": {}, + "aliases": [ + "indexa" + ], + "primary_terms": { + "0": 1, + "1": 1, + "2": 1, + "3": 1, + "4": 1 + }, + "in_sync_allocations": { + "0": [ + "WFVdvk1eQrmOfemq6UoBIw" + ], + "1": [ + "_ASJYn1bRO2MnCN0HKH5BA" + ], + "2": [ + "o5kBd295ReKi2g4g1bpjyA" + ], + "3": [ + "orpHABU0S8eJ0Nd82U_SQA" + ], + "4": [ + "6I6OQ1QfTgCvHigkIdmuPA" + ] + } + } + }, + "index-graveyard": { + "tombstones": [] + } + }, + "routing_table": { + "indices": { + "index2": { + "shards": { + "0": [ + { + "state": "STARTED", + "primary": true, + "node": "ejy2E2sMTg6nqnjhF9KfsQ", + "relocating_node": null, + "shard": 0, + "index": "index2", + "allocation_id": { + "id": "gbBnb3KFQOyXHl3eaVV6nA" + } + }, + { + "state": "UNASSIGNED", + "primary": false, + "node": null, + "relocating_node": null, + "shard": 0, + "index": "index2", + "recovery_source": { + "type": "PEER" + }, + "unassigned_info": { + "reason": "INDEX_CREATED", + "at": "2018-10-15T08:28:10.325Z", + "delayed": false, + "allocation_status": "no_attempt" + } + } + ], + "1": [ + { + "state": "STARTED", + "primary": true, + "node": "ejy2E2sMTg6nqnjhF9KfsQ", + "relocating_node": null, + "shard": 1, + "index": "index2", + "allocation_id": { + "id": "sNsOyQBnSdKQuFETNtRYng" + } + }, + { + "state": "UNASSIGNED", + "primary": false, + "node": null, + "relocating_node": null, + "shard": 1, + "index": "index2", + "recovery_source": { + "type": "PEER" + }, + "unassigned_info": { + "reason": "INDEX_CREATED", + "at": "2018-10-15T08:28:10.325Z", + "delayed": false, + "allocation_status": "no_attempt" + } + } + ], + "2": [ + { + "state": "STARTED", + "primary": true, + "node": "ejy2E2sMTg6nqnjhF9KfsQ", + "relocating_node": null, + "shard": 2, + "index": "index2", + "allocation_id": { + "id": "4UZfdxQeT7CMUxI9Fl5tYA" + } + }, + { + "state": "UNASSIGNED", + "primary": false, + "node": null, + "relocating_node": null, + "shard": 2, + "index": "index2", + "recovery_source": { + "type": "PEER" + }, + "unassigned_info": { + "reason": "INDEX_CREATED", + "at": "2018-10-15T08:28:10.325Z", + "delayed": false, + "allocation_status": "no_attempt" + } + } + ], + "3": [ + { + "state": "STARTED", + "primary": true, + "node": "ejy2E2sMTg6nqnjhF9KfsQ", + "relocating_node": null, + "shard": 3, + "index": "index2", + "allocation_id": { + "id": "70TzfLRxQ_KL-tT81-QO4w" + } + }, + { + "state": "UNASSIGNED", + "primary": false, + "node": null, + "relocating_node": null, + "shard": 3, + "index": "index2", + "recovery_source": { + "type": "PEER" + }, + "unassigned_info": { + "reason": "INDEX_CREATED", + "at": "2018-10-15T08:28:10.325Z", + "delayed": false, + "allocation_status": "no_attempt" + } + } + ], + "4": [ + { + "state": "STARTED", + "primary": true, + "node": "ejy2E2sMTg6nqnjhF9KfsQ", + "relocating_node": null, + "shard": 4, + "index": "index2", + "allocation_id": { + "id": "GhBM4mIdTAG-IVjGYFazkQ" + } + }, + { + "state": "UNASSIGNED", + "primary": false, + "node": null, + "relocating_node": null, + "shard": 4, + "index": "index2", + "recovery_source": { + "type": "PEER" + }, + "unassigned_info": { + "reason": "INDEX_CREATED", + "at": "2018-10-15T08:28:10.325Z", + "delayed": false, + "allocation_status": "no_attempt" + } + } + ], + "5": [ + { + "state": "STARTED", + "primary": true, + "node": "ejy2E2sMTg6nqnjhF9KfsQ", + "relocating_node": null, + "shard": 4, + "index": "index2", + "allocation_id": { + "id": "GhBM4mIdTAG-IVjGYFazkQ" + } + }, + { + "state": "UNASSIGNED", + "primary": false, + "node": null, + "relocating_node": null, + "shard": 4, + "index": "index2", + "recovery_source": { + "type": "PEER" + }, + "unassigned_info": { + "reason": "INDEX_CREATED", + "at": "2018-10-15T08:28:10.325Z", + "delayed": false, + "allocation_status": "no_attempt" + } + } + ] + } + }, + "index1": { + "shards": { + "0": [ + { + "state": "STARTED", + "primary": true, + "node": "ejy2E2sMTg6nqnjhF9KfsQ", + "relocating_node": null, + "shard": 0, + "index": "index1", + "allocation_id": { + "id": "WFVdvk1eQrmOfemq6UoBIw" + } + }, + { + "state": "UNASSIGNED", + "primary": false, + "node": null, + "relocating_node": null, + "shard": 0, + "index": "index1", + "recovery_source": { + "type": "PEER" + }, + "unassigned_info": { + "reason": "INDEX_CREATED", + "at": "2018-10-15T08:28:05.063Z", + "delayed": false, + "allocation_status": "no_attempt" + } + } + ], + "1": [ + { + "state": "STARTED", + "primary": true, + "node": "ejy2E2sMTg6nqnjhF9KfsQ", + "relocating_node": null, + "shard": 1, + "index": "index1", + "allocation_id": { + "id": "_ASJYn1bRO2MnCN0HKH5BA" + } + }, + { + "state": "UNASSIGNED", + "primary": false, + "node": null, + "relocating_node": null, + "shard": 1, + "index": "index1", + "recovery_source": { + "type": "PEER" + }, + "unassigned_info": { + "reason": "INDEX_CREATED", + "at": "2018-10-15T08:28:05.063Z", + "delayed": false, + "allocation_status": "no_attempt" + } + } + ], + "2": [ + { + "state": "STARTED", + "primary": true, + "node": "ejy2E2sMTg6nqnjhF9KfsQ", + "relocating_node": null, + "shard": 2, + "index": "index1", + "allocation_id": { + "id": "o5kBd295ReKi2g4g1bpjyA" + } + }, + { + "state": "UNASSIGNED", + "primary": false, + "node": null, + "relocating_node": null, + "shard": 2, + "index": "index1", + "recovery_source": { + "type": "PEER" + }, + "unassigned_info": { + "reason": "INDEX_CREATED", + "at": "2018-10-15T08:28:05.063Z", + "delayed": false, + "allocation_status": "no_attempt" + } + } + ], + "3": [ + { + "state": "STARTED", + "primary": true, + "node": "ejy2E2sMTg6nqnjhF9KfsQ", + "relocating_node": null, + "shard": 3, + "index": "index1", + "allocation_id": { + "id": "orpHABU0S8eJ0Nd82U_SQA" + } + }, + { + "state": "UNASSIGNED", + "primary": false, + "node": null, + "relocating_node": null, + "shard": 3, + "index": "index1", + "recovery_source": { + "type": "PEER" + }, + "unassigned_info": { + "reason": "INDEX_CREATED", + "at": "2018-10-15T08:28:05.063Z", + "delayed": false, + "allocation_status": "no_attempt" + } + } + ], + "4": [ + { + "state": "STARTED", + "primary": true, + "node": "ejy2E2sMTg6nqnjhF9KfsQ", + "relocating_node": null, + "shard": 4, + "index": "index1", + "allocation_id": { + "id": "6I6OQ1QfTgCvHigkIdmuPA" + } + }, + { + "state": "UNASSIGNED", + "primary": false, + "node": null, + "relocating_node": null, + "shard": 4, + "index": "index1", + "recovery_source": { + "type": "PEER" + }, + "unassigned_info": { + "reason": "INDEX_CREATED", + "at": "2018-10-15T08:28:05.063Z", + "delayed": false, + "allocation_status": "no_attempt" + } + } + ] + } + } + } + }, + "routing_nodes": { + "unassigned": [ + { + "state": "UNASSIGNED", + "primary": false, + "node": null, + "relocating_node": null, + "shard": 2, + "index": "index2", + "recovery_source": { + "type": "PEER" + }, + "unassigned_info": { + "reason": "INDEX_CREATED", + "at": "2018-10-15T08:28:10.325Z", + "delayed": false, + "allocation_status": "no_attempt" + } + }, + { + "state": "UNASSIGNED", + "primary": false, + "node": null, + "relocating_node": null, + "shard": 1, + "index": "index2", + "recovery_source": { + "type": "PEER" + }, + "unassigned_info": { + "reason": "INDEX_CREATED", + "at": "2018-10-15T08:28:10.325Z", + "delayed": false, + "allocation_status": "no_attempt" + } + }, + { + "state": "UNASSIGNED", + "primary": false, + "node": null, + "relocating_node": null, + "shard": 4, + "index": "index2", + "recovery_source": { + "type": "PEER" + }, + "unassigned_info": { + "reason": "INDEX_CREATED", + "at": "2018-10-15T08:28:10.325Z", + "delayed": false, + "allocation_status": "no_attempt" + } + }, + { + "state": "UNASSIGNED", + "primary": false, + "node": null, + "relocating_node": null, + "shard": 3, + "index": "index2", + "recovery_source": { + "type": "PEER" + }, + "unassigned_info": { + "reason": "INDEX_CREATED", + "at": "2018-10-15T08:28:10.325Z", + "delayed": false, + "allocation_status": "no_attempt" + } + }, + { + "state": "UNASSIGNED", + "primary": false, + "node": null, + "relocating_node": null, + "shard": 0, + "index": "index2", + "recovery_source": { + "type": "PEER" + }, + "unassigned_info": { + "reason": "INDEX_CREATED", + "at": "2018-10-15T08:28:10.325Z", + "delayed": false, + "allocation_status": "no_attempt" + } + }, + { + "state": "UNASSIGNED", + "primary": false, + "node": null, + "relocating_node": null, + "shard": 2, + "index": "index1", + "recovery_source": { + "type": "PEER" + }, + "unassigned_info": { + "reason": "INDEX_CREATED", + "at": "2018-10-15T08:28:05.063Z", + "delayed": false, + "allocation_status": "no_attempt" + } + }, + { + "state": "UNASSIGNED", + "primary": false, + "node": null, + "relocating_node": null, + "shard": 1, + "index": "index1", + "recovery_source": { + "type": "PEER" + }, + "unassigned_info": { + "reason": "INDEX_CREATED", + "at": "2018-10-15T08:28:05.063Z", + "delayed": false, + "allocation_status": "no_attempt" + } + }, + { + "state": "UNASSIGNED", + "primary": false, + "node": null, + "relocating_node": null, + "shard": 4, + "index": "index1", + "recovery_source": { + "type": "PEER" + }, + "unassigned_info": { + "reason": "INDEX_CREATED", + "at": "2018-10-15T08:28:05.063Z", + "delayed": false, + "allocation_status": "no_attempt" + } + }, + { + "state": "UNASSIGNED", + "primary": false, + "node": null, + "relocating_node": null, + "shard": 3, + "index": "index1", + "recovery_source": { + "type": "PEER" + }, + "unassigned_info": { + "reason": "INDEX_CREATED", + "at": "2018-10-15T08:28:05.063Z", + "delayed": false, + "allocation_status": "no_attempt" + } + }, + { + "state": "UNASSIGNED", + "primary": false, + "node": null, + "relocating_node": null, + "shard": 0, + "index": "index1", + "recovery_source": { + "type": "PEER" + }, + "unassigned_info": { + "reason": "INDEX_CREATED", + "at": "2018-10-15T08:28:05.063Z", + "delayed": false, + "allocation_status": "no_attempt" + } + } + ], + "nodes": { + "ejy2E2sMTg6nqnjhF9KfsQ": [ + { + "state": "STARTED", + "primary": true, + "node": "ejy2E2sMTg6nqnjhF9KfsQ", + "relocating_node": null, + "shard": 2, + "index": "index2", + "allocation_id": { + "id": "4UZfdxQeT7CMUxI9Fl5tYA" + } + }, + { + "state": "STARTED", + "primary": true, + "node": "ejy2E2sMTg6nqnjhF9KfsQ", + "relocating_node": null, + "shard": 1, + "index": "index2", + "allocation_id": { + "id": "sNsOyQBnSdKQuFETNtRYng" + } + }, + { + "state": "STARTED", + "primary": true, + "node": "ejy2E2sMTg6nqnjhF9KfsQ", + "relocating_node": null, + "shard": 4, + "index": "index2", + "allocation_id": { + "id": "GhBM4mIdTAG-IVjGYFazkQ" + } + }, + { + "state": "STARTED", + "primary": true, + "node": "ejy2E2sMTg6nqnjhF9KfsQ", + "relocating_node": null, + "shard": 3, + "index": "index2", + "allocation_id": { + "id": "70TzfLRxQ_KL-tT81-QO4w" + } + }, + { + "state": "STARTED", + "primary": true, + "node": "ejy2E2sMTg6nqnjhF9KfsQ", + "relocating_node": null, + "shard": 0, + "index": "index2", + "allocation_id": { + "id": "gbBnb3KFQOyXHl3eaVV6nA" + } + }, + { + "state": "STARTED", + "primary": true, + "node": "ejy2E2sMTg6nqnjhF9KfsQ", + "relocating_node": null, + "shard": 2, + "index": "index1", + "allocation_id": { + "id": "o5kBd295ReKi2g4g1bpjyA" + } + }, + { + "state": "STARTED", + "primary": true, + "node": "ejy2E2sMTg6nqnjhF9KfsQ", + "relocating_node": null, + "shard": 1, + "index": "index1", + "allocation_id": { + "id": "_ASJYn1bRO2MnCN0HKH5BA" + } + }, + { + "state": "STARTED", + "primary": true, + "node": "ejy2E2sMTg6nqnjhF9KfsQ", + "relocating_node": null, + "shard": 4, + "index": "index1", + "allocation_id": { + "id": "6I6OQ1QfTgCvHigkIdmuPA" + } + }, + { + "state": "STARTED", + "primary": true, + "node": "ejy2E2sMTg6nqnjhF9KfsQ", + "relocating_node": null, + "shard": 3, + "index": "index1", + "allocation_id": { + "id": "orpHABU0S8eJ0Nd82U_SQA" + } + }, + { + "state": "STARTED", + "primary": true, + "node": "ejy2E2sMTg6nqnjhF9KfsQ", + "relocating_node": null, + "shard": 0, + "index": "index1", + "allocation_id": { + "id": "WFVdvk1eQrmOfemq6UoBIw" + } + } + ] + } + } + } \ No newline at end of file diff --git a/fs_brokers/apache_hdfs_broker/build.xml.deprecated b/fs_brokers/apache_hdfs_broker/build.xml.deprecated deleted file mode 100644 index 52fbe9552b..0000000000 --- a/fs_brokers/apache_hdfs_broker/build.xml.deprecated +++ /dev/null @@ -1,95 +0,0 @@ - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - diff --git a/fs_brokers/apache_hdfs_broker/pom.xml b/fs_brokers/apache_hdfs_broker/pom.xml new file mode 100644 index 0000000000..6ccb7265ad --- /dev/null +++ b/fs_brokers/apache_hdfs_broker/pom.xml @@ -0,0 +1,458 @@ + + + + + + 4.0.0 + + org.apache + doris-apache-hdfs-broker + 3.4.0 + jar + + doris-apache-hdfs-broker + https://github.com/apache/incubator-doris + + + ${basedir}/../../ + UTF-8 + 1.8 + 1.8 + + + + + + custome-env + + + env.CUSTOM_MAVEN_REPO + + + + + + custom-nexus + ${env.CUSTOM_MAVEN_REPO} + + + + + + custom-nexus + ${env.CUSTOM_MAVEN_REPO} + + + + + + + general-env + + + !env.CUSTOM_MAVEN_REPO + + + + + + + cloudera-thirdparty + https://repository.cloudera.com/content/repositories/third-party/ + + + + oracleReleases + http://download.oracle.com/maven + + + + + + + spring-plugins + http://repo.spring.io/plugins-release/ + + + + + + + + + commons-cli + commons-cli + 1.4 + + + + + commons-codec + commons-codec + 1.9 + + + + + commons-collections + commons-collections + 3.2.2 + + + + + commons-configuration + commons-configuration + 1.6 + + + + + commons-daemon + commons-daemon + 1.0.13 + + + + + commons-io + commons-io + 2.4 + + + + + commons-lang + commons-lang + 2.6 + + + + + org.apache.commons + commons-lang3 + 3.3.2 + + + + + commons-logging + commons-logging + 1.1.3 + + + + + com.google.guava + guava + 15.0 + + + + + org.apache.hadoop + hadoop-annotations + 2.7.3 + + + + + org.apache.hadoop + hadoop-auth + 2.7.3 + + + + + org.apache.hadoop + hadoop-common + 2.7.3 + + + + + org.apache.hadoop + hadoop-hdfs + 2.7.3 + + + + + org.apache.htrace + htrace-core + 3.1.0-incubating + + + + + junit + junit + 4.12 + test + + + + + org.apache.thrift + libthrift + 0.9.3 + + + + + log4j + log4j + 1.2.17 + + + + + org.apache.logging.log4j + log4j-api + 2.2 + + + + + org.apache.logging.log4j + log4j-core + 2.2 + + + + + com.google.protobuf + protobuf-java + 2.5.0 + + + + + org.slf4j + slf4j-api + 1.6.4 + + + + + org.slf4j + slf4j-log4j12 + 1.7.5 + test + + + + + apache_hdfs_broker + + + + exec-maven-plugin + org.codehaus.mojo + 1.6.0 + + + mkdir-thrift-dir + process-resources + + exec + + + false + mkdir + + -p + ${basedir}/src/main/resources/thrift + ${basedir}/src/main/thrift + + ${skip.plugin} + + + + copy-thrift + process-resources + + exec + + + cp + + ${palo.home}/gensrc/thrift/PaloBrokerService.thrift + ${basedir}/src/main/resources/thrift/ + + ${skip.plugin} + + + + + + + + org.apache.thrift.tools + maven-thrift-plugin + 0.1.11 + + + ${basedir}/src/main/resources/thrift/ + ${basedir}/src/main/thrift/ + ${skip.plugin} + + + + thrift-sources + generate-sources + + compile + + + + + + + + org.apache.maven.plugins + maven-dependency-plugin + 3.1.1 + + + copy-dependencies + package + + copy-dependencies + + + ${project.build.directory}/lib + false + false + true + ${skip.plugin} + + + + + + + + + + + org.eclipse.m2e + lifecycle-mapping + 1.0.0 + + + + + + exec-maven-plugin + org.codehaus.mojo + [1.6,) + + exec + + + + + + + + + + org.apache.thrift.tools + maven-thrift-plugin + [0.1.11,) + + compile + + + + + + + + + + org.codehaus.mojo + exec-maven-plugin + [1.6,) + + exec + + + + + + + + + + org.apache.maven.plugins + maven-dependency-plugin + [3.1.1,) + + copy-dependencies + + + + + + + + + + org.apache.maven.plugins + maven-compiler-plugin + [3.1,) + + compile + testCompile + + + + + + + + + + org.apache.maven.plugins + maven-resources-plugin + [2.6,) + + resources + testResources + + + + + + + + + + + + + + + + + diff --git a/gensrc/proto/descriptors.proto b/gensrc/proto/descriptors.proto new file mode 100644 index 0000000000..60f53b2d23 --- /dev/null +++ b/gensrc/proto/descriptors.proto @@ -0,0 +1,59 @@ +// Copyright (c) 2018, Baidu.com, Inc. All Rights Reserved + +// Licensed 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. + +syntax="proto2"; + +package palo; + +import "types.proto"; + +message PSlotDescriptor { + required int32 id = 1; + required int32 parent = 2; // tuple id which this slot is belong to + required PTypeDesc slot_type = 3; + required int32 column_pos = 4; // in originating table + required int32 byte_offset = 5; // into tuple + required int32 null_indicator_byte = 6; + required int32 null_indicator_bit = 7; + required string col_name = 8; + required int32 slot_idx = 9; + required bool is_materialized = 10; +}; + +message PTupleDescriptor { + required int32 id = 1; + required int32 byte_size = 2; + required int32 num_null_bytes = 3; + optional int64 table_id = 4; + optional int32 num_null_slots = 5; +}; + +message POlapTableIndexSchema { + required int64 id = 1; + repeated string columns = 2; + required int32 schema_hash = 3; +}; + +message POlapTableSchemaParam { + required int64 db_id = 1; + required int64 table_id = 2; + required int64 version = 3; + + // Logical columns, contain all column that in logical table + repeated PSlotDescriptor slot_descs = 4; + required PTupleDescriptor tuple_desc = 5; + repeated POlapTableIndexSchema indexes = 6; +}; + diff --git a/gensrc/thrift/PaloExternalDataSourceService.thrift b/gensrc/thrift/PaloExternalDataSourceService.thrift new file mode 100644 index 0000000000..49aa668dcc --- /dev/null +++ b/gensrc/thrift/PaloExternalDataSourceService.thrift @@ -0,0 +1,221 @@ +// Modifications copyright (C) 2017, Baidu.com, Inc. +// Copyright 2017 The Apache Software Foundation + +// 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. + +namespace java com.baidu.palo.thrift +namespace cpp palo + +include "Exprs.thrift" +include "Opcodes.thrift" +include "Status.thrift" +include "Types.thrift" + +// A result set column descriptor. +// this definition id different from column desc in palo, the column desc in palo only support scalar type, does not support map, array +// so that should convert palo column desc into ExtColumnDesc +struct TExtColumnDesc { + // The column name as given in the Create .. statement. Always set. + 1: optional string name + // The column type. Always set. + 2: optional Types.TTypeDesc type +} + +// Metadata used to describe the schema (column names, types, comments) +// of result sets. +struct TExtTableSchema { + // List of columns. Always set. + 1: optional list cols +} + +struct TExtLiteral { + 1: required Exprs.TExprNodeType node_type + 2: optional Exprs.TBoolLiteral bool_literal + 3: optional Exprs.TDateLiteral date_literal + 4: optional Exprs.TFloatLiteral float_literal + 5: optional Exprs.TIntLiteral int_literal + 6: optional Exprs.TStringLiteral string_literal + 7: optional Exprs.TDecimalLiteral decimal_literal + 8: optional Exprs.TLargeIntLiteral large_int_literal +} + +// Binary predicates that can be pushed to the external data source and +// are of the form . Sources can choose to accept or reject +// predicates via the return value of prepare(), see TPrepareResult. +// The column and the value are guaranteed to be type compatible in Impala, +// but they are not necessarily the same type, so the data source +// implementation may need to do an implicit cast. +struct TExtBinaryPredicate { + // Column on which the predicate is applied. Always set. + 1: optional TExtColumnDesc col + // Comparison operator. Always set. + 2: optional Opcodes.TExprOpcode op + // Value on the right side of the binary predicate. Always set. + 3: optional TExtLiteral value +} + +// a union of all predicates +struct TExtPredicate { + 1: required Exprs.TExprNodeType node_type + 2: optional TExtBinaryPredicate binary_predicate +} + +// A union over all possible return types for a column of data +// Currently only used by ExternalDataSource types +// +struct TExtColumnData { + // One element in the list for every row in the column indicating if there is + // a value in the vals list or a null. + 1: required list is_null; + + // Only one is set, only non-null values are set. this indicates one column data for a row batch + 2: optional list bool_vals; + 3: optional binary byte_vals; + 4: optional list short_vals; + 5: optional list int_vals; + 6: optional list long_vals; + 7: optional list double_vals; + 8: optional list string_vals; + 9: optional list binary_vals; +} + +// Serialized batch of rows returned by getNext(). +// one row batch contains mult rows, and the result is arranged in column style +struct TExtRowBatch { + // Each TColumnData contains the data for an entire column. Always set. + 1: optional list cols + + // The number of rows returned. For count(*) queries, there may not be + // any materialized columns so cols will be an empty list and this value + // will indicate how many rows are returned. When there are materialized + // columns, this number should be the same as the size of each + // TColumnData.is_null list. + 2: optional i64 num_rows +} + +// Parameters to prepare(). +struct TExtPrepareParams { + // The name of the table. Always set. + 1: optional string table_name + + // A string specified for the table that is passed to the external data source. + // Always set, may be an empty string. + 2: optional string init_string + + // A list of conjunctive (AND) clauses, each of which contains a list of + // disjunctive (OR) binary predicates. Always set, may be an empty list. + 3: optional list> predicates +} + +// Returned by prepare(). +struct TExtPrepareResult { + 1: required Status.TStatus status + + // Estimate of the total number of rows returned when applying the predicates indicated + // by accepted_conjuncts. Not set if the data source does not support providing + // this statistic. + 2: optional i64 num_rows_estimate + + // Accepted conjuncts. References the 'predicates' parameter in the prepare() + // call. It contains the 0-based indices of the top-level list elements (the + // AND elements) that the library will be able to apply during the scan. Those + // elements that aren’t referenced in accepted_conjuncts will be evaluated by + // Impala itself. + 3: optional list accepted_conjuncts +} + +// Parameters to open(). +struct TExtOpenParams { + // A unique identifier for the query. Always set. + 1: optional Types.TUniqueId query_id + + // The name of the table. Always set. + 2: optional string table_name + + // A string specified for the table that is passed to the external data source. + // Always set, may be an empty string. + 3: optional map properties + + // The authenticated user name. Always set. + 4: optional string authenticated_user_name + + // The schema of the rows that the scan needs to return. Always set. + 5: optional TExtTableSchema row_schema + + // The expected size of the row batches it returns in the subsequent getNext() calls. + // Always set. + 6: optional i32 batch_size + + 7: optional list> predicates + + // The query limit, if specified. + 8: optional i64 limit +} + +// Returned by open(). +struct TExtOpenResult { + 1: required Status.TStatus status + + // An opaque handle used in subsequent getNext()/close() calls. Required. + 2: optional string scan_handle +} + +// Parameters to getNext() +struct TExtGetNextParams { + // The opaque handle returned by the previous open() call. Always set. + 1: optional string scan_handle // es search context id + 2: optional i64 offset // es should check the offset to prevent duplicate rpc calls +} + +// Returned by getNext(). +struct TExtGetNextResult { + 1: required Status.TStatus status + + // If true, reached the end of the result stream; subsequent calls to + // getNext() won’t return any more results. Required. + 2: optional bool eos + + // A batch of rows to return, if any exist. The number of rows in the batch + // should be less than or equal to the batch_size specified in TOpenParams. + 3: optional TExtRowBatch rows +} + +// Parameters to close() +struct TExtCloseParams { + // The opaque handle returned by the previous open() call. Always set. + 1: optional string scan_handle +} + +// Returned by close(). +struct TExtCloseResult { + 1: required Status.TStatus status +} + +// 这个data source可以认为是palo统一的外部data source的入口 +service TExtDataSourceService { + // 1. palo be call this api to send index, type, shard id to es + // 2. es will open a search context and prepare data, will return a context id + TExtOpenResult open(1: TExtOpenParams params); + // 1. palo be will send a search context id to es + // 2. es will find the search context and find a batch rows and send to palo + // 3. palo will run the remaining predicates when receving data + // 4. es should check the offset when receive the request + TExtGetNextResult getNext(1: TExtGetNextParams params); + // 1. es will release the context when receiving the data + TExtCloseResult close(1: TExtCloseParams params); +} \ No newline at end of file