[Feature] support to hive sink (#29171)
Signed-off-by: stephen <stephen5217@163.com>
This commit is contained in:
parent
681452c5ca
commit
691380242d
|
|
@ -187,6 +187,7 @@ set(EXEC_FILES
|
|||
pipeline/scan/schema_scan_operator.cpp
|
||||
pipeline/scan/schema_scan_context.cpp
|
||||
pipeline/sink/iceberg_table_sink_operator.cpp
|
||||
pipeline/sink/hive_table_sink_operator.cpp
|
||||
pipeline/scan/morsel.cpp
|
||||
pipeline/scan/chunk_buffer_limiter.cpp
|
||||
pipeline/sink/file_sink_operator.cpp
|
||||
|
|
|
|||
|
|
@ -46,6 +46,7 @@
|
|||
#include "gen_cpp/InternalService_types.h"
|
||||
#include "runtime/data_stream_sender.h"
|
||||
#include "runtime/export_sink.h"
|
||||
#include "runtime/hive_table_sink.h"
|
||||
#include "runtime/iceberg_table_sink.h"
|
||||
#include "runtime/memory_scratch_sink.h"
|
||||
#include "runtime/multi_cast_data_stream_sink.h"
|
||||
|
|
@ -152,6 +153,13 @@ Status DataSink::create_data_sink(RuntimeState* state, const TDataSink& thrift_s
|
|||
*sink = std::make_unique<IcebergTableSink>(state->obj_pool(), output_exprs);
|
||||
break;
|
||||
}
|
||||
case TDataSinkType::HIVE_TABLE_SINK: {
|
||||
if (!thrift_sink.__isset.hive_table_sink) {
|
||||
return Status::InternalError("Missing hive table sink");
|
||||
}
|
||||
*sink = std::make_unique<HiveTableSink>(state->obj_pool(), output_exprs);
|
||||
break;
|
||||
}
|
||||
|
||||
default:
|
||||
std::stringstream error_msg;
|
||||
|
|
|
|||
|
|
@ -35,6 +35,7 @@
|
|||
#include "exec/pipeline/scan/scan_operator.h"
|
||||
#include "exec/pipeline/sink/export_sink_operator.h"
|
||||
#include "exec/pipeline/sink/file_sink_operator.h"
|
||||
#include "exec/pipeline/sink/hive_table_sink_operator.h"
|
||||
#include "exec/pipeline/sink/iceberg_table_sink_operator.h"
|
||||
#include "exec/pipeline/sink/memory_scratch_sink_operator.h"
|
||||
#include "exec/pipeline/sink/mysql_table_sink_operator.h"
|
||||
|
|
@ -50,6 +51,7 @@
|
|||
#include "runtime/descriptors.h"
|
||||
#include "runtime/exec_env.h"
|
||||
#include "runtime/export_sink.h"
|
||||
#include "runtime/hive_table_sink.h"
|
||||
#include "runtime/iceberg_table_sink.h"
|
||||
#include "runtime/memory_scratch_sink.h"
|
||||
#include "runtime/multi_cast_data_stream_sink.h"
|
||||
|
|
@ -932,6 +934,37 @@ Status FragmentExecutor::_decompose_data_sink_to_operator(RuntimeState* runtime_
|
|||
partition_expr_ctxs, source_operator_dop,
|
||||
desired_iceberg_sink_dop);
|
||||
}
|
||||
} else if (typeid(*datasink) == typeid(starrocks::HiveTableSink)) {
|
||||
auto* hive_table_sink = down_cast<starrocks::HiveTableSink*>(datasink.get());
|
||||
auto output_expr = hive_table_sink->get_output_expr();
|
||||
const auto& t_hive_sink = thrift_sink.hive_table_sink;
|
||||
const auto partition_col_size = t_hive_sink.partition_column_names.size();
|
||||
std::vector<TExpr> partition_expr(output_exprs.end() - partition_col_size, output_exprs.end());
|
||||
std::vector<ExprContext*> partition_expr_ctxs;
|
||||
|
||||
RETURN_IF_ERROR(Expr::create_expr_trees(runtime_state->obj_pool(), partition_expr, &partition_expr_ctxs,
|
||||
runtime_state));
|
||||
|
||||
auto* source_operator =
|
||||
down_cast<SourceOperatorFactory*>(fragment_ctx->pipelines().back()->source_operator_factory());
|
||||
|
||||
size_t desired_hive_sink_dop = request.pipeline_sink_dop();
|
||||
size_t source_operator_dop = source_operator->degree_of_parallelism();
|
||||
OpFactoryPtr hive_table_sink_op = std::make_shared<HiveTableSinkOperatorFactory>(
|
||||
context->next_operator_id(), fragment_ctx, thrift_sink.hive_table_sink,
|
||||
hive_table_sink->get_output_expr(), partition_expr_ctxs);
|
||||
|
||||
if (t_hive_sink.partition_column_names.size() == 0 || t_hive_sink.is_static_partition_sink) {
|
||||
if (source_operator_dop != desired_hive_sink_dop) {
|
||||
context->maybe_interpolate_local_passthrough_exchange_for_sink(
|
||||
runtime_state, hive_table_sink_op, source_operator_dop, desired_hive_sink_dop);
|
||||
} else {
|
||||
fragment_ctx->pipelines().back()->get_op_factories().emplace_back(std::move(hive_table_sink_op));
|
||||
}
|
||||
} else {
|
||||
context->maybe_interpolate_local_key_partition_exchange_for_sink(
|
||||
runtime_state, hive_table_sink_op, partition_expr_ctxs, source_operator_dop, desired_hive_sink_dop);
|
||||
}
|
||||
}
|
||||
|
||||
return Status::OK();
|
||||
|
|
|
|||
|
|
@ -35,6 +35,7 @@ const int32_t Operator::s_pseudo_plan_node_id_for_olap_table_sink = -98;
|
|||
const int32_t Operator::s_pseudo_plan_node_id_for_result_sink = -99;
|
||||
const int32_t Operator::s_pseudo_plan_node_id_upper_bound = -100;
|
||||
const int32_t Operator::s_pseudo_plan_node_id_for_iceberg_table_sink = -101;
|
||||
const int32_t Operator::s_pseudo_plan_node_id_for_hive_table_sink = -102;
|
||||
|
||||
Operator::Operator(OperatorFactory* factory, int32_t id, std::string name, int32_t plan_node_id,
|
||||
int32_t driver_sequence)
|
||||
|
|
|
|||
|
|
@ -183,6 +183,7 @@ public:
|
|||
static const int32_t s_pseudo_plan_node_id_for_olap_table_sink;
|
||||
static const int32_t s_pseudo_plan_node_id_for_result_sink;
|
||||
static const int32_t s_pseudo_plan_node_id_for_iceberg_table_sink;
|
||||
static const int32_t s_pseudo_plan_node_id_for_hive_table_sink;
|
||||
static const int32_t s_pseudo_plan_node_id_upper_bound;
|
||||
|
||||
RuntimeProfile* runtime_profile() { return _runtime_profile.get(); }
|
||||
|
|
|
|||
|
|
@ -0,0 +1,257 @@
|
|||
// Copyright 2021-present StarRocks, Inc. All rights reserved.
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// https://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 "hive_table_sink_operator.h"
|
||||
|
||||
#include <utility>
|
||||
|
||||
#include "exec/parquet_builder.h"
|
||||
#include "util/path_util.h"
|
||||
|
||||
namespace starrocks::pipeline {
|
||||
|
||||
static const std::string HIVE_UNPARTITIONED_TABLE_LOCATION = "hive_unpartitioned_table_fake_location";
|
||||
|
||||
Status HiveTableSinkOperator::prepare(RuntimeState* state) {
|
||||
RETURN_IF_ERROR(Operator::prepare(state));
|
||||
return Status::OK();
|
||||
}
|
||||
|
||||
void HiveTableSinkOperator::close(RuntimeState* state) {
|
||||
for (const auto& writer : _partition_writers) {
|
||||
if (!writer.second->closed()) {
|
||||
writer.second->close(state);
|
||||
}
|
||||
}
|
||||
Operator::close(state);
|
||||
}
|
||||
|
||||
bool HiveTableSinkOperator::need_input() const {
|
||||
for (const auto& writer : _partition_writers) {
|
||||
if (!writer.second->writable()) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
bool HiveTableSinkOperator::is_finished() const {
|
||||
if (_partition_writers.size() == 0) {
|
||||
return _is_finished.load();
|
||||
}
|
||||
for (const auto& writer : _partition_writers) {
|
||||
if (!writer.second->closed()) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
Status HiveTableSinkOperator::set_finishing(RuntimeState* state) {
|
||||
for (const auto& writer : _partition_writers) {
|
||||
if (!writer.second->closed()) {
|
||||
writer.second->close(state);
|
||||
}
|
||||
}
|
||||
|
||||
if (_partition_writers.size() == 0) {
|
||||
_is_finished = true;
|
||||
}
|
||||
return Status::OK();
|
||||
}
|
||||
|
||||
bool HiveTableSinkOperator::pending_finish() const {
|
||||
return !is_finished();
|
||||
}
|
||||
|
||||
Status HiveTableSinkOperator::set_cancelled(RuntimeState* state) {
|
||||
return Status::OK();
|
||||
}
|
||||
|
||||
StatusOr<ChunkPtr> HiveTableSinkOperator::pull_chunk(RuntimeState* state) {
|
||||
return Status::InternalError("Shouldn't pull chunk from hive table sink operator");
|
||||
}
|
||||
|
||||
Status HiveTableSinkOperator::push_chunk(RuntimeState* state, const ChunkPtr& chunk) {
|
||||
TableInfo tableInfo;
|
||||
tableInfo.schema = _parquet_file_schema;
|
||||
tableInfo.compress_type = _compression_codec;
|
||||
tableInfo.cloud_conf = _cloud_conf;
|
||||
|
||||
if (_partition_column_names.size() == 0) {
|
||||
if (_partition_writers.empty()) {
|
||||
tableInfo.partition_location = _location;
|
||||
auto writer = std::make_unique<RollingAsyncParquetWriter>(tableInfo, _output_expr, _common_metrics.get(),
|
||||
add_hive_commit_info, state, _driver_sequence);
|
||||
_partition_writers.insert({HIVE_UNPARTITIONED_TABLE_LOCATION, std::move(writer)});
|
||||
}
|
||||
|
||||
_partition_writers[HIVE_UNPARTITIONED_TABLE_LOCATION]->append_chunk(chunk.get(), state);
|
||||
} else if (_is_static_partition_insert && !_partition_writers.empty()) {
|
||||
_partition_writers.begin()->second->append_chunk(chunk.get(), state);
|
||||
} else {
|
||||
Columns partitions_columns;
|
||||
partitions_columns.resize(_partition_expr.size());
|
||||
for (size_t i = 0; i < partitions_columns.size(); ++i) {
|
||||
ASSIGN_OR_RETURN(partitions_columns[i], _partition_expr[i]->evaluate(chunk.get()));
|
||||
DCHECK(partitions_columns[i] != nullptr);
|
||||
}
|
||||
|
||||
std::vector<std::string> partition_column_values;
|
||||
for (const ColumnPtr& column : partitions_columns) {
|
||||
if (column->has_null()) {
|
||||
return Status::NotSupported("Partition value can't be null.");
|
||||
}
|
||||
|
||||
std::string partition_value;
|
||||
RETURN_IF_ERROR(partition_value_to_string(ColumnHelper::get_data_column(column.get()), partition_value));
|
||||
partition_column_values.emplace_back(partition_value);
|
||||
}
|
||||
|
||||
DCHECK(_partition_column_names.size() == partition_column_values.size());
|
||||
|
||||
string partition_location = _get_partition_location(partition_column_values);
|
||||
|
||||
auto partition_writer = _partition_writers.find(partition_location);
|
||||
if (partition_writer == _partition_writers.end()) {
|
||||
tableInfo.partition_location = partition_location;
|
||||
std::vector<ExprContext*> data_col_exprs(_output_expr.begin(),
|
||||
_output_expr.begin() + _data_column_names.size());
|
||||
auto writer = std::make_unique<RollingAsyncParquetWriter>(tableInfo, data_col_exprs, _common_metrics.get(),
|
||||
add_hive_commit_info, state, _driver_sequence);
|
||||
_partition_writers.insert({partition_location, std::move(writer)});
|
||||
_partition_writers[partition_location]->append_chunk(chunk.get(), state);
|
||||
} else {
|
||||
partition_writer->second->append_chunk(chunk.get(), state);
|
||||
}
|
||||
}
|
||||
return Status::OK();
|
||||
}
|
||||
|
||||
std::string HiveTableSinkOperator::_get_partition_location(const std::vector<std::string>& values) {
|
||||
std::string partition_location = _location;
|
||||
for (size_t i = 0; i < _partition_column_names.size(); i++) {
|
||||
partition_location += _partition_column_names[i] + "=" + values[i] + "/";
|
||||
}
|
||||
return partition_location;
|
||||
}
|
||||
|
||||
Status HiveTableSinkOperator::partition_value_to_string(Column* column, std::string& partition_value) {
|
||||
auto v = column->get(0);
|
||||
if (column->is_date()) {
|
||||
partition_value = v.get_date().to_string();
|
||||
return Status::OK();
|
||||
}
|
||||
|
||||
bool not_support = false;
|
||||
v.visit([&](auto& variant) {
|
||||
std::visit(
|
||||
[&](auto&& arg) {
|
||||
using T = std::decay_t<decltype(arg)>;
|
||||
if constexpr (std::is_same_v<T, Slice>) {
|
||||
partition_value = arg.to_string();
|
||||
} else if constexpr (std::is_same_v<T, uint8_t> || std::is_same_v<T, int16_t> ||
|
||||
std::is_same_v<T, uint16_t> || std::is_same_v<T, uint24_t> ||
|
||||
std::is_same_v<T, int32_t> || std::is_same_v<T, uint32_t> ||
|
||||
std::is_same_v<T, int64_t> || std::is_same_v<T, uint64_t>) {
|
||||
partition_value = std::to_string(arg);
|
||||
} else if constexpr (std::is_same_v<T, int8_t>) {
|
||||
// iceberg has no smallint type. we can safely use int8 as boolean.
|
||||
partition_value = arg == 0 ? "false" : "true";
|
||||
} else {
|
||||
not_support = true;
|
||||
}
|
||||
},
|
||||
variant);
|
||||
});
|
||||
|
||||
if (not_support) {
|
||||
return Status::NotSupported(fmt::format("Partition value can't be {}", column->get_name()));
|
||||
}
|
||||
|
||||
return Status::OK();
|
||||
}
|
||||
|
||||
HiveTableSinkOperatorFactory::HiveTableSinkOperatorFactory(int32_t id, FragmentContext* fragment_ctx,
|
||||
const THiveTableSink& thrift_sink,
|
||||
vector<TExpr> t_output_expr,
|
||||
std::vector<ExprContext*> partition_expr_ctxs)
|
||||
: OperatorFactory(id, "hive_table_sink", Operator::s_pseudo_plan_node_id_for_hive_table_sink),
|
||||
_t_output_expr(std::move(t_output_expr)),
|
||||
_partition_expr_ctxs(std::move(partition_expr_ctxs)),
|
||||
_data_column_names(thrift_sink.data_column_names),
|
||||
_partition_column_names(thrift_sink.partition_column_names),
|
||||
_fragment_ctx(std::move(fragment_ctx)),
|
||||
_location(thrift_sink.staging_dir),
|
||||
_file_format(thrift_sink.file_format),
|
||||
_compression_codec(thrift_sink.compression_type),
|
||||
_cloud_conf(thrift_sink.cloud_configuration),
|
||||
is_static_partition_insert(thrift_sink.is_static_partition_sink) {
|
||||
DCHECK(thrift_sink.__isset.data_column_names);
|
||||
DCHECK(thrift_sink.__isset.partition_column_names);
|
||||
DCHECK(thrift_sink.__isset.staging_dir);
|
||||
DCHECK(thrift_sink.__isset.file_format);
|
||||
DCHECK(thrift_sink.__isset.compression_type);
|
||||
DCHECK(thrift_sink.__isset.cloud_configuration);
|
||||
DCHECK(thrift_sink.__isset.is_static_partition_sink);
|
||||
}
|
||||
|
||||
Status HiveTableSinkOperatorFactory::prepare(RuntimeState* state) {
|
||||
RETURN_IF_ERROR(OperatorFactory::prepare(state));
|
||||
|
||||
RETURN_IF_ERROR(Expr::create_expr_trees(state->obj_pool(), _t_output_expr, &_output_expr_ctxs, state));
|
||||
RETURN_IF_ERROR(Expr::prepare(_output_expr_ctxs, state));
|
||||
RETURN_IF_ERROR(Expr::open(_output_expr_ctxs, state));
|
||||
|
||||
RETURN_IF_ERROR(Expr::prepare(_partition_expr_ctxs, state));
|
||||
RETURN_IF_ERROR(Expr::open(_partition_expr_ctxs, state));
|
||||
|
||||
if (_file_format == "parquet") {
|
||||
std::vector<ExprContext*> data_col_exprs(_output_expr_ctxs.begin(),
|
||||
_output_expr_ctxs.begin() + _data_column_names.size());
|
||||
auto result = parquet::ParquetBuildHelper::make_schema(
|
||||
_data_column_names, data_col_exprs, std::vector<parquet::FileColumnId>(_data_column_names.size()));
|
||||
if (!result.ok()) {
|
||||
return Status::NotSupported(result.status().message());
|
||||
}
|
||||
_parquet_file_schema = result.ValueOrDie();
|
||||
}
|
||||
|
||||
return Status::OK();
|
||||
}
|
||||
|
||||
void HiveTableSinkOperatorFactory::close(RuntimeState* state) {
|
||||
Expr::close(_partition_expr_ctxs, state);
|
||||
Expr::close(_output_expr_ctxs, state);
|
||||
OperatorFactory::close(state);
|
||||
}
|
||||
|
||||
void HiveTableSinkOperator::add_hive_commit_info(starrocks::parquet::AsyncFileWriter* writer, RuntimeState* state) {
|
||||
THiveFileInfo hive_file_info;
|
||||
hive_file_info.__set_file_name(path_util::base_name(writer->file_location()));
|
||||
hive_file_info.__set_partition_path(writer->partition_location());
|
||||
hive_file_info.__set_record_count(writer->metadata()->num_rows());
|
||||
hive_file_info.__set_file_size_in_bytes(writer->file_size());
|
||||
|
||||
TSinkCommitInfo commit_info;
|
||||
commit_info.__set_hive_file_info(hive_file_info);
|
||||
|
||||
// update runtime state
|
||||
state->add_sink_commit_info(commit_info);
|
||||
state->update_num_rows_load_sink(hive_file_info.record_count);
|
||||
}
|
||||
|
||||
} // namespace starrocks::pipeline
|
||||
|
|
@ -0,0 +1,135 @@
|
|||
// Copyright 2021-present StarRocks, Inc. All rights reserved.
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// https://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 <gen_cpp/DataSinks_types.h>
|
||||
#include <parquet/arrow/writer.h>
|
||||
|
||||
#include <utility>
|
||||
|
||||
#include "common/logging.h"
|
||||
#include "exec/parquet_writer.h"
|
||||
#include "exec/pipeline/fragment_context.h"
|
||||
#include "exec/pipeline/operator.h"
|
||||
#include "fs/fs.h"
|
||||
|
||||
namespace starrocks {
|
||||
namespace pipeline {
|
||||
|
||||
class HiveTableSinkOperator final : public Operator {
|
||||
public:
|
||||
HiveTableSinkOperator(OperatorFactory* factory, int32_t id, int32_t plan_node_id, int32_t driver_sequence,
|
||||
std::string location, std::string file_format, TCompressionType::type compression_codec,
|
||||
const TCloudConfiguration& cloud_conf, FragmentContext* fragment_ctx,
|
||||
const std::shared_ptr<::parquet::schema::GroupNode>& schema,
|
||||
const std::vector<ExprContext*>& output_expr_ctxs,
|
||||
const vector<ExprContext*>& partition_output_expr,
|
||||
const vector<std::string>& partition_column_names,
|
||||
const vector<std::string>& data_column_names, bool is_static_partition_insert)
|
||||
: Operator(factory, id, "hive_table_sink", plan_node_id, driver_sequence),
|
||||
_location(std::move(location)),
|
||||
_file_format(std::move(file_format)),
|
||||
_compression_codec(std::move(compression_codec)),
|
||||
_cloud_conf(cloud_conf),
|
||||
_parquet_file_schema(std::move(schema)),
|
||||
_output_expr(output_expr_ctxs),
|
||||
_partition_expr(partition_output_expr),
|
||||
_is_static_partition_insert(is_static_partition_insert),
|
||||
_partition_column_names(partition_column_names),
|
||||
_data_column_names(data_column_names) {}
|
||||
|
||||
~HiveTableSinkOperator() override = default;
|
||||
|
||||
Status prepare(RuntimeState* state) override;
|
||||
|
||||
void close(RuntimeState* state) override;
|
||||
|
||||
bool has_output() const override { return false; }
|
||||
|
||||
bool need_input() const override;
|
||||
|
||||
bool is_finished() const override;
|
||||
|
||||
Status set_finishing(RuntimeState* state) override;
|
||||
|
||||
bool pending_finish() const override;
|
||||
|
||||
Status set_cancelled(RuntimeState* state) override;
|
||||
|
||||
StatusOr<ChunkPtr> pull_chunk(RuntimeState* state) override;
|
||||
|
||||
Status push_chunk(RuntimeState* state, const ChunkPtr& chunk) override;
|
||||
|
||||
static void add_hive_commit_info(starrocks::parquet::AsyncFileWriter* writer, RuntimeState* state);
|
||||
|
||||
static Status partition_value_to_string(Column* column, std::string& partition_value);
|
||||
|
||||
private:
|
||||
std::string _get_partition_location(const std::vector<std::string>& values);
|
||||
|
||||
std::string _location;
|
||||
std::string _file_format;
|
||||
TCompressionType::type _compression_codec;
|
||||
TCloudConfiguration _cloud_conf;
|
||||
|
||||
std::shared_ptr<::parquet::schema::GroupNode> _parquet_file_schema;
|
||||
std::vector<ExprContext*> _output_expr;
|
||||
std::vector<ExprContext*> _partition_expr;
|
||||
std::unordered_map<std::string, std::unique_ptr<starrocks::RollingAsyncParquetWriter>> _partition_writers;
|
||||
std::atomic<bool> _is_finished = false;
|
||||
bool _is_static_partition_insert = false;
|
||||
std::vector<std::string> _partition_column_names;
|
||||
std::vector<std::string> _data_column_names;
|
||||
};
|
||||
|
||||
class HiveTableSinkOperatorFactory final : public OperatorFactory {
|
||||
public:
|
||||
HiveTableSinkOperatorFactory(int32_t id, FragmentContext* fragment_ctx, const THiveTableSink& t_hive_table_sink,
|
||||
vector<TExpr> t_output_expr, std::vector<ExprContext*> partition_expr_ctxs);
|
||||
|
||||
~HiveTableSinkOperatorFactory() override = default;
|
||||
|
||||
OperatorPtr create(int32_t degree_of_parallelism, int32_t driver_sequence) override {
|
||||
return std::make_shared<HiveTableSinkOperator>(
|
||||
this, _id, _plan_node_id, driver_sequence, _location, _file_format, _compression_codec, _cloud_conf,
|
||||
_fragment_ctx, _parquet_file_schema,
|
||||
std::vector<ExprContext*>(_output_expr_ctxs.begin(),
|
||||
_output_expr_ctxs.begin() + _data_column_names.size()),
|
||||
_partition_expr_ctxs, _partition_column_names, _data_column_names, is_static_partition_insert);
|
||||
}
|
||||
|
||||
Status prepare(RuntimeState* state) override;
|
||||
|
||||
void close(RuntimeState* state) override;
|
||||
|
||||
private:
|
||||
std::vector<TExpr> _t_output_expr;
|
||||
std::vector<ExprContext*> _output_expr_ctxs;
|
||||
std::vector<ExprContext*> _partition_expr_ctxs;
|
||||
|
||||
std::vector<std::string> _data_column_names;
|
||||
std::vector<std::string> _partition_column_names;
|
||||
|
||||
FragmentContext* _fragment_ctx = nullptr;
|
||||
std::string _location;
|
||||
std::string _file_format;
|
||||
TCompressionType::type _compression_codec;
|
||||
TCloudConfiguration _cloud_conf;
|
||||
std::shared_ptr<::parquet::schema::GroupNode> _parquet_file_schema;
|
||||
bool is_static_partition_insert = false;
|
||||
};
|
||||
|
||||
} // namespace pipeline
|
||||
} // namespace starrocks
|
||||
|
|
@ -93,6 +93,7 @@ set(RUNTIME_FILES
|
|||
schema_table_sink.cpp
|
||||
command_executor.cpp
|
||||
iceberg_table_sink.cpp
|
||||
hive_table_sink.cpp
|
||||
)
|
||||
|
||||
set(RUNTIME_FILES ${RUNTIME_FILES}
|
||||
|
|
|
|||
|
|
@ -0,0 +1,58 @@
|
|||
// Copyright 2021-present StarRocks, Inc. All rights reserved.
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// https://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 "hive_table_sink.h"
|
||||
|
||||
#include "exprs/expr.h"
|
||||
#include "runtime/runtime_state.h"
|
||||
#include "util/runtime_profile.h"
|
||||
|
||||
namespace starrocks {
|
||||
|
||||
HiveTableSink::HiveTableSink(ObjectPool* pool, const std::vector<TExpr>& t_exprs)
|
||||
: _pool(pool), _t_output_expr(t_exprs) {}
|
||||
|
||||
HiveTableSink::~HiveTableSink() = default;
|
||||
|
||||
Status HiveTableSink::init(const TDataSink& thrift_sink, RuntimeState* state) {
|
||||
RETURN_IF_ERROR(DataSink::init(thrift_sink, state));
|
||||
RETURN_IF_ERROR(prepare(state));
|
||||
RETURN_IF_ERROR(open(state));
|
||||
return Status::OK();
|
||||
}
|
||||
|
||||
Status HiveTableSink::prepare(RuntimeState* state) {
|
||||
RETURN_IF_ERROR(DataSink::prepare(state));
|
||||
RETURN_IF_ERROR(Expr::prepare(_output_expr_ctxs, state));
|
||||
std::stringstream title;
|
||||
title << "IcebergTableSink (frag_id=" << state->fragment_instance_id() << ")";
|
||||
_profile = _pool->add(new RuntimeProfile(title.str()));
|
||||
return Status::OK();
|
||||
}
|
||||
|
||||
Status HiveTableSink::open(RuntimeState* state) {
|
||||
RETURN_IF_ERROR(Expr::open(_output_expr_ctxs, state));
|
||||
return Status::OK();
|
||||
}
|
||||
|
||||
Status HiveTableSink::send_chunk(RuntimeState* state, Chunk* chunk) {
|
||||
return Status::OK();
|
||||
}
|
||||
|
||||
Status HiveTableSink::close(RuntimeState* state, Status exec_status) {
|
||||
Expr::close(_output_expr_ctxs, state);
|
||||
return Status::OK();
|
||||
}
|
||||
|
||||
} // namespace starrocks
|
||||
|
|
@ -0,0 +1,51 @@
|
|||
// Copyright 2021-present StarRocks, Inc. All rights reserved.
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// https://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "common/logging.h"
|
||||
#include "exec/data_sink.h"
|
||||
|
||||
namespace starrocks {
|
||||
|
||||
class ExprContext;
|
||||
|
||||
class HiveTableSink : public DataSink {
|
||||
public:
|
||||
HiveTableSink(ObjectPool* pool, const std::vector<TExpr>& t_exprs);
|
||||
|
||||
~HiveTableSink() override;
|
||||
|
||||
Status init(const TDataSink& thrift_sink, RuntimeState* state) override;
|
||||
|
||||
Status prepare(RuntimeState* state) override;
|
||||
|
||||
Status open(RuntimeState* state) override;
|
||||
|
||||
Status send_chunk(RuntimeState* state, Chunk* chunk) override;
|
||||
|
||||
Status close(RuntimeState* state, Status exec_status) override;
|
||||
|
||||
RuntimeProfile* profile() override { return _profile; }
|
||||
|
||||
std::vector<TExpr> get_output_expr() const { return _t_output_expr; }
|
||||
|
||||
private:
|
||||
ObjectPool* _pool;
|
||||
const std::vector<TExpr>& _t_output_expr;
|
||||
std::vector<ExprContext*> _output_expr_ctxs;
|
||||
RuntimeProfile* _profile = nullptr;
|
||||
};
|
||||
|
||||
} // namespace starrocks
|
||||
|
|
@ -815,6 +815,10 @@ under the License.
|
|||
<version>2.4.0</version>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>io.airlift</groupId>
|
||||
<artifactId>concurrent</artifactId>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
|
||||
<build>
|
||||
|
|
|
|||
|
|
@ -56,6 +56,7 @@ import com.starrocks.common.StarRocksFEMetaVersion;
|
|||
import com.starrocks.common.io.Text;
|
||||
import com.starrocks.connector.RemoteFileInfo;
|
||||
import com.starrocks.connector.exception.StarRocksConnectorException;
|
||||
import com.starrocks.connector.hive.HiveStorageFormat;
|
||||
import com.starrocks.persist.ModifyTableColumnOperationLog;
|
||||
import com.starrocks.qe.ConnectContext;
|
||||
import com.starrocks.server.CatalogMgr;
|
||||
|
|
@ -82,15 +83,6 @@ import java.util.stream.Collectors;
|
|||
import static com.starrocks.server.CatalogMgr.ResourceMappingCatalog.getResourceMappingCatalogName;
|
||||
import static com.starrocks.server.CatalogMgr.ResourceMappingCatalog.isResourceMappingCatalog;
|
||||
|
||||
/**
|
||||
* External hive table
|
||||
* At the very beginning, hive table is only designed for spark load, and property hive.metastore.uris is used to
|
||||
* record hive metastore uris.
|
||||
* But when hive table supports query and there is a lot of hive tables,
|
||||
* using hive.resource property is more convenient to change hive config.
|
||||
* So we still remains the hive.metastore.uris property for compatible, but hive table only set hive.metastore.uris
|
||||
* dose not support query.
|
||||
*/
|
||||
public class HiveTable extends Table implements HiveMetaStoreTable {
|
||||
private static final Logger LOG = LogManager.getLogger(HiveTable.class);
|
||||
|
||||
|
|
@ -128,13 +120,16 @@ public class HiveTable extends Table implements HiveMetaStoreTable {
|
|||
// This error will happen when appending files to an existed partition on user side.
|
||||
private boolean useMetadataCache = true;
|
||||
|
||||
private HiveStorageFormat storageFormat;
|
||||
|
||||
public HiveTable() {
|
||||
super(TableType.HIVE);
|
||||
}
|
||||
|
||||
public HiveTable(long id, String name, List<Column> fullSchema, String resourceName, String catalog,
|
||||
String hiveDbName, String hiveTableName, String tableLocation, long createTime,
|
||||
List<String> partColumnNames, List<String> dataColumnNames, Map<String, String> properties) {
|
||||
List<String> partColumnNames, List<String> dataColumnNames, Map<String, String> properties,
|
||||
HiveStorageFormat storageFormat) {
|
||||
super(id, name, TableType.HIVE, fullSchema);
|
||||
this.resourceName = resourceName;
|
||||
this.catalogName = catalog;
|
||||
|
|
@ -145,6 +140,7 @@ public class HiveTable extends Table implements HiveMetaStoreTable {
|
|||
this.partColumnNames = partColumnNames;
|
||||
this.dataColumnNames = dataColumnNames;
|
||||
this.hiveProperties = properties;
|
||||
this.storageFormat = storageFormat;
|
||||
}
|
||||
|
||||
public String getHiveDbTable() {
|
||||
|
|
@ -170,6 +166,10 @@ public class HiveTable extends Table implements HiveMetaStoreTable {
|
|||
return hiveTableName;
|
||||
}
|
||||
|
||||
public HiveStorageFormat getStorageFormat() {
|
||||
return storageFormat;
|
||||
}
|
||||
|
||||
public boolean isUseMetadataCache() {
|
||||
if (ConnectContext.get() != null && ConnectContext.get().getSessionVariable().isEnableHiveMetadataCacheWithInsert()) {
|
||||
return true;
|
||||
|
|
@ -485,6 +485,11 @@ public class HiveTable extends Table implements HiveMetaStoreTable {
|
|||
return foreignKeyConstraints;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean supportInsert() {
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hashCode(getCatalogName(), hiveDbName, getTableIdentifier());
|
||||
|
|
@ -522,6 +527,7 @@ public class HiveTable extends Table implements HiveMetaStoreTable {
|
|||
private List<String> partitionColNames = Lists.newArrayList();
|
||||
private List<String> dataColNames = Lists.newArrayList();
|
||||
private Map<String, String> properties = Maps.newHashMap();
|
||||
private HiveStorageFormat storageFormat;
|
||||
|
||||
public Builder() {
|
||||
}
|
||||
|
|
@ -586,9 +592,14 @@ public class HiveTable extends Table implements HiveMetaStoreTable {
|
|||
return this;
|
||||
}
|
||||
|
||||
public Builder setStorageFormat(HiveStorageFormat storageFormat) {
|
||||
this.storageFormat = storageFormat;
|
||||
return this;
|
||||
}
|
||||
|
||||
public HiveTable build() {
|
||||
return new HiveTable(id, tableName, fullSchema, resourceName, catalogName, hiveDbName, hiveTableName,
|
||||
tableLocation, createTime, partitionColNames, dataColNames, properties);
|
||||
tableLocation, createTime, partitionColNames, dataColNames, properties, storageFormat);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -277,7 +277,6 @@ public class IcebergTable extends Table {
|
|||
return Objects.equal(catalogName, otherTable.getCatalogName()) &&
|
||||
Objects.equal(remoteDbName, otherTable.remoteDbName) &&
|
||||
Objects.equal(tableIdentifier, otherTable.getTableIdentifier());
|
||||
|
||||
}
|
||||
|
||||
public static Builder builder() {
|
||||
|
|
|
|||
|
|
@ -102,7 +102,11 @@ public class CachingRemoteFileIO implements RemoteFileIO {
|
|||
}
|
||||
|
||||
public void updateRemoteFiles(RemotePathKey pathKey) {
|
||||
cache.put(pathKey, loadRemoteFiles(pathKey));
|
||||
if (fileIO instanceof CachingRemoteFileIO) {
|
||||
((CachingRemoteFileIO) fileIO).updateRemoteFiles(pathKey);
|
||||
} else {
|
||||
cache.put(pathKey, loadRemoteFiles(pathKey));
|
||||
}
|
||||
}
|
||||
|
||||
public synchronized void invalidateAll() {
|
||||
|
|
|
|||
|
|
@ -21,7 +21,7 @@ import com.starrocks.credential.CloudConfiguration;
|
|||
import static com.google.common.base.Preconditions.checkArgument;
|
||||
import static java.util.Objects.requireNonNull;
|
||||
|
||||
public class CatalogConnector {
|
||||
public class CatalogConnector implements Connector {
|
||||
private final Connector normalConnector;
|
||||
private final Connector informationSchemaConnector;
|
||||
|
||||
|
|
|
|||
|
|
@ -628,4 +628,23 @@ public class PartitionUtil {
|
|||
throw UnsupportedException.unsupportedException("unsupported partition expr:" + partitionExpr);
|
||||
}
|
||||
}
|
||||
|
||||
public static String getPartitionName(String basePath, String partitionPath) {
|
||||
String basePathWithSlash = getPathWithSlash(basePath);
|
||||
String partitionPathWithSlash = getPathWithSlash(partitionPath);
|
||||
|
||||
if (basePathWithSlash.equals(partitionPathWithSlash)) {
|
||||
return "";
|
||||
}
|
||||
|
||||
Preconditions.checkState(partitionPath.startsWith(basePathWithSlash),
|
||||
"Can't infer partition name. base path: %s, partition path: %s", basePath, partitionPath);
|
||||
|
||||
partitionPath = partitionPath.endsWith("/") ? partitionPath.substring(0, partitionPath.length() - 1) : partitionPath;
|
||||
return partitionPath.substring(basePathWithSlash.length());
|
||||
}
|
||||
|
||||
public static String getPathWithSlash(String path) {
|
||||
return path.endsWith("/") ? path : path + "/";
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -18,34 +18,59 @@ package com.starrocks.connector;
|
|||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.starrocks.connector.exception.StarRocksConnectorException;
|
||||
import com.starrocks.connector.hive.HiveWriteUtils;
|
||||
import com.starrocks.connector.hive.Partition;
|
||||
import com.starrocks.sql.PlannerProfile;
|
||||
import jline.internal.Log;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.LocatedFileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.RemoteIterator;
|
||||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.Executor;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Future;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.function.Function;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static com.starrocks.connector.hive.HiveWriteUtils.checkedDelete;
|
||||
import static com.starrocks.connector.hive.HiveWriteUtils.createDirectory;
|
||||
import static com.starrocks.connector.hive.HiveWriteUtils.fileCreatedByQuery;
|
||||
|
||||
public class RemoteFileOperations {
|
||||
private static final Logger LOG = LogManager.getLogger(RemoteFileOperations.class);
|
||||
public static final String HMS_PARTITIONS_REMOTE_FILES = "HMS.PARTITIONS.LIST_FS_PARTITIONS";
|
||||
protected CachingRemoteFileIO remoteFileIO;
|
||||
private final ExecutorService executor;
|
||||
private final ExecutorService pullRemoteFileExecutor;
|
||||
private final Executor updateRemoteFilesExecutor;
|
||||
private final boolean isRecursive;
|
||||
private final boolean enableCatalogLevelCache;
|
||||
private final Configuration conf;
|
||||
|
||||
public RemoteFileOperations(CachingRemoteFileIO remoteFileIO,
|
||||
ExecutorService executor,
|
||||
ExecutorService pullRemoteFileExecutor,
|
||||
Executor updateRemoteFilesExecutor,
|
||||
boolean isRecursive,
|
||||
boolean enableCatalogLevelCache) {
|
||||
boolean enableCatalogLevelCache,
|
||||
Configuration conf) {
|
||||
this.remoteFileIO = remoteFileIO;
|
||||
this.executor = executor;
|
||||
this.pullRemoteFileExecutor = pullRemoteFileExecutor;
|
||||
this.updateRemoteFilesExecutor = updateRemoteFilesExecutor;
|
||||
this.isRecursive = isRecursive;
|
||||
this.enableCatalogLevelCache = enableCatalogLevelCache;
|
||||
this.conf = conf;
|
||||
}
|
||||
|
||||
public List<RemoteFileInfo> getRemoteFiles(List<Partition> partitions) {
|
||||
|
|
@ -82,7 +107,7 @@ public class RemoteFileOperations {
|
|||
try (PlannerProfile.ScopedTimer ignored = PlannerProfile.getScopedTimer(HMS_PARTITIONS_REMOTE_FILES)) {
|
||||
for (Partition partition : partitions) {
|
||||
RemotePathKey pathKey = RemotePathKey.of(partition.getFullPath(), isRecursive, hudiTableLocation);
|
||||
Future<Map<RemotePathKey, List<RemoteFileDesc>>> future = executor.submit(() ->
|
||||
Future<Map<RemotePathKey, List<RemoteFileDesc>>> future = pullRemoteFileExecutor.submit(() ->
|
||||
remoteFileIO.getRemoteFiles(pathKey, useCache));
|
||||
futures.add(future);
|
||||
}
|
||||
|
|
@ -128,6 +153,11 @@ public class RemoteFileOperations {
|
|||
}
|
||||
}
|
||||
|
||||
public void refreshPartitionFilesCache(Path path) {
|
||||
RemotePathKey remotePathKey = RemotePathKey.of(path.toString(), isRecursive);
|
||||
remoteFileIO.updateRemoteFiles(remotePathKey);
|
||||
}
|
||||
|
||||
private List<RemoteFileInfo> fillFileInfo(
|
||||
Map<RemotePathKey, List<RemoteFileDesc>> files,
|
||||
Map<RemotePathKey, Partition> partitions) {
|
||||
|
|
@ -157,4 +187,105 @@ public class RemoteFileOperations {
|
|||
public void invalidateAll() {
|
||||
remoteFileIO.invalidateAll();
|
||||
}
|
||||
|
||||
public Executor getUpdateFsExecutor() {
|
||||
return updateRemoteFilesExecutor;
|
||||
}
|
||||
|
||||
public void asyncRenameFiles(
|
||||
List<CompletableFuture<?>> renameFileFutures,
|
||||
AtomicBoolean cancelled,
|
||||
Path writePath,
|
||||
Path targetPath,
|
||||
List<String> fileNames) {
|
||||
FileSystem fileSystem;
|
||||
try {
|
||||
fileSystem = FileSystem.get(writePath.toUri(), conf);
|
||||
} catch (Exception e) {
|
||||
Log.error("Failed to get fileSystem", e);
|
||||
throw new StarRocksConnectorException("Failed to move data files to target location. " +
|
||||
"Failed to get file system on path %s. msg: %s", writePath, e.getMessage());
|
||||
}
|
||||
|
||||
for (String fileName : fileNames) {
|
||||
Path source = new Path(writePath, fileName);
|
||||
Path target = new Path(targetPath, fileName);
|
||||
renameFileFutures.add(CompletableFuture.runAsync(() -> {
|
||||
if (cancelled.get()) {
|
||||
return;
|
||||
}
|
||||
try {
|
||||
if (fileSystem.exists(target)) {
|
||||
throw new StarRocksConnectorException("Failed to move data files from %s to target location %s. msg:" +
|
||||
" target location already exists", source, target);
|
||||
}
|
||||
|
||||
if (!fileSystem.rename(source, target)) {
|
||||
throw new StarRocksConnectorException("Failed to move data files from %s to target location %s. msg:" +
|
||||
" rename operation failed", source, target);
|
||||
}
|
||||
} catch (IOException e) {
|
||||
LOG.error("Failed to rename data files", e);
|
||||
throw new StarRocksConnectorException("Failed to move data files from %s to final location %s. msg: %s",
|
||||
source, target, e.getMessage());
|
||||
}
|
||||
}, updateRemoteFilesExecutor));
|
||||
}
|
||||
}
|
||||
|
||||
public void renameDirectory(Path source, Path target, Runnable runWhenPathNotExist) {
|
||||
if (pathExists(target)) {
|
||||
throw new StarRocksConnectorException("Unable to rename from %s to %s. msg: target directory already exists",
|
||||
source, target);
|
||||
}
|
||||
|
||||
if (!pathExists(target.getParent())) {
|
||||
createDirectory(target.getParent(), conf);
|
||||
}
|
||||
|
||||
runWhenPathNotExist.run();
|
||||
|
||||
try {
|
||||
if (!FileSystem.get(source.toUri(), conf).rename(source, target)) {
|
||||
throw new StarRocksConnectorException("Failed to rename %s to %s: rename returned false", source, target);
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new StarRocksConnectorException("Failed to rename %s to %s, msg: %s", source, target, e.getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
public void removeNotCurrentQueryFiles(Path partitionPath, String queryId) {
|
||||
try {
|
||||
FileSystem fileSystem = FileSystem.get(partitionPath.toUri(), conf);
|
||||
RemoteIterator<LocatedFileStatus> iterator = fileSystem.listFiles(partitionPath, false);
|
||||
while (iterator.hasNext()) {
|
||||
Path file = iterator.next().getPath();
|
||||
if (!fileCreatedByQuery(file.getName(), queryId)) {
|
||||
checkedDelete(fileSystem, file, false);
|
||||
}
|
||||
}
|
||||
} catch (Exception e) {
|
||||
LOG.error("Failed to delete partition {} files when overwriting on s3", partitionPath, e);
|
||||
throw new StarRocksConnectorException("Failed to delete partition %s files during overwrite. msg: %s",
|
||||
partitionPath, e.getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
public boolean pathExists(Path path) {
|
||||
return HiveWriteUtils.pathExists(path, conf);
|
||||
}
|
||||
|
||||
public boolean deleteIfExists(Path path, boolean recursive) {
|
||||
return HiveWriteUtils.deleteIfExists(path, recursive, conf);
|
||||
}
|
||||
|
||||
public FileStatus[] listStatus(Path path) {
|
||||
try {
|
||||
FileSystem fileSystem = FileSystem.get(path.toUri(), conf);
|
||||
return fileSystem.listStatus(path);
|
||||
} catch (Exception e) {
|
||||
LOG.error("Failed to list path {}", path, e);
|
||||
throw new StarRocksConnectorException("Failed to list path %s. msg: %s", path.toString(), e.getMessage());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -271,6 +271,11 @@ public class CachingHiveMetastore implements IHiveMetastore {
|
|||
return get(partitionKeysCache, hivePartitionValue);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean partitionExists(String dbName, String tableName, List<String> partitionValues) {
|
||||
return metastore.partitionExists(dbName, tableName, partitionValues);
|
||||
}
|
||||
|
||||
private List<String> loadPartitionKeys(HivePartitionValue hivePartitionValue) {
|
||||
return metastore.getPartitionKeysByValue(hivePartitionValue.getHiveTableName().getDatabaseName(),
|
||||
hivePartitionValue.getHiveTableName().getTableName(), hivePartitionValue.getPartitionValues());
|
||||
|
|
@ -300,6 +305,30 @@ public class CachingHiveMetastore implements IHiveMetastore {
|
|||
return metastore.getPartition(key.getDatabaseName(), key.getTableName(), key.getPartitionValues());
|
||||
}
|
||||
|
||||
public void addPartitions(String dbName, String tableName, List<HivePartitionWithStats> partitions) {
|
||||
try {
|
||||
metastore.addPartitions(dbName, tableName, partitions);
|
||||
} finally {
|
||||
refreshTable(dbName, tableName, true);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void dropPartition(String dbName, String tableName, List<String> partValues, boolean deleteData) {
|
||||
List<String> partitionColNames = getTable(dbName, tableName).getPartitionColumnNames();
|
||||
try {
|
||||
metastore.dropPartition(dbName, tableName, partValues, deleteData);
|
||||
} finally {
|
||||
String partitionName = PartitionUtil.toHivePartitionName(partitionColNames, partValues);
|
||||
HivePartitionName hivePartitionName = HivePartitionName.of(dbName, tableName, partitionName);
|
||||
refreshPartition(Lists.newArrayList(hivePartitionName));
|
||||
}
|
||||
}
|
||||
|
||||
public void alterPartition(HivePartitionWithStats partition) {
|
||||
metastore.alterPartition(partition);
|
||||
}
|
||||
|
||||
public Map<String, Partition> getPartitionsByNames(String dbName, String tblName, List<String> partitionNames) {
|
||||
List<HivePartitionName> hivePartitionNames = partitionNames.stream()
|
||||
.map(partitionName -> HivePartitionName.of(dbName, tblName, partitionName))
|
||||
|
|
@ -342,6 +371,23 @@ public class CachingHiveMetastore implements IHiveMetastore {
|
|||
return metastore.getTableStatistics(hiveTableName.getDatabaseName(), hiveTableName.getTableName());
|
||||
}
|
||||
|
||||
public void updateTableStatistics(String dbName, String tableName, Function<HivePartitionStats, HivePartitionStats> update) {
|
||||
try {
|
||||
metastore.updateTableStatistics(dbName, tableName, update);
|
||||
} finally {
|
||||
refreshTable(dbName, tableName, true);
|
||||
}
|
||||
}
|
||||
|
||||
public void updatePartitionStatistics(String dbName, String tableName, String partitionName,
|
||||
Function<HivePartitionStats, HivePartitionStats> update) {
|
||||
try {
|
||||
metastore.updatePartitionStatistics(dbName, tableName, partitionName, update);
|
||||
} finally {
|
||||
refreshPartition(Lists.newArrayList(HivePartitionName.of(dbName, tableName, partitionName)));
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<String, HivePartitionStats> getPartitionStatistics(Table table, List<String> partitionNames) {
|
||||
String dbName = ((HiveMetaStoreTable) table).getDbName();
|
||||
|
|
|
|||
|
|
@ -0,0 +1,713 @@
|
|||
// Copyright 2021-present StarRocks, Inc. All rights reserved.
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// https://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package com.starrocks.connector.hive;
|
||||
|
||||
import com.google.common.base.Joiner;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.starrocks.catalog.HiveTable;
|
||||
import com.starrocks.common.Pair;
|
||||
import com.starrocks.common.Version;
|
||||
import com.starrocks.connector.RemoteFileOperations;
|
||||
import com.starrocks.connector.exception.StarRocksConnectorException;
|
||||
import com.starrocks.qe.ConnectContext;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.LinkedHashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.Queue;
|
||||
import java.util.Set;
|
||||
import java.util.StringJoiner;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
import java.util.concurrent.ConcurrentLinkedQueue;
|
||||
import java.util.concurrent.Executor;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static com.google.common.base.Verify.verify;
|
||||
import static com.starrocks.connector.PartitionUtil.toPartitionValues;
|
||||
import static com.starrocks.connector.hive.HiveMetadata.STARROCKS_QUERY_ID;
|
||||
import static com.starrocks.connector.hive.HivePartitionStats.ReduceOperator.SUBTRACT;
|
||||
import static com.starrocks.connector.hive.HivePartitionStats.fromCommonStats;
|
||||
import static com.starrocks.connector.hive.HiveWriteUtils.fileCreatedByQuery;
|
||||
import static com.starrocks.connector.hive.HiveWriteUtils.isS3Url;
|
||||
import static io.airlift.concurrent.MoreFutures.getFutureValue;
|
||||
import static java.util.Objects.requireNonNull;
|
||||
|
||||
public class HiveCommitter {
|
||||
private static final Logger LOG = LogManager.getLogger(HiveCommitter.class);
|
||||
private static final int PARTITION_COMMIT_BATCH_SIZE = 20;
|
||||
private final HiveTable table;
|
||||
private final HiveMetastoreOperations hmsOps;
|
||||
private final RemoteFileOperations fileOps;
|
||||
private final Executor updateStatsExecutor;
|
||||
private final AtomicBoolean fsTaskCancelled = new AtomicBoolean(false);
|
||||
private final List<CompletableFuture<?>> fsTaskFutures = new ArrayList<>();
|
||||
private final Queue<DirectoryCleanUpTask> clearTasksForAbort = new ConcurrentLinkedQueue<>();
|
||||
private final List<Path> clearPathsForFinish = new ArrayList<>();
|
||||
private final Set<Path> remoteFilesCacheToRefresh = new LinkedHashSet<>();
|
||||
private final AddPartitionsTask addPartitionsTask = new AddPartitionsTask();
|
||||
private final List<RenameDirectoryTask> renameDirTasksForAbort = new ArrayList<>();
|
||||
|
||||
private final List<UpdateStatisticsTask> updateStatisticsTasks = new ArrayList<>();
|
||||
private final Path stagingDir;
|
||||
|
||||
public HiveCommitter(HiveMetastoreOperations hmsOps, RemoteFileOperations fileOps,
|
||||
Executor updateStatsExecutor, HiveTable table, Path stagingDir) {
|
||||
this.hmsOps = hmsOps;
|
||||
this.fileOps = fileOps;
|
||||
this.updateStatsExecutor = updateStatsExecutor;
|
||||
this.table = table;
|
||||
this.stagingDir = stagingDir;
|
||||
}
|
||||
|
||||
public void commit(List<PartitionUpdate> partitionUpdates) {
|
||||
try {
|
||||
prepare(partitionUpdates);
|
||||
doCommit();
|
||||
} catch (Throwable t) {
|
||||
LOG.warn("Rolling back due to commit failure", t);
|
||||
try {
|
||||
cancelNotStartFsTasks();
|
||||
undoUpdateStatsTasks();
|
||||
undoAddPartitionsTask();
|
||||
waitAsyncFsTaskSuppressThrowable();
|
||||
runClearTasksForAbort();
|
||||
runRenameDirTasksForAbort();
|
||||
} catch (RuntimeException e) {
|
||||
t.addSuppressed(new Exception("Failed to roll back after commit failure", e));
|
||||
}
|
||||
throw t;
|
||||
} finally {
|
||||
runRefreshFilesCacheTasks();
|
||||
runClearPathsForFinish();
|
||||
clearStagingDir();
|
||||
}
|
||||
}
|
||||
|
||||
public void prepare(List<PartitionUpdate> partitionUpdates) {
|
||||
List<Pair<PartitionUpdate, HivePartitionStats>> insertExistsPartitions = new ArrayList<>();
|
||||
for (PartitionUpdate pu : partitionUpdates) {
|
||||
PartitionUpdate.UpdateMode mode = pu.getUpdateMode();
|
||||
HivePartitionStats updateStats = fromCommonStats(pu.getRowCount(), pu.getTotalSizeInBytes());
|
||||
if (table.isUnPartitioned()) {
|
||||
if (partitionUpdates.size() != 1) {
|
||||
throw new StarRocksConnectorException("There are multiple updates in the unpartition table: %s.%s",
|
||||
table.getDbName(), table.getTableName());
|
||||
}
|
||||
|
||||
if (mode == PartitionUpdate.UpdateMode.APPEND) {
|
||||
prepareAppendTable(pu, updateStats);
|
||||
} else if (mode == PartitionUpdate.UpdateMode.OVERWRITE) {
|
||||
prepareOverwriteTable(pu, updateStats);
|
||||
}
|
||||
} else {
|
||||
if (mode == PartitionUpdate.UpdateMode.NEW) {
|
||||
prepareAddPartition(pu, updateStats);
|
||||
} else if (mode == PartitionUpdate.UpdateMode.APPEND) {
|
||||
insertExistsPartitions.add(Pair.create(pu, updateStats));
|
||||
} else if (mode == PartitionUpdate.UpdateMode.OVERWRITE) {
|
||||
prepareOverwritePartition(pu, updateStats);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (!insertExistsPartitions.isEmpty()) {
|
||||
prepareAppendPartition(insertExistsPartitions);
|
||||
}
|
||||
}
|
||||
|
||||
public void doCommit() {
|
||||
waitAsyncFsTasks();
|
||||
runAddPartitionsTask();
|
||||
runUpdateStatsTasks();
|
||||
}
|
||||
|
||||
private void prepareAppendTable(PartitionUpdate pu, HivePartitionStats updateStats) {
|
||||
Path targetPath = new Path(table.getTableLocation());
|
||||
remoteFilesCacheToRefresh.add(targetPath);
|
||||
clearTasksForAbort.add(new DirectoryCleanUpTask(targetPath, false));
|
||||
if (!pu.isS3Url()) {
|
||||
fileOps.asyncRenameFiles(fsTaskFutures, fsTaskCancelled, pu.getWritePath(), pu.getTargetPath(), pu.getFileNames());
|
||||
}
|
||||
updateStatisticsTasks.add(new UpdateStatisticsTask(
|
||||
table.getDbName(),
|
||||
table.getTableName(),
|
||||
Optional.empty(),
|
||||
updateStats,
|
||||
true));
|
||||
}
|
||||
|
||||
private void prepareOverwriteTable(PartitionUpdate pu, HivePartitionStats updateStats) {
|
||||
Path writePath = pu.getWritePath();
|
||||
Path targetPath = pu.getTargetPath();
|
||||
remoteFilesCacheToRefresh.add(targetPath);
|
||||
|
||||
Path oldTableStagingPath = new Path(targetPath.getParent(), "_temp_" + targetPath.getName() + "_" +
|
||||
ConnectContext.get().getQueryId().toString());
|
||||
fileOps.renameDirectory(targetPath, oldTableStagingPath,
|
||||
() -> renameDirTasksForAbort.add(new RenameDirectoryTask(oldTableStagingPath, targetPath)));
|
||||
clearPathsForFinish.add(oldTableStagingPath);
|
||||
|
||||
fileOps.renameDirectory(writePath, targetPath,
|
||||
() -> clearTasksForAbort.add(new DirectoryCleanUpTask(targetPath, true)));
|
||||
|
||||
UpdateStatisticsTask updateStatsTask = new UpdateStatisticsTask(table.getDbName(), table.getTableName(),
|
||||
Optional.empty(), updateStats, false);
|
||||
updateStatisticsTasks.add(updateStatsTask);
|
||||
}
|
||||
|
||||
private void prepareAddPartition(PartitionUpdate pu, HivePartitionStats updateStats) {
|
||||
Path writePath = pu.getWritePath();
|
||||
Path targetPath = pu.getTargetPath();
|
||||
fsTaskFutures.add(CompletableFuture.runAsync(() -> {
|
||||
if (fsTaskCancelled.get()) {
|
||||
return;
|
||||
}
|
||||
|
||||
if (!pu.isS3Url()) {
|
||||
fileOps.renameDirectory(
|
||||
writePath,
|
||||
targetPath,
|
||||
() -> clearTasksForAbort.add(new DirectoryCleanUpTask(targetPath, true)));
|
||||
} else {
|
||||
clearTasksForAbort.add(new DirectoryCleanUpTask(targetPath, true));
|
||||
}
|
||||
}, fileOps.getUpdateFsExecutor()));
|
||||
|
||||
HivePartition hivePartition = buildHivePartition(pu);
|
||||
HivePartitionWithStats partitionWithStats = new HivePartitionWithStats(pu.getName(), hivePartition, updateStats);
|
||||
addPartitionsTask.addPartition(partitionWithStats);
|
||||
}
|
||||
|
||||
private void prepareAppendPartition(List<Pair<PartitionUpdate, HivePartitionStats>> partitions) {
|
||||
for (List<Pair<PartitionUpdate, HivePartitionStats>> partitionBatch : Iterables.partition(partitions, 100)) {
|
||||
List<String> partitionNames = partitionBatch.stream()
|
||||
.map(pair -> pair.first.getName())
|
||||
.collect(Collectors.toList());
|
||||
|
||||
Map<String, Partition> partitionsByNames = hmsOps.getPartitionByNames(table, partitionNames);
|
||||
|
||||
for (int i = 0; i < partitionsByNames.size(); i++) {
|
||||
String partitionName = partitionNames.get(i);
|
||||
if (partitionsByNames.get(partitionName) == null) {
|
||||
throw new StarRocksConnectorException("Partition [%s] not found", partitionName);
|
||||
}
|
||||
|
||||
PartitionUpdate pu = partitionBatch.get(i).first;
|
||||
HivePartitionStats updateStats = partitionBatch.get(i).second;
|
||||
|
||||
Path writePath = pu.getWritePath();
|
||||
Path targetPath = pu.getTargetPath();
|
||||
remoteFilesCacheToRefresh.add(targetPath);
|
||||
clearTasksForAbort.add(new DirectoryCleanUpTask(targetPath, false));
|
||||
|
||||
if (!pu.isS3Url()) {
|
||||
fileOps.asyncRenameFiles(fsTaskFutures, fsTaskCancelled, writePath, targetPath, pu.getFileNames());
|
||||
}
|
||||
|
||||
UpdateStatisticsTask updateStatsTask = new UpdateStatisticsTask(table.getDbName(), table.getTableName(),
|
||||
Optional.of(pu.getName()), updateStats, true);
|
||||
updateStatisticsTasks.add(updateStatsTask);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void prepareOverwritePartition(PartitionUpdate pu, HivePartitionStats updateStats) {
|
||||
Path writePath = pu.getWritePath();
|
||||
Path targetPath = pu.getTargetPath();
|
||||
|
||||
if (pu.isS3Url()) {
|
||||
String queryId = ConnectContext.get().getQueryId().toString();
|
||||
fileOps.removeNotCurrentQueryFiles(targetPath, queryId);
|
||||
} else {
|
||||
Path oldPartitionStagingPath = new Path(targetPath.getParent(), "_temp_" + targetPath.getName()
|
||||
+ "_" + ConnectContext.get().getQueryId().toString());
|
||||
|
||||
fileOps.renameDirectory(
|
||||
targetPath,
|
||||
oldPartitionStagingPath,
|
||||
() -> renameDirTasksForAbort.add(new RenameDirectoryTask(oldPartitionStagingPath, targetPath)));
|
||||
clearPathsForFinish.add(oldPartitionStagingPath);
|
||||
|
||||
fileOps.renameDirectory(
|
||||
writePath,
|
||||
targetPath,
|
||||
() -> clearTasksForAbort.add(new DirectoryCleanUpTask(targetPath, true)));
|
||||
}
|
||||
|
||||
remoteFilesCacheToRefresh.add(targetPath);
|
||||
UpdateStatisticsTask updateStatsTask = new UpdateStatisticsTask(table.getDbName(), table.getTableName(),
|
||||
Optional.of(pu.getName()), updateStats, false);
|
||||
updateStatisticsTasks.add(updateStatsTask);
|
||||
}
|
||||
|
||||
private void waitAsyncFsTasks() {
|
||||
for (CompletableFuture<?> future : fsTaskFutures) {
|
||||
getFutureValue(future, StarRocksConnectorException.class);
|
||||
}
|
||||
}
|
||||
|
||||
private void runAddPartitionsTask() {
|
||||
if (!addPartitionsTask.isEmpty()) {
|
||||
addPartitionsTask.run(hmsOps);
|
||||
}
|
||||
}
|
||||
|
||||
private void runUpdateStatsTasks() {
|
||||
ImmutableList.Builder<CompletableFuture<?>> updateStatsFutures = ImmutableList.builder();
|
||||
List<String> failedUpdateStatsTaskDescs = new ArrayList<>();
|
||||
List<Throwable> suppressedExceptions = new ArrayList<>();
|
||||
for (UpdateStatisticsTask task : updateStatisticsTasks) {
|
||||
updateStatsFutures.add(CompletableFuture.runAsync(() -> {
|
||||
try {
|
||||
task.run(hmsOps);
|
||||
} catch (Throwable t) {
|
||||
addSuppressedExceptions(suppressedExceptions, t, failedUpdateStatsTaskDescs, task.getDescription());
|
||||
}
|
||||
}, updateStatsExecutor));
|
||||
}
|
||||
|
||||
for (CompletableFuture<?> executeUpdateFuture : updateStatsFutures.build()) {
|
||||
getFutureValue(executeUpdateFuture);
|
||||
}
|
||||
|
||||
if (!suppressedExceptions.isEmpty()) {
|
||||
StringBuilder message = new StringBuilder();
|
||||
message.append("Failed to update following tasks: ");
|
||||
Joiner.on("; ").appendTo(message, failedUpdateStatsTaskDescs);
|
||||
StarRocksConnectorException exception = new StarRocksConnectorException(message.toString());
|
||||
suppressedExceptions.forEach(exception::addSuppressed);
|
||||
throw exception;
|
||||
}
|
||||
}
|
||||
|
||||
private void cancelNotStartFsTasks() {
|
||||
fsTaskCancelled.set(true);
|
||||
}
|
||||
|
||||
private void undoUpdateStatsTasks() {
|
||||
ImmutableList.Builder<CompletableFuture<?>> undoUpdateFutures = ImmutableList.builder();
|
||||
for (UpdateStatisticsTask task : updateStatisticsTasks) {
|
||||
undoUpdateFutures.add(CompletableFuture.runAsync(() -> {
|
||||
try {
|
||||
task.undo(hmsOps);
|
||||
} catch (Throwable throwable) {
|
||||
LOG.error("Failed to rollback: {}", task.getDescription(), throwable);
|
||||
}
|
||||
}, updateStatsExecutor));
|
||||
}
|
||||
|
||||
for (CompletableFuture<?> undoUpdateFuture : undoUpdateFutures.build()) {
|
||||
getFutureValue(undoUpdateFuture);
|
||||
}
|
||||
}
|
||||
|
||||
private void undoAddPartitionsTask() {
|
||||
if (addPartitionsTask.isEmpty()) {
|
||||
return;
|
||||
}
|
||||
|
||||
HivePartition firstPartition = addPartitionsTask.getPartitions().get(0).getHivePartition();
|
||||
String dbName = firstPartition.getDatabaseName();
|
||||
String tableName = firstPartition.getTableName();
|
||||
List<List<String>> rollbackFailedPartitions = addPartitionsTask.rollback(hmsOps);
|
||||
LOG.error("Failed to rollback: add_partition for partition values {}.{}.{}",
|
||||
dbName, tableName, rollbackFailedPartitions);
|
||||
}
|
||||
|
||||
private void waitAsyncFsTaskSuppressThrowable() {
|
||||
for (CompletableFuture<?> future : fsTaskFutures) {
|
||||
try {
|
||||
future.get();
|
||||
} catch (InterruptedException e) {
|
||||
Thread.currentThread().interrupt();
|
||||
} catch (Throwable t) {
|
||||
// ignore
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void runClearTasksForAbort() {
|
||||
for (DirectoryCleanUpTask cleanUpTask : clearTasksForAbort) {
|
||||
recursiveDeleteItems(cleanUpTask.getPath(), cleanUpTask.isDeleteEmptyDir());
|
||||
}
|
||||
}
|
||||
|
||||
private void runRenameDirTasksForAbort() {
|
||||
for (RenameDirectoryTask directoryRenameTask : renameDirTasksForAbort) {
|
||||
try {
|
||||
if (fileOps.pathExists(directoryRenameTask.getRenameFrom())) {
|
||||
fileOps.renameDirectory(directoryRenameTask.getRenameFrom(), directoryRenameTask.getRenameTo(), () -> {});
|
||||
}
|
||||
} catch (Throwable t) {
|
||||
LOG.error("Failed to undo rename dir from {} to {}",
|
||||
directoryRenameTask.getRenameFrom(), directoryRenameTask.getRenameTo(), t);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void runRefreshFilesCacheTasks() {
|
||||
for (Path path : remoteFilesCacheToRefresh) {
|
||||
fileOps.refreshPartitionFilesCache(path);
|
||||
}
|
||||
}
|
||||
|
||||
private void runClearPathsForFinish() {
|
||||
for (Path path : clearPathsForFinish) {
|
||||
try {
|
||||
if (!fileOps.deleteIfExists(path, true)) {
|
||||
LOG.warn("Failed to recursively delete path {}", path);
|
||||
}
|
||||
} catch (Exception e) {
|
||||
LOG.warn("Failed to recursively delete path {}", path, e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void clearStagingDir() {
|
||||
if (!isS3Url(stagingDir.toString()) && !fileOps.deleteIfExists(stagingDir, true)) {
|
||||
LOG.warn("Failed to clear staging dir {}", stagingDir);
|
||||
}
|
||||
}
|
||||
|
||||
private HivePartition buildHivePartition(PartitionUpdate partitionUpdate) {
|
||||
return HivePartition.builder()
|
||||
.setDatabaseName(table.getDbName())
|
||||
.setTableName(table.getTableName())
|
||||
.setColumns(table.getDataColumnNames().stream()
|
||||
.map(table::getColumn)
|
||||
.collect(Collectors.toList()))
|
||||
.setValues(toPartitionValues(partitionUpdate.getName()))
|
||||
.setParameters(ImmutableMap.<String, String>builder()
|
||||
.put("starrocks_version", Version.STARROCKS_VERSION + "-" + Version.STARROCKS_COMMIT_HASH)
|
||||
.put(STARROCKS_QUERY_ID, ConnectContext.get().getQueryId().toString())
|
||||
.buildOrThrow())
|
||||
.setStorageFormat(table.getStorageFormat())
|
||||
.setLocation(partitionUpdate.getTargetPath().toString())
|
||||
.build();
|
||||
}
|
||||
|
||||
private static class RenameDirectoryTask {
|
||||
private final Path renameFrom;
|
||||
private final Path renameTo;
|
||||
|
||||
public RenameDirectoryTask(Path renameFrom, Path renameTo) {
|
||||
this.renameFrom = requireNonNull(renameFrom, "renameFrom is null");
|
||||
this.renameTo = requireNonNull(renameTo, "renameTo is null");
|
||||
}
|
||||
|
||||
public Path getRenameFrom() {
|
||||
return renameFrom;
|
||||
}
|
||||
|
||||
public Path getRenameTo() {
|
||||
return renameTo;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return new StringJoiner(", ", RenameDirectoryTask.class.getSimpleName() + "[", "]")
|
||||
.add("renameFrom=" + renameFrom)
|
||||
.add("renameTo=" + renameTo)
|
||||
.toString();
|
||||
}
|
||||
}
|
||||
|
||||
private static class DirectoryCleanUpTask {
|
||||
private final Path path;
|
||||
private final boolean deleteEmptyDir;
|
||||
|
||||
public DirectoryCleanUpTask(Path path, boolean deleteEmptyDir) {
|
||||
this.path = path;
|
||||
this.deleteEmptyDir = deleteEmptyDir;
|
||||
}
|
||||
|
||||
public Path getPath() {
|
||||
return path;
|
||||
}
|
||||
|
||||
public boolean isDeleteEmptyDir() {
|
||||
return deleteEmptyDir;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return new StringJoiner(", ", DirectoryCleanUpTask.class.getSimpleName() + "[", "]")
|
||||
.add("path=" + path)
|
||||
.add("deleteEmptyDir=" + deleteEmptyDir)
|
||||
.toString();
|
||||
}
|
||||
}
|
||||
|
||||
public static class AddPartitionsTask {
|
||||
private final List<HivePartitionWithStats> partitions = new ArrayList<>();
|
||||
private List<List<String>> createdPartitionValues = new ArrayList<>();
|
||||
|
||||
public boolean isEmpty() {
|
||||
return partitions.isEmpty();
|
||||
}
|
||||
|
||||
public List<HivePartitionWithStats> getPartitions() {
|
||||
return partitions;
|
||||
}
|
||||
|
||||
public void addPartition(HivePartitionWithStats partition) {
|
||||
partitions.add(partition);
|
||||
}
|
||||
|
||||
public void run(HiveMetastoreOperations hmsOps) {
|
||||
HivePartition firstPartition = partitions.get(0).getHivePartition();
|
||||
String dbName = firstPartition.getDatabaseName();
|
||||
String tableName = firstPartition.getTableName();
|
||||
List<List<HivePartitionWithStats>> batchedPartitions = Lists.partition(partitions, PARTITION_COMMIT_BATCH_SIZE);
|
||||
for (List<HivePartitionWithStats> batch : batchedPartitions) {
|
||||
try {
|
||||
hmsOps.addPartitions(dbName, tableName, batch);
|
||||
for (HivePartitionWithStats partition : batch) {
|
||||
createdPartitionValues.add(partition.getHivePartition().getValues());
|
||||
}
|
||||
} catch (Throwable t) {
|
||||
LOG.error("Failed to add partition", t);
|
||||
boolean addedSuccess = true;
|
||||
for (HivePartitionWithStats partition : batch) {
|
||||
try {
|
||||
Partition remotePartition = hmsOps.getPartition(
|
||||
dbName, tableName, partition.getHivePartition().getValues());
|
||||
|
||||
if (checkIsSamePartition(remotePartition, partition.getHivePartition())) {
|
||||
createdPartitionValues.add(partition.getHivePartition().getValues());
|
||||
} else {
|
||||
addedSuccess = false;
|
||||
}
|
||||
} catch (Throwable ignored) {
|
||||
addedSuccess = false;
|
||||
}
|
||||
}
|
||||
|
||||
if (!addedSuccess) {
|
||||
throw t;
|
||||
}
|
||||
}
|
||||
}
|
||||
partitions.clear();
|
||||
}
|
||||
|
||||
public List<List<String>> rollback(HiveMetastoreOperations hmsOps) {
|
||||
HivePartition firstPartition = partitions.get(0).getHivePartition();
|
||||
String dbName = firstPartition.getDatabaseName();
|
||||
String tableName = firstPartition.getTableName();
|
||||
List<List<String>> rollbackFailedPartitions = new ArrayList<>();
|
||||
for (List<String> createdPartitionValue : createdPartitionValues) {
|
||||
try {
|
||||
hmsOps.dropPartition(dbName, tableName, createdPartitionValue, false);
|
||||
} catch (Throwable t) {
|
||||
LOG.warn("Failed to drop partition on {}.{}.{} when rollback",
|
||||
dbName, tableName, rollbackFailedPartitions);
|
||||
rollbackFailedPartitions.add(createdPartitionValue);
|
||||
}
|
||||
}
|
||||
createdPartitionValues = rollbackFailedPartitions;
|
||||
return rollbackFailedPartitions;
|
||||
}
|
||||
}
|
||||
|
||||
// 1. alter table or alter partition
|
||||
// 2. update table or partition statistics
|
||||
private static class UpdateStatisticsTask {
|
||||
private final String dbName;
|
||||
private final String tableName;
|
||||
private final Optional<String> partitionName;
|
||||
private final HivePartitionStats updatePartitionStat;
|
||||
private final boolean merge;
|
||||
|
||||
private boolean done;
|
||||
|
||||
public UpdateStatisticsTask(String dbName, String tableName, Optional<String> partitionName,
|
||||
HivePartitionStats statistics, boolean merge) {
|
||||
this.dbName = requireNonNull(dbName, "dbName is null");
|
||||
this.tableName = requireNonNull(tableName, "tableName is null");
|
||||
this.partitionName = requireNonNull(partitionName, "partitionName is null");
|
||||
this.updatePartitionStat = requireNonNull(statistics, "statistics is null");
|
||||
this.merge = merge;
|
||||
}
|
||||
|
||||
public void run(HiveMetastoreOperations hmsOps) {
|
||||
if (partitionName.isPresent()) {
|
||||
hmsOps.updatePartitionStatistics(dbName, tableName, partitionName.get(), this::updateStatistics);
|
||||
} else {
|
||||
hmsOps.updateTableStatistics(dbName, tableName, this::updateStatistics);
|
||||
}
|
||||
done = true;
|
||||
}
|
||||
|
||||
public void undo(HiveMetastoreOperations hmsOps) {
|
||||
if (!done) {
|
||||
return;
|
||||
}
|
||||
if (partitionName.isPresent()) {
|
||||
hmsOps.updatePartitionStatistics(dbName, tableName, partitionName.get(), this::resetStatistics);
|
||||
} else {
|
||||
hmsOps.updateTableStatistics(dbName, tableName, this::resetStatistics);
|
||||
}
|
||||
}
|
||||
|
||||
public String getDescription() {
|
||||
if (partitionName.isPresent()) {
|
||||
return "alter partition parameters " + tableName + partitionName.get();
|
||||
} else {
|
||||
return "alter table parameters " + tableName;
|
||||
}
|
||||
}
|
||||
|
||||
private HivePartitionStats updateStatistics(HivePartitionStats currentStats) {
|
||||
return merge ? HivePartitionStats.merge(currentStats, updatePartitionStat) : updatePartitionStat;
|
||||
}
|
||||
|
||||
private HivePartitionStats resetStatistics(HivePartitionStats currentStatistics) {
|
||||
return HivePartitionStats.reduce(currentStatistics, updatePartitionStat, SUBTRACT);
|
||||
}
|
||||
}
|
||||
|
||||
public static class DeleteRecursivelyResult {
|
||||
private final boolean dirNotExists;
|
||||
private final List<String> notDeletedEligibleItems;
|
||||
|
||||
public DeleteRecursivelyResult(boolean dirNotExists, List<String> notDeletedEligibleItems) {
|
||||
this.dirNotExists = dirNotExists;
|
||||
this.notDeletedEligibleItems = notDeletedEligibleItems;
|
||||
}
|
||||
|
||||
public boolean dirNotExists() {
|
||||
return dirNotExists;
|
||||
}
|
||||
|
||||
public List<String> getNotDeletedEligibleItems() {
|
||||
return notDeletedEligibleItems;
|
||||
}
|
||||
}
|
||||
|
||||
public static Optional<String> getQueryId(Map<String, String> params) {
|
||||
params = params != null ? params : new HashMap<>();
|
||||
return Optional.ofNullable(params.get(STARROCKS_QUERY_ID));
|
||||
}
|
||||
|
||||
public static boolean checkIsSamePartition(Partition remotePartition, HivePartition curPartition) {
|
||||
if (remotePartition == null) {
|
||||
return false;
|
||||
}
|
||||
|
||||
if (!getQueryId(remotePartition.getParameters()).isPresent()) {
|
||||
return false;
|
||||
}
|
||||
|
||||
return getQueryId(remotePartition.getParameters()).equals(getQueryId(curPartition.getParameters()));
|
||||
}
|
||||
|
||||
private void recursiveDeleteItems(Path directory, boolean deleteEmptyDir) {
|
||||
DeleteRecursivelyResult deleteResult = recursiveDeleteFiles(directory, deleteEmptyDir);
|
||||
|
||||
if (!deleteResult.getNotDeletedEligibleItems().isEmpty()) {
|
||||
LOG.error("Failed to delete directory {}. Some eligible items can't be deleted: {}.",
|
||||
directory.toString(), deleteResult.getNotDeletedEligibleItems());
|
||||
} else if (deleteEmptyDir && !deleteResult.dirNotExists()) {
|
||||
LOG.error("Failed to delete directory {} due to dir isn't empty", directory.toString());
|
||||
}
|
||||
}
|
||||
|
||||
public DeleteRecursivelyResult recursiveDeleteFiles(Path directory, boolean deleteEmptyDir) {
|
||||
try {
|
||||
if (!fileOps.pathExists(directory)) {
|
||||
return new DeleteRecursivelyResult(true, ImmutableList.of());
|
||||
}
|
||||
} catch (StarRocksConnectorException e) {
|
||||
ImmutableList.Builder<String> notDeletedEligibleItems = ImmutableList.builder();
|
||||
notDeletedEligibleItems.add(directory.toString() + "/*");
|
||||
return new DeleteRecursivelyResult(false, notDeletedEligibleItems.build());
|
||||
}
|
||||
|
||||
return doRecursiveDeleteFiles(directory, ConnectContext.get().getQueryId().toString(), deleteEmptyDir);
|
||||
}
|
||||
|
||||
private DeleteRecursivelyResult doRecursiveDeleteFiles(Path directory, String queryId, boolean deleteEmptyDir) {
|
||||
FileStatus[] allFiles;
|
||||
try {
|
||||
allFiles = fileOps.listStatus(directory);
|
||||
} catch (StarRocksConnectorException e) {
|
||||
ImmutableList.Builder<String> notDeletedEligibleItems = ImmutableList.builder();
|
||||
notDeletedEligibleItems.add(directory + "/*");
|
||||
return new DeleteRecursivelyResult(false, notDeletedEligibleItems.build());
|
||||
}
|
||||
|
||||
boolean isEmptyDir = true;
|
||||
List<String> notDeletedEligibleItems = new ArrayList<>();
|
||||
for (FileStatus fileStatus : allFiles) {
|
||||
if (fileStatus.isFile()) {
|
||||
Path filePath = fileStatus.getPath();
|
||||
String fileName = filePath.getName();
|
||||
boolean eligible = fileCreatedByQuery(fileName, queryId);
|
||||
|
||||
if (eligible) {
|
||||
if (!fileOps.deleteIfExists(filePath, false)) {
|
||||
isEmptyDir = false;
|
||||
notDeletedEligibleItems.add(filePath.toString());
|
||||
}
|
||||
} else {
|
||||
isEmptyDir = false;
|
||||
}
|
||||
} else if (fileStatus.isDirectory()) {
|
||||
DeleteRecursivelyResult subResult = doRecursiveDeleteFiles(fileStatus.getPath(), queryId, deleteEmptyDir);
|
||||
if (!subResult.dirNotExists()) {
|
||||
isEmptyDir = false;
|
||||
}
|
||||
if (!subResult.getNotDeletedEligibleItems().isEmpty()) {
|
||||
notDeletedEligibleItems.addAll(subResult.getNotDeletedEligibleItems());
|
||||
}
|
||||
} else {
|
||||
isEmptyDir = false;
|
||||
notDeletedEligibleItems.add(fileStatus.getPath().toString());
|
||||
}
|
||||
}
|
||||
|
||||
if (isEmptyDir && deleteEmptyDir) {
|
||||
verify(notDeletedEligibleItems.isEmpty());
|
||||
if (!fileOps.deleteIfExists(directory, false)) {
|
||||
return new DeleteRecursivelyResult(false, ImmutableList.of(directory + "/"));
|
||||
}
|
||||
// all items of the location have been deleted.
|
||||
return new DeleteRecursivelyResult(true, ImmutableList.of());
|
||||
}
|
||||
|
||||
return new DeleteRecursivelyResult(false, notDeletedEligibleItems);
|
||||
}
|
||||
|
||||
private synchronized void addSuppressedExceptions(
|
||||
List<Throwable> suppressedExceptions, Throwable t,
|
||||
List<String> descriptions, String description) {
|
||||
descriptions.add(description);
|
||||
if (suppressedExceptions.size() < 3) {
|
||||
suppressedExceptions.add(t);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -62,6 +62,8 @@ public class HiveConnector implements Connector {
|
|||
internalMgr.getHiveMetastoreConf(),
|
||||
internalMgr.getRemoteFileConf(),
|
||||
internalMgr.getPullRemoteFileExecutor(),
|
||||
internalMgr.getupdateRemoteFilesExecutor(),
|
||||
internalMgr.getUpdateStatisticsExecutor(),
|
||||
internalMgr.isSearchRecursive(),
|
||||
internalMgr.enableHmsEventsIncrementalSync(),
|
||||
hdfsEnvironment.getConfiguration(),
|
||||
|
|
|
|||
|
|
@ -30,6 +30,7 @@ import com.starrocks.sql.analyzer.SemanticException;
|
|||
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.Executor;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Executors;
|
||||
|
||||
|
|
@ -51,9 +52,12 @@ public class HiveConnectorInternalMgr {
|
|||
private ExecutorService refreshHiveMetastoreExecutor;
|
||||
private ExecutorService refreshRemoteFileExecutor;
|
||||
private ExecutorService pullRemoteFileExecutor;
|
||||
private ExecutorService updateRemoteFilesExecutor;
|
||||
private ExecutorService updateStatisticsExecutor;
|
||||
|
||||
private final boolean isRecursive;
|
||||
private final int loadRemoteFileMetadataThreadNum;
|
||||
private final int updateRemoteFileMetadataThreadNum;
|
||||
private final boolean enableHmsEventsIncrementalSync;
|
||||
|
||||
private final boolean enableBackgroundRefreshHiveMetadata;
|
||||
|
|
@ -72,6 +76,8 @@ public class HiveConnectorInternalMgr {
|
|||
this.isRecursive = Boolean.parseBoolean(properties.getOrDefault("enable_recursive_listing", "true"));
|
||||
this.loadRemoteFileMetadataThreadNum = Integer.parseInt(properties.getOrDefault("remote_file_load_thread_num",
|
||||
String.valueOf(Config.remote_file_metadata_load_concurrency)));
|
||||
this.updateRemoteFileMetadataThreadNum = Integer.parseInt(properties.getOrDefault("remote_file_update_thread_num",
|
||||
String.valueOf(Config.remote_file_metadata_load_concurrency / 4)));
|
||||
this.enableHmsEventsIncrementalSync = Boolean.parseBoolean(properties.getOrDefault("enable_hms_events_incremental_sync",
|
||||
String.valueOf(Config.enable_hms_events_incremental_sync)));
|
||||
|
||||
|
|
@ -156,6 +162,21 @@ public class HiveConnectorInternalMgr {
|
|||
return pullRemoteFileExecutor;
|
||||
}
|
||||
|
||||
public ExecutorService getupdateRemoteFilesExecutor() {
|
||||
if (updateRemoteFilesExecutor == null) {
|
||||
updateRemoteFilesExecutor = Executors.newFixedThreadPool(updateRemoteFileMetadataThreadNum,
|
||||
new ThreadFactoryBuilder().setNameFormat("update-hive-remote-files-%d").build());
|
||||
}
|
||||
|
||||
return updateRemoteFilesExecutor;
|
||||
}
|
||||
|
||||
public Executor getUpdateStatisticsExecutor() {
|
||||
Executor baseExecutor = Executors.newCachedThreadPool(
|
||||
new ThreadFactoryBuilder().setNameFormat("hive-metastore-update-%d").build());
|
||||
return new ReentrantExecutor(baseExecutor, remoteFileConf.getRefreshMaxThreadNum());
|
||||
}
|
||||
|
||||
public boolean isSearchRecursive() {
|
||||
return isRecursive;
|
||||
}
|
||||
|
|
|
|||
|
|
@ -155,7 +155,7 @@ public class HiveMetaClient {
|
|||
argClasses = argClasses == null ? ClassUtils.getCompatibleParamClasses(args) : argClasses;
|
||||
Method method = client.hiveClient.getClass().getDeclaredMethod(methodName, argClasses);
|
||||
return (T) method.invoke(client.hiveClient, args);
|
||||
} catch (Exception e) {
|
||||
} catch (Throwable e) {
|
||||
LOG.error(messageIfError, e);
|
||||
connectionException = new StarRocksConnectorException(messageIfError + ", msg: " + e.getMessage(), e);
|
||||
throw connectionException;
|
||||
|
|
@ -213,6 +213,22 @@ public class HiveMetaClient {
|
|||
}
|
||||
}
|
||||
|
||||
public void alterTable(String dbName, String tableName, Table newTable) {
|
||||
Class<?>[] argClasses = {String.class, String.class, Table.class};
|
||||
try (PlannerProfile.ScopedTimer ignored = PlannerProfile.getScopedTimer("HMS.alterTable")) {
|
||||
callRPC("alter_table", "Failed to alter table " + dbName + "." + tableName,
|
||||
argClasses, dbName, tableName, newTable);
|
||||
}
|
||||
}
|
||||
|
||||
public void alterPartition(String dbName, String tableName, Partition newPartition) {
|
||||
Class<?>[] argClasses = {String.class, String.class, Partition.class};
|
||||
try (PlannerProfile.ScopedTimer ignored = PlannerProfile.getScopedTimer("HMS.alterPartition")) {
|
||||
callRPC("alter_partition", "Failed to alter partition " + dbName + "." + tableName + newPartition.getValues(),
|
||||
argClasses, dbName, tableName, newPartition);
|
||||
}
|
||||
}
|
||||
|
||||
public List<String> getPartitionKeys(String dbName, String tableName) {
|
||||
try (PlannerProfile.ScopedTimer ignored = PlannerProfile.getScopedTimer("HMS.listPartitionNames")) {
|
||||
return callRPC("listPartitionNames", String.format("Failed to get partitionKeys on [%s.%s]", dbName, tableName),
|
||||
|
|
@ -247,6 +263,20 @@ public class HiveMetaClient {
|
|||
}
|
||||
}
|
||||
|
||||
public void addPartitions(String dbName, String tableName, List<Partition> partitions) {
|
||||
try (PlannerProfile.ScopedTimer ignored = PlannerProfile.getScopedTimer("HMS.addPartitions")) {
|
||||
callRPC("add_partitions", String.format("Failed to add partitions on %s.%s",
|
||||
dbName, tableName), partitions);
|
||||
}
|
||||
}
|
||||
|
||||
public void dropPartition(String dbName, String tableName, List<String> partValues, boolean deleteData) {
|
||||
try (PlannerProfile.ScopedTimer ignored = PlannerProfile.getScopedTimer("HMS.dropPartition")) {
|
||||
callRPC("dropPartition", String.format("Failed to drop partition on %s.%s.%s",
|
||||
dbName, tableName, partValues), dbName, tableName, partValues, deleteData);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Both 'getPartitionsByNames' and 'getPartitionColumnStatistics' could throw exception or no response
|
||||
* when querying too many partitions at present. Due to statistics don't affect accuracy, user could adjust
|
||||
|
|
|
|||
|
|
@ -12,7 +12,6 @@
|
|||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
|
||||
package com.starrocks.connector.hive;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
|
|
@ -32,6 +31,7 @@ import com.starrocks.connector.PartitionInfo;
|
|||
import com.starrocks.connector.RemoteFileInfo;
|
||||
import com.starrocks.connector.RemoteFileOperations;
|
||||
import com.starrocks.connector.exception.StarRocksConnectorException;
|
||||
import com.starrocks.connector.hive.PartitionUpdate.UpdateMode;
|
||||
import com.starrocks.qe.SessionVariable;
|
||||
import com.starrocks.server.GlobalStateMgr;
|
||||
import com.starrocks.sql.ast.CreateTableStmt;
|
||||
|
|
@ -41,34 +41,43 @@ import com.starrocks.sql.optimizer.operator.scalar.ColumnRefOperator;
|
|||
import com.starrocks.sql.optimizer.operator.scalar.ScalarOperator;
|
||||
import com.starrocks.sql.optimizer.statistics.ColumnStatistic;
|
||||
import com.starrocks.sql.optimizer.statistics.Statistics;
|
||||
import com.starrocks.thrift.TSinkCommitInfo;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.concurrent.Executor;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static com.starrocks.connector.PartitionUtil.toHivePartitionName;
|
||||
import static com.starrocks.connector.PartitionUtil.toPartitionValues;
|
||||
import static com.starrocks.server.CatalogMgr.ResourceMappingCatalog.isResourceMappingCatalog;
|
||||
|
||||
public class HiveMetadata implements ConnectorMetadata {
|
||||
private static final Logger LOG = LogManager.getLogger(HiveMetadata.class);
|
||||
public static final String STARROCKS_QUERY_ID = "starrocks_query_id";
|
||||
private final String catalogName;
|
||||
private final HiveMetastoreOperations hmsOps;
|
||||
private final RemoteFileOperations fileOps;
|
||||
private final HiveStatisticsProvider statisticsProvider;
|
||||
private final Optional<CacheUpdateProcessor> cacheUpdateProcessor;
|
||||
private Executor updateExecutor;
|
||||
|
||||
public HiveMetadata(String catalogName,
|
||||
HiveMetastoreOperations hmsOps,
|
||||
RemoteFileOperations fileOperations,
|
||||
HiveStatisticsProvider statisticsProvider,
|
||||
Optional<CacheUpdateProcessor> cacheUpdateProcessor) {
|
||||
Optional<CacheUpdateProcessor> cacheUpdateProcessor,
|
||||
Executor updateExecutor) {
|
||||
this.catalogName = catalogName;
|
||||
this.hmsOps = hmsOps;
|
||||
this.fileOps = fileOperations;
|
||||
this.statisticsProvider = statisticsProvider;
|
||||
this.cacheUpdateProcessor = cacheUpdateProcessor;
|
||||
this.updateExecutor = updateExecutor;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
@ -115,6 +124,25 @@ public class HiveMetadata implements ConnectorMetadata {
|
|||
return hmsOps.createTable(stmt);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void dropTable(DropTableStmt stmt) throws DdlException {
|
||||
String dbName = stmt.getDbName();
|
||||
String tableName = stmt.getTableName();
|
||||
if (isResourceMappingCatalog(catalogName)) {
|
||||
HiveMetaStoreTable hmsTable = (HiveMetaStoreTable) GlobalStateMgr.getCurrentState()
|
||||
.getMetadata().getTable(dbName, tableName);
|
||||
cacheUpdateProcessor.ifPresent(processor -> processor.invalidateTable(
|
||||
hmsTable.getDbName(), hmsTable.getTableName(), hmsTable.getTableLocation()));
|
||||
} else {
|
||||
if (!stmt.isForceDrop()) {
|
||||
throw new DdlException(String.format("Table location will be cleared." +
|
||||
" 'Force' must be set when dropping a hive table." +
|
||||
" Please execute 'drop table %s.%s.%s force'", stmt.getCatalogName(), dbName, tableName));
|
||||
}
|
||||
hmsOps.dropTable(dbName, tableName);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Table getTable(String dbName, String tblName) {
|
||||
Table table;
|
||||
|
|
@ -234,22 +262,39 @@ public class HiveMetadata implements ConnectorMetadata {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void dropTable(DropTableStmt stmt) throws DdlException {
|
||||
String dbName = stmt.getDbName();
|
||||
String tableName = stmt.getTableName();
|
||||
if (isResourceMappingCatalog(catalogName)) {
|
||||
HiveMetaStoreTable hmsTable = (HiveMetaStoreTable) GlobalStateMgr.getCurrentState()
|
||||
.getMetadata().getTable(dbName, tableName);
|
||||
cacheUpdateProcessor.ifPresent(processor -> processor.invalidateTable(
|
||||
hmsTable.getDbName(), hmsTable.getTableName(), hmsTable.getTableLocation()));
|
||||
} else {
|
||||
if (!stmt.isForceDrop()) {
|
||||
throw new DdlException(String.format("Table location will be cleared." +
|
||||
" 'Force' must be set when dropping a hive table." +
|
||||
" Please execute 'drop table %s.%s.%s force'", stmt.getCatalogName(), dbName, tableName));
|
||||
public void finishSink(String dbName, String tableName, List<TSinkCommitInfo> commitInfos) {
|
||||
HiveTable table = (HiveTable) getTable(dbName, tableName);
|
||||
String stagingDir = commitInfos.get(0).getStaging_dir();
|
||||
boolean isOverwrite = commitInfos.get(0).isIs_overwrite();
|
||||
|
||||
List<PartitionUpdate> partitionUpdates = commitInfos.stream()
|
||||
.map(TSinkCommitInfo::getHive_file_info)
|
||||
.map(fileInfo -> PartitionUpdate.get(fileInfo, stagingDir, table.getTableLocation()))
|
||||
.collect(Collectors.collectingAndThen(Collectors.toList(), PartitionUpdate::merge));
|
||||
|
||||
List<String> partitionColNames = table.getPartitionColumnNames();
|
||||
for (PartitionUpdate partitionUpdate : partitionUpdates) {
|
||||
PartitionUpdate.UpdateMode mode;
|
||||
if (table.isUnPartitioned()) {
|
||||
mode = isOverwrite ? UpdateMode.OVERWRITE : UpdateMode.APPEND;
|
||||
partitionUpdate.setUpdateMode(mode);
|
||||
break;
|
||||
} else {
|
||||
List<String> partitionValues = toPartitionValues(partitionUpdate.getName());
|
||||
Preconditions.checkState(partitionColNames.size() == partitionValues.size(),
|
||||
"Partition columns names size doesn't equal partition values size. %s vs %s",
|
||||
partitionColNames.size(), partitionValues.size());
|
||||
if (hmsOps.partitionExists(dbName, tableName, partitionValues)) {
|
||||
mode = isOverwrite ? UpdateMode.OVERWRITE : UpdateMode.APPEND;
|
||||
} else {
|
||||
mode = PartitionUpdate.UpdateMode.NEW;
|
||||
}
|
||||
partitionUpdate.setUpdateMode(mode);
|
||||
}
|
||||
hmsOps.dropTable(dbName, tableName);
|
||||
}
|
||||
|
||||
HiveCommitter committer = new HiveCommitter(hmsOps, fileOps, updateExecutor, table, new Path(stagingDir));
|
||||
committer.commit(partitionUpdates);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
|||
|
|
@ -23,6 +23,7 @@ import com.starrocks.connector.RemoteFileOperations;
|
|||
import org.apache.hadoop.conf.Configuration;
|
||||
|
||||
import java.util.Optional;
|
||||
import java.util.concurrent.Executor;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
|
||||
import static com.starrocks.connector.hive.CachingHiveMetastore.createQueryLevelInstance;
|
||||
|
|
@ -34,6 +35,8 @@ public class HiveMetadataFactory {
|
|||
private final long perQueryMetastoreMaxNum;
|
||||
private final long perQueryCacheRemotePathMaxNum;
|
||||
private final ExecutorService pullRemoteFileExecutor;
|
||||
private final Executor updateRemoteFilesExecutor;
|
||||
private final Executor updateStatisticsExecutor;
|
||||
private final boolean isRecursive;
|
||||
private final boolean enableHmsEventsIncrementalSync;
|
||||
private final Configuration configuration;
|
||||
|
|
@ -45,6 +48,8 @@ public class HiveMetadataFactory {
|
|||
CachingHiveMetastoreConf hmsConf,
|
||||
CachingRemoteFileConf fileConf,
|
||||
ExecutorService pullRemoteFileExecutor,
|
||||
Executor updateRemoteFilesExecutor,
|
||||
Executor updateStatisticsExecutor,
|
||||
boolean isRecursive,
|
||||
boolean enableHmsEventsIncrementalSync,
|
||||
Configuration configuration,
|
||||
|
|
@ -55,6 +60,8 @@ public class HiveMetadataFactory {
|
|||
this.perQueryMetastoreMaxNum = hmsConf.getPerQueryCacheMaxNum();
|
||||
this.perQueryCacheRemotePathMaxNum = fileConf.getPerQueryCacheMaxSize();
|
||||
this.pullRemoteFileExecutor = pullRemoteFileExecutor;
|
||||
this.updateRemoteFilesExecutor = updateRemoteFilesExecutor;
|
||||
this.updateStatisticsExecutor = updateStatisticsExecutor;
|
||||
this.isRecursive = isRecursive;
|
||||
this.enableHmsEventsIncrementalSync = enableHmsEventsIncrementalSync;
|
||||
this.configuration = configuration;
|
||||
|
|
@ -69,13 +76,15 @@ public class HiveMetadataFactory {
|
|||
RemoteFileOperations remoteFileOperations = new RemoteFileOperations(
|
||||
CachingRemoteFileIO.createQueryLevelInstance(remoteFileIO, perQueryCacheRemotePathMaxNum),
|
||||
pullRemoteFileExecutor,
|
||||
updateRemoteFilesExecutor,
|
||||
isRecursive,
|
||||
remoteFileIO instanceof CachingRemoteFileIO);
|
||||
remoteFileIO instanceof CachingRemoteFileIO,
|
||||
configuration);
|
||||
HiveStatisticsProvider statisticsProvider = new HiveStatisticsProvider(hiveMetastoreOperations, remoteFileOperations);
|
||||
|
||||
Optional<CacheUpdateProcessor> cacheUpdateProcessor = getCacheUpdateProcessor();
|
||||
return new HiveMetadata(catalogName, hiveMetastoreOperations,
|
||||
remoteFileOperations, statisticsProvider, cacheUpdateProcessor);
|
||||
return new HiveMetadata(catalogName, hiveMetastoreOperations, remoteFileOperations,
|
||||
statisticsProvider, cacheUpdateProcessor, updateStatisticsExecutor);
|
||||
}
|
||||
|
||||
public synchronized Optional<CacheUpdateProcessor> getCacheUpdateProcessor() {
|
||||
|
|
|
|||
|
|
@ -15,6 +15,7 @@
|
|||
|
||||
package com.starrocks.connector.hive;
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.starrocks.catalog.Database;
|
||||
import com.starrocks.catalog.HiveMetaStoreTable;
|
||||
|
|
@ -33,6 +34,7 @@ import org.apache.logging.log4j.LogManager;
|
|||
import org.apache.logging.log4j.Logger;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
|
|
@ -41,10 +43,14 @@ import java.util.stream.Collectors;
|
|||
|
||||
import static com.google.common.collect.ImmutableList.toImmutableList;
|
||||
import static com.google.common.collect.ImmutableMap.toImmutableMap;
|
||||
import static com.google.common.collect.Iterables.getOnlyElement;
|
||||
import static com.starrocks.connector.hive.HiveMetastoreApiConverter.toHiveCommonStats;
|
||||
import static com.starrocks.connector.hive.HiveMetastoreApiConverter.toMetastoreApiPartition;
|
||||
import static com.starrocks.connector.hive.HiveMetastoreApiConverter.toMetastoreApiTable;
|
||||
import static com.starrocks.connector.hive.HiveMetastoreApiConverter.updateStatisticsParameters;
|
||||
import static com.starrocks.connector.hive.HiveMetastoreApiConverter.validateHiveTableType;
|
||||
import static com.starrocks.connector.hive.HiveMetastoreOperations.LOCATION_PROPERTY;
|
||||
import static com.starrocks.connector.hive.Partition.TRANSIENT_LAST_DDL_TIME;
|
||||
|
||||
public class HiveMetastore implements IHiveMetastore {
|
||||
|
||||
|
|
@ -127,6 +133,12 @@ public class HiveMetastore implements IHiveMetastore {
|
|||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean partitionExists(String dbName, String tableName, List<String> partitionValues) {
|
||||
return !client.getPartitionKeysByValue(dbName, tableName, partitionValues).isEmpty();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Partition getPartition(String dbName, String tblName, List<String> partitionValues) {
|
||||
StorageDescriptor sd;
|
||||
Map<String, String> params;
|
||||
|
|
@ -171,6 +183,28 @@ public class HiveMetastore implements IHiveMetastore {
|
|||
return resultBuilder.build();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addPartitions(String dbName, String tableName, List<HivePartitionWithStats> partitions) {
|
||||
List<org.apache.hadoop.hive.metastore.api.Partition> hivePartitions = partitions.stream()
|
||||
.map(HiveMetastoreApiConverter::toMetastoreApiPartition)
|
||||
.collect(Collectors.toList());
|
||||
client.addPartitions(dbName, tableName, hivePartitions);
|
||||
// TODO(stephen): add partition column statistics
|
||||
}
|
||||
|
||||
@Override
|
||||
public void dropPartition(String dbName, String tableName, List<String> partValues, boolean deleteData) {
|
||||
client.dropPartition(dbName, tableName, partValues, deleteData);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void alterPartition(HivePartitionWithStats partition) {
|
||||
String dbName = partition.getHivePartition().getDatabaseName();
|
||||
String tableName = partition.getHivePartition().getTableName();
|
||||
org.apache.hadoop.hive.metastore.api.Partition hivePartition = toMetastoreApiPartition(partition);
|
||||
client.alterPartition(dbName, tableName, hivePartition);
|
||||
}
|
||||
|
||||
public HivePartitionStats getTableStatistics(String dbName, String tblName) {
|
||||
org.apache.hadoop.hive.metastore.api.Table table = client.getTable(dbName, tblName);
|
||||
HiveCommonStats commonStats = toHiveCommonStats(table.getParameters());
|
||||
|
|
@ -188,6 +222,49 @@ public class HiveMetastore implements IHiveMetastore {
|
|||
return new HivePartitionStats(commonStats, columnStatistics);
|
||||
}
|
||||
|
||||
public void updateTableStatistics(String dbName, String tableName, Function<HivePartitionStats, HivePartitionStats> update) {
|
||||
org.apache.hadoop.hive.metastore.api.Table originTable = client.getTable(dbName, tableName);
|
||||
if (originTable == null) {
|
||||
throw new StarRocksConnectorException("Table '%s.%s' not found", dbName, tableName);
|
||||
}
|
||||
|
||||
org.apache.hadoop.hive.metastore.api.Table newTable = originTable.deepCopy();
|
||||
HiveCommonStats curCommonStats = toHiveCommonStats(originTable.getParameters());
|
||||
HivePartitionStats curPartitionStats = new HivePartitionStats(curCommonStats, new HashMap<>());
|
||||
HivePartitionStats updatedStats = update.apply(curPartitionStats);
|
||||
|
||||
HiveCommonStats commonStats = updatedStats.getCommonStats();
|
||||
Map<String, String> originParams = newTable.getParameters();
|
||||
originParams.put(TRANSIENT_LAST_DDL_TIME, String.valueOf(System.currentTimeMillis() / 1000));
|
||||
newTable.setParameters(updateStatisticsParameters(originParams, commonStats));
|
||||
client.alterTable(dbName, tableName, newTable);
|
||||
|
||||
//TODO(stephen): update table column statistics
|
||||
}
|
||||
|
||||
public void updatePartitionStatistics(String dbName, String tableName, String partitionName,
|
||||
Function<HivePartitionStats, HivePartitionStats> update) {
|
||||
List<org.apache.hadoop.hive.metastore.api.Partition> partitions = client.getPartitionsByNames(
|
||||
dbName, tableName, ImmutableList.of(partitionName));
|
||||
if (partitions.size() != 1) {
|
||||
throw new StarRocksConnectorException("Metastore returned multiple partitions for name: " + partitionName);
|
||||
}
|
||||
|
||||
org.apache.hadoop.hive.metastore.api.Partition originPartition = getOnlyElement(partitions);
|
||||
HiveCommonStats curCommonStats = toHiveCommonStats(originPartition.getParameters());
|
||||
HivePartitionStats curPartitionStats = new HivePartitionStats(curCommonStats, new HashMap<>());
|
||||
HivePartitionStats updatedStats = update.apply(curPartitionStats);
|
||||
|
||||
org.apache.hadoop.hive.metastore.api.Partition modifiedPartition = originPartition.deepCopy();
|
||||
HiveCommonStats commonStats = updatedStats.getCommonStats();
|
||||
Map<String, String> originParams = modifiedPartition.getParameters();
|
||||
originParams.put(TRANSIENT_LAST_DDL_TIME, String.valueOf(System.currentTimeMillis() / 1000));
|
||||
modifiedPartition.setParameters(updateStatisticsParameters(modifiedPartition.getParameters(), commonStats));
|
||||
client.alterPartition(dbName, tableName, modifiedPartition);
|
||||
|
||||
//TODO(stephen): update partition column statistics
|
||||
}
|
||||
|
||||
public Map<String, HivePartitionStats> getPartitionStatistics(Table table, List<String> partitionNames) {
|
||||
HiveMetaStoreTable hmsTbl = (HiveMetaStoreTable) table;
|
||||
String dbName = hmsTbl.getDbName();
|
||||
|
|
|
|||
|
|
@ -17,6 +17,7 @@ package com.starrocks.connector.hive;
|
|||
import com.google.common.base.Strings;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.starrocks.catalog.Column;
|
||||
|
|
@ -59,8 +60,10 @@ import java.util.List;
|
|||
import java.util.Locale;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static com.google.common.base.Strings.emptyToNull;
|
||||
import static com.google.common.collect.ImmutableList.toImmutableList;
|
||||
import static com.starrocks.catalog.HudiTable.HUDI_BASE_PATH;
|
||||
import static com.starrocks.catalog.HudiTable.HUDI_TABLE_COLUMN_NAMES;
|
||||
|
|
@ -70,15 +73,18 @@ import static com.starrocks.catalog.HudiTable.HUDI_TABLE_SERDE_LIB;
|
|||
import static com.starrocks.catalog.HudiTable.HUDI_TABLE_TYPE;
|
||||
import static com.starrocks.connector.ColumnTypeConverter.fromHudiType;
|
||||
import static com.starrocks.connector.ColumnTypeConverter.fromHudiTypeToHiveTypeString;
|
||||
import static com.starrocks.connector.hive.HiveMetastoreOperations.FILE_FORMAT;
|
||||
import static com.starrocks.connector.hive.HiveMetadata.STARROCKS_QUERY_ID;
|
||||
import static com.starrocks.connector.hive.RemoteFileInputFormat.fromHdfsInputFormatClass;
|
||||
import static com.starrocks.server.CatalogMgr.ResourceMappingCatalog.toResourceName;
|
||||
import static java.util.Objects.requireNonNull;
|
||||
import static org.apache.hadoop.hive.common.StatsSetupConst.NUM_FILES;
|
||||
import static org.apache.hadoop.hive.common.StatsSetupConst.ROW_COUNT;
|
||||
import static org.apache.hadoop.hive.common.StatsSetupConst.TOTAL_SIZE;
|
||||
|
||||
public class HiveMetastoreApiConverter {
|
||||
private static final Logger LOG = LogManager.getLogger(HiveMetastoreApiConverter.class);
|
||||
private static final String SPARK_SQL_SOURCE_PROVIDER = "spark.sql.sources.provider";
|
||||
private static final Set<String> STATS_PROPERTIES = ImmutableSet.of(ROW_COUNT, TOTAL_SIZE, NUM_FILES);
|
||||
|
||||
private static boolean isDeltaLakeTable(Map<String, String> tableParams) {
|
||||
return tableParams.containsKey(SPARK_SQL_SOURCE_PROVIDER) &&
|
||||
|
|
@ -133,6 +139,7 @@ public class HiveMetastoreApiConverter {
|
|||
.collect(Collectors.toList()))
|
||||
.setFullSchema(toFullSchemasForHiveTable(table))
|
||||
.setTableLocation(toTableLocation(table.getSd(), table.getParameters()))
|
||||
.setStorageFormat(HiveStorageFormat.get(fromHdfsInputFormatClass(table.getSd().getInputFormat()).name()))
|
||||
.setCreateTime(table.getCreateTime());
|
||||
return tableBuilder.build();
|
||||
}
|
||||
|
|
@ -142,18 +149,19 @@ public class HiveMetastoreApiConverter {
|
|||
apiTable.setDbName(table.getDbName());
|
||||
apiTable.setTableName(table.getTableName());
|
||||
apiTable.setTableType("MANAGED_TABLE");
|
||||
apiTable.setOwner(System.getenv("HADOOP_USER_NAME"));
|
||||
apiTable.setParameters(toApiTableProperties(table));
|
||||
apiTable.setPartitionKeys(table.getPartitionColumns().stream()
|
||||
.map(HiveMetastoreApiConverter::toMetastoreApiFieldSchema)
|
||||
.collect(Collectors.toList()));
|
||||
apiTable.setSd(makeStorageDescriptor(table));
|
||||
apiTable.setSd(makeStorageDescriptorFromHiveTable(table));
|
||||
return apiTable;
|
||||
}
|
||||
|
||||
private static StorageDescriptor makeStorageDescriptor(HiveTable table) {
|
||||
private static StorageDescriptor makeStorageDescriptorFromHiveTable(HiveTable table) {
|
||||
SerDeInfo serdeInfo = new SerDeInfo();
|
||||
serdeInfo.setName(table.getTableName());
|
||||
HiveStorageFormat storageFormat = HiveStorageFormat.get(table.getHiveProperties().getOrDefault(FILE_FORMAT, "parquet"));
|
||||
HiveStorageFormat storageFormat = table.getStorageFormat();
|
||||
serdeInfo.setSerializationLib(storageFormat.getSerde());
|
||||
|
||||
StorageDescriptor sd = new StorageDescriptor();
|
||||
|
|
@ -170,6 +178,25 @@ public class HiveMetastoreApiConverter {
|
|||
return sd;
|
||||
}
|
||||
|
||||
private static StorageDescriptor makeStorageDescriptorFromHivePartition(HivePartition partition) {
|
||||
SerDeInfo serdeInfo = new SerDeInfo();
|
||||
HiveStorageFormat hiveSd = partition.getStorage();
|
||||
serdeInfo.setName(partition.getTableName());
|
||||
serdeInfo.setSerializationLib(hiveSd.getSerde());
|
||||
|
||||
StorageDescriptor sd = new StorageDescriptor();
|
||||
sd.setLocation(emptyToNull(partition.getLocation()));
|
||||
sd.setCols(partition.getColumns().stream()
|
||||
.map(HiveMetastoreApiConverter::toMetastoreApiFieldSchema)
|
||||
.collect(toImmutableList()));
|
||||
sd.setSerdeInfo(serdeInfo);
|
||||
sd.setInputFormat(hiveSd.getInputFormat());
|
||||
sd.setOutputFormat(hiveSd.getOutputFormat());
|
||||
sd.setParameters(ImmutableMap.of());
|
||||
|
||||
return sd;
|
||||
}
|
||||
|
||||
public static FieldSchema toMetastoreApiFieldSchema(Column column) {
|
||||
return new FieldSchema(column.getName(), ColumnTypeConverter.toHiveType(column.getType()), column.getComment());
|
||||
}
|
||||
|
|
@ -177,12 +204,12 @@ public class HiveMetastoreApiConverter {
|
|||
public static Map<String, String> toApiTableProperties(HiveTable table) {
|
||||
ImmutableMap.Builder<String, String> tableProperties = ImmutableMap.builder();
|
||||
|
||||
tableProperties.put("numFiles", "-1");
|
||||
tableProperties.put("totalSize", "-1");
|
||||
tableProperties.put(ROW_COUNT, "0");
|
||||
tableProperties.put(TOTAL_SIZE, "0");
|
||||
tableProperties.put("comment", table.getComment());
|
||||
tableProperties.put("starrocks_version", Version.STARROCKS_VERSION + "-" + Version.STARROCKS_COMMIT_HASH);
|
||||
if (ConnectContext.get() != null && ConnectContext.get().getQueryId() != null) {
|
||||
tableProperties.put("starrocks_query_id", ConnectContext.get().getQueryId().toString());
|
||||
tableProperties.put(STARROCKS_QUERY_ID, ConnectContext.get().getQueryId().toString());
|
||||
}
|
||||
|
||||
return tableProperties.build();
|
||||
|
|
@ -267,6 +294,25 @@ public class HiveMetastoreApiConverter {
|
|||
return partitionBuilder.build();
|
||||
}
|
||||
|
||||
public static org.apache.hadoop.hive.metastore.api.Partition toMetastoreApiPartition(
|
||||
HivePartitionWithStats partitionWithStatistics) {
|
||||
org.apache.hadoop.hive.metastore.api.Partition partition =
|
||||
toMetastoreApiPartition(partitionWithStatistics.getHivePartition());
|
||||
partition.setParameters(updateStatisticsParameters(
|
||||
partition.getParameters(), partitionWithStatistics.getHivePartitionStats().getCommonStats()));
|
||||
return partition;
|
||||
}
|
||||
|
||||
public static org.apache.hadoop.hive.metastore.api.Partition toMetastoreApiPartition(HivePartition hivePartition) {
|
||||
org.apache.hadoop.hive.metastore.api.Partition result = new org.apache.hadoop.hive.metastore.api.Partition();
|
||||
result.setDbName(hivePartition.getDatabaseName());
|
||||
result.setTableName(hivePartition.getTableName());
|
||||
result.setValues(hivePartition.getValues());
|
||||
result.setSd(makeStorageDescriptorFromHivePartition(hivePartition));
|
||||
result.setParameters(hivePartition.getParameters());
|
||||
return result;
|
||||
}
|
||||
|
||||
public static List<Column> toFullSchemasForHiveTable(Table table) {
|
||||
List<FieldSchema> fieldSchemas = getAllFieldSchemas(table);
|
||||
List<Column> fullSchema = Lists.newArrayList();
|
||||
|
|
@ -480,6 +526,26 @@ public class HiveMetastoreApiConverter {
|
|||
return hiveColumnStatistics;
|
||||
}
|
||||
|
||||
public static Map<String, String> updateStatisticsParameters(Map<String, String> parameters, HiveCommonStats statistics) {
|
||||
ImmutableMap.Builder<String, String> result = ImmutableMap.builder();
|
||||
|
||||
parameters.forEach((key, value) -> {
|
||||
if (!(STATS_PROPERTIES.contains(key))) {
|
||||
result.put(key, value);
|
||||
}
|
||||
});
|
||||
|
||||
result.put(ROW_COUNT, String.valueOf(statistics.getRowNums()));
|
||||
result.put(TOTAL_SIZE, String.valueOf(statistics.getTotalFileBytes()));
|
||||
|
||||
if (!parameters.containsKey("STATS_GENERATED_VIA_STATS_TASK")) {
|
||||
result.put("STATS_GENERATED_VIA_STATS_TASK", "workaround for potential lack of HIVE-12730");
|
||||
}
|
||||
|
||||
|
||||
return result.buildOrThrow();
|
||||
}
|
||||
|
||||
public static void validateHiveTableType(String hiveTableType) {
|
||||
if (hiveTableType == null) {
|
||||
throw new StarRocksConnectorException("Unknown hive table type.");
|
||||
|
|
|
|||
|
|
@ -42,6 +42,7 @@ import java.util.HashMap;
|
|||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.function.Function;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static com.google.common.base.Preconditions.checkState;
|
||||
|
|
@ -172,6 +173,7 @@ public class HiveMetastoreOperations {
|
|||
.setFullSchema(stmt.getColumns())
|
||||
.setTableLocation(tablePath.toString())
|
||||
.setProperties(stmt.getProperties())
|
||||
.setStorageFormat(HiveStorageFormat.get(properties.getOrDefault(FILE_FORMAT, "parquet")))
|
||||
.setCreateTime(System.currentTimeMillis());
|
||||
Table table = builder.build();
|
||||
try {
|
||||
|
|
@ -215,6 +217,18 @@ public class HiveMetastoreOperations {
|
|||
return metastore.getPartition(dbName, tableName, partitionValues);
|
||||
}
|
||||
|
||||
public void addPartitions(String dbName, String tableName, List<HivePartitionWithStats> partitions) {
|
||||
metastore.addPartitions(dbName, tableName, partitions);
|
||||
}
|
||||
|
||||
public void dropPartition(String dbName, String tableName, List<String> partitionValues, boolean deleteData) {
|
||||
metastore.dropPartition(dbName, tableName, partitionValues, deleteData);
|
||||
}
|
||||
|
||||
public boolean partitionExists(String dbName, String tableName, List<String> partitionValues) {
|
||||
return metastore.partitionExists(dbName, tableName, partitionValues);
|
||||
}
|
||||
|
||||
public Map<String, Partition> getPartitionByPartitionKeys(Table table, List<PartitionKey> partitionKeys) {
|
||||
String dbName = ((HiveMetaStoreTable) table).getDbName();
|
||||
String tblName = ((HiveMetaStoreTable) table).getTableName();
|
||||
|
|
@ -267,6 +281,15 @@ public class HiveMetastoreOperations {
|
|||
metastore.invalidateAll();
|
||||
}
|
||||
|
||||
public void updateTableStatistics(String dbName, String tableName, Function<HivePartitionStats, HivePartitionStats> update) {
|
||||
metastore.updateTableStatistics(dbName, tableName, update);
|
||||
}
|
||||
|
||||
public void updatePartitionStatistics(String dbName, String tableName, String partitionName,
|
||||
Function<HivePartitionStats, HivePartitionStats> update) {
|
||||
metastore.updatePartitionStatistics(dbName, tableName, partitionName, update);
|
||||
}
|
||||
|
||||
public Path getDefaultLocation(String dbName, String tableName) {
|
||||
Database database = getDb(dbName);
|
||||
|
||||
|
|
|
|||
|
|
@ -0,0 +1,126 @@
|
|||
// Copyright 2021-present StarRocks, Inc. All rights reserved.
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// https://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package com.starrocks.connector.hive;
|
||||
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.starrocks.catalog.Column;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
public class HivePartition {
|
||||
private final String databaseName;
|
||||
private final String tableName;
|
||||
private final List<String> values;
|
||||
private String location;
|
||||
private final HiveStorageFormat storage;
|
||||
private final List<Column> columns;
|
||||
private final Map<String, String> parameters;
|
||||
|
||||
public HivePartition(String databaseName, String tableName, List<String> values, String location,
|
||||
HiveStorageFormat storage, List<Column> columns, Map<String, String> parameters) {
|
||||
this.databaseName = databaseName;
|
||||
this.tableName = tableName;
|
||||
this.values = values;
|
||||
this.location = location;
|
||||
this.storage = storage;
|
||||
this.columns = columns;
|
||||
this.parameters = parameters;
|
||||
}
|
||||
|
||||
public String getDatabaseName() {
|
||||
return databaseName;
|
||||
}
|
||||
|
||||
public String getTableName() {
|
||||
return tableName;
|
||||
}
|
||||
|
||||
public List<String> getValues() {
|
||||
return values;
|
||||
}
|
||||
|
||||
public HiveStorageFormat getStorage() {
|
||||
return storage;
|
||||
}
|
||||
|
||||
public List<Column> getColumns() {
|
||||
return columns;
|
||||
}
|
||||
|
||||
public Map<String, String> getParameters() {
|
||||
return parameters;
|
||||
}
|
||||
|
||||
public String getLocation() {
|
||||
return location;
|
||||
}
|
||||
|
||||
public static Builder builder() {
|
||||
return new Builder();
|
||||
}
|
||||
|
||||
public static class Builder {
|
||||
private String databaseName;
|
||||
private String tableName;
|
||||
private HiveStorageFormat storageFormat;
|
||||
private List<String> values;
|
||||
private List<Column> columns;
|
||||
private String location;
|
||||
private Map<String, String> parameters = ImmutableMap.of();
|
||||
|
||||
private Builder() {
|
||||
}
|
||||
|
||||
public Builder setDatabaseName(String databaseName) {
|
||||
this.databaseName = databaseName;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder setTableName(String tableName) {
|
||||
this.tableName = tableName;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder setValues(List<String> values) {
|
||||
this.values = values;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder setStorageFormat(HiveStorageFormat storageFormat) {
|
||||
this.storageFormat = storageFormat;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder setColumns(List<Column> columns) {
|
||||
this.columns = columns;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder setParameters(Map<String, String> parameters) {
|
||||
this.parameters = parameters;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder setLocation(String location) {
|
||||
this.location = location;
|
||||
return this;
|
||||
}
|
||||
|
||||
public HivePartition build() {
|
||||
return new HivePartition(databaseName, tableName, values, location, storageFormat, columns, parameters);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -29,6 +29,11 @@ public class HivePartitionStats {
|
|||
return EMPTY;
|
||||
}
|
||||
|
||||
public static HivePartitionStats fromCommonStats(long rowNums, long totalFileBytes) {
|
||||
HiveCommonStats commonStats = new HiveCommonStats(rowNums, totalFileBytes);
|
||||
return new HivePartitionStats(commonStats, ImmutableMap.of());
|
||||
}
|
||||
|
||||
public HivePartitionStats(HiveCommonStats commonStats, Map<String, HiveColumnStats> columnStats) {
|
||||
this.commonStats = commonStats;
|
||||
this.columnStats = columnStats;
|
||||
|
|
@ -50,4 +55,56 @@ public class HivePartitionStats {
|
|||
sb.append('}');
|
||||
return sb.toString();
|
||||
}
|
||||
|
||||
// only used to update the parameters of partition or table.
|
||||
// TODO(stephen): collect and merge colum statistics.
|
||||
public static HivePartitionStats merge(HivePartitionStats current, HivePartitionStats update) {
|
||||
if (current.getCommonStats().getRowNums() == -1 || update.getCommonStats().getRowNums() <= 0) {
|
||||
return current;
|
||||
} else if (current.getCommonStats().getRowNums() == 0 && update.getCommonStats().getRowNums() > 0) {
|
||||
return update;
|
||||
}
|
||||
|
||||
return new HivePartitionStats(
|
||||
reduce(current.getCommonStats(), update.getCommonStats(), ReduceOperator.ADD),
|
||||
// TODO(stephen): collect and merge column statistics
|
||||
current.getColumnStats());
|
||||
}
|
||||
|
||||
public static HivePartitionStats reduce(HivePartitionStats first, HivePartitionStats second, ReduceOperator operator) {
|
||||
return HivePartitionStats.fromCommonStats(
|
||||
reduce(first.getCommonStats().getRowNums(), second.getCommonStats().getRowNums(), operator),
|
||||
reduce(first.getCommonStats().getTotalFileBytes(), second.getCommonStats().getTotalFileBytes(), operator));
|
||||
}
|
||||
|
||||
public static HiveCommonStats reduce(HiveCommonStats current, HiveCommonStats update, ReduceOperator operator) {
|
||||
return new HiveCommonStats(
|
||||
reduce(current.getRowNums(), update.getRowNums(), operator),
|
||||
reduce(current.getTotalFileBytes(), update.getTotalFileBytes(), operator));
|
||||
}
|
||||
|
||||
public static long reduce(long current, long update, ReduceOperator operator) {
|
||||
if (current >= 0 && update >= 0) {
|
||||
switch (operator) {
|
||||
case ADD:
|
||||
return current + update;
|
||||
case SUBTRACT:
|
||||
return current - update;
|
||||
case MAX:
|
||||
return Math.max(current, update);
|
||||
case MIN:
|
||||
return Math.min(current, update);
|
||||
}
|
||||
throw new IllegalArgumentException("Unexpected operator: " + operator);
|
||||
}
|
||||
|
||||
return 0;
|
||||
}
|
||||
|
||||
public enum ReduceOperator {
|
||||
ADD,
|
||||
SUBTRACT,
|
||||
MIN,
|
||||
MAX,
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -0,0 +1,39 @@
|
|||
// Copyright 2021-present StarRocks, Inc. All rights reserved.
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// https://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package com.starrocks.connector.hive;
|
||||
|
||||
public class HivePartitionWithStats {
|
||||
String partitionName;
|
||||
HivePartition hivePartition;
|
||||
HivePartitionStats hivePartitionStats;
|
||||
|
||||
public HivePartitionWithStats(String partitionName, HivePartition hivePartition, HivePartitionStats hivePartitionStats) {
|
||||
this.partitionName = partitionName;
|
||||
this.hivePartition = hivePartition;
|
||||
this.hivePartitionStats = hivePartitionStats;
|
||||
}
|
||||
|
||||
public String getPartitionName() {
|
||||
return partitionName;
|
||||
}
|
||||
|
||||
public HivePartition getHivePartition() {
|
||||
return hivePartition;
|
||||
}
|
||||
|
||||
public HivePartitionStats getHivePartitionStats() {
|
||||
return hivePartitionStats;
|
||||
}
|
||||
}
|
||||
|
|
@ -14,6 +14,8 @@
|
|||
|
||||
package com.starrocks.connector.hive;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.starrocks.catalog.HiveTable;
|
||||
import com.starrocks.common.DdlException;
|
||||
import com.starrocks.connector.exception.StarRocksConnectorException;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
|
|
@ -22,8 +24,10 @@ import org.apache.hadoop.fs.Path;
|
|||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
|
||||
import java.io.FileNotFoundException;
|
||||
import java.io.IOException;
|
||||
import java.util.Map;
|
||||
import java.util.UUID;
|
||||
|
||||
import static com.starrocks.connector.hive.HiveMetastoreOperations.EXTERNAL_LOCATION_PROPERTY;
|
||||
import static com.starrocks.connector.hive.HiveMetastoreOperations.LOCATION_PROPERTY;
|
||||
|
|
@ -49,8 +53,8 @@ public class HiveWriteUtils {
|
|||
FileSystem fileSystem = FileSystem.get(path.toUri(), conf);
|
||||
return fileSystem.exists(path);
|
||||
} catch (Exception e) {
|
||||
LOG.error("Failed checking path {}", path, e);
|
||||
throw new StarRocksConnectorException("Failed checking path: " + path + " msg:" + e.getMessage());
|
||||
LOG.error("Failed to check path {}", path, e);
|
||||
throw new StarRocksConnectorException("Failed to check path: " + path + ". msg: " + e.getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -76,4 +80,54 @@ public class HiveWriteUtils {
|
|||
throw new StarRocksConnectorException("Failed to create directory: " + path, e);
|
||||
}
|
||||
}
|
||||
|
||||
public static String getStagingDir(HiveTable table, String tempStagingDir) {
|
||||
String stagingDir;
|
||||
String location = table.getTableLocation();
|
||||
if (isS3Url(location)) {
|
||||
stagingDir = location;
|
||||
} else {
|
||||
Path tempRoot = new Path(location, tempStagingDir);
|
||||
Path tempStagingPath = new Path(tempRoot, UUID.randomUUID().toString());
|
||||
stagingDir = tempStagingPath.toString();
|
||||
}
|
||||
return stagingDir.endsWith("/") ? stagingDir : stagingDir + "/";
|
||||
}
|
||||
|
||||
public static boolean fileCreatedByQuery(String fileName, String queryId) {
|
||||
Preconditions.checkState(fileName.length() > queryId.length() && queryId.length() > 8,
|
||||
"file name or query id is invalid");
|
||||
String checkQueryId = queryId.substring(0, queryId.length() - 8);
|
||||
return fileName.startsWith(checkQueryId) || fileName.endsWith(checkQueryId);
|
||||
}
|
||||
|
||||
public static void checkedDelete(FileSystem fileSystem, Path file, boolean recursive) throws IOException {
|
||||
try {
|
||||
if (!fileSystem.delete(file, recursive)) {
|
||||
if (fileSystem.exists(file)) {
|
||||
throw new IOException("Failed to delete " + file);
|
||||
}
|
||||
}
|
||||
} catch (FileNotFoundException ignored) {
|
||||
// ignore
|
||||
}
|
||||
}
|
||||
|
||||
public static boolean deleteIfExists(Path path, boolean recursive, Configuration conf) {
|
||||
try {
|
||||
FileSystem fileSystem = FileSystem.get(path.toUri(), conf);
|
||||
if (fileSystem.delete(path, recursive)) {
|
||||
return true;
|
||||
}
|
||||
|
||||
return !fileSystem.exists(path);
|
||||
} catch (FileNotFoundException ignored) {
|
||||
return true;
|
||||
} catch (IOException ignored) {
|
||||
LOG.error("Failed to delete remote path {}", path);
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -23,6 +23,7 @@ import com.starrocks.catalog.Table;
|
|||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.function.Function;
|
||||
|
||||
public interface IHiveMetastore {
|
||||
|
||||
|
|
@ -54,12 +55,25 @@ public interface IHiveMetastore {
|
|||
|
||||
Partition getPartition(String dbName, String tableName, List<String> partitionValues);
|
||||
|
||||
void addPartitions(String dbName, String tableName, List<HivePartitionWithStats> partitions);
|
||||
|
||||
void alterPartition(HivePartitionWithStats partition);
|
||||
|
||||
void dropPartition(String dbName, String tableName, List<String> partValues, boolean deleteData);
|
||||
|
||||
boolean partitionExists(String dbName, String tableName, List<String> partitionValues);
|
||||
|
||||
Map<String, Partition> getPartitionsByNames(String dbName, String tableName, List<String> partitionNames);
|
||||
|
||||
HivePartitionStats getTableStatistics(String dbName, String tableName);
|
||||
|
||||
Map<String, HivePartitionStats> getPartitionStatistics(Table table, List<String> partitions);
|
||||
|
||||
void updateTableStatistics(String dbName, String tableName, Function<HivePartitionStats, HivePartitionStats> update);
|
||||
|
||||
void updatePartitionStatistics(String dbName, String tableName, String partitionName,
|
||||
Function<HivePartitionStats, HivePartitionStats> update);
|
||||
|
||||
// return refreshed partitions in cache for partitioned table, return empty list for unpartitioned table
|
||||
default List<HivePartitionName> refreshTable(String hiveDbName, String hiveTblName, boolean onlyCachedPartitions) {
|
||||
return Lists.newArrayList();
|
||||
|
|
|
|||
|
|
@ -0,0 +1,133 @@
|
|||
// Copyright 2021-present StarRocks, Inc. All rights reserved.
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// https://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package com.starrocks.connector.hive;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.base.Strings;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Multimaps;
|
||||
import com.starrocks.connector.PartitionUtil;
|
||||
import com.starrocks.thrift.THiveFileInfo;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
|
||||
import static com.starrocks.connector.PartitionUtil.getPathWithSlash;
|
||||
|
||||
public class PartitionUpdate {
|
||||
private final String name;
|
||||
private UpdateMode updateMode;
|
||||
private final Path writePath;
|
||||
private final Path targetPath;
|
||||
private final List<String> fileNames;
|
||||
private final long rowCount;
|
||||
private final long totalSizeInBytes;
|
||||
|
||||
public PartitionUpdate(String name, Path writePath, Path targetPath,
|
||||
List<String> fileNames, long rowCount, long totalSizeInBytes) {
|
||||
this.name = name;
|
||||
this.writePath = writePath;
|
||||
this.targetPath = targetPath;
|
||||
this.fileNames = fileNames;
|
||||
this.rowCount = rowCount;
|
||||
this.totalSizeInBytes = totalSizeInBytes;
|
||||
}
|
||||
|
||||
public String getName() {
|
||||
return name;
|
||||
}
|
||||
|
||||
public UpdateMode getUpdateMode() {
|
||||
return updateMode;
|
||||
}
|
||||
|
||||
public Path getWritePath() {
|
||||
return writePath;
|
||||
}
|
||||
|
||||
public Path getTargetPath() {
|
||||
return targetPath;
|
||||
}
|
||||
|
||||
public List<String> getFileNames() {
|
||||
return fileNames;
|
||||
}
|
||||
|
||||
public long getRowCount() {
|
||||
return rowCount;
|
||||
}
|
||||
|
||||
public long getTotalSizeInBytes() {
|
||||
return totalSizeInBytes;
|
||||
}
|
||||
|
||||
public boolean isS3Url() {
|
||||
return HiveWriteUtils.isS3Url(targetPath.toString());
|
||||
}
|
||||
|
||||
public PartitionUpdate setUpdateMode(UpdateMode updateMode) {
|
||||
this.updateMode = updateMode;
|
||||
return this;
|
||||
}
|
||||
|
||||
public enum UpdateMode {
|
||||
NEW,
|
||||
APPEND,
|
||||
OVERWRITE,
|
||||
}
|
||||
|
||||
public static PartitionUpdate get(THiveFileInfo fileInfo, String stagingDir, String tableLocation) {
|
||||
Preconditions.checkState(fileInfo.isSetPartition_path() && !Strings.isNullOrEmpty(fileInfo.getPartition_path()),
|
||||
"Missing partition path");
|
||||
|
||||
String partitionName = PartitionUtil.getPartitionName(stagingDir, fileInfo.getPartition_path());
|
||||
|
||||
String tableLocationWithSlash = getPathWithSlash(tableLocation);
|
||||
String stagingDirWithSlash = getPathWithSlash(stagingDir);
|
||||
|
||||
String writePath = stagingDirWithSlash + partitionName + "/";
|
||||
String targetPath = tableLocationWithSlash + partitionName + "/";
|
||||
|
||||
return new PartitionUpdate(partitionName, new Path(writePath), new Path(targetPath),
|
||||
Lists.newArrayList(fileInfo.getFile_name()), fileInfo.getRecord_count(), fileInfo.getFile_size_in_bytes());
|
||||
}
|
||||
|
||||
public static List<PartitionUpdate> merge(List<PartitionUpdate> unMergedUpdates) {
|
||||
ImmutableList.Builder<PartitionUpdate> partitionUpdates = ImmutableList.builder();
|
||||
for (Collection<PartitionUpdate> partitionGroup : Multimaps.index(unMergedUpdates, PartitionUpdate::getName)
|
||||
.asMap().values()) {
|
||||
PartitionUpdate firstPartition = partitionGroup.iterator().next();
|
||||
|
||||
ImmutableList.Builder<String> allFileNames = ImmutableList.builder();
|
||||
long totalRowCount = 0;
|
||||
long fileSizeInBytes = 0;
|
||||
for (PartitionUpdate partition : partitionGroup) {
|
||||
allFileNames.addAll(partition.getFileNames());
|
||||
totalRowCount += partition.getRowCount();
|
||||
fileSizeInBytes += partition.getTotalSizeInBytes();
|
||||
}
|
||||
|
||||
partitionUpdates.add(new PartitionUpdate(firstPartition.getName(),
|
||||
firstPartition.getWritePath(),
|
||||
firstPartition.getTargetPath(),
|
||||
allFileNames.build(),
|
||||
totalRowCount,
|
||||
fileSizeInBytes));
|
||||
}
|
||||
return partitionUpdates.build();
|
||||
}
|
||||
}
|
||||
|
|
@ -72,8 +72,10 @@ public class HudiMetadataFactory {
|
|||
RemoteFileOperations remoteFileOperations = new RemoteFileOperations(
|
||||
CachingRemoteFileIO.createQueryLevelInstance(remoteFileIO, perQueryCacheRemotePathMaxNum),
|
||||
pullRemoteFileExecutor,
|
||||
pullRemoteFileExecutor,
|
||||
isRecursive,
|
||||
remoteFileIO instanceof CachingRemoteFileIO);
|
||||
remoteFileIO instanceof CachingRemoteFileIO,
|
||||
hadoopConf);
|
||||
HiveStatisticsProvider statisticsProvider = new HiveStatisticsProvider(hiveMetastoreOperations, remoteFileOperations);
|
||||
Optional<CacheUpdateProcessor> cacheUpdateProcessor = getCacheUpdateProcessor();
|
||||
|
||||
|
|
|
|||
|
|
@ -39,6 +39,7 @@ import com.google.common.collect.Sets;
|
|||
import com.google.gson.annotations.SerializedName;
|
||||
import com.starrocks.catalog.AuthorizationInfo;
|
||||
import com.starrocks.catalog.Database;
|
||||
import com.starrocks.catalog.HiveTable;
|
||||
import com.starrocks.catalog.IcebergTable;
|
||||
import com.starrocks.catalog.Table;
|
||||
import com.starrocks.catalog.system.SystemTable;
|
||||
|
|
@ -213,7 +214,7 @@ public class InsertLoadJob extends LoadJob {
|
|||
return true;
|
||||
}
|
||||
|
||||
if (table instanceof SystemTable || table instanceof IcebergTable) {
|
||||
if (table instanceof SystemTable || table instanceof IcebergTable || table instanceof HiveTable) {
|
||||
return false;
|
||||
} else {
|
||||
return true;
|
||||
|
|
|
|||
|
|
@ -34,6 +34,7 @@
|
|||
|
||||
package com.starrocks.planner;
|
||||
|
||||
import com.starrocks.catalog.HiveTable;
|
||||
import com.starrocks.catalog.IcebergTable;
|
||||
import com.starrocks.catalog.MysqlTable;
|
||||
import com.starrocks.catalog.OlapTable;
|
||||
|
|
@ -95,6 +96,8 @@ public abstract class DataSink {
|
|||
return true;
|
||||
} else if (table instanceof IcebergTable) {
|
||||
return true;
|
||||
} else if (table instanceof HiveTable) {
|
||||
return true;
|
||||
}
|
||||
|
||||
return false;
|
||||
|
|
|
|||
|
|
@ -0,0 +1,121 @@
|
|||
// Copyright 2021-present StarRocks, Inc. All rights reserved.
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// https://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package com.starrocks.planner;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.starrocks.analysis.TupleDescriptor;
|
||||
import com.starrocks.catalog.HiveTable;
|
||||
import com.starrocks.connector.Connector;
|
||||
import com.starrocks.connector.hive.HiveWriteUtils;
|
||||
import com.starrocks.credential.CloudConfiguration;
|
||||
import com.starrocks.qe.SessionVariable;
|
||||
import com.starrocks.server.GlobalStateMgr;
|
||||
import com.starrocks.thrift.TCloudConfiguration;
|
||||
import com.starrocks.thrift.TCompressionType;
|
||||
import com.starrocks.thrift.TDataSink;
|
||||
import com.starrocks.thrift.TDataSinkType;
|
||||
import com.starrocks.thrift.TExplainLevel;
|
||||
import com.starrocks.thrift.THiveTableSink;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
import static com.starrocks.analysis.OutFileClause.PARQUET_COMPRESSION_TYPE_MAP;
|
||||
import static com.starrocks.connector.hive.HiveMetastoreOperations.FILE_FORMAT;
|
||||
|
||||
public class HiveTableSink extends DataSink {
|
||||
|
||||
protected final TupleDescriptor desc;
|
||||
private final String fileFormat;
|
||||
private final String stagingDir;
|
||||
private final List<String> dataColNames;
|
||||
private final List<String> partitionColNames;
|
||||
private final String compressionType;
|
||||
private final boolean isStaticPartitionSink;
|
||||
private final String tableIdentifier;
|
||||
private final CloudConfiguration cloudConfiguration;
|
||||
|
||||
public HiveTableSink(HiveTable hiveTable, TupleDescriptor desc, boolean isStaticPartitionSink, SessionVariable sessionVariable) {
|
||||
this.desc = desc;
|
||||
this.stagingDir = HiveWriteUtils.getStagingDir(hiveTable, sessionVariable.getHiveTempStagingDir());
|
||||
this.partitionColNames = hiveTable.getPartitionColumnNames();
|
||||
this.dataColNames = hiveTable.getDataColumnNames();
|
||||
this.tableIdentifier = hiveTable.getUUID();
|
||||
this.isStaticPartitionSink = isStaticPartitionSink;
|
||||
this.fileFormat = hiveTable.getHiveProperties().getOrDefault(FILE_FORMAT, "parquet");
|
||||
this.compressionType = hiveTable.getHiveProperties().getOrDefault("compression_codec", "gzip");
|
||||
String catalogName = hiveTable.getCatalogName();
|
||||
Connector connector = GlobalStateMgr.getCurrentState().getConnectorMgr().getConnector(catalogName);
|
||||
Preconditions.checkState(connector != null,
|
||||
String.format("connector of catalog %s should not be null", catalogName));
|
||||
|
||||
this.cloudConfiguration = connector.getCloudConfiguration();
|
||||
|
||||
Preconditions.checkState(cloudConfiguration != null,
|
||||
String.format("cloudConfiguration of catalog %s should not be null", catalogName));
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getExplainString(String prefix, TExplainLevel explainLevel) {
|
||||
StringBuilder strBuilder = new StringBuilder();
|
||||
strBuilder.append(prefix + "Hive TABLE SINK\n");
|
||||
strBuilder.append(prefix + " TABLE: " + tableIdentifier + "\n");
|
||||
strBuilder.append(prefix + " TUPLE ID: " + desc.getId() + "\n");
|
||||
strBuilder.append(prefix + " " + DataPartition.RANDOM.getExplainString(explainLevel));
|
||||
return strBuilder.toString();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected TDataSink toThrift() {
|
||||
TDataSink tDataSink = new TDataSink(TDataSinkType.HIVE_TABLE_SINK);
|
||||
THiveTableSink tHiveTableSink = new THiveTableSink();
|
||||
tHiveTableSink.setData_column_names(dataColNames);
|
||||
tHiveTableSink.setPartition_column_names(partitionColNames);
|
||||
tHiveTableSink.setStaging_dir(stagingDir);
|
||||
tHiveTableSink.setFile_format(fileFormat);
|
||||
tHiveTableSink.setIs_static_partition_sink(isStaticPartitionSink);
|
||||
TCompressionType compression = PARQUET_COMPRESSION_TYPE_MAP.get(compressionType);
|
||||
tHiveTableSink.setCompression_type(compression);
|
||||
TCloudConfiguration tCloudConfiguration = new TCloudConfiguration();
|
||||
cloudConfiguration.toThrift(tCloudConfiguration);
|
||||
tHiveTableSink.setCloud_configuration(tCloudConfiguration);
|
||||
tDataSink.setHive_table_sink(tHiveTableSink);
|
||||
|
||||
return tDataSink;
|
||||
}
|
||||
|
||||
@Override
|
||||
public PlanNodeId getExchNodeId() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DataPartition getOutputPartition() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean canUsePipeLine() {
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean canUseRuntimeAdaptiveDop() {
|
||||
return true;
|
||||
}
|
||||
|
||||
public String getStagingDir() {
|
||||
return stagingDir;
|
||||
}
|
||||
}
|
||||
|
|
@ -95,10 +95,6 @@ public class IcebergTableSink extends DataSink {
|
|||
String.format("cloudConfiguration of catalog %s should not be null", catalogName));
|
||||
}
|
||||
|
||||
public static boolean isUnSupportedPartitionColumnType(Type type) {
|
||||
return type.isFloat() || type.isDecimalOfAnyVersion() || type.isDatetime();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getExplainString(String prefix, TExplainLevel explainLevel) {
|
||||
StringBuilder strBuilder = new StringBuilder();
|
||||
|
|
|
|||
|
|
@ -155,6 +155,8 @@ public class PlanFragment extends TreeNode<PlanFragment> {
|
|||
private TCacheParam cacheParam = null;
|
||||
private boolean hasOlapTableSink = false;
|
||||
private boolean hasIcebergTableSink = false;
|
||||
private boolean hasHiveTableSink = false;
|
||||
|
||||
private boolean forceSetTableSinkDop = false;
|
||||
private boolean forceAssignScanRangesPerDriverSeq = false;
|
||||
|
||||
|
|
@ -257,7 +259,7 @@ public class PlanFragment extends TreeNode<PlanFragment> {
|
|||
}
|
||||
|
||||
public boolean hasTableSink() {
|
||||
return hasIcebergTableSink() || hasOlapTableSink();
|
||||
return hasIcebergTableSink() || hasOlapTableSink() || hasHiveTableSink();
|
||||
}
|
||||
|
||||
public boolean hasOlapTableSink() {
|
||||
|
|
@ -276,6 +278,14 @@ public class PlanFragment extends TreeNode<PlanFragment> {
|
|||
this.hasIcebergTableSink = true;
|
||||
}
|
||||
|
||||
public boolean hasHiveTableSink() {
|
||||
return this.hasHiveTableSink;
|
||||
}
|
||||
|
||||
public void setHasHiveTableSink() {
|
||||
this.hasHiveTableSink = true;
|
||||
}
|
||||
|
||||
public boolean forceSetTableSinkDop() {
|
||||
return this.forceSetTableSinkDop;
|
||||
}
|
||||
|
|
|
|||
|
|
@ -489,6 +489,8 @@ public class SessionVariable implements Serializable, Writable, Cloneable {
|
|||
|
||||
public static final String ENABLE_COLLECT_TABLE_LEVEL_SCAN_STATS = "enable_collect_table_level_scan_stats";
|
||||
|
||||
public static final String HIVE_TEMP_STAGING_DIR = "hive_temp_staging_dir";
|
||||
|
||||
public static final List<String> DEPRECATED_VARIABLES = ImmutableList.<String>builder()
|
||||
.add(CODEGEN_LEVEL)
|
||||
.add(MAX_EXECUTION_TIME)
|
||||
|
|
@ -1240,8 +1242,20 @@ public class SessionVariable implements Serializable, Writable, Cloneable {
|
|||
@VarAttr(name = ENABLE_COLLECT_TABLE_LEVEL_SCAN_STATS)
|
||||
private boolean enableCollectTableLevelScanStats = true;
|
||||
|
||||
@VarAttr(name = HIVE_TEMP_STAGING_DIR)
|
||||
private String hiveTempStagingDir = "/tmp/starrocks";
|
||||
|
||||
private int exprChildrenLimit = -1;
|
||||
|
||||
public String getHiveTempStagingDir() {
|
||||
return hiveTempStagingDir;
|
||||
}
|
||||
|
||||
public SessionVariable setHiveTempStagingDir(String hiveTempStagingDir) {
|
||||
this.hiveTempStagingDir = hiveTempStagingDir;
|
||||
return this;
|
||||
}
|
||||
|
||||
public int getExprChildrenLimit() {
|
||||
return exprChildrenLimit;
|
||||
}
|
||||
|
|
|
|||
|
|
@ -93,6 +93,7 @@ import com.starrocks.mysql.MysqlSerializer;
|
|||
import com.starrocks.persist.CreateInsertOverwriteJobLog;
|
||||
import com.starrocks.persist.gson.GsonUtils;
|
||||
import com.starrocks.planner.HdfsScanNode;
|
||||
import com.starrocks.planner.HiveTableSink;
|
||||
import com.starrocks.planner.OlapScanNode;
|
||||
import com.starrocks.planner.OlapTableSink;
|
||||
import com.starrocks.planner.PlanFragment;
|
||||
|
|
@ -1589,7 +1590,7 @@ public class StmtExecutor {
|
|||
|
||||
if (parsedStmt instanceof InsertStmt && ((InsertStmt) parsedStmt).isOverwrite() &&
|
||||
!((InsertStmt) parsedStmt).hasOverwriteJob() &&
|
||||
!(targetTable instanceof IcebergTable)) {
|
||||
!(targetTable.isIcebergTable() || targetTable.isHiveTable())) {
|
||||
handleInsertOverwrite((InsertStmt) parsedStmt);
|
||||
return;
|
||||
}
|
||||
|
|
@ -1630,8 +1631,8 @@ public class StmtExecutor {
|
|||
sourceType,
|
||||
context.getSessionVariable().getQueryTimeoutS(),
|
||||
authenticateParams);
|
||||
} else if (targetTable instanceof SystemTable || targetTable instanceof IcebergTable) {
|
||||
// schema table and iceberg table does not need txn
|
||||
} else if (targetTable instanceof SystemTable || (targetTable.isIcebergTable() || targetTable.isHiveTable())) {
|
||||
// schema table and iceberg and hive table does not need txn
|
||||
} else {
|
||||
transactionId = GlobalStateMgr.getCurrentGlobalTransactionMgr().beginTransaction(
|
||||
database.getId(),
|
||||
|
|
@ -1701,7 +1702,7 @@ public class StmtExecutor {
|
|||
}
|
||||
|
||||
context.setStatisticsJob(AnalyzerUtils.isStatisticsJob(context, parsedStmt));
|
||||
if (!targetTable.isIcebergTable()) {
|
||||
if (!(targetTable.isIcebergTable() || targetTable.isHiveTable())) {
|
||||
jobId = context.getGlobalStateMgr().getLoadMgr().registerLoadJob(
|
||||
label,
|
||||
database.getFullName(),
|
||||
|
|
@ -1799,7 +1800,8 @@ public class StmtExecutor {
|
|||
TransactionCommitFailedException.FILTER_DATA_IN_STRICT_MODE + ", tracking sql = " +
|
||||
trackingSql
|
||||
);
|
||||
} else if (targetTable instanceof SystemTable || targetTable instanceof IcebergTable) {
|
||||
} else if (targetTable instanceof SystemTable || (targetTable.isHiveTable() ||
|
||||
targetTable.isIcebergTable())) {
|
||||
// schema table does not need txn
|
||||
} else {
|
||||
GlobalStateMgr.getCurrentGlobalTransactionMgr().abortTransaction(
|
||||
|
|
@ -1844,6 +1846,22 @@ public class StmtExecutor {
|
|||
context.getGlobalStateMgr().getMetadataMgr().finishSink(catalogName, dbName, tableName, commitInfos);
|
||||
txnStatus = TransactionStatus.VISIBLE;
|
||||
label = "FAKE_ICEBERG_SINK_LABEL";
|
||||
} else if (targetTable instanceof HiveTable) {
|
||||
List<TSinkCommitInfo> commitInfos = coord.getSinkCommitInfos();
|
||||
HiveTableSink hiveTableSink = (HiveTableSink) execPlan.getFragments().get(0).getSink();
|
||||
String stagingDir = hiveTableSink.getStagingDir();
|
||||
if (stmt instanceof InsertStmt) {
|
||||
InsertStmt insertStmt = (InsertStmt) stmt;
|
||||
for (TSinkCommitInfo commitInfo : commitInfos) {
|
||||
commitInfo.setStaging_dir(stagingDir);
|
||||
if (insertStmt.isOverwrite()) {
|
||||
commitInfo.setIs_overwrite(true);
|
||||
}
|
||||
}
|
||||
}
|
||||
context.getGlobalStateMgr().getMetadataMgr().finishSink(catalogName, dbName, tableName, commitInfos);
|
||||
txnStatus = TransactionStatus.VISIBLE;
|
||||
label = "FAKE_HIVE_SINK_LABEL";
|
||||
} else {
|
||||
if (isExplainAnalyze) {
|
||||
GlobalStateMgr.getCurrentGlobalTransactionMgr()
|
||||
|
|
|
|||
|
|
@ -17,6 +17,7 @@ package com.starrocks.qe.scheduler.dag;
|
|||
import com.google.common.collect.Maps;
|
||||
import com.starrocks.common.util.DebugUtil;
|
||||
import com.starrocks.planner.DataSink;
|
||||
import com.starrocks.planner.HiveTableSink;
|
||||
import com.starrocks.planner.IcebergTableSink;
|
||||
import com.starrocks.planner.PlanFragment;
|
||||
import com.starrocks.planner.PlanFragmentId;
|
||||
|
|
@ -297,7 +298,7 @@ public class FragmentInstance {
|
|||
|
||||
DataSink dataSink = fragment.getSink();
|
||||
int dop = fragment.getPipelineDop();
|
||||
if (!(dataSink instanceof IcebergTableSink)) {
|
||||
if (!(dataSink instanceof IcebergTableSink || dataSink instanceof HiveTableSink)) {
|
||||
return dop;
|
||||
} else {
|
||||
int sessionVarSinkDop = ConnectContext.get().getSessionVariable().getPipelineSinkDop();
|
||||
|
|
|
|||
|
|
@ -68,6 +68,7 @@ public class HiveTableFactory extends ExternalTableFactory {
|
|||
.setDataColumnNames(oHiveTable.getDataColumnNames())
|
||||
.setFullSchema(columns)
|
||||
.setTableLocation(oHiveTable.getTableLocation())
|
||||
.setStorageFormat(oHiveTable.getStorageFormat())
|
||||
.setCreateTime(oHiveTable.getCreateTime());
|
||||
|
||||
HiveTable hiveTable = tableBuilder.build();
|
||||
|
|
|
|||
|
|
@ -27,6 +27,7 @@ import com.starrocks.analysis.SlotRef;
|
|||
import com.starrocks.analysis.StringLiteral;
|
||||
import com.starrocks.analysis.TupleDescriptor;
|
||||
import com.starrocks.catalog.Column;
|
||||
import com.starrocks.catalog.HiveTable;
|
||||
import com.starrocks.catalog.IcebergTable;
|
||||
import com.starrocks.catalog.KeysType;
|
||||
import com.starrocks.catalog.MaterializedView;
|
||||
|
|
@ -37,6 +38,7 @@ import com.starrocks.catalog.Type;
|
|||
import com.starrocks.common.Config;
|
||||
import com.starrocks.common.Pair;
|
||||
import com.starrocks.planner.DataSink;
|
||||
import com.starrocks.planner.HiveTableSink;
|
||||
import com.starrocks.planner.IcebergTableSink;
|
||||
import com.starrocks.planner.MysqlTableSink;
|
||||
import com.starrocks.planner.OlapTableSink;
|
||||
|
|
@ -257,12 +259,16 @@ public class InsertPlanner {
|
|||
descriptorTable.addReferencedTable(targetTable);
|
||||
dataSink = new IcebergTableSink((IcebergTable) targetTable, tupleDesc,
|
||||
isKeyPartitionStaticInsert(insertStmt, queryRelation));
|
||||
} else if (targetTable instanceof HiveTable) {
|
||||
dataSink = new HiveTableSink((HiveTable) targetTable, tupleDesc,
|
||||
isKeyPartitionStaticInsert(insertStmt, queryRelation), session.getSessionVariable());
|
||||
} else {
|
||||
throw new SemanticException("Unknown table type " + insertStmt.getTargetTable().getType());
|
||||
}
|
||||
|
||||
PlanFragment sinkFragment = execPlan.getFragments().get(0);
|
||||
if (canUsePipeline && (targetTable instanceof OlapTable || targetTable instanceof IcebergTable)) {
|
||||
if (canUsePipeline && (targetTable instanceof OlapTable || targetTable.isIcebergTable() ||
|
||||
targetTable.isHiveTable())) {
|
||||
if (shuffleServiceEnable) {
|
||||
// For shuffle insert into, we only support tablet sink dop = 1
|
||||
// because for tablet sink dop > 1, local passthourgh exchange will influence the order of sending,
|
||||
|
|
@ -281,7 +287,9 @@ public class InsertPlanner {
|
|||
if (targetTable instanceof OlapTable) {
|
||||
sinkFragment.setHasOlapTableSink();
|
||||
sinkFragment.setForceAssignScanRangesPerDriverSeq();
|
||||
} else {
|
||||
} else if (targetTable.isHiveTable()) {
|
||||
sinkFragment.setHasHiveTableSink();
|
||||
} else if (targetTable.isIcebergTable()) {
|
||||
sinkFragment.setHasIcebergTableSink();
|
||||
}
|
||||
|
||||
|
|
@ -687,8 +695,16 @@ public class InsertPlanner {
|
|||
|
||||
private boolean needToSkip(InsertStmt stmt, int columnIdx) {
|
||||
Table targetTable = stmt.getTargetTable();
|
||||
return stmt.isSpecifyKeyPartition() &&
|
||||
((IcebergTable) targetTable).partitionColumnIndexes().contains(columnIdx);
|
||||
boolean skip = false;
|
||||
if (stmt.isSpecifyKeyPartition()) {
|
||||
if (targetTable.isIcebergTable()) {
|
||||
return ((IcebergTable) targetTable).partitionColumnIndexes().contains(columnIdx);
|
||||
} else if (targetTable.isHiveTable()) {
|
||||
return columnIdx >= targetTable.getFullSchema().size() - targetTable.getPartitionColumnNames().size();
|
||||
}
|
||||
}
|
||||
|
||||
return skip;
|
||||
}
|
||||
|
||||
private boolean isKeyPartitionStaticInsert(InsertStmt insertStmt, QueryRelation queryRelation) {
|
||||
|
|
@ -696,12 +712,12 @@ public class InsertPlanner {
|
|||
return false;
|
||||
}
|
||||
|
||||
if (!(insertStmt.getTargetTable() instanceof IcebergTable)) {
|
||||
Table targetTable = insertStmt.getTargetTable();
|
||||
if (!(targetTable.isHiveTable() || targetTable.isIcebergTable())) {
|
||||
return false;
|
||||
}
|
||||
|
||||
IcebergTable icebergTable = (IcebergTable) insertStmt.getTargetTable();
|
||||
if (icebergTable.isUnPartitioned()) {
|
||||
if (targetTable.isUnPartitioned()) {
|
||||
return false;
|
||||
}
|
||||
|
||||
|
|
@ -720,7 +736,7 @@ public class InsertPlanner {
|
|||
|
||||
List<String> targetColumnNames;
|
||||
if (insertStmt.getTargetColumnNames() == null) {
|
||||
targetColumnNames = icebergTable.getColumns().stream()
|
||||
targetColumnNames = targetTable.getColumns().stream()
|
||||
.map(Column::getName).collect(Collectors.toList());
|
||||
} else {
|
||||
targetColumnNames = Lists.newArrayList(insertStmt.getTargetColumnNames());
|
||||
|
|
@ -728,7 +744,7 @@ public class InsertPlanner {
|
|||
|
||||
for (int i = 0; i < targetColumnNames.size(); i++) {
|
||||
String columnName = targetColumnNames.get(i);
|
||||
if (icebergTable.getPartitionColumnNames().contains(columnName)) {
|
||||
if (targetTable.getPartitionColumnNames().contains(columnName)) {
|
||||
Expr expr = listItems.get(i).getExpr();
|
||||
if (expr instanceof NullLiteral) {
|
||||
throw new SemanticException("partition value can't be null");
|
||||
|
|
|
|||
|
|
@ -25,16 +25,16 @@ import com.starrocks.catalog.Column;
|
|||
import com.starrocks.catalog.Database;
|
||||
import com.starrocks.catalog.ExternalOlapTable;
|
||||
import com.starrocks.catalog.HiveTable;
|
||||
import com.starrocks.catalog.IcebergTable;
|
||||
import com.starrocks.catalog.MaterializedView;
|
||||
import com.starrocks.catalog.OlapTable;
|
||||
import com.starrocks.catalog.Partition;
|
||||
import com.starrocks.catalog.Table;
|
||||
import com.starrocks.catalog.Type;
|
||||
import com.starrocks.common.AnalysisException;
|
||||
import com.starrocks.common.ErrorCode;
|
||||
import com.starrocks.common.ErrorReport;
|
||||
import com.starrocks.connector.hive.HiveWriteUtils;
|
||||
import com.starrocks.external.starrocks.TableMetaSyncer;
|
||||
import com.starrocks.planner.IcebergTableSink;
|
||||
import com.starrocks.qe.ConnectContext;
|
||||
import com.starrocks.server.CatalogMgr;
|
||||
import com.starrocks.sql.ast.DefaultValueExpr;
|
||||
|
|
@ -94,8 +94,8 @@ public class InsertAnalyzer {
|
|||
}
|
||||
|
||||
if (insertStmt.isOverwrite()) {
|
||||
if (!(table instanceof OlapTable) && !table.isIcebergTable()) {
|
||||
throw unsupportedException("Only support insert overwrite olap table and iceberg table");
|
||||
if (!(table instanceof OlapTable) && !table.isIcebergTable() && !table.isHiveTable()) {
|
||||
throw unsupportedException("Only support insert overwrite olap/iceberg/hive table");
|
||||
}
|
||||
if (table instanceof OlapTable && ((OlapTable) table).getState() != NORMAL) {
|
||||
String msg =
|
||||
|
|
@ -106,15 +106,16 @@ public class InsertAnalyzer {
|
|||
}
|
||||
|
||||
if (!table.supportInsert()) {
|
||||
if (table.isIcebergTable()) {
|
||||
throw unsupportedException("Only support insert into iceberg table with parquet file format");
|
||||
if (table.isIcebergTable() || table.isHiveTable()) {
|
||||
throw unsupportedException(String.format("Only support insert into %s table with parquet file format",
|
||||
table.getType()));
|
||||
}
|
||||
throw unsupportedException("Only support insert into olap table or mysql table or iceberg table");
|
||||
throw unsupportedException("Only support insert into olap/mysql/iceberg/hive table");
|
||||
}
|
||||
|
||||
if (table instanceof IcebergTable && CatalogMgr.isInternalCatalog(catalogName)) {
|
||||
throw unsupportedException("Doesn't support iceberg table sink in the internal catalog. " +
|
||||
"You need to use iceberg catalog.");
|
||||
if ((table.isHiveTable() || table.isIcebergTable()) && CatalogMgr.isInternalCatalog(catalogName)) {
|
||||
throw unsupportedException(String.format("Doesn't support %s table sink in the internal catalog. " +
|
||||
"You need to use %s catalog.", table.getType(), table.getType()));
|
||||
}
|
||||
|
||||
List<Long> targetPartitionIds = Lists.newArrayList();
|
||||
|
|
@ -162,9 +163,12 @@ public class InsertAnalyzer {
|
|||
}
|
||||
}
|
||||
|
||||
if (table instanceof IcebergTable) {
|
||||
IcebergTable icebergTable = (IcebergTable) table;
|
||||
List<String> tablePartitionColumnNames = icebergTable.getPartitionColumnNames();
|
||||
if (table.isIcebergTable() || table.isHiveTable()) {
|
||||
if (table.isHiveTable() && table.isUnPartitioned() &&
|
||||
HiveWriteUtils.isS3Url(table.getTableLocation()) && insertStmt.isOverwrite()) {
|
||||
throw new SemanticException("Unsupported insert overwrite hive unpartitioned table with s3 location");
|
||||
}
|
||||
List<String> tablePartitionColumnNames = table.getPartitionColumnNames();
|
||||
if (insertStmt.getTargetColumnNames() != null) {
|
||||
for (String partitionColName : tablePartitionColumnNames) {
|
||||
if (!insertStmt.getTargetColumnNames().contains(partitionColName)) {
|
||||
|
|
@ -172,13 +176,17 @@ public class InsertAnalyzer {
|
|||
}
|
||||
}
|
||||
} else if (insertStmt.isStaticKeyPartitionInsert()) {
|
||||
checkStaticKeyPartitionInsert(insertStmt, icebergTable, targetPartitionNames);
|
||||
checkStaticKeyPartitionInsert(insertStmt, table, targetPartitionNames);
|
||||
}
|
||||
|
||||
for (Column column : icebergTable.getPartitionColumns()) {
|
||||
if (IcebergTableSink.isUnSupportedPartitionColumnType(column.getType())) {
|
||||
throw new SemanticException("Unsupported partition column type [%s] for iceberg table sink",
|
||||
column.getType().canonicalName());
|
||||
List<Column> partitionColumns = tablePartitionColumnNames.stream()
|
||||
.map(table::getColumn)
|
||||
.collect(Collectors.toList());
|
||||
|
||||
for (Column column : partitionColumns) {
|
||||
if (isUnSupportedPartitionColumnType(column.getType())) {
|
||||
throw new SemanticException("Unsupported partition column type [%s] for %s table sink",
|
||||
column.getType().canonicalName(), table.getType());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -229,7 +237,7 @@ public class InsertAnalyzer {
|
|||
}
|
||||
|
||||
int mentionedColumnSize = mentionedColumns.size();
|
||||
if (table instanceof IcebergTable && insertStmt.isStaticKeyPartitionInsert()) {
|
||||
if ((table.isIcebergTable() || table.isHiveTable()) && insertStmt.isStaticKeyPartitionInsert()) {
|
||||
// full column size = mentioned column size + partition column size for static partition insert
|
||||
mentionedColumnSize -= table.getPartitionColumnNames().size();
|
||||
}
|
||||
|
|
@ -335,4 +343,8 @@ public class InsertAnalyzer {
|
|||
}
|
||||
return copiedTable;
|
||||
}
|
||||
|
||||
public static boolean isUnSupportedPartitionColumnType(Type type) {
|
||||
return type.isFloat() || type.isDecimalOfAnyVersion() || type.isDatetime();
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -21,7 +21,6 @@ import com.starrocks.analysis.Expr;
|
|||
import com.starrocks.analysis.RedirectStatus;
|
||||
import com.starrocks.analysis.TableName;
|
||||
import com.starrocks.catalog.Column;
|
||||
import com.starrocks.catalog.IcebergTable;
|
||||
import com.starrocks.catalog.Table;
|
||||
import com.starrocks.sql.parser.NodePosition;
|
||||
|
||||
|
|
@ -201,7 +200,8 @@ public class InsertStmt extends DmlStmt {
|
|||
}
|
||||
|
||||
public boolean isSpecifyKeyPartition() {
|
||||
return targetTable != null && targetTable instanceof IcebergTable && isStaticKeyPartitionInsert();
|
||||
return targetTable != null && (targetTable.isHiveTable() || targetTable.isIcebergTable()) &&
|
||||
isStaticKeyPartitionInsert();
|
||||
}
|
||||
|
||||
public boolean isStaticKeyPartitionInsert() {
|
||||
|
|
|
|||
|
|
@ -174,6 +174,7 @@ import java.util.ArrayList;
|
|||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Map.Entry;
|
||||
|
|
@ -1112,10 +1113,15 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
|
|||
throw new TException("method not implemented");
|
||||
}
|
||||
|
||||
@Override
|
||||
public int add_partitions(List<Partition> partitions)
|
||||
throws InvalidObjectException, AlreadyExistsException, MetaException, TException {
|
||||
throw new TException("method not implemented");
|
||||
public int add_partitions(List<Partition> new_parts) throws TException {
|
||||
if (new_parts != null && !new_parts.isEmpty() && !((Partition)new_parts.get(0)).isSetCatName()) {
|
||||
String defaultCat = MetaStoreUtils.getDefaultCatalog(this.conf);
|
||||
new_parts.forEach((p) -> {
|
||||
p.setCatName(defaultCat);
|
||||
});
|
||||
}
|
||||
|
||||
return this.client.add_partitions(new_parts);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
@ -1384,30 +1390,21 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
|
|||
name, deleteData, envContext);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void alter_table(String databaseName, String tblName, Table table)
|
||||
throws InvalidOperationException, MetaException, TException {
|
||||
throw new TException("method not implemented");
|
||||
|
||||
public void alter_table(String dbname, String tbl_name, Table new_tbl) throws TException {
|
||||
this.alter_table_with_environmentContext(dbname, tbl_name, new_tbl, (EnvironmentContext)null);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void alter_table(String catName, String dbName, String tblName, Table newTable)
|
||||
throws InvalidOperationException, MetaException, TException {
|
||||
throw new TException("method not implemented");
|
||||
public void alter_table(String defaultDatabaseName, String tblName, Table table, boolean cascade) throws TException {
|
||||
EnvironmentContext environmentContext = new EnvironmentContext();
|
||||
if (cascade) {
|
||||
environmentContext.putToProperties("CASCADE", "true");
|
||||
}
|
||||
|
||||
this.alter_table_with_environmentContext(defaultDatabaseName, tblName, table, environmentContext);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void alter_table(String catName, String dbName, String tblName, Table newTable,
|
||||
EnvironmentContext envContext) throws InvalidOperationException, MetaException, TException {
|
||||
throw new TException("method not implemented");
|
||||
}
|
||||
|
||||
@Override
|
||||
public void alter_table(String defaultDatabaseName, String tblName, Table table, boolean cascade)
|
||||
throws InvalidOperationException, MetaException, TException {
|
||||
throw new TException("method not implemented");
|
||||
public void alter_table(String catName, String dbName, String tblName, Table newTable, EnvironmentContext envContext) throws TException {
|
||||
this.client.alter_table_with_environment_context(MetaStoreUtils.prependCatalogToDbName(catName, dbName, this.conf), tblName, newTable, envContext);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
@ -1506,28 +1503,67 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
|
|||
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean dropPartition(String db_name, String tbl_name, List<String> part_vals, boolean deleteData)
|
||||
throws NoSuchObjectException, MetaException, TException {
|
||||
throw new TException("method not implemented");
|
||||
public boolean dropPartition(String dbName, String tableName, String partName, boolean deleteData) throws TException {
|
||||
return this.dropPartition(MetaStoreUtils.getDefaultCatalog(this.conf), dbName, tableName, partName, deleteData);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean dropPartition(String catName, String db_name, String tbl_name, List<String> part_vals,
|
||||
boolean deleteData) throws NoSuchObjectException, MetaException, TException {
|
||||
throw new TException("method not implemented");
|
||||
public boolean dropPartition(String catName, String db_name, String tbl_name, String name, boolean deleteData) throws TException {
|
||||
return this.client.drop_partition_by_name_with_environment_context(MetaStoreUtils.prependCatalogToDbName(catName, db_name, this.conf), tbl_name, name, deleteData, (EnvironmentContext)null);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean dropPartition(String db_name, String tbl_name, List<String> part_vals, PartitionDropOptions options)
|
||||
throws NoSuchObjectException, MetaException, TException {
|
||||
throw new TException("method not implemented");
|
||||
private static EnvironmentContext getEnvironmentContextWithIfPurgeSet() {
|
||||
Map<String, String> warehouseOptions = new HashMap();
|
||||
warehouseOptions.put("ifPurge", "TRUE");
|
||||
return new EnvironmentContext(warehouseOptions);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean dropPartition(String catName, String db_name, String tbl_name, List<String> part_vals,
|
||||
PartitionDropOptions options) throws NoSuchObjectException, MetaException, TException {
|
||||
throw new TException("method not implemented");
|
||||
/** @deprecated */
|
||||
@Deprecated
|
||||
public boolean dropPartition(String db_name, String tbl_name, List<String> part_vals, EnvironmentContext env_context) throws TException {
|
||||
return this.client.drop_partition_with_environment_context(MetaStoreUtils.prependCatalogToDbName(db_name, this.conf), tbl_name, part_vals, true, env_context);
|
||||
}
|
||||
|
||||
/** @deprecated */
|
||||
@Deprecated
|
||||
public boolean dropPartition(String dbName, String tableName, String partName, boolean dropData, EnvironmentContext ec) throws TException {
|
||||
return this.client.drop_partition_by_name_with_environment_context(MetaStoreUtils.prependCatalogToDbName(dbName, this.conf), tableName, partName, dropData, ec);
|
||||
}
|
||||
|
||||
/** @deprecated */
|
||||
@Deprecated
|
||||
public boolean dropPartition(String dbName, String tableName, List<String> partVals) throws TException {
|
||||
return this.client.drop_partition(MetaStoreUtils.prependCatalogToDbName(dbName, this.conf), tableName, partVals, true);
|
||||
}
|
||||
|
||||
public boolean dropPartition(String db_name, String tbl_name, List<String> part_vals, boolean deleteData) throws TException {
|
||||
return this.dropPartition(MetaStoreUtils.getDefaultCatalog(this.conf), db_name, tbl_name, part_vals, PartitionDropOptions.instance().deleteData(deleteData));
|
||||
}
|
||||
|
||||
public boolean dropPartition(String catName, String db_name, String tbl_name, List<String> part_vals, boolean deleteData) throws TException {
|
||||
return this.dropPartition(catName, db_name, tbl_name, part_vals, PartitionDropOptions.instance().deleteData(deleteData));
|
||||
}
|
||||
|
||||
public boolean dropPartition(String db_name, String tbl_name, List<String> part_vals, PartitionDropOptions options) throws TException {
|
||||
return this.dropPartition(MetaStoreUtils.getDefaultCatalog(this.conf), db_name, tbl_name, part_vals, options);
|
||||
}
|
||||
|
||||
public boolean dropPartition(String catName, String db_name, String tbl_name, List<String> part_vals, PartitionDropOptions options) throws TException {
|
||||
if (options == null) {
|
||||
options = PartitionDropOptions.instance();
|
||||
}
|
||||
|
||||
if (part_vals != null) {
|
||||
Iterator var6 = part_vals.iterator();
|
||||
|
||||
while(var6.hasNext()) {
|
||||
String partVal = (String)var6.next();
|
||||
if (partVal == null) {
|
||||
throw new MetaException("The partition value must not be null.");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return this.client.drop_partition_with_environment_context(MetaStoreUtils.prependCatalogToDbName(catName, db_name, this.conf), tbl_name, part_vals, options.deleteData, options.purgeData ? getEnvironmentContextWithIfPurgeSet() : null);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
@ -1573,47 +1609,17 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
|
|||
throw new TException("method not implemented");
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean dropPartition(String db_name, String tbl_name, String name, boolean deleteData)
|
||||
throws NoSuchObjectException, MetaException, TException {
|
||||
throw new TException("method not implemented");
|
||||
public void alter_partition(String dbName, String tblName, Partition newPart) throws InvalidOperationException, MetaException, TException {
|
||||
this.alter_partition(MetaStoreUtils.getDefaultCatalog(this.conf), dbName, tblName, newPart, (EnvironmentContext)null);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean dropPartition(String catName, String db_name, String tbl_name, String name, boolean deleteData)
|
||||
throws NoSuchObjectException, MetaException, TException {
|
||||
throw new TException("method not implemented");
|
||||
public void alter_partition(String dbName, String tblName, Partition newPart, EnvironmentContext environmentContext) throws InvalidOperationException, MetaException, TException {
|
||||
this.alter_partition(MetaStoreUtils.getDefaultCatalog(this.conf), dbName, tblName, newPart, environmentContext);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void alter_partition(String dbName, String tblName, Partition newPart)
|
||||
throws InvalidOperationException, MetaException, TException {
|
||||
throw new TException("method not implemented");
|
||||
|
||||
public void alter_partition(String catName, String dbName, String tblName, Partition newPart, EnvironmentContext environmentContext) throws TException {
|
||||
this.client.alter_partition_with_environment_context(MetaStoreUtils.prependCatalogToDbName(catName, dbName, this.conf), tblName, newPart, environmentContext);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void alter_partition(String catName, String dbName, String tblName, Partition newPart)
|
||||
throws InvalidOperationException, MetaException, TException {
|
||||
throw new TException("method not implemented");
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void alter_partition(String dbName, String tblName, Partition newPart, EnvironmentContext environmentContext)
|
||||
throws InvalidOperationException, MetaException, TException {
|
||||
throw new TException("method not implemented");
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void alter_partition(String catName, String dbName, String tblName, Partition newPart,
|
||||
EnvironmentContext environmentContext)
|
||||
throws InvalidOperationException, MetaException, TException {
|
||||
throw new TException("method not implemented");
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void alter_partitions(String dbName, String tblName, List<Partition> newParts)
|
||||
throws InvalidOperationException, MetaException, TException {
|
||||
|
|
|
|||
|
|
@ -59,6 +59,7 @@ import org.junit.Test;
|
|||
|
||||
import java.util.List;
|
||||
|
||||
import static com.starrocks.connector.hive.HiveClassNames.MAPRED_PARQUET_INPUT_FORMAT_CLASS;
|
||||
import static com.starrocks.server.CatalogMgr.ResourceMappingCatalog.getResourceMappingCatalogName;
|
||||
|
||||
public class HiveTableTest {
|
||||
|
|
@ -88,6 +89,7 @@ public class HiveTableTest {
|
|||
List<FieldSchema> unPartKeys = Lists.newArrayList(new FieldSchema("col2", "INT", ""));
|
||||
String hdfsPath = "hdfs://127.0.0.1:10000/hive";
|
||||
StorageDescriptor sd = new StorageDescriptor();
|
||||
sd.setInputFormat(MAPRED_PARQUET_INPUT_FORMAT_CLASS);
|
||||
sd.setCols(unPartKeys);
|
||||
sd.setLocation(hdfsPath);
|
||||
Table msTable = new Table();
|
||||
|
|
@ -100,6 +102,7 @@ public class HiveTableTest {
|
|||
msTable.setCreateTime(createTime);
|
||||
|
||||
HiveTable oTable = HiveMetastoreApiConverter.toHiveTable(msTable, getResourceMappingCatalogName("hive0", "hive"));
|
||||
Assert.assertTrue(oTable.supportInsert());
|
||||
new Expectations() {
|
||||
{
|
||||
GlobalStateMgr.getCurrentState().getMetadataMgr();
|
||||
|
|
|
|||
|
|
@ -17,10 +17,6 @@ package com.starrocks.connector;
|
|||
|
||||
import com.google.common.collect.Lists;
|
||||
import com.starrocks.common.FeConstants;
|
||||
import com.starrocks.connector.CachingRemoteFileIO;
|
||||
import com.starrocks.connector.RemoteFileBlockDesc;
|
||||
import com.starrocks.connector.RemoteFileDesc;
|
||||
import com.starrocks.connector.RemotePathKey;
|
||||
import com.starrocks.connector.hive.HiveRemoteFileIO;
|
||||
import com.starrocks.connector.hive.MockedRemoteFileSystem;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
|
|
@ -73,5 +69,7 @@ public class CachingRemoteFileIOTest {
|
|||
Map<RemotePathKey, List<RemoteFileDesc>> presentRemoteFileInfos =
|
||||
cachingFileIO.getPresentRemoteFiles(Lists.newArrayList(pathKey));
|
||||
Assert.assertEquals(1, presentRemoteFileInfos.size());
|
||||
|
||||
queryLevelCache.updateRemoteFiles(pathKey);
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -22,6 +22,7 @@ import com.starrocks.catalog.Type;
|
|||
import com.starrocks.connector.exception.StarRocksConnectorException;
|
||||
import com.starrocks.connector.hive.HiveClassNames;
|
||||
import com.starrocks.connector.hive.HiveMetastoreApiConverter;
|
||||
import com.starrocks.connector.hive.HiveStorageFormat;
|
||||
import mockit.Expectations;
|
||||
import mockit.Mocked;
|
||||
import org.apache.avro.Schema;
|
||||
|
|
@ -163,6 +164,7 @@ public class HiveMetastoreApiConverterTest {
|
|||
.setFullSchema(Lists.newArrayList(new Column("c1", Type.INT), new Column("p1", Type.INT)))
|
||||
.setDataColumnNames(Lists.newArrayList("c1"))
|
||||
.setTableLocation("table_location")
|
||||
.setStorageFormat(HiveStorageFormat.PARQUET)
|
||||
.build();
|
||||
hiveTable.setComment("my_comment");
|
||||
Table table = HiveMetastoreApiConverter.toMetastoreApiTable(hiveTable);
|
||||
|
|
@ -178,6 +180,6 @@ public class HiveMetastoreApiConverterTest {
|
|||
Assert.assertEquals(HiveClassNames.MAPRED_PARQUET_OUTPUT_FORMAT_CLASS, table.getSd().getOutputFormat());
|
||||
|
||||
Assert.assertEquals("my_comment", table.getParameters().get("comment"));
|
||||
Assert.assertEquals("-1", table.getParameters().get("numFiles"));
|
||||
Assert.assertEquals("0", table.getParameters().get("numRows"));
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -31,6 +31,7 @@ import com.starrocks.catalog.ScalarType;
|
|||
import com.starrocks.catalog.Table;
|
||||
import com.starrocks.catalog.Type;
|
||||
import com.starrocks.common.AnalysisException;
|
||||
import com.starrocks.common.ExceptionChecker;
|
||||
import com.starrocks.common.UserException;
|
||||
import com.starrocks.connector.exception.StarRocksConnectorException;
|
||||
import com.starrocks.connector.hive.HiveMetaClient;
|
||||
|
|
@ -50,6 +51,7 @@ import java.util.Optional;
|
|||
|
||||
import static com.starrocks.connector.PartitionUtil.createPartitionKey;
|
||||
import static com.starrocks.connector.PartitionUtil.fromPartitionKey;
|
||||
import static com.starrocks.connector.PartitionUtil.getPartitionName;
|
||||
import static com.starrocks.connector.PartitionUtil.getSuffixName;
|
||||
import static com.starrocks.connector.PartitionUtil.toPartitionValues;
|
||||
|
||||
|
|
@ -266,4 +268,20 @@ public class PartitionUtilTest {
|
|||
upperBound.pushColumn(new DateLiteral(2022, 12, 03), PrimitiveType.DATE);
|
||||
Assert.assertTrue(partitionMap.get("p20221202").upperEndpoint().equals(upperBound));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetPartition() {
|
||||
String base = "hdfs://hadoop01:9000/mytable";
|
||||
String tableLocation = "hdfs://hadoop01:9000/mytable/";
|
||||
Assert.assertTrue(getPartitionName(base, tableLocation).isEmpty());
|
||||
|
||||
String errorPath = "hdfs://aaa/bbb";
|
||||
ExceptionChecker.expectThrowsWithMsg(
|
||||
IllegalStateException.class,
|
||||
"Can't infer partition name. base path",
|
||||
() -> PartitionUtil.getPartitionName(base, errorPath));
|
||||
|
||||
String partitionPath = "hdfs://hadoop01:9000/mytable/year=2023/month=12/day=30";
|
||||
Assert.assertEquals("year=2023/month=12/day=30", PartitionUtil.getPartitionName(base, partitionPath));
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -12,35 +12,40 @@
|
|||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
|
||||
package com.starrocks.connector;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
import com.starrocks.common.ExceptionChecker;
|
||||
import com.starrocks.common.FeConstants;
|
||||
import com.starrocks.connector.CachingRemoteFileIO;
|
||||
import com.starrocks.connector.RemoteFileBlockDesc;
|
||||
import com.starrocks.connector.RemoteFileDesc;
|
||||
import com.starrocks.connector.RemoteFileInfo;
|
||||
import com.starrocks.connector.RemoteFileOperations;
|
||||
import com.starrocks.connector.RemotePathKey;
|
||||
import com.starrocks.connector.exception.StarRocksConnectorException;
|
||||
import com.starrocks.connector.hive.HiveMetaClient;
|
||||
import com.starrocks.connector.hive.HiveMetastore;
|
||||
import com.starrocks.connector.hive.HiveMetastoreTest;
|
||||
import com.starrocks.connector.hive.HiveRemoteFileIO;
|
||||
import com.starrocks.connector.hive.HiveWriteUtils;
|
||||
import com.starrocks.connector.hive.MockedRemoteFileSystem;
|
||||
import com.starrocks.connector.hive.Partition;
|
||||
import com.starrocks.connector.hive.RemoteFileInputFormat;
|
||||
import mockit.Mock;
|
||||
import mockit.MockUp;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.net.URI;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
|
||||
import static com.starrocks.connector.hive.MockedRemoteFileSystem.TEST_FILES;
|
||||
import static io.airlift.concurrent.MoreFutures.getFutureValue;
|
||||
|
||||
public class RemoteFileOperationsTest {
|
||||
@Test
|
||||
|
|
@ -53,7 +58,8 @@ public class RemoteFileOperationsTest {
|
|||
ExecutorService executorToLoad = Executors.newFixedThreadPool(5);
|
||||
|
||||
CachingRemoteFileIO cachingFileIO = new CachingRemoteFileIO(hiveRemoteFileIO, executorToRefresh, 10, 10, 10);
|
||||
RemoteFileOperations ops = new RemoteFileOperations(cachingFileIO, executorToLoad, false, true);
|
||||
RemoteFileOperations ops = new RemoteFileOperations(cachingFileIO, executorToLoad, executorToLoad,
|
||||
false, true, new Configuration());
|
||||
|
||||
String tableLocation = "hdfs://127.0.0.1:10000/hive.db/hive_tbl";
|
||||
RemotePathKey pathKey = RemotePathKey.of(tableLocation, false);
|
||||
|
|
@ -97,5 +103,174 @@ public class RemoteFileOperationsTest {
|
|||
Assert.assertEquals(1, presentRemoteFileInfos.size());
|
||||
|
||||
Assert.assertEquals(2, ops.getPresentFilesInCache(partitions.values()).size());
|
||||
|
||||
ops.refreshPartitionFilesCache(new Path(tableLocation));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void asyncRenameFilesTest() {
|
||||
HiveRemoteFileIO hiveRemoteFileIO = new HiveRemoteFileIO(new Configuration());
|
||||
FileSystem fs = new MockedRemoteFileSystem(TEST_FILES);
|
||||
hiveRemoteFileIO.setFileSystem(fs);
|
||||
FeConstants.runningUnitTest = true;
|
||||
ExecutorService executorToRefresh = Executors.newFixedThreadPool(5);
|
||||
ExecutorService executorToLoad = Executors.newFixedThreadPool(5);
|
||||
|
||||
CachingRemoteFileIO cachingFileIO = new CachingRemoteFileIO(hiveRemoteFileIO, executorToRefresh, 10, 10, 10);
|
||||
RemoteFileOperations ops = new RemoteFileOperations(cachingFileIO, executorToLoad, executorToLoad,
|
||||
false, true, new Configuration());
|
||||
|
||||
List<CompletableFuture<?>> futures = new ArrayList<>();
|
||||
Path writePath = new Path("hdfs://hadoop01:9000/tmp/starrocks/queryid");
|
||||
Path targetPath = new Path("hdfs://hadoop01:9000/user/hive/warehouse/test.db/t1");
|
||||
List<String> fileNames = Lists.newArrayList("file1");
|
||||
ExceptionChecker.expectThrowsWithMsg(
|
||||
StarRocksConnectorException.class,
|
||||
"Failed to move data files to target location." +
|
||||
" Failed to get file system on path hdfs://hadoop01:9000/tmp/starrocks/queryid",
|
||||
() -> ops.asyncRenameFiles(futures, new AtomicBoolean(true), writePath, targetPath, fileNames));
|
||||
|
||||
|
||||
RemoteFileOperations ops1 = new RemoteFileOperations(cachingFileIO, executorToLoad, Executors.newSingleThreadExecutor(),
|
||||
false, true, new Configuration());
|
||||
|
||||
FileSystem mockedFs = new MockedRemoteFileSystem(TEST_FILES) {
|
||||
@Override
|
||||
public boolean exists(Path path) {
|
||||
return true;
|
||||
}
|
||||
};
|
||||
|
||||
new MockUp<FileSystem>() {
|
||||
@Mock
|
||||
public FileSystem get(URI uri, Configuration conf) throws IOException {
|
||||
return mockedFs;
|
||||
}
|
||||
};
|
||||
|
||||
ExceptionChecker.expectThrowsWithMsg(
|
||||
StarRocksConnectorException.class,
|
||||
"Failed to move data files from hdfs://hadoop01:9000/tmp/starrocks/queryid/file1 to" +
|
||||
" target location hdfs://hadoop01:9000/user/hive/warehouse/test.db/t1/file1." +
|
||||
" msg: target location already exists",
|
||||
() -> {
|
||||
ops1.asyncRenameFiles(futures, new AtomicBoolean(false), writePath, targetPath, fileNames);
|
||||
getFutureValue(futures.get(0), StarRocksConnectorException.class);
|
||||
});
|
||||
|
||||
new MockUp<FileSystem>() {
|
||||
@Mock
|
||||
public FileSystem get(URI uri, Configuration conf) throws IOException {
|
||||
return fs;
|
||||
}
|
||||
};
|
||||
|
||||
ExceptionChecker.expectThrowsWithMsg(
|
||||
StarRocksConnectorException.class,
|
||||
"Failed to move data files from hdfs://hadoop01:9000/tmp/starrocks/queryid/file1 to" +
|
||||
" target location hdfs://hadoop01:9000/user/hive/warehouse/test.db/t1/file1." +
|
||||
" msg: rename operation failed",
|
||||
() -> {
|
||||
futures.clear();
|
||||
ops.asyncRenameFiles(futures, new AtomicBoolean(false), writePath, targetPath, fileNames);
|
||||
getFutureValue(futures.get(0), StarRocksConnectorException.class);
|
||||
});
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRenameDir() {
|
||||
HiveRemoteFileIO hiveRemoteFileIO = new HiveRemoteFileIO(new Configuration());
|
||||
FileSystem fs = new MockedRemoteFileSystem(TEST_FILES);
|
||||
hiveRemoteFileIO.setFileSystem(fs);
|
||||
FeConstants.runningUnitTest = true;
|
||||
ExecutorService executorToRefresh = Executors.newSingleThreadExecutor();
|
||||
ExecutorService executorToLoad = Executors.newSingleThreadExecutor();
|
||||
CachingRemoteFileIO cachingFileIO = new CachingRemoteFileIO(hiveRemoteFileIO, executorToRefresh, 10, 10, 10);
|
||||
RemoteFileOperations ops = new RemoteFileOperations(cachingFileIO, executorToLoad, executorToLoad,
|
||||
false, true, new Configuration());
|
||||
new MockUp<HiveWriteUtils>() {
|
||||
@Mock
|
||||
public boolean pathExists(Path path, Configuration conf) {
|
||||
return true;
|
||||
}
|
||||
};
|
||||
|
||||
Path writePath = new Path("hdfs://hadoop01:9000/tmp/starrocks/queryid");
|
||||
Path targetPath = new Path("hdfs://hadoop01:9000/user/hive/warehouse/test.db/t1");
|
||||
|
||||
ExceptionChecker.expectThrowsWithMsg(
|
||||
StarRocksConnectorException.class,
|
||||
"Unable to rename from hdfs://hadoop01:9000/tmp/starrocks/queryid to " +
|
||||
"hdfs://hadoop01:9000/user/hive/warehouse/test.db/t1. msg: target directory already exists",
|
||||
() -> ops.renameDirectory(writePath, targetPath, () -> {}));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRenameDirFailed() {
|
||||
HiveRemoteFileIO hiveRemoteFileIO = new HiveRemoteFileIO(new Configuration());
|
||||
FileSystem fs = new MockedRemoteFileSystem(TEST_FILES);
|
||||
hiveRemoteFileIO.setFileSystem(fs);
|
||||
FeConstants.runningUnitTest = true;
|
||||
ExecutorService executorToRefresh = Executors.newSingleThreadExecutor();
|
||||
ExecutorService executorToLoad = Executors.newSingleThreadExecutor();
|
||||
CachingRemoteFileIO cachingFileIO = new CachingRemoteFileIO(hiveRemoteFileIO, executorToRefresh, 10, 10, 10);
|
||||
RemoteFileOperations ops = new RemoteFileOperations(cachingFileIO, executorToLoad, executorToLoad,
|
||||
false, true, new Configuration());
|
||||
|
||||
|
||||
Path writePath = new Path("hdfs://hadoop01:9000/tmp/starrocks/queryid");
|
||||
Path targetPath = new Path("hdfs://hadoop01:9000/user/hive/warehouse/test.db/t1");
|
||||
FileSystem mockedFs = new MockedRemoteFileSystem(TEST_FILES) {
|
||||
@Override
|
||||
public boolean exists(Path path) {
|
||||
if (path.equals(targetPath.getParent())) {
|
||||
return true;
|
||||
} else {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
new MockUp<FileSystem>() {
|
||||
@Mock
|
||||
public FileSystem get(URI uri, Configuration conf) throws IOException {
|
||||
return mockedFs;
|
||||
}
|
||||
};
|
||||
|
||||
ExceptionChecker.expectThrowsWithMsg(
|
||||
StarRocksConnectorException.class,
|
||||
"Failed to rename",
|
||||
() -> ops.renameDirectory(writePath, targetPath, () -> {}));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRemoveNotCurrentQueryFiles() {
|
||||
HiveRemoteFileIO hiveRemoteFileIO = new HiveRemoteFileIO(new Configuration());
|
||||
FileSystem fs = new MockedRemoteFileSystem(TEST_FILES);
|
||||
hiveRemoteFileIO.setFileSystem(fs);
|
||||
FeConstants.runningUnitTest = true;
|
||||
ExecutorService executorToRefresh = Executors.newSingleThreadExecutor();
|
||||
ExecutorService executorToLoad = Executors.newSingleThreadExecutor();
|
||||
CachingRemoteFileIO cachingFileIO = new CachingRemoteFileIO(hiveRemoteFileIO, executorToRefresh, 10, 10, 10);
|
||||
RemoteFileOperations ops = new RemoteFileOperations(cachingFileIO, executorToLoad, executorToLoad,
|
||||
false, true, new Configuration());
|
||||
Path targetPath = new Path("hdfs://hadoop01:9000/user/hive/warehouse/test.db/t1");
|
||||
|
||||
ExceptionChecker.expectThrowsWithMsg(
|
||||
StarRocksConnectorException.class,
|
||||
"Failed to delete partition",
|
||||
() -> ops.removeNotCurrentQueryFiles(targetPath, "aaa"));
|
||||
|
||||
new MockUp<FileSystem>() {
|
||||
@Mock
|
||||
public FileSystem get(URI uri, Configuration conf) throws IOException {
|
||||
return fs;
|
||||
}
|
||||
};
|
||||
ExceptionChecker.expectThrowsWithMsg(
|
||||
StarRocksConnectorException.class,
|
||||
"file name or query id is invalid",
|
||||
() -> ops.removeNotCurrentQueryFiles(targetPath, "aaa"));
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -16,11 +16,13 @@ package com.starrocks.connector.hive;
|
|||
|
||||
import com.google.common.collect.Lists;
|
||||
import com.starrocks.analysis.StringLiteral;
|
||||
import com.starrocks.catalog.Column;
|
||||
import com.starrocks.catalog.Database;
|
||||
import com.starrocks.catalog.HivePartitionKey;
|
||||
import com.starrocks.catalog.HiveTable;
|
||||
import com.starrocks.catalog.PrimitiveType;
|
||||
import com.starrocks.catalog.ScalarType;
|
||||
import com.starrocks.catalog.Type;
|
||||
import com.starrocks.connector.PartitionUtil;
|
||||
import com.starrocks.connector.exception.StarRocksConnectorException;
|
||||
import mockit.Expectations;
|
||||
|
|
@ -31,6 +33,8 @@ import org.junit.Before;
|
|||
import org.junit.Test;
|
||||
|
||||
import java.lang.reflect.InvocationTargetException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
|
|
@ -252,4 +256,59 @@ public class CachingHiveMetastoreTest {
|
|||
List<String> value = PartitionUtil.fromPartitionKey(hivePartitionKey);
|
||||
Assert.assertEquals(HiveMetaClient.PARTITION_NULL_VALUE, value.get(0));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testPartitionExist() {
|
||||
CachingHiveMetastore cachingHiveMetastore = new CachingHiveMetastore(
|
||||
metastore, executor, expireAfterWriteSec, refreshAfterWriteSec, 1000, false);
|
||||
Assert.assertTrue(cachingHiveMetastore.partitionExists("db", "tbl", Lists.newArrayList()));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAddPartitions() {
|
||||
CachingHiveMetastore cachingHiveMetastore = new CachingHiveMetastore(
|
||||
metastore, executor, expireAfterWriteSec, refreshAfterWriteSec, 1000, false);
|
||||
cachingHiveMetastore.addPartitions("db", "table", new ArrayList<>());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDropPartition() {
|
||||
CachingHiveMetastore cachingHiveMetastore = new CachingHiveMetastore(
|
||||
metastore, executor, expireAfterWriteSec, refreshAfterWriteSec, 1000, false);
|
||||
cachingHiveMetastore.dropPartition("db", "table", Lists.newArrayList("1"), false);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAlterPartition() {
|
||||
CachingHiveMetastore cachingHiveMetastore = new CachingHiveMetastore(
|
||||
metastore, executor, expireAfterWriteSec, refreshAfterWriteSec, 1000, false);
|
||||
HivePartition hivePartition = HivePartition.builder()
|
||||
.setColumns(Lists.newArrayList(new Column("c1", Type.INT)))
|
||||
.setStorageFormat(HiveStorageFormat.PARQUET)
|
||||
.setDatabaseName("db")
|
||||
.setTableName("table")
|
||||
.setLocation("location")
|
||||
.setValues(Lists.newArrayList("p1=1"))
|
||||
.setParameters(new HashMap<>()).build();
|
||||
|
||||
HivePartitionStats hivePartitionStats = HivePartitionStats.empty();
|
||||
HivePartitionWithStats hivePartitionWithStats = new HivePartitionWithStats("p1=1", hivePartition, hivePartitionStats);
|
||||
cachingHiveMetastore.alterPartition(hivePartitionWithStats);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUpdateTableStats() {
|
||||
CachingHiveMetastore cachingHiveMetastore = new CachingHiveMetastore(
|
||||
metastore, executor, expireAfterWriteSec, refreshAfterWriteSec, 1000, false);
|
||||
HivePartitionStats partitionStats = HivePartitionStats.empty();
|
||||
cachingHiveMetastore.updateTableStatistics("db", "table", ignore -> partitionStats);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUpdatePartitionStats() {
|
||||
CachingHiveMetastore cachingHiveMetastore = new CachingHiveMetastore(
|
||||
metastore, executor, expireAfterWriteSec, refreshAfterWriteSec, 1000, false);
|
||||
HivePartitionStats partitionStats = HivePartitionStats.empty();
|
||||
cachingHiveMetastore.updatePartitionStatistics("db", "table", "p1=1", ignore -> partitionStats);
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -38,6 +38,7 @@ import com.starrocks.connector.RemoteFileOperations;
|
|||
import com.starrocks.connector.RemotePathKey;
|
||||
import com.starrocks.connector.exception.StarRocksConnectorException;
|
||||
import com.starrocks.qe.ConnectContext;
|
||||
import com.starrocks.sql.analyzer.AnalyzeTestUtil;
|
||||
import com.starrocks.sql.ast.DropTableStmt;
|
||||
import com.starrocks.sql.optimizer.Memo;
|
||||
import com.starrocks.sql.optimizer.OptimizerContext;
|
||||
|
|
@ -45,9 +46,17 @@ import com.starrocks.sql.optimizer.base.ColumnRefFactory;
|
|||
import com.starrocks.sql.optimizer.operator.scalar.ColumnRefOperator;
|
||||
import com.starrocks.sql.optimizer.statistics.ColumnStatistic;
|
||||
import com.starrocks.sql.optimizer.statistics.Statistics;
|
||||
import com.starrocks.thrift.THiveFileInfo;
|
||||
import com.starrocks.thrift.TSinkCommitInfo;
|
||||
import com.starrocks.utframe.UtFrameUtils;
|
||||
import mockit.Expectations;
|
||||
import mockit.Mock;
|
||||
import mockit.MockUp;
|
||||
import mockit.Mocked;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.junit.After;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
|
|
@ -57,9 +66,13 @@ import java.util.HashMap;
|
|||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.function.Function;
|
||||
|
||||
import static com.starrocks.connector.hive.HiveMetadata.STARROCKS_QUERY_ID;
|
||||
import static com.starrocks.connector.hive.MockedRemoteFileSystem.TEST_FILES;
|
||||
|
||||
public class HiveMetadataTest {
|
||||
|
|
@ -98,7 +111,8 @@ public class HiveMetadataTest {
|
|||
hiveRemoteFileIO.setFileSystem(fs);
|
||||
cachingRemoteFileIO = CachingRemoteFileIO.createCatalogLevelInstance(
|
||||
hiveRemoteFileIO, executorForRemoteFileRefresh, 100, 10, 10);
|
||||
fileOps = new RemoteFileOperations(cachingRemoteFileIO, executorForPullFiles, false, true);
|
||||
fileOps = new RemoteFileOperations(cachingRemoteFileIO, executorForPullFiles, executorForPullFiles,
|
||||
false, true, new Configuration());
|
||||
statisticsProvider = new HiveStatisticsProvider(hmsOps, fileOps);
|
||||
|
||||
UtFrameUtils.createMinStarRocksCluster();
|
||||
|
|
@ -106,7 +120,8 @@ public class HiveMetadataTest {
|
|||
connectContext = UtFrameUtils.createDefaultCtx();
|
||||
columnRefFactory = new ColumnRefFactory();
|
||||
optimizerContext = new OptimizerContext(new Memo(), columnRefFactory, connectContext);
|
||||
hiveMetadata = new HiveMetadata("hive_catalog", hmsOps, fileOps, statisticsProvider, Optional.empty());
|
||||
hiveMetadata = new HiveMetadata("hive_catalog", hmsOps, fileOps, statisticsProvider,
|
||||
Optional.empty(), executorForHmsRefresh);
|
||||
}
|
||||
|
||||
@After
|
||||
|
|
@ -317,4 +332,313 @@ public class HiveMetadataTest {
|
|||
|
||||
hiveMetadata.dropTable(new DropTableStmt(false, tableName, true));
|
||||
}
|
||||
|
||||
@Test(expected = StarRocksConnectorException.class)
|
||||
public void testFinishSink() {
|
||||
String stagingDir = "hdfs://127.0.0.1:10000/tmp/starrocks/queryid";
|
||||
THiveFileInfo fileInfo = new THiveFileInfo();
|
||||
fileInfo.setFile_name("myfile.parquet");
|
||||
fileInfo.setPartition_path("hdfs://127.0.0.1:10000/tmp/starrocks/queryid/col1=2");
|
||||
fileInfo.setRecord_count(10);
|
||||
fileInfo.setFile_size_in_bytes(100);
|
||||
TSinkCommitInfo tSinkCommitInfo = new TSinkCommitInfo();
|
||||
tSinkCommitInfo.setStaging_dir(stagingDir);
|
||||
tSinkCommitInfo.setIs_overwrite(false);
|
||||
tSinkCommitInfo.setHive_file_info(fileInfo);
|
||||
hiveMetadata.finishSink("hive_db", "hive_table", Lists.newArrayList(tSinkCommitInfo));
|
||||
}
|
||||
|
||||
|
||||
@Test(expected = StarRocksConnectorException.class)
|
||||
public void testAddPartition() throws Exception {
|
||||
String stagingDir = "hdfs://127.0.0.1:10000/tmp/starrocks/queryid";
|
||||
THiveFileInfo fileInfo = new THiveFileInfo();
|
||||
fileInfo.setFile_name("myfile.parquet");
|
||||
fileInfo.setPartition_path("hdfs://127.0.0.1:10000/tmp/starrocks/queryid/col1=2");
|
||||
fileInfo.setRecord_count(10);
|
||||
fileInfo.setFile_size_in_bytes(100);
|
||||
TSinkCommitInfo tSinkCommitInfo = new TSinkCommitInfo();
|
||||
tSinkCommitInfo.setStaging_dir(stagingDir);
|
||||
tSinkCommitInfo.setIs_overwrite(false);
|
||||
tSinkCommitInfo.setHive_file_info(fileInfo);
|
||||
|
||||
new MockUp<HiveMetastoreOperations>() {
|
||||
@Mock
|
||||
public boolean partitionExists(String dbName, String tableName, List<String> partitionValues) {
|
||||
return false;
|
||||
}
|
||||
};
|
||||
|
||||
new MockUp<RemoteFileOperations>() {
|
||||
@Mock
|
||||
public void renameDirectory(Path source, Path target, Runnable runWhenPathNotExist) {
|
||||
}
|
||||
};
|
||||
|
||||
AnalyzeTestUtil.init();
|
||||
hiveMetadata.finishSink("hive_db", "hive_table", Lists.newArrayList(tSinkCommitInfo));
|
||||
|
||||
new MockUp<HiveMetastoreOperations>() {
|
||||
@Mock
|
||||
public void addPartitions(String dbName, String tableName, List<HivePartitionWithStats> partitions) {
|
||||
throw new StarRocksConnectorException("add partition failed");
|
||||
}
|
||||
};
|
||||
hiveMetadata.finishSink("hive_db", "hive_table", Lists.newArrayList(tSinkCommitInfo));
|
||||
}
|
||||
|
||||
@Test(expected = StarRocksConnectorException.class)
|
||||
public void testAppendPartition() {
|
||||
String stagingDir = "hdfs://127.0.0.1:10000/tmp/starrocks/queryid";
|
||||
THiveFileInfo fileInfo = new THiveFileInfo();
|
||||
fileInfo.setFile_name("myfile.parquet");
|
||||
fileInfo.setPartition_path("hdfs://127.0.0.1:10000/tmp/starrocks/queryid/col1=2");
|
||||
fileInfo.setRecord_count(10);
|
||||
fileInfo.setFile_size_in_bytes(100);
|
||||
TSinkCommitInfo tSinkCommitInfo = new TSinkCommitInfo();
|
||||
tSinkCommitInfo.setStaging_dir(stagingDir);
|
||||
tSinkCommitInfo.setIs_overwrite(false);
|
||||
tSinkCommitInfo.setHive_file_info(fileInfo);
|
||||
|
||||
new MockUp<RemoteFileOperations>() {
|
||||
@Mock
|
||||
public void asyncRenameFiles(
|
||||
List<CompletableFuture<?>> renameFileFutures,
|
||||
AtomicBoolean cancelled,
|
||||
Path writePath,
|
||||
Path targetPath,
|
||||
List<String> fileNames) {
|
||||
|
||||
}
|
||||
};
|
||||
|
||||
hiveMetadata.finishSink("hive_db", "hive_table", Lists.newArrayList(tSinkCommitInfo));
|
||||
|
||||
new MockUp<HiveMetastoreOperations>() {
|
||||
@Mock
|
||||
public void updatePartitionStatistics(String dbName, String tableName, String partitionName,
|
||||
Function<HivePartitionStats, HivePartitionStats> update) {
|
||||
throw new StarRocksConnectorException("ERROR");
|
||||
}
|
||||
};
|
||||
hiveMetadata.finishSink("hive_db", "hive_table", Lists.newArrayList(tSinkCommitInfo));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testOverwritePartition() throws Exception {
|
||||
String stagingDir = "hdfs://127.0.0.1:10000/tmp/starrocks/queryid";
|
||||
THiveFileInfo fileInfo = new THiveFileInfo();
|
||||
fileInfo.setFile_name("myfile.parquet");
|
||||
fileInfo.setPartition_path("hdfs://127.0.0.1:10000/tmp/starrocks/queryid/col1=2");
|
||||
fileInfo.setRecord_count(10);
|
||||
fileInfo.setFile_size_in_bytes(100);
|
||||
TSinkCommitInfo tSinkCommitInfo = new TSinkCommitInfo();
|
||||
tSinkCommitInfo.setStaging_dir(stagingDir);
|
||||
tSinkCommitInfo.setIs_overwrite(true);
|
||||
tSinkCommitInfo.setHive_file_info(fileInfo);
|
||||
|
||||
new MockUp<RemoteFileOperations>() {
|
||||
@Mock
|
||||
public void renameDirectory(Path source, Path target, Runnable runWhenPathNotExist) {
|
||||
}
|
||||
};
|
||||
|
||||
AnalyzeTestUtil.init();
|
||||
hiveMetadata.finishSink("hive_db", "hive_table", Lists.newArrayList(tSinkCommitInfo));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAppendTable() throws Exception {
|
||||
String stagingDir = "hdfs://127.0.0.1:10000/tmp/starrocks/queryid";
|
||||
THiveFileInfo fileInfo = new THiveFileInfo();
|
||||
fileInfo.setFile_name("myfile.parquet");
|
||||
fileInfo.setPartition_path("hdfs://127.0.0.1:10000/tmp/starrocks/queryid/");
|
||||
fileInfo.setRecord_count(10);
|
||||
fileInfo.setFile_size_in_bytes(100);
|
||||
TSinkCommitInfo tSinkCommitInfo = new TSinkCommitInfo();
|
||||
tSinkCommitInfo.setStaging_dir(stagingDir);
|
||||
tSinkCommitInfo.setIs_overwrite(false);
|
||||
tSinkCommitInfo.setHive_file_info(fileInfo);
|
||||
|
||||
new MockUp<RemoteFileOperations>() {
|
||||
@Mock
|
||||
public void asyncRenameFiles(
|
||||
List<CompletableFuture<?>> renameFileFutures,
|
||||
AtomicBoolean cancelled,
|
||||
Path writePath,
|
||||
Path targetPath,
|
||||
List<String> fileNames) {
|
||||
|
||||
}
|
||||
};
|
||||
|
||||
AnalyzeTestUtil.init();
|
||||
hiveMetadata.finishSink("hive_db", "unpartitioned_table", Lists.newArrayList(tSinkCommitInfo));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testOverwriteTable() throws Exception {
|
||||
String stagingDir = "hdfs://127.0.0.1:10000/tmp/starrocks/queryid";
|
||||
THiveFileInfo fileInfo = new THiveFileInfo();
|
||||
fileInfo.setFile_name("myfile.parquet");
|
||||
fileInfo.setPartition_path("hdfs://127.0.0.1:10000/tmp/starrocks/queryid/");
|
||||
fileInfo.setRecord_count(10);
|
||||
fileInfo.setFile_size_in_bytes(100);
|
||||
TSinkCommitInfo tSinkCommitInfo = new TSinkCommitInfo();
|
||||
tSinkCommitInfo.setStaging_dir(stagingDir);
|
||||
tSinkCommitInfo.setIs_overwrite(true);
|
||||
tSinkCommitInfo.setHive_file_info(fileInfo);
|
||||
|
||||
new MockUp<RemoteFileOperations>() {
|
||||
@Mock
|
||||
public void renameDirectory(Path source, Path target, Runnable runWhenPathNotExist) {
|
||||
}
|
||||
};
|
||||
|
||||
AnalyzeTestUtil.init();
|
||||
hiveMetadata.finishSink("hive_db", "unpartitioned_table", Lists.newArrayList(tSinkCommitInfo));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void tesRecursiveDeleteFiles(@Mocked HiveMetastoreOperations hmsOps,
|
||||
@Mocked RemoteFileOperations fileOps,
|
||||
@Mocked HiveTable hiveTable) throws Exception {
|
||||
HiveCommitter hiveCommitter = new HiveCommitter(hmsOps, fileOps, Executors.newSingleThreadExecutor(),
|
||||
hiveTable, new Path("hdfs://hadoop01:9000/hive"));
|
||||
HiveCommitter.DeleteRecursivelyResult result = hiveCommitter.recursiveDeleteFiles(new Path("hdfs://aaa"), false);
|
||||
Assert.assertTrue(result.dirNotExists());
|
||||
Assert.assertTrue(result.getNotDeletedEligibleItems().isEmpty());
|
||||
|
||||
new Expectations(fileOps) {
|
||||
{
|
||||
fileOps.pathExists((Path) any);
|
||||
result = new StarRocksConnectorException("ERROR");
|
||||
minTimes = 1;
|
||||
}
|
||||
};
|
||||
|
||||
result = hiveCommitter.recursiveDeleteFiles(new Path("hdfs://aaa"), false);
|
||||
Assert.assertFalse(result.dirNotExists());
|
||||
Assert.assertEquals(Lists.newArrayList("hdfs://aaa/*"), result.getNotDeletedEligibleItems());
|
||||
|
||||
AnalyzeTestUtil.init();
|
||||
|
||||
new Expectations(fileOps) {
|
||||
{
|
||||
fileOps.pathExists((Path) any);
|
||||
result = true;
|
||||
minTimes = 1;
|
||||
|
||||
fileOps.listStatus((Path) any);
|
||||
result = new StarRocksConnectorException("ERROR");
|
||||
minTimes = 1;
|
||||
}
|
||||
};
|
||||
result = hiveCommitter.recursiveDeleteFiles(new Path("hdfs://aaa"), false);
|
||||
Assert.assertFalse(result.dirNotExists());
|
||||
Assert.assertEquals(Lists.newArrayList("hdfs://aaa/*"), result.getNotDeletedEligibleItems());
|
||||
|
||||
Path path = new Path("hdfs://hadoop01:9000/user/hive/warehouse/t1/my.parquet");
|
||||
FileStatus fileStatus = new FileStatus(10, false, 0, 0, 0, 0, null, null, null, path);
|
||||
FileStatus[] mockedStatus = new FileStatus[1];
|
||||
mockedStatus[0] = fileStatus;
|
||||
|
||||
new Expectations(fileOps) {
|
||||
{
|
||||
fileOps.listStatus((Path) any);
|
||||
result = mockedStatus;
|
||||
minTimes = 1;
|
||||
|
||||
fileOps.deleteIfExists((Path) any, false);
|
||||
result = true;
|
||||
minTimes = 1;
|
||||
}
|
||||
};
|
||||
|
||||
new MockUp<HiveWriteUtils>() {
|
||||
@Mock
|
||||
public boolean fileCreatedByQuery(String fileName, String queryId) {
|
||||
return true;
|
||||
}
|
||||
};
|
||||
|
||||
result = hiveCommitter.recursiveDeleteFiles(new Path("hdfs://aaa"), false);
|
||||
Assert.assertFalse(result.dirNotExists());
|
||||
Assert.assertTrue(result.getNotDeletedEligibleItems().isEmpty());
|
||||
|
||||
result = hiveCommitter.recursiveDeleteFiles(new Path("hdfs://aaa"), true);
|
||||
Assert.assertTrue(result.dirNotExists());
|
||||
Assert.assertTrue(result.getNotDeletedEligibleItems().isEmpty());
|
||||
|
||||
new Expectations(fileOps) {
|
||||
{
|
||||
fileOps.deleteIfExists((Path) any, false);
|
||||
result = false;
|
||||
minTimes = 1;
|
||||
}
|
||||
};
|
||||
result = hiveCommitter.recursiveDeleteFiles(new Path("hdfs://aaa"), true);
|
||||
Assert.assertFalse(result.dirNotExists());
|
||||
Assert.assertEquals(fileStatus.getPath().toString(), result.getNotDeletedEligibleItems().get(0));
|
||||
|
||||
new MockUp<HiveWriteUtils>() {
|
||||
@Mock
|
||||
public boolean fileCreatedByQuery(String fileName, String queryId) {
|
||||
return false;
|
||||
}
|
||||
};
|
||||
result = hiveCommitter.recursiveDeleteFiles(new Path("hdfs://aaa"), false);
|
||||
Assert.assertFalse(result.dirNotExists());
|
||||
Assert.assertTrue(result.getNotDeletedEligibleItems().isEmpty());
|
||||
|
||||
FileStatus fileStatus1 = new FileStatus(10, true, 0, 0, 0, 0, null, null, null, path);
|
||||
FileStatus[] mockedStatus1 = new FileStatus[1];
|
||||
mockedStatus1[0] = fileStatus1;
|
||||
|
||||
new Expectations(fileOps) {
|
||||
{
|
||||
fileOps.listStatus(new Path("hdfs://aaa"));
|
||||
result = mockedStatus1;
|
||||
minTimes = 1;
|
||||
|
||||
fileOps.listStatus((Path) any);
|
||||
result = mockedStatus;
|
||||
minTimes = 1;
|
||||
}
|
||||
};
|
||||
result = hiveCommitter.recursiveDeleteFiles(new Path("hdfs://aaa"), false);
|
||||
Assert.assertFalse(result.dirNotExists());
|
||||
Assert.assertTrue(result.getNotDeletedEligibleItems().isEmpty());
|
||||
}
|
||||
|
||||
@Test(expected = StarRocksConnectorException.class)
|
||||
public void testHiveCommitterPrepare(@Mocked HiveMetastoreOperations hmsOps,
|
||||
@Mocked RemoteFileOperations fileOps,
|
||||
@Mocked HiveTable hiveTable) {
|
||||
HiveCommitter hiveCommitter = new HiveCommitter(hmsOps, fileOps, Executors.newSingleThreadExecutor(),
|
||||
hiveTable, new Path("hdfs://hadoop01:9000/hive"));
|
||||
new Expectations() {
|
||||
{
|
||||
hiveTable.isUnPartitioned();
|
||||
result = true;
|
||||
minTimes = 1;
|
||||
}
|
||||
};
|
||||
|
||||
PartitionUpdate pu1 = new PartitionUpdate("", null, null, null, 1, 1);
|
||||
PartitionUpdate pu2 = new PartitionUpdate("", null, null, null, 1, 1);
|
||||
hiveCommitter.prepare(Lists.newArrayList(pu1, pu2));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testIsSamePartition() {
|
||||
Assert.assertFalse(HiveCommitter.checkIsSamePartition(null, null));
|
||||
Assert.assertFalse(HiveCommitter.checkIsSamePartition(new Partition(new HashMap<>(), null, null, null, false), null));
|
||||
|
||||
Map<String, String> map = new HashMap<>();
|
||||
map.put(STARROCKS_QUERY_ID, "abcd");
|
||||
Partition remotePartition = new Partition(map, null, null, null, false);
|
||||
HivePartition hivePartition = new HivePartition(null, null, null, null, null, null, map);
|
||||
Assert.assertTrue(HiveCommitter.checkIsSamePartition(remotePartition, hivePartition));
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -12,15 +12,16 @@
|
|||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
|
||||
package com.starrocks.connector.hive;
|
||||
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.starrocks.catalog.Column;
|
||||
import com.starrocks.catalog.Database;
|
||||
import com.starrocks.catalog.HiveTable;
|
||||
import com.starrocks.catalog.ScalarType;
|
||||
import com.starrocks.catalog.Type;
|
||||
import com.starrocks.connector.PartitionUtil;
|
||||
import com.starrocks.connector.exception.StarRocksConnectorException;
|
||||
import org.apache.hadoop.hive.conf.HiveConf;
|
||||
|
|
@ -36,6 +37,8 @@ import org.apache.hadoop.hive.metastore.api.Table;
|
|||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
|
|
@ -111,6 +114,56 @@ public class HiveMetastoreTest {
|
|||
Assert.assertEquals("100", partition.getParameters().get(TOTAL_SIZE));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testPartitionExists() {
|
||||
HiveMetaClient client = new MockedHiveMetaClient();
|
||||
HiveMetastore metastore = new HiveMetastore(client, "hive_catalog");
|
||||
Assert.assertTrue(metastore.partitionExists("db1", "tbl1", new ArrayList<>()));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAddPartitions() {
|
||||
HiveMetaClient client = new MockedHiveMetaClient();
|
||||
HiveMetastore metastore = new HiveMetastore(client, "hive_catalog");
|
||||
HivePartition hivePartition = HivePartition.builder()
|
||||
.setColumns(Lists.newArrayList(new Column("c1", Type.INT)))
|
||||
.setStorageFormat(HiveStorageFormat.PARQUET)
|
||||
.setDatabaseName("db")
|
||||
.setTableName("table")
|
||||
.setLocation("location")
|
||||
.setValues(Lists.newArrayList("p1=1"))
|
||||
.setParameters(new HashMap<>()).build();
|
||||
|
||||
HivePartitionStats hivePartitionStats = HivePartitionStats.empty();
|
||||
HivePartitionWithStats hivePartitionWithStats = new HivePartitionWithStats("p1=1", hivePartition, hivePartitionStats);
|
||||
metastore.addPartitions("db", "table", Lists.newArrayList(hivePartitionWithStats));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDropPartition() {
|
||||
HiveMetaClient client = new MockedHiveMetaClient();
|
||||
HiveMetastore metastore = new HiveMetastore(client, "hive_catalog");
|
||||
metastore.dropPartition("db", "table", Lists.newArrayList("k1=1"), false);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAlterPartition() {
|
||||
HiveMetaClient client = new MockedHiveMetaClient();
|
||||
HiveMetastore metastore = new HiveMetastore(client, "hive_catalog");
|
||||
HivePartition hivePartition = HivePartition.builder()
|
||||
.setColumns(Lists.newArrayList(new Column("c1", Type.INT)))
|
||||
.setStorageFormat(HiveStorageFormat.PARQUET)
|
||||
.setDatabaseName("db")
|
||||
.setTableName("table")
|
||||
.setLocation("location")
|
||||
.setValues(Lists.newArrayList("p1=1"))
|
||||
.setParameters(new HashMap<>()).build();
|
||||
|
||||
HivePartitionStats hivePartitionStats = HivePartitionStats.empty();
|
||||
HivePartitionWithStats hivePartitionWithStats = new HivePartitionWithStats("p1=1", hivePartition, hivePartitionStats);
|
||||
metastore.alterPartition(hivePartitionWithStats);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetPartitionByNames() {
|
||||
HiveMetaClient client = new MockedHiveMetaClient();
|
||||
|
|
@ -171,6 +224,22 @@ public class HiveMetastoreTest {
|
|||
Assert.assertEquals(5, columnStatistics2.getNdv());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUpdateTableStatistics() {
|
||||
HiveMetaClient client = new MockedHiveMetaClient();
|
||||
HiveMetastore metastore = new HiveMetastore(client, "hive_catalog");
|
||||
HivePartitionStats partitionStats = HivePartitionStats.empty();
|
||||
metastore.updateTableStatistics("db", "table", ignore -> partitionStats);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUpdatePartitionStatistics() {
|
||||
HiveMetaClient client = new MockedHiveMetaClient();
|
||||
HiveMetastore metastore = new HiveMetastore(client, "hive_catalog");
|
||||
HivePartitionStats partitionStats = HivePartitionStats.empty();
|
||||
metastore.updatePartitionStatistics("db", "table", "p1=1", ignore -> partitionStats);
|
||||
}
|
||||
|
||||
public static class MockedHiveMetaClient extends HiveMetaClient {
|
||||
|
||||
public MockedHiveMetaClient() {
|
||||
|
|
@ -230,17 +299,35 @@ public class HiveMetastoreTest {
|
|||
Table msTable1 = new Table();
|
||||
msTable1.setDbName(dbName);
|
||||
msTable1.setTableName(tblName);
|
||||
msTable1.setPartitionKeys(partKeys);
|
||||
|
||||
msTable1.setSd(sd);
|
||||
msTable1.setTableType("MANAGED_TABLE");
|
||||
msTable1.setParameters(ImmutableMap.of(ROW_COUNT, "50", TOTAL_SIZE, "100"));
|
||||
|
||||
if (!tblName.equals("unpartitioned_table")) {
|
||||
msTable1.setPartitionKeys(partKeys);
|
||||
} else {
|
||||
msTable1.setPartitionKeys(new ArrayList<>());
|
||||
}
|
||||
|
||||
return msTable1;
|
||||
}
|
||||
|
||||
public void alterTable(String dbName, String tableName, Table newTable) {
|
||||
|
||||
}
|
||||
|
||||
public List<String> getPartitionKeys(String dbName, String tableName) {
|
||||
return Lists.newArrayList("col1");
|
||||
}
|
||||
|
||||
public List<String> getPartitionKeysByValue(String dbName, String tableName, List<String> partitionValues) {
|
||||
return Lists.newArrayList("col1");
|
||||
}
|
||||
|
||||
public void addPartitions(String dbName, String tableName, List<Partition> partitions) {
|
||||
}
|
||||
|
||||
public Partition getPartition(HiveTableName name, List<String> partitionValues) {
|
||||
StorageDescriptor sd = new StorageDescriptor();
|
||||
String hdfsPath = "hdfs://127.0.0.1:10000/hive";
|
||||
|
|
@ -269,6 +356,12 @@ public class HiveMetastoreTest {
|
|||
return partition;
|
||||
}
|
||||
|
||||
public void dropPartition(String dbName, String tableName, List<String> partValues, boolean deleteData) {
|
||||
}
|
||||
|
||||
public void alterPartition(String dbName, String tableName, Partition newPartition) {
|
||||
}
|
||||
|
||||
public List<Partition> getPartitionsByNames(String dbName, String tblName, List<String> partitionNames) {
|
||||
String hdfsPath = "hdfs://127.0.0.1:10000/hive.db/hive_tbl/";
|
||||
List<Partition> res = Lists.newArrayList();
|
||||
|
|
|
|||
|
|
@ -0,0 +1,57 @@
|
|||
// Copyright 2021-present StarRocks, Inc. All rights reserved.
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// https://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package com.starrocks.connector.hive;
|
||||
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
public class HivePartitionStatsTest {
|
||||
@Test
|
||||
public void testFromCommonStats() {
|
||||
long rowNums = 5;
|
||||
long fileSize = 100;
|
||||
HivePartitionStats hivePartitionStats = HivePartitionStats.fromCommonStats(rowNums, fileSize);
|
||||
Assert.assertEquals(5, hivePartitionStats.getCommonStats().getRowNums());
|
||||
Assert.assertEquals(100, hivePartitionStats.getCommonStats().getTotalFileBytes());
|
||||
Assert.assertTrue(hivePartitionStats.getColumnStats().isEmpty());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMerge() {
|
||||
HivePartitionStats current = HivePartitionStats.empty();
|
||||
HivePartitionStats update = HivePartitionStats.empty();
|
||||
Assert.assertEquals(current, HivePartitionStats.merge(current, update));
|
||||
|
||||
current = HivePartitionStats.fromCommonStats(5, 100);
|
||||
update = HivePartitionStats.empty();
|
||||
Assert.assertEquals(current, HivePartitionStats.merge(current, update));
|
||||
|
||||
current = HivePartitionStats.fromCommonStats(0, 0);
|
||||
update = HivePartitionStats.fromCommonStats(5, 100);
|
||||
Assert.assertEquals(update, HivePartitionStats.merge(current, update));
|
||||
|
||||
current = HivePartitionStats.fromCommonStats(5, 100);
|
||||
Assert.assertEquals(10, HivePartitionStats.merge(current, update).getCommonStats().getRowNums());
|
||||
Assert.assertEquals(200, HivePartitionStats.merge(current, update).getCommonStats().getTotalFileBytes());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testReduce() {
|
||||
Assert.assertEquals(10, HivePartitionStats.reduce(5, 5, HivePartitionStats.ReduceOperator.ADD));
|
||||
Assert.assertEquals(0, HivePartitionStats.reduce(5, 5, HivePartitionStats.ReduceOperator.SUBTRACT));
|
||||
Assert.assertEquals(5, HivePartitionStats.reduce(5, 6, HivePartitionStats.ReduceOperator.MIN));
|
||||
Assert.assertEquals(6, HivePartitionStats.reduce(5, 6, HivePartitionStats.ReduceOperator.MAX));
|
||||
}
|
||||
}
|
||||
|
|
@ -88,7 +88,8 @@ public class HiveStatisticsProviderTest {
|
|||
hiveRemoteFileIO.setFileSystem(fs);
|
||||
cachingRemoteFileIO = CachingRemoteFileIO.createCatalogLevelInstance(
|
||||
hiveRemoteFileIO, executorForRemoteFileRefresh, 100, 10, 10);
|
||||
fileOps = new RemoteFileOperations(cachingRemoteFileIO, executorForPullFiles, false, true);
|
||||
fileOps = new RemoteFileOperations(cachingRemoteFileIO, executorForPullFiles, executorForPullFiles,
|
||||
false, true, new Configuration());
|
||||
statisticsProvider = new HiveStatisticsProvider(hmsOps, fileOps);
|
||||
|
||||
UtFrameUtils.createMinStarRocksCluster();
|
||||
|
|
|
|||
|
|
@ -51,7 +51,7 @@ public class HiveWriteUtilsTest {
|
|||
public void testPathExists() {
|
||||
Path path = new Path("hdfs://127.0.0.1:9000/user/hive/warehouse/db");
|
||||
ExceptionChecker.expectThrowsWithMsg(StarRocksConnectorException.class,
|
||||
"Failed checking path",
|
||||
"Failed to check path",
|
||||
() -> HiveWriteUtils.pathExists(path, new Configuration()));
|
||||
|
||||
new MockUp<FileSystem>() {
|
||||
|
|
|
|||
|
|
@ -65,6 +65,7 @@ import java.util.stream.Collectors;
|
|||
|
||||
import static com.starrocks.common.util.DateUtils.DATE_FORMATTER_UNIX;
|
||||
import static com.starrocks.connector.hive.CachingHiveMetastore.createCatalogLevelInstance;
|
||||
import static com.starrocks.connector.hive.HiveClassNames.MAPRED_PARQUET_INPUT_FORMAT_CLASS;
|
||||
import static com.starrocks.sql.optimizer.Utils.getLongFromDateTime;
|
||||
import static java.lang.Double.NEGATIVE_INFINITY;
|
||||
import static java.lang.Double.POSITIVE_INFINITY;
|
||||
|
|
@ -342,8 +343,8 @@ public class MockedHiveMetadata implements ConnectorMetadata {
|
|||
cols.add(new FieldSchema("r_name", "string", null));
|
||||
cols.add(new FieldSchema("r_comment", "string", null));
|
||||
StorageDescriptor sd =
|
||||
new StorageDescriptor(cols, "", "", "", false, -1, null, Lists.newArrayList(), Lists.newArrayList(),
|
||||
Maps.newHashMap());
|
||||
new StorageDescriptor(cols, "", MAPRED_PARQUET_INPUT_FORMAT_CLASS, "", false,
|
||||
-1, null, Lists.newArrayList(), Lists.newArrayList(), Maps.newHashMap());
|
||||
|
||||
CaseInsensitiveMap<String, ColumnStatistic> regionStats = new CaseInsensitiveMap<>();
|
||||
regionStats.put("r_regionkey", new ColumnStatistic(0, 4, 0, 4, 5));
|
||||
|
|
@ -363,8 +364,8 @@ public class MockedHiveMetadata implements ConnectorMetadata {
|
|||
cols.add(new FieldSchema("n_name", "string", null));
|
||||
cols.add(new FieldSchema("n_regionkey", "int", null));
|
||||
cols.add(new FieldSchema("n_comment", "string", null));
|
||||
sd = new StorageDescriptor(cols, "", "", "", false, -1, null, Lists.newArrayList(), Lists.newArrayList(),
|
||||
Maps.newHashMap());
|
||||
sd = new StorageDescriptor(cols, "", MAPRED_PARQUET_INPUT_FORMAT_CLASS, "", false,
|
||||
-1, null, Lists.newArrayList(), Lists.newArrayList(), Maps.newHashMap());
|
||||
|
||||
Map<String, ColumnStatistic> nationStats = new CaseInsensitiveMap<>();
|
||||
nationStats.put("n_nationkey", new ColumnStatistic(0, 24, 0, 4, 25));
|
||||
|
|
@ -387,8 +388,8 @@ public class MockedHiveMetadata implements ConnectorMetadata {
|
|||
cols.add(new FieldSchema("s_phone", "string", null));
|
||||
cols.add(new FieldSchema("s_acctbal", "decimal(15,2)", null));
|
||||
cols.add(new FieldSchema("s_comment", "string", null));
|
||||
sd = new StorageDescriptor(cols, "", "", "", false, -1, null, Lists.newArrayList(), Lists.newArrayList(),
|
||||
Maps.newHashMap());
|
||||
sd = new StorageDescriptor(cols, "", MAPRED_PARQUET_INPUT_FORMAT_CLASS, "", false,
|
||||
-1, null, Lists.newArrayList(), Lists.newArrayList(), Maps.newHashMap());
|
||||
|
||||
CaseInsensitiveMap<String, ColumnStatistic> supplierStats = new CaseInsensitiveMap<>();
|
||||
supplierStats.put("s_suppkey", new ColumnStatistic(1, 1000000.0, 0, 4, 1000000));
|
||||
|
|
@ -416,8 +417,8 @@ public class MockedHiveMetadata implements ConnectorMetadata {
|
|||
cols.add(new FieldSchema("p_container", "string", null));
|
||||
cols.add(new FieldSchema("p_retailprice", "decimal(15,2)", null));
|
||||
cols.add(new FieldSchema("p_comment", "string", null));
|
||||
sd = new StorageDescriptor(cols, "", "", "", false, -1, null, Lists.newArrayList(), Lists.newArrayList(),
|
||||
Maps.newHashMap());
|
||||
sd = new StorageDescriptor(cols, "", MAPRED_PARQUET_INPUT_FORMAT_CLASS, "", false,
|
||||
-1, null, Lists.newArrayList(), Lists.newArrayList(), Maps.newHashMap());
|
||||
|
||||
CaseInsensitiveMap<String, ColumnStatistic> partStats = new CaseInsensitiveMap<>();
|
||||
partStats.put("p_partkey", new ColumnStatistic(1, 20000000, 0, 8, 20000000));
|
||||
|
|
@ -443,8 +444,8 @@ public class MockedHiveMetadata implements ConnectorMetadata {
|
|||
cols.add(new FieldSchema("ps_availqty", "int", null));
|
||||
cols.add(new FieldSchema("ps_supplycost", "decimal(15,2)", null));
|
||||
cols.add(new FieldSchema("ps_comment", "string", null));
|
||||
sd = new StorageDescriptor(cols, "", "", "", false, -1, null, Lists.newArrayList(), Lists.newArrayList(),
|
||||
Maps.newHashMap());
|
||||
sd = new StorageDescriptor(cols, "", MAPRED_PARQUET_INPUT_FORMAT_CLASS, "", false,
|
||||
-1, null, Lists.newArrayList(), Lists.newArrayList(), Maps.newHashMap());
|
||||
|
||||
CaseInsensitiveMap<String, ColumnStatistic> partSuppStats = new CaseInsensitiveMap<>();
|
||||
partSuppStats.put("ps_partkey", new ColumnStatistic(1, 20000000, 0, 8, 20000000));
|
||||
|
|
@ -469,8 +470,8 @@ public class MockedHiveMetadata implements ConnectorMetadata {
|
|||
cols.add(new FieldSchema("c_acctbal", "decimal(15,2)", null));
|
||||
cols.add(new FieldSchema("c_mktsegment", "string", null));
|
||||
cols.add(new FieldSchema("c_comment", "string", null));
|
||||
sd = new StorageDescriptor(cols, "", "", "", false, -1, null, Lists.newArrayList(), Lists.newArrayList(),
|
||||
Maps.newHashMap());
|
||||
sd = new StorageDescriptor(cols, "", MAPRED_PARQUET_INPUT_FORMAT_CLASS,
|
||||
"", false, -1, null, Lists.newArrayList(), Lists.newArrayList(), Maps.newHashMap());
|
||||
|
||||
Map<String, ColumnStatistic> customerStats = new CaseInsensitiveMap<>();
|
||||
customerStats.put("c_custkey", new ColumnStatistic(1, 15000000, 0, 8, 15000000));
|
||||
|
|
@ -499,7 +500,8 @@ public class MockedHiveMetadata implements ConnectorMetadata {
|
|||
cols.add(new FieldSchema("o_clerk", "string", null));
|
||||
cols.add(new FieldSchema("o_shippriority", "int", null));
|
||||
cols.add(new FieldSchema("o_comment", "string", null));
|
||||
sd = new StorageDescriptor(cols, "", "", "", false, -1, null, Lists.newArrayList(), Lists.newArrayList(),
|
||||
sd = new StorageDescriptor(cols, "", MAPRED_PARQUET_INPUT_FORMAT_CLASS,
|
||||
"", false, -1, null, Lists.newArrayList(), Lists.newArrayList(),
|
||||
Maps.newHashMap());
|
||||
|
||||
CaseInsensitiveMap<String, ColumnStatistic> ordersStats = new CaseInsensitiveMap<>();
|
||||
|
|
@ -540,7 +542,8 @@ public class MockedHiveMetadata implements ConnectorMetadata {
|
|||
cols.add(new FieldSchema("l_shipinstruct", "string", null));
|
||||
cols.add(new FieldSchema("l_shipmode", "string", null));
|
||||
cols.add(new FieldSchema("l_comment", "string", null));
|
||||
sd = new StorageDescriptor(cols, "", "", "", false, -1, null, Lists.newArrayList(), Lists.newArrayList(),
|
||||
sd = new StorageDescriptor(cols, "", MAPRED_PARQUET_INPUT_FORMAT_CLASS,
|
||||
"", false, -1, null, Lists.newArrayList(), Lists.newArrayList(),
|
||||
Maps.newHashMap());
|
||||
|
||||
Map<String, ColumnStatistic> lineitemStats = new CaseInsensitiveMap<>();
|
||||
|
|
@ -605,7 +608,8 @@ public class MockedHiveMetadata implements ConnectorMetadata {
|
|||
cols.add(new FieldSchema("o_comment", "string", null));
|
||||
|
||||
StorageDescriptor sd =
|
||||
new StorageDescriptor(cols, "", "", "", false, -1, null, Lists.newArrayList(), Lists.newArrayList(),
|
||||
new StorageDescriptor(cols, "", MAPRED_PARQUET_INPUT_FORMAT_CLASS,
|
||||
"", false, -1, null, Lists.newArrayList(), Lists.newArrayList(),
|
||||
Maps.newHashMap());
|
||||
Table orders = new Table("orders", "partitioned_db", null, 0, 0, 0, sd,
|
||||
ImmutableList.of(new FieldSchema("o_orderdate", "Date", null)), Maps.newHashMap(),
|
||||
|
|
@ -673,7 +677,8 @@ public class MockedHiveMetadata implements ConnectorMetadata {
|
|||
cols.add(new FieldSchema("l_shipmode", "string", null));
|
||||
cols.add(new FieldSchema("l_comment", "string", null));
|
||||
StorageDescriptor sd =
|
||||
new StorageDescriptor(cols, "", "", "", false, -1, null, Lists.newArrayList(), Lists.newArrayList(),
|
||||
new StorageDescriptor(cols, "", MAPRED_PARQUET_INPUT_FORMAT_CLASS,
|
||||
"", false, -1, null, Lists.newArrayList(), Lists.newArrayList(),
|
||||
Maps.newHashMap());
|
||||
Table lineItemPar = new Table("lineitem_par", "partitioned_db", null, 0, 0, 0, sd,
|
||||
ImmutableList.of(new FieldSchema("l_shipdate", "Date", null)), Maps.newHashMap(),
|
||||
|
|
@ -743,7 +748,8 @@ public class MockedHiveMetadata implements ConnectorMetadata {
|
|||
cols.add(new FieldSchema("l_shipmode", "string", null));
|
||||
cols.add(new FieldSchema("l_comment", "string", null));
|
||||
StorageDescriptor sd =
|
||||
new StorageDescriptor(cols, "", "", "", false, -1, null, Lists.newArrayList(), Lists.newArrayList(),
|
||||
new StorageDescriptor(cols, "", MAPRED_PARQUET_INPUT_FORMAT_CLASS,
|
||||
"", false, -1, null, Lists.newArrayList(), Lists.newArrayList(),
|
||||
Maps.newHashMap());
|
||||
Table lineItemPar = new Table("lineitem_mul_par", "partitioned_db", null, 0, 0, 0, sd,
|
||||
ImmutableList.of(new FieldSchema("l_shipdate", "Date", null),
|
||||
|
|
@ -825,8 +831,8 @@ public class MockedHiveMetadata implements ConnectorMetadata {
|
|||
cols.add(new FieldSchema("l_shipmode", "string", null));
|
||||
cols.add(new FieldSchema("l_comment", "string", null));
|
||||
StorageDescriptor sd =
|
||||
new StorageDescriptor(cols, "", "", "", false, -1, null, Lists.newArrayList(), Lists.newArrayList(),
|
||||
Maps.newHashMap());
|
||||
new StorageDescriptor(cols, "", MAPRED_PARQUET_INPUT_FORMAT_CLASS, "", false, -1,
|
||||
null, Lists.newArrayList(), Lists.newArrayList(), Maps.newHashMap());
|
||||
Table lineItemPar = new Table("lineitem_mul_par2", "partitioned_db", null, 0, 0, 0, sd,
|
||||
ImmutableList.of(new FieldSchema("l_shipdate", "Date", null),
|
||||
new FieldSchema("l_returnflag", "string", null)),
|
||||
|
|
@ -904,7 +910,8 @@ public class MockedHiveMetadata implements ConnectorMetadata {
|
|||
cols.add(new FieldSchema("c2", "string", null));
|
||||
cols.add(new FieldSchema("c3", "string", null));
|
||||
StorageDescriptor sd =
|
||||
new StorageDescriptor(cols, "", "", "", false, -1, null, Lists.newArrayList(), Lists.newArrayList(),
|
||||
new StorageDescriptor(cols, "", MAPRED_PARQUET_INPUT_FORMAT_CLASS,
|
||||
"", false, -1, null, Lists.newArrayList(), Lists.newArrayList(),
|
||||
Maps.newHashMap());
|
||||
Table mockTable = new Table(tableName, dbName, null, 0, 0, 0, sd,
|
||||
ImmutableList.of(new FieldSchema("par_col", "int", null)), Maps.newHashMap(), null,
|
||||
|
|
@ -964,8 +971,8 @@ public class MockedHiveMetadata implements ConnectorMetadata {
|
|||
cols.add(new FieldSchema("c3", "string", null));
|
||||
|
||||
StorageDescriptor sd =
|
||||
new StorageDescriptor(cols, "", "", "", false, -1, null, Lists.newArrayList(), Lists.newArrayList(),
|
||||
Maps.newHashMap());
|
||||
new StorageDescriptor(cols, "", MAPRED_PARQUET_INPUT_FORMAT_CLASS, "", false,
|
||||
-1, null, Lists.newArrayList(), Lists.newArrayList(), Maps.newHashMap());
|
||||
Table t1 = new Table("t1_par", "partitioned_db", null, 0, 0, 0, sd,
|
||||
ImmutableList.of(new FieldSchema("par_col", "int", null),
|
||||
new FieldSchema("par_date", "date", null)), Maps.newHashMap(), null, null,
|
||||
|
|
@ -1026,7 +1033,8 @@ public class MockedHiveMetadata implements ConnectorMetadata {
|
|||
cols.add(new FieldSchema("c1", "int", null));
|
||||
cols.add(new FieldSchema("c2", "string", null));
|
||||
cols.add(new FieldSchema("c3", "string", null));
|
||||
StorageDescriptor sd = new StorageDescriptor(cols, "", "", "", false, -1, null, Lists.newArrayList(),
|
||||
StorageDescriptor sd = new StorageDescriptor(cols, "", MAPRED_PARQUET_INPUT_FORMAT_CLASS,
|
||||
"", false, -1, null, Lists.newArrayList(),
|
||||
Lists.newArrayList(), Maps.newHashMap());
|
||||
Table t2 = new Table("t2_par", "partitioned_db", null, 0, 0, 0, sd,
|
||||
ImmutableList.of(new FieldSchema("par_col", "int", null),
|
||||
|
|
@ -1088,7 +1096,8 @@ public class MockedHiveMetadata implements ConnectorMetadata {
|
|||
cols.add(new FieldSchema("c1", "int", null));
|
||||
cols.add(new FieldSchema("c2", "string", null));
|
||||
cols.add(new FieldSchema("c3", "string", null));
|
||||
StorageDescriptor sd = new StorageDescriptor(cols, "", "", "", false, -1, null, Lists.newArrayList(),
|
||||
StorageDescriptor sd = new StorageDescriptor(cols, "", MAPRED_PARQUET_INPUT_FORMAT_CLASS,
|
||||
"", false, -1, null, Lists.newArrayList(),
|
||||
Lists.newArrayList(), Maps.newHashMap());
|
||||
Table partTbl1 = new Table("part_tbl1", "partitioned_db", null, 0, 0, 0, sd,
|
||||
ImmutableList.of(new FieldSchema("par_date", "date", null)), Maps.newHashMap(),
|
||||
|
|
@ -1141,7 +1150,7 @@ public class MockedHiveMetadata implements ConnectorMetadata {
|
|||
cols.add(new FieldSchema("c1", "int", null));
|
||||
cols.add(new FieldSchema("c2", "string", null));
|
||||
cols.add(new FieldSchema("c3", "string", null));
|
||||
StorageDescriptor sd = new StorageDescriptor(cols, "", "", "", false,
|
||||
StorageDescriptor sd = new StorageDescriptor(cols, "", MAPRED_PARQUET_INPUT_FORMAT_CLASS, "", false,
|
||||
-1, null, Lists.newArrayList(),
|
||||
Lists.newArrayList(), Maps.newHashMap());
|
||||
Table partTbl2 = new Table("part_tbl2", "partitioned_db", null, 0, 0, 0, sd,
|
||||
|
|
@ -1204,11 +1213,10 @@ public class MockedHiveMetadata implements ConnectorMetadata {
|
|||
new HiveMetastoreOperations(cachingHiveMetastore, false, new Configuration(), MetastoreType.HMS,
|
||||
"hive_catalog");
|
||||
RemoteFileIO remoteFileIO = new HiveRemoteFileIO(new Configuration());
|
||||
CachingRemoteFileIO cacheIO =
|
||||
CachingRemoteFileIO.createCatalogLevelInstance(remoteFileIO, Executors.newSingleThreadExecutor(), 0, 0,
|
||||
0);
|
||||
RemoteFileOperations fileOps =
|
||||
new RemoteFileOperations(cacheIO, Executors.newSingleThreadExecutor(), false, false);
|
||||
CachingRemoteFileIO cacheIO = CachingRemoteFileIO.createCatalogLevelInstance(remoteFileIO,
|
||||
Executors.newSingleThreadExecutor(), 0, 0, 0);
|
||||
RemoteFileOperations fileOps = new RemoteFileOperations(cacheIO, Executors.newSingleThreadExecutor(),
|
||||
Executors.newSingleThreadExecutor(), false, false, new Configuration());
|
||||
|
||||
HiveStatisticsProvider hiveStatisticsProvider = new HiveStatisticsProvider(hmsOps, fileOps);
|
||||
try {
|
||||
|
|
|
|||
|
|
@ -114,7 +114,7 @@ public class MockedRemoteFileSystem extends FileSystem {
|
|||
|
||||
@Override
|
||||
public boolean rename(Path path, Path path1) throws IOException {
|
||||
throw new UnsupportedOperationException();
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
|||
|
|
@ -0,0 +1,84 @@
|
|||
// Copyright 2021-present StarRocks, Inc. All rights reserved.
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// https://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package com.starrocks.connector.hive;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
import com.starrocks.common.ExceptionChecker;
|
||||
import com.starrocks.thrift.THiveFileInfo;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
public class PartitionUpdateTest {
|
||||
|
||||
@Test
|
||||
public void testGetPartitionUpdate() {
|
||||
String stagingDir = "hdfs://hadoop01:9000/tmp/starrocks/queryid";
|
||||
String tableLocation = "hdfs://hadoop01:9000/user/hive/warehouse/test.db/t1";
|
||||
THiveFileInfo fileInfo = new THiveFileInfo();
|
||||
fileInfo.setFile_name("myfile.parquet");
|
||||
fileInfo.setPartition_path("hdfs://hadoop01:9000/tmp/starrocks/queryid/k2=2");
|
||||
fileInfo.setRecord_count(10);
|
||||
fileInfo.setFile_size_in_bytes(100);
|
||||
PartitionUpdate pu = PartitionUpdate.get(fileInfo, stagingDir, tableLocation);
|
||||
pu.setUpdateMode(PartitionUpdate.UpdateMode.NEW);
|
||||
Assert.assertEquals("k2=2", pu.getName());
|
||||
Assert.assertEquals(Lists.newArrayList("myfile.parquet"), pu.getFileNames());
|
||||
Assert.assertEquals(10, pu.getRowCount());
|
||||
Assert.assertEquals(100, pu.getTotalSizeInBytes());
|
||||
Assert.assertEquals(PartitionUpdate.UpdateMode.NEW, pu.getUpdateMode());
|
||||
Assert.assertEquals("hdfs://hadoop01:9000/tmp/starrocks/queryid/k2=2", pu.getWritePath().toString());
|
||||
Assert.assertEquals("hdfs://hadoop01:9000/user/hive/warehouse/test.db/t1/k2=2", pu.getTargetPath().toString());
|
||||
Assert.assertFalse(pu.isS3Url());
|
||||
|
||||
THiveFileInfo fileInfo1 = new THiveFileInfo();
|
||||
ExceptionChecker.expectThrowsWithMsg(
|
||||
IllegalStateException.class,
|
||||
"Missing partition path",
|
||||
() -> PartitionUpdate.get(fileInfo1, stagingDir, tableLocation));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMergePartitionUpdates() {
|
||||
PartitionUpdate first = new PartitionUpdate(
|
||||
"k2=1",
|
||||
new Path("s3://starrocks-dla/writePath/k2=1"),
|
||||
new Path("s3://starrocks-dla/writePath/k2=1"),
|
||||
Lists.newArrayList("file1", "file2"),
|
||||
1,
|
||||
10);
|
||||
|
||||
PartitionUpdate second = new PartitionUpdate(
|
||||
"k2=1",
|
||||
new Path("s3://starrocks-dla/writePath/k2=1"),
|
||||
new Path("s3://starrocks-dla/writePath/k2=1"),
|
||||
Lists.newArrayList("file3", "file4"),
|
||||
1,
|
||||
10);
|
||||
|
||||
List<PartitionUpdate> puList = PartitionUpdate.merge(Lists.newArrayList(first, second));
|
||||
Assert.assertEquals(1, puList.size());
|
||||
PartitionUpdate merged = puList.get(0);
|
||||
Assert.assertEquals("k2=1", merged.getName());
|
||||
Assert.assertEquals(Lists.newArrayList("file1", "file2", "file3", "file4"), merged.getFileNames());
|
||||
Assert.assertEquals(2, merged.getRowCount());
|
||||
Assert.assertEquals(20, merged.getTotalSizeInBytes());
|
||||
Assert.assertEquals("s3://starrocks-dla/writePath/k2=1", merged.getWritePath().toString());
|
||||
Assert.assertEquals("s3://starrocks-dla/writePath/k2=1", merged.getTargetPath().toString());
|
||||
Assert.assertTrue(merged.isS3Url());
|
||||
}
|
||||
}
|
||||
|
|
@ -12,10 +12,8 @@
|
|||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
|
||||
package com.starrocks.connector.hive;
|
||||
|
||||
import com.starrocks.connector.hive.RemoteFileInputFormat;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
|
|
|
|||
|
|
@ -74,7 +74,8 @@ public class HudiMetadataTest {
|
|||
hudiRemoteFileIO = new HudiRemoteFileIO(new Configuration());
|
||||
cachingRemoteFileIO = CachingRemoteFileIO.createCatalogLevelInstance(
|
||||
hudiRemoteFileIO, executorForRemoteFileRefresh, 100, 10, 10);
|
||||
fileOps = new RemoteFileOperations(cachingRemoteFileIO, executorForPullFiles, false, true);
|
||||
fileOps = new RemoteFileOperations(cachingRemoteFileIO, executorForPullFiles, executorForPullFiles,
|
||||
false, true, new Configuration());
|
||||
statisticsProvider = new HiveStatisticsProvider(hmsOps, fileOps);
|
||||
|
||||
UtFrameUtils.createMinStarRocksCluster();
|
||||
|
|
|
|||
|
|
@ -0,0 +1,116 @@
|
|||
// Copyright 2021-present StarRocks, Inc. All rights reserved.
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// https://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package com.starrocks.planner;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
import com.starrocks.analysis.TupleDescriptor;
|
||||
import com.starrocks.analysis.TupleId;
|
||||
import com.starrocks.catalog.Column;
|
||||
import com.starrocks.catalog.HiveTable;
|
||||
import com.starrocks.catalog.Type;
|
||||
import com.starrocks.connector.CatalogConnector;
|
||||
import com.starrocks.connector.ConnectorMgr;
|
||||
import com.starrocks.connector.ConnectorTableId;
|
||||
import com.starrocks.connector.hive.HiveStorageFormat;
|
||||
import com.starrocks.credential.CloudConfigurationFactory;
|
||||
import com.starrocks.qe.SessionVariable;
|
||||
import com.starrocks.sql.analyzer.AnalyzeTestUtil;
|
||||
import com.starrocks.thrift.TCloudType;
|
||||
import com.starrocks.thrift.TCompressionType;
|
||||
import com.starrocks.thrift.TDataSink;
|
||||
import com.starrocks.thrift.TDataSinkType;
|
||||
import com.starrocks.thrift.TExplainLevel;
|
||||
import com.starrocks.thrift.THiveTableSink;
|
||||
import com.starrocks.utframe.StarRocksAssert;
|
||||
import com.starrocks.utframe.UtFrameUtils;
|
||||
import mockit.Expectations;
|
||||
import mockit.Mocked;
|
||||
import org.junit.Assert;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.HashMap;
|
||||
|
||||
import static com.starrocks.server.CatalogMgr.ResourceMappingCatalog.toResourceName;
|
||||
import static com.starrocks.sql.analyzer.AnalyzeTestUtil.getStarRocksAssert;
|
||||
|
||||
public class HiveTableSinkTest {
|
||||
@BeforeClass
|
||||
public static void beforeClass() throws Exception {
|
||||
UtFrameUtils.createMinStarRocksCluster();
|
||||
AnalyzeTestUtil.init();
|
||||
StarRocksAssert starRocksAssert = getStarRocksAssert();
|
||||
String createHiveCatalogStmt = "create external catalog hive_catalog properties (\"type\"=\"hive\", " +
|
||||
"\"hive.metastore.uris\"=\"thrift://hms:9083\")";
|
||||
starRocksAssert.withCatalog(createHiveCatalogStmt);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testHiveTableSink(@Mocked CatalogConnector hiveConnector) {
|
||||
HiveTable.Builder builder = HiveTable.builder()
|
||||
.setId(ConnectorTableId.CONNECTOR_ID_GENERATOR.getNextId().asInt())
|
||||
.setTableName("hive_table")
|
||||
.setCatalogName("hive_catalog")
|
||||
.setResourceName(toResourceName("hive_catalog", "hive"))
|
||||
.setHiveDbName("hive_db")
|
||||
.setHiveTableName("hive_table")
|
||||
.setPartitionColumnNames(Lists.newArrayList("p1"))
|
||||
.setDataColumnNames(Lists.newArrayList("c1"))
|
||||
.setFullSchema(Lists.newArrayList(new Column("c1", Type.INT), new Column("p1", Type.INT)))
|
||||
.setTableLocation("hdfs://hadoop01:9000/tableLocation")
|
||||
.setProperties(new HashMap<>())
|
||||
.setStorageFormat(HiveStorageFormat.PARQUET)
|
||||
.setCreateTime(System.currentTimeMillis());
|
||||
|
||||
new Expectations() {
|
||||
{
|
||||
hiveConnector.getCloudConfiguration();
|
||||
result = CloudConfigurationFactory.buildDefaultCloudConfiguration();
|
||||
minTimes = 1;
|
||||
}
|
||||
};
|
||||
|
||||
ConnectorMgr connectorMgr = AnalyzeTestUtil.getConnectContext().getGlobalStateMgr().getConnectorMgr();
|
||||
new Expectations(connectorMgr) {
|
||||
{
|
||||
connectorMgr.getConnector("hive_catalog");
|
||||
result = hiveConnector;
|
||||
minTimes = 1;
|
||||
}
|
||||
};
|
||||
|
||||
TupleDescriptor desc = new TupleDescriptor(new TupleId(0));
|
||||
HiveTableSink sink = new HiveTableSink(builder.build(), desc, true, new SessionVariable());
|
||||
Assert.assertNull(sink.getExchNodeId());
|
||||
Assert.assertNull(sink.getOutputPartition());
|
||||
Assert.assertNull(sink.getOutputPartition());
|
||||
Assert.assertTrue(sink.canUsePipeLine());
|
||||
Assert.assertTrue(sink.canUseRuntimeAdaptiveDop());
|
||||
Assert.assertTrue(sink.getStagingDir().contains("/tmp/starrocks"));
|
||||
Assert.assertTrue(sink.getExplainString("SINK", TExplainLevel.NORMAL).contains(
|
||||
"SINKHive TABLE SINK\n" +
|
||||
"SINK TABLE: hive_catalog.hive_db.hive_table"));
|
||||
TDataSink tDataSink = sink.toThrift();
|
||||
Assert.assertEquals(TDataSinkType.HIVE_TABLE_SINK, tDataSink.getType());
|
||||
THiveTableSink tHiveTableSink = tDataSink.getHive_table_sink();
|
||||
Assert.assertTrue(tHiveTableSink.getStaging_dir().startsWith("hdfs://hadoop01:9000/tmp/starrocks"));
|
||||
Assert.assertEquals("parquet", tHiveTableSink.getFile_format());
|
||||
Assert.assertEquals("c1", tHiveTableSink.getData_column_names().get(0));
|
||||
Assert.assertEquals("p1", tHiveTableSink.getPartition_column_names().get(0));
|
||||
Assert.assertEquals(TCompressionType.GZIP, tHiveTableSink.getCompression_type());
|
||||
Assert.assertTrue(tHiveTableSink.is_static_partition_sink);
|
||||
Assert.assertEquals(TCloudType.DEFAULT, tHiveTableSink.getCloud_configuration().cloud_type);
|
||||
}
|
||||
}
|
||||
|
|
@ -38,6 +38,8 @@ import org.junit.Test;
|
|||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
|
||||
import static com.starrocks.connector.hive.HiveClassNames.MAPRED_PARQUET_INPUT_FORMAT_CLASS;
|
||||
|
||||
public class MetadataMgrTest {
|
||||
@BeforeClass
|
||||
public static void beforeClass() throws Exception {
|
||||
|
|
@ -141,7 +143,7 @@ public class MetadataMgrTest {
|
|||
StorageDescriptor sd = new StorageDescriptor();
|
||||
sd.setCols(unPartKeys);
|
||||
sd.setLocation(hdfsPath);
|
||||
sd.setInputFormat("org.apache.hadoop.hive.ql.io.HiveInputFormat");
|
||||
sd.setInputFormat(MAPRED_PARQUET_INPUT_FORMAT_CLASS);
|
||||
Table msTable1 = new Table();
|
||||
msTable1.setDbName("hive_db");
|
||||
msTable1.setTableName("hive_table");
|
||||
|
|
|
|||
|
|
@ -18,8 +18,8 @@ import com.google.common.collect.ImmutableList;
|
|||
import com.google.common.collect.Lists;
|
||||
import com.starrocks.catalog.Column;
|
||||
import com.starrocks.catalog.Database;
|
||||
import com.starrocks.catalog.HiveTable;
|
||||
import com.starrocks.catalog.IcebergTable;
|
||||
import com.starrocks.catalog.Table;
|
||||
import com.starrocks.catalog.Type;
|
||||
import com.starrocks.server.MetadataMgr;
|
||||
import com.starrocks.utframe.StarRocksAssert;
|
||||
|
|
@ -108,16 +108,6 @@ public class AnalyzeInsertTest {
|
|||
analyzeFail("insert into iceberg_catalog.db.err_tbl values (1)",
|
||||
"Table err_tbl is not found");
|
||||
|
||||
new Expectations(metadata) {
|
||||
{
|
||||
metadata.getTable(anyString, anyString, anyString);
|
||||
result = new HiveTable();
|
||||
minTimes = 0;
|
||||
}
|
||||
};
|
||||
analyzeFail("insert into iceberg_catalog.db.hive_tbl values (1)",
|
||||
"Only support insert into olap table or mysql table or iceberg table");
|
||||
|
||||
new Expectations(metadata) {
|
||||
{
|
||||
metadata.getTable(anyString, anyString, anyString);
|
||||
|
|
@ -166,6 +156,10 @@ public class AnalyzeInsertTest {
|
|||
icebergTable.getColumn(anyString);
|
||||
result = ImmutableList.of(new Column("p1", Type.ARRAY_DATE));
|
||||
minTimes = 0;
|
||||
|
||||
icebergTable.isIcebergTable();
|
||||
result = true;
|
||||
minTimes = 0;
|
||||
}
|
||||
};
|
||||
|
||||
|
|
@ -210,13 +204,13 @@ public class AnalyzeInsertTest {
|
|||
result = Lists.newArrayList("p1", "p2");
|
||||
minTimes = 1;
|
||||
|
||||
icebergTable.getPartitionColumns();
|
||||
result = Lists.newArrayList(new Column("p1", Type.DATETIME));
|
||||
icebergTable.getType();
|
||||
result = Table.TableType.ICEBERG;
|
||||
minTimes = 1;
|
||||
}
|
||||
};
|
||||
|
||||
analyzeFail("insert into iceberg_catalog.db.tbl select 1, 2, \"2023-01-01 12:34:45\"",
|
||||
"Unsupported partition column type [DATETIME] for iceberg table sink.");
|
||||
"Unsupported partition column type [DATETIME] for ICEBERG table sink.");
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -815,9 +815,6 @@ public class InsertPlanTest extends PlanTestBase {
|
|||
|
||||
icebergTable.getPartitionColumnNames();
|
||||
result = new ArrayList<>();
|
||||
|
||||
icebergTable.getPartitionColumns();
|
||||
result = Lists.newArrayList(new Column("k1", Type.INT), new Column("k2", Type.INT));
|
||||
}
|
||||
};
|
||||
|
||||
|
|
|
|||
|
|
@ -708,6 +708,12 @@ under the License.
|
|||
<version>4.0.9</version>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>io.airlift</groupId>
|
||||
<artifactId>concurrent</artifactId>
|
||||
<version>202</version>
|
||||
</dependency>
|
||||
|
||||
</dependencies>
|
||||
</dependencyManagement>
|
||||
|
||||
|
|
|
|||
|
|
@ -52,7 +52,8 @@ enum TDataSinkType {
|
|||
MEMORY_SCRATCH_SINK,
|
||||
MULTI_CAST_DATA_STREAM_SINK,
|
||||
SCHEMA_TABLE_SINK,
|
||||
ICEBERG_TABLE_SINK
|
||||
ICEBERG_TABLE_SINK,
|
||||
HIVE_TABLE_SINK
|
||||
}
|
||||
|
||||
enum TResultSinkType {
|
||||
|
|
@ -228,6 +229,16 @@ struct TIcebergTableSink {
|
|||
6: optional CloudConfiguration.TCloudConfiguration cloud_configuration
|
||||
}
|
||||
|
||||
struct THiveTableSink {
|
||||
1: optional string staging_dir
|
||||
2: optional string file_format
|
||||
3: optional list<string> data_column_names
|
||||
4: optional list<string> partition_column_names
|
||||
5: optional Types.TCompressionType compression_type
|
||||
6: optional bool is_static_partition_sink
|
||||
7: optional CloudConfiguration.TCloudConfiguration cloud_configuration
|
||||
}
|
||||
|
||||
struct TDataSink {
|
||||
1: required TDataSinkType type
|
||||
2: optional TDataStreamSink stream_sink
|
||||
|
|
@ -239,4 +250,5 @@ struct TDataSink {
|
|||
9: optional TMultiCastDataStreamSink multi_cast_stream_sink
|
||||
10: optional TSchemaTableSink schema_table_sink
|
||||
11: optional TIcebergTableSink iceberg_table_sink
|
||||
12: optional THiveTableSink hive_table_sink
|
||||
}
|
||||
|
|
|
|||
|
|
@ -545,9 +545,18 @@ struct TIcebergDataFile {
|
|||
7: optional TIcebergColumnStats column_stats;
|
||||
}
|
||||
|
||||
struct THiveFileInfo {
|
||||
1: optional string file_name
|
||||
2: optional string partition_path
|
||||
4: optional i64 record_count
|
||||
5: optional i64 file_size_in_bytes
|
||||
}
|
||||
|
||||
struct TSinkCommitInfo {
|
||||
1: optional TIcebergDataFile iceberg_data_file
|
||||
2: optional THiveFileInfo hive_file_info
|
||||
// ... for other tables sink commit info
|
||||
|
||||
100: optional bool is_overwrite;
|
||||
101: optional string staging_dir
|
||||
}
|
||||
|
|
|
|||
Loading…
Reference in New Issue