Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[Feature][Kudu] Support multi-table source read #5878

Merged
merged 7 commits into from
Nov 21, 2023
Merged
Show file tree
Hide file tree
Changes from 4 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
70 changes: 54 additions & 16 deletions docs/en/connector-v2/source/Kudu.md
Original file line number Diff line number Diff line change
Expand Up @@ -42,22 +42,23 @@ The tested kudu version is 1.11.1.

## Source Options

| Name | Type | Required | Default | Description |
|-------------------------------------------|--------|----------|------------------------------------------------|----------------------------------------------------------------------------------------------------------|
| kudu_masters | String | Yes | - | Kudu master address. Separated by ',',such as '192.168.88.110:7051'. |
| table_name | String | Yes | - | The name of kudu table. |
| client_worker_count | Int | No | 2 * Runtime.getRuntime().availableProcessors() | Kudu worker count. Default value is twice the current number of cpu cores. |
| client_default_operation_timeout_ms | Long | No | 30000 | Kudu normal operation time out. |
| client_default_admin_operation_timeout_ms | Long | No | 30000 | Kudu admin operation time out. |
| enable_kerberos | Bool | No | false | Kerberos principal enable. |
| kerberos_principal | String | No | - | Kerberos principal. Note that all zeta nodes require have this file. |
| kerberos_keytab | String | No | - | Kerberos keytab. Note that all zeta nodes require have this file. |
| kerberos_krb5conf | String | No | - | Kerberos krb5 conf. Note that all zeta nodes require have this file. |
| scan_token_query_timeout | Long | No | 30000 | The timeout for connecting scan token. If not set, it will be the same as operationTimeout. |
| scan_token_batch_size_bytes | Int | No | 1024 * 1024 | Kudu scan bytes. The maximum number of bytes read at a time, the default is 1MB. |
| filter | Int | No | 1024 * 1024 | Kudu scan filter expressions,Not supported yet. |
| schema | Map | No | 1024 * 1024 | SeaTunnel Schema. |
| common-options | | No | - | Source plugin common parameters, please refer to [Source Common Options](common-options.md) for details. |
| Name | Type | Required | Default | Description |
|-------------------------------------------|--------|----------|------------------------------------------------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
| kudu_masters | String | Yes | - | Kudu master address. Separated by ',',such as '192.168.88.110:7051'. |
| table_name | String | Yes | - | The name of kudu table. |
| client_worker_count | Int | No | 2 * Runtime.getRuntime().availableProcessors() | Kudu worker count. Default value is twice the current number of cpu cores. |
| client_default_operation_timeout_ms | Long | No | 30000 | Kudu normal operation time out. |
| client_default_admin_operation_timeout_ms | Long | No | 30000 | Kudu admin operation time out. |
| enable_kerberos | Bool | No | false | Kerberos principal enable. |
| kerberos_principal | String | No | - | Kerberos principal. Note that all zeta nodes require have this file. |
| kerberos_keytab | String | No | - | Kerberos keytab. Note that all zeta nodes require have this file. |
| kerberos_krb5conf | String | No | - | Kerberos krb5 conf. Note that all zeta nodes require have this file. |
| scan_token_query_timeout | Long | No | 30000 | The timeout for connecting scan token. If not set, it will be the same as operationTimeout. |
| scan_token_batch_size_bytes | Int | No | 1024 * 1024 | Kudu scan bytes. The maximum number of bytes read at a time, the default is 1MB. |
| filter | Int | No | 1024 * 1024 | Kudu scan filter expressions,Not supported yet. |
| schema | Map | No | 1024 * 1024 | SeaTunnel Schema. |
| table_list | Array | No | - | The list of tables to be read. you can use this configuration instead of `table_path` example: ```table_list = [{ table_name = "kudu_source_table_1"},{ table_name = "kudu_source_table_2"}] ``` |
| common-options | | No | - | Source plugin common parameters, please refer to [Source Common Options](common-options.md) for details. |

## Task Example

Expand Down Expand Up @@ -103,6 +104,43 @@ sink {
}
```

### Multiple Table

```hocon
env {
# You can set engine configuration here
execution.parallelism = 1
job.mode = "STREAMING"
execution.checkpoint.interval = 5000
}

source {
# This is a example source plugin **only for test and demonstrate the feature source plugin**
kudu{
kudu_masters = "kudu-master:7051"
table_list = [
{
table_name = "kudu_source_table_1"
},{
table_name = "kudu_source_table_2"
}
]
result_table_name = "kudu"
}
}

