mirror of
https://github.com/apache/impala.git
synced 2025-12-19 09:58:28 -05:00
IMPALA-14092 Part2: Support querying of paimon data table via JNI
This patch mainly implement the querying of paimon data table
through JNI based scanner.
Features implemented:
- support column pruning.
The partition pruning and predicate push down will be submitted
as the third part of the patch.
We implemented this by treating the paimon table as normal
unpartitioned table. When querying paimon table:
- PaimonScanNode will decide paimon splits need to be scanned,
and then transfer splits to BE do the jni-based scan operation.
- We also collect the required columns that need to be scanned,
and pass the columns to Scanner for column pruning. This is
implemented by passing the field ids of the columns to BE,
instead of column position to support schema evolution.
- In the original implementation, PaimonJniScanner will directly
pass paimon row object to BE, and call corresponding paimon row
field accessor, which is a java method to convert row fields to
impala row batch tuples. We find it is slow due to overhead of
JVM method calling.
To minimize the overhead, we refashioned the implementation,
the PaimonJniScanner will convert the paimon row batches to
arrow recordbatch, which stores data in offheap region of
impala JVM. And PaimonJniScanner will pass the arrow offheap
record batch memory pointer to the BE backend.
BE PaimonJniScanNode will directly read data from JVM offheap
region, and convert the arrow record batch to impala row batch.
The benchmark shows the later implementation is 2.x better
than the original implementation.
The lifecycle of arrow row batch is mainly like this:
the arrow row batch is generated in FE,and passed to BE.
After the record batch is imported to BE successfully,
BE will be in charge of freeing the row batch.
There are two free paths: the normal path, and the
exception path. For the normal path, when the arrow batch
is totally consumed by BE, BE will call jni to fetch the next arrow
batch. For this case, the arrow batch is freed automatically.
For the exceptional path, it happends when query is cancelled, or memory
failed to allocate. For these corner cases, arrow batch is freed in the
method close if it is not totally consumed by BE.
Current supported impala data types for query includes:
- BOOLEAN
- TINYINT
- SMALLINT
- INTEGER
- BIGINT
- FLOAT
- DOUBLE
- STRING
- DECIMAL(P,S)
- TIMESTAMP
- CHAR(N)
- VARCHAR(N)
- BINARY
- DATE
TODO:
- Patches pending submission:
- Support tpcds/tpch data-loading
for paimon data table.
- Virtual Column query support for querying
paimon data table.
- Query support with time travel.
- Query support for paimon meta tables.
- WIP:
- Snapshot incremental read.
- Complex type query support.
- Native paimon table scanner, instead of
jni based.
Testing:
- Create tests table in functional_schema_template.sql
- Add TestPaimonScannerWithLimit in test_scanners.py
- Add test_paimon_query in test_paimon.py.
- Already passed the tpcds/tpch test for paimon table, due to the
testing table data is currently generated by spark, and it is
not supported by impala now, we have to do this since hive
doesn't support generating paimon table for dynamic-partitioned
tables. we plan to submit a separate patch for tpcds/tpch data
loading and associated tpcds/tpch query tests.
- JVM Offheap memory leak tests, have run looped tpch tests for
1 day, no obvious offheap memory increase is observed,
offheap memory usage is within 10M.
Change-Id: Ie679a89a8cc21d52b583422336b9f747bdf37384
Reviewed-on: http://gerrit.cloudera.org:8080/23613
Tested-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
Reviewed-by: Zoltan Borok-Nagy <boroknagyz@cloudera.com>
Reviewed-by: Riza Suminto <riza.suminto@cloudera.com>
This commit is contained in:
@@ -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
|
||||
|
||||
@@ -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
|
||||
|
||||
@@ -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")
|
||||
|
||||
@@ -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<int, const char*>::const_iterator i =
|
||||
_TPlanNodeType_VALUES_TO_NAMES.find(tnode.node_type);
|
||||
@@ -432,6 +436,7 @@ void ExecNode::CollectScanNodes(vector<ExecNode*>* 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) {
|
||||
|
||||
108
be/src/exec/paimon/CMakeLists.txt
Normal file
108
be/src/exec/paimon/CMakeLists.txt
Normal file
@@ -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
|
||||
# <false, arrow::FieldPath, std::__cxx11::basic_string<char>,
|
||||
# std::vector<arrow::FieldRef, std::allocator<arrow::FieldRef> > >' to
|
||||
# 'std::variant<arrow::FieldPath, std::__cxx11::basic_string<char>,
|
||||
# std::vector<arrow::FieldRef, std::allocator<arrow::FieldRef> > >'
|
||||
# return static_cast<variant<_Types...>&>(__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()
|
||||
315
be/src/exec/paimon/paimon-jni-row-reader.cc
Normal file
315
be/src/exec/paimon/paimon-jni-row-reader.cc
Normal file
@@ -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 <jni.h>
|
||||
#include <cstdint>
|
||||
#include <memory>
|
||||
#include <string_view>
|
||||
#include <type_traits>
|
||||
#include <arrow/array/array_base.h>
|
||||
#include <arrow/array/array_binary.h>
|
||||
#include <arrow/type_fwd.h>
|
||||
#include <glog/logging.h>
|
||||
|
||||
#include <arrow/api.h>
|
||||
#include <arrow/c/bridge.h>
|
||||
#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<arrow::Array> 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 <typename T, typename AT>
|
||||
Status PaimonJniRowReader::WriteSlot(const AT* arrow_array, int row_idx, void* slot) {
|
||||
T value = arrow_array->Value(row_idx);
|
||||
*reinterpret_cast<T*>(slot) = value;
|
||||
return Status::OK();
|
||||
}
|
||||
|
||||
template <typename T, typename AT>
|
||||
Status PaimonJniRowReader::CastAndWriteSlot(
|
||||
const arrow::Array* arrow_array, const int row_idx, void* slot) {
|
||||
auto derived_array = static_cast<const AT*>(arrow_array);
|
||||
return WriteSlot<T, AT>(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</* IS_CHAR */ true>(
|
||||
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</* IS_BINARY */ true>(
|
||||
array, row_index, slot, tuple_data_pool));
|
||||
} else {
|
||||
RETURN_IF_ERROR(WriteStringOrBinarySlot</* IS_BINARY */ false>(
|
||||
array, row_index, slot, tuple_data_pool));
|
||||
}
|
||||
break;
|
||||
}
|
||||
case TYPE_BOOLEAN: {
|
||||
RETURN_IF_ERROR(
|
||||
(CastAndWriteSlot<bool, arrow::BooleanArray>(array, row_index, slot)));
|
||||
break;
|
||||
}
|
||||
case TYPE_DATE: {
|
||||
RETURN_IF_ERROR(WriteDateSlot(array, row_index, slot));
|
||||
break;
|
||||
}
|
||||
case TYPE_TINYINT: {
|
||||
RETURN_IF_ERROR(
|
||||
(CastAndWriteSlot<int8_t, arrow::Int8Array>(array, row_index, slot)));
|
||||
break;
|
||||
}
|
||||
case TYPE_SMALLINT: {
|
||||
RETURN_IF_ERROR(
|
||||
(CastAndWriteSlot<int16_t, arrow::Int16Array>(array, row_index, slot)));
|
||||
break;
|
||||
}
|
||||
case TYPE_INT: {
|
||||
RETURN_IF_ERROR(
|
||||
(CastAndWriteSlot<int32_t, arrow::Int32Array>(array, row_index, slot)));
|
||||
break;
|
||||
}
|
||||
case TYPE_BIGINT: {
|
||||
RETURN_IF_ERROR(
|
||||
(CastAndWriteSlot<int64_t, arrow::Int64Array>(array, row_index, slot)));
|
||||
break;
|
||||
}
|
||||
case TYPE_FLOAT: {
|
||||
RETURN_IF_ERROR(
|
||||
(CastAndWriteSlot<float, arrow::FloatArray>(array, row_index, slot)));
|
||||
break;
|
||||
}
|
||||
case TYPE_DOUBLE: {
|
||||
RETURN_IF_ERROR(
|
||||
(CastAndWriteSlot<double, arrow::DoubleArray>(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<const arrow::Date32Array*>(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<int32_t*>(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<Decimal4Value*>(slot);
|
||||
if (UNLIKELY(
|
||||
(ParquetPlainEncoder::Decode<Decimal4Value,
|
||||
parquet::Type::FIXED_LEN_BYTE_ARRAY>(buffer, buffer + len, len, val))
|
||||
< 0)) {
|
||||
return Status(ERROR_INVALID_DECIMAL);
|
||||
}
|
||||
break;
|
||||
}
|
||||
case 8: {
|
||||
Decimal8Value* val = reinterpret_cast<Decimal8Value*>(slot);
|
||||
if (UNLIKELY(
|
||||
(ParquetPlainEncoder::Decode<Decimal8Value,
|
||||
parquet::Type::FIXED_LEN_BYTE_ARRAY>(buffer, buffer + len, len, val))
|
||||
< 0)) {
|
||||
return Status(ERROR_INVALID_DECIMAL);
|
||||
}
|
||||
break;
|
||||
}
|
||||
case 16: {
|
||||
Decimal16Value* val = reinterpret_cast<Decimal16Value*>(slot);
|
||||
if (UNLIKELY(
|
||||
(ParquetPlainEncoder::Decode<Decimal16Value,
|
||||
parquet::Type::FIXED_LEN_BYTE_ARRAY>(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<const arrow::BinaryArray*>(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<const arrow::TimestampType&>(*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<TimestampValue*>(slot) = TimestampValue::FromUnixTimeNanos(
|
||||
value / NANOS_PER_SEC, value % NANOS_PER_SEC, tz);
|
||||
break;
|
||||
case ::arrow::TimeUnit::MICRO:
|
||||
*reinterpret_cast<TimestampValue*>(slot) =
|
||||
TimestampValue::FromUnixTimeMicros(value, tz);
|
||||
break;
|
||||
case ::arrow::TimeUnit::MILLI:
|
||||
*reinterpret_cast<TimestampValue*>(slot) =
|
||||
TimestampValue::FromUnixTimeMicros(value * 1000L, tz);
|
||||
break;
|
||||
case ::arrow::TimeUnit::SECOND:
|
||||
*reinterpret_cast<TimestampValue*>(slot) = TimestampValue::FromUnixTime(value, tz);
|
||||
break;
|
||||
}
|
||||
|
||||
return Status::OK();
|
||||
}
|
||||
|
||||
template <bool IS_CHAR>
|
||||
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<const arrow::StringArray*>(array);
|
||||
std::string_view v = nchar_array->Value(row_idx);
|
||||
|
||||
int src_len = v.size();
|
||||
int unpadded_len = std::min<int>(dst_len, src_len);
|
||||
// Allocate memory and copy the bytes from the JVM to the RowBatch.
|
||||
char* dst_char = reinterpret_cast<char*>(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 <bool IS_BINARY>
|
||||
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<const arrow::BinaryArray*>(array);
|
||||
v = binary_array->Value(row_idx);
|
||||
} else {
|
||||
const arrow::StringArray* string_array =
|
||||
static_cast<const arrow::StringArray*>(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<char*>(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<StringValue*>(slot)->Assign(buffer, jbuffer_size);
|
||||
return Status::OK();
|
||||
}
|
||||
} // namespace impala
|
||||
82
be/src/exec/paimon/paimon-jni-row-reader.h
Normal file
82
be/src/exec/paimon/paimon-jni-row-reader.h
Normal file
@@ -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 <arrow/record_batch.h>
|
||||
|
||||
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 <typename T, typename AT>
|
||||
Status WriteSlot(const AT* arrow_array, int row_idx, void* slot) WARN_UNUSED_RESULT;
|
||||
|
||||
template <typename T, typename AT>
|
||||
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 <bool IS_BINARY>
|
||||
Status WriteStringOrBinarySlot(const arrow::Array* array, const int row_idx, void* slot,
|
||||
MemPool* tuple_data_pool) WARN_UNUSED_RESULT;
|
||||
|
||||
template <bool IS_CHAR>
|
||||
Status WriteVarCharOrCharSlot(const arrow::Array* array, const int row_idx, int max_len,
|
||||
void* slot, MemPool* tuple_data_pool) WARN_UNUSED_RESULT;
|
||||
};
|
||||
|
||||
} // namespace impala
|
||||
267
be/src/exec/paimon/paimon-jni-scan-node.cc
Normal file
267
be/src/exec/paimon/paimon-jni-scan-node.cc
Normal file
@@ -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 <jni.h>
|
||||
#include <vector>
|
||||
#include <arrow/c/bridge.h>
|
||||
#include <glog/logging.h>
|
||||
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<int32_t> 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<std::string> 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<TPaimonJniScanParam>(
|
||||
&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<int32_t>& 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*>(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<Tuple*>(
|
||||
reinterpret_cast<uint8_t*>(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);
|
||||
}
|
||||
154
be/src/exec/paimon/paimon-jni-scan-node.h
Normal file
154
be/src/exec/paimon/paimon-jni-scan-node.h
Normal file
@@ -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 <jni.h>
|
||||
#include <arrow/record_batch.h>
|
||||
|
||||
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<int32_t>& 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<PaimonJniScanner> jni_scanner_;
|
||||
|
||||
/// Helper class to transform Paimon rows to Impala tuples.
|
||||
std::unique_ptr<PaimonJniRowReader> 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<MemTracker> 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<arrow::RecordBatch> 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
|
||||
108
be/src/exec/paimon/paimon-jni-scanner.cc
Normal file
108
be/src/exec/paimon/paimon-jni-scanner.cc
Normal file
@@ -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 <jni.h>
|
||||
#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_, "<init>", "([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<const jbyte*>(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
|
||||
77
be/src/exec/paimon/paimon-jni-scanner.h
Normal file
77
be/src/exec/paimon/paimon-jni-scanner.h
Normal file
@@ -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 <jni.h>
|
||||
#include <string>
|
||||
#include <arrow/c/bridge.h>
|
||||
|
||||
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
|
||||
31
be/src/exec/paimon/paimon-scan-plan-node.cc
Normal file
31
be/src/exec/paimon/paimon-scan-plan-node.cc
Normal file
@@ -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();
|
||||
}
|
||||
36
be/src/exec/paimon/paimon-scan-plan-node.h
Normal file
36
be/src/exec/paimon/paimon-scan-plan-node.h
Normal file
@@ -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 <jni.h>
|
||||
#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
|
||||
@@ -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;
|
||||
}
|
||||
|
||||
@@ -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_; }
|
||||
|
||||
@@ -70,7 +70,7 @@ static const string SCHEDULER_WARNING_KEY("Scheduler Warning");
|
||||
static const vector<TPlanNodeType::type> 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
|
||||
|
||||
@@ -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();
|
||||
|
||||
@@ -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",
|
||||
|
||||
@@ -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
|
||||
|
||||
54
cmake_modules/FindArrow.cmake
Normal file
54
cmake_modules/FindArrow.cmake
Normal file
@@ -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
|
||||
)
|
||||
@@ -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.
|
||||
|
||||
@@ -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<binary> splits
|
||||
// Field id list for projection.
|
||||
3: required list<i32> 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 "<a> = <b>"
|
||||
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
|
||||
|
||||
@@ -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 {
|
||||
|
||||
119
fe/pom.xml
119
fe/pom.xml
@@ -558,7 +558,126 @@ under the License.
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.paimon</groupId>
|
||||
<artifactId>paimon-arrow</artifactId>
|
||||
<version>${paimon.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.arrow</groupId>
|
||||
<artifactId>arrow-vector</artifactId>
|
||||
<version>${arrow.version}</version>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>log4j</groupId>
|
||||
<artifactId>log4j</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>org.apache.logging.log4j</groupId>
|
||||
<artifactId>log4j-1.2-api</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>org.apache.logging.log4j</groupId>
|
||||
<artifactId>log4j-slf4j-impl</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>org.apache.ant</groupId>
|
||||
<artifactId>ant</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>org.apache.ant</groupId>
|
||||
<artifactId>ant-launcher</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<artifactId>flatbuffers-java</artifactId>
|
||||
<groupId>com.google.flatbuffers</groupId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.apache.arrow</groupId>
|
||||
<artifactId>arrow-c-data</artifactId>
|
||||
<version>${arrow.version}</version>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>log4j</groupId>
|
||||
<artifactId>log4j</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>org.apache.logging.log4j</groupId>
|
||||
<artifactId>log4j-1.2-api</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>org.apache.logging.log4j</groupId>
|
||||
<artifactId>log4j-slf4j-impl</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>org.apache.ant</groupId>
|
||||
<artifactId>ant</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>org.apache.ant</groupId>
|
||||
<artifactId>ant-launcher</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.apache.arrow</groupId>
|
||||
<artifactId>arrow-memory-core</artifactId>
|
||||
<version>${arrow.version}</version>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>log4j</groupId>
|
||||
<artifactId>log4j</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>org.apache.logging.log4j</groupId>
|
||||
<artifactId>log4j-1.2-api</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>org.apache.logging.log4j</groupId>
|
||||
<artifactId>log4j-slf4j-impl</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>org.apache.ant</groupId>
|
||||
<artifactId>ant</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>org.apache.ant</groupId>
|
||||
<artifactId>ant-launcher</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.apache.arrow</groupId>
|
||||
<artifactId>arrow-memory-unsafe</artifactId>
|
||||
<version>${arrow.version}</version>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>log4j</groupId>
|
||||
<artifactId>log4j</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>org.apache.logging.log4j</groupId>
|
||||
<artifactId>log4j-1.2-api</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>org.apache.logging.log4j</groupId>
|
||||
<artifactId>log4j-slf4j-impl</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>org.apache.ant</groupId>
|
||||
<artifactId>ant</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>org.apache.ant</groupId>
|
||||
<artifactId>ant-launcher</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
|
||||
<reporting>
|
||||
|
||||
@@ -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()));
|
||||
}
|
||||
|
||||
@@ -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());
|
||||
}
|
||||
|
||||
@@ -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<Type, Integer> 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));
|
||||
}
|
||||
|
||||
@@ -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<Column> 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
|
||||
|
||||
@@ -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()));
|
||||
}
|
||||
|
||||
@@ -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;
|
||||
}
|
||||
}
|
||||
@@ -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<String> fieldNames = structTypeInfo.getAllStructFieldNames();
|
||||
ArrayList<TypeInfo> 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);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -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;
|
||||
}
|
||||
|
||||
@@ -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_);
|
||||
}
|
||||
}
|
||||
@@ -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.
|
||||
* <p>
|
||||
|
||||
@@ -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<Column> toImpalaColumn(Table table) throws ImpalaRuntimeException {
|
||||
RowType rowType = table.rowType();
|
||||
List<DataField> dataFields = rowType.getFields();
|
||||
List<String> partitionKeys = table.partitionKeys()
|
||||
.stream()
|
||||
.map(String::toLowerCase)
|
||||
.collect(Collectors.toList());
|
||||
List<Column> impalaFields = convertToImpalaSchema(rowType);
|
||||
List<Column> impalaNonPartitionedFields = Lists.newArrayList();
|
||||
List<Column> impalaPartitionedFields = Lists.newArrayList();
|
||||
List<Column> 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;
|
||||
}
|
||||
}
|
||||
|
||||
303
fe/src/main/java/org/apache/impala/planner/PaimonScanNode.java
Normal file
303
fe/src/main/java/org/apache/impala/planner/PaimonScanNode.java
Normal file
@@ -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<Integer> 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<Integer> fieldIdMap_ = Sets.newHashSet();
|
||||
// Splits generated for paimon scanning during planning stage.
|
||||
private List<Split> splits_;
|
||||
|
||||
public PaimonScanNode(PlanNodeId id, TupleDescriptor desc, List<Expr> 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<DataField> 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<TScanRangeLocation> 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<Expr> 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();
|
||||
}
|
||||
}
|
||||
@@ -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<Expr> conjuncts_;
|
||||
private MultiAggregateInfo aggInfo_;
|
||||
|
||||
private FePaimonTable table_;
|
||||
|
||||
public PaimonScanPlanner(Analyzer analyzer, PlannerContext ctx, TableRef paimonTblRef,
|
||||
List<Expr> 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;
|
||||
}
|
||||
}
|
||||
@@ -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());
|
||||
|
||||
@@ -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<Predicate> predicates_;
|
||||
|
||||
public PaimonSplit(Split split, ArrayList<Predicate> predicates) {
|
||||
split_ = split;
|
||||
predicates_ = predicates;
|
||||
}
|
||||
|
||||
public Split getSplit() { return split_; }
|
||||
|
||||
public List<Predicate> getPredicates() { return predicates_; }
|
||||
}
|
||||
@@ -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;
|
||||
}
|
||||
}
|
||||
@@ -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);
|
||||
}
|
||||
}
|
||||
@@ -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());
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -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 <a href="https://github.com/apache/paimon/pull/6695">...</a>}
|
||||
* 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(); }
|
||||
}
|
||||
@@ -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 <a href="https://github.com/apache/paimon/pull/6695">...</a>}
|
||||
* 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_; }
|
||||
}
|
||||
@@ -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 <a href="https://github.com/apache/paimon/pull/6695">...</a>}
|
||||
* 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<Field> 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<Field> 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);
|
||||
}
|
||||
}
|
||||
@@ -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<PaimonSplit> splits_ = null;
|
||||
// Paimon schema after projection.
|
||||
private RowType projectedSchema_;
|
||||
// Paimon data record iterator.
|
||||
private RecordReaderIterator<InternalRow> 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<Predicate> predicates = splits_.get(0).getPredicates();
|
||||
if (predicates != null && !predicates.isEmpty()) {
|
||||
readBuilder.withFilter(predicates);
|
||||
}
|
||||
// Create Iterator for given splits.
|
||||
List<Split> splits =
|
||||
splits_.stream().map(PaimonSplit::getSplit).collect(Collectors.toList());
|
||||
RecordReader<InternalRow> 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_);
|
||||
}
|
||||
}
|
||||
@@ -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
|
||||
|
||||
@@ -42,6 +42,7 @@ under the License.
|
||||
<hive.dist.type>${env.IMPALA_HIVE_DIST_TYPE}</hive.dist.type>
|
||||
<hudi.version>${env.IMPALA_HUDI_VERSION}</hudi.version>
|
||||
<paimon.version>${env.IMPALA_PAIMON_VERSION}</paimon.version>
|
||||
<arrow.version>${env.IMPALA_ARROW_VERSION}</arrow.version>
|
||||
<ranger.version>${env.IMPALA_RANGER_VERSION}</ranger.version>
|
||||
<postgres.jdbc.version>${env.IMPALA_POSTGRES_JDBC_DRIVER_VERSION}</postgres.jdbc.version>
|
||||
<hbase.version>${env.IMPALA_HBASE_VERSION}</hbase.version>
|
||||
|
||||
115
testdata/data/paimon_test/README.md
vendored
Normal file
115
testdata/data/paimon_test/README.md
vendored
Normal file
@@ -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.
|
||||
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
65
testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_paimon/schema/schema-0
vendored
Normal file
65
testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/alltypes_paimon/schema/schema-0
vendored
Normal file
@@ -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
|
||||
}
|
||||
@@ -0,0 +1 @@
|
||||
1
|
||||
@@ -0,0 +1 @@
|
||||
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
|
||||
}
|
||||
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
@@ -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
|
||||
}
|
||||
@@ -0,0 +1 @@
|
||||
1
|
||||
@@ -0,0 +1 @@
|
||||
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
|
||||
}
|
||||
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Some files were not shown because too many files have changed in this diff Show More
Reference in New Issue
Block a user