From 3c29d14ba3e9d73ebade7f6e760f27e83608c9ab Mon Sep 17 00:00:00 2001 From: morningman Date: Thu, 12 Feb 2026 17:26:30 +0800 Subject: [PATCH 01/13] init --- .../pipeline/exec/tvf_table_sink_operator.cpp | 32 +++ .../pipeline/exec/tvf_table_sink_operator.h | 87 +++++++ be/src/pipeline/pipeline_fragment_context.cpp | 9 + be/src/vec/sink/writer/vtvf_table_writer.cpp | 245 ++++++++++++++++++ be/src/vec/sink/writer/vtvf_table_writer.h | 89 +++++++ .../org/apache/doris/nereids/DorisParser.g4 | 6 +- .../nereids/analyzer/UnboundTVFTableSink.java | 130 ++++++++++ .../translator/PhysicalPlanTranslator.java | 18 ++ .../nereids/parser/LogicalPlanBuilder.java | 23 ++ .../apache/doris/nereids/rules/RuleSet.java | 2 + .../apache/doris/nereids/rules/RuleType.java | 2 + .../nereids/rules/analysis/BindSink.java | 42 ++- ...calTVFTableSinkToPhysicalTVFTableSink.java | 46 ++++ .../doris/nereids/trees/plans/PlanType.java | 4 + .../commands/insert/InsertIntoTVFCommand.java | 158 +++++++++++ .../plans/logical/LogicalTVFTableSink.java | 135 ++++++++++ .../plans/physical/PhysicalTVFTableSink.java | 157 +++++++++++ .../trees/plans/visitor/SinkVisitor.java | 17 ++ .../apache/doris/planner/TVFTableSink.java | 178 +++++++++++++ gensrc/thrift/DataSinks.thrift | 18 ++ 20 files changed, 1396 insertions(+), 2 deletions(-) create mode 100644 be/src/pipeline/exec/tvf_table_sink_operator.cpp create mode 100644 be/src/pipeline/exec/tvf_table_sink_operator.h create mode 100644 be/src/vec/sink/writer/vtvf_table_writer.cpp create mode 100644 be/src/vec/sink/writer/vtvf_table_writer.h create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundTVFTableSink.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalTVFTableSinkToPhysicalTVFTableSink.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertIntoTVFCommand.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalTVFTableSink.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalTVFTableSink.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/planner/TVFTableSink.java diff --git a/be/src/pipeline/exec/tvf_table_sink_operator.cpp b/be/src/pipeline/exec/tvf_table_sink_operator.cpp new file mode 100644 index 00000000000000..c5e82ea2762638 --- /dev/null +++ b/be/src/pipeline/exec/tvf_table_sink_operator.cpp @@ -0,0 +1,32 @@ +// 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 "tvf_table_sink_operator.h" + +#include "common/status.h" + +namespace doris::pipeline { +#include "common/compile_check_begin.h" + +Status TVFTableSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& info) { + RETURN_IF_ERROR(Base::init(state, info)); + SCOPED_TIMER(exec_time_counter()); + SCOPED_TIMER(_init_timer); + return Status::OK(); +} + +} // namespace doris::pipeline diff --git a/be/src/pipeline/exec/tvf_table_sink_operator.h b/be/src/pipeline/exec/tvf_table_sink_operator.h new file mode 100644 index 00000000000000..922a4b7d96d634 --- /dev/null +++ b/be/src/pipeline/exec/tvf_table_sink_operator.h @@ -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. + +#pragma once + +#include "operator.h" +#include "vec/sink/writer/vtvf_table_writer.h" + +namespace doris::pipeline { +#include "common/compile_check_begin.h" + +class TVFTableSinkOperatorX; + +class TVFTableSinkLocalState final + : public AsyncWriterSink { +public: + using Base = AsyncWriterSink; + using Parent = TVFTableSinkOperatorX; + ENABLE_FACTORY_CREATOR(TVFTableSinkLocalState); + TVFTableSinkLocalState(DataSinkOperatorXBase* parent, RuntimeState* state) + : Base(parent, state) {}; + Status init(RuntimeState* state, LocalSinkStateInfo& info) override; + Status open(RuntimeState* state) override { + SCOPED_TIMER(exec_time_counter()); + SCOPED_TIMER(_open_timer); + return Base::open(state); + } + + friend class TVFTableSinkOperatorX; +}; + +class TVFTableSinkOperatorX final : public DataSinkOperatorX { +public: + using Base = DataSinkOperatorX; + TVFTableSinkOperatorX(ObjectPool* pool, int operator_id, const RowDescriptor& row_desc, + const std::vector& t_output_expr) + : Base(operator_id, 0, 0), + _row_desc(row_desc), + _t_output_expr(t_output_expr), + _pool(pool) {}; + + Status init(const TDataSink& thrift_sink) override { + RETURN_IF_ERROR(Base::init(thrift_sink)); + RETURN_IF_ERROR(vectorized::VExpr::create_expr_trees(_t_output_expr, _output_vexpr_ctxs)); + return Status::OK(); + } + + Status prepare(RuntimeState* state) override { + RETURN_IF_ERROR(Base::prepare(state)); + RETURN_IF_ERROR(vectorized::VExpr::prepare(_output_vexpr_ctxs, state, _row_desc)); + return vectorized::VExpr::open(_output_vexpr_ctxs, state); + } + + Status sink(RuntimeState* state, vectorized::Block* in_block, bool eos) override { + auto& local_state = get_local_state(state); + SCOPED_TIMER(local_state.exec_time_counter()); + COUNTER_UPDATE(local_state.rows_input_counter(), (int64_t)in_block->rows()); + return local_state.sink(state, in_block, eos); + } + +private: + friend class TVFTableSinkLocalState; + template + requires(std::is_base_of_v) + friend class AsyncWriterSink; + const RowDescriptor& _row_desc; + vectorized::VExprContextSPtrs _output_vexpr_ctxs; + const std::vector& _t_output_expr; + ObjectPool* _pool = nullptr; +}; + +#include "common/compile_check_end.h" +} // namespace doris::pipeline diff --git a/be/src/pipeline/pipeline_fragment_context.cpp b/be/src/pipeline/pipeline_fragment_context.cpp index 417eb88baa96bc..a2ef7406a07335 100644 --- a/be/src/pipeline/pipeline_fragment_context.cpp +++ b/be/src/pipeline/pipeline_fragment_context.cpp @@ -62,6 +62,7 @@ #include "pipeline/exec/hashjoin_build_sink.h" #include "pipeline/exec/hashjoin_probe_operator.h" #include "pipeline/exec/hive_table_sink_operator.h" +#include "pipeline/exec/tvf_table_sink_operator.h" #include "pipeline/exec/iceberg_table_sink_operator.h" #include "pipeline/exec/jdbc_scan_operator.h" #include "pipeline/exec/jdbc_table_sink_operator.h" @@ -1186,6 +1187,14 @@ Status PipelineFragmentContext::_create_data_sink(ObjectPool* pool, const TDataS _sink.reset(new BlackholeSinkOperatorX(next_sink_operator_id())); break; } + case TDataSinkType::TVF_TABLE_SINK: { + if (!thrift_sink.__isset.tvf_table_sink) { + return Status::InternalError("Missing TVF table sink."); + } + _sink = std::make_shared(pool, next_sink_operator_id(), row_desc, + output_exprs); + break; + } default: return Status::InternalError("Unsuported sink type in pipeline: {}", thrift_sink.type); } diff --git a/be/src/vec/sink/writer/vtvf_table_writer.cpp b/be/src/vec/sink/writer/vtvf_table_writer.cpp new file mode 100644 index 00000000000000..65160dc4eeba4b --- /dev/null +++ b/be/src/vec/sink/writer/vtvf_table_writer.cpp @@ -0,0 +1,245 @@ +// 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 "vec/sink/writer/vtvf_table_writer.h" + +#include +#include + +#include + +#include "common/status.h" +#include "io/file_factory.h" +#include "io/fs/local_file_system.h" +#include "runtime/runtime_state.h" +#include "vec/core/block.h" +#include "vec/exprs/vexpr.h" +#include "vec/exprs/vexpr_context.h" +#include "vec/runtime/vcsv_transformer.h" +#include "vec/runtime/vorc_transformer.h" +#include "vec/runtime/vparquet_transformer.h" + +namespace doris::vectorized { + +VTVFTableWriter::VTVFTableWriter(const TDataSink& t_sink, const VExprContextSPtrs& output_exprs, + std::shared_ptr dep, + std::shared_ptr fin_dep) + : AsyncResultWriter(output_exprs, dep, fin_dep) { + _tvf_sink = t_sink.tvf_table_sink; +} + +Status VTVFTableWriter::open(RuntimeState* state, RuntimeProfile* profile) { + _state = state; + + // Init profile counters + RuntimeProfile* writer_profile = profile->create_child("VTVFTableWriter", true, true); + _written_rows_counter = ADD_COUNTER(writer_profile, "NumWrittenRows", TUnit::UNIT); + _written_data_bytes = ADD_COUNTER(writer_profile, "WrittenDataBytes", TUnit::BYTES); + _file_write_timer = ADD_TIMER(writer_profile, "FileWriteTime"); + _writer_close_timer = ADD_TIMER(writer_profile, "FileWriterCloseTime"); + + _file_path = _tvf_sink.file_path; + _max_file_size_bytes = _tvf_sink.__isset.max_file_size_bytes ? _tvf_sink.max_file_size_bytes : 0; + _delete_existing_files_flag = + _tvf_sink.__isset.delete_existing_files ? _tvf_sink.delete_existing_files : true; + + // Delete existing files if requested + if (_delete_existing_files_flag) { + RETURN_IF_ERROR(_delete_existing_files()); + } + + return _create_next_file_writer(); +} + +Status VTVFTableWriter::write(RuntimeState* state, vectorized::Block& block) { + COUNTER_UPDATE(_written_rows_counter, block.rows()); + + { + SCOPED_TIMER(_file_write_timer); + RETURN_IF_ERROR(_vfile_writer->write(block)); + } + + _current_written_bytes = _vfile_writer->written_len(); + + // Auto-split if max file size is set + if (_max_file_size_bytes > 0) { + RETURN_IF_ERROR(_create_new_file_if_exceed_size()); + } + + return Status::OK(); +} + +Status VTVFTableWriter::close(Status status) { + if (!status.ok()) { + return status; + } + + SCOPED_TIMER(_writer_close_timer); + return _close_file_writer(true); +} + +Status VTVFTableWriter::_create_file_writer(const std::string& file_name) { + TFileType::type file_type = _tvf_sink.file_type; + std::map properties; + if (_tvf_sink.__isset.properties) { + properties = _tvf_sink.properties; + } + + _file_writer_impl = DORIS_TRY(FileFactory::create_file_writer( + file_type, _state->exec_env(), {}, properties, file_name, + { + .write_file_cache = false, + .sync_file_data = false, + })); + + TFileFormatType::type format = _tvf_sink.file_format; + switch (format) { + case TFileFormatType::FORMAT_CSV_PLAIN: { + std::string column_separator = + _tvf_sink.__isset.column_separator ? _tvf_sink.column_separator : ","; + std::string line_delimiter = + _tvf_sink.__isset.line_delimiter ? _tvf_sink.line_delimiter : "\n"; + TFileCompressType::type compress_type = TFileCompressType::PLAIN; + if (_tvf_sink.__isset.compression_type) { + compress_type = _tvf_sink.compression_type; + } + _vfile_writer.reset(new VCSVTransformer(_state, _file_writer_impl.get(), + _vec_output_expr_ctxs, false, {}, {}, column_separator, + line_delimiter, false, compress_type)); + break; + } + case TFileFormatType::FORMAT_PARQUET: { + // Build parquet schemas from columns + std::vector parquet_schemas; + if (_tvf_sink.__isset.columns) { + for (const auto& col : _tvf_sink.columns) { + TParquetSchema schema; + schema.__set_schema_column_name(col.column_name); + parquet_schemas.push_back(schema); + } + } + _vfile_writer.reset(new VParquetTransformer( + _state, _file_writer_impl.get(), _vec_output_expr_ctxs, parquet_schemas, false, + {TParquetCompressionType::SNAPPY, TParquetVersion::PARQUET_1_0, false, false})); + break; + } + case TFileFormatType::FORMAT_ORC: { + TFileCompressType::type compress_type = TFileCompressType::PLAIN; + if (_tvf_sink.__isset.compression_type) { + compress_type = _tvf_sink.compression_type; + } + _vfile_writer.reset(new VOrcTransformer(_state, _file_writer_impl.get(), + _vec_output_expr_ctxs, "", {}, false, + compress_type)); + break; + } + default: + return Status::InternalError("Unsupported TVF sink file format: {}", format); + } + + LOG(INFO) << "TVF table writer created file: " << file_name + << ", format: " << format + << ", query_id: " << print_id(_state->query_id()); + + return _vfile_writer->open(); +} + +Status VTVFTableWriter::_create_next_file_writer() { + std::string file_name; + RETURN_IF_ERROR(_get_next_file_name(&file_name)); + return _create_file_writer(file_name); +} + +Status VTVFTableWriter::_close_file_writer(bool done) { + if (_vfile_writer) { + RETURN_IF_ERROR(_vfile_writer->close()); + COUNTER_UPDATE(_written_data_bytes, _vfile_writer->written_len()); + _vfile_writer.reset(nullptr); + } else if (_file_writer_impl && _file_writer_impl->state() != io::FileWriter::State::CLOSED) { + RETURN_IF_ERROR(_file_writer_impl->close()); + } + + if (!done) { + RETURN_IF_ERROR(_create_next_file_writer()); + } + return Status::OK(); +} + +Status VTVFTableWriter::_create_new_file_if_exceed_size() { + if (_max_file_size_bytes <= 0 || _current_written_bytes < _max_file_size_bytes) { + return Status::OK(); + } + SCOPED_TIMER(_writer_close_timer); + RETURN_IF_ERROR(_close_file_writer(false)); + _current_written_bytes = 0; + return Status::OK(); +} + +Status VTVFTableWriter::_get_next_file_name(std::string* file_name) { + // Determine file extension + std::string ext; + switch (_tvf_sink.file_format) { + case TFileFormatType::FORMAT_CSV_PLAIN: + ext = "csv"; + break; + case TFileFormatType::FORMAT_PARQUET: + ext = "parquet"; + break; + case TFileFormatType::FORMAT_ORC: + ext = "orc"; + break; + default: + ext = "dat"; + break; + } + + if (_file_idx == 0 && _max_file_size_bytes <= 0) { + // Single file mode: use the path as-is if it already has extension + if (_file_path.find('.') != std::string::npos) { + *file_name = _file_path; + } else { + *file_name = fmt::format("{}.{}", _file_path, ext); + } + } else { + // Multi-file (auto-split) mode: append index + // Strip extension from base path if present + std::string base = _file_path; + auto dot_pos = base.rfind('.'); + if (dot_pos != std::string::npos) { + base = base.substr(0, dot_pos); + } + *file_name = fmt::format("{}_{}.{}", base, _file_idx, ext); + } + _file_idx++; + return Status::OK(); +} + +Status VTVFTableWriter::_delete_existing_files() { + if (_tvf_sink.file_type == TFileType::FILE_LOCAL) { + // For local files, try to delete the file if it exists + bool exists = false; + RETURN_IF_ERROR(io::global_local_filesystem()->exists(_file_path, &exists)); + if (exists) { + RETURN_IF_ERROR(io::global_local_filesystem()->delete_file(_file_path)); + } + } + // For S3/HDFS, we don't delete existing files by default + // as it requires more complex handling (e.g., directory listing) + return Status::OK(); +} + +} // namespace doris::vectorized diff --git a/be/src/vec/sink/writer/vtvf_table_writer.h b/be/src/vec/sink/writer/vtvf_table_writer.h new file mode 100644 index 00000000000000..9e21fb828acfa1 --- /dev/null +++ b/be/src/vec/sink/writer/vtvf_table_writer.h @@ -0,0 +1,89 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#pragma once + +#include +#include + +#include +#include + +#include "common/status.h" +#include "io/fs/file_writer.h" +#include "util/runtime_profile.h" +#include "vec/exprs/vexpr_fwd.h" +#include "vec/runtime/vfile_format_transformer.h" +#include "vec/sink/writer/async_result_writer.h" + +namespace doris { +class RuntimeState; +class RuntimeProfile; + +namespace vectorized { +class Block; + +/** + * VTVFTableWriter writes query result blocks to files (local/s3/hdfs) + * via the TVF sink path: INSERT INTO tvf_name(properties) SELECT ... + * + * It inherits from AsyncResultWriter to perform IO in a separate thread pool, + * avoiding blocking the pipeline execution engine. + */ +class VTVFTableWriter final : public AsyncResultWriter { +public: + VTVFTableWriter(const TDataSink& t_sink, const VExprContextSPtrs& output_exprs, + std::shared_ptr dep, + std::shared_ptr fin_dep); + + ~VTVFTableWriter() override = default; + + Status open(RuntimeState* state, RuntimeProfile* profile) override; + + Status write(RuntimeState* state, vectorized::Block& block) override; + + Status close(Status status) override; + +private: + Status _create_file_writer(const std::string& file_name); + Status _create_next_file_writer(); + Status _close_file_writer(bool done); + Status _create_new_file_if_exceed_size(); + Status _get_next_file_name(std::string* file_name); + Status _delete_existing_files(); + + TTVFTableSink _tvf_sink; + RuntimeState* _state = nullptr; + + std::unique_ptr _file_writer_impl; + std::unique_ptr _vfile_writer; + + int64_t _current_written_bytes = 0; + int64_t _max_file_size_bytes = 0; + int _file_idx = 0; + bool _delete_existing_files_flag = true; + std::string _file_path; + + // profile counters + RuntimeProfile::Counter* _written_rows_counter = nullptr; + RuntimeProfile::Counter* _written_data_bytes = nullptr; + RuntimeProfile::Counter* _file_write_timer = nullptr; + RuntimeProfile::Counter* _writer_close_timer = nullptr; +}; + +} // namespace vectorized +} // namespace doris diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 index be2c75019006a4..4120543f798d25 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 @@ -134,7 +134,11 @@ optSpecBranch ; supportedDmlStatement - : explain? cte? INSERT (INTO | OVERWRITE TABLE) + : explain? cte? INSERT INTO tvfName=identifier + LEFT_PAREN tvfProperties=propertyItemList RIGHT_PAREN + (WITH LABEL labelName=identifier)? + query #insertIntoTVF + | explain? cte? INSERT (INTO | OVERWRITE TABLE) (tableName=multipartIdentifier (optSpecBranch)? | DORIS_INTERNAL_TABLE_ID LEFT_PAREN tableId=INTEGER_VALUE RIGHT_PAREN) partitionSpec? // partition define (WITH LABEL labelName=identifier)? cols=identifierList? // label and columns define diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundTVFTableSink.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundTVFTableSink.java new file mode 100644 index 00000000000000..cfb44ffd27cfe3 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundTVFTableSink.java @@ -0,0 +1,130 @@ +// 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.doris.nereids.analyzer; + +import org.apache.doris.nereids.exceptions.UnboundException; +import org.apache.doris.nereids.memo.GroupExpression; +import org.apache.doris.nereids.properties.LogicalProperties; +import org.apache.doris.nereids.trees.expressions.NamedExpression; +import org.apache.doris.nereids.trees.expressions.Slot; +import org.apache.doris.nereids.trees.plans.BlockFuncDepsPropagation; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.algebra.Sink; +import org.apache.doris.nereids.trees.plans.commands.info.DMLCommandType; +import org.apache.doris.nereids.trees.plans.logical.UnboundLogicalSink; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.nereids.util.Utils; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; + +import java.util.List; +import java.util.Map; +import java.util.Optional; + +/** + * Unbound TVF table sink for INSERT INTO tvf_name(properties) SELECT ... + */ +public class UnboundTVFTableSink extends UnboundLogicalSink + implements Unbound, Sink, BlockFuncDepsPropagation { + + private final String tvfName; + private final Map properties; + + public UnboundTVFTableSink(String tvfName, Map properties, + DMLCommandType dmlCommandType, CHILD_TYPE child) { + super(ImmutableList.of(tvfName), + PlanType.LOGICAL_UNBOUND_TVF_TABLE_SINK, + ImmutableList.of(), + Optional.empty(), + Optional.empty(), + ImmutableList.of(), + dmlCommandType, + child); + this.tvfName = tvfName; + this.properties = properties; + } + + public UnboundTVFTableSink(String tvfName, Map properties, + DMLCommandType dmlCommandType, + Optional groupExpression, + Optional logicalProperties, CHILD_TYPE child) { + super(ImmutableList.of(tvfName), + PlanType.LOGICAL_UNBOUND_TVF_TABLE_SINK, + ImmutableList.of(), + groupExpression, + logicalProperties, + ImmutableList.of(), + dmlCommandType, + child); + this.tvfName = tvfName; + this.properties = properties; + } + + public String getTvfName() { + return tvfName; + } + + public Map getProperties() { + return properties; + } + + @Override + public Plan withChildren(List children) { + Preconditions.checkArgument(children.size() == 1, "UnboundTVFTableSink only accepts one child"); + return new UnboundTVFTableSink<>(tvfName, properties, getDMLCommandType(), + groupExpression, Optional.empty(), children.get(0)); + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitUnboundTVFTableSink(this, context); + } + + @Override + public Plan withGroupExpression(Optional groupExpression) { + return new UnboundTVFTableSink<>(tvfName, properties, getDMLCommandType(), + groupExpression, Optional.of(getLogicalProperties()), child()); + } + + @Override + public Plan withGroupExprLogicalPropChildren(Optional groupExpression, + Optional logicalProperties, List children) { + Preconditions.checkArgument(children.size() == 1, "UnboundTVFTableSink only accepts one child"); + return new UnboundTVFTableSink<>(tvfName, properties, getDMLCommandType(), + groupExpression, logicalProperties, children.get(0)); + } + + @Override + public UnboundTVFTableSink withOutputExprs(List outputExprs) { + throw new UnboundException("could not call withOutputExprs on UnboundTVFTableSink"); + } + + @Override + public List computeOutput() { + throw new UnboundException("output"); + } + + @Override + public String toString() { + return Utils.toSqlString("UnboundTVFTableSink[" + id.asInt() + "]", + "tvfName", tvfName, + "properties", properties); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java index 055ed7ab5d62b4..6173363632685d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java @@ -118,6 +118,7 @@ import org.apache.doris.nereids.trees.plans.physical.AbstractPhysicalSort; import org.apache.doris.nereids.trees.plans.physical.PhysicalAssertNumRows; import org.apache.doris.nereids.trees.plans.physical.PhysicalBlackholeSink; +import org.apache.doris.nereids.trees.plans.physical.PhysicalTVFTableSink; import org.apache.doris.nereids.trees.plans.physical.PhysicalCTEAnchor; import org.apache.doris.nereids.trees.plans.physical.PhysicalCTEConsumer; import org.apache.doris.nereids.trees.plans.physical.PhysicalCTEProducer; @@ -184,6 +185,7 @@ import org.apache.doris.planner.AssertNumRowsNode; import org.apache.doris.planner.BackendPartitionedSchemaScanNode; import org.apache.doris.planner.BlackholeSink; +import org.apache.doris.planner.TVFTableSink; import org.apache.doris.planner.CTEScanNode; import org.apache.doris.planner.DataPartition; import org.apache.doris.planner.DataStreamSink; @@ -457,6 +459,22 @@ public PlanFragment visitPhysicalBlackholeSink(PhysicalBlackholeSink tvfSink, + PlanTranslatorContext context) { + PlanFragment rootFragment = tvfSink.child().accept(this, context); + rootFragment.setOutputPartition(DataPartition.UNPARTITIONED); + TVFTableSink sink = new TVFTableSink( + tvfSink.getTvfName(), tvfSink.getProperties(), tvfSink.getCols()); + try { + sink.bindDataSink(); + } catch (Exception e) { + throw new RuntimeException("Failed to bind TVF table sink", e); + } + rootFragment.setSink(sink); + return rootFragment; + } + @Override public PlanFragment visitPhysicalResultSink(PhysicalResultSink physicalResultSink, PlanTranslatorContext context) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index 3291f25651f077..ec15a689d25226 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -241,6 +241,7 @@ import org.apache.doris.nereids.DorisParser.IndexDefContext; import org.apache.doris.nereids.DorisParser.IndexDefsContext; import org.apache.doris.nereids.DorisParser.InlineTableContext; +import org.apache.doris.nereids.DorisParser.InsertIntoTVFContext; import org.apache.doris.nereids.DorisParser.InsertTableContext; import org.apache.doris.nereids.DorisParser.InstallPluginContext; import org.apache.doris.nereids.DorisParser.IntegerLiteralContext; @@ -488,6 +489,7 @@ import org.apache.doris.nereids.analyzer.UnboundAlias; import org.apache.doris.nereids.analyzer.UnboundBlackholeSink; import org.apache.doris.nereids.analyzer.UnboundBlackholeSink.UnboundBlackholeSinkContext; +import org.apache.doris.nereids.analyzer.UnboundTVFTableSink; import org.apache.doris.nereids.analyzer.UnboundFunction; import org.apache.doris.nereids.analyzer.UnboundInlineTable; import org.apache.doris.nereids.analyzer.UnboundOneRowRelation; @@ -1000,6 +1002,7 @@ import org.apache.doris.nereids.trees.plans.commands.info.WarmUpItem; import org.apache.doris.nereids.trees.plans.commands.insert.BatchInsertIntoTableCommand; import org.apache.doris.nereids.trees.plans.commands.insert.InsertIntoTableCommand; +import org.apache.doris.nereids.trees.plans.commands.insert.InsertIntoTVFCommand; import org.apache.doris.nereids.trees.plans.commands.insert.InsertOverwriteTableCommand; import org.apache.doris.nereids.trees.plans.commands.insert.WarmupSelectCommand; import org.apache.doris.nereids.trees.plans.commands.load.CreateRoutineLoadCommand; @@ -1384,6 +1387,26 @@ public TlsOptions visitRequireClause(DorisParser.RequireClauseContext ctx) { return TlsOptions.of(options); } + @Override + public LogicalPlan visitInsertIntoTVF(InsertIntoTVFContext ctx) { + String tvfName = ctx.tvfName.getText(); + Map properties = visitPropertyItemList(ctx.tvfProperties); + Optional labelName = ctx.labelName == null + ? Optional.empty() : Optional.of(ctx.labelName.getText()); + LogicalPlan plan = visitQuery(ctx.query()); + + UnboundTVFTableSink sink = new UnboundTVFTableSink<>( + tvfName, properties, DMLCommandType.INSERT, plan); + + Optional cte = Optional.empty(); + if (ctx.cte() != null) { + cte = Optional.ofNullable(withCte(plan, ctx.cte())); + } + + LogicalPlan command = new InsertIntoTVFCommand(sink, labelName, cte); + return withExplain(command, ctx.explain()); + } + /** * This function may be used in some task like InsertTask, RefreshDictionary, etc. the target could be many type of * tables. diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleSet.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleSet.java index d8c119103a8a71..0ea4455d043a8e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleSet.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleSet.java @@ -60,6 +60,7 @@ import org.apache.doris.nereids.rules.implementation.AggregateStrategies; import org.apache.doris.nereids.rules.implementation.LogicalAssertNumRowsToPhysicalAssertNumRows; import org.apache.doris.nereids.rules.implementation.LogicalBlackholeSinkToPhysicalBlackholeSink; +import org.apache.doris.nereids.rules.implementation.LogicalTVFTableSinkToPhysicalTVFTableSink; import org.apache.doris.nereids.rules.implementation.LogicalCTEAnchorToPhysicalCTEAnchor; import org.apache.doris.nereids.rules.implementation.LogicalCTEConsumerToPhysicalCTEConsumer; import org.apache.doris.nereids.rules.implementation.LogicalCTEProducerToPhysicalCTEProducer; @@ -241,6 +242,7 @@ public class RuleSet { .add(new LogicalDeferMaterializeResultSinkToPhysicalDeferMaterializeResultSink()) .add(new LogicalDictionarySinkToPhysicalDictionarySink()) .add(new LogicalBlackholeSinkToPhysicalBlackholeSink()) + .add(new LogicalTVFTableSinkToPhysicalTVFTableSink()) .build(); // left-zig-zag tree is used when column stats are not available. diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java index a5f3d1612ef1e8..3fc2568480845c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java @@ -41,6 +41,7 @@ public enum RuleType { BINDING_INSERT_JDBC_TABLE(RuleTypeClass.REWRITE), BINDING_INSERT_TARGET_TABLE(RuleTypeClass.REWRITE), BINDING_INSERT_DICTIONARY_TABLE(RuleTypeClass.REWRITE), + BINDING_INSERT_TVF_TABLE(RuleTypeClass.REWRITE), INIT_MATERIALIZATION_HOOK_FOR_FILE_SINK(RuleTypeClass.REWRITE), INIT_MATERIALIZATION_HOOK_FOR_TABLE_SINK(RuleTypeClass.REWRITE), INIT_MATERIALIZATION_HOOK_FOR_RESULT_SINK(RuleTypeClass.REWRITE), @@ -558,6 +559,7 @@ public enum RuleType { LOGICAL_DEFER_MATERIALIZE_RESULT_SINK_TO_PHYSICAL_DEFER_MATERIALIZE_RESULT_SINK_RULE(RuleTypeClass.IMPLEMENTATION), LOGICAL_FILE_SINK_TO_PHYSICAL_FILE_SINK_RULE(RuleTypeClass.IMPLEMENTATION), LOGICAL_DICTIONARY_SINK_TO_PHYSICAL_DICTIONARY_SINK_RULE(RuleTypeClass.IMPLEMENTATION), + LOGICAL_TVF_TABLE_SINK_TO_PHYSICAL_TVF_TABLE_SINK_RULE(RuleTypeClass.IMPLEMENTATION), LOGICAL_ASSERT_NUM_ROWS_TO_PHYSICAL_ASSERT_NUM_ROWS(RuleTypeClass.IMPLEMENTATION), STORAGE_LAYER_AGGREGATE_WITHOUT_PROJECT(RuleTypeClass.IMPLEMENTATION), STORAGE_LAYER_AGGREGATE_WITH_PROJECT(RuleTypeClass.IMPLEMENTATION), diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindSink.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindSink.java index 8cf2e105816ac7..4282f372eed373 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindSink.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindSink.java @@ -47,6 +47,7 @@ import org.apache.doris.nereids.analyzer.UnboundIcebergTableSink; import org.apache.doris.nereids.analyzer.UnboundJdbcTableSink; import org.apache.doris.nereids.analyzer.UnboundSlot; +import org.apache.doris.nereids.analyzer.UnboundTVFTableSink; import org.apache.doris.nereids.analyzer.UnboundTableSink; import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.parser.NereidsParser; @@ -82,6 +83,7 @@ import org.apache.doris.nereids.trees.plans.logical.LogicalOneRowRelation; import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; import org.apache.doris.nereids.trees.plans.logical.LogicalProject; +import org.apache.doris.nereids.trees.plans.logical.LogicalTVFTableSink; import org.apache.doris.nereids.trees.plans.logical.LogicalTableSink; import org.apache.doris.nereids.trees.plans.logical.UnboundLogicalSink; import org.apache.doris.nereids.trees.plans.visitor.InferPlanOutputAlias; @@ -154,7 +156,8 @@ public List buildRules() { RuleType.BINDING_INSERT_JDBC_TABLE.build(unboundJdbcTableSink().thenApply(this::bindJdbcTableSink)), RuleType.BINDING_INSERT_DICTIONARY_TABLE .build(unboundDictionarySink().thenApply(this::bindDictionarySink)), - RuleType.BINDING_INSERT_BLACKHOLE_SINK.build(unboundBlackholeSink().thenApply(this::bindBlackHoleSink)) + RuleType.BINDING_INSERT_BLACKHOLE_SINK.build(unboundBlackholeSink().thenApply(this::bindBlackHoleSink)), + RuleType.BINDING_INSERT_TVF_TABLE.build(unboundTVFTableSink().thenApply(this::bindTVFTableSink)) ); } @@ -556,6 +559,43 @@ private Plan bindBlackHoleSink(MatchingContext> ctx) return boundSink; } + private Plan bindTVFTableSink(MatchingContext> ctx) { + UnboundTVFTableSink sink = ctx.root; + String tvfName = sink.getTvfName().toLowerCase(); + Map properties = sink.getProperties(); + + // Validate tvfName + if (!tvfName.equals("local") && !tvfName.equals("s3") && !tvfName.equals("hdfs")) { + throw new AnalysisException( + "INSERT INTO TVF only supports local/s3/hdfs, but got: " + tvfName); + } + + // Validate required properties + if (!properties.containsKey("file_path")) { + throw new AnalysisException("TVF sink requires 'file_path' property"); + } + if (!properties.containsKey("format")) { + throw new AnalysisException("TVF sink requires 'format' property"); + } + if (tvfName.equals("local") && !properties.containsKey("backend_id")) { + throw new AnalysisException("local TVF sink requires 'backend_id' property"); + } + + LogicalPlan child = ((LogicalPlan) sink.child()); + + // Derive columns from child query output + List cols = child.getOutput().stream() + .map(slot -> new Column(slot.getName(), slot.getDataType().toCatalogDataType())) + .collect(ImmutableList.toImmutableList()); + + List outputExprs = child.getOutput().stream() + .map(NamedExpression.class::cast) + .collect(ImmutableList.toImmutableList()); + + return new LogicalTVFTableSink<>(tvfName, properties, cols, outputExprs, + Optional.empty(), Optional.empty(), child); + } + private Plan bindHiveTableSink(MatchingContext> ctx) { UnboundHiveTableSink sink = ctx.root; Pair pair = bind(ctx.cascadesContext, sink); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalTVFTableSinkToPhysicalTVFTableSink.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalTVFTableSinkToPhysicalTVFTableSink.java new file mode 100644 index 00000000000000..70dae10ec2a196 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalTVFTableSinkToPhysicalTVFTableSink.java @@ -0,0 +1,46 @@ +// 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.doris.nereids.rules.implementation; + +import org.apache.doris.nereids.rules.Rule; +import org.apache.doris.nereids.rules.RuleType; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.logical.LogicalTVFTableSink; +import org.apache.doris.nereids.trees.plans.physical.PhysicalTVFTableSink; + +import java.util.Optional; + +/** + * Implementation rule to convert LogicalTVFTableSink to PhysicalTVFTableSink. + */ +public class LogicalTVFTableSinkToPhysicalTVFTableSink extends OneImplementationRuleFactory { + @Override + public Rule build() { + return logicalTVFTableSink().thenApply(ctx -> { + LogicalTVFTableSink sink = ctx.root; + return new PhysicalTVFTableSink<>( + sink.getTvfName(), + sink.getProperties(), + sink.getCols(), + sink.getOutputExprs(), + Optional.empty(), + sink.getLogicalProperties(), + sink.child()); + }).toRule(RuleType.LOGICAL_TVF_TABLE_SINK_TO_PHYSICAL_TVF_TABLE_SINK_RULE); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java index 0c5a16fc3efa76..5b89fa7b89bd59 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java @@ -61,6 +61,8 @@ public enum PlanType { LOGICAL_UNBOUND_RESULT_SINK, LOGICAL_UNBOUND_DICTIONARY_SINK, LOGICAL_UNBOUND_BLACKHOLE_SINK, + LOGICAL_UNBOUND_TVF_TABLE_SINK, + LOGICAL_TVF_TABLE_SINK, // logical others LOGICAL_AGGREGATE, @@ -121,6 +123,7 @@ public enum PlanType { PHYSICAL_RESULT_SINK, PHYSICAL_BLACKHOLE_SINK, PHYSICAL_DICTIONARY_SINK, + PHYSICAL_TVF_TABLE_SINK, // physical others PHYSICAL_HASH_AGGREGATE, @@ -163,6 +166,7 @@ public enum PlanType { INSERT_INTO_TABLE_COMMAND, INSERT_INTO_DICTIONARY_COMMAND, INSERT_INTO_BLACKHOLE_COMMAND, + INSERT_INTO_TVF_COMMAND, WARMUP_SELECT_COMMAND, BATCH_INSERT_INTO_TABLE_COMMAND, INSERT_OVERWRITE_TABLE_COMMAND, diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertIntoTVFCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertIntoTVFCommand.java new file mode 100644 index 00000000000000..12d824ca225fc5 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertIntoTVFCommand.java @@ -0,0 +1,158 @@ +// 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.doris.nereids.trees.plans.commands.insert; + +import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.EnvFactory; +import org.apache.doris.common.ErrorCode; +import org.apache.doris.common.ErrorReport; +import org.apache.doris.common.Status; +import org.apache.doris.mysql.privilege.PrivPredicate; +import org.apache.doris.nereids.NereidsPlanner; +import org.apache.doris.nereids.glue.LogicalPlanAdapter; +import org.apache.doris.nereids.trees.TreeNode; +import org.apache.doris.nereids.trees.plans.Explainable; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.commands.Command; +import org.apache.doris.nereids.trees.plans.commands.ForwardWithSync; +import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.ConnectContext.ConnectType; +import org.apache.doris.qe.Coordinator; +import org.apache.doris.qe.QeProcessorImpl; +import org.apache.doris.qe.QueryInfo; +import org.apache.doris.qe.StmtExecutor; +import org.apache.doris.thrift.TStatusCode; +import org.apache.doris.thrift.TUniqueId; + +import com.google.common.base.Preconditions; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.List; +import java.util.Optional; + +/** + * Command for INSERT INTO tvf_name(properties) SELECT ... + * This command is independent from InsertIntoTableCommand since TVF sink + * has no real table, no transaction, and no table lock. + */ +public class InsertIntoTVFCommand extends Command implements ForwardWithSync, Explainable { + + private static final Logger LOG = LogManager.getLogger(InsertIntoTVFCommand.class); + + private final LogicalPlan logicalQuery; + private final Optional labelName; + private final Optional cte; + + public InsertIntoTVFCommand(LogicalPlan logicalQuery, Optional labelName, + Optional cte) { + super(PlanType.INSERT_INTO_TVF_COMMAND); + this.logicalQuery = logicalQuery; + this.labelName = labelName; + this.cte = cte; + } + + @Override + public void run(ConnectContext ctx, StmtExecutor executor) throws Exception { + // 1. Check privilege + if (!Env.getCurrentEnv().getAccessManager().checkGlobalPriv(ctx, PrivPredicate.ADMIN) + && !Env.getCurrentEnv().getAccessManager().checkGlobalPriv(ctx, PrivPredicate.LOAD)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, + "INSERT INTO TVF requires ADMIN or LOAD privilege"); + } + + // 2. Prepare the plan + LogicalPlan plan = logicalQuery; + if (cte.isPresent()) { + plan = (LogicalPlan) cte.get().withChildren(plan); + } + + LogicalPlanAdapter logicalPlanAdapter = new LogicalPlanAdapter(plan, ctx.getStatementContext()); + NereidsPlanner planner = new NereidsPlanner(ctx.getStatementContext()); + planner.plan(logicalPlanAdapter, ctx.getSessionVariable().toThrift()); + + executor.setPlanner(planner); + executor.checkBlockRules(); + + if (ctx.getConnectType() == ConnectType.MYSQL && ctx.getMysqlChannel() != null) { + ctx.getMysqlChannel().reset(); + } + + // 3. Create coordinator + Coordinator coordinator = EnvFactory.getInstance().createCoordinator( + ctx, planner, ctx.getStatsErrorEstimator()); + + TUniqueId queryId = ctx.queryId(); + QeProcessorImpl.INSTANCE.registerQuery(queryId, + new QueryInfo(ctx, "INSERT INTO TVF", coordinator)); + + try { + coordinator.exec(); + + // Wait for completion + if (coordinator.join(ctx.getExecTimeout())) { + if (!coordinator.isDone()) { + coordinator.cancel(new Status(TStatusCode.INTERNAL_ERROR, "Insert into TVF timeout")); + ctx.getState().setError(ErrorCode.ERR_UNKNOWN_ERROR, "Insert into TVF timeout"); + return; + } + } + + if (coordinator.getExecStatus().ok()) { + String label = labelName.orElse( + String.format("tvf_insert_%x_%x", ctx.queryId().hi, ctx.queryId().lo)); + ctx.getState().setOk(0, 0, "Insert into TVF succeeded. label: " + label); + } else { + String errMsg = coordinator.getExecStatus().getErrorMsg(); + LOG.warn("insert into TVF failed, error: {}", errMsg); + ctx.getState().setError(ErrorCode.ERR_UNKNOWN_ERROR, errMsg); + } + } catch (Exception e) { + LOG.warn("insert into TVF failed", e); + ctx.getState().setError(ErrorCode.ERR_UNKNOWN_ERROR, + e.getMessage() == null ? "unknown error" : e.getMessage()); + } finally { + QeProcessorImpl.INSTANCE.unregisterQuery(queryId); + coordinator.close(); + } + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitCommand(this, context); + } + + @Override + public Plan getExplainPlan(ConnectContext ctx) { + return this.logicalQuery; + } + + @Override + public List> children() { + return List.of(logicalQuery); + } + + @Override + public Plan withChildren(List children) { + Preconditions.checkArgument(children.size() == 1); + return new InsertIntoTVFCommand((LogicalPlan) children.get(0), labelName, cte); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalTVFTableSink.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalTVFTableSink.java new file mode 100644 index 00000000000000..00b7e754782d77 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalTVFTableSink.java @@ -0,0 +1,135 @@ +// 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.doris.nereids.trees.plans.logical; + +import org.apache.doris.catalog.Column; +import org.apache.doris.nereids.memo.GroupExpression; +import org.apache.doris.nereids.properties.LogicalProperties; +import org.apache.doris.nereids.trees.expressions.NamedExpression; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.PropagateFuncDeps; +import org.apache.doris.nereids.trees.plans.algebra.Sink; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.nereids.util.Utils; + +import com.google.common.base.Preconditions; + +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; + +/** + * Logical TVF table sink for INSERT INTO tvf_name(properties) SELECT ... + */ +public class LogicalTVFTableSink extends LogicalSink + implements Sink, PropagateFuncDeps { + + private final String tvfName; + private final Map properties; + private final List cols; + + public LogicalTVFTableSink(String tvfName, Map properties, + List cols, List outputExprs, CHILD_TYPE child) { + super(PlanType.LOGICAL_TVF_TABLE_SINK, outputExprs, child); + this.tvfName = Objects.requireNonNull(tvfName, "tvfName should not be null"); + this.properties = Objects.requireNonNull(properties, "properties should not be null"); + this.cols = Objects.requireNonNull(cols, "cols should not be null"); + } + + public LogicalTVFTableSink(String tvfName, Map properties, + List cols, List outputExprs, + Optional groupExpression, + Optional logicalProperties, CHILD_TYPE child) { + super(PlanType.LOGICAL_TVF_TABLE_SINK, outputExprs, groupExpression, logicalProperties, child); + this.tvfName = Objects.requireNonNull(tvfName, "tvfName should not be null"); + this.properties = Objects.requireNonNull(properties, "properties should not be null"); + this.cols = Objects.requireNonNull(cols, "cols should not be null"); + } + + public String getTvfName() { + return tvfName; + } + + public Map getProperties() { + return properties; + } + + public List getCols() { + return cols; + } + + @Override + public LogicalTVFTableSink withChildren(List children) { + Preconditions.checkArgument(children.size() == 1, + "LogicalTVFTableSink's children size must be 1, but real is %s", children.size()); + return new LogicalTVFTableSink<>(tvfName, properties, cols, outputExprs, children.get(0)); + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitLogicalTVFTableSink(this, context); + } + + @Override + public LogicalTVFTableSink withGroupExpression(Optional groupExpression) { + return new LogicalTVFTableSink<>(tvfName, properties, cols, outputExprs, + groupExpression, Optional.of(getLogicalProperties()), child()); + } + + @Override + public LogicalTVFTableSink withGroupExprLogicalPropChildren(Optional groupExpression, + Optional logicalProperties, List children) { + Preconditions.checkArgument(children.size() == 1, "LogicalTVFTableSink only accepts one child"); + return new LogicalTVFTableSink<>(tvfName, properties, cols, outputExprs, + groupExpression, logicalProperties, children.get(0)); + } + + @Override + public LogicalTVFTableSink withOutputExprs(List outputExprs) { + return new LogicalTVFTableSink<>(tvfName, properties, cols, outputExprs, child()); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + LogicalTVFTableSink that = (LogicalTVFTableSink) o; + return tvfName.equals(that.tvfName) + && properties.equals(that.properties) + && cols.equals(that.cols) + && outputExprs.equals(that.outputExprs); + } + + @Override + public int hashCode() { + return Objects.hash(tvfName, properties, cols, outputExprs); + } + + @Override + public String toString() { + return Utils.toSqlString("LogicalTVFTableSink[" + id.asInt() + "]", + "tvfName", tvfName, + "outputExprs", outputExprs); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalTVFTableSink.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalTVFTableSink.java new file mode 100644 index 00000000000000..1220a98bb2f4b0 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalTVFTableSink.java @@ -0,0 +1,157 @@ +// 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.doris.nereids.trees.plans.physical; + +import org.apache.doris.catalog.Column; +import org.apache.doris.nereids.memo.GroupExpression; +import org.apache.doris.nereids.properties.LogicalProperties; +import org.apache.doris.nereids.properties.PhysicalProperties; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.NamedExpression; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.algebra.Sink; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.nereids.util.Utils; +import org.apache.doris.statistics.Statistics; + +import com.google.common.base.Preconditions; +import org.jetbrains.annotations.Nullable; + +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; + +/** + * Physical TVF table sink for INSERT INTO tvf_name(properties) SELECT ... + */ +public class PhysicalTVFTableSink extends PhysicalSink + implements Sink { + + private final String tvfName; + private final Map properties; + private final List cols; + + public PhysicalTVFTableSink(String tvfName, Map properties, + List cols, List outputExprs, + Optional groupExpression, + LogicalProperties logicalProperties, + CHILD_TYPE child) { + this(tvfName, properties, cols, outputExprs, groupExpression, + logicalProperties, PhysicalProperties.GATHER, null, child); + } + + public PhysicalTVFTableSink(String tvfName, Map properties, + List cols, List outputExprs, + Optional groupExpression, + LogicalProperties logicalProperties, + @Nullable PhysicalProperties physicalProperties, + Statistics statistics, CHILD_TYPE child) { + super(PlanType.PHYSICAL_TVF_TABLE_SINK, outputExprs, groupExpression, + logicalProperties, physicalProperties, statistics, child); + this.tvfName = Objects.requireNonNull(tvfName, "tvfName should not be null"); + this.properties = Objects.requireNonNull(properties, "properties should not be null"); + this.cols = Objects.requireNonNull(cols, "cols should not be null"); + } + + public String getTvfName() { + return tvfName; + } + + public Map getProperties() { + return properties; + } + + public List getCols() { + return cols; + } + + @Override + public PhysicalTVFTableSink withChildren(List children) { + Preconditions.checkArgument(children.size() == 1, + "PhysicalTVFTableSink's children size must be 1, but real is %s", children.size()); + return new PhysicalTVFTableSink<>(tvfName, properties, cols, outputExprs, + groupExpression, getLogicalProperties(), physicalProperties, statistics, children.get(0)); + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitPhysicalTVFTableSink(this, context); + } + + @Override + public List getExpressions() { + return outputExprs; + } + + @Override + public PhysicalTVFTableSink withGroupExpression(Optional groupExpression) { + return new PhysicalTVFTableSink<>(tvfName, properties, cols, outputExprs, + groupExpression, getLogicalProperties(), physicalProperties, statistics, child()); + } + + @Override + public PhysicalTVFTableSink withGroupExprLogicalPropChildren(Optional groupExpression, + Optional logicalProperties, List children) { + Preconditions.checkArgument(children.size() == 1, + "PhysicalTVFTableSink's children size must be 1, but real is %s", children.size()); + return new PhysicalTVFTableSink<>(tvfName, properties, cols, outputExprs, + groupExpression, logicalProperties.get(), physicalProperties, statistics, children.get(0)); + } + + @Override + public PhysicalTVFTableSink withPhysicalPropertiesAndStats( + PhysicalProperties physicalProperties, Statistics statistics) { + return new PhysicalTVFTableSink<>(tvfName, properties, cols, outputExprs, + groupExpression, getLogicalProperties(), physicalProperties, statistics, child()); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + PhysicalTVFTableSink that = (PhysicalTVFTableSink) o; + return tvfName.equals(that.tvfName) + && properties.equals(that.properties) + && cols.equals(that.cols) + && outputExprs.equals(that.outputExprs); + } + + @Override + public int hashCode() { + return Objects.hash(tvfName, properties, cols, outputExprs); + } + + @Override + public String toString() { + return Utils.toSqlString("PhysicalTVFTableSink[" + id.asInt() + "]", + "tvfName", tvfName, + "outputExprs", outputExprs); + } + + @Override + public PhysicalTVFTableSink resetLogicalProperties() { + return new PhysicalTVFTableSink<>(tvfName, properties, cols, outputExprs, + groupExpression, null, physicalProperties, statistics, child()); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/SinkVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/SinkVisitor.java index 411398ad2e0e8d..c39cf96d3b5474 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/SinkVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/SinkVisitor.java @@ -23,6 +23,7 @@ import org.apache.doris.nereids.analyzer.UnboundIcebergTableSink; import org.apache.doris.nereids.analyzer.UnboundJdbcTableSink; import org.apache.doris.nereids.analyzer.UnboundResultSink; +import org.apache.doris.nereids.analyzer.UnboundTVFTableSink; import org.apache.doris.nereids.analyzer.UnboundTableSink; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.logical.LogicalBlackholeSink; @@ -35,6 +36,7 @@ import org.apache.doris.nereids.trees.plans.logical.LogicalOlapTableSink; import org.apache.doris.nereids.trees.plans.logical.LogicalResultSink; import org.apache.doris.nereids.trees.plans.logical.LogicalSink; +import org.apache.doris.nereids.trees.plans.logical.LogicalTVFTableSink; import org.apache.doris.nereids.trees.plans.logical.LogicalTableSink; import org.apache.doris.nereids.trees.plans.physical.PhysicalBlackholeSink; import org.apache.doris.nereids.trees.plans.physical.PhysicalDeferMaterializeResultSink; @@ -46,6 +48,7 @@ import org.apache.doris.nereids.trees.plans.physical.PhysicalOlapTableSink; import org.apache.doris.nereids.trees.plans.physical.PhysicalResultSink; import org.apache.doris.nereids.trees.plans.physical.PhysicalSink; +import org.apache.doris.nereids.trees.plans.physical.PhysicalTVFTableSink; import org.apache.doris.nereids.trees.plans.physical.PhysicalTableSink; /** @@ -93,6 +96,10 @@ default R visitUnboundBlackholeSink(UnboundBlackholeSink unbound return visitLogicalSink(unboundBlackholeSink, context); } + default R visitUnboundTVFTableSink(UnboundTVFTableSink unboundTVFTableSink, C context) { + return visitLogicalSink(unboundTVFTableSink, context); + } + // ******************************* // logical // ******************************* @@ -139,6 +146,11 @@ default R visitLogicalBlackholeSink( return visitLogicalSink(logicalBlackholeSink, context); } + default R visitLogicalTVFTableSink( + LogicalTVFTableSink logicalTVFTableSink, C context) { + return visitLogicalSink(logicalTVFTableSink, context); + } + // ******************************* // physical // ******************************* @@ -184,4 +196,9 @@ default R visitPhysicalDeferMaterializeResultSink( PhysicalDeferMaterializeResultSink sink, C context) { return visitPhysicalSink(sink, context); } + + default R visitPhysicalTVFTableSink( + PhysicalTVFTableSink tvfTableSink, C context) { + return visitPhysicalSink(tvfTableSink, context); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/TVFTableSink.java b/fe/fe-core/src/main/java/org/apache/doris/planner/TVFTableSink.java new file mode 100644 index 00000000000000..8a2d73d881a534 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/TVFTableSink.java @@ -0,0 +1,178 @@ +// 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.doris.planner; + +import org.apache.doris.catalog.Column; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.thrift.TColumn; +import org.apache.doris.thrift.TDataSink; +import org.apache.doris.thrift.TDataSinkType; +import org.apache.doris.thrift.TExplainLevel; +import org.apache.doris.thrift.TFileCompressType; +import org.apache.doris.thrift.TFileFormatType; +import org.apache.doris.thrift.TFileType; +import org.apache.doris.thrift.TTVFTableSink; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +/** + * TVFTableSink is used for INSERT INTO tvf_name(properties) SELECT ... + * It writes query results to files via TVF (local/s3/hdfs). + */ +public class TVFTableSink extends DataSink { + private final String tvfName; + private final Map properties; + private final List cols; + private TDataSink tDataSink; + + public TVFTableSink(String tvfName, Map properties, List cols) { + this.tvfName = tvfName; + this.properties = properties; + this.cols = cols; + } + + public void bindDataSink() throws AnalysisException { + TTVFTableSink tSink = new TTVFTableSink(); + tSink.setTvfName(tvfName); + + String filePath = properties.get("file_path"); + tSink.setFilePath(filePath); + + // Set file format + String format = properties.getOrDefault("format", "csv").toLowerCase(); + TFileFormatType formatType = getFormatType(format); + tSink.setFileFormat(formatType); + + // Set file type based on TVF name + TFileType fileType = getFileType(tvfName); + tSink.setFileType(fileType); + + // Set all properties for BE to access + tSink.setProperties(properties); + + // Set columns + List tColumns = new ArrayList<>(); + for (Column col : cols) { + tColumns.add(col.toThrift()); + } + tSink.setColumns(tColumns); + + // Set column separator + String columnSeparator = properties.getOrDefault("column_separator", ","); + tSink.setColumnSeparator(columnSeparator); + + // Set line delimiter + String lineDelimiter = properties.getOrDefault("line_delimiter", "\n"); + tSink.setLineDelimiter(lineDelimiter); + + // Set max file size + String maxFileSizeStr = properties.get("max_file_size"); + if (maxFileSizeStr != null) { + tSink.setMaxFileSizeBytes(Long.parseLong(maxFileSizeStr)); + } + + // Set delete existing files flag + String deleteExisting = properties.getOrDefault("delete_existing_files", "true"); + tSink.setDeleteExistingFiles(Boolean.parseBoolean(deleteExisting)); + + // Set compression type + String compression = properties.get("compression_type"); + if (compression != null) { + tSink.setCompressionType(getCompressType(compression)); + } + + // Set backend id for local TVF + String backendIdStr = properties.get("backend_id"); + if (backendIdStr != null) { + tSink.setBackendId(Long.parseLong(backendIdStr)); + } + + // Set hadoop config for hdfs/s3 + if (tvfName.equals("hdfs") || tvfName.equals("s3")) { + tSink.setHadoopConfig(properties); + } + + tDataSink = new TDataSink(TDataSinkType.TVF_TABLE_SINK); + tDataSink.setTvfTableSink(tSink); + } + + private TFileFormatType getFormatType(String format) throws AnalysisException { + switch (format) { + case "csv": + return TFileFormatType.FORMAT_CSV_PLAIN; + case "parquet": + return TFileFormatType.FORMAT_PARQUET; + case "orc": + return TFileFormatType.FORMAT_ORC; + default: + throw new AnalysisException("Unsupported TVF sink format: " + format + + ". Supported formats: csv, parquet, orc"); + } + } + + private TFileType getFileType(String tvfName) throws AnalysisException { + switch (tvfName.toLowerCase()) { + case "local": + return TFileType.FILE_LOCAL; + case "s3": + return TFileType.FILE_S3; + case "hdfs": + return TFileType.FILE_HDFS; + default: + throw new AnalysisException("Unsupported TVF type: " + tvfName); + } + } + + private TFileCompressType getCompressType(String compressType) { + switch (compressType.toLowerCase()) { + case "snappy": + return TFileCompressType.SNAPPYBLOCK; + case "lz4": + return TFileCompressType.LZ4BLOCK; + case "gzip": + case "gz": + return TFileCompressType.GZ; + case "zstd": + return TFileCompressType.ZSTD; + default: + return TFileCompressType.PLAIN; + } + } + + @Override + public String getExplainString(String prefix, TExplainLevel explainLevel) { + StringBuilder strBuilder = new StringBuilder(); + strBuilder.append(prefix).append("TVF TABLE SINK\n"); + strBuilder.append(prefix).append(" tvfName: ").append(tvfName).append("\n"); + strBuilder.append(prefix).append(" filePath: ").append(properties.get("file_path")).append("\n"); + strBuilder.append(prefix).append(" format: ").append(properties.getOrDefault("format", "csv")).append("\n"); + return strBuilder.toString(); + } + + @Override + protected TDataSink toThrift() { + return tDataSink; + } + + @Override + public DataPartition getOutputPartition() { + return null; + } +} diff --git a/gensrc/thrift/DataSinks.thrift b/gensrc/thrift/DataSinks.thrift index f151c58917bdc5..c5e176ee155d33 100644 --- a/gensrc/thrift/DataSinks.thrift +++ b/gensrc/thrift/DataSinks.thrift @@ -42,6 +42,7 @@ enum TDataSinkType { ICEBERG_TABLE_SINK = 14, DICTIONARY_SINK = 15, BLACKHOLE_SINK = 16, + TVF_TABLE_SINK = 17, } enum TResultSinkType { @@ -452,6 +453,22 @@ struct TDictionarySink { struct TBlackholeSink { } +struct TTVFTableSink { + 1: optional string tvf_name // "local", "s3", "hdfs" + 2: optional string file_path + 3: optional PlanNodes.TFileFormatType file_format + 4: optional Types.TFileType file_type // FILE_LOCAL, FILE_S3, FILE_HDFS + 5: optional map properties + 6: optional list columns + 7: optional string column_separator + 8: optional string line_delimiter + 9: optional i64 max_file_size_bytes + 10: optional bool delete_existing_files + 11: optional map hadoop_config + 12: optional PlanNodes.TFileCompressType compression_type + 13: optional i64 backend_id // local TVF: specify BE +} + struct TDataSink { 1: required TDataSinkType type 2: optional TDataStreamSink stream_sink @@ -468,4 +485,5 @@ struct TDataSink { 14: optional TIcebergTableSink iceberg_table_sink 15: optional TDictionarySink dictionary_sink 16: optional TBlackholeSink blackhole_sink + 17: optional TTVFTableSink tvf_table_sink } From 297e20f4c14b1391144f168d9fe28db000ace187 Mon Sep 17 00:00:00 2001 From: morningman Date: Thu, 12 Feb 2026 22:21:49 +0800 Subject: [PATCH 02/13] 2 csv pass --- be/src/pipeline/exec/operator.cpp | 3 + be/src/pipeline/pipeline_fragment_context.cpp | 2 +- be/src/vec/sink/writer/vtvf_table_writer.cpp | 56 +++++++------ be/src/vec/sink/writer/vtvf_table_writer.h | 4 + fe/.idea/vcs.xml | 24 ++---- .../translator/PhysicalPlanTranslator.java | 1 + .../nereids/rules/analysis/BindSink.java | 82 +++++++++++++++++-- .../rules/analysis/CollectRelation.java | 5 ++ .../commands/insert/InsertIntoTVFCommand.java | 19 +---- .../apache/doris/planner/TVFTableSink.java | 9 +- 10 files changed, 139 insertions(+), 66 deletions(-) diff --git a/be/src/pipeline/exec/operator.cpp b/be/src/pipeline/exec/operator.cpp index d4fd3f6276dad5..ff92b598b79fb2 100644 --- a/be/src/pipeline/exec/operator.cpp +++ b/be/src/pipeline/exec/operator.cpp @@ -80,6 +80,7 @@ #include "pipeline/exec/spill_sort_source_operator.h" #include "pipeline/exec/streaming_aggregation_operator.h" #include "pipeline/exec/table_function_operator.h" +#include "pipeline/exec/tvf_table_sink_operator.h" #include "pipeline/exec/union_sink_operator.h" #include "pipeline/exec/union_source_operator.h" #include "pipeline/local_exchange/local_exchange_sink_operator.h" @@ -817,6 +818,7 @@ DECLARE_OPERATOR(ResultFileSinkLocalState) DECLARE_OPERATOR(OlapTableSinkLocalState) DECLARE_OPERATOR(OlapTableSinkV2LocalState) DECLARE_OPERATOR(HiveTableSinkLocalState) +DECLARE_OPERATOR(TVFTableSinkLocalState) DECLARE_OPERATOR(IcebergTableSinkLocalState) DECLARE_OPERATOR(AnalyticSinkLocalState) DECLARE_OPERATOR(BlackholeSinkLocalState) @@ -936,6 +938,7 @@ template class AsyncWriterSink; template class AsyncWriterSink; template class AsyncWriterSink; +template class AsyncWriterSink; #ifdef BE_TEST template class OperatorX; diff --git a/be/src/pipeline/pipeline_fragment_context.cpp b/be/src/pipeline/pipeline_fragment_context.cpp index a2ef7406a07335..4dfa58e45a0ac5 100644 --- a/be/src/pipeline/pipeline_fragment_context.cpp +++ b/be/src/pipeline/pipeline_fragment_context.cpp @@ -62,7 +62,6 @@ #include "pipeline/exec/hashjoin_build_sink.h" #include "pipeline/exec/hashjoin_probe_operator.h" #include "pipeline/exec/hive_table_sink_operator.h" -#include "pipeline/exec/tvf_table_sink_operator.h" #include "pipeline/exec/iceberg_table_sink_operator.h" #include "pipeline/exec/jdbc_scan_operator.h" #include "pipeline/exec/jdbc_table_sink_operator.h" @@ -101,6 +100,7 @@ #include "pipeline/exec/spill_sort_source_operator.h" #include "pipeline/exec/streaming_aggregation_operator.h" #include "pipeline/exec/table_function_operator.h" +#include "pipeline/exec/tvf_table_sink_operator.h" #include "pipeline/exec/union_sink_operator.h" #include "pipeline/exec/union_source_operator.h" #include "pipeline/local_exchange/local_exchange_sink_operator.h" diff --git a/be/src/vec/sink/writer/vtvf_table_writer.cpp b/be/src/vec/sink/writer/vtvf_table_writer.cpp index 65160dc4eeba4b..fbe579c9a680b6 100644 --- a/be/src/vec/sink/writer/vtvf_table_writer.cpp +++ b/be/src/vec/sink/writer/vtvf_table_writer.cpp @@ -20,8 +20,6 @@ #include #include -#include - #include "common/status.h" #include "io/file_factory.h" #include "io/fs/local_file_system.h" @@ -53,10 +51,18 @@ Status VTVFTableWriter::open(RuntimeState* state, RuntimeProfile* profile) { _writer_close_timer = ADD_TIMER(writer_profile, "FileWriterCloseTime"); _file_path = _tvf_sink.file_path; - _max_file_size_bytes = _tvf_sink.__isset.max_file_size_bytes ? _tvf_sink.max_file_size_bytes : 0; + _max_file_size_bytes = + _tvf_sink.__isset.max_file_size_bytes ? _tvf_sink.max_file_size_bytes : 0; _delete_existing_files_flag = _tvf_sink.__isset.delete_existing_files ? _tvf_sink.delete_existing_files : true; + VLOG_DEBUG << "TVF table writer open, query_id=" << print_id(_state->query_id()) + << ", tvf_name=" << _tvf_sink.tvf_name << ", file_path=" << _tvf_sink.file_path + << ", file_format=" << _tvf_sink.file_format << ", file_type=" << _tvf_sink.file_type + << ", max_file_size_bytes=" << _max_file_size_bytes + << ", delete_existing_files=" << _delete_existing_files_flag + << ", columns_count=" << (_tvf_sink.__isset.columns ? _tvf_sink.columns.size() : 0); + // Delete existing files if requested if (_delete_existing_files_flag) { RETURN_IF_ERROR(_delete_existing_files()); @@ -99,41 +105,38 @@ Status VTVFTableWriter::_create_file_writer(const std::string& file_name) { properties = _tvf_sink.properties; } - _file_writer_impl = DORIS_TRY(FileFactory::create_file_writer( - file_type, _state->exec_env(), {}, properties, file_name, - { - .write_file_cache = false, - .sync_file_data = false, - })); + _file_writer_impl = DORIS_TRY(FileFactory::create_file_writer(file_type, _state->exec_env(), {}, + properties, file_name, + { + .write_file_cache = false, + .sync_file_data = false, + })); TFileFormatType::type format = _tvf_sink.file_format; switch (format) { case TFileFormatType::FORMAT_CSV_PLAIN: { - std::string column_separator = - _tvf_sink.__isset.column_separator ? _tvf_sink.column_separator : ","; - std::string line_delimiter = - _tvf_sink.__isset.line_delimiter ? _tvf_sink.line_delimiter : "\n"; + _column_separator = _tvf_sink.__isset.column_separator ? _tvf_sink.column_separator : ","; + _line_delimiter = _tvf_sink.__isset.line_delimiter ? _tvf_sink.line_delimiter : "\n"; TFileCompressType::type compress_type = TFileCompressType::PLAIN; if (_tvf_sink.__isset.compression_type) { compress_type = _tvf_sink.compression_type; } - _vfile_writer.reset(new VCSVTransformer(_state, _file_writer_impl.get(), - _vec_output_expr_ctxs, false, {}, {}, column_separator, - line_delimiter, false, compress_type)); + _vfile_writer.reset(new VCSVTransformer( + _state, _file_writer_impl.get(), _vec_output_expr_ctxs, false, {}, {}, + _column_separator, _line_delimiter, false, compress_type)); break; } case TFileFormatType::FORMAT_PARQUET: { - // Build parquet schemas from columns - std::vector parquet_schemas; + _parquet_schemas.clear(); if (_tvf_sink.__isset.columns) { for (const auto& col : _tvf_sink.columns) { TParquetSchema schema; schema.__set_schema_column_name(col.column_name); - parquet_schemas.push_back(schema); + _parquet_schemas.push_back(schema); } } _vfile_writer.reset(new VParquetTransformer( - _state, _file_writer_impl.get(), _vec_output_expr_ctxs, parquet_schemas, false, + _state, _file_writer_impl.get(), _vec_output_expr_ctxs, _parquet_schemas, false, {TParquetCompressionType::SNAPPY, TParquetVersion::PARQUET_1_0, false, false})); break; } @@ -142,8 +145,14 @@ Status VTVFTableWriter::_create_file_writer(const std::string& file_name) { if (_tvf_sink.__isset.compression_type) { compress_type = _tvf_sink.compression_type; } + _orc_column_names.clear(); + if (_tvf_sink.__isset.columns) { + for (const auto& col : _tvf_sink.columns) { + _orc_column_names.push_back(col.column_name); + } + } _vfile_writer.reset(new VOrcTransformer(_state, _file_writer_impl.get(), - _vec_output_expr_ctxs, "", {}, false, + _vec_output_expr_ctxs, "", _orc_column_names, false, compress_type)); break; } @@ -151,9 +160,8 @@ Status VTVFTableWriter::_create_file_writer(const std::string& file_name) { return Status::InternalError("Unsupported TVF sink file format: {}", format); } - LOG(INFO) << "TVF table writer created file: " << file_name - << ", format: " << format - << ", query_id: " << print_id(_state->query_id()); + VLOG_DEBUG << "TVF table writer created file: " << file_name << ", format=" << format + << ", query_id=" << print_id(_state->query_id()); return _vfile_writer->open(); } diff --git a/be/src/vec/sink/writer/vtvf_table_writer.h b/be/src/vec/sink/writer/vtvf_table_writer.h index 9e21fb828acfa1..7b6d1239860293 100644 --- a/be/src/vec/sink/writer/vtvf_table_writer.h +++ b/be/src/vec/sink/writer/vtvf_table_writer.h @@ -77,6 +77,10 @@ class VTVFTableWriter final : public AsyncResultWriter { int _file_idx = 0; bool _delete_existing_files_flag = true; std::string _file_path; + std::string _column_separator; + std::string _line_delimiter; + std::vector _parquet_schemas; + std::vector _orc_column_names; // profile counters RuntimeProfile::Counter* _written_rows_counter = nullptr; diff --git a/fe/.idea/vcs.xml b/fe/.idea/vcs.xml index 7b2cdb1cbbd39a..e5158c553f8f33 100644 --- a/fe/.idea/vcs.xml +++ b/fe/.idea/vcs.xml @@ -1,20 +1,4 @@ - - + + + + + - + \ No newline at end of file diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java index 6173363632685d..8f3a285da006ec 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java @@ -465,6 +465,7 @@ public PlanFragment visitPhysicalTVFTableSink(PhysicalTVFTableSink> ctx) { LogicalPlan child = ((LogicalPlan) sink.child()); - // Derive columns from child query output - List cols = child.getOutput().stream() - .map(slot -> new Column(slot.getName(), slot.getDataType().toCatalogDataType())) - .collect(ImmutableList.toImmutableList()); + // Determine target schema: if append mode and file exists, use existing file schema; + // otherwise derive from child query output. + boolean deleteExisting = Boolean.parseBoolean( + properties.getOrDefault("delete_existing_files", "true")); + List cols = null; + if (!deleteExisting) { + cols = tryGetExistingFileSchema(tvfName, properties); + } + if (cols == null) { + cols = child.getOutput().stream() + .map(slot -> new Column(slot.getName(), slot.getDataType().toCatalogDataType())) + .collect(ImmutableList.toImmutableList()); + } + + // Validate column count + if (cols.size() != child.getOutput().size()) { + throw new AnalysisException( + "insert into cols should be corresponding to the query output" + + ", target columns: " + cols.size() + + ", query output: " + child.getOutput().size()); + } + + // Build columnToOutput mapping and reuse getOutputProjectByCoercion for type cast, + // same as OlapTable INSERT INTO. + Map columnToOutput = Maps.newLinkedHashMap(); + for (int i = 0; i < cols.size(); i++) { + Column col = cols.get(i); + NamedExpression childExpr = (NamedExpression) child.getOutput().get(i); + Alias output = new Alias(TypeCoercionUtils.castIfNotSameType( + childExpr, DataType.fromCatalogType(col.getType())), col.getName()); + columnToOutput.put(col.getName(), output); + } + LogicalProject projectWithCast = getOutputProjectByCoercion(cols, child, columnToOutput); - List outputExprs = child.getOutput().stream() + List outputExprs = projectWithCast.getOutput().stream() .map(NamedExpression.class::cast) .collect(ImmutableList.toImmutableList()); return new LogicalTVFTableSink<>(tvfName, properties, cols, outputExprs, - Optional.empty(), Optional.empty(), child); + Optional.empty(), Optional.empty(), projectWithCast); + } + + /** + * Try to instantiate the corresponding TVF to read the existing file's schema. + * Returns null if the file does not exist or schema inference fails. + */ + private List tryGetExistingFileSchema(String tvfName, Map properties) { + try { + ExternalFileTableValuedFunction tvf; + Map propsCopy = new HashMap<>(properties); + switch (tvfName) { + case "local": + tvf = new LocalTableValuedFunction(propsCopy); + break; + case "s3": + tvf = new S3TableValuedFunction(propsCopy); + break; + case "hdfs": + tvf = new HdfsTableValuedFunction(propsCopy); + break; + default: + return null; + } + List columns = tvf.getTableColumns(); + if (columns != null && !columns.isEmpty()) { + return columns; + } + } catch (Exception e) { + // File does not exist or schema inference failed — fall back to child query schema + LOG.info("TVF sink: could not read existing file schema for append mode, " + + "will use child query schema. Reason: " + e.getMessage()); + } + return null; } private Plan bindHiveTableSink(MatchingContext> ctx) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CollectRelation.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CollectRelation.java index 6d0b20801676aa..fee51734485afa 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CollectRelation.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CollectRelation.java @@ -32,6 +32,7 @@ import org.apache.doris.nereids.analyzer.UnboundDictionarySink; import org.apache.doris.nereids.analyzer.UnboundRelation; import org.apache.doris.nereids.analyzer.UnboundResultSink; +import org.apache.doris.nereids.analyzer.UnboundTVFTableSink; import org.apache.doris.nereids.analyzer.UnboundTableSink; import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.parser.NereidsParser; @@ -139,6 +140,10 @@ private Plan collectFromAny(MatchingContext ctx) { } private Plan collectFromUnboundSink(MatchingContext> ctx) { + // TVF sink (local/s3/hdfs) is not a real table, skip table collection + if (ctx.root instanceof UnboundTVFTableSink) { + return null; + } List nameParts = ctx.root.getNameParts(); switch (nameParts.size()) { case 1: diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertIntoTVFCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertIntoTVFCommand.java index 12d824ca225fc5..2066a67b04be5c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertIntoTVFCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertIntoTVFCommand.java @@ -25,7 +25,6 @@ import org.apache.doris.mysql.privilege.PrivPredicate; import org.apache.doris.nereids.NereidsPlanner; import org.apache.doris.nereids.glue.LogicalPlanAdapter; -import org.apache.doris.nereids.trees.TreeNode; import org.apache.doris.nereids.trees.plans.Explainable; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.PlanType; @@ -37,16 +36,14 @@ import org.apache.doris.qe.ConnectContext.ConnectType; import org.apache.doris.qe.Coordinator; import org.apache.doris.qe.QeProcessorImpl; -import org.apache.doris.qe.QueryInfo; +import org.apache.doris.qe.QeProcessorImpl.QueryInfo; import org.apache.doris.qe.StmtExecutor; import org.apache.doris.thrift.TStatusCode; import org.apache.doris.thrift.TUniqueId; -import com.google.common.base.Preconditions; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; -import java.util.List; import java.util.Optional; /** @@ -108,7 +105,8 @@ public void run(ConnectContext ctx, StmtExecutor executor) throws Exception { coordinator.exec(); // Wait for completion - if (coordinator.join(ctx.getExecTimeout())) { + int timeoutS = ctx.getExecTimeoutS(); + if (coordinator.join(timeoutS)) { if (!coordinator.isDone()) { coordinator.cancel(new Status(TStatusCode.INTERNAL_ERROR, "Insert into TVF timeout")); ctx.getState().setError(ErrorCode.ERR_UNKNOWN_ERROR, "Insert into TVF timeout"); @@ -144,15 +142,4 @@ public R accept(PlanVisitor visitor, C context) { public Plan getExplainPlan(ConnectContext ctx) { return this.logicalQuery; } - - @Override - public List> children() { - return List.of(logicalQuery); - } - - @Override - public Plan withChildren(List children) { - Preconditions.checkArgument(children.size() == 1); - return new InsertIntoTVFCommand((LogicalPlan) children.get(0), labelName, cte); - } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/TVFTableSink.java b/fe/fe-core/src/main/java/org/apache/doris/planner/TVFTableSink.java index 8a2d73d881a534..a99e8a1eb603c3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/TVFTableSink.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/TVFTableSink.java @@ -37,12 +37,14 @@ * It writes query results to files via TVF (local/s3/hdfs). */ public class TVFTableSink extends DataSink { + private final PlanNodeId exchNodeId; private final String tvfName; private final Map properties; private final List cols; private TDataSink tDataSink; - public TVFTableSink(String tvfName, Map properties, List cols) { + public TVFTableSink(PlanNodeId exchNodeId, String tvfName, Map properties, List cols) { + this.exchNodeId = exchNodeId; this.tvfName = tvfName; this.properties = properties; this.cols = cols; @@ -175,4 +177,9 @@ protected TDataSink toThrift() { public DataPartition getOutputPartition() { return null; } + + @Override + public PlanNodeId getExchNodeId() { + return exchNodeId; + } } From ad62293aa6f995d962954fc631132e55a5ad414a Mon Sep 17 00:00:00 2001 From: morningman Date: Fri, 13 Feb 2026 00:08:27 +0800 Subject: [PATCH 03/13] local test --- .../translator/PhysicalPlanTranslator.java | 3 +- .../properties/RequestPropertyDeriver.java | 9 + .../tvf/insert/test_insert_into_local_tvf.out | 127 ++++ .../insert/test_insert_into_local_tvf.groovy | 617 ++++++++++++++++++ 4 files changed, 755 insertions(+), 1 deletion(-) create mode 100644 regression-test/data/external_table_p0/tvf/insert/test_insert_into_local_tvf.out create mode 100644 regression-test/suites/external_table_p0/tvf/insert/test_insert_into_local_tvf.groovy diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java index 8f3a285da006ec..3e8bc18a7ec857 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java @@ -42,6 +42,7 @@ import org.apache.doris.catalog.TableIf; import org.apache.doris.common.Config; import org.apache.doris.common.Pair; +import org.apache.doris.common.util.Util; import org.apache.doris.datasource.ExternalTable; import org.apache.doris.datasource.FileQueryScanNode; import org.apache.doris.datasource.doris.RemoteDorisExternalTable; @@ -470,7 +471,7 @@ public PlanFragment visitPhysicalTVFTableSink(PhysicalTVFTableSink tvfTableSink, PlanContext context) { + // TVF sink writes to a single file on a single BE, so all data must be gathered + addRequestPropertyToChildren(PhysicalProperties.GATHER); + return null; + } + @Override public Void visitPhysicalDictionarySink(PhysicalDictionarySink dictionarySink, PlanContext context) { diff --git a/regression-test/data/external_table_p0/tvf/insert/test_insert_into_local_tvf.out b/regression-test/data/external_table_p0/tvf/insert/test_insert_into_local_tvf.out new file mode 100644 index 00000000000000..2dd0e051ab5b73 --- /dev/null +++ b/regression-test/data/external_table_p0/tvf/insert/test_insert_into_local_tvf.out @@ -0,0 +1,127 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !csv_basic_types -- +0,-1,-100,-1000,-100000,-1.1,-2.2,-123.45,2020-02-29,2020-02-29 00:00:00,,special_chars +0,2,200,2000,200000,3.3,4.4,678.90,2024-06-15,2024-06-15 12:30:00,foo,bar +1,1,100,1000,100000,1.1,2.2,123.45,2024-01-01,2024-01-01 10:00:00,hello,world +1,3,300,3000,300000,5.5,6.6,999.99,2024-12-31,2024-12-31 23:59:59,test,data +\\N,\\N,\\N,\\N,\\N,\\N,\\N,\\N,\\N,\\N,\\N,\\N + +-- !parquet_basic_types -- +\N \N \N \N \N \N \N \N \N \N \N \N +false -1 -100 -1000 -100000 -1.1 -2.2 -123.45 2020-02-29 2020-02-29T00:00 special_chars +false 2 200 2000 200000 3.3 4.4 678.90 2024-06-15 2024-06-15T12:30 foo bar +true 1 100 1000 100000 1.1 2.2 123.45 2024-01-01 2024-01-01T10:00 hello world +true 3 300 3000 300000 5.5 6.6 999.99 2024-12-31 2024-12-31T23:59:59 test data + +-- !orc_basic_types -- +\N \N \N \N \N \N \N \N \N \N \N \N +false -1 -100 -1000 -100000 -1.1 -2.2 -123.45 2020-02-29 2020-02-29T00:00 special_chars +false 2 200 2000 200000 3.3 4.4 678.90 2024-06-15 2024-06-15T12:30 foo bar +true 1 100 1000 100000 1.1 2.2 123.45 2024-01-01 2024-01-01T10:00 hello world +true 3 300 3000 300000 5.5 6.6 999.99 2024-12-31 2024-12-31T23:59:59 test data + +-- !parquet_complex_types -- +1 [1, 2, 3] {"a":1, "b":2} {"f1":1, "f2":"hello"} +2 [4, 5] {"x":10} {"f1":2, "f2":"world"} +3 [] {} {"f1":3, "f2":""} +4 \N \N \N + +-- !orc_complex_types -- +1 [1, 2, 3] {"a":1, "b":2} {"f1":1, "f2":"hello"} +2 [4, 5] {"x":10} {"f1":2, "f2":"world"} +3 [] {} {"f1":3, "f2":""} +4 \N \N \N + +-- !csv_sep_comma -- +-1000 special_chars +1000 hello world +2000 foo bar +3000 test data + +-- !csv_sep_tab -- +-1000 special_chars +1000 hello world +2000 foo bar +3000 test data + +-- !csv_sep_pipe -- +-1000 special_chars +1000 hello world +2000 foo bar +3000 test data + +-- !csv_sep_multi -- +-1000 special_chars +1000 hello world +2000 foo bar +3000 test data + +-- !csv_line_crlf -- +-1000,,special_chars +1000,hello,world +2000,foo,bar +3000,test,data + +-- !csv_compress_gz -- +-1000,,special_chars +1000,hello,world +2000,foo,bar +3000,test,data + +-- !csv_compress_zstd -- +-1000,,special_chars +1000,hello,world +2000,foo,bar +3000,test,data + +-- !csv_compress_snappy -- +-1000,,special_chars +1000,hello,world +2000,foo,bar +3000,test,data + +-- !overwrite_first -- +-1000, +1000,hello +2000,foo +3000,test +\\N,\\N + +-- !overwrite_second -- +1000,hello +2000,foo + +-- !append_first -- +1000 hello + +-- !append_second -- +1000 foo + +-- !const_expr -- +1,hello,3.14,2024-01-01 + +-- !where_groupby -- +0,2,1000 +1,2,4000 + +-- !join_query -- +1000,hello,label_a +2000,foo,label_b +3000,test,label_c + +-- !subquery -- +-1000,,special_chars +1000,hello,world +2000,foo,bar +3000,test,data + +-- !type_cast -- +-1000,-1.100000023841858,2020-02-29 +1000,1.100000023841858,2024-01-01 +2000,3.299999952316284,2024-06-15 +3000,5.5,2024-12-31 + +-- !union_query -- +1000,hello +2000,foo + diff --git a/regression-test/suites/external_table_p0/tvf/insert/test_insert_into_local_tvf.groovy b/regression-test/suites/external_table_p0/tvf/insert/test_insert_into_local_tvf.groovy new file mode 100644 index 00000000000000..95042345c12430 --- /dev/null +++ b/regression-test/suites/external_table_p0/tvf/insert/test_insert_into_local_tvf.groovy @@ -0,0 +1,617 @@ +// 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. + +suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { + + List> backends = sql """ show backends """ + assertTrue(backends.size() > 0) + def be_id = backends[0][0] + def be_host = backends[0][1] + def basePath = "/tmp/test_insert_into_local_tvf" + + // Clean and create basePath on the BE node + sshExec("root", be_host, "rm -rf ${basePath}", false) + sshExec("root", be_host, "mkdir -p ${basePath}") + sshExec("root", be_host, "chmod 777 ${basePath}") + + // ============ Source tables ============ + + sql """ DROP TABLE IF EXISTS insert_tvf_test_src """ + sql """ + CREATE TABLE IF NOT EXISTS insert_tvf_test_src ( + c_bool BOOLEAN, + c_tinyint TINYINT, + c_smallint SMALLINT, + c_int INT, + c_bigint BIGINT, + c_float FLOAT, + c_double DOUBLE, + c_decimal DECIMAL(10,2), + c_date DATE, + c_datetime DATETIME, + c_varchar VARCHAR(100), + c_string STRING + ) DISTRIBUTED BY HASH(c_int) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO insert_tvf_test_src VALUES + (true, 1, 100, 1000, 100000, 1.1, 2.2, 123.45, '2024-01-01', '2024-01-01 10:00:00', 'hello', 'world'), + (false, 2, 200, 2000, 200000, 3.3, 4.4, 678.90, '2024-06-15', '2024-06-15 12:30:00', 'foo', 'bar'), + (true, 3, 300, 3000, 300000, 5.5, 6.6, 999.99, '2024-12-31', '2024-12-31 23:59:59', 'test', 'data'), + (NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL), + (false, -1, -100, -1000, -100000, -1.1, -2.2, -123.45,'2020-02-29', '2020-02-29 00:00:00', '', 'special_chars'); + """ + + sql """ DROP TABLE IF EXISTS insert_tvf_complex_src """ + sql """ + CREATE TABLE IF NOT EXISTS insert_tvf_complex_src ( + c_int INT, + c_array ARRAY, + c_map MAP, + c_struct STRUCT + ) DISTRIBUTED BY HASH(c_int) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO insert_tvf_complex_src VALUES + (1, [1, 2, 3], {'a': 1, 'b': 2}, {1, 'hello'}), + (2, [4, 5], {'x': 10}, {2, 'world'}), + (3, [], {}, {3, ''}), + (4, NULL, NULL, NULL); + """ + + sql """ DROP TABLE IF EXISTS insert_tvf_join_src """ + sql """ + CREATE TABLE IF NOT EXISTS insert_tvf_join_src ( + c_int INT, + c_label STRING + ) DISTRIBUTED BY HASH(c_int) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ INSERT INTO insert_tvf_join_src VALUES (1000, 'label_a'), (2000, 'label_b'), (3000, 'label_c'); """ + + // ============ 1. CSV basic types ============ + + sql """ + INSERT INTO local( + "file_path" = "${basePath}/basic_csv.csv", + "backend_id" = "${be_id}", + "format" = "csv" + ) SELECT * FROM insert_tvf_test_src ORDER BY c_int; + """ + + order_qt_csv_basic_types """ + SELECT * FROM local( + "file_path" = "${basePath}/basic_csv.csv", + "backend_id" = "${be_id}", + "format" = "csv" + ) ORDER BY c1; + """ + + // ============ 2. Parquet basic types ============ + + sql """ + INSERT INTO local( + "file_path" = "${basePath}/basic_parquet.parquet", + "backend_id" = "${be_id}", + "format" = "parquet" + ) SELECT * FROM insert_tvf_test_src ORDER BY c_int; + """ + + order_qt_parquet_basic_types """ + SELECT * FROM local( + "file_path" = "${basePath}/basic_parquet.parquet", + "backend_id" = "${be_id}", + "format" = "parquet" + ) ORDER BY c_int; + """ + + // ============ 3. ORC basic types ============ + + sql """ + INSERT INTO local( + "file_path" = "${basePath}/basic_orc.orc", + "backend_id" = "${be_id}", + "format" = "orc" + ) SELECT * FROM insert_tvf_test_src ORDER BY c_int; + """ + + order_qt_orc_basic_types """ + SELECT * FROM local( + "file_path" = "${basePath}/basic_orc.orc", + "backend_id" = "${be_id}", + "format" = "orc" + ) ORDER BY c_int; + """ + + // ============ 4. Parquet complex types ============ + + sql """ + INSERT INTO local( + "file_path" = "${basePath}/complex_parquet.parquet", + "backend_id" = "${be_id}", + "format" = "parquet" + ) SELECT * FROM insert_tvf_complex_src ORDER BY c_int; + """ + + order_qt_parquet_complex_types """ + SELECT * FROM local( + "file_path" = "${basePath}/complex_parquet.parquet", + "backend_id" = "${be_id}", + "format" = "parquet" + ) ORDER BY c_int; + """ + + // ============ 5. ORC complex types ============ + + sql """ + INSERT INTO local( + "file_path" = "${basePath}/complex_orc.orc", + "backend_id" = "${be_id}", + "format" = "orc" + ) SELECT * FROM insert_tvf_complex_src ORDER BY c_int; + """ + + order_qt_orc_complex_types """ + SELECT * FROM local( + "file_path" = "${basePath}/complex_orc.orc", + "backend_id" = "${be_id}", + "format" = "orc" + ) ORDER BY c_int; + """ + + // ============ 6. CSV separator: comma ============ + + sql """ + INSERT INTO local( + "file_path" = "${basePath}/sep_comma.csv", + "backend_id" = "${be_id}", + "format" = "csv", + "column_separator" = "," + ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; + """ + + order_qt_csv_sep_comma """ + SELECT * FROM local( + "file_path" = "${basePath}/sep_comma.csv", + "backend_id" = "${be_id}", + "format" = "csv", + "column_separator" = "," + ) ORDER BY c1; + """ + + // ============ 7. CSV separator: tab ============ + + sql """ + INSERT INTO local( + "file_path" = "${basePath}/sep_tab.csv", + "backend_id" = "${be_id}", + "format" = "csv", + "column_separator" = "\t" + ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; + """ + + order_qt_csv_sep_tab """ + SELECT * FROM local( + "file_path" = "${basePath}/sep_tab.csv", + "backend_id" = "${be_id}", + "format" = "csv", + "column_separator" = "\t" + ) ORDER BY c1; + """ + + // ============ 8. CSV separator: pipe ============ + + sql """ + INSERT INTO local( + "file_path" = "${basePath}/sep_pipe.csv", + "backend_id" = "${be_id}", + "format" = "csv", + "column_separator" = "|" + ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; + """ + + order_qt_csv_sep_pipe """ + SELECT * FROM local( + "file_path" = "${basePath}/sep_pipe.csv", + "backend_id" = "${be_id}", + "format" = "csv", + "column_separator" = "|" + ) ORDER BY c1; + """ + + // ============ 9. CSV separator: multi-char ============ + + sql """ + INSERT INTO local( + "file_path" = "${basePath}/sep_multi.csv", + "backend_id" = "${be_id}", + "format" = "csv", + "column_separator" = ";;" + ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; + """ + + order_qt_csv_sep_multi """ + SELECT * FROM local( + "file_path" = "${basePath}/sep_multi.csv", + "backend_id" = "${be_id}", + "format" = "csv", + "column_separator" = ";;" + ) ORDER BY c1; + """ + + // ============ 10. CSV line delimiter: CRLF ============ + + sql """ + INSERT INTO local( + "file_path" = "${basePath}/line_crlf.csv", + "backend_id" = "${be_id}", + "format" = "csv", + "line_delimiter" = "\r\n" + ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; + """ + + order_qt_csv_line_crlf """ + SELECT * FROM local( + "file_path" = "${basePath}/line_crlf.csv", + "backend_id" = "${be_id}", + "format" = "csv", + "line_delimiter" = "\r\n" + ) ORDER BY c1; + """ + + // ============ 11. CSV compress: gz ============ + + sql """ + INSERT INTO local( + "file_path" = "${basePath}/compress_gz.csv.gz", + "backend_id" = "${be_id}", + "format" = "csv", + "compression_type" = "gz" + ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; + """ + + order_qt_csv_compress_gz """ + SELECT * FROM local( + "file_path" = "${basePath}/compress_gz.csv.gz", + "backend_id" = "${be_id}", + "format" = "csv", + "compress_type" = "gz" + ) ORDER BY c1; + """ + + // ============ 12. CSV compress: zstd ============ + + sql """ + INSERT INTO local( + "file_path" = "${basePath}/compress_zstd.csv.zst", + "backend_id" = "${be_id}", + "format" = "csv", + "compression_type" = "zstd" + ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; + """ + + order_qt_csv_compress_zstd """ + SELECT * FROM local( + "file_path" = "${basePath}/compress_zstd.csv.zst", + "backend_id" = "${be_id}", + "format" = "csv", + "compress_type" = "zstd" + ) ORDER BY c1; + """ + + // ============ 13. CSV compress: lz4 ============ + + // TODO: lz4 read meet error: LZ4F_getFrameInfo error: ERROR_frameType_unknown + // sql """ + // INSERT INTO local( + // "file_path" = "${basePath}/compress_lz4.csv.lz4", + // "backend_id" = "${be_id}", + // "format" = "csv", + // "compression_type" = "lz4" + // ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; + // """ + + // order_qt_csv_compress_lz4 """ + // SELECT * FROM local( + // "file_path" = "${basePath}/compress_lz4.csv.lz4", + // "backend_id" = "${be_id}", + // "format" = "csv", + // "compress_type" = "lz4" + // ) ORDER BY c1; + // """ + + // ============ 14. CSV compress: snappy ============ + + sql """ + INSERT INTO local( + "file_path" = "${basePath}/compress_snappy.csv.snappy", + "backend_id" = "${be_id}", + "format" = "csv", + "compression_type" = "snappy" + ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; + """ + + order_qt_csv_compress_snappy """ + SELECT * FROM local( + "file_path" = "${basePath}/compress_snappy.csv.snappy", + "backend_id" = "${be_id}", + "format" = "csv", + "compress_type" = "snappyblock" + ) ORDER BY c1; + """ + + // ============ 15. Overwrite mode (delete_existing_files=true) ============ + + // First write: 5 rows + sql """ + INSERT INTO local( + "file_path" = "${basePath}/overwrite.csv", + "backend_id" = "${be_id}", + "format" = "csv" + ) SELECT c_int, c_varchar FROM insert_tvf_test_src ORDER BY c_int; + """ + + order_qt_overwrite_first """ + SELECT * FROM local( + "file_path" = "${basePath}/overwrite.csv", + "backend_id" = "${be_id}", + "format" = "csv" + ) ORDER BY c1; + """ + + // Second write: 2 rows with overwrite + sql """ + INSERT INTO local( + "file_path" = "${basePath}/overwrite.csv", + "backend_id" = "${be_id}", + "format" = "csv", + "delete_existing_files" = "true" + ) SELECT c_int, c_varchar FROM insert_tvf_test_src WHERE c_int > 0 ORDER BY c_int LIMIT 2; + """ + + order_qt_overwrite_second """ + SELECT * FROM local( + "file_path" = "${basePath}/overwrite.csv", + "backend_id" = "${be_id}", + "format" = "csv" + ) ORDER BY c1; + """ + + // ============ 16. Append mode (delete_existing_files=false) ============ + + // First write + sql """ + INSERT INTO local( + "file_path" = "${basePath}/append.parquet", + "backend_id" = "${be_id}", + "format" = "parquet", + "delete_existing_files" = "false" + ) SELECT c_int, c_varchar FROM insert_tvf_test_src WHERE c_int = 1000; + """ + + order_qt_append_first """ + SELECT * FROM local( + "file_path" = "${basePath}/append.parquet", + "backend_id" = "${be_id}", + "format" = "parquet" + ) ORDER BY c_int; + """ + + // Second write (append) + sql """ + INSERT INTO local( + "file_path" = "${basePath}/append.parquet", + "backend_id" = "${be_id}", + "format" = "parquet", + "delete_existing_files" = "false" + ) SELECT c_int, c_varchar FROM insert_tvf_test_src WHERE c_int = 2000; + """ + + order_qt_append_second """ + SELECT * FROM local( + "file_path" = "${basePath}/append.parquet", + "backend_id" = "${be_id}", + "format" = "parquet" + ) ORDER BY c_int; + """ + + // ============ 17. Complex SELECT: constant expressions ============ + + sql """ + INSERT INTO local( + "file_path" = "${basePath}/const_expr.csv", + "backend_id" = "${be_id}", + "format" = "csv" + ) SELECT 1, 'hello', 3.14, CAST('2024-01-01' AS DATE); + """ + + order_qt_const_expr """ + SELECT * FROM local( + "file_path" = "${basePath}/const_expr.csv", + "backend_id" = "${be_id}", + "format" = "csv" + ) ORDER BY c1; + """ + + // ============ 18. Complex SELECT: WHERE + GROUP BY ============ + + sql """ + INSERT INTO local( + "file_path" = "${basePath}/where_groupby.csv", + "backend_id" = "${be_id}", + "format" = "csv" + ) SELECT c_bool, COUNT(*), SUM(c_int) FROM insert_tvf_test_src WHERE c_int IS NOT NULL GROUP BY c_bool ORDER BY c_bool; + """ + + order_qt_where_groupby """ + SELECT * FROM local( + "file_path" = "${basePath}/where_groupby.csv", + "backend_id" = "${be_id}", + "format" = "csv" + ) ORDER BY c1; + """ + + // ============ 19. Complex SELECT: JOIN ============ + + sql """ + INSERT INTO local( + "file_path" = "${basePath}/join_query.csv", + "backend_id" = "${be_id}", + "format" = "csv" + ) SELECT a.c_int, a.c_varchar, b.c_label + FROM insert_tvf_test_src a INNER JOIN insert_tvf_join_src b ON a.c_int = b.c_int + ORDER BY a.c_int; + """ + + order_qt_join_query """ + SELECT * FROM local( + "file_path" = "${basePath}/join_query.csv", + "backend_id" = "${be_id}", + "format" = "csv" + ) ORDER BY c1; + """ + + // ============ 20. Complex SELECT: subquery ============ + + sql """ + INSERT INTO local( + "file_path" = "${basePath}/subquery.csv", + "backend_id" = "${be_id}", + "format" = "csv" + ) SELECT * FROM (SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int) sub; + """ + + order_qt_subquery """ + SELECT * FROM local( + "file_path" = "${basePath}/subquery.csv", + "backend_id" = "${be_id}", + "format" = "csv" + ) ORDER BY c1; + """ + + // ============ 21. Complex SELECT: type cast ============ + + sql """ + INSERT INTO local( + "file_path" = "${basePath}/type_cast.csv", + "backend_id" = "${be_id}", + "format" = "csv" + ) SELECT CAST(c_int AS BIGINT), CAST(c_float AS DOUBLE), CAST(c_date AS STRING) + FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; + """ + + order_qt_type_cast """ + SELECT * FROM local( + "file_path" = "${basePath}/type_cast.csv", + "backend_id" = "${be_id}", + "format" = "csv" + ) ORDER BY c1; + """ + + // ============ 22. Complex SELECT: UNION ALL ============ + + sql """ + INSERT INTO local( + "file_path" = "${basePath}/union_query.csv", + "backend_id" = "${be_id}", + "format" = "csv" + ) SELECT c_int, c_varchar FROM insert_tvf_test_src WHERE c_int = 1000 + UNION ALL + SELECT c_int, c_varchar FROM insert_tvf_test_src WHERE c_int = 2000; + """ + + order_qt_union_query """ + SELECT * FROM local( + "file_path" = "${basePath}/union_query.csv", + "backend_id" = "${be_id}", + "format" = "csv" + ) ORDER BY c1; + """ + + // ============ 23. Error: missing file_path ============ + + test { + sql """ + INSERT INTO local( + "backend_id" = "${be_id}", + "format" = "csv" + ) SELECT 1; + """ + exception "file_path" + } + + // ============ 24. Error: missing format ============ + + test { + sql """ + INSERT INTO local( + "file_path" = "${basePath}/err.csv", + "backend_id" = "${be_id}" + ) SELECT 1; + """ + exception "format" + } + + // ============ 25. Error: missing backend_id for local ============ + + test { + sql """ + INSERT INTO local( + "file_path" = "${basePath}/err.csv", + "format" = "csv" + ) SELECT 1; + """ + exception "backend_id" + } + + // ============ 26. Error: unsupported TVF name ============ + + test { + sql """ + INSERT INTO unknown_tvf( + "file_path" = "/tmp/err.csv", + "format" = "csv" + ) SELECT 1; + """ + exception "INSERT INTO TVF only supports" + } + + // ============ 27. Error: unsupported format ============ + + test { + sql """ + INSERT INTO local( + "file_path" = "${basePath}/err.json", + "backend_id" = "${be_id}", + "format" = "json" + ) SELECT 1; + """ + exception "Unsupported TVF sink format" + } + + // ============ Cleanup ============ + + sshExec("root", be_host, "rm -rf ${basePath}", false) + sql """ DROP TABLE IF EXISTS insert_tvf_test_src """ + sql """ DROP TABLE IF EXISTS insert_tvf_complex_src """ + sql """ DROP TABLE IF EXISTS insert_tvf_join_src """ +} From 4f9b9a8e99a8737035b22f403ea987b902b8b9f2 Mon Sep 17 00:00:00 2001 From: morningman Date: Fri, 13 Feb 2026 00:47:04 +0800 Subject: [PATCH 04/13] s3 test --- .../apache/doris/planner/TVFTableSink.java | 126 +++-- .../tvf/insert/test_insert_into_local_tvf.out | 94 ++-- .../tvf/insert/test_insert_into_s3_tvf.out | 133 +++++ .../insert/test_insert_into_local_tvf.groovy | 38 +- .../tvf/insert/test_insert_into_s3_tvf.groovy | 527 ++++++++++++++++++ 5 files changed, 798 insertions(+), 120 deletions(-) create mode 100644 regression-test/data/external_table_p0/tvf/insert/test_insert_into_s3_tvf.out create mode 100644 regression-test/suites/external_table_p0/tvf/insert/test_insert_into_s3_tvf.groovy diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/TVFTableSink.java b/fe/fe-core/src/main/java/org/apache/doris/planner/TVFTableSink.java index a99e8a1eb603c3..205a113a3064df 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/TVFTableSink.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/TVFTableSink.java @@ -19,15 +19,21 @@ import org.apache.doris.catalog.Column; import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.util.Util; +import org.apache.doris.datasource.property.fileformat.CsvFileFormatProperties; +import org.apache.doris.datasource.property.fileformat.FileFormatProperties; +import org.apache.doris.datasource.property.fileformat.OrcFileFormatProperties; +import org.apache.doris.datasource.property.storage.StorageProperties; import org.apache.doris.thrift.TColumn; import org.apache.doris.thrift.TDataSink; import org.apache.doris.thrift.TDataSinkType; import org.apache.doris.thrift.TExplainLevel; -import org.apache.doris.thrift.TFileCompressType; import org.apache.doris.thrift.TFileFormatType; import org.apache.doris.thrift.TFileType; import org.apache.doris.thrift.TTVFTableSink; +import com.google.common.collect.Maps; + import java.util.ArrayList; import java.util.List; import java.util.Map; @@ -35,6 +41,9 @@ /** * TVFTableSink is used for INSERT INTO tvf_name(properties) SELECT ... * It writes query results to files via TVF (local/s3/hdfs). + * + * Property parsing reuses the same StorageProperties and FileFormatProperties + * infrastructure as the read-side TVF (SELECT * FROM s3/hdfs/local(...)). */ public class TVFTableSink extends DataSink { private final PlanNodeId exchNodeId; @@ -54,20 +63,54 @@ public void bindDataSink() throws AnalysisException { TTVFTableSink tSink = new TTVFTableSink(); tSink.setTvfName(tvfName); - String filePath = properties.get("file_path"); - tSink.setFilePath(filePath); + // --- 1. Parse file format properties (reuse read-side FileFormatProperties) --- + // Make a mutable copy; FileFormatProperties.analyzeFileFormatProperties removes consumed keys. + Map propsCopy = Maps.newTreeMap(String.CASE_INSENSITIVE_ORDER); + propsCopy.putAll(properties); + + String formatStr = propsCopy.getOrDefault("format", "csv").toLowerCase(); + propsCopy.remove("format"); + + // Also consume "compression_type" as alias for "compress_type" (write-side convention) + if (propsCopy.containsKey("compression_type") && !propsCopy.containsKey("compress_type")) { + propsCopy.put("compress_type", propsCopy.remove("compression_type")); + } + + FileFormatProperties fileFormatProps = FileFormatProperties.createFileFormatProperties(formatStr); + fileFormatProps.analyzeFileFormatProperties(propsCopy, true); - // Set file format - String format = properties.getOrDefault("format", "csv").toLowerCase(); - TFileFormatType formatType = getFormatType(format); + TFileFormatType formatType = fileFormatProps.getFileFormatType(); + if (!Util.isCsvFormat(formatType) && formatType != TFileFormatType.FORMAT_PARQUET + && formatType != TFileFormatType.FORMAT_ORC) { + throw new AnalysisException("Unsupported format: " + formatType.name()); + } tSink.setFileFormat(formatType); // Set file type based on TVF name TFileType fileType = getFileType(tvfName); tSink.setFileType(fileType); - // Set all properties for BE to access - tSink.setProperties(properties); + // --- 2. Parse storage/connection properties (reuse read-side StorageProperties) --- + Map backendConnectProps; + if (tvfName.equals("local")) { + // Local TVF: pass properties as-is (same as LocalProperties.getBackendConfigProperties) + backendConnectProps = new java.util.HashMap<>(propsCopy); + } else { + // S3/HDFS: use StorageProperties to normalize connection property keys + // (e.g. "s3.endpoint" -> "AWS_ENDPOINT", "hadoop.username" -> hadoop config) + try { + StorageProperties storageProps = StorageProperties.createPrimary(propsCopy); + backendConnectProps = storageProps.getBackendConfigProperties(); + } catch (Exception e) { + throw new AnalysisException("Failed to parse storage properties: " + e.getMessage(), e); + } + } + + String filePath = properties.get("file_path"); + tSink.setFilePath(filePath); + + // Set normalized properties for BE + tSink.setProperties(backendConnectProps); // Set columns List tColumns = new ArrayList<>(); @@ -76,59 +119,44 @@ public void bindDataSink() throws AnalysisException { } tSink.setColumns(tColumns); - // Set column separator - String columnSeparator = properties.getOrDefault("column_separator", ","); - tSink.setColumnSeparator(columnSeparator); - - // Set line delimiter - String lineDelimiter = properties.getOrDefault("line_delimiter", "\n"); - tSink.setLineDelimiter(lineDelimiter); + // --- 3. Set format-specific sink options --- + if (fileFormatProps instanceof CsvFileFormatProperties) { + CsvFileFormatProperties csvProps = (CsvFileFormatProperties) fileFormatProps; + csvProps.checkSupportedCompressionType(true); + tSink.setColumnSeparator(csvProps.getColumnSeparator()); + tSink.setLineDelimiter(csvProps.getLineDelimiter()); + tSink.setCompressionType(csvProps.getCompressionType()); + } else if (fileFormatProps instanceof OrcFileFormatProperties) { + tSink.setCompressionType(((OrcFileFormatProperties) fileFormatProps).getOrcCompressionType()); + } + // Parquet compression is handled by BE via parquet writer options - // Set max file size + // --- 4. Set sink-specific options --- + // Max file size String maxFileSizeStr = properties.get("max_file_size"); if (maxFileSizeStr != null) { tSink.setMaxFileSizeBytes(Long.parseLong(maxFileSizeStr)); } - // Set delete existing files flag + // Delete existing files flag String deleteExisting = properties.getOrDefault("delete_existing_files", "true"); tSink.setDeleteExistingFiles(Boolean.parseBoolean(deleteExisting)); - // Set compression type - String compression = properties.get("compression_type"); - if (compression != null) { - tSink.setCompressionType(getCompressType(compression)); - } - - // Set backend id for local TVF + // Backend id for local TVF String backendIdStr = properties.get("backend_id"); if (backendIdStr != null) { tSink.setBackendId(Long.parseLong(backendIdStr)); } - // Set hadoop config for hdfs/s3 - if (tvfName.equals("hdfs") || tvfName.equals("s3")) { - tSink.setHadoopConfig(properties); + // Set hadoop config for hdfs/s3 (BE uses this for file writer creation) + if (!tvfName.equals("local")) { + tSink.setHadoopConfig(backendConnectProps); } tDataSink = new TDataSink(TDataSinkType.TVF_TABLE_SINK); tDataSink.setTvfTableSink(tSink); } - private TFileFormatType getFormatType(String format) throws AnalysisException { - switch (format) { - case "csv": - return TFileFormatType.FORMAT_CSV_PLAIN; - case "parquet": - return TFileFormatType.FORMAT_PARQUET; - case "orc": - return TFileFormatType.FORMAT_ORC; - default: - throw new AnalysisException("Unsupported TVF sink format: " + format - + ". Supported formats: csv, parquet, orc"); - } - } - private TFileType getFileType(String tvfName) throws AnalysisException { switch (tvfName.toLowerCase()) { case "local": @@ -142,22 +170,6 @@ private TFileType getFileType(String tvfName) throws AnalysisException { } } - private TFileCompressType getCompressType(String compressType) { - switch (compressType.toLowerCase()) { - case "snappy": - return TFileCompressType.SNAPPYBLOCK; - case "lz4": - return TFileCompressType.LZ4BLOCK; - case "gzip": - case "gz": - return TFileCompressType.GZ; - case "zstd": - return TFileCompressType.ZSTD; - default: - return TFileCompressType.PLAIN; - } - } - @Override public String getExplainString(String prefix, TExplainLevel explainLevel) { StringBuilder strBuilder = new StringBuilder(); diff --git a/regression-test/data/external_table_p0/tvf/insert/test_insert_into_local_tvf.out b/regression-test/data/external_table_p0/tvf/insert/test_insert_into_local_tvf.out index 2dd0e051ab5b73..5058c9346c59c6 100644 --- a/regression-test/data/external_table_p0/tvf/insert/test_insert_into_local_tvf.out +++ b/regression-test/data/external_table_p0/tvf/insert/test_insert_into_local_tvf.out @@ -1,10 +1,10 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !csv_basic_types -- -0,-1,-100,-1000,-100000,-1.1,-2.2,-123.45,2020-02-29,2020-02-29 00:00:00,,special_chars -0,2,200,2000,200000,3.3,4.4,678.90,2024-06-15,2024-06-15 12:30:00,foo,bar -1,1,100,1000,100000,1.1,2.2,123.45,2024-01-01,2024-01-01 10:00:00,hello,world -1,3,300,3000,300000,5.5,6.6,999.99,2024-12-31,2024-12-31 23:59:59,test,data -\\N,\\N,\\N,\\N,\\N,\\N,\\N,\\N,\\N,\\N,\\N,\\N +\N \N \N \N \N \N \N \N \N \N \N \N +0 -1 -100 -1000 -100000 -1.1 -2.2 -123.45 2020-02-29 2020-02-29 00:00:00 special_chars +0 2 200 2000 200000 3.3 4.4 678.90 2024-06-15 2024-06-15 12:30:00 foo bar +1 1 100 1000 100000 1.1 2.2 123.45 2024-01-01 2024-01-01 10:00:00 hello world +1 3 300 3000 300000 5.5 6.6 999.99 2024-12-31 2024-12-31 23:59:59 test data -- !parquet_basic_types -- \N \N \N \N \N \N \N \N \N \N \N \N @@ -57,39 +57,45 @@ true 3 300 3000 300000 5.5 6.6 999.99 2024-12-31 2024-12-31T23:59:59 test data 3000 test data -- !csv_line_crlf -- --1000,,special_chars -1000,hello,world -2000,foo,bar -3000,test,data +-1000 special_chars +1000 hello world +2000 foo bar +3000 test data -- !csv_compress_gz -- --1000,,special_chars -1000,hello,world -2000,foo,bar -3000,test,data +-1000 special_chars +1000 hello world +2000 foo bar +3000 test data -- !csv_compress_zstd -- --1000,,special_chars -1000,hello,world -2000,foo,bar -3000,test,data +-1000 special_chars +1000 hello world +2000 foo bar +3000 test data + +-- !csv_compress_lz4 -- +-1000 special_chars +1000 hello world +2000 foo bar +3000 test data -- !csv_compress_snappy -- --1000,,special_chars -1000,hello,world -2000,foo,bar -3000,test,data +-1000 special_chars +1000 hello world +2000 foo bar +3000 test data -- !overwrite_first -- --1000, -1000,hello -2000,foo -3000,test -\\N,\\N +\N \N +-1000 +1000 hello +2000 foo +3000 test -- !overwrite_second -- -1000,hello -2000,foo +1000 hello +2000 foo -- !append_first -- 1000 hello @@ -98,30 +104,30 @@ true 3 300 3000 300000 5.5 6.6 999.99 2024-12-31 2024-12-31T23:59:59 test data 1000 foo -- !const_expr -- -1,hello,3.14,2024-01-01 +1 hello 3.14 2024-01-01 -- !where_groupby -- -0,2,1000 -1,2,4000 +0 2 1000 +1 2 4000 -- !join_query -- -1000,hello,label_a -2000,foo,label_b -3000,test,label_c +1000 hello label_a +2000 foo label_b +3000 test label_c -- !subquery -- --1000,,special_chars -1000,hello,world -2000,foo,bar -3000,test,data +-1000 special_chars +1000 hello world +2000 foo bar +3000 test data -- !type_cast -- --1000,-1.100000023841858,2020-02-29 -1000,1.100000023841858,2024-01-01 -2000,3.299999952316284,2024-06-15 -3000,5.5,2024-12-31 +-1000 -1.100000023841858 2020-02-29 +1000 1.100000023841858 2024-01-01 +2000 3.299999952316284 2024-06-15 +3000 5.5 2024-12-31 -- !union_query -- -1000,hello -2000,foo +1000 hello +2000 foo diff --git a/regression-test/data/external_table_p0/tvf/insert/test_insert_into_s3_tvf.out b/regression-test/data/external_table_p0/tvf/insert/test_insert_into_s3_tvf.out new file mode 100644 index 00000000000000..60632c73288b1a --- /dev/null +++ b/regression-test/data/external_table_p0/tvf/insert/test_insert_into_s3_tvf.out @@ -0,0 +1,133 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !s3_csv_basic_types -- +\N \N \N \N \N \N \N \N \N \N \N \N +0 -1 -100 -1000 -100000 -1.1 -2.2 -123.45 2020-02-29 2020-02-29 00:00:00 special_chars +0 2 200 2000 200000 3.3 4.4 678.90 2024-06-15 2024-06-15 12:30:00 foo bar +1 1 100 1000 100000 1.1 2.2 123.45 2024-01-01 2024-01-01 10:00:00 hello world +1 3 300 3000 300000 5.5 6.6 999.99 2024-12-31 2024-12-31 23:59:59 test data + +-- !s3_parquet_basic_types -- +\N \N \N \N \N \N \N \N \N \N \N \N +false -1 -100 -1000 -100000 -1.1 -2.2 -123.45 2020-02-29 2020-02-29T00:00 special_chars +false 2 200 2000 200000 3.3 4.4 678.90 2024-06-15 2024-06-15T12:30 foo bar +true 1 100 1000 100000 1.1 2.2 123.45 2024-01-01 2024-01-01T10:00 hello world +true 3 300 3000 300000 5.5 6.6 999.99 2024-12-31 2024-12-31T23:59:59 test data + +-- !s3_orc_basic_types -- +\N \N \N \N \N \N \N \N \N \N \N \N +false -1 -100 -1000 -100000 -1.1 -2.2 -123.45 2020-02-29 2020-02-29T00:00 special_chars +false 2 200 2000 200000 3.3 4.4 678.90 2024-06-15 2024-06-15T12:30 foo bar +true 1 100 1000 100000 1.1 2.2 123.45 2024-01-01 2024-01-01T10:00 hello world +true 3 300 3000 300000 5.5 6.6 999.99 2024-12-31 2024-12-31T23:59:59 test data + +-- !s3_parquet_complex_types -- +1 [1, 2, 3] {"a":1, "b":2} {"f1":1, "f2":"hello"} +2 [4, 5] {"x":10} {"f1":2, "f2":"world"} +3 [] {} {"f1":3, "f2":""} +4 \N \N \N + +-- !s3_orc_complex_types -- +1 [1, 2, 3] {"a":1, "b":2} {"f1":1, "f2":"hello"} +2 [4, 5] {"x":10} {"f1":2, "f2":"world"} +3 [] {} {"f1":3, "f2":""} +4 \N \N \N + +-- !s3_csv_sep_comma -- +-1000 special_chars +1000 hello world +2000 foo bar +3000 test data + +-- !s3_csv_sep_tab -- +-1000 special_chars +1000 hello world +2000 foo bar +3000 test data + +-- !s3_csv_sep_pipe -- +-1000 special_chars +1000 hello world +2000 foo bar +3000 test data + +-- !s3_csv_sep_multi -- +-1000 special_chars +1000 hello world +2000 foo bar +3000 test data + +-- !s3_csv_line_crlf -- +-1000 special_chars +1000 hello world +2000 foo bar +3000 test data + +-- !s3_csv_compress_gz -- +-1000 special_chars +1000 hello world +2000 foo bar +3000 test data + +-- !s3_csv_compress_zstd -- +-1000 special_chars +1000 hello world +2000 foo bar +3000 test data + +-- !s3_csv_compress_lz4 -- +-1000 special_chars +1000 hello world +2000 foo bar +3000 test data + +-- !s3_csv_compress_snappy -- +-1000 special_chars +1000 hello world +2000 foo bar +3000 test data + +-- !s3_overwrite_first -- +\N \N +-1000 +1000 hello +2000 foo +3000 test + +-- !s3_overwrite_second -- +1000 hello +2000 foo + +-- !s3_append_first -- +1000 hello + +-- !s3_append_second -- +1000 hello + +-- !s3_const_expr -- +1 hello 3.14 2024-01-01 + +-- !s3_where_groupby -- +0 2 1000 +1 2 4000 + +-- !s3_join_query -- +1000 hello label_a +2000 foo label_b +3000 test label_c + +-- !s3_subquery -- +-1000 special_chars +1000 hello world +2000 foo bar +3000 test data + +-- !s3_type_cast -- +-1000 -1.100000023841858 2020-02-29 +1000 1.100000023841858 2024-01-01 +2000 3.299999952316284 2024-06-15 +3000 5.5 2024-12-31 + +-- !s3_union_query -- +1000 hello +2000 foo + diff --git a/regression-test/suites/external_table_p0/tvf/insert/test_insert_into_local_tvf.groovy b/regression-test/suites/external_table_p0/tvf/insert/test_insert_into_local_tvf.groovy index 95042345c12430..375310c9cfb342 100644 --- a/regression-test/suites/external_table_p0/tvf/insert/test_insert_into_local_tvf.groovy +++ b/regression-test/suites/external_table_p0/tvf/insert/test_insert_into_local_tvf.groovy @@ -321,23 +321,23 @@ suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { // ============ 13. CSV compress: lz4 ============ // TODO: lz4 read meet error: LZ4F_getFrameInfo error: ERROR_frameType_unknown - // sql """ - // INSERT INTO local( - // "file_path" = "${basePath}/compress_lz4.csv.lz4", - // "backend_id" = "${be_id}", - // "format" = "csv", - // "compression_type" = "lz4" - // ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; - // """ - - // order_qt_csv_compress_lz4 """ - // SELECT * FROM local( - // "file_path" = "${basePath}/compress_lz4.csv.lz4", - // "backend_id" = "${be_id}", - // "format" = "csv", - // "compress_type" = "lz4" - // ) ORDER BY c1; - // """ + sql """ + INSERT INTO local( + "file_path" = "${basePath}/compress_lz4.csv.lz4", + "backend_id" = "${be_id}", + "format" = "csv", + "compression_type" = "lz4block" + ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; + """ + + order_qt_csv_compress_lz4 """ + SELECT * FROM local( + "file_path" = "${basePath}/compress_lz4.csv.lz4", + "backend_id" = "${be_id}", + "format" = "csv", + "compress_type" = "lz4block" + ) ORDER BY c1; + """ // ============ 14. CSV compress: snappy ============ @@ -346,7 +346,7 @@ suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { "file_path" = "${basePath}/compress_snappy.csv.snappy", "backend_id" = "${be_id}", "format" = "csv", - "compression_type" = "snappy" + "compression_type" = "snappyblock" ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; """ @@ -605,7 +605,7 @@ suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { "format" = "json" ) SELECT 1; """ - exception "Unsupported TVF sink format" + exception "Unsupported" } // ============ Cleanup ============ diff --git a/regression-test/suites/external_table_p0/tvf/insert/test_insert_into_s3_tvf.groovy b/regression-test/suites/external_table_p0/tvf/insert/test_insert_into_s3_tvf.groovy new file mode 100644 index 00000000000000..910a3edf6eb679 --- /dev/null +++ b/regression-test/suites/external_table_p0/tvf/insert/test_insert_into_s3_tvf.groovy @@ -0,0 +1,527 @@ +// 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. + +suite("test_insert_into_s3_tvf", "p0,external,external_docker") { + + String ak = getS3AK() + String sk = getS3SK() + String s3_endpoint = getS3Endpoint() + String region = getS3Region() + String bucket = context.config.otherConfigs.get("s3BucketName") + + if (ak == null || ak == "" || sk == null || sk == "") { + logger.info("S3 not configured, skip") + return + } + + def s3BasePath = "${bucket}/regression/insert_tvf_test" + + def s3WriteProps = { String path, String format -> + return """ + "file_path" = "s3://${s3BasePath}/${path}", + "format" = "${format}", + "s3.endpoint" = "${s3_endpoint}", + "s3.access_key" = "${ak}", + "s3.secret_key" = "${sk}", + "s3.region" = "${region}" + """ + } + + def s3ReadProps = { String path, String format -> + return """ + "uri" = "https://${bucket}.${s3_endpoint}/regression/insert_tvf_test/${path}", + "s3.access_key" = "${ak}", + "s3.secret_key" = "${sk}", + "format" = "${format}", + "region" = "${region}" + """ + } + + // ============ Source tables ============ + + sql """ DROP TABLE IF EXISTS insert_tvf_test_src """ + sql """ + CREATE TABLE IF NOT EXISTS insert_tvf_test_src ( + c_bool BOOLEAN, + c_tinyint TINYINT, + c_smallint SMALLINT, + c_int INT, + c_bigint BIGINT, + c_float FLOAT, + c_double DOUBLE, + c_decimal DECIMAL(10,2), + c_date DATE, + c_datetime DATETIME, + c_varchar VARCHAR(100), + c_string STRING + ) DISTRIBUTED BY HASH(c_int) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO insert_tvf_test_src VALUES + (true, 1, 100, 1000, 100000, 1.1, 2.2, 123.45, '2024-01-01', '2024-01-01 10:00:00', 'hello', 'world'), + (false, 2, 200, 2000, 200000, 3.3, 4.4, 678.90, '2024-06-15', '2024-06-15 12:30:00', 'foo', 'bar'), + (true, 3, 300, 3000, 300000, 5.5, 6.6, 999.99, '2024-12-31', '2024-12-31 23:59:59', 'test', 'data'), + (NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL), + (false, -1, -100, -1000, -100000, -1.1, -2.2, -123.45,'2020-02-29', '2020-02-29 00:00:00', '', 'special_chars'); + """ + + sql """ DROP TABLE IF EXISTS insert_tvf_complex_src """ + sql """ + CREATE TABLE IF NOT EXISTS insert_tvf_complex_src ( + c_int INT, + c_array ARRAY, + c_map MAP, + c_struct STRUCT + ) DISTRIBUTED BY HASH(c_int) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO insert_tvf_complex_src VALUES + (1, [1, 2, 3], {'a': 1, 'b': 2}, {1, 'hello'}), + (2, [4, 5], {'x': 10}, {2, 'world'}), + (3, [], {}, {3, ''}), + (4, NULL, NULL, NULL); + """ + + sql """ DROP TABLE IF EXISTS insert_tvf_join_src """ + sql """ + CREATE TABLE IF NOT EXISTS insert_tvf_join_src ( + c_int INT, + c_label STRING + ) DISTRIBUTED BY HASH(c_int) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ INSERT INTO insert_tvf_join_src VALUES (1000, 'label_a'), (2000, 'label_b'), (3000, 'label_c'); """ + + // ============ 1. S3 CSV basic types ============ + + sql """ + INSERT INTO s3( + ${s3WriteProps("basic_csv.csv", "csv")} + ) SELECT * FROM insert_tvf_test_src ORDER BY c_int; + """ + + order_qt_s3_csv_basic_types """ + SELECT * FROM s3( + ${s3ReadProps("basic_csv.csv", "csv")} + ) ORDER BY c1; + """ + + // ============ 2. S3 Parquet basic types ============ + + sql """ + INSERT INTO s3( + ${s3WriteProps("basic_parquet.parquet", "parquet")} + ) SELECT * FROM insert_tvf_test_src ORDER BY c_int; + """ + + order_qt_s3_parquet_basic_types """ + SELECT * FROM s3( + ${s3ReadProps("basic_parquet.parquet", "parquet")} + ) ORDER BY c_int; + """ + + // ============ 3. S3 ORC basic types ============ + + sql """ + INSERT INTO s3( + ${s3WriteProps("basic_orc.orc", "orc")} + ) SELECT * FROM insert_tvf_test_src ORDER BY c_int; + """ + + order_qt_s3_orc_basic_types """ + SELECT * FROM s3( + ${s3ReadProps("basic_orc.orc", "orc")} + ) ORDER BY c_int; + """ + + // ============ 4. S3 Parquet complex types ============ + + sql """ + INSERT INTO s3( + ${s3WriteProps("complex_parquet.parquet", "parquet")} + ) SELECT * FROM insert_tvf_complex_src ORDER BY c_int; + """ + + order_qt_s3_parquet_complex_types """ + SELECT * FROM s3( + ${s3ReadProps("complex_parquet.parquet", "parquet")} + ) ORDER BY c_int; + """ + + // ============ 5. S3 ORC complex types ============ + + sql """ + INSERT INTO s3( + ${s3WriteProps("complex_orc.orc", "orc")} + ) SELECT * FROM insert_tvf_complex_src ORDER BY c_int; + """ + + order_qt_s3_orc_complex_types """ + SELECT * FROM s3( + ${s3ReadProps("complex_orc.orc", "orc")} + ) ORDER BY c_int; + """ + + // ============ 6. S3 CSV separator: comma ============ + + sql """ + INSERT INTO s3( + ${s3WriteProps("sep_comma.csv", "csv")}, + "column_separator" = "," + ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; + """ + + order_qt_s3_csv_sep_comma """ + SELECT * FROM s3( + ${s3ReadProps("sep_comma.csv", "csv")}, + "column_separator" = "," + ) ORDER BY c1; + """ + + // ============ 7. S3 CSV separator: tab ============ + + sql """ + INSERT INTO s3( + ${s3WriteProps("sep_tab.csv", "csv")}, + "column_separator" = "\t" + ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; + """ + + order_qt_s3_csv_sep_tab """ + SELECT * FROM s3( + ${s3ReadProps("sep_tab.csv", "csv")}, + "column_separator" = "\t" + ) ORDER BY c1; + """ + + // ============ 8. S3 CSV separator: pipe ============ + + sql """ + INSERT INTO s3( + ${s3WriteProps("sep_pipe.csv", "csv")}, + "column_separator" = "|" + ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; + """ + + order_qt_s3_csv_sep_pipe """ + SELECT * FROM s3( + ${s3ReadProps("sep_pipe.csv", "csv")}, + "column_separator" = "|" + ) ORDER BY c1; + """ + + // ============ 9. S3 CSV separator: multi-char ============ + + sql """ + INSERT INTO s3( + ${s3WriteProps("sep_multi.csv", "csv")}, + "column_separator" = ";;" + ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; + """ + + order_qt_s3_csv_sep_multi """ + SELECT * FROM s3( + ${s3ReadProps("sep_multi.csv", "csv")}, + "column_separator" = ";;" + ) ORDER BY c1; + """ + + // ============ 10. S3 CSV line delimiter: CRLF ============ + + sql """ + INSERT INTO s3( + ${s3WriteProps("line_crlf.csv", "csv")}, + "line_delimiter" = "\r\n" + ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; + """ + + order_qt_s3_csv_line_crlf """ + SELECT * FROM s3( + ${s3ReadProps("line_crlf.csv", "csv")}, + "line_delimiter" = "\r\n" + ) ORDER BY c1; + """ + + // ============ 11. S3 CSV compress: gz ============ + + sql """ + INSERT INTO s3( + ${s3WriteProps("compress_gz.csv.gz", "csv")}, + "compression_type" = "gz" + ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; + """ + + order_qt_s3_csv_compress_gz """ + SELECT * FROM s3( + ${s3ReadProps("compress_gz.csv.gz", "csv")}, + "compress_type" = "gz" + ) ORDER BY c1; + """ + + // ============ 12. S3 CSV compress: zstd ============ + + sql """ + INSERT INTO s3( + ${s3WriteProps("compress_zstd.csv.zst", "csv")}, + "compression_type" = "zstd" + ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; + """ + + order_qt_s3_csv_compress_zstd """ + SELECT * FROM s3( + ${s3ReadProps("compress_zstd.csv.zst", "csv")}, + "compress_type" = "zstd" + ) ORDER BY c1; + """ + + // ============ 13. S3 CSV compress: lz4 ============ + + sql """ + INSERT INTO s3( + ${s3WriteProps("compress_lz4.csv.lz4", "csv")}, + "compression_type" = "lz4block" + ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; + """ + + order_qt_s3_csv_compress_lz4 """ + SELECT * FROM s3( + ${s3ReadProps("compress_lz4.csv.lz4", "csv")}, + "compress_type" = "lz4block" + ) ORDER BY c1; + """ + + // ============ 14. S3 CSV compress: snappy ============ + + sql """ + INSERT INTO s3( + ${s3WriteProps("compress_snappy.csv.snappy", "csv")}, + "compression_type" = "snappyblock" + ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; + """ + + order_qt_s3_csv_compress_snappy """ + SELECT * FROM s3( + ${s3ReadProps("compress_snappy.csv.snappy", "csv")}, + "compress_type" = "snappyblock" + ) ORDER BY c1; + """ + + // ============ 15. S3 Overwrite mode ============ + + // First write: 5 rows + sql """ + INSERT INTO s3( + ${s3WriteProps("overwrite.csv", "csv")} + ) SELECT c_int, c_varchar FROM insert_tvf_test_src ORDER BY c_int; + """ + + order_qt_s3_overwrite_first """ + SELECT * FROM s3( + ${s3ReadProps("overwrite.csv", "csv")} + ) ORDER BY c1; + """ + + // Second write: 2 rows with overwrite + sql """ + INSERT INTO s3( + ${s3WriteProps("overwrite.csv", "csv")}, + "delete_existing_files" = "true" + ) SELECT c_int, c_varchar FROM insert_tvf_test_src WHERE c_int > 0 ORDER BY c_int LIMIT 2; + """ + + order_qt_s3_overwrite_second """ + SELECT * FROM s3( + ${s3ReadProps("overwrite.csv", "csv")} + ) ORDER BY c1; + """ + + // ============ 16. S3 Append mode ============ + + // First write + sql """ + INSERT INTO s3( + ${s3WriteProps("append.parquet", "parquet")}, + "delete_existing_files" = "false" + ) SELECT c_int, c_varchar FROM insert_tvf_test_src WHERE c_int = 1000; + """ + + order_qt_s3_append_first """ + SELECT * FROM s3( + ${s3ReadProps("append.parquet", "parquet")} + ) ORDER BY c_int; + """ + + // Second write (append) + sql """ + INSERT INTO s3( + ${s3WriteProps("append.parquet", "parquet")}, + "delete_existing_files" = "false" + ) SELECT c_int, c_varchar FROM insert_tvf_test_src WHERE c_int = 2000; + """ + + order_qt_s3_append_second """ + SELECT * FROM s3( + ${s3ReadProps("append.parquet", "parquet")} + ) ORDER BY c_int; + """ + + // ============ 17. S3 Complex SELECT: constant expressions ============ + + sql """ + INSERT INTO s3( + ${s3WriteProps("const_expr.csv", "csv")} + ) SELECT 1, 'hello', 3.14, CAST('2024-01-01' AS DATE); + """ + + order_qt_s3_const_expr """ + SELECT * FROM s3( + ${s3ReadProps("const_expr.csv", "csv")} + ) ORDER BY c1; + """ + + // ============ 18. S3 Complex SELECT: WHERE + GROUP BY ============ + + sql """ + INSERT INTO s3( + ${s3WriteProps("where_groupby.csv", "csv")} + ) SELECT c_bool, COUNT(*), SUM(c_int) FROM insert_tvf_test_src WHERE c_int IS NOT NULL GROUP BY c_bool ORDER BY c_bool; + """ + + order_qt_s3_where_groupby """ + SELECT * FROM s3( + ${s3ReadProps("where_groupby.csv", "csv")} + ) ORDER BY c1; + """ + + // ============ 19. S3 Complex SELECT: JOIN ============ + + sql """ + INSERT INTO s3( + ${s3WriteProps("join_query.csv", "csv")} + ) SELECT a.c_int, a.c_varchar, b.c_label + FROM insert_tvf_test_src a INNER JOIN insert_tvf_join_src b ON a.c_int = b.c_int + ORDER BY a.c_int; + """ + + order_qt_s3_join_query """ + SELECT * FROM s3( + ${s3ReadProps("join_query.csv", "csv")} + ) ORDER BY c1; + """ + + // ============ 20. S3 Complex SELECT: subquery ============ + + sql """ + INSERT INTO s3( + ${s3WriteProps("subquery.csv", "csv")} + ) SELECT * FROM (SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int) sub; + """ + + order_qt_s3_subquery """ + SELECT * FROM s3( + ${s3ReadProps("subquery.csv", "csv")} + ) ORDER BY c1; + """ + + // ============ 21. S3 Complex SELECT: type cast ============ + + sql """ + INSERT INTO s3( + ${s3WriteProps("type_cast.csv", "csv")} + ) SELECT CAST(c_int AS BIGINT), CAST(c_float AS DOUBLE), CAST(c_date AS STRING) + FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; + """ + + order_qt_s3_type_cast """ + SELECT * FROM s3( + ${s3ReadProps("type_cast.csv", "csv")} + ) ORDER BY c1; + """ + + // ============ 22. S3 Complex SELECT: UNION ALL ============ + + sql """ + INSERT INTO s3( + ${s3WriteProps("union_query.csv", "csv")} + ) SELECT c_int, c_varchar FROM insert_tvf_test_src WHERE c_int = 1000 + UNION ALL + SELECT c_int, c_varchar FROM insert_tvf_test_src WHERE c_int = 2000; + """ + + order_qt_s3_union_query """ + SELECT * FROM s3( + ${s3ReadProps("union_query.csv", "csv")} + ) ORDER BY c1; + """ + + // ============ 23. Error: missing file_path ============ + + test { + sql """ + INSERT INTO s3( + "format" = "csv", + "s3.endpoint" = "${s3_endpoint}", + "s3.access_key" = "${ak}", + "s3.secret_key" = "${sk}", + "s3.region" = "${region}" + ) SELECT 1; + """ + exception "file_path" + } + + // ============ 24. Error: missing format ============ + + test { + sql """ + INSERT INTO s3( + "file_path" = "s3://${s3BasePath}/err.csv", + "s3.endpoint" = "${s3_endpoint}", + "s3.access_key" = "${ak}", + "s3.secret_key" = "${sk}", + "s3.region" = "${region}" + ) SELECT 1; + """ + exception "format" + } + + // ============ 25. Error: unsupported format ============ + + test { + sql """ + INSERT INTO s3( + "file_path" = "s3://${s3BasePath}/err.json", + "format" = "json", + "s3.endpoint" = "${s3_endpoint}", + "s3.access_key" = "${ak}", + "s3.secret_key" = "${sk}", + "s3.region" = "${region}" + ) SELECT 1; + """ + exception "Unsupported" + } + + // ============ Cleanup ============ + + sql """ DROP TABLE IF EXISTS insert_tvf_test_src """ + sql """ DROP TABLE IF EXISTS insert_tvf_complex_src """ + sql """ DROP TABLE IF EXISTS insert_tvf_join_src """ +} From 4c64caa67531afc0c539bc4ccb9b68bc13a3b516 Mon Sep 17 00:00:00 2001 From: morningman Date: Fri, 13 Feb 2026 00:48:17 +0800 Subject: [PATCH 05/13] hdfs pass --- .../tvf/insert/test_insert_into_hdfs_tvf.out | 133 +++++ .../insert/test_insert_into_hdfs_tvf.groovy | 516 ++++++++++++++++++ 2 files changed, 649 insertions(+) create mode 100644 regression-test/data/external_table_p0/tvf/insert/test_insert_into_hdfs_tvf.out create mode 100644 regression-test/suites/external_table_p0/tvf/insert/test_insert_into_hdfs_tvf.groovy diff --git a/regression-test/data/external_table_p0/tvf/insert/test_insert_into_hdfs_tvf.out b/regression-test/data/external_table_p0/tvf/insert/test_insert_into_hdfs_tvf.out new file mode 100644 index 00000000000000..1999c61a06e2a0 --- /dev/null +++ b/regression-test/data/external_table_p0/tvf/insert/test_insert_into_hdfs_tvf.out @@ -0,0 +1,133 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !hdfs_csv_basic_types -- +\N \N \N \N \N \N \N \N \N \N \N \N +0 -1 -100 -1000 -100000 -1.1 -2.2 -123.45 2020-02-29 2020-02-29 00:00:00 special_chars +0 2 200 2000 200000 3.3 4.4 678.90 2024-06-15 2024-06-15 12:30:00 foo bar +1 1 100 1000 100000 1.1 2.2 123.45 2024-01-01 2024-01-01 10:00:00 hello world +1 3 300 3000 300000 5.5 6.6 999.99 2024-12-31 2024-12-31 23:59:59 test data + +-- !hdfs_parquet_basic_types -- +\N \N \N \N \N \N \N \N \N \N \N \N +false -1 -100 -1000 -100000 -1.1 -2.2 -123.45 2020-02-29 2020-02-29T00:00 special_chars +false 2 200 2000 200000 3.3 4.4 678.90 2024-06-15 2024-06-15T12:30 foo bar +true 1 100 1000 100000 1.1 2.2 123.45 2024-01-01 2024-01-01T10:00 hello world +true 3 300 3000 300000 5.5 6.6 999.99 2024-12-31 2024-12-31T23:59:59 test data + +-- !hdfs_orc_basic_types -- +\N \N \N \N \N \N \N \N \N \N \N \N +false -1 -100 -1000 -100000 -1.1 -2.2 -123.45 2020-02-29 2020-02-29T00:00 special_chars +false 2 200 2000 200000 3.3 4.4 678.90 2024-06-15 2024-06-15T12:30 foo bar +true 1 100 1000 100000 1.1 2.2 123.45 2024-01-01 2024-01-01T10:00 hello world +true 3 300 3000 300000 5.5 6.6 999.99 2024-12-31 2024-12-31T23:59:59 test data + +-- !hdfs_parquet_complex_types -- +1 [1, 2, 3] {"a":1, "b":2} {"f1":1, "f2":"hello"} +2 [4, 5] {"x":10} {"f1":2, "f2":"world"} +3 [] {} {"f1":3, "f2":""} +4 \N \N \N + +-- !hdfs_orc_complex_types -- +1 [1, 2, 3] {"a":1, "b":2} {"f1":1, "f2":"hello"} +2 [4, 5] {"x":10} {"f1":2, "f2":"world"} +3 [] {} {"f1":3, "f2":""} +4 \N \N \N + +-- !hdfs_csv_sep_comma -- +-1000 special_chars +1000 hello world +2000 foo bar +3000 test data + +-- !hdfs_csv_sep_tab -- +-1000 special_chars +1000 hello world +2000 foo bar +3000 test data + +-- !hdfs_csv_sep_pipe -- +-1000 special_chars +1000 hello world +2000 foo bar +3000 test data + +-- !hdfs_csv_sep_multi -- +-1000 special_chars +1000 hello world +2000 foo bar +3000 test data + +-- !hdfs_csv_line_crlf -- +-1000 special_chars +1000 hello world +2000 foo bar +3000 test data + +-- !hdfs_csv_compress_gz -- +-1000 special_chars +1000 hello world +2000 foo bar +3000 test data + +-- !hdfs_csv_compress_zstd -- +-1000 special_chars +1000 hello world +2000 foo bar +3000 test data + +-- !hdfs_csv_compress_lz4 -- +-1000 special_chars +1000 hello world +2000 foo bar +3000 test data + +-- !hdfs_csv_compress_snappy -- +-1000 special_chars +1000 hello world +2000 foo bar +3000 test data + +-- !hdfs_overwrite_first -- +\N \N +-1000 +1000 hello +2000 foo +3000 test + +-- !hdfs_overwrite_second -- +1000 hello +2000 foo + +-- !hdfs_append_first -- +1000 hello + +-- !hdfs_append_second -- +2000 foo + +-- !hdfs_const_expr -- +1 hello 3.14 2024-01-01 + +-- !hdfs_where_groupby -- +0 2 1000 +1 2 4000 + +-- !hdfs_join_query -- +1000 hello label_a +2000 foo label_b +3000 test label_c + +-- !hdfs_subquery -- +-1000 special_chars +1000 hello world +2000 foo bar +3000 test data + +-- !hdfs_type_cast -- +-1000 -1.100000023841858 2020-02-29 +1000 1.100000023841858 2024-01-01 +2000 3.299999952316284 2024-06-15 +3000 5.5 2024-12-31 + +-- !hdfs_union_query -- +1000 hello +2000 foo + diff --git a/regression-test/suites/external_table_p0/tvf/insert/test_insert_into_hdfs_tvf.groovy b/regression-test/suites/external_table_p0/tvf/insert/test_insert_into_hdfs_tvf.groovy new file mode 100644 index 00000000000000..3ad1cfdb955a75 --- /dev/null +++ b/regression-test/suites/external_table_p0/tvf/insert/test_insert_into_hdfs_tvf.groovy @@ -0,0 +1,516 @@ +// 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. + +suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { + + String hdfs_port = context.config.otherConfigs.get("hive2HdfsPort") + String externalEnvIp = context.config.otherConfigs.get("externalEnvIp") + def hdfsUserName = "doris" + def defaultFS = "hdfs://${externalEnvIp}:${hdfs_port}" + def hdfsBasePath = "/tmp/test_insert_into_hdfs_tvf" + + String enabled = context.config.otherConfigs.get("enableHiveTest") + if (enabled == null || !enabled.equalsIgnoreCase("true")) { + logger.info("enableHiveTest not true, skip") + return + } + + def hdfsWriteProps = { String path, String format -> + return """ + "file_path" = "${hdfsBasePath}/${path}", + "format" = "${format}", + "hadoop.username" = "${hdfsUserName}", + "fs.defaultFS" = "${defaultFS}" + """ + } + + def hdfsReadProps = { String path, String format -> + return """ + "uri" = "${defaultFS}${hdfsBasePath}/${path}", + "hadoop.username" = "${hdfsUserName}", + "format" = "${format}" + """ + } + + // ============ Source tables ============ + + sql """ DROP TABLE IF EXISTS insert_tvf_test_src """ + sql """ + CREATE TABLE IF NOT EXISTS insert_tvf_test_src ( + c_bool BOOLEAN, + c_tinyint TINYINT, + c_smallint SMALLINT, + c_int INT, + c_bigint BIGINT, + c_float FLOAT, + c_double DOUBLE, + c_decimal DECIMAL(10,2), + c_date DATE, + c_datetime DATETIME, + c_varchar VARCHAR(100), + c_string STRING + ) DISTRIBUTED BY HASH(c_int) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO insert_tvf_test_src VALUES + (true, 1, 100, 1000, 100000, 1.1, 2.2, 123.45, '2024-01-01', '2024-01-01 10:00:00', 'hello', 'world'), + (false, 2, 200, 2000, 200000, 3.3, 4.4, 678.90, '2024-06-15', '2024-06-15 12:30:00', 'foo', 'bar'), + (true, 3, 300, 3000, 300000, 5.5, 6.6, 999.99, '2024-12-31', '2024-12-31 23:59:59', 'test', 'data'), + (NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL), + (false, -1, -100, -1000, -100000, -1.1, -2.2, -123.45,'2020-02-29', '2020-02-29 00:00:00', '', 'special_chars'); + """ + + sql """ DROP TABLE IF EXISTS insert_tvf_complex_src """ + sql """ + CREATE TABLE IF NOT EXISTS insert_tvf_complex_src ( + c_int INT, + c_array ARRAY, + c_map MAP, + c_struct STRUCT + ) DISTRIBUTED BY HASH(c_int) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO insert_tvf_complex_src VALUES + (1, [1, 2, 3], {'a': 1, 'b': 2}, {1, 'hello'}), + (2, [4, 5], {'x': 10}, {2, 'world'}), + (3, [], {}, {3, ''}), + (4, NULL, NULL, NULL); + """ + + sql """ DROP TABLE IF EXISTS insert_tvf_join_src """ + sql """ + CREATE TABLE IF NOT EXISTS insert_tvf_join_src ( + c_int INT, + c_label STRING + ) DISTRIBUTED BY HASH(c_int) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ INSERT INTO insert_tvf_join_src VALUES (1000, 'label_a'), (2000, 'label_b'), (3000, 'label_c'); """ + + // ============ 1. HDFS CSV basic types ============ + + sql """ + INSERT INTO hdfs( + ${hdfsWriteProps("basic_csv.csv", "csv")} + ) SELECT * FROM insert_tvf_test_src ORDER BY c_int; + """ + + order_qt_hdfs_csv_basic_types """ + SELECT * FROM hdfs( + ${hdfsReadProps("basic_csv.csv", "csv")} + ) ORDER BY c1; + """ + + // ============ 2. HDFS Parquet basic types ============ + + sql """ + INSERT INTO hdfs( + ${hdfsWriteProps("basic_parquet.parquet", "parquet")} + ) SELECT * FROM insert_tvf_test_src ORDER BY c_int; + """ + + order_qt_hdfs_parquet_basic_types """ + SELECT * FROM hdfs( + ${hdfsReadProps("basic_parquet.parquet", "parquet")} + ) ORDER BY c_int; + """ + + // ============ 3. HDFS ORC basic types ============ + + sql """ + INSERT INTO hdfs( + ${hdfsWriteProps("basic_orc.orc", "orc")} + ) SELECT * FROM insert_tvf_test_src ORDER BY c_int; + """ + + order_qt_hdfs_orc_basic_types """ + SELECT * FROM hdfs( + ${hdfsReadProps("basic_orc.orc", "orc")} + ) ORDER BY c_int; + """ + + // ============ 4. HDFS Parquet complex types ============ + + sql """ + INSERT INTO hdfs( + ${hdfsWriteProps("complex_parquet.parquet", "parquet")} + ) SELECT * FROM insert_tvf_complex_src ORDER BY c_int; + """ + + order_qt_hdfs_parquet_complex_types """ + SELECT * FROM hdfs( + ${hdfsReadProps("complex_parquet.parquet", "parquet")} + ) ORDER BY c_int; + """ + + // ============ 5. HDFS ORC complex types ============ + + sql """ + INSERT INTO hdfs( + ${hdfsWriteProps("complex_orc.orc", "orc")} + ) SELECT * FROM insert_tvf_complex_src ORDER BY c_int; + """ + + order_qt_hdfs_orc_complex_types """ + SELECT * FROM hdfs( + ${hdfsReadProps("complex_orc.orc", "orc")} + ) ORDER BY c_int; + """ + + // ============ 6. HDFS CSV separator: comma ============ + + sql """ + INSERT INTO hdfs( + ${hdfsWriteProps("sep_comma.csv", "csv")}, + "column_separator" = "," + ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; + """ + + order_qt_hdfs_csv_sep_comma """ + SELECT * FROM hdfs( + ${hdfsReadProps("sep_comma.csv", "csv")}, + "column_separator" = "," + ) ORDER BY c1; + """ + + // ============ 7. HDFS CSV separator: tab ============ + + sql """ + INSERT INTO hdfs( + ${hdfsWriteProps("sep_tab.csv", "csv")}, + "column_separator" = "\t" + ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; + """ + + order_qt_hdfs_csv_sep_tab """ + SELECT * FROM hdfs( + ${hdfsReadProps("sep_tab.csv", "csv")}, + "column_separator" = "\t" + ) ORDER BY c1; + """ + + // ============ 8. HDFS CSV separator: pipe ============ + + sql """ + INSERT INTO hdfs( + ${hdfsWriteProps("sep_pipe.csv", "csv")}, + "column_separator" = "|" + ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; + """ + + order_qt_hdfs_csv_sep_pipe """ + SELECT * FROM hdfs( + ${hdfsReadProps("sep_pipe.csv", "csv")}, + "column_separator" = "|" + ) ORDER BY c1; + """ + + // ============ 9. HDFS CSV separator: multi-char ============ + + sql """ + INSERT INTO hdfs( + ${hdfsWriteProps("sep_multi.csv", "csv")}, + "column_separator" = ";;" + ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; + """ + + order_qt_hdfs_csv_sep_multi """ + SELECT * FROM hdfs( + ${hdfsReadProps("sep_multi.csv", "csv")}, + "column_separator" = ";;" + ) ORDER BY c1; + """ + + // ============ 10. HDFS CSV line delimiter: CRLF ============ + + sql """ + INSERT INTO hdfs( + ${hdfsWriteProps("line_crlf.csv", "csv")}, + "line_delimiter" = "\r\n" + ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; + """ + + order_qt_hdfs_csv_line_crlf """ + SELECT * FROM hdfs( + ${hdfsReadProps("line_crlf.csv", "csv")}, + "line_delimiter" = "\r\n" + ) ORDER BY c1; + """ + + // ============ 11. HDFS CSV compress: gz ============ + + sql """ + INSERT INTO hdfs( + ${hdfsWriteProps("compress_gz.csv.gz", "csv")}, + "compression_type" = "gz" + ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; + """ + + order_qt_hdfs_csv_compress_gz """ + SELECT * FROM hdfs( + ${hdfsReadProps("compress_gz.csv.gz", "csv")}, + "compress_type" = "gz" + ) ORDER BY c1; + """ + + // ============ 12. HDFS CSV compress: zstd ============ + + sql """ + INSERT INTO hdfs( + ${hdfsWriteProps("compress_zstd.csv.zst", "csv")}, + "compression_type" = "zstd" + ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; + """ + + order_qt_hdfs_csv_compress_zstd """ + SELECT * FROM hdfs( + ${hdfsReadProps("compress_zstd.csv.zst", "csv")}, + "compress_type" = "zstd" + ) ORDER BY c1; + """ + + // ============ 13. HDFS CSV compress: lz4 ============ + + sql """ + INSERT INTO hdfs( + ${hdfsWriteProps("compress_lz4.csv.lz4", "csv")}, + "compression_type" = "lz4block" + ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; + """ + + order_qt_hdfs_csv_compress_lz4 """ + SELECT * FROM hdfs( + ${hdfsReadProps("compress_lz4.csv.lz4", "csv")}, + "compress_type" = "lz4block" + ) ORDER BY c1; + """ + + // ============ 14. HDFS CSV compress: snappy ============ + + sql """ + INSERT INTO hdfs( + ${hdfsWriteProps("compress_snappy.csv.snappy", "csv")}, + "compression_type" = "snappyblock" + ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; + """ + + order_qt_hdfs_csv_compress_snappy """ + SELECT * FROM hdfs( + ${hdfsReadProps("compress_snappy.csv.snappy", "csv")}, + "compress_type" = "snappyblock" + ) ORDER BY c1; + """ + + // ============ 15. HDFS Overwrite mode ============ + + // First write: 5 rows + sql """ + INSERT INTO hdfs( + ${hdfsWriteProps("overwrite.csv", "csv")} + ) SELECT c_int, c_varchar FROM insert_tvf_test_src ORDER BY c_int; + """ + + order_qt_hdfs_overwrite_first """ + SELECT * FROM hdfs( + ${hdfsReadProps("overwrite.csv", "csv")} + ) ORDER BY c1; + """ + + // Second write: 2 rows with overwrite + sql """ + INSERT INTO hdfs( + ${hdfsWriteProps("overwrite.csv", "csv")}, + "delete_existing_files" = "true" + ) SELECT c_int, c_varchar FROM insert_tvf_test_src WHERE c_int > 0 ORDER BY c_int LIMIT 2; + """ + + order_qt_hdfs_overwrite_second """ + SELECT * FROM hdfs( + ${hdfsReadProps("overwrite.csv", "csv")} + ) ORDER BY c1; + """ + + // ============ 16. HDFS Append mode ============ + + // First write + sql """ + INSERT INTO hdfs( + ${hdfsWriteProps("append.parquet", "parquet")}, + "delete_existing_files" = "false" + ) SELECT c_int, c_varchar FROM insert_tvf_test_src WHERE c_int = 1000; + """ + + order_qt_hdfs_append_first """ + SELECT * FROM hdfs( + ${hdfsReadProps("append.parquet", "parquet")} + ) ORDER BY c_int; + """ + + // Second write (append) + sql """ + INSERT INTO hdfs( + ${hdfsWriteProps("append.parquet", "parquet")}, + "delete_existing_files" = "false" + ) SELECT c_int, c_varchar FROM insert_tvf_test_src WHERE c_int = 2000; + """ + + order_qt_hdfs_append_second """ + SELECT * FROM hdfs( + ${hdfsReadProps("append.parquet", "parquet")} + ) ORDER BY c_int; + """ + + // ============ 17. HDFS Complex SELECT: constant expressions ============ + + sql """ + INSERT INTO hdfs( + ${hdfsWriteProps("const_expr.csv", "csv")} + ) SELECT 1, 'hello', 3.14, CAST('2024-01-01' AS DATE); + """ + + order_qt_hdfs_const_expr """ + SELECT * FROM hdfs( + ${hdfsReadProps("const_expr.csv", "csv")} + ) ORDER BY c1; + """ + + // ============ 18. HDFS Complex SELECT: WHERE + GROUP BY ============ + + sql """ + INSERT INTO hdfs( + ${hdfsWriteProps("where_groupby.csv", "csv")} + ) SELECT c_bool, COUNT(*), SUM(c_int) FROM insert_tvf_test_src WHERE c_int IS NOT NULL GROUP BY c_bool ORDER BY c_bool; + """ + + order_qt_hdfs_where_groupby """ + SELECT * FROM hdfs( + ${hdfsReadProps("where_groupby.csv", "csv")} + ) ORDER BY c1; + """ + + // ============ 19. HDFS Complex SELECT: JOIN ============ + + sql """ + INSERT INTO hdfs( + ${hdfsWriteProps("join_query.csv", "csv")} + ) SELECT a.c_int, a.c_varchar, b.c_label + FROM insert_tvf_test_src a INNER JOIN insert_tvf_join_src b ON a.c_int = b.c_int + ORDER BY a.c_int; + """ + + order_qt_hdfs_join_query """ + SELECT * FROM hdfs( + ${hdfsReadProps("join_query.csv", "csv")} + ) ORDER BY c1; + """ + + // ============ 20. HDFS Complex SELECT: subquery ============ + + sql """ + INSERT INTO hdfs( + ${hdfsWriteProps("subquery.csv", "csv")} + ) SELECT * FROM (SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int) sub; + """ + + order_qt_hdfs_subquery """ + SELECT * FROM hdfs( + ${hdfsReadProps("subquery.csv", "csv")} + ) ORDER BY c1; + """ + + // ============ 21. HDFS Complex SELECT: type cast ============ + + sql """ + INSERT INTO hdfs( + ${hdfsWriteProps("type_cast.csv", "csv")} + ) SELECT CAST(c_int AS BIGINT), CAST(c_float AS DOUBLE), CAST(c_date AS STRING) + FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; + """ + + order_qt_hdfs_type_cast """ + SELECT * FROM hdfs( + ${hdfsReadProps("type_cast.csv", "csv")} + ) ORDER BY c1; + """ + + // ============ 22. HDFS Complex SELECT: UNION ALL ============ + + sql """ + INSERT INTO hdfs( + ${hdfsWriteProps("union_query.csv", "csv")} + ) SELECT c_int, c_varchar FROM insert_tvf_test_src WHERE c_int = 1000 + UNION ALL + SELECT c_int, c_varchar FROM insert_tvf_test_src WHERE c_int = 2000; + """ + + order_qt_hdfs_union_query """ + SELECT * FROM hdfs( + ${hdfsReadProps("union_query.csv", "csv")} + ) ORDER BY c1; + """ + + // ============ 23. Error: missing file_path ============ + + test { + sql """ + INSERT INTO hdfs( + "format" = "csv", + "hadoop.username" = "${hdfsUserName}", + "fs.defaultFS" = "${defaultFS}" + ) SELECT 1; + """ + exception "file_path" + } + + // ============ 24. Error: missing format ============ + + test { + sql """ + INSERT INTO hdfs( + "file_path" = "${hdfsBasePath}/err.csv", + "hadoop.username" = "${hdfsUserName}", + "fs.defaultFS" = "${defaultFS}" + ) SELECT 1; + """ + exception "format" + } + + // ============ 25. Error: unsupported format ============ + + test { + sql """ + INSERT INTO hdfs( + "file_path" = "${hdfsBasePath}/err.json", + "format" = "json", + "hadoop.username" = "${hdfsUserName}", + "fs.defaultFS" = "${defaultFS}" + ) SELECT 1; + """ + exception "Unsupported" + } + + // ============ Cleanup ============ + + sql """ DROP TABLE IF EXISTS insert_tvf_test_src """ + sql """ DROP TABLE IF EXISTS insert_tvf_complex_src """ + sql """ DROP TABLE IF EXISTS insert_tvf_join_src """ +} From 1f2b34a4ea488023a6d5dd11b77006aa086717e1 Mon Sep 17 00:00:00 2001 From: morningman Date: Fri, 13 Feb 2026 01:14:53 +0800 Subject: [PATCH 06/13] 2 --- be/src/vec/sink/writer/vtvf_table_writer.cpp | 1 + .../commands/insert/InsertIntoTVFCommand.java | 9 ++- .../insert/test_insert_into_hdfs_tvf.groovy | 65 ++++++++++++------- .../insert/test_insert_into_local_tvf.groovy | 65 ++++++++++++------- .../tvf/insert/test_insert_into_s3_tvf.groovy | 65 ++++++++++++------- 5 files changed, 138 insertions(+), 67 deletions(-) diff --git a/be/src/vec/sink/writer/vtvf_table_writer.cpp b/be/src/vec/sink/writer/vtvf_table_writer.cpp index fbe579c9a680b6..2c7dc6189f4f09 100644 --- a/be/src/vec/sink/writer/vtvf_table_writer.cpp +++ b/be/src/vec/sink/writer/vtvf_table_writer.cpp @@ -73,6 +73,7 @@ Status VTVFTableWriter::open(RuntimeState* state, RuntimeProfile* profile) { Status VTVFTableWriter::write(RuntimeState* state, vectorized::Block& block) { COUNTER_UPDATE(_written_rows_counter, block.rows()); + state->update_num_rows_load_total(block.rows()); { SCOPED_TIMER(_file_write_timer); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertIntoTVFCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertIntoTVFCommand.java index 2066a67b04be5c..8bfc3d8e5195a2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertIntoTVFCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertIntoTVFCommand.java @@ -23,6 +23,7 @@ import org.apache.doris.common.ErrorReport; import org.apache.doris.common.Status; import org.apache.doris.mysql.privilege.PrivPredicate; +import org.apache.doris.task.LoadEtlTask; import org.apache.doris.nereids.NereidsPlanner; import org.apache.doris.nereids.glue.LogicalPlanAdapter; import org.apache.doris.nereids.trees.plans.Explainable; @@ -117,7 +118,13 @@ public void run(ConnectContext ctx, StmtExecutor executor) throws Exception { if (coordinator.getExecStatus().ok()) { String label = labelName.orElse( String.format("tvf_insert_%x_%x", ctx.queryId().hi, ctx.queryId().lo)); - ctx.getState().setOk(0, 0, "Insert into TVF succeeded. label: " + label); + long loadedRows = 0; + String loadedRowsStr = coordinator.getLoadCounters() + .get(LoadEtlTask.DPP_NORMAL_ALL); + if (loadedRowsStr != null) { + loadedRows = Long.parseLong(loadedRowsStr); + } + ctx.getState().setOk(loadedRows, 0, "Insert into TVF succeeded. label: " + label); } else { String errMsg = coordinator.getExecStatus().getErrorMsg(); LOG.warn("insert into TVF failed, error: {}", errMsg); diff --git a/regression-test/suites/external_table_p0/tvf/insert/test_insert_into_hdfs_tvf.groovy b/regression-test/suites/external_table_p0/tvf/insert/test_insert_into_hdfs_tvf.groovy index 3ad1cfdb955a75..bd4a1f9ce07f47 100644 --- a/regression-test/suites/external_table_p0/tvf/insert/test_insert_into_hdfs_tvf.groovy +++ b/regression-test/suites/external_table_p0/tvf/insert/test_insert_into_hdfs_tvf.groovy @@ -110,7 +110,8 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { sql """ INSERT INTO hdfs( - ${hdfsWriteProps("basic_csv.csv", "csv")} + ${hdfsWriteProps("basic_csv.csv", "csv")}, + "delete_existing_files" = "true" ) SELECT * FROM insert_tvf_test_src ORDER BY c_int; """ @@ -124,7 +125,8 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { sql """ INSERT INTO hdfs( - ${hdfsWriteProps("basic_parquet.parquet", "parquet")} + ${hdfsWriteProps("basic_parquet.parquet", "parquet")}, + "delete_existing_files" = "true" ) SELECT * FROM insert_tvf_test_src ORDER BY c_int; """ @@ -138,7 +140,8 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { sql """ INSERT INTO hdfs( - ${hdfsWriteProps("basic_orc.orc", "orc")} + ${hdfsWriteProps("basic_orc.orc", "orc")}, + "delete_existing_files" = "true" ) SELECT * FROM insert_tvf_test_src ORDER BY c_int; """ @@ -152,7 +155,8 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { sql """ INSERT INTO hdfs( - ${hdfsWriteProps("complex_parquet.parquet", "parquet")} + ${hdfsWriteProps("complex_parquet.parquet", "parquet")}, + "delete_existing_files" = "true" ) SELECT * FROM insert_tvf_complex_src ORDER BY c_int; """ @@ -166,7 +170,8 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { sql """ INSERT INTO hdfs( - ${hdfsWriteProps("complex_orc.orc", "orc")} + ${hdfsWriteProps("complex_orc.orc", "orc")}, + "delete_existing_files" = "true" ) SELECT * FROM insert_tvf_complex_src ORDER BY c_int; """ @@ -181,7 +186,8 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { sql """ INSERT INTO hdfs( ${hdfsWriteProps("sep_comma.csv", "csv")}, - "column_separator" = "," + "column_separator" = ",", + "delete_existing_files" = "true" ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; """ @@ -197,7 +203,8 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { sql """ INSERT INTO hdfs( ${hdfsWriteProps("sep_tab.csv", "csv")}, - "column_separator" = "\t" + "column_separator" = "\t", + "delete_existing_files" = "true" ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; """ @@ -213,7 +220,8 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { sql """ INSERT INTO hdfs( ${hdfsWriteProps("sep_pipe.csv", "csv")}, - "column_separator" = "|" + "column_separator" = "|", + "delete_existing_files" = "true" ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; """ @@ -229,7 +237,8 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { sql """ INSERT INTO hdfs( ${hdfsWriteProps("sep_multi.csv", "csv")}, - "column_separator" = ";;" + "column_separator" = ";;", + "delete_existing_files" = "true" ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; """ @@ -245,7 +254,8 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { sql """ INSERT INTO hdfs( ${hdfsWriteProps("line_crlf.csv", "csv")}, - "line_delimiter" = "\r\n" + "line_delimiter" = "\r\n", + "delete_existing_files" = "true" ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; """ @@ -261,7 +271,8 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { sql """ INSERT INTO hdfs( ${hdfsWriteProps("compress_gz.csv.gz", "csv")}, - "compression_type" = "gz" + "compression_type" = "gz", + "delete_existing_files" = "true" ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; """ @@ -277,7 +288,8 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { sql """ INSERT INTO hdfs( ${hdfsWriteProps("compress_zstd.csv.zst", "csv")}, - "compression_type" = "zstd" + "compression_type" = "zstd", + "delete_existing_files" = "true" ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; """ @@ -293,7 +305,8 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { sql """ INSERT INTO hdfs( ${hdfsWriteProps("compress_lz4.csv.lz4", "csv")}, - "compression_type" = "lz4block" + "compression_type" = "lz4block", + "delete_existing_files" = "true" ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; """ @@ -309,7 +322,8 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { sql """ INSERT INTO hdfs( ${hdfsWriteProps("compress_snappy.csv.snappy", "csv")}, - "compression_type" = "snappyblock" + "compression_type" = "snappyblock", + "delete_existing_files" = "true" ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; """ @@ -325,7 +339,8 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { // First write: 5 rows sql """ INSERT INTO hdfs( - ${hdfsWriteProps("overwrite.csv", "csv")} + ${hdfsWriteProps("overwrite.csv", "csv")}, + "delete_existing_files" = "true" ) SELECT c_int, c_varchar FROM insert_tvf_test_src ORDER BY c_int; """ @@ -355,7 +370,7 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { sql """ INSERT INTO hdfs( ${hdfsWriteProps("append.parquet", "parquet")}, - "delete_existing_files" = "false" + "delete_existing_files" = "true" ) SELECT c_int, c_varchar FROM insert_tvf_test_src WHERE c_int = 1000; """ @@ -383,7 +398,8 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { sql """ INSERT INTO hdfs( - ${hdfsWriteProps("const_expr.csv", "csv")} + ${hdfsWriteProps("const_expr.csv", "csv")}, + "delete_existing_files" = "true" ) SELECT 1, 'hello', 3.14, CAST('2024-01-01' AS DATE); """ @@ -397,7 +413,8 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { sql """ INSERT INTO hdfs( - ${hdfsWriteProps("where_groupby.csv", "csv")} + ${hdfsWriteProps("where_groupby.csv", "csv")}, + "delete_existing_files" = "true" ) SELECT c_bool, COUNT(*), SUM(c_int) FROM insert_tvf_test_src WHERE c_int IS NOT NULL GROUP BY c_bool ORDER BY c_bool; """ @@ -411,7 +428,8 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { sql """ INSERT INTO hdfs( - ${hdfsWriteProps("join_query.csv", "csv")} + ${hdfsWriteProps("join_query.csv", "csv")}, + "delete_existing_files" = "true" ) SELECT a.c_int, a.c_varchar, b.c_label FROM insert_tvf_test_src a INNER JOIN insert_tvf_join_src b ON a.c_int = b.c_int ORDER BY a.c_int; @@ -427,7 +445,8 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { sql """ INSERT INTO hdfs( - ${hdfsWriteProps("subquery.csv", "csv")} + ${hdfsWriteProps("subquery.csv", "csv")}, + "delete_existing_files" = "true" ) SELECT * FROM (SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int) sub; """ @@ -441,7 +460,8 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { sql """ INSERT INTO hdfs( - ${hdfsWriteProps("type_cast.csv", "csv")} + ${hdfsWriteProps("type_cast.csv", "csv")}, + "delete_existing_files" = "true" ) SELECT CAST(c_int AS BIGINT), CAST(c_float AS DOUBLE), CAST(c_date AS STRING) FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; """ @@ -456,7 +476,8 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { sql """ INSERT INTO hdfs( - ${hdfsWriteProps("union_query.csv", "csv")} + ${hdfsWriteProps("union_query.csv", "csv")}, + "delete_existing_files" = "true" ) SELECT c_int, c_varchar FROM insert_tvf_test_src WHERE c_int = 1000 UNION ALL SELECT c_int, c_varchar FROM insert_tvf_test_src WHERE c_int = 2000; diff --git a/regression-test/suites/external_table_p0/tvf/insert/test_insert_into_local_tvf.groovy b/regression-test/suites/external_table_p0/tvf/insert/test_insert_into_local_tvf.groovy index 375310c9cfb342..80dee8ff381bab 100644 --- a/regression-test/suites/external_table_p0/tvf/insert/test_insert_into_local_tvf.groovy +++ b/regression-test/suites/external_table_p0/tvf/insert/test_insert_into_local_tvf.groovy @@ -94,7 +94,8 @@ suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { INSERT INTO local( "file_path" = "${basePath}/basic_csv.csv", "backend_id" = "${be_id}", - "format" = "csv" + "format" = "csv", + "delete_existing_files" = "true" ) SELECT * FROM insert_tvf_test_src ORDER BY c_int; """ @@ -112,7 +113,8 @@ suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { INSERT INTO local( "file_path" = "${basePath}/basic_parquet.parquet", "backend_id" = "${be_id}", - "format" = "parquet" + "format" = "parquet", + "delete_existing_files" = "true" ) SELECT * FROM insert_tvf_test_src ORDER BY c_int; """ @@ -130,7 +132,8 @@ suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { INSERT INTO local( "file_path" = "${basePath}/basic_orc.orc", "backend_id" = "${be_id}", - "format" = "orc" + "format" = "orc", + "delete_existing_files" = "true" ) SELECT * FROM insert_tvf_test_src ORDER BY c_int; """ @@ -148,7 +151,8 @@ suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { INSERT INTO local( "file_path" = "${basePath}/complex_parquet.parquet", "backend_id" = "${be_id}", - "format" = "parquet" + "format" = "parquet", + "delete_existing_files" = "true" ) SELECT * FROM insert_tvf_complex_src ORDER BY c_int; """ @@ -166,7 +170,8 @@ suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { INSERT INTO local( "file_path" = "${basePath}/complex_orc.orc", "backend_id" = "${be_id}", - "format" = "orc" + "format" = "orc", + "delete_existing_files" = "true" ) SELECT * FROM insert_tvf_complex_src ORDER BY c_int; """ @@ -185,7 +190,8 @@ suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { "file_path" = "${basePath}/sep_comma.csv", "backend_id" = "${be_id}", "format" = "csv", - "column_separator" = "," + "column_separator" = ",", + "delete_existing_files" = "true" ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; """ @@ -205,7 +211,8 @@ suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { "file_path" = "${basePath}/sep_tab.csv", "backend_id" = "${be_id}", "format" = "csv", - "column_separator" = "\t" + "column_separator" = "\t", + "delete_existing_files" = "true" ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; """ @@ -225,7 +232,8 @@ suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { "file_path" = "${basePath}/sep_pipe.csv", "backend_id" = "${be_id}", "format" = "csv", - "column_separator" = "|" + "column_separator" = "|", + "delete_existing_files" = "true" ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; """ @@ -245,7 +253,8 @@ suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { "file_path" = "${basePath}/sep_multi.csv", "backend_id" = "${be_id}", "format" = "csv", - "column_separator" = ";;" + "column_separator" = ";;", + "delete_existing_files" = "true" ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; """ @@ -265,7 +274,8 @@ suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { "file_path" = "${basePath}/line_crlf.csv", "backend_id" = "${be_id}", "format" = "csv", - "line_delimiter" = "\r\n" + "line_delimiter" = "\r\n", + "delete_existing_files" = "true" ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; """ @@ -285,7 +295,8 @@ suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { "file_path" = "${basePath}/compress_gz.csv.gz", "backend_id" = "${be_id}", "format" = "csv", - "compression_type" = "gz" + "compression_type" = "gz", + "delete_existing_files" = "true" ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; """ @@ -305,7 +316,8 @@ suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { "file_path" = "${basePath}/compress_zstd.csv.zst", "backend_id" = "${be_id}", "format" = "csv", - "compression_type" = "zstd" + "compression_type" = "zstd", + "delete_existing_files" = "true" ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; """ @@ -326,7 +338,8 @@ suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { "file_path" = "${basePath}/compress_lz4.csv.lz4", "backend_id" = "${be_id}", "format" = "csv", - "compression_type" = "lz4block" + "compression_type" = "lz4block", + "delete_existing_files" = "true" ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; """ @@ -346,7 +359,8 @@ suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { "file_path" = "${basePath}/compress_snappy.csv.snappy", "backend_id" = "${be_id}", "format" = "csv", - "compression_type" = "snappyblock" + "compression_type" = "snappyblock", + "delete_existing_files" = "true" ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; """ @@ -366,7 +380,8 @@ suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { INSERT INTO local( "file_path" = "${basePath}/overwrite.csv", "backend_id" = "${be_id}", - "format" = "csv" + "format" = "csv", + "delete_existing_files" = "true" ) SELECT c_int, c_varchar FROM insert_tvf_test_src ORDER BY c_int; """ @@ -404,7 +419,7 @@ suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { "file_path" = "${basePath}/append.parquet", "backend_id" = "${be_id}", "format" = "parquet", - "delete_existing_files" = "false" + "delete_existing_files" = "true" ) SELECT c_int, c_varchar FROM insert_tvf_test_src WHERE c_int = 1000; """ @@ -440,7 +455,8 @@ suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { INSERT INTO local( "file_path" = "${basePath}/const_expr.csv", "backend_id" = "${be_id}", - "format" = "csv" + "format" = "csv", + "delete_existing_files" = "true" ) SELECT 1, 'hello', 3.14, CAST('2024-01-01' AS DATE); """ @@ -458,7 +474,8 @@ suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { INSERT INTO local( "file_path" = "${basePath}/where_groupby.csv", "backend_id" = "${be_id}", - "format" = "csv" + "format" = "csv", + "delete_existing_files" = "true" ) SELECT c_bool, COUNT(*), SUM(c_int) FROM insert_tvf_test_src WHERE c_int IS NOT NULL GROUP BY c_bool ORDER BY c_bool; """ @@ -476,7 +493,8 @@ suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { INSERT INTO local( "file_path" = "${basePath}/join_query.csv", "backend_id" = "${be_id}", - "format" = "csv" + "format" = "csv", + "delete_existing_files" = "true" ) SELECT a.c_int, a.c_varchar, b.c_label FROM insert_tvf_test_src a INNER JOIN insert_tvf_join_src b ON a.c_int = b.c_int ORDER BY a.c_int; @@ -496,7 +514,8 @@ suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { INSERT INTO local( "file_path" = "${basePath}/subquery.csv", "backend_id" = "${be_id}", - "format" = "csv" + "format" = "csv", + "delete_existing_files" = "true" ) SELECT * FROM (SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int) sub; """ @@ -514,7 +533,8 @@ suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { INSERT INTO local( "file_path" = "${basePath}/type_cast.csv", "backend_id" = "${be_id}", - "format" = "csv" + "format" = "csv", + "delete_existing_files" = "true" ) SELECT CAST(c_int AS BIGINT), CAST(c_float AS DOUBLE), CAST(c_date AS STRING) FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; """ @@ -533,7 +553,8 @@ suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { INSERT INTO local( "file_path" = "${basePath}/union_query.csv", "backend_id" = "${be_id}", - "format" = "csv" + "format" = "csv", + "delete_existing_files" = "true" ) SELECT c_int, c_varchar FROM insert_tvf_test_src WHERE c_int = 1000 UNION ALL SELECT c_int, c_varchar FROM insert_tvf_test_src WHERE c_int = 2000; diff --git a/regression-test/suites/external_table_p0/tvf/insert/test_insert_into_s3_tvf.groovy b/regression-test/suites/external_table_p0/tvf/insert/test_insert_into_s3_tvf.groovy index 910a3edf6eb679..8b1c198a2c7f3d 100644 --- a/regression-test/suites/external_table_p0/tvf/insert/test_insert_into_s3_tvf.groovy +++ b/regression-test/suites/external_table_p0/tvf/insert/test_insert_into_s3_tvf.groovy @@ -115,7 +115,8 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { sql """ INSERT INTO s3( - ${s3WriteProps("basic_csv.csv", "csv")} + ${s3WriteProps("basic_csv.csv", "csv")}, + "delete_existing_files" = "true" ) SELECT * FROM insert_tvf_test_src ORDER BY c_int; """ @@ -129,7 +130,8 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { sql """ INSERT INTO s3( - ${s3WriteProps("basic_parquet.parquet", "parquet")} + ${s3WriteProps("basic_parquet.parquet", "parquet")}, + "delete_existing_files" = "true" ) SELECT * FROM insert_tvf_test_src ORDER BY c_int; """ @@ -143,7 +145,8 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { sql """ INSERT INTO s3( - ${s3WriteProps("basic_orc.orc", "orc")} + ${s3WriteProps("basic_orc.orc", "orc")}, + "delete_existing_files" = "true" ) SELECT * FROM insert_tvf_test_src ORDER BY c_int; """ @@ -157,7 +160,8 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { sql """ INSERT INTO s3( - ${s3WriteProps("complex_parquet.parquet", "parquet")} + ${s3WriteProps("complex_parquet.parquet", "parquet")}, + "delete_existing_files" = "true" ) SELECT * FROM insert_tvf_complex_src ORDER BY c_int; """ @@ -171,7 +175,8 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { sql """ INSERT INTO s3( - ${s3WriteProps("complex_orc.orc", "orc")} + ${s3WriteProps("complex_orc.orc", "orc")}, + "delete_existing_files" = "true" ) SELECT * FROM insert_tvf_complex_src ORDER BY c_int; """ @@ -186,7 +191,8 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { sql """ INSERT INTO s3( ${s3WriteProps("sep_comma.csv", "csv")}, - "column_separator" = "," + "column_separator" = ",", + "delete_existing_files" = "true" ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; """ @@ -202,7 +208,8 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { sql """ INSERT INTO s3( ${s3WriteProps("sep_tab.csv", "csv")}, - "column_separator" = "\t" + "column_separator" = "\t", + "delete_existing_files" = "true" ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; """ @@ -218,7 +225,8 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { sql """ INSERT INTO s3( ${s3WriteProps("sep_pipe.csv", "csv")}, - "column_separator" = "|" + "column_separator" = "|", + "delete_existing_files" = "true" ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; """ @@ -234,7 +242,8 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { sql """ INSERT INTO s3( ${s3WriteProps("sep_multi.csv", "csv")}, - "column_separator" = ";;" + "column_separator" = ";;", + "delete_existing_files" = "true" ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; """ @@ -250,7 +259,8 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { sql """ INSERT INTO s3( ${s3WriteProps("line_crlf.csv", "csv")}, - "line_delimiter" = "\r\n" + "line_delimiter" = "\r\n", + "delete_existing_files" = "true" ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; """ @@ -266,7 +276,8 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { sql """ INSERT INTO s3( ${s3WriteProps("compress_gz.csv.gz", "csv")}, - "compression_type" = "gz" + "compression_type" = "gz", + "delete_existing_files" = "true" ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; """ @@ -282,7 +293,8 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { sql """ INSERT INTO s3( ${s3WriteProps("compress_zstd.csv.zst", "csv")}, - "compression_type" = "zstd" + "compression_type" = "zstd", + "delete_existing_files" = "true" ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; """ @@ -298,7 +310,8 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { sql """ INSERT INTO s3( ${s3WriteProps("compress_lz4.csv.lz4", "csv")}, - "compression_type" = "lz4block" + "compression_type" = "lz4block", + "delete_existing_files" = "true" ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; """ @@ -314,7 +327,8 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { sql """ INSERT INTO s3( ${s3WriteProps("compress_snappy.csv.snappy", "csv")}, - "compression_type" = "snappyblock" + "compression_type" = "snappyblock", + "delete_existing_files" = "true" ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; """ @@ -330,7 +344,8 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { // First write: 5 rows sql """ INSERT INTO s3( - ${s3WriteProps("overwrite.csv", "csv")} + ${s3WriteProps("overwrite.csv", "csv")}, + "delete_existing_files" = "true" ) SELECT c_int, c_varchar FROM insert_tvf_test_src ORDER BY c_int; """ @@ -360,7 +375,7 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { sql """ INSERT INTO s3( ${s3WriteProps("append.parquet", "parquet")}, - "delete_existing_files" = "false" + "delete_existing_files" = "true" ) SELECT c_int, c_varchar FROM insert_tvf_test_src WHERE c_int = 1000; """ @@ -388,7 +403,8 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { sql """ INSERT INTO s3( - ${s3WriteProps("const_expr.csv", "csv")} + ${s3WriteProps("const_expr.csv", "csv")}, + "delete_existing_files" = "true" ) SELECT 1, 'hello', 3.14, CAST('2024-01-01' AS DATE); """ @@ -402,7 +418,8 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { sql """ INSERT INTO s3( - ${s3WriteProps("where_groupby.csv", "csv")} + ${s3WriteProps("where_groupby.csv", "csv")}, + "delete_existing_files" = "true" ) SELECT c_bool, COUNT(*), SUM(c_int) FROM insert_tvf_test_src WHERE c_int IS NOT NULL GROUP BY c_bool ORDER BY c_bool; """ @@ -416,7 +433,8 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { sql """ INSERT INTO s3( - ${s3WriteProps("join_query.csv", "csv")} + ${s3WriteProps("join_query.csv", "csv")}, + "delete_existing_files" = "true" ) SELECT a.c_int, a.c_varchar, b.c_label FROM insert_tvf_test_src a INNER JOIN insert_tvf_join_src b ON a.c_int = b.c_int ORDER BY a.c_int; @@ -432,7 +450,8 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { sql """ INSERT INTO s3( - ${s3WriteProps("subquery.csv", "csv")} + ${s3WriteProps("subquery.csv", "csv")}, + "delete_existing_files" = "true" ) SELECT * FROM (SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int) sub; """ @@ -446,7 +465,8 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { sql """ INSERT INTO s3( - ${s3WriteProps("type_cast.csv", "csv")} + ${s3WriteProps("type_cast.csv", "csv")}, + "delete_existing_files" = "true" ) SELECT CAST(c_int AS BIGINT), CAST(c_float AS DOUBLE), CAST(c_date AS STRING) FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; """ @@ -461,7 +481,8 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { sql """ INSERT INTO s3( - ${s3WriteProps("union_query.csv", "csv")} + ${s3WriteProps("union_query.csv", "csv")}, + "delete_existing_files" = "true" ) SELECT c_int, c_varchar FROM insert_tvf_test_src WHERE c_int = 1000 UNION ALL SELECT c_int, c_varchar FROM insert_tvf_test_src WHERE c_int = 2000; From 6813554f85c0f7c91f95af718412ae5fe82a511c Mon Sep 17 00:00:00 2001 From: morningman Date: Fri, 13 Feb 2026 08:08:21 +0800 Subject: [PATCH 07/13] fix cases --- be/src/vec/sink/writer/vtvf_table_writer.cpp | 44 +-- be/src/vec/sink/writer/vtvf_table_writer.h | 2 - .../nereids/rules/analysis/BindSink.java | 68 ++--- .../commands/insert/InsertIntoTVFCommand.java | 57 ++++ .../apache/doris/planner/TVFTableSink.java | 5 +- .../insert/test_insert_into_hdfs_tvf.groovy | 122 ++++---- .../insert/test_insert_into_local_tvf.groovy | 260 +++++++++++------- .../tvf/insert/test_insert_into_s3_tvf.groovy | 124 +++++---- 8 files changed, 377 insertions(+), 305 deletions(-) diff --git a/be/src/vec/sink/writer/vtvf_table_writer.cpp b/be/src/vec/sink/writer/vtvf_table_writer.cpp index 2c7dc6189f4f09..e001461d570203 100644 --- a/be/src/vec/sink/writer/vtvf_table_writer.cpp +++ b/be/src/vec/sink/writer/vtvf_table_writer.cpp @@ -22,7 +22,6 @@ #include "common/status.h" #include "io/file_factory.h" -#include "io/fs/local_file_system.h" #include "runtime/runtime_state.h" #include "vec/core/block.h" #include "vec/exprs/vexpr.h" @@ -53,21 +52,13 @@ Status VTVFTableWriter::open(RuntimeState* state, RuntimeProfile* profile) { _file_path = _tvf_sink.file_path; _max_file_size_bytes = _tvf_sink.__isset.max_file_size_bytes ? _tvf_sink.max_file_size_bytes : 0; - _delete_existing_files_flag = - _tvf_sink.__isset.delete_existing_files ? _tvf_sink.delete_existing_files : true; VLOG_DEBUG << "TVF table writer open, query_id=" << print_id(_state->query_id()) << ", tvf_name=" << _tvf_sink.tvf_name << ", file_path=" << _tvf_sink.file_path << ", file_format=" << _tvf_sink.file_format << ", file_type=" << _tvf_sink.file_type << ", max_file_size_bytes=" << _max_file_size_bytes - << ", delete_existing_files=" << _delete_existing_files_flag << ", columns_count=" << (_tvf_sink.__isset.columns ? _tvf_sink.columns.size() : 0); - // Delete existing files if requested - if (_delete_existing_files_flag) { - RETURN_IF_ERROR(_delete_existing_files()); - } - return _create_next_file_writer(); } @@ -199,7 +190,6 @@ Status VTVFTableWriter::_create_new_file_if_exceed_size() { } Status VTVFTableWriter::_get_next_file_name(std::string* file_name) { - // Determine file extension std::string ext; switch (_tvf_sink.file_format) { case TFileFormatType::FORMAT_CSV_PLAIN: @@ -216,39 +206,11 @@ Status VTVFTableWriter::_get_next_file_name(std::string* file_name) { break; } - if (_file_idx == 0 && _max_file_size_bytes <= 0) { - // Single file mode: use the path as-is if it already has extension - if (_file_path.find('.') != std::string::npos) { - *file_name = _file_path; - } else { - *file_name = fmt::format("{}.{}", _file_path, ext); - } - } else { - // Multi-file (auto-split) mode: append index - // Strip extension from base path if present - std::string base = _file_path; - auto dot_pos = base.rfind('.'); - if (dot_pos != std::string::npos) { - base = base.substr(0, dot_pos); - } - *file_name = fmt::format("{}_{}.{}", base, _file_idx, ext); - } + // file_path is a prefix, generate: {prefix}{query_id}_{idx}.{ext} + std::string query_id_str = print_id(_state->query_id()); + *file_name = fmt::format("{}{}_{}.{}", _file_path, query_id_str, _file_idx, ext); _file_idx++; return Status::OK(); } -Status VTVFTableWriter::_delete_existing_files() { - if (_tvf_sink.file_type == TFileType::FILE_LOCAL) { - // For local files, try to delete the file if it exists - bool exists = false; - RETURN_IF_ERROR(io::global_local_filesystem()->exists(_file_path, &exists)); - if (exists) { - RETURN_IF_ERROR(io::global_local_filesystem()->delete_file(_file_path)); - } - } - // For S3/HDFS, we don't delete existing files by default - // as it requires more complex handling (e.g., directory listing) - return Status::OK(); -} - } // namespace doris::vectorized diff --git a/be/src/vec/sink/writer/vtvf_table_writer.h b/be/src/vec/sink/writer/vtvf_table_writer.h index 7b6d1239860293..17c3fc7a159e3e 100644 --- a/be/src/vec/sink/writer/vtvf_table_writer.h +++ b/be/src/vec/sink/writer/vtvf_table_writer.h @@ -64,7 +64,6 @@ class VTVFTableWriter final : public AsyncResultWriter { Status _close_file_writer(bool done); Status _create_new_file_if_exceed_size(); Status _get_next_file_name(std::string* file_name); - Status _delete_existing_files(); TTVFTableSink _tvf_sink; RuntimeState* _state = nullptr; @@ -75,7 +74,6 @@ class VTVFTableWriter final : public AsyncResultWriter { int64_t _current_written_bytes = 0; int64_t _max_file_size_bytes = 0; int _file_idx = 0; - bool _delete_existing_files_flag = true; std::string _file_path; std::string _column_separator; std::string _line_delimiter; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindSink.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindSink.java index a980656a0468d6..4b5874c61e68fa 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindSink.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindSink.java @@ -97,10 +97,7 @@ import org.apache.doris.qe.AutoCloseSessionVariable; import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.SessionVariable; -import org.apache.doris.tablefunction.ExternalFileTableValuedFunction; -import org.apache.doris.tablefunction.HdfsTableValuedFunction; -import org.apache.doris.tablefunction.LocalTableValuedFunction; -import org.apache.doris.tablefunction.S3TableValuedFunction; + import org.apache.doris.thrift.TPartialUpdateNewRowPolicy; import com.google.common.base.Preconditions; @@ -589,22 +586,28 @@ private Plan bindTVFTableSink(MatchingContext> ctx) { throw new AnalysisException("local TVF sink requires 'backend_id' property"); } - LogicalPlan child = ((LogicalPlan) sink.child()); + // Validate file_path must not contain wildcards + String filePath = properties.get("file_path"); + if (filePath.contains("*") || filePath.contains("?") || filePath.contains("[")) { + throw new AnalysisException( + "TVF sink file_path must not contain wildcards: " + filePath); + } - // Determine target schema: if append mode and file exists, use existing file schema; - // otherwise derive from child query output. + // local TVF does not support delete_existing_files=true boolean deleteExisting = Boolean.parseBoolean( - properties.getOrDefault("delete_existing_files", "true")); - List cols = null; - if (!deleteExisting) { - cols = tryGetExistingFileSchema(tvfName, properties); - } - if (cols == null) { - cols = child.getOutput().stream() - .map(slot -> new Column(slot.getName(), slot.getDataType().toCatalogDataType())) - .collect(ImmutableList.toImmutableList()); + properties.getOrDefault("delete_existing_files", "false")); + if (tvfName.equals("local") && deleteExisting) { + throw new AnalysisException( + "delete_existing_files=true is not supported for local TVF"); } + LogicalPlan child = ((LogicalPlan) sink.child()); + + // Always derive schema from child query output + List cols = child.getOutput().stream() + .map(slot -> new Column(slot.getName(), slot.getDataType().toCatalogDataType())) + .collect(ImmutableList.toImmutableList()); + // Validate column count if (cols.size() != child.getOutput().size()) { throw new AnalysisException( @@ -633,39 +636,6 @@ private Plan bindTVFTableSink(MatchingContext> ctx) { Optional.empty(), Optional.empty(), projectWithCast); } - /** - * Try to instantiate the corresponding TVF to read the existing file's schema. - * Returns null if the file does not exist or schema inference fails. - */ - private List tryGetExistingFileSchema(String tvfName, Map properties) { - try { - ExternalFileTableValuedFunction tvf; - Map propsCopy = new HashMap<>(properties); - switch (tvfName) { - case "local": - tvf = new LocalTableValuedFunction(propsCopy); - break; - case "s3": - tvf = new S3TableValuedFunction(propsCopy); - break; - case "hdfs": - tvf = new HdfsTableValuedFunction(propsCopy); - break; - default: - return null; - } - List columns = tvf.getTableColumns(); - if (columns != null && !columns.isEmpty()) { - return columns; - } - } catch (Exception e) { - // File does not exist or schema inference failed — fall back to child query schema - LOG.info("TVF sink: could not read existing file schema for append mode, " - + "will use child query schema. Reason: " + e.getMessage()); - } - return null; - } - private Plan bindHiveTableSink(MatchingContext> ctx) { UnboundHiveTableSink sink = ctx.root; Pair pair = bind(ctx.cascadesContext, sink); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertIntoTVFCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertIntoTVFCommand.java index 8bfc3d8e5195a2..c35df25d98d1b3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertIntoTVFCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertIntoTVFCommand.java @@ -22,6 +22,10 @@ import org.apache.doris.common.ErrorCode; import org.apache.doris.common.ErrorReport; import org.apache.doris.common.Status; +import org.apache.doris.common.UserException; +import org.apache.doris.datasource.property.storage.StorageProperties; +import org.apache.doris.fs.FileSystemFactory; +import org.apache.doris.fs.remote.RemoteFileSystem; import org.apache.doris.mysql.privilege.PrivPredicate; import org.apache.doris.task.LoadEtlTask; import org.apache.doris.nereids.NereidsPlanner; @@ -32,6 +36,8 @@ import org.apache.doris.nereids.trees.plans.commands.Command; import org.apache.doris.nereids.trees.plans.commands.ForwardWithSync; import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; +import org.apache.doris.nereids.trees.plans.physical.PhysicalPlan; +import org.apache.doris.nereids.trees.plans.physical.PhysicalTVFTableSink; import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.ConnectContext.ConnectType; @@ -45,6 +51,8 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import java.util.HashMap; +import java.util.Map; import java.util.Optional; /** @@ -90,6 +98,20 @@ public void run(ConnectContext ctx, StmtExecutor executor) throws Exception { executor.setPlanner(planner); executor.checkBlockRules(); + // FE-side deletion of existing files (before BE execution) + PhysicalPlan physicalPlan = planner.getPhysicalPlan(); + if (physicalPlan instanceof PhysicalTVFTableSink) { + PhysicalTVFTableSink tvfSink = (PhysicalTVFTableSink) physicalPlan; + String sinkTvfName = tvfSink.getTvfName(); + Map sinkProps = tvfSink.getProperties(); + boolean deleteExisting = Boolean.parseBoolean( + sinkProps.getOrDefault("delete_existing_files", "false")); + + if (deleteExisting && !"local".equals(sinkTvfName)) { + deleteExistingFilesInFE(sinkTvfName, sinkProps); + } + } + if (ctx.getConnectType() == ConnectType.MYSQL && ctx.getMysqlChannel() != null) { ctx.getMysqlChannel().reset(); } @@ -149,4 +171,39 @@ public R accept(PlanVisitor visitor, C context) { public Plan getExplainPlan(ConnectContext ctx) { return this.logicalQuery; } + + private void deleteExistingFilesInFE(String tvfName, Map props) + throws Exception { + String filePath = props.get("file_path"); + // Extract parent directory from prefix path: s3://bucket/path/to/prefix_ -> s3://bucket/path/to/ + String parentDir = extractParentDirectory(filePath); + LOG.info("TVF sink: deleting existing files in directory: {}", parentDir); + + // Copy props for building StorageProperties (exclude write-specific params) + Map fsCopyProps = new HashMap<>(props); + fsCopyProps.remove("file_path"); + fsCopyProps.remove("format"); + fsCopyProps.remove("delete_existing_files"); + fsCopyProps.remove("max_file_size"); + fsCopyProps.remove("column_separator"); + fsCopyProps.remove("line_delimiter"); + fsCopyProps.remove("compression_type"); + fsCopyProps.remove("compress_type"); + + StorageProperties storageProps = StorageProperties.createPrimary(fsCopyProps); + RemoteFileSystem fs = FileSystemFactory.get(storageProps); + org.apache.doris.backup.Status deleteStatus = fs.deleteDirectory(parentDir); + if (!deleteStatus.ok()) { + throw new UserException("Failed to delete existing files in " + + parentDir + ": " + deleteStatus.getErrMsg()); + } + } + + private static String extractParentDirectory(String prefixPath) { + int lastSlash = prefixPath.lastIndexOf('/'); + if (lastSlash >= 0) { + return prefixPath.substring(0, lastSlash + 1); + } + return prefixPath; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/TVFTableSink.java b/fe/fe-core/src/main/java/org/apache/doris/planner/TVFTableSink.java index 205a113a3064df..1d6d44d52aef61 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/TVFTableSink.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/TVFTableSink.java @@ -138,9 +138,8 @@ public void bindDataSink() throws AnalysisException { tSink.setMaxFileSizeBytes(Long.parseLong(maxFileSizeStr)); } - // Delete existing files flag - String deleteExisting = properties.getOrDefault("delete_existing_files", "true"); - tSink.setDeleteExistingFiles(Boolean.parseBoolean(deleteExisting)); + // Delete existing files is handled by FE (InsertIntoTVFCommand), always tell BE not to delete + tSink.setDeleteExistingFiles(false); // Backend id for local TVF String backendIdStr = properties.get("backend_id"); diff --git a/regression-test/suites/external_table_p0/tvf/insert/test_insert_into_hdfs_tvf.groovy b/regression-test/suites/external_table_p0/tvf/insert/test_insert_into_hdfs_tvf.groovy index bd4a1f9ce07f47..35da72c89d8975 100644 --- a/regression-test/suites/external_table_p0/tvf/insert/test_insert_into_hdfs_tvf.groovy +++ b/regression-test/suites/external_table_p0/tvf/insert/test_insert_into_hdfs_tvf.groovy @@ -29,6 +29,7 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { return } + // file_path is now a prefix; BE generates: {prefix}{query_id}_{idx}.{ext} def hdfsWriteProps = { String path, String format -> return """ "file_path" = "${hdfsBasePath}/${path}", @@ -38,6 +39,7 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { """ } + // Read uses wildcard to match generated file names def hdfsReadProps = { String path, String format -> return """ "uri" = "${defaultFS}${hdfsBasePath}/${path}", @@ -110,14 +112,14 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { sql """ INSERT INTO hdfs( - ${hdfsWriteProps("basic_csv.csv", "csv")}, + ${hdfsWriteProps("basic_csv/data_", "csv")}, "delete_existing_files" = "true" ) SELECT * FROM insert_tvf_test_src ORDER BY c_int; """ order_qt_hdfs_csv_basic_types """ SELECT * FROM hdfs( - ${hdfsReadProps("basic_csv.csv", "csv")} + ${hdfsReadProps("basic_csv/*", "csv")} ) ORDER BY c1; """ @@ -125,14 +127,14 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { sql """ INSERT INTO hdfs( - ${hdfsWriteProps("basic_parquet.parquet", "parquet")}, + ${hdfsWriteProps("basic_parquet/data_", "parquet")}, "delete_existing_files" = "true" ) SELECT * FROM insert_tvf_test_src ORDER BY c_int; """ order_qt_hdfs_parquet_basic_types """ SELECT * FROM hdfs( - ${hdfsReadProps("basic_parquet.parquet", "parquet")} + ${hdfsReadProps("basic_parquet/*", "parquet")} ) ORDER BY c_int; """ @@ -140,14 +142,14 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { sql """ INSERT INTO hdfs( - ${hdfsWriteProps("basic_orc.orc", "orc")}, + ${hdfsWriteProps("basic_orc/data_", "orc")}, "delete_existing_files" = "true" ) SELECT * FROM insert_tvf_test_src ORDER BY c_int; """ order_qt_hdfs_orc_basic_types """ SELECT * FROM hdfs( - ${hdfsReadProps("basic_orc.orc", "orc")} + ${hdfsReadProps("basic_orc/*", "orc")} ) ORDER BY c_int; """ @@ -155,14 +157,14 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { sql """ INSERT INTO hdfs( - ${hdfsWriteProps("complex_parquet.parquet", "parquet")}, + ${hdfsWriteProps("complex_parquet/data_", "parquet")}, "delete_existing_files" = "true" ) SELECT * FROM insert_tvf_complex_src ORDER BY c_int; """ order_qt_hdfs_parquet_complex_types """ SELECT * FROM hdfs( - ${hdfsReadProps("complex_parquet.parquet", "parquet")} + ${hdfsReadProps("complex_parquet/*", "parquet")} ) ORDER BY c_int; """ @@ -170,14 +172,14 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { sql """ INSERT INTO hdfs( - ${hdfsWriteProps("complex_orc.orc", "orc")}, + ${hdfsWriteProps("complex_orc/data_", "orc")}, "delete_existing_files" = "true" ) SELECT * FROM insert_tvf_complex_src ORDER BY c_int; """ order_qt_hdfs_orc_complex_types """ SELECT * FROM hdfs( - ${hdfsReadProps("complex_orc.orc", "orc")} + ${hdfsReadProps("complex_orc/*", "orc")} ) ORDER BY c_int; """ @@ -185,7 +187,7 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { sql """ INSERT INTO hdfs( - ${hdfsWriteProps("sep_comma.csv", "csv")}, + ${hdfsWriteProps("sep_comma/data_", "csv")}, "column_separator" = ",", "delete_existing_files" = "true" ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; @@ -193,7 +195,7 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { order_qt_hdfs_csv_sep_comma """ SELECT * FROM hdfs( - ${hdfsReadProps("sep_comma.csv", "csv")}, + ${hdfsReadProps("sep_comma/*", "csv")}, "column_separator" = "," ) ORDER BY c1; """ @@ -202,7 +204,7 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { sql """ INSERT INTO hdfs( - ${hdfsWriteProps("sep_tab.csv", "csv")}, + ${hdfsWriteProps("sep_tab/data_", "csv")}, "column_separator" = "\t", "delete_existing_files" = "true" ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; @@ -210,7 +212,7 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { order_qt_hdfs_csv_sep_tab """ SELECT * FROM hdfs( - ${hdfsReadProps("sep_tab.csv", "csv")}, + ${hdfsReadProps("sep_tab/*", "csv")}, "column_separator" = "\t" ) ORDER BY c1; """ @@ -219,7 +221,7 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { sql """ INSERT INTO hdfs( - ${hdfsWriteProps("sep_pipe.csv", "csv")}, + ${hdfsWriteProps("sep_pipe/data_", "csv")}, "column_separator" = "|", "delete_existing_files" = "true" ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; @@ -227,7 +229,7 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { order_qt_hdfs_csv_sep_pipe """ SELECT * FROM hdfs( - ${hdfsReadProps("sep_pipe.csv", "csv")}, + ${hdfsReadProps("sep_pipe/*", "csv")}, "column_separator" = "|" ) ORDER BY c1; """ @@ -236,7 +238,7 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { sql """ INSERT INTO hdfs( - ${hdfsWriteProps("sep_multi.csv", "csv")}, + ${hdfsWriteProps("sep_multi/data_", "csv")}, "column_separator" = ";;", "delete_existing_files" = "true" ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; @@ -244,7 +246,7 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { order_qt_hdfs_csv_sep_multi """ SELECT * FROM hdfs( - ${hdfsReadProps("sep_multi.csv", "csv")}, + ${hdfsReadProps("sep_multi/*", "csv")}, "column_separator" = ";;" ) ORDER BY c1; """ @@ -253,7 +255,7 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { sql """ INSERT INTO hdfs( - ${hdfsWriteProps("line_crlf.csv", "csv")}, + ${hdfsWriteProps("line_crlf/data_", "csv")}, "line_delimiter" = "\r\n", "delete_existing_files" = "true" ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; @@ -261,7 +263,7 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { order_qt_hdfs_csv_line_crlf """ SELECT * FROM hdfs( - ${hdfsReadProps("line_crlf.csv", "csv")}, + ${hdfsReadProps("line_crlf/*", "csv")}, "line_delimiter" = "\r\n" ) ORDER BY c1; """ @@ -270,7 +272,7 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { sql """ INSERT INTO hdfs( - ${hdfsWriteProps("compress_gz.csv.gz", "csv")}, + ${hdfsWriteProps("compress_gz/data_", "csv")}, "compression_type" = "gz", "delete_existing_files" = "true" ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; @@ -278,7 +280,7 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { order_qt_hdfs_csv_compress_gz """ SELECT * FROM hdfs( - ${hdfsReadProps("compress_gz.csv.gz", "csv")}, + ${hdfsReadProps("compress_gz/*", "csv")}, "compress_type" = "gz" ) ORDER BY c1; """ @@ -287,7 +289,7 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { sql """ INSERT INTO hdfs( - ${hdfsWriteProps("compress_zstd.csv.zst", "csv")}, + ${hdfsWriteProps("compress_zstd/data_", "csv")}, "compression_type" = "zstd", "delete_existing_files" = "true" ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; @@ -295,7 +297,7 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { order_qt_hdfs_csv_compress_zstd """ SELECT * FROM hdfs( - ${hdfsReadProps("compress_zstd.csv.zst", "csv")}, + ${hdfsReadProps("compress_zstd/*", "csv")}, "compress_type" = "zstd" ) ORDER BY c1; """ @@ -304,7 +306,7 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { sql """ INSERT INTO hdfs( - ${hdfsWriteProps("compress_lz4.csv.lz4", "csv")}, + ${hdfsWriteProps("compress_lz4/data_", "csv")}, "compression_type" = "lz4block", "delete_existing_files" = "true" ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; @@ -312,7 +314,7 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { order_qt_hdfs_csv_compress_lz4 """ SELECT * FROM hdfs( - ${hdfsReadProps("compress_lz4.csv.lz4", "csv")}, + ${hdfsReadProps("compress_lz4/*", "csv")}, "compress_type" = "lz4block" ) ORDER BY c1; """ @@ -321,7 +323,7 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { sql """ INSERT INTO hdfs( - ${hdfsWriteProps("compress_snappy.csv.snappy", "csv")}, + ${hdfsWriteProps("compress_snappy/data_", "csv")}, "compression_type" = "snappyblock", "delete_existing_files" = "true" ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; @@ -329,38 +331,39 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { order_qt_hdfs_csv_compress_snappy """ SELECT * FROM hdfs( - ${hdfsReadProps("compress_snappy.csv.snappy", "csv")}, + ${hdfsReadProps("compress_snappy/*", "csv")}, "compress_type" = "snappyblock" ) ORDER BY c1; """ // ============ 15. HDFS Overwrite mode ============ + // delete_existing_files=true is handled by FE for HDFS // First write: 5 rows sql """ INSERT INTO hdfs( - ${hdfsWriteProps("overwrite.csv", "csv")}, + ${hdfsWriteProps("overwrite/data_", "csv")}, "delete_existing_files" = "true" ) SELECT c_int, c_varchar FROM insert_tvf_test_src ORDER BY c_int; """ order_qt_hdfs_overwrite_first """ SELECT * FROM hdfs( - ${hdfsReadProps("overwrite.csv", "csv")} + ${hdfsReadProps("overwrite/*", "csv")} ) ORDER BY c1; """ - // Second write: 2 rows with overwrite + // Second write: 2 rows with overwrite (FE deletes the directory first) sql """ INSERT INTO hdfs( - ${hdfsWriteProps("overwrite.csv", "csv")}, + ${hdfsWriteProps("overwrite/data_", "csv")}, "delete_existing_files" = "true" ) SELECT c_int, c_varchar FROM insert_tvf_test_src WHERE c_int > 0 ORDER BY c_int LIMIT 2; """ order_qt_hdfs_overwrite_second """ SELECT * FROM hdfs( - ${hdfsReadProps("overwrite.csv", "csv")} + ${hdfsReadProps("overwrite/*", "csv")} ) ORDER BY c1; """ @@ -369,28 +372,27 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { // First write sql """ INSERT INTO hdfs( - ${hdfsWriteProps("append.parquet", "parquet")}, + ${hdfsWriteProps("append/data_", "parquet")}, "delete_existing_files" = "true" ) SELECT c_int, c_varchar FROM insert_tvf_test_src WHERE c_int = 1000; """ order_qt_hdfs_append_first """ SELECT * FROM hdfs( - ${hdfsReadProps("append.parquet", "parquet")} + ${hdfsReadProps("append/*", "parquet")} ) ORDER BY c_int; """ - // Second write (append) + // Second write (append — different query_id produces different file name) sql """ INSERT INTO hdfs( - ${hdfsWriteProps("append.parquet", "parquet")}, - "delete_existing_files" = "false" + ${hdfsWriteProps("append/data_", "parquet")} ) SELECT c_int, c_varchar FROM insert_tvf_test_src WHERE c_int = 2000; """ order_qt_hdfs_append_second """ SELECT * FROM hdfs( - ${hdfsReadProps("append.parquet", "parquet")} + ${hdfsReadProps("append/*", "parquet")} ) ORDER BY c_int; """ @@ -398,14 +400,14 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { sql """ INSERT INTO hdfs( - ${hdfsWriteProps("const_expr.csv", "csv")}, + ${hdfsWriteProps("const_expr/data_", "csv")}, "delete_existing_files" = "true" ) SELECT 1, 'hello', 3.14, CAST('2024-01-01' AS DATE); """ order_qt_hdfs_const_expr """ SELECT * FROM hdfs( - ${hdfsReadProps("const_expr.csv", "csv")} + ${hdfsReadProps("const_expr/*", "csv")} ) ORDER BY c1; """ @@ -413,14 +415,14 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { sql """ INSERT INTO hdfs( - ${hdfsWriteProps("where_groupby.csv", "csv")}, + ${hdfsWriteProps("where_groupby/data_", "csv")}, "delete_existing_files" = "true" ) SELECT c_bool, COUNT(*), SUM(c_int) FROM insert_tvf_test_src WHERE c_int IS NOT NULL GROUP BY c_bool ORDER BY c_bool; """ order_qt_hdfs_where_groupby """ SELECT * FROM hdfs( - ${hdfsReadProps("where_groupby.csv", "csv")} + ${hdfsReadProps("where_groupby/*", "csv")} ) ORDER BY c1; """ @@ -428,7 +430,7 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { sql """ INSERT INTO hdfs( - ${hdfsWriteProps("join_query.csv", "csv")}, + ${hdfsWriteProps("join_query/data_", "csv")}, "delete_existing_files" = "true" ) SELECT a.c_int, a.c_varchar, b.c_label FROM insert_tvf_test_src a INNER JOIN insert_tvf_join_src b ON a.c_int = b.c_int @@ -437,7 +439,7 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { order_qt_hdfs_join_query """ SELECT * FROM hdfs( - ${hdfsReadProps("join_query.csv", "csv")} + ${hdfsReadProps("join_query/*", "csv")} ) ORDER BY c1; """ @@ -445,14 +447,14 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { sql """ INSERT INTO hdfs( - ${hdfsWriteProps("subquery.csv", "csv")}, + ${hdfsWriteProps("subquery/data_", "csv")}, "delete_existing_files" = "true" ) SELECT * FROM (SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int) sub; """ order_qt_hdfs_subquery """ SELECT * FROM hdfs( - ${hdfsReadProps("subquery.csv", "csv")} + ${hdfsReadProps("subquery/*", "csv")} ) ORDER BY c1; """ @@ -460,7 +462,7 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { sql """ INSERT INTO hdfs( - ${hdfsWriteProps("type_cast.csv", "csv")}, + ${hdfsWriteProps("type_cast/data_", "csv")}, "delete_existing_files" = "true" ) SELECT CAST(c_int AS BIGINT), CAST(c_float AS DOUBLE), CAST(c_date AS STRING) FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; @@ -468,7 +470,7 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { order_qt_hdfs_type_cast """ SELECT * FROM hdfs( - ${hdfsReadProps("type_cast.csv", "csv")} + ${hdfsReadProps("type_cast/*", "csv")} ) ORDER BY c1; """ @@ -476,7 +478,7 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { sql """ INSERT INTO hdfs( - ${hdfsWriteProps("union_query.csv", "csv")}, + ${hdfsWriteProps("union_query/data_", "csv")}, "delete_existing_files" = "true" ) SELECT c_int, c_varchar FROM insert_tvf_test_src WHERE c_int = 1000 UNION ALL @@ -485,7 +487,7 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { order_qt_hdfs_union_query """ SELECT * FROM hdfs( - ${hdfsReadProps("union_query.csv", "csv")} + ${hdfsReadProps("union_query/*", "csv")} ) ORDER BY c1; """ @@ -507,7 +509,7 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { test { sql """ INSERT INTO hdfs( - "file_path" = "${hdfsBasePath}/err.csv", + "file_path" = "${hdfsBasePath}/err/data_", "hadoop.username" = "${hdfsUserName}", "fs.defaultFS" = "${defaultFS}" ) SELECT 1; @@ -520,7 +522,7 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { test { sql """ INSERT INTO hdfs( - "file_path" = "${hdfsBasePath}/err.json", + "file_path" = "${hdfsBasePath}/err/data_", "format" = "json", "hadoop.username" = "${hdfsUserName}", "fs.defaultFS" = "${defaultFS}" @@ -529,6 +531,20 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { exception "Unsupported" } + // ============ 26. Error: wildcard in file_path ============ + + test { + sql """ + INSERT INTO hdfs( + "file_path" = "${hdfsBasePath}/wildcard/*.csv", + "format" = "csv", + "hadoop.username" = "${hdfsUserName}", + "fs.defaultFS" = "${defaultFS}" + ) SELECT 1; + """ + exception "wildcards" + } + // ============ Cleanup ============ sql """ DROP TABLE IF EXISTS insert_tvf_test_src """ diff --git a/regression-test/suites/external_table_p0/tvf/insert/test_insert_into_local_tvf.groovy b/regression-test/suites/external_table_p0/tvf/insert/test_insert_into_local_tvf.groovy index 80dee8ff381bab..fe98120f29eee6 100644 --- a/regression-test/suites/external_table_p0/tvf/insert/test_insert_into_local_tvf.groovy +++ b/regression-test/suites/external_table_p0/tvf/insert/test_insert_into_local_tvf.groovy @@ -89,19 +89,22 @@ suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { sql """ INSERT INTO insert_tvf_join_src VALUES (1000, 'label_a'), (2000, 'label_b'), (3000, 'label_c'); """ // ============ 1. CSV basic types ============ + // file_path is a prefix; BE generates: {prefix}{query_id}_{idx}.{ext} + // Read back using wildcard on the prefix + + sshExec("root", be_host, "rm -f ${basePath}/basic_csv_*") sql """ INSERT INTO local( - "file_path" = "${basePath}/basic_csv.csv", + "file_path" = "${basePath}/basic_csv_", "backend_id" = "${be_id}", - "format" = "csv", - "delete_existing_files" = "true" + "format" = "csv" ) SELECT * FROM insert_tvf_test_src ORDER BY c_int; """ order_qt_csv_basic_types """ SELECT * FROM local( - "file_path" = "${basePath}/basic_csv.csv", + "file_path" = "${basePath}/basic_csv_*", "backend_id" = "${be_id}", "format" = "csv" ) ORDER BY c1; @@ -109,18 +112,19 @@ suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { // ============ 2. Parquet basic types ============ + sshExec("root", be_host, "rm -f ${basePath}/basic_parquet_*") + sql """ INSERT INTO local( - "file_path" = "${basePath}/basic_parquet.parquet", + "file_path" = "${basePath}/basic_parquet_", "backend_id" = "${be_id}", - "format" = "parquet", - "delete_existing_files" = "true" + "format" = "parquet" ) SELECT * FROM insert_tvf_test_src ORDER BY c_int; """ order_qt_parquet_basic_types """ SELECT * FROM local( - "file_path" = "${basePath}/basic_parquet.parquet", + "file_path" = "${basePath}/basic_parquet_*", "backend_id" = "${be_id}", "format" = "parquet" ) ORDER BY c_int; @@ -128,18 +132,19 @@ suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { // ============ 3. ORC basic types ============ + sshExec("root", be_host, "rm -f ${basePath}/basic_orc_*") + sql """ INSERT INTO local( - "file_path" = "${basePath}/basic_orc.orc", + "file_path" = "${basePath}/basic_orc_", "backend_id" = "${be_id}", - "format" = "orc", - "delete_existing_files" = "true" + "format" = "orc" ) SELECT * FROM insert_tvf_test_src ORDER BY c_int; """ order_qt_orc_basic_types """ SELECT * FROM local( - "file_path" = "${basePath}/basic_orc.orc", + "file_path" = "${basePath}/basic_orc_*", "backend_id" = "${be_id}", "format" = "orc" ) ORDER BY c_int; @@ -147,18 +152,19 @@ suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { // ============ 4. Parquet complex types ============ + sshExec("root", be_host, "rm -f ${basePath}/complex_parquet_*") + sql """ INSERT INTO local( - "file_path" = "${basePath}/complex_parquet.parquet", + "file_path" = "${basePath}/complex_parquet_", "backend_id" = "${be_id}", - "format" = "parquet", - "delete_existing_files" = "true" + "format" = "parquet" ) SELECT * FROM insert_tvf_complex_src ORDER BY c_int; """ order_qt_parquet_complex_types """ SELECT * FROM local( - "file_path" = "${basePath}/complex_parquet.parquet", + "file_path" = "${basePath}/complex_parquet_*", "backend_id" = "${be_id}", "format" = "parquet" ) ORDER BY c_int; @@ -166,18 +172,19 @@ suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { // ============ 5. ORC complex types ============ + sshExec("root", be_host, "rm -f ${basePath}/complex_orc_*") + sql """ INSERT INTO local( - "file_path" = "${basePath}/complex_orc.orc", + "file_path" = "${basePath}/complex_orc_", "backend_id" = "${be_id}", - "format" = "orc", - "delete_existing_files" = "true" + "format" = "orc" ) SELECT * FROM insert_tvf_complex_src ORDER BY c_int; """ order_qt_orc_complex_types """ SELECT * FROM local( - "file_path" = "${basePath}/complex_orc.orc", + "file_path" = "${basePath}/complex_orc_*", "backend_id" = "${be_id}", "format" = "orc" ) ORDER BY c_int; @@ -185,19 +192,20 @@ suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { // ============ 6. CSV separator: comma ============ + sshExec("root", be_host, "rm -f ${basePath}/sep_comma_*") + sql """ INSERT INTO local( - "file_path" = "${basePath}/sep_comma.csv", + "file_path" = "${basePath}/sep_comma_", "backend_id" = "${be_id}", "format" = "csv", - "column_separator" = ",", - "delete_existing_files" = "true" + "column_separator" = "," ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; """ order_qt_csv_sep_comma """ SELECT * FROM local( - "file_path" = "${basePath}/sep_comma.csv", + "file_path" = "${basePath}/sep_comma_*", "backend_id" = "${be_id}", "format" = "csv", "column_separator" = "," @@ -206,19 +214,20 @@ suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { // ============ 7. CSV separator: tab ============ + sshExec("root", be_host, "rm -f ${basePath}/sep_tab_*") + sql """ INSERT INTO local( - "file_path" = "${basePath}/sep_tab.csv", + "file_path" = "${basePath}/sep_tab_", "backend_id" = "${be_id}", "format" = "csv", - "column_separator" = "\t", - "delete_existing_files" = "true" + "column_separator" = "\t" ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; """ order_qt_csv_sep_tab """ SELECT * FROM local( - "file_path" = "${basePath}/sep_tab.csv", + "file_path" = "${basePath}/sep_tab_*", "backend_id" = "${be_id}", "format" = "csv", "column_separator" = "\t" @@ -227,19 +236,20 @@ suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { // ============ 8. CSV separator: pipe ============ + sshExec("root", be_host, "rm -f ${basePath}/sep_pipe_*") + sql """ INSERT INTO local( - "file_path" = "${basePath}/sep_pipe.csv", + "file_path" = "${basePath}/sep_pipe_", "backend_id" = "${be_id}", "format" = "csv", - "column_separator" = "|", - "delete_existing_files" = "true" + "column_separator" = "|" ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; """ order_qt_csv_sep_pipe """ SELECT * FROM local( - "file_path" = "${basePath}/sep_pipe.csv", + "file_path" = "${basePath}/sep_pipe_*", "backend_id" = "${be_id}", "format" = "csv", "column_separator" = "|" @@ -248,19 +258,20 @@ suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { // ============ 9. CSV separator: multi-char ============ + sshExec("root", be_host, "rm -f ${basePath}/sep_multi_*") + sql """ INSERT INTO local( - "file_path" = "${basePath}/sep_multi.csv", + "file_path" = "${basePath}/sep_multi_", "backend_id" = "${be_id}", "format" = "csv", - "column_separator" = ";;", - "delete_existing_files" = "true" + "column_separator" = ";;" ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; """ order_qt_csv_sep_multi """ SELECT * FROM local( - "file_path" = "${basePath}/sep_multi.csv", + "file_path" = "${basePath}/sep_multi_*", "backend_id" = "${be_id}", "format" = "csv", "column_separator" = ";;" @@ -269,19 +280,20 @@ suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { // ============ 10. CSV line delimiter: CRLF ============ + sshExec("root", be_host, "rm -f ${basePath}/line_crlf_*") + sql """ INSERT INTO local( - "file_path" = "${basePath}/line_crlf.csv", + "file_path" = "${basePath}/line_crlf_", "backend_id" = "${be_id}", "format" = "csv", - "line_delimiter" = "\r\n", - "delete_existing_files" = "true" + "line_delimiter" = "\r\n" ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; """ order_qt_csv_line_crlf """ SELECT * FROM local( - "file_path" = "${basePath}/line_crlf.csv", + "file_path" = "${basePath}/line_crlf_*", "backend_id" = "${be_id}", "format" = "csv", "line_delimiter" = "\r\n" @@ -290,19 +302,20 @@ suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { // ============ 11. CSV compress: gz ============ + sshExec("root", be_host, "rm -f ${basePath}/compress_gz_*") + sql """ INSERT INTO local( - "file_path" = "${basePath}/compress_gz.csv.gz", + "file_path" = "${basePath}/compress_gz_", "backend_id" = "${be_id}", "format" = "csv", - "compression_type" = "gz", - "delete_existing_files" = "true" + "compression_type" = "gz" ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; """ order_qt_csv_compress_gz """ SELECT * FROM local( - "file_path" = "${basePath}/compress_gz.csv.gz", + "file_path" = "${basePath}/compress_gz_*", "backend_id" = "${be_id}", "format" = "csv", "compress_type" = "gz" @@ -311,19 +324,20 @@ suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { // ============ 12. CSV compress: zstd ============ + sshExec("root", be_host, "rm -f ${basePath}/compress_zstd_*") + sql """ INSERT INTO local( - "file_path" = "${basePath}/compress_zstd.csv.zst", + "file_path" = "${basePath}/compress_zstd_", "backend_id" = "${be_id}", "format" = "csv", - "compression_type" = "zstd", - "delete_existing_files" = "true" + "compression_type" = "zstd" ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; """ order_qt_csv_compress_zstd """ SELECT * FROM local( - "file_path" = "${basePath}/compress_zstd.csv.zst", + "file_path" = "${basePath}/compress_zstd_*", "backend_id" = "${be_id}", "format" = "csv", "compress_type" = "zstd" @@ -333,19 +347,20 @@ suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { // ============ 13. CSV compress: lz4 ============ // TODO: lz4 read meet error: LZ4F_getFrameInfo error: ERROR_frameType_unknown + sshExec("root", be_host, "rm -f ${basePath}/compress_lz4_*") + sql """ INSERT INTO local( - "file_path" = "${basePath}/compress_lz4.csv.lz4", + "file_path" = "${basePath}/compress_lz4_", "backend_id" = "${be_id}", "format" = "csv", - "compression_type" = "lz4block", - "delete_existing_files" = "true" + "compression_type" = "lz4block" ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; """ order_qt_csv_compress_lz4 """ SELECT * FROM local( - "file_path" = "${basePath}/compress_lz4.csv.lz4", + "file_path" = "${basePath}/compress_lz4_*", "backend_id" = "${be_id}", "format" = "csv", "compress_type" = "lz4block" @@ -354,96 +369,100 @@ suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { // ============ 14. CSV compress: snappy ============ + sshExec("root", be_host, "rm -f ${basePath}/compress_snappy_*") + sql """ INSERT INTO local( - "file_path" = "${basePath}/compress_snappy.csv.snappy", + "file_path" = "${basePath}/compress_snappy_", "backend_id" = "${be_id}", "format" = "csv", - "compression_type" = "snappyblock", - "delete_existing_files" = "true" + "compression_type" = "snappyblock" ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; """ order_qt_csv_compress_snappy """ SELECT * FROM local( - "file_path" = "${basePath}/compress_snappy.csv.snappy", + "file_path" = "${basePath}/compress_snappy_*", "backend_id" = "${be_id}", "format" = "csv", "compress_type" = "snappyblock" ) ORDER BY c1; """ - // ============ 15. Overwrite mode (delete_existing_files=true) ============ + // ============ 15. Overwrite mode ============ + // local TVF does not support delete_existing_files=true, so use shell cleanup to simulate overwrite // First write: 5 rows + sshExec("root", be_host, "rm -f ${basePath}/overwrite_*") + sql """ INSERT INTO local( - "file_path" = "${basePath}/overwrite.csv", + "file_path" = "${basePath}/overwrite_", "backend_id" = "${be_id}", - "format" = "csv", - "delete_existing_files" = "true" + "format" = "csv" ) SELECT c_int, c_varchar FROM insert_tvf_test_src ORDER BY c_int; """ order_qt_overwrite_first """ SELECT * FROM local( - "file_path" = "${basePath}/overwrite.csv", + "file_path" = "${basePath}/overwrite_*", "backend_id" = "${be_id}", "format" = "csv" ) ORDER BY c1; """ - // Second write: 2 rows with overwrite + // Clean files via shell, then write 2 rows + sshExec("root", be_host, "rm -f ${basePath}/overwrite_*") + sql """ INSERT INTO local( - "file_path" = "${basePath}/overwrite.csv", + "file_path" = "${basePath}/overwrite_", "backend_id" = "${be_id}", - "format" = "csv", - "delete_existing_files" = "true" + "format" = "csv" ) SELECT c_int, c_varchar FROM insert_tvf_test_src WHERE c_int > 0 ORDER BY c_int LIMIT 2; """ order_qt_overwrite_second """ SELECT * FROM local( - "file_path" = "${basePath}/overwrite.csv", + "file_path" = "${basePath}/overwrite_*", "backend_id" = "${be_id}", "format" = "csv" ) ORDER BY c1; """ - // ============ 16. Append mode (delete_existing_files=false) ============ + // ============ 16. Append mode (default, delete_existing_files=false) ============ + + sshExec("root", be_host, "rm -f ${basePath}/append_*") // First write sql """ INSERT INTO local( - "file_path" = "${basePath}/append.parquet", + "file_path" = "${basePath}/append_", "backend_id" = "${be_id}", - "format" = "parquet", - "delete_existing_files" = "true" + "format" = "parquet" ) SELECT c_int, c_varchar FROM insert_tvf_test_src WHERE c_int = 1000; """ order_qt_append_first """ SELECT * FROM local( - "file_path" = "${basePath}/append.parquet", + "file_path" = "${basePath}/append_*", "backend_id" = "${be_id}", "format" = "parquet" ) ORDER BY c_int; """ - // Second write (append) + // Second write (append — different query_id produces different file name) sql """ INSERT INTO local( - "file_path" = "${basePath}/append.parquet", + "file_path" = "${basePath}/append_", "backend_id" = "${be_id}", - "format" = "parquet", - "delete_existing_files" = "false" + "format" = "parquet" ) SELECT c_int, c_varchar FROM insert_tvf_test_src WHERE c_int = 2000; """ order_qt_append_second """ SELECT * FROM local( - "file_path" = "${basePath}/append.parquet", + "file_path" = "${basePath}/append_*", "backend_id" = "${be_id}", "format" = "parquet" ) ORDER BY c_int; @@ -451,18 +470,19 @@ suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { // ============ 17. Complex SELECT: constant expressions ============ + sshExec("root", be_host, "rm -f ${basePath}/const_expr_*") + sql """ INSERT INTO local( - "file_path" = "${basePath}/const_expr.csv", + "file_path" = "${basePath}/const_expr_", "backend_id" = "${be_id}", - "format" = "csv", - "delete_existing_files" = "true" + "format" = "csv" ) SELECT 1, 'hello', 3.14, CAST('2024-01-01' AS DATE); """ order_qt_const_expr """ SELECT * FROM local( - "file_path" = "${basePath}/const_expr.csv", + "file_path" = "${basePath}/const_expr_*", "backend_id" = "${be_id}", "format" = "csv" ) ORDER BY c1; @@ -470,18 +490,19 @@ suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { // ============ 18. Complex SELECT: WHERE + GROUP BY ============ + sshExec("root", be_host, "rm -f ${basePath}/where_groupby_*") + sql """ INSERT INTO local( - "file_path" = "${basePath}/where_groupby.csv", + "file_path" = "${basePath}/where_groupby_", "backend_id" = "${be_id}", - "format" = "csv", - "delete_existing_files" = "true" + "format" = "csv" ) SELECT c_bool, COUNT(*), SUM(c_int) FROM insert_tvf_test_src WHERE c_int IS NOT NULL GROUP BY c_bool ORDER BY c_bool; """ order_qt_where_groupby """ SELECT * FROM local( - "file_path" = "${basePath}/where_groupby.csv", + "file_path" = "${basePath}/where_groupby_*", "backend_id" = "${be_id}", "format" = "csv" ) ORDER BY c1; @@ -489,12 +510,13 @@ suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { // ============ 19. Complex SELECT: JOIN ============ + sshExec("root", be_host, "rm -f ${basePath}/join_query_*") + sql """ INSERT INTO local( - "file_path" = "${basePath}/join_query.csv", + "file_path" = "${basePath}/join_query_", "backend_id" = "${be_id}", - "format" = "csv", - "delete_existing_files" = "true" + "format" = "csv" ) SELECT a.c_int, a.c_varchar, b.c_label FROM insert_tvf_test_src a INNER JOIN insert_tvf_join_src b ON a.c_int = b.c_int ORDER BY a.c_int; @@ -502,7 +524,7 @@ suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { order_qt_join_query """ SELECT * FROM local( - "file_path" = "${basePath}/join_query.csv", + "file_path" = "${basePath}/join_query_*", "backend_id" = "${be_id}", "format" = "csv" ) ORDER BY c1; @@ -510,18 +532,19 @@ suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { // ============ 20. Complex SELECT: subquery ============ + sshExec("root", be_host, "rm -f ${basePath}/subquery_*") + sql """ INSERT INTO local( - "file_path" = "${basePath}/subquery.csv", + "file_path" = "${basePath}/subquery_", "backend_id" = "${be_id}", - "format" = "csv", - "delete_existing_files" = "true" + "format" = "csv" ) SELECT * FROM (SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int) sub; """ order_qt_subquery """ SELECT * FROM local( - "file_path" = "${basePath}/subquery.csv", + "file_path" = "${basePath}/subquery_*", "backend_id" = "${be_id}", "format" = "csv" ) ORDER BY c1; @@ -529,19 +552,20 @@ suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { // ============ 21. Complex SELECT: type cast ============ + sshExec("root", be_host, "rm -f ${basePath}/type_cast_*") + sql """ INSERT INTO local( - "file_path" = "${basePath}/type_cast.csv", + "file_path" = "${basePath}/type_cast_", "backend_id" = "${be_id}", - "format" = "csv", - "delete_existing_files" = "true" + "format" = "csv" ) SELECT CAST(c_int AS BIGINT), CAST(c_float AS DOUBLE), CAST(c_date AS STRING) FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; """ order_qt_type_cast """ SELECT * FROM local( - "file_path" = "${basePath}/type_cast.csv", + "file_path" = "${basePath}/type_cast_*", "backend_id" = "${be_id}", "format" = "csv" ) ORDER BY c1; @@ -549,12 +573,13 @@ suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { // ============ 22. Complex SELECT: UNION ALL ============ + sshExec("root", be_host, "rm -f ${basePath}/union_query_*") + sql """ INSERT INTO local( - "file_path" = "${basePath}/union_query.csv", + "file_path" = "${basePath}/union_query_", "backend_id" = "${be_id}", - "format" = "csv", - "delete_existing_files" = "true" + "format" = "csv" ) SELECT c_int, c_varchar FROM insert_tvf_test_src WHERE c_int = 1000 UNION ALL SELECT c_int, c_varchar FROM insert_tvf_test_src WHERE c_int = 2000; @@ -562,7 +587,7 @@ suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { order_qt_union_query """ SELECT * FROM local( - "file_path" = "${basePath}/union_query.csv", + "file_path" = "${basePath}/union_query_*", "backend_id" = "${be_id}", "format" = "csv" ) ORDER BY c1; @@ -585,7 +610,7 @@ suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { test { sql """ INSERT INTO local( - "file_path" = "${basePath}/err.csv", + "file_path" = "${basePath}/err_", "backend_id" = "${be_id}" ) SELECT 1; """ @@ -597,7 +622,7 @@ suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { test { sql """ INSERT INTO local( - "file_path" = "${basePath}/err.csv", + "file_path" = "${basePath}/err_", "format" = "csv" ) SELECT 1; """ @@ -609,7 +634,7 @@ suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { test { sql """ INSERT INTO unknown_tvf( - "file_path" = "/tmp/err.csv", + "file_path" = "/tmp/err_", "format" = "csv" ) SELECT 1; """ @@ -621,7 +646,7 @@ suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { test { sql """ INSERT INTO local( - "file_path" = "${basePath}/err.json", + "file_path" = "${basePath}/err_", "backend_id" = "${be_id}", "format" = "json" ) SELECT 1; @@ -629,6 +654,33 @@ suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { exception "Unsupported" } + // ============ 28. Error: wildcard in file_path ============ + + test { + sql """ + INSERT INTO local( + "file_path" = "${basePath}/wildcard_*.csv", + "backend_id" = "${be_id}", + "format" = "csv" + ) SELECT 1; + """ + exception "wildcards" + } + + // ============ 29. Error: delete_existing_files=true on local TVF ============ + + test { + sql """ + INSERT INTO local( + "file_path" = "${basePath}/err_", + "backend_id" = "${be_id}", + "format" = "csv", + "delete_existing_files" = "true" + ) SELECT 1; + """ + exception "delete_existing_files" + } + // ============ Cleanup ============ sshExec("root", be_host, "rm -rf ${basePath}", false) diff --git a/regression-test/suites/external_table_p0/tvf/insert/test_insert_into_s3_tvf.groovy b/regression-test/suites/external_table_p0/tvf/insert/test_insert_into_s3_tvf.groovy index 8b1c198a2c7f3d..b2824fd5cffe63 100644 --- a/regression-test/suites/external_table_p0/tvf/insert/test_insert_into_s3_tvf.groovy +++ b/regression-test/suites/external_table_p0/tvf/insert/test_insert_into_s3_tvf.groovy @@ -30,6 +30,7 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { def s3BasePath = "${bucket}/regression/insert_tvf_test" + // file_path is now a prefix; BE generates: {prefix}{query_id}_{idx}.{ext} def s3WriteProps = { String path, String format -> return """ "file_path" = "s3://${s3BasePath}/${path}", @@ -41,6 +42,7 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { """ } + // Read uses wildcard to match generated file names def s3ReadProps = { String path, String format -> return """ "uri" = "https://${bucket}.${s3_endpoint}/regression/insert_tvf_test/${path}", @@ -115,14 +117,14 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { sql """ INSERT INTO s3( - ${s3WriteProps("basic_csv.csv", "csv")}, + ${s3WriteProps("basic_csv/data_", "csv")}, "delete_existing_files" = "true" ) SELECT * FROM insert_tvf_test_src ORDER BY c_int; """ order_qt_s3_csv_basic_types """ SELECT * FROM s3( - ${s3ReadProps("basic_csv.csv", "csv")} + ${s3ReadProps("basic_csv/*", "csv")} ) ORDER BY c1; """ @@ -130,14 +132,14 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { sql """ INSERT INTO s3( - ${s3WriteProps("basic_parquet.parquet", "parquet")}, + ${s3WriteProps("basic_parquet/data_", "parquet")}, "delete_existing_files" = "true" ) SELECT * FROM insert_tvf_test_src ORDER BY c_int; """ order_qt_s3_parquet_basic_types """ SELECT * FROM s3( - ${s3ReadProps("basic_parquet.parquet", "parquet")} + ${s3ReadProps("basic_parquet/*", "parquet")} ) ORDER BY c_int; """ @@ -145,14 +147,14 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { sql """ INSERT INTO s3( - ${s3WriteProps("basic_orc.orc", "orc")}, + ${s3WriteProps("basic_orc/data_", "orc")}, "delete_existing_files" = "true" ) SELECT * FROM insert_tvf_test_src ORDER BY c_int; """ order_qt_s3_orc_basic_types """ SELECT * FROM s3( - ${s3ReadProps("basic_orc.orc", "orc")} + ${s3ReadProps("basic_orc/*", "orc")} ) ORDER BY c_int; """ @@ -160,14 +162,14 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { sql """ INSERT INTO s3( - ${s3WriteProps("complex_parquet.parquet", "parquet")}, + ${s3WriteProps("complex_parquet/data_", "parquet")}, "delete_existing_files" = "true" ) SELECT * FROM insert_tvf_complex_src ORDER BY c_int; """ order_qt_s3_parquet_complex_types """ SELECT * FROM s3( - ${s3ReadProps("complex_parquet.parquet", "parquet")} + ${s3ReadProps("complex_parquet/*", "parquet")} ) ORDER BY c_int; """ @@ -175,14 +177,14 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { sql """ INSERT INTO s3( - ${s3WriteProps("complex_orc.orc", "orc")}, + ${s3WriteProps("complex_orc/data_", "orc")}, "delete_existing_files" = "true" ) SELECT * FROM insert_tvf_complex_src ORDER BY c_int; """ order_qt_s3_orc_complex_types """ SELECT * FROM s3( - ${s3ReadProps("complex_orc.orc", "orc")} + ${s3ReadProps("complex_orc/*", "orc")} ) ORDER BY c_int; """ @@ -190,7 +192,7 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { sql """ INSERT INTO s3( - ${s3WriteProps("sep_comma.csv", "csv")}, + ${s3WriteProps("sep_comma/data_", "csv")}, "column_separator" = ",", "delete_existing_files" = "true" ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; @@ -198,7 +200,7 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { order_qt_s3_csv_sep_comma """ SELECT * FROM s3( - ${s3ReadProps("sep_comma.csv", "csv")}, + ${s3ReadProps("sep_comma/*", "csv")}, "column_separator" = "," ) ORDER BY c1; """ @@ -207,7 +209,7 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { sql """ INSERT INTO s3( - ${s3WriteProps("sep_tab.csv", "csv")}, + ${s3WriteProps("sep_tab/data_", "csv")}, "column_separator" = "\t", "delete_existing_files" = "true" ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; @@ -215,7 +217,7 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { order_qt_s3_csv_sep_tab """ SELECT * FROM s3( - ${s3ReadProps("sep_tab.csv", "csv")}, + ${s3ReadProps("sep_tab/*", "csv")}, "column_separator" = "\t" ) ORDER BY c1; """ @@ -224,7 +226,7 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { sql """ INSERT INTO s3( - ${s3WriteProps("sep_pipe.csv", "csv")}, + ${s3WriteProps("sep_pipe/data_", "csv")}, "column_separator" = "|", "delete_existing_files" = "true" ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; @@ -232,7 +234,7 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { order_qt_s3_csv_sep_pipe """ SELECT * FROM s3( - ${s3ReadProps("sep_pipe.csv", "csv")}, + ${s3ReadProps("sep_pipe/*", "csv")}, "column_separator" = "|" ) ORDER BY c1; """ @@ -241,7 +243,7 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { sql """ INSERT INTO s3( - ${s3WriteProps("sep_multi.csv", "csv")}, + ${s3WriteProps("sep_multi/data_", "csv")}, "column_separator" = ";;", "delete_existing_files" = "true" ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; @@ -249,7 +251,7 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { order_qt_s3_csv_sep_multi """ SELECT * FROM s3( - ${s3ReadProps("sep_multi.csv", "csv")}, + ${s3ReadProps("sep_multi/*", "csv")}, "column_separator" = ";;" ) ORDER BY c1; """ @@ -258,7 +260,7 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { sql """ INSERT INTO s3( - ${s3WriteProps("line_crlf.csv", "csv")}, + ${s3WriteProps("line_crlf/data_", "csv")}, "line_delimiter" = "\r\n", "delete_existing_files" = "true" ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; @@ -266,7 +268,7 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { order_qt_s3_csv_line_crlf """ SELECT * FROM s3( - ${s3ReadProps("line_crlf.csv", "csv")}, + ${s3ReadProps("line_crlf/*", "csv")}, "line_delimiter" = "\r\n" ) ORDER BY c1; """ @@ -275,7 +277,7 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { sql """ INSERT INTO s3( - ${s3WriteProps("compress_gz.csv.gz", "csv")}, + ${s3WriteProps("compress_gz/data_", "csv")}, "compression_type" = "gz", "delete_existing_files" = "true" ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; @@ -283,7 +285,7 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { order_qt_s3_csv_compress_gz """ SELECT * FROM s3( - ${s3ReadProps("compress_gz.csv.gz", "csv")}, + ${s3ReadProps("compress_gz/*", "csv")}, "compress_type" = "gz" ) ORDER BY c1; """ @@ -292,7 +294,7 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { sql """ INSERT INTO s3( - ${s3WriteProps("compress_zstd.csv.zst", "csv")}, + ${s3WriteProps("compress_zstd/data_", "csv")}, "compression_type" = "zstd", "delete_existing_files" = "true" ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; @@ -300,7 +302,7 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { order_qt_s3_csv_compress_zstd """ SELECT * FROM s3( - ${s3ReadProps("compress_zstd.csv.zst", "csv")}, + ${s3ReadProps("compress_zstd/*", "csv")}, "compress_type" = "zstd" ) ORDER BY c1; """ @@ -309,7 +311,7 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { sql """ INSERT INTO s3( - ${s3WriteProps("compress_lz4.csv.lz4", "csv")}, + ${s3WriteProps("compress_lz4/data_", "csv")}, "compression_type" = "lz4block", "delete_existing_files" = "true" ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; @@ -317,7 +319,7 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { order_qt_s3_csv_compress_lz4 """ SELECT * FROM s3( - ${s3ReadProps("compress_lz4.csv.lz4", "csv")}, + ${s3ReadProps("compress_lz4/*", "csv")}, "compress_type" = "lz4block" ) ORDER BY c1; """ @@ -326,7 +328,7 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { sql """ INSERT INTO s3( - ${s3WriteProps("compress_snappy.csv.snappy", "csv")}, + ${s3WriteProps("compress_snappy/data_", "csv")}, "compression_type" = "snappyblock", "delete_existing_files" = "true" ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; @@ -334,38 +336,39 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { order_qt_s3_csv_compress_snappy """ SELECT * FROM s3( - ${s3ReadProps("compress_snappy.csv.snappy", "csv")}, + ${s3ReadProps("compress_snappy/*", "csv")}, "compress_type" = "snappyblock" ) ORDER BY c1; """ // ============ 15. S3 Overwrite mode ============ + // delete_existing_files=true is handled by FE for S3 // First write: 5 rows sql """ INSERT INTO s3( - ${s3WriteProps("overwrite.csv", "csv")}, + ${s3WriteProps("overwrite/data_", "csv")}, "delete_existing_files" = "true" ) SELECT c_int, c_varchar FROM insert_tvf_test_src ORDER BY c_int; """ order_qt_s3_overwrite_first """ SELECT * FROM s3( - ${s3ReadProps("overwrite.csv", "csv")} + ${s3ReadProps("overwrite/*", "csv")} ) ORDER BY c1; """ - // Second write: 2 rows with overwrite + // Second write: 2 rows with overwrite (FE deletes the directory first) sql """ INSERT INTO s3( - ${s3WriteProps("overwrite.csv", "csv")}, + ${s3WriteProps("overwrite/data_", "csv")}, "delete_existing_files" = "true" ) SELECT c_int, c_varchar FROM insert_tvf_test_src WHERE c_int > 0 ORDER BY c_int LIMIT 2; """ order_qt_s3_overwrite_second """ SELECT * FROM s3( - ${s3ReadProps("overwrite.csv", "csv")} + ${s3ReadProps("overwrite/*", "csv")} ) ORDER BY c1; """ @@ -374,28 +377,27 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { // First write sql """ INSERT INTO s3( - ${s3WriteProps("append.parquet", "parquet")}, + ${s3WriteProps("append/data_", "parquet")}, "delete_existing_files" = "true" ) SELECT c_int, c_varchar FROM insert_tvf_test_src WHERE c_int = 1000; """ order_qt_s3_append_first """ SELECT * FROM s3( - ${s3ReadProps("append.parquet", "parquet")} + ${s3ReadProps("append/*", "parquet")} ) ORDER BY c_int; """ - // Second write (append) + // Second write (append — different query_id produces different file name) sql """ INSERT INTO s3( - ${s3WriteProps("append.parquet", "parquet")}, - "delete_existing_files" = "false" + ${s3WriteProps("append/data_", "parquet")} ) SELECT c_int, c_varchar FROM insert_tvf_test_src WHERE c_int = 2000; """ order_qt_s3_append_second """ SELECT * FROM s3( - ${s3ReadProps("append.parquet", "parquet")} + ${s3ReadProps("append/*", "parquet")} ) ORDER BY c_int; """ @@ -403,14 +405,14 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { sql """ INSERT INTO s3( - ${s3WriteProps("const_expr.csv", "csv")}, + ${s3WriteProps("const_expr/data_", "csv")}, "delete_existing_files" = "true" ) SELECT 1, 'hello', 3.14, CAST('2024-01-01' AS DATE); """ order_qt_s3_const_expr """ SELECT * FROM s3( - ${s3ReadProps("const_expr.csv", "csv")} + ${s3ReadProps("const_expr/*", "csv")} ) ORDER BY c1; """ @@ -418,14 +420,14 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { sql """ INSERT INTO s3( - ${s3WriteProps("where_groupby.csv", "csv")}, + ${s3WriteProps("where_groupby/data_", "csv")}, "delete_existing_files" = "true" ) SELECT c_bool, COUNT(*), SUM(c_int) FROM insert_tvf_test_src WHERE c_int IS NOT NULL GROUP BY c_bool ORDER BY c_bool; """ order_qt_s3_where_groupby """ SELECT * FROM s3( - ${s3ReadProps("where_groupby.csv", "csv")} + ${s3ReadProps("where_groupby/*", "csv")} ) ORDER BY c1; """ @@ -433,7 +435,7 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { sql """ INSERT INTO s3( - ${s3WriteProps("join_query.csv", "csv")}, + ${s3WriteProps("join_query/data_", "csv")}, "delete_existing_files" = "true" ) SELECT a.c_int, a.c_varchar, b.c_label FROM insert_tvf_test_src a INNER JOIN insert_tvf_join_src b ON a.c_int = b.c_int @@ -442,7 +444,7 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { order_qt_s3_join_query """ SELECT * FROM s3( - ${s3ReadProps("join_query.csv", "csv")} + ${s3ReadProps("join_query/*", "csv")} ) ORDER BY c1; """ @@ -450,14 +452,14 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { sql """ INSERT INTO s3( - ${s3WriteProps("subquery.csv", "csv")}, + ${s3WriteProps("subquery/data_", "csv")}, "delete_existing_files" = "true" ) SELECT * FROM (SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int) sub; """ order_qt_s3_subquery """ SELECT * FROM s3( - ${s3ReadProps("subquery.csv", "csv")} + ${s3ReadProps("subquery/*", "csv")} ) ORDER BY c1; """ @@ -465,7 +467,7 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { sql """ INSERT INTO s3( - ${s3WriteProps("type_cast.csv", "csv")}, + ${s3WriteProps("type_cast/data_", "csv")}, "delete_existing_files" = "true" ) SELECT CAST(c_int AS BIGINT), CAST(c_float AS DOUBLE), CAST(c_date AS STRING) FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; @@ -473,7 +475,7 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { order_qt_s3_type_cast """ SELECT * FROM s3( - ${s3ReadProps("type_cast.csv", "csv")} + ${s3ReadProps("type_cast/*", "csv")} ) ORDER BY c1; """ @@ -481,7 +483,7 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { sql """ INSERT INTO s3( - ${s3WriteProps("union_query.csv", "csv")}, + ${s3WriteProps("union_query/data_", "csv")}, "delete_existing_files" = "true" ) SELECT c_int, c_varchar FROM insert_tvf_test_src WHERE c_int = 1000 UNION ALL @@ -490,7 +492,7 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { order_qt_s3_union_query """ SELECT * FROM s3( - ${s3ReadProps("union_query.csv", "csv")} + ${s3ReadProps("union_query/*", "csv")} ) ORDER BY c1; """ @@ -514,7 +516,7 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { test { sql """ INSERT INTO s3( - "file_path" = "s3://${s3BasePath}/err.csv", + "file_path" = "s3://${s3BasePath}/err/data_", "s3.endpoint" = "${s3_endpoint}", "s3.access_key" = "${ak}", "s3.secret_key" = "${sk}", @@ -529,7 +531,7 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { test { sql """ INSERT INTO s3( - "file_path" = "s3://${s3BasePath}/err.json", + "file_path" = "s3://${s3BasePath}/err/data_", "format" = "json", "s3.endpoint" = "${s3_endpoint}", "s3.access_key" = "${ak}", @@ -540,6 +542,22 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { exception "Unsupported" } + // ============ 26. Error: wildcard in file_path ============ + + test { + sql """ + INSERT INTO s3( + "file_path" = "s3://${s3BasePath}/wildcard/*.csv", + "format" = "csv", + "s3.endpoint" = "${s3_endpoint}", + "s3.access_key" = "${ak}", + "s3.secret_key" = "${sk}", + "s3.region" = "${region}" + ) SELECT 1; + """ + exception "wildcards" + } + // ============ Cleanup ============ sql """ DROP TABLE IF EXISTS insert_tvf_test_src """ From d719863767b52c20b0277147d3907a4a22c0e39a Mon Sep 17 00:00:00 2001 From: morningman Date: Fri, 13 Feb 2026 08:20:11 +0800 Subject: [PATCH 08/13] format --- .../doris/nereids/glue/translator/PhysicalPlanTranslator.java | 4 ++-- .../org/apache/doris/nereids/parser/LogicalPlanBuilder.java | 4 ++-- .../src/main/java/org/apache/doris/nereids/rules/RuleSet.java | 2 +- .../org/apache/doris/nereids/rules/analysis/BindSink.java | 1 - .../trees/plans/commands/insert/InsertIntoTVFCommand.java | 2 +- 5 files changed, 6 insertions(+), 7 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java index 3e8bc18a7ec857..f99f1b337e7ccf 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java @@ -119,7 +119,6 @@ import org.apache.doris.nereids.trees.plans.physical.AbstractPhysicalSort; import org.apache.doris.nereids.trees.plans.physical.PhysicalAssertNumRows; import org.apache.doris.nereids.trees.plans.physical.PhysicalBlackholeSink; -import org.apache.doris.nereids.trees.plans.physical.PhysicalTVFTableSink; import org.apache.doris.nereids.trees.plans.physical.PhysicalCTEAnchor; import org.apache.doris.nereids.trees.plans.physical.PhysicalCTEConsumer; import org.apache.doris.nereids.trees.plans.physical.PhysicalCTEProducer; @@ -166,6 +165,7 @@ import org.apache.doris.nereids.trees.plans.physical.PhysicalSetOperation; import org.apache.doris.nereids.trees.plans.physical.PhysicalStorageLayerAggregate; import org.apache.doris.nereids.trees.plans.physical.PhysicalTVFRelation; +import org.apache.doris.nereids.trees.plans.physical.PhysicalTVFTableSink; import org.apache.doris.nereids.trees.plans.physical.PhysicalTopN; import org.apache.doris.nereids.trees.plans.physical.PhysicalUnion; import org.apache.doris.nereids.trees.plans.physical.PhysicalWindow; @@ -186,7 +186,6 @@ import org.apache.doris.planner.AssertNumRowsNode; import org.apache.doris.planner.BackendPartitionedSchemaScanNode; import org.apache.doris.planner.BlackholeSink; -import org.apache.doris.planner.TVFTableSink; import org.apache.doris.planner.CTEScanNode; import org.apache.doris.planner.DataPartition; import org.apache.doris.planner.DataStreamSink; @@ -220,6 +219,7 @@ import org.apache.doris.planner.SelectNode; import org.apache.doris.planner.SetOperationNode; import org.apache.doris.planner.SortNode; +import org.apache.doris.planner.TVFTableSink; import org.apache.doris.planner.TableFunctionNode; import org.apache.doris.planner.UnionNode; import org.apache.doris.qe.ConnectContext; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index ec15a689d25226..758e103bf092db 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -489,7 +489,6 @@ import org.apache.doris.nereids.analyzer.UnboundAlias; import org.apache.doris.nereids.analyzer.UnboundBlackholeSink; import org.apache.doris.nereids.analyzer.UnboundBlackholeSink.UnboundBlackholeSinkContext; -import org.apache.doris.nereids.analyzer.UnboundTVFTableSink; import org.apache.doris.nereids.analyzer.UnboundFunction; import org.apache.doris.nereids.analyzer.UnboundInlineTable; import org.apache.doris.nereids.analyzer.UnboundOneRowRelation; @@ -498,6 +497,7 @@ import org.apache.doris.nereids.analyzer.UnboundSlot; import org.apache.doris.nereids.analyzer.UnboundStar; import org.apache.doris.nereids.analyzer.UnboundTVFRelation; +import org.apache.doris.nereids.analyzer.UnboundTVFTableSink; import org.apache.doris.nereids.analyzer.UnboundTableSinkCreator; import org.apache.doris.nereids.analyzer.UnboundVariable; import org.apache.doris.nereids.analyzer.UnboundVariable.VariableType; @@ -1001,8 +1001,8 @@ import org.apache.doris.nereids.trees.plans.commands.info.TagOptions; import org.apache.doris.nereids.trees.plans.commands.info.WarmUpItem; import org.apache.doris.nereids.trees.plans.commands.insert.BatchInsertIntoTableCommand; -import org.apache.doris.nereids.trees.plans.commands.insert.InsertIntoTableCommand; import org.apache.doris.nereids.trees.plans.commands.insert.InsertIntoTVFCommand; +import org.apache.doris.nereids.trees.plans.commands.insert.InsertIntoTableCommand; import org.apache.doris.nereids.trees.plans.commands.insert.InsertOverwriteTableCommand; import org.apache.doris.nereids.trees.plans.commands.insert.WarmupSelectCommand; import org.apache.doris.nereids.trees.plans.commands.load.CreateRoutineLoadCommand; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleSet.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleSet.java index 0ea4455d043a8e..3b46251c426b0d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleSet.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleSet.java @@ -60,7 +60,6 @@ import org.apache.doris.nereids.rules.implementation.AggregateStrategies; import org.apache.doris.nereids.rules.implementation.LogicalAssertNumRowsToPhysicalAssertNumRows; import org.apache.doris.nereids.rules.implementation.LogicalBlackholeSinkToPhysicalBlackholeSink; -import org.apache.doris.nereids.rules.implementation.LogicalTVFTableSinkToPhysicalTVFTableSink; import org.apache.doris.nereids.rules.implementation.LogicalCTEAnchorToPhysicalCTEAnchor; import org.apache.doris.nereids.rules.implementation.LogicalCTEConsumerToPhysicalCTEConsumer; import org.apache.doris.nereids.rules.implementation.LogicalCTEProducerToPhysicalCTEProducer; @@ -98,6 +97,7 @@ import org.apache.doris.nereids.rules.implementation.LogicalSchemaScanToPhysicalSchemaScan; import org.apache.doris.nereids.rules.implementation.LogicalSortToPhysicalQuickSort; import org.apache.doris.nereids.rules.implementation.LogicalTVFRelationToPhysicalTVFRelation; +import org.apache.doris.nereids.rules.implementation.LogicalTVFTableSinkToPhysicalTVFTableSink; import org.apache.doris.nereids.rules.implementation.LogicalTopNToPhysicalTopN; import org.apache.doris.nereids.rules.implementation.LogicalUnionToPhysicalUnion; import org.apache.doris.nereids.rules.implementation.LogicalWindowToPhysicalWindow; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindSink.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindSink.java index 4b5874c61e68fa..d7197a577bf9e4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindSink.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindSink.java @@ -97,7 +97,6 @@ import org.apache.doris.qe.AutoCloseSessionVariable; import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.SessionVariable; - import org.apache.doris.thrift.TPartialUpdateNewRowPolicy; import com.google.common.base.Preconditions; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertIntoTVFCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertIntoTVFCommand.java index c35df25d98d1b3..4847cd028fa3bb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertIntoTVFCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertIntoTVFCommand.java @@ -27,7 +27,6 @@ import org.apache.doris.fs.FileSystemFactory; import org.apache.doris.fs.remote.RemoteFileSystem; import org.apache.doris.mysql.privilege.PrivPredicate; -import org.apache.doris.task.LoadEtlTask; import org.apache.doris.nereids.NereidsPlanner; import org.apache.doris.nereids.glue.LogicalPlanAdapter; import org.apache.doris.nereids.trees.plans.Explainable; @@ -45,6 +44,7 @@ import org.apache.doris.qe.QeProcessorImpl; import org.apache.doris.qe.QeProcessorImpl.QueryInfo; import org.apache.doris.qe.StmtExecutor; +import org.apache.doris.task.LoadEtlTask; import org.apache.doris.thrift.TStatusCode; import org.apache.doris.thrift.TUniqueId; From bd2e21082652c3bc0b88c2181f53de6ebcb9e3fc Mon Sep 17 00:00:00 2001 From: morningman Date: Fri, 13 Feb 2026 09:20:50 +0800 Subject: [PATCH 09/13] format --- .licenserc.yaml | 1 + .../doris/nereids/analyzer/UnboundTVFTableSink.java | 10 +++++++++- 2 files changed, 10 insertions(+), 1 deletion(-) diff --git a/.licenserc.yaml b/.licenserc.yaml index ef44aa8e5818ad..947e0873afeb14 100644 --- a/.licenserc.yaml +++ b/.licenserc.yaml @@ -47,6 +47,7 @@ header: - "fe/fe-core/src/main/java/software/amazon/awssdk/core/client/builder/SdkDefaultClientBuilder.java" - "fe/fe-core/src/main/antlr4/org/apache/doris/nereids/JavaLexer.g4" - "fe/fe-core/src/main/antlr4/org/apache/doris/nereids/JavaParser.g4" + - "fe/.idea/vcs.xml" - "be/dict/ik/*" - "be/dict/pinyin/*" - "be/src/common/signal_handler.h" diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundTVFTableSink.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundTVFTableSink.java index cfb44ffd27cfe3..0ac62b26dfaefd 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundTVFTableSink.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundTVFTableSink.java @@ -47,6 +47,14 @@ public class UnboundTVFTableSink extends UnboundLogical private final String tvfName; private final Map properties; + /** + * For insert into tvf + * + * @param tvfName + * @param properties + * @param dmlCommandType + * @param child + */ public UnboundTVFTableSink(String tvfName, Map properties, DMLCommandType dmlCommandType, CHILD_TYPE child) { super(ImmutableList.of(tvfName), @@ -61,7 +69,7 @@ public UnboundTVFTableSink(String tvfName, Map properties, this.properties = properties; } - public UnboundTVFTableSink(String tvfName, Map properties, + private UnboundTVFTableSink(String tvfName, Map properties, DMLCommandType dmlCommandType, Optional groupExpression, Optional logicalProperties, CHILD_TYPE child) { From 2b8beda20aec116d41de255e5edbe0f3deb38325 Mon Sep 17 00:00:00 2001 From: morningman Date: Fri, 13 Feb 2026 10:21:20 +0800 Subject: [PATCH 10/13] refactor --- be/src/vec/runtime/vcsv_transformer.h | 4 +- .../vfile_format_transformer_factory.cpp | 85 +++++++++++++++++++ .../vfile_format_transformer_factory.h | 43 ++++++++++ be/src/vec/runtime/vparquet_transformer.cpp | 9 +- be/src/vec/runtime/vparquet_transformer.h | 4 +- be/src/vec/sink/writer/vtvf_table_writer.cpp | 67 ++------------- be/src/vec/sink/writer/vtvf_table_writer.h | 6 +- .../tvf/insert/test_insert_into_hdfs_tvf.out | 1 + .../tvf/insert/test_insert_into_local_tvf.out | 3 +- .../tvf/insert/test_insert_into_s3_tvf.out | 1 + 10 files changed, 149 insertions(+), 74 deletions(-) create mode 100644 be/src/vec/runtime/vfile_format_transformer_factory.cpp create mode 100644 be/src/vec/runtime/vfile_format_transformer_factory.h diff --git a/be/src/vec/runtime/vcsv_transformer.h b/be/src/vec/runtime/vcsv_transformer.h index abfe5c3198858b..cb09bf8568898f 100644 --- a/be/src/vec/runtime/vcsv_transformer.h +++ b/be/src/vec/runtime/vcsv_transformer.h @@ -65,8 +65,8 @@ class VCSVTransformer final : public VFileFormatTransformer { std::string _gen_csv_header_types(); std::string _csv_header; - std::string_view _column_separator; - std::string_view _line_delimiter; + std::string _column_separator; + std::string _line_delimiter; doris::io::FileWriter* _file_writer = nullptr; // Used to buffer the export data of plain text diff --git a/be/src/vec/runtime/vfile_format_transformer_factory.cpp b/be/src/vec/runtime/vfile_format_transformer_factory.cpp new file mode 100644 index 00000000000000..bbd415e2452e28 --- /dev/null +++ b/be/src/vec/runtime/vfile_format_transformer_factory.cpp @@ -0,0 +1,85 @@ +// 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 "vec/runtime/vfile_format_transformer_factory.h" + +#include + +#include +#include + +#include "vec/runtime/vcsv_transformer.h" +#include "vec/runtime/vorc_transformer.h" +#include "vec/runtime/vparquet_transformer.h" + +namespace doris::vectorized { + +Status create_tvf_format_transformer(const TTVFTableSink& tvf_sink, RuntimeState* state, + io::FileWriter* file_writer, + const VExprContextSPtrs& output_vexpr_ctxs, + std::unique_ptr* result) { + TFileFormatType::type format = tvf_sink.file_format; + switch (format) { + case TFileFormatType::FORMAT_CSV_PLAIN: { + std::string column_separator = + tvf_sink.__isset.column_separator ? tvf_sink.column_separator : ","; + std::string line_delimiter = + tvf_sink.__isset.line_delimiter ? tvf_sink.line_delimiter : "\n"; + TFileCompressType::type compress_type = TFileCompressType::PLAIN; + if (tvf_sink.__isset.compression_type) { + compress_type = tvf_sink.compression_type; + } + result->reset(new VCSVTransformer(state, file_writer, output_vexpr_ctxs, false, {}, {}, + column_separator, line_delimiter, false, compress_type)); + break; + } + case TFileFormatType::FORMAT_PARQUET: { + std::vector parquet_schemas; + if (tvf_sink.__isset.columns) { + for (const auto& col : tvf_sink.columns) { + TParquetSchema schema; + schema.__set_schema_column_name(col.column_name); + parquet_schemas.push_back(schema); + } + } + result->reset(new VParquetTransformer( + state, file_writer, output_vexpr_ctxs, parquet_schemas, false, + {TParquetCompressionType::SNAPPY, TParquetVersion::PARQUET_1_0, false, false})); + break; + } + case TFileFormatType::FORMAT_ORC: { + TFileCompressType::type compress_type = TFileCompressType::PLAIN; + if (tvf_sink.__isset.compression_type) { + compress_type = tvf_sink.compression_type; + } + std::vector orc_column_names; + if (tvf_sink.__isset.columns) { + for (const auto& col : tvf_sink.columns) { + orc_column_names.push_back(col.column_name); + } + } + result->reset(new VOrcTransformer(state, file_writer, output_vexpr_ctxs, "", + orc_column_names, false, compress_type)); + break; + } + default: + return Status::InternalError("Unsupported TVF sink file format: {}", format); + } + return Status::OK(); +} + +} // namespace doris::vectorized diff --git a/be/src/vec/runtime/vfile_format_transformer_factory.h b/be/src/vec/runtime/vfile_format_transformer_factory.h new file mode 100644 index 00000000000000..2a690db93bcd32 --- /dev/null +++ b/be/src/vec/runtime/vfile_format_transformer_factory.h @@ -0,0 +1,43 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#pragma once + +#include + +#include + +#include "common/status.h" +#include "vec/exprs/vexpr_fwd.h" + +namespace doris { +class RuntimeState; +namespace io { +class FileWriter; +} // namespace io +} // namespace doris + +namespace doris::vectorized { + +class VFileFormatTransformer; + +Status create_tvf_format_transformer(const TTVFTableSink& tvf_sink, RuntimeState* state, + io::FileWriter* file_writer, + const VExprContextSPtrs& output_vexpr_ctxs, + std::unique_ptr* result); + +} // namespace doris::vectorized diff --git a/be/src/vec/runtime/vparquet_transformer.cpp b/be/src/vec/runtime/vparquet_transformer.cpp index e9f5549b5103eb..d75513b0b0ac1d 100644 --- a/be/src/vec/runtime/vparquet_transformer.cpp +++ b/be/src/vec/runtime/vparquet_transformer.cpp @@ -165,7 +165,6 @@ VParquetTransformer::VParquetTransformer(RuntimeState* state, doris::io::FileWri const iceberg::Schema* iceberg_schema) : VFileFormatTransformer(state, output_vexpr_ctxs, output_object_data), _column_names(std::move(column_names)), - _parquet_schemas(nullptr), _parquet_options(parquet_options), _iceberg_schema_json(iceberg_schema_json), _iceberg_schema(iceberg_schema) { @@ -174,12 +173,12 @@ VParquetTransformer::VParquetTransformer(RuntimeState* state, doris::io::FileWri VParquetTransformer::VParquetTransformer(RuntimeState* state, doris::io::FileWriter* file_writer, const VExprContextSPtrs& output_vexpr_ctxs, - const std::vector& parquet_schemas, + std::vector parquet_schemas, bool output_object_data, const ParquetFileOptions& parquet_options, const std::string* iceberg_schema_json) : VFileFormatTransformer(state, output_vexpr_ctxs, output_object_data), - _parquet_schemas(&parquet_schemas), + _parquet_schemas(std::move(parquet_schemas)), _parquet_options(parquet_options), _iceberg_schema_json(iceberg_schema_json) { _iceberg_schema = nullptr; @@ -228,9 +227,9 @@ Status VParquetTransformer::_parse_schema() { std::shared_ptr type; RETURN_IF_ERROR(convert_to_arrow_type(_output_vexpr_ctxs[i]->root()->data_type(), &type, _state->timezone())); - if (_parquet_schemas != nullptr) { + if (!_parquet_schemas.empty()) { std::shared_ptr field = - arrow::field(_parquet_schemas->operator[](i).schema_column_name, type, + arrow::field(_parquet_schemas[i].schema_column_name, type, _output_vexpr_ctxs[i]->root()->is_nullable()); fields.emplace_back(field); } else { diff --git a/be/src/vec/runtime/vparquet_transformer.h b/be/src/vec/runtime/vparquet_transformer.h index 7ae58aff74d4b7..b654b3314146a2 100644 --- a/be/src/vec/runtime/vparquet_transformer.h +++ b/be/src/vec/runtime/vparquet_transformer.h @@ -97,7 +97,7 @@ class VParquetTransformer final : public VFileFormatTransformer { VParquetTransformer(RuntimeState* state, doris::io::FileWriter* file_writer, const VExprContextSPtrs& output_vexpr_ctxs, - const std::vector& parquet_schemas, bool output_object_data, + std::vector parquet_schemas, bool output_object_data, const ParquetFileOptions& parquet_options, const std::string* iceberg_schema_json = nullptr); @@ -123,7 +123,7 @@ class VParquetTransformer final : public VFileFormatTransformer { std::shared_ptr _arrow_schema; std::vector _column_names; - const std::vector* _parquet_schemas = nullptr; + std::vector _parquet_schemas; const ParquetFileOptions _parquet_options; const std::string* _iceberg_schema_json; uint64_t _write_size = 0; diff --git a/be/src/vec/sink/writer/vtvf_table_writer.cpp b/be/src/vec/sink/writer/vtvf_table_writer.cpp index e001461d570203..7d82e8ef471d2e 100644 --- a/be/src/vec/sink/writer/vtvf_table_writer.cpp +++ b/be/src/vec/sink/writer/vtvf_table_writer.cpp @@ -18,7 +18,6 @@ #include "vec/sink/writer/vtvf_table_writer.h" #include -#include #include "common/status.h" #include "io/file_factory.h" @@ -26,9 +25,6 @@ #include "vec/core/block.h" #include "vec/exprs/vexpr.h" #include "vec/exprs/vexpr_context.h" -#include "vec/runtime/vcsv_transformer.h" -#include "vec/runtime/vorc_transformer.h" -#include "vec/runtime/vparquet_transformer.h" namespace doris::vectorized { @@ -97,62 +93,15 @@ Status VTVFTableWriter::_create_file_writer(const std::string& file_name) { properties = _tvf_sink.properties; } - _file_writer_impl = DORIS_TRY(FileFactory::create_file_writer(file_type, _state->exec_env(), {}, - properties, file_name, - { - .write_file_cache = false, - .sync_file_data = false, - })); - - TFileFormatType::type format = _tvf_sink.file_format; - switch (format) { - case TFileFormatType::FORMAT_CSV_PLAIN: { - _column_separator = _tvf_sink.__isset.column_separator ? _tvf_sink.column_separator : ","; - _line_delimiter = _tvf_sink.__isset.line_delimiter ? _tvf_sink.line_delimiter : "\n"; - TFileCompressType::type compress_type = TFileCompressType::PLAIN; - if (_tvf_sink.__isset.compression_type) { - compress_type = _tvf_sink.compression_type; - } - _vfile_writer.reset(new VCSVTransformer( - _state, _file_writer_impl.get(), _vec_output_expr_ctxs, false, {}, {}, - _column_separator, _line_delimiter, false, compress_type)); - break; - } - case TFileFormatType::FORMAT_PARQUET: { - _parquet_schemas.clear(); - if (_tvf_sink.__isset.columns) { - for (const auto& col : _tvf_sink.columns) { - TParquetSchema schema; - schema.__set_schema_column_name(col.column_name); - _parquet_schemas.push_back(schema); - } - } - _vfile_writer.reset(new VParquetTransformer( - _state, _file_writer_impl.get(), _vec_output_expr_ctxs, _parquet_schemas, false, - {TParquetCompressionType::SNAPPY, TParquetVersion::PARQUET_1_0, false, false})); - break; - } - case TFileFormatType::FORMAT_ORC: { - TFileCompressType::type compress_type = TFileCompressType::PLAIN; - if (_tvf_sink.__isset.compression_type) { - compress_type = _tvf_sink.compression_type; - } - _orc_column_names.clear(); - if (_tvf_sink.__isset.columns) { - for (const auto& col : _tvf_sink.columns) { - _orc_column_names.push_back(col.column_name); - } - } - _vfile_writer.reset(new VOrcTransformer(_state, _file_writer_impl.get(), - _vec_output_expr_ctxs, "", _orc_column_names, false, - compress_type)); - break; - } - default: - return Status::InternalError("Unsupported TVF sink file format: {}", format); - } + _file_writer_impl = DORIS_TRY(FileFactory::create_file_writer( + file_type, _state->exec_env(), {}, properties, file_name, + {.write_file_cache = false, .sync_file_data = false})); + + RETURN_IF_ERROR(create_tvf_format_transformer(_tvf_sink, _state, _file_writer_impl.get(), + _vec_output_expr_ctxs, &_vfile_writer)); - VLOG_DEBUG << "TVF table writer created file: " << file_name << ", format=" << format + VLOG_DEBUG << "TVF table writer created file: " << file_name + << ", format=" << _tvf_sink.file_format << ", query_id=" << print_id(_state->query_id()); return _vfile_writer->open(); diff --git a/be/src/vec/sink/writer/vtvf_table_writer.h b/be/src/vec/sink/writer/vtvf_table_writer.h index 17c3fc7a159e3e..5c69b4354aed55 100644 --- a/be/src/vec/sink/writer/vtvf_table_writer.h +++ b/be/src/vec/sink/writer/vtvf_table_writer.h @@ -18,7 +18,6 @@ #pragma once #include -#include #include #include @@ -28,6 +27,7 @@ #include "util/runtime_profile.h" #include "vec/exprs/vexpr_fwd.h" #include "vec/runtime/vfile_format_transformer.h" +#include "vec/runtime/vfile_format_transformer_factory.h" #include "vec/sink/writer/async_result_writer.h" namespace doris { @@ -75,10 +75,6 @@ class VTVFTableWriter final : public AsyncResultWriter { int64_t _max_file_size_bytes = 0; int _file_idx = 0; std::string _file_path; - std::string _column_separator; - std::string _line_delimiter; - std::vector _parquet_schemas; - std::vector _orc_column_names; // profile counters RuntimeProfile::Counter* _written_rows_counter = nullptr; diff --git a/regression-test/data/external_table_p0/tvf/insert/test_insert_into_hdfs_tvf.out b/regression-test/data/external_table_p0/tvf/insert/test_insert_into_hdfs_tvf.out index 1999c61a06e2a0..f1a9d8fe85e64c 100644 --- a/regression-test/data/external_table_p0/tvf/insert/test_insert_into_hdfs_tvf.out +++ b/regression-test/data/external_table_p0/tvf/insert/test_insert_into_hdfs_tvf.out @@ -101,6 +101,7 @@ true 3 300 3000 300000 5.5 6.6 999.99 2024-12-31 2024-12-31T23:59:59 test data 1000 hello -- !hdfs_append_second -- +1000 hello 2000 foo -- !hdfs_const_expr -- diff --git a/regression-test/data/external_table_p0/tvf/insert/test_insert_into_local_tvf.out b/regression-test/data/external_table_p0/tvf/insert/test_insert_into_local_tvf.out index 5058c9346c59c6..e05a406c439bde 100644 --- a/regression-test/data/external_table_p0/tvf/insert/test_insert_into_local_tvf.out +++ b/regression-test/data/external_table_p0/tvf/insert/test_insert_into_local_tvf.out @@ -101,7 +101,8 @@ true 3 300 3000 300000 5.5 6.6 999.99 2024-12-31 2024-12-31T23:59:59 test data 1000 hello -- !append_second -- -1000 foo +1000 hello +2000 foo -- !const_expr -- 1 hello 3.14 2024-01-01 diff --git a/regression-test/data/external_table_p0/tvf/insert/test_insert_into_s3_tvf.out b/regression-test/data/external_table_p0/tvf/insert/test_insert_into_s3_tvf.out index 60632c73288b1a..71260ab859bf0b 100644 --- a/regression-test/data/external_table_p0/tvf/insert/test_insert_into_s3_tvf.out +++ b/regression-test/data/external_table_p0/tvf/insert/test_insert_into_s3_tvf.out @@ -102,6 +102,7 @@ true 3 300 3000 300000 5.5 6.6 999.99 2024-12-31 2024-12-31T23:59:59 test data -- !s3_append_second -- 1000 hello +2000 foo -- !s3_const_expr -- 1 hello 3.14 2024-01-01 From 884ce04912a6c377656267a64b2c5b54ff9e08f8 Mon Sep 17 00:00:00 2001 From: morningman Date: Fri, 13 Feb 2026 10:32:56 +0800 Subject: [PATCH 11/13] format --- .../apache/doris/nereids/analyzer/UnboundTVFTableSink.java | 5 ----- 1 file changed, 5 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundTVFTableSink.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundTVFTableSink.java index 0ac62b26dfaefd..ae2031b30fd60b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundTVFTableSink.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundTVFTableSink.java @@ -49,11 +49,6 @@ public class UnboundTVFTableSink extends UnboundLogical /** * For insert into tvf - * - * @param tvfName - * @param properties - * @param dmlCommandType - * @param child */ public UnboundTVFTableSink(String tvfName, Map properties, DMLCommandType dmlCommandType, CHILD_TYPE child) { From 83884bc523ac235b514126e55eeaaff3b315b014 Mon Sep 17 00:00:00 2001 From: morningman Date: Fri, 13 Feb 2026 16:09:11 +0800 Subject: [PATCH 12/13] fix cases --- .../insert/test_insert_into_hdfs_tvf.groovy | 72 +++++++++--------- .../insert/test_insert_into_local_tvf.groovy | 74 +++++++++---------- .../tvf/insert/test_insert_into_s3_tvf.groovy | 74 +++++++++---------- 3 files changed, 110 insertions(+), 110 deletions(-) diff --git a/regression-test/suites/external_table_p0/tvf/insert/test_insert_into_hdfs_tvf.groovy b/regression-test/suites/external_table_p0/tvf/insert/test_insert_into_hdfs_tvf.groovy index 35da72c89d8975..dd3a03bc154e19 100644 --- a/regression-test/suites/external_table_p0/tvf/insert/test_insert_into_hdfs_tvf.groovy +++ b/regression-test/suites/external_table_p0/tvf/insert/test_insert_into_hdfs_tvf.groovy @@ -50,9 +50,9 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { // ============ Source tables ============ - sql """ DROP TABLE IF EXISTS insert_tvf_test_src """ + sql """ DROP TABLE IF EXISTS test_insert_into_hdfs_tvf_src """ sql """ - CREATE TABLE IF NOT EXISTS insert_tvf_test_src ( + CREATE TABLE IF NOT EXISTS test_insert_into_hdfs_tvf_src ( c_bool BOOLEAN, c_tinyint TINYINT, c_smallint SMALLINT, @@ -70,7 +70,7 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { """ sql """ - INSERT INTO insert_tvf_test_src VALUES + INSERT INTO test_insert_into_hdfs_tvf_src VALUES (true, 1, 100, 1000, 100000, 1.1, 2.2, 123.45, '2024-01-01', '2024-01-01 10:00:00', 'hello', 'world'), (false, 2, 200, 2000, 200000, 3.3, 4.4, 678.90, '2024-06-15', '2024-06-15 12:30:00', 'foo', 'bar'), (true, 3, 300, 3000, 300000, 5.5, 6.6, 999.99, '2024-12-31', '2024-12-31 23:59:59', 'test', 'data'), @@ -78,9 +78,9 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { (false, -1, -100, -1000, -100000, -1.1, -2.2, -123.45,'2020-02-29', '2020-02-29 00:00:00', '', 'special_chars'); """ - sql """ DROP TABLE IF EXISTS insert_tvf_complex_src """ + sql """ DROP TABLE IF EXISTS test_insert_into_hdfs_tvf_complex_src """ sql """ - CREATE TABLE IF NOT EXISTS insert_tvf_complex_src ( + CREATE TABLE IF NOT EXISTS test_insert_into_hdfs_tvf_complex_src ( c_int INT, c_array ARRAY, c_map MAP, @@ -90,23 +90,23 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { """ sql """ - INSERT INTO insert_tvf_complex_src VALUES + INSERT INTO test_insert_into_hdfs_tvf_complex_src VALUES (1, [1, 2, 3], {'a': 1, 'b': 2}, {1, 'hello'}), (2, [4, 5], {'x': 10}, {2, 'world'}), (3, [], {}, {3, ''}), (4, NULL, NULL, NULL); """ - sql """ DROP TABLE IF EXISTS insert_tvf_join_src """ + sql """ DROP TABLE IF EXISTS test_insert_into_hdfs_tvf_join_src """ sql """ - CREATE TABLE IF NOT EXISTS insert_tvf_join_src ( + CREATE TABLE IF NOT EXISTS test_insert_into_hdfs_tvf_join_src ( c_int INT, c_label STRING ) DISTRIBUTED BY HASH(c_int) BUCKETS 1 PROPERTIES("replication_num" = "1"); """ - sql """ INSERT INTO insert_tvf_join_src VALUES (1000, 'label_a'), (2000, 'label_b'), (3000, 'label_c'); """ + sql """ INSERT INTO test_insert_into_hdfs_tvf_join_src VALUES (1000, 'label_a'), (2000, 'label_b'), (3000, 'label_c'); """ // ============ 1. HDFS CSV basic types ============ @@ -114,7 +114,7 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { INSERT INTO hdfs( ${hdfsWriteProps("basic_csv/data_", "csv")}, "delete_existing_files" = "true" - ) SELECT * FROM insert_tvf_test_src ORDER BY c_int; + ) SELECT * FROM test_insert_into_hdfs_tvf_src ORDER BY c_int; """ order_qt_hdfs_csv_basic_types """ @@ -129,7 +129,7 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { INSERT INTO hdfs( ${hdfsWriteProps("basic_parquet/data_", "parquet")}, "delete_existing_files" = "true" - ) SELECT * FROM insert_tvf_test_src ORDER BY c_int; + ) SELECT * FROM test_insert_into_hdfs_tvf_src ORDER BY c_int; """ order_qt_hdfs_parquet_basic_types """ @@ -144,7 +144,7 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { INSERT INTO hdfs( ${hdfsWriteProps("basic_orc/data_", "orc")}, "delete_existing_files" = "true" - ) SELECT * FROM insert_tvf_test_src ORDER BY c_int; + ) SELECT * FROM test_insert_into_hdfs_tvf_src ORDER BY c_int; """ order_qt_hdfs_orc_basic_types """ @@ -159,7 +159,7 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { INSERT INTO hdfs( ${hdfsWriteProps("complex_parquet/data_", "parquet")}, "delete_existing_files" = "true" - ) SELECT * FROM insert_tvf_complex_src ORDER BY c_int; + ) SELECT * FROM test_insert_into_hdfs_tvf_complex_src ORDER BY c_int; """ order_qt_hdfs_parquet_complex_types """ @@ -174,7 +174,7 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { INSERT INTO hdfs( ${hdfsWriteProps("complex_orc/data_", "orc")}, "delete_existing_files" = "true" - ) SELECT * FROM insert_tvf_complex_src ORDER BY c_int; + ) SELECT * FROM test_insert_into_hdfs_tvf_complex_src ORDER BY c_int; """ order_qt_hdfs_orc_complex_types """ @@ -190,7 +190,7 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { ${hdfsWriteProps("sep_comma/data_", "csv")}, "column_separator" = ",", "delete_existing_files" = "true" - ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; + ) SELECT c_int, c_varchar, c_string FROM test_insert_into_hdfs_tvf_src WHERE c_int IS NOT NULL ORDER BY c_int; """ order_qt_hdfs_csv_sep_comma """ @@ -207,7 +207,7 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { ${hdfsWriteProps("sep_tab/data_", "csv")}, "column_separator" = "\t", "delete_existing_files" = "true" - ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; + ) SELECT c_int, c_varchar, c_string FROM test_insert_into_hdfs_tvf_src WHERE c_int IS NOT NULL ORDER BY c_int; """ order_qt_hdfs_csv_sep_tab """ @@ -224,7 +224,7 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { ${hdfsWriteProps("sep_pipe/data_", "csv")}, "column_separator" = "|", "delete_existing_files" = "true" - ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; + ) SELECT c_int, c_varchar, c_string FROM test_insert_into_hdfs_tvf_src WHERE c_int IS NOT NULL ORDER BY c_int; """ order_qt_hdfs_csv_sep_pipe """ @@ -241,7 +241,7 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { ${hdfsWriteProps("sep_multi/data_", "csv")}, "column_separator" = ";;", "delete_existing_files" = "true" - ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; + ) SELECT c_int, c_varchar, c_string FROM test_insert_into_hdfs_tvf_src WHERE c_int IS NOT NULL ORDER BY c_int; """ order_qt_hdfs_csv_sep_multi """ @@ -258,7 +258,7 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { ${hdfsWriteProps("line_crlf/data_", "csv")}, "line_delimiter" = "\r\n", "delete_existing_files" = "true" - ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; + ) SELECT c_int, c_varchar, c_string FROM test_insert_into_hdfs_tvf_src WHERE c_int IS NOT NULL ORDER BY c_int; """ order_qt_hdfs_csv_line_crlf """ @@ -275,7 +275,7 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { ${hdfsWriteProps("compress_gz/data_", "csv")}, "compression_type" = "gz", "delete_existing_files" = "true" - ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; + ) SELECT c_int, c_varchar, c_string FROM test_insert_into_hdfs_tvf_src WHERE c_int IS NOT NULL ORDER BY c_int; """ order_qt_hdfs_csv_compress_gz """ @@ -292,7 +292,7 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { ${hdfsWriteProps("compress_zstd/data_", "csv")}, "compression_type" = "zstd", "delete_existing_files" = "true" - ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; + ) SELECT c_int, c_varchar, c_string FROM test_insert_into_hdfs_tvf_src WHERE c_int IS NOT NULL ORDER BY c_int; """ order_qt_hdfs_csv_compress_zstd """ @@ -309,7 +309,7 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { ${hdfsWriteProps("compress_lz4/data_", "csv")}, "compression_type" = "lz4block", "delete_existing_files" = "true" - ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; + ) SELECT c_int, c_varchar, c_string FROM test_insert_into_hdfs_tvf_src WHERE c_int IS NOT NULL ORDER BY c_int; """ order_qt_hdfs_csv_compress_lz4 """ @@ -326,7 +326,7 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { ${hdfsWriteProps("compress_snappy/data_", "csv")}, "compression_type" = "snappyblock", "delete_existing_files" = "true" - ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; + ) SELECT c_int, c_varchar, c_string FROM test_insert_into_hdfs_tvf_src WHERE c_int IS NOT NULL ORDER BY c_int; """ order_qt_hdfs_csv_compress_snappy """ @@ -344,7 +344,7 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { INSERT INTO hdfs( ${hdfsWriteProps("overwrite/data_", "csv")}, "delete_existing_files" = "true" - ) SELECT c_int, c_varchar FROM insert_tvf_test_src ORDER BY c_int; + ) SELECT c_int, c_varchar FROM test_insert_into_hdfs_tvf_src ORDER BY c_int; """ order_qt_hdfs_overwrite_first """ @@ -358,7 +358,7 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { INSERT INTO hdfs( ${hdfsWriteProps("overwrite/data_", "csv")}, "delete_existing_files" = "true" - ) SELECT c_int, c_varchar FROM insert_tvf_test_src WHERE c_int > 0 ORDER BY c_int LIMIT 2; + ) SELECT c_int, c_varchar FROM test_insert_into_hdfs_tvf_src WHERE c_int > 0 ORDER BY c_int LIMIT 2; """ order_qt_hdfs_overwrite_second """ @@ -374,7 +374,7 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { INSERT INTO hdfs( ${hdfsWriteProps("append/data_", "parquet")}, "delete_existing_files" = "true" - ) SELECT c_int, c_varchar FROM insert_tvf_test_src WHERE c_int = 1000; + ) SELECT c_int, c_varchar FROM test_insert_into_hdfs_tvf_src WHERE c_int = 1000; """ order_qt_hdfs_append_first """ @@ -387,7 +387,7 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { sql """ INSERT INTO hdfs( ${hdfsWriteProps("append/data_", "parquet")} - ) SELECT c_int, c_varchar FROM insert_tvf_test_src WHERE c_int = 2000; + ) SELECT c_int, c_varchar FROM test_insert_into_hdfs_tvf_src WHERE c_int = 2000; """ order_qt_hdfs_append_second """ @@ -417,7 +417,7 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { INSERT INTO hdfs( ${hdfsWriteProps("where_groupby/data_", "csv")}, "delete_existing_files" = "true" - ) SELECT c_bool, COUNT(*), SUM(c_int) FROM insert_tvf_test_src WHERE c_int IS NOT NULL GROUP BY c_bool ORDER BY c_bool; + ) SELECT c_bool, COUNT(*), SUM(c_int) FROM test_insert_into_hdfs_tvf_src WHERE c_int IS NOT NULL GROUP BY c_bool ORDER BY c_bool; """ order_qt_hdfs_where_groupby """ @@ -433,7 +433,7 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { ${hdfsWriteProps("join_query/data_", "csv")}, "delete_existing_files" = "true" ) SELECT a.c_int, a.c_varchar, b.c_label - FROM insert_tvf_test_src a INNER JOIN insert_tvf_join_src b ON a.c_int = b.c_int + FROM test_insert_into_hdfs_tvf_src a INNER JOIN test_insert_into_hdfs_tvf_join_src b ON a.c_int = b.c_int ORDER BY a.c_int; """ @@ -449,7 +449,7 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { INSERT INTO hdfs( ${hdfsWriteProps("subquery/data_", "csv")}, "delete_existing_files" = "true" - ) SELECT * FROM (SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int) sub; + ) SELECT * FROM (SELECT c_int, c_varchar, c_string FROM test_insert_into_hdfs_tvf_src WHERE c_int IS NOT NULL ORDER BY c_int) sub; """ order_qt_hdfs_subquery """ @@ -465,7 +465,7 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { ${hdfsWriteProps("type_cast/data_", "csv")}, "delete_existing_files" = "true" ) SELECT CAST(c_int AS BIGINT), CAST(c_float AS DOUBLE), CAST(c_date AS STRING) - FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; + FROM test_insert_into_hdfs_tvf_src WHERE c_int IS NOT NULL ORDER BY c_int; """ order_qt_hdfs_type_cast """ @@ -480,9 +480,9 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { INSERT INTO hdfs( ${hdfsWriteProps("union_query/data_", "csv")}, "delete_existing_files" = "true" - ) SELECT c_int, c_varchar FROM insert_tvf_test_src WHERE c_int = 1000 + ) SELECT c_int, c_varchar FROM test_insert_into_hdfs_tvf_src WHERE c_int = 1000 UNION ALL - SELECT c_int, c_varchar FROM insert_tvf_test_src WHERE c_int = 2000; + SELECT c_int, c_varchar FROM test_insert_into_hdfs_tvf_src WHERE c_int = 2000; """ order_qt_hdfs_union_query """ @@ -547,7 +547,7 @@ suite("test_insert_into_hdfs_tvf", "external,hive,tvf,external_docker") { // ============ Cleanup ============ - sql """ DROP TABLE IF EXISTS insert_tvf_test_src """ - sql """ DROP TABLE IF EXISTS insert_tvf_complex_src """ - sql """ DROP TABLE IF EXISTS insert_tvf_join_src """ + sql """ DROP TABLE IF EXISTS test_insert_into_hdfs_tvf_src """ + sql """ DROP TABLE IF EXISTS test_insert_into_hdfs_tvf_complex_src """ + sql """ DROP TABLE IF EXISTS test_insert_into_hdfs_tvf_join_src """ } diff --git a/regression-test/suites/external_table_p0/tvf/insert/test_insert_into_local_tvf.groovy b/regression-test/suites/external_table_p0/tvf/insert/test_insert_into_local_tvf.groovy index fe98120f29eee6..cf5f539c6512e9 100644 --- a/regression-test/suites/external_table_p0/tvf/insert/test_insert_into_local_tvf.groovy +++ b/regression-test/suites/external_table_p0/tvf/insert/test_insert_into_local_tvf.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { +suite("test_insert_into_local_tvf", "tvf,external,external_docker") { List> backends = sql """ show backends """ assertTrue(backends.size() > 0) @@ -30,9 +30,9 @@ suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { // ============ Source tables ============ - sql """ DROP TABLE IF EXISTS insert_tvf_test_src """ + sql """ DROP TABLE IF EXISTS test_insert_into_local_tvf_src """ sql """ - CREATE TABLE IF NOT EXISTS insert_tvf_test_src ( + CREATE TABLE IF NOT EXISTS test_insert_into_local_tvf_src ( c_bool BOOLEAN, c_tinyint TINYINT, c_smallint SMALLINT, @@ -50,7 +50,7 @@ suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { """ sql """ - INSERT INTO insert_tvf_test_src VALUES + INSERT INTO test_insert_into_local_tvf_src VALUES (true, 1, 100, 1000, 100000, 1.1, 2.2, 123.45, '2024-01-01', '2024-01-01 10:00:00', 'hello', 'world'), (false, 2, 200, 2000, 200000, 3.3, 4.4, 678.90, '2024-06-15', '2024-06-15 12:30:00', 'foo', 'bar'), (true, 3, 300, 3000, 300000, 5.5, 6.6, 999.99, '2024-12-31', '2024-12-31 23:59:59', 'test', 'data'), @@ -58,9 +58,9 @@ suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { (false, -1, -100, -1000, -100000, -1.1, -2.2, -123.45,'2020-02-29', '2020-02-29 00:00:00', '', 'special_chars'); """ - sql """ DROP TABLE IF EXISTS insert_tvf_complex_src """ + sql """ DROP TABLE IF EXISTS test_insert_into_local_tvf_complex_src """ sql """ - CREATE TABLE IF NOT EXISTS insert_tvf_complex_src ( + CREATE TABLE IF NOT EXISTS test_insert_into_local_tvf_complex_src ( c_int INT, c_array ARRAY, c_map MAP, @@ -70,23 +70,23 @@ suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { """ sql """ - INSERT INTO insert_tvf_complex_src VALUES + INSERT INTO test_insert_into_local_tvf_complex_src VALUES (1, [1, 2, 3], {'a': 1, 'b': 2}, {1, 'hello'}), (2, [4, 5], {'x': 10}, {2, 'world'}), (3, [], {}, {3, ''}), (4, NULL, NULL, NULL); """ - sql """ DROP TABLE IF EXISTS insert_tvf_join_src """ + sql """ DROP TABLE IF EXISTS test_insert_into_local_tvf_join_src """ sql """ - CREATE TABLE IF NOT EXISTS insert_tvf_join_src ( + CREATE TABLE IF NOT EXISTS test_insert_into_local_tvf_join_src ( c_int INT, c_label STRING ) DISTRIBUTED BY HASH(c_int) BUCKETS 1 PROPERTIES("replication_num" = "1"); """ - sql """ INSERT INTO insert_tvf_join_src VALUES (1000, 'label_a'), (2000, 'label_b'), (3000, 'label_c'); """ + sql """ INSERT INTO test_insert_into_local_tvf_join_src VALUES (1000, 'label_a'), (2000, 'label_b'), (3000, 'label_c'); """ // ============ 1. CSV basic types ============ // file_path is a prefix; BE generates: {prefix}{query_id}_{idx}.{ext} @@ -99,7 +99,7 @@ suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { "file_path" = "${basePath}/basic_csv_", "backend_id" = "${be_id}", "format" = "csv" - ) SELECT * FROM insert_tvf_test_src ORDER BY c_int; + ) SELECT * FROM test_insert_into_local_tvf_src ORDER BY c_int; """ order_qt_csv_basic_types """ @@ -119,7 +119,7 @@ suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { "file_path" = "${basePath}/basic_parquet_", "backend_id" = "${be_id}", "format" = "parquet" - ) SELECT * FROM insert_tvf_test_src ORDER BY c_int; + ) SELECT * FROM test_insert_into_local_tvf_src ORDER BY c_int; """ order_qt_parquet_basic_types """ @@ -139,7 +139,7 @@ suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { "file_path" = "${basePath}/basic_orc_", "backend_id" = "${be_id}", "format" = "orc" - ) SELECT * FROM insert_tvf_test_src ORDER BY c_int; + ) SELECT * FROM test_insert_into_local_tvf_src ORDER BY c_int; """ order_qt_orc_basic_types """ @@ -159,7 +159,7 @@ suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { "file_path" = "${basePath}/complex_parquet_", "backend_id" = "${be_id}", "format" = "parquet" - ) SELECT * FROM insert_tvf_complex_src ORDER BY c_int; + ) SELECT * FROM test_insert_into_local_tvf_complex_src ORDER BY c_int; """ order_qt_parquet_complex_types """ @@ -179,7 +179,7 @@ suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { "file_path" = "${basePath}/complex_orc_", "backend_id" = "${be_id}", "format" = "orc" - ) SELECT * FROM insert_tvf_complex_src ORDER BY c_int; + ) SELECT * FROM test_insert_into_local_tvf_complex_src ORDER BY c_int; """ order_qt_orc_complex_types """ @@ -200,7 +200,7 @@ suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { "backend_id" = "${be_id}", "format" = "csv", "column_separator" = "," - ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; + ) SELECT c_int, c_varchar, c_string FROM test_insert_into_local_tvf_src WHERE c_int IS NOT NULL ORDER BY c_int; """ order_qt_csv_sep_comma """ @@ -222,7 +222,7 @@ suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { "backend_id" = "${be_id}", "format" = "csv", "column_separator" = "\t" - ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; + ) SELECT c_int, c_varchar, c_string FROM test_insert_into_local_tvf_src WHERE c_int IS NOT NULL ORDER BY c_int; """ order_qt_csv_sep_tab """ @@ -244,7 +244,7 @@ suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { "backend_id" = "${be_id}", "format" = "csv", "column_separator" = "|" - ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; + ) SELECT c_int, c_varchar, c_string FROM test_insert_into_local_tvf_src WHERE c_int IS NOT NULL ORDER BY c_int; """ order_qt_csv_sep_pipe """ @@ -266,7 +266,7 @@ suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { "backend_id" = "${be_id}", "format" = "csv", "column_separator" = ";;" - ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; + ) SELECT c_int, c_varchar, c_string FROM test_insert_into_local_tvf_src WHERE c_int IS NOT NULL ORDER BY c_int; """ order_qt_csv_sep_multi """ @@ -288,7 +288,7 @@ suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { "backend_id" = "${be_id}", "format" = "csv", "line_delimiter" = "\r\n" - ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; + ) SELECT c_int, c_varchar, c_string FROM test_insert_into_local_tvf_src WHERE c_int IS NOT NULL ORDER BY c_int; """ order_qt_csv_line_crlf """ @@ -310,7 +310,7 @@ suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { "backend_id" = "${be_id}", "format" = "csv", "compression_type" = "gz" - ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; + ) SELECT c_int, c_varchar, c_string FROM test_insert_into_local_tvf_src WHERE c_int IS NOT NULL ORDER BY c_int; """ order_qt_csv_compress_gz """ @@ -332,7 +332,7 @@ suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { "backend_id" = "${be_id}", "format" = "csv", "compression_type" = "zstd" - ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; + ) SELECT c_int, c_varchar, c_string FROM test_insert_into_local_tvf_src WHERE c_int IS NOT NULL ORDER BY c_int; """ order_qt_csv_compress_zstd """ @@ -355,7 +355,7 @@ suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { "backend_id" = "${be_id}", "format" = "csv", "compression_type" = "lz4block" - ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; + ) SELECT c_int, c_varchar, c_string FROM test_insert_into_local_tvf_src WHERE c_int IS NOT NULL ORDER BY c_int; """ order_qt_csv_compress_lz4 """ @@ -377,7 +377,7 @@ suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { "backend_id" = "${be_id}", "format" = "csv", "compression_type" = "snappyblock" - ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; + ) SELECT c_int, c_varchar, c_string FROM test_insert_into_local_tvf_src WHERE c_int IS NOT NULL ORDER BY c_int; """ order_qt_csv_compress_snappy """ @@ -400,7 +400,7 @@ suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { "file_path" = "${basePath}/overwrite_", "backend_id" = "${be_id}", "format" = "csv" - ) SELECT c_int, c_varchar FROM insert_tvf_test_src ORDER BY c_int; + ) SELECT c_int, c_varchar FROM test_insert_into_local_tvf_src ORDER BY c_int; """ order_qt_overwrite_first """ @@ -419,7 +419,7 @@ suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { "file_path" = "${basePath}/overwrite_", "backend_id" = "${be_id}", "format" = "csv" - ) SELECT c_int, c_varchar FROM insert_tvf_test_src WHERE c_int > 0 ORDER BY c_int LIMIT 2; + ) SELECT c_int, c_varchar FROM test_insert_into_local_tvf_src WHERE c_int > 0 ORDER BY c_int LIMIT 2; """ order_qt_overwrite_second """ @@ -440,7 +440,7 @@ suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { "file_path" = "${basePath}/append_", "backend_id" = "${be_id}", "format" = "parquet" - ) SELECT c_int, c_varchar FROM insert_tvf_test_src WHERE c_int = 1000; + ) SELECT c_int, c_varchar FROM test_insert_into_local_tvf_src WHERE c_int = 1000; """ order_qt_append_first """ @@ -457,7 +457,7 @@ suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { "file_path" = "${basePath}/append_", "backend_id" = "${be_id}", "format" = "parquet" - ) SELECT c_int, c_varchar FROM insert_tvf_test_src WHERE c_int = 2000; + ) SELECT c_int, c_varchar FROM test_insert_into_local_tvf_src WHERE c_int = 2000; """ order_qt_append_second """ @@ -497,7 +497,7 @@ suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { "file_path" = "${basePath}/where_groupby_", "backend_id" = "${be_id}", "format" = "csv" - ) SELECT c_bool, COUNT(*), SUM(c_int) FROM insert_tvf_test_src WHERE c_int IS NOT NULL GROUP BY c_bool ORDER BY c_bool; + ) SELECT c_bool, COUNT(*), SUM(c_int) FROM test_insert_into_local_tvf_src WHERE c_int IS NOT NULL GROUP BY c_bool ORDER BY c_bool; """ order_qt_where_groupby """ @@ -518,7 +518,7 @@ suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { "backend_id" = "${be_id}", "format" = "csv" ) SELECT a.c_int, a.c_varchar, b.c_label - FROM insert_tvf_test_src a INNER JOIN insert_tvf_join_src b ON a.c_int = b.c_int + FROM test_insert_into_local_tvf_src a INNER JOIN test_insert_into_local_tvf_join_src b ON a.c_int = b.c_int ORDER BY a.c_int; """ @@ -539,7 +539,7 @@ suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { "file_path" = "${basePath}/subquery_", "backend_id" = "${be_id}", "format" = "csv" - ) SELECT * FROM (SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int) sub; + ) SELECT * FROM (SELECT c_int, c_varchar, c_string FROM test_insert_into_local_tvf_src WHERE c_int IS NOT NULL ORDER BY c_int) sub; """ order_qt_subquery """ @@ -560,7 +560,7 @@ suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { "backend_id" = "${be_id}", "format" = "csv" ) SELECT CAST(c_int AS BIGINT), CAST(c_float AS DOUBLE), CAST(c_date AS STRING) - FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; + FROM test_insert_into_local_tvf_src WHERE c_int IS NOT NULL ORDER BY c_int; """ order_qt_type_cast """ @@ -580,9 +580,9 @@ suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { "file_path" = "${basePath}/union_query_", "backend_id" = "${be_id}", "format" = "csv" - ) SELECT c_int, c_varchar FROM insert_tvf_test_src WHERE c_int = 1000 + ) SELECT c_int, c_varchar FROM test_insert_into_local_tvf_src WHERE c_int = 1000 UNION ALL - SELECT c_int, c_varchar FROM insert_tvf_test_src WHERE c_int = 2000; + SELECT c_int, c_varchar FROM test_insert_into_local_tvf_src WHERE c_int = 2000; """ order_qt_union_query """ @@ -684,7 +684,7 @@ suite("test_insert_into_local_tvf", "p0,tvf,external,external_docker") { // ============ Cleanup ============ sshExec("root", be_host, "rm -rf ${basePath}", false) - sql """ DROP TABLE IF EXISTS insert_tvf_test_src """ - sql """ DROP TABLE IF EXISTS insert_tvf_complex_src """ - sql """ DROP TABLE IF EXISTS insert_tvf_join_src """ + sql """ DROP TABLE IF EXISTS test_insert_into_local_tvf_src """ + sql """ DROP TABLE IF EXISTS test_insert_into_local_tvf_complex_src """ + sql """ DROP TABLE IF EXISTS test_insert_into_local_tvf_join_src """ } diff --git a/regression-test/suites/external_table_p0/tvf/insert/test_insert_into_s3_tvf.groovy b/regression-test/suites/external_table_p0/tvf/insert/test_insert_into_s3_tvf.groovy index b2824fd5cffe63..0e1ebbf27f925a 100644 --- a/regression-test/suites/external_table_p0/tvf/insert/test_insert_into_s3_tvf.groovy +++ b/regression-test/suites/external_table_p0/tvf/insert/test_insert_into_s3_tvf.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite("test_insert_into_s3_tvf", "p0,external,external_docker") { +suite("test_insert_into_s3_tvf", "external,external_docker") { String ak = getS3AK() String sk = getS3SK() @@ -55,9 +55,9 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { // ============ Source tables ============ - sql """ DROP TABLE IF EXISTS insert_tvf_test_src """ + sql """ DROP TABLE IF EXISTS test_insert_into_s3_tvf_src """ sql """ - CREATE TABLE IF NOT EXISTS insert_tvf_test_src ( + CREATE TABLE IF NOT EXISTS test_insert_into_s3_tvf_src ( c_bool BOOLEAN, c_tinyint TINYINT, c_smallint SMALLINT, @@ -75,7 +75,7 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { """ sql """ - INSERT INTO insert_tvf_test_src VALUES + INSERT INTO test_insert_into_s3_tvf_src VALUES (true, 1, 100, 1000, 100000, 1.1, 2.2, 123.45, '2024-01-01', '2024-01-01 10:00:00', 'hello', 'world'), (false, 2, 200, 2000, 200000, 3.3, 4.4, 678.90, '2024-06-15', '2024-06-15 12:30:00', 'foo', 'bar'), (true, 3, 300, 3000, 300000, 5.5, 6.6, 999.99, '2024-12-31', '2024-12-31 23:59:59', 'test', 'data'), @@ -83,9 +83,9 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { (false, -1, -100, -1000, -100000, -1.1, -2.2, -123.45,'2020-02-29', '2020-02-29 00:00:00', '', 'special_chars'); """ - sql """ DROP TABLE IF EXISTS insert_tvf_complex_src """ + sql """ DROP TABLE IF EXISTS test_insert_into_s3_tvf_complex_src """ sql """ - CREATE TABLE IF NOT EXISTS insert_tvf_complex_src ( + CREATE TABLE IF NOT EXISTS test_insert_into_s3_tvf_complex_src ( c_int INT, c_array ARRAY, c_map MAP, @@ -95,23 +95,23 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { """ sql """ - INSERT INTO insert_tvf_complex_src VALUES + INSERT INTO test_insert_into_s3_tvf_complex_src VALUES (1, [1, 2, 3], {'a': 1, 'b': 2}, {1, 'hello'}), (2, [4, 5], {'x': 10}, {2, 'world'}), (3, [], {}, {3, ''}), (4, NULL, NULL, NULL); """ - sql """ DROP TABLE IF EXISTS insert_tvf_join_src """ + sql """ DROP TABLE IF EXISTS test_insert_into_s3_tvf_join_src """ sql """ - CREATE TABLE IF NOT EXISTS insert_tvf_join_src ( + CREATE TABLE IF NOT EXISTS test_insert_into_s3_tvf_join_src ( c_int INT, c_label STRING ) DISTRIBUTED BY HASH(c_int) BUCKETS 1 PROPERTIES("replication_num" = "1"); """ - sql """ INSERT INTO insert_tvf_join_src VALUES (1000, 'label_a'), (2000, 'label_b'), (3000, 'label_c'); """ + sql """ INSERT INTO test_insert_into_s3_tvf_join_src VALUES (1000, 'label_a'), (2000, 'label_b'), (3000, 'label_c'); """ // ============ 1. S3 CSV basic types ============ @@ -119,7 +119,7 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { INSERT INTO s3( ${s3WriteProps("basic_csv/data_", "csv")}, "delete_existing_files" = "true" - ) SELECT * FROM insert_tvf_test_src ORDER BY c_int; + ) SELECT * FROM test_insert_into_s3_tvf_src ORDER BY c_int; """ order_qt_s3_csv_basic_types """ @@ -134,7 +134,7 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { INSERT INTO s3( ${s3WriteProps("basic_parquet/data_", "parquet")}, "delete_existing_files" = "true" - ) SELECT * FROM insert_tvf_test_src ORDER BY c_int; + ) SELECT * FROM test_insert_into_s3_tvf_src ORDER BY c_int; """ order_qt_s3_parquet_basic_types """ @@ -149,7 +149,7 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { INSERT INTO s3( ${s3WriteProps("basic_orc/data_", "orc")}, "delete_existing_files" = "true" - ) SELECT * FROM insert_tvf_test_src ORDER BY c_int; + ) SELECT * FROM test_insert_into_s3_tvf_src ORDER BY c_int; """ order_qt_s3_orc_basic_types """ @@ -164,7 +164,7 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { INSERT INTO s3( ${s3WriteProps("complex_parquet/data_", "parquet")}, "delete_existing_files" = "true" - ) SELECT * FROM insert_tvf_complex_src ORDER BY c_int; + ) SELECT * FROM test_insert_into_s3_tvf_complex_src ORDER BY c_int; """ order_qt_s3_parquet_complex_types """ @@ -179,7 +179,7 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { INSERT INTO s3( ${s3WriteProps("complex_orc/data_", "orc")}, "delete_existing_files" = "true" - ) SELECT * FROM insert_tvf_complex_src ORDER BY c_int; + ) SELECT * FROM test_insert_into_s3_tvf_complex_src ORDER BY c_int; """ order_qt_s3_orc_complex_types """ @@ -195,7 +195,7 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { ${s3WriteProps("sep_comma/data_", "csv")}, "column_separator" = ",", "delete_existing_files" = "true" - ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; + ) SELECT c_int, c_varchar, c_string FROM test_insert_into_s3_tvf_src WHERE c_int IS NOT NULL ORDER BY c_int; """ order_qt_s3_csv_sep_comma """ @@ -212,7 +212,7 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { ${s3WriteProps("sep_tab/data_", "csv")}, "column_separator" = "\t", "delete_existing_files" = "true" - ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; + ) SELECT c_int, c_varchar, c_string FROM test_insert_into_s3_tvf_src WHERE c_int IS NOT NULL ORDER BY c_int; """ order_qt_s3_csv_sep_tab """ @@ -229,7 +229,7 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { ${s3WriteProps("sep_pipe/data_", "csv")}, "column_separator" = "|", "delete_existing_files" = "true" - ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; + ) SELECT c_int, c_varchar, c_string FROM test_insert_into_s3_tvf_src WHERE c_int IS NOT NULL ORDER BY c_int; """ order_qt_s3_csv_sep_pipe """ @@ -246,7 +246,7 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { ${s3WriteProps("sep_multi/data_", "csv")}, "column_separator" = ";;", "delete_existing_files" = "true" - ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; + ) SELECT c_int, c_varchar, c_string FROM test_insert_into_s3_tvf_src WHERE c_int IS NOT NULL ORDER BY c_int; """ order_qt_s3_csv_sep_multi """ @@ -263,7 +263,7 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { ${s3WriteProps("line_crlf/data_", "csv")}, "line_delimiter" = "\r\n", "delete_existing_files" = "true" - ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; + ) SELECT c_int, c_varchar, c_string FROM test_insert_into_s3_tvf_src WHERE c_int IS NOT NULL ORDER BY c_int; """ order_qt_s3_csv_line_crlf """ @@ -280,7 +280,7 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { ${s3WriteProps("compress_gz/data_", "csv")}, "compression_type" = "gz", "delete_existing_files" = "true" - ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; + ) SELECT c_int, c_varchar, c_string FROM test_insert_into_s3_tvf_src WHERE c_int IS NOT NULL ORDER BY c_int; """ order_qt_s3_csv_compress_gz """ @@ -297,7 +297,7 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { ${s3WriteProps("compress_zstd/data_", "csv")}, "compression_type" = "zstd", "delete_existing_files" = "true" - ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; + ) SELECT c_int, c_varchar, c_string FROM test_insert_into_s3_tvf_src WHERE c_int IS NOT NULL ORDER BY c_int; """ order_qt_s3_csv_compress_zstd """ @@ -314,7 +314,7 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { ${s3WriteProps("compress_lz4/data_", "csv")}, "compression_type" = "lz4block", "delete_existing_files" = "true" - ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; + ) SELECT c_int, c_varchar, c_string FROM test_insert_into_s3_tvf_src WHERE c_int IS NOT NULL ORDER BY c_int; """ order_qt_s3_csv_compress_lz4 """ @@ -331,7 +331,7 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { ${s3WriteProps("compress_snappy/data_", "csv")}, "compression_type" = "snappyblock", "delete_existing_files" = "true" - ) SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; + ) SELECT c_int, c_varchar, c_string FROM test_insert_into_s3_tvf_src WHERE c_int IS NOT NULL ORDER BY c_int; """ order_qt_s3_csv_compress_snappy """ @@ -349,7 +349,7 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { INSERT INTO s3( ${s3WriteProps("overwrite/data_", "csv")}, "delete_existing_files" = "true" - ) SELECT c_int, c_varchar FROM insert_tvf_test_src ORDER BY c_int; + ) SELECT c_int, c_varchar FROM test_insert_into_s3_tvf_src ORDER BY c_int; """ order_qt_s3_overwrite_first """ @@ -363,7 +363,7 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { INSERT INTO s3( ${s3WriteProps("overwrite/data_", "csv")}, "delete_existing_files" = "true" - ) SELECT c_int, c_varchar FROM insert_tvf_test_src WHERE c_int > 0 ORDER BY c_int LIMIT 2; + ) SELECT c_int, c_varchar FROM test_insert_into_s3_tvf_src WHERE c_int > 0 ORDER BY c_int LIMIT 2; """ order_qt_s3_overwrite_second """ @@ -379,7 +379,7 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { INSERT INTO s3( ${s3WriteProps("append/data_", "parquet")}, "delete_existing_files" = "true" - ) SELECT c_int, c_varchar FROM insert_tvf_test_src WHERE c_int = 1000; + ) SELECT c_int, c_varchar FROM test_insert_into_s3_tvf_src WHERE c_int = 1000; """ order_qt_s3_append_first """ @@ -392,7 +392,7 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { sql """ INSERT INTO s3( ${s3WriteProps("append/data_", "parquet")} - ) SELECT c_int, c_varchar FROM insert_tvf_test_src WHERE c_int = 2000; + ) SELECT c_int, c_varchar FROM test_insert_into_s3_tvf_src WHERE c_int = 2000; """ order_qt_s3_append_second """ @@ -422,7 +422,7 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { INSERT INTO s3( ${s3WriteProps("where_groupby/data_", "csv")}, "delete_existing_files" = "true" - ) SELECT c_bool, COUNT(*), SUM(c_int) FROM insert_tvf_test_src WHERE c_int IS NOT NULL GROUP BY c_bool ORDER BY c_bool; + ) SELECT c_bool, COUNT(*), SUM(c_int) FROM test_insert_into_s3_tvf_src WHERE c_int IS NOT NULL GROUP BY c_bool ORDER BY c_bool; """ order_qt_s3_where_groupby """ @@ -438,7 +438,7 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { ${s3WriteProps("join_query/data_", "csv")}, "delete_existing_files" = "true" ) SELECT a.c_int, a.c_varchar, b.c_label - FROM insert_tvf_test_src a INNER JOIN insert_tvf_join_src b ON a.c_int = b.c_int + FROM test_insert_into_s3_tvf_src a INNER JOIN test_insert_into_s3_tvf_join_src b ON a.c_int = b.c_int ORDER BY a.c_int; """ @@ -454,7 +454,7 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { INSERT INTO s3( ${s3WriteProps("subquery/data_", "csv")}, "delete_existing_files" = "true" - ) SELECT * FROM (SELECT c_int, c_varchar, c_string FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int) sub; + ) SELECT * FROM (SELECT c_int, c_varchar, c_string FROM test_insert_into_s3_tvf_src WHERE c_int IS NOT NULL ORDER BY c_int) sub; """ order_qt_s3_subquery """ @@ -470,7 +470,7 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { ${s3WriteProps("type_cast/data_", "csv")}, "delete_existing_files" = "true" ) SELECT CAST(c_int AS BIGINT), CAST(c_float AS DOUBLE), CAST(c_date AS STRING) - FROM insert_tvf_test_src WHERE c_int IS NOT NULL ORDER BY c_int; + FROM test_insert_into_s3_tvf_src WHERE c_int IS NOT NULL ORDER BY c_int; """ order_qt_s3_type_cast """ @@ -485,9 +485,9 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { INSERT INTO s3( ${s3WriteProps("union_query/data_", "csv")}, "delete_existing_files" = "true" - ) SELECT c_int, c_varchar FROM insert_tvf_test_src WHERE c_int = 1000 + ) SELECT c_int, c_varchar FROM test_insert_into_s3_tvf_src WHERE c_int = 1000 UNION ALL - SELECT c_int, c_varchar FROM insert_tvf_test_src WHERE c_int = 2000; + SELECT c_int, c_varchar FROM test_insert_into_s3_tvf_src WHERE c_int = 2000; """ order_qt_s3_union_query """ @@ -560,7 +560,7 @@ suite("test_insert_into_s3_tvf", "p0,external,external_docker") { // ============ Cleanup ============ - sql """ DROP TABLE IF EXISTS insert_tvf_test_src """ - sql """ DROP TABLE IF EXISTS insert_tvf_complex_src """ - sql """ DROP TABLE IF EXISTS insert_tvf_join_src """ + sql """ DROP TABLE IF EXISTS test_insert_into_s3_tvf_src """ + sql """ DROP TABLE IF EXISTS test_insert_into_s3_tvf_complex_src """ + sql """ DROP TABLE IF EXISTS test_insert_into_s3_tvf_join_src """ } From fcc6e220b17f401a1cebf7e18e89b7745b099458 Mon Sep 17 00:00:00 2001 From: morningman Date: Fri, 13 Feb 2026 15:44:46 +0800 Subject: [PATCH 13/13] support jni writer --- .../vfile_format_transformer_factory.cpp | 23 +++ .../vec/runtime/vjni_format_transformer.cpp | 137 ++++++++++++++++++ be/src/vec/runtime/vjni_format_transformer.h | 74 ++++++++++ be/src/vec/sink/writer/vtvf_table_writer.cpp | 29 ++-- build.sh | 4 +- .../apache/doris/common/jni/JniWriter.java | 99 +++++++++++++ fe/be-java-extensions/java-writer/pom.xml | 75 ++++++++++ .../doris/writer/LocalFileJniWriter.java | 117 +++++++++++++++ .../src/main/resources/package.xml | 41 ++++++ fe/be-java-extensions/pom.xml | 1 + .../apache/doris/planner/TVFTableSink.java | 10 ++ gensrc/thrift/DataSinks.thrift | 7 + 12 files changed, 606 insertions(+), 11 deletions(-) create mode 100644 be/src/vec/runtime/vjni_format_transformer.cpp create mode 100644 be/src/vec/runtime/vjni_format_transformer.h create mode 100644 fe/be-java-extensions/java-common/src/main/java/org/apache/doris/common/jni/JniWriter.java create mode 100644 fe/be-java-extensions/java-writer/pom.xml create mode 100644 fe/be-java-extensions/java-writer/src/main/java/org/apache/doris/writer/LocalFileJniWriter.java create mode 100644 fe/be-java-extensions/java-writer/src/main/resources/package.xml diff --git a/be/src/vec/runtime/vfile_format_transformer_factory.cpp b/be/src/vec/runtime/vfile_format_transformer_factory.cpp index bbd415e2452e28..69751cc637f55c 100644 --- a/be/src/vec/runtime/vfile_format_transformer_factory.cpp +++ b/be/src/vec/runtime/vfile_format_transformer_factory.cpp @@ -23,6 +23,7 @@ #include #include "vec/runtime/vcsv_transformer.h" +#include "vec/runtime/vjni_format_transformer.h" #include "vec/runtime/vorc_transformer.h" #include "vec/runtime/vparquet_transformer.h" @@ -32,6 +33,28 @@ Status create_tvf_format_transformer(const TTVFTableSink& tvf_sink, RuntimeState io::FileWriter* file_writer, const VExprContextSPtrs& output_vexpr_ctxs, std::unique_ptr* result) { + // JNI writer path: delegate to Java-side writer + if (tvf_sink.__isset.writer_type && tvf_sink.writer_type == TTVFWriterType::JNI) { + if (!tvf_sink.__isset.writer_class) { + return Status::InternalError("writer_class is required when writer_type is JNI"); + } + std::map writer_params; + if (tvf_sink.__isset.properties) { + writer_params = tvf_sink.properties; + } + writer_params["file_path"] = tvf_sink.file_path; + if (tvf_sink.__isset.column_separator) { + writer_params["column_separator"] = tvf_sink.column_separator; + } + if (tvf_sink.__isset.line_delimiter) { + writer_params["line_delimiter"] = tvf_sink.line_delimiter; + } + result->reset(new VJniFormatTransformer(state, output_vexpr_ctxs, tvf_sink.writer_class, + std::move(writer_params))); + return Status::OK(); + } + + // Native writer path TFileFormatType::type format = tvf_sink.file_format; switch (format) { case TFileFormatType::FORMAT_CSV_PLAIN: { diff --git a/be/src/vec/runtime/vjni_format_transformer.cpp b/be/src/vec/runtime/vjni_format_transformer.cpp new file mode 100644 index 00000000000000..f309a2861ccc23 --- /dev/null +++ b/be/src/vec/runtime/vjni_format_transformer.cpp @@ -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. + +#include "vec/runtime/vjni_format_transformer.h" + +#include "runtime/runtime_state.h" +#include "vec/exec/jni_connector.h" + +namespace doris::vectorized { + +VJniFormatTransformer::VJniFormatTransformer(RuntimeState* state, + const VExprContextSPtrs& output_vexpr_ctxs, + std::string writer_class, + std::map writer_params) + : VFileFormatTransformer(state, output_vexpr_ctxs, false), + _writer_class(std::move(writer_class)), + _writer_params(std::move(writer_params)) {} + +Status VJniFormatTransformer::_init_jni_writer(JNIEnv* env, int batch_size) { + // Load writer class via the same class loader as JniScanner + Jni::GlobalClass jni_writer_cls; + RETURN_IF_ERROR( + Jni::Util::get_jni_scanner_class(env, _writer_class.c_str(), &jni_writer_cls)); + + // Get constructor: (int batchSize, Map params) + Jni::MethodId writer_constructor; + RETURN_IF_ERROR( + jni_writer_cls.get_method(env, "", "(ILjava/util/Map;)V", &writer_constructor)); + + // Convert C++ params map to Java HashMap + Jni::LocalObject hashmap_object; + RETURN_IF_ERROR(Jni::Util::convert_to_java_map(env, _writer_params, &hashmap_object)); + + // Create writer instance + RETURN_IF_ERROR(jni_writer_cls.new_object(env, writer_constructor) + .with_arg((jint)batch_size) + .with_arg(hashmap_object) + .call(&_jni_writer_obj)); + + // Resolve method IDs + RETURN_IF_ERROR(jni_writer_cls.get_method(env, "open", "()V", &_jni_writer_open)); + RETURN_IF_ERROR( + jni_writer_cls.get_method(env, "write", "(Ljava/util/Map;)V", &_jni_writer_write)); + RETURN_IF_ERROR(jni_writer_cls.get_method(env, "close", "()V", &_jni_writer_close)); + RETURN_IF_ERROR(jni_writer_cls.get_method(env, "getStatistics", "()Ljava/util/Map;", + &_jni_writer_get_statistics)); + return Status::OK(); +} + +Status VJniFormatTransformer::open() { + JNIEnv* env = nullptr; + RETURN_IF_ERROR(Jni::Env::Get(&env)); + + int batch_size = _state->batch_size(); + RETURN_IF_ERROR(_init_jni_writer(env, batch_size)); + + RETURN_IF_ERROR(_jni_writer_obj.call_void_method(env, _jni_writer_open).call()); + RETURN_ERROR_IF_EXC(env); + + _opened = true; + return Status::OK(); +} + +Status VJniFormatTransformer::write(const Block& block) { + if (block.rows() == 0) { + return Status::OK(); + } + + JNIEnv* env = nullptr; + RETURN_IF_ERROR(Jni::Env::Get(&env)); + + // 1. Convert Block to Java table metadata (column addresses) + Block* mutable_block = const_cast(&block); + std::unique_ptr input_table; + RETURN_IF_ERROR(JniConnector::to_java_table(mutable_block, input_table)); + + // 2. Cache schema on first call + if (!_schema_cached) { + auto schema = JniConnector::parse_table_schema(mutable_block); + _cached_required_fields = schema.first; + _cached_columns_types = schema.second; + _schema_cached = true; + } + + // 3. Build input params map for Java writer + std::map input_params = { + {"meta_address", std::to_string((long)input_table.get())}, + {"required_fields", _cached_required_fields}, + {"columns_types", _cached_columns_types}}; + + // 4. Convert to Java Map and call writer.write(inputParams) + Jni::LocalObject input_map; + RETURN_IF_ERROR(Jni::Util::convert_to_java_map(env, input_params, &input_map)); + + RETURN_IF_ERROR( + _jni_writer_obj.call_void_method(env, _jni_writer_write).with_arg(input_map).call()); + RETURN_ERROR_IF_EXC(env); + + _cur_written_rows += block.rows(); + return Status::OK(); +} + +Status VJniFormatTransformer::close() { + if (_closed || !_opened) { + return Status::OK(); + } + _closed = true; + + JNIEnv* env = nullptr; + RETURN_IF_ERROR(Jni::Env::Get(&env)); + + RETURN_IF_ERROR(_jni_writer_obj.call_void_method(env, _jni_writer_close).call()); + RETURN_ERROR_IF_EXC(env); + + return Status::OK(); +} + +int64_t VJniFormatTransformer::written_len() { + // JNI writer manages file size on Java side; return 0 to disable C++ auto-split. + return 0; +} + +} // namespace doris::vectorized diff --git a/be/src/vec/runtime/vjni_format_transformer.h b/be/src/vec/runtime/vjni_format_transformer.h new file mode 100644 index 00000000000000..aa63d65e516614 --- /dev/null +++ b/be/src/vec/runtime/vjni_format_transformer.h @@ -0,0 +1,74 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#pragma once + +#include +#include + +#include "util/jni-util.h" +#include "vfile_format_transformer.h" + +namespace doris::vectorized { +#include "common/compile_check_begin.h" + +/** + * VJniFormatTransformer is a VFileFormatTransformer implementation that delegates + * write operations to a Java-side JniWriter via JNI. It sits alongside + * VCSVTransformer/VParquetTransformer/VOrcTransformer as a peer implementation. + * + * The Java writer class must extend org.apache.doris.common.jni.JniWriter and + * follow the same constructor signature as JniScanner: (int batchSize, Map params). + */ +class VJniFormatTransformer final : public VFileFormatTransformer { +public: + VJniFormatTransformer(RuntimeState* state, const VExprContextSPtrs& output_vexpr_ctxs, + std::string writer_class, + std::map writer_params); + + ~VJniFormatTransformer() override = default; + + Status open() override; + Status write(const Block& block) override; + Status close() override; + int64_t written_len() override; + +private: + Status _init_jni_writer(JNIEnv* env, int batch_size); + + std::string _writer_class; + std::map _writer_params; + + // JNI handles (same pattern as JniConnector) + Jni::GlobalObject _jni_writer_obj; + Jni::MethodId _jni_writer_open; + Jni::MethodId _jni_writer_write; + Jni::MethodId _jni_writer_close; + Jni::MethodId _jni_writer_get_statistics; + + // Schema cache (computed on first write, reused afterwards) + bool _schema_cached = false; + std::string _cached_required_fields; + std::string _cached_columns_types; + + bool _opened = false; + bool _closed = false; +}; + +} // namespace doris::vectorized + +#include "common/compile_check_end.h" diff --git a/be/src/vec/sink/writer/vtvf_table_writer.cpp b/be/src/vec/sink/writer/vtvf_table_writer.cpp index 7d82e8ef471d2e..df5ad05f5c82b9 100644 --- a/be/src/vec/sink/writer/vtvf_table_writer.cpp +++ b/be/src/vec/sink/writer/vtvf_table_writer.cpp @@ -87,21 +87,30 @@ Status VTVFTableWriter::close(Status status) { } Status VTVFTableWriter::_create_file_writer(const std::string& file_name) { - TFileType::type file_type = _tvf_sink.file_type; - std::map properties; - if (_tvf_sink.__isset.properties) { - properties = _tvf_sink.properties; + bool use_jni = _tvf_sink.__isset.writer_type && + _tvf_sink.writer_type == TTVFWriterType::JNI; + + if (!use_jni) { + // Native path: create file writer via FileFactory + TFileType::type file_type = _tvf_sink.file_type; + std::map properties; + if (_tvf_sink.__isset.properties) { + properties = _tvf_sink.properties; + } + + _file_writer_impl = DORIS_TRY(FileFactory::create_file_writer( + file_type, _state->exec_env(), {}, properties, file_name, + {.write_file_cache = false, .sync_file_data = false})); } - _file_writer_impl = DORIS_TRY(FileFactory::create_file_writer( - file_type, _state->exec_env(), {}, properties, file_name, - {.write_file_cache = false, .sync_file_data = false})); - - RETURN_IF_ERROR(create_tvf_format_transformer(_tvf_sink, _state, _file_writer_impl.get(), - _vec_output_expr_ctxs, &_vfile_writer)); + // Factory creates either JNI or native transformer + RETURN_IF_ERROR(create_tvf_format_transformer( + _tvf_sink, _state, use_jni ? nullptr : _file_writer_impl.get(), + _vec_output_expr_ctxs, &_vfile_writer)); VLOG_DEBUG << "TVF table writer created file: " << file_name << ", format=" << _tvf_sink.file_format + << ", use_jni=" << use_jni << ", query_id=" << print_id(_state->query_id()); return _vfile_writer->open(); diff --git a/build.sh b/build.sh index 580f5d3a047db4..2bb49e2673f300 100755 --- a/build.sh +++ b/build.sh @@ -584,6 +584,7 @@ if [[ "${BUILD_BE_JAVA_EXTENSIONS}" -eq 1 ]]; then # modules+=("be-java-extensions/lakesoul-scanner") modules+=("be-java-extensions/preload-extensions") modules+=("be-java-extensions/${HADOOP_DEPS_NAME}") + modules+=("be-java-extensions/java-writer") # If the BE_EXTENSION_IGNORE variable is not empty, remove the modules that need to be ignored from FE_MODULES if [[ -n "${BE_EXTENSION_IGNORE}" ]]; then @@ -920,6 +921,7 @@ EOF extensions_modules+=("preload-extensions") extensions_modules+=("iceberg-metadata-scanner") extensions_modules+=("${HADOOP_DEPS_NAME}") + extensions_modules+=("java-writer") if [[ -n "${BE_EXTENSION_IGNORE}" ]]; then IFS=',' read -r -a ignore_modules <<<"${BE_EXTENSION_IGNORE}" @@ -1048,4 +1050,4 @@ if [[ -n "${DORIS_POST_BUILD_HOOK}" ]]; then eval "${DORIS_POST_BUILD_HOOK}" fi -exit 0 \ No newline at end of file +exit 0 diff --git a/fe/be-java-extensions/java-common/src/main/java/org/apache/doris/common/jni/JniWriter.java b/fe/be-java-extensions/java-common/src/main/java/org/apache/doris/common/jni/JniWriter.java new file mode 100644 index 00000000000000..2dfefd2ac79cf7 --- /dev/null +++ b/fe/be-java-extensions/java-common/src/main/java/org/apache/doris/common/jni/JniWriter.java @@ -0,0 +1,99 @@ +// 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.doris.common.jni; + +import org.apache.doris.common.jni.vec.ColumnType; +import org.apache.doris.common.jni.vec.VectorTable; + +import java.io.IOException; +import java.util.Collections; +import java.util.Map; + +/** + * JniWriter is the base class for JNI-based writers, symmetric to JniScanner. + * Constructor signature: (int batchSize, Map params) matches JniScanner + * to reuse the same class loading mechanism (Jni::Util::get_jni_scanner_class). + * + * Lifecycle: open() -> write() [repeated] -> close() + */ +public abstract class JniWriter { + protected int batchSize; + protected Map params; + protected ColumnType[] columnTypes; + protected String[] fields; + protected long writeTime = 0; + protected long readTableTime = 0; + + public JniWriter(int batchSize, Map params) { + this.batchSize = batchSize; + this.params = params; + } + + public abstract void open() throws IOException; + + /** + * JNI entry point: receives C++ Block metadata, creates a ReadableTable, + * then delegates to writeInternal. + */ + public void write(Map inputParams) throws IOException { + // Parse and cache schema on first call + if (columnTypes == null) { + String requiredFields = inputParams.get("required_fields"); + String columnsTypes = inputParams.get("columns_types"); + if (requiredFields != null && !requiredFields.isEmpty()) { + fields = requiredFields.split(","); + String[] typeStrs = columnsTypes.split("#"); + columnTypes = new ColumnType[typeStrs.length]; + for (int i = 0; i < typeStrs.length; i++) { + columnTypes[i] = ColumnType.parseType(fields[i], typeStrs[i]); + } + } else { + fields = new String[0]; + columnTypes = new ColumnType[0]; + } + } + + long startRead = System.nanoTime(); + VectorTable inputTable = VectorTable.createReadableTable(inputParams); + readTableTime += System.nanoTime() - startRead; + + long startWrite = System.nanoTime(); + writeInternal(inputTable); + writeTime += System.nanoTime() - startWrite; + } + + protected abstract void writeInternal(VectorTable inputTable) throws IOException; + + public abstract void close() throws IOException; + + /** + * Performance metrics. Key format: "metricType:metricName" + * Supported types: timer, counter, bytes (same as JniScanner). + */ + public Map getStatistics() { + return Collections.emptyMap(); + } + + public long getWriteTime() { + return writeTime; + } + + public long getReadTableTime() { + return readTableTime; + } +} diff --git a/fe/be-java-extensions/java-writer/pom.xml b/fe/be-java-extensions/java-writer/pom.xml new file mode 100644 index 00000000000000..e0bbaf20389cfc --- /dev/null +++ b/fe/be-java-extensions/java-writer/pom.xml @@ -0,0 +1,75 @@ + + + + + be-java-extensions + org.apache.doris + ${revision} + + 4.0.0 + + java-writer + + + 8 + 8 + + + + + org.apache.doris + java-common + ${project.version} + provided + + + + + java-writer + ${project.basedir}/target/ + + + org.apache.maven.plugins + maven-assembly-plugin + + + src/main/resources/package.xml + + + + + + + + + + make-assembly + package + + single + + + + + + + diff --git a/fe/be-java-extensions/java-writer/src/main/java/org/apache/doris/writer/LocalFileJniWriter.java b/fe/be-java-extensions/java-writer/src/main/java/org/apache/doris/writer/LocalFileJniWriter.java new file mode 100644 index 00000000000000..0b0bab8aa5f697 --- /dev/null +++ b/fe/be-java-extensions/java-writer/src/main/java/org/apache/doris/writer/LocalFileJniWriter.java @@ -0,0 +1,117 @@ +// 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.doris.writer; + +import org.apache.doris.common.jni.JniWriter; +import org.apache.doris.common.jni.vec.VectorTable; + +import org.apache.log4j.Logger; + +import java.io.BufferedWriter; +import java.io.FileWriter; +import java.io.IOException; +import java.util.Map; + +/** + * LocalFileJniWriter writes C++ Block data to local CSV files via JNI. + * Loaded by C++ as: org/apache/doris/writer/LocalFileJniWriter + */ +public class LocalFileJniWriter extends JniWriter { + private static final Logger LOG = Logger.getLogger(LocalFileJniWriter.class); + + private String filePath; + private String columnSeparator; + private String lineDelimiter; + private BufferedWriter fileWriter; + private long writtenRows = 0; + private long writtenBytes = 0; + + public LocalFileJniWriter(int batchSize, Map params) { + super(batchSize, params); + this.filePath = params.get("file_path"); + this.columnSeparator = params.getOrDefault("column_separator", ","); + this.lineDelimiter = params.getOrDefault("line_delimiter", "\n"); + LOG.info("LocalFileJniWriter created: filePath=" + filePath + + ", columnSeparator=" + columnSeparator + + ", batchSize=" + batchSize); + } + + @Override + public void open() throws IOException { + LOG.info("LocalFileJniWriter opening file: " + filePath); + fileWriter = new BufferedWriter(new FileWriter(filePath)); + LOG.info("LocalFileJniWriter opened file successfully: " + filePath); + } + + @Override + protected void writeInternal(VectorTable inputTable) throws IOException { + int numRows = inputTable.getNumRows(); + int numCols = inputTable.getNumColumns(); + LOG.info("LocalFileJniWriter writeInternal: numRows=" + numRows + ", numCols=" + numCols); + if (numRows == 0) { + return; + } + + Object[][] data = inputTable.getMaterializedData(); + StringBuilder sb = new StringBuilder(); + + for (int row = 0; row < numRows; row++) { + for (int col = 0; col < numCols; col++) { + if (col > 0) { + sb.append(columnSeparator); + } + Object val = data[col][row]; + if (val != null) { + sb.append(val.toString()); + } else { + sb.append("\\N"); + } + } + sb.append(lineDelimiter); + } + + String output = sb.toString(); + fileWriter.write(output); + writtenRows += numRows; + writtenBytes += output.getBytes().length; + LOG.info("LocalFileJniWriter wrote " + numRows + " rows, totalWrittenRows=" + writtenRows + + ", totalWrittenBytes=" + writtenBytes); + } + + @Override + public void close() throws IOException { + LOG.info("LocalFileJniWriter closing: filePath=" + filePath + + ", totalWrittenRows=" + writtenRows + ", totalWrittenBytes=" + writtenBytes); + if (fileWriter != null) { + fileWriter.flush(); + fileWriter.close(); + fileWriter = null; + } + LOG.info("LocalFileJniWriter closed successfully: " + filePath); + } + + @Override + public Map getStatistics() { + Map stats = new java.util.HashMap<>(); + stats.put("counter:WrittenRows", String.valueOf(writtenRows)); + stats.put("bytes:WrittenBytes", String.valueOf(writtenBytes)); + stats.put("timer:WriteTime", String.valueOf(writeTime)); + stats.put("timer:ReadTableTime", String.valueOf(readTableTime)); + return stats; + } +} diff --git a/fe/be-java-extensions/java-writer/src/main/resources/package.xml b/fe/be-java-extensions/java-writer/src/main/resources/package.xml new file mode 100644 index 00000000000000..4bbb2610603363 --- /dev/null +++ b/fe/be-java-extensions/java-writer/src/main/resources/package.xml @@ -0,0 +1,41 @@ + + + + jar-with-dependencies + + jar + + false + + + / + true + true + runtime + + + **/Log4j2Plugins.dat + + + + + diff --git a/fe/be-java-extensions/pom.xml b/fe/be-java-extensions/pom.xml index 8151cd179d167a..eab071c9c5ac6d 100644 --- a/fe/be-java-extensions/pom.xml +++ b/fe/be-java-extensions/pom.xml @@ -34,6 +34,7 @@ under the License. preload-extensions trino-connector-scanner hadoop-deps + java-writer diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/TVFTableSink.java b/fe/fe-core/src/main/java/org/apache/doris/planner/TVFTableSink.java index 1d6d44d52aef61..c511336767c3df 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/TVFTableSink.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/TVFTableSink.java @@ -31,6 +31,7 @@ import org.apache.doris.thrift.TFileFormatType; import org.apache.doris.thrift.TFileType; import org.apache.doris.thrift.TTVFTableSink; +import org.apache.doris.thrift.TTVFWriterType; import com.google.common.collect.Maps; @@ -152,6 +153,15 @@ public void bindDataSink() throws AnalysisException { tSink.setHadoopConfig(backendConnectProps); } + // Set writer_type: JNI if writer_class is specified, otherwise NATIVE + String writerClass = properties.get("writer_class"); + if (writerClass != null) { + tSink.setWriterType(TTVFWriterType.JNI); + tSink.setWriterClass(writerClass); + } else { + tSink.setWriterType(TTVFWriterType.NATIVE); + } + tDataSink = new TDataSink(TDataSinkType.TVF_TABLE_SINK); tDataSink.setTvfTableSink(tSink); } diff --git a/gensrc/thrift/DataSinks.thrift b/gensrc/thrift/DataSinks.thrift index c5e176ee155d33..cb817b7dbe8803 100644 --- a/gensrc/thrift/DataSinks.thrift +++ b/gensrc/thrift/DataSinks.thrift @@ -453,6 +453,11 @@ struct TDictionarySink { struct TBlackholeSink { } +enum TTVFWriterType { + NATIVE = 0, + JNI = 1 +} + struct TTVFTableSink { 1: optional string tvf_name // "local", "s3", "hdfs" 2: optional string file_path @@ -467,6 +472,8 @@ struct TTVFTableSink { 11: optional map hadoop_config 12: optional PlanNodes.TFileCompressType compression_type 13: optional i64 backend_id // local TVF: specify BE + 14: optional TTVFWriterType writer_type // NATIVE or JNI + 15: optional string writer_class // Java class name (required when writer_type=JNI) } struct TDataSink {