transform {
}

sink {
Assert {
rules {
table-names = ["kudu_source_table_1", "kudu_source_table_2"]
}
}
}
```

## Changelog

### 2.2.0-beta 2022-09-26
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -90,9 +90,6 @@ public class CommonConfig implements Serializable {
"Kerberos krb5 conf. Note that all zeta nodes require have this file.");

protected String masters;

protected String table;

protected Integer workerCount;

protected Long operationTimeout;
Expand All @@ -106,7 +103,6 @@ public class CommonConfig implements Serializable {

public CommonConfig(ReadonlyConfig config) {
this.masters = config.get(MASTER);
this.table = config.get(TABLE_NAME);
this.workerCount = config.get(WORKER_COUNT);
this.operationTimeout = config.get(OPERATION_TIMEOUT);
this.adminOperationTimeout = config.get(ADMIN_OPERATION_TIMEOUT);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -74,6 +74,8 @@ public class KuduSinkConfig extends CommonConfig {

private SaveMode saveMode;

private String table;

private SessionConfiguration.FlushMode flushMode;

private int maxBufferSize;
Expand All @@ -99,6 +101,7 @@ public static SaveMode fromStr(String str) {

public KuduSinkConfig(ReadonlyConfig config) {
super(config);
this.table = config.get(TABLE_NAME);
this.saveMode = config.get(SAVE_MODE);
this.flushMode = fromStrFlushMode(config.get(FLUSH_MODE));
this.maxBufferSize = config.get(BATCH_SIZE);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,8 @@

package org.apache.seatunnel.connectors.seatunnel.kudu.config;

import org.apache.seatunnel.shade.com.fasterxml.jackson.core.type.TypeReference;

import org.apache.seatunnel.api.configuration.Option;
import org.apache.seatunnel.api.configuration.Options;
import org.apache.seatunnel.api.configuration.ReadonlyConfig;
Expand All @@ -26,6 +28,9 @@
import lombok.Getter;
import lombok.ToString;

import java.util.List;
import java.util.Map;

@Getter
@ToString
public class KuduSourceConfig extends CommonConfig {
Expand All @@ -50,16 +55,22 @@ public class KuduSourceConfig extends CommonConfig {
.noDefaultValue()
.withDescription("Kudu scan filter expressions");

public static final Option<List<Map<String, Object>>> TABLE_LIST =
Options.key("table_list")
.type(new TypeReference<List<Map<String, Object>>>() {})
.noDefaultValue()
.withDescription("table list config");

private int batchSizeBytes;

protected Long queryTimeout;

private String filter;
private List<KuduSourceTableConfig> tableConfigList;

public KuduSourceConfig(ReadonlyConfig config) {
super(config);
this.batchSizeBytes = config.get(SCAN_BATCH_SIZE_BYTES);
this.queryTimeout = config.get(QUERY_TIMEOUT);
this.filter = config.get(FILTER);
this.tableConfigList = KuduSourceTableConfig.of(config);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,85 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.seatunnel.connectors.seatunnel.kudu.config;

import org.apache.seatunnel.api.configuration.ReadonlyConfig;
import org.apache.seatunnel.api.table.catalog.Catalog;
import org.apache.seatunnel.api.table.catalog.CatalogTable;
import org.apache.seatunnel.api.table.catalog.CatalogTableUtil;
import org.apache.seatunnel.api.table.catalog.TablePath;
import org.apache.seatunnel.api.table.catalog.schema.TableSchemaOptions;
import org.apache.seatunnel.api.table.factory.FactoryUtil;
import org.apache.seatunnel.connectors.seatunnel.kudu.catalog.KuduCatalog;
import org.apache.seatunnel.connectors.seatunnel.kudu.catalog.KuduCatalogFactory;

import com.google.common.collect.Lists;
import lombok.Getter;

import java.io.Serializable;
import java.util.List;
import java.util.Optional;
import java.util.stream.Collectors;

@Getter
public class KuduSourceTableConfig implements Serializable {

private final TablePath tablePath;

private final CatalogTable catalogTable;

private String filter;

private KuduSourceTableConfig(String tablePath, CatalogTable catalogTable) {
this.tablePath = TablePath.of(tablePath);
this.catalogTable = catalogTable;
}

public static List<KuduSourceTableConfig> of(ReadonlyConfig config) {
Optional<Catalog> optionalCatalog =
FactoryUtil.createOptionalCatalog(
KuduCatalogFactory.IDENTIFIER,
config,
KuduSourceTableConfig.class.getClassLoader(),
KuduCatalogFactory.IDENTIFIER);

try (KuduCatalog kuduCatalog = (KuduCatalog) optionalCatalog.get()) {
kuduCatalog.open();
if (config.getOptional(KuduSourceConfig.TABLE_LIST).isPresent()) {
return config.get(KuduSourceConfig.TABLE_LIST).stream()
.map(ReadonlyConfig::fromMap)
.map(readonlyConfig -> parseKuduSourceConfig(readonlyConfig, kuduCatalog))
.collect(Collectors.toList());
}
KuduSourceTableConfig kuduSourceTableConfig =
parseKuduSourceConfig(config, kuduCatalog);
return Lists.newArrayList(kuduSourceTableConfig);
}
}

public static KuduSourceTableConfig parseKuduSourceConfig(
ReadonlyConfig config, KuduCatalog kuduCatalog) {
CatalogTable catalogTable;
String tableName = config.get(CommonConfig.TABLE_NAME);
if (config.getOptional(TableSchemaOptions.SCHEMA).isPresent()) {
catalogTable = CatalogTableUtil.buildWithConfig(config);
} else {
catalogTable = kuduCatalog.getTable(TablePath.of(config.get(CommonConfig.TABLE_NAME)));
}
return new KuduSourceTableConfig(tableName, catalogTable);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
import org.apache.seatunnel.api.table.type.SeaTunnelRow;
import org.apache.seatunnel.api.table.type.SeaTunnelRowType;
import org.apache.seatunnel.connectors.seatunnel.kudu.config.KuduSourceConfig;
import org.apache.seatunnel.connectors.seatunnel.kudu.config.KuduSourceTableConfig;
import org.apache.seatunnel.connectors.seatunnel.kudu.exception.KuduConnectorErrorCode;
import org.apache.seatunnel.connectors.seatunnel.kudu.exception.KuduConnectorException;
import org.apache.seatunnel.connectors.seatunnel.kudu.source.KuduSourceSplit;
Expand Down Expand Up @@ -51,15 +52,12 @@
public class KuduInputFormat implements Serializable {

private final KuduSourceConfig kuduSourceConfig;
private final SeaTunnelRowType rowTypeInfo;

/** Declare the global variable KuduClient and use it to manipulate the Kudu table */
public KuduClient kuduClient;

public KuduInputFormat(
@NonNull KuduSourceConfig kuduSourceConfig, SeaTunnelRowType rowTypeInfo) {
public KuduInputFormat(@NonNull KuduSourceConfig kuduSourceConfig) {
this.kuduSourceConfig = kuduSourceConfig;
this.rowTypeInfo = rowTypeInfo;
}

public void openInputFormat() {
Expand All @@ -68,7 +66,7 @@ public void openInputFormat() {
}
}

public SeaTunnelRow toInternal(RowResult rs) throws SQLException {
public SeaTunnelRow toInternal(RowResult rs, SeaTunnelRowType rowTypeInfo) throws SQLException {
List<Object> fields = new ArrayList<>();
SeaTunnelDataType<?>[] seaTunnelDataTypes = rowTypeInfo.getFieldTypes();
for (int i = 0; i < seaTunnelDataTypes.length; i++) {
Expand Down Expand Up @@ -96,21 +94,22 @@ public void closeInputFormat() {
}
}

public Set<KuduSourceSplit> createInputSplits() throws IOException {
public Set<KuduSourceSplit> createInputSplits(KuduSourceTableConfig kuduSourceTableConfig)
throws IOException {
List<KuduScanToken> scanTokens =
KuduUtil.getKuduScanToken(kuduSourceConfig, rowTypeInfo.getFieldNames());
KuduUtil.getKuduScanToken(kuduClient, kuduSourceConfig, kuduSourceTableConfig);
Set<KuduSourceSplit> allSplit = new HashSet<>(scanTokens.size());
for (int i = 0; i < scanTokens.size(); i++) {
allSplit.add(new KuduSourceSplit(i, scanTokens.get(i).serialize()));
allSplit.add(
new KuduSourceSplit(
kuduSourceTableConfig.getTablePath(),
i,
scanTokens.get(i).serialize()));
}
return allSplit;
}

public KuduScanner scanner(byte[] token) throws IOException {
return KuduScanToken.deserializeIntoScanner(token, kuduClient);
}

public SeaTunnelRowType getRowTypeInfo() {
return this.rowTypeInfo;
}
}
Loading
Loading