[Feature] Add native format reader to access StarRocks data bypass BE Server. (#54470)

Signed-off-by: plotor <zhenchao.wang@hotmail.com>
This commit is contained in:
Zhenchao Wang 2025-03-13 10:13:03 +08:00 committed by GitHub
parent 1acf89f27c
commit bd53c8fd47
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
45 changed files with 4864 additions and 131 deletions

View File

@ -912,7 +912,6 @@ set(STARROCKS_DEPENDENCIES
arrow_flight_sql
gRPC::grpc
gRPC::grpc++
jemalloc # required by arrow
parquet
orc
cctz
@ -943,6 +942,13 @@ set(STARROCKS_DEPENDENCIES
${WL_END_GROUP}
)
if (NOT BUILD_FORMAT_LIB)
set(STARROCKS_DEPENDENCIES
${STARROCKS_DEPENDENCIES}
jemalloc # required by arrow
)
endif()
if (${WITH_STARCACHE} STREQUAL "ON")
set(STARROCKS_DEPENDENCIES
${STARROCKS_DEPENDENCIES}

View File

@ -368,6 +368,7 @@ Status UpdateConfigAction::update_config(const std::string& name, const std::str
UPDATE_STARLET_CONFIG(starlet_delete_files_max_key_in_batch, delete_files_max_key_in_batch);
#undef UPDATE_STARLET_CONFIG
#ifndef BUILD_FORMAT_LIB
_config_callback.emplace("starlet_filesystem_instance_cache_capacity", [&]() -> Status {
LOG(INFO) << "set starlet_filesystem_instance_cache_capacity:"
<< config::starlet_filesystem_instance_cache_capacity;
@ -376,6 +377,8 @@ Status UpdateConfigAction::update_config(const std::string& name, const std::str
}
return Status::OK();
});
#endif
#endif // USE_STAROS
});

View File

@ -12,7 +12,6 @@
# See the License for the specific language governing permissions and
# limitations under the License.
set(CMAKE_VERBOSE_MAKEFILE ON)
set(LIBRARY_OUTPUT_PATH "${BUILD_DIR}/src/starrocks_format")
@ -42,7 +41,7 @@ set(STARROCKS_LIBS
StarRocksGen
Webserver
TestUtil
BlockCache
Cache
${WL_END_GROUP}
)

View File

@ -22,10 +22,12 @@
#include "common/config.h"
#include "fs/fs_s3.h"
#include "runtime/exec_env.h"
#include "runtime/time_types.h"
#include "storage/lake/fixed_location_provider.h"
#include "storage/lake/tablet_manager.h"
#include "storage/olap_define.h"
#include "util/mem_info.h"
#include "util/timezone_utils.h"
namespace starrocks::lake {
@ -56,10 +58,14 @@ void starrocks_format_initialize(void) {
Aws::InitAPI(aws_sdk_options);
MemInfo::init();
date::init_date_cache();
TimezoneUtils::init_time_zones();
auto ge_init_stat = GlobalEnv::GetInstance()->init();
CHECK(ge_init_stat.ok()) << "init global env error";
auto lake_location_provider = std::make_shared<FixedLocationProvider>("");
_lake_tablet_manager = new lake::TabletManager(lake_location_provider, config::lake_metadata_cache_limit);
LOG(INFO) << "starrocks format module has been initialized successfully";

View File

@ -37,8 +37,15 @@ Status LakeDelvecLoader::load(const TabletSegmentId& tsid, int64_t version, DelV
Status LakeDelvecLoader::load_from_file(const TabletSegmentId& tsid, int64_t version, DelVectorPtr* pdelvec) {
(*pdelvec).reset(new DelVector());
// 2. find in delvec file
ASSIGN_OR_RETURN(auto metadata,
_tablet_manager->get_tablet_metadata(tsid.tablet_id, version, _fill_cache, 0, _lake_io_opts.fs));
TabletMetadataPtr metadata;
if (_lake_io_opts.location_provider) {
const std::string filepath = _lake_io_opts.location_provider->tablet_metadata_location(tsid.tablet_id, version);
ASSIGN_OR_RETURN(metadata, _tablet_manager->get_tablet_metadata(filepath, _fill_cache, 0, _lake_io_opts.fs));
} else {
ASSIGN_OR_RETURN(metadata, _tablet_manager->get_tablet_metadata(tsid.tablet_id, version, _fill_cache, 0,
_lake_io_opts.fs));
}
RETURN_IF_ERROR(
lake::get_del_vec(_tablet_manager, *metadata, tsid.segment_id, _fill_cache, _lake_io_opts, pdelvec->get()));
return Status::OK();

View File

@ -1 +1,222 @@
# StarRocks Format Library
# StarRocks Format Library
StarRocks Format Library is an SDK for reading and writing StarRocks data files, which provides C++ and Java version of tablet data file reader and writer. With the Format SDK, you can bypass the StarRocks BE node and directly read and write tablet data files that stored in the remote file system (e.g. S3, HDFS) in the shared-data mode. So that a piece of data can be accessed (read and write) by multiple computing engines (e.g. Spark, StarRocks) at the same time.
## Why we do it?
StarRocks is a next-gen, high-performance analytical data warehouse. However, the limitations of the MPP architecture still cause it to face some problems in large-data ETL scenarios. For example:
- __Low resource utilization__: StarRocks clusters adopt the resource reservation mode mostly, but in order to support large-data ETL scenarios, which require a large amount of resource overhead in a relatively short period of time, plan redundant resources in advance may reduce the overall resource utilization of the cluster.
- __Poor resource isolation__: In terms of resource isolation, StarRocks adopts Group rather than Query level isolation. In large-data ETL scenarios, there is a risk that a query with large resource overhead will run out of resources, thereby starving some small queries to death.
- __Lack of failure tolerance__: Due to the lack of task-level failure tolerance mechanism, when an ETL job fails, it will usually still fail even if rerun it manually.
In order to solve the above problems, we have proposed the idea of __bypassing StarRocks BE node to directly read and write tablet data files in the shared-data mode__. Taking the Apache Spark as an example_In fact, we are not bound to it, and we can support more computing engines by design._, the overall architecture design is as follows:
![starrocks-bypass-load](https://github.com/user-attachments/assets/00d37763-5b6c-42ac-a980-40a604746c59)
For complex query scenarios, users can submit jobs through the `spark-sql` client. Spark interacts with StarRocks to obtain tables's metadata information, manages data write transaction, and do read/write operation on shared storage through the Format SDK directly. Finally, for one piece of StarRocks's data files, multiple computing engines can read and write it.
## How to build
The building process is divided into two steps. __Firstly__, execute the following command to compile `be` and `format-lib` module to generate the `libstarrocks_format.so` file:
```bash
# build be
./build.sh --clean --be --enable-shared-data
# build format-lib
./build.sh --format-lib --enable-shared-data
```
__Then__ execute the following command to compile the `format-sdk` module:
```bash
cd ./format-sdk
mvn clean package -DskipTests
```
After that, you will get two files: `libstarrocks_format_wrapper.so` and `format-sdk.jar`.
## How to use
Assume there is a product sales information table, the table schema is as follows:
```sql
CREATE TABLE `tb_sales`
(
`product_id` BIGINT NOT NULL COMMENT "Product ID",
`date` DATE NOT NULL COMMENT "Sale Date",
`quantity` INT NOT NULL COMMENT "Sale Quantity",
`amount` BIGINT NOT NULL COMMENT "Sale Amount"
) ENGINE = OLAP PRIMARY KEY(`product_id`, `date`)
DISTRIBUTED BY HASH(`product_id`)
PROPERTIES (
"enable_persistent_index" = "true"
);
```
Next, we will take a Java program as an example to show how to use the Format SDK to read and write data files of this table.
> Note: you need to introduce the `format-sdk.jar` dependency into your project.
- __Write by StarRocksWriter__
You can write a program to use `StarRocksWriter` to perform write operations according to the following steps:
1. Request the metadata and partition information of the table by `RestClient`;
2. Begin a transaction;
3. Create and open a `StarRocksWriter` which binding to the target Tablet;
4. Create an Arrow `VectorSchemaRoot` object, and fill it with custom data according to the table schema definition;
5. Call the `StarRocksWriter#write` method to write the filled `VectorSchemaRoot` object to StarRocks;
6. Batch as needed, and call the `StarRocksWriter#flush` method to flush the data to the file system;
7. Repeat steps 4 to 6 above until all data is written;
8. Close and release `StarRocksWriter`, it's automatically closed and released using the `try-with-resources` statement in this example;
9. Commit or rollback the transaction.
Example code as follows
```java
final String DEFAULT_CATALOG = "default_catalog";
final String DB_NAME = "bypass";
final String TB_NAME = "tb_sales";
Config config = Config.newBuilder()
.feHttpUrl("http://127.0.0.1:8030")
.feJdbcUrl("jdbc:mysql://127.0.0.1:9030")
.database(DB_NAME)
.username("root")
.password("******")
.s3Endpoint("https://tos-s3-cn-beijing.volces.com")
.s3EndpointRegion("cn-beijing")
.s3ConnectionSslEnabled(false)
.s3PathStyleAccess(false)
.s3AccessKey("******")
.s3SecretKey("******")
.build();
String label = String.format(
"bypass_%s_%s_%s", DB_NAME, TB_NAME, RandomStringUtils.randomAlphabetic(8)
);
try (RestClient restClient = RestClient.newBuilder()
.feHttpEndpoints("http://127.0.0.1:8030")
.username("root")
.password("******")
.build()) {
// Request the metadata and partition information of the table by RestClient
TableSchema tableSchema = restClient.getTableSchema(DEFAULT_CATALOG, DB_NAME, TB_NAME);
List<TablePartition> tablePartitions = restClient.listTablePartitions(DEFAULT_CATALOG, DB_NAME, TB_NAME, false);
Schema arrowSchema = ArrowUtils.toArrowSchema(tableSchema, ZoneId.systemDefault());
// Begin a transaction
TransactionResult beginTxnResult = restClient.beginTransaction(DEFAULT_CATALOG, DB_NAME, TB_NAME, label);
assertTrue(beginTxnResult.isOk());
List<TabletCommitInfo> commitTablets = new ArrayList<>();
TablePartition tablePartition = tablePartitions.get(0);
for (int i = 0; i < tablePartition.getTablets().size(); i++) {
TablePartition.Tablet tablet = tablePartition.getTablets().get(i);
Long tabletId = tablet.getId();
// Create and open a StarRocksWriter which binding to the target Tablet
try (StarRocksWriter tabletWriter = new StarRocksWriter(
tabletId, tablePartition.getStoragePath(), beginTxnResult.getTxnId(), arrowSchema, config)) {
tabletWriter.open();
try (VectorSchemaRoot root = VectorSchemaRoot.create(arrowSchema, tabletWriter.getAllocator())) {
// Fill the VectorSchemaRoot object with custom data according to the table schema definition
fillSampleData(i, root);
// Write the filled VectorSchemaRoot object to StarRocks
tabletWriter.write(root);
// Flush the data to the file system
tabletWriter.flush();
}
tabletWriter.finish();
}
commitTablets.add(new TabletCommitInfo(tabletId, tablet.getPrimaryComputeNodeId()));
}
// Commit or rollback the transaction
TransactionResult prepareTxnResult = restClient.prepareTransaction(
DEFAULT_CATALOG, DB_NAME, beginTxnResult.getLabel(), commitTablets, null);
if (prepareTxnResult.isOk()) {
TransactionResult commitTxnResult = restClient.commitTransaction(
DEFAULT_CATALOG, DB_NAME, prepareTxnResult.getLabel());
assertTrue(commitTxnResult.isOk());
} else {
TransactionResult rollbackTxnResult = restClient.rollbackTransaction(
DEFAULT_CATALOG, DB_NAME, prepareTxnResult.getLabel(), null);
assertTrue(rollbackTxnResult.isOk());
}
}
```
- __Read by StarRocksReader__
You can write a program to use `StarRocksWriter` to perform read operations according to the following steps:
1. Request the metadata and partition information of the table by `RestClient`;
2. Create and open a `StarRocksReader` which binding to the target Tablet;
3. `StarRocksReader` supports iterator access, so you can iterate and read the data;
4. Close and release `StarRocksReader`, it's automatically closed and released using the `try-with-resources` statement in this example.
Example code as follows
```java
final String DEFAULT_CATALOG = "default_catalog";
final String DB_NAME = "bypass";
final String TB_NAME = "tb_sales";
Config config = Config.newBuilder()
.feHttpUrl("http://127.0.0.1:8030")
.feJdbcUrl("jdbc:mysql://127.0.0.1:9030")
.database(DB_NAME)
.username("root")
.password("******")
.s3Endpoint("https://tos-s3-cn-beijing.volces.com")
.s3EndpointRegion("cn-beijing")
.s3ConnectionSslEnabled(false)
.s3PathStyleAccess(false)
.s3AccessKey("******")
.s3SecretKey("******")
.build();
try (RestClient restClient = RestClient.newBuilder()
.feHttpEndpoints("http://127.0.0.1:8030")
.username("root")
.password("******")
.build()) {
// Request the metadata and partition information of the table by RestClient
TableSchema tableSchema = restClient.getTableSchema(DEFAULT_CATALOG, DB_NAME, TB_NAME);
List<TablePartition> tablePartitions = restClient.listTablePartitions(DEFAULT_CATALOG, DB_NAME, TB_NAME, false);
Schema arrowSchema = ArrowUtils.toArrowSchema(tableSchema, ZoneId.systemDefault());
TablePartition partition = tablePartitions.get(0);
for (TablePartition.Tablet tablet : partition.getTablets()) {
Long tabletId = tablet.getId();
// Create and open a StarRocksReader which binding to the target Tablet
try (StarRocksReader reader = new StarRocksReader(
tabletId,
partition.getStoragePath(),
partition.getVisibleVersion(),
arrowSchema,
arrowSchema,
config)) {
reader.open();
// StarRocksReader supports iterator access, so you can iterate and read the data
while (reader.hasNext()) {
try (VectorSchemaRoot root = reader.next()) {
System.out.println(root.contentToTSVString());
}
}
}
}
}
```

View File

@ -56,6 +56,12 @@
<version>${arrow.version}</version>
</dependency>
<dependency>
<groupId>org.apache.httpcomponents</groupId>
<artifactId>httpclient</artifactId>
<version>4.5.14</version>
</dependency>
<dependency>
<groupId>org.slf4j</groupId>
<artifactId>slf4j-api</artifactId>
@ -70,6 +76,13 @@
<scope>compile</scope>
</dependency>
<dependency>
<groupId>com.mysql</groupId>
<artifactId>mysql-connector-j</artifactId>
<version>8.0.33</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.logging.log4j</groupId>
<artifactId>log4j-slf4j-impl</artifactId>
@ -162,6 +175,7 @@
</execution>
</executions>
</plugin>
<plugin>
<groupId>org.codehaus.mojo</groupId>
<artifactId>exec-maven-plugin</artifactId>

View File

@ -27,6 +27,7 @@
#include "column/column.h"
#include "column/column_helper.h"
#include "column/field.h"
#include "format/format_utils.h"
#include "types/logical_type.h"
#include "util/json.h"
#include "util/slice.h"
@ -99,6 +100,55 @@ public:
}
return arrow::Status::OK();
}
arrow::Result<std::shared_ptr<arrow::Array>> toArrowArray(const std::shared_ptr<Column>& column) override {
using ArrowBuilderType = typename arrow::TypeTraits<ArrowType>::BuilderType;
std::unique_ptr<ArrowBuilderType> builder =
std::make_unique<ArrowBuilderType>(_arrow_type, const_cast<arrow::MemoryPool*>(_pool));
size_t num_rows = column->size();
ARROW_RETURN_NOT_OK(builder->Reserve(num_rows));
for (size_t i = 0; i < num_rows; ++i) {
if (column->is_null(i)) {
ARROW_RETURN_NOT_OK(builder->AppendNull());
continue;
}
auto* data_column = ColumnHelper::get_data_column(column.get());
const SrCppType* column_data = down_cast<const SrColumnType*>(data_column)->get_data().data();
if constexpr (SR_TYPE == TYPE_CHAR || SR_TYPE == TYPE_VARCHAR || SR_TYPE == TYPE_VARBINARY) {
Slice slice = column_data[i];
ARROW_RETURN_NOT_OK(builder->Append(slice.data, slice.size));
} else if constexpr (SR_TYPE == TYPE_LARGEINT) {
SrCppType sr_value = column_data[i];
std::string value =
DecimalV3Cast::to_string<int128_t>(sr_value, starrocks::decimal_precision_limit<int128_t>, 0);
ARROW_RETURN_NOT_OK(builder->Append(value));
} else if constexpr (SR_TYPE == TYPE_JSON) {
auto item = down_cast<const SrColumnType*>(data_column)->get_object(i);
FORMAT_ASSIGN_OR_RAISE_ARROW_STATUS(auto json_value, item->to_string());
ARROW_RETURN_NOT_OK(builder->Append(json_value));
} else if constexpr (SR_TYPE == TYPE_OBJECT) { // bitmap column
auto item = down_cast<const SrColumnType*>(data_column)->get_object(i);
std::string buf;
size_t serialize_size = item->get_size_in_bytes();
buf.resize(serialize_size);
item->write(buf.data());
ARROW_RETURN_NOT_OK(builder->Append(buf.data(), serialize_size));
} else if constexpr (SR_TYPE == TYPE_HLL) { // hll column
auto item = down_cast<const SrColumnType*>(data_column)->get_object(i);
std::string buf;
size_t serialize_size = item->max_serialized_size();
buf.resize(serialize_size);
size_t size = item->serialize(reinterpret_cast<uint8_t*>(buf.data()));
ARROW_RETURN_NOT_OK(builder->Append(buf.data(), size));
} else {
return arrow::Status::TypeError("Can't convert starrocks type ", _sr_field->type()->type(),
" to arrow binary.");
}
}
return builder->Finish();
}
};
} // namespace starrocks::lake::format

View File

@ -22,7 +22,7 @@
namespace starrocks::lake::format {
class ColumnConverter;
using ColumnConverterVector = std::vector<std::shared_ptr<ColumnConverter> >;
using ColumnConverterVector = std::vector<std::shared_ptr<ColumnConverter>>;
constexpr int32_t ARROW_CONVERTER_ID(arrow::Type::type arrow_type_id, LogicalType sr_logical_type) {
DCHECK(arrow_type_id >= arrow::Type::NA && arrow_type_id < arrow::Type::MAX_ID)
@ -36,9 +36,9 @@ constexpr int32_t ARROW_CONVERTER_ID(arrow::Type::type arrow_type_id, LogicalTyp
class ColumnConverter {
public:
// Create a Converter for the given data type
static arrow::Result<std::shared_ptr<ColumnConverter> > create(const std::shared_ptr<arrow::DataType>& arrow_type,
const std::shared_ptr<Field>& sr_field,
const arrow::MemoryPool* pool);
static arrow::Result<std::shared_ptr<ColumnConverter>> create(const std::shared_ptr<arrow::DataType>& arrow_type,
const std::shared_ptr<Field>& sr_field,
const arrow::MemoryPool* pool);
static arrow::Result<ColumnConverterVector> create_children_converter(
const std::shared_ptr<arrow::DataType>& arrow_type, const std::shared_ptr<Field>& sr_field,
@ -58,8 +58,13 @@ public:
*/
virtual arrow::Status toSrColumn(std::shared_ptr<arrow::Array> array, ColumnPtr& column) = 0;
/**
* Convert starrocks column to arrow array.
*/
virtual arrow::Result<std::shared_ptr<arrow::Array>> toArrowArray(const std::shared_ptr<Column>& column) = 0;
protected:
arrow::Result<std::shared_ptr<arrow::Buffer> > convert_null_bitmap(const Buffer<uint8_t>& null_bytes);
arrow::Result<std::shared_ptr<arrow::Buffer>> convert_null_bitmap(const Buffer<uint8_t>& null_bytes);
static ColumnPtr get_data_column(const ColumnPtr& column);
@ -71,7 +76,7 @@ protected:
const arrow::MemoryPool* _pool;
std::vector<std::shared_ptr<ColumnConverter> > _children;
std::vector<std::shared_ptr<ColumnConverter>> _children;
};
} // namespace starrocks::lake::format

View File

@ -95,6 +95,34 @@ public:
return arrow::Status::OK();
}
arrow::Result<std::shared_ptr<arrow::Array>> toArrowArray(const std::shared_ptr<Column>& column) override {
// convert data column,include list:offsets, values, map: offsets, keys, values, struct: children columns.
const auto data_column = arrow::internal::checked_pointer_cast<SrColumnType>(get_data_column(column));
ARROW_ASSIGN_OR_RAISE(std::vector<starrocks::ColumnPtr> sr_sub_columns, get_children_columns(data_column));
std::vector<std::shared_ptr<arrow::Array>> arrays;
arrays.resize(sr_sub_columns.size());
if (_children.size() < arrays.size()) {
return arrow::Status::Invalid("Converter size (", _children.size(), ") is less than arrow array size(",
arrays.size(), ")");
}
// convert children data column
for (size_t idx = 0; idx < arrays.size(); ++idx) {
ARROW_ASSIGN_OR_RAISE(arrays[idx], _children[idx]->toArrowArray(sr_sub_columns[idx]));
}
// convert null bitmap
std::shared_ptr<arrow::Buffer> null_bitmap;
if (column->is_nullable()) {
auto nullable = down_cast<NullableColumn*>(column.get());
auto& null_bytes = nullable->immutable_null_column_data();
ARROW_ASSIGN_OR_RAISE(null_bitmap, convert_null_bitmap(null_bytes));
}
return make_nested_array<ArrowArrayType>(arrays, null_bitmap);
}
private:
template <class ArrowArrayClass, typename = std::enable_if_t<std::is_same_v<ArrowArrayClass, arrow::ListArray> ||
std::is_same_v<ArrowArrayClass, arrow::MapArray> ||

View File

@ -153,6 +153,58 @@ public:
return arrow::Status::OK();
}
arrow::Result<std::shared_ptr<arrow::Array>> toArrowArray(const std::shared_ptr<Column>& column) override {
using ArrowBuilderType = typename arrow::TypeTraits<ArrowType>::BuilderType;
const auto& real_arrow_type = arrow::internal::checked_pointer_cast<ArrowType>(_arrow_type);
std::unique_ptr<ArrowBuilderType> builder =
std::make_unique<ArrowBuilderType>(_arrow_type, const_cast<arrow::MemoryPool*>(_pool));
size_t num_rows = column->size();
ARROW_RETURN_NOT_OK(builder->Reserve(num_rows));
for (size_t i = 0; i < num_rows; ++i) {
if (column->is_null(i)) {
ARROW_RETURN_NOT_OK(builder->AppendNull());
continue;
}
auto* data_column = ColumnHelper::get_data_column(column.get());
const SrCppType* column_data = down_cast<const SrColumnType*>(data_column)->get_data().data();
if constexpr (SR_TYPE == TYPE_DATE || SR_TYPE == TYPE_DATETIME) {
if constexpr (std::is_base_of_v<arrow::Date32Type, ArrowType>) {
ARROW_RETURN_NOT_OK(builder->Append((column_data[i].to_unixtime() / MILLIS_PER_DAY)));
} else if constexpr (std::is_base_of_v<arrow::Date64Type, ArrowType>) {
ARROW_RETURN_NOT_OK(builder->Append(column_data[i].to_unixtime()));
} else {
auto millis = column_data[i].to_unixtime(_ctz);
if (real_arrow_type->unit() == arrow::TimeUnit::MILLI) {
ARROW_RETURN_NOT_OK(builder->Append(millis));
} else if (real_arrow_type->unit() == arrow::TimeUnit::MICRO) {
ARROW_RETURN_NOT_OK(builder->Append(millis * MICROS_PER_MILLIS));
} else {
return arrow::Status::Invalid("Unsupported timeunit ", real_arrow_type->unit());
}
}
} else if constexpr (SR_TYPE == TYPE_DECIMAL32 || SR_TYPE == TYPE_DECIMAL64 || SR_TYPE == TYPE_DECIMAL128) {
int128_t c_value = column_data[i];
int64_t high = c_value >> 64;
uint64_t low = c_value;
arrow::Decimal128 value(high, low);
ARROW_RETURN_NOT_OK(builder->Append(value));
} else if constexpr (SR_TYPE == TYPE_LARGEINT) {
int128_t c_value = column_data[i];
int64_t high = c_value >> 64;
uint64_t low = c_value;
arrow::Decimal256 value(arrow::Decimal128(high, low));
ARROW_RETURN_NOT_OK(builder->Append(value));
} else {
ARROW_RETURN_NOT_OK(builder->Append((column_data[i])));
}
}
return builder->Finish();
}
};
} // namespace starrocks::lake::format

View File

@ -76,6 +76,38 @@ public:
}
};
class ChunkToRecordBatchConverterImpl : public ChunkToRecordBatchConverter {
public:
ChunkToRecordBatchConverterImpl(const std::shared_ptr<Schema>& sr_schema,
const std::shared_ptr<arrow::Schema>& arrow_schema, arrow::MemoryPool* pool)
: ChunkToRecordBatchConverter(sr_schema, arrow_schema, pool) {}
arrow::Result<std::shared_ptr<arrow::RecordBatch>> convert(const Chunk* chunk) override {
if (chunk == nullptr) {
return nullptr;
}
size_t column_size = chunk->num_columns();
if (column_size != _arrow_schema->num_fields()) {
return arrow::Status::Invalid("Field number in chunk is ", column_size, ", expected ",
_arrow_schema->num_fields());
}
std::vector<std::shared_ptr<arrow::Array>> arrays(column_size);
for (size_t idx = 0; idx < column_size; ++idx) {
ARROW_ASSIGN_OR_RAISE(arrays[idx], _converters[idx]->toArrowArray(chunk->columns()[idx]));
}
return arrow::RecordBatch::Make(_arrow_schema, chunk->num_rows(), std::move(arrays));
}
};
arrow::Result<std::shared_ptr<ChunkToRecordBatchConverter>> ChunkToRecordBatchConverter::create(
const std::shared_ptr<Schema>& sr_schema, const std::shared_ptr<arrow::Schema>& arrow_schema,
arrow::MemoryPool* pool) {
return std::make_shared<ChunkToRecordBatchConverterImpl>(sr_schema, arrow_schema, pool);
}
arrow::Result<std::shared_ptr<RecordBatchToChunkConverter>> RecordBatchToChunkConverter::create(
const std::shared_ptr<Schema>& sr_schema, const std::shared_ptr<arrow::Schema>& arrow_schema,
arrow::MemoryPool* pool) {

View File

@ -18,5 +18,7 @@
add_library(format STATIC
starrocks_format_writer.h
starrocks_format_writer.cpp
starrocks_format_reader.h
starrocks_format_reader.cpp
format_utils.h
)

View File

@ -0,0 +1,501 @@
// 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.
#define FMT_HEADER_ONLY
#include <arrow/array/builder_primitive.h>
#include <arrow/c/bridge.h>
#include <arrow/record_batch.h>
#include <arrow/result.h>
#include <arrow/status.h>
#include <arrow/type_fwd.h>
#include <arrow/util/key_value_metadata.h>
#include <utility>
#include "fmt/format.h"
// project dependencies
#include "convert/starrocks_arrow_converter.h"
#include "starrocks_format_reader.h"
// starrocks dependencies
#include "column/chunk.h"
#include "common/status.h"
#include "exec/olap_scan_prepare.h"
#include "exec/pipeline/runtime_filter_types.h"
#include "format_utils.h"
#include "gen_cpp/QueryPlanExtra_types.h"
#include "options.h"
#include "runtime/descriptors.h"
#include "starrocks_format/starrocks_lib.h"
#include "storage/chunk_helper.h"
#include "storage/column_predicate.h"
#include "storage/lake/fixed_location_provider.h"
#include "storage/lake/tablet_reader.h"
#include "storage/lake/versioned_tablet.h"
#include "storage/olap_common.h"
#include "storage/predicate_parser.h"
#include "storage/tablet_schema.h"
#include "util/thrift_util.h"
#include "util/url_coding.h"
namespace starrocks::lake::format {
class StarRocksFormatReaderImpl : public StarRocksFormatReader {
public:
StarRocksFormatReaderImpl(int64_t tablet_id, std::string tablet_root_path, int64_t version,
std::shared_ptr<arrow::Schema> required_schema,
std::shared_ptr<arrow::Schema> output_schema,
std::unordered_map<std::string, std::string> options)
: _tablet_id(tablet_id),
_version(version),
_required_schema(std::move(required_schema)),
_output_schema(std::move(output_schema)),
_tablet_root_path(std::move(tablet_root_path)),
_options(std::move(options)) {
auto itr = _options.find(SR_FORMAT_CHUNK_SIZE);
if (itr != _options.end() && !itr->second.empty()) {
_chunk_size = stoi(itr->second);
} else {
_chunk_size = config::vector_chunk_size;
}
}
arrow::Status open() override {
/*
* support the below file system options, same as hadoop aws fs options
* fs.s3a.path.style.access default false
* fs.s3a.access.key
* fs.s3a.secret.key
* fs.s3a.endpoint
* fs.s3a.endpoint.region
* fs.s3a.connection.ssl.enabled
* fs.s3a.retry.limit
* fs.s3a.retry.interval
*/
auto fs_options = filter_map_by_key_prefix(_options, "fs.");
auto provider = std::make_shared<FixedLocationProvider>(_tablet_root_path);
auto metadata_location = provider->tablet_metadata_location(_tablet_id, _version);
FORMAT_ASSIGN_OR_RAISE_ARROW_STATUS(auto fs, FileSystem::Create(metadata_location, FSOptions(fs_options)));
FORMAT_ASSIGN_OR_RAISE_ARROW_STATUS(auto metadata,
_lake_tablet_manager->get_tablet_metadata(metadata_location, true, 0, fs));
// get tablet schema;
_tablet_schema = std::make_shared<TabletSchema>(metadata->schema());
bool using_column_uid = false;
auto itr = _options.find(SR_FORMAT_USING_COLUMN_UID);
if (itr != _options.end() && !itr->second.empty()) {
using_column_uid = itr->second.compare("true");
}
// get scan column index from tablet schema.
std::vector<uint32_t> required_column_indexes;
ARROW_RETURN_NOT_OK(schema_to_column_index(_required_schema, required_column_indexes, using_column_uid));
// append key columns first.
for (size_t i = 0; i < _tablet_schema->num_key_columns(); i++) {
_scan_column_indexes.push_back(i);
}
for (auto index : required_column_indexes) {
if (!_tablet_schema->column(index).is_key()) {
_scan_column_indexes.push_back(index);
}
}
std::sort(_scan_column_indexes.begin(), _scan_column_indexes.end());
_scan_schema = std::make_shared<Schema>(ChunkHelper::convert_schema(_tablet_schema, _scan_column_indexes));
// create tablet reader
_tablet = std::make_unique<VersionedTablet>(_lake_tablet_manager, metadata);
FORMAT_ASSIGN_OR_RAISE_ARROW_STATUS(_tablet_reader, _tablet->new_reader(*_scan_schema));
// get output column index from tablet schema
std::vector<uint32_t> output_column_indexes;
ARROW_RETURN_NOT_OK(schema_to_column_index(_output_schema, output_column_indexes, using_column_uid));
// if scan columns not same as output column. we need project again after filter
if (std::equal(_scan_column_indexes.begin(), _scan_column_indexes.end(), output_column_indexes.begin(),
output_column_indexes.end())) {
_output_tablet_schema = _scan_schema;
} else {
_need_project = true;
_output_tablet_schema =
std::make_shared<Schema>(ChunkHelper::convert_schema(_tablet_schema, output_column_indexes));
ARROW_RETURN_NOT_OK(build_output_index_map(_output_tablet_schema, _scan_schema));
}
ARROW_ASSIGN_OR_RAISE(_arrow_converter,
ChunkToRecordBatchConverter::create(_output_tablet_schema, _output_schema,
arrow::default_memory_pool()));
ARROW_RETURN_NOT_OK(_arrow_converter->init());
TabletReaderParams read_params;
read_params.reader_type = READER_BYPASS_QUERY;
read_params.is_pipeline = false;
read_params.skip_aggregation = false;
read_params.chunk_size = _chunk_size;
read_params.use_page_cache = false;
read_params.lake_io_opts.fill_data_cache = false;
read_params.lake_io_opts.fs = fs;
read_params.lake_io_opts.location_provider = provider;
// parse query plan
auto query_plan_iter = _options.find(SR_FORMAT_QUERY_PLAN);
if (query_plan_iter != _options.end() && !query_plan_iter->second.empty()) {
_state = std::make_shared<RuntimeState>(TQueryGlobals());
ARROW_RETURN_NOT_OK(to_arrow_status(_state->init_query_global_dict(std::vector<TGlobalDict>())));
ARROW_RETURN_NOT_OK(to_arrow_status(parse_query_plan(query_plan_iter->second)));
ARROW_RETURN_NOT_OK(to_arrow_status(init_reader_params(read_params)));
}
ARROW_RETURN_NOT_OK(to_arrow_status(_tablet_reader->prepare()));
ARROW_RETURN_NOT_OK(to_arrow_status(_tablet_reader->open(read_params)));
return arrow::Status::OK();
}
arrow::Status get_next(ArrowArray* c_arrow_array) override {
auto chunk = ChunkHelper::new_chunk(*_output_tablet_schema, _chunk_size);
Status status = do_get_next(chunk);
std::shared_ptr<arrow::RecordBatch> record_batch;
if (status.ok()) {
ARROW_ASSIGN_OR_RAISE(record_batch, _arrow_converter->convert(chunk.get()))
} else if (status.is_end_of_file()) {
ARROW_ASSIGN_OR_RAISE(record_batch, arrow::RecordBatch::MakeEmpty(_output_schema));
} else {
return to_arrow_status(status);
}
if (record_batch) {
return ExportRecordBatch(*record_batch, c_arrow_array);
}
return arrow::Status::OK();
}
void close() override {
if (_tablet_reader) {
_tablet_reader->close();
}
if (_tablet_reader) {
_tablet_reader.reset();
}
_predicate_free_pool.clear();
}
private:
arrow::Status build_output_index_map(const std::shared_ptr<Schema>& output, const std::shared_ptr<Schema>& input) {
DCHECK(output);
DCHECK(input);
std::unordered_map<ColumnId, size_t> input_indexes;
for (size_t i = 0; i < input->num_fields(); i++) {
input_indexes[input->field(i)->id()] = i;
}
_index_map.resize(output->num_fields());
for (size_t i = 0; i < output->num_fields(); i++) {
if (input_indexes.count(output->field(i)->id()) == 0) {
return arrow::Status::Invalid("Output column ", output->field(i)->name(),
" isn't in scan column list.");
}
_index_map[i] = input_indexes[output->field(i)->id()];
}
return arrow::Status::OK();
}
arrow::Result<int32_t> get_column_id(const std::shared_ptr<arrow::Field>& field) {
auto metadata = field->metadata();
if (!field->HasMetadata() || !metadata->Contains(SR_FORMAT_COLUMN_ID)) {
return arrow::Status::Invalid("Missing arrow field metadata ", SR_FORMAT_COLUMN_ID);
}
try {
ARROW_ASSIGN_OR_RAISE(string str_unique_id, metadata->Get(SR_FORMAT_COLUMN_ID));
return std::stoi(str_unique_id);
} catch (const std::invalid_argument& e) {
return arrow::Status::Invalid("Invalid argument: ", e.what());
} catch (const std::out_of_range& e) {
return arrow::Status::Invalid("Out of range: ", e.what());
}
}
arrow::Status schema_to_column_index(const std::shared_ptr<arrow::Schema>& schema,
std::vector<uint32_t>& column_indexes, bool using_column_uid) {
std::stringstream ss;
for (int col_idx = 0; col_idx < schema->num_fields(); col_idx++) {
int32_t index = -1;
if (using_column_uid) {
ARROW_ASSIGN_OR_RAISE(int column_id, get_column_id(schema->field(col_idx)));
index = _tablet_schema->field_index(column_id);
if (index < 0) {
ss << "Invalid field(" << col_idx << ") unique id: " << column_id;
}
} else {
index = _tablet_schema->field_index(schema->field(col_idx)->name());
if (index < 0) {
ss << "Invalid field(" << col_idx << ") name: " << schema->field(col_idx)->name();
}
}
if (index < 0) {
return arrow::Status::Invalid(ss.str());
}
column_indexes.push_back(index);
}
return arrow::Status::OK();
}
Status parse_query_plan(std::string& encoded_query_plan) {
std::string query_plan_info;
if (!base64_decode(encoded_query_plan, &query_plan_info)) {
return Status::InvalidArgument("Decode query plan error: " + encoded_query_plan);
}
const auto* buf = (const uint8_t*)query_plan_info.data();
uint32_t len = query_plan_info.size();
// deserialize TQueryPlanInfo
TQueryPlanInfo t_query_plan_info;
RETURN_IF_ERROR(deserialize_thrift_msg(buf, &len, TProtocolType::BINARY, &t_query_plan_info));
TPlanNode* plan_node = nullptr;
for (auto& node : t_query_plan_info.plan_fragment.plan.nodes) {
if (node.node_type == TPlanNodeType::LAKE_SCAN_NODE) {
if (!plan_node) {
plan_node = &node;
} else {
return Status::InvalidArgument("There should be only one lake scan node in query plan!");
}
}
}
// There should be a lake scan plan node, because only one table in query plan.
if (!plan_node) {
return Status::InvalidArgument("There is no lake scan node in query plan!");
}
// get tuple descriptor
RETURN_IF_ERROR(DescriptorTbl::create(_state.get(), &_obj_pool, t_query_plan_info.desc_tbl, &_desc_tbl, 4096));
auto tuple_id = plan_node->lake_scan_node.tuple_id;
_tuple_desc = _desc_tbl->get_tuple_descriptor(tuple_id);
for (auto slot : _tuple_desc->slots()) {
DCHECK(slot->is_materialized());
int32_t index = _tablet_schema->field_index(slot->col_name());
if (index < 0) {
return Status::InternalError("Invalid field name: " + slot->col_name());
}
// set query slots for pushdown filter
auto itr = std::find(_scan_column_indexes.begin(), _scan_column_indexes.end(), index);
if (itr != _scan_column_indexes.end()) {
_query_slots.push_back(slot);
}
}
// get conjuncts
if (plan_node->__isset.conjuncts && plan_node->conjuncts.size() > 0) {
RETURN_IF_ERROR(Expr::create_expr_trees(&_obj_pool, plan_node->conjuncts, &_conjunct_ctxs, _state.get()));
RETURN_IF_ERROR(Expr::prepare(_conjunct_ctxs, _state.get()));
for (auto ctx : _conjunct_ctxs) {
Status status = ctx->open(_state.get());
}
RETURN_IF_ERROR(Expr::open(_conjunct_ctxs, _state.get()));
}
return Status::OK();
}
Status init_reader_params(TabletReaderParams& params) {
auto key_column_names = std::make_shared<std::vector<std::string>>();
if (PRIMARY_KEYS == _tablet_schema->keys_type() && _tablet_schema->sort_key_idxes().size() > 0) {
for (auto sort_key_index : _tablet_schema->sort_key_idxes()) {
TabletColumn col = _tablet_schema->column(sort_key_index);
key_column_names->emplace_back(col.name());
}
} else {
for (const auto& col : _tablet_schema->columns()) {
if (col.is_key()) {
key_column_names->emplace_back(col.name());
}
}
}
ScanConjunctsManagerOptions conjuncts_manager_opts = ScanConjunctsManagerOptions{
.conjunct_ctxs_ptr = &_conjunct_ctxs,
.tuple_desc = _tuple_desc,
.obj_pool = &_obj_pool,
.key_column_names = key_column_names.get(),
.runtime_state = _state.get(),
};
_conjuncts_manager = std::make_unique<ScanConjunctsManager>(std::move(conjuncts_manager_opts));
RETURN_IF_ERROR(_conjuncts_manager->parse_conjuncts());
auto parser = _obj_pool.add(new OlapPredicateParser(_tablet_schema));
std::vector<PredicatePtr> preds;
RETURN_IF_ERROR(_conjuncts_manager->get_predicate_tree(parser, preds));
PredicateAndNode and_node;
for (auto& p : preds) {
if (parser->can_pushdown(p.get())) {
and_node.add_child(PredicateColumnNode{p.get()});
} else {
_not_push_down_predicates.add(p.get());
}
_predicate_free_pool.emplace_back(std::move(p));
}
params.pred_tree = PredicateTree::create(std::move(and_node));
_conjuncts_manager->get_not_push_down_conjuncts(&_not_push_down_conjuncts);
std::vector<std::unique_ptr<OlapScanRange>> key_ranges;
RETURN_IF_ERROR(_conjuncts_manager->get_key_ranges(&key_ranges));
std::vector<OlapScanRange*> scanner_ranges;
int scanners_per_tablet = 64;
int num_ranges = key_ranges.size();
int ranges_per_scanner = std::max(1, num_ranges / scanners_per_tablet);
for (int i = 0; i < num_ranges;) {
scanner_ranges.push_back(key_ranges[i].get());
i++;
for (int j = 1; i < num_ranges && j < ranges_per_scanner &&
key_ranges[i]->end_include == key_ranges[i - 1]->end_include;
++j, ++i) {
scanner_ranges.push_back(key_ranges[i].get());
}
}
for (const auto& key_range : scanner_ranges) {
if (key_range->begin_scan_range.size() == 1 &&
key_range->begin_scan_range.get_value(0) == NEGATIVE_INFINITY) {
continue;
}
params.range = key_range->begin_include ? TabletReaderParams::RangeStartOperation::GE
: TabletReaderParams::RangeStartOperation::GT;
params.end_range = key_range->end_include ? TabletReaderParams::RangeEndOperation::LE
: TabletReaderParams::RangeEndOperation::LT;
params.start_key.push_back(key_range->begin_scan_range);
params.end_key.push_back(key_range->end_scan_range);
}
return Status::OK();
}
Status do_get_next(ChunkUniquePtr& chunk_ptr) {
auto* output_chunk = chunk_ptr.get();
if (!_scan_chunk) {
_scan_chunk = ChunkHelper::new_chunk(_tablet_reader->output_schema(), _chunk_size);
}
_scan_chunk->reset();
do {
RETURN_IF_ERROR(_tablet_reader->get_next(_scan_chunk.get()));
// If there is no filter, _query_slots will be empty.
for (auto slot : _query_slots) {
size_t column_index = _scan_chunk->schema()->get_field_index_by_name(slot->col_name());
_scan_chunk->set_slot_id_to_index(slot->id(), column_index);
}
if (!_not_push_down_predicates.empty()) {
// SCOPED_TIMER(_expr_filter_timer);
size_t row_num = _scan_chunk->num_rows();
_selection.clear();
_selection.resize(row_num);
auto status = _not_push_down_predicates.evaluate(_scan_chunk.get(), _selection.data(), 0, row_num);
_scan_chunk->filter(_selection);
DCHECK_CHUNK(_scan_chunk);
}
if (!_not_push_down_conjuncts.empty()) {
// SCOPED_TIMER(_expr_filter_timer);
auto status = ExecNode::eval_conjuncts(_not_push_down_conjuncts, _scan_chunk.get());
DCHECK_CHUNK(_scan_chunk.get());
}
if (_need_project) {
Columns& input_columns = _scan_chunk->columns();
for (size_t i = 0; i < _index_map.size(); i++) {
output_chunk->get_column_by_index(i).swap(input_columns[_index_map[i]]);
}
} else {
auto scan_chunk = _scan_chunk.get();
output_chunk->swap_chunk(*(scan_chunk));
}
} while (output_chunk->num_rows() == 0);
return Status::OK();
}
private:
int64_t _tablet_id;
int64_t _version;
std::shared_ptr<arrow::Schema> _required_schema;
std::shared_ptr<arrow::Schema> _output_schema;
std::string _tablet_root_path;
std::unordered_map<std::string, std::string> _options;
int32_t _chunk_size;
std::shared_ptr<TabletSchema> _tablet_schema;
std::unique_ptr<VersionedTablet> _tablet;
std::shared_ptr<TabletReader> _tablet_reader;
// internal tablet reader schema
std::vector<uint32_t> _scan_column_indexes;
std::unordered_set<uint32_t> _unused_output_column_ids;
std::shared_ptr<Schema> _scan_schema;
std::shared_ptr<Chunk> _scan_chunk;
// format reader output schema
std::shared_ptr<Schema> _output_tablet_schema;
// mapping from index of column in output chunk to index of column in input chunk.
std::vector<size_t> _index_map;
// need choose select columns when scan schema are not same as output schema
bool _need_project = false;
// filter pushdown use the vars
std::shared_ptr<RuntimeState> _state;
ObjectPool _obj_pool;
// _desc_tbl, tuple_desc, _query_slots, _conjunct_ctxs memory are maintained by _obj_pool
DescriptorTbl* _desc_tbl = nullptr;
TupleDescriptor* _tuple_desc = nullptr;
// slot descriptors for each one of |output_columns|. used by _not_push_down_conjuncts.
std::vector<SlotDescriptor*> _query_slots;
std::vector<ExprContext*> _conjunct_ctxs;
std::unique_ptr<ScanConjunctsManager> _conjuncts_manager = nullptr;
using PredicatePtr = std::unique_ptr<ColumnPredicate>;
// The conjuncts couldn't push down to storage engine
std::vector<ExprContext*> _not_push_down_conjuncts;
ConjunctivePredicates _not_push_down_predicates;
std::vector<PredicatePtr> _predicate_free_pool;
Buffer<uint8_t> _selection;
std::shared_ptr<ChunkToRecordBatchConverter> _arrow_converter;
};
arrow::Result<StarRocksFormatReader*> StarRocksFormatReader::create(
int64_t tablet_id, std::string tablet_root_path, int64_t version, ArrowSchema* required_arrow_schema,
ArrowSchema* output_arrow_schema, std::unordered_map<std::string, std::string> options) {
ARROW_ASSIGN_OR_RAISE(std::shared_ptr<arrow::Schema> required_schema, arrow::ImportSchema(required_arrow_schema));
ARROW_ASSIGN_OR_RAISE(std::shared_ptr<arrow::Schema> output_schema, arrow::ImportSchema(output_arrow_schema));
return create(tablet_id, std::move(tablet_root_path), version, required_schema, output_schema, std::move(options));
}
arrow::Result<StarRocksFormatReader*> StarRocksFormatReader::create(
int64_t tablet_id, std::string tablet_root_path, int64_t version,
std::shared_ptr<arrow::Schema> required_schema, std::shared_ptr<arrow::Schema> output_schema,
std::unordered_map<std::string, std::string> options) {
StarRocksFormatReaderImpl* format_Reader = new StarRocksFormatReaderImpl(
tablet_id, std::move(tablet_root_path), version, required_schema, output_schema, std::move(options));
return format_Reader;
}
} // namespace starrocks::lake::format

View File

@ -0,0 +1,92 @@
// 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 <arrow/array.h>
#include <arrow/c/abi.h>
#include <arrow/result.h>
#include <arrow/type.h>
namespace starrocks::lake::format {
/**
* StarRocksFormatReader is a tablet file reader, which is used to bypass BE Server to read tablet files that stored
* on the file systems (e.g. S3, HDFS) in share-data mode. These data files can be written by StarRocks or by the
* StarRocksFormatWriter provided by this SDK.
*
* StarRocksFormatReader can be regarded as a wrapper of StarRocks TabletReader, and all file operation methods will
* eventually be mapped to the corresponding methods in TabletReader.
*
* You can refer to the following examples for use:
* @code
* // Create and open Reader
* auto&& result = StarRocksFormatReader::create(...);
* StarRocksFormatReader* reader = std::move(result).ValueUnsafe();
* reader->open();
*
* // Read tablet data iteratively
* reader->get_next(arrow_array);
*
* // Close Reader
* reader->close();
*/
class StarRocksFormatReader {
public:
static arrow::Result<StarRocksFormatReader*> create(int64_t tablet_id, std::string tablet_root_path,
int64_t version, ArrowSchema* required_schema,
ArrowSchema* output_schema,
std::unordered_map<std::string, std::string> options);
/**
* Create a starrocks format reader instance, which bound to a specific tablet.
*
* @param tablet_id The target tablet id.
* @param tablet_root_path The target tablet root path.
* @param version The target tablet version.
* @param required_schema The required schema, also contains the union of output fields and pushdown filter fields.
* @param output_schema The output fields.
* @param options key-value parameters, e.g. S3 connection authentication configuration.
*/
static arrow::Result<StarRocksFormatReader*> create(int64_t tablet_id, std::string tablet_root_path,
int64_t version, std::shared_ptr<arrow::Schema> required_schema,
std::shared_ptr<arrow::Schema> output_schema,
std::unordered_map<std::string, std::string> options);
virtual ~StarRocksFormatReader() = default;
/**
* Open the format reader, the StarRocks TabletReader will be initialized and opened during this time.
*/
virtual arrow::Status open() = 0;
/**
* Get the starrocks chunk via TabletReader, and convert it to arrow array by column converter.
*/
virtual arrow::Status get_next(ArrowArray* c_arrow_array) = 0;
/**
* Close this reader, the TabletReader will be closed and reset during this time.
*/
virtual void close() = 0;
public:
StarRocksFormatReader(StarRocksFormatReader&&) = delete;
StarRocksFormatReader& operator=(StarRocksFormatReader&&) = delete;
protected:
StarRocksFormatReader() = default;
};
} // namespace starrocks::lake::format

View File

@ -20,6 +20,7 @@
// project dependencies
#include "convert/starrocks_arrow_converter.h"
#include "fmt/format.h"
#include "format_utils.h"
#include "starrocks_format_writer.h"
@ -129,7 +130,7 @@ private:
} else if (is_del(f.path)) {
op_write->add_dels(std::move(f.path));
} else {
return Status::InternalError(fmt::format("unknown file {}", f.path));
return Status::InternalError(fmt::format("Unknown file {}", f.path));
}
}
op_write->mutable_rowset()->set_num_rows(_tablet_writer->num_rows());
@ -140,10 +141,10 @@ private:
Status save_txn_log(const TxnLogPtr& log) {
if (UNLIKELY(!log->has_tablet_id())) {
return Status::InvalidArgument("missing tablet id in txn log");
return Status::InvalidArgument("Missing tablet id in txn log");
}
if (UNLIKELY(!log->has_txn_id())) {
return Status::InvalidArgument("missing id in txn log");
return Status::InvalidArgument("Missing id in txn log");
}
auto txn_log_path = _loc_provider->txn_log_location(log->tablet_id(), log->txn_id());
@ -158,7 +159,6 @@ private:
// TODO: construct prefix in LocationProvider
std::string prefix = fmt::format("{:016X}_", _tablet_id);
auto root = _loc_provider->metadata_root_location(_tablet_id);
auto scan_cb = [&](std::string_view name) {
if (HasPrefixString(name, prefix)) {
objects.emplace_back(join_path(root, name));
@ -195,17 +195,17 @@ private:
/* static methods */
arrow::Result<StarRocksFormatWriter*> StarRocksFormatWriter::create(
int64_t tablet_id, const std::string& tablet_root_path, int64_t txn_id, const ArrowSchema* output_arrow_schema,
const std::unordered_map<std::string, std::string>& options) {
int64_t tablet_id, const std::string tablet_root_path, int64_t txn_id, const ArrowSchema* output_arrow_schema,
const std::unordered_map<std::string, std::string> options) {
ARROW_ASSIGN_OR_RAISE(std::shared_ptr<arrow::Schema> output_schema,
arrow::ImportSchema(const_cast<struct ArrowSchema*>(output_arrow_schema)));
return create(tablet_id, std::move(tablet_root_path), txn_id, std::move(output_schema), std::move(options));
}
arrow::Result<StarRocksFormatWriter*> StarRocksFormatWriter::create(
int64_t tablet_id, const std::string& tablet_root_path, int64_t txn_id,
const std::shared_ptr<arrow::Schema>& output_schema,
const std::unordered_map<std::string, std::string>& options) {
int64_t tablet_id, const std::string tablet_root_path, int64_t txn_id,
const std::shared_ptr<arrow::Schema> output_schema,
const std::unordered_map<std::string, std::string> options) {
StarRocksFormatWriterImpl* format_writer = new StarRocksFormatWriterImpl(
tablet_id, std::move(tablet_root_path), txn_id, std::move(output_schema), std::move(options));
return format_writer;

View File

@ -14,6 +14,7 @@
#pragma once
#include <arrow/array.h>
#include <arrow/c/abi.h>
#include <arrow/c/bridge.h>
#include <arrow/result.h>
#include <arrow/type.h>
@ -51,9 +52,9 @@ namespace starrocks::lake::format {
*/
class StarRocksFormatWriter {
public:
static arrow::Result<StarRocksFormatWriter*> create(int64_t tablet_id, const std::string& tablet_root_path,
int64_t txn_id, const ArrowSchema* output_schema,
const std::unordered_map<std::string, std::string>& options);
static arrow::Result<StarRocksFormatWriter*> create(int64_t tablet_id, std::string tablet_root_path, int64_t txn_id,
const ArrowSchema* output_schema,
const std::unordered_map<std::string, std::string> options);
/**
* Create a starrocks format writer instance, which bound to a specific tablet.
@ -64,10 +65,9 @@ public:
* @param output_schema The table schema that the target tablet belongs.
* @param options key-value parameters, e.g. S3 connection authentication configuration.
*/
static arrow::Result<StarRocksFormatWriter*> create(int64_t tablet_id, const std::string& tablet_root_path,
int64_t txn_id,
const std::shared_ptr<arrow::Schema>& output_schema,
const std::unordered_map<std::string, std::string>& options);
static arrow::Result<StarRocksFormatWriter*> create(int64_t tablet_id, std::string tablet_root_path, int64_t txn_id,
const std::shared_ptr<arrow::Schema> output_schema,
const std::unordered_map<std::string, std::string> options);
virtual ~StarRocksFormatWriter() = default;

View File

@ -20,7 +20,18 @@
#include <unordered_map>
#include <vector>
#define SAFE_CALL_WRITER_FUNCATION(writer, body) \
#define SAFE_CALL_READER_FUNCTION(reader, body) \
if (reader != nullptr) { \
try { \
body; \
} catch (const std::exception& ex) { \
env->ThrowNew(kNativeOptExceptionClass, ex.what()); \
} \
} else { \
env->ThrowNew(kNativeOptExceptionClass, "Invalid tablet reader handler!"); \
}
#define SAFE_CALL_WRITER_FUNCTION(writer, body) \
if (writer != nullptr) { \
try { \
body; \

View File

@ -17,6 +17,7 @@
#include <string>
#include "format/starrocks_format_reader.h"
#include "format/starrocks_format_writer.h"
#include "jni_utils.h"
#include "starrocks_format/starrocks_lib.h"
@ -83,21 +84,20 @@ JNIEXPORT jlong JNICALL Java_com_starrocks_format_StarRocksWriter_createNativeWr
return reinterpret_cast<int64_t>(format_writer);
}
JNIEXPORT jlong JNICALL Java_com_starrocks_format_StarRocksWriter_nativeOpen(JNIEnv* env, jobject jobj, jlong handler) {
JNIEXPORT void JNICALL Java_com_starrocks_format_StarRocksWriter_nativeOpen(JNIEnv* env, jobject jobj, jlong handler) {
StarRocksFormatWriter* format_writer = reinterpret_cast<StarRocksFormatWriter*>(handler);
SAFE_CALL_WRITER_FUNCATION(format_writer, {
SAFE_CALL_WRITER_FUNCTION(format_writer, {
arrow::Status st = format_writer->open();
if (!st.ok()) {
env->ThrowNew(kNativeOptExceptionClass, st.message().c_str());
}
});
return 0;
}
JNIEXPORT jlong JNICALL Java_com_starrocks_format_StarRocksWriter_nativeWrite(JNIEnv* env, jobject jobj, jlong handler,
jlong jArrowArray) {
JNIEXPORT void JNICALL Java_com_starrocks_format_StarRocksWriter_nativeWrite(JNIEnv* env, jobject jobj, jlong handler,
jlong jArrowArray) {
StarRocksFormatWriter* format_writer = reinterpret_cast<StarRocksFormatWriter*>(handler);
SAFE_CALL_WRITER_FUNCATION(format_writer, {
SAFE_CALL_WRITER_FUNCTION(format_writer, {
const ArrowArray* c_array_import = reinterpret_cast<struct ArrowArray*>(jArrowArray);
if (c_array_import != nullptr) {
arrow::Status st = format_writer->write(c_array_import);
@ -106,49 +106,109 @@ JNIEXPORT jlong JNICALL Java_com_starrocks_format_StarRocksWriter_nativeWrite(JN
}
}
});
return 0;
}
JNIEXPORT jlong JNICALL Java_com_starrocks_format_StarRocksWriter_nativeFlush(JNIEnv* env, jobject jobj,
jlong handler) {
JNIEXPORT void JNICALL Java_com_starrocks_format_StarRocksWriter_nativeFlush(JNIEnv* env, jobject jobj, jlong handler) {
StarRocksFormatWriter* format_writer = reinterpret_cast<StarRocksFormatWriter*>(handler);
SAFE_CALL_WRITER_FUNCATION(format_writer, {
SAFE_CALL_WRITER_FUNCTION(format_writer, {
arrow::Status st = format_writer->flush();
if (!st.ok()) {
env->ThrowNew(kNativeOptExceptionClass, st.message().c_str());
}
});
return 0;
}
JNIEXPORT jlong JNICALL Java_com_starrocks_format_StarRocksWriter_nativeFinish(JNIEnv* env, jobject jobj,
jlong handler) {
JNIEXPORT void JNICALL Java_com_starrocks_format_StarRocksWriter_nativeFinish(JNIEnv* env, jobject jobj,
jlong handler) {
StarRocksFormatWriter* format_writer = reinterpret_cast<StarRocksFormatWriter*>(handler);
SAFE_CALL_WRITER_FUNCATION(format_writer, {
SAFE_CALL_WRITER_FUNCTION(format_writer, {
arrow::Status st = format_writer->finish();
if (!st.ok()) {
env->ThrowNew(kNativeOptExceptionClass, st.message().c_str());
}
});
return 0;
}
JNIEXPORT jlong JNICALL Java_com_starrocks_format_StarRocksWriter_nativeClose(JNIEnv* env, jobject jobj,
jlong handler) {
JNIEXPORT void JNICALL Java_com_starrocks_format_StarRocksWriter_nativeClose(JNIEnv* env, jobject jobj, jlong handler) {
StarRocksFormatWriter* format_writer = reinterpret_cast<StarRocksFormatWriter*>(handler);
SAFE_CALL_WRITER_FUNCATION(format_writer, { format_writer->close(); });
return 0;
SAFE_CALL_WRITER_FUNCTION(format_writer, { format_writer->close(); });
}
JNIEXPORT void JNICALL Java_com_starrocks_format_jni_LibraryHelper_releaseWriter(JNIEnv* env, jobject jobj,
jlong writerAddress) {
JNIEXPORT void JNICALL Java_com_starrocks_format_StarRocksWriter_nativeRelease(JNIEnv* env, jobject jobj,
jlong writerAddress) {
StarRocksFormatWriter* tablet_writer = reinterpret_cast<StarRocksFormatWriter*>(writerAddress);
if (tablet_writer != nullptr) {
delete tablet_writer;
}
}
/* reader functions */
JNIEXPORT jlong JNICALL Java_com_starrocks_format_StarRocksReader_createNativeReader(
JNIEnv* env, jobject jobj, jlong jtablet_id, jlong jversion, jlong jrequired_schema, jlong joutput_schema,
jstring jtable_root_path, jobject joptions) {
int64_t tablet_id = jtablet_id;
int64_t version = jversion;
// get schema
if (jrequired_schema == 0) {
env->ThrowNew(kNativeOptExceptionClass, "required_schema should not be null");
return 0;
}
if (joutput_schema == 0) {
env->ThrowNew(kNativeOptExceptionClass, "output_schema should not be null");
return 0;
}
std::string table_root_path = jstring_to_cstring(env, jtable_root_path);
std::unordered_map<std::string, std::string> options = jhashmap_to_cmap(env, joptions);
auto&& result = StarRocksFormatReader::create(
tablet_id, std::move(table_root_path), version, reinterpret_cast<struct ArrowSchema*>(jrequired_schema),
reinterpret_cast<struct ArrowSchema*>(joutput_schema), std::move(options));
if (!result.ok()) {
env->ThrowNew(kNativeOptExceptionClass, result.status().message().c_str());
return 0;
}
StarRocksFormatReader* format_reader = std::move(result).ValueUnsafe();
return reinterpret_cast<int64_t>(format_reader);
}
JNIEXPORT void JNICALL Java_com_starrocks_format_StarRocksReader_nativeOpen(JNIEnv* env, jobject jobj, jlong handler) {
StarRocksFormatReader* format_reader = reinterpret_cast<StarRocksFormatReader*>(handler);
SAFE_CALL_READER_FUNCTION(format_reader, {
arrow::Status st = format_reader->open();
if (!st.ok()) {
env->ThrowNew(kNativeOptExceptionClass, st.message().c_str());
}
});
}
JNIEXPORT void JNICALL Java_com_starrocks_format_StarRocksReader_nativeGetNext(JNIEnv* env, jobject jobj, jlong handler,
jlong jArrowArray) {
StarRocksFormatReader* format_reader = reinterpret_cast<StarRocksFormatReader*>(handler);
SAFE_CALL_READER_FUNCTION(format_reader, {
ArrowArray* c_array_export = reinterpret_cast<struct ArrowArray*>(jArrowArray);
arrow::Status st = format_reader->get_next(c_array_export);
if (!st.ok()) {
env->ThrowNew(kNativeOptExceptionClass, st.message().c_str());
}
});
}
JNIEXPORT void JNICALL Java_com_starrocks_format_StarRocksReader_nativeClose(JNIEnv* env, jobject jobj, jlong handler) {
StarRocksFormatReader* format_reader = reinterpret_cast<StarRocksFormatReader*>(handler);
SAFE_CALL_READER_FUNCTION(format_reader, { format_reader->close(); });
}
JNIEXPORT void JNICALL Java_com_starrocks_format_StarRocksReader_nativeRelease(JNIEnv* env, jobject jobj,
jlong chunkAddress) {
StarRocksFormatReader* format_reader = reinterpret_cast<StarRocksFormatReader*>(chunkAddress);
if (format_reader != nullptr) {
delete format_reader;
}
}
#ifdef __cplusplus
}
#endif

View File

@ -25,4 +25,6 @@ constexpr const char* SR_FORMAT_USING_COLUMN_UID = "starrocks.format.using_colum
constexpr const char* SR_FORMAT_QUERY_PLAN = "starrocks.format.query_plan";
constexpr const char* SR_FORMAT_COLUMN_ID = "starrocks.format.column.id";
} // namespace starrocks::lake::format

View File

@ -0,0 +1,340 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
package com.starrocks.format;
import org.apache.commons.collections4.MapUtils;
import java.lang.annotation.ElementType;
import java.lang.annotation.Inherited;
import java.lang.annotation.Retention;
import java.lang.annotation.RetentionPolicy;
import java.lang.annotation.Target;
import java.lang.reflect.Field;
import java.util.HashMap;
import java.util.Map;
import java.util.Objects;
public class Config {
@Inherited
@Target({ElementType.FIELD, ElementType.METHOD})
@Retention(RetentionPolicy.RUNTIME)
public @interface Option {
String value();
}
@Option("starrocks.fe.http.url")
private final String feHttpUrl;
@Option("starrocks.fe.jdbc.url")
private final String feJdbcUrl;
@Option("starrocks.database")
private final String database;
@Option("starrocks.user")
private final String username;
@Option("starrocks.password")
private final String password;
@Option("starrocks.format.query_plan")
private final String queryPlan;
@Option("starrocks.format.using_column_uid")
private final Boolean usingColumnUid;
@Option("starrocks.format.unreleased.warning.threshold")
private final Integer unreleasedWarningThreshold;
/* S3 Options */
@Option("fs.s3a.endpoint")
private final String s3Endpoint;
@Option("fs.s3a.endpoint.region")
private final String s3EndpointRegion;
@Option("fs.s3a.connection.ssl.enabled")
private final Boolean s3ConnectionSslEnabled;
@Option("fs.s3a.path.style.access")
private final Boolean s3PathStyleAccess;
@Option("fs.s3a.access.key")
private final String s3AccessKey;
@Option("fs.s3a.secret.key")
private final String s3SecretKey;
private final Map<String, Object> customConfigs;
private Config(Builder builder) {
this.feHttpUrl = builder.feHttpUrl;
this.feJdbcUrl = builder.feJdbcUrl;
this.database = builder.database;
this.username = builder.username;
this.password = builder.password;
this.queryPlan = builder.queryPlan;
this.usingColumnUid = builder.usingColumnUid;
this.unreleasedWarningThreshold = builder.unreleasedWarningThreshold;
this.s3Endpoint = builder.s3Endpoint;
this.s3EndpointRegion = builder.s3EndpointRegion;
this.s3ConnectionSslEnabled = builder.s3ConnectionSslEnabled;
this.s3PathStyleAccess = builder.s3PathStyleAccess;
this.s3AccessKey = builder.s3AccessKey;
this.s3SecretKey = builder.s3SecretKey;
this.customConfigs = new HashMap<>(builder.customConfigs);
}
public static Builder newBuilder() {
return new Builder();
}
public Builder toBuilder() {
return new Builder()
.feHttpUrl(feHttpUrl)
.feJdbcUrl(feJdbcUrl)
.database(database)
.username(username)
.password(password)
.queryPlan(queryPlan)
.usingColumnUid(usingColumnUid)
.unreleasedWarningThreshold(unreleasedWarningThreshold)
.s3Endpoint(s3Endpoint)
.s3EndpointRegion(s3EndpointRegion)
.s3ConnectionSslEnabled(s3ConnectionSslEnabled)
.s3PathStyleAccess(s3PathStyleAccess)
.s3AccessKey(s3AccessKey)
.s3SecretKey(s3SecretKey);
}
/**
* Config Builder.
*/
public static class Builder {
private String feHttpUrl;
private String feJdbcUrl;
private String database;
private String username;
private String password;
private String queryPlan;
private Boolean usingColumnUid;
private Integer unreleasedWarningThreshold = 128;
private String s3Endpoint;
private String s3EndpointRegion;
private Boolean s3ConnectionSslEnabled;
private Boolean s3PathStyleAccess;
private String s3AccessKey;
private String s3SecretKey;
private final Map<String, Object> customConfigs = new HashMap<>();
private Builder() {
}
public Builder feHttpUrl(String feHttpUrl) {
this.feHttpUrl = feHttpUrl;
return this;
}
public Builder feJdbcUrl(String feJdbcUrl) {
this.feJdbcUrl = feJdbcUrl;
return this;
}
public Builder database(String database) {
this.database = database;
return this;
}
public Builder username(String username) {
this.username = username;
return this;
}
public Builder password(String password) {
this.password = password;
return this;
}
public Builder queryPlan(String queryPlan) {
this.queryPlan = queryPlan;
return this;
}
public Builder usingColumnUid(Boolean usingColumnUid) {
this.usingColumnUid = usingColumnUid;
return this;
}
public Builder unreleasedWarningThreshold(Integer unreleasedWarningThreshold) {
this.unreleasedWarningThreshold = unreleasedWarningThreshold;
return this;
}
public Builder s3Endpoint(String s3Endpoint) {
this.s3Endpoint = s3Endpoint;
return this;
}
public Builder s3EndpointRegion(String s3EndpointRegion) {
this.s3EndpointRegion = s3EndpointRegion;
return this;
}
public Builder s3ConnectionSslEnabled(Boolean s3ConnectionSslEnabled) {
this.s3ConnectionSslEnabled = s3ConnectionSslEnabled;
return this;
}
public Builder s3PathStyleAccess(Boolean s3PathStyleAccess) {
this.s3PathStyleAccess = s3PathStyleAccess;
return this;
}
public Builder s3AccessKey(String s3AccessKey) {
this.s3AccessKey = s3AccessKey;
return this;
}
public Builder s3SecretKey(String s3SecretKey) {
this.s3SecretKey = s3SecretKey;
return this;
}
public Builder customConfig(String name, Object value) {
this.customConfigs.put(name, value);
return this;
}
public Config build() {
return new Config(this);
}
}
/**
* Convert to {@link Map} object.
*/
public Map<String, String> toMap() {
Map<String, String> mapping = new HashMap<>();
Field[] fields = this.getClass().getDeclaredFields();
for (Field field : fields) {
if (!field.isAnnotationPresent(Option.class)) {
continue;
}
Option option = field.getAnnotation(Option.class);
String propName = option.value();
try {
Object propValue = field.get(this);
if (null != propValue) {
mapping.put(propName, Objects.toString(propValue));
}
} catch (IllegalAccessException e) {
throw new IllegalStateException("Resolve option error: " + propName, e);
}
}
if (MapUtils.isNotEmpty(this.customConfigs)) {
for (Map.Entry<String, Object> entry : this.customConfigs.entrySet()) {
if (mapping.containsKey(entry.getKey())) {
throw new IllegalStateException("Duplicate config: " + entry.getKey());
}
mapping.put(entry.getKey(), Objects.toString(entry.getValue()));
}
}
return mapping;
}
/* getters */
public String getFeHttpUrl() {
return feHttpUrl;
}
public String getFeJdbcUrl() {
return feJdbcUrl;
}
public String getDatabase() {
return database;
}
public String getUsername() {
return username;
}
public String getPassword() {
return password;
}
public String getQueryPlan() {
return queryPlan;
}
public Boolean getUsingColumnUid() {
return usingColumnUid;
}
public Integer getUnreleasedWarningThreshold() {
return unreleasedWarningThreshold;
}
public String getS3Endpoint() {
return s3Endpoint;
}
public String getS3EndpointRegion() {
return s3EndpointRegion;
}
public Boolean getS3ConnectionSslEnabled() {
return s3ConnectionSslEnabled;
}
public Boolean getS3PathStyleAccess() {
return s3PathStyleAccess;
}
public String getS3AccessKey() {
return s3AccessKey;
}
public String getS3SecretKey() {
return s3SecretKey;
}
}

View File

@ -0,0 +1,104 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
package com.starrocks.format;
import com.starrocks.format.jni.LibraryHelper;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
public abstract class DataAccessor implements AutoCloseable {
protected final Logger log = LoggerFactory.getLogger(this.getClass());
static {
LibraryHelper.load();
}
protected static final AtomicInteger UNRELEASED_COUNTER = new AtomicInteger(0);
protected final AtomicBoolean released = new AtomicBoolean(false);
/**
* Native read or write pointer.
*/
protected long nativePointer = 0L;
protected final void release() {
if (released.compareAndSet(false, true)) {
this.doRelease(nativePointer);
nativePointer = 0L;
UNRELEASED_COUNTER.decrementAndGet();
}
}
protected void doRelease(long nativePointer) {
}
@FunctionalInterface
protected interface ThrowingProcedure<E extends Exception> {
void invoke() throws E;
}
@FunctionalInterface
protected interface ThrowingSupplier<T, E extends Exception> {
T get() throws E;
}
protected <E extends Exception> void checkAndDo(ThrowingProcedure<E> supplier) throws E {
if (0 == nativePointer) {
throw new IllegalStateException("Native reader or writer may not be created correctly.");
}
if (released.get()) {
throw new IllegalStateException("Native reader or writer is released.");
}
supplier.invoke();
}
protected <T, E extends Exception> T checkAndDo(ThrowingSupplier<T, E> supplier) throws E {
if (0 == nativePointer) {
throw new IllegalStateException("Native reader or writer may not be created correctly.");
}
if (released.get()) {
throw new IllegalStateException("Native reader or writer is released.");
}
return supplier.get();
}
protected void checkUnreleasedInstances(int unreleasedWarningThreshold) {
if (UNRELEASED_COUNTER.incrementAndGet() >= unreleasedWarningThreshold) {
log.warn("Found more than {} unreleased instances. Did you forget to call the release method when exiting {}? " +
"If this is expected, you can turn off this warning log by increasing the '{}' parameter.",
UNRELEASED_COUNTER.get(),
this.getClass().getCanonicalName(),
"starrocks.format.unreleased.warning.threshold");
}
}
}

View File

@ -0,0 +1,132 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
package com.starrocks.format;
import com.starrocks.format.jni.NativeOperateException;
import org.apache.arrow.c.ArrowArray;
import org.apache.arrow.c.ArrowSchema;
import org.apache.arrow.c.Data;
import org.apache.arrow.memory.BufferAllocator;
import org.apache.arrow.memory.RootAllocator;
import org.apache.arrow.vector.VectorSchemaRoot;
import org.apache.arrow.vector.types.pojo.Schema;
import java.util.Iterator;
import java.util.Map;
import java.util.Optional;
import static java.util.Objects.requireNonNull;
public class StarRocksReader extends DataAccessor implements Iterator<VectorSchemaRoot> {
private final BufferAllocator allocator;
private final Schema requiredSchema;
private final Schema outputSchema;
private VectorSchemaRoot data;
public StarRocksReader(long tabletId,
String tabletRootPath,
long version,
Schema requiredSchema,
Schema outputSchema,
Config config) {
this.allocator = new RootAllocator();
this.requiredSchema = requireNonNull(requiredSchema, "Null required schema");
if (null == requiredSchema.getFields() || requiredSchema.getFields().isEmpty()) {
throw new IllegalArgumentException("Empty required schema fields");
}
this.outputSchema = requireNonNull(outputSchema, "Null output schema");
ArrowSchema requiredArrowSchema = ArrowSchema.allocateNew(allocator);
Data.exportSchema(allocator, this.requiredSchema, null, requiredArrowSchema);
ArrowSchema outputArrowSchema = ArrowSchema.allocateNew(allocator);
Data.exportSchema(allocator, this.outputSchema, null, outputArrowSchema);
this.nativePointer = createNativeReader(
tabletId,
version,
requiredArrowSchema.memoryAddress(),
outputArrowSchema.memoryAddress(),
tabletRootPath,
config.toMap()
);
Optional.ofNullable(config.getUnreleasedWarningThreshold())
.ifPresent(this::checkUnreleasedInstances);
}
public void open() {
checkAndDo(() -> nativeOpen(nativePointer));
}
@Override
public boolean hasNext() {
data = checkAndDo(() -> {
VectorSchemaRoot root = VectorSchemaRoot.create(outputSchema, allocator);
try (ArrowArray array = ArrowArray.allocateNew(allocator)) {
try {
nativeGetNext(nativePointer, array.memoryAddress());
Data.importIntoVectorSchemaRoot(allocator, array, root, null);
} catch (Exception e) {
array.release();
throw new NativeOperateException(e.getMessage(), e);
}
}
return root;
});
return null != data && data.getRowCount() > 0;
}
@Override
public VectorSchemaRoot next() {
return checkAndDo(() -> data);
}
@Override
public void close() throws Exception {
checkAndDo(() -> nativeClose(nativePointer));
this.release();
}
@Override
protected void doRelease(long nativePointer) {
nativeRelease(nativePointer);
}
/* native methods */
public native long createNativeReader(long tabletId,
long version,
long requiredArrowSchemaAddr,
long outputArrowSchemaAddr,
String tableRootPath,
Map<String, String> options);
public native void nativeOpen(long nativePointer);
public native void nativeGetNext(long nativePointer, long arrowArray);
public native void nativeClose(long nativePointer);
public native void nativeRelease(long nativePointer);
}

View File

@ -17,7 +17,6 @@
package com.starrocks.format;
import com.starrocks.format.jni.LibraryHelper;
import org.apache.arrow.c.ArrowArray;
import org.apache.arrow.c.ArrowSchema;
import org.apache.arrow.c.Data;
@ -27,20 +26,11 @@ import org.apache.arrow.vector.VectorSchemaRoot;
import org.apache.arrow.vector.types.pojo.Schema;
import java.util.Map;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.Optional;
import static java.util.Objects.requireNonNull;
public class StarRocksWriter implements AutoCloseable {
private static final LibraryHelper LIB_HELPER = new LibraryHelper();
/**
* The c++ StarRocksFormatWriter pointer
*/
private long nativeWriterPoint = 0L;
private final AtomicBoolean released = new AtomicBoolean(false);
public class StarRocksWriter extends DataAccessor {
private final BufferAllocator allocator;
private final Schema schema;
@ -49,7 +39,7 @@ public class StarRocksWriter implements AutoCloseable {
String tabletRootPath,
long txnId,
Schema schema,
Map<String, String> config) {
Config config) {
this.allocator = new RootAllocator();
this.schema = requireNonNull(schema, "Null schema.");
if (null == schema.getFields() || schema.getFields().isEmpty()) {
@ -59,63 +49,54 @@ public class StarRocksWriter implements AutoCloseable {
ArrowSchema arrowSchema = ArrowSchema.allocateNew(allocator);
Data.exportSchema(allocator, schema, null, arrowSchema);
this.nativeWriterPoint = createNativeWriter(
this.nativePointer = createNativeWriter(
tabletId,
txnId,
arrowSchema.memoryAddress(),
requireNonNull(tabletRootPath, "Null tablet root path."),
requireNonNull(config, "Null config."));
requireNonNull(config, "Null config.").toMap()
);
Optional.ofNullable(config.getUnreleasedWarningThreshold())
.ifPresent(this::checkUnreleasedInstances);
}
public void open() {
checkAndDo(() -> nativeOpen(nativeWriterPoint));
checkAndDo(() -> nativeOpen(nativePointer));
}
public long write(VectorSchemaRoot root) {
return checkAndDo(() -> {
public void write(VectorSchemaRoot root) {
checkAndDo(() -> {
try (ArrowArray array = ArrowArray.allocateNew(allocator)) {
Data.exportVectorSchemaRoot(allocator, root, null, array);
return nativeWrite(nativeWriterPoint, array.memoryAddress());
nativeWrite(nativePointer, array.memoryAddress());
}
});
}
public long flush() {
return checkAndDo(() -> nativeFlush(nativeWriterPoint));
public void flush() {
checkAndDo(() -> nativeFlush(nativePointer));
}
public long finish() {
return checkAndDo(() -> nativeFinish(nativeWriterPoint));
public void finish() {
checkAndDo(() -> nativeFinish(nativePointer));
}
@Override
public void close() throws Exception {
checkAndDo(() -> nativeClose(nativeWriterPoint));
checkAndDo(() -> nativeClose(nativePointer));
this.release();
}
public void release() {
if (released.compareAndSet(false, true)) {
LIB_HELPER.releaseWriter(nativeWriterPoint);
nativeWriterPoint = 0L;
}
@Override
protected void doRelease(long nativePointer) {
nativeRelease(nativePointer);
}
public BufferAllocator getAllocator() {
return allocator;
}
private <T, E extends Exception> T checkAndDo(ThrowingSupplier<T, E> supplier) throws E {
if (0 == nativeWriterPoint) {
throw new IllegalStateException("Native writer may not be created correctly.");
}
if (released.get()) {
throw new IllegalStateException("Native writer is released.");
}
return supplier.get();
}
/* native methods */
public native long createNativeWriter(long tabletId,
@ -124,14 +105,16 @@ public class StarRocksWriter implements AutoCloseable {
String tableRootPath,
Map<String, String> options);
public native long nativeOpen(long nativeWriter);
public native void nativeOpen(long nativePointer);
public native long nativeWrite(long nativeWriter, long arrowArray);
public native void nativeWrite(long nativePointer, long arrowArray);
public native long nativeFlush(long nativeWriter);
public native void nativeFlush(long nativePointer);
public native long nativeFinish(long nativeWriter);
public native void nativeFinish(long nativePointer);
public native long nativeClose(long nativeWriter);
public native void nativeClose(long nativePointer);
public native void nativeRelease(long nativePointer);
}

View File

@ -1,22 +0,0 @@
// 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.format;
@FunctionalInterface
public interface ThrowingSupplier<T, E extends Exception> {
T get() throws E;
}

View File

@ -41,21 +41,13 @@ public class LibraryHelper {
public static final String PROP_FORMAT_LIB_NAME = "com.starrocks.format.lib.name";
public static final String PROP_FORMAT_WRAPPER_LIB_NAME = "com.starrocks.format.wrapper.lib.name";
static {
load();
private LibraryHelper() {
}
public LibraryHelper() {
}
public native void releaseWriter(long writerAddress);
public native void releaseReader(long readerAddress);
/**
* Load starrocks_format.so and starrocks_format_wrapper.so.
*/
private static void load() {
public static synchronized void load() {
long start = System.currentTimeMillis();
try {
// 1. load starrocks_format.so

View File

@ -0,0 +1,41 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
package com.starrocks.format.rest;
import org.apache.http.client.methods.HttpRequestBase;
import java.util.Optional;
public class RequestException extends Exception {
private static final long serialVersionUID = 3313208780769850911L;
public RequestException(HttpRequestBase request, String message) {
super(wrapMessage(request, message));
}
public RequestException(HttpRequestBase request, String message, Throwable cause) {
super(wrapMessage(request, message), cause);
}
private static String wrapMessage(HttpRequestBase request, String message) {
String wrapMsg = request.toString() + " error";
return Optional.ofNullable(message).map(wm -> wrapMsg + ": " + wm).orElse(wrapMsg);
}
}

View File

@ -0,0 +1,138 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
package com.starrocks.format.rest;
import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
import com.fasterxml.jackson.annotation.JsonProperty;
import java.util.List;
@JsonIgnoreProperties(ignoreUnknown = true)
public class ResponseContent<T> {
@JsonProperty("code")
private String code;
@JsonProperty("message")
private String message;
@JsonProperty("result")
private T result;
public ResponseContent() {
}
public boolean isOk() {
return 0 == Integer.parseInt(this.getCode());
}
public boolean notOk() {
return !isOk();
}
public static class PagedResult<I> {
@JsonProperty("pageNum")
private Integer pageNum;
@JsonProperty("pageSize")
private Integer pageSize;
/**
* Total pages.
*/
@JsonProperty("pages")
private Integer pages;
/**
* Total elements.
*/
@JsonProperty("total")
private Integer total;
@JsonProperty("items")
private List<I> items;
public PagedResult() {
}
public Integer getPageNum() {
return pageNum;
}
public void setPageNum(Integer pageNum) {
this.pageNum = pageNum;
}
public Integer getPageSize() {
return pageSize;
}
public void setPageSize(Integer pageSize) {
this.pageSize = pageSize;
}
public Integer getPages() {
return pages;
}
public void setPages(Integer pages) {
this.pages = pages;
}
public Integer getTotal() {
return total;
}
public void setTotal(Integer total) {
this.total = total;
}
public List<I> getItems() {
return items;
}
public void setItems(List<I> items) {
this.items = items;
}
}
public String getCode() {
return code;
}
public void setCode(String code) {
this.code = code;
}
public String getMessage() {
return message;
}
public void setMessage(String message) {
this.message = message;
}
public T getResult() {
return result;
}
public void setResult(T result) {
this.result = result;
}
}

View File

@ -0,0 +1,712 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
package com.starrocks.format.rest;
import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.core.type.TypeReference;
import com.fasterxml.jackson.databind.DeserializationFeature;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.starrocks.format.rest.ResponseContent.PagedResult;
import com.starrocks.format.rest.model.QueryPlan;
import com.starrocks.format.rest.model.TablePartition;
import com.starrocks.format.rest.model.TableSchema;
import com.starrocks.format.rest.model.TabletCommitInfo;
import com.starrocks.format.rest.model.TabletFailInfo;
import org.apache.commons.collections4.CollectionUtils;
import org.apache.commons.lang3.RandomUtils;
import org.apache.commons.lang3.StringUtils;
import org.apache.commons.lang3.Validate;
import org.apache.http.HttpEntity;
import org.apache.http.HttpRequest;
import org.apache.http.HttpRequestInterceptor;
import org.apache.http.HttpResponse;
import org.apache.http.HttpStatus;
import org.apache.http.ParseException;
import org.apache.http.ProtocolException;
import org.apache.http.StatusLine;
import org.apache.http.auth.Credentials;
import org.apache.http.auth.UsernamePasswordCredentials;
import org.apache.http.client.RedirectStrategy;
import org.apache.http.client.config.RequestConfig;
import org.apache.http.client.methods.CloseableHttpResponse;
import org.apache.http.client.methods.HttpGet;
import org.apache.http.client.methods.HttpPost;
import org.apache.http.client.methods.HttpRequestBase;
import org.apache.http.client.methods.HttpUriRequest;
import org.apache.http.client.protocol.HttpClientContext;
import org.apache.http.client.utils.URIBuilder;
import org.apache.http.entity.ContentType;
import org.apache.http.entity.StringEntity;
import org.apache.http.impl.auth.BasicScheme;
import org.apache.http.impl.client.CloseableHttpClient;
import org.apache.http.impl.client.HttpClients;
import org.apache.http.impl.client.LaxRedirectStrategy;
import org.apache.http.protocol.HTTP;
import org.apache.http.protocol.HttpContext;
import org.apache.http.util.EntityUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.net.URI;
import java.net.URISyntaxException;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Optional;
import java.util.function.Consumer;
import java.util.stream.Collectors;
/**
* RestClient is an HTTP client, that used to interact with the Rest API of StarRocks FE. You can construct it as follows:
* <pre>
* <code>
* RestClient.newBuilder()
* .feHttpEndpoints("http://127.0.0.1:8030")
* .username("root")
* .password("******")
* .build();
* </code>
* </pre>
*/
public class RestClient implements AutoCloseable {
private static final Logger LOG = LoggerFactory.getLogger(RestClient.class);
private static final String HTTP_SCHEME_PREFIX = "http://";
private static final String HTTPS_SCHEME_PREFIX = "https://";
private static final String HEADER_DATABASE = "db";
private static final String HEADER_TABLE = "table";
private static final String HEADER_LABEL = "label";
private static final String HEADER_TIMEOUT = "timeout";
private static final String PARAM_TEMPORARY = "temporary";
private static final String PARAM_PAGE_NUM = "page_num";
private static final String PARAM_PAGE_SIZE = "page_size";
private static final String PARAM_SOURCE_TYPE = "source_type";
private static final String BODY_COMMITTED_TABLETS = "committed_tablets";
private static final String BODY_FAILED_TABLETS = "failed_tablets";
private static final int DEFAULT_PAGE_SIZE = 100;
// FIXME maybe we should extract and import LoadJobSourceType from fe module
private static final int BYPASS_WRITE_JOB_SOURCE_TYPE = 11;
private static final String GET_TABLE_SCHEMA_URL_FORMAT =
"http://%s/api/v2/catalogs/%s/databases/%s/tables/%s/schema";
private static final String GET_TABLE_PARTITION_URL_FORMAT =
"http://%s/api/v2/catalogs/%s/databases/%s/tables/%s/partition";
private static final String GET_QUERY_PLAN_URL_FORMAT = "http://%s/api/%s/%s/_query_plan";
private static final String OPERATE_TRANSACTION_URL_FORMAT = "http://%s/api/transaction/%s";
private static final ObjectMapper JSON_PARSER = new ObjectMapper();
static {
JSON_PARSER.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
}
private final List<String> feHttpEndpoints;
private final Credentials credentials;
private int connectTimeoutMillis = 30 * 1000;
private int socketTimeoutMillis = 30 * 1000;
private int retries = 3;
private RestClient(Builder builder) {
if (CollectionUtils.isEmpty(builder.feHttpEndpoints)) {
throw new IllegalArgumentException("missing fe endpoints");
}
this.feHttpEndpoints = builder.feHttpEndpoints;
if (StringUtils.isBlank(builder.username)) {
throw new IllegalArgumentException("missing username");
}
this.credentials = new UsernamePasswordCredentials(builder.username, builder.password);
Optional.ofNullable(builder.connectTimeoutMillis)
.ifPresent(millis -> this.connectTimeoutMillis = millis);
Optional.ofNullable(builder.socketTimeoutMillis)
.ifPresent(millis -> this.socketTimeoutMillis = millis);
Optional.ofNullable(builder.retries)
.ifPresent(retries -> this.retries = retries);
}
public static Builder newBuilder() {
return new Builder();
}
@Override
public void close() throws Exception {
LOG.info("Close rest client to {}", StringUtils.join(this.feHttpEndpoints, "; "));
}
/**
* Get sql's query plan.
*/
public QueryPlan getQueryPlan(String dbName,
String tableName,
String selectClause,
String filterClause) throws RequestException {
String sql = String.format(
"SELECT %s FROM `%s`.`%s`", Optional.ofNullable(selectClause).orElse("*"), dbName, tableName);
if (StringUtils.isNotBlank(filterClause)) {
sql += String.format(" WHERE %s", filterClause);
}
return getQueryPlan(dbName, tableName, sql);
}
/**
* Get sql's query plan.
*/
public QueryPlan getQueryPlan(String dbName, String tableName, String sql) throws RequestException {
LOG.info("Get query plan for table[{}.{}] by sql[{}]", dbName, tableName, sql);
HttpPost request = new HttpPost(toUri(
String.format(GET_QUERY_PLAN_URL_FORMAT, getRandomFeEndpoint(), dbName, tableName)
));
try {
Map<String, Object> body = new HashMap<>(1);
body.put("sql", sql);
StringEntity entity = new StringEntity(
JSON_PARSER.writeValueAsString(body), StandardCharsets.UTF_8);
entity.setContentType(ContentType.APPLICATION_JSON.toString());
request.setEntity(entity);
} catch (JsonProcessingException e) {
throw new IllegalStateException("Write sql as json error: " + sql, e);
}
return this.doRequest(request, httpEntity -> {
if (null == httpEntity) {
throw new RequestException(request, "null response entity");
}
try {
return JSON_PARSER.readValue(
EntityUtils.toString(httpEntity, StandardCharsets.UTF_8),
new TypeReference<QueryPlan>() {
});
} catch (IOException | ParseException e) {
throw new RequestException(request, e.getMessage(), e);
}
});
}
/**
* Request to get table schema.
*/
public TableSchema getTableSchema(String catalog, String database, String table) throws RequestException {
LOG.info("Get table schema for {}.{}.{}", catalog, database, table);
HttpGet request = new HttpGet(
toUri(String.format(GET_TABLE_SCHEMA_URL_FORMAT,
getRandomFeEndpoint(), catalog, database, table))
);
return this.doRequest(
request,
httpEntity -> {
if (null == httpEntity) {
throw new RequestException(request, "null response entity");
}
ResponseContent<TableSchema> respContent;
try {
respContent = JSON_PARSER.readValue(
EntityUtils.toString(httpEntity, StandardCharsets.UTF_8),
new TypeReference<ResponseContent<TableSchema>>() {
});
if (respContent.isOk()) {
return respContent.getResult();
}
} catch (IOException | ParseException e) {
throw new RequestException(request, e.getMessage(), e);
}
throw new RequestException(request, respContent.getMessage());
});
}
/**
* Request to list all table partitions.
*/
public List<TablePartition> listTablePartitions(String catalog,
String database,
String table,
boolean temporary) throws RequestException {
List<TablePartition> partitions = new ArrayList<>();
int pageNum = 0;
while (true) {
PagedResult<TablePartition> pagedResult =
listTablePartitions(catalog, database, table, temporary, pageNum, DEFAULT_PAGE_SIZE);
partitions.addAll(pagedResult.getItems());
if (++pageNum >= pagedResult.getPages()) {
break;
}
}
return partitions;
}
/**
* Request to list table partitions by page.
*/
public PagedResult<TablePartition> listTablePartitions(String catalog,
String database,
String table,
boolean temporary,
int pageNum,
int pageSize) throws RequestException {
Validate.isTrue(pageNum >= 0, "Invalid pageNum: " + pageNum);
Validate.isTrue(pageSize > 0, "Invalid pageSize: " + pageSize);
LOG.info("List table partitions for {}.{}.{}, temporary:{}, pageNum: {}, pageSize: {}",
catalog, database, table, temporary, pageNum, pageSize);
HttpGet request = new HttpGet(toTablePartitionUri(catalog, database, table, temporary, pageNum, pageSize));
return this.doRequest(
request,
httpEntity -> {
if (null == httpEntity) {
throw new RequestException(request, "null response entity");
}
ResponseContent<PagedResult<TablePartition>> respContent;
try {
respContent = JSON_PARSER.readValue(
EntityUtils.toString(httpEntity, StandardCharsets.UTF_8),
new TypeReference<ResponseContent<PagedResult<TablePartition>>>() {
});
if (respContent.isOk()) {
return respContent.getResult();
}
} catch (IOException e) {
throw new RequestException(request, e.getMessage(), e);
}
throw new RequestException(request, respContent.getMessage());
});
}
private URI toTablePartitionUri(String catalog,
String database,
String table,
boolean temporary,
int pageNum,
int pageSize) {
String uri = String.format(
GET_TABLE_PARTITION_URL_FORMAT,
getRandomFeEndpoint(), catalog, database, table
);
return toUri(uri, uriBuilder -> uriBuilder
.addParameter(PARAM_TEMPORARY, Objects.toString(temporary))
.addParameter(PARAM_PAGE_NUM, Objects.toString(pageNum))
.addParameter(PARAM_PAGE_SIZE, Objects.toString(pageSize))
);
}
/**
* Request to begin transaction.
*/
public TransactionResult beginTransaction(String catalog,
String database,
String table,
String label) throws RequestException {
return this.beginTransaction(catalog, database, table, label, null);
}
/**
* Request to begin transaction.
*/
public TransactionResult beginTransaction(String catalog,
String database,
String table,
String label,
Integer timeoutSecs) throws RequestException {
return this.doTransaction(TxnOperation.TXN_BEGIN, label, request -> {
request.addHeader(HEADER_DATABASE, database);
request.addHeader(HEADER_TABLE, table);
request.addHeader(HEADER_LABEL, label);
if (null != timeoutSecs) {
Validate.isTrue(timeoutSecs > 0, "Invalid timeout: " + timeoutSecs);
request.addHeader(HEADER_TIMEOUT, Objects.toString(timeoutSecs));
}
});
}
/**
* Request to prepare transaction.
*/
public TransactionResult prepareTransaction(String catalog,
String database,
String label,
List<TabletCommitInfo> successTablets,
List<TabletFailInfo> failureTablets) throws RequestException {
return this.doTransaction(TxnOperation.TXN_PREPARE, label, request -> {
request.addHeader(HEADER_DATABASE, database);
request.addHeader(HEADER_LABEL, label);
try {
String body = JSON_PARSER.writeValueAsString(new HashMap<String, Object>(2) {
private static final long serialVersionUID = 6981271088717642861L;
{
Optional.ofNullable(successTablets).ifPresent(tablets -> put(BODY_COMMITTED_TABLETS, tablets));
Optional.ofNullable(failureTablets).ifPresent(tablets -> put(BODY_FAILED_TABLETS, tablets));
}
});
StringEntity entity = new StringEntity(body, StandardCharsets.UTF_8);
entity.setContentType(ContentType.APPLICATION_JSON.toString());
request.setEntity(entity);
} catch (JsonProcessingException e) {
throw new IllegalArgumentException(
"Write success or failure tablets as json error, " + e.getMessage(), e);
}
});
}
/**
* Request to commit transaction.
*/
public TransactionResult commitTransaction(String catalog,
String database,
String label) throws RequestException {
return this.doTransaction(TxnOperation.TXN_COMMIT, label, request -> {
request.addHeader(HEADER_DATABASE, database);
request.addHeader(HEADER_LABEL, label);
});
}
/**
* Request to rollback transaction.
*/
public TransactionResult rollbackTransaction(String catalog,
String database,
String label,
List<TabletFailInfo> failureTablets) throws RequestException {
return this.doTransaction(TxnOperation.TXN_ROLLBACK, label, request -> {
request.addHeader(HEADER_DATABASE, database);
request.addHeader(HEADER_LABEL, label);
try {
String body = JSON_PARSER.writeValueAsString(new HashMap<String, Object>(1) {
private static final long serialVersionUID = -293525488977240959L;
{
Optional.ofNullable(failureTablets).ifPresent(tablets -> put(BODY_FAILED_TABLETS, tablets));
}
});
StringEntity entity = new StringEntity(body, StandardCharsets.UTF_8);
entity.setContentType(ContentType.APPLICATION_JSON.toString());
request.setEntity(entity);
} catch (JsonProcessingException e) {
throw new IllegalArgumentException(
"Write failure tablets as json error, " + e.getMessage(), e);
}
});
}
private TransactionResult doTransaction(TxnOperation txnOpt,
String label,
Consumer<HttpPost> requestConsumer) throws RequestException {
HttpPost request = new HttpPost(toUri(
String.format(OPERATE_TRANSACTION_URL_FORMAT, getRandomFeEndpoint(), txnOpt),
uriBuilder -> {
if (TxnOperation.TXN_BEGIN.equals(txnOpt)) {
uriBuilder.addParameter(PARAM_SOURCE_TYPE, Objects.toString(BYPASS_WRITE_JOB_SOURCE_TYPE));
}
}
));
if (null != requestConsumer) {
requestConsumer.accept(request);
}
LOG.info("Request to {} transaction, label: {}", txnOpt, label);
return this.doRequest(request, httpEntity -> {
if (null == httpEntity) {
throw new RequestException(request, "null response entity");
}
TransactionResult txnResult;
try {
txnResult = JSON_PARSER.readValue(
EntityUtils.toString(httpEntity, StandardCharsets.UTF_8),
new TypeReference<TransactionResult>() {
});
if (txnResult.isOk()) {
return txnResult;
}
} catch (IOException | ParseException e) {
throw new RequestException(request, e.getMessage(), e);
}
throw new RequestException(request, txnResult.getMessage());
});
}
private URI toUri(String uriString) {
return this.toUri(uriString, null);
}
private URI toUri(String uriString, Consumer<URIBuilder> uriBuilderConsumer) {
try {
URIBuilder uriBuilder = new URIBuilder(uriString);
if (null != uriBuilderConsumer) {
uriBuilderConsumer.accept(uriBuilder);
}
return uriBuilder.build();
} catch (URISyntaxException e) {
throw new IllegalStateException("Invalid uri: " + uriString, e);
}
}
private String getRandomFeEndpoint() {
return feHttpEndpoints.get(RandomUtils.nextInt(0, feHttpEndpoints.size()));
}
private String doRequest(HttpRequestBase request) throws RequestException {
return this.doRequest(request, httpEntity -> {
if (null == httpEntity) {
return null;
}
try {
return EntityUtils.toString(httpEntity, StandardCharsets.UTF_8);
} catch (IOException | ParseException e) {
throw new RequestException(request, e.getMessage(), e);
}
});
}
private <T> T doRequest(HttpRequestBase request,
ThrowingFunction<HttpEntity, T, RequestException> responseEntityParser) throws RequestException {
RequestConfig requestConfig = RequestConfig.custom()
.setConnectTimeout(connectTimeoutMillis)
.setSocketTimeout(socketTimeoutMillis)
.setRedirectsEnabled(true)
.build();
request.setConfig(requestConfig);
RedirectStrategy redirectStrategy = new LaxRedirectStrategy() {
@Override
public HttpUriRequest getRedirect(HttpRequest req, HttpResponse rep, HttpContext ctx)
throws ProtocolException {
String method = req.getRequestLine().getMethod();
if (HttpPost.METHOD_NAME.equalsIgnoreCase(method)) {
// FIXME deep copy?
request.setURI(getLocationURI(req, rep, ctx));
return request;
}
return super.getRedirect(req, rep, ctx);
}
};
Throwable e = null;
int retryCnt = 0;
while (retryCnt++ < retries) {
try (CloseableHttpClient httpClient = HttpClients.custom()
.setRedirectStrategy(redirectStrategy)
.addInterceptorFirst((HttpRequestInterceptor) (req, ctx) -> req.removeHeaders(HTTP.CONTENT_LEN))
.build()) {
HttpClientContext context = HttpClientContext.create();
request.addHeader(new BasicScheme().authenticate(credentials, request, context));
if (LOG.isDebugEnabled()) {
LOG.debug("Send request: {}", request);
}
try (CloseableHttpResponse response = httpClient.execute(request, context)) {
StatusLine respStatus = response.getStatusLine();
HttpEntity httpEntity = response.getEntity();
if (HttpStatus.SC_OK == respStatus.getStatusCode()) {
return responseEntityParser.apply(httpEntity);
}
String message = response.getStatusLine().toString();
if (null != httpEntity) {
message = message + ", " + EntityUtils.toString(httpEntity, StandardCharsets.UTF_8);
}
e = new RequestException(request, message);
if (LOG.isDebugEnabled()) {
LOG.debug("Request {} with retries {} error, reason: {}", request, retryCnt, message);
}
}
} catch (Throwable ex) {
e = ex;
}
}
LOG.error("Request {} error", request, e);
if (e instanceof RequestException) {
throw (RequestException) e;
}
throw new RequestException(
request,
Optional.ofNullable(e).map(Throwable::getMessage).orElse(null),
e
);
}
/**
* Rest client builder.
*/
public static class Builder {
private List<String> feHttpEndpoints;
private String username;
private String password;
private Integer connectTimeoutMillis;
private Integer socketTimeoutMillis;
private Integer retries;
private Builder() {
}
/**
* StarRocks FE HTTP endpoint (e.g. {@code http://127.0.0.1:8030}), and multiple endpoints can be separated by commas.
*
* @see #feHttpEndpoints(List)
*/
public Builder feHttpEndpoints(String feEndpoints) {
return this.feHttpEndpoints(feEndpoints.split(",\\s*"));
}
/**
* StarRocks FE HTTP endpoints(e.g. {@code http://127.0.0.1:8030}), and multiple endpoints are allowed.
*
* @see #feHttpEndpoints(List)
*/
public Builder feHttpEndpoints(String[] feEndpoints) {
return this.feHttpEndpoints(Arrays.asList(feEndpoints));
}
/**
* StarRocks FE HTTP endpoints(e.g. {@code http://127.0.0.1:8030}), and multiple endpoints are allowed.
*/
public Builder feHttpEndpoints(List<String> feEndpoints) {
this.feHttpEndpoints = feEndpoints.stream()
.filter(StringUtils::isNotBlank)
.map(String::trim)
.map(String::toLowerCase)
.map(elem -> {
if (elem.startsWith(HTTP_SCHEME_PREFIX)) {
elem = elem.substring(HTTP_SCHEME_PREFIX.length());
}
if (elem.startsWith(HTTPS_SCHEME_PREFIX)) {
elem = elem.substring(HTTPS_SCHEME_PREFIX.length());
}
return elem;
})
.distinct()
.collect(Collectors.toList());
if (CollectionUtils.isEmpty(this.feHttpEndpoints)) {
throw new IllegalArgumentException("Invalid fe endpoints: " + StringUtils.join(feEndpoints, ", "));
}
return this;
}
/**
* Username to access the target StarRocks cluster.
*/
public Builder username(String username) {
this.username = username;
return this;
}
/**
* Password to access the target StarRocks cluster.
*/
public Builder password(String password) {
this.password = password;
return this;
}
/**
* HTTP connection timeout in milliseconds, and default is 30s.
*/
public Builder connectTimeoutMillis(Integer connectTimeoutMillis) {
if (null == connectTimeoutMillis || connectTimeoutMillis <= 0) {
throw new IllegalArgumentException(
"Invalid connect timeout: " + connectTimeoutMillis);
}
this.connectTimeoutMillis = connectTimeoutMillis;
return this;
}
/**
* HTTP socket timeout in milliseconds, and default is 30s.
*/
public Builder socketTimeoutMillis(Integer socketTimeoutMillis) {
if (null == socketTimeoutMillis || socketTimeoutMillis <= 0) {
throw new IllegalArgumentException(
"Invalid socket timeout: " + socketTimeoutMillis);
}
this.socketTimeoutMillis = socketTimeoutMillis;
return this;
}
/**
* HTTP request retry count, and default is 3.
*/
public Builder retries(Integer retries) {
if (null == retries || retries <= 0) {
throw new IllegalArgumentException("Invalid retries: " + retries);
}
this.retries = retries;
return this;
}
public RestClient build() {
return new RestClient(this);
}
}
@FunctionalInterface
private interface ThrowingFunction<T, R, E extends Exception> {
R apply(T t) throws E;
}
}

View File

@ -0,0 +1,77 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
package com.starrocks.format.rest;
import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
import com.fasterxml.jackson.annotation.JsonProperty;
import java.util.StringJoiner;
@JsonIgnoreProperties(ignoreUnknown = true)
public class TransactionResult {
@JsonProperty("Status")
private String status;
@JsonProperty("Message")
private String message;
@JsonProperty("Label")
private String label;
@JsonProperty("TxnId")
private Long txnId;
public TransactionResult() {
}
public boolean isOk() {
return "OK".equalsIgnoreCase(getStatus());
}
public boolean notOk() {
return !isOk();
}
@Override
public String toString() {
return new StringJoiner(", ", "[", "]")
.add("status='" + status + "'")
.add("message='" + message + "'")
.add("label='" + label + "'")
.add("txnId=" + txnId)
.toString();
}
public String getStatus() {
return status;
}
public String getMessage() {
return message;
}
public String getLabel() {
return label;
}
public Long getTxnId() {
return txnId;
}
}

View File

@ -0,0 +1,47 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
package com.starrocks.format.rest;
/**
* Transaction operation for request.
*/
public enum TxnOperation {
TXN_BEGIN("begin"),
TXN_PREPARE("prepare"),
TXN_COMMIT("commit"),
TXN_ROLLBACK("rollback");
private final String value;
TxnOperation(String value) {
this.value = value;
}
@Override
public String toString() {
return getValue();
}
public String getValue() {
return value;
}
}

View File

@ -0,0 +1,275 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
package com.starrocks.format.rest.model;
import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
import com.fasterxml.jackson.annotation.JsonProperty;
import java.io.Serializable;
import java.util.List;
@JsonIgnoreProperties(ignoreUnknown = true)
public class Column implements Serializable {
private static final long serialVersionUID = -2443476992489195839L;
@JsonProperty("name")
private String name;
@JsonProperty("type")
private Type type;
@JsonProperty("aggregationType")
private String aggregationType;
@JsonProperty("isKey")
private Boolean key;
@JsonProperty("isAllowNull")
private Boolean allowNull;
@JsonProperty("isAutoIncrement")
private Boolean autoIncrement;
@JsonProperty("defaultValueType")
private String defaultValueType;
@JsonProperty("defaultValue")
private String defaultValue;
@JsonProperty("defaultExpr")
private String defaultExpr;
@JsonProperty("comment")
private String comment;
@JsonProperty("uniqueId")
private Integer uniqueId;
public Column() {
}
@JsonIgnoreProperties(ignoreUnknown = true)
public static class Type implements Serializable {
private static final long serialVersionUID = 5342044260068193334L;
@JsonProperty("name")
private String name;
@JsonProperty("typeSize")
private Integer typeSize;
@JsonProperty("columnSize")
private Integer columnSize;
@JsonProperty("precision")
private Integer precision;
@JsonProperty("scale")
private Integer scale;
@JsonProperty("itemType")
private Type itemType;
@JsonProperty("named")
private Boolean named;
@JsonProperty("fields")
private List<Column> fields;
@JsonProperty("keyType")
private Type keyType;
@JsonProperty("valueType")
private Type valueType;
public Type() {
}
public String getName() {
return name;
}
public void setName(String name) {
this.name = name;
}
public Integer getTypeSize() {
return typeSize;
}
public void setTypeSize(Integer typeSize) {
this.typeSize = typeSize;
}
public Integer getColumnSize() {
return columnSize;
}
public void setColumnSize(Integer columnSize) {
this.columnSize = columnSize;
}
public Integer getPrecision() {
return precision;
}
public void setPrecision(Integer precision) {
this.precision = precision;
}
public Integer getScale() {
return scale;
}
public void setScale(Integer scale) {
this.scale = scale;
}
public Type getItemType() {
return itemType;
}
public void setItemType(Type itemType) {
this.itemType = itemType;
}
public Boolean getNamed() {
return named;
}
public void setNamed(Boolean named) {
this.named = named;
}
public List<Column> getFields() {
return fields;
}
public void setFields(List<Column> fields) {
this.fields = fields;
}
public Type getKeyType() {
return keyType;
}
public void setKeyType(Type keyType) {
this.keyType = keyType;
}
public Type getValueType() {
return valueType;
}
public void setValueType(Type valueType) {
this.valueType = valueType;
}
}
public String getName() {
return name;
}
public void setName(String name) {
this.name = name;
}
public Type getType() {
return type;
}
public void setType(Type type) {
this.type = type;
}
public String getAggregationType() {
return aggregationType;
}
public void setAggregationType(String aggregationType) {
this.aggregationType = aggregationType;
}
public Boolean getKey() {
return key;
}
public void setKey(Boolean key) {
this.key = key;
}
public Boolean getAllowNull() {
return allowNull;
}
public void setAllowNull(Boolean allowNull) {
this.allowNull = allowNull;
}
public Boolean getAutoIncrement() {
return autoIncrement;
}
public void setAutoIncrement(Boolean autoIncrement) {
this.autoIncrement = autoIncrement;
}
public String getDefaultValueType() {
return defaultValueType;
}
public void setDefaultValueType(String defaultValueType) {
this.defaultValueType = defaultValueType;
}
public String getDefaultValue() {
return defaultValue;
}
public void setDefaultValue(String defaultValue) {
this.defaultValue = defaultValue;
}
public String getDefaultExpr() {
return defaultExpr;
}
public void setDefaultExpr(String defaultExpr) {
this.defaultExpr = defaultExpr;
}
public String getComment() {
return comment;
}
public void setComment(String comment) {
this.comment = comment;
}
public Integer getUniqueId() {
return uniqueId;
}
public void setUniqueId(Integer uniqueId) {
this.uniqueId = uniqueId;
}
}

View File

@ -0,0 +1,63 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
package com.starrocks.format.rest.model;
import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
import com.fasterxml.jackson.annotation.JsonProperty;
import java.util.List;
@JsonIgnoreProperties(ignoreUnknown = true)
public class DistributionInfo {
@JsonProperty("type")
private String type;
@JsonProperty("bucketNum")
private int bucketNum;
@JsonProperty("distributionColumns")
private List<Column> distributionColumns;
public DistributionInfo() {
}
public String getType() {
return type;
}
public void setType(String type) {
this.type = type;
}
public int getBucketNum() {
return bucketNum;
}
public void setBucketNum(int bucketNum) {
this.bucketNum = bucketNum;
}
public List<Column> getDistributionColumns() {
return distributionColumns;
}
public void setDistributionColumns(List<Column> distributionColumns) {
this.distributionColumns = distributionColumns;
}
}

View File

@ -0,0 +1,94 @@
// 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.format.rest.model;
import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
import com.fasterxml.jackson.annotation.JsonProperty;
import java.util.List;
import java.util.Map;
@JsonIgnoreProperties(ignoreUnknown = true)
public class Index {
@JsonProperty("indexId")
private Long indexId;
@JsonProperty("indexName")
private String indexName;
@JsonProperty("indexType")
private String indexType;
@JsonProperty("columns")
private List<String> columns;
@JsonProperty("comment")
private String comment;
@JsonProperty("properties")
private Map<String, String> properties;
public Index() {
}
public Long getIndexId() {
return indexId;
}
public void setIndexId(Long indexId) {
this.indexId = indexId;
}
public String getIndexName() {
return indexName;
}
public void setIndexName(String indexName) {
this.indexName = indexName;
}
public String getIndexType() {
return indexType;
}
public void setIndexType(String indexType) {
this.indexType = indexType;
}
public List<String> getColumns() {
return columns;
}
public void setColumns(List<String> columns) {
this.columns = columns;
}
public String getComment() {
return comment;
}
public void setComment(String comment) {
this.comment = comment;
}
public Map<String, String> getProperties() {
return properties;
}
public void setProperties(Map<String, String> properties) {
this.properties = properties;
}
}

View File

@ -0,0 +1,63 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
package com.starrocks.format.rest.model;
import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
import com.fasterxml.jackson.annotation.JsonProperty;
import java.util.List;
@JsonIgnoreProperties(ignoreUnknown = true)
public class MaterializedIndexMeta {
@JsonProperty("indexId")
private Long indexId;
@JsonProperty("keysType")
private String keysType;
@JsonProperty("columns")
private List<Column> columns;
public MaterializedIndexMeta() {
}
public Long getIndexId() {
return indexId;
}
public void setIndexId(Long indexId) {
this.indexId = indexId;
}
public String getKeysType() {
return keysType;
}
public void setKeysType(String keysType) {
this.keysType = keysType;
}
public List<Column> getColumns() {
return columns;
}
public void setColumns(List<Column> columns) {
this.columns = columns;
}
}

View File

@ -0,0 +1,52 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
package com.starrocks.format.rest.model;
import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
import com.fasterxml.jackson.annotation.JsonProperty;
import java.util.List;
@JsonIgnoreProperties(ignoreUnknown = true)
public class PartitionInfo {
@JsonProperty("type")
private String type;
@JsonProperty("partitionColumns")
private List<Column> partitionColumns;
public PartitionInfo() {
}
public String getType() {
return type;
}
public void setType(String type) {
this.type = type;
}
public List<Column> getPartitionColumns() {
return partitionColumns;
}
public void setPartitionColumns(List<Column> partitionColumns) {
this.partitionColumns = partitionColumns;
}
}

View File

@ -0,0 +1,120 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
package com.starrocks.format.rest.model;
import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
import com.fasterxml.jackson.annotation.JsonProperty;
import java.util.List;
import java.util.Map;
@JsonIgnoreProperties(ignoreUnknown = true)
public class QueryPlan {
@JsonProperty("status")
private Integer status;
@JsonProperty("exception")
private String message;
@JsonProperty("opaqued_query_plan")
private String opaquedQueryPlan;
@JsonProperty("partitions")
private Map<String, TabletNode> partitions;
public Integer getStatus() {
return status;
}
public void setStatus(Integer status) {
this.status = status;
}
public String getMessage() {
return message;
}
public void setMessage(String message) {
this.message = message;
}
public String getOpaquedQueryPlan() {
return opaquedQueryPlan;
}
public void setOpaquedQueryPlan(String opaquedQueryPlan) {
this.opaquedQueryPlan = opaquedQueryPlan;
}
public Map<String, TabletNode> getPartitions() {
return partitions;
}
public void setPartitions(Map<String, TabletNode> partitions) {
this.partitions = partitions;
}
public static class TabletNode {
@JsonProperty("routings")
private List<String> endpoints;
private Integer version;
private Long versionHash;
private Long schemaHash;
public TabletNode() {
}
public List<String> getEndpoints() {
return endpoints;
}
public void setEndpoints(List<String> endpoints) {
this.endpoints = endpoints;
}
public Integer getVersion() {
return version;
}
public void setVersion(Integer version) {
this.version = version;
}
public Long getVersionHash() {
return versionHash;
}
public void setVersionHash(Long versionHash) {
this.versionHash = versionHash;
}
public Long getSchemaHash() {
return schemaHash;
}
public void setSchemaHash(Long schemaHash) {
this.schemaHash = schemaHash;
}
}
}

View File

@ -0,0 +1,225 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
package com.starrocks.format.rest.model;
import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
import com.fasterxml.jackson.annotation.JsonProperty;
import java.util.List;
import java.util.Set;
@JsonIgnoreProperties(ignoreUnknown = true)
public class TablePartition {
@JsonProperty("id")
private Long id;
@JsonProperty("name")
private String name;
@JsonProperty("bucketNum")
private Integer bucketNum;
@JsonProperty("distributionType")
private String distributionType;
@JsonProperty("visibleVersion")
private Long visibleVersion;
@JsonProperty("visibleVersionTime")
private Long visibleVersionTime;
@JsonProperty("nextVersion")
private Long nextVersion;
@JsonProperty("isMinPartition")
private Boolean isMinPartition;
@JsonProperty("isMaxPartition")
private Boolean isMaxPartition;
@JsonProperty("startKeys")
private List<Object> startKeys;
@JsonProperty("endKeys")
private List<Object> endKeys;
@JsonProperty("inKeys")
private List<List<Object>> inKeys;
@JsonProperty("storagePath")
private String storagePath;
@JsonProperty("tablets")
private List<Tablet> tablets;
public TablePartition() {
}
public Long getId() {
return id;
}
public void setId(Long id) {
this.id = id;
}
public String getName() {
return name;
}
public void setName(String name) {
this.name = name;
}
public Integer getBucketNum() {
return bucketNum;
}
public void setBucketNum(Integer bucketNum) {
this.bucketNum = bucketNum;
}
public String getDistributionType() {
return distributionType;
}
public void setDistributionType(String distributionType) {
this.distributionType = distributionType;
}
public Long getVisibleVersion() {
return visibleVersion;
}
public void setVisibleVersion(Long visibleVersion) {
this.visibleVersion = visibleVersion;
}
public Long getVisibleVersionTime() {
return visibleVersionTime;
}
public void setVisibleVersionTime(Long visibleVersionTime) {
this.visibleVersionTime = visibleVersionTime;
}
public Long getNextVersion() {
return nextVersion;
}
public void setNextVersion(Long nextVersion) {
this.nextVersion = nextVersion;
}
public Boolean getMinPartition() {
return isMinPartition;
}
public void setMinPartition(Boolean minPartition) {
isMinPartition = minPartition;
}
public Boolean getMaxPartition() {
return isMaxPartition;
}
public void setMaxPartition(Boolean maxPartition) {
isMaxPartition = maxPartition;
}
public List<Object> getStartKeys() {
return startKeys;
}
public void setStartKeys(List<Object> startKeys) {
this.startKeys = startKeys;
}
public List<Object> getEndKeys() {
return endKeys;
}
public void setEndKeys(List<Object> endKeys) {
this.endKeys = endKeys;
}
public List<List<Object>> getInKeys() {
return inKeys;
}
public void setInKeys(List<List<Object>> inKeys) {
this.inKeys = inKeys;
}
public String getStoragePath() {
return storagePath;
}
public void setStoragePath(String storagePath) {
this.storagePath = storagePath;
}
public List<Tablet> getTablets() {
return tablets;
}
public void setTablets(List<Tablet> tablets) {
this.tablets = tablets;
}
public static class Tablet {
@JsonProperty("id")
private Long id;
@JsonProperty("primaryComputeNodeId")
private Long primaryComputeNodeId;
@JsonProperty("backendIds")
private Set<Long> backendIds;
public Tablet() {
}
public Long getId() {
return id;
}
public void setId(Long id) {
this.id = id;
}
public Long getPrimaryComputeNodeId() {
return primaryComputeNodeId;
}
public void setPrimaryComputeNodeId(Long primaryComputeNodeId) {
this.primaryComputeNodeId = primaryComputeNodeId;
}
public Set<Long> getBackendIds() {
return backendIds;
}
public void setBackendIds(Set<Long> backendIds) {
this.backendIds = backendIds;
}
}
}

View File

@ -0,0 +1,207 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
package com.starrocks.format.rest.model;
import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
import com.fasterxml.jackson.annotation.JsonProperty;
import java.util.List;
import java.util.Map;
@JsonIgnoreProperties(ignoreUnknown = true)
public class TableSchema {
@JsonProperty("id")
private Long id;
@JsonProperty("name")
private String name;
@JsonProperty("tableType")
private String tableType;
@JsonProperty("keysType")
private String keysType;
@JsonProperty("comment")
private String comment;
@JsonProperty("createTime")
private Long createTime;
@JsonProperty("columns")
private List<Column> columns;
@JsonProperty("indexMetas")
private List<MaterializedIndexMeta> indexMetas;
@JsonProperty("partitionInfo")
private PartitionInfo partitionInfo;
@JsonProperty("defaultDistributionInfo")
private DistributionInfo defaultDistributionInfo;
@JsonProperty("colocateGroup")
private String colocateGroup;
@JsonProperty("indexes")
private List<Index> indexes;
@JsonProperty("baseIndexId")
private Long baseIndexId;
@JsonProperty("maxIndexId")
private Long maxIndexId;
@JsonProperty("maxColUniqueId")
private Integer maxColUniqueId;
@JsonProperty("properties")
private Map<String, String> properties;
public TableSchema() {
}
public Long getId() {
return id;
}
public void setId(Long id) {
this.id = id;
}
public String getName() {
return name;
}
public void setName(String name) {
this.name = name;
}
public String getTableType() {
return tableType;
}
public void setTableType(String tableType) {
this.tableType = tableType;
}
public String getKeysType() {
return keysType;
}
public void setKeysType(String keysType) {
this.keysType = keysType;
}
public String getComment() {
return comment;
}
public void setComment(String comment) {
this.comment = comment;
}
public Long getCreateTime() {
return createTime;
}
public void setCreateTime(Long createTime) {
this.createTime = createTime;
}
public List<Column> getColumns() {
return columns;
}
public void setColumns(List<Column> columns) {
this.columns = columns;
}
public List<MaterializedIndexMeta> getIndexMetas() {
return indexMetas;
}
public void setIndexMetas(List<MaterializedIndexMeta> indexMetas) {
this.indexMetas = indexMetas;
}
public PartitionInfo getPartitionInfo() {
return partitionInfo;
}
public void setPartitionInfo(PartitionInfo partitionInfo) {
this.partitionInfo = partitionInfo;
}
public DistributionInfo getDefaultDistributionInfo() {
return defaultDistributionInfo;
}
public void setDefaultDistributionInfo(DistributionInfo defaultDistributionInfo) {
this.defaultDistributionInfo = defaultDistributionInfo;
}
public String getColocateGroup() {
return colocateGroup;
}
public void setColocateGroup(String colocateGroup) {
this.colocateGroup = colocateGroup;
}
public List<Index> getIndexes() {
return indexes;
}
public void setIndexes(List<Index> indexes) {
this.indexes = indexes;
}
public Long getBaseIndexId() {
return baseIndexId;
}
public void setBaseIndexId(Long baseIndexId) {
this.baseIndexId = baseIndexId;
}
public Long getMaxIndexId() {
return maxIndexId;
}
public void setMaxIndexId(Long maxIndexId) {
this.maxIndexId = maxIndexId;
}
public Integer getMaxColUniqueId() {
return maxColUniqueId;
}
public void setMaxColUniqueId(Integer maxColUniqueId) {
this.maxColUniqueId = maxColUniqueId;
}
public Map<String, String> getProperties() {
return properties;
}
public void setProperties(Map<String, String> properties) {
this.properties = properties;
}
}

View File

@ -0,0 +1,54 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
package com.starrocks.format.rest.model;
import java.io.Serializable;
public class TabletCommitInfo implements Serializable {
private static final long serialVersionUID = 8431256353347497743L;
private Long tabletId;
private Long backendId;
public TabletCommitInfo() {
}
public TabletCommitInfo(Long tabletId, Long backendId) {
this.tabletId = tabletId;
this.backendId = backendId;
}
public Long getTabletId() {
return tabletId;
}
public void setTabletId(Long tabletId) {
this.tabletId = tabletId;
}
public Long getBackendId() {
return backendId;
}
public void setBackendId(Long backendId) {
this.backendId = backendId;
}
}

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.
package com.starrocks.format.rest.model;
import java.io.Serializable;
public class TabletFailInfo implements Serializable {
private static final long serialVersionUID = -9209558728067591603L;
private Long tabletId;
private Long backendId;
public TabletFailInfo() {
}
public TabletFailInfo(Long tabletId, Long backendId) {
this.tabletId = tabletId;
this.backendId = backendId;
}
public Long getTabletId() {
return tabletId;
}
public void setTabletId(Long tabletId) {
this.tabletId = tabletId;
}
public Long getBackendId() {
return backendId;
}
public void setBackendId(Long backendId) {
this.backendId = backendId;
}
}

View File

@ -0,0 +1,496 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
package com.starrocks.format;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.starrocks.format.util.ArrowUtils;
import com.starrocks.format.util.DataType;
import com.starrocks.proto.LakeTypes;
import com.starrocks.proto.TabletSchema.ColumnPB;
import com.starrocks.proto.TabletSchema.TabletSchemaPB;
import org.apache.arrow.vector.BigIntVector;
import org.apache.arrow.vector.BitVector;
import org.apache.arrow.vector.DateDayVector;
import org.apache.arrow.vector.DateMilliVector;
import org.apache.arrow.vector.DecimalVector;
import org.apache.arrow.vector.FieldVector;
import org.apache.arrow.vector.Float4Vector;
import org.apache.arrow.vector.Float8Vector;
import org.apache.arrow.vector.IntVector;
import org.apache.arrow.vector.SmallIntVector;
import org.apache.arrow.vector.TimeStampMicroTZVector;
import org.apache.arrow.vector.TinyIntVector;
import org.apache.arrow.vector.VarBinaryVector;
import org.apache.arrow.vector.VarCharVector;
import org.apache.arrow.vector.VectorSchemaRoot;
import org.apache.arrow.vector.complex.ListVector;
import org.apache.arrow.vector.complex.MapVector;
import org.apache.arrow.vector.complex.StructVector;
import org.apache.arrow.vector.complex.impl.UnionMapWriter;
import org.apache.arrow.vector.types.pojo.ArrowType;
import org.apache.arrow.vector.types.pojo.Field;
import org.apache.arrow.vector.types.pojo.FieldType;
import org.apache.arrow.vector.types.pojo.Schema;
import org.apache.commons.lang3.StringUtils;
import java.io.File;
import java.io.FileOutputStream;
import java.io.IOException;
import java.math.BigDecimal;
import java.math.RoundingMode;
import java.nio.ByteBuffer;
import java.nio.charset.StandardCharsets;
import java.sql.Date;
import java.time.LocalDateTime;
import java.time.ZoneId;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
import static com.starrocks.format.util.ArrowUtils.MK_COLUMN_AGG_TYPE;
import static com.starrocks.format.util.ArrowUtils.MK_COLUMN_ID;
import static com.starrocks.format.util.ArrowUtils.MK_COLUMN_IS_AUTO_INCREMENT;
import static com.starrocks.format.util.ArrowUtils.MK_COLUMN_IS_KEY;
import static com.starrocks.format.util.ArrowUtils.MK_COLUMN_MAX_LENGTH;
import static com.starrocks.format.util.ArrowUtils.MK_COLUMN_TYPE;
import static com.starrocks.format.util.ArrowUtils.MK_TABLE_ID;
import static com.starrocks.format.util.ArrowUtils.MK_TABLE_KEYS_TYPE;
import static org.junit.jupiter.api.Assertions.assertTrue;
public class BaseFormatTest {
protected static final ObjectMapper JSON = new ObjectMapper();
protected static final ZoneId DEFAULT_TZ = ZoneId.systemDefault();
protected static long writeTabletMeta(String tabletRootPath, long version, TabletSchemaPB schema)
throws IOException {
long tabletId = schema.getId();
File dataDir = new File(tabletRootPath + "/data");
assertTrue(dataDir.mkdirs());
File logDir = new File(tabletRootPath + "/log");
assertTrue(logDir.mkdirs());
File metaDir = new File(tabletRootPath + "/meta");
assertTrue(metaDir.mkdirs());
LakeTypes.TabletMetadataPB metadata = LakeTypes.TabletMetadataPB.newBuilder()
.setSchema(schema)
.build();
File mf1 = new File(metaDir, String.format("%016X_%016X.meta", tabletId, version));
try (FileOutputStream fos = new FileOutputStream(mf1)) {
System.out.println("Write meta to " + mf1.getAbsolutePath());
metadata.writeTo(fos);
}
File mf2 = new File(metaDir, String.format("%016X_%016X.meta", tabletId, version + 1L));
try (FileOutputStream fos = new FileOutputStream(mf2)) {
System.out.println("Write meta to " + mf2.getAbsolutePath());
metadata.writeTo(fos);
}
return version + 1L;
}
protected static Schema toArrowSchema(TabletSchemaPB tabletSchema) {
Map<String, String> metadata = new HashMap<String, String>() {
private static final long serialVersionUID = -1564086805896748379L;
{
put(MK_TABLE_ID, String.valueOf(tabletSchema.getId()));
put(MK_TABLE_KEYS_TYPE, String.valueOf(tabletSchema.getKeysType()));
}
};
List<Field> fields = tabletSchema.getColumnList().stream()
.map(BaseFormatTest::toArrowField)
.collect(Collectors.toList());
return new Schema(fields, metadata);
}
public static Field toArrowField(ColumnPB column) {
ArrowType arrowType = ArrowUtils.toArrowType(
column.getType(),
DEFAULT_TZ,
column.getPrecision(),
column.getFrac()
);
Map<String, String> metadata = new HashMap<String, String>() {
{
put(MK_COLUMN_ID, String.valueOf(column.getUniqueId()));
put(MK_COLUMN_TYPE, column.getType());
put(MK_COLUMN_IS_KEY, String.valueOf(column.getIsKey()));
put(MK_COLUMN_MAX_LENGTH, String.valueOf(column.getLength()));
put(MK_COLUMN_AGG_TYPE, StringUtils.defaultIfBlank(column.getAggregation(), "NONE"));
put(MK_COLUMN_IS_AUTO_INCREMENT, String.valueOf(column.getIsAutoIncrement()));
}
};
List<Field> children = new ArrayList<>();
if (DataType.MAP.is(column.getType())) {
List<Field> mapChildren = new ArrayList<>();
for (ColumnPB child : column.getChildrenColumnsList()) {
Field childField = toArrowField(child);
mapChildren.add(childField);
}
Field childField = new Field("entries",
new FieldType(false, ArrowType.Struct.INSTANCE, null, metadata), mapChildren);
children.add(childField);
} else {
for (ColumnPB child : column.getChildrenColumnsList()) {
Field childField = toArrowField(child);
children.add(childField);
}
}
return new Field(column.getName(), new FieldType(column.getIsNullable(), arrowType, null, metadata), children);
}
protected static void fillRowData(VectorSchemaRoot vsr, int startRowId, int numRows) throws Exception {
for (int colIdx = 0; colIdx < vsr.getSchema().getFields().size(); colIdx++) {
Field column = vsr.getSchema().getFields().get(colIdx);
FieldVector fieldVector = vsr.getVector(colIdx);
for (int rowIdx = 0; rowIdx < numRows; rowIdx++) {
int rowId = startRowId + rowIdx;
if ("id".equalsIgnoreCase(column.getName())) {
((BigIntVector) fieldVector).setSafe(rowIdx, rowId);
continue;
}
fillColData(column, fieldVector, rowIdx, rowId, 0);
}
fieldVector.setValueCount(numRows);
}
vsr.setRowCount(numRows);
}
private static void fillColData(Field field, FieldVector fieldVector, int rowIdx, int rowId, int depth)
throws Exception {
int sign = (rowId % 2 == 0) ? -1 : 1;
DataType colType = DataType.of(field.getFieldType().getMetadata().get(MK_COLUMN_TYPE));
switch (colType) {
case BOOLEAN:
((BitVector) fieldVector).setSafe(rowIdx, rowId % 2);
break;
case TINYINT:
TinyIntVector tinyIntVector = (TinyIntVector) fieldVector;
if (rowId == 0) {
tinyIntVector.setSafe(rowIdx, Byte.MAX_VALUE);
} else if (rowId == 1) {
tinyIntVector.setSafe(rowIdx, Byte.MIN_VALUE);
} else {
tinyIntVector.setSafe(rowIdx, rowId * sign);
}
break;
case SMALLINT:
SmallIntVector smallIntVector = (SmallIntVector) fieldVector;
if (rowId == 0) {
smallIntVector.setSafe(rowIdx, Short.MAX_VALUE);
} else if (rowId == 1) {
smallIntVector.setSafe(rowIdx, Short.MIN_VALUE);
} else {
smallIntVector.setSafe(rowIdx, (short) (rowId * 10 * sign));
}
break;
case INT:
IntVector intVector = (IntVector) fieldVector;
if (rowId == 0) {
intVector.setSafe(rowIdx, Integer.MAX_VALUE);
} else if (rowId == 1) {
intVector.setSafe(rowIdx, Integer.MIN_VALUE);
} else {
intVector.setSafe(rowIdx, rowId * 100 * sign + depth);
}
break;
case BIGINT:
BigIntVector bigIntVector = (BigIntVector) fieldVector;
if (rowId == 0) {
bigIntVector.setSafe(rowIdx, Long.MAX_VALUE);
} else if (rowId == 1) {
bigIntVector.setSafe(rowIdx, Long.MIN_VALUE);
} else {
bigIntVector.setSafe(rowIdx, rowId * 1000L * sign);
}
break;
case LARGEINT:
VarCharVector largeIntVector = (VarCharVector) fieldVector;
if (rowId == 0) {
largeIntVector.setSafe(rowIdx,
"170141183460469231731687303715884105727".getBytes(StandardCharsets.UTF_8));
} else if (rowId == 1) {
largeIntVector.setSafe(rowIdx,
"-170141183460469231731687303715884105727".getBytes(StandardCharsets.UTF_8));
} else {
largeIntVector.setSafe(rowIdx, String.valueOf(rowId * 10000L * sign).getBytes(StandardCharsets.UTF_8));
}
break;
case FLOAT:
((Float4Vector) fieldVector).setSafe(rowIdx, 123.45678901234f * rowId * sign);
break;
case DOUBLE:
((Float8Vector) fieldVector).setSafe(rowIdx, 23456.78901234 * rowId * sign);
break;
case DECIMAL:
// decimal v2 type
BigDecimal bdv2;
if (rowId == 0) {
bdv2 = new BigDecimal("-12345678901234567890123.4567");
} else if (rowId == 1) {
bdv2 = new BigDecimal("999999999999999999999999.9999");
} else {
bdv2 = new BigDecimal("1234.56789");
bdv2 = bdv2.multiply(BigDecimal.valueOf(sign));
}
((DecimalVector) fieldVector).setSafe(rowIdx, bdv2);
break;
case DECIMAL32:
case DECIMAL64:
case DECIMAL128:
ArrowType.Decimal decimalType = (ArrowType.Decimal) field.getFieldType().getType();
BigDecimal bd;
if (rowId == 0) {
if (decimalType.getPrecision() <= 9) {
bd = new BigDecimal("9999999.5678");
} else if (decimalType.getPrecision() <= 18) {
bd = new BigDecimal("999999999999999.56789");
} else {
bd = new BigDecimal("9999999999999999999999999999999999.56789");
}
} else if (rowId == 1) {
if (decimalType.getPrecision() <= 9) {
bd = new BigDecimal("-9999999.5678");
} else if (decimalType.getPrecision() <= 18) {
bd = new BigDecimal("-999999999999999.56789");
} else {
bd = new BigDecimal("-9999999999999999999999999999999999.56789");
}
} else {
if (decimalType.getPrecision() <= 9) {
bd = new BigDecimal("12345.5678");
} else if (decimalType.getPrecision() <= 18) {
bd = new BigDecimal("123456789012.56789");
} else {
bd = new BigDecimal("12345678901234567890123.56789");
}
bd = bd.multiply(BigDecimal.valueOf((long) rowId * sign));
}
bd = bd.setScale(decimalType.getScale(), RoundingMode.HALF_UP);
((DecimalVector) fieldVector).setSafe(rowIdx, bd);
break;
case DATE:
Date dt;
if (rowId == 0) {
dt = Date.valueOf("1900-1-1");
} else if (rowId == 1) {
dt = Date.valueOf("4096-12-31");
} else {
dt = Date.valueOf("2023-10-31");
dt.setYear(123 + rowId * sign + depth);
}
if (fieldVector instanceof DateDayVector) {
((DateDayVector) fieldVector).setSafe(rowIdx, (int) dt.toLocalDate().toEpochDay());
} else if (fieldVector instanceof DateMilliVector) {
((DateMilliVector) fieldVector).setSafe(rowIdx, dt.toLocalDate().toEpochDay() * 24 * 3600 * 1000);
} else {
throw new IllegalStateException("unsupported column type: " + field.getType());
}
break;
case DATETIME:
LocalDateTime ts;
if (rowId == 0) {
ts = LocalDateTime.parse("1800-11-20T12:34:56");
} else if (rowId == 1) {
ts = LocalDateTime.parse("4096-11-30T11:22:33");
} else {
ts = LocalDateTime.parse("2023-12-30T22:33:44");
ts = ts.withYear(1900 + 123 + rowId * sign);
}
((TimeStampMicroTZVector) fieldVector)
.setSafe(rowIdx, ts.atZone(ZoneId.systemDefault()).toInstant().toEpochMilli() * 1000L);
break;
case CHAR:
case VARCHAR:
String strValue = field.getName() + ":name" + rowId;
if (depth > 0) {
strValue += ",d:" + depth;
}
((VarCharVector) fieldVector).setSafe(rowIdx, strValue.getBytes());
break;
case BINARY:
case VARBINARY:
String valuePrefix = field.getName() + ":name" + rowId + ":";
ByteBuffer buffer = ByteBuffer.allocate(valuePrefix.getBytes().length + 4);
buffer.put(valuePrefix.getBytes());
buffer.putInt(rowId);
((VarBinaryVector) fieldVector).setSafe(rowIdx, buffer.array());
break;
case BITMAP:
case OBJECT:
byte[] bitmapValue = new byte[] {0x00};
switch (rowId % 4) {
case 0:
bitmapValue = new byte[] {0x01, 0x00, 0x00, 0x00, 0x00};
break;
case 1:
bitmapValue = new byte[] {0x01, (byte) 0xE8, 0x03, 0x00, 0x00};
break;
case 3:
bitmapValue = new byte[] {0x1, (byte) 0xB8, 0xB, 0x0, 0x0};
break;
}
((VarBinaryVector) fieldVector).setSafe(rowIdx, bitmapValue);
break;
case HLL:
byte[] hllValue = new byte[] {0x00};
switch (rowId % 4) {
case 0:
hllValue = new byte[] {0x00};
break;
case 1:
hllValue = new byte[] {
0x1, 0x1, 0x44, 0x6, (byte) 0xC3, (byte) 0x80, (byte) 0x9E, (byte) 0x9D, (byte) 0xE6, 0x14
};
break;
case 3:
hllValue = new byte[] {0x1, 0x1, (byte) 0x9A, 0x5, (byte) 0xE4, (byte) 0xE6, 0x65, 0x76, 0x4, 0x28};
break;
}
((VarBinaryVector) fieldVector).setSafe(rowIdx, hllValue);
break;
case ARRAY: {
ListVector listVector = (ListVector) fieldVector;
List<FieldVector> children = fieldVector.getChildrenFromFields();
int elementSize = (rowId + depth) % 4;
listVector.startNewValue(rowIdx);
for (FieldVector childVector : children) {
if (childVector instanceof IntVector) {
int intVal = rowId * 100 * sign;
int startOffset = childVector.getValueCount();
for (int arrayIndex = 0; arrayIndex < elementSize; arrayIndex++) {
((IntVector) childVector).setSafe(startOffset + arrayIndex, intVal + depth + arrayIndex);
}
childVector.setValueCount(startOffset + elementSize);
}
}
listVector.endValue(rowIdx, elementSize);
}
break;
case JSON: {
Map<String, Object> jsonMap = new HashMap<>();
jsonMap.put("rowid", rowId);
boolean boolVal = rowId % 2 == 0;
jsonMap.put("bool", boolVal);
int intVal = 0;
if (rowId == 0) {
intVal = Integer.MAX_VALUE;
} else if (rowId == 1) {
intVal = Integer.MIN_VALUE;
} else {
intVal = rowId * 100 * sign;
}
jsonMap.put("int", intVal);
jsonMap.put("varchar", field.getName() + ":name" + rowId);
String json = JSON.writeValueAsString(jsonMap);
((VarCharVector) fieldVector).setSafe(rowIdx, json.getBytes(), 0, json.getBytes().length);
}
break;
case MAP: {
int elementSize = rowId % 4;
((ListVector) fieldVector).startNewValue(rowIdx);
UnionMapWriter mapWriter = ((MapVector) fieldVector).getWriter();
mapWriter.setPosition(rowIdx);
mapWriter.startMap();
int intVal = rowId * 100 * sign;
for (int arrayIndex = 0; arrayIndex < elementSize; arrayIndex++) {
mapWriter.startEntry();
mapWriter.key().integer().writeInt(intVal + depth + arrayIndex);
mapWriter.value().varChar().writeVarChar("mapvalue:" + (intVal + depth + arrayIndex));
mapWriter.endEntry();
}
mapWriter.endMap();
}
break;
case STRUCT:
List<FieldVector> children = ((StructVector) fieldVector).getChildrenFromFields();
for (FieldVector childVector : children) {
fillColData(childVector.getField(), childVector, rowId, rowIdx, depth + 1);
}
((StructVector) fieldVector).setIndexDefined(rowIdx);
break;
default:
throw new IllegalStateException("unsupported column type: " + field.getType());
}
}
protected static class ColumnType {
private DataType dataType;
private Integer length;
private Integer precision;
private Integer scale;
public ColumnType(DataType dataType, Integer length) {
this.dataType = dataType;
this.length = length;
}
public ColumnType(DataType dataType, Integer length, Integer precision, Integer scale) {
this.dataType = dataType;
this.length = length;
this.precision = precision;
this.scale = scale;
}
public DataType getDataType() {
return dataType;
}
public void setDataType(DataType dataType) {
this.dataType = dataType;
}
public Integer getLength() {
return length;
}
public void setLength(Integer length) {
this.length = length;
}
public Integer getPrecision() {
return precision;
}
public void setPrecision(Integer precision) {
this.precision = precision;
}
public Integer getScale() {
return scale;
}
public void setScale(Integer scale) {
this.scale = scale;
}
}
}

View File

@ -17,8 +17,128 @@
package com.starrocks.format;
class StarRocksWriterTest {
import com.starrocks.format.util.DataType;
import com.starrocks.proto.TabletSchema.ColumnPB;
import com.starrocks.proto.TabletSchema.KeysType;
import com.starrocks.proto.TabletSchema.TabletSchemaPB;
import com.starrocks.proto.Types;
import org.apache.arrow.vector.VectorSchemaRoot;
import org.apache.arrow.vector.types.pojo.Schema;
import org.apache.commons.lang3.RandomUtils;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.io.TempDir;
// TODO add Unit Tests
import java.nio.file.Path;
class StarRocksWriterTest extends BaseFormatTest {
@Test
public void testWrite(@TempDir Path tempDir) throws Exception {
String tabletRootPath = tempDir.toAbsolutePath().toString();
final long tabletId = RandomUtils.nextLong(0, Integer.MAX_VALUE);
TabletSchemaPB.Builder schemaBuilder = TabletSchemaPB.newBuilder()
.setId(tabletId)
.setKeysType(KeysType.DUP_KEYS)
.setCompressionType(Types.CompressionTypePB.LZ4_FRAME);
int colUniqueId = 0;
// add key column
schemaBuilder.addColumn(ColumnPB.newBuilder()
.setUniqueId(colUniqueId++)
.setName("id")
.setType(DataType.BIGINT.getLiteral())
.setIsKey(true)
.setIsNullable(false)
.setLength(8)
.setIndexLength(8)
.build());
ColumnType[] columnTypes = new ColumnType[] {
new ColumnType(DataType.BOOLEAN, 1),
new ColumnType(DataType.TINYINT, 1),
new ColumnType(DataType.SMALLINT, 2),
new ColumnType(DataType.INT, 4),
new ColumnType(DataType.BIGINT, 8),
new ColumnType(DataType.LARGEINT, 16),
new ColumnType(DataType.FLOAT, 4),
new ColumnType(DataType.DOUBLE, 8),
new ColumnType(DataType.DECIMAL64, 8),
new ColumnType(DataType.DATE, 16),
new ColumnType(DataType.DATETIME, 16),
new ColumnType(DataType.CHAR, 32),
new ColumnType(DataType.VARCHAR, 1024),
new ColumnType(DataType.JSON, 16)
};
for (ColumnType columnType : columnTypes) {
ColumnPB.Builder columnBuilder = ColumnPB.newBuilder()
.setUniqueId(colUniqueId++)
.setName("c_" + columnType.getDataType().getLiteral().toLowerCase())
.setType(columnType.getDataType().getLiteral())
.setIsKey(false)
.setIsNullable(true)
.setLength(columnType.getLength())
.setIndexLength(columnType.getLength())
.setAggregation("NONE");
if (DataType.DECIMAL32.equals(columnType.getDataType())) {
columnBuilder.setPrecision(9);
columnBuilder.setFrac(2);
} else if (DataType.DECIMAL64.equals(columnType.getDataType())) {
columnBuilder.setPrecision(18);
columnBuilder.setFrac(3);
} else if (DataType.DECIMAL128.equals(columnType.getDataType())) {
columnBuilder.setPrecision(38);
columnBuilder.setFrac(4);
}
schemaBuilder.addColumn(columnBuilder.build());
}
TabletSchemaPB tabletSchema = schemaBuilder
.setNextColumnUniqueId(colUniqueId)
// sort key index always the key column index
.addSortKeyIdxes(0)
// short key size is less than sort keys
.setNumShortKeyColumns(1)
.setNumRowsPerRowBlock(1024)
.build();
long visibleVersion = writeTabletMeta(tabletRootPath, 1, tabletSchema);
final long txnId = RandomUtils.nextLong(1, Integer.MAX_VALUE);
Schema schema = toArrowSchema(tabletSchema);
try (StarRocksWriter writer = new StarRocksWriter(
tabletId,
tabletRootPath,
txnId,
schema,
Config.newBuilder().build())) {
writer.open();
// write use chunk interface
VectorSchemaRoot vsr = VectorSchemaRoot.create(schema, writer.getAllocator());
fillRowData(vsr, 0, 5);
writer.write(vsr);
vsr.close();
writer.flush();
writer.finish();
}
try (StarRocksReader reader = new StarRocksReader(
tabletId,
tabletRootPath,
visibleVersion,
schema,
schema,
Config.newBuilder().build())) {
reader.open();
while (reader.hasNext()) {
VectorSchemaRoot root = reader.next();
System.out.println(root.contentToTSVString());
}
}
}
}

View File

@ -0,0 +1,174 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
package com.starrocks.format.util;
import com.starrocks.format.rest.model.Column;
import com.starrocks.format.rest.model.MaterializedIndexMeta;
import com.starrocks.format.rest.model.TableSchema;
import org.apache.arrow.vector.types.DateUnit;
import org.apache.arrow.vector.types.FloatingPointPrecision;
import org.apache.arrow.vector.types.TimeUnit;
import org.apache.arrow.vector.types.pojo.ArrowType;
import org.apache.arrow.vector.types.pojo.Field;
import org.apache.arrow.vector.types.pojo.FieldType;
import org.apache.arrow.vector.types.pojo.Schema;
import org.apache.commons.lang3.StringUtils;
import java.time.ZoneId;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.stream.Collectors;
public class ArrowUtils {
public static final String MK_TABLE_ID = "starrocks.format.table.id";
public static final String MK_TABLE_KEYS_TYPE = "starrocks.format.table.keysType";
public static final String MK_COLUMN_ID = "starrocks.format.column.id";
public static final String MK_COLUMN_TYPE = "starrocks.format.column.type";
public static final String MK_COLUMN_IS_KEY = "starrocks.format.column.isKey";
public static final String MK_COLUMN_MAX_LENGTH = "starrocks.format.column.maxLength";
public static final String MK_COLUMN_AGG_TYPE = "starrocks.format.column.aggType";
public static final String MK_COLUMN_IS_AUTO_INCREMENT = "starrocks.format.column.isAutoIncrement";
public static Schema toArrowSchema(TableSchema tableSchema, ZoneId tz) {
MaterializedIndexMeta indexMeta = tableSchema.getIndexMetas().get(0);
Map<String, String> metadata = new HashMap<>();
metadata.put(MK_TABLE_ID, String.valueOf(indexMeta.getIndexId()));
metadata.put(MK_TABLE_KEYS_TYPE, indexMeta.getKeysType());
List<Field> fields = indexMeta.getColumns().stream()
.map(column -> toArrowField(column, tz)).collect(Collectors.toList());
return new Schema(fields, metadata);
}
public static Field toArrowField(Column column, ZoneId tz) {
ArrowType arrowType = toArrowType(
column.getType().getName(),
tz,
Optional.ofNullable(column.getType().getPrecision()).orElse(0),
Optional.ofNullable(column.getType().getScale()).orElse(0)
);
Map<String, String> metadata = new HashMap<>();
metadata.put(MK_COLUMN_ID, String.valueOf(column.getUniqueId()));
metadata.put(MK_COLUMN_TYPE, column.getType().getName());
metadata.put(MK_COLUMN_IS_KEY, String.valueOf(column.getKey()));
metadata.put(MK_COLUMN_MAX_LENGTH, String.valueOf(column.getType().getColumnSize()));
metadata.put(MK_COLUMN_AGG_TYPE, StringUtils.defaultIfBlank(column.getAggregationType(), "NONE"));
metadata.put(MK_COLUMN_IS_AUTO_INCREMENT, String.valueOf(column.getAutoIncrement()));
List<Field> children = getChildren(column.getType(), tz);
return new Field(
column.getName(),
new FieldType(Optional.ofNullable(column.getAllowNull()).orElse(false), arrowType, null, metadata),
children
);
}
public static Field toArrowField(String fieldName, Column.Type columnType, ZoneId tz) {
ArrowType arrowType = toArrowType(
columnType.getName(),
tz,
Optional.ofNullable(columnType.getPrecision()).orElse(0),
Optional.ofNullable(columnType.getScale()).orElse(0)
);
Map<String, String> metadata = new HashMap<>();
metadata.put(MK_COLUMN_TYPE, columnType.getName());
metadata.put(MK_COLUMN_MAX_LENGTH, String.valueOf(columnType.getColumnSize()));
List<Field> children = getChildren(columnType, tz);
return new Field(fieldName, new FieldType(false, arrowType, null, metadata), children);
}
public static List<Field> getChildren(Column.Type columnType, ZoneId tz) {
if (DataType.isScalar(columnType.getName())) {
return new ArrayList<>(0);
}
List<Field> children = new ArrayList<>();
if (DataType.MAP.is(columnType.getName())) {
Field keyField = toArrowField("key", columnType.getKeyType(), tz);
Field valueField = toArrowField("value", columnType.getValueType(), tz);
Map<String, String> metadata = new HashMap<>();
metadata.put(MK_COLUMN_TYPE, "STRUCT");
Field childField = new Field("entries",
new FieldType(false, ArrowType.Struct.INSTANCE, null, metadata), Arrays.asList(keyField, valueField));
children.add(childField);
} else if (DataType.ARRAY.is(columnType.getName())) {
Field itemField = toArrowField("item1", columnType.getItemType(), tz);
children.add(itemField);
} else if (DataType.STRUCT.is(columnType.getName())) {
for (Column child : columnType.getFields()) {
Field childField = toArrowField(child, tz);
children.add(childField);
}
}
return children;
}
public static ArrowType toArrowType(String srType, ZoneId tz, int precision, int scale) {
DataType dataType = DataType.of(srType);
switch (dataType) {
case BOOLEAN:
return ArrowType.Bool.INSTANCE;
case TINYINT:
return new ArrowType.Int(8, true);
case SMALLINT:
return new ArrowType.Int(16, true);
case INT:
return new ArrowType.Int(32, true);
case BIGINT:
return new ArrowType.Int(64, true);
case FLOAT:
return new ArrowType.FloatingPoint(FloatingPointPrecision.SINGLE);
case DOUBLE:
return new ArrowType.FloatingPoint(FloatingPointPrecision.DOUBLE);
case DECIMAL:
case DECIMAL32:
case DECIMAL64:
case DECIMAL128:
return new ArrowType.Decimal(precision, scale, 128);
case DATE:
return new ArrowType.Date(DateUnit.DAY);
case DATETIME:
return new ArrowType.Timestamp(TimeUnit.MICROSECOND, tz.getId());
case CHAR:
case VARCHAR:
case JSON:
case LARGEINT:
return ArrowType.Utf8.INSTANCE;
case BINARY:
case VARBINARY:
case OBJECT:
case BITMAP:
case HLL:
return ArrowType.Binary.INSTANCE;
case ARRAY:
return ArrowType.List.INSTANCE;
case STRUCT:
return ArrowType.Struct.INSTANCE;
case MAP:
return new ArrowType.Map(false);
default:
throw new UnsupportedOperationException("Unsupported data type: " + dataType);
}
}
}

View File

@ -0,0 +1,104 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
package com.starrocks.format.util;
import java.util.Arrays;
import java.util.Optional;
public enum DataType {
BOOLEAN(1, "BOOLEAN"),
TINYINT(2, "TINYINT"),
SMALLINT(3, "SMALLINT"),
INT(4, "INT"),
BIGINT(5, "BIGINT"),
LARGEINT(6, "LARGEINT"),
FLOAT(7, "FLOAT"),
DOUBLE(8, "DOUBLE"),
DECIMAL(9, "DECIMAL"),
DECIMAL32(10, "DECIMAL32"),
DECIMAL64(11, "DECIMAL64"),
DECIMAL128(12, "DECIMAL128"),
DATE(13, "DATE"),
DATETIME(14, "DATETIME"),
CHAR(15, "CHAR"),
VARCHAR(16, "VARCHAR"),
BINARY(17, "BINARY"),
VARBINARY(18, "VARBINARY"),
BITMAP(19, "BITMAP"),
OBJECT(20, "OBJECT"),
HLL(21, "HLL"),
ARRAY(22, "ARRAY"),
JSON(23, "JSON"),
MAP(24, "MAP"),
STRUCT(25, "STRUCT");
private final int id;
private final String literal;
DataType(int id, String literal) {
this.id = id;
this.literal = literal;
}
public static DataType of(String nameOrLiteral) {
return elegantOf(nameOrLiteral)
.orElseThrow(() -> new IllegalArgumentException("Unsupported data type: " + nameOrLiteral));
}
public static Optional<DataType> elegantOf(String nameOrLiteral) {
return Arrays.stream(values())
.filter(dt -> dt.name().equalsIgnoreCase(nameOrLiteral)
|| dt.getLiteral().equalsIgnoreCase(nameOrLiteral))
.findFirst();
}
public static boolean isScalar(String nameOrLiteral) {
return !isArray(nameOrLiteral) && !isStruct(nameOrLiteral) && !isMap(nameOrLiteral);
}
public static boolean isArray(String nameOrLiteral) {
return DataType.ARRAY.equals(of(nameOrLiteral));
}
public static boolean isStruct(String nameOrLiteral) {
return DataType.STRUCT.equals(of(nameOrLiteral));
}
public static boolean isMap(String nameOrLiteral) {
return DataType.MAP.equals(of(nameOrLiteral));
}
public boolean is(String type) {
return name().equalsIgnoreCase(type) || getLiteral().equalsIgnoreCase(type);
}
public boolean not(String type) {
return !is(type);
}
public int getId() {
return id;
}
public String getLiteral() {
return literal;
}
}