Skip to content

[PLUGIN-1815] Commit/Rollback added in Committer. #584

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

Open
wants to merge 1 commit into
base: develop
Choose a base branch
from
Open
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
Expand Up @@ -203,6 +203,7 @@ public void prepareRun(BatchSinkContext context) {
configAccessor.setInitQueries(dbSinkConfig.getInitQueries());
configAccessor.getConfiguration().set(DBConfiguration.DRIVER_CLASS_PROPERTY, driverClass.getName());
configAccessor.getConfiguration().set(DBConfiguration.URL_PROPERTY, connectionString);
configAccessor.getConfiguration().set(ETLDBOutputFormat.STAGE_NAME, context.getStageName());
String fullyQualifiedTableName = dbSchemaName == null ? dbSinkConfig.getEscapedTableName()
: dbSinkConfig.getEscapedDbSchemaName() + "." + dbSinkConfig.getEscapedTableName();
configAccessor.getConfiguration().set(DBConfiguration.OUTPUT_TABLE_NAME_PROPERTY, fullyQualifiedTableName);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,8 @@
import io.cdap.plugin.db.TransactionIsolationLevel;
import io.cdap.plugin.util.DBUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.mapreduce.JobContext;
import org.apache.hadoop.mapreduce.OutputCommitter;
import org.apache.hadoop.mapreduce.RecordWriter;
import org.apache.hadoop.mapreduce.TaskAttemptContext;
import org.apache.hadoop.mapreduce.lib.db.DBConfiguration;
Expand All @@ -43,6 +45,7 @@
import java.sql.Statement;
import java.util.Map;
import java.util.Properties;
import java.util.concurrent.ConcurrentHashMap;

import static io.cdap.plugin.db.ConnectionConfigAccessor.OPERATION_NAME;
import static io.cdap.plugin.db.ConnectionConfigAccessor.RELATION_TABLE_KEY;
Expand All @@ -56,15 +59,92 @@
public class ETLDBOutputFormat<K extends DBWritable, V> extends DBOutputFormat<K, V> {
// Batch size before submitting a batch to the SQL engine. If set to 0, no batches will be submitted until commit.
public static final String COMMIT_BATCH_SIZE = "io.cdap.plugin.db.output.commit.batch.size";
public static final String STAGE_NAME = "io.cdap.plugin.db.output.stage_name";
public static final int DEFAULT_COMMIT_BATCH_SIZE = 1000;
private static final Character ESCAPE_CHAR = '"';

// Format for connection map's key will be "taskAttemptId_stageName"
private static final String CONNECTION_MAP_KEY_FORMAT = "%s_%s";

// CONNECTION_MAP will be used to store connections with "taskAttemptId_stageName" as key and
// connection object as value. Making it static to be accessed from multiple task attempts within same executor.
private static final Map<String, Connection> CONNECTION_MAP = new ConcurrentHashMap<>();
private static final Logger LOG = LoggerFactory.getLogger(ETLDBOutputFormat.class);

private Configuration conf;
private Driver driver;
private JDBCDriverShim driverShim;

@Override
public OutputCommitter getOutputCommitter(TaskAttemptContext context)
throws IOException, InterruptedException {
return new OutputCommitter() {
@Override
public void setupJob(JobContext jobContext) throws IOException {
// do nothing
}

@Override
public void setupTask(TaskAttemptContext taskContext) throws IOException {
// do nothing
}

@Override
public boolean needsTaskCommit(TaskAttemptContext taskContext) throws IOException {
return true;
}

@Override
public void commitTask(TaskAttemptContext taskContext) throws IOException {
conf = context.getConfiguration();
String stageName = conf.get(STAGE_NAME);
String connectionId = getConnectionMapKeyFormat(context.getTaskAttemptID().toString(), stageName);
Connection connection;
if ((connection = CONNECTION_MAP.remove(connectionId)) != null) {
try {
connection.commit();
} catch (SQLException e) {
try {
connection.rollback();
} catch (SQLException ex) {
LOG.warn(StringUtils.stringifyException(ex));
}
throw new IOException(e);
} finally {
try {
connection.close();
LOG.debug("Connection Closed after committing the task with taskAttemptId {}", connectionId);
} catch (SQLException ex) {
LOG.warn(StringUtils.stringifyException(ex));
}
}
}
}

@Override
public void abortTask(TaskAttemptContext taskContext) throws IOException {
conf = context.getConfiguration();
String stageName = conf.get(STAGE_NAME);
String connectionId = getConnectionMapKeyFormat(context.getTaskAttemptID().toString(), stageName);
Connection connection;
if ((connection = CONNECTION_MAP.remove(connectionId)) != null) {
try {
connection.rollback();
} catch (SQLException e) {
throw new IOException(e);
} finally {
try {
connection.close();
LOG.debug("Connection Closed after rollback the task with taskAttemptId {}", connectionId);
} catch (SQLException ex) {
LOG.warn(StringUtils.stringifyException(ex));
}
}
}
}
};
}

@Override
public RecordWriter<K, V> getRecordWriter(TaskAttemptContext context) throws IOException {
conf = context.getConfiguration();
Expand All @@ -81,6 +161,11 @@ public RecordWriter<K, V> getRecordWriter(TaskAttemptContext context) throws IOE

try {
Connection connection = getConnection(conf);
String stageName = conf.get(STAGE_NAME);
// If using multiple sinks, task attemptID can be same in that case, appending stage in the end for uniqueness.
String connectionId = getConnectionMapKeyFormat(context.getTaskAttemptID().toString(), stageName);
CONNECTION_MAP.put(connectionId, connection);
LOG.debug("Connection Added to the map with connectionId : {}", connectionId);
PreparedStatement statement = connection.prepareStatement(constructQueryOnOperation(tableName, fieldNames,
operationName, listKeys));
return new DBRecordWriter(connection, statement) {
Expand All @@ -98,23 +183,15 @@ public void close(TaskAttemptContext context) throws IOException {
if (!emptyData) {
getStatement().executeBatch();
}
getConnection().commit();
} catch (SQLException e) {
try {
getConnection().rollback();
} catch (SQLException ex) {
LOG.warn(StringUtils.stringifyException(ex));
}
throw new IOException(e);
} finally {
try {
getStatement().close();
getConnection().close();
} catch (SQLException ex) {
throw new IOException(ex);
}
}

try {
DriverManager.deregisterDriver(driverShim);
} catch (SQLException e) {
Expand Down Expand Up @@ -298,4 +375,8 @@ public String constructUpdateQuery(String table, String[] fieldNames, String[] l
return query.toString();
}
}

private String getConnectionMapKeyFormat(String taskAttemptId, String stageName) {
return String.format(CONNECTION_MAP_KEY_FORMAT, taskAttemptId, stageName);
}
}
Loading