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][Connector V2] expose configurable options in Cassandra #3681

Merged
merged 12 commits into from
May 19, 2023
26 changes: 13 additions & 13 deletions docs/en/connector-v2/sink/Cassandra.md
Original file line number Diff line number Diff line change
Expand Up @@ -13,19 +13,19 @@ Write data to Apache Cassandra.

## Options

| name | type | required | default value |
|-------------------|--------|----------|---------------|
| host | String | Yes | - |
| keyspace | String | Yes | - |
| table | String | Yes | - |
| username | String | No | - |
| password | String | No | - |
| datacenter | String | No | datacenter1 |
| consistency_level | String | No | LOCAL_ONE |
| fields | String | No | LOCAL_ONE |
| batch_size | String | No | 5000 |
| batch_type | String | No | UNLOGGER |
| async_write | String | No | true |
| name | type | required | default value |
|-------------------|----------|----------|---------------|
| host | String | Yes | - |
| keyspace | String | Yes | - |
| table | String | Yes | - |
| username | String | No | - |
| password | String | No | - |
| datacenter | String | No | datacenter1 |
| consistency_level | String | No | LOCAL_ONE |
| fields | String | No | LOCAL_ONE |
| batch_size | int | No | 5000 |
| batch_type | String | No | UNLOGGER |
| async_write | boolean | No | true |

### host [string]

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,100 +17,82 @@

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

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

import com.datastax.oss.driver.api.core.ConsistencyLevel;
import com.datastax.oss.driver.api.core.DefaultConsistencyLevel;
import com.datastax.oss.driver.api.core.cql.DefaultBatchType;
import lombok.Data;
import lombok.NoArgsConstructor;
import lombok.NonNull;
import lombok.ToString;

import java.io.Serializable;
import java.util.List;

