[GitHub] jiazhai commented on a change in pull request #2440: Issue 2313: create a JDBC sink connector

2018-08-27 Thread GitBox
jiazhai commented on a change in pull request #2440: Issue 2313: create a JDBC 
sink connector
URL: https://github.com/apache/incubator-pulsar/pull/2440#discussion_r213151503
 
 

 ##
 File path: 
tests/integration/src/test/java/org/apache/pulsar/tests/integration/suites/PulsarTestSuite.java
 ##
 @@ -60,10 +62,19 @@ protected PulsarClusterSpecBuilder 
beforeSetupCluster(String clusterName, Pulsar
 .withCreateContainerCmdModifier(createContainerCmd -> 
createContainerCmd
 .withName(kafkaServiceName)
 .withHostName(clusterName + "-" + kafkaServiceName)));
+
 final String cassandraServiceName = "cassandra";
 externalServices.put(
 cassandraServiceName,
 new CassandraContainer(clusterName));
+
+// use mySQL for jdbc test
+final String jdbcServiceName = "jdbc";
+externalServices.put(
+jdbcServiceName,
+new MySQLContainer()
 
 Review comment:
   Thanks, this is already done in PulsarCluster


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] jiazhai commented on a change in pull request #2440: Issue 2313: create a JDBC sink connector

2018-08-27 Thread GitBox
jiazhai commented on a change in pull request #2440: Issue 2313: create a JDBC 
sink connector
URL: https://github.com/apache/incubator-pulsar/pull/2440#discussion_r213151155
 
 

 ##
 File path: 
pulsar-io/jdbc/src/main/java/org/apache/pulsar/io/jdbc/JdbcUtils.java
 ##
 @@ -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 org.apache.pulsar.io.jdbc;
