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

[Hbase] Support Hbase multi-table sink #6969

Closed
wants to merge 5 commits into from
Closed
Show file tree
Hide file tree
Changes from all 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
73 changes: 72 additions & 1 deletion docs/en/connector-v2/sink/Hbase.md
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,7 @@ Output data to Hbase
| name | type | required | default value |
|--------------------|---------|----------|-----------------|
| zookeeper_quorum | string | yes | - |
| table | string | yes | - |
| table | string | no | - |
| rowkey_column | list | yes | - |
| family_name | config | yes | - |
| rowkey_delimiter | string | no | "" |
Expand Down Expand Up @@ -114,6 +114,77 @@ Hbase {

```

### Multiple Table

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

source {
FakeSource {
tables_configs = [
{
schema = {
table = "hbase_sink_1"
fields {
name = STRING
c_string = STRING
c_double = DOUBLE
c_bigint = BIGINT
c_float = FLOAT
c_int = INT
c_smallint = SMALLINT
c_boolean = BOOLEAN
time = BIGINT
}
}
rows = [
{
kind = INSERT
fields = ["label_1", "sink_1", 4.3, 200, 2.5, 2, 5, true, 1627529632356]
}
]
},
{
schema = {
table = "hbase_sink_2"
fields {
name = STRING
c_string = STRING
c_double = DOUBLE
c_bigint = BIGINT
c_float = FLOAT
c_int = INT
c_smallint = SMALLINT
c_boolean = BOOLEAN
time = BIGINT
}
}
rows = [
{
kind = INSERT
fields = ["label_2", "sink_2", 4.3, 200, 2.5, 2, 5, true, 1627529632357]
}
]
}
]
}
}

sink {
Hbase {
zookeeper_quorum = "hbase:2181"
rowkey_column = ["name"]
family_name {
all_columns = info
}
}
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can we add table like jdbc sink and can support ${database_name},${schema_name},${table_name} expression?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Of course, I'll fix it.

}
```

## Changelog

### next version
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@

import org.apache.seatunnel.shade.com.typesafe.config.Config;

import org.apache.seatunnel.api.configuration.ReadonlyConfig;
import org.apache.seatunnel.common.config.TypesafeConfigUtils;

import lombok.Builder;
Expand Down Expand Up @@ -69,41 +70,24 @@ public class HbaseParameters implements Serializable {

@Builder.Default private HbaseConfig.EnCoding enCoding = ENCODING.defaultValue();

public static HbaseParameters buildWithConfig(Config pluginConfig) {
public static HbaseParameters buildWithConfig(ReadonlyConfig config) {
HbaseParametersBuilder builder = HbaseParameters.builder();

// required parameters
builder.zookeeperQuorum(pluginConfig.getString(ZOOKEEPER_QUORUM.key()));
builder.table(pluginConfig.getString(TABLE.key()));
builder.rowkeyColumns(pluginConfig.getStringList(ROWKEY_COLUMNS.key()));
builder.familyNames(
TypesafeConfigUtils.configToMap(pluginConfig.getConfig(FAMILY_NAME.key())));

// optional parameters
if (pluginConfig.hasPath(ROWKEY_DELIMITER.key())) {
builder.rowkeyDelimiter(pluginConfig.getString(ROWKEY_DELIMITER.key()));
}
if (pluginConfig.hasPath(VERSION_COLUMN.key())) {
builder.versionColumn(pluginConfig.getString(VERSION_COLUMN.key()));
}
if (pluginConfig.hasPath(NULL_MODE.key())) {
String nullMode = pluginConfig.getString(NULL_MODE.key());
builder.nullMode(HbaseConfig.NullMode.valueOf(nullMode.toUpperCase()));
}
if (pluginConfig.hasPath(WAL_WRITE.key())) {
builder.walWrite(pluginConfig.getBoolean(WAL_WRITE.key()));
}
if (pluginConfig.hasPath(WRITE_BUFFER_SIZE.key())) {
builder.writeBufferSize(pluginConfig.getInt(WRITE_BUFFER_SIZE.key()));
}
if (pluginConfig.hasPath(ENCODING.key())) {
String encoding = pluginConfig.getString(ENCODING.key());
builder.enCoding(HbaseConfig.EnCoding.valueOf(encoding.toUpperCase()));
}
if (pluginConfig.hasPath(HBASE_EXTRA_CONFIG.key())) {
Config extraConfig = pluginConfig.getConfig(HBASE_EXTRA_CONFIG.key());
builder.hbaseExtraConfig(TypesafeConfigUtils.configToMap(extraConfig));
}
builder.zookeeperQuorum(config.get(ZOOKEEPER_QUORUM));
builder.rowkeyColumns(config.get(ROWKEY_COLUMNS));
builder.familyNames(config.get(FAMILY_NAME));

builder.table(config.get(TABLE));
builder.rowkeyDelimiter(config.get(ROWKEY_DELIMITER));
builder.versionColumn(config.get(VERSION_COLUMN));
String nullMode = String.valueOf(config.get(NULL_MODE));
builder.nullMode(HbaseConfig.NullMode.valueOf(nullMode.toUpperCase()));
builder.walWrite(config.get(WAL_WRITE));
builder.writeBufferSize(config.get(WRITE_BUFFER_SIZE));
String encoding = String.valueOf(config.get(ENCODING));
builder.enCoding(HbaseConfig.EnCoding.valueOf(encoding.toUpperCase()));
builder.hbaseExtraConfig(config.get(HBASE_EXTRA_CONFIG));
return builder.build();
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,33 +19,21 @@

import org.apache.seatunnel.shade.com.typesafe.config.Config;

import org.apache.seatunnel.api.common.PrepareFailException;
import org.apache.seatunnel.api.common.SeaTunnelAPIErrorCode;
import org.apache.seatunnel.api.sink.SeaTunnelSink;
import org.apache.seatunnel.api.sink.SinkWriter;
import org.apache.seatunnel.api.sink.SupportMultiTableSink;
import org.apache.seatunnel.api.table.catalog.CatalogTable;
import org.apache.seatunnel.api.table.type.SeaTunnelRow;
import org.apache.seatunnel.api.table.type.SeaTunnelRowType;
import org.apache.seatunnel.common.config.CheckConfigUtil;
import org.apache.seatunnel.common.config.CheckResult;
import org.apache.seatunnel.common.constants.PluginType;
import org.apache.seatunnel.connectors.seatunnel.common.sink.AbstractSimpleSink;
import org.apache.seatunnel.connectors.seatunnel.common.sink.AbstractSinkWriter;
import org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseParameters;
import org.apache.seatunnel.connectors.seatunnel.hbase.exception.HbaseConnectorException;

import com.google.auto.service.AutoService;

import java.io.IOException;
import java.util.ArrayList;
import java.util.List;

import static org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseConfig.FAMILY_NAME;
import static org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseConfig.ROWKEY_COLUMNS;
import static org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseConfig.TABLE;
import static org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseConfig.ZOOKEEPER_QUORUM;

@AutoService(SeaTunnelSink.class)
public class HbaseSink extends AbstractSimpleSink<SeaTunnelRow, Void> {
public class HbaseSink extends AbstractSimpleSink<SeaTunnelRow, Void>
implements SupportMultiTableSink {

private Config pluginConfig;

Expand All @@ -62,29 +50,9 @@ public String getPluginName() {
return HbaseSinkFactory.IDENTIFIER;
}

@Override
public void prepare(Config pluginConfig) throws PrepareFailException {
this.pluginConfig = pluginConfig;
CheckResult result =
CheckConfigUtil.checkAllExists(
pluginConfig,
ZOOKEEPER_QUORUM.key(),
TABLE.key(),
ROWKEY_COLUMNS.key(),
FAMILY_NAME.key());
if (!result.isSuccess()) {
throw new HbaseConnectorException(
SeaTunnelAPIErrorCode.CONFIG_VALIDATION_FAILED,
String.format(
"PluginName: %s, PluginType: %s, Message: %s",
getPluginName(), PluginType.SINK, result.getMsg()));
}
this.hbaseParameters = HbaseParameters.buildWithConfig(pluginConfig);
}

@Override
public void setTypeInfo(SeaTunnelRowType seaTunnelRowType) {
this.seaTunnelRowType = seaTunnelRowType;
public HbaseSink(HbaseParameters hbaseParameters, CatalogTable catalogTable) {
this.hbaseParameters = hbaseParameters;
this.seaTunnelRowType = catalogTable.getTableSchema().toPhysicalRowDataType();
for (String rowkeyColumn : hbaseParameters.getRowkeyColumns()) {
this.rowkeyColumnIndexes.add(seaTunnelRowType.indexOf(rowkeyColumn));
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,12 +17,22 @@

package org.apache.seatunnel.connectors.seatunnel.hbase.sink;

import org.apache.seatunnel.api.configuration.ReadonlyConfig;
import org.apache.seatunnel.api.configuration.util.OptionRule;
import org.apache.seatunnel.api.sink.SinkReplaceNameConstant;
import org.apache.seatunnel.api.table.catalog.CatalogTable;
import org.apache.seatunnel.api.table.catalog.TableIdentifier;
import org.apache.seatunnel.api.table.connector.TableSink;
import org.apache.seatunnel.api.table.factory.Factory;
import org.apache.seatunnel.api.table.factory.TableSinkFactory;
import org.apache.seatunnel.api.table.factory.TableSinkFactoryContext;
import org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseParameters;

import com.google.auto.service.AutoService;

import java.util.HashMap;
import java.util.Map;

import static org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseConfig.ENCODING;
import static org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseConfig.FAMILY_NAME;
import static org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseConfig.HBASE_EXTRA_CONFIG;
Expand Down Expand Up @@ -59,4 +69,50 @@ public OptionRule optionRule() {
HBASE_EXTRA_CONFIG)
.build();
}

@Override
public TableSink createSink(TableSinkFactoryContext context) {
ReadonlyConfig readonlyConfig = context.getOptions();
CatalogTable catalogTable = context.getCatalogTable();
ReadonlyConfig finalReadonlyConfig =
generateCurrentReadonlyConfig(readonlyConfig, catalogTable);
HbaseParameters hbaseParameters = HbaseParameters.buildWithConfig(finalReadonlyConfig);
return () -> new HbaseSink(hbaseParameters, catalogTable);
}

private ReadonlyConfig generateCurrentReadonlyConfig(
ReadonlyConfig readonlyConfig, CatalogTable catalogTable) {
Map<String, String> configMap = readonlyConfig.toMap();

readonlyConfig
.getOptional(TABLE)
.ifPresent(
tableName -> {
String replacedPath =
replaceCatalogTableInPath(tableName, catalogTable);
configMap.put(TABLE.key(), replacedPath);
});

return ReadonlyConfig.fromMap(new HashMap<>(configMap));
}

private String replaceCatalogTableInPath(String originTableName, CatalogTable catalogTable) {
String tableName = originTableName;
TableIdentifier tableIdentifier = catalogTable.getTableId();
if (tableIdentifier != null) {
if (tableIdentifier.getSchemaName() != null) {
tableName =
tableName.replace(
SinkReplaceNameConstant.REPLACE_SCHEMA_NAME_KEY,
tableIdentifier.getSchemaName());
}
if (tableIdentifier.getTableName() != null) {
tableName =
tableName.replace(
SinkReplaceNameConstant.REPLACE_TABLE_NAME_KEY,
tableIdentifier.getTableName());
}
}
return tableName;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@

package org.apache.seatunnel.connectors.seatunnel.hbase.sink;

import org.apache.seatunnel.api.sink.SupportMultiTableSinkWriter;
import org.apache.seatunnel.api.table.type.SeaTunnelDataType;
import org.apache.seatunnel.api.table.type.SeaTunnelRow;
import org.apache.seatunnel.api.table.type.SeaTunnelRowType;
Expand Down Expand Up @@ -44,7 +45,8 @@
import java.util.stream.Collectors;
import java.util.stream.IntStream;

public class HbaseSinkWriter extends AbstractSinkWriter<SeaTunnelRow, Void> {
public class HbaseSinkWriter extends AbstractSinkWriter<SeaTunnelRow, Void>
implements SupportMultiTableSinkWriter<Void> {

private static final String ALL_COLUMNS = "all_columns";

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -56,6 +56,9 @@
public class HbaseIT extends TestSuiteBase implements TestResource {

private static final String TABLE_NAME = "seatunnel_test";
private static final String MULTI_TABLE_ONE_NAME = "hbase_sink_1";

private static final String MULTI_TABLE_TWO_NAME = "hbase_sink_2";

private static final String FAMILY_NAME = "info";

Expand All @@ -76,6 +79,9 @@ public void startUp() throws Exception {
log.info("initial");
hbaseCluster.createTable(TABLE_NAME, Arrays.asList(FAMILY_NAME));
table = TableName.valueOf(TABLE_NAME);
// Create table for hbase multi-table sink test
hbaseCluster.createTable(MULTI_TABLE_ONE_NAME, Arrays.asList(FAMILY_NAME));
hbaseCluster.createTable(MULTI_TABLE_TWO_NAME, Arrays.asList(FAMILY_NAME));
}

@AfterAll
Expand All @@ -92,15 +98,8 @@ public void testHbaseSink(TestContainer container) throws IOException, Interrupt
deleteData(table);
Container.ExecResult sinkExecResult = container.executeJob("/fake-to-hbase.conf");
Assertions.assertEquals(0, sinkExecResult.getExitCode());
Table hbaseTable = hbaseConnection.getTable(table);
Scan scan = new Scan();
ResultScanner scanner = hbaseTable.getScanner(scan);
ArrayList<Result> results = new ArrayList<>();
for (Result result : scanner) {
results.add(result);
}
ArrayList<Result> results = readData(table);
Assertions.assertEquals(results.size(), 5);
scanner.close();
Container.ExecResult sourceExecResult = container.executeJob("/hbase-to-assert.conf");
Assertions.assertEquals(0, sourceExecResult.getExitCode());
}
Expand Down Expand Up @@ -133,6 +132,25 @@ public void testHbaseSinkWithArray(TestContainer container)
scanner.close();
}

@DisabledOnContainer(
value = {},
type = {EngineType.SPARK, EngineType.FLINK},
disabledReason = "Currently SPARK/FLINK do not support multiple table write")
public void testHbaseMultiTableSink(TestContainer container)
throws IOException, InterruptedException {
TableName multiTable1 = TableName.valueOf(MULTI_TABLE_ONE_NAME);
TableName multiTable2 = TableName.valueOf(MULTI_TABLE_TWO_NAME);
deleteData(multiTable1);
deleteData(multiTable2);
Container.ExecResult sinkExecResult =
container.executeJob("/fake-to-hbase-with-multipletable.conf");
Assertions.assertEquals(0, sinkExecResult.getExitCode());
ArrayList<Result> results = readData(multiTable1);
Assertions.assertEquals(results.size(), 1);
results = readData(multiTable2);
Assertions.assertEquals(results.size(), 1);
}

private void deleteData(TableName table) throws IOException {
Table hbaseTable = hbaseConnection.getTable(table);
Scan scan = new Scan();
Expand All @@ -143,4 +161,16 @@ private void deleteData(TableName table) throws IOException {
hbaseTable.delete(deleteRow);
}
}

public ArrayList<Result> readData(TableName table) throws IOException {
Table hbaseTable = hbaseConnection.getTable(table);
Scan scan = new Scan();
ResultScanner scanner = hbaseTable.getScanner(scan);
ArrayList<Result> results = new ArrayList<>();
for (Result result : scanner) {
results.add(result);
}
scanner.close();
return results;
}
}
Loading
Loading