@Data
@ToString
@NoArgsConstructor
public class CassandraConfig implements Serializable {

public static final String HOST = "host";
public static final String USERNAME = "username";
public static final String PASSWORD = "password";
public static final String DATACENTER = "datacenter";
public static final String KEYSPACE = "keyspace";
public static final String TABLE = "table";
public static final String CQL = "cql";
public static final String FIELDS = "fields";
public static final String CONSISTENCY_LEVEL = "consistency_level";
public static final String BATCH_SIZE = "batch_size";
public static final String BATCH_TYPE = "batch_type";
public static final String ASYNC_WRITE = "async_write";

private String host;
private String username;
private String password;
private String datacenter;
private String keyspace;
private String table;
private String cql;
private List<String> fields;
private ConsistencyLevel consistencyLevel;
private Integer batchSize;
private DefaultBatchType batchType;
private Boolean asyncWrite;

public CassandraConfig(@NonNull String host, @NonNull String keyspace) {
this.host = host;
this.keyspace = keyspace;
}

public static CassandraConfig getCassandraConfig(Config config) {
CassandraConfig cassandraConfig = new CassandraConfig(
config.getString(HOST),
config.getString(KEYSPACE)
);
if (config.hasPath(USERNAME)) {
cassandraConfig.setUsername(config.getString(USERNAME));
}
if (config.hasPath(PASSWORD)) {
cassandraConfig.setPassword(config.getString(PASSWORD));
}
if (config.hasPath(DATACENTER)) {
cassandraConfig.setDatacenter(config.getString(DATACENTER));
} else {
cassandraConfig.setDatacenter("datacenter1");
}
if (config.hasPath(TABLE)) {
cassandraConfig.setTable(config.getString(TABLE));
}
if (config.hasPath(CQL)) {
cassandraConfig.setCql(config.getString(CQL));
}
if (config.hasPath(FIELDS)) {
cassandraConfig.setFields(config.getStringList(FIELDS));
}
if (config.hasPath(CONSISTENCY_LEVEL)) {
cassandraConfig.setConsistencyLevel(DefaultConsistencyLevel.valueOf(config.getString(CONSISTENCY_LEVEL)));
} else {
cassandraConfig.setConsistencyLevel(DefaultConsistencyLevel.LOCAL_ONE);
}
if (config.hasPath(BATCH_SIZE)) {
cassandraConfig.setBatchSize(config.getInt(BATCH_SIZE));
} else {
cassandraConfig.setBatchSize(Integer.parseInt("5000"));
}
if (config.hasPath(BATCH_TYPE)) {
cassandraConfig.setBatchType(DefaultBatchType.valueOf(config.getString(BATCH_TYPE)));
} else {
cassandraConfig.setBatchType(DefaultBatchType.UNLOGGED);
}
if (config.hasPath(ASYNC_WRITE)) {
cassandraConfig.setAsyncWrite(config.getBoolean(ASYNC_WRITE));
} else {
cassandraConfig.setAsyncWrite(true);
}
return cassandraConfig;
}
import org.apache.seatunnel.api.configuration.Option;
import org.apache.seatunnel.api.configuration.Options;

public class CassandraConfig {

public static final Integer DEFAULT_BATCH_SIZE = 5000;

public static final Option<String> HOST =
Options.key("host")
.stringType()
.noDefaultValue()
.withDescription("");

public static final Option<String> KEYSPACE =
Options.key("keyspace")
.stringType()
.noDefaultValue()
.withDescription("");

public static final Option<String> USERNAME =
Options.key("username")
.stringType()
.noDefaultValue()
.withDescription("");
public static final Option<String> PASSWORD =
Options.key("password")
.stringType()
.noDefaultValue()
.withDescription("");
public static final Option<String> DATACENTER =
Options.key("datacenter")
.stringType()
.defaultValue("datacenter1")
.withDescription("");

public static final Option<String> CONSISTENCY_LEVEL =
Options.key("consistency_level")
.stringType()
.defaultValue("LOCAL_ONE")
.withDescription("");

public static final Option<String> TABLE =
Options.key("table")
.stringType()
.noDefaultValue()
.withDescription("");

public static final Option<String> FIELDS =
Options.key("fields")
.stringType()
.defaultValue("LOCAL_ONE")
.withDescription("");

public static final Option<Integer> BATCH_SIZE =
Options.key("batch_size")
.intType()
.defaultValue(DEFAULT_BATCH_SIZE)
.withDescription("");

public static final Option<String> BATCH_TYPE =
Options.key("batch_type")
.stringType()
.defaultValue("UNLOGGER")
.withDescription("");

public static final Option<Boolean> ASYNC_WRITE =
Options.key("async_write")
.booleanType()
.defaultValue(true)
.withDescription("");

public static final Option<String> CQL =
Options.key("cql")
.stringType()
.noDefaultValue()
.withDescription("");


}
Original file line number Diff line number Diff line change
@@ -0,0 +1,92 @@
/*
* 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.cassandra.config;

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

import com.datastax.oss.driver.api.core.ConsistencyLevel;
import com.datastax.oss.driver.api.core.DefaultConsistencyLevel;
import com.datastax.oss.driver.api.core.cql.DefaultBatchType;
import lombok.Getter;
import lombok.Setter;

import java.io.Serializable;
import java.util.List;

@Setter
@Getter
public class CassandraParameters implements Serializable {
private String host;
private String username;
private String password;
private String datacenter;
private String keyspace;
private String table;
private String cql;
private List<String> fields;
private ConsistencyLevel consistencyLevel;
private Integer batchSize;
private DefaultBatchType batchType;
private Boolean asyncWrite;

public void buildWithConfig(Config config) {
this.host = config.getString(CassandraConfig.HOST.key());
this.keyspace = config.getString(CassandraConfig.KEYSPACE.key());

if (config.hasPath(CassandraConfig.USERNAME.key())) {
this.username = config.getString(CassandraConfig.USERNAME.key());
}
if (config.hasPath(CassandraConfig.PASSWORD.key())) {
this.password = config.getString(CassandraConfig.PASSWORD.key());
}
if (config.hasPath(CassandraConfig.DATACENTER.key())) {
this.datacenter = config.getString(CassandraConfig.DATACENTER.key());
} else {
this.datacenter = CassandraConfig.DATACENTER.defaultValue();
}
if (config.hasPath(CassandraConfig.TABLE.key())) {
this.table = config.getString(CassandraConfig.TABLE.key());
}
if (config.hasPath(CassandraConfig.CQL.key())) {
this.cql = config.getString(CassandraConfig.CQL.key());
}
if (config.hasPath(CassandraConfig.FIELDS.key())) {
this.fields = config.getStringList(CassandraConfig.FIELDS.key());
}
if (config.hasPath(CassandraConfig.CONSISTENCY_LEVEL.key())) {
this.consistencyLevel = DefaultConsistencyLevel.valueOf(config.getString(CassandraConfig.CONSISTENCY_LEVEL.key()));
} else {
this.consistencyLevel = DefaultConsistencyLevel.valueOf(CassandraConfig.CONSISTENCY_LEVEL.defaultValue());
}
if (config.hasPath(CassandraConfig.BATCH_SIZE.key())) {
this.batchSize = config.getInt(CassandraConfig.BATCH_SIZE.key());
} else {
this.batchSize = CassandraConfig.BATCH_SIZE.defaultValue();
}
if (config.hasPath(CassandraConfig.BATCH_TYPE.key())) {
this.batchType = DefaultBatchType.valueOf(config.getString(CassandraConfig.BATCH_TYPE.key()));
} else {
this.batchType = DefaultBatchType.valueOf(CassandraConfig.BATCH_TYPE.defaultValue());
Copy link
Contributor

Choose a reason for hiding this comment

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

Same as above.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

ok。

Copy link
Contributor Author

Choose a reason for hiding this comment

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

But I was refered to the writing of Redis.
Is this necessary at this stage?

Copy link
Contributor

Choose a reason for hiding this comment

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

This is only suggested, not required.

}
if (config.hasPath(CassandraConfig.ASYNC_WRITE.key())) {
this.asyncWrite = config.getBoolean(CassandraConfig.ASYNC_WRITE.key());
} else {
this.asyncWrite = true;
Copy link
Contributor

Choose a reason for hiding this comment

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

The default value is assigned when the variable is declared. So we can get rid of the else branch.

}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -17,9 +17,14 @@

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

import static org.apache.seatunnel.connectors.seatunnel.cassandra.config.CassandraConfig.CQL;
import static org.apache.seatunnel.connectors.seatunnel.cassandra.config.CassandraConfig.DATACENTER;
import static org.apache.seatunnel.connectors.seatunnel.cassandra.config.CassandraConfig.FIELDS;
import static org.apache.seatunnel.connectors.seatunnel.cassandra.config.CassandraConfig.HOST;
import static org.apache.seatunnel.connectors.seatunnel.cassandra.config.CassandraConfig.KEYSPACE;
import static org.apache.seatunnel.connectors.seatunnel.cassandra.config.CassandraConfig.PASSWORD;
import static org.apache.seatunnel.connectors.seatunnel.cassandra.config.CassandraConfig.TABLE;
import static org.apache.seatunnel.connectors.seatunnel.cassandra.config.CassandraConfig.USERNAME;

import org.apache.seatunnel.api.common.PrepareFailException;
import org.apache.seatunnel.api.common.SeaTunnelAPIErrorCode;
Expand All @@ -32,7 +37,7 @@
import org.apache.seatunnel.common.config.CheckResult;
import org.apache.seatunnel.common.constants.PluginType;
import org.apache.seatunnel.connectors.seatunnel.cassandra.client.CassandraClient;
import org.apache.seatunnel.connectors.seatunnel.cassandra.config.CassandraConfig;
import org.apache.seatunnel.connectors.seatunnel.cassandra.config.CassandraParameters;
import org.apache.seatunnel.connectors.seatunnel.cassandra.exception.CassandraConnectorErrorCode;
import org.apache.seatunnel.connectors.seatunnel.cassandra.exception.CassandraConnectorException;
import org.apache.seatunnel.connectors.seatunnel.common.sink.AbstractSimpleSink;
Expand All @@ -51,8 +56,7 @@
@AutoService(SeaTunnelSink.class)
public class CassandraSink extends AbstractSimpleSink<SeaTunnelRow, Void> {

private CassandraConfig cassandraConfig;

private final CassandraParameters cassandraParameters = new CassandraParameters();
private SeaTunnelRowType seaTunnelRowType;

private ColumnDefinitions tableSchema;
Expand All @@ -63,34 +67,34 @@ public String getPluginName() {
}

@Override
public void prepare(Config config) throws PrepareFailException {
CheckResult checkResult = CheckConfigUtil.checkAllExists(config, HOST, KEYSPACE, TABLE);
public void prepare(Config pluginConfig) throws PrepareFailException {
CheckResult checkResult = CheckConfigUtil.checkAllExists(pluginConfig, HOST.key(), KEYSPACE.key(), CQL.key());
if (!checkResult.isSuccess()) {
throw new CassandraConnectorException(SeaTunnelAPIErrorCode.CONFIG_VALIDATION_FAILED,
String.format("PluginName: %s, PluginType: %s, Message: %s",
getPluginName(), PluginType.SINK, checkResult.getMsg()));
}
this.cassandraConfig = CassandraConfig.getCassandraConfig(config);
try (CqlSession session = CassandraClient.getCqlSessionBuilder(
cassandraConfig.getHost(),
cassandraConfig.getKeyspace(),
cassandraConfig.getUsername(),
cassandraConfig.getPassword(),
cassandraConfig.getDatacenter()
pluginConfig.getString(HOST.key()),
pluginConfig.getString(KEYSPACE.key()),
pluginConfig.getString(USERNAME.key()),
pluginConfig.getString(PASSWORD.key()),
pluginConfig.getString(DATACENTER.key())
).build()) {
List<String> fields = cassandraConfig.getFields();
this.tableSchema = CassandraClient.getTableSchema(session, cassandraConfig.getTable());
this.cassandraParameters.buildWithConfig(pluginConfig);
List<String> fields = pluginConfig.getStringList(FIELDS.key());
this.tableSchema = CassandraClient.getTableSchema(session, pluginConfig.getString(TABLE.key()));
if (fields == null || fields.isEmpty()) {
List<String> newFields = new ArrayList<>();
for (int i = 0; i < tableSchema.size(); i++) {
newFields.add(tableSchema.get(i).getName().asInternal());
}
cassandraConfig.setFields(newFields);
this.cassandraParameters.setFields(newFields);
} else {
for (String field : fields) {
if (!tableSchema.contains(field)) {
throw new CassandraConnectorException(CassandraConnectorErrorCode.FIELD_NOT_IN_TABLE,
"Field " + field + " does not exist in table " + config.getString(TABLE));
"Field " + field + " does not exist in table " + pluginConfig.getString(TABLE.key()));
}
}
}
Expand All @@ -113,6 +117,6 @@ public SeaTunnelDataType<SeaTunnelRow> getConsumedType() {

@Override
public AbstractSinkWriter<SeaTunnelRow, Void> createWriter(SinkWriter.Context context) throws IOException {
return new CassandraSinkWriter(cassandraConfig, seaTunnelRowType, tableSchema);
return new CassandraSinkWriter(cassandraParameters, seaTunnelRowType, tableSchema);
}
}
Loading