+
+import static com.google.common.base.Preconditions.checkState;
+
+import com.google.common.collect.Lists;
+import java.sql.Connection;
+import java.sql.DatabaseMetaData;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Properties;
+import java.util.stream.IntStream;
+import lombok.Data;
+import lombok.EqualsAndHashCode;
+import lombok.Getter;
+import lombok.Setter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+
+/**
+ * Jdbc Utils
+ */
+@Slf4j
+public class JdbcUtils {
+
+@Data(staticConstructor = "of")
+@Setter
+@Getter
+@EqualsAndHashCode
+@ToString
+public static class TableId {
+private final String catalogName;
+private final String schemaName;
+private final String tableName;
+}
+
+@Data(staticConstructor = "of")
+@Setter
+@Getter
+@EqualsAndHashCode
+@ToString
+public static class ColumnId {
+private final TableId tableId;
+private final String name;
+// SQL type from java.sql.Types
+private final int type;
+private final String typeName;
+// column position in table
+private final int position;
+}
+
+@Data(staticConstructor = "of")
+@Setter
+@Getter
+@EqualsAndHashCode
+@ToString
+public static class TableDefinition {
+private final TableId tableId;
+private final List columns;
+}
+
+/**
+ * Given a driver type(such as mysql), return its jdbc driver class name.
+ * TODO: test and support more types
+ */
+public static String getDriverClassName(String driver) throws Exception {
+if (driver.equals("mysql")) {
+return "com.mysql.jdbc.Driver";
+} if (driver.equals("sqlite")) {
+return "org.sqlite.JDBC";
+} else {
+throw new Exception("Not tested jdbc driver type: " + driver);
 
 Review comment:
   Thanks, we may better tested before add more types, fail early is better. 
For new type, we should also add dependency in pom file.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] jiazhai commented on a change in pull request #2440: Issue 2313: create a JDBC sink connector

2018-08-27 Thread GitBox
jiazhai commented on a change in pull request #2440: Issue 2313: create a JDBC 
sink connector
URL: https://github.com/apache/incubator-pulsar/pull/2440#discussion_r213150850
 
 

 ##
 File path: 
pulsar-io/jdbc/src/main/java/org/apache/pulsar/io/jdbc/JdbcAbstractSink.java
 ##
 @@ -0,0 +1,190 @@
+/**
+ * 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.pulsar.io.jdbc;
+
+import static 
jersey.repackaged.com.google.common.base.Preconditions.checkState;
+
+import com.google.common.collect.Lists;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.Statement;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import lombok.Getter;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.functions.api.Record;
+import org.apache.pulsar.io.core.Sink;
+import org.apache.pulsar.io.core.SinkContext;
+
+/**
+ * A Simple abstract class for Jdbc sink
+ * Users need to implement extractKeyValue function to use this sink
+ */
+@Slf4j
+public abstract class JdbcAbstractSink implements Sink {
+// - Runtime fields
+private JdbcSinkConfig jdbcSinkConfig;
+@Getter
+private Connection connection;
+private String tableName;
+
+private JdbcUtils.TableId tableId;
+private PreparedStatement insertStatement;
+
+// TODO: turn to getSchema from 
SinkContext.getTopicSchema.getSchema(inputTopic)
+protected String schema;
+protected JdbcUtils.TableDefinition tableDefinition;
+
+// for flush
+private List> incomingList;
+private List> swapList;
+private AtomicBoolean isFlushing;
+private int timeoutMs;
+private int batchSize;
+private ScheduledExecutorService flushExecutor;
+
+@Override
+public void open(Map config, SinkContext sinkContext) 
throws Exception {
+jdbcSinkConfig = JdbcSinkConfig.load(config);
+
+String jdbcUrl = jdbcSinkConfig.getJdbcUrl();
+if (jdbcSinkConfig.getJdbcUrl() == null) {
+throw new IllegalArgumentException("Required jdbc Url not set.");
+}
+
+Properties properties = new Properties();
+String username = jdbcSinkConfig.getUserName();
+String password = jdbcSinkConfig.getPassword();
+if (username != null) {
+properties.setProperty("user", username);
+}
+if (password != null) {
+properties.setProperty("password", password);
+}
+
+connection = JdbcUtils.getConnection(jdbcUrl, properties);
+log.info("Connection opened");
+
+schema = jdbcSinkConfig.getSchema();
+tableName = jdbcSinkConfig.getTableName();
+tableId = JdbcUtils.getTableId(connection, tableName);
+tableDefinition = JdbcUtils.getTableDefinition(connection, tableId);
+insertStatement = JdbcUtils.buildInsertStatement(connection, 
JdbcUtils.buildInsertSql(tableDefinition));
+
+timeoutMs = jdbcSinkConfig.getTimeoutMs();
+batchSize = jdbcSinkConfig.getBatchSize();
+incomingList = Lists.newArrayList();
+swapList = Lists.newArrayList();
+isFlushing = new AtomicBoolean(false);
+
+flushExecutor = Executors.newScheduledThreadPool(1);
+flushExecutor.scheduleAtFixedRate(() -> flush(), timeoutMs, timeoutMs, 
TimeUnit.MILLISECONDS);
+}
+
+@Override
+public void close() throws Exception {
+if (!connection.getAutoCommit()) {
+connection.commit();
+}
+flushExecutor.shutdown();
+if (connection != null) {
+connection.close();
+}
+log.info("Connection Closed");
+}
+
+@Override
+public void write(Record record) throws Exception {
+int number;
+synchronized (incomingList) {
+incomingList.add(record);
+number = incomingList.size();
+}
+
+if (number == batchSize) {
+flushExecutor.schedule(() -> flush(), 0, TimeUnit.MILLISECONDS);
+}
+}
+
+// bind value with a 

[GitHub] jiazhai commented on a change in pull request #2440: Issue 2313: create a JDBC sink connector

2018-08-25 Thread GitBox
jiazhai commented on a change in pull request #2440: Issue 2313: create a JDBC 
sink connector
URL: https://github.com/apache/incubator-pulsar/pull/2440#discussion_r212812761
 
 

 ##
 File path: 
pulsar-io/jdbc/src/main/java/org/apache/pulsar/io/jdbc/JdbcAbstractSink.java
 ##
 @@ -0,0 +1,187 @@
+/**
+ * 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.pulsar.io.jdbc;
+
+import static 
jersey.repackaged.com.google.common.base.Preconditions.checkState;
+
+import com.google.common.collect.Lists;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.Statement;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import lombok.Getter;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.functions.api.Record;
+import org.apache.pulsar.io.core.Sink;
+import org.apache.pulsar.io.core.SinkContext;
+import org.apache.pulsar.io.jdbc.JdbcUtils.TableDefinition;
+
+/**
+ * A Simple abstract class for Jdbc sink
+ * Users need to implement extractKeyValue function to use this sink
+ */
+@Slf4j
+public abstract class JdbcAbstractSink implements Sink {
+// - Runtime fields
+private JdbcSinkConfig jdbcSinkConfig;
+@Getter
+private Connection connection;
+private String tableName;
+private String schema;
+
+private JdbcUtils.TableId tableId;
+private JdbcUtils.TableDefinition tableDefinition;
+private PreparedStatement insertStatement;
+
+// for flush
+private List> incomingList;
+private List> swapList;
+private AtomicBoolean isFlushing;
+private int timeoutMs;
+private int batchSize;
+private ScheduledExecutorService flushExecutor;
+
+@Override
+public void open(Map config, SinkContext sinkContext) 
throws Exception {
+// TODO: currently assume schema is same as created table schema;
+// turn to getSchema from 
SinkContext.getTopicSchema.getSchema(inputTopic)?
+
+jdbcSinkConfig = JdbcSinkConfig.load(config);
+
+String jdbcUrl = jdbcSinkConfig.getJdbcUrl();
+if (jdbcSinkConfig.getJdbcUrl() == null) {
+throw new IllegalArgumentException("Required jdbc Url not set.");
+}
+
+Properties properties = new Properties();
+String username = jdbcSinkConfig.getUserName();
+String password = jdbcSinkConfig.getPassword();
+if (username != null) {
+properties.setProperty("user", username);
+}
+if (password != null) {
+properties.setProperty("password", password);
+}
+
+connection = JdbcUtils.getConnection(jdbcUrl, properties);
+log.info("Connection open");
+
+schema = jdbcSinkConfig.getSchema();
+tableName = jdbcSinkConfig.getTableName();
+tableId = JdbcUtils.getTableId(connection, tableName);
+tableDefinition = JdbcUtils.getTableDefinition(connection, tableId);
+insertStatement = JdbcUtils.buildInsertStatement(connection, 
JdbcUtils.buildInsertSql(tableDefinition));
+
+timeoutMs = jdbcSinkConfig.getTimeoutMs();
+batchSize = jdbcSinkConfig.getBatchSize();
+incomingList = Lists.newArrayList();
+swapList = Lists.newArrayList();
+isFlushing = new AtomicBoolean(false);
+
+flushExecutor = Executors.newScheduledThreadPool(1);
+flushExecutor.scheduleAtFixedRate(() -> flush(), timeoutMs, timeoutMs, 
TimeUnit.MILLISECONDS);
+}
+
+@Override
+public void close() throws Exception {
+if (!connection.getAutoCommit()) {
+connection.commit();
+}
+flushExecutor.shutdown();
+if (connection != null) {
+connection.close();
+}
+log.info("Connection Closed");
+}
+
+@Override
+public void write(Record record) throws Exception{
+synchronized (incomingList) {
+incomingList.add(record);
+}
+
+if (incomingList.size() >= batchSize) {
+flushExecutor.schedule(() ->