diff --git a/CMakeLists.txt b/CMakeLists.txt index 11f19a87e..cfd75ddc5 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -435,6 +435,9 @@ include_directories(${OPENTELEMETRY_CPP_INCLUDE_DIR}) # See be/src/kudu/security/krb5_realm_override.cc for more information. set(KRB5_REALM_OVERRIDE -Wl,--undefined=krb5_realm_override_loaded krb5_realm_override) +# find Arrow headers and libs +find_package(Arrow REQUIRED) +IMPALA_ADD_THIRDPARTY_LIB(arrow ${ARROW_INCLUDE_DIR} ${ARROW_STATIC_LIB} "") ################################################################### # System dependencies diff --git a/be/CMakeLists.txt b/be/CMakeLists.txt index 88ffb0cc8..12af823db 100644 --- a/be/CMakeLists.txt +++ b/be/CMakeLists.txt @@ -528,6 +528,7 @@ set (IMPALA_LIBS ExecSequence ExecText ExecIcebergMetadata + ExecPaimon Exprs ExprsIr GlobalFlags @@ -681,7 +682,8 @@ set (IMPALA_DEPENDENCIES java_jvm kudu_client cctz - curl) + curl + arrow) # When building with Clang, linking fails because it is trying to # use a symbol in kudu_client, but that symbol is discarded. To diff --git a/be/src/exec/CMakeLists.txt b/be/src/exec/CMakeLists.txt index 441708a78..243705b13 100644 --- a/be/src/exec/CMakeLists.txt +++ b/be/src/exec/CMakeLists.txt @@ -25,6 +25,7 @@ add_subdirectory(rcfile) add_subdirectory(sequence) add_subdirectory(text) add_subdirectory(iceberg-metadata) +add_subdirectory(paimon) # where to put generated libraries set(LIBRARY_OUTPUT_PATH "${BUILD_OUTPUT_ROOT_DIRECTORY}/exec") diff --git a/be/src/exec/exec-node.cc b/be/src/exec/exec-node.cc index 78b8ff555..0a34ede5b 100644 --- a/be/src/exec/exec-node.cc +++ b/be/src/exec/exec-node.cc @@ -44,6 +44,7 @@ #include "exec/kudu/kudu-scan-node.h" #include "exec/kudu/kudu-util.h" #include "exec/nested-loop-join-node.h" +#include "exec/paimon/paimon-scan-plan-node.h" #include "exec/partial-sort-node.h" #include "exec/partitioned-hash-join-node.h" #include "exec/select-node.h" @@ -239,6 +240,9 @@ Status PlanNode::CreatePlanNode( case TPlanNodeType::ICEBERG_MERGE_NODE: *node = pool->Add(new IcebergMergePlanNode()); break; + case TPlanNodeType::PAIMON_SCAN_NODE: + *node = pool->Add(new PaimonScanPlanNode()); + break; default: map::const_iterator i = _TPlanNodeType_VALUES_TO_NAMES.find(tnode.node_type); @@ -432,6 +436,7 @@ void ExecNode::CollectScanNodes(vector* nodes) { CollectNodes(TPlanNodeType::HDFS_SCAN_NODE, nodes); CollectNodes(TPlanNodeType::HBASE_SCAN_NODE, nodes); CollectNodes(TPlanNodeType::KUDU_SCAN_NODE, nodes); + CollectNodes(TPlanNodeType::PAIMON_SCAN_NODE, nodes); } Status ExecNode::ExecDebugActionImpl(TExecNodePhase::type phase, RuntimeState* state) { diff --git a/be/src/exec/paimon/CMakeLists.txt b/be/src/exec/paimon/CMakeLists.txt new file mode 100644 index 000000000..f1bf79616 --- /dev/null +++ b/be/src/exec/paimon/CMakeLists.txt @@ -0,0 +1,108 @@ +# 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. + +# where to put generated libraries +set(LIBRARY_OUTPUT_PATH "${BUILD_OUTPUT_ROOT_DIRECTORY}/exec/paimon") + +# where to put generated binaries +set(EXECUTABLE_OUTPUT_PATH "${BUILD_OUTPUT_ROOT_DIRECTORY}/exec/paimon") + +add_library(ExecPaimon + paimon-jni-row-reader.cc + paimon-jni-scan-node.cc + paimon-jni-scanner.cc + paimon-scan-plan-node.cc +) + +# Below is a work-around to fix clang compile issue. For the normal build option, +# the compile will be successful. The issue will occur if CMAKE_BUILD_TYPE has +# following types: ADDRESS_SANITIZER, TIDY, UBSAN,UBSAN_FULL, TSAN, TSAN_FULL. +# For the normal build option, will invoke g++, however for the above build type, +# build script will use clang++ instead of g++ to compile, in this case, following +# error will occur: +# The detailed errors are: +# error: cannot cast private base class 'std::__detail::__variant::_Variant_storage +# , +# std::vector > >' to +# 'std::variant, +# std::vector > >' +# return static_cast&>(__rhs); +# +# +# looks like it hit bug: https://bugs.llvm.org/show_bug.cgi?id=31852 +# and the following patch is already applied: +# https://gcc.gnu.org/cgit/gcc/commit/?id=aafaa3254ec6fc3d5e3a15a40185950d3af04432 +# in the libc++ header of our toolchain. +# but looks like the patch above is only fixes the method get +# it doesn't fix the method static_cast. +# here we use the same work-ground for method static_cast. +# NOTE: the fix below only applies to current version of toolchain. +# need to adapt if toolchain is upgraded. +set(PAIMON_PATCH_GCC_HEADER_DIR + gcc-$ENV{IMPALA_GCC_VERSION}/include/c++/$ENV{IMPALA_GCC_VERSION}) + +set(PAIMON_PATCH_WORK_DIR + $ENV{IMPALA_TOOLCHAIN_PACKAGES_HOME}/${PAIMON_PATCH_GCC_HEADER_DIR}) + +set(PAIMON_PATCH_TARGET_FILE + ${PAIMON_PATCH_WORK_DIR}/variant) + +# change this if upgraded toolchain still haven't fix the issue. +set(PAIMON_PATCH_TARGET_FILE_CHECKSUM_ORIG + 4daf8153a09ee07bab2ac339e21b9725e17a40854a42284c85b3d2ba3c0862e3) + +set(PAIMON_PATCH_SENTINEL_FILE + ${PAIMON_PATCH_WORK_DIR}/variant_patched) +file(SHA256 ${PAIMON_PATCH_TARGET_FILE} + PAIMON_PATCH_TARGET_FILE_CHECKSUM_CURR) +message("Current Hash Is: " + ${PAIMON_PATCH_TARGET_FILE_CHECKSUM_CURR}) +message("Original Hash Is: " + ${PAIMON_PATCH_TARGET_FILE_CHECKSUM_ORIG}) + +if(PAIMON_PATCH_TARGET_FILE_CHECKSUM_ORIG STREQUAL PAIMON_PATCH_TARGET_FILE_CHECKSUM_CURR + ) + + message(STATUS "apply variant patch to fix clang++ static_cast issue.") + # To fix the compile error, the following tiny patch should be applied to + # toolchain-packages-gcc10.4.0/gcc-10.4.0/include/c++/10.4.0/variant + file(WRITE /tmp/variant.patch + "1296a1297,1299 + > #if defined(__clang__) && __clang_major__ <= 7 + > : public __detail::__variant::_Variant_base<_Types...>, + > #else + 1297a1301 + > #endif + ") + add_custom_command( + OUTPUT ${PAIMON_PATCH_SENTINEL_FILE} + COMMAND cd ${PAIMON_PATCH_WORK_DIR} && + patch -b -i /tmp/variant.patch ${PAIMON_PATCH_TARGET_FILE} && + touch ${PAIMON_PATCH_SENTINEL_FILE} + DEPENDS ${PAIMON_PATCH_TARGET_FILE} + COMMENT "Patching variant file in GCC include directory to fix static_cast issue." + VERBATIM + ) + add_custom_target(patch_variant + DEPENDS ${PAIMON_PATCH_SENTINEL_FILE} + COMMENT "Variant patch applied." + ) + add_dependencies(ExecPaimon patch_variant gen-deps) +else() + message(STATUS "skip apply patch since hash is changed.") + add_dependencies(ExecPaimon gen-deps) +endif() diff --git a/be/src/exec/paimon/paimon-jni-row-reader.cc b/be/src/exec/paimon/paimon-jni-row-reader.cc new file mode 100644 index 000000000..00990c537 --- /dev/null +++ b/be/src/exec/paimon/paimon-jni-row-reader.cc @@ -0,0 +1,315 @@ +// 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 "exec/paimon/paimon-jni-row-reader.h" + +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include "common/global-types.h" +#include "exec/exec-node.inline.h" +#include "exec/parquet/parquet-common.h" +#include "exec/read-write-util.h" +#include "gutil/walltime.h" +#include "runtime/collection-value-builder.h" +#include "runtime/decimal-value.h" +#include "runtime/runtime-state.h" +#include "runtime/timestamp-value.h" +#include "runtime/timestamp-value.inline.h" +#include "runtime/tuple-row.h" +#include "util/jni-util.h" + +namespace impala { + +PaimonJniRowReader::PaimonJniRowReader() {} + +Status PaimonJniRowReader::MaterializeTuple(const arrow::RecordBatch& recordBatch, + const int row_index, const TupleDescriptor* tuple_desc, Tuple* tuple, + MemPool* tuple_data_pool, RuntimeState* state) { + DCHECK(tuple != nullptr); + DCHECK(tuple_data_pool != nullptr); + DCHECK(tuple_desc != nullptr); + int col = 0; + DCHECK(recordBatch.num_columns() == tuple_desc->slots().size()); + for (const SlotDescriptor* slot_desc : tuple_desc->slots()) { + std::shared_ptr arr = recordBatch.column(col); + DCHECK(arr != nullptr); + RETURN_IF_ERROR( + WriteSlot(arr.get(), row_index, slot_desc, tuple, tuple_data_pool, state)); + col++; + } + return Status::OK(); +} + +template +Status PaimonJniRowReader::WriteSlot(const AT* arrow_array, int row_idx, void* slot) { + T value = arrow_array->Value(row_idx); + *reinterpret_cast(slot) = value; + return Status::OK(); +} + +template +Status PaimonJniRowReader::CastAndWriteSlot( + const arrow::Array* arrow_array, const int row_idx, void* slot) { + auto derived_array = static_cast(arrow_array); + return WriteSlot(derived_array, row_idx, slot); +} + +Status PaimonJniRowReader::WriteSlot(const arrow::Array* array, int row_index, + const SlotDescriptor* slot_desc, Tuple* tuple, MemPool* tuple_data_pool, + RuntimeState* state) { + if (array->IsNull(row_index)) { + tuple->SetNull(slot_desc->null_indicator_offset()); + return Status::OK(); + } + void* slot = tuple->GetSlot(slot_desc->tuple_offset()); + const ColumnType& type = slot_desc->type(); + switch (type.type) { + case TYPE_CHAR: { + RETURN_IF_ERROR(WriteVarCharOrCharSlot( + array, row_index, slot_desc->type().len, slot, tuple_data_pool)); + break; + } + case TYPE_STRING: + case TYPE_VARCHAR: { + if (type.IsBinaryType()) { // byte[] + RETURN_IF_ERROR(WriteStringOrBinarySlot( + array, row_index, slot, tuple_data_pool)); + } else { + RETURN_IF_ERROR(WriteStringOrBinarySlot( + array, row_index, slot, tuple_data_pool)); + } + break; + } + case TYPE_BOOLEAN: { + RETURN_IF_ERROR( + (CastAndWriteSlot(array, row_index, slot))); + break; + } + case TYPE_DATE: { + RETURN_IF_ERROR(WriteDateSlot(array, row_index, slot)); + break; + } + case TYPE_TINYINT: { + RETURN_IF_ERROR( + (CastAndWriteSlot(array, row_index, slot))); + break; + } + case TYPE_SMALLINT: { + RETURN_IF_ERROR( + (CastAndWriteSlot(array, row_index, slot))); + break; + } + case TYPE_INT: { + RETURN_IF_ERROR( + (CastAndWriteSlot(array, row_index, slot))); + break; + } + case TYPE_BIGINT: { + RETURN_IF_ERROR( + (CastAndWriteSlot(array, row_index, slot))); + break; + } + case TYPE_FLOAT: { + RETURN_IF_ERROR( + (CastAndWriteSlot(array, row_index, slot))); + break; + } + case TYPE_DOUBLE: { + RETURN_IF_ERROR( + (CastAndWriteSlot(array, row_index, slot))); + break; + } + case TYPE_DECIMAL: { + RETURN_IF_ERROR(WriteDecimalSlot(array, row_index, type, slot)); + break; + } + case TYPE_TIMESTAMP: { + RETURN_IF_ERROR( + WriteTimeStampSlot(array, state->local_time_zone(), row_index, slot)); + break; + } + case TYPE_STRUCT: { + // TODO: implement struct type support later + tuple->SetNull(slot_desc->null_indicator_offset()); + break; + } + case TYPE_ARRAY: { + // TODO: implement array type support later + tuple->SetNull(slot_desc->null_indicator_offset()); + break; + } + case TYPE_MAP: { + // TODO: implement map type support later + tuple->SetNull(slot_desc->null_indicator_offset()); + break; + } + default: + DCHECK(false) << "Unsupported column type: " << slot_desc->type().type; + tuple->SetNull(slot_desc->null_indicator_offset()); + } + return Status::OK(); +} + +Status PaimonJniRowReader::WriteDateSlot( + const arrow::Array* array, const int row_idx, void* slot) { + const arrow::Date32Array* date_array = static_cast(array); + int32_t days_since_epoch = date_array->Value(row_idx); + + // This will set the value to DateValue::INVALID_DAYS_SINCE_EPOCH if it is out of + // range. + DateValue result(days_since_epoch); + *reinterpret_cast(slot) = result.Value(); + return Status::OK(); +} + +// Sets the decimal value in the slot. Inline method to avoid nested switch statements. +static const std::string ERROR_INVALID_DECIMAL = "Invalid Decimal Format"; +inline Status SetDecimalVal( + const ColumnType& type, const uint8_t* buffer, int len, void* slot) { + switch (type.GetByteSize()) { + case 4: { + Decimal4Value* val = reinterpret_cast(slot); + if (UNLIKELY( + (ParquetPlainEncoder::Decode(buffer, buffer + len, len, val)) + < 0)) { + return Status(ERROR_INVALID_DECIMAL); + } + break; + } + case 8: { + Decimal8Value* val = reinterpret_cast(slot); + if (UNLIKELY( + (ParquetPlainEncoder::Decode(buffer, buffer + len, len, val)) + < 0)) { + return Status(ERROR_INVALID_DECIMAL); + } + break; + } + case 16: { + Decimal16Value* val = reinterpret_cast(slot); + if (UNLIKELY( + (ParquetPlainEncoder::Decode(buffer, buffer + len, len, val)) + < 0)) { + return Status(ERROR_INVALID_DECIMAL); + } + break; + } + default: + DCHECK(false); + } + return Status::OK(); +} + +Status PaimonJniRowReader::WriteDecimalSlot( + const arrow::Array* array, const int row_idx, const ColumnType& type, void* slot) { + const arrow::BinaryArray* binary_array = static_cast(array); + int byte_length = 0; + const uint8_t* data = binary_array->GetValue(row_idx, &byte_length); + DCHECK(byte_length > 0 && byte_length <= 16); + return SetDecimalVal(type, data, byte_length, slot); +} + +Status PaimonJniRowReader::WriteTimeStampSlot( + const arrow::Array* array, const Timezone* timezone, const int row_idx, void* slot) { + const arrow::TimestampArray* date_array = (const arrow::TimestampArray*)array; + int64_t value = date_array->Value(row_idx); + const auto& type = static_cast(*date_array->type()); + const std::string& tz_name = type.timezone(); + const Timezone* tz = tz_name.empty() ? UTCPTR : timezone; + switch (type.unit()) { + case ::arrow::TimeUnit::NANO: + *reinterpret_cast(slot) = TimestampValue::FromUnixTimeNanos( + value / NANOS_PER_SEC, value % NANOS_PER_SEC, tz); + break; + case ::arrow::TimeUnit::MICRO: + *reinterpret_cast(slot) = + TimestampValue::FromUnixTimeMicros(value, tz); + break; + case ::arrow::TimeUnit::MILLI: + *reinterpret_cast(slot) = + TimestampValue::FromUnixTimeMicros(value * 1000L, tz); + break; + case ::arrow::TimeUnit::SECOND: + *reinterpret_cast(slot) = TimestampValue::FromUnixTime(value, tz); + break; + } + + return Status::OK(); +} + +template +Status PaimonJniRowReader::WriteVarCharOrCharSlot(const arrow::Array* array, + const int row_idx, int dst_len, void* slot, MemPool* tuple_data_pool) { + const arrow::StringArray* nchar_array = static_cast(array); + std::string_view v = nchar_array->Value(row_idx); + + int src_len = v.size(); + int unpadded_len = std::min(dst_len, src_len); + // Allocate memory and copy the bytes from the JVM to the RowBatch. + char* dst_char = reinterpret_cast(slot); + memcpy(dst_char, v.data(), unpadded_len); + StringValue::PadWithSpaces(dst_char, dst_len, unpadded_len); + return Status::OK(); +} + +/// Obtain bytes from arrow string/binary batch first, Then the data has to be copied +/// to the tuple_data_pool, because the Fe PaimonJniScanner releases the JVM offheap +/// memory later. +template +Status PaimonJniRowReader::WriteStringOrBinarySlot( + const arrow::Array* array, const int row_idx, void* slot, MemPool* tuple_data_pool) { + std::string_view v; + uint32_t jbuffer_size = 0; + if constexpr (IS_BINARY) { + const arrow::BinaryArray* binary_array = + static_cast(array); + v = binary_array->Value(row_idx); + } else { + const arrow::StringArray* string_array = + static_cast(array); + v = string_array->Value(row_idx); + } + + jbuffer_size = v.size(); + // Allocate memory and copy the bytes from the JVM to the RowBatch. + char* buffer = + reinterpret_cast(tuple_data_pool->TryAllocateUnaligned(jbuffer_size)); + if (UNLIKELY(buffer == nullptr)) { + string details = strings::Substitute("Failed to allocate $0 bytes for $1.", + jbuffer_size, IS_BINARY ? "binary" : "string"); + return tuple_data_pool->mem_tracker()->MemLimitExceeded( + nullptr, details, jbuffer_size); + } + + memcpy(buffer, v.data(), jbuffer_size); + reinterpret_cast(slot)->Assign(buffer, jbuffer_size); + return Status::OK(); +} +} // namespace impala diff --git a/be/src/exec/paimon/paimon-jni-row-reader.h b/be/src/exec/paimon/paimon-jni-row-reader.h new file mode 100644 index 000000000..9464f00ba --- /dev/null +++ b/be/src/exec/paimon/paimon-jni-row-reader.h @@ -0,0 +1,82 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#pragma once + +#include "common/global-types.h" +#include "common/status.h" +#include "runtime/types.h" + +#include + +namespace impala { + +class MemPool; +class RuntimeState; +class ScanNode; +class Status; +class SlotDescriptor; +class Tuple; +class TupleDescriptor; + +/// Row reader for Paimon table scans, it translates a row of arrow RecordBatch to +/// Impala row batch tuples. +class PaimonJniRowReader { + public: + PaimonJniRowReader(); + + /// Materialize the Arrow batch into Impala rows. + Status MaterializeTuple(const arrow::RecordBatch& recordBatch, const int row_idx, + const TupleDescriptor* tuple_desc, Tuple* tuple, MemPool* tuple_data_pool, + RuntimeState* state); + + private: + // Writes a row denoted by 'row_idx' from an arrow batch into the target tuple. + Status WriteSlot(const arrow::Array* array, const int row_idx, + const SlotDescriptor* slot_desc, Tuple* tuple, MemPool* tuple_data_pool, + RuntimeState* state) WARN_UNUSED_RESULT; + + /// Template method that writes a value from 'arrow_array' to 'slot'. + /// 'T' is the type of the slot, + /// 'AT' is the proper subtype of the arrow::Array. + template + Status WriteSlot(const AT* arrow_array, int row_idx, void* slot) WARN_UNUSED_RESULT; + + template + Status CastAndWriteSlot( + const arrow::Array* arrow_array, const int row_idx, void* slot) WARN_UNUSED_RESULT; + + Status WriteDateSlot( + const arrow::Array* array, const int row_idx, void* slot) WARN_UNUSED_RESULT; + + Status WriteDecimalSlot(const arrow::Array* array, const int row_idx, + const ColumnType& type, void* slot) WARN_UNUSED_RESULT; + + /// Paimon TimeStamp is parsed into TimestampValue. + Status WriteTimeStampSlot(const arrow::Array* array, const Timezone* timezone, + const int row_idx, void* slot) WARN_UNUSED_RESULT; + + template + Status WriteStringOrBinarySlot(const arrow::Array* array, const int row_idx, void* slot, + MemPool* tuple_data_pool) WARN_UNUSED_RESULT; + + template + Status WriteVarCharOrCharSlot(const arrow::Array* array, const int row_idx, int max_len, + void* slot, MemPool* tuple_data_pool) WARN_UNUSED_RESULT; +}; + +} // namespace impala diff --git a/be/src/exec/paimon/paimon-jni-scan-node.cc b/be/src/exec/paimon/paimon-jni-scan-node.cc new file mode 100644 index 000000000..63cb689a4 --- /dev/null +++ b/be/src/exec/paimon/paimon-jni-scan-node.cc @@ -0,0 +1,267 @@ +// 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 "exec/paimon/paimon-jni-scan-node.h" +#include "common/status.h" +#include "exec/exec-node-util.h" +#include "exec/exec-node.inline.h" +#include "exec/paimon/paimon-jni-row-reader.h" +#include "exec/paimon/paimon-jni-scanner.h" +#include "exec/paimon/paimon-scan-plan-node.h" +#include "rpc/thrift-util.h" +#include "runtime/exec-env.h" +#include "runtime/runtime-state.h" +#include "runtime/tuple-row.h" +#include "util/jni-util.h" +#include "util/periodic-counter-updater.h" +#include "util/runtime-profile-counters.h" + +#include +#include +#include +#include +using namespace impala; + +PaimonJniScanNode::PaimonJniScanNode( + ObjectPool* pool, const PaimonScanPlanNode& pnode, const DescriptorTbl& descs) + : ScanNode(pool, pnode, descs), + tuple_id_(pnode.tnode_->paimon_table_scan_node.tuple_id), + table_name_(pnode.tnode_->paimon_table_scan_node.table_name), + splits_empty_(false), + paimon_last_arrow_record_batch_consumed_bytes_(0), + arrow_record_batch_row_count_(0), + arrow_record_batch_row_index_(0) {} + +Status PaimonJniScanNode::Prepare(RuntimeState* state) { + RETURN_IF_ERROR(ScanNode::Prepare(state)); + DCHECK(scan_range_params_ != NULL) + << "Must call SetScanRanges() before calling Prepare()"; + + scan_open_timer_ = ADD_TIMER(runtime_profile(), "ScanOpenTime"); + paimon_api_scan_timer_ = ADD_TIMER(runtime_profile(), "PaimonApiScanTime"); + + tuple_desc_ = state->desc_tbl().GetTupleDescriptor(tuple_id_); + if (tuple_desc_ == nullptr) { + return Status( + "Failed to get tuple descriptor, tuple id: " + std::to_string(tuple_id_)); + } + arrow_batch_mem_tracker_.reset( + new MemTracker(-1, "Arrow Batch", this->mem_tracker(), false)); + /// Construct the jni scan param, the param will be used in PaimonJniScanner. + paimon_jni_scan_param_.__set_paimon_table_obj( + plan_node_.tnode_->paimon_table_scan_node.paimon_table_obj); + /// update projection id, will get the top-level field ids of each tuple. + std::vector field_ids; + RETURN_IF_ERROR(CollectProjectionFieldIds(tuple_desc_, field_ids)); + paimon_jni_scan_param_.__set_projection(field_ids); + LOG(INFO) << table_name_ << " Contains " << field_ids.size() << " field ids." + << std::endl; + paimon_jni_scan_param_.__set_mem_limit_bytes( + arrow_batch_mem_tracker_->GetLowestLimit(MemLimit::HARD)); + paimon_jni_scan_param_.__set_batch_size(state->batch_size()); + paimon_jni_scan_param_.__set_fragment_id(state->fragment_instance_id()); + std::vector scan_range_vector; + for (const ScanRangeParamsPB& params : *scan_range_params_) { + DCHECK(params.scan_range().has_file_metadata()); + const std::string& split = params.scan_range().file_metadata(); + scan_range_vector.push_back(split); + } + paimon_jni_scan_param_.__set_splits(scan_range_vector); + /// Check if splits is empty + splits_empty_ = scan_range_vector.empty(); + impala::ThriftSerializer serializer(false); + /// serialize the jni scan param to binary. + RETURN_IF_ERROR(serializer.SerializeToString( + &paimon_jni_scan_param_, &paimon_jni_scan_param_serialized_)); + return Status::OK(); +} + +Status PaimonJniScanNode::Open(RuntimeState* state) { + SCOPED_TIMER(scan_open_timer_); + RETURN_IF_ERROR(ScanNode::Open(state)); + /// Skip if splits is empty. + if (splits_empty_) { + return Status::OK(); + } + JNIEnv* env = JniUtil::GetJNIEnv(); + if (env == nullptr) return Status("Failed to get/create JVM"); + jni_scanner_.reset( + new PaimonJniScanner(paimon_jni_scan_param_serialized_, tuple_desc_, table_name_)); + RETURN_IF_ERROR(jni_scanner_->Init(env)); + paimon_row_reader_.reset(new PaimonJniRowReader()); + SCOPED_TIMER(paimon_api_scan_timer_); + RETURN_IF_ERROR(jni_scanner_->ScanTable(env)); + return Status::OK(); +} + +Status PaimonJniScanNode::CollectProjectionFieldIds( + const TupleDescriptor* tuple_desc_, vector& projection) { + for (const SlotDescriptor* slot_desc : tuple_desc_->slots()) { + int field_id = -1; + if (slot_desc->col_path().size() == 1) { + // Top level slots have ColumnDescriptors that store the field ids. + field_id = tuple_desc_->table_desc()->GetColumnDesc(slot_desc).field_id(); + } else { + // TODO: support the nested field later. + return Status("Paimon Scanner currently doesn't support nested type now"); + } + DCHECK_NE(field_id, -1); + projection.push_back(field_id); + } + return Status::OK(); +} + +Status PaimonJniScanNode::GetNextBatchIfNeeded(bool* is_empty_batch) { + /// Check if we need to fetch next arrow record batch from jni. if yes, + /// fetch it. + if (paimon_arrow_record_batch_holder_ == nullptr + || arrow_record_batch_row_index_ >= arrow_record_batch_row_count_) { + SCOPED_TIMER(paimon_api_scan_timer_); + JNIEnv* env = JniUtil::GetJNIEnv(); + struct ArrowArray* array; + struct ArrowSchema* schema; + long row_count; + long offheap_consumed_bytes; + + DCHECK(is_empty_batch != nullptr); + + RETURN_IF_ERROR(jni_scanner_->GetNextBatchDirect( + env, &array, &schema, &row_count, &offheap_consumed_bytes)); + + *is_empty_batch = false; + if (row_count > 0) { + auto resultImportVectorSchemaRoot = arrow::ImportRecordBatch(array, schema); + /// since the result type is arrow::Status, need to check status manually + /// without using macro RETURN_IF_ERROR + if (!resultImportVectorSchemaRoot.ok()) { + return Status(resultImportVectorSchemaRoot.status().message()); + } + paimon_arrow_record_batch_holder_ = resultImportVectorSchemaRoot.ValueUnsafe(); + arrow_record_batch_row_count_ = row_count; + DCHECK_EQ(row_count, paimon_arrow_record_batch_holder_->num_rows()); + arrow_record_batch_row_index_ = 0; + /// update allocated offheap memory reported from Jni. + /// need to do for each batch since we need to check the + /// memory usage hit the limit of mem tracker. + OffheapTrackFree(); + RETURN_IF_ERROR(OffheapTrackAllocation(offheap_consumed_bytes)); + + } else { + /// No more batches to fetch. + paimon_arrow_record_batch_holder_ = nullptr; + arrow_record_batch_row_count_ = 0; + arrow_record_batch_row_index_ = 0; + OffheapTrackFree(); + } + } + + *is_empty_batch = arrow_record_batch_row_count_ > 0 + && arrow_record_batch_row_index_ < arrow_record_batch_row_count_; + + return Status::OK(); +} + +Status PaimonJniScanNode::GetNext(RuntimeState* state, RowBatch* row_batch, bool* eos) { + SCOPED_TIMER(runtime_profile_->total_time_counter()); + ScopedGetNextEventAdder ea(this, eos); + RETURN_IF_ERROR(ExecDebugAction(TExecNodePhase::GETNEXT, state)); + RETURN_IF_CANCELLED(state); + RETURN_IF_ERROR(QueryMaintenance(state)); + + /// Return eos if empty splits or reached limit. + if (splits_empty_ || ReachedLimit()) { + *eos = true; + return Status::OK(); + } + + /// Set eos to false initially for the batch. + *eos = false; + + /// Allocate buffer for RowBatch and init the tuple + uint8_t* tuple_buffer; + int64_t tuple_buffer_size; + RETURN_IF_ERROR( + row_batch->ResizeAndAllocateTupleBuffer(state, &tuple_buffer_size, &tuple_buffer)); + Tuple* tuple = reinterpret_cast(tuple_buffer); + tuple->Init(tuple_buffer_size); + + SCOPED_TIMER(materialize_tuple_timer()); + while (!ReachedLimit() && !row_batch->AtCapacity()) { + /// Break the loop if is canceled, or maintainance is needed. + if (state->is_cancelled() || !QueryMaintenance(state).ok()) { + break; + } + int row_idx = row_batch->AddRow(); + TupleRow* tuple_row = row_batch->GetRow(row_idx); + tuple_row->SetTuple(0, tuple); + + /// Get the next arrow batch from 'org.apache.impala.util.paimon.PaimonJniScanner' + /// if the current arrow batch is already consumed. + bool fetched; + RETURN_IF_ERROR(GetNextBatchIfNeeded(&fetched)); + /// When fetched is false, there are no more arrow batches to read + if (!fetched) { + *eos = true; + break; + } + DCHECK(paimon_arrow_record_batch_holder_ != nullptr); + COUNTER_ADD(rows_read_counter(), 1); + RETURN_IF_ERROR(paimon_row_reader_->MaterializeTuple( + *paimon_arrow_record_batch_holder_, arrow_record_batch_row_index_, tuple_desc_, + tuple, row_batch->tuple_data_pool(), state)); + /// Evaluate conjuncts on this tuple row + if (ExecNode::EvalConjuncts( + conjunct_evals().data(), conjunct_evals().size(), tuple_row)) { + row_batch->CommitLastRow(); + tuple = reinterpret_cast( + reinterpret_cast(tuple) + tuple_desc_->byte_size()); + IncrementNumRowsReturned(1); + } else { + /// Reset the null bits, everyhing else will be overwritten + Tuple::ClearNullBits( + tuple, tuple_desc_->null_bytes_offset(), tuple_desc_->num_null_bytes()); + } + /// will process the next row of arrow RecordBatch in next iteration. + arrow_record_batch_row_index_++; + } + if (ReachedLimit()) { + *eos = true; + } + return Status::OK(); +} + +void PaimonJniScanNode::Close(RuntimeState* state) { + if (is_closed()) return; + /// Eagerly release arrow batch before calling + /// frontend method close, in case of + /// mem leak. leak was observed when query was + /// canceled. + if (paimon_arrow_record_batch_holder_ != nullptr) { + paimon_arrow_record_batch_holder_.reset(); + arrow_record_batch_row_index_ = 0; + arrow_record_batch_row_count_ = 0; + } + /// Close jni scanner if splits is not empty. + if (!splits_empty_) { + jni_scanner_->Close(state); + } + OffheapTrackFree(); + if (arrow_batch_mem_tracker_ != nullptr) { + arrow_batch_mem_tracker_->Close(); + } + ScanNode::Close(state); +} diff --git a/be/src/exec/paimon/paimon-jni-scan-node.h b/be/src/exec/paimon/paimon-jni-scan-node.h new file mode 100644 index 000000000..9ff10f3eb --- /dev/null +++ b/be/src/exec/paimon/paimon-jni-scan-node.h @@ -0,0 +1,154 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#pragma once + +#include "common/global-types.h" +#include "exec/paimon/paimon-jni-row-reader.h" +#include "exec/paimon/paimon-jni-scanner.h" +#include "exec/scan-node.h" +#include "runtime/descriptors.h" + +#include +#include + +namespace impala { + +class ExecNode; +class PaimonJniRowReader; +class RuntimeState; +class Status; + +/// Scan node for an Paimon table. +/// Since Paimon API can be used to scan both paimon data tables and metadata tables. +/// The current jni scanner works as a generic solution to scan both data table and +/// metadata tables. +/// For scanning these paimon tables this scanner calls into the JVM and creates an +/// 'PaimonJniScanner' object that does the scanning. Once the Paimon scan is done, +/// To minimize the jni overhead, the PaimonJniScanner will first write batch of +/// InternalRows to the arrow BatchRecord into the offheap, and the offheap memory +/// pointer will pass to the native side to directly read the batch record and +/// convert the arrow format into native impala RowBatch. The benchmark shows 2.x +/// better performance than pure jni implementation. +/// +/// The flow of scanning is: +/// 1. Backend: Get the splits/table obj from plan node, generate thrift +/// encoded param and passes to the JNI scanner. +/// 2. Backend: Creates an PaimonJniScanner object on the Java heap. +/// 3. Backend: Triggers a table scan on the Frontend +/// 4. Frontend: Executes the scan +/// 5. Backend: Calls GetNext that calls PaimonJniScanner's GetNext +/// 6. Frontend: PaimonJniScanner's GetNextBatchDirect will return the offheap pointer, +/// as well as consumed offheap bytes size to the arrow row batch. +/// 7. Backend: Consume the arrow RecordBatch into impala RowBatch. +/// +/// Note: +/// This scan node can be executed on any executor. + +class PaimonScanPlanNode; +class PaimonJniScanNode : public ScanNode { + public: + PaimonJniScanNode( + ObjectPool* pool, const PaimonScanPlanNode& pnode, const DescriptorTbl& descs); + + /// Initializes counters, executes Paimon table scan and initializes accessors. + Status Prepare(RuntimeState* state) override; + + /// Creates the Paimon row reader. + Status Open(RuntimeState* state) override; + + /// Fills the next rowbatch with the results returned by the Paimon scan. + Status GetNext(RuntimeState* state, RowBatch* row_batch, bool* eos) override; + + /// Finalize and close this operator. + void Close(RuntimeState* state) override; + + protected: + Status CollectProjectionFieldIds( + const TupleDescriptor* tuple_desc, vector& projection); + Status OffheapTrackAllocation(long bytes); + void OffheapTrackFree(); + + private: + /// Adapter that helps preparing the table and executes an Paimon table scan + /// on Java side. Allows the ScanNode to fetch the Arrow RecordBatch from the + /// Java Heap. + std::unique_ptr jni_scanner_; + + /// Helper class to transform Paimon rows to Impala tuples. + std::unique_ptr paimon_row_reader_; + + /// Get the next arrow row record batch from the jni scanner. + /// returns false if no record is available anymore, true if valid row is returned. + Status GetNextBatchIfNeeded(bool* is_empty_batch); + + /// The TupleId and TupleDescriptor of the tuple that this scan node will populate. + const TupleId tuple_id_; + const TupleDescriptor* tuple_desc_ = nullptr; + + /// The table name. + const std::string table_name_; + + // Paimon scan param. + TPaimonJniScanParam paimon_jni_scan_param_; + /// Indicate whether the splits is empty. + /// It is used to control whether the + /// jni scanner should be created. If splits is empty, + /// it will bypass the JNI operation, in other words, + /// all jni related operation will be skipped,directly + /// set eof flag to true, and return empty scan + /// result. + bool splits_empty_; + /// MemTracker for tracing arrow used JVM offheap memory. + /// Initialized in Prepare(). Owned by RuntimeState. + std::unique_ptr arrow_batch_mem_tracker_; + /// last consumed offheap bytes for arrow batch + long paimon_last_arrow_record_batch_consumed_bytes_; + /// Thrift serialized paimon scan param + std::string paimon_jni_scan_param_serialized_; + /// current unconsumed arrow record batch. + std::shared_ptr paimon_arrow_record_batch_holder_; + /// current row_count of the arrow record batch. + long arrow_record_batch_row_count_; + /// current row_index of the arrow record batch. + long arrow_record_batch_row_index_; + /// Paimon scan specific counters. + RuntimeProfile::Counter* scan_open_timer_; + RuntimeProfile::Counter* paimon_api_scan_timer_; +}; + +inline Status PaimonJniScanNode::OffheapTrackAllocation(long consumed_bytes) { + if (consumed_bytes > 0) { + if (arrow_batch_mem_tracker_->TryConsume(consumed_bytes)) { + paimon_last_arrow_record_batch_consumed_bytes_ = consumed_bytes; + return Status::OK(); + } else { + return Status::MemLimitExceeded("Arrow batch size exceed the mem limit."); + } + } else { + return Status::OK(); + } +} + +inline void PaimonJniScanNode::OffheapTrackFree() { + if (paimon_last_arrow_record_batch_consumed_bytes_ > 0) { + arrow_batch_mem_tracker_->Release(paimon_last_arrow_record_batch_consumed_bytes_); + paimon_last_arrow_record_batch_consumed_bytes_ = 0; + } +} + +} // namespace impala diff --git a/be/src/exec/paimon/paimon-jni-scanner.cc b/be/src/exec/paimon/paimon-jni-scanner.cc new file mode 100644 index 000000000..460ff9930 --- /dev/null +++ b/be/src/exec/paimon/paimon-jni-scanner.cc @@ -0,0 +1,108 @@ +// 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 "exec/paimon/paimon-jni-scanner.h" +#include +#include "util/jni-util.h" + +namespace impala { + +PaimonJniScanner::PaimonJniScanner(const std::string& scan_param, + const TupleDescriptor* tuple_desc, const std::string& table_name) + : paimon_scan_param_(scan_param), tuple_desc_(tuple_desc), table_name_(table_name) {} + +Status PaimonJniScanner::InitJNI() { + DCHECK(paimon_jni_scanner_cl_ == nullptr) << "InitJNI() already called!"; + JNIEnv* env = JniUtil::GetJNIEnv(); + if (env == nullptr) return Status("Failed to get/create JVM"); + + // Global class references: + RETURN_IF_ERROR(JniUtil::GetGlobalClassRef( + env, "org/apache/impala/util/paimon/PaimonJniScanner", &paimon_jni_scanner_cl_)); + + // Method ids: + RETURN_IF_ERROR(JniUtil::GetMethodID( + env, paimon_jni_scanner_cl_, "", "([B)V", &paimon_jni_scanner_ctor_)); + RETURN_IF_ERROR(JniUtil::GetMethodID( + env, paimon_jni_scanner_cl_, "ScanTable", "()V", &paimon_jni_scanner_scan_table_)); + RETURN_IF_ERROR(JniUtil::GetMethodID(env, paimon_jni_scanner_cl_, "GetNextBatch", + "([J)J", &paimon_jni_scanner_get_next_)); + RETURN_IF_ERROR(JniUtil::GetMethodID( + env, paimon_jni_scanner_cl_, "close", "()V", &paimon_jni_scanner_close_)); + + return Status::OK(); +} + +Status PaimonJniScanner::Init(JNIEnv* env) { + jbyteArray jbytes_scan_param = env->NewByteArray(paimon_scan_param_.size()); + RETURN_ERROR_IF_EXC(env); + env->SetByteArrayRegion(jbytes_scan_param, 0, paimon_scan_param_.size(), + reinterpret_cast(paimon_scan_param_.data())); + RETURN_ERROR_IF_EXC(env); + jobject j_jni_scanner = + env->NewObject(paimon_jni_scanner_cl_, paimon_jni_scanner_ctor_, jbytes_scan_param); + RETURN_ERROR_IF_EXC(env); + RETURN_IF_ERROR(JniUtil::LocalToGlobalRef(env, j_jni_scanner, &j_jni_scanner_)); + return Status::OK(); +} + +Status PaimonJniScanner::ScanTable(JNIEnv* env) { + env->CallObjectMethod(j_jni_scanner_, paimon_jni_scanner_scan_table_); + RETURN_ERROR_IF_EXC(env); + return Status::OK(); +} + +Status PaimonJniScanner::GetNextBatchDirect(JNIEnv* env, struct ArrowArray** array, + struct ArrowSchema** schema, long* rows, long* offheap_used) { + /// Will pass a long array to java method, and the returned are two memory address. + /// 1st is the schema memory address, the second is the memory address of arrow + /// array vector. the two memory address are in the offheap region of JVM, + /// and the offheap usage in bytes. + jlongArray address_array = env->NewLongArray(3); + RETURN_ERROR_IF_EXC(env); + jlong result = + env->CallLongMethod(j_jni_scanner_, paimon_jni_scanner_get_next_, address_array); + RETURN_ERROR_IF_EXC(env); + jlong values[3]; + env->GetLongArrayRegion(address_array, 0, 3, &values[0]); + *schema = (struct ArrowSchema*)values[0]; + *array = (struct ArrowArray*)values[1]; + *offheap_used = values[2]; + *rows = result; + env->DeleteLocalRef(address_array); + return Status::OK(); +} + +void PaimonJniScanner::Close(RuntimeState* state) { + JNIEnv* env = JniUtil::GetJNIEnv(); + if (env != nullptr) { + if (j_jni_scanner_ != nullptr) { + /// Call close method to free resources of java PaimonJniScanner. + env->CallObjectMethod(j_jni_scanner_, paimon_jni_scanner_close_); + env->DeleteGlobalRef(j_jni_scanner_); + } + } +} + +string PaimonJniScanner::DebugString() { + std::stringstream out; + out << "PaimonJniScanner: [ Paimon table name: " << table_name_ << "; "; + out << tuple_desc_->DebugString() << "]"; + return out.str(); +} + +} // namespace impala diff --git a/be/src/exec/paimon/paimon-jni-scanner.h b/be/src/exec/paimon/paimon-jni-scanner.h new file mode 100644 index 000000000..6684a6c35 --- /dev/null +++ b/be/src/exec/paimon/paimon-jni-scanner.h @@ -0,0 +1,77 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#pragma once + +#include "common/status.h" +#include "runtime/descriptors.h" + +#include +#include +#include + +namespace impala { + +class RuntimeState; + +/// Adapter class of the FE PaimonJniScanner, wraps the JNI calls as C++ methods. +class PaimonJniScanner { + public: + PaimonJniScanner(const std::string& scan_param, const TupleDescriptor* tuple_desc, + const std::string& table_name); + + /// JNI setup. Creates global references for Java classes and finds method ids. + /// Initializes static members, should be called once per process lifecycle. + static Status InitJNI() WARN_UNUSED_RESULT; + + // Initializes this object, creates the java metadata scanner object. + Status Init(JNIEnv* env) WARN_UNUSED_RESULT; + + /// Executes an Paimon scan through JNI. + Status ScanTable(JNIEnv* env) WARN_UNUSED_RESULT; + + /// Gets the next arrow batch from 'org.apache.impala.util.paimon.PaimonJniScanner'. + Status GetNextBatchDirect(JNIEnv* env, struct ArrowArray** array, + struct ArrowSchema** schema, long* rows, long* offheap_used) WARN_UNUSED_RESULT; + + /// Removes global references. + void Close(RuntimeState* state); + + private: + /// Global class references created with JniUtil. + inline static jclass paimon_jni_scanner_cl_ = nullptr; + + /// Method references created with JniUtil. + inline static jmethodID paimon_jni_scanner_ctor_ = nullptr; + inline static jmethodID paimon_jni_scanner_scan_table_ = nullptr; + inline static jmethodID paimon_jni_scanner_get_next_ = nullptr; + inline static jmethodID paimon_jni_scanner_close_ = nullptr; + + /// The Paimon table scan parameters. + const std::string& paimon_scan_param_; + /// Top level TupleDescriptor. + const TupleDescriptor* tuple_desc_; + /// metastore table name + const std::string& table_name_; + /// Paimon scanner Java object, it helps preparing the table and + /// executes an Paimon table scan. Allows the ScanNode to fetch the row batch from + /// the Java Off Heap. + jobject j_jni_scanner_; + + std::string DebugString(); +}; +} // namespace impala diff --git a/be/src/exec/paimon/paimon-scan-plan-node.cc b/be/src/exec/paimon/paimon-scan-plan-node.cc new file mode 100644 index 000000000..cb3738761 --- /dev/null +++ b/be/src/exec/paimon/paimon-scan-plan-node.cc @@ -0,0 +1,31 @@ +// 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 "common/status.h" +#include "exec/paimon/paimon-scan-plan-node.h" +#include "exec/paimon/paimon-jni-scan-node.h" +#include "runtime/exec-env.h" +#include "runtime/runtime-state.h" +#include "util/jni-util.h" + +using namespace impala; + +Status PaimonScanPlanNode::CreateExecNode(RuntimeState* state, ExecNode** node) const { + ObjectPool* pool = state->obj_pool(); + *node = pool->Add(new PaimonJniScanNode(pool, *this, state->desc_tbl())); + return Status::OK(); +} \ No newline at end of file diff --git a/be/src/exec/paimon/paimon-scan-plan-node.h b/be/src/exec/paimon/paimon-scan-plan-node.h new file mode 100644 index 000000000..28bccbeab --- /dev/null +++ b/be/src/exec/paimon/paimon-scan-plan-node.h @@ -0,0 +1,36 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#pragma once + +#include +#include "exec/scan-node.h" + +namespace impala { + +class ExecNode; +class RuntimeState; +class ScanPlanNode; +class Status; + +class PaimonScanPlanNode : public ScanPlanNode { + public: + Status CreateExecNode(RuntimeState* state, ExecNode** node) const override; + ~PaimonScanPlanNode() {} +}; + +} // namespace impala diff --git a/be/src/runtime/descriptors.cc b/be/src/runtime/descriptors.cc index 0227cc88a..d7de726d1 100644 --- a/be/src/runtime/descriptors.cc +++ b/be/src/runtime/descriptors.cc @@ -368,6 +368,16 @@ string SystemTableDescriptor::DebugString() const { return out.str(); } +PaimonTableDescriptor::PaimonTableDescriptor(const TTableDescriptor& tdesc) + : TableDescriptor(tdesc), paimon_api_table_(tdesc.paimonTable) {} + +string PaimonTableDescriptor::DebugString() const { + stringstream out; + out << "PaimonTable(" << TableDescriptor::DebugString(); + out << ")"; + return out.str(); +} + TupleDescriptor::TupleDescriptor(const TTupleDescriptor& tdesc) : id_(tdesc.id), byte_size_(tdesc.byteSize), @@ -626,6 +636,9 @@ Status DescriptorTbl::CreateTblDescriptorInternal(const TTableDescriptor& tdesc, case TTableType::SYSTEM_TABLE: *desc = pool->Add(new SystemTableDescriptor(tdesc)); break; + case TTableType::PAIMON_TABLE: + *desc = pool->Add(new PaimonTableDescriptor(tdesc)); + break; default: DCHECK(false) << "invalid table type: " << tdesc.tableType; } diff --git a/be/src/runtime/descriptors.h b/be/src/runtime/descriptors.h index b0962b891..071c32a17 100644 --- a/be/src/runtime/descriptors.h +++ b/be/src/runtime/descriptors.h @@ -604,6 +604,16 @@ class SystemTableDescriptor : public TableDescriptor { TSystemTableName::type table_name_; }; +// Descriptor for a Paimon Table +class PaimonTableDescriptor : public TableDescriptor { + public: + PaimonTableDescriptor(const TTableDescriptor& tdesc); + virtual std::string DebugString() const; + + private: + impala::TPaimonTable paimon_api_table_; +}; + class TupleDescriptor { public: int byte_size() const { return byte_size_; } diff --git a/be/src/scheduling/scheduler.cc b/be/src/scheduling/scheduler.cc index 71d9cd796..8f59dd8ce 100644 --- a/be/src/scheduling/scheduler.cc +++ b/be/src/scheduling/scheduler.cc @@ -70,7 +70,7 @@ static const string SCHEDULER_WARNING_KEY("Scheduler Warning"); static const vector SCAN_NODE_TYPES{TPlanNodeType::HDFS_SCAN_NODE, TPlanNodeType::HBASE_SCAN_NODE, TPlanNodeType::DATA_SOURCE_NODE, TPlanNodeType::KUDU_SCAN_NODE, TPlanNodeType::ICEBERG_METADATA_SCAN_NODE, - TPlanNodeType::SYSTEM_TABLE_SCAN_NODE}; + TPlanNodeType::SYSTEM_TABLE_SCAN_NODE, TPlanNodeType::PAIMON_SCAN_NODE}; // Consistent scheduling requires picking up to k distinct candidates out of n nodes. // Since each iteration can pick a node that it already picked (i.e. it is sampling with diff --git a/be/src/service/impalad-main.cc b/be/src/service/impalad-main.cc index a37e25dae..1233ace4b 100644 --- a/be/src/service/impalad-main.cc +++ b/be/src/service/impalad-main.cc @@ -31,6 +31,7 @@ #include "exec/hbase/hbase-table-writer.h" #include "exec/iceberg-metadata/iceberg-metadata-scanner.h" #include "exec/iceberg-metadata/iceberg-row-reader.h" +#include "exec/paimon/paimon-jni-scanner.h" #include "exprs/hive-udf-call.h" #include "exprs/timezone_db.h" #include "gen-cpp/ImpalaService.h" @@ -68,6 +69,7 @@ int ImpaladMain(int argc, char** argv) { ABORT_IF_ERROR(HBaseTableWriter::InitJNI()); ABORT_IF_ERROR(IcebergMetadataScanner::InitJNI()); ABORT_IF_ERROR(IcebergRowReader::InitJNI()); + ABORT_IF_ERROR(PaimonJniScanner::InitJNI()); ABORT_IF_ERROR(HiveUdfCall::InitEnv()); ABORT_IF_ERROR(JniCatalogCacheUpdateIterator::InitJNI()); InitFeSupport(); diff --git a/bin/bootstrap_toolchain.py b/bin/bootstrap_toolchain.py index ecc0354f9..c58a17b6c 100755 --- a/bin/bootstrap_toolchain.py +++ b/bin/bootstrap_toolchain.py @@ -473,7 +473,7 @@ def get_toolchain_downloads(): gcc_package = ToolchainPackage("gcc") toolchain_packages += [llvm_package, llvm_package_asserts, gcc_package] toolchain_packages += [ToolchainPackage(p) for p in - ["avro", "binutils", "boost", "breakpad", "bzip2", "calloncehack", "cctz", + ["arrow", "avro", "binutils", "boost", "breakpad", "bzip2", "calloncehack", "cctz", "cloudflarezlib", "cmake", "crcutil", "curl", "flatbuffers", "gdb", "gflags", "glog", "gperftools", "jwt-cpp", "libev", "libunwind", "lz4", "mold", "openldap", "opentelemetry-cpp", "orc", "protobuf", "python", "rapidjson", "re2", diff --git a/bin/impala-config.sh b/bin/impala-config.sh index 848732b0d..ffe94fa7c 100755 --- a/bin/impala-config.sh +++ b/bin/impala-config.sh @@ -207,6 +207,8 @@ export IMPALA_TPC_H_VERSION=2.17.0 unset IMPALA_TPC_H_URL export IMPALA_ZLIB_VERSION=1.3.1 unset IMPALA_ZLIB_URL +export IMPALA_ARROW_VERSION=13.0.0 +unset IMPALA_ARROW_URL export IMPALA_CLOUDFLAREZLIB_VERSION=7aa510344e unset IMPALA_CLOUDFLAREZLIB_URL export IMPALA_CALLONCEHACK_VERSION=1.0.0 diff --git a/cmake_modules/FindArrow.cmake b/cmake_modules/FindArrow.cmake new file mode 100644 index 000000000..8b44f7b5c --- /dev/null +++ b/cmake_modules/FindArrow.cmake @@ -0,0 +1,54 @@ +############################################################################## +# 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. +############################################################################## + +# - Find Arrow (headers and libarrow.a) with ARROW_ROOT hinting a location +# This module defines +# ARROW_INCLUDE_DIR, directory containing headers +# ARROW_STATIC_LIB, path to libarrow.a +# ARROW_FOUND +set(ARROW_ROOT $ENV{IMPALA_TOOLCHAIN_PACKAGES_HOME}/arrow-$ENV{IMPALA_ARROW_VERSION}) + +set(ARROW_SEARCH_HEADER_PATHS ${ARROW_ROOT}/include) + +set(ARROW_SEARCH_LIB_PATH ${ARROW_ROOT}/lib) + +find_path(ARROW_INCLUDE_DIR NAMES arrow/api.h arrow/c/bridge.h PATHS + ${ARROW_SEARCH_HEADER_PATHS} + # make sure we don't accidentally pick up a different version + NO_DEFAULT_PATH) + +find_library(ARROW_STATIC_LIB NAMES libarrow.a libarrow_bundled_dependencies.a PATHS + ${ARROW_SEARCH_LIB_PATH}) + +if(NOT ARROW_STATIC_LIB) + message(FATAL_ERROR "Arrow includes and libraries NOT found. " + "Looked for headers in ${ARROW_SEARCH_HEADER_PATHS}, " + "and for libs in ${ARROW_SEARCH_LIB_PATH}") + set(ARROW_FOUND FALSE) +else() + set(ARROW_FOUND TRUE) +endif () + +set(ARROW_FOUND ${ARROW_STATIC_LIB_FOUND}) + +mark_as_advanced( + ARROW_INCLUDE_DIR + ARROW_STATIC_LIB + ARROW_FOUND +) diff --git a/common/thrift/CatalogObjects.thrift b/common/thrift/CatalogObjects.thrift index 6f466a58f..7b0fd2f88 100644 --- a/common/thrift/CatalogObjects.thrift +++ b/common/thrift/CatalogObjects.thrift @@ -334,6 +334,11 @@ struct TColumn { // Key and value field id for Iceberg column with Map type. 22: optional i32 iceberg_field_map_key_id 23: optional i32 iceberg_field_map_value_id + // The followings are Paimon-specific column properties, + // will reuse the iceberg_field_id, is_key, is_nullable + // for Paimon table. + 26: optional bool is_paimon_column + } // Represents an HDFS file in a partition. diff --git a/common/thrift/PlanNodes.thrift b/common/thrift/PlanNodes.thrift index f8f44fcff..cba7d2131 100644 --- a/common/thrift/PlanNodes.thrift +++ b/common/thrift/PlanNodes.thrift @@ -57,6 +57,7 @@ enum TPlanNodeType { TUPLE_CACHE_NODE = 20 SYSTEM_TABLE_SCAN_NODE = 21 ICEBERG_MERGE_NODE = 22 + PAIMON_SCAN_NODE=23 } // phases of an execution node @@ -417,6 +418,28 @@ struct TSystemTableScanNode { 2: required CatalogObjects.TSystemTableName table_name } +struct TPaimonJniScanParam { + // Serialized paimon api table object. + 1: required binary paimon_table_obj + // Thrift serialized splits for the Jni Scanner. + 2: required list splits + // Field id list for projection. + 3: required list projection + // mem limit from backend. + // not set means no limit. + 4: optional i64 mem_limit_bytes + // arrow batch size + 5: optional i32 batch_size + // fragment id + 6: Types.TUniqueId fragment_id; +} + +struct TPaimonScanNode { + 1: required Types.TTupleId tuple_id + 2: required binary paimon_table_obj + 3: required string table_name; +} + struct TEqJoinCondition { // left-hand side of " = " 1: required Exprs.TExpr left; @@ -838,6 +861,7 @@ struct TPlanNode { 28: optional TTupleCacheNode tuple_cache_node 29: optional TSystemTableScanNode system_table_scan_node + 31: optional TPaimonScanNode paimon_table_scan_node } // A flattened representation of a tree of PlanNodes, obtained by depth-first diff --git a/common/thrift/Types.thrift b/common/thrift/Types.thrift index 4903dfac0..9d25ac818 100644 --- a/common/thrift/Types.thrift +++ b/common/thrift/Types.thrift @@ -77,8 +77,10 @@ struct TScalarType { struct TStructField { 1: required string name 2: optional string comment - // Valid for Iceberg tables + // Valid for Iceberg and Paimon tables. 3: optional i32 field_id + // Valid for paimon tables. + 4: optional bool is_nullable } struct TTypeNode { diff --git a/fe/pom.xml b/fe/pom.xml index 18cd27e4c..248ba8ce4 100644 --- a/fe/pom.xml +++ b/fe/pom.xml @@ -558,7 +558,126 @@ under the License. + + org.apache.paimon + paimon-arrow + ${paimon.version} + + + org.apache.arrow + arrow-vector + ${arrow.version} + + + log4j + log4j + + + org.apache.logging.log4j + log4j-1.2-api + + + org.apache.logging.log4j + log4j-slf4j-impl + + + org.apache.ant + ant + + + org.apache.ant + ant-launcher + + + flatbuffers-java + com.google.flatbuffers + + + + + org.apache.arrow + arrow-c-data + ${arrow.version} + + + log4j + log4j + + + org.apache.logging.log4j + log4j-1.2-api + + + org.apache.logging.log4j + log4j-slf4j-impl + + + org.apache.ant + ant + + + org.apache.ant + ant-launcher + + + + + + org.apache.arrow + arrow-memory-core + ${arrow.version} + + + log4j + log4j + + + org.apache.logging.log4j + log4j-1.2-api + + + org.apache.logging.log4j + log4j-slf4j-impl + + + org.apache.ant + ant + + + org.apache.ant + ant-launcher + + + + + + org.apache.arrow + arrow-memory-unsafe + ${arrow.version} + + + log4j + log4j + + + org.apache.logging.log4j + log4j-1.2-api + + + org.apache.logging.log4j + log4j-slf4j-impl + + + org.apache.ant + ant + + + org.apache.ant + ant-launcher + + + diff --git a/fe/src/main/java/org/apache/impala/catalog/Column.java b/fe/src/main/java/org/apache/impala/catalog/Column.java index 4251f14d9..ca85b6546 100644 --- a/fe/src/main/java/org/apache/impala/catalog/Column.java +++ b/fe/src/main/java/org/apache/impala/catalog/Column.java @@ -22,6 +22,8 @@ import java.util.List; import org.apache.hadoop.hive.metastore.api.ColumnStatisticsData; import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.impala.catalog.paimon.PaimonColumn; +import org.apache.impala.catalog.paimon.PaimonStructField; import org.apache.impala.common.ImpalaRuntimeException; import org.apache.impala.thrift.TColumn; import org.apache.impala.thrift.TColumnDescriptor; @@ -102,6 +104,10 @@ public class Column { col = new IcebergColumn(columnDesc.getColumnName(), type, comment, position, columnDesc.getIceberg_field_id(), columnDesc.getIceberg_field_map_key_id(), columnDesc.getIceberg_field_map_value_id(), columnDesc.isIs_nullable()); + } else if (columnDesc.isIs_paimon_column()) { + Preconditions.checkState(columnDesc.isSetIceberg_field_id()); + col = new PaimonColumn(columnDesc.getColumnName(), type, comment, position, + columnDesc.getIceberg_field_id(), columnDesc.isIs_nullable()); } else if (columnDesc.isIs_hbase_column()) { // HBase table column. The HBase column qualifier (column name) is not be set for // the HBase row key, so it being set in the thrift struct is not a precondition. @@ -159,6 +165,10 @@ public class Column { IcebergColumn iCol = (IcebergColumn) col; fields.add(new IcebergStructField(iCol.getName(), iCol.getType(), iCol.getComment(), iCol.getFieldId())); + } else if (col instanceof PaimonColumn) { + PaimonColumn pCol = (PaimonColumn) col; + fields.add(new PaimonStructField(pCol.getName(), pCol.getType(), + pCol.getComment(), pCol.getFieldId(), pCol.isNullable())); } else { fields.add(new StructField(col.getName(), col.getType(), col.getComment())); } diff --git a/fe/src/main/java/org/apache/impala/catalog/Table.java b/fe/src/main/java/org/apache/impala/catalog/Table.java index 315a334f2..ef7808076 100644 --- a/fe/src/main/java/org/apache/impala/catalog/Table.java +++ b/fe/src/main/java/org/apache/impala/catalog/Table.java @@ -40,6 +40,8 @@ import org.apache.hadoop.hive.metastore.api.PrincipalType; import org.apache.impala.analysis.TableName; import org.apache.impala.catalog.events.InFlightEvents; import org.apache.impala.catalog.monitor.CatalogMonitor; +import org.apache.impala.catalog.paimon.PaimonColumn; +import org.apache.impala.catalog.paimon.PaimonStructField; import org.apache.impala.catalog.paimon.PaimonTable; import org.apache.impala.catalog.paimon.PaimonUtil; import org.apache.impala.common.ImpalaRuntimeException; @@ -667,6 +669,10 @@ public abstract class Table extends CatalogObjectImpl implements FeTable { IcebergColumn iCol = (IcebergColumn) col; return new IcebergStructField(iCol.getName(), iCol.getType(), iCol.getComment(), iCol.getFieldId()); + } else if (col instanceof PaimonColumn) { + PaimonColumn pCol = (PaimonColumn) col; + return new PaimonStructField(pCol.getName(), pCol.getType(), pCol.getComment(), + pCol.getFieldId(), pCol.isNullable()); } else { return new StructField(col.getName(), col.getType(), col.getComment()); } diff --git a/fe/src/main/java/org/apache/impala/catalog/Type.java b/fe/src/main/java/org/apache/impala/catalog/Type.java index 640daad15..3b9afe79a 100644 --- a/fe/src/main/java/org/apache/impala/catalog/Type.java +++ b/fe/src/main/java/org/apache/impala/catalog/Type.java @@ -25,6 +25,7 @@ import org.apache.impala.analysis.CreateTableStmt; import org.apache.impala.analysis.Parser; import org.apache.impala.analysis.StatementBase; import org.apache.impala.analysis.TypeDef; +import org.apache.impala.catalog.paimon.PaimonStructField; import org.apache.impala.common.AnalysisException; import org.apache.impala.common.Pair; import org.apache.impala.thrift.TColumnType; @@ -527,9 +528,16 @@ public abstract class Type { Pair res = fromThrift(col, nodeIdx); nodeIdx = res.second.intValue(); if (thriftField.isSetField_id()) { - // We create 'IcebergStructField' for Iceberg tables which have field id. - structFields.add(new IcebergStructField(name, res.first, comment, - thriftField.getField_id())); + if (!thriftField.isSetIs_nullable()) { + // We create 'IcebergStructField' for Iceberg tables which have field id. + // if nullable is not set. + structFields.add(new IcebergStructField( + name, res.first, comment, thriftField.getField_id())); + } else { + // nullable is set, it is a PaimonStructField + structFields.add(new PaimonStructField(name, res.first, comment, + thriftField.getField_id(), thriftField.isIs_nullable())); + } } else { structFields.add(new StructField(name, res.first, comment)); } diff --git a/fe/src/main/java/org/apache/impala/catalog/local/LocalPaimonTable.java b/fe/src/main/java/org/apache/impala/catalog/local/LocalPaimonTable.java index 55622cd6a..54ad744ed 100644 --- a/fe/src/main/java/org/apache/impala/catalog/local/LocalPaimonTable.java +++ b/fe/src/main/java/org/apache/impala/catalog/local/LocalPaimonTable.java @@ -19,7 +19,7 @@ package org.apache.impala.catalog.local; import com.google.common.base.Preconditions; -import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.impala.catalog.Column; import org.apache.impala.catalog.TableLoadingException; import org.apache.impala.catalog.paimon.FePaimonTable; import org.apache.impala.catalog.paimon.PaimonUtil; @@ -29,6 +29,7 @@ import org.apache.log4j.Logger; import org.apache.paimon.table.Table; import java.io.IOException; +import java.util.List; import java.util.Set; /** @@ -45,18 +46,28 @@ public class LocalPaimonTable extends LocalTable implements FePaimonTable { Preconditions.checkNotNull(msTbl); Preconditions.checkNotNull(ref); try { - LocalPaimonTable localPaimonTable = new LocalPaimonTable(db, msTbl, ref); + Table table = PaimonUtil.createFileStoreTable(msTbl); + List paimonColumns = PaimonUtil.toImpalaColumn(table); + ColumnMap colMap = new ColumnMap(paimonColumns, + /*numClusteringCols=*/table.partitionKeys().size(), + db.getName() + "." + msTbl.getTableName(), + /*isFullAcidSchema=*/false); + LocalPaimonTable localPaimonTable = + new LocalPaimonTable(db, msTbl, ref, colMap, table); return localPaimonTable; - } catch (MetaException ex) { + } catch (Exception ex) { throw new TableLoadingException("Failed to load table" + msTbl.getTableName(), ex); } } protected LocalPaimonTable(LocalDb db, org.apache.hadoop.hive.metastore.api.Table msTbl, - MetaProvider.TableMetaRef ref) throws MetaException { - super(db, msTbl, ref); - table_ = PaimonUtil.createFileStoreTable(msTbl); + MetaProvider.TableMetaRef ref, ColumnMap columnMap, Table table) { + super(db, msTbl, ref, columnMap); + table_ = table; + /// TODO: add virtual column later if it is supported. + /// addVirtualColumns(ref.getVirtualColumns()); applyPaimonTableStatsIfPresent(); + applyPaimonColumnStatsIfPresent(); } @Override diff --git a/fe/src/main/java/org/apache/impala/catalog/local/LocalTable.java b/fe/src/main/java/org/apache/impala/catalog/local/LocalTable.java index 2f780d3f5..ea74b0ca4 100644 --- a/fe/src/main/java/org/apache/impala/catalog/local/LocalTable.java +++ b/fe/src/main/java/org/apache/impala/catalog/local/LocalTable.java @@ -49,6 +49,8 @@ import org.apache.impala.catalog.SystemTable; import org.apache.impala.catalog.TableLoadingException; import org.apache.impala.catalog.VirtualColumn; import org.apache.impala.catalog.local.MetaProvider.TableMetaRef; +import org.apache.impala.catalog.paimon.PaimonColumn; +import org.apache.impala.catalog.paimon.PaimonStructField; import org.apache.impala.catalog.paimon.PaimonUtil; import org.apache.impala.common.Pair; import org.apache.impala.common.RuntimeEnv; @@ -451,6 +453,10 @@ abstract class LocalTable implements FeTable { IcebergColumn iCol = (IcebergColumn) col; fields.add(new IcebergStructField(iCol.getName(), iCol.getType(), iCol.getComment(), iCol.getFieldId())); + } else if (col instanceof PaimonColumn) { + PaimonColumn pCol = (PaimonColumn) col; + fields.add(new PaimonStructField(pCol.getName(), pCol.getType(), + pCol.getComment(), pCol.getFieldId(), pCol.isNullable())); } else { fields.add(new StructField(col.getName(), col.getType(), col.getComment())); } diff --git a/fe/src/main/java/org/apache/impala/catalog/paimon/PaimonColumn.java b/fe/src/main/java/org/apache/impala/catalog/paimon/PaimonColumn.java new file mode 100644 index 000000000..2f2f9ca84 --- /dev/null +++ b/fe/src/main/java/org/apache/impala/catalog/paimon/PaimonColumn.java @@ -0,0 +1,66 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.impala.catalog.paimon; + +import org.apache.impala.catalog.Column; +import org.apache.impala.catalog.Type; +import org.apache.impala.thrift.TColumn; +import org.apache.impala.thrift.TColumnDescriptor; + +/** + * Represents a Paimon column. + * + * This class extends Column with the Paimon-specific field id. Field ids are used in + * schema evolution to uniquely identify columns.. + */ +public class PaimonColumn extends Column { + private final int fieldId_; + // False for required Paimon field, true for optional Paimon field + private final boolean isNullable_; + + public PaimonColumn(String name, Type type, String comment, int position, int fieldId, + boolean isNullable) { + super(name.toLowerCase(), type, comment, position); + fieldId_ = fieldId; + isNullable_ = isNullable; + } + + public PaimonColumn(String name, Type type, String comment, int position, int fieldId) { + this(name, type, comment, position, fieldId, true); + } + + public int getFieldId() { return fieldId_; } + + public boolean isNullable() { return isNullable_; } + + @Override + public TColumn toThrift() { + TColumn tcol = super.toThrift(); + tcol.setIs_paimon_column(true); + tcol.setIceberg_field_id(fieldId_); + tcol.setIs_nullable(isNullable_); + return tcol; + } + + @Override + public TColumnDescriptor toDescriptor() { + TColumnDescriptor desc = super.toDescriptor(); + desc.setIcebergFieldId(fieldId_); + return desc; + } +} diff --git a/fe/src/main/java/org/apache/impala/catalog/paimon/PaimonHiveTypeUtils.java b/fe/src/main/java/org/apache/impala/catalog/paimon/PaimonHiveTypeUtils.java index a35e8fa51..b62ceab29 100644 --- a/fe/src/main/java/org/apache/impala/catalog/paimon/PaimonHiveTypeUtils.java +++ b/fe/src/main/java/org/apache/impala/catalog/paimon/PaimonHiveTypeUtils.java @@ -63,11 +63,11 @@ import org.apache.paimon.types.VariantType; * Utils for paimon and hive Type conversions, the class is from * org.apache.paimon.hive.HiveTypeUtils, refactor to fix the * following incompatible conversion issue: - * paimon type LocalZonedTimestampType will convert to - * org.apache.hadoop.hive.serde2.typeinfo.TimestampLocalTZTypeInfo + * paimon type ${@link LocalZonedTimestampType} will convert to + * ${@link org.apache.hadoop.hive.serde2.typeinfo.TimestampLocalTZTypeInfo} * however, it is not supported in impala, TableLoadingException * will raise while loading the table in method: - * apache.impala.catalog.FeCatalogUtils#parseColumnType + * ${@link org.apache.impala.catalog.FeCatalogUtils#parseColumnType} * To fix the issue LocalZonedTimestampType will be converted to * hive timestamp type. */ @@ -206,31 +206,26 @@ public class PaimonHiveTypeUtils { } static DataType visit(TypeInfo type, HiveToPaimonTypeVisitor visitor) { - if (!(type instanceof StructTypeInfo)) { - if (type instanceof MapTypeInfo) { - MapTypeInfo mapTypeInfo = (MapTypeInfo)type; - return DataTypes.MAP(visit(mapTypeInfo.getMapKeyTypeInfo(), visitor), - visit(mapTypeInfo.getMapValueTypeInfo(), visitor)); - } else if (type instanceof ListTypeInfo) { - ListTypeInfo listTypeInfo = (ListTypeInfo)type; - return DataTypes.ARRAY( - visit(listTypeInfo.getListElementTypeInfo(), visitor)); - } else { - return visitor.atomic(type); - } - } else { + if (type instanceof StructTypeInfo) { StructTypeInfo structTypeInfo = (StructTypeInfo)type; ArrayList fieldNames = structTypeInfo.getAllStructFieldNames(); ArrayList typeInfos = structTypeInfo .getAllStructFieldTypeInfos(); RowType.Builder builder = RowType.builder(); - for(int i = 0; i < fieldNames.size(); ++i) { builder.field((String)fieldNames.get(i), visit((TypeInfo)typeInfos.get(i), visitor)); } - return builder.build(); + } else if (type instanceof MapTypeInfo) { + MapTypeInfo mapTypeInfo = (MapTypeInfo) type; + return DataTypes.MAP(visit(mapTypeInfo.getMapKeyTypeInfo(), visitor), + visit(mapTypeInfo.getMapValueTypeInfo(), visitor)); + } else if (type instanceof ListTypeInfo) { + ListTypeInfo listTypeInfo = (ListTypeInfo) type; + return DataTypes.ARRAY(visit(listTypeInfo.getListElementTypeInfo(), visitor)); + } else { + return visitor.atomic(type); } } diff --git a/fe/src/main/java/org/apache/impala/catalog/paimon/PaimonImpalaTypeUtils.java b/fe/src/main/java/org/apache/impala/catalog/paimon/PaimonImpalaTypeUtils.java index 9405168c0..c66b6a3a5 100644 --- a/fe/src/main/java/org/apache/impala/catalog/paimon/PaimonImpalaTypeUtils.java +++ b/fe/src/main/java/org/apache/impala/catalog/paimon/PaimonImpalaTypeUtils.java @@ -180,8 +180,9 @@ public class PaimonImpalaTypeUtils { rowType.getFields() .stream() .map(dataField - -> new StructField( - dataField.name().toLowerCase(), dataField.type().accept(this))) + -> new PaimonStructField(dataField.name().toLowerCase(), + dataField.type().accept(this), dataField.description(), + dataField.id(), dataField.type().isNullable())) .collect(Collectors.toList()); return new StructType(structFields); @@ -254,12 +255,12 @@ public class PaimonImpalaTypeUtils { public static boolean isSupportedPrimitiveType(PrimitiveType primitiveType) { Preconditions.checkNotNull(primitiveType); switch (primitiveType) { + case DOUBLE: + case FLOAT: case BIGINT: case INT: case SMALLINT: case TINYINT: - case DOUBLE: - case FLOAT: case BOOLEAN: case STRING: case TIMESTAMP: @@ -267,7 +268,6 @@ public class PaimonImpalaTypeUtils { case DATE: case BINARY: case CHAR: - case DATETIME: case VARCHAR: return true; default: return false; } diff --git a/fe/src/main/java/org/apache/impala/catalog/paimon/PaimonStructField.java b/fe/src/main/java/org/apache/impala/catalog/paimon/PaimonStructField.java new file mode 100644 index 000000000..87079fbf1 --- /dev/null +++ b/fe/src/main/java/org/apache/impala/catalog/paimon/PaimonStructField.java @@ -0,0 +1,87 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.impala.catalog.paimon; + +import java.util.Objects; + +import org.apache.impala.catalog.StructField; +import org.apache.impala.catalog.Type; +import org.apache.impala.thrift.TColumnType; +import org.apache.impala.thrift.TStructField; +import org.apache.impala.thrift.TTypeNode; + +/** + * Represents a Paimon StructField. + * + * This class extends StructField with Paimon-specific field. + * Paimon uses field IDs for schema evolution and compatibility, similar to Iceberg. + * We keep field id by this class, so we can use field id to resolve column on backend. + */ +public class PaimonStructField extends StructField { + private final int fieldId_; + // False for required Paimon field, true for optional Paimon field + private final boolean isNullable_; + + public PaimonStructField(String name, Type type, String comment, int fieldId) { + this(name, type, comment, fieldId, true); + } + + public PaimonStructField( + String name, Type type, String comment, int fieldId, boolean isNullable) { + super(name, type, comment); + fieldId_ = fieldId; + isNullable_ = isNullable; + } + + public int getFieldId() { return fieldId_; } + + public boolean isNullable() { return isNullable_; } + + @Override + public void toThrift(TColumnType container, TTypeNode node) { + TStructField field = new TStructField(); + field.setName(name_); + if (comment_ != null) field.setComment(comment_); + field.setField_id(fieldId_); + // Paimon-specific metadata - nullable and key properties could be added to + // extended metadata if the Thrift definition supports it + field.setIs_nullable(isNullable_); + node.struct_fields.add(field); + type_.toThrift(container); + } + + @Override + public boolean equals(Object other) { + if (!(other instanceof PaimonStructField)) return false; + PaimonStructField otherStructField = (PaimonStructField) other; + return otherStructField.name_.equals(name_) && otherStructField.type_.equals(type_) + && otherStructField.fieldId_ == fieldId_ + && otherStructField.isNullable_ == isNullable_; + } + + @Override + public int hashCode() { + return Objects.hash(name_, type_, fieldId_, isNullable_); + } + + @Override + public String toString() { + return String.format("PaimonStructField{name=%s, type=%s, fieldId=%d, nullable=%s}", + name_, type_, fieldId_, isNullable_); + } +} \ No newline at end of file diff --git a/fe/src/main/java/org/apache/impala/catalog/paimon/PaimonTable.java b/fe/src/main/java/org/apache/impala/catalog/paimon/PaimonTable.java index f6ade36a5..45ea962de 100644 --- a/fe/src/main/java/org/apache/impala/catalog/paimon/PaimonTable.java +++ b/fe/src/main/java/org/apache/impala/catalog/paimon/PaimonTable.java @@ -28,6 +28,7 @@ import org.apache.hadoop.hive.metastore.IMetaStoreClient; import org.apache.hadoop.hive.metastore.api.FieldSchema; import org.apache.impala.catalog.Column; import org.apache.impala.catalog.Db; +import org.apache.impala.catalog.StructType; import org.apache.impala.catalog.Table; import org.apache.impala.catalog.TableLoadingException; import org.apache.impala.catalog.VirtualColumn; @@ -135,7 +136,8 @@ public class PaimonTable extends Table implements FePaimonTable { public void loadSchemaFromPaimon() throws TableLoadingException, ImpalaRuntimeException { loadSchema(); - addVirtualColumns(); + // TODO: add virtual column later if it is supported. + // addVirtualColumns(); } /** @@ -213,6 +215,17 @@ public class PaimonTable extends Table implements FePaimonTable { addVirtualColumn(VirtualColumn.BUCKET_ID); } + @Override + public void addColumn(Column col) { + Preconditions.checkState(col instanceof PaimonColumn); + PaimonColumn pCol = (PaimonColumn) col; + colsByPos_.add(pCol); + colsByName_.put(pCol.getName().toLowerCase(), col); + ((StructType) type_.getItemType()) + .addField(new PaimonStructField(col.getName(), col.getType(), col.getComment(), + pCol.getFieldId(), pCol.isNullable())); + } + /** * Loads the metadata of a Paimon table. *

