{
+ private static final long serialVersionUID = -7994311331389155692L;
+
+ private static final Logger LOG = LoggerFactory.getLogger(CassandraOutputFormat.class);
+
+ private String address;
+ private String tableName;
+ private String userName;
+ private String password;
+ private String database;
+ private Integer maxRequestsPerConnection;
+ private Integer coreConnectionsPerHost;
+ private Integer maxConnectionsPerHost;
+ private Integer maxQueueSize;
+ private Integer readTimeoutMillis;
+ private Integer connectTimeoutMillis;
+ private Integer poolTimeoutMillis;
+
+ protected String[] fieldNames;
+ TypeInformation>[] fieldTypes;
+
+ private int batchInterval = 5000;
+
+ private Cluster cluster;
+ private Session session = null;
+
+ private int batchCount = 0;
+
+ private transient Counter outRecords;
+
+ private transient Meter outRecordsRate;
+
+ public CassandraOutputFormat() {
+ }
+
+ @Override
+ public void configure(Configuration parameters) {
+ }
+
+ /**
+ * Connects to the target database and initializes the prepared statement.
+ *
+ * @param taskNumber The number of the parallel instance.
+ * @throws IOException Thrown, if the output could not be opened due to an
+ * I/O problem.
+ */
+ @Override
+ public void open(int taskNumber, int numTasks) throws IOException {
+ try {
+ if (session == null) {
+ QueryOptions queryOptions = new QueryOptions();
+ //The default consistency level for queries: ConsistencyLevel.TWO.
+ queryOptions.setConsistencyLevel(ConsistencyLevel.QUORUM);
+ Integer maxRequestsPerConnection = this.maxRequestsPerConnection == null ? 1 : this.maxRequestsPerConnection;
+ Integer coreConnectionsPerHost = this.coreConnectionsPerHost == null ? 8 : this.coreConnectionsPerHost;
+ Integer maxConnectionsPerHost = this.maxConnectionsPerHost == null ? 32768 : this.maxConnectionsPerHost;
+ Integer maxQueueSize = this.maxQueueSize == null ? 100000 : this.maxQueueSize;
+ Integer readTimeoutMillis = this.readTimeoutMillis == null ? 60000 : this.readTimeoutMillis;
+ Integer connectTimeoutMillis = this.connectTimeoutMillis == null ? 60000 : this.connectTimeoutMillis;
+ Integer poolTimeoutMillis = this.poolTimeoutMillis == null ? 60000 : this.poolTimeoutMillis;
+ Integer cassandraPort = 0;
+
+ ArrayList serversList = new ArrayList();
+ //Read timeout or connection timeout Settings
+ SocketOptions so = new SocketOptions()
+ .setReadTimeoutMillis(readTimeoutMillis)
+ .setConnectTimeoutMillis(connectTimeoutMillis);
+
+ //The cluster USES hostdistance.local in the same machine room
+ //Hostdistance. REMOTE is used for different machine rooms
+ //Ignore use HostDistance. IGNORED
+ PoolingOptions poolingOptions = new PoolingOptions()
+ //Each connection allows a maximum of 64 concurrent requests
+ .setMaxRequestsPerConnection(HostDistance.LOCAL, maxRequestsPerConnection)
+ //Have at least two connections to each machine in the cluster
+ .setCoreConnectionsPerHost(HostDistance.LOCAL, coreConnectionsPerHost)
+ //There are up to eight connections to each machine in the cluster
+ .setMaxConnectionsPerHost(HostDistance.LOCAL, maxConnectionsPerHost)
+ .setMaxQueueSize(maxQueueSize)
+ .setPoolTimeoutMillis(poolTimeoutMillis);
+ //重试策略
+ RetryPolicy retryPolicy = DowngradingConsistencyRetryPolicy.INSTANCE;
+
+ for (String server : address.split(",")) {
+ cassandraPort = Integer.parseInt(server.split(":")[1]);
+ serversList.add(InetAddress.getByName(server.split(":")[0]));
+ }
+
+ if (userName == null || userName.isEmpty() || password == null || password.isEmpty()) {
+ cluster = Cluster.builder().addContactPoints(serversList).withRetryPolicy(retryPolicy)
+ .withPort(cassandraPort)
+ .withPoolingOptions(poolingOptions).withSocketOptions(so)
+ .withQueryOptions(queryOptions).build();
+ } else {
+ cluster = Cluster.builder().addContactPoints(serversList).withRetryPolicy(retryPolicy)
+ .withPort(cassandraPort)
+ .withPoolingOptions(poolingOptions).withSocketOptions(so)
+ .withCredentials(userName, password)
+ .withQueryOptions(queryOptions).build();
+ }
+ // 建立连接 连接已存在的键空间
+ session = cluster.connect(database);
+ LOG.info("connect cassandra is successed!");
+ initMetric();
+ }
+ } catch (Exception e) {
+ LOG.error("connect cassandra is error:" + e.getMessage());
+ }
+ }
+
+ private void initMetric() {
+ outRecords = getRuntimeContext().getMetricGroup().counter(MetricConstant.DT_NUM_RECORDS_OUT);
+ outRecordsRate = getRuntimeContext().getMetricGroup().meter(MetricConstant.DT_NUM_RECORDS_OUT_RATE, new MeterView(outRecords, 20));
+ }
+
+ /**
+ * Adds a record to the prepared statement.
+ *
+ * When this method is called, the output format is guaranteed to be opened.
+ *
+ *
+ * WARNING: this may fail when no column types specified (because a best effort approach is attempted in order to
+ * insert a null value but it's not guaranteed that the JDBC driver handles PreparedStatement.setObject(pos, null))
+ *
+ * @param tuple2 The records to add to the output.
+ * @throws IOException Thrown, if the records could not be added due to an I/O problem.
+ * @see PreparedStatement
+ */
+ @Override
+ public void writeRecord(Tuple2 tuple2) throws IOException {
+ Tuple2 tupleTrans = tuple2;
+ Boolean retract = tupleTrans.getField(0);
+ Row row = tupleTrans.getField(1);
+ try {
+ if (retract) {
+ insertWrite(row);
+ outRecords.inc();
+ } else {
+ //do nothing
+ }
+ } catch (Exception e) {
+ throw new IllegalArgumentException("writeRecord() failed", e);
+ }
+ }
+
+ private void insertWrite(Row row) {
+ try {
+ String cql = buildSql(row);
+ if (cql != null) {
+ ResultSet resultSet = session.execute(cql);
+ resultSet.wasApplied();
+ }
+ } catch (Exception e) {
+ LOG.error("[upsert] is error:" + e.getMessage());
+ }
+ }
+
+ private String buildSql(Row row) {
+ StringBuffer fields = new StringBuffer();
+ StringBuffer values = new StringBuffer();
+ for (int index = 0; index < row.getArity(); index++) {
+ if (row.getField(index) == null) {
+ } else {
+ fields.append(fieldNames[index] + ",");
+ values.append("'" + row.getField(index) + "'" + ",");
+ }
+ }
+ fields.deleteCharAt(fields.length() - 1);
+ values.deleteCharAt(values.length() - 1);
+ String cql = "INSERT INTO " + database + "." + tableName + " (" + fields.toString() + ") "
+ + " VALUES (" + values.toString() + ")";
+ return cql;
+ }
+
+ /**
+ * Executes prepared statement and closes all resources of this instance.
+ *
+ * @throws IOException Thrown, if the input could not be closed properly.
+ */
+ @Override
+ public void close() throws IOException {
+ try {
+ if (session != null) {
+ session.close();
+ }
+ } catch (Exception e) {
+ LOG.error("Error while closing session.", e);
+ }
+ try {
+ if (cluster != null) {
+ cluster.close();
+ }
+ } catch (Exception e) {
+ LOG.error("Error while closing cluster.", e);
+ }
+ LOG.info("close cassandra is successed!");
+ }
+
+ public static CassandraFormatBuilder buildOutputFormat() {
+ return new CassandraFormatBuilder();
+ }
+
+ public static class CassandraFormatBuilder {
+ private final CassandraOutputFormat format;
+
+ protected CassandraFormatBuilder() {
+ this.format = new CassandraOutputFormat();
+ }
+
+ public CassandraFormatBuilder setUsername(String username) {
+ format.userName = username;
+ return this;
+ }
+
+ public CassandraFormatBuilder setPassword(String password) {
+ format.password = password;
+ return this;
+ }
+
+ public CassandraFormatBuilder setAddress(String address) {
+ format.address = address;
+ return this;
+ }
+
+ public CassandraFormatBuilder setTableName(String tableName) {
+ format.tableName = tableName;
+ return this;
+ }
+
+ public CassandraFormatBuilder setDatabase(String database) {
+ format.database = database;
+ return this;
+ }
+
+ public CassandraFormatBuilder setFieldNames(String[] fieldNames) {
+ format.fieldNames = fieldNames;
+ return this;
+ }
+
+ public CassandraFormatBuilder setFieldTypes(TypeInformation>[] fieldTypes) {
+ format.fieldTypes = fieldTypes;
+ return this;
+ }
+
+ public CassandraFormatBuilder setMaxRequestsPerConnection(Integer maxRequestsPerConnection) {
+ format.maxRequestsPerConnection = maxRequestsPerConnection;
+ return this;
+ }
+
+ public CassandraFormatBuilder setCoreConnectionsPerHost(Integer coreConnectionsPerHost) {
+ format.coreConnectionsPerHost = coreConnectionsPerHost;
+ return this;
+ }
+
+ public CassandraFormatBuilder setMaxConnectionsPerHost(Integer maxConnectionsPerHost) {
+ format.maxConnectionsPerHost = maxConnectionsPerHost;
+ return this;
+ }
+
+ public CassandraFormatBuilder setMaxQueueSize(Integer maxQueueSize) {
+ format.maxQueueSize = maxQueueSize;
+ return this;
+ }
+
+ public CassandraFormatBuilder setReadTimeoutMillis(Integer readTimeoutMillis) {
+ format.readTimeoutMillis = readTimeoutMillis;
+ return this;
+ }
+
+ public CassandraFormatBuilder setConnectTimeoutMillis(Integer connectTimeoutMillis) {
+ format.connectTimeoutMillis = connectTimeoutMillis;
+ return this;
+ }
+
+ public CassandraFormatBuilder setPoolTimeoutMillis(Integer poolTimeoutMillis) {
+ format.poolTimeoutMillis = poolTimeoutMillis;
+ return this;
+ }
+
+ /**
+ * Finalizes the configuration and checks validity.
+ *
+ * @return Configured RetractJDBCOutputFormat
+ */
+ public CassandraOutputFormat finish() {
+ if (format.userName == null) {
+ LOG.info("Username was not supplied separately.");
+ }
+ if (format.password == null) {
+ LOG.info("Password was not supplied separately.");
+ }
+ if (format.address == null) {
+ throw new IllegalArgumentException("No address URL supplied.");
+ }
+ if (format.database == null) {
+ throw new IllegalArgumentException("No dababase suplied");
+ }
+ if (format.tableName == null) {
+ throw new IllegalArgumentException("No tableName supplied");
+ }
+ return format;
+ }
+ }
+}
diff --git a/cassandra/cassandra-sink/src/main/java/com/dtstack/flink/sql/sink/cassandra/CassandraSink.java b/cassandra/cassandra-sink/src/main/java/com/dtstack/flink/sql/sink/cassandra/CassandraSink.java
new file mode 100644
index 000000000..eb7b23b53
--- /dev/null
+++ b/cassandra/cassandra-sink/src/main/java/com/dtstack/flink/sql/sink/cassandra/CassandraSink.java
@@ -0,0 +1,133 @@
+/*
+ * 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.dtstack.flink.sql.sink.cassandra;
+
+
+import com.dtstack.flink.sql.sink.IStreamSinkGener;
+import com.dtstack.flink.sql.sink.cassandra.table.CassandraTableInfo;
+import com.dtstack.flink.sql.table.TargetTableInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.typeutils.RowTypeInfo;
+import org.apache.flink.api.java.typeutils.TupleTypeInfo;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.functions.sink.OutputFormatSinkFunction;
+import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
+import org.apache.flink.table.sinks.RetractStreamTableSink;
+import org.apache.flink.table.sinks.TableSink;
+import org.apache.flink.types.Row;
+
+/**
+ * Reason:
+ * Date: 2018/11/22
+ *
+ * @author xuqianjin
+ */
+public class CassandraSink implements RetractStreamTableSink, IStreamSinkGener {
+
+
+ protected String[] fieldNames;
+ TypeInformation>[] fieldTypes;
+ protected String address;
+ protected String tableName;
+ protected String userName;
+ protected String password;
+ protected String database;
+ protected Integer maxRequestsPerConnection;
+ protected Integer coreConnectionsPerHost;
+ protected Integer maxConnectionsPerHost;
+ protected Integer maxQueueSize;
+ protected Integer readTimeoutMillis;
+ protected Integer connectTimeoutMillis;
+ protected Integer poolTimeoutMillis;
+
+ public CassandraSink() {
+ // TO DO NOTHING
+ }
+
+ @Override
+ public CassandraSink genStreamSink(TargetTableInfo targetTableInfo) {
+ CassandraTableInfo cassandraTableInfo = (CassandraTableInfo) targetTableInfo;
+ this.address = cassandraTableInfo.getAddress();
+ this.tableName = cassandraTableInfo.getTableName();
+ this.userName = cassandraTableInfo.getUserName();
+ this.password = cassandraTableInfo.getPassword();
+ this.database = cassandraTableInfo.getDatabase();
+ this.maxRequestsPerConnection = cassandraTableInfo.getMaxRequestsPerConnection();
+ this.coreConnectionsPerHost = cassandraTableInfo.getCoreConnectionsPerHost();
+ this.maxConnectionsPerHost = cassandraTableInfo.getMaxConnectionsPerHost();
+ this.maxQueueSize = cassandraTableInfo.getMaxQueueSize();
+ this.readTimeoutMillis = cassandraTableInfo.getReadTimeoutMillis();
+ this.connectTimeoutMillis = cassandraTableInfo.getConnectTimeoutMillis();
+ this.poolTimeoutMillis = cassandraTableInfo.getPoolTimeoutMillis();
+ return this;
+ }
+
+ @Override
+ public void emitDataStream(DataStream> dataStream) {
+ CassandraOutputFormat.CassandraFormatBuilder builder = CassandraOutputFormat.buildOutputFormat();
+ builder.setAddress(this.address)
+ .setDatabase(this.database)
+ .setTableName(this.tableName)
+ .setPassword(this.password)
+ .setUsername(this.userName)
+ .setMaxRequestsPerConnection(this.maxRequestsPerConnection)
+ .setCoreConnectionsPerHost(this.coreConnectionsPerHost)
+ .setMaxConnectionsPerHost(this.maxConnectionsPerHost)
+ .setMaxQueueSize(this.maxQueueSize)
+ .setReadTimeoutMillis(this.readTimeoutMillis)
+ .setConnectTimeoutMillis(this.connectTimeoutMillis)
+ .setPoolTimeoutMillis(this.poolTimeoutMillis)
+ .setFieldNames(this.fieldNames)
+ .setFieldTypes(this.fieldTypes);
+
+ CassandraOutputFormat outputFormat = builder.finish();
+ RichSinkFunction richSinkFunction = new OutputFormatSinkFunction(outputFormat);
+ dataStream.addSink(richSinkFunction);
+ }
+
+ @Override
+ public TableSink> configure(String[] fieldNames, TypeInformation>[] fieldTypes) {
+ this.fieldNames = fieldNames;
+ this.fieldTypes = fieldTypes;
+ return this;
+ }
+
+ @Override
+ public TupleTypeInfo> getOutputType() {
+ return new TupleTypeInfo(org.apache.flink.table.api.Types.BOOLEAN(), getRecordType());
+ }
+
+ @Override
+ public TypeInformation getRecordType() {
+ return new RowTypeInfo(fieldTypes, fieldNames);
+ }
+
+ @Override
+ public String[] getFieldNames() {
+ return fieldNames;
+ }
+
+ @Override
+ public TypeInformation>[] getFieldTypes() {
+ return fieldTypes;
+ }
+
+}
diff --git a/cassandra/cassandra-sink/src/main/java/com/dtstack/flink/sql/sink/cassandra/table/CassandraSinkParser.java b/cassandra/cassandra-sink/src/main/java/com/dtstack/flink/sql/sink/cassandra/table/CassandraSinkParser.java
new file mode 100644
index 000000000..4c68e71ae
--- /dev/null
+++ b/cassandra/cassandra-sink/src/main/java/com/dtstack/flink/sql/sink/cassandra/table/CassandraSinkParser.java
@@ -0,0 +1,84 @@
+/*
+ * 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.dtstack.flink.sql.sink.cassandra.table;
+
+import com.dtstack.flink.sql.table.AbsTableParser;
+import com.dtstack.flink.sql.table.TableInfo;
+import com.dtstack.flink.sql.util.MathUtil;
+
+import java.util.Map;
+
+import static com.dtstack.flink.sql.table.TableInfo.PARALLELISM_KEY;
+
+/**
+ * Reason:
+ * Date: 2018/11/22
+ *
+ * @author xuqianjin
+ */
+public class CassandraSinkParser extends AbsTableParser {
+
+ public static final String ADDRESS_KEY = "address";
+
+ public static final String TABLE_NAME_KEY = "tableName";
+
+ public static final String USER_NAME_KEY = "userName";
+
+ public static final String PASSWORD_KEY = "password";
+
+ public static final String DATABASE_KEY = "database";
+
+ public static final String MAX_REQUEST_PER_CONNECTION_KEY = "maxRequestsPerConnection";
+
+ public static final String CORE_CONNECTIONS_PER_HOST_KEY = "coreConnectionsPerHost";
+
+ public static final String MAX_CONNECTIONS_PER_HOST_KEY = "maxConnectionsPerHost";
+
+ public static final String MAX_QUEUE_SIZE_KEY = "maxQueueSize";
+
+ public static final String READ_TIMEOUT_MILLIS_KEY = "readTimeoutMillis";
+
+ public static final String CONNECT_TIMEOUT_MILLIS_KEY = "connectTimeoutMillis";
+
+ public static final String POOL_TIMEOUT_MILLIS_KEY = "poolTimeoutMillis";
+
+ @Override
+ public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) {
+ CassandraTableInfo cassandraTableInfo = new CassandraTableInfo();
+ cassandraTableInfo.setName(tableName);
+ parseFieldsInfo(fieldsInfo, cassandraTableInfo);
+
+ cassandraTableInfo.setParallelism(MathUtil.getIntegerVal(props.get(PARALLELISM_KEY.toLowerCase())));
+ cassandraTableInfo.setAddress(MathUtil.getString(props.get(ADDRESS_KEY.toLowerCase())));
+ cassandraTableInfo.setTableName(MathUtil.getString(props.get(TABLE_NAME_KEY.toLowerCase())));
+ cassandraTableInfo.setDatabase(MathUtil.getString(props.get(DATABASE_KEY.toLowerCase())));
+ cassandraTableInfo.setUserName(MathUtil.getString(props.get(USER_NAME_KEY.toLowerCase())));
+ cassandraTableInfo.setPassword(MathUtil.getString(props.get(PASSWORD_KEY.toLowerCase())));
+ cassandraTableInfo.setMaxRequestsPerConnection(MathUtil.getIntegerVal(props.get(MAX_REQUEST_PER_CONNECTION_KEY.toLowerCase())));
+ cassandraTableInfo.setCoreConnectionsPerHost(MathUtil.getIntegerVal(props.get(CORE_CONNECTIONS_PER_HOST_KEY.toLowerCase())));
+ cassandraTableInfo.setMaxConnectionsPerHost(MathUtil.getIntegerVal(props.get(MAX_CONNECTIONS_PER_HOST_KEY.toLowerCase())));
+ cassandraTableInfo.setMaxQueueSize(MathUtil.getIntegerVal(props.get(MAX_QUEUE_SIZE_KEY.toLowerCase())));
+ cassandraTableInfo.setReadTimeoutMillis(MathUtil.getIntegerVal(props.get(READ_TIMEOUT_MILLIS_KEY.toLowerCase())));
+ cassandraTableInfo.setConnectTimeoutMillis(MathUtil.getIntegerVal(props.get(CONNECT_TIMEOUT_MILLIS_KEY.toLowerCase())));
+ cassandraTableInfo.setPoolTimeoutMillis(MathUtil.getIntegerVal(props.get(POOL_TIMEOUT_MILLIS_KEY.toLowerCase())));
+
+ return cassandraTableInfo;
+ }
+}
diff --git a/cassandra/cassandra-sink/src/main/java/com/dtstack/flink/sql/sink/cassandra/table/CassandraTableInfo.java b/cassandra/cassandra-sink/src/main/java/com/dtstack/flink/sql/sink/cassandra/table/CassandraTableInfo.java
new file mode 100644
index 000000000..7d52b23bb
--- /dev/null
+++ b/cassandra/cassandra-sink/src/main/java/com/dtstack/flink/sql/sink/cassandra/table/CassandraTableInfo.java
@@ -0,0 +1,161 @@
+/*
+ * 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.dtstack.flink.sql.sink.cassandra.table;
+
+import com.dtstack.flink.sql.table.TargetTableInfo;
+import org.apache.flink.calcite.shaded.com.google.common.base.Preconditions;
+
+/**
+ * Reason:
+ * Date: 2018/11/22
+ *
+ * @author xuqianjin
+ */
+public class CassandraTableInfo extends TargetTableInfo {
+
+ private static final String CURR_TYPE = "cassandra";
+
+ private String address;
+ private String tableName;
+ private String userName;
+ private String password;
+ private String database;
+ private Integer maxRequestsPerConnection;
+ private Integer coreConnectionsPerHost;
+ private Integer maxConnectionsPerHost;
+ private Integer maxQueueSize;
+ private Integer readTimeoutMillis;
+ private Integer connectTimeoutMillis;
+ private Integer poolTimeoutMillis;
+
+ public CassandraTableInfo() {
+ setType(CURR_TYPE);
+ }
+
+ public String getAddress() {
+ return address;
+ }
+
+ public void setAddress(String address) {
+ this.address = address;
+ }
+
+ public String getDatabase() {
+ return database;
+ }
+
+ public void setDatabase(String database) {
+ this.database = database;
+ }
+
+ public String getTableName() {
+ return tableName;
+ }
+
+ public void setTableName(String tableName) {
+ this.tableName = tableName;
+ }
+
+ public String getUserName() {
+ return userName;
+ }
+
+ public void setUserName(String userName) {
+ this.userName = userName;
+ }
+
+ public String getPassword() {
+ return password;
+ }
+
+ public void setPassword(String password) {
+ this.password = password;
+ }
+
+ public Integer getMaxRequestsPerConnection() {
+ return maxRequestsPerConnection;
+ }
+
+ public void setMaxRequestsPerConnection(Integer maxRequestsPerConnection) {
+ this.maxRequestsPerConnection = maxRequestsPerConnection;
+ }
+
+ public Integer getCoreConnectionsPerHost() {
+ return coreConnectionsPerHost;
+ }
+
+ public void setCoreConnectionsPerHost(Integer coreConnectionsPerHost) {
+ this.coreConnectionsPerHost = coreConnectionsPerHost;
+ }
+
+ public Integer getMaxConnectionsPerHost() {
+ return maxConnectionsPerHost;
+ }
+
+ public void setMaxConnectionsPerHost(Integer maxConnectionsPerHost) {
+ this.maxConnectionsPerHost = maxConnectionsPerHost;
+ }
+
+ public Integer getMaxQueueSize() {
+ return maxQueueSize;
+ }
+
+ public void setMaxQueueSize(Integer maxQueueSize) {
+ this.maxQueueSize = maxQueueSize;
+ }
+
+ public Integer getReadTimeoutMillis() {
+ return readTimeoutMillis;
+ }
+
+ public void setReadTimeoutMillis(Integer readTimeoutMillis) {
+ this.readTimeoutMillis = readTimeoutMillis;
+ }
+
+ public Integer getConnectTimeoutMillis() {
+ return connectTimeoutMillis;
+ }
+
+ public void setConnectTimeoutMillis(Integer connectTimeoutMillis) {
+ this.connectTimeoutMillis = connectTimeoutMillis;
+ }
+
+ public Integer getPoolTimeoutMillis() {
+ return poolTimeoutMillis;
+ }
+
+ public void setPoolTimeoutMillis(Integer poolTimeoutMillis) {
+ this.poolTimeoutMillis = poolTimeoutMillis;
+ }
+
+ @Override
+ public boolean check() {
+ Preconditions.checkNotNull(address, "Cassandra field of ADDRESS is required");
+ Preconditions.checkNotNull(database, "Cassandra field of database is required");
+ Preconditions.checkNotNull(tableName, "Cassandra field of tableName is required");
+ return true;
+ }
+
+ @Override
+ public String getType() {
+ // return super.getType().toLowerCase() + TARGET_SUFFIX;
+ return super.getType().toLowerCase();
+ }
+}
diff --git a/cassandra/cassandra-sink/src/test/java/com/dtstack/flinkx/AppTest.java b/cassandra/cassandra-sink/src/test/java/com/dtstack/flinkx/AppTest.java
new file mode 100644
index 000000000..33a0233ac
--- /dev/null
+++ b/cassandra/cassandra-sink/src/test/java/com/dtstack/flinkx/AppTest.java
@@ -0,0 +1,58 @@
+/*
+ * 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.dtstack.flinkx;
+
+import junit.framework.Test;
+import junit.framework.TestCase;
+import junit.framework.TestSuite;
+
+/**
+ * Unit test for simple App.
+ */
+public class AppTest
+ extends TestCase
+{
+ /**
+ * Create the test case
+ *
+ * @param testName name of the test case
+ */
+ public AppTest( String testName )
+ {
+ super( testName );
+ }
+
+ /**
+ * @return the suite of tests being tested
+ */
+ public static Test suite()
+ {
+ return new TestSuite( AppTest.class );
+ }
+
+ /**
+ * Rigourous Test :-)
+ */
+ public void testApp()
+ {
+ assertTrue( true );
+ }
+}
diff --git a/cassandra/pom.xml b/cassandra/pom.xml
new file mode 100644
index 000000000..f49de388b
--- /dev/null
+++ b/cassandra/pom.xml
@@ -0,0 +1,39 @@
+
+
+
+ flink.sql
+ com.dtstack.flink
+ 1.0-SNAPSHOT
+
+ 4.0.0
+ sql.cassandra
+ pom
+
+
+ cassandra-sink
+ cassandra-side
+
+
+
+
+ junit
+ junit
+ 3.8.1
+ test
+
+
+ com.dtstack.flink
+ sql.core
+ 1.0-SNAPSHOT
+ provided
+
+
+ com.datastax.cassandra
+ cassandra-driver-core
+ 3.6.0
+
+
+
+
\ No newline at end of file
diff --git a/docs/cassandraSide.md b/docs/cassandraSide.md
new file mode 100644
index 000000000..131560047
--- /dev/null
+++ b/docs/cassandraSide.md
@@ -0,0 +1,85 @@
+
+## 1.格式:
+```
+ CREATE TABLE tableName(
+ colName cloType,
+ ...
+ PRIMARY KEY(keyInfo),
+ PERIOD FOR SYSTEM_TIME
+ )WITH(
+ type ='cassandra',
+ address ='ip:port[,ip:port]',
+ userName='dbUserName',
+ password='dbPwd',
+ tableName='tableName',
+ database='database',
+ cache ='LRU',
+ cacheSize ='10000',
+ cacheTTLMs ='60000',
+ parallelism ='1',
+ partitionedJoin='false'
+ );
+```
+
+# 2.支持版本
+ cassandra-3.6.x
+
+## 3.表结构定义
+
+ |参数名称|含义|
+ |----|---|
+ | tableName | 注册到flink的表名称(可选填;不填默认和hbase对应的表名称相同)|
+ | colName | 列名称|
+ | colType | 列类型 [colType支持的类型](colType.md)|
+ | PERIOD FOR SYSTEM_TIME | 关键字表明该定义的表为维表信息|
+ | PRIMARY KEY(keyInfo) | 维表主键定义;多个列之间用逗号隔开|
+
+## 4.参数
+
+ |参数名称|含义|是否必填|默认值|
+ |----|---|---|----|
+ | type |表明 输出表类型 cassandra|是||
+ | address | 连接cassandra数据库 jdbcUrl |是||
+ | userName | cassandra连接用户名|否||
+ | password | cassandra连接密码|否||
+ | tableName | cassandra表名称|是||
+ | database | cassandra表名称|是||
+ | cache | 维表缓存策略(NONE/LRU)|否|NONE|
+ | partitionedJoin | 是否在維表join之前先根据 設定的key 做一次keyby操作(可以減少维表的数据缓存量)|否|false|
+ | maxRequestsPerConnection | 每个连接最多允许64个并发请求|否|NONE|
+ | coreConnectionsPerHost | 和Cassandra集群里的每个机器都至少有2个连接|否|NONE|
+ | maxConnectionsPerHost | 和Cassandra集群里的每个机器都最多有6个连接|否|NONE|
+ | maxQueueSize | Cassandra队列大小|否|NONE|
+ | readTimeoutMillis | Cassandra读超时|否|NONE|
+ | connectTimeoutMillis | Cassandra连接超时|否|NONE|
+ | poolTimeoutMillis | Cassandra线程池超时|否|NONE|
+
+ ----------
+ > 缓存策略
+ * NONE: 不做内存缓存
+ * LRU:
+ * cacheSize: 缓存的条目数量
+ * cacheTTLMs:缓存的过期时间(ms)
+
+
+## 5.样例
+```
+create table sideTable(
+ CHANNEL varchar,
+ XCCOUNT int,
+ PRIMARY KEY(channel),
+ PERIOD FOR SYSTEM_TIME
+ )WITH(
+ type ='cassandra',
+ address ='172.21.32.1:9042,172.21.32.1:9042',
+ database ='test',
+ tableName ='sidetest',
+ cache ='LRU',
+ parallelism ='1',
+ partitionedJoin='false'
+ );
+
+
+```
+
+
diff --git a/docs/cassandraSink.md b/docs/cassandraSink.md
new file mode 100644
index 000000000..8ea38e104
--- /dev/null
+++ b/docs/cassandraSink.md
@@ -0,0 +1,63 @@
+## 1.格式:
+```
+CREATE TABLE tableName(
+ colName colType,
+ ...
+ colNameX colType
+ )WITH(
+ type ='cassandra',
+ address ='ip:port[,ip:port]',
+ userName ='userName',
+ password ='pwd',
+ database ='databaseName',
+ tableName ='tableName',
+ parallelism ='parllNum'
+ );
+
+```
+
+## 2.支持版本
+ cassandra-3.6.x
+
+## 3.表结构定义
+
+|参数名称|含义|
+|----|---|
+| tableName| 在 sql 中使用的名称;即注册到flink-table-env上的名称|
+| colName | 列名称|
+| colType | 列类型 [colType支持的类型](colType.md)|
+
+## 4.参数:
+
+|参数名称|含义|是否必填|默认值|
+|----|----|----|----|
+|type |表明 输出表类型 cassandra|是||
+|address | 连接cassandra数据库 jdbcUrl |是||
+|userName | cassandra连接用户名|否||
+|password | cassandra连接密码|否||
+|tableName | cassandra表名称|是||
+|database | cassandra表名称|是||
+|parallelism | 并行度设置|否|1|
+|maxRequestsPerConnection | 每个连接最多允许64个并发请求|否|NONE|
+|coreConnectionsPerHost | 和Cassandra集群里的每个机器都至少有2个连接|否|NONE|
+|maxConnectionsPerHost | 和Cassandra集群里的每个机器都最多有6个连接|否|NONE|
+|maxQueueSize | Cassandra队列大小|否|NONE|
+|readTimeoutMillis | Cassandra读超时|否|NONE|
+|connectTimeoutMillis | Cassandra连接超时|否|NONE|
+|poolTimeoutMillis | Cassandra线程池超时|否|NONE|
+
+## 5.样例:
+```
+CREATE TABLE MyResult(
+ channel VARCHAR,
+ pv VARCHAR
+ )WITH(
+ type ='cassandra',
+ address ='172.21.32.1:9042,172.21.32.1:9042',
+ userName ='dtstack',
+ password ='abc123',
+ database ='test',
+ tableName ='pv',
+ parallelism ='1'
+ )
+ ```
\ No newline at end of file
diff --git a/mongo/mongo-side/mongo-all-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAllReqRow.java b/mongo/mongo-side/mongo-all-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAllReqRow.java
index c1c812cd0..d1a38ca5f 100644
--- a/mongo/mongo-side/mongo-all-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAllReqRow.java
+++ b/mongo/mongo-side/mongo-all-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAllReqRow.java
@@ -152,7 +152,6 @@ public void flatMap(Row value, Collector out) throws Exception {
}
for (Map one : cacheList) {
- System.out.println(fillData(value, one));
out.collect(fillData(value, one));
}
}
diff --git a/pom.xml b/pom.xml
index a31cc7392..7ef6c1fdd 100644
--- a/pom.xml
+++ b/pom.xml
@@ -5,6 +5,9 @@
com.dtstack.flink
flink.sql
1.0-SNAPSHOT
+ pom
+ flink.sql
+ http://maven.apache.org
core
kafka09
@@ -19,11 +22,9 @@
rdb
sqlserver
oracle
+ cassandra
- pom
- flink.sql
- http://maven.apache.org
UTF-8