[Feature] support to hive sink (#29171)

Signed-off-by: stephen <stephen5217@163.com>
This commit is contained in:
stephen 2023-08-24 11:01:34 +08:00 committed by GitHub
parent 681452c5ca
commit 691380242d
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
70 changed files with 3565 additions and 238 deletions

View File

@ -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

View File

@ -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;

View File

@ -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();

View File

@ -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)

View File

@ -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(); }

View File

@ -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

View File

@ -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

View File

@ -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}

View File

@ -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

View File

@ -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

View File

@ -815,6 +815,10 @@ under the License.
<version>2.4.0</version>
</dependency>
<dependency>
<groupId>io.airlift</groupId>
<artifactId>concurrent</artifactId>
</dependency>
</dependencies>
<build>

View File

@ -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);
}
}
}

View File

@ -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() {

View File

@ -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() {

View File

@ -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;

View File

@ -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 + "/";
}
}

View File

@ -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());
}
}
}

View File

@ -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();

View File

@ -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);
}
}
}

View File

@ -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(),

View File

@ -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;
}

View File

@ -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

View File

@ -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

View File

@ -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() {

View File

@ -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();

View File

@ -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.");

View File

@ -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);

View File

@ -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);
}
}
}

View File

@ -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,
}
}

View File

@ -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;
}
}

View File

@ -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;
}
}

View File

@ -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();

View File

@ -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();
}
}

View File

@ -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();

View File

@ -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;

View File

@ -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;

View File

@ -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;
}
}

View File

@ -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();

View File

@ -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;
}

View File

@ -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;
}

View File

@ -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()

View File

@ -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();

View File

@ -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();

View File

@ -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");

View File

@ -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();
}
}

View File

@ -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() {

View File

@ -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 {

View File

@ -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();

View File

@ -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);
}
}

View File

@ -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"));
}
}

View File

@ -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));
}
}

View File

@ -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"));
}
}

View File

@ -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);
}
}

View File

@ -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));
}
}

View File

@ -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();

View File

@ -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));
}
}

View File

@ -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();

View File

@ -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>() {

View File

@ -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 {

View File

@ -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

View File

@ -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());
}
}

View File

@ -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;

View File

@ -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();

View File

@ -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);
}
}

View File

@ -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");

View File

@ -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.");
}
}

View File

@ -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));
}
};

View File

@ -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>

View File

@ -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
}

View File

@ -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
}