diff --git a/fe/src/main/java/org/apache/impala/catalog/paimon/PaimonUtil.java b/fe/src/main/java/org/apache/impala/catalog/paimon/PaimonUtil.java index f73870864..df3284009 100644 --- a/fe/src/main/java/org/apache/impala/catalog/paimon/PaimonUtil.java +++ b/fe/src/main/java/org/apache/impala/catalog/paimon/PaimonUtil.java @@ -104,7 +104,6 @@ import org.apache.paimon.types.SmallIntType; import org.apache.paimon.types.TinyIntType; import org.apache.paimon.utils.InternalRowPartitionComputer; import org.apache.thrift.TException; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -200,6 +199,17 @@ public class PaimonUtil { return ret; } + /** + * function to close autoClosable object quitely. + */ + public static void closeQuitely(AutoCloseable closeable) { + if (closeable != null) { + try { + closeable.close(); + } catch (Exception e) { LOG.warn("Error closing " + closeable, e); } + } + } + /** * Converts Paimon schema to an Impala schema. */ @@ -209,7 +219,8 @@ public class PaimonUtil { int pos = 0; for (DataField dataField : schema.getFields()) { Type colType = PaimonImpalaTypeUtils.toImpalaType(dataField.type()); - ret.add(new Column(dataField.name().toLowerCase(), colType, pos++)); + ret.add(new PaimonColumn(dataField.name().toLowerCase(), colType, + dataField.description(), pos++, dataField.id(), dataField.type().isNullable())); } return ret; } @@ -861,4 +872,43 @@ public class PaimonUtil { } return result; } + + /** + * convert paimon api table schema to impala columns + */ + public static List toImpalaColumn(Table table) throws ImpalaRuntimeException { + RowType rowType = table.rowType(); + List dataFields = rowType.getFields(); + List partitionKeys = table.partitionKeys() + .stream() + .map(String::toLowerCase) + .collect(Collectors.toList()); + List impalaFields = convertToImpalaSchema(rowType); + List impalaNonPartitionedFields = Lists.newArrayList(); + List impalaPartitionedFields = Lists.newArrayList(); + List columns = Lists.newArrayList(); + // lookup the clustering columns + for (String name : partitionKeys) { + int colIndex = PaimonUtil.getFieldIndexByNameIgnoreCase(rowType, name); + Preconditions.checkArgument(colIndex >= 0); + impalaPartitionedFields.add(impalaFields.get(colIndex)); + } + // put non-clustering columns in natural order + for (int i = 0; i < dataFields.size(); i++) { + if (!partitionKeys.contains(dataFields.get(i).name().toLowerCase())) { + impalaNonPartitionedFields.add(impalaFields.get(i)); + } + } + + int colPos = 0; + for (Column col : impalaPartitionedFields) { + col.setPosition(colPos++); + columns.add(col); + } + for (Column col : impalaNonPartitionedFields) { + col.setPosition(colPos++); + columns.add(col); + } + return columns; + } } diff --git a/fe/src/main/java/org/apache/impala/planner/PaimonScanNode.java b/fe/src/main/java/org/apache/impala/planner/PaimonScanNode.java new file mode 100644 index 000000000..509c3f525 --- /dev/null +++ b/fe/src/main/java/org/apache/impala/planner/PaimonScanNode.java @@ -0,0 +1,303 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.impala.planner; + +import com.google.common.base.MoreObjects; +import com.google.common.base.Preconditions; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; +import org.apache.commons.lang3.SerializationUtils; +import org.apache.impala.analysis.Analyzer; +import org.apache.impala.analysis.Expr; +import org.apache.impala.analysis.MultiAggregateInfo; +import org.apache.impala.analysis.SlotDescriptor; +import org.apache.impala.analysis.TupleDescriptor; +import org.apache.impala.catalog.paimon.FePaimonTable; +import org.apache.impala.catalog.paimon.PaimonColumn; +import org.apache.impala.common.AnalysisException; +import org.apache.impala.common.ImpalaException; +import org.apache.impala.common.ImpalaRuntimeException; +import org.apache.impala.common.ThriftSerializationCtx; +import org.apache.impala.planner.paimon.PaimonSplit; +import org.apache.impala.thrift.TExplainLevel; +import org.apache.impala.thrift.TNetworkAddress; +import org.apache.impala.thrift.TPaimonScanNode; +import org.apache.impala.thrift.TPlanNode; +import org.apache.impala.thrift.TPlanNodeType; +import org.apache.impala.thrift.TQueryOptions; +import org.apache.impala.thrift.TScanRange; +import org.apache.impala.thrift.TScanRangeLocation; +import org.apache.impala.thrift.TScanRangeLocationList; +import org.apache.impala.thrift.TScanRangeSpec; +import org.apache.impala.util.ExecutorMembershipSnapshot; +import org.apache.paimon.table.Table; +import org.apache.paimon.table.source.DataSplit; +import org.apache.paimon.table.source.ReadBuilder; +import org.apache.paimon.table.source.Split; +import org.apache.paimon.types.DataField; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +/** + * Jni-based scan node of a single paimon table. + */ +public class PaimonScanNode extends ScanNode { + private final static Logger LOG = LoggerFactory.getLogger(PaimonScanNode.class); + private final static long PAIMON_ROW_AVG_SIZE_OVERHEAD = 4L; + // FeTable object + private final FePaimonTable table_; + + // paimon table object + private final Table paimonApiTable_; + + // Indexes for the set of hosts that will be used for the query. + // From analyzer.getHostIndex().getIndex(address) + private final Set hostIndexSet_ = new HashSet<>(); + // Array of top level field ids used for top level column scan pruning. + private int[] projection_; + // top level field id map + private Set fieldIdMap_ = Sets.newHashSet(); + // Splits generated for paimon scanning during planning stage. + private List splits_; + + public PaimonScanNode(PlanNodeId id, TupleDescriptor desc, List conjuncts, + MultiAggregateInfo aggInfo, FePaimonTable table) { + super(id, desc, "SCAN PAIMON"); + conjuncts_ = conjuncts; + aggInfo_ = aggInfo; + table_ = table; + paimonApiTable_ = table.getPaimonApiTable(); + } + + @Override + public void init(Analyzer analyzer) throws ImpalaException { + super.init(analyzer); + conjuncts_ = orderConjunctsByCost(conjuncts_); + // TODO: implement predicate push down later here. + + // materialize slots in remaining conjuncts_ + analyzer.materializeSlots(conjuncts_); + collectProjectionId(); + computeMemLayout(analyzer); + computeScanRangeLocations(analyzer); + computePaimonStats(analyzer); + } + + public void computePaimonStats(Analyzer analyzer) { + computeNumNodes(analyzer); + // Update the cardinality, hint value will be used when table has no stats. + inputCardinality_ = cardinality_ = estimateTableRowCount(); + cardinality_ = applyConjunctsSelectivity(cardinality_); + cardinality_ = capCardinalityAtLimit(cardinality_); + avgRowSize_ = estimateAvgRowSize(); + if (LOG.isTraceEnabled()) { + LOG.trace("computeStats paimonScan: cardinality=" + Long.toString(cardinality_)); + } + } + + /** + * Collect and analyze top-level columns. + */ + public void collectProjectionId() throws AnalysisException { + projection_ = new int[desc_.getSlots().size()]; + for (int i = 0; i < desc_.getSlots().size(); i++) { + SlotDescriptor sd = desc_.getSlots().get(i); + if (sd.isVirtualColumn()) { + throw new AnalysisException("Paimon Scanner doesn't support virtual columns."); + } + if (sd.getPath().getRawPath() != null && sd.getPath().getRawPath().size() > 1) { + throw new AnalysisException("Paimon Scanner doesn't support nested columns."); + } + PaimonColumn paimonColumn = (PaimonColumn) desc_.getSlots().get(i).getColumn(); + projection_[i] = paimonColumn.getFieldId(); + fieldIdMap_.add(paimonColumn.getFieldId()); + } + Preconditions.checkArgument(projection_.length == desc_.getSlots().size()); + LOG.info(String.format("table %s projection fields: %s", table_.getFullName(), + Arrays.toString(projection_))); + } + + protected long estimateSplitRowCount(Split s) { + if (s instanceof DataSplit) { + DataSplit dataSplit = (DataSplit) s; + if (dataSplit.mergedRowCountAvailable()) { return dataSplit.mergedRowCount(); } + } + return s.rowCount(); + } + + protected long estimateTableRowCount() { + return splits_.stream() + .map(this ::estimateSplitRowCount) + .reduce(Long::sum) + .orElse(-1L); + } + + protected long estimateAvgRowSize() { + List dataColumns = paimonApiTable_.rowType().getFields(); + return dataColumns.stream() + .filter(df -> fieldIdMap_.contains(df.id())) + .mapToInt(column -> column.type().defaultSize()) + .sum() + + PAIMON_ROW_AVG_SIZE_OVERHEAD; + } + /** + * Compute the scan range locations for the given table using the scan tokens. + */ + private void computeScanRangeLocations(Analyzer analyzer) + throws ImpalaRuntimeException { + scanRangeSpecs_ = new TScanRangeSpec(); + ReadBuilder readBuilder = paimonApiTable_.newReadBuilder(); + + // 2. Plan splits in 'Coordinator'. + splits_ = readBuilder.newScan().plan().splits(); + + if (splits_.size() <= 0) { + LOG.info("no paimon data available"); + return; + } + + for (Split split : splits_) { + List locations = new ArrayList<>(); + // TODO: Currently, set to dummy network address for random executor scheduling, + // don't forget to get actual location for data locality after native table scan + // is supported. + // + { + TNetworkAddress address = new TNetworkAddress("localhost", 12345); + // Use the network address to look up the host in the global list + Integer hostIndex = analyzer.getHostIndex().getOrAddIndex(address); + locations.add(new TScanRangeLocation(hostIndex)); + hostIndexSet_.add(hostIndex); + } + + TScanRange scanRange = new TScanRange(); + // TODO: apply predicate push down later. + PaimonSplit paimonSplit = new PaimonSplit(split, null); + byte[] split_data_serialized = SerializationUtils.serialize(paimonSplit); + scanRange.setFile_metadata(split_data_serialized); + TScanRangeLocationList locs = new TScanRangeLocationList(); + locs.setScan_range(scanRange); + locs.setLocations(locations); + scanRangeSpecs_.addToConcrete_ranges(locs); + } + } + + @Override + protected double computeSelectivity() { + List allConjuncts = Lists.newArrayList(Iterables.concat(conjuncts_)); + return computeCombinedSelectivity(allConjuncts); + } + + /** + * Estimate the number of impalad nodes that this scan node will execute on (which is + * ultimately determined by the scheduling done by the backend's Scheduler). + * As of now, scan ranges are scheduled round-robin, since they have no location + * information. . + */ + protected void computeNumNodes(Analyzer analyzer) { + ExecutorMembershipSnapshot cluster = ExecutorMembershipSnapshot.getCluster(); + final int maxInstancesPerNode = getMaxInstancesPerNode(analyzer); + final int maxPossibleInstances = + analyzer.numExecutorsForPlanning() * maxInstancesPerNode; + int totalNodes = 0; + int totalInstances = 0; + int numRemoteRanges = splits_.size(); + + // The remote ranges are round-robined across all the impalads. + int numRemoteNodes = Math.min(numRemoteRanges, analyzer.numExecutorsForPlanning()); + // The remote assignments may overlap, but we don't know by how much + // so conservatively assume no overlap. + totalNodes = Math.min(numRemoteNodes, analyzer.numExecutorsForPlanning()); + + totalInstances = Math.min(numRemoteRanges, totalNodes * maxInstancesPerNode); + + numNodes_ = Math.max(totalNodes, 1); + numInstances_ = Math.max(totalInstances, 1); + } + + @Override + public void computeNodeResourceProfile(TQueryOptions queryOptions) { + // current batch size is from query options, so estimated bytes + // is calculated as BATCH_SIZE * average row size * 2 + long batchSize = getRowBatchSize(queryOptions); + long memSize = batchSize * (long) getAvgRowSize() * 2; + nodeResourceProfile_ = + new ResourceProfileBuilder().setMemEstimateBytes(memSize).build(); + } + + @Override + protected void toThrift(TPlanNode msg, ThriftSerializationCtx serialCtx) { + toThrift(msg); + } + + @Override + protected void toThrift(TPlanNode node) { + node.node_type = TPlanNodeType.PAIMON_SCAN_NODE; + node.paimon_table_scan_node = new TPaimonScanNode(desc_.getId().asInt(), + ByteBuffer.wrap(SerializationUtils.serialize(paimonApiTable_)), + table_.getFullName()); + } + + @Override + public void computeProcessingCost(TQueryOptions queryOptions) { + processingCost_ = computeScanProcessingCost(queryOptions); + } + + @Override + protected String getNodeExplainString( + String prefix, String detailPrefix, TExplainLevel detailLevel) { + StringBuilder result = new StringBuilder(); + + String aliasStr = desc_.hasExplicitAlias() ? " " + desc_.getAlias() : ""; + result.append(String.format("%s%s:%s [%s%s]\n", prefix, id_.toString(), displayName_, + table_.getFullName(), aliasStr)); + + switch (detailLevel) { + case MINIMAL: break; + case STANDARD: // Fallthrough intended. + case EXTENDED: // Fallthrough intended. + case VERBOSE: { + if (!conjuncts_.isEmpty()) { + result.append(detailPrefix + + "predicates: " + Expr.getExplainString(conjuncts_, detailLevel) + "\n"); + } + if (!runtimeFilters_.isEmpty()) { + result.append(detailPrefix + "runtime filters: "); + result.append(getRuntimeFilterExplainString(false, detailLevel)); + } + } + } + return result.toString(); + } + + @Override + protected String debugString() { + MoreObjects.ToStringHelper helper = MoreObjects.toStringHelper(this); + helper.addValue(super.debugString()); + helper.addValue("paimonTable=" + table_.getFullName()); + return helper.toString(); + } +} diff --git a/fe/src/main/java/org/apache/impala/planner/PaimonScanPlanner.java b/fe/src/main/java/org/apache/impala/planner/PaimonScanPlanner.java new file mode 100644 index 000000000..a9d300a6a --- /dev/null +++ b/fe/src/main/java/org/apache/impala/planner/PaimonScanPlanner.java @@ -0,0 +1,64 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.impala.planner; + +import com.google.common.base.Preconditions; +import org.apache.impala.analysis.Analyzer; +import org.apache.impala.analysis.Expr; +import org.apache.impala.analysis.MultiAggregateInfo; +import org.apache.impala.analysis.TableRef; +import org.apache.impala.catalog.paimon.FePaimonTable; +import org.apache.impala.common.ImpalaException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.List; + +/** + * ScanNode factory class for Paimon, currently, only Jni based is supported + * Will add native scanNode later. + */ +public class PaimonScanPlanner { + private static final Logger LOG = LoggerFactory.getLogger(PaimonScanPlanner.class); + + private Analyzer analyzer_; + private PlannerContext ctx_; + private TableRef tblRef_; + private List conjuncts_; + private MultiAggregateInfo aggInfo_; + + private FePaimonTable table_; + + public PaimonScanPlanner(Analyzer analyzer, PlannerContext ctx, TableRef paimonTblRef, + List conjuncts, MultiAggregateInfo aggInfo) throws ImpalaException { + Preconditions.checkState(paimonTblRef.getTable() instanceof FePaimonTable); + analyzer_ = analyzer; + ctx_ = ctx; + tblRef_ = paimonTblRef; + conjuncts_ = conjuncts; + aggInfo_ = aggInfo; + table_ = (FePaimonTable) paimonTblRef.getTable(); + } + + public PlanNode createPaimonScanPlan() throws ImpalaException { + PaimonScanNode ret = new PaimonScanNode( + ctx_.getNextNodeId(), tblRef_.getDesc(), conjuncts_, aggInfo_, table_); + ret.init(analyzer_); + return ret; + } +} diff --git a/fe/src/main/java/org/apache/impala/planner/SingleNodePlanner.java b/fe/src/main/java/org/apache/impala/planner/SingleNodePlanner.java index a6e83b62c..effe17fee 100644 --- a/fe/src/main/java/org/apache/impala/planner/SingleNodePlanner.java +++ b/fe/src/main/java/org/apache/impala/planner/SingleNodePlanner.java @@ -1910,6 +1910,10 @@ public class SingleNodePlanner implements SingleNodePlannerIntf { conjuncts); scanNode.init(analyzer); return scanNode; + } else if (table instanceof FePaimonTable) { + PaimonScanPlanner paimonScanPlanner = + new PaimonScanPlanner(analyzer, ctx_, tblRef, conjuncts, aggInfo); + return paimonScanPlanner.createPaimonScanPlan(); } else if (table instanceof FeHBaseTable) { // HBase table scanNode = new HBaseScanNode(ctx_.getNextNodeId(), tblRef.getDesc()); diff --git a/fe/src/main/java/org/apache/impala/planner/paimon/PaimonSplit.java b/fe/src/main/java/org/apache/impala/planner/paimon/PaimonSplit.java new file mode 100644 index 000000000..c1430dd50 --- /dev/null +++ b/fe/src/main/java/org/apache/impala/planner/paimon/PaimonSplit.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.impala.planner.paimon; + +import org.apache.paimon.predicate.Predicate; +import org.apache.paimon.table.source.Split; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.List; + +/** + * Paimon split entity used by paimon jni scanner. + * */ +public class PaimonSplit implements Serializable { + // Paimon split instance to perform scan. + private final Split split_; + // predicates that can be pushed to paimon source. + private final ArrayList predicates_; + + public PaimonSplit(Split split, ArrayList predicates) { + split_ = split; + predicates_ = predicates; + } + + public Split getSplit() { return split_; } + + public List getPredicates() { return predicates_; } +} diff --git a/fe/src/main/java/org/apache/impala/util/paimon/ArrowRootAllocation.java b/fe/src/main/java/org/apache/impala/util/paimon/ArrowRootAllocation.java new file mode 100644 index 000000000..41eaaf28c --- /dev/null +++ b/fe/src/main/java/org/apache/impala/util/paimon/ArrowRootAllocation.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.impala.util.paimon; + +import org.apache.arrow.memory.RootAllocator; + +/* + * + * Arrow Root Allocation singleton + * Note: all Arrow code should use this as root allocator. + * + * */ +public class ArrowRootAllocation { + private static RootAllocator ROOT_ALLOCATOR; + + private ArrowRootAllocation() {} + + public static RootAllocator rootAllocator() { + synchronized (ArrowRootAllocation.class) { + if (ROOT_ALLOCATOR == null) { + ROOT_ALLOCATOR = new RootAllocator(Long.MAX_VALUE); + Runtime.getRuntime().addShutdownHook(new Thread(() -> ROOT_ALLOCATOR.close())); + } + } + return ROOT_ALLOCATOR; + } +} diff --git a/fe/src/main/java/org/apache/impala/util/paimon/PaimonArrowFieldTypeFactory.java b/fe/src/main/java/org/apache/impala/util/paimon/PaimonArrowFieldTypeFactory.java new file mode 100644 index 000000000..eecbd8890 --- /dev/null +++ b/fe/src/main/java/org/apache/impala/util/paimon/PaimonArrowFieldTypeFactory.java @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.impala.util.paimon; + +import org.apache.arrow.vector.types.pojo.ArrowType; +import org.apache.arrow.vector.types.pojo.FieldType; +import org.apache.paimon.arrow.ArrowFieldTypeConversion; +import org.apache.paimon.types.DecimalType; + +/** + * It is an extension of {@link ArrowFieldTypeConversion.ArrowFieldTypeVisitor} class. + * To change the decimal conversion behavior. + * Paimon decimal type will convert to arrow Decimal128 type, it involves byte copy + * and padding, which will cause additional overhead to pass data to BE. + * To Eliminate the overhead, will directly pass the decimal unscaled bytes to BE, So + * Arrow binary type will be used instead of Decimal128 data type. + */ +public class PaimonArrowFieldTypeFactory + extends ArrowFieldTypeConversion.ArrowFieldTypeVisitor { + @Override + public FieldType visit(DecimalType decimalType) { + return new FieldType(decimalType.isNullable(), new ArrowType.Binary(), null); + } +} diff --git a/fe/src/main/java/org/apache/impala/util/paimon/PaimonArrowFieldWriterFactory.java b/fe/src/main/java/org/apache/impala/util/paimon/PaimonArrowFieldWriterFactory.java new file mode 100644 index 000000000..ad6873418 --- /dev/null +++ b/fe/src/main/java/org/apache/impala/util/paimon/PaimonArrowFieldWriterFactory.java @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.impala.util.paimon; + +import org.apache.arrow.vector.FieldVector; +import org.apache.arrow.vector.VarBinaryVector; +import org.apache.paimon.arrow.writer.ArrowFieldWriter; +import org.apache.paimon.arrow.writer.ArrowFieldWriterFactory; +import org.apache.paimon.arrow.writer.ArrowFieldWriterFactoryVisitor; +import org.apache.paimon.data.DataGetters; +import org.apache.paimon.data.Decimal; +import org.apache.paimon.data.columnar.ColumnVector; +import org.apache.paimon.data.columnar.DecimalColumnVector; +import org.apache.paimon.types.DecimalType; + +import javax.annotation.Nullable; + +/** + * It is an extension of {@link ArrowFieldWriterFactoryVisitor} class. + * To change the decimal field writer behavior. + * Will directly convert paimon decimal type to arrow binary value. + */ +public class PaimonArrowFieldWriterFactory extends ArrowFieldWriterFactoryVisitor { + @Override + public ArrowFieldWriterFactory visit(DecimalType decimalType) { + return (fieldVector, isNullable) + -> new DecimalWriter(fieldVector, decimalType.getPrecision(), + decimalType.getScale(), isNullable); + } + + public static class DecimalWriter extends ArrowFieldWriter { + // decimal precision + private final int precision_; + // decimal scale + private final int scale_; + + public DecimalWriter( + FieldVector fieldVector, int precision, int scale, boolean isNullable) { + super(fieldVector, isNullable); + this.precision_ = precision; + this.scale_ = scale; + } + + protected void doWrite(ColumnVector columnVector, @Nullable int[] pickedInColumn, + int startIndex, int batchRows) { + VarBinaryVector decimalVector = (VarBinaryVector) this.fieldVector; + + for (int i = 0; i < batchRows; ++i) { + int row = this.getRowNumber(startIndex, i, pickedInColumn); + if (columnVector.isNullAt(row)) { + decimalVector.setNull(i); + } else { + Decimal value = ((DecimalColumnVector) columnVector) + .getDecimal(row, this.precision_, this.scale_); + byte[] bytes = value.toUnscaledBytes(); + decimalVector.setSafe(i, bytes); + } + } + } + + protected void doWrite(int rowIndex, DataGetters getters, int pos) { + ((VarBinaryVector) this.fieldVector) + .setSafe(rowIndex, + getters.getDecimal(pos, this.precision_, this.scale_).toUnscaledBytes()); + } + } +} diff --git a/fe/src/main/java/org/apache/impala/util/paimon/PaimonArrowFormatNativeWriter.java b/fe/src/main/java/org/apache/impala/util/paimon/PaimonArrowFormatNativeWriter.java new file mode 100644 index 000000000..ff65ff96b --- /dev/null +++ b/fe/src/main/java/org/apache/impala/util/paimon/PaimonArrowFormatNativeWriter.java @@ -0,0 +1,94 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.impala.util.paimon; + +import org.apache.arrow.c.ArrowArray; +import org.apache.arrow.c.ArrowSchema; +import org.apache.arrow.memory.BufferAllocator; +import org.apache.arrow.vector.VectorSchemaRoot; +import org.apache.paimon.arrow.vector.ArrowCStruct; +import org.apache.paimon.arrow.vector.ArrowFormatCWriter; +import org.apache.paimon.data.InternalRow; +import org.apache.paimon.types.RowType; + +/** + * The wrapper of {@link PaimonArrowFormatWriter} to expose JVM off heap address to + * BE. + * TODO: this class is based on ${@link ArrowFormatCWriter} to allow the customization + * of Field writer. will remove if relevant PR is accepted by paimon + * community. Refer to + * ${@link ...} + * for more detail. + */ +public class PaimonArrowFormatNativeWriter implements AutoCloseable { + // arrow array vector + private final ArrowArray array_; + // arrow schema + private final ArrowSchema schema_; + // arrow RecordBatch writer. + private final PaimonArrowFormatWriter realWriter_; + + public PaimonArrowFormatNativeWriter( + RowType rowType, int writeBatchSize, boolean caseSensitive) { + this(new PaimonArrowFormatWriter(rowType, writeBatchSize, caseSensitive)); + } + + public PaimonArrowFormatNativeWriter(RowType rowType, int writeBatchSize, + boolean caseSensitive, BufferAllocator allocator) { + this(new PaimonArrowFormatWriter(rowType, writeBatchSize, caseSensitive, allocator)); + } + + private PaimonArrowFormatNativeWriter(PaimonArrowFormatWriter arrowFormatWriter) { + this.realWriter_ = arrowFormatWriter; + BufferAllocator allocator = realWriter_.getAllocator(); + array_ = ArrowArray.allocateNew(allocator); + schema_ = ArrowSchema.allocateNew(allocator); + } + + public boolean write(InternalRow currentRow) { return realWriter_.write(currentRow); } + + public ArrowCStruct flush() { + realWriter_.flush(); + VectorSchemaRoot vectorSchemaRoot = realWriter_.getVectorSchemaRoot(); + return PaimonArrowUtils.serializeToCStruct( + vectorSchemaRoot, array_, schema_, realWriter_.getAllocator()); + } + + public void reset() { realWriter_.reset(); } + + public boolean empty() { return realWriter_.empty(); } + + public void release() { + array_.release(); + schema_.release(); + } + + @Override + public void close() { + array_.close(); + schema_.close(); + realWriter_.close(); + } + + public VectorSchemaRoot getVectorSchemaRoot() { + return realWriter_.getVectorSchemaRoot(); + } + + public BufferAllocator getAllocator() { return realWriter_.getAllocator(); } +} diff --git a/fe/src/main/java/org/apache/impala/util/paimon/PaimonArrowFormatWriter.java b/fe/src/main/java/org/apache/impala/util/paimon/PaimonArrowFormatWriter.java new file mode 100644 index 000000000..12d427c31 --- /dev/null +++ b/fe/src/main/java/org/apache/impala/util/paimon/PaimonArrowFormatWriter.java @@ -0,0 +1,126 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.impala.util.paimon; + +import org.apache.arrow.memory.BufferAllocator; +import org.apache.arrow.memory.RootAllocator; +import org.apache.arrow.vector.VectorSchemaRoot; +import org.apache.arrow.vector.util.OversizedAllocationException; +import org.apache.paimon.arrow.vector.ArrowFormatWriter; +import org.apache.paimon.arrow.writer.ArrowFieldWriter; +import org.apache.paimon.arrow.writer.ArrowFieldWriterFactoryVisitor; +import org.apache.paimon.data.InternalRow; +import org.apache.paimon.types.DataType; +import org.apache.paimon.types.RowType; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Write paimon Internal rows to arrow RecordBatch batch in Java side. + * TODO: this class is based on ${@link ArrowFormatWriter} to allow the customization + * of Field writer. will remove if relevant PR is accepted by paimon + * community. Refer to + * ${@link ...} + * for more detail. + */ +public class PaimonArrowFormatWriter implements AutoCloseable { + private static final Logger LOG = + LoggerFactory.getLogger(PaimonArrowFormatWriter.class); + // writer factory + private static final ArrowFieldWriterFactoryVisitor FIELD_WRITER_FACTORY = + new PaimonArrowFieldWriterFactory(); + // field type factory + private static final PaimonArrowFieldTypeFactory FIELD_TYPE_FACTORY = + new PaimonArrowFieldTypeFactory(); + // arrow vector schema root + private final VectorSchemaRoot vectorSchemaRoot_; + // arrow field writers + private final ArrowFieldWriter[] fieldWriters_; + // arrow RecordBatch batch size + private final int batchSize_; + // buffer allocator. + private final BufferAllocator allocator_; + // rowid for current batch. + private int rowId_; + + public PaimonArrowFormatWriter( + RowType rowType, int writeBatchSize, boolean caseSensitive) { + this(rowType, writeBatchSize, caseSensitive, new RootAllocator()); + } + + public PaimonArrowFormatWriter(RowType rowType, int writeBatchSize, + boolean caseSensitive, BufferAllocator allocator) { + this(rowType, writeBatchSize, caseSensitive, allocator, FIELD_WRITER_FACTORY); + } + + public PaimonArrowFormatWriter(RowType rowType, int writeBatchSize, + boolean caseSensitive, BufferAllocator allocator, + ArrowFieldWriterFactoryVisitor fieldWriterFactory) { + this.allocator_ = allocator; + + vectorSchemaRoot_ = PaimonArrowUtils.createVectorSchemaRoot( + rowType, allocator, caseSensitive, FIELD_TYPE_FACTORY); + + fieldWriters_ = new ArrowFieldWriter[rowType.getFieldCount()]; + + for (int i = 0; i < fieldWriters_.length; i++) { + DataType type = rowType.getFields().get(i).type(); + fieldWriters_[i] = type.accept(fieldWriterFactory) + .create(vectorSchemaRoot_.getVector(i), type.isNullable()); + } + + this.batchSize_ = writeBatchSize; + } + + public void flush() { vectorSchemaRoot_.setRowCount(rowId_); } + + public boolean write(InternalRow currentRow) { + if (rowId_ >= batchSize_) { return false; } + for (int i = 0; i < currentRow.getFieldCount(); i++) { + try { + fieldWriters_[i].write(rowId_, currentRow, i); + } catch (OversizedAllocationException | IndexOutOfBoundsException e) { + // maybe out of memory + LOG.warn("Arrow field writer failed while writing", e); + return false; + } + } + + rowId_++; + return true; + } + + public boolean empty() { return rowId_ == 0; } + + public void reset() { + for (ArrowFieldWriter fieldWriter : fieldWriters_) { fieldWriter.reset(); } + rowId_ = 0; + } + + @Override + public void close() { + vectorSchemaRoot_.close(); + allocator_.close(); + } + + public VectorSchemaRoot getVectorSchemaRoot() { return vectorSchemaRoot_; } + + public BufferAllocator getAllocator() { return allocator_; } +} diff --git a/fe/src/main/java/org/apache/impala/util/paimon/PaimonArrowUtils.java b/fe/src/main/java/org/apache/impala/util/paimon/PaimonArrowUtils.java new file mode 100644 index 000000000..059cc6297 --- /dev/null +++ b/fe/src/main/java/org/apache/impala/util/paimon/PaimonArrowUtils.java @@ -0,0 +1,137 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.impala.util.paimon; + +import org.apache.arrow.c.ArrowArray; +import org.apache.arrow.c.ArrowSchema; +import org.apache.arrow.c.Data; +import org.apache.arrow.memory.BufferAllocator; +import org.apache.arrow.vector.VectorSchemaRoot; +import org.apache.arrow.vector.complex.ListVector; +import org.apache.arrow.vector.complex.MapVector; +import org.apache.arrow.vector.types.Types; +import org.apache.arrow.vector.types.pojo.Field; +import org.apache.arrow.vector.types.pojo.FieldType; +import org.apache.arrow.vector.types.pojo.Schema; +import org.apache.paimon.arrow.ArrowFieldTypeConversion; +import org.apache.paimon.arrow.ArrowUtils; +import org.apache.paimon.arrow.vector.ArrowCStruct; +import org.apache.paimon.table.SpecialFields; +import org.apache.paimon.types.ArrayType; +import org.apache.paimon.types.DataField; +import org.apache.paimon.types.DataType; +import org.apache.paimon.types.MapType; +import org.apache.paimon.types.RowType; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; + +import static org.apache.paimon.utils.StringUtils.toLowerCaseIfNeed; + +/** + * Utilities for creating Arrow objects. + * TODO: this class is based on ${@link ArrowUtils} to allow the customization + * of Field writer. will remove if relevant PR is accepted by paimon + * community. Refer to + * ${@link ...} + * for more detail. + */ +public class PaimonArrowUtils { + static final String PARQUET_FIELD_ID = "PARQUET:field_id"; + + public static VectorSchemaRoot createVectorSchemaRoot(RowType rowType, + BufferAllocator allocator, boolean caseSensitive, + ArrowFieldTypeConversion.ArrowFieldTypeVisitor visitor) { + List fields = + rowType.getFields() + .stream() + .map(f + -> toArrowField(toLowerCaseIfNeed(f.name(), caseSensitive), f.id(), + f.type(), visitor, 0)) + .collect(Collectors.toList()); + return VectorSchemaRoot.create(new Schema(fields), allocator); + } + + public static Field toArrowField(String fieldName, int fieldId, DataType dataType, + ArrowFieldTypeConversion.ArrowFieldTypeVisitor visitor, int depth) { + FieldType fieldType = dataType.accept(visitor); + fieldType = new FieldType(fieldType.isNullable(), fieldType.getType(), + fieldType.getDictionary(), + Collections.singletonMap(PARQUET_FIELD_ID, String.valueOf(fieldId))); + List children = null; + if (dataType instanceof ArrayType) { + Field field = toArrowField(ListVector.DATA_VECTOR_NAME, fieldId, + ((ArrayType) dataType).getElementType(), visitor, depth + 1); + FieldType typeInner = field.getFieldType(); + field = new Field(field.getName(), + new FieldType(typeInner.isNullable(), typeInner.getType(), + typeInner.getDictionary(), + Collections.singletonMap(PARQUET_FIELD_ID, + String.valueOf( + SpecialFields.getArrayElementFieldId(fieldId, depth + 1)))), + field.getChildren()); + children = Collections.singletonList(field); + } else if (dataType instanceof MapType) { + MapType mapType = (MapType) dataType; + + Field keyField = toArrowField(MapVector.KEY_NAME, fieldId, + mapType.getKeyType().notNull(), visitor, depth + 1); + FieldType keyType = keyField.getFieldType(); + keyField = new Field(keyField.getName(), + new FieldType(keyType.isNullable(), keyType.getType(), keyType.getDictionary(), + Collections.singletonMap(PARQUET_FIELD_ID, + String.valueOf(SpecialFields.getMapKeyFieldId(fieldId, depth + 1)))), + keyField.getChildren()); + + Field valueField = toArrowField(MapVector.VALUE_NAME, fieldId, + mapType.getValueType().notNull(), visitor, depth + 1); + FieldType valueType = valueField.getFieldType(); + valueField = new Field(valueField.getName(), + new FieldType(valueType.isNullable(), valueType.getType(), + valueType.getDictionary(), + Collections.singletonMap(PARQUET_FIELD_ID, + String.valueOf(SpecialFields.getMapValueFieldId(fieldId, depth + 1)))), + valueField.getChildren()); + + FieldType structType = new FieldType(false, Types.MinorType.STRUCT.getType(), null, + Collections.singletonMap(PARQUET_FIELD_ID, String.valueOf(fieldId))); + Field mapField = new Field(MapVector.DATA_VECTOR_NAME, + // data vector, key vector and value vector CANNOT be null + structType, Arrays.asList(keyField, valueField)); + + children = Collections.singletonList(mapField); + } else if (dataType instanceof RowType) { + RowType rowType = (RowType) dataType; + children = new ArrayList<>(); + for (DataField field : rowType.getFields()) { + children.add(toArrowField(field.name(), field.id(), field.type(), visitor, 0)); + } + } + return new Field(fieldName, fieldType, children); + } + + public static ArrowCStruct serializeToCStruct(VectorSchemaRoot vsr, ArrowArray array, + ArrowSchema schema, BufferAllocator bufferAllocator) { + Data.exportVectorSchemaRoot(bufferAllocator, vsr, null, array, schema); + return ArrowCStruct.of(array, schema); + } +} diff --git a/fe/src/main/java/org/apache/impala/util/paimon/PaimonJniScanner.java b/fe/src/main/java/org/apache/impala/util/paimon/PaimonJniScanner.java new file mode 100644 index 000000000..6fdf36d21 --- /dev/null +++ b/fe/src/main/java/org/apache/impala/util/paimon/PaimonJniScanner.java @@ -0,0 +1,221 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.impala.util.paimon; + +import com.google.common.collect.Lists; +import org.apache.arrow.memory.BufferAllocator; +import org.apache.commons.lang3.SerializationUtils; +import org.apache.impala.catalog.paimon.PaimonUtil; +import org.apache.impala.common.ImpalaException; +import org.apache.impala.common.JniUtil; +import org.apache.impala.planner.paimon.PaimonSplit; +import org.apache.impala.thrift.TPaimonJniScanParam; +import org.apache.paimon.arrow.vector.ArrowCStruct; +import org.apache.paimon.data.InternalRow; +import org.apache.paimon.predicate.Predicate; +import org.apache.paimon.reader.RecordReader; +import org.apache.paimon.reader.RecordReaderIterator; +import org.apache.paimon.table.Table; +import org.apache.paimon.table.source.ReadBuilder; +import org.apache.paimon.table.source.Split; +import org.apache.paimon.types.DataField; +import org.apache.paimon.types.RowType; +import org.apache.thrift.protocol.TBinaryProtocol; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; + +/** + * The Fe Paimon Jni Scanner, used by backend PaimonJniScanner. + */ +public class PaimonJniScanner implements AutoCloseable { + private final static Logger LOG = LoggerFactory.getLogger(PaimonJniScanner.class); + + public final static int DEFAULT_ROWBATCH_SIZE = 1024; + public final static long DEFAULT_INITIAL_RESERVATION = 32 * 1024; + + private final static TBinaryProtocol.Factory protocolFactory_ = + new TBinaryProtocol.Factory(); + // Paimon api table. + private Table table_ = null; + // Paimon splits assigned to the scanner. + private List splits_ = null; + // Paimon schema after projection. + private RowType projectedSchema_; + // Paimon data record iterator. + private RecordReaderIterator iterator_; + // batch size; + int batchSize_; + // paimon to arrow RecordBatch writer. + private PaimonArrowFormatNativeWriter writer_; + // arrow off heap allocator. + private BufferAllocator bufferAllocator_; + // total rows metrics. + private long totalRows_ = 0; + // upper bound mem limit. + // -1 means no limit. + private long allocator_mem_limit_ = -1; + + /** + * Constructor for PaimonJniScanner, will be called in Open + * method of BE PaimonJniScanNode. + * @param jni_scan_param_thrift: thrift form of paimon scan param. + * */ + public PaimonJniScanner(byte[] jni_scan_param_thrift) { + TPaimonJniScanParam paimonJniScanParam = new TPaimonJniScanParam(); + try { + JniUtil.deserializeThrift( + protocolFactory_, paimonJniScanParam, jni_scan_param_thrift); + } catch (ImpalaException ex) { LOG.error("failed to get paimon jni scan param"); } + // table + table_ = SerializationUtils.deserialize(paimonJniScanParam.getPaimon_table_obj()); + // splits + splits_ = Lists.newArrayList(); + for (ByteBuffer split_data : paimonJniScanParam.getSplits()) { + ByteBuffer split_data_serialized = split_data.compact(); + splits_.add(SerializationUtils.deserialize(split_data_serialized.array())); + } + // projection field ids + int[] projectionFieldIds = + paimonJniScanParam.getProjection().stream().mapToInt(Integer::intValue).toArray(); + // projected fields and schema + DataField[] projectedFields = + Arrays.stream(projectionFieldIds) + .mapToObj(fieldId -> table_.rowType().getField(fieldId)) + .toArray(DataField[] ::new); + projectedSchema_ = RowType.of(projectedFields); + // get batch size + batchSize_ = paimonJniScanParam.getBatch_size(); + if (batchSize_ <= 0) { batchSize_ = DEFAULT_ROWBATCH_SIZE; } + // get mem limit + allocator_mem_limit_ = paimonJniScanParam.getMem_limit_bytes(); + String allocatorName = + "paimonscan_" + table_.uuid() + paimonJniScanParam.getFragment_id().toString(); + // create allocator + if (allocator_mem_limit_ > 0) { + bufferAllocator_ = ArrowRootAllocation.rootAllocator().newChildAllocator( + allocatorName, DEFAULT_INITIAL_RESERVATION, allocator_mem_limit_); + } else { + bufferAllocator_ = ArrowRootAllocation.rootAllocator().newChildAllocator( + allocatorName, DEFAULT_INITIAL_RESERVATION, Long.MAX_VALUE); + } + LOG.info(String.format("Open with mem_limit: %d bytes, batch_size:%d rows, " + + "Projection field ids:%s", + allocator_mem_limit_, batchSize_, Arrays.toString(projectionFieldIds))); + } + + /** + * Perform table splits scanning, will be called in Open + * method of BE PaimonJniScanNode. + * */ + public void ScanTable() { + // If we are on a stack frame that was created through JNI we need to set the context + // class loader as Paimon might use reflection to dynamically load classes and + // methods. + JniUtil.setContextClassLoaderForThisThread(this.getClass().getClassLoader()); + writer_ = new PaimonArrowFormatNativeWriter( + projectedSchema_, batchSize_, false, bufferAllocator_); + // Create and scan the metadata table + initReader(); + } + + /** + * Get the next arrow batch, will be called in GetNext + * method of BE PaimonJniScanNode. + * @param address: return three long values to BE + * address[0]: schema address of arrow batch. + * address[1]: vector address of arrow batch. + * address[2]: offheap memory consumption for current batch. + * */ + public long GetNextBatch(long[] address) { + if (!writer_.empty()) { writer_.reset(); } + int rows = 0; + for (int i = 0; i < batchSize_; i++) { + if (iterator_.hasNext()) { + boolean result = writer_.write(iterator_.next()); + if (result) { rows++; } + } else { + break; + } + } + totalRows_ += rows; + if (rows > 0) { + ArrowCStruct cStruct = writer_.flush(); + address[0] = cStruct.schemaAddress(); + address[1] = cStruct.arrayAddress(); + address[2] = bufferAllocator_.getAllocatedMemory(); + return rows; + } else { + return 0; + } + } + + protected boolean initReader() { + try { + ReadBuilder readBuilder = table_.newReadBuilder().withReadType(projectedSchema_); + // Apply push down predicates if present. + // Currently predicates are always null/empty, + // All conjuncts are evaluated by the C++ scanner. + List predicates = splits_.get(0).getPredicates(); + if (predicates != null && !predicates.isEmpty()) { + readBuilder.withFilter(predicates); + } + // Create Iterator for given splits. + List splits = + splits_.stream().map(PaimonSplit::getSplit).collect(Collectors.toList()); + RecordReader reader = readBuilder.newRead().createReader(splits); + iterator_ = new RecordReaderIterator<>(reader); + LOG.info( + String.format("Reading %d splits for %s", splits.size(), table_.fullName())); + return true; + } catch (Exception ex) { + LOG.error("failed to init reader for " + table_.fullName(), ex); + return false; + } + } + + /** + * Perform clean up operation , will be called in Close + * method of BE PaimonJniScanNode. + * */ + @Override + public void close() throws Exception { + // release writer resources. + PaimonUtil.closeQuitely(writer_); + + // release arrow allocator resources owned by current scanner. + PaimonUtil.closeQuitely(bufferAllocator_); + // used to check mem leak in more detail if arrow allocation + // debug is turned on. + if (bufferAllocator_.getAllocatedMemory() > 0) { + LOG.error( + String.format("Leaked memory for %s is %d bytes, dump:%s", table_.fullName(), + bufferAllocator_.getAllocatedMemory(), bufferAllocator_.toVerboseString())); + } + LOG.info(String.format("Peak memory for %s is %d bytes, total rows: %d", + table_.fullName(), bufferAllocator_.getPeakMemoryAllocation(), totalRows_)); + + // release iterator resources + PaimonUtil.closeQuitely(iterator_); + } +} diff --git a/fe/src/test/java/org/apache/impala/catalog/paimon/ImpalaTypeUtilsTest.java b/fe/src/test/java/org/apache/impala/catalog/paimon/ImpalaTypeUtilsTest.java index 53418b952..9544f4dec 100644 --- a/fe/src/test/java/org/apache/impala/catalog/paimon/ImpalaTypeUtilsTest.java +++ b/fe/src/test/java/org/apache/impala/catalog/paimon/ImpalaTypeUtilsTest.java @@ -97,11 +97,11 @@ public class ImpalaTypeUtilsTest { // Test row type RowType rowType = new RowType(Arrays.asList(new DataField(0, "id", new IntType()), new DataField(1, "name", DataTypes.STRING()))); - StructType expectedStructType = new StructType(Arrays.asList( - new StructField("id", Type.INT, - rowType.getField(0).description()), - new StructField("name", Type.STRING, - rowType.getField(1).description()))); + StructType expectedStructType = new StructType( + Arrays.asList(new PaimonStructField("id", Type.INT, + rowType.getField(0).description(), rowType.getField(0).id()), + new PaimonStructField("name", Type.STRING, rowType.getField(1).description(), + rowType.getField(1).id()))); assertEquals(expectedStructType, PaimonImpalaTypeUtils.toImpalaType(rowType)); // doesn't support time diff --git a/java/pom.xml b/java/pom.xml index 4eed32513..f9abb3b84 100644 --- a/java/pom.xml +++ b/java/pom.xml @@ -42,6 +42,7 @@ under the License. ${env.IMPALA_HIVE_DIST_TYPE} ${env.IMPALA_HUDI_VERSION} ${env.IMPALA_PAIMON_VERSION} + ${env.IMPALA_ARROW_VERSION} ${env.IMPALA_RANGER_VERSION} ${env.IMPALA_POSTGRES_JDBC_DRIVER_VERSION} ${env.IMPALA_HBASE_VERSION} diff --git a/testdata/data/paimon_test/README.md b/testdata/data/paimon_test/README.md new file mode 100644 index 000000000..752b2b9c6 --- /dev/null +++ b/testdata/data/paimon_test/README.md @@ -0,0 +1,115 @@ +# Paimon Test Tables + +This README.md explain the schema and usage of paimon test tables in the folder. +the table data in the folder will be directly copied to hdfs and external paimon +tables will be created accordingly for these tables. + +## paimon_non_partitioned +The subset of table ratings in movielens dataset, it is not partitioned. +the schema is: ++-----------+--------+---------+ +| userid | int | | +| movieid | int | | +| rating | float | | +| timestamp | bigint | | +-------------------------------- +## paimon_partitioned +The subset of table ratings in movielens dataset, it is partitioned. +the schema is: ++-----------+--------+---------+ +| userid | int | | +| movieid | int | | +| rating | float | | +| timestamp | bigint | | +-------------------------------- +## paimon_primitive_alltypes +The table contains all primitive types supported by paimon: +schema is: ++---------------+---------------+---------+ +| name | type | comment | ++---------------+---------------+---------+ +| bool_value | boolean | | +| tiny_value | tinyint | | +| small_value | smallint | | +| int_value | int | | +| big_value | bigint | | +| float_value | float | | +| double_value | double | | +| decimal_value | decimal(10,2) | | +| char_value | char(10) | | +| varchar_value | varchar(100) | | +| binary_value | binary | | +| date_value | date | | +| ts_ltz_value | timestamp | | +| ts_value | timestamp | | ++---------------+---------------+---------+ +## paimon_decimal_tbl +The table is used to support decimal related test,with various precision and scale. +the schema is: ++------+----------------+---------+ +| name | type | comment | ++------+----------------+---------+ +| d1 | decimal(9,0) | | +| d2 | decimal(10,0) | | +| d3 | decimal(20,10) | | +| d4 | decimal(38,38) | | +| d5 | decimal(10,5) | | ++------+----------------+---------+ +## paimon_decimal_tbl +The table is used to support decimal related test,with various precision and scale. +the schema is: ++------+----------------+---------+ +| name | type | comment | ++------+----------------+---------+ +| d1 | decimal(9,0) | | +| d2 | decimal(10,0) | | +| d3 | decimal(20,10) | | +| d4 | decimal(38,38) | | +| d5 | decimal(10,5) | | ++------+----------------+---------+ +## alltypes_paimon +the table is table alltypes with paimon format, it is used to support test test_scanner +for paimon format. +the schema is: ++-----------------+-----------+---------------+ +| name | type | comment | ++-----------------+-----------+---------------+ +| id | int | Add a comment | +| bool_col | boolean | | +| tinyint_col | tinyint | | +| smallint_col | smallint | | +| int_col | int | | +| bigint_col | bigint | | +| float_col | float | | +| double_col | double | | +| date_string_col | string | | +| string_col | string | | +| timestamp_col | timestamp | | +| year | int | | +| month | int | | ++-----------------+-----------+---------------+ +## alltypes_structs_paimon +the table is table alltypes_structs with paimon format, it is used to support negative +cases for complex and nested field query suuport. ++------------+---------------------------+---------+ +| name | type | comment | ++------------+---------------------------+---------+ +| id | int | | +| struct_val | struct< | | +| | bool_col:boolean, | | +| | tinyint_col:tinyint, | | +| | smallint_col:smallint, | | +| | int_col:int, | | +| | bigint_col:bigint, | | +| | float_col:float, | | +| | double_col:double, | | +| | date_string_col:string, | | +| | string_col:string | | +| | > | | +| year | int | | +| month | int | | +---------------------------------------------------- +## TODO: +Most of testing tables should be removed later once paimon write is supported +for impala, if these table generation can be easily implemented using DML or +CTAS statement. \ No newline at end of file diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_paimon/bucket-0/data-275038ac-cf80-4caa-b9c3-43854884f257-0.parquet b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_paimon/bucket-0/data-275038ac-cf80-4caa-b9c3-43854884f257-0.parquet new file mode 100644 index 000000000..bd689f6d4 Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_paimon/bucket-0/data-275038ac-cf80-4caa-b9c3-43854884f257-0.parquet differ diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_paimon/bucket-0/data-4724d1a4-637d-4275-8439-d4edd1a4583e-0.parquet b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_paimon/bucket-0/data-4724d1a4-637d-4275-8439-d4edd1a4583e-0.parquet new file mode 100644 index 000000000..c4fab82ec Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_paimon/bucket-0/data-4724d1a4-637d-4275-8439-d4edd1a4583e-0.parquet differ diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_paimon/bucket-0/data-7201ded8-6883-42ef-a39f-a99fc8a83938-0.parquet b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_paimon/bucket-0/data-7201ded8-6883-42ef-a39f-a99fc8a83938-0.parquet new file mode 100644 index 000000000..576ebb557 Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_paimon/bucket-0/data-7201ded8-6883-42ef-a39f-a99fc8a83938-0.parquet differ diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_paimon/bucket-0/data-7e80aa3a-5863-4cb5-bdb4-84d0d9002dd0-0.parquet b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_paimon/bucket-0/data-7e80aa3a-5863-4cb5-bdb4-84d0d9002dd0-0.parquet new file mode 100644 index 000000000..db82bbf86 Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_paimon/bucket-0/data-7e80aa3a-5863-4cb5-bdb4-84d0d9002dd0-0.parquet differ diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_paimon/bucket-0/data-81c20e92-de0c-4ea1-bf5e-989ecb844645-0.parquet b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_paimon/bucket-0/data-81c20e92-de0c-4ea1-bf5e-989ecb844645-0.parquet new file mode 100644 index 000000000..e6228a08a Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_paimon/bucket-0/data-81c20e92-de0c-4ea1-bf5e-989ecb844645-0.parquet differ diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_paimon/bucket-0/data-9a6dbc25-fd98-471d-adca-02aa6c8f2751-0.parquet b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_paimon/bucket-0/data-9a6dbc25-fd98-471d-adca-02aa6c8f2751-0.parquet new file mode 100644 index 000000000..8a4561541 Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_paimon/bucket-0/data-9a6dbc25-fd98-471d-adca-02aa6c8f2751-0.parquet differ diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_paimon/bucket-0/data-9b24cb51-8c3a-448b-a0f3-605ed71e7caa-0.parquet b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_paimon/bucket-0/data-9b24cb51-8c3a-448b-a0f3-605ed71e7caa-0.parquet new file mode 100644 index 000000000..d2fe63a2a Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_paimon/bucket-0/data-9b24cb51-8c3a-448b-a0f3-605ed71e7caa-0.parquet differ diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_paimon/bucket-0/data-c079a770-9eb6-4b91-b3c8-f6c84fbaf7ce-0.parquet b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_paimon/bucket-0/data-c079a770-9eb6-4b91-b3c8-f6c84fbaf7ce-0.parquet new file mode 100644 index 000000000..282b23ab5 Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_paimon/bucket-0/data-c079a770-9eb6-4b91-b3c8-f6c84fbaf7ce-0.parquet differ diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_paimon/bucket-0/data-d943b655-eed4-4ee0-9a2e-024ff4596832-0.parquet b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_paimon/bucket-0/data-d943b655-eed4-4ee0-9a2e-024ff4596832-0.parquet new file mode 100644 index 000000000..4f64e81db Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_paimon/bucket-0/data-d943b655-eed4-4ee0-9a2e-024ff4596832-0.parquet differ diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_paimon/bucket-0/data-dffe00a3-de35-4a74-96df-cd0fb7d08070-0.parquet b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_paimon/bucket-0/data-dffe00a3-de35-4a74-96df-cd0fb7d08070-0.parquet new file mode 100644 index 000000000..24163a021 Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_paimon/bucket-0/data-dffe00a3-de35-4a74-96df-cd0fb7d08070-0.parquet differ diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_paimon/bucket-0/data-e6940374-4033-48c7-b6d9-ebca0e7d0a4d-0.parquet b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_paimon/bucket-0/data-e6940374-4033-48c7-b6d9-ebca0e7d0a4d-0.parquet new file mode 100644 index 000000000..f8ad4f544 Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_paimon/bucket-0/data-e6940374-4033-48c7-b6d9-ebca0e7d0a4d-0.parquet differ diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_paimon/bucket-0/data-eef44c7a-9856-4817-8f1f-a2a7dbc331c5-0.parquet b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_paimon/bucket-0/data-eef44c7a-9856-4817-8f1f-a2a7dbc331c5-0.parquet new file mode 100644 index 000000000..5e4ad052b Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_paimon/bucket-0/data-eef44c7a-9856-4817-8f1f-a2a7dbc331c5-0.parquet differ diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_paimon/manifest/manifest-fab63fbc-acd6-424a-ac4e-2a848ea51f52-0 b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_paimon/manifest/manifest-fab63fbc-acd6-424a-ac4e-2a848ea51f52-0 new file mode 100644 index 000000000..4a2f6fc3b Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_paimon/manifest/manifest-fab63fbc-acd6-424a-ac4e-2a848ea51f52-0 differ diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_paimon/manifest/manifest-list-fa3d4f50-7f45-4d37-b5e7-22e6cf3f2cb0-0 b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_paimon/manifest/manifest-list-fa3d4f50-7f45-4d37-b5e7-22e6cf3f2cb0-0 new file mode 100644 index 000000000..d6ce64370 Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_paimon/manifest/manifest-list-fa3d4f50-7f45-4d37-b5e7-22e6cf3f2cb0-0 differ diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_paimon/manifest/manifest-list-fa3d4f50-7f45-4d37-b5e7-22e6cf3f2cb0-1 b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_paimon/manifest/manifest-list-fa3d4f50-7f45-4d37-b5e7-22e6cf3f2cb0-1 new file mode 100644 index 000000000..83a4052ba Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_paimon/manifest/manifest-list-fa3d4f50-7f45-4d37-b5e7-22e6cf3f2cb0-1 differ diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_paimon/schema/schema-0 b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_paimon/schema/schema-0 new file mode 100644 index 000000000..e340e367c --- /dev/null +++ b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_paimon/schema/schema-0 @@ -0,0 +1,65 @@ +{ + "version" : 3, + "id" : 0, + "fields" : [ { + "id" : 0, + "name" : "id", + "type" : "INT", + "description" : "Add a comment" + }, { + "id" : 1, + "name" : "bool_col", + "type" : "BOOLEAN" + }, { + "id" : 2, + "name" : "tinyint_col", + "type" : "TINYINT" + }, { + "id" : 3, + "name" : "smallint_col", + "type" : "SMALLINT" + }, { + "id" : 4, + "name" : "int_col", + "type" : "INT" + }, { + "id" : 5, + "name" : "bigint_col", + "type" : "BIGINT" + }, { + "id" : 6, + "name" : "float_col", + "type" : "FLOAT" + }, { + "id" : 7, + "name" : "double_col", + "type" : "DOUBLE" + }, { + "id" : 8, + "name" : "date_string_col", + "type" : "STRING" + }, { + "id" : 9, + "name" : "string_col", + "type" : "STRING" + }, { + "id" : 10, + "name" : "timestamp_col", + "type" : "TIMESTAMP(6) WITH LOCAL TIME ZONE" + }, { + "id" : 11, + "name" : "year", + "type" : "INT" + }, { + "id" : 12, + "name" : "month", + "type" : "INT" + } ], + "highestFieldId" : 12, + "partitionKeys" : [ ], + "primaryKeys" : [ ], + "options" : { + "owner" : "jichen" + }, + "timeMillis" : 1758785836201 +} \ No newline at end of file diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_paimon/snapshot/EARLIEST b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_paimon/snapshot/EARLIEST new file mode 100644 index 000000000..56a6051ca --- /dev/null +++ b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_paimon/snapshot/EARLIEST @@ -0,0 +1 @@ +1 \ No newline at end of file diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_paimon/snapshot/LATEST b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_paimon/snapshot/LATEST new file mode 100644 index 000000000..56a6051ca --- /dev/null +++ b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_paimon/snapshot/LATEST @@ -0,0 +1 @@ +1 \ No newline at end of file diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_paimon/snapshot/snapshot-1 b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_paimon/snapshot/snapshot-1 new file mode 100644 index 000000000..1cbbe3127 --- /dev/null +++ b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_paimon/snapshot/snapshot-1 @@ -0,0 +1,18 @@ +{ + "version" : 3, + "id" : 1, + "schemaId" : 0, + "baseManifestList" : "manifest-list-fa3d4f50-7f45-4d37-b5e7-22e6cf3f2cb0-0", + "baseManifestListSize" : 884, + "deltaManifestList" : "manifest-list-fa3d4f50-7f45-4d37-b5e7-22e6cf3f2cb0-1", + "deltaManifestListSize" : 986, + "changelogManifestList" : null, + "commitUser" : "68feb84e-06bb-43d3-a5bc-0f3d6e801075", + "commitIdentifier" : 9223372036854775807, + "commitKind" : "APPEND", + "timeMillis" : 1758785841642, + "logOffsets" : { }, + "totalRecordCount" : 7300, + "deltaRecordCount" : 7300, + "changelogRecordCount" : 0 +} \ No newline at end of file diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_structs_paimon/bucket-0/data-04a2d395-e55d-4720-bdf9-d0aea3177a01-0.parquet b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_structs_paimon/bucket-0/data-04a2d395-e55d-4720-bdf9-d0aea3177a01-0.parquet new file mode 100644 index 000000000..84f227b34 Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_structs_paimon/bucket-0/data-04a2d395-e55d-4720-bdf9-d0aea3177a01-0.parquet differ diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_structs_paimon/bucket-0/data-0ff3cbcd-bd56-43ef-8b00-543a83cc5e6f-0.parquet b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_structs_paimon/bucket-0/data-0ff3cbcd-bd56-43ef-8b00-543a83cc5e6f-0.parquet new file mode 100644 index 000000000..fd24e7cb0 Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_structs_paimon/bucket-0/data-0ff3cbcd-bd56-43ef-8b00-543a83cc5e6f-0.parquet differ diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_structs_paimon/bucket-0/data-15fe0349-ce6f-4228-9031-f1d556dfc808-0.parquet b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_structs_paimon/bucket-0/data-15fe0349-ce6f-4228-9031-f1d556dfc808-0.parquet new file mode 100644 index 000000000..acbc979e6 Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_structs_paimon/bucket-0/data-15fe0349-ce6f-4228-9031-f1d556dfc808-0.parquet differ diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_structs_paimon/bucket-0/data-43259d5c-33d4-4fa2-bed0-40e8966ed9e0-0.parquet b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_structs_paimon/bucket-0/data-43259d5c-33d4-4fa2-bed0-40e8966ed9e0-0.parquet new file mode 100644 index 000000000..aa26cfeac Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_structs_paimon/bucket-0/data-43259d5c-33d4-4fa2-bed0-40e8966ed9e0-0.parquet differ diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_structs_paimon/bucket-0/data-4e2d864e-0aa9-4b5e-8b6d-4261540072f3-0.parquet b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_structs_paimon/bucket-0/data-4e2d864e-0aa9-4b5e-8b6d-4261540072f3-0.parquet new file mode 100644 index 000000000..06d457051 Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_structs_paimon/bucket-0/data-4e2d864e-0aa9-4b5e-8b6d-4261540072f3-0.parquet differ diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_structs_paimon/bucket-0/data-602c116d-3e51-40b6-914d-950170ece098-0.parquet b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_structs_paimon/bucket-0/data-602c116d-3e51-40b6-914d-950170ece098-0.parquet new file mode 100644 index 000000000..72f84d0f2 Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_structs_paimon/bucket-0/data-602c116d-3e51-40b6-914d-950170ece098-0.parquet differ diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_structs_paimon/bucket-0/data-6a5bf48f-e00f-45dc-8d44-62582cf6d881-0.parquet b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_structs_paimon/bucket-0/data-6a5bf48f-e00f-45dc-8d44-62582cf6d881-0.parquet new file mode 100644 index 000000000..c569e3be2 Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_structs_paimon/bucket-0/data-6a5bf48f-e00f-45dc-8d44-62582cf6d881-0.parquet differ diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_structs_paimon/bucket-0/data-72d3fa1b-ff44-4088-a2d6-22902d35e795-0.parquet b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_structs_paimon/bucket-0/data-72d3fa1b-ff44-4088-a2d6-22902d35e795-0.parquet new file mode 100644 index 000000000..bcc0e68e9 Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_structs_paimon/bucket-0/data-72d3fa1b-ff44-4088-a2d6-22902d35e795-0.parquet differ diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_structs_paimon/bucket-0/data-7f917551-249f-4a13-9ec9-43be271f7b21-0.parquet b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_structs_paimon/bucket-0/data-7f917551-249f-4a13-9ec9-43be271f7b21-0.parquet new file mode 100644 index 000000000..b2d96f6b7 Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_structs_paimon/bucket-0/data-7f917551-249f-4a13-9ec9-43be271f7b21-0.parquet differ diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_structs_paimon/bucket-0/data-8b7a002b-cf8d-4d17-ba96-0389d2dfb456-0.parquet b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_structs_paimon/bucket-0/data-8b7a002b-cf8d-4d17-ba96-0389d2dfb456-0.parquet new file mode 100644 index 000000000..3426f43ac Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_structs_paimon/bucket-0/data-8b7a002b-cf8d-4d17-ba96-0389d2dfb456-0.parquet differ diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_structs_paimon/bucket-0/data-b5785d5b-c764-429f-82c5-aa76bbf15a63-0.parquet b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_structs_paimon/bucket-0/data-b5785d5b-c764-429f-82c5-aa76bbf15a63-0.parquet new file mode 100644 index 000000000..df6a4324c Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_structs_paimon/bucket-0/data-b5785d5b-c764-429f-82c5-aa76bbf15a63-0.parquet differ diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_structs_paimon/bucket-0/data-f71174f8-719a-4ed5-84d0-244a6c26604b-0.parquet b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_structs_paimon/bucket-0/data-f71174f8-719a-4ed5-84d0-244a6c26604b-0.parquet new file mode 100644 index 000000000..df2b72d14 Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_structs_paimon/bucket-0/data-f71174f8-719a-4ed5-84d0-244a6c26604b-0.parquet differ diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_structs_paimon/manifest/manifest-85ea4264-6480-4517-9d44-2a5ba022a922-0 b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_structs_paimon/manifest/manifest-85ea4264-6480-4517-9d44-2a5ba022a922-0 new file mode 100644 index 000000000..d897e9527 Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_structs_paimon/manifest/manifest-85ea4264-6480-4517-9d44-2a5ba022a922-0 differ diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_structs_paimon/manifest/manifest-list-67f31438-ac2d-4d03-9bd9-4e1d55120c5c-0 b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_structs_paimon/manifest/manifest-list-67f31438-ac2d-4d03-9bd9-4e1d55120c5c-0 new file mode 100644 index 000000000..de568fda8 Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_structs_paimon/manifest/manifest-list-67f31438-ac2d-4d03-9bd9-4e1d55120c5c-0 differ diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_structs_paimon/manifest/manifest-list-67f31438-ac2d-4d03-9bd9-4e1d55120c5c-1 b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_structs_paimon/manifest/manifest-list-67f31438-ac2d-4d03-9bd9-4e1d55120c5c-1 new file mode 100644 index 000000000..b3b74c1a6 Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_structs_paimon/manifest/manifest-list-67f31438-ac2d-4d03-9bd9-4e1d55120c5c-1 differ diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_structs_paimon/schema/schema-0 b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_structs_paimon/schema/schema-0 new file mode 100644 index 000000000..958b0ee0b --- /dev/null +++ b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_structs_paimon/schema/schema-0 @@ -0,0 +1,67 @@ +{ + "version" : 3, + "id" : 0, + "fields" : [ { + "id" : 0, + "name" : "id", + "type" : "INT" + }, { + "id" : 1, + "name" : "struct_val", + "type" : { + "type" : "ROW", + "fields" : [ { + "id" : 2, + "name" : "bool_col", + "type" : "BOOLEAN" + }, { + "id" : 3, + "name" : "tinyint_col", + "type" : "TINYINT" + }, { + "id" : 4, + "name" : "smallint_col", + "type" : "SMALLINT" + }, { + "id" : 5, + "name" : "int_col", + "type" : "INT" + }, { + "id" : 6, + "name" : "bigint_col", + "type" : "BIGINT" + }, { + "id" : 7, + "name" : "float_col", + "type" : "FLOAT" + }, { + "id" : 8, + "name" : "double_col", + "type" : "DOUBLE" + }, { + "id" : 9, + "name" : "date_string_col", + "type" : "STRING" + }, { + "id" : 10, + "name" : "string_col", + "type" : "STRING" + } ] + } + }, { + "id" : 11, + "name" : "year", + "type" : "INT" + }, { + "id" : 12, + "name" : "month", + "type" : "INT" + } ], + "highestFieldId" : 12, + "partitionKeys" : [ ], + "primaryKeys" : [ ], + "options" : { + "owner" : "jichen" + }, + "timeMillis" : 1758803654196 +} \ No newline at end of file diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_structs_paimon/snapshot/EARLIEST b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_structs_paimon/snapshot/EARLIEST new file mode 100644 index 000000000..56a6051ca --- /dev/null +++ b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_structs_paimon/snapshot/EARLIEST @@ -0,0 +1 @@ +1 \ No newline at end of file diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_structs_paimon/snapshot/LATEST b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_structs_paimon/snapshot/LATEST new file mode 100644 index 000000000..56a6051ca --- /dev/null +++ b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_structs_paimon/snapshot/LATEST @@ -0,0 +1 @@ +1 \ No newline at end of file diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_structs_paimon/snapshot/snapshot-1 b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_structs_paimon/snapshot/snapshot-1 new file mode 100644 index 000000000..2dd5d96f4 --- /dev/null +++ b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_structs_paimon/snapshot/snapshot-1 @@ -0,0 +1,18 @@ +{ + "version" : 3, + "id" : 1, + "schemaId" : 0, + "baseManifestList" : "manifest-list-67f31438-ac2d-4d03-9bd9-4e1d55120c5c-0", + "baseManifestListSize" : 884, + "deltaManifestList" : "manifest-list-67f31438-ac2d-4d03-9bd9-4e1d55120c5c-1", + "deltaManifestListSize" : 985, + "changelogManifestList" : null, + "commitUser" : "790f59d4-de3f-4f08-ba82-37036c5b04d1", + "commitIdentifier" : 9223372036854775807, + "commitKind" : "APPEND", + "timeMillis" : 1758803655199, + "logOffsets" : { }, + "totalRecordCount" : 7300, + "deltaRecordCount" : 7300, + "changelogRecordCount" : 0 +} \ No newline at end of file diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_decimal_tbl/bucket-0/data-58e01102-7ce1-4410-8022-50afc44aa215-0.parquet b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_decimal_tbl/bucket-0/data-58e01102-7ce1-4410-8022-50afc44aa215-0.parquet new file mode 100644 index 000000000..2fa67f519 Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_decimal_tbl/bucket-0/data-58e01102-7ce1-4410-8022-50afc44aa215-0.parquet differ diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_decimal_tbl/bucket-0/data-6d2bcd70-7bfc-4e04-8a37-52bb441bccc1-0.parquet b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_decimal_tbl/bucket-0/data-6d2bcd70-7bfc-4e04-8a37-52bb441bccc1-0.parquet new file mode 100644 index 000000000..d5ffb3d29 Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_decimal_tbl/bucket-0/data-6d2bcd70-7bfc-4e04-8a37-52bb441bccc1-0.parquet differ diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_decimal_tbl/bucket-0/data-7f39c604-2f10-4bea-bd61-32787ba71065-0.parquet b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_decimal_tbl/bucket-0/data-7f39c604-2f10-4bea-bd61-32787ba71065-0.parquet new file mode 100644 index 000000000..16d3665ec Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_decimal_tbl/bucket-0/data-7f39c604-2f10-4bea-bd61-32787ba71065-0.parquet differ diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_decimal_tbl/bucket-0/data-8def1f02-46fd-4114-8c89-cbae117a1b61-0.parquet b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_decimal_tbl/bucket-0/data-8def1f02-46fd-4114-8c89-cbae117a1b61-0.parquet new file mode 100644 index 000000000..d818d98ea Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_decimal_tbl/bucket-0/data-8def1f02-46fd-4114-8c89-cbae117a1b61-0.parquet differ diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_decimal_tbl/bucket-0/data-a100aae7-ed6e-4f04-8648-be059c9e2942-0.parquet b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_decimal_tbl/bucket-0/data-a100aae7-ed6e-4f04-8648-be059c9e2942-0.parquet new file mode 100644 index 000000000..ac3ea6e8d Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_decimal_tbl/bucket-0/data-a100aae7-ed6e-4f04-8648-be059c9e2942-0.parquet differ diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_decimal_tbl/bucket-0/data-bddf22a5-302c-4e14-bf86-dd897d868b42-0.parquet b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_decimal_tbl/bucket-0/data-bddf22a5-302c-4e14-bf86-dd897d868b42-0.parquet new file mode 100644 index 000000000..917bf2f0e Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_decimal_tbl/bucket-0/data-bddf22a5-302c-4e14-bf86-dd897d868b42-0.parquet differ diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_decimal_tbl/bucket-0/data-cf7233f7-0e36-49e0-bd10-7b1f94ca1408-0.parquet b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_decimal_tbl/bucket-0/data-cf7233f7-0e36-49e0-bd10-7b1f94ca1408-0.parquet new file mode 100644 index 000000000..602ba4b08 Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_decimal_tbl/bucket-0/data-cf7233f7-0e36-49e0-bd10-7b1f94ca1408-0.parquet differ diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_decimal_tbl/bucket-0/data-ddc2526d-a35f-4c0f-91e8-def352b6432e-0.parquet b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_decimal_tbl/bucket-0/data-ddc2526d-a35f-4c0f-91e8-def352b6432e-0.parquet new file mode 100644 index 000000000..254d834b2 Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_decimal_tbl/bucket-0/data-ddc2526d-a35f-4c0f-91e8-def352b6432e-0.parquet differ diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_decimal_tbl/manifest/manifest-1e64bd23-89a3-40ab-8f11-b11467e340fe-0 b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_decimal_tbl/manifest/manifest-1e64bd23-89a3-40ab-8f11-b11467e340fe-0 new file mode 100644 index 000000000..7b51f9e9c Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_decimal_tbl/manifest/manifest-1e64bd23-89a3-40ab-8f11-b11467e340fe-0 differ diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_decimal_tbl/manifest/manifest-8c8eef70-e1cc-4019-bd2b-1df0c773321a-0 b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_decimal_tbl/manifest/manifest-8c8eef70-e1cc-4019-bd2b-1df0c773321a-0 new file mode 100644 index 000000000..1f4cbd258 Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_decimal_tbl/manifest/manifest-8c8eef70-e1cc-4019-bd2b-1df0c773321a-0 differ diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_decimal_tbl/manifest/manifest-list-b15e96d1-cea5-44c2-a5b4-dc8c7041463f-0 b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_decimal_tbl/manifest/manifest-list-b15e96d1-cea5-44c2-a5b4-dc8c7041463f-0 new file mode 100644 index 000000000..870f2c22d Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_decimal_tbl/manifest/manifest-list-b15e96d1-cea5-44c2-a5b4-dc8c7041463f-0 differ diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_decimal_tbl/manifest/manifest-list-b15e96d1-cea5-44c2-a5b4-dc8c7041463f-1 b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_decimal_tbl/manifest/manifest-list-b15e96d1-cea5-44c2-a5b4-dc8c7041463f-1 new file mode 100644 index 000000000..8a2ba77bb Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_decimal_tbl/manifest/manifest-list-b15e96d1-cea5-44c2-a5b4-dc8c7041463f-1 differ diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_decimal_tbl/manifest/manifest-list-b9930407-daa0-43cc-b4fa-107aef9a5ffc-0 b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_decimal_tbl/manifest/manifest-list-b9930407-daa0-43cc-b4fa-107aef9a5ffc-0 new file mode 100644 index 000000000..98c37faef Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_decimal_tbl/manifest/manifest-list-b9930407-daa0-43cc-b4fa-107aef9a5ffc-0 differ diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_decimal_tbl/manifest/manifest-list-b9930407-daa0-43cc-b4fa-107aef9a5ffc-1 b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_decimal_tbl/manifest/manifest-list-b9930407-daa0-43cc-b4fa-107aef9a5ffc-1 new file mode 100644 index 000000000..013f03a3c Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_decimal_tbl/manifest/manifest-list-b9930407-daa0-43cc-b4fa-107aef9a5ffc-1 differ diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_decimal_tbl/schema/schema-0 b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_decimal_tbl/schema/schema-0 new file mode 100644 index 000000000..cab2ba164 --- /dev/null +++ b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_decimal_tbl/schema/schema-0 @@ -0,0 +1,34 @@ +{ + "version" : 3, + "id" : 0, + "fields" : [ { + "id" : 0, + "name" : "d1", + "type" : "DECIMAL(9, 0)" + }, { + "id" : 1, + "name" : "d2", + "type" : "DECIMAL(10, 0)" + }, { + "id" : 2, + "name" : "d3", + "type" : "DECIMAL(20, 10)" + }, { + "id" : 3, + "name" : "d4", + "type" : "DECIMAL(38, 38)" + }, { + "id" : 4, + "name" : "d5", + "type" : "DECIMAL(10, 5)" + } ], + "highestFieldId" : 4, + "partitionKeys" : [ ], + "primaryKeys" : [ ], + "options" : { + "owner" : "jichen", + "path" : "hdfs://localhost:20500/test-warehouse/paimon_test/paimon_catalog/warehouse/functional.db/paimon_decimal_tbl", + "external" : "true" + }, + "timeMillis" : 1757003848556 +} \ No newline at end of file diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_decimal_tbl/snapshot/EARLIEST b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_decimal_tbl/snapshot/EARLIEST new file mode 100644 index 000000000..56a6051ca --- /dev/null +++ b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_decimal_tbl/snapshot/EARLIEST @@ -0,0 +1 @@ +1 \ No newline at end of file diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_decimal_tbl/snapshot/LATEST b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_decimal_tbl/snapshot/LATEST new file mode 100644 index 000000000..d8263ee98 --- /dev/null +++ b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_decimal_tbl/snapshot/LATEST @@ -0,0 +1 @@ +2 \ No newline at end of file diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_decimal_tbl/snapshot/snapshot-1 b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_decimal_tbl/snapshot/snapshot-1 new file mode 100644 index 000000000..76180f207 --- /dev/null +++ b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_decimal_tbl/snapshot/snapshot-1 @@ -0,0 +1,18 @@ +{ + "version" : 3, + "id" : 1, + "schemaId" : 0, + "baseManifestList" : "manifest-list-b15e96d1-cea5-44c2-a5b4-dc8c7041463f-0", + "baseManifestListSize" : 884, + "deltaManifestList" : "manifest-list-b15e96d1-cea5-44c2-a5b4-dc8c7041463f-1", + "deltaManifestListSize" : 989, + "changelogManifestList" : null, + "commitUser" : "0144f4f7-db7f-412f-91ad-82ef14642b76", + "commitIdentifier" : 9223372036854775807, + "commitKind" : "APPEND", + "timeMillis" : 1757003851433, + "logOffsets" : { }, + "totalRecordCount" : 3, + "deltaRecordCount" : 3, + "changelogRecordCount" : 0 +} \ No newline at end of file diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_decimal_tbl/snapshot/snapshot-2 b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_decimal_tbl/snapshot/snapshot-2 new file mode 100644 index 000000000..dd3d042cd --- /dev/null +++ b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_decimal_tbl/snapshot/snapshot-2 @@ -0,0 +1,18 @@ +{ + "version" : 3, + "id" : 2, + "schemaId" : 0, + "baseManifestList" : "manifest-list-b9930407-daa0-43cc-b4fa-107aef9a5ffc-0", + "baseManifestListSize" : 989, + "deltaManifestList" : "manifest-list-b9930407-daa0-43cc-b4fa-107aef9a5ffc-1", + "deltaManifestListSize" : 989, + "changelogManifestList" : null, + "commitUser" : "3836dff6-6464-496a-914e-dcdd8d34b26f", + "commitIdentifier" : 9223372036854775807, + "commitKind" : "OVERWRITE", + "timeMillis" : 1757042364907, + "logOffsets" : { }, + "totalRecordCount" : 5, + "deltaRecordCount" : 2, + "changelogRecordCount" : 0 +} \ No newline at end of file diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_primitive_alltypes/bucket-0/data-3728bff4-abe9-4ab9-8837-081b21a80e70-0.parquet b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_primitive_alltypes/bucket-0/data-3728bff4-abe9-4ab9-8837-081b21a80e70-0.parquet new file mode 100644 index 000000000..c51ab3bd0 Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_primitive_alltypes/bucket-0/data-3728bff4-abe9-4ab9-8837-081b21a80e70-0.parquet differ diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_primitive_alltypes/bucket-0/data-493438a3-8481-4c98-8125-856ff5b42a12-0.parquet b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_primitive_alltypes/bucket-0/data-493438a3-8481-4c98-8125-856ff5b42a12-0.parquet new file mode 100644 index 000000000..34befe691 Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_primitive_alltypes/bucket-0/data-493438a3-8481-4c98-8125-856ff5b42a12-0.parquet differ diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_primitive_alltypes/bucket-0/data-59e8fabf-ccba-47d6-a88a-0571d4e0e7a0-0.parquet b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_primitive_alltypes/bucket-0/data-59e8fabf-ccba-47d6-a88a-0571d4e0e7a0-0.parquet new file mode 100644 index 000000000..a20ba6568 Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_primitive_alltypes/bucket-0/data-59e8fabf-ccba-47d6-a88a-0571d4e0e7a0-0.parquet differ diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_primitive_alltypes/bucket-0/data-6db45077-3cc7-46cf-b0c8-f7b5f42c19f7-0.parquet b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_primitive_alltypes/bucket-0/data-6db45077-3cc7-46cf-b0c8-f7b5f42c19f7-0.parquet new file mode 100644 index 000000000..60caeae99 Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_primitive_alltypes/bucket-0/data-6db45077-3cc7-46cf-b0c8-f7b5f42c19f7-0.parquet differ diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_primitive_alltypes/bucket-0/data-a73a0979-5cb2-4876-b7eb-1a8d8dcd90f7-0.parquet b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_primitive_alltypes/bucket-0/data-a73a0979-5cb2-4876-b7eb-1a8d8dcd90f7-0.parquet new file mode 100644 index 000000000..bfc4c1549 Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_primitive_alltypes/bucket-0/data-a73a0979-5cb2-4876-b7eb-1a8d8dcd90f7-0.parquet differ diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_primitive_alltypes/bucket-0/data-af00a36c-e45c-48c3-8004-9055399d7147-0.parquet b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_primitive_alltypes/bucket-0/data-af00a36c-e45c-48c3-8004-9055399d7147-0.parquet new file mode 100644 index 000000000..adc707e63 Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_primitive_alltypes/bucket-0/data-af00a36c-e45c-48c3-8004-9055399d7147-0.parquet differ diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_primitive_alltypes/bucket-0/data-b9ea2c17-a695-4a31-98da-ac1da8ca242f-0.parquet b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_primitive_alltypes/bucket-0/data-b9ea2c17-a695-4a31-98da-ac1da8ca242f-0.parquet new file mode 100644 index 000000000..314f6e0a1 Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_primitive_alltypes/bucket-0/data-b9ea2c17-a695-4a31-98da-ac1da8ca242f-0.parquet differ diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_primitive_alltypes/manifest/manifest-32d94d25-0a6f-499e-9953-e4d2dd8de146-0 b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_primitive_alltypes/manifest/manifest-32d94d25-0a6f-499e-9953-e4d2dd8de146-0 new file mode 100644 index 000000000..2d2002461 Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_primitive_alltypes/manifest/manifest-32d94d25-0a6f-499e-9953-e4d2dd8de146-0 differ diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_primitive_alltypes/manifest/manifest-65f28661-1546-4a1a-930e-f31118e55bd9-0 b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_primitive_alltypes/manifest/manifest-65f28661-1546-4a1a-930e-f31118e55bd9-0 new file mode 100644 index 000000000..b6f7e9404 Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_primitive_alltypes/manifest/manifest-65f28661-1546-4a1a-930e-f31118e55bd9-0 differ diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_primitive_alltypes/manifest/manifest-list-5db2c8c6-e253-4fba-b0c2-71dc19d66678-0 b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_primitive_alltypes/manifest/manifest-list-5db2c8c6-e253-4fba-b0c2-71dc19d66678-0 new file mode 100644 index 000000000..5bc4b347b Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_primitive_alltypes/manifest/manifest-list-5db2c8c6-e253-4fba-b0c2-71dc19d66678-0 differ diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_primitive_alltypes/manifest/manifest-list-5db2c8c6-e253-4fba-b0c2-71dc19d66678-1 b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_primitive_alltypes/manifest/manifest-list-5db2c8c6-e253-4fba-b0c2-71dc19d66678-1 new file mode 100644 index 000000000..1845eecc1 Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_primitive_alltypes/manifest/manifest-list-5db2c8c6-e253-4fba-b0c2-71dc19d66678-1 differ diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_primitive_alltypes/manifest/manifest-list-c6343add-3b4b-4a00-a470-de159bd83b3d-0 b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_primitive_alltypes/manifest/manifest-list-c6343add-3b4b-4a00-a470-de159bd83b3d-0 new file mode 100644 index 000000000..6bb5ca5c7 Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_primitive_alltypes/manifest/manifest-list-c6343add-3b4b-4a00-a470-de159bd83b3d-0 differ diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_primitive_alltypes/manifest/manifest-list-c6343add-3b4b-4a00-a470-de159bd83b3d-1 b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_primitive_alltypes/manifest/manifest-list-c6343add-3b4b-4a00-a470-de159bd83b3d-1 new file mode 100644 index 000000000..e3031178f Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_primitive_alltypes/manifest/manifest-list-c6343add-3b4b-4a00-a470-de159bd83b3d-1 differ diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_primitive_alltypes/schema/schema-0 b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_primitive_alltypes/schema/schema-0 new file mode 100644 index 000000000..024ab5ebb --- /dev/null +++ b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_primitive_alltypes/schema/schema-0 @@ -0,0 +1,66 @@ +{ + "version" : 3, + "id" : 0, + "fields" : [ { + "id" : 0, + "name" : "bool_value", + "type" : "BOOLEAN" + }, { + "id" : 1, + "name" : "tiny_value", + "type" : "TINYINT" + }, { + "id" : 2, + "name" : "small_value", + "type" : "SMALLINT" + }, { + "id" : 3, + "name" : "int_value", + "type" : "INT" + }, { + "id" : 4, + "name" : "big_value", + "type" : "BIGINT" + }, { + "id" : 5, + "name" : "float_value", + "type" : "FLOAT" + }, { + "id" : 6, + "name" : "double_value", + "type" : "DOUBLE" + }, { + "id" : 7, + "name" : "decimal_value", + "type" : "DECIMAL(10, 2)" + }, { + "id" : 8, + "name" : "char_value", + "type" : "CHAR(10)" + }, { + "id" : 9, + "name" : "varchar_value", + "type" : "VARCHAR(100)" + }, { + "id" : 10, + "name" : "binary_value", + "type" : "BINARY(100)" + }, { + "id" : 11, + "name" : "date_value", + "type" : "DATE" + }, { + "id" : 12, + "name" : "ts_ltz_value", + "type" : "TIMESTAMP(6) WITH LOCAL TIME ZONE" + }, { + "id" : 13, + "name" : "ts_value", + "type" : "TIMESTAMP(6)" + } ], + "highestFieldId" : 13, + "partitionKeys" : [ ], + "primaryKeys" : [ ], + "options" : { }, + "timeMillis" : 1754907467147 +} \ No newline at end of file diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_primitive_alltypes/snapshot/EARLIEST b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_primitive_alltypes/snapshot/EARLIEST new file mode 100644 index 000000000..56a6051ca --- /dev/null +++ b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_primitive_alltypes/snapshot/EARLIEST @@ -0,0 +1 @@ +1 \ No newline at end of file diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_primitive_alltypes/snapshot/LATEST b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_primitive_alltypes/snapshot/LATEST new file mode 100644 index 000000000..d8263ee98 --- /dev/null +++ b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_primitive_alltypes/snapshot/LATEST @@ -0,0 +1 @@ +2 \ No newline at end of file diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_primitive_alltypes/snapshot/snapshot-1 b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_primitive_alltypes/snapshot/snapshot-1 new file mode 100644 index 000000000..ca6afedf2 --- /dev/null +++ b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_primitive_alltypes/snapshot/snapshot-1 @@ -0,0 +1,18 @@ +{ + "version" : 3, + "id" : 1, + "schemaId" : 0, + "baseManifestList" : "manifest-list-5db2c8c6-e253-4fba-b0c2-71dc19d66678-0", + "baseManifestListSize" : 884, + "deltaManifestList" : "manifest-list-5db2c8c6-e253-4fba-b0c2-71dc19d66678-1", + "deltaManifestListSize" : 985, + "changelogManifestList" : null, + "commitUser" : "77fe8f4e-7a57-4193-b07c-b0e961e36e14", + "commitIdentifier" : 9223372036854775807, + "commitKind" : "APPEND", + "timeMillis" : 1754907470727, + "logOffsets" : { }, + "totalRecordCount" : 1, + "deltaRecordCount" : 1, + "changelogRecordCount" : 0 +} \ No newline at end of file diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_primitive_alltypes/snapshot/snapshot-2 b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_primitive_alltypes/snapshot/snapshot-2 new file mode 100644 index 000000000..cc3729393 --- /dev/null +++ b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_primitive_alltypes/snapshot/snapshot-2 @@ -0,0 +1,18 @@ +{ + "version" : 3, + "id" : 2, + "schemaId" : 0, + "baseManifestList" : "manifest-list-c6343add-3b4b-4a00-a470-de159bd83b3d-0", + "baseManifestListSize" : 985, + "deltaManifestList" : "manifest-list-c6343add-3b4b-4a00-a470-de159bd83b3d-1", + "deltaManifestListSize" : 989, + "changelogManifestList" : null, + "commitUser" : "97054845-d99a-46b8-8721-9ec01ea673df", + "commitIdentifier" : 9223372036854775807, + "commitKind" : "APPEND", + "timeMillis" : 1757001982034, + "logOffsets" : { }, + "totalRecordCount" : 7, + "deltaRecordCount" : 6, + "changelogRecordCount" : 0 +} \ No newline at end of file diff --git a/testdata/datasets/functional/functional_schema_template.sql b/testdata/datasets/functional/functional_schema_template.sql index d23e03fe1..8c9d0e629 100644 --- a/testdata/datasets/functional/functional_schema_template.sql +++ b/testdata/datasets/functional/functional_schema_template.sql @@ -4837,5 +4837,53 @@ STORED AS PAIMON LOCATION '/test-warehouse/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned'; ---- DEPENDENT_LOAD `hadoop fs -mkdir -p /test-warehouse/paimon_test/paimon_catalog/warehouse/functional.db && \ -hadoop fs -put -f ${IMPALA_HOME}//testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned /test-warehouse/paimon_test/paimon_catalog/warehouse/functional.db -==== \ No newline at end of file +hadoop fs -put -f ${IMPALA_HOME}/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned /test-warehouse/paimon_test/paimon_catalog/warehouse/functional.db +==== +---- DATASET +functional +---- BASE_TABLE_NAME +paimon_primitive_alltypes +---- CREATE +CREATE EXTERNAL TABLE IF NOT EXISTS {db_name}{db_suffix}.{table_name} +STORED AS PAIMON +LOCATION '/test-warehouse/paimon_test/paimon_catalog/warehouse/functional.db/paimon_primitive_alltypes'; +---- DEPENDENT_LOAD +`hadoop fs -mkdir -p /test-warehouse/paimon_test/paimon_catalog/warehouse/functional.db && \ +hadoop fs -put -f ${IMPALA_HOME}/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_primitive_alltypes /test-warehouse/paimon_test/paimon_catalog/warehouse/functional.db +==== +---- DATASET +functional +---- BASE_TABLE_NAME +paimon_decimal_tbl +---- CREATE +CREATE EXTERNAL TABLE IF NOT EXISTS {db_name}{db_suffix}.{table_name} +STORED AS PAIMON +LOCATION '/test-warehouse/paimon_test/paimon_catalog/warehouse/functional.db/paimon_decimal_tbl'; +---- DEPENDENT_LOAD +`hadoop fs -mkdir -p /test-warehouse/paimon_test/paimon_catalog/warehouse/functional.db && \ +hadoop fs -put -f ${IMPALA_HOME}/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_decimal_tbl /test-warehouse/paimon_test/paimon_catalog/warehouse/functional.db +==== +---- DATASET +functional +---- BASE_TABLE_NAME +alltypes_paimon +---- CREATE +CREATE EXTERNAL TABLE IF NOT EXISTS {db_name}{db_suffix}.{table_name} +STORED AS PAIMON +LOCATION '/test-warehouse/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_paimon'; +---- DEPENDENT_LOAD +`hadoop fs -mkdir -p /test-warehouse/paimon_test/paimon_catalog/warehouse/functional.db && \ +hadoop fs -put -f ${IMPALA_HOME}/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_paimon /test-warehouse/paimon_test/paimon_catalog/warehouse/functional.db +==== +---- DATASET +functional +---- BASE_TABLE_NAME +alltypes_structs_paimon +---- CREATE +CREATE EXTERNAL TABLE IF NOT EXISTS {db_name}{db_suffix}.{table_name} +STORED AS PAIMON +LOCATION '/test-warehouse/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_structs_paimon'; +---- DEPENDENT_LOAD +`hadoop fs -mkdir -p /test-warehouse/paimon_test/paimon_catalog/warehouse/functional.db && \ +hadoop fs -put -f ${IMPALA_HOME}/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_structs_paimon /test-warehouse/paimon_test/paimon_catalog/warehouse/functional.db + diff --git a/testdata/datasets/functional/schema_constraints.csv b/testdata/datasets/functional/schema_constraints.csv index ec1d997e0..568e5c04d 100644 --- a/testdata/datasets/functional/schema_constraints.csv +++ b/testdata/datasets/functional/schema_constraints.csv @@ -444,4 +444,8 @@ table_name:timestamp_with_tz_str, constraint:restrict_to, table_format:parquet/n # Paimon tests are executed in the PARQUET file format dimension table_name:paimon_partitioned, constraint:restrict_to, table_format:parquet/none/none -table_name:paimon_non_partitioned, constraint:restrict_to, table_format:parquet/none/none \ No newline at end of file +table_name:paimon_non_partitioned, constraint:restrict_to, table_format:parquet/none/none +table_name:paimon_primitive_alltypes, constraint:restrict_to, table_format:parquet/none/none +table_name:paimon_decimal_tbl, constraint:restrict_to, table_format:parquet/none/none +table_name:alltypes_paimon, constraint:restrict_to, table_format:parquet/none/none +table_name:alltypes_structs_paimon, constraint:restrict_to, table_format:parquet/none/none \ No newline at end of file diff --git a/testdata/workloads/functional-query/queries/QueryTest/paimon-negative.test b/testdata/workloads/functional-query/queries/QueryTest/paimon-negative.test index 73e2fa1c4..9d0a52341 100644 --- a/testdata/workloads/functional-query/queries/QueryTest/paimon-negative.test +++ b/testdata/workloads/functional-query/queries/QueryTest/paimon-negative.test @@ -161,8 +161,4 @@ ON a.userid = source.userid WHEN MATCHED THEN UPDATE SET movieid = source.movieid; ---- CATCH AnalysisException: Target table must be an Iceberg table: functional_parquet.paimon_partitioned -==== ----- QUERY -SELECT * FROM functional_parquet.paimon_partitioned; ----- CATCH -NotImplementedException: Query is not supported for PAIMON table now + diff --git a/testdata/workloads/functional-query/queries/QueryTest/paimon-query.test b/testdata/workloads/functional-query/queries/QueryTest/paimon-query.test new file mode 100644 index 000000000..075cfb8ae --- /dev/null +++ b/testdata/workloads/functional-query/queries/QueryTest/paimon-query.test @@ -0,0 +1,168 @@ +==== +---- QUERY +# Test the Paimon DataSource +# count(*) with a predicate evaluated by Impala +select count(*) from functional_parquet.paimon_primitive_alltypes +where float_value = 0 and varchar_value is not NULL +---- RESULTS +1 +---- TYPES +BIGINT +==== +---- QUERY +# count(*) with no predicates has no materialized slots +select count(*) from functional_parquet.paimon_primitive_alltypes +---- RESULTS +7 +---- TYPES +BIGINT +==== +---- QUERY +# Gets all types including a row with a NULL value. The predicate pushed to +# the DataSource. +select bool_value,tiny_value,small_value,int_value,big_value,float_value,double_value,decimal_value,char_value,varchar_value,binary_value,date_value,ts_value +from functional_parquet.paimon_primitive_alltypes +where int_value > 12 limit 5; +---- RESULTS +true,127,32767,2147483647,9223372036854775807,3.402823466385289e+38,1.797693134862316e+308,1234567.89,'char ','varchar','\x01\x02\x03',2023-03-01,2025-08-11 18:17:27.653000000 +true,3,3,13,30,3.299999952316284,30.3,345.67,'char3 ','varchar3','NULL',2009-01-02,2009-01-02 00:13:00.480000000 +false,4,4,14,40,4.400000095367432,40.4,456.78,'char4 ','varchar4','NULL',2009-01-02,2009-01-02 00:14:00.510000000 +true,0,0,20,0,0.0,0.0,0.00,'char0 ','varchar0','NULL',2009-01-03,2009-01-03 00:20:00.900000000 +---- TYPES +BOOLEAN, TINYINT, SMALLINT, INT, BIGINT, FLOAT, DOUBLE, DECIMAL, CHAR, VARCHAR, BINARY, DATE, TIMESTAMP +==== +---- QUERY +# Gets specified columns. +select int_value, bool_value, small_value, float_value, double_value, date_value +from functional_parquet.paimon_primitive_alltypes +where int_value > 12 limit 5; +---- RESULTS +2147483647,true,32767,3.402823466385289e+38,1.797693134862316e+308,2023-03-01 +13,true,3,3.299999952316284,30.3,2009-01-02 +14,false,4,4.400000095367432,40.4,2009-01-02 +20,true,0,0.0,0.0,2009-01-03 +---- TYPES +INT, BOOLEAN, SMALLINT, FLOAT, DOUBLE, DATE +==== +---- QUERY +# Inner join with a non Paimon table +select a.int_value, b.int_col +from functional_parquet.paimon_primitive_alltypes a inner join functional.alltypes b on (a.int_value = b.id) +where a.int_value = 11 +---- RESULTS +11,1 +---- TYPES +INT, INT +==== +---- QUERY +# Gets specified columns based on date predicate with operator '='. +select int_value, bool_value, small_value, float_value, double_value, date_value +from functional_parquet.paimon_primitive_alltypes +where date_value = DATE '2009-01-02' order by int_value limit 5; +---- RESULTS +11,true,1,1.100000023841858,10.1,2009-01-02 +12,false,2,2.200000047683716,20.2,2009-01-02 +13,true,3,3.299999952316284,30.3,2009-01-02 +14,false,4,4.400000095367432,40.4,2009-01-02 +---- TYPES +INT, BOOLEAN, SMALLINT, FLOAT, DOUBLE, DATE +==== +---- QUERY +select * from functional_parquet.paimon_decimal_tbl; +---- RESULTS +1234,2222,1.2345678900,0.12345678900000000000000000000000000000,12345.78900 +2345,111,12.3456789000,0.12345678900000000000000000000000000000,3.14100 +12345,333,123.4567890000,0.12345678900000000000000000000000000000,11.22000 +12345,333,1234.5678900000,0.12345678900000000000000000000000000000,0.10000 +132842,333,12345.6789000000,0.12345678900000000000000000000000000000,0.77889 +---- TYPES +DECIMAL,DECIMAL,DECIMAL,DECIMAL,DECIMAL +==== +---- QUERY +select * from functional_parquet.paimon_decimal_tbl where d3 < 10.00; +---- RESULTS +1234,2222,1.2345678900,0.12345678900000000000000000000000000000,12345.78900 +---- TYPES +DECIMAL,DECIMAL,DECIMAL,DECIMAL,DECIMAL +==== +---- QUERY +select * from functional_parquet.paimon_decimal_tbl where d3 <= 123.45678900; +---- RESULTS +1234,2222,1.2345678900,0.12345678900000000000000000000000000000,12345.78900 +2345,111,12.3456789000,0.12345678900000000000000000000000000000,3.14100 +12345,333,123.4567890000,0.12345678900000000000000000000000000000,11.22000 +---- TYPES +DECIMAL,DECIMAL,DECIMAL,DECIMAL,DECIMAL +==== +---- QUERY +select * from functional_parquet.paimon_decimal_tbl where d3 = 123.45678900; +---- RESULTS +12345,333,123.4567890000,0.12345678900000000000000000000000000000,11.22000 +---- TYPES +DECIMAL,DECIMAL,DECIMAL,DECIMAL,DECIMAL +==== +---- QUERY +select * from functional_parquet.paimon_decimal_tbl where d3 != 123.45678900; +---- RESULTS +1234,2222,1.2345678900,0.12345678900000000000000000000000000000,12345.78900 +2345,111,12.3456789000,0.12345678900000000000000000000000000000,3.14100 +12345,333,1234.5678900000,0.12345678900000000000000000000000000000,0.10000 +132842,333,12345.6789000000,0.12345678900000000000000000000000000000,0.77889 +---- TYPES +DECIMAL,DECIMAL,DECIMAL,DECIMAL,DECIMAL +==== +---- QUERY +select * from functional_parquet.paimon_decimal_tbl where d3 >= 123.45678900; +---- RESULTS +12345,333,123.4567890000,0.12345678900000000000000000000000000000,11.22000 +12345,333,1234.5678900000,0.12345678900000000000000000000000000000,0.10000 +132842,333,12345.6789000000,0.12345678900000000000000000000000000000,0.77889 +---- TYPES +DECIMAL,DECIMAL,DECIMAL,DECIMAL,DECIMAL +==== +---- QUERY +select * from functional_parquet.paimon_decimal_tbl where d3 > 123.45678900; +---- RESULTS +12345,333,1234.5678900000,0.12345678900000000000000000000000000000,0.10000 +132842,333,12345.6789000000,0.12345678900000000000000000000000000000,0.77889 +---- TYPES +DECIMAL,DECIMAL,DECIMAL,DECIMAL,DECIMAL +==== +---- QUERY +select * from functional_parquet.paimon_decimal_tbl where d3 >= 123.45678900 and d5 < 10.0; +---- RESULTS +12345,333,1234.5678900000,0.12345678900000000000000000000000000000,0.10000 +132842,333,12345.6789000000,0.12345678900000000000000000000000000000,0.77889 +---- TYPES +DECIMAL,DECIMAL,DECIMAL,DECIMAL,DECIMAL +==== +---- QUERY +SELECT * FROM functional_parquet.alltypes_structs_paimon WHERE year=2009 AND month=1 AND id > 300 +---- RESULTS +301,2009,1 +302,2009,1 +303,2009,1 +304,2009,1 +305,2009,1 +306,2009,1 +307,2009,1 +308,2009,1 +309,2009,1 +---- TYPES +INT,INT,INT +==== +---- QUERY +select struct_val from functional_parquet.alltypes_structs_paimon where year=2009 and month=1 and id > 300 +---- CATCH +AnalysisException: Local Paimon Table: functional_parquet.alltypes_structs_paimon is not supported when querying STRUCT type STRUCT +==== +---- QUERY +select struct_val.bool_col from functional_parquet.alltypes_structs_paimon where year=2009 and month=1 and id > 300 +---- CATCH +AnalysisException: Paimon Scanner doesn't support nested columns. +==== +---- QUERY +select INPUT__FILE__NAME from functional_parquet.alltypes_structs_paimon where year=2009 and month=1 and id > 300 +---- CATCH +AnalysisException: Could not resolve column/field reference: 'input__file__name' + diff --git a/tests/query_test/test_paimon.py b/tests/query_test/test_paimon.py index a0297950b..0a92d096b 100644 --- a/tests/query_test/test_paimon.py +++ b/tests/query_test/test_paimon.py @@ -52,3 +52,7 @@ class TestCreatingPaimonTable(ImpalaTestSuite): def test_paimon_negative(self, vector, unique_database): self.run_test_case('QueryTest/paimon-negative', vector, unique_database) + + def test_paimon_query(self, vector, unique_database): + self.run_test_case('QueryTest/paimon-query', + vector, unique_database) diff --git a/tests/query_test/test_scanners.py b/tests/query_test/test_scanners.py index c044cf0a5..04d3bb4b9 100644 --- a/tests/query_test/test_scanners.py +++ b/tests/query_test/test_scanners.py @@ -2014,3 +2014,38 @@ class TestSingleFileTable(ImpalaTestSuite): select_stmt = "select count(*) from {db}.{tbl}".format(**params) res = self.execute_query_expect_success(self.client, select_stmt, options) assert res.data[0].split("\t")[0] == '1' + + +class TestPaimonScannerWithLimit(ImpalaTestSuite): + """Test paimon scanners with a simple limit clause. The limit clause triggers + cancellation in the scanner code paths.""" + + @classmethod + def add_test_dimensions(cls): + super(TestPaimonScannerWithLimit, cls).add_test_dimensions() + # Use a small batch size so changing the limit affects the timing of cancellation + cls.ImpalaTestMatrix.add_dimension( + create_exec_option_dimension(batch_sizes=[100])) + cls.ImpalaTestMatrix.add_constraint( + lambda v: v.get_value('table_format').file_format == 'parquet') + + def test_limit(self, vector): + vector.get_value('exec_option')['abort_on_error'] = 1 + self._test_limit(vector) + # IMPALA-3337: when continuing on error, the error log should not show errors + # (e.g. "Cancelled"). + vector.get_value('exec_option')['abort_on_error'] = 0 + self._test_limit(vector) + + def _test_limit(self, vector): + iterations = 50 + query_template = "select * from functional_parquet.alltypes_paimon limit %s" + for i in range(1, iterations): + # Vary the limit to vary the timing of cancellation + limit = (i * 100) % 1001 + 1 + query = query_template % limit + result = self.execute_query(query, vector.get_value('exec_option'), + table_format=vector.get_value('table_format')) + assert len(result.data) == limit + # IMPALA-3337: The error log should be empty. + assert not result.log