Skip to content
Draft
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
Original file line number Diff line number Diff line change
@@ -0,0 +1,109 @@
/*
* 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.wayang.basic.operators;

import org.apache.wayang.basic.data.Record;
import org.apache.wayang.basic.types.RecordType;
import org.apache.wayang.core.plan.wayangplan.UnarySink;
import org.apache.wayang.core.types.DataSetType;

import java.util.Properties;

/**
* {@link UnarySink} that writes Records to a database table.
*/

public class TableSink extends UnarySink<Record> {
private final String tableName;

private String[] columnNames;

private final Properties props;

private String mode;

/**
* Creates a new instance.
*
* @param props database connection properties
* @param tableName name of the table to be written
* @param columnNames names of the columns in the tables
*/
public TableSink(Properties props, String mode, String tableName, String... columnNames) {
this(props, mode, tableName, columnNames, DataSetType.createDefault(Record.class));
}

public TableSink(Properties props, String mode, String tableName, String[] columnNames, DataSetType<Record> type) {
super(type);
this.tableName = tableName;
this.columnNames = columnNames;
this.props = props;
this.mode = mode;
}

/**
* Copies an instance (exclusive of broadcasts).
*
* @param that that should be copied
*/
public TableSink(TableSink that) {
super(that);
this.tableName = that.getTableName();
this.columnNames = that.getColumnNames();
this.props = that.getProperties();
this.mode = that.getMode();
}

public String getTableName() {
return this.tableName;
}

protected void setColumnNames(String[] columnNames) {
this.columnNames = columnNames;
}

public String[] getColumnNames() {
return this.columnNames;
}

public Properties getProperties() {
return this.props;
}

public String getMode() {
return mode;
}

public void setMode(String mode) {
this.mode = mode;
}

/**
* Constructs an appropriate output {@link DataSetType} for the given column names.
*
* @param columnNames the column names or an empty array if unknown
* @return the output {@link DataSetType}, which will be based upon a {@link RecordType} unless no {@code columnNames}
* is empty
*/
private static DataSetType<Record> createOutputDataSetType(String[] columnNames) {
return columnNames.length == 0 ?
DataSetType.createDefault(Record.class) :
DataSetType.createDefault(new RecordType(columnNames));
}
}
6 changes: 6 additions & 0 deletions wayang-platforms/wayang-java/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -78,6 +78,12 @@
<artifactId>log4j-slf4j-impl</artifactId>
<version>2.20.0</version>
</dependency>
<dependency>
<groupId>org.postgresql</groupId>
<artifactId>postgresql</artifactId>
<version>42.7.2</version>
<scope>test</scope>
</dependency>
<!-- Mockito for mocking -->
<dependency>
<groupId>org.mockito</groupId>
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,203 @@
/*
* 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.wayang.java.operators;

import org.apache.wayang.basic.data.Record;
import org.apache.wayang.basic.operators.TableSink;
import org.apache.wayang.core.optimizer.OptimizationContext;
import org.apache.wayang.core.plan.wayangplan.ExecutionOperator;
import org.apache.wayang.core.platform.ChannelDescriptor;
import org.apache.wayang.core.platform.ChannelInstance;
import org.apache.wayang.core.platform.lineage.ExecutionLineageNode;
import org.apache.wayang.core.types.DataSetType;
import org.apache.wayang.core.util.Tuple;
import org.apache.wayang.java.channels.CollectionChannel;
import org.apache.wayang.java.channels.JavaChannelInstance;
import org.apache.wayang.java.channels.StreamChannel;
import org.apache.wayang.java.execution.JavaExecutor;

import java.sql.Connection;
import java.sql.DriverManager;
import java.sql.PreparedStatement;
import java.sql.SQLException;
import java.sql.Statement;
import java.util.Arrays;
import java.util.Collection;
import java.util.Iterator;
import java.util.List;
import java.util.Properties;


public class JavaTableSink extends TableSink implements JavaExecutionOperator {

private void setRecordValue(PreparedStatement ps, int index, Object value) throws SQLException {
if (value == null) {
ps.setNull(index, java.sql.Types.NULL);
} else if (value instanceof Integer) {
ps.setInt(index, (Integer) value);
} else if (value instanceof Long) {
ps.setLong(index, (Long) value);
} else if (value instanceof Double) {
ps.setDouble(index, (Double) value);
} else if (value instanceof Float) {
ps.setFloat(index, (Float) value);
} else if (value instanceof Boolean) {
ps.setBoolean(index, (Boolean) value);
} else {
ps.setString(index, value.toString());
}
}

public JavaTableSink(Properties props, String mode, String tableName) {
this(props, mode, tableName, null);
}

public JavaTableSink(Properties props, String mode, String tableName, String... columnNames) {
super(props, mode, tableName, columnNames);

}

public JavaTableSink(Properties props, String mode, String tableName, String[] columnNames, DataSetType<Record> type) {
super(props, mode, tableName, columnNames, type);

}

public JavaTableSink(TableSink that) {
super(that);
}

@Override
public Tuple<Collection<ExecutionLineageNode>, Collection<ChannelInstance>> evaluate(
ChannelInstance[] inputs,
ChannelInstance[] outputs,
JavaExecutor javaExecutor,
OptimizationContext.OperatorContext operatorContext) {
assert inputs.length == 1;
assert outputs.length == 0;
JavaChannelInstance input = (JavaChannelInstance) inputs[0];

// The stream is converted to an Iterator so that we can read the first element w/o consuming the entire stream.
Iterator<Record> recordIterator = input.<Record>provideStream().iterator();
// We read the first element to derive the Record schema.
Record schemaRecord = recordIterator.next();

// We assume that all records have the same length and only check the first record.
int recordLength = schemaRecord.size();
if (this.getColumnNames() != null) {
assert recordLength == this.getColumnNames().length;
} else {
String[] columnNames = new String[recordLength];
for (int i = 0; i < recordLength; i++) {
columnNames[i] = "c_" + i;
}
this.setColumnNames(columnNames);
}

// TODO: Check if we need this property.
this.getProperties().setProperty("streamingBatchInsert", "True");

Connection conn;
try {
Class.forName(this.getProperties().getProperty("driver"));
conn = DriverManager.getConnection(this.getProperties().getProperty("url"), this.getProperties());
conn.setAutoCommit(false);

Statement stmt = conn.createStatement();

// Drop existing table if the mode is 'overwrite'.
if (this.getMode().equals("overwrite")) {
stmt.execute("DROP TABLE IF EXISTS " + this.getTableName());
}

// Create a new table if the specified table name does not exist yet.
StringBuilder sb = new StringBuilder();
sb.append("CREATE TABLE IF NOT EXISTS ").append(this.getTableName()).append(" (");
String separator = "";
for (int i = 0; i < recordLength; i++) {
sb.append(separator).append(this.getColumnNames()[i]).append(" VARCHAR(255)");
separator = ", ";
}
sb.append(")");
stmt.execute(sb.toString());

// Create a prepared statement to insert value from the recordIterator.
sb = new StringBuilder();
sb.append("INSERT INTO ").append(this.getTableName()).append(" (");
separator = "";
for (int i = 0; i < recordLength; i++) {
sb.append(separator).append(this.getColumnNames()[i]);
separator = ", ";
}
sb.append(") VALUES (");
separator = "";
for (int i = 0; i < recordLength; i++) {
sb.append(separator).append("?");
separator = ", ";
}
sb.append(")");
PreparedStatement ps = conn.prepareStatement(sb.toString());

// The schema Record has to be pushed to the database too.
for (int i = 0; i < recordLength; i++) {
setRecordValue(ps, i + 1, schemaRecord.getField(i));
}
ps.addBatch();

// Iterate through all remaining records and add them to the prepared statement
recordIterator.forEachRemaining(
r -> {
try {
for (int i = 0; i < recordLength; i++) {
setRecordValue(ps, i + 1, r.getField(i));
}
ps.addBatch();
} catch (SQLException e) {
e.printStackTrace();
}
}
);

ps.executeBatch();
conn.commit();
conn.close();
} catch (ClassNotFoundException e) {
System.out.println("Please specify a correct database driver.");
e.printStackTrace();
} catch (SQLException e) {
e.printStackTrace();
}

return ExecutionOperator.modelEagerExecution(inputs, outputs, operatorContext);
}

@Override
public String getLoadProfileEstimatorConfigurationKey() {
return "rheem.java.tablesink.load";
}

@Override
public List<ChannelDescriptor> getSupportedInputChannels(int index) {
return Arrays.asList(CollectionChannel.DESCRIPTOR, StreamChannel.DESCRIPTOR);
}

@Override
public List<ChannelDescriptor> getSupportedOutputChannels(int index) {
throw new UnsupportedOperationException("This operator has no outputs.");
}
}
Loading
Loading