diff --git a/.gitignore b/.gitignore index fdc2214ab..0d983e567 100644 --- a/.gitignore +++ b/.gitignore @@ -14,3 +14,12 @@ mainGeneratedRest/ mainGeneratedDataTemplate/ .DS_Store ligradle/ + +# Kafka Web View data +kafkawebview_data/ + +# Local server properties +server-local.properties + +# Local Brooklin data +.brooklin/ diff --git a/build.gradle b/build.gradle index 4b94ec65a..7234dfe36 100644 --- a/build.gradle +++ b/build.gradle @@ -154,18 +154,25 @@ project(':datastream-common') { compile "com.intellij:annotations:$intellijAnnotationsVersion" compile "com.google.guava:guava:$guavaVersion" compile "tech.allegro.schema.json2avro:converter:$json2avroVersion" + compile "com.wayfair.commons:commons-metrics:$wayfairMetricsVersion" + testCompile "org.mockito:mockito-core:$mockitoVersion" } } project(':datastream-bigquery') { + checkstyle { + configFile = file("${project.rootDir}/checkstyle/checkstyle.xml") + sourceSets = [ getProject().sourceSets.main ] + toolVersion = "6.7" + ignoreFailures = false + } dependencies { compile "org.apache.avro:avro:$avroVersion" compile "org.apache.kafka:kafka-clients:$kafkaClientsVersion" compile "io.confluent:kafka-avro-serializer:$kafkaSchemaRegistryClientVersion" - compile "io.confluent:kafka-schema-registry:$kafkaSchemaRegistryVersion" compile "io.confluent:kafka-schema-registry-client:$kafkaSchemaRegistryClientVersion" compile "com.google.cloud:google-cloud-storage:$googleCloudStorageVersion" @@ -174,6 +181,7 @@ project(':datastream-bigquery') { compile project(':datastream-server') compile project(':datastream-utils') + testCompile project(':datastream-testcommon') testCompile "org.mockito:mockito-core:$mockitoVersion" tasks.create(name: "copyDependentLibs", type: Copy) { @@ -199,7 +207,6 @@ project(':datastream-cloud-storage') { compile "org.apache.parquet:parquet-column:$parquetWriterVersion" compile "io.confluent:kafka-avro-serializer:$kafkaSchemaRegistryClientVersion" - compile "io.confluent:kafka-schema-registry:$kafkaSchemaRegistryVersion" compile "io.confluent:kafka-schema-registry-client:$kafkaSchemaRegistryClientVersion" compile "com.google.cloud:google-cloud-storage:$googleCloudStorageVersion" @@ -225,6 +232,7 @@ project(':datastream-server-api') { dependencies { compile project(':datastream-common') compile project(':datastream-utils') + testCompile "org.mockito:mockito-core:$mockitoVersion" } } @@ -352,7 +360,7 @@ project(':datastream-tools') { compile "commons-cli:commons-cli:$commonsCliVersion" } - tasks.create(name: "releaseTarGz", dependsOn: configurations.archives.artifacts, type: Tar) { + tasks.create(name: "releaseTarGz", group: "distribution", dependsOn: configurations.archives.artifacts, type: Tar) { description = "Create Brooklin release tarball" baseName = "${rootProject.name}" into { "${property('baseName')}-$rootProject.version" } @@ -387,6 +395,15 @@ project(':datastream-tools') { from(project(':datastream-kafka-connector').configurations.runtime) { into("libs/") } duplicatesStrategy 'exclude' } + tasks.create(name: "copyDependentLibs", type: Copy) { + from (configurations.runtime) { + } + into "$buildDir/dependent-libs" + } + + jar { + dependsOn 'copyDependentLibs' + } } project(':datastream-client') { diff --git a/config/log4j.properties b/config/log4j.properties index f3405d78e..a3c9c300e 100644 --- a/config/log4j.properties +++ b/config/log4j.properties @@ -5,7 +5,9 @@ log4j.appender.stdout.layout.ConversionPattern=[%d] %p %m (%c)%n log4j.appender.datastreamServiceAppender=org.apache.log4j.DailyRollingFileAppender log4j.appender.datastreamServiceAppender.DatePattern='.'yyyy-MM-dd-HH log4j.appender.datastreamServiceAppender.File=${datastream.logs.dir}/server.log +log4j.appender.file.MaxFileSize=200MB log4j.appender.datastreamServiceAppender.layout=org.apache.log4j.PatternLayout -log4j.appender.datastreamServiceAppender.layout.ConversionPattern=[%d] %p %m (%c)%n +log4j.appender.datastreamServiceAppender.layout.ConversionPattern=[%d{dd MMM yyyy HH:mm:ss,SSS}] [%p] [%t] [%c:%L] %m %n -log4j.rootLogger=INFO, datastreamServiceAppender , stdout \ No newline at end of file +log4j.rootLogger=INFO, datastreamServiceAppender +log4j.logger.com.linkedin.datastream=DEBUG \ No newline at end of file diff --git a/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/Batch.java b/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/Batch.java index e6aa012b0..bcabf6566 100644 --- a/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/Batch.java +++ b/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/Batch.java @@ -5,23 +5,24 @@ */ package com.linkedin.datastream.bigquery; -import java.nio.BufferUnderflowException; import java.util.ArrayList; import java.util.List; +import java.util.Optional; import org.apache.avro.generic.GenericRecord; -import org.apache.kafka.common.errors.SerializationException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import com.google.cloud.bigquery.InsertAllRequest; import com.google.cloud.bigquery.Schema; -import io.confluent.kafka.serializers.KafkaAvroDeserializer; - +import com.linkedin.datastream.bigquery.schema.BigquerySchemaEvolver; import com.linkedin.datastream.bigquery.translator.RecordTranslator; import com.linkedin.datastream.bigquery.translator.SchemaTranslator; import com.linkedin.datastream.common.DatastreamRecordMetadata; import com.linkedin.datastream.common.Package; import com.linkedin.datastream.metrics.DynamicMetricsManager; +import com.linkedin.datastream.serde.Deserializer; import com.linkedin.datastream.server.api.transport.buffered.AbstractBatch; /** @@ -29,6 +30,8 @@ */ public class Batch extends AbstractBatch { + private static final Logger _logger = LoggerFactory.getLogger(Batch.class); + private final int _maxBatchSize; private final int _maxBatchAge; private final BigqueryBatchCommitter _committer; @@ -38,38 +41,44 @@ public class Batch extends AbstractBatch { private org.apache.avro.Schema _avroSchema; private Schema _schema; - private SchemaRegistry _schemaRegistry; - private String _destination; - private KafkaAvroDeserializer _deserializer; + private final BigqueryDatastreamDestination _destination; + private final Deserializer _valueDeserializer; + private final BigquerySchemaEvolver _schemaEvolver; + private final org.apache.avro.Schema _fixedAvroSchema; /** * Constructor for Batch. * @param maxBatchSize any batch bigger than this threshold will be committed to BQ. * @param maxBatchAge any batch older than this threshold will be committed to BQ. * @param maxInflightWriteLogCommits maximum allowed batches in the commit backlog - * @param schemaRegistry schema registry client object + * @param valueDeserializer a Deserializer * @param committer committer object */ - public Batch(int maxBatchSize, - int maxBatchAge, - int maxInflightWriteLogCommits, - SchemaRegistry schemaRegistry, - BigqueryBatchCommitter committer) { + public Batch( + final BigqueryDatastreamDestination destination, + final int maxBatchSize, + final int maxBatchAge, + final int maxInflightWriteLogCommits, + final Deserializer valueDeserializer, + final BigqueryBatchCommitter committer, + final BigquerySchemaEvolver schemaEvolver, + final org.apache.avro.Schema fixedAvroSchema + ) { super(maxInflightWriteLogCommits); + this._destination = destination; this._maxBatchSize = maxBatchSize; this._maxBatchAge = maxBatchAge; this._committer = committer; + this._valueDeserializer = valueDeserializer; + this._schemaEvolver = schemaEvolver; + this._fixedAvroSchema = fixedAvroSchema; this._batch = new ArrayList<>(); this._batchCreateTimeStamp = System.currentTimeMillis(); this._schema = null; - this._destination = null; - this._schemaRegistry = schemaRegistry; - this._deserializer = _schemaRegistry.getDeserializer(); } private void reset() { _batch.clear(); - _destination = null; _ackCallbacks.clear(); _recordMetadata.clear(); _sourceTimestamps.clear(); @@ -82,7 +91,7 @@ public void write(Package aPackage) throws InterruptedException { // skip null records if (aPackage.getRecord().getValue() == null) { - LOG.info("Null record received from topic {}, partition {}, and offset {}", + _logger.info("Null record received from topic {}, partition {}, and offset {}", aPackage.getTopic(), aPackage.getPartition(), aPackage.getOffset()); DynamicMetricsManager.getInstance().createOrUpdateMeter( this.getClass().getSimpleName(), @@ -94,81 +103,84 @@ public void write(Package aPackage) throws InterruptedException { return; } - if (_destination == null) { - String[] datasetRetentionTableSuffix = aPackage.getDestination().split("/"); - if (datasetRetentionTableSuffix.length == 3) { - _destination = datasetRetentionTableSuffix[0] + - "/" + aPackage.getTopic() + datasetRetentionTableSuffix[2] + - "/" + datasetRetentionTableSuffix[1]; - } else { - _destination = datasetRetentionTableSuffix[0] + - "/" + aPackage.getTopic() + - "/" + datasetRetentionTableSuffix[1]; - } - } - - if (_schema == null) { - _avroSchema = _schemaRegistry.getSchemaByTopic(aPackage.getTopic()); - _schema = SchemaTranslator.translate(_avroSchema); - _committer.setDestTableSchema(_destination, _schema); - } - GenericRecord record; try { - record = (GenericRecord) _deserializer.deserialize( + record = (GenericRecord) _valueDeserializer.deserialize((byte[]) aPackage.getRecord().getValue()); + } catch (Exception e) { + _logger.warn("Error deserializing message at Topic {} - Partition {} - Offset {} - Reason {} - Exception {} for destination {}", aPackage.getTopic(), - (byte[]) aPackage.getRecord().getValue()); - } catch (SerializationException e) { - if (e.getCause() instanceof BufferUnderflowException) { - LOG.warn("Skipping message at Topic {} - Partition {} - Offset {} - Reason {} - Exception {}", - aPackage.getTopic(), - aPackage.getPartition(), - aPackage.getOffset(), - e.getMessage(), - e.getCause().getClass()); - DynamicMetricsManager.getInstance().createOrUpdateMeter( - this.getClass().getSimpleName(), - aPackage.getTopic(), - "deserializerErrorCount", - 1); - aPackage.getAckCallback().onCompletion(new DatastreamRecordMetadata( - aPackage.getCheckpoint(), aPackage.getTopic(), aPackage.getPartition()), null); - return; - } else { - throw e; - } + aPackage.getPartition(), + aPackage.getOffset(), + e.getMessage(), + e.getCause().getClass(), + _destination, + e); + DynamicMetricsManager.getInstance().createOrUpdateMeter( + this.getClass().getSimpleName(), + aPackage.getTopic(), + "deserializerErrorCount", + 1); + throw e; } - _batch.add(RecordTranslator.translate(record, _avroSchema)); + if (_fixedAvroSchema == null) { + processAvroSchema(record.getSchema()); + } else { + processAvroSchema(_fixedAvroSchema); + } + + _batch.add(RecordTranslator.translate(record)); _ackCallbacks.add(aPackage.getAckCallback()); _recordMetadata.add(new DatastreamRecordMetadata(aPackage.getCheckpoint(), aPackage.getTopic(), aPackage.getPartition())); _sourceTimestamps.add(aPackage.getTimestamp()); - } else if (aPackage.isTryFlushSignal() || aPackage.isForceFlushSignal()) { - if (_batch.isEmpty()) { - LOG.debug("Nothing to flush."); - return; + } + + if (aPackage.isForceFlushSignal() || _batch.size() >= _maxBatchSize || + System.currentTimeMillis() - _batchCreateTimeStamp >= _maxBatchAge) { + flush(aPackage.isForceFlushSignal()); + } + } + + private void processAvroSchema(final org.apache.avro.Schema avroSchema) { + final Optional newBQSchema; + if (_avroSchema == null) { + newBQSchema = Optional.of(SchemaTranslator.translate(avroSchema)); + } else if (!_avroSchema.equals(avroSchema)) { + final Schema evolvedSchema = _schemaEvolver.evolveSchema(_schema, SchemaTranslator.translate(avroSchema)); + if (!_schema.equals(evolvedSchema)) { + newBQSchema = Optional.of(evolvedSchema); + } else { + newBQSchema = Optional.empty(); } + } else { + newBQSchema = Optional.empty(); } + newBQSchema.ifPresent(schema -> { + _avroSchema = avroSchema; + _schema = schema; + _committer.setDestTableSchema(_destination, _schema); + }); + } - if (_batch.size() >= _maxBatchSize || - System.currentTimeMillis() - _batchCreateTimeStamp >= _maxBatchAge || - aPackage.isForceFlushSignal()) { + private void flush(final boolean force) throws InterruptedException { + if (_batch.isEmpty()) { + _logger.debug("Nothing to flush."); + } else { waitForRoomInCommitBacklog(); incrementInflightWriteLogCommits(); _committer.commit( new ArrayList<>(_batch), - _destination, + _destination.toString(), new ArrayList<>(_ackCallbacks), new ArrayList<>(_recordMetadata), new ArrayList<>(_sourceTimestamps), - () -> decrementInflightWriteLogCommitsAndNotify() + this::decrementInflightWriteLogCommitsAndNotify ); reset(); - - if (aPackage.isForceFlushSignal()) { + if (force) { waitForCommitBacklogToClear(); } } diff --git a/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/BatchBuilder.java b/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/BatchBuilder.java index 8ad8b979e..468c66379 100644 --- a/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/BatchBuilder.java +++ b/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/BatchBuilder.java @@ -6,13 +6,13 @@ package com.linkedin.datastream.bigquery; import java.util.Map; +import java.util.function.Function; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import com.linkedin.datastream.common.DatastreamRecordMetadata; import com.linkedin.datastream.common.Package; -import com.linkedin.datastream.common.VerifiableProperties; import com.linkedin.datastream.metrics.DynamicMetricsManager; import com.linkedin.datastream.server.api.transport.buffered.AbstractBatch; import com.linkedin.datastream.server.api.transport.buffered.AbstractBatchBuilder; @@ -22,14 +22,9 @@ */ public class BatchBuilder extends AbstractBatchBuilder { - private static final Logger LOG = LoggerFactory.getLogger(BatchBuilder.class.getName()); - private static final String CONFIG_SCHEMA_REGISTRY = "schemaRegistry"; + private static final Logger _logger = LoggerFactory.getLogger(BatchBuilder.class.getName()); - private final int _maxBatchSize; - private final int _maxBatchAge; - private final int _maxInflightCommits; - private final BigqueryBatchCommitter _committer; - private final SchemaRegistry _schemaRegistry; + private final Function newBatchSupplier; /** * Constructor for BatchBuilder @@ -38,21 +33,28 @@ public class BatchBuilder extends AbstractBatchBuilder { * @param maxInflightCommits maximum allowed batches in the commit backlog. * @param committer committer object. * @param queueSize queue size of the batch builder. - * @param translatorProperties configuration options for translator. + * @param destinationConfigurations a Map of BigqueryDatastreamDestination to BigqueryDatastreamConfiguration */ - public BatchBuilder(int maxBatchSize, - int maxBatchAge, - int maxInflightCommits, - BigqueryBatchCommitter committer, - int queueSize, - VerifiableProperties translatorProperties) { + BatchBuilder(final int maxBatchSize, + final int maxBatchAge, + final int maxInflightCommits, + final BigqueryBatchCommitter committer, + final int queueSize, + final Map destinationConfigurations) { super(queueSize); - this._maxBatchSize = maxBatchSize; - this._maxBatchAge = maxBatchAge; - this._maxInflightCommits = maxInflightCommits; - this._committer = committer; - this._schemaRegistry = new SchemaRegistry( - new VerifiableProperties(translatorProperties.getDomainProperties(CONFIG_SCHEMA_REGISTRY))); + newBatchSupplier = destination -> { + final BigqueryDatastreamConfiguration config = destinationConfigurations.get(destination); + return new Batch( + destination, + maxBatchSize, + maxBatchAge, + maxInflightCommits, + config.getValueDeserializer(), + committer, + config.getSchemaEvolver(), + config.getFixedSchema().orElse(null) + ); + }; } @Override @@ -70,18 +72,14 @@ public void run() { try { if (aPackage.isDataPackage()) { _registry.computeIfAbsent(aPackage.getTopic() + "-" + aPackage.getPartition(), - key -> new Batch(_maxBatchSize, - _maxBatchAge, - _maxInflightCommits, - _schemaRegistry, - _committer)).write(aPackage); + topicAndPartition -> newBatchSupplier.apply(BigqueryDatastreamDestination.parse(aPackage.getDestination()))) + .write(aPackage); } else { // broadcast signal for (Map.Entry entry : _registry.entrySet()) { entry.getValue().write(aPackage); } } - aPackage.markAsDelivered(); } catch (InterruptedException e) { Thread.currentThread().interrupt(); break; @@ -92,14 +90,18 @@ public void run() { aPackage.getTopic(), "errorCount", 1); - LOG.error("Unable to write to batch {}", e); - aPackage.getAckCallback().onCompletion(new DatastreamRecordMetadata( - aPackage.getCheckpoint(), aPackage.getTopic(), aPackage.getPartition()), e); + _logger.error("Unable to write to batch", e); + aPackage.getAckCallback().onCompletion( + new DatastreamRecordMetadata(aPackage.getCheckpoint(), aPackage.getTopic(), aPackage.getPartition()), + e + ); } else { - LOG.error("Unable to process flush signal {}", e); + _logger.error("Unable to process flush signal", e); } + } finally { + aPackage.markAsDelivered(); } } - LOG.info("Batch builder stopped."); + _logger.info("Batch builder stopped."); } } diff --git a/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/BigqueryBatchCommitter.java b/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/BigqueryBatchCommitter.java index 632354728..c89e239c5 100644 --- a/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/BigqueryBatchCommitter.java +++ b/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/BigqueryBatchCommitter.java @@ -5,43 +5,43 @@ */ package com.linkedin.datastream.bigquery; -import java.io.FileInputStream; -import java.io.FileNotFoundException; -import java.io.IOException; -import java.text.SimpleDateFormat; -import java.util.Calendar; +import java.time.Duration; +import java.time.LocalDate; +import java.time.ZoneOffset; +import java.time.format.DateTimeFormatter; +import java.time.temporal.ChronoUnit; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.TimeZone; +import java.util.Optional; +import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; +import java.util.stream.IntStream; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.google.auth.Credentials; -import com.google.auth.oauth2.GoogleCredentials; - import com.google.cloud.bigquery.BigQuery; import com.google.cloud.bigquery.BigQueryException; -import com.google.cloud.bigquery.BigQueryOptions; import com.google.cloud.bigquery.InsertAllRequest; import com.google.cloud.bigquery.InsertAllResponse; import com.google.cloud.bigquery.Schema; import com.google.cloud.bigquery.StandardTableDefinition; +import com.google.cloud.bigquery.Table; import com.google.cloud.bigquery.TableDefinition; import com.google.cloud.bigquery.TableId; import com.google.cloud.bigquery.TableInfo; import com.google.cloud.bigquery.TimePartitioning; +import com.linkedin.datastream.bigquery.schema.BigquerySchemaEvolver; import com.linkedin.datastream.common.DatastreamRecordMetadata; -import com.linkedin.datastream.common.DatastreamTransientException; import com.linkedin.datastream.common.SendCallback; -import com.linkedin.datastream.common.VerifiableProperties; import com.linkedin.datastream.metrics.DynamicMetricsManager; import com.linkedin.datastream.server.api.transport.buffered.BatchCommitter; import com.linkedin.datastream.server.api.transport.buffered.CommitCallback; @@ -50,83 +50,30 @@ * This class commits submitted batches to BQ tables. */ public class BigqueryBatchCommitter implements BatchCommitter> { - private static final Logger LOG = LoggerFactory.getLogger(BigqueryBatchCommitter.class.getName()); - - private ConcurrentMap _destTableSchemas; - private Map _destTableCreated; + private final Logger log = LoggerFactory.getLogger(BigqueryBatchCommitter.class.getName()); - private static final String CONFIG_THREADS = "threads"; + private final ConcurrentMap _destTableSchemas; + private final Map _datastreamConfigurations; + private final ConcurrentMap _initializedDestinationConfiguration; private final ExecutorService _executor; - private final int _numOfCommitterThreads; private final BigQuery _bigquery; - private static String sanitizeTableName(String tableName) { - return tableName.replaceAll("[^A-Za-z0-9_]+", "_"); - } + private final DateTimeFormatter partitionDateFormatter = DateTimeFormatter.ofPattern("yyyyMMdd"); /** - * Constructor for BigqueryBatchCommitter - * @param properties configuration options + * Constructor. + * @param bigQuery a BigQuery instance + * @param numThreads the number of committer threads */ - public BigqueryBatchCommitter(VerifiableProperties properties) { - String credentialsPath = properties.getString("credentialsPath"); - String projectId = properties.getString("projectId"); - try { - Credentials credentials = GoogleCredentials - .fromStream(new FileInputStream(credentialsPath)); - this._bigquery = BigQueryOptions.newBuilder() - .setProjectId(projectId) - .setCredentials(credentials).build().getService(); - } catch (FileNotFoundException e) { - LOG.error("Credentials path {} does not exist", credentialsPath); - throw new RuntimeException(e); - } catch (IOException e) { - LOG.error("Unable to read credentials: {}", credentialsPath); - throw new RuntimeException(e); - } - - this._numOfCommitterThreads = properties.getInt(CONFIG_THREADS, 1); - this._executor = Executors.newFixedThreadPool(_numOfCommitterThreads); - + public BigqueryBatchCommitter(final BigQuery bigQuery, final int numThreads, final Map datastreamConfigurations) { + this._bigquery = bigQuery; + this._executor = Executors.newFixedThreadPool(numThreads); this._destTableSchemas = new ConcurrentHashMap<>(); - this._destTableCreated = new HashMap<>(); - } - - private synchronized void createTableIfAbsent(String destination) { - if (_destTableCreated.containsKey(destination)) { - return; - } - String[] datasetTableNameRetention = destination.split("/"); - - try { - TableId tableId = TableId.of(datasetTableNameRetention[0], sanitizeTableName(datasetTableNameRetention[1])); - TableDefinition tableDefinition; - long partitionRetentionDays = Long.parseLong(datasetTableNameRetention[2]); - if (partitionRetentionDays > 0) { - tableDefinition = StandardTableDefinition.newBuilder() - .setSchema(_destTableSchemas.get(destination)) - .setTimePartitioning( - TimePartitioning.of(TimePartitioning.Type.DAY, partitionRetentionDays * 86400000L)) - .build(); - } else { - tableDefinition = StandardTableDefinition.newBuilder() - .setSchema(_destTableSchemas.get(destination)) - .setTimePartitioning(TimePartitioning.of(TimePartitioning.Type.DAY)) - .build(); - } - TableInfo tableInfo = TableInfo.newBuilder(tableId, tableDefinition).build(); - if (_bigquery.getTable(tableId) != null) { - LOG.debug("Table {} already exist", destination); - return; - } - _bigquery.create(tableInfo); - LOG.info("Table {} created successfully", destination); - } catch (BigQueryException e) { - LOG.warn("Failed to create table {} - {}", destination, e); - throw e; - } + _datastreamConfigurations = datastreamConfigurations; + _initializedDestinationConfiguration = new ConcurrentHashMap<>(); } /** @@ -134,92 +81,94 @@ private synchronized void createTableIfAbsent(String destination) { * @param dest dataset and table * @param schema table schema */ - public void setDestTableSchema(String dest, Schema schema) { - _destTableSchemas.putIfAbsent(dest, schema); + public void setDestTableSchema(final BigqueryDatastreamDestination dest, final Schema schema) { + _destTableSchemas.put(dest, schema); } @Override public void commit(List batch, - String destination, + final String destinationStr, List ackCallbacks, List recordMetadata, List sourceTimestamps, CommitCallback callback) { if (batch.isEmpty()) { + callback.commited(); return; } final Runnable committerTask = () -> { - Exception exception = null; - InsertAllResponse response = null; + final String classSimpleName = this.getClass().getSimpleName(); - try { - createTableIfAbsent(destination); - - TimeZone utcTimeZone = TimeZone.getTimeZone("UTC"); - SimpleDateFormat timeFmt = new SimpleDateFormat("yyyyMMdd"); - timeFmt.setTimeZone(utcTimeZone); - - String[] datasetTable = destination.split("/"); - TableId tableId = TableId.of(datasetTable[0], - sanitizeTableName(datasetTable[1]) + "$" + timeFmt.format( - Calendar.getInstance(utcTimeZone).getTimeInMillis())); - - LOG.debug("Committing a batch to dataset {} and table {}", datasetTable[0], sanitizeTableName(datasetTable[1])); - long start = System.currentTimeMillis(); - - response = _bigquery.insertAll( - InsertAllRequest.newBuilder(tableId, batch) - .build()); - - DynamicMetricsManager.getInstance().createOrUpdateHistogram( - this.getClass().getSimpleName(), - recordMetadata.get(0).getTopic(), - "insertAllExecTime", - System.currentTimeMillis() - start); - } catch (Exception e) { - LOG.warn("Failed to insert a rows {}", response); - exception = new DatastreamTransientException(e); - } + final BigqueryDatastreamDestination destination = BigqueryDatastreamDestination.parse(destinationStr); + final BigqueryDatastreamConfiguration datastreamConfiguration = Optional.ofNullable(_datastreamConfigurations.get(destination)) + .orElseThrow(() -> new IllegalStateException(String.format("configuration not defined for destination: %s", destination))); + final String tableName = sanitizeTableName(destination.getDestinatonName()); + final TableId tableId = TableId.of(destination.getProjectId(), destination.getDatasetId(), tableName); + final String partition = partitionDateFormatter.format(LocalDate.now(ZoneOffset.UTC)); - for (int i = 0; i < ackCallbacks.size(); i++) { - if (exception != null) { - // entire batch failed - DynamicMetricsManager.getInstance().createOrUpdateMeter( - this.getClass().getSimpleName(), - recordMetadata.get(i).getTopic(), - "errorCount", - 1); - ackCallbacks.get(i).onCompletion(recordMetadata.get(i), exception); - // force to check if table exists next time - _destTableCreated.remove(destination); - } else { - Long key = Long.valueOf(i); - if (response != null && response.hasErrors() && response.getInsertErrors().containsKey(key)) { - LOG.warn("Failed to insert a row {} {}", i, response.getInsertErrors().get(key)); - DynamicMetricsManager.getInstance().createOrUpdateMeter( - this.getClass().getSimpleName(), - recordMetadata.get(i).getTopic(), - "errorCount", - 1); - ackCallbacks.get(i).onCompletion(recordMetadata.get(i), - new DatastreamTransientException(response.getInsertErrors().get(key).toString())); - } else { - DynamicMetricsManager.getInstance().createOrUpdateMeter( - this.getClass().getSimpleName(), - recordMetadata.get(i).getTopic(), - "commitCount", - 1); - DynamicMetricsManager.getInstance().createOrUpdateHistogram( - this.getClass().getSimpleName(), - recordMetadata.get(i).getTopic(), - "eteLatency", - System.currentTimeMillis() - sourceTimestamps.get(i)); - ackCallbacks.get(i).onCompletion(recordMetadata.get(i), null); + // Initialize destination the first time it is encountered or if the destination configuration has changed + _initializedDestinationConfiguration.compute(destination, (d, config) -> { + if (config != datastreamConfiguration) { + try { + createOrUpdateTable(tableId, _destTableSchemas.get(destination), + datastreamConfiguration.getSchemaEvolver(), datastreamConfiguration.getPartitionExpirationDays().orElse(null), + datastreamConfiguration.getLabels(), datastreamConfiguration.isCreateDestinationTableEnabled(), + classSimpleName, recordMetadata.get(0).getTopic()); + log.info("Initialized table {} for destination {}", tableId, destination); + } catch (final Exception e) { + log.warn("Unexpected error initializing table {} for destination {}", tableId, destination, e); } } + return datastreamConfiguration; + }); + + final TableId insertTableId = TableId.of(tableId.getProject(), tableId.getDataset(), String.format("%s$%s", tableId.getTable(), partition)); + + log.debug("Committing a batch with size {} to table {}", batch.size(), insertTableId); + + final long start = System.currentTimeMillis(); + Map insertErrors = insertRowsAndMapErrorsWithRetry(insertTableId, batch); + final long end = System.currentTimeMillis(); + DynamicMetricsManager.getInstance() + .createOrUpdateHistogram(this.getClass().getSimpleName(), recordMetadata.get(0).getTopic(), "insertAllExecTime", end - start); + + // If we manage the destination table and encountered insert errors, try creating/updating the destination table before retrying + if (!insertErrors.isEmpty()) { + try { + final boolean tableUpdatedOrCreated = createOrUpdateTable(tableId, _destTableSchemas.get(destination), + datastreamConfiguration.getSchemaEvolver(), datastreamConfiguration.getPartitionExpirationDays().orElse(null), + datastreamConfiguration.getLabels(), datastreamConfiguration.isCreateDestinationTableEnabled(), + classSimpleName, recordMetadata.get(0).getTopic()); + if (tableUpdatedOrCreated) { + log.info("Table created/updated for destination {}. Retrying batch...", destination); + insertErrors = insertRowsAndMapErrorsWithRetry(insertTableId, batch); + } + } catch (final Exception e) { + insertErrors = IntStream.range(0, batch.size()).boxed().collect(Collectors.toMap(i -> i, i -> e)); + } } + final Map finalInsertErrors = insertErrors; + IntStream.range(0, ackCallbacks.size()).forEach(i -> { + final DatastreamRecordMetadata currentRecordMetadata = recordMetadata.get(i); + final SendCallback ackCallback = ackCallbacks.get(i); + final String topic = currentRecordMetadata.getTopic(); + + if (!finalInsertErrors.containsKey(i)) { + DynamicMetricsManager.getInstance().createOrUpdateMeter(classSimpleName, topic, "commitCount", 1); + final long currentRecordSourceTimestamp = sourceTimestamps.get(i); + DynamicMetricsManager.getInstance().createOrUpdateHistogram(classSimpleName, topic, "eteLatency", + System.currentTimeMillis() - currentRecordSourceTimestamp); + ackCallback.onCompletion(currentRecordMetadata, null); + } else { + final Exception insertError = finalInsertErrors.get(i); + log.warn("Failed to insert a row {} {}", i, insertError.getMessage()); + DynamicMetricsManager.getInstance().createOrUpdateMeter(classSimpleName, topic, "errorCount", 1); + ackCallback.onCompletion(currentRecordMetadata, insertError); + } + }); + callback.commited(); }; @@ -231,13 +180,209 @@ public void shutdown() { _executor.shutdown(); try { if (!_executor.awaitTermination(5, TimeUnit.SECONDS)) { - LOG.warn("Batch Committer shutdown timed out."); + log.warn("Batch Committer shutdown timed out."); } } catch (InterruptedException e) { - LOG.warn("Interrupted while awaiting committer termination."); + log.warn("Interrupted while awaiting committer termination."); Thread.currentThread().interrupt(); } - LOG.info("BQ Batch committer stopped."); + log.info("BQ Batch committer stopped."); + } + + Map insertRowsAndMapErrorsWithRetry(final TableId insertTableId, final List batch) { + Map insertErrors; + if (batch.size() > 1) { + try { + final InsertAllResponse response = _bigquery.insertAll(InsertAllRequest.newBuilder(insertTableId, batch).build()); + insertErrors = response.getInsertErrors().entrySet().stream().collect(Collectors.toMap( + entry -> entry.getKey().intValue(), + entry -> new TransientStreamingInsertException(entry.getValue().toString()) + )); + } catch (final Exception e) { + if (isBatchSizeLimitException(e)) { + log.warn("Batch size limit hit for table {} with batch size of {}. Retrying with reduced batch sizes...", insertTableId, batch.size(), e); + final int halfIndex = batch.size() / 2; + final Map firstBatchErrors = insertRowsAndMapErrorsWithRetry(insertTableId, batch.subList(0, halfIndex)); + final Map secondBatchErrors = insertRowsAndMapErrorsWithRetry(insertTableId, batch.subList(halfIndex, batch.size())); + insertErrors = new HashMap<>(firstBatchErrors); + for (Map.Entry entry : secondBatchErrors.entrySet()) { + insertErrors.put(entry.getKey() + halfIndex, entry.getValue()); + } + } else { + final TransientStreamingInsertException wrappedException = new TransientStreamingInsertException(e); + insertErrors = IntStream.range(0, batch.size()).boxed().collect(Collectors.toMap(i -> i, i -> wrappedException)); + } + } + } else { + insertErrors = insertRowsAndMapErrors(insertTableId, batch); + } + return insertErrors; + } + + static String sanitizeTableName(String tableName) { + return tableName.replaceAll("[^A-Za-z0-9_]+", "_"); + } + + private Map insertRowsAndMapErrors(final TableId insertTableId, final List batch) { + Map insertErrors; + if (!batch.isEmpty()) { + try { + final InsertAllResponse response = _bigquery.insertAll(InsertAllRequest.newBuilder(insertTableId, batch).build()); + insertErrors = response.getInsertErrors().entrySet().stream().collect(Collectors.toMap( + entry -> entry.getKey().intValue(), + entry -> new TransientStreamingInsertException(entry.getValue().toString()) + )); + } catch (final Exception e) { + final TransientStreamingInsertException wrappedException = new TransientStreamingInsertException(e); + insertErrors = IntStream.range(0, batch.size()).boxed().collect(Collectors.toMap(i -> i, i -> wrappedException)); + } + } else { + insertErrors = Collections.emptyMap(); + } + return insertErrors; + } + + + private boolean createOrUpdateTable(final TableId tableId, final Schema desiredTableSchema, + final BigquerySchemaEvolver schemaEvolver, final Long partitionExpirationDays, + final List labels, + final boolean createDestinationTableEnabled, + final String classSimpleName, + final String topic) { + final TimePartitioning timePartitioning = Optional.ofNullable(partitionExpirationDays) + .filter(partitionRetentionDays -> partitionRetentionDays > 0) + .map(partitionRetentionDays -> TimePartitioning.of(TimePartitioning.Type.DAY, Duration.of(partitionRetentionDays, ChronoUnit.DAYS).toMillis())) + .orElse(TimePartitioning.of(TimePartitioning.Type.DAY)); + + final Optional optionalExistingTable = Optional.ofNullable(_bigquery.getTable(tableId)); + final boolean tableCreatedOrUpdated; + if (optionalExistingTable.isPresent()) { + try { + tableCreatedOrUpdated = updateTable(tableId, desiredTableSchema, timePartitioning, optionalExistingTable.get(), schemaEvolver, labels); + if (tableCreatedOrUpdated) { + log.info("Updated schema/labels on table {}", tableId); + DynamicMetricsManager.getInstance().createOrUpdateMeter(classSimpleName, topic, "tableUpdateCount", 1); + } + } catch (final Exception e) { + log.error("Unexpected error while updating schema/labels on table {}", tableId, e); + DynamicMetricsManager.getInstance().createOrUpdateMeter(classSimpleName, topic, "tableUpdateErrorCount", 1); + throw e; + } + } else { + if (createDestinationTableEnabled) { + try { + tableCreatedOrUpdated = createTable(tableId, desiredTableSchema, timePartitioning, labels); + if (tableCreatedOrUpdated) { + log.info("Created table {}", tableId); + DynamicMetricsManager.getInstance().createOrUpdateMeter(classSimpleName, topic, "tableCreateCount", 1); + } + } catch (final Exception e) { + log.error("Unexpected error while creating table {}", tableId, e); + DynamicMetricsManager.getInstance().createOrUpdateMeter(classSimpleName, topic, "tableCreateErrorCount", 1); + throw e; + } + } else { + tableCreatedOrUpdated = false; + } + } + return tableCreatedOrUpdated; + } + + private boolean updateTable(final TableId tableId, final Schema desiredTableSchema, + final TimePartitioning timePartitioning, final Table existingTable, + final BigquerySchemaEvolver schemaEvolver, + final List labels) { + final Schema existingTableSchema = Optional.ofNullable(existingTable.getDefinition().getSchema()) + .orElseThrow(() -> new IllegalStateException(String.format("schema not defined for table: %s", tableId))); + final Set existingLabels = existingTable.getLabels().entrySet().stream() + .map(entry -> new BigqueryLabel(entry.getKey(), entry.getValue())).collect(Collectors.toSet()); + final Optional> optionalLabelMap; + if (!existingLabels.containsAll(labels)) { + optionalLabelMap = Optional.of(labels.stream().collect(Collectors.toMap(BigqueryLabel::getName, BigqueryLabel::getValue))); + } else { + optionalLabelMap = Optional.empty(); + } + final Optional optionalEvolvedSchema; + if (!desiredTableSchema.equals(existingTableSchema)) { + final Schema evolvedSchema = schemaEvolver.evolveSchema(existingTableSchema, desiredTableSchema); + if (!existingTableSchema.equals(evolvedSchema)) { + optionalEvolvedSchema = Optional.of(evolvedSchema); + } else { + optionalEvolvedSchema = Optional.empty(); + } + } else { + optionalEvolvedSchema = Optional.empty(); + } + boolean tableUpdated; + if (optionalLabelMap.isPresent() || optionalEvolvedSchema.isPresent()) { + try { + final Table.Builder tableBuilder = existingTable.toBuilder(); + optionalEvolvedSchema.ifPresent(schema -> tableBuilder.setDefinition(createTableDefinition(schema, timePartitioning))); + optionalLabelMap.ifPresent(tableBuilder::setLabels); + tableBuilder.build().update(); + tableUpdated = true; + } catch (BigQueryException e) { + final Table currentTable = _bigquery.getTable(tableId); + if (optionalEvolvedSchema.isPresent()) { + final Schema evolvedSchema = optionalEvolvedSchema.get(); + final Schema currentTableSchema = currentTable.getDefinition().getSchema(); + if (evolvedSchema.equals(currentTableSchema)) { + log.info("Schema already evolved for table {}", tableId); + tableUpdated = true; + } else if (!existingTableSchema.equals(currentTableSchema)) { + log.warn("Concurrent table schema update exception encountered for table {}. Retrying update with new base schema...", tableId, e); + tableUpdated = updateTable(tableId, desiredTableSchema, timePartitioning, currentTable, schemaEvolver, labels); + } else { + log.error("Failed to update schema for table {}", tableId, e); + throw e; + } + } else { + final Set currentLabels = currentTable.getLabels().entrySet().stream() + .map(entry -> new BigqueryLabel(entry.getKey(), entry.getValue())).collect(Collectors.toSet()); + if (currentLabels.containsAll(labels)) { + log.info("Labels already updated for table {}", tableId); + tableUpdated = true; + } else if (!existingTable.getLabels().equals(currentTable.getLabels())) { + log.warn("Concurrent table label update exception encountered for table {}. Retrying update...", tableId, e); + tableUpdated = updateTable(tableId, desiredTableSchema, timePartitioning, currentTable, schemaEvolver, labels); + } else { + log.error("Failed to update labels for table {}", tableId, e); + throw e; + } + } + } + } else { + log.debug("No update required for table {}", tableId); + tableUpdated = false; + } + return tableUpdated; + } + + private boolean createTable(final TableId tableId, final Schema desiredTableSchema, final TimePartitioning timePartitioning, + final List labels) { + final Map labelsMap = labels.stream().collect(Collectors.toMap(BigqueryLabel::getName, BigqueryLabel::getValue)); + final TableInfo tableInfo = TableInfo.newBuilder(tableId, createTableDefinition(desiredTableSchema, timePartitioning)).setLabels(labelsMap).build(); + try { + _bigquery.create(tableInfo); + } catch (BigQueryException e) { + log.warn("Failed to create table {}", tableId, e); + throw e; + } + log.info("Table {} created successfully", tableId); + return true; + } + + private static TableDefinition createTableDefinition(final Schema schema, final TimePartitioning timePartitioning) { + return StandardTableDefinition.newBuilder() + .setSchema(schema) + .setTimePartitioning(timePartitioning) + .build(); + } + + private static boolean isBatchSizeLimitException(final Exception e) { + final String message = e.getMessage(); + return message != null && + (message.startsWith("Request payload size exceeds the limit") || message.startsWith("too many rows present in the request")); } } diff --git a/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/BigqueryBufferedTransportProvider.java b/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/BigqueryBufferedTransportProvider.java new file mode 100644 index 000000000..963a5fca0 --- /dev/null +++ b/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/BigqueryBufferedTransportProvider.java @@ -0,0 +1,61 @@ +/** + * Copyright 2020 Wayfair LLC. All rights reserved. + * Licensed under the BSD 2-Clause License. See the LICENSE file in the project root for license information. + * See the NOTICE file in the project root for additional information regarding copyright ownership. + */ +package com.linkedin.datastream.bigquery; + +import java.util.List; +import java.util.concurrent.TimeUnit; + +import com.linkedin.datastream.server.api.transport.buffered.AbstractBatchBuilder; +import com.linkedin.datastream.server.api.transport.buffered.AbstractBufferedTransportProvider; + +/** + * Implementation of a Bigquery BufferedTransportProvider. + */ +public class BigqueryBufferedTransportProvider extends AbstractBufferedTransportProvider { + + private final BigqueryBatchCommitter _committer; + private final int _maxBatchAge; + + /** + * Constructor. + * @param transportProviderName a String + * @param batchBuilders a list of objects tha extend AbstractBatchBuilder + * @param committer a BigqueryBatchCommitter + * @param maxBatchAge an int + */ + public BigqueryBufferedTransportProvider(final String transportProviderName, + final List batchBuilders, + final BigqueryBatchCommitter committer, + final int maxBatchAge) { + super(transportProviderName, batchBuilders); + _committer = committer; + _maxBatchAge = maxBatchAge; + init(); + } + + private void init() { + for (AbstractBatchBuilder batchBuilder : _batchBuilders) { + batchBuilder.start(); + } + + // send periodic flush signal to commit stale objects + _scheduler.scheduleAtFixedRate( + () -> { + for (AbstractBatchBuilder objectBuilder: _batchBuilders) { + _logger.info("Try flush signal sent."); + objectBuilder.assign(new com.linkedin.datastream.common.Package.PackageBuilder().buildTryFlushSignalPackage()); + } + }, + _maxBatchAge / 2, + _maxBatchAge / 2, + TimeUnit.MILLISECONDS); + } + + @Override + protected void shutdownCommitter() { + _committer.shutdown(); + } +} diff --git a/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/BigqueryCachedSchemaRegistryClient.java b/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/BigqueryCachedSchemaRegistryClient.java new file mode 100644 index 000000000..93e583eaa --- /dev/null +++ b/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/BigqueryCachedSchemaRegistryClient.java @@ -0,0 +1,487 @@ +/* + * Copyright 2020 Wayfair LLC. All rights reserved. + * Licensed under the BSD 2-Clause License. See the LICENSE file in the project root for license information. + * See the NOTICE file in the project root for additional information regarding copyright ownership. + */ + +package com.linkedin.datastream.bigquery; + +import java.io.IOException; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; + +import org.apache.avro.Schema; +import org.apache.kafka.common.config.ConfigException; + +import io.confluent.kafka.schemaregistry.client.SchemaMetadata; +import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient; +import io.confluent.kafka.schemaregistry.client.rest.RestService; +import io.confluent.kafka.schemaregistry.client.rest.Versions; +import io.confluent.kafka.schemaregistry.client.rest.entities.Config; +import io.confluent.kafka.schemaregistry.client.rest.entities.SchemaString; +import io.confluent.kafka.schemaregistry.client.rest.entities.requests.ConfigUpdateRequest; +import io.confluent.kafka.schemaregistry.client.rest.entities.requests.ModeGetResponse; +import io.confluent.kafka.schemaregistry.client.rest.entities.requests.ModeUpdateRequest; +import io.confluent.kafka.schemaregistry.client.rest.exceptions.RestClientException; +import io.confluent.kafka.schemaregistry.client.security.basicauth.BasicAuthCredentialProvider; +import io.confluent.kafka.schemaregistry.client.security.basicauth.BasicAuthCredentialProviderFactory; +import io.confluent.kafka.schemaregistry.client.security.bearerauth.BearerAuthCredentialProvider; +import io.confluent.kafka.schemaregistry.client.security.bearerauth.BearerAuthCredentialProviderFactory; + +/** + * Thread-safe Schema Registry Client with client side caching. + * + * This implementation is a fork of Confluent's cached Schema Registry client. + * The implementation was forked to gain control over the Avro schema parser, so we can validate schemas with invalid default values. + * + * Original source: + * https://github.com/confluentinc/schema-registry/blob/v5.3.0/client/src/main/java/io/confluent/kafka/schemaregistry/client/CachedSchemaRegistryClient.java + */ +public class BigqueryCachedSchemaRegistryClient implements SchemaRegistryClient { + + private final RestService restService; + private final int identityMapCapacity; + private final Map> schemaCache; + private final Map> idCache; + private final Map> versionCache; + private final Boolean validateSchemaDefaults; + private final Boolean validateSchemaFieldNames; + + public static final Map DEFAULT_REQUEST_PROPERTIES; + + static { + final Map properties = new HashMap<>(); + properties.put("Content-Type", Versions.SCHEMA_REGISTRY_V1_JSON_WEIGHTED); + DEFAULT_REQUEST_PROPERTIES = Collections.unmodifiableMap(properties); + } + + /** + * Constructor. + * @param baseUrl a comma delimited String pointing to schema registry endpoints + * @param identityMapCapacity the maximum number of schemas to cache + */ + public BigqueryCachedSchemaRegistryClient(String baseUrl, int identityMapCapacity) { + this(new RestService(baseUrl), identityMapCapacity); + } + + /** + * Constructor. + * @param baseUrls a list of Strings pointing to schema registry endpoints + * @param identityMapCapacity the maximum number of schemas to cache + */ + public BigqueryCachedSchemaRegistryClient(List baseUrls, int identityMapCapacity) { + this(new RestService(baseUrls), identityMapCapacity); + } + + /** + * Constructor. + * @param restService a RestService + * @param identityMapCapacity the maximum number of schemas to cache + */ + public BigqueryCachedSchemaRegistryClient(RestService restService, int identityMapCapacity) { + this(restService, identityMapCapacity, null); + } + + /** + * Constructor. + * @param baseUrl a comma delimited String pointing to schema registry endpoints + * @param identityMapCapacity the maximum number of schemas to cache + * @param originals a configuration Map + */ + public BigqueryCachedSchemaRegistryClient( + String baseUrl, + int identityMapCapacity, + Map originals) { + this(baseUrl, identityMapCapacity, originals, null); + } + + /** + * Constructor. + * @param baseUrls a list of Strings pointing to schema registry endpoints + * @param identityMapCapacity the maximum number of schemas to cache + * @param originals a configuration Map + */ + public BigqueryCachedSchemaRegistryClient( + List baseUrls, + int identityMapCapacity, + Map originals) { + this(baseUrls, identityMapCapacity, originals, null); + } + + /** + * Constructor. + * @param restService a RestService + * @param identityMapCapacity the maximum number of schemas to cache + * @param configs a configuration Map + */ + public BigqueryCachedSchemaRegistryClient( + RestService restService, + int identityMapCapacity, + Map configs) { + this(restService, identityMapCapacity, configs, null); + } + + /** + * Constructor. + * @param baseUrl a comma delimited String pointing to schema registry endpoints + * @param identityMapCapacity the maximum number of schemas to cache + * @param originals a configuration Map + * @param httpHeaders a mapping of HTTP headers + */ + public BigqueryCachedSchemaRegistryClient( + String baseUrl, + int identityMapCapacity, + Map originals, + Map httpHeaders) { + this(new RestService(baseUrl), identityMapCapacity, originals, httpHeaders); + } + + /** + * Constructor. + * @param baseUrls a list of Strings pointing to schema registry endpoints + * @param identityMapCapacity the maximum number of schemas to cache + * @param originals a configuration Map + * @param httpHeaders a mapping of HTTP headers + */ + public BigqueryCachedSchemaRegistryClient( + List baseUrls, + int identityMapCapacity, + Map originals, + Map httpHeaders) { + this(new RestService(baseUrls), identityMapCapacity, originals, httpHeaders); + } + + /** + * Constructor. + * @param restService a RestService + * @param identityMapCapacity the maximum number of schemas to cache + * @param configs a configuration Map + * @param httpHeaders a mapping of HTTP headers + */ + public BigqueryCachedSchemaRegistryClient( + RestService restService, + int identityMapCapacity, + Map configs, + Map httpHeaders) { + this.identityMapCapacity = identityMapCapacity; + this.schemaCache = new HashMap<>(); + this.idCache = new HashMap<>(); + this.versionCache = new HashMap<>(); + this.restService = restService; + this.idCache.put(null, new HashMap<>()); + configureRestService(configs, httpHeaders); + + final Optional> optionalConfigs = Optional.ofNullable(configs); + validateSchemaFieldNames = optionalConfigs.map(c -> c.get(BigquerySchemaRegistryClientConfig.SCHEMA_REGISTRY_PARSER_VALIDATE_FIELD_NAMES)) + .map(v -> Boolean.valueOf(v.toString())).orElse(null); + validateSchemaDefaults = optionalConfigs.map(c -> c.get(BigquerySchemaRegistryClientConfig.SCHEMA_REGISTRY_PARSER_VALIDATE_DEFAULTS)) + .map(v -> Boolean.valueOf(v.toString())).orElse(null); + } + + private void configureRestService(Map configs, Map httpHeaders) { + if (httpHeaders != null) { + restService.setHttpHeaders(httpHeaders); + } + + if (configs != null) { + String basicCredentialsSource = + (String) configs.get(BigquerySchemaRegistryClientConfig.BASIC_AUTH_CREDENTIALS_SOURCE); + String bearerCredentialsSource = + (String) configs.get(BigquerySchemaRegistryClientConfig.BEARER_AUTH_CREDENTIALS_SOURCE); + + if (isNonEmpty(basicCredentialsSource) && isNonEmpty(bearerCredentialsSource)) { + throw new ConfigException(String.format( + "Only one of '%s' and '%s' may be specified", + BigquerySchemaRegistryClientConfig.BASIC_AUTH_CREDENTIALS_SOURCE, + BigquerySchemaRegistryClientConfig.BEARER_AUTH_CREDENTIALS_SOURCE + )); + + } else if (isNonEmpty(basicCredentialsSource)) { + BasicAuthCredentialProvider basicAuthCredentialProvider = + BasicAuthCredentialProviderFactory.getBasicAuthCredentialProvider( + basicCredentialsSource, + configs + ); + restService.setBasicAuthCredentialProvider(basicAuthCredentialProvider); + + } else if (isNonEmpty(bearerCredentialsSource)) { + BearerAuthCredentialProvider bearerAuthCredentialProvider = + BearerAuthCredentialProviderFactory.getBearerAuthCredentialProvider( + bearerCredentialsSource, + configs + ); + restService.setBearerAuthCredentialProvider(bearerAuthCredentialProvider); + } + } + } + + private static boolean isNonEmpty(String s) { + return s != null && !s.isEmpty(); + } + + private int registerAndGetId(String subject, Schema schema) + throws IOException, RestClientException { + return restService.registerSchema(schema.toString(), subject); + } + + private int registerAndGetId(String subject, Schema schema, int version, int id) + throws IOException, RestClientException { + return restService.registerSchema(schema.toString(), subject, version, id); + } + + private Schema getSchemaByIdFromRegistry(final int id) throws IOException, RestClientException { + final SchemaString restSchema = restService.getId(id); + + final Schema.Parser parser = new Schema.Parser(); + Optional.ofNullable(validateSchemaFieldNames).ifPresent(parser::setValidate); + Optional.ofNullable(validateSchemaDefaults).ifPresent(parser::setValidateDefaults); + + return parser.parse(restSchema.getSchemaString()); + } + + private int getVersionFromRegistry(String subject, Schema schema) + throws IOException, RestClientException { + io.confluent.kafka.schemaregistry.client.rest.entities.Schema response = + restService.lookUpSubjectVersion(schema.toString(), subject, true); + return response.getVersion(); + } + + private int getIdFromRegistry(String subject, Schema schema) + throws IOException, RestClientException { + io.confluent.kafka.schemaregistry.client.rest.entities.Schema response = + restService.lookUpSubjectVersion(schema.toString(), subject, false); + return response.getId(); + } + + @Override + public synchronized int register(String subject, Schema schema) + throws IOException, RestClientException { + return register(subject, schema, 0, -1); + } + + @Override + public synchronized int register(String subject, Schema schema, int version, int id) + throws IOException, RestClientException { + final Map schemaIdMap = + schemaCache.computeIfAbsent(subject, k -> new HashMap<>()); + + final Integer cachedId = schemaIdMap.get(schema); + if (cachedId != null) { + if (id >= 0 && id != cachedId) { + throw new IllegalStateException("Schema already registered with id " + + cachedId + " instead of input id " + id); + } + return cachedId; + } + + if (schemaIdMap.size() >= identityMapCapacity) { + throw new IllegalStateException("Too many schema objects created for " + subject + "!"); + } + + final int retrievedId = id >= 0 + ? registerAndGetId(subject, schema, version, id) + : registerAndGetId(subject, schema); + schemaIdMap.put(schema, retrievedId); + idCache.get(null).put(retrievedId, schema); + return retrievedId; + } + + @Override + public Schema getByID(final int id) throws IOException, RestClientException { + return getById(id); + } + + @Override + public synchronized Schema getById(int id) throws IOException, RestClientException { + return getBySubjectAndId(null, id); + } + + @Override + public Schema getBySubjectAndID(final String subject, final int id) + throws IOException, RestClientException { + return getBySubjectAndId(subject, id); + } + + @Override + public synchronized Schema getBySubjectAndId(String subject, int id) + throws IOException, RestClientException { + + final Map idSchemaMap = idCache + .computeIfAbsent(subject, k -> new HashMap<>()); + + final Schema cachedSchema = idSchemaMap.get(id); + if (cachedSchema != null) { + return cachedSchema; + } + + final Schema retrievedSchema = getSchemaByIdFromRegistry(id); + idSchemaMap.put(id, retrievedSchema); + return retrievedSchema; + } + + @Override + public SchemaMetadata getSchemaMetadata(String subject, int version) + throws IOException, RestClientException { + io.confluent.kafka.schemaregistry.client.rest.entities.Schema response + = restService.getVersion(subject, version); + int id = response.getId(); + String schema = response.getSchema(); + return new SchemaMetadata(id, version, schema); + } + + @Override + public synchronized SchemaMetadata getLatestSchemaMetadata(String subject) + throws IOException, RestClientException { + io.confluent.kafka.schemaregistry.client.rest.entities.Schema response + = restService.getLatestVersion(subject); + int id = response.getId(); + int version = response.getVersion(); + String schema = response.getSchema(); + return new SchemaMetadata(id, version, schema); + } + + @Override + public synchronized int getVersion(String subject, Schema schema) + throws IOException, RestClientException { + final Map schemaVersionMap = + versionCache.computeIfAbsent(subject, k -> new HashMap<>()); + + final Integer cachedVersion = schemaVersionMap.get(schema); + if (cachedVersion != null) { + return cachedVersion; + } + + if (schemaVersionMap.size() >= identityMapCapacity) { + throw new IllegalStateException("Too many schema objects created for " + subject + "!"); + } + + final int retrievedVersion = getVersionFromRegistry(subject, schema); + schemaVersionMap.put(schema, retrievedVersion); + return retrievedVersion; + } + + @Override + public List getAllVersions(String subject) + throws IOException, RestClientException { + return restService.getAllVersions(subject); + } + + @Override + public synchronized int getId(String subject, Schema schema) + throws IOException, RestClientException { + final Map schemaIdMap = + schemaCache.computeIfAbsent(subject, k -> new HashMap<>()); + + final Integer cachedId = schemaIdMap.get(schema); + if (cachedId != null) { + return cachedId; + } + + if (schemaIdMap.size() >= identityMapCapacity) { + throw new IllegalStateException("Too many schema objects created for " + subject + "!"); + } + + final int retrievedId = getIdFromRegistry(subject, schema); + schemaIdMap.put(schema, retrievedId); + idCache.get(null).put(retrievedId, schema); + return retrievedId; + } + + @Override + public List deleteSubject(String subject) throws IOException, RestClientException { + return deleteSubject(DEFAULT_REQUEST_PROPERTIES, subject); + } + + @Override + public synchronized List deleteSubject( + Map requestProperties, String subject) + throws IOException, RestClientException { + Objects.requireNonNull(subject, "subject"); + versionCache.remove(subject); + idCache.remove(subject); + schemaCache.remove(subject); + return restService.deleteSubject(requestProperties, subject); + } + + @Override + public Integer deleteSchemaVersion(String subject, String version) + throws IOException, RestClientException { + return deleteSchemaVersion(DEFAULT_REQUEST_PROPERTIES, subject, version); + } + + @Override + public synchronized Integer deleteSchemaVersion( + Map requestProperties, + String subject, + String version) + throws IOException, RestClientException { + versionCache + .getOrDefault(subject, Collections.emptyMap()) + .values() + .remove(Integer.valueOf(version)); + return restService.deleteSchemaVersion(requestProperties, subject, version); + } + + @Override + public boolean testCompatibility(String subject, Schema schema) + throws IOException, RestClientException { + return restService.testCompatibility(schema.toString(), subject, "latest"); + } + + @Override + public String updateCompatibility(String subject, String compatibility) + throws IOException, RestClientException { + ConfigUpdateRequest response = restService.updateCompatibility(compatibility, subject); + return response.getCompatibilityLevel(); + } + + @Override + public String getCompatibility(String subject) throws IOException, RestClientException { + Config response = restService.getConfig(subject); + return response.getCompatibilityLevel(); + } + + @Override + public String setMode(String mode) + throws IOException, RestClientException { + ModeUpdateRequest response = restService.setMode(mode); + return response.getMode(); + } + + @Override + public String setMode(String mode, String subject) + throws IOException, RestClientException { + ModeUpdateRequest response = restService.setMode(mode, subject); + return response.getMode(); + } + + @Override + public String getMode() throws IOException, RestClientException { + ModeGetResponse response = restService.getMode(); + return response.getMode(); + } + + @Override + public String getMode(String subject) throws IOException, RestClientException { + ModeGetResponse response = restService.getMode(subject); + return response.getMode(); + } + + @Override + public Collection getAllSubjects() throws IOException, RestClientException { + return restService.getAllSubjects(); + } + + @Override + public void reset() { + schemaCache.clear(); + idCache.clear(); + versionCache.clear(); + idCache.put(null, new HashMap<>()); + } +} \ No newline at end of file diff --git a/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/BigqueryDatastreamConfiguration.java b/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/BigqueryDatastreamConfiguration.java new file mode 100644 index 000000000..d9bdb1081 --- /dev/null +++ b/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/BigqueryDatastreamConfiguration.java @@ -0,0 +1,220 @@ +/** + * Copyright 2020 Wayfair LLC. All rights reserved. + * Licensed under the BSD 2-Clause License. See the LICENSE file in the project root for license information. + * See the NOTICE file in the project root for additional information regarding copyright ownership. + */ +package com.linkedin.datastream.bigquery; + + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Objects; +import java.util.Optional; + +import org.apache.avro.Schema; + +import com.linkedin.datastream.bigquery.schema.BigquerySchemaEvolver; +import com.linkedin.datastream.bigquery.schema.FixedBigquerySchemaEvolver; +import com.linkedin.datastream.serde.Deserializer; + +/** + * A class to hold Bigquery Datastream configurations. + */ +public class BigqueryDatastreamConfiguration { + + private final BigqueryDatastreamDestination _destination; + private final BigquerySchemaEvolver _schemaEvolver; + private final boolean _createDestinationTable; + private final Deserializer _valueDeserializer; + private final Serializer _valueSerializer; + private final Long _partitionExpirationDays; + private final BigqueryDatastreamConfiguration _deadLetterTableConfiguration; + private final List _labels; + private final Schema _fixedSchema; + + /** + * Constructor. + * @param builder the Builder + */ + private BigqueryDatastreamConfiguration(final Builder builder) { + if (builder._schemaEvolver instanceof FixedBigquerySchemaEvolver && builder._fixedSchema == null) { + throw new IllegalArgumentException("fixedSchema must not be null when schemaEvolver type is FixedBigquerySchemaEvolver"); + } + _destination = builder._destination; + _schemaEvolver = builder._schemaEvolver; + _createDestinationTable = builder._createDestinationTable; + _partitionExpirationDays = builder._partitionExpirationDays; + _deadLetterTableConfiguration = builder._deadLetterTableConfiguration; + if (builder._labels != null) { + _labels = new ArrayList<>(builder._labels); + } else { + _labels = Collections.emptyList(); + } + _valueDeserializer = builder._valueDeserializer; + _valueSerializer = builder._valueSerializer; + _fixedSchema = builder._fixedSchema; + } + + public BigqueryDatastreamDestination getDestination() { + return _destination; + } + + public Optional getPartitionExpirationDays() { + return Optional.ofNullable(_partitionExpirationDays); + } + + public BigquerySchemaEvolver getSchemaEvolver() { + return _schemaEvolver; + } + + public boolean isCreateDestinationTableEnabled() { + return _createDestinationTable; + } + + public Deserializer getValueDeserializer() { + return _valueDeserializer; + } + + public Serializer getValueSerializer() { + return _valueSerializer; + } + + public Optional getDeadLetterTableConfiguration() { + return Optional.ofNullable(_deadLetterTableConfiguration); + } + + public List getLabels() { + return Collections.unmodifiableList(_labels); + } + + public Optional getFixedSchema() { + return Optional.ofNullable(_fixedSchema); + } + + /** + * Builder for BigqueryDatastreamConfiguration. + */ + public static class Builder { + private final BigqueryDatastreamDestination _destination; + private final BigquerySchemaEvolver _schemaEvolver; + private final boolean _createDestinationTable; + private final Deserializer _valueDeserializer; + private final Serializer _valueSerializer; + + private Long _partitionExpirationDays; + private BigqueryDatastreamConfiguration _deadLetterTableConfiguration; + private List _labels; + private Schema _fixedSchema; + + /** + * Constructor. + * @param schemaEvolver the BigquerySchemaEvolver + * @param createDestinationTable a boolean + * @param valueDeserializer a value Deserializer + * @param valueSerializer a value Serializer + */ + public Builder(final BigqueryDatastreamDestination destination, final BigquerySchemaEvolver schemaEvolver, + final boolean createDestinationTable, + final Deserializer valueDeserializer, final Serializer valueSerializer + ) { + _schemaEvolver = schemaEvolver; + _createDestinationTable = createDestinationTable; + _valueDeserializer = valueDeserializer; + _valueSerializer = valueSerializer; + _destination = destination; + + _deadLetterTableConfiguration = null; + _partitionExpirationDays = null; + _labels = null; + _fixedSchema = null; + } + + /** + * Set partition expiration days. + * @param partitionExpirationDays a long value + * @return the Builder + */ + public Builder withPartitionExpirationDays(final long partitionExpirationDays) { + _partitionExpirationDays = partitionExpirationDays; + return this; + } + + /** + * Set labels list. + * @param labels a List of BigqueryLabel objects + * @return the Builder + */ + public Builder withLabels(final List labels) { + _labels = labels; + return this; + } + + /** + * Set the fixed schema. + * @param fixedSchema an Avro Schema + * @return the Builder + */ + public Builder withFixedSchema(final Schema fixedSchema) { + _fixedSchema = fixedSchema; + return this; + } + + /** + * Set the dead letter table configuration. + * @param deadLetterTableConfiguration the BigqueryDatastreamConfiguration + * @return the Builder + */ + public Builder withDeadLetterTableConfiguration(final BigqueryDatastreamConfiguration deadLetterTableConfiguration) { + _deadLetterTableConfiguration = deadLetterTableConfiguration; + return this; + } + + /** + * Build the BigqueryDatastreamConfiguration. + * @return the BigqueryDatastreamConfiguration + */ + public BigqueryDatastreamConfiguration build() { + return new BigqueryDatastreamConfiguration(this); + } + } + + /** + * Create a new Builder. + * @param destination a BigqueryDatastreamDestination + * @param schemaEvolver a BigquerySchemaEvolver + * @param createDestinationTable a boolean + * @param valueDeserializer a value Deserializer + * @param valueSerializer a value Serializer + * @return the Builder + */ + public static Builder builder(final BigqueryDatastreamDestination destination, + final BigquerySchemaEvolver schemaEvolver, final boolean createDestinationTable, + final Deserializer valueDeserializer, final Serializer valueSerializer) { + return new Builder(destination, schemaEvolver, createDestinationTable, valueDeserializer, valueSerializer); + } + + @Override + public boolean equals(final Object o) { + if (this == o) { + return true; + } else if (o == null || getClass() != o.getClass()) { + return false; + } + final BigqueryDatastreamConfiguration that = (BigqueryDatastreamConfiguration) o; + return _schemaEvolver.equals(that._schemaEvolver) && _createDestinationTable == that._createDestinationTable + && _valueDeserializer.equals(that._valueDeserializer) + && _valueSerializer.equals(that._valueSerializer) + && _destination.equals(that._destination) + && Objects.equals(_deadLetterTableConfiguration, that._deadLetterTableConfiguration) + && Objects.equals(_partitionExpirationDays, that._partitionExpirationDays) + && Objects.equals(_labels, that._labels) + && Objects.equals(_fixedSchema, that._fixedSchema); + } + + @Override + public int hashCode() { + return Objects.hash(_schemaEvolver, _createDestinationTable, _valueDeserializer, _valueSerializer, + _destination, _deadLetterTableConfiguration, _partitionExpirationDays, _labels, _fixedSchema); + } +} diff --git a/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/BigqueryDatastreamConfigurationFactory.java b/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/BigqueryDatastreamConfigurationFactory.java new file mode 100644 index 000000000..815140ca5 --- /dev/null +++ b/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/BigqueryDatastreamConfigurationFactory.java @@ -0,0 +1,127 @@ +/* + * Copyright 2020 Wayfair LLC. All rights reserved. + * Licensed under the BSD 2-Clause License. See the LICENSE file in the project root for license information. + * See the NOTICE file in the project root for additional information regarding copyright ownership. + */ + +package com.linkedin.datastream.bigquery; + +import java.io.IOException; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; + +import org.apache.avro.Schema; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient; +import io.confluent.kafka.schemaregistry.client.rest.exceptions.RestClientException; +import io.confluent.kafka.serializers.AbstractKafkaAvroSerDeConfig; +import io.confluent.kafka.serializers.KafkaAvroDeserializer; +import io.confluent.kafka.serializers.KafkaAvroDeserializerConfig; +import io.confluent.kafka.serializers.KafkaAvroSerializer; + +import com.linkedin.datastream.bigquery.schema.BigquerySchemaEvolver; +import com.linkedin.datastream.bigquery.schema.FixedBigquerySchemaEvolver; +import com.linkedin.datastream.common.DatastreamRuntimeException; + +/** + * A factory for creating BigqueryDatastreamConfiguration. + */ +public class BigqueryDatastreamConfigurationFactory { + + private final Logger logger = LoggerFactory.getLogger(BigqueryDatastreamConfigurationFactory.class); + + /** + * Create a BigqueryDatastreamConfiguration. + * @param destination a BigqueryDatastreamDestination + * @param datastreamName the name of the Datastream + * @param schemaRegistryLocation a String pointing to a schema registry endpoint + * @param schemaEvolver a BigquerySchemaEvolver + * @param autoCreateTable a boolean controlling if the destination table should be auto-created + * @param partitionExpirationDays a Long to define partition expiration days, if null partitions are not set to expire + * @param deadLetterTableConfiguration a BigqueryDatastreamConfiguration for the dead letter table associated with this Datastream + * @param labels a List of BigqueryLabel to set on the destination table + * @param schemaId an optional Integer for a fixed schema id + * @param relaxedAvroSchemaValidation an optional Boolean to control relaxed Avro schema validation for this Datastream + * @return the BigqueryDatastreamConfiguration + */ + public BigqueryDatastreamConfiguration createBigqueryDatastreamConfiguration( + final BigqueryDatastreamDestination destination, + final String datastreamName, + final String schemaRegistryLocation, + final BigquerySchemaEvolver schemaEvolver, + final boolean autoCreateTable, + final Long partitionExpirationDays, + final BigqueryDatastreamConfiguration deadLetterTableConfiguration, + final List labels, + final Integer schemaId, + final Boolean relaxedAvroSchemaValidation) { + + // If relaxed Avro schema validation is defined, then explicitly enable/disable validations via config, otherwise use validation defaults + final Map schemaRegistryConfig = Optional.ofNullable(relaxedAvroSchemaValidation) + // Convert relaxedAvroSchemaValidation boolean to enableValidation boolean + .map(b -> !b) + .map(enableValidation -> { + final Map config = new HashMap<>(2); + config.put(BigquerySchemaRegistryClientConfig.SCHEMA_REGISTRY_PARSER_VALIDATE_FIELD_NAMES, enableValidation); + config.put(BigquerySchemaRegistryClientConfig.SCHEMA_REGISTRY_PARSER_VALIDATE_DEFAULTS, enableValidation); + return Collections.unmodifiableMap(config); + }).orElse(Collections.emptyMap()); + final SchemaRegistryClient schemaRegistryClient = new BigqueryCachedSchemaRegistryClient( + schemaRegistryLocation, + AbstractKafkaAvroSerDeConfig.MAX_SCHEMAS_PER_SUBJECT_DEFAULT, + schemaRegistryConfig + ); + + final Map valueSerDeConfig = new HashMap<>(); + valueSerDeConfig.put(KafkaAvroDeserializerConfig.SCHEMA_REGISTRY_URL_CONFIG, schemaRegistryLocation); + + final KafkaSerializer valueSerializer = new KafkaSerializer(new KafkaAvroSerializer(schemaRegistryClient, valueSerDeConfig)); + final KafkaDeserializer valueDeserializer = new KafkaDeserializer(new KafkaAvroDeserializer(schemaRegistryClient, valueSerDeConfig)); + + + final BigqueryDatastreamConfiguration.Builder configBuilder = BigqueryDatastreamConfiguration.builder( + destination, + schemaEvolver, + autoCreateTable, + valueDeserializer, + valueSerializer + ); + + if (schemaEvolver instanceof FixedBigquerySchemaEvolver) { + if (schemaId == null) { + throw new IllegalArgumentException("schema ID is required for fixed schema evolution mode"); + } + try { + final Schema schema = schemaRegistryClient.getById(schemaId); + if (schema != null) { + configBuilder.withFixedSchema(schema); + } else { + logger.error("Required schema not found with ID {} for datastream with name '{}'", schemaId, datastreamName); + throw new IllegalStateException("required schema not found for datastream"); + } + + } catch (IOException | RestClientException e) { + logger.error("Error fetching schema with ID {} for datastream with name '{}'", schemaId, datastreamName, e); + throw new DatastreamRuntimeException("Error fetching schema for datastream", e); + } + } else if (schemaId != null) { + logger.warn("Schema ID provided for datastream with name '{}', but schema evolution mode is not set to fixed. Ignoring...", datastreamName); + } + + Optional.ofNullable(partitionExpirationDays).ifPresent(configBuilder::withPartitionExpirationDays); + + Optional.ofNullable(deadLetterTableConfiguration).ifPresent(configBuilder::withDeadLetterTableConfiguration); + + if (labels != null && !labels.isEmpty()) { + configBuilder.withLabels(labels); + } + + return configBuilder.build(); + } + +} diff --git a/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/BigqueryDatastreamDestination.java b/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/BigqueryDatastreamDestination.java new file mode 100644 index 000000000..49820ec01 --- /dev/null +++ b/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/BigqueryDatastreamDestination.java @@ -0,0 +1,123 @@ +/** + * Copyright 2020 Wayfair LLC. All rights reserved. + * Licensed under the BSD 2-Clause License. See the LICENSE file in the project root for license information. + * See the NOTICE file in the project root for additional information regarding copyright ownership. + */ +package com.linkedin.datastream.bigquery; + +import java.net.URI; +import java.net.URISyntaxException; +import java.util.Objects; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +import org.apache.commons.lang3.Validate; +import org.apache.http.client.utils.URIBuilder; + +/** + * A class to hold Bigquery Datastream destination information. + */ +public class BigqueryDatastreamDestination { + private final String projectId; + private final String datasetId; + private final String destinatonName; + private final URI uri; + private final boolean _wildcardDestination; + + private static final String SCHEME = "brooklin-bigquery"; + private static final Pattern DESTINATION_PATTERN = Pattern.compile("^" + SCHEME + "://([^.]+)\\.([^.]+)\\.(.+)$"); + private static final Pattern WILDCARD_DESTINATION_PATTERN = Pattern.compile("^[^*]*\\*?[^*]*$"); + + /** + * Constructor. + * @param projectId a String + * @param datasetId a String + * @param destinationName a String + */ + public BigqueryDatastreamDestination(final String projectId, final String datasetId, final String destinationName) { + Validate.notBlank(projectId, "projectId cannot be blank"); + Validate.notBlank(datasetId, "datasetId cannot be blank"); + Validate.notBlank(datasetId, "destinationName cannot be blank"); + Validate.matchesPattern(destinationName, WILDCARD_DESTINATION_PATTERN.pattern(), "wildcard destinationName must contain a single *"); + + this.projectId = projectId; + this.datasetId = datasetId; + this.destinatonName = destinationName; + uri = toUri(this); + _wildcardDestination = WILDCARD_DESTINATION_PATTERN.matcher(destinationName).matches(); + } + + /** + * Utility function to parse a destination String into a BigqueryDatastreamDestination + * @param destination a String + * @return the BigqueryDatastreamDestination + */ + public static BigqueryDatastreamDestination parse(final String destination) { + final Matcher destinationMatcher = DESTINATION_PATTERN.matcher(destination); + Validate.isTrue(destinationMatcher.matches(), "destination is malformed"); + return new BigqueryDatastreamDestination(destinationMatcher.group(1), destinationMatcher.group(2), destinationMatcher.group(3)); + } + + @Override + public String toString() { + return uri.toString(); + } + + public URI getUri() { + return uri; + } + + private static URI toUri(final BigqueryDatastreamDestination destination) { + final String host = String.join(".", destination.projectId, destination.datasetId, destination.destinatonName); + final URIBuilder builder = new URIBuilder() + .setScheme(SCHEME) + .setHost(host); + try { + return builder.build(); + } catch (final URISyntaxException e) { + throw new IllegalStateException(e); + } + } + + public String getProjectId() { + return projectId; + } + + public String getDatasetId() { + return datasetId; + } + + public String getDestinatonName() { + return destinatonName; + } + + public boolean isWildcardDestination() { + return _wildcardDestination; + } + + /** + * Create a new BigqueryDatastreamDestination by replacing the wildcard with a value. + * @param value the placeholder value + * @return the BigqueryDatastreamDestination + */ + public BigqueryDatastreamDestination replaceWildcard(final String value) { + Validate.isTrue(_wildcardDestination, "destination must be a wildcard destination"); + return new BigqueryDatastreamDestination(projectId, datasetId, destinatonName.replace("*", value)); + } + + @Override + public boolean equals(final Object o) { + if (this == o) { + return true; + } else if (o == null || getClass() != o.getClass()) { + return false; + } + final BigqueryDatastreamDestination that = (BigqueryDatastreamDestination) o; + return projectId.equals(that.projectId) && datasetId.equals(that.datasetId) && destinatonName.equals(that.destinatonName); + } + + @Override + public int hashCode() { + return Objects.hash(projectId, datasetId, destinatonName); + } +} diff --git a/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/BigqueryExceptionType.java b/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/BigqueryExceptionType.java new file mode 100644 index 000000000..a46a8262e --- /dev/null +++ b/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/BigqueryExceptionType.java @@ -0,0 +1,17 @@ +/** + * Copyright 2020 Wayfair LLC. All rights reserved. + * Licensed under the BSD 2-Clause License. See the LICENSE file in the project root for license information. + * See the NOTICE file in the project root for additional information regarding copyright ownership. + */ +package com.linkedin.datastream.bigquery; + +/** + * An enumeration of Bigquery exception types. + */ +public enum BigqueryExceptionType { + SchemaEvolution, + Deserialization, + SchemaTranslation, + InsertError, + Other +} diff --git a/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/BigqueryLabel.java b/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/BigqueryLabel.java new file mode 100644 index 000000000..fa14733c6 --- /dev/null +++ b/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/BigqueryLabel.java @@ -0,0 +1,98 @@ +/* + * Copyright 2020 Wayfair LLC. All rights reserved. + * Licensed under the BSD 2-Clause License. See the LICENSE file in the project root for license information. + * See the NOTICE file in the project root for additional information regarding copyright ownership. + */ + +package com.linkedin.datastream.bigquery; + +import java.util.Objects; + +import org.apache.commons.lang3.Validate; + +/** + * A class representing a label on a BigQuery resource. + */ +public class BigqueryLabel { + private final String _name; + private final String _value; + + /** + * Constructor. + * @param name a String + * @param value a String + */ + public BigqueryLabel(final String name, final String value) { + Validate.notBlank(name, "name cannot be blank"); + Validate.notNull(value, "value cannot be null"); + if (!value.isEmpty()) { + Validate.notBlank(value, "value must not be blank when not empty"); + } + Validate.isTrue(name.equals(name.toLowerCase()), "name must be lowercase"); + Validate.isTrue(value.equals(value.toLowerCase()), "value must be lowercase"); + _name = name; + _value = value; + } + + /** + * Constructor. + * @param name a String + */ + public BigqueryLabel(final String name) { + this(name, ""); + } + + public String getName() { + return _name; + } + + public String getValue() { + return _value; + } + + /** + * Construct a BigqueryLabel with the given name. + * @param name a String + * @return a BigqueryLabel + */ + public static BigqueryLabel of(final String name) { + return new BigqueryLabel(name); + } + + /** + * Construct a BigqueryLabel with the given name and value. + * @param name a String + * @param value a String + * @return the BigqueryLabel + */ + public static BigqueryLabel of(final String name, final String value) { + return new BigqueryLabel(name, value); + } + + @Override + public String toString() { + final String label; + if (!_value.isEmpty()) { + label = String.join(":", _name, _value); + } else { + label = _name; + } + return label; + } + + @Override + public boolean equals(final Object o) { + if (this == o) { + return true; + } else if (o == null || getClass() != o.getClass()) { + return false; + } + final BigqueryLabel label = (BigqueryLabel) o; + return _name.equals(label._name) && _value.equals(label._value); + } + + @Override + public int hashCode() { + return Objects.hash(_name, _value); + } +} diff --git a/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/BigquerySchemaRegistryClientConfig.java b/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/BigquerySchemaRegistryClientConfig.java new file mode 100644 index 000000000..9e6e35de1 --- /dev/null +++ b/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/BigquerySchemaRegistryClientConfig.java @@ -0,0 +1,24 @@ +/* + * Copyright 2020 Wayfair LLC. All rights reserved. + * Licensed under the BSD 2-Clause License. See the LICENSE file in the project root for license information. + * See the NOTICE file in the project root for additional information regarding copyright ownership. + */ + +package com.linkedin.datastream.bigquery; + +/** + * A class that holds Schema Registry Client configuration constants. + */ +public class BigquerySchemaRegistryClientConfig extends io.confluent.kafka.schemaregistry.client.SchemaRegistryClientConfig { + + /** + * A configuration to enable/disable validating schema field names when parsing Avro schemas. + */ + public static final String SCHEMA_REGISTRY_PARSER_VALIDATE_FIELD_NAMES = "schema.registry.parser.validate.names"; + + /** + * A configuration to enable/disable validating schema field defaults when parsing Avro schemas. + */ + public static final String SCHEMA_REGISTRY_PARSER_VALIDATE_DEFAULTS = "schema.registry.parser.validate.defaults"; + +} diff --git a/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/BigqueryTransportProvider.java b/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/BigqueryTransportProvider.java index a5d438840..cb9833fa9 100644 --- a/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/BigqueryTransportProvider.java +++ b/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/BigqueryTransportProvider.java @@ -5,147 +5,249 @@ */ package com.linkedin.datastream.bigquery; -import java.util.concurrent.TimeUnit; - +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.stream.Collectors; + +import org.apache.avro.Schema; +import org.apache.avro.SchemaBuilder; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericRecordBuilder; +import org.apache.commons.lang3.exception.ExceptionUtils; +import org.apache.kafka.common.errors.SerializationException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.linkedin.datastream.common.VerifiableProperties; -import com.linkedin.datastream.server.api.transport.buffered.AbstractBatchBuilder; -import com.linkedin.datastream.server.api.transport.buffered.AbstractBufferedTransportProvider; +import com.linkedin.datastream.bigquery.schema.IncompatibleSchemaEvolutionException; +import com.linkedin.datastream.bigquery.translator.SchemaTranslationException; +import com.linkedin.datastream.common.BrooklinEnvelope; +import com.linkedin.datastream.common.BrooklinEnvelopeMetadataConstants; +import com.linkedin.datastream.common.DatastreamTransientException; +import com.linkedin.datastream.common.SendCallback; +import com.linkedin.datastream.serde.Deserializer; +import com.linkedin.datastream.server.DatastreamProducerRecord; +import com.linkedin.datastream.server.DatastreamProducerRecordBuilder; +import com.linkedin.datastream.server.api.transport.TransportProvider; + +import static com.linkedin.datastream.server.api.transport.buffered.AbstractBufferedTransportProvider.KAFKA_ORIGIN_CLUSTER; +import static com.linkedin.datastream.server.api.transport.buffered.AbstractBufferedTransportProvider.KAFKA_ORIGIN_OFFSET; +import static com.linkedin.datastream.server.api.transport.buffered.AbstractBufferedTransportProvider.KAFKA_ORIGIN_PARTITION; +import static com.linkedin.datastream.server.api.transport.buffered.AbstractBufferedTransportProvider.KAFKA_ORIGIN_TOPIC; /** * This is a Bigquery Transport provider that writes events to specified bigquery table. */ -public class BigqueryTransportProvider extends AbstractBufferedTransportProvider { - - private static final Logger LOG = LoggerFactory.getLogger(BigqueryTransportProvider.class.getName()); - - private final BigqueryBatchCommitter _committer; - - private BigqueryTransportProvider(BigqueryTransportProviderBuilder builder) { - super(builder._transportProviderName); - - this._committer = builder._committer; - - // initialize and start object builders - for (int i = 0; i < builder._batchBuilderCount; i++) { - _batchBuilders.add(new BatchBuilder( - builder._maxBatchSize, - builder._maxBatchAge, - builder._maxInflightBatchCommits, - builder._committer, - builder._batchBuilderQueueSize, - builder._translatorProperties)); - } - for (AbstractBatchBuilder batchBuilder : _batchBuilders) { - batchBuilder.start(); - } +public class BigqueryTransportProvider implements TransportProvider { + + private static final Logger _logger = LoggerFactory.getLogger(BigqueryTransportProvider.class.getName()); + + static final Schema EXCEPTION_TYPE_SCHEMA = Schema.createEnum("exceptionType", null, + "com.linkedin.datastream.bigquery", + Arrays.stream(BigqueryExceptionType.values()).map(BigqueryExceptionType::name).collect(Collectors.toList())); + + static final Schema EXCEPTION_RECORD_SCHEMA = SchemaBuilder.builder("com.linkedin.datastream.bigquery") + .record("ExceptionRecord").fields() + .requiredString("kafkaCluster") + .requiredString("kafkaTopic") + .requiredString("kafkaPartition") + .requiredString("kafkaOffset") + .requiredString("kafkaEventTimestamp") + .name("exceptionType").type(EXCEPTION_TYPE_SCHEMA).noDefault() + .requiredString("exceptionClass") + .requiredString("exceptionMessage") + .requiredString("exceptionStackTrace") + .requiredBytes("eventValueBytes") + .optionalString("eventValueString") + .endRecord(); + + private final BigqueryBufferedTransportProvider _bufferedTransportProvider; + private final Serializer _valueSerializer; + private final Deserializer _valueDeserializer; + private final BigqueryDatastreamConfiguration _datastreamConfiguration; + private final ConcurrentMap _destinationConfigurations; + private final ConcurrentHashMap.KeySetView _destinations; - // send periodic flush signal to commit stale objects - _scheduler.scheduleAtFixedRate( - () -> { - for (AbstractBatchBuilder objectBuilder: _batchBuilders) { - LOG.info("Try flush signal sent."); - objectBuilder.assign(new com.linkedin.datastream.common.Package.PackageBuilder().buildTryFlushSignalPackage()); - } - }, - builder._maxBatchAge / 2, - builder._maxBatchAge / 2, - TimeUnit.MILLISECONDS); + /** + * Constructor. + * @param bufferedTransportProvider a BigqueryBufferedTransportProvider + * @param valueSerializer a Serializer + * @param valueDeserializer a Deserializer + * @param datastreamConfiguration a BigqueryDatastreamConfiguration + * @param destinationConfigurations a mapping of BigqueryDatastreamDestination to BigqueryDatastreamConfiguration + */ + public BigqueryTransportProvider(final BigqueryBufferedTransportProvider bufferedTransportProvider, + final Serializer valueSerializer, + final Deserializer valueDeserializer, + final BigqueryDatastreamConfiguration datastreamConfiguration, + final ConcurrentMap destinationConfigurations) { + _bufferedTransportProvider = bufferedTransportProvider; + _valueSerializer = valueSerializer; + _valueDeserializer = valueDeserializer; + _datastreamConfiguration = datastreamConfiguration; + _destinationConfigurations = destinationConfigurations; + _destinations = ConcurrentHashMap.newKeySet(); } @Override - protected void shutdownCommitter() { - _committer.shutdown(); + public void send(final String destination, final DatastreamProducerRecord record, final SendCallback onComplete) { + final BigqueryDatastreamDestination datastreamDestination = parseOrConstructDestination(destination, record); + registerConfigurationForDestination(datastreamDestination, _datastreamConfiguration); + final SendCallback callbackHandler = _datastreamConfiguration.getDeadLetterTableConfiguration() + .map(deadLetterTableConfiguration -> exceptionHandlingCallbackHandler(datastreamDestination, record, onComplete, deadLetterTableConfiguration)) + .orElse(onComplete); + _bufferedTransportProvider.send(destination, record, callbackHandler); } - /** - * Builder class for {@link com.linkedin.datastream.bigquery.BigqueryTransportProvider} - */ - public static class BigqueryTransportProviderBuilder { - private String _transportProviderName; - private int _batchBuilderQueueSize; - private int _batchBuilderCount; - private int _maxBatchSize; - private int _maxBatchAge; - private int _maxInflightBatchCommits; - private BigqueryBatchCommitter _committer; - private VerifiableProperties _translatorProperties; - - /** - * Set the name of the transport provider - */ - public BigqueryTransportProviderBuilder setTransportProviderName(String transportProviderName) { - this._transportProviderName = transportProviderName; - return this; - } - - /** - * Set batch builder's queue size - */ - public BigqueryTransportProviderBuilder setBatchBuilderQueueSize(int batchBuilderQueueSize) { - this._batchBuilderQueueSize = batchBuilderQueueSize; - return this; + private BigqueryDatastreamDestination parseOrConstructDestination(final String destination, final DatastreamProducerRecord record) { + final BigqueryDatastreamDestination originalDestination = BigqueryDatastreamDestination.parse(destination); + final BigqueryDatastreamDestination finalDestination; + // Check for a datastream with a wildcard destination and replace the destination with the Kafka topic name. + // This condition should only occur for some legacy datastreams. + if (originalDestination.isWildcardDestination()) { + finalDestination = record.getEvents().stream().findAny() + .map(event -> originalDestination.replaceWildcard(event.getMetadata().get(KAFKA_ORIGIN_TOPIC))) + .orElse(originalDestination); + } else { + finalDestination = originalDestination; } + return finalDestination; + } - /** - * Set number of batch builders - */ - public BigqueryTransportProviderBuilder setBatchBuilderCount(int batchBuilderCount) { - this._batchBuilderCount = batchBuilderCount; - return this; - } + private SendCallback exceptionHandlingCallbackHandler(final BigqueryDatastreamDestination datastreamDestination, + final DatastreamProducerRecord record, + final SendCallback onComplete, + final BigqueryDatastreamConfiguration deadLetterTableConfiguration) { + return (metadata, exception) -> { + // If no exception is encountered or just a transient exception, then call the provided send callback. + // For transient exceptions, we expect the exception will resolve after some number of retries, + // so call the regular send callback to let the connector handle the retries. + // For all other exceptions, we expect the exception is unrecoverable and will not resolve with a retry, + // so create an exception record and send it to the exceptions/dead-letter table. + if (exception == null || exception instanceof DatastreamTransientException) { + onComplete.onCompletion(metadata, exception); + } else { + final BigqueryDatastreamDestination deadLetterTableDestination; + if (deadLetterTableConfiguration.getDestination().isWildcardDestination()) { + deadLetterTableDestination = deadLetterTableConfiguration.getDestination().replaceWildcard(datastreamDestination.getDestinatonName()); + } else { + deadLetterTableDestination = deadLetterTableConfiguration.getDestination(); + } + + final DatastreamProducerRecord exceptionRecord; + try { + exceptionRecord = createExceptionRecord(deadLetterTableDestination.getDestinatonName(), record, exception); + } catch (final Exception e) { + _logger.error("Unable to create BigQuery exception record", e); + onComplete.onCompletion(metadata, e); + return; + } + registerConfigurationForDestination(deadLetterTableDestination, deadLetterTableConfiguration); + // Send an exception record + _bufferedTransportProvider.send(deadLetterTableDestination.toString(), exceptionRecord, + (exceptionRecordMetadata, exceptionRecordException) -> + onComplete.onCompletion(metadata, + // Call the callback with the original exception if an exception is encountered + // while trying to insert an exception record + Optional.ofNullable(exceptionRecordException).map(e -> exception).orElse(null) + ) + ); + } + }; + } - /** - * Set max batch size - */ - public BigqueryTransportProviderBuilder setMaxBatchSize(int maxBatchSize) { - this._maxBatchSize = maxBatchSize; - return this; - } + private void registerConfigurationForDestination(final BigqueryDatastreamDestination destination, final BigqueryDatastreamConfiguration configuration) { + _destinations.add(destination); + _destinationConfigurations.put(destination, configuration); + } - /** - * Set max batch age - */ - public BigqueryTransportProviderBuilder setMaxBatchAge(int maxBatchAge) { - this._maxBatchAge = maxBatchAge; - return this; - } + Set getDestinations() { + return Collections.unmodifiableSet(_destinations); + } - /** - * Set max inflight commits - */ - public BigqueryTransportProviderBuilder setMaxInflightBatchCommits(int maxInflightBatchCommits) { - this._maxInflightBatchCommits = maxInflightBatchCommits; - return this; - } + @Override + public void close() { + } - /** - * Set batch committer - */ - public BigqueryTransportProviderBuilder setCommitter(BigqueryBatchCommitter committer) { - this._committer = committer; - return this; - } + @Override + public void flush() { + _bufferedTransportProvider.flush(); + } - /** - * Set translator configuration options - */ - public BigqueryTransportProviderBuilder setTranslatorProperties(VerifiableProperties translatorProperties) { - this._translatorProperties = translatorProperties; - return this; - } + private DatastreamProducerRecord createExceptionRecord(final String exceptionsTopicName, final DatastreamProducerRecord record, final Exception exception) { + final DatastreamProducerRecordBuilder exceptionRecordBuilder = new DatastreamProducerRecordBuilder(); + exceptionRecordBuilder.setSourceCheckpoint(record.getCheckpoint()); + exceptionRecordBuilder.setEventsSourceTimestamp(record.getEventsSourceTimestamp()); + record.getPartition().ifPresent(exceptionRecordBuilder::setPartition); + record.getPartitionKey().ifPresent(exceptionRecordBuilder::setPartitionKey); + record.getDestination().ifPresent(exceptionRecordBuilder::setDestination); + + final BigqueryExceptionType exceptionType = classifyException(exception); + final String exceptionClassName = exception.getClass().getName(); + final String exceptionMessage = Optional.ofNullable(exception.getMessage()).orElse(""); + final String exceptionStackTrace = ExceptionUtils.getStackTrace(exception); + record.getEventsSendTimestamp().ifPresent(exceptionRecordBuilder::setEventsSendTimestamp); + record.getEvents().stream().map(event -> { + final String cluster = event.getMetadata().get(KAFKA_ORIGIN_CLUSTER); + final String topic = event.getMetadata().get(KAFKA_ORIGIN_TOPIC); + final String partition = event.getMetadata().get(KAFKA_ORIGIN_PARTITION); + final String offset = event.getMetadata().get(KAFKA_ORIGIN_OFFSET); + final String eventTimestamp = event.getMetadata().get(BrooklinEnvelopeMetadataConstants.EVENT_TIMESTAMP); + final GenericRecordBuilder exceptionAvroRecordBuilder = new GenericRecordBuilder(EXCEPTION_RECORD_SCHEMA) + .set("kafkaCluster", cluster) + .set("kafkaTopic", topic) + .set("kafkaPartition", partition) + .set("kafkaOffset", offset) + .set("kafkaEventTimestamp", eventTimestamp) + .set("exceptionType", new GenericData.EnumSymbol(EXCEPTION_TYPE_SCHEMA, exceptionType.name())) + .set("exceptionClass", exceptionClassName) + .set("exceptionMessage", exceptionMessage) + .set("exceptionStackTrace", exceptionStackTrace); + if (event.value().isPresent()) { + final byte[] valueBytes = (byte[]) event.value().get(); + exceptionAvroRecordBuilder.set("eventValueBytes", ByteBuffer.wrap(valueBytes)); + try { + final Object value = _valueDeserializer.deserialize(valueBytes); + exceptionAvroRecordBuilder.set("eventValueString", value.toString()); + } catch (final Exception e) { + _logger.warn("Unable to deserialize event value", e); + } + } else { + exceptionAvroRecordBuilder.set("eventValueBytes", ByteBuffer.wrap(new byte[0])); + } + // Use a hard-coded topic name to serialize all exception records, so we only have to manage the exception record schema in one place + final byte[] exceptionRecordBytes = _valueSerializer.serialize("brooklin-bigquery-transport-exceptions", exceptionAvroRecordBuilder.build()); + final Map exceptionRecordMetadata = new HashMap<>(5); + exceptionRecordMetadata.put(KAFKA_ORIGIN_CLUSTER, cluster); + exceptionRecordMetadata.put(KAFKA_ORIGIN_TOPIC, exceptionsTopicName); + exceptionRecordMetadata.put(KAFKA_ORIGIN_PARTITION, partition); + exceptionRecordMetadata.put(KAFKA_ORIGIN_OFFSET, offset); + exceptionRecordMetadata.put(BrooklinEnvelopeMetadataConstants.EVENT_TIMESTAMP, eventTimestamp); + return new BrooklinEnvelope(new byte[0], exceptionRecordBytes, exceptionRecordMetadata); + }).forEachOrdered(exceptionRecordBuilder::addEvent); + return exceptionRecordBuilder.build(); + } - /** - * Build the BigqueryTransportProvider. - * @return - * BigqueryTransportProvider that is created. - */ - public BigqueryTransportProvider build() { - return new BigqueryTransportProvider(this); + private static BigqueryExceptionType classifyException(final Exception e) { + final BigqueryExceptionType exceptionType; + if (e instanceof IncompatibleSchemaEvolutionException) { + exceptionType = BigqueryExceptionType.SchemaEvolution; + } else if (e instanceof TransientStreamingInsertException) { + exceptionType = BigqueryExceptionType.InsertError; + } else if (e instanceof SerializationException) { + exceptionType = BigqueryExceptionType.Deserialization; + } else if (e instanceof SchemaTranslationException) { + exceptionType = BigqueryExceptionType.SchemaTranslation; + } else { + exceptionType = BigqueryExceptionType.Other; } + return exceptionType; } - } diff --git a/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/BigqueryTransportProviderAdmin.java b/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/BigqueryTransportProviderAdmin.java index cf34a1e27..6ac7e5b80 100644 --- a/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/BigqueryTransportProviderAdmin.java +++ b/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/BigqueryTransportProviderAdmin.java @@ -6,19 +6,36 @@ package com.linkedin.datastream.bigquery; import java.time.Duration; -import java.util.Properties; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.regex.Pattern; +import java.util.stream.Collectors; +import org.apache.commons.lang3.StringUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import com.linkedin.datastream.bigquery.schema.BigquerySchemaEvolver; +import com.linkedin.datastream.bigquery.schema.BigquerySchemaEvolverFactory; +import com.linkedin.datastream.bigquery.schema.BigquerySchemaEvolverType; import com.linkedin.datastream.common.Datastream; import com.linkedin.datastream.common.DatastreamDestination; -import com.linkedin.datastream.common.VerifiableProperties; +import com.linkedin.datastream.common.DatastreamRuntimeException; +import com.linkedin.datastream.common.DatastreamSource; +import com.linkedin.datastream.common.DatastreamUtils; import com.linkedin.datastream.server.DatastreamTask; +import com.linkedin.datastream.server.Pair; import com.linkedin.datastream.server.api.connector.DatastreamValidationException; import com.linkedin.datastream.server.api.transport.TransportProvider; import com.linkedin.datastream.server.api.transport.TransportProviderAdmin; + /** * {@link TransportProviderAdmin} implementation for {@link BigqueryTransportProvider} * @@ -28,49 +45,127 @@ * */ public class BigqueryTransportProviderAdmin implements TransportProviderAdmin { - private static final Logger LOG = LoggerFactory.getLogger(BigqueryTransportProviderAdmin.class); - private static final String CONFIG_BATCHBUILDER_QUEUE_SIZE = "batchBuilderQueueSize"; - private static final String CONFIG_BATCHBUILDER_THREAD_COUNT = "batchBuilderThreadCount"; - private static final String CONFIG_MAX_BATCH_SIZE = "maxBatchSize"; - private static final String CONFIG_MAX_BATCH_AGE = "maxBatchAge"; - private static final String CONFIG_MAX_INFLIGHT_COMMITS = "maxInflightCommits"; + protected static final String METADATA_DATASET_KEY = "dataset"; + protected static final String METADATA_TABLE_SUFFIX_KEY = "tableSuffix"; + protected static final String METADATA_PARTITION_EXPIRATION_DAYS_KEY = "partitionExpirationDays"; + protected static final String METADATA_SCHEMA_EVOLUTION_MODE_KEY = "schemaEvolutionMode"; + protected static final String METADATA_DEAD_LETTER_TABLE_KEY = "deadLetterTable"; + protected static final String METADATA_DISABLE_DEAD_LETTER_TABLE_KEY = "disableDeadLetterTable"; + protected static final String METADATA_AUTO_CREATE_TABLE_KEY = "autoCreateTables"; + protected static final String METADATA_LABELS_KEY = "labels"; + protected static final String METADATA_SCHEMA_REGISTRY_LOCATION_KEY = "schemaRegistryLocation"; + protected static final String METADATA_SCHEMA_ID_KEY = "schemaID"; + protected static final String METADATA_RELAX_AVRO_SCHEMA_VALIDATION = "relaxAvroSchemaValidation"; + + private static final int DEFAULT_NUMBER_PARTITIONS = 1; + + private final BigqueryBufferedTransportProvider _bufferedTransportProvider; + private final String _legacyDefaultProjectId; + private final String _legacyDefaultSchemaRegistryUrl; + + private final BigquerySchemaEvolver _defaultSchemaEvolver; + + private final BigqueryTransportProviderFactory _bigqueryTransportProviderFactory; + private final BigqueryDatastreamConfigurationFactory _bigqueryDatastreamConfigurationFactory; - private static final String CONFIG_TRANSLATOR_DOMAIN_PREFIX = "translator"; + private final Map _bigquerySchemaEvolverMap; + private final ConcurrentMap _datastreamConfigByDestination; + private final ConcurrentMap _datastreamTransportProvider; + private final ConcurrentMap> _transportProviderTasks; - private static final String CONFIG_COMMITTER_DOMAIN_PREFIX = "committer"; + private final Pattern _legacyDatastreamDestinationConnectionStringPattern; - private BigqueryTransportProvider _transportProvider; + private final Logger logger = LoggerFactory.getLogger(BigqueryTransportProviderAdmin.class); /** - * Constructor for BigqueryTransportProviderAdmin. - * @param props TransportProviderAdmin configuration properties, e.g. number of committer threads, file format. + * Constructor. */ - public BigqueryTransportProviderAdmin(String transportProviderName, Properties props) { - VerifiableProperties tpProperties = new VerifiableProperties(props); - - VerifiableProperties committerProperties = new VerifiableProperties(tpProperties.getDomainProperties( - CONFIG_COMMITTER_DOMAIN_PREFIX, false)); - - _transportProvider = new BigqueryTransportProvider.BigqueryTransportProviderBuilder() - .setTransportProviderName(transportProviderName) - .setBatchBuilderQueueSize(tpProperties.getInt(CONFIG_BATCHBUILDER_QUEUE_SIZE, 1000)) - .setBatchBuilderCount(tpProperties.getInt(CONFIG_BATCHBUILDER_THREAD_COUNT, 5)) - .setMaxBatchSize(tpProperties.getInt(CONFIG_MAX_BATCH_SIZE, 100000)) - .setMaxBatchAge(tpProperties.getInt(CONFIG_MAX_BATCH_AGE, 500)) - .setMaxInflightBatchCommits(tpProperties.getInt(CONFIG_MAX_INFLIGHT_COMMITS, 1)) - .setCommitter(new BigqueryBatchCommitter(committerProperties)) - .setTranslatorProperties(new VerifiableProperties(tpProperties.getDomainProperties(CONFIG_TRANSLATOR_DOMAIN_PREFIX))) - .build(); + public BigqueryTransportProviderAdmin(final BigqueryBufferedTransportProvider bufferedTransportProvider, + final BigquerySchemaEvolver defaultSchemaEvolver, + final ConcurrentMap datastreamConfigByDestination, + final Map bigquerySchemaEvolverMap, + final BigqueryTransportProviderFactory bigqueryTransportProviderFactory, + final BigqueryDatastreamConfigurationFactory bigqueryDatastreamConfigurationFactory, + final String legacyDefaultProjectId, + final String legacyDefaultSchemaRegistryUrl) { + this._bufferedTransportProvider = bufferedTransportProvider; + this._defaultSchemaEvolver = defaultSchemaEvolver; + this._datastreamConfigByDestination = datastreamConfigByDestination; + this._bigquerySchemaEvolverMap = bigquerySchemaEvolverMap; + this._bigqueryTransportProviderFactory = bigqueryTransportProviderFactory; + this._bigqueryDatastreamConfigurationFactory = bigqueryDatastreamConfigurationFactory; + + this._legacyDefaultProjectId = legacyDefaultProjectId; + this._legacyDefaultSchemaRegistryUrl = legacyDefaultSchemaRegistryUrl; + + _legacyDatastreamDestinationConnectionStringPattern = Pattern.compile("^[^/]+/[^/]+(/[^/]*)?$"); + + _datastreamTransportProvider = new ConcurrentHashMap<>(); + _transportProviderTasks = new ConcurrentHashMap<>(); } + @Override - public TransportProvider assignTransportProvider(DatastreamTask task) { - return _transportProvider; + public TransportProvider assignTransportProvider(final DatastreamTask task) { + // Assume that the task has a single datastream + final Datastream datastream = task.getDatastreams().get(0); + // For legacy datastreams, update the destination connection string and late-init the destination name when record is sent + if (isLegacyDatastreamDestinationConnectionString(datastream.getDestination().getConnectionString())) { + try { + updateConnectionStringOnLegacyDatastream(datastream, "*"); + } catch (final DatastreamValidationException e) { + logger.error("Unable to assign invalid datastream with name: {}", datastream.getName(), e); + throw new DatastreamRuntimeException("Unable to assign invalid datastream", e); + } + } + + return _datastreamTransportProvider.compute(datastream, (d, transportProvider) -> { + if (transportProvider == null) { + final BigqueryDatastreamConfiguration configuration = getConfigurationFromDatastream(d); + transportProvider = _bigqueryTransportProviderFactory.createTransportProvider(_bufferedTransportProvider, + configuration.getValueSerializer(), configuration.getValueDeserializer(), configuration, _datastreamConfigByDestination); + } + _transportProviderTasks.compute(transportProvider, (tp, taskSet) -> { + if (taskSet == null) { + taskSet = ConcurrentHashMap.newKeySet(); + } + taskSet.add(task); + return taskSet; + }); + return transportProvider; + }); + } + + Map getDatastreamTransportProviders() { + return Collections.unmodifiableMap(_datastreamTransportProvider); + } + + Map> getTransportProviderTasks() { + return Collections.unmodifiableMap(_transportProviderTasks); } @Override - public void unassignTransportProvider(DatastreamTask task) { + public void unassignTransportProvider(final DatastreamTask task) { + // Assume that the task has a single datastream + final Datastream datastream = task.getDatastreams().get(0); + _datastreamTransportProvider.computeIfPresent(datastream, (d, transportProvider) -> { + // Remove the task from the transport provider tasks set and clear out the mapping if the tasks set is empty + final Set remainingTransportProviderTasks = _transportProviderTasks.computeIfPresent(transportProvider, (tp, tasks) -> { + tasks.remove(task); + return tasks.isEmpty() ? null : tasks; + }); + + // If the transport provider has no more tasks assigned, then close and remove the mapping, else retain the mapping + if (remainingTransportProviderTasks == null || remainingTransportProviderTasks.isEmpty()) { + // Do not remove destination configurations as we do not know when the last event is committed + //transportProvider.getDestinations().forEach(_datastreamConfigByDestination::remove); + transportProvider.close(); + return null; + } else { + return transportProvider; + } + }); } @Override @@ -80,17 +175,44 @@ public void initializeDestinationForDatastream(Datastream datastream, String des datastream.setDestination(new DatastreamDestination()); } - if (!datastream.getMetadata().containsKey("dataset")) { - throw new DatastreamValidationException("Metadata dataset is not set in the datastream definition."); + final DatastreamSource source = datastream.getSource(); + final DatastreamDestination destination = datastream.getDestination(); + + // Skip the destination partition validation for datastreams that have connector-managed destinations + // (i.e. mirroring connectors) + if (!DatastreamUtils.isConnectorManagedDestination(datastream) && (!destination.hasPartitions() || destination.getPartitions() <= 0)) { + if (source.hasPartitions()) { + destination.setPartitions(source.getPartitions()); + } else { + logger.warn("Unable to set the number of partitions in a destination, set to default {}", DEFAULT_NUMBER_PARTITIONS); + destination.setPartitions(DEFAULT_NUMBER_PARTITIONS); + } } - String destination = datastream.getMetadata().get("dataset") - + "/" - + (datastream.getMetadata().containsKey("partitionExpirationDays") ? datastream.getMetadata().get("partitionExpirationDays") : "-1") - + "/" - + (datastream.getMetadata().containsKey("tableSuffix") ? datastream.getMetadata().get("tableSuffix") : ""); + if (destination.hasConnectionString() && !isLegacyDatastreamDestinationConnectionString(destination.getConnectionString())) { + try { + BigqueryDatastreamDestination.parse(destination.getConnectionString()); + } catch (IllegalArgumentException e) { + throw new DatastreamValidationException("Bigquery datastream destination is malformed", e); + } + } else { + logger.warn("Updating connection string on legacy datastream with name: {}", datastream.getName()); + updateConnectionStringOnLegacyDatastream(datastream, destinationName); + } + } + + private void updateConnectionStringOnLegacyDatastream(final Datastream datastream, final String destinationName) throws DatastreamValidationException { + final String dataset = datastream.getMetadata().get(METADATA_DATASET_KEY); + if (StringUtils.isBlank(dataset)) { + throw new DatastreamValidationException("Metadata dataset is not set in the datastream definition."); + } - datastream.getDestination().setConnectionString(destination); + final BigqueryDatastreamDestination datastreamDestination = new BigqueryDatastreamDestination( + _legacyDefaultProjectId, + dataset, + destinationName + datastream.getMetadata().getOrDefault(METADATA_TABLE_SUFFIX_KEY, "") + ); + datastream.getDestination().setConnectionString(datastreamDestination.toString()); } @Override @@ -105,4 +227,85 @@ public void dropDestination(Datastream datastream) { public Duration getRetention(Datastream datastream) { return Duration.ofSeconds(0); } + + BigqueryDatastreamConfiguration getConfigurationFromDatastream(final Datastream datastream) { + final Map metadata = datastream.getMetadata(); + final String schemaRegistryLocation = metadata.getOrDefault(METADATA_SCHEMA_REGISTRY_LOCATION_KEY, _legacyDefaultSchemaRegistryUrl); + final BigquerySchemaEvolver schemaEvolver = Optional.ofNullable(metadata.get(METADATA_SCHEMA_EVOLUTION_MODE_KEY)).map( + name -> Optional.ofNullable(_bigquerySchemaEvolverMap.get(name)) + .orElseThrow(() -> new IllegalStateException(String.format("schema evolver not found with name: %s", name)) + )).orElse(_defaultSchemaEvolver); + final boolean autoCreateDestinationTable = Optional.ofNullable(metadata.get(METADATA_AUTO_CREATE_TABLE_KEY)) + .map(Boolean::valueOf).orElse(true); + final Long partitionExpirationDays = Optional.ofNullable(metadata.get(METADATA_PARTITION_EXPIRATION_DAYS_KEY)) + .map(Long::valueOf).orElse(null); + + final BigqueryDatastreamDestination destination = BigqueryDatastreamDestination.parse(datastream.getDestination().getConnectionString()); + + final BigqueryDatastreamConfiguration deadLetterTableConfiguration; + if (!Boolean.parseBoolean(metadata.get(METADATA_DISABLE_DEAD_LETTER_TABLE_KEY))) { + final BigqueryDatastreamDestination deadLetterTable = BigqueryDatastreamDestination + .parse(metadata.getOrDefault(METADATA_DEAD_LETTER_TABLE_KEY, destination.toString() + "_exceptions")); + deadLetterTableConfiguration = _bigqueryDatastreamConfigurationFactory.createBigqueryDatastreamConfiguration( + deadLetterTable, + datastream.getName(), + schemaRegistryLocation, + BigquerySchemaEvolverFactory.createBigquerySchemaEvolver(BigquerySchemaEvolverType.dynamic), + true, + null, + null, + null, + null, + null + ); + } else { + deadLetterTableConfiguration = null; + } + + final List labels = Optional.ofNullable(metadata.get(METADATA_LABELS_KEY)) + .map(this::parseLabelsString).orElse(null); + final Integer schemaId = Optional.ofNullable(metadata.get(METADATA_SCHEMA_ID_KEY)) + .map(Integer::valueOf).orElse(null); + final Boolean relaxAvroSchemaValidation = Optional.ofNullable(metadata.get(METADATA_RELAX_AVRO_SCHEMA_VALIDATION)) + .map(Boolean::valueOf).orElse(null); + + return _bigqueryDatastreamConfigurationFactory.createBigqueryDatastreamConfiguration( + destination, + datastream.getName(), + schemaRegistryLocation, + schemaEvolver, + autoCreateDestinationTable, + partitionExpirationDays, + deadLetterTableConfiguration, + labels, + schemaId, + relaxAvroSchemaValidation + ); + } + + private boolean isLegacyDatastreamDestinationConnectionString(final String destinationConnectionString) { + return _legacyDatastreamDestinationConnectionStringPattern.matcher(destinationConnectionString).matches(); + } + + protected List parseLabelsString(final String labelsString) { + // Parse to map first to find overlapping label names + final Map labelsMap = Arrays.stream(labelsString.split(",")) + .filter(StringUtils::isNotBlank) + .map(label -> { + final String[] parts = label.split(":"); + if (parts.length == 0 || parts.length > 2) { + throw new IllegalArgumentException("invalid label: " + label); + } + final String name = parts[0]; + final String value; + if (parts.length == 2) { + value = parts[1]; + } else { + value = ""; + } + return Pair.of(name, value); + }).collect(Collectors.toMap(Pair::getKey, Pair::getValue)); + return labelsMap.entrySet().stream().map(entry -> new BigqueryLabel(entry.getKey(), entry.getValue())).collect(Collectors.toList()); + } + } diff --git a/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/BigqueryTransportProviderAdminFactory.java b/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/BigqueryTransportProviderAdminFactory.java index 520394301..5cc6de0e9 100644 --- a/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/BigqueryTransportProviderAdminFactory.java +++ b/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/BigqueryTransportProviderAdminFactory.java @@ -5,8 +5,30 @@ */ package com.linkedin.datastream.bigquery; +import java.io.FileInputStream; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.util.Arrays; +import java.util.List; +import java.util.Map; import java.util.Properties; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.auth.Credentials; +import com.google.auth.oauth2.GoogleCredentials; +import com.google.cloud.bigquery.BigQuery; +import com.google.cloud.bigquery.BigQueryOptions; + +import com.linkedin.datastream.bigquery.schema.BigquerySchemaEvolver; +import com.linkedin.datastream.bigquery.schema.BigquerySchemaEvolverFactory; +import com.linkedin.datastream.bigquery.schema.BigquerySchemaEvolverType; +import com.linkedin.datastream.common.VerifiableProperties; import com.linkedin.datastream.server.api.transport.TransportProviderAdmin; import com.linkedin.datastream.server.api.transport.TransportProviderAdminFactory; @@ -14,9 +36,91 @@ * Simple Bigquery Transport provider factory that creates one producer for the entire system */ public class BigqueryTransportProviderAdminFactory implements TransportProviderAdminFactory { + + private final Logger log = LoggerFactory.getLogger(BigqueryTransportProviderAdminFactory.class); + + private static final String CONFIG_BATCHBUILDER_QUEUE_SIZE = "batchBuilderQueueSize"; + private static final String CONFIG_BATCHBUILDER_THREAD_COUNT = "batchBuilderThreadCount"; + private static final String CONFIG_MAX_BATCH_SIZE = "maxBatchSize"; + private static final String CONFIG_MAX_BATCH_AGE = "maxBatchAge"; + private static final String CONFIG_MAX_INFLIGHT_COMMITS = "maxInflightCommits"; + + private static final String CONFIG_SCHEMA_REGISTRY_URL = "translator.schemaRegistry.URL"; + + private static final String CONFIG_COMMITTER_DOMAIN_PREFIX = "committer"; + private static final String CONFIG_COMMITTER_THREADS = "threads"; + private static final String CONFIG_COMMITTER_PROJECT_ID = "projectId"; + private static final String CONFIG_COMMITTER_CREDENTIALS_PATH = "credentialsPath"; + @Override public TransportProviderAdmin createTransportProviderAdmin(String transportProviderName, Properties transportProviderProperties) { - return new BigqueryTransportProviderAdmin(transportProviderName, transportProviderProperties); + final VerifiableProperties tpProperties = new VerifiableProperties(transportProviderProperties); + + // Create a map to define datastream destination to configuration mappings and share with multiple components. + // Datastream destination to configuration mappings are initialized when new datastreams are created or assigned for processing. + // TODO: potentially replace the shared map with a datastream configuration registry implementation + final ConcurrentMap datastreamConfigByDestination = new ConcurrentHashMap<>(); + final String committerProjectId; + final BigqueryBatchCommitter committer; + { + final VerifiableProperties committerProperties = new VerifiableProperties(tpProperties.getDomainProperties(CONFIG_COMMITTER_DOMAIN_PREFIX)); + committerProjectId = committerProperties.getString(CONFIG_COMMITTER_PROJECT_ID); + final String committerCredentialsPath = committerProperties.getString(CONFIG_COMMITTER_CREDENTIALS_PATH); + final BigQuery bigQuery = constructBigQueryClient(committerProjectId, committerCredentialsPath); + final int committerThreads = committerProperties.getInt(CONFIG_COMMITTER_THREADS, 1); + committer = new BigqueryBatchCommitter(bigQuery, committerThreads, datastreamConfigByDestination); + } + + final String legacyDefaultSchemaRegistryUrl = tpProperties.getProperty(CONFIG_SCHEMA_REGISTRY_URL); + + final int maxBatchAge = tpProperties.getInt(CONFIG_MAX_BATCH_AGE, 500); + final List batchBuilders; + final Map bigquerySchemaEvolverMap; + final BigquerySchemaEvolver defaultSchemaEvolver; + { + bigquerySchemaEvolverMap = Arrays.stream(BigquerySchemaEvolverType.values()) + .collect(Collectors.toMap(BigquerySchemaEvolverType::getModeName, BigquerySchemaEvolverFactory::createBigquerySchemaEvolver)); + defaultSchemaEvolver = BigquerySchemaEvolverFactory.createBigquerySchemaEvolver(BigquerySchemaEvolverType.dynamic); + + // Adjusted default max batch size to 500 based on Google's recommendation: https://cloud.google.com/bigquery/quotas#streaming_inserts + final int maxBatchSize = tpProperties.getInt(CONFIG_MAX_BATCH_SIZE, 500); + final int maxInFlightCommits = tpProperties.getInt(CONFIG_MAX_INFLIGHT_COMMITS, 1); + final int batchQueueSize = tpProperties.getInt(CONFIG_BATCHBUILDER_QUEUE_SIZE, 1000); + final int batchBuilderCount = tpProperties.getInt(CONFIG_BATCHBUILDER_THREAD_COUNT, 5); + batchBuilders = IntStream.range(0, batchBuilderCount) + .mapToObj(i -> new BatchBuilder( + maxBatchSize, + maxBatchAge, + maxInFlightCommits, + committer, + batchQueueSize, + datastreamConfigByDestination)).collect(Collectors.toList()); + } + + final BigqueryBufferedTransportProvider bufferedTransportProvider = new BigqueryBufferedTransportProvider(transportProviderName, batchBuilders, + committer, maxBatchAge); + return new BigqueryTransportProviderAdmin(bufferedTransportProvider, + defaultSchemaEvolver, datastreamConfigByDestination, bigquerySchemaEvolverMap, new BigqueryTransportProviderFactory(), + new BigqueryDatastreamConfigurationFactory(), + committerProjectId, + legacyDefaultSchemaRegistryUrl); } + + private BigQuery constructBigQueryClient(final String projectId, final String credentialsPath) { + try { + Credentials credentials = GoogleCredentials + .fromStream(new FileInputStream(credentialsPath)); + return BigQueryOptions.newBuilder() + .setProjectId(projectId) + .setCredentials(credentials).build().getService(); + } catch (FileNotFoundException e) { + log.error("Credentials path {} does not exist", credentialsPath); + throw new RuntimeException(e); + } catch (IOException e) { + log.error("Unable to read credentials: {}", credentialsPath); + throw new RuntimeException(e); + } + } + } diff --git a/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/BigqueryTransportProviderFactory.java b/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/BigqueryTransportProviderFactory.java new file mode 100644 index 000000000..1a6c23fb0 --- /dev/null +++ b/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/BigqueryTransportProviderFactory.java @@ -0,0 +1,36 @@ +/* + * Copyright 2020 Wayfair LLC. All rights reserved. + * Licensed under the BSD 2-Clause License. See the LICENSE file in the project root for license information. + * See the NOTICE file in the project root for additional information regarding copyright ownership. + */ + +package com.linkedin.datastream.bigquery; + +import java.util.concurrent.ConcurrentMap; + +import com.linkedin.datastream.serde.Deserializer; + +/** + * Factory for BigqueryTransportProvider. + */ +public class BigqueryTransportProviderFactory { + + /** + * Create a BigqueryTransportProvider. + * @param bufferedTransportProvider a BigqueryBufferedTransportProvider + * @param valueSerializer a Serializer + * @param valueDeserializer a Deserializer + * @param datastreamConfiguration a BigqueryDatastreamConfiguration + * @param destinationConfigs a mapping of BigqueryDatastreamDestination to BigqueryDatastreamConfiguration + * @return the BigqueryTransportProvider + */ + public BigqueryTransportProvider createTransportProvider(final BigqueryBufferedTransportProvider bufferedTransportProvider, + final Serializer valueSerializer, + final Deserializer valueDeserializer, + final BigqueryDatastreamConfiguration datastreamConfiguration, + final ConcurrentMap + destinationConfigs) { + return new BigqueryTransportProvider(bufferedTransportProvider, valueSerializer, valueDeserializer, + datastreamConfiguration, destinationConfigs); + } +} diff --git a/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/KafkaDeserializer.java b/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/KafkaDeserializer.java new file mode 100644 index 000000000..c2a168fcd --- /dev/null +++ b/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/KafkaDeserializer.java @@ -0,0 +1,29 @@ +/** + * Copyright 2020 Wayfair LLC. All rights reserved. + * Licensed under the BSD 2-Clause License. See the LICENSE file in the project root for license information. + * See the NOTICE file in the project root for additional information regarding copyright ownership. + */ +package com.linkedin.datastream.bigquery; + +import com.linkedin.datastream.serde.Deserializer; + +/** + * An implementation of Deserializer for deserializing Kafka objects. + */ +public class KafkaDeserializer implements Deserializer { + + private final org.apache.kafka.common.serialization.Deserializer deserializer; + + /** + * Constructor. + * @param deserializer a org.apache.kafka.common.serialization.Deserializer that deserializes from a byte array to an object + */ + public KafkaDeserializer(final org.apache.kafka.common.serialization.Deserializer deserializer) { + this.deserializer = deserializer; + } + + @Override + public Object deserialize(final byte[] data) { + return deserializer.deserialize(null, data); + } +} diff --git a/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/KafkaSerializer.java b/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/KafkaSerializer.java new file mode 100644 index 000000000..c9714096a --- /dev/null +++ b/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/KafkaSerializer.java @@ -0,0 +1,29 @@ +/** + * Copyright 2020 Wayfair LLC. All rights reserved. + * Licensed under the BSD 2-Clause License. See the LICENSE file in the project root for license information. + * See the NOTICE file in the project root for additional information regarding copyright ownership. + */ +package com.linkedin.datastream.bigquery; + +/** + * An implementation of a Serializer for serializing Kafka objects. + */ +public class KafkaSerializer implements Serializer { + + private final org.apache.kafka.common.serialization.Serializer serializer; + + + /** + * Constructor. + * @param serializer a org.apache.kafka.common.serialization.Serializer that serializes from an object to a byte array + */ + public KafkaSerializer(final org.apache.kafka.common.serialization.Serializer serializer) { + this.serializer = serializer; + } + + @Override + public byte[] serialize(final String topic, final Object data) { + return serializer.serialize(topic, data); + } + +} diff --git a/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/SchemaRegistry.java b/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/SchemaRegistry.java deleted file mode 100644 index 6d9c40c9d..000000000 --- a/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/SchemaRegistry.java +++ /dev/null @@ -1,79 +0,0 @@ -/** - * Copyright 2020 Wayfair LLC. All rights reserved. - * Licensed under the BSD 2-Clause License. See the LICENSE file in the project root for license information. - * See the NOTICE file in the project root for additional information regarding copyright ownership. - */ -package com.linkedin.datastream.bigquery; - -import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; - -import org.apache.avro.Schema; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import io.confluent.kafka.schemaregistry.client.CachedSchemaRegistryClient; -import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient; -import io.confluent.kafka.serializers.KafkaAvroDeserializer; - -import com.linkedin.datastream.common.VerifiableProperties; - -/** - * This class is a wrapper to schema registry client. - */ -public class SchemaRegistry { - private static final Logger LOG = LoggerFactory.getLogger(SchemaRegistry.class); - private final static Map SCHEMAS = new ConcurrentHashMap<>(); - - private static final String CONFIG_SCHEMA_REGISTRY_URL = "URL"; - private static final String CONFIG_SCHEMA_NAME_SUFFIX = "schemaNameSuffix"; - private static final String DEFAULT_CONFLUENT_SCHEMA_NAME_SUFFIX = "-value"; - - private KafkaAvroDeserializer _deserializer; - private String _schemaRegistryURL; - private String _schemaNameSuffix; - private SchemaRegistryClient _schemaRegistryClient; - - /** - * Constructor for SchemaRegistry. - * @param props schema registry client properties. - */ - public SchemaRegistry(VerifiableProperties props) { - this._schemaRegistryURL = props.getString(CONFIG_SCHEMA_REGISTRY_URL); - this._schemaRegistryClient = new CachedSchemaRegistryClient(_schemaRegistryURL, Integer.MAX_VALUE); - this._schemaNameSuffix = props.getString(CONFIG_SCHEMA_NAME_SUFFIX, DEFAULT_CONFLUENT_SCHEMA_NAME_SUFFIX); - this._deserializer = new KafkaAvroDeserializer(_schemaRegistryClient); - } - - /** - * Returns avro schema of a given topic. - * @param topic topic name - * @return avro schema - */ - public Schema getSchemaByTopic(String topic) { - String key = _schemaRegistryURL + "-" + topic; - Schema schema = SCHEMAS.computeIfAbsent(key, (k) -> { - try { - String schemaName = topic + _schemaNameSuffix; - return new Schema.Parser().parse(_schemaRegistryClient.getLatestSchemaMetadata(schemaName).getSchema()); - } catch (Exception e) { - LOG.error("Unable to find schema for {} - {}", key, e); - return null; - } - }); - - if (schema == null) { - throw new IllegalStateException("Avro schema not found for topic " + topic); - } - return schema; - } - - /** - * returns the deserializer which shares the underlying schema registry client - * @return avro deserializer - */ - public KafkaAvroDeserializer getDeserializer() { - return _deserializer; - } -} diff --git a/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/Serializer.java b/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/Serializer.java new file mode 100644 index 000000000..098b847eb --- /dev/null +++ b/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/Serializer.java @@ -0,0 +1,21 @@ +/** + * Copyright 2020 Wayfair LLC. All rights reserved. + * Licensed under the BSD 2-Clause License. See the LICENSE file in the project root for license information. + * See the NOTICE file in the project root for additional information regarding copyright ownership. + */ +package com.linkedin.datastream.bigquery; + +/** + * Serializer interface for Brooklin which is used to convert an object to bytes. + */ +public interface Serializer { + + /** + * Serialize an object for a given topic to a byte array. + * @param topic a String + * @param data an Object to serialize + * @return a byte array + */ + byte[] serialize(final String topic, final Object data); + +} diff --git a/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/TransientStreamingInsertException.java b/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/TransientStreamingInsertException.java new file mode 100644 index 000000000..d287014de --- /dev/null +++ b/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/TransientStreamingInsertException.java @@ -0,0 +1,31 @@ +/** + * Copyright 2020 Wayfair LLC. All rights reserved. + * Licensed under the BSD 2-Clause License. See the LICENSE file in the project root for license information. + * See the NOTICE file in the project root for additional information regarding copyright ownership. + */ +package com.linkedin.datastream.bigquery; + +import com.linkedin.datastream.common.DatastreamTransientException; + +/** + * A DatastreamTransientException that is thrown when there is an error performing a streaming insert into BigQuery. + */ +public class TransientStreamingInsertException extends DatastreamTransientException { + private static final long serialVersionUID = 1; + + /** + * Constructor. + * @param cause a Throwable + */ + public TransientStreamingInsertException(final Throwable cause) { + super(cause); + } + + /** + * Constructor. + * @param message a String + */ + public TransientStreamingInsertException(final String message) { + super(message); + } +} diff --git a/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/schema/BigquerySchemaEvolver.java b/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/schema/BigquerySchemaEvolver.java new file mode 100644 index 000000000..d691453b9 --- /dev/null +++ b/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/schema/BigquerySchemaEvolver.java @@ -0,0 +1,24 @@ +/** + * Copyright 2020 Wayfair LLC. All rights reserved. + * Licensed under the BSD 2-Clause License. See the LICENSE file in the project root for license information. + * See the NOTICE file in the project root for additional information regarding copyright ownership. + */ +package com.linkedin.datastream.bigquery.schema; + +import com.google.cloud.bigquery.Schema; + +/** + * Interface to handle evolving BigQuery schemas. + */ +public interface BigquerySchemaEvolver { + + /** + * Combine a base and new schema to create an evolved schema. + * @param baseSchema the Schema to use as the base + * @param newSchema the Schema to combine with the base + * @return a Schema representing the evolution of the base and new schemas + * @throws IncompatibleSchemaEvolutionException if the schemas could not be evolved in a way that is compatible + */ + Schema evolveSchema(Schema baseSchema, Schema newSchema) throws IncompatibleSchemaEvolutionException; + +} diff --git a/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/schema/BigquerySchemaEvolverFactory.java b/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/schema/BigquerySchemaEvolverFactory.java new file mode 100644 index 000000000..cbf872b88 --- /dev/null +++ b/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/schema/BigquerySchemaEvolverFactory.java @@ -0,0 +1,40 @@ +/** + * Copyright 2020 Wayfair LLC. All rights reserved. + * Licensed under the BSD 2-Clause License. See the LICENSE file in the project root for license information. + * See the NOTICE file in the project root for additional information regarding copyright ownership. + */ +package com.linkedin.datastream.bigquery.schema; + +/** + * A factory to construct BigquerySchemaEvolver instances. + */ +public class BigquerySchemaEvolverFactory { + + private static final BigquerySchemaEvolver DYNAMIC_SCHEMA_EVOLVER = new DynamicBigquerySchemaEvolver(); + private static final BigquerySchemaEvolver FIXED_SCHEMA_EVOLVER = new FixedBigquerySchemaEvolver(); + private static final BigquerySchemaEvolver NOOP_SCHEMA_EVOLVER = new NoopBigquerySchemaEvolver(); + + /** + * Factory method to create a BigquerySchemaEvolver based on the BigquerySchemaEvolverType + * @param schemaEvolverType a BigquerySchemaEvolverType + * @return the BigquerySchemaEvolver + */ + public static BigquerySchemaEvolver createBigquerySchemaEvolver(final BigquerySchemaEvolverType schemaEvolverType) { + final BigquerySchemaEvolver schemaEvolver; + switch (schemaEvolverType) { + case dynamic: + schemaEvolver = DYNAMIC_SCHEMA_EVOLVER; + break; + case fixed: + schemaEvolver = FIXED_SCHEMA_EVOLVER; + break; + case noop: + schemaEvolver = NOOP_SCHEMA_EVOLVER; + break; + default: + throw new IllegalStateException("Unsupported BigquerySchemaEvolverType: " + schemaEvolverType); + } + return schemaEvolver; + } + +} diff --git a/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/schema/BigquerySchemaEvolverType.java b/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/schema/BigquerySchemaEvolverType.java new file mode 100644 index 000000000..06ec6c815 --- /dev/null +++ b/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/schema/BigquerySchemaEvolverType.java @@ -0,0 +1,25 @@ +/** + * Copyright 2020 Wayfair LLC. All rights reserved. + * Licensed under the BSD 2-Clause License. See the LICENSE file in the project root for license information. + * See the NOTICE file in the project root for additional information regarding copyright ownership. + */ +package com.linkedin.datastream.bigquery.schema; + +/** + * An enumeration of BigQuerySchemaEvolver types. + */ +public enum BigquerySchemaEvolverType { + dynamic("DynamicSchema"), + fixed("FixedSchema"), + noop("NoopSchema"); + + private final String _modeName; + + BigquerySchemaEvolverType(final String modeName) { + this._modeName = modeName; + } + + public String getModeName() { + return _modeName; + } +} diff --git a/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/schema/DynamicBigquerySchemaEvolver.java b/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/schema/DynamicBigquerySchemaEvolver.java new file mode 100644 index 000000000..0ea8069f7 --- /dev/null +++ b/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/schema/DynamicBigquerySchemaEvolver.java @@ -0,0 +1,184 @@ +/** + * Copyright 2020 Wayfair LLC. All rights reserved. + * Licensed under the BSD 2-Clause License. See the LICENSE file in the project root for license information. + * See the NOTICE file in the project root for additional information regarding copyright ownership. + */ +package com.linkedin.datastream.bigquery.schema; + +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; + +import org.apache.commons.lang3.Validate; + +import com.google.cloud.bigquery.Field; +import com.google.cloud.bigquery.FieldList; +import com.google.cloud.bigquery.Schema; +import com.google.cloud.bigquery.StandardSQLTypeName; +import com.google.common.collect.ImmutableList; + + +/** + * A dynamic implementation of a BigQuerySchemaEvolver. + */ +public class DynamicBigquerySchemaEvolver implements BigquerySchemaEvolver { + + protected DynamicBigquerySchemaEvolver() { } + + @Override + public Schema evolveSchema(final Schema baseSchema, final Schema newSchema) throws IncompatibleSchemaEvolutionException { + return Schema.of(mergeFields(baseSchema.getFields(), newSchema.getFields())); + } + + FieldList mergeFields(final FieldList baseFields, final FieldList newFields) throws IncompatibleSchemaEvolutionException { + if (baseFields == newFields) { + return newFields; + } else { + final Map newSchemaFieldsByName = newFields.stream() + .collect(Collectors.toMap(Field::getName, field -> field)); + final Set existingFieldsNames = baseFields.stream().map(Field::getName).collect(Collectors.toSet()); + final List evolvedSchemaFields = baseFields.stream() + .map(field -> mergeFields(field, newSchemaFieldsByName.get(field.getName()))) + .collect(Collectors.toList()); + final List addedFields = newFields.stream() + .filter(field -> !existingFieldsNames.contains(field.getName())) + .map(field -> mergeFields(null, field)).collect(Collectors.toList()); + return FieldList.of(ImmutableList.builder().addAll(evolvedSchemaFields).addAll(addedFields).build()); + } + } + + Field makeNullable(final Field field) { + final Field nullableField; + if (field.getMode() == Field.Mode.REPEATED) { + nullableField = field; + } else { + nullableField = field.toBuilder().setMode(Field.Mode.NULLABLE).build(); + } + return nullableField; + } + + Field addedField(final Field field) { + return makeNullable(field); + } + + Field removedField(final Field field) { + return makeNullable(field); + } + + Field mergeFields(final Field baseField, final Field newField) throws IncompatibleSchemaEvolutionException { + final Field mergedField; + if (baseField == null && newField == null) { + throw new IllegalArgumentException("baseField and newField cannot both be null"); + } else if (baseField == null) { + mergedField = addedField(newField); + } else if (newField == null) { + mergedField = removedField(baseField); + } else if (baseField.equals(newField)) { + mergedField = newField; + } else { + Validate.isTrue(baseField.getName().equalsIgnoreCase(newField.getName()), "baseField and newField names do not match"); + final Field.Builder mergedFieldBuilder = baseField.toBuilder(); + if (StandardSQLTypeName.STRUCT.equals(baseField.getType().getStandardType())) { + mergedFieldBuilder.setType(mergeFieldTypes(baseField.getType().getStandardType(), newField.getType().getStandardType()), + mergeFields(baseField.getSubFields(), newField.getSubFields())); + } else { + mergedFieldBuilder.setType(mergeFieldTypes(baseField.getType().getStandardType(), newField.getType().getStandardType())); + } + + final Field.Mode mergedFieldMode = mergeFieldModes( + Optional.ofNullable(baseField.getMode()).orElse(Field.Mode.NULLABLE), + Optional.ofNullable(newField.getMode()).orElse(Field.Mode.NULLABLE) + ); + if (baseField.getMode() != null || mergedFieldMode != Field.Mode.NULLABLE) { + mergedFieldBuilder.setMode(mergedFieldMode); + } + mergedField = mergedFieldBuilder.build(); + } + return mergedField; + } + + StandardSQLTypeName mergeFieldTypes(final StandardSQLTypeName baseType, final StandardSQLTypeName newType) throws IncompatibleSchemaEvolutionException { + Validate.notNull(baseType, "baseType cannot be null"); + Validate.notNull(newType, "newType cannot be null"); + + // Support BigQuery type coercion described in https://cloud.google.com/bigquery/docs/reference/standard-sql/conversion_rules + final StandardSQLTypeName mergedType; + switch (newType) { + case INT64: + switch (baseType) { + case FLOAT64: + case NUMERIC: + case INT64: + mergedType = baseType; + break; + default: + throw new IncompatibleSchemaEvolutionException(String.format("Cannot coerce type %s into %s", newType, baseType)); + } + break; + case NUMERIC: + switch (baseType) { + case FLOAT64: + case NUMERIC: + mergedType = baseType; + break; + default: + throw new IncompatibleSchemaEvolutionException(String.format("Cannot coerce type %s into %s", newType, baseType)); + } + break; + default: + if (newType == baseType) { + mergedType = baseType; + } else { + throw new IncompatibleSchemaEvolutionException(String.format("Cannot coerce type %s into %s", newType, baseType)); + } + break; + } + return mergedType; + } + + Field.Mode mergeFieldModes(final Field.Mode baseMode, final Field.Mode newMode) { + Validate.notNull(baseMode, "baseMode cannot be null"); + Validate.notNull(newMode, "newMode cannot be null"); + final Field.Mode mergedMode; + switch (baseMode) { + case REPEATED: + switch (newMode) { + case REPEATED: + case NULLABLE: + mergedMode = Field.Mode.REPEATED; + break; + default: + throw new IncompatibleSchemaEvolutionException(String.format("Cannot change field mode from %s to %s", baseMode, newMode)); + } + break; + case REQUIRED: + switch (newMode) { + case REQUIRED: + mergedMode = Field.Mode.REQUIRED; + break; + case NULLABLE: + mergedMode = Field.Mode.NULLABLE; + break; + default: + throw new IncompatibleSchemaEvolutionException(String.format("Cannot change field mode from %s to %s", baseMode, newMode)); + } + break; + case NULLABLE: + switch (newMode) { + case NULLABLE: + case REQUIRED: + mergedMode = Field.Mode.NULLABLE; + break; + default: + throw new IncompatibleSchemaEvolutionException(String.format("Cannot change field mode from %s to %s", baseMode, newMode)); + } + break; + default: + throw new IncompatibleSchemaEvolutionException(String.format("Cannot change field mode from %s to %s", baseMode, newMode)); + } + return mergedMode; + } + +} diff --git a/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/schema/FixedBigquerySchemaEvolver.java b/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/schema/FixedBigquerySchemaEvolver.java new file mode 100644 index 000000000..2a27e1501 --- /dev/null +++ b/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/schema/FixedBigquerySchemaEvolver.java @@ -0,0 +1,22 @@ +/** + * Copyright 2020 Wayfair LLC. All rights reserved. + * Licensed under the BSD 2-Clause License. See the LICENSE file in the project root for license information. + * See the NOTICE file in the project root for additional information regarding copyright ownership. + */ +package com.linkedin.datastream.bigquery.schema; + +import com.google.cloud.bigquery.Schema; + +/** + * A Fixed schema implementation of a BigquerySchemaEvolver. + */ +public class FixedBigquerySchemaEvolver implements BigquerySchemaEvolver { + + protected FixedBigquerySchemaEvolver() { } + + @Override + public Schema evolveSchema(final Schema baseSchema, final Schema newSchema) throws IncompatibleSchemaEvolutionException { + return baseSchema; + } + +} diff --git a/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/schema/IncompatibleSchemaEvolutionException.java b/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/schema/IncompatibleSchemaEvolutionException.java new file mode 100644 index 000000000..3e9012887 --- /dev/null +++ b/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/schema/IncompatibleSchemaEvolutionException.java @@ -0,0 +1,25 @@ +/** + * Copyright 2020 Wayfair LLC. All rights reserved. + * Licensed under the BSD 2-Clause License. See the LICENSE file in the project root for license information. + * See the NOTICE file in the project root for additional information regarding copyright ownership. + */ +package com.linkedin.datastream.bigquery.schema; + +import com.linkedin.datastream.common.DatastreamRuntimeException; + +/** + * An exception indicating that a requested schema evolution was incompatible. + */ +public class IncompatibleSchemaEvolutionException extends DatastreamRuntimeException { + + private static final long serialVersionUID = -1; + + /** + * Constructor. + * @param message the message + */ + public IncompatibleSchemaEvolutionException(final String message) { + super(message); + } + +} diff --git a/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/schema/NoopBigquerySchemaEvolver.java b/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/schema/NoopBigquerySchemaEvolver.java new file mode 100644 index 000000000..69fd7de95 --- /dev/null +++ b/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/schema/NoopBigquerySchemaEvolver.java @@ -0,0 +1,22 @@ +/** + * Copyright 2020 Wayfair LLC. All rights reserved. + * Licensed under the BSD 2-Clause License. See the LICENSE file in the project root for license information. + * See the NOTICE file in the project root for additional information regarding copyright ownership. + */ +package com.linkedin.datastream.bigquery.schema; + +import com.google.cloud.bigquery.Schema; + +/** + * A NoOp implementation of a BigquerySchemaEvolver. + */ +public class NoopBigquerySchemaEvolver implements BigquerySchemaEvolver { + + protected NoopBigquerySchemaEvolver() { } + + @Override + public Schema evolveSchema(final Schema baseSchema, final Schema newSchema) throws IncompatibleSchemaEvolutionException { + return baseSchema; + } + +} diff --git a/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/translator/LogicalTypeIdentifier.java b/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/translator/LogicalTypeIdentifier.java index 2fa04cd80..6a80c6880 100644 --- a/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/translator/LogicalTypeIdentifier.java +++ b/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/translator/LogicalTypeIdentifier.java @@ -9,11 +9,14 @@ import java.util.List; import org.apache.avro.Schema; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * This class has methods to identify logical types in avro. */ class LogicalTypeIdentifier { + static Logger _logger = LoggerFactory.getLogger(LogicalTypeIdentifier.class); private static final String DEBEZIUM_LOGICAL_TYPE_PROPERTY = "connect.name"; @@ -48,6 +51,10 @@ class LogicalTypeIdentifier { private static final String AVRO_TIMESTAMPMICROS_NAME = "timestamp-micros"; private static final String AVRO_DECIMAL_NAME = "decimal"; + public final static String BROOKLIN_LOGICAL_TYPE_KEY = "brooklin_type_key"; + public final static String BROOKLIN_ZONED_TIMESTAMP = "brooklin_zoned_timestamp"; + public final static String BROOKLIN_DATETIME = "brooklin_datetime"; + private static final List AVRO_TIME_TYPES = Arrays.asList(AVRO_TIMEMILLIS_NAME, AVRO_TIMEMICROS_NAME); private static final List AVRO_TIMESTAMP_TYPES = Arrays.asList(AVRO_TIMESTAMPMILLIS_NAME, AVRO_TIMESTAMPMICROS_NAME); @@ -79,7 +86,13 @@ static boolean isTimeType(Schema avroSchema) { * @return true if the avro schema is timestamp logical type */ static boolean isTimestampType(Schema avroSchema) { - return avroSchema.getLogicalType() != null && AVRO_TIMESTAMP_TYPES.contains(avroSchema.getLogicalType().getName()); + return isBrooklinZonedTimestamp(avroSchema) || isDebeziumTimestampLogicalType(avroSchema); + } + + private static boolean isDebeziumTimestampLogicalType(Schema avroSchema) { + return avroSchema.getLogicalType() != null && AVRO_TIMESTAMP_TYPES.contains(avroSchema.getLogicalType().getName()) + || (avroSchema.getProp(DEBEZIUM_LOGICAL_TYPE_PROPERTY) != null && + DEBEZIUM_TIMESTAMP_TYPES.contains(avroSchema.getProp(DEBEZIUM_LOGICAL_TYPE_PROPERTY))); } /** @@ -88,6 +101,15 @@ static boolean isTimestampType(Schema avroSchema) { * @return true if the avro schema is datetime logical type */ static boolean isDatetimeType(Schema avroSchema) { + return isBrooklinDataTime(avroSchema) || isDebeziumDateTime(avroSchema); + } + + private static boolean isBrooklinDataTime(Schema avroSchema) { + return (avroSchema.getProp(BROOKLIN_LOGICAL_TYPE_KEY) != null && + avroSchema.getProp(BROOKLIN_LOGICAL_TYPE_KEY).equals(BROOKLIN_DATETIME)); + } + + static boolean isDebeziumDateTime(Schema avroSchema) { return (avroSchema.getProp(DEBEZIUM_LOGICAL_TYPE_PROPERTY) != null && DEBEZIUM_TIMESTAMP_TYPES.contains(avroSchema.getProp(DEBEZIUM_LOGICAL_TYPE_PROPERTY))); } @@ -167,16 +189,31 @@ static boolean isMicroTimestamp(Schema avroSchema) { * @return true if the avro schema is zoned timestamp logical type */ static boolean isZonedTimestamp(Schema avroSchema) { + return isBrooklinZonedTimestamp(avroSchema) || + isDebeziumZonedTimestamp(avroSchema); + } + + static boolean isDebeziumZonedTimestamp(Schema avroSchema) { return (avroSchema.getProp(DEBEZIUM_LOGICAL_TYPE_PROPERTY) != null && avroSchema.getProp(DEBEZIUM_LOGICAL_TYPE_PROPERTY).equals(DEBEZIUM_ZONEDTIMESTAMP_NAME)); } + static boolean isBrooklinZonedTimestamp(Schema avroSchema) { + return (avroSchema.getProp(BROOKLIN_LOGICAL_TYPE_KEY) != null && + avroSchema.getProp(BROOKLIN_LOGICAL_TYPE_KEY).equals(BROOKLIN_ZONED_TIMESTAMP)); + } + /** * checks if avro schema is Datetime type with millisecond precision * @param avroSchema avro schema to inspect * @return true if the avro schema is datetime logical type with millisecond precision */ static boolean isMilliDatetime(Schema avroSchema) { + // All brooklin datetime is milli type for now + return isBrooklinDataTime(avroSchema) || isDebeziumMilliDateTime(avroSchema); + } + + static boolean isDebeziumMilliDateTime(Schema avroSchema) { return (avroSchema.getProp(DEBEZIUM_LOGICAL_TYPE_PROPERTY) != null && avroSchema.getProp(DEBEZIUM_LOGICAL_TYPE_PROPERTY).equals(DEBEZIUM_TIMESTAMP_NAME)); } diff --git a/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/translator/LogicalTypeTranslator.java b/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/translator/LogicalTypeTranslator.java index 318871f04..cb78c8ff1 100644 --- a/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/translator/LogicalTypeTranslator.java +++ b/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/translator/LogicalTypeTranslator.java @@ -32,7 +32,9 @@ class LogicalTypeTranslator { * @return BQ date */ static String translateDateType(long daysSinceEpoch) { - return new SimpleDateFormat(DATE_FORMAT).format(new Date(daysSinceEpoch * 86400000L)); + SimpleDateFormat dateTimeFmt = new SimpleDateFormat(DATE_FORMAT); + dateTimeFmt.setTimeZone(TimeZone.getTimeZone("UTC")); + return dateTimeFmt.format(new Date(daysSinceEpoch * 86400000L)); } /** diff --git a/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/translator/RecordTranslator.java b/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/translator/RecordTranslator.java index a9954daa7..53069a2dc 100644 --- a/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/translator/RecordTranslator.java +++ b/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/translator/RecordTranslator.java @@ -25,6 +25,8 @@ import org.apache.avro.generic.GenericFixed; import org.apache.avro.generic.GenericRecord; import org.apache.avro.util.Utf8; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import com.google.api.client.util.DateTime; import com.google.api.services.bigquery.model.TableRow; @@ -34,6 +36,7 @@ * This class translates given avro record into BQ row object. */ public class RecordTranslator { + private static final Logger _logger = LoggerFactory.getLogger(RecordTranslator.class); /** * BiqQuery restricts to 9 decimal digits of scale @@ -187,11 +190,13 @@ private static Map.Entry translatePrimitiveTypeObject(Object rec if (decimalType.getScale() > BQ_MAX_DECIMAL_SCALE) { scaledBigDecimal = scaledBigDecimal.setScale(BQ_MAX_DECIMAL_SCALE, RoundingMode.HALF_UP); } - result = new AbstractMap.SimpleEntry<>(name, scaledBigDecimal); + // When precision > 19, have to use String otherwise Bigquery will round off the value. + result = new AbstractMap.SimpleEntry<>(name, scaledBigDecimal.toPlainString()); } else { result = new AbstractMap.SimpleEntry<>(name, Base64.getEncoder().encodeToString(((ByteBuffer) record).array())); } + break; default: return result; @@ -372,7 +377,7 @@ private static Map.Entry translateMapTypeObject(Object record, S private static Map.Entry translateArrayTypeObject(Object record, Schema avroSchema, String name) { Map.Entry result; if (avroSchema.getElementType().getType() == Schema.Type.ARRAY) { - throw new IllegalArgumentException("Array of array types are not supported."); + throw new SchemaTranslationException("Array of array types are not supported."); } if (avroSchema.getElementType().getType() == Schema.Type.RECORD) { @@ -423,7 +428,7 @@ private static TableRow translateRecord(GenericData.Record record) { Schema avroSchema = record.getSchema(); if (avroSchema.getType() != Schema.Type.RECORD) { - throw new IllegalArgumentException("Object is not a Avro Record type."); + throw new SchemaTranslationException("Object is not a Avro Record type."); } TableRow fields = new TableRow(); @@ -436,12 +441,7 @@ private static TableRow translateRecord(GenericData.Record record) { fields.put(avroField.name(), translateRecord((GenericData.Record) record.get(avroField.name()))); } } else if (avroField.schema().getType() == Schema.Type.UNION) { - if (record.get(avroField.name()) != null && LogicalTypeIdentifier.isLSN(avroField.name())) { - fields.put(avroField.name(), - LogicalTypeTranslator.translateLSN(String.valueOf(record.get(avroField.name())))); - } else { - fields.put(avroField.name(), translateUnionTypeObject(record.get(avroField.name()), avroField.schema())); - } + fields.put(avroField.name(), translateUnionTypeObject(record.get(avroField.name()), avroField.schema())); } else if (avroField.schema().getType() == Schema.Type.FIXED) { fields.put(avroField.name(), translateFixedTypeObject(record.get(avroField.name()), avroField.schema())); } else if (avroField.schema().getType() == Schema.Type.MAP) { @@ -456,22 +456,40 @@ private static TableRow translateRecord(GenericData.Record record) { } private static TableRow getMetadata() { - return new TableRow().set("ingest_timestamp", new DateTime(new Date(System.currentTimeMillis()))); + return getMetadata(new DateTime(new Date(System.currentTimeMillis()))); + } + + /** + * Get metadata for this record. + * @param ingestTimestamp a DateTime representing the ingestion timestamp + * @return a TableRow containing the metadata + */ + public static TableRow getMetadata(final DateTime ingestTimestamp) { + return new TableRow().set("ingest_timestamp", ingestTimestamp); } /** * translate given avro record into BQ row object. * @param avroRecord avro record - * @param avroSchema avro schema * @return BQ row */ - public static InsertAllRequest.RowToInsert translate(GenericRecord avroRecord, Schema avroSchema) { - if (avroSchema.getType() != org.apache.avro.Schema.Type.RECORD) { - throw new IllegalArgumentException("The root of the record's schema should be a RECORD type."); + public static InsertAllRequest.RowToInsert translate(final GenericRecord avroRecord) { + return translate(avroRecord, getMetadata()); + } + + /** + * Translate the given avro record and metadata into a BQ row object. + * @param avroRecord a GenericRecord + * @param metadata a TableRow containing metadata + * @return BQ row + */ + public static InsertAllRequest.RowToInsert translate(final GenericRecord avroRecord, final TableRow metadata) { + if (avroRecord.getSchema().getType() != org.apache.avro.Schema.Type.RECORD) { + throw new SchemaTranslationException("The root of the record's schema should be a RECORD type."); } TableRow row = translateRecord((GenericData.Record) avroRecord); - row.set("__metadata", getMetadata()); + row.set("__metadata", metadata); return InsertAllRequest.RowToInsert.of(row); } diff --git a/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/translator/SchemaTranslationException.java b/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/translator/SchemaTranslationException.java new file mode 100644 index 000000000..4faaf4178 --- /dev/null +++ b/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/translator/SchemaTranslationException.java @@ -0,0 +1,24 @@ +/** + * Copyright 2020 Wayfair LLC. All rights reserved. + * Licensed under the BSD 2-Clause License. See the LICENSE file in the project root for license information. + * See the NOTICE file in the project root for additional information regarding copyright ownership. + */ +package com.linkedin.datastream.bigquery.translator; + +import com.linkedin.datastream.common.DatastreamRuntimeException; + +/** + * A DatastreamRuntimeException that is thrown when a schema fails to translate. + */ +public class SchemaTranslationException extends DatastreamRuntimeException { + private static final long serialVersionUID = 1; + + /** + * Constructor. + * @param message a String + */ + public SchemaTranslationException(final String message) { + super(message); + } + +} diff --git a/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/translator/SchemaTranslator.java b/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/translator/SchemaTranslator.java index 629c8beb2..758368b24 100644 --- a/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/translator/SchemaTranslator.java +++ b/datastream-bigquery/src/main/java/com/linkedin/datastream/bigquery/translator/SchemaTranslator.java @@ -9,6 +9,8 @@ import java.util.List; import org.apache.avro.LogicalType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import com.google.cloud.bigquery.Field; import com.google.cloud.bigquery.FieldList; @@ -20,6 +22,8 @@ */ public class SchemaTranslator { + static Logger _logger = LoggerFactory.getLogger(SchemaTranslator.class); + private static class FieldTypePair { Field field; StandardSQLTypeName type; @@ -67,7 +71,7 @@ private static FieldTypePair translateNonRecordSchema(org.apache.avro.Schema avr break; case ARRAY: if (avroSchema.getElementType().getType() == org.apache.avro.Schema.Type.ARRAY) { - throw new IllegalArgumentException("Array of array types are not supported."); + throw new SchemaTranslationException("Array of array types are not supported."); } if (avroSchema.getElementType().getType() == org.apache.avro.Schema.Type.RECORD) { @@ -318,7 +322,7 @@ private static FieldTypePair translateNonRecordSchema(org.apache.avro.Schema avr case NULL: return null; default: - throw new IllegalArgumentException("Avro type not recognized."); + throw new SchemaTranslationException("Avro type not recognized."); } if (avroSchema.getDoc() != null) { @@ -345,7 +349,7 @@ private static Field getMetadata() { */ public static Schema translate(org.apache.avro.Schema avroSchema) { if (avroSchema.getType() != org.apache.avro.Schema.Type.RECORD) { - throw new IllegalArgumentException("The root of the record's schema should be a RECORD type."); + throw new SchemaTranslationException("The root of the record's schema should be a RECORD type."); } FieldList fields = translateRecordSchema(avroSchema); diff --git a/datastream-bigquery/src/test/java/com/linkedin/datastream/bigquery/BatchTests.java b/datastream-bigquery/src/test/java/com/linkedin/datastream/bigquery/BatchTests.java new file mode 100644 index 000000000..84f4bf4ca --- /dev/null +++ b/datastream-bigquery/src/test/java/com/linkedin/datastream/bigquery/BatchTests.java @@ -0,0 +1,154 @@ +/** + * Copyright 2020 Wayfair LLC. All rights reserved. + * Licensed under the BSD 2-Clause License. See the LICENSE file in the project root for license information. + * See the NOTICE file in the project root for additional information regarding copyright ownership. + */ +package com.linkedin.datastream.bigquery; + +import com.codahale.metrics.MetricRegistry; +import com.linkedin.datastream.bigquery.schema.BigquerySchemaEvolver; +import com.linkedin.datastream.bigquery.schema.BigquerySchemaEvolverFactory; +import com.linkedin.datastream.bigquery.schema.BigquerySchemaEvolverType; +import com.linkedin.datastream.bigquery.translator.SchemaTranslator; +import com.linkedin.datastream.common.Package; +import com.linkedin.datastream.common.Record; +import com.linkedin.datastream.metrics.DynamicMetricsManager; +import com.linkedin.datastream.serde.Deserializer; +import io.confluent.kafka.schemaregistry.client.MockSchemaRegistryClient; +import io.confluent.kafka.schemaregistry.client.rest.exceptions.RestClientException; +import io.confluent.kafka.serializers.KafkaAvroDeserializer; +import io.confluent.kafka.serializers.KafkaAvroSerializer; +import org.apache.avro.SchemaBuilder; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.GenericRecordBuilder; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +import java.io.IOException; +import java.time.Instant; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; + +public class BatchTests { + + @BeforeClass + public static void beforeClass() { + DynamicMetricsManager.createInstance(new MetricRegistry(), BigqueryBatchCommitterTests.class.getSimpleName()); + } + + private MockSchemaRegistryClient schemaRegistryClient; + private KafkaAvroSerializer serializer; + private BigqueryBatchCommitter committer; + private BigquerySchemaEvolver schemaEvolver; + private Deserializer deserializer; + + @BeforeMethod + void beforeTest() { + schemaRegistryClient = new MockSchemaRegistryClient(); + deserializer = new KafkaDeserializer(new KafkaAvroDeserializer(schemaRegistryClient)); + serializer = new KafkaAvroSerializer(schemaRegistryClient); + committer = mock(BigqueryBatchCommitter.class); + schemaEvolver = BigquerySchemaEvolverFactory.createBigquerySchemaEvolver(BigquerySchemaEvolverType.dynamic); + } + + @Test + public void testInitSchemaOnFirstWrite() throws InterruptedException, IOException, RestClientException { + final org.apache.avro.Schema avroSchema = SchemaBuilder.builder("com.linkedin") + .record("test_message").fields().name("message").type("string").noDefault() + .endRecord(); + + final GenericRecord record = new GenericRecordBuilder(avroSchema) + .set("message", "test") + .build(); + + final String topicName = "testTopic"; + + schemaRegistryClient.register(topicName + "-value", avroSchema); + + final String projectId = "project_name"; + final String datasetName = "dataset_name"; + final String tableName = "table_name"; + final BigqueryDatastreamDestination destination = new BigqueryDatastreamDestination(projectId, datasetName, tableName); + + final Batch batch = new Batch(destination, 10, 10000, 1, deserializer, committer, schemaEvolver, null); + + final Package aPackage = new Package.PackageBuilder() + .setTopic(topicName) + .setDestination(destination.toString()) + .setOffset("0") + .setPartition("0") + .setCheckpoint("test") + .setTimestamp(Instant.now().toEpochMilli()) + .setRecord(new Record(null, serializer.serialize(topicName, record))) + .build(); + + batch.write(aPackage); + + final com.google.cloud.bigquery.Schema bqSchema = SchemaTranslator.translate(avroSchema); + verify(committer).setDestTableSchema(destination, bqSchema); + } + + @Test + public void testEvolveSchemaOnWrite() throws InterruptedException, IOException, RestClientException { + final org.apache.avro.Schema avroSchema = SchemaBuilder.builder("com.linkedin") + .record("test_message").fields().name("message").type("string").noDefault() + .endRecord(); + final BigqueryDatastreamDestination destination = new BigqueryDatastreamDestination("project_name", "dataset_name", "table_name"); + final Batch batch = new Batch(destination, 10, 10000, 1, deserializer, committer, schemaEvolver, null); + { + final GenericRecord record = new GenericRecordBuilder(avroSchema) + .set("message", "test") + .build(); + + final String topicName = "testTopic"; + + schemaRegistryClient.register(topicName + "-value", avroSchema); + + final Package aPackage = new Package.PackageBuilder() + .setTopic(topicName) + .setDestination(destination.toString()) + .setOffset("0") + .setPartition("0") + .setCheckpoint("test") + .setTimestamp(Instant.now().toEpochMilli()) + .setRecord(new Record(null, serializer.serialize(topicName, record))) + .build(); + + batch.write(aPackage); + + final com.google.cloud.bigquery.Schema bqSchema = SchemaTranslator.translate(avroSchema); + verify(committer).setDestTableSchema(destination, bqSchema); + } + final org.apache.avro.Schema newAvroSchema = SchemaBuilder.builder("com.linkedin") + .record("test_message").fields() + .name("message").type("string").noDefault() + .name("new_int").type("int").noDefault() + .endRecord(); + + final GenericRecord record = new GenericRecordBuilder(newAvroSchema) + .set("message", "test") + .set("new_int", 123) + .build(); + + final String topicName = "testTopic"; + + schemaRegistryClient.register(topicName + "-value", newAvroSchema); + + final Package aPackage = new Package.PackageBuilder() + .setTopic(topicName) + .setDestination(destination.toString()) + .setOffset("1") + .setPartition("0") + .setCheckpoint("test") + .setTimestamp(Instant.now().toEpochMilli()) + .setRecord(new Record(null, serializer.serialize(topicName, record))) + .build(); + + batch.write(aPackage); + + final com.google.cloud.bigquery.Schema bqSchema = schemaEvolver.evolveSchema(SchemaTranslator.translate(avroSchema), SchemaTranslator.translate(newAvroSchema)); + verify(committer).setDestTableSchema(destination, bqSchema); + } +} diff --git a/datastream-bigquery/src/test/java/com/linkedin/datastream/bigquery/BigqueryBatchCommitterTests.java b/datastream-bigquery/src/test/java/com/linkedin/datastream/bigquery/BigqueryBatchCommitterTests.java new file mode 100644 index 000000000..adf8bf688 --- /dev/null +++ b/datastream-bigquery/src/test/java/com/linkedin/datastream/bigquery/BigqueryBatchCommitterTests.java @@ -0,0 +1,697 @@ +/** + * Copyright 2020 Wayfair LLC. All rights reserved. + * Licensed under the BSD 2-Clause License. See the LICENSE file in the project root for license information. + * See the NOTICE file in the project root for additional information regarding copyright ownership. + */ +package com.linkedin.datastream.bigquery; + +import java.time.LocalDate; +import java.time.ZoneOffset; +import java.time.format.DateTimeFormatter; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +import org.mockito.ArgumentCaptor; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +import com.codahale.metrics.MetricRegistry; +import com.google.cloud.bigquery.BigQuery; +import com.google.cloud.bigquery.BigQueryError; +import com.google.cloud.bigquery.BigQueryException; +import com.google.cloud.bigquery.Field; +import com.google.cloud.bigquery.InsertAllRequest; +import com.google.cloud.bigquery.InsertAllResponse; +import com.google.cloud.bigquery.Schema; +import com.google.cloud.bigquery.StandardSQLTypeName; +import com.google.cloud.bigquery.StandardTableDefinition; +import com.google.cloud.bigquery.Table; +import com.google.cloud.bigquery.TableDefinition; +import com.google.cloud.bigquery.TableId; +import com.google.cloud.bigquery.TableInfo; +import com.google.cloud.bigquery.TimePartitioning; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; + +import com.linkedin.datastream.bigquery.schema.BigquerySchemaEvolver; +import com.linkedin.datastream.bigquery.schema.BigquerySchemaEvolverFactory; +import com.linkedin.datastream.bigquery.schema.BigquerySchemaEvolverType; +import com.linkedin.datastream.common.DatastreamRecordMetadata; +import com.linkedin.datastream.common.SendCallback; +import com.linkedin.datastream.metrics.DynamicMetricsManager; +import com.linkedin.datastream.serde.Deserializer; +import com.linkedin.datastream.server.api.transport.buffered.CommitCallback; + +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.atLeastOnce; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertTrue; + +public class BigqueryBatchCommitterTests { + + @BeforeClass + public static void beforeClass() { + DynamicMetricsManager.createInstance(new MetricRegistry(), BigqueryBatchCommitterTests.class.getSimpleName()); + } + + private BigQuery bigQuery; + private BigqueryBatchCommitter batchCommitter; + private BigquerySchemaEvolver schemaEvolver; + private Map destinationConfiguraitons; + + @BeforeMethod + void beforeTest() { + bigQuery = mock(BigQuery.class); + schemaEvolver = BigquerySchemaEvolverFactory.createBigquerySchemaEvolver(BigquerySchemaEvolverType.dynamic); + destinationConfiguraitons = new HashMap<>(); + batchCommitter = new BigqueryBatchCommitter(bigQuery, 1, destinationConfiguraitons); + } + + @Test + public void testCreateTableOnCommit() throws InterruptedException { + final TableId tableId = TableId.of("project_name", "dataset_name", "table_name"); + final BigqueryDatastreamDestination destination = new BigqueryDatastreamDestination(tableId.getProject(), tableId.getDataset(), tableId.getTable()); + final List labels = Arrays.asList(BigqueryLabel.of("test"), BigqueryLabel.of("name", "value")); + destinationConfiguraitons.put(destination, BigqueryDatastreamConfiguration.builder(destination, schemaEvolver, true, mock(Deserializer.class), mock(Serializer.class)) + .withLabels(labels).build()); + final Schema schema = Schema.of( + Field.of("string", StandardSQLTypeName.STRING), + Field.of("int", StandardSQLTypeName.INT64) + ); + batchCommitter.setDestTableSchema(destination, schema); + + when(bigQuery.getTable(tableId)).thenReturn(null); + + final CommitCallback commitCallback = mock(CommitCallback.class); + final CountDownLatch latch = new CountDownLatch(1); + doAnswer(invocation -> { + latch.countDown(); + return null; + }).when(commitCallback).commited(); + + final ImmutableList rowsToInsert = ImmutableList.of( + InsertAllRequest.RowToInsert.of(ImmutableMap.of( + "string", "test", + "int", 123 + )) + ); + final TableId insertTableId = TableId.of(tableId.getProject(), tableId.getDataset(), + String.format("%s$%s", tableId.getTable(), LocalDate.now(ZoneOffset.UTC).format(DateTimeFormatter.ofPattern("yyyyMMdd")))); + final InsertAllRequest insertAllRequest = InsertAllRequest.of(insertTableId, rowsToInsert); + final BigQueryException bigqueryException = new BigQueryException(404, "Table not found", new BigQueryError("notFound", null, "Table not found")); + final AtomicBoolean tableCreated = new AtomicBoolean(false); + when(bigQuery.insertAll(insertAllRequest)).then(invocation -> { + if (tableCreated.get()) { + return mock(InsertAllResponse.class); + } else { + throw bigqueryException; + } + }); + + final TableDefinition tableDefinition = StandardTableDefinition.newBuilder().setSchema(schema) + .setTimePartitioning(TimePartitioning.of(TimePartitioning.Type.DAY)) + .build(); + final TableInfo tableInfo = TableInfo.newBuilder(tableId, tableDefinition) + .setLabels(labels.stream().collect(Collectors.toMap(BigqueryLabel::getName, BigqueryLabel::getValue))) + .build(); + when(bigQuery.create(tableInfo)).then(invocation -> { + if (!tableCreated.get()) { + tableCreated.set(true); + return mock(Table.class); + } else { + throw new IllegalStateException("Table already created"); + } + }); + + final SendCallback mockedRowCallback = mock(SendCallback.class); + final ImmutableList callbacks = ImmutableList.of(mockedRowCallback); + final ImmutableList metadata = ImmutableList.of( + new DatastreamRecordMetadata("test", "test", 0) + ); + final ImmutableList timestamps = ImmutableList.of(System.currentTimeMillis()); + batchCommitter.commit(rowsToInsert, destination.toString(), callbacks, metadata, timestamps, commitCallback); + + latch.await(1, TimeUnit.SECONDS); + + verify(bigQuery).getTable(tableId); + verify(bigQuery).create(tableInfo); + verify(bigQuery).insertAll(insertAllRequest); + verify(mockedRowCallback).onCompletion(metadata.get(0), null); + } + + @Test + public void testCreateTableFailure() throws InterruptedException { + final TableId tableId = TableId.of("project_name", "dataset_name", "table_name"); + final BigqueryDatastreamDestination destination = new BigqueryDatastreamDestination(tableId.getProject(), tableId.getDataset(), tableId.getTable()); + destinationConfiguraitons.put(destination, BigqueryDatastreamConfiguration.builder(destination, schemaEvolver, true, mock(Deserializer.class), mock(Serializer.class)).build()); + final Schema schema = Schema.of( + Field.of("string", StandardSQLTypeName.STRING), + Field.of("int", StandardSQLTypeName.INT64) + ); + batchCommitter.setDestTableSchema(destination, schema); + + when(bigQuery.getTable(tableId)).thenReturn(null); + + final CommitCallback commitCallback = mock(CommitCallback.class); + final CountDownLatch latch = new CountDownLatch(1); + doAnswer(invocation -> { + latch.countDown(); + return null; + }).when(commitCallback).commited(); + + final ImmutableList rowsToInsert = ImmutableList.of( + InsertAllRequest.RowToInsert.of(ImmutableMap.of( + "string", "test", + "int", 123 + )) + ); + + final TableDefinition tableDefinition = StandardTableDefinition.newBuilder().setSchema(schema) + .setTimePartitioning(TimePartitioning.of(TimePartitioning.Type.DAY)) + .build(); + final TableInfo tableInfo = TableInfo.newBuilder(tableId, tableDefinition).build(); + + final BigQueryException exception = new BigQueryException(400, "Test create table failure"); + when(bigQuery.create(tableInfo)).thenThrow(exception); + final SendCallback sendCallback = mock(SendCallback.class); + final DatastreamRecordMetadata metadata = new DatastreamRecordMetadata("test", "test", 0); + batchCommitter.commit(rowsToInsert, destination.toString(), ImmutableList.of(sendCallback), ImmutableList.of(metadata), + ImmutableList.of(), commitCallback); + latch.await(1, TimeUnit.SECONDS); + + verify(bigQuery, times(2)).getTable(tableId); + verify(bigQuery).insertAll(any(InsertAllRequest.class)); + + final ArgumentCaptor exceptionArgumentCaptor = ArgumentCaptor.forClass(Exception.class); + verify(sendCallback).onCompletion(eq(metadata), exceptionArgumentCaptor.capture()); + final Exception actualException = exceptionArgumentCaptor.getValue(); + assertEquals(actualException, exception); + } + + + @Test + public void testTableNotCreatedWhenAutoCreateIsDisabled() throws InterruptedException { + final TableId tableId = TableId.of("project_name", "dataset_name", "table_name"); + final BigqueryDatastreamDestination destination = new BigqueryDatastreamDestination(tableId.getProject(), tableId.getDataset(), tableId.getTable()); + destinationConfiguraitons.put(destination, BigqueryDatastreamConfiguration.builder(destination, schemaEvolver, false, mock(Deserializer.class), mock(Serializer.class)).build()); + final Schema schema = Schema.of( + Field.of("string", StandardSQLTypeName.STRING), + Field.of("int", StandardSQLTypeName.INT64) + ); + batchCommitter.setDestTableSchema(destination, schema); + + final CommitCallback commitCallback = mock(CommitCallback.class); + final CountDownLatch latch = new CountDownLatch(1); + doAnswer(invocation -> { + latch.countDown(); + return null; + }).when(commitCallback).commited(); + + final ImmutableList rowsToInsert = ImmutableList.of( + InsertAllRequest.RowToInsert.of(ImmutableMap.of( + "string", "test", + "int", 123 + )) + ); + final TableId insertTableId = TableId.of(tableId.getProject(), tableId.getDataset(), + String.format("%s$%s", tableId.getTable(), LocalDate.now(ZoneOffset.UTC).format(DateTimeFormatter.ofPattern("yyyyMMdd")))); + final InsertAllRequest insertAllRequest = InsertAllRequest.of(insertTableId, rowsToInsert); + final BigQueryException bigqueryException = new BigQueryException(404, "Table not found", new BigQueryError("notFound", null, "Table not found")); + when(bigQuery.insertAll(insertAllRequest)).thenThrow(bigqueryException); + + final List capturedRecordMetadata = new LinkedList<>(); + final List capturedExceptions = new LinkedList<>(); + + final SendCallback rowCallback = ((metadata, exception) -> { + capturedRecordMetadata.add(metadata); + Optional.ofNullable(exception).ifPresent(capturedExceptions::add); + }); + final ImmutableList callbacks = ImmutableList.of(rowCallback); + final ImmutableList metadata = ImmutableList.of( + new DatastreamRecordMetadata("test", "test", 0) + ); + final ImmutableList timestamps = ImmutableList.of(System.currentTimeMillis()); + batchCommitter.commit(rowsToInsert, destination.toString(), callbacks, metadata, timestamps, commitCallback); + + latch.await(1, TimeUnit.SECONDS); + + verify(bigQuery, atLeastOnce()).getTable(any(TableId.class)); + verify(bigQuery, never()).create(any(TableInfo.class)); + verify(bigQuery, atLeastOnce()).insertAll(insertAllRequest); + assertEquals(capturedRecordMetadata, metadata); + assertFalse(capturedExceptions.isEmpty()); + capturedExceptions.forEach(e -> { + assertTrue(e instanceof TransientStreamingInsertException); + assertEquals(e.getCause(), bigqueryException); + }); + } + + @Test + public void testEvolveTableSchemaOnCommit() throws InterruptedException { + final TableId tableId = TableId.of("project_name", "dataset_name", "table_name"); + final BigqueryDatastreamDestination destination = new BigqueryDatastreamDestination(tableId.getProject(), tableId.getDataset(), tableId.getTable()); + final List labels = Arrays.asList(BigqueryLabel.of("test"), BigqueryLabel.of("name", "value")); + destinationConfiguraitons.put(destination, BigqueryDatastreamConfiguration.builder(destination, schemaEvolver, true, mock(Deserializer.class), mock(Serializer.class)) + .withLabels(labels).build()); + final Schema newSchema = Schema.of( + Field.of("string", StandardSQLTypeName.STRING), + Field.of("int", StandardSQLTypeName.INT64), + Field.of("new_string", StandardSQLTypeName.STRING) + ); + batchCommitter.setDestTableSchema(destination, newSchema); + + final Schema existingSchema = Schema.of( + Field.of("string", StandardSQLTypeName.STRING), + Field.of("int", StandardSQLTypeName.INT64) + ); + final TableDefinition existingTableDefinition = StandardTableDefinition.of(existingSchema); + final Table existingTable = mock(Table.class); + + when(existingTable.getDefinition()).thenReturn(existingTableDefinition); + when(existingTable.getLabels()).thenReturn(Collections.emptyMap()); + + final Schema evolvedSchema = schemaEvolver.evolveSchema(existingSchema, newSchema); + final TableDefinition evolvedTableDefinition = StandardTableDefinition.newBuilder() + .setSchema(evolvedSchema) + .setTimePartitioning(TimePartitioning.of(TimePartitioning.Type.DAY)) + .build(); + final Table.Builder tableBuilder = mock(Table.Builder.class); + when(existingTable.toBuilder()).thenReturn(tableBuilder); + when(tableBuilder.setDefinition(evolvedTableDefinition)).thenReturn(tableBuilder); + final Map labelsMap = labels.stream().collect(Collectors.toMap(BigqueryLabel::getName, BigqueryLabel::getValue)); + when(tableBuilder.setLabels(labelsMap)).thenReturn(tableBuilder); + + final Table evolvedTable = mock(Table.class); + when(tableBuilder.build()).thenReturn(evolvedTable); + + final TableId insertTableId = TableId.of(tableId.getProject(), tableId.getDataset(), + String.format("%s$%s", tableId.getTable(), LocalDate.now(ZoneOffset.UTC).format(DateTimeFormatter.ofPattern("yyyyMMdd")))); + final ImmutableList rowsToInsert = ImmutableList.of( + InsertAllRequest.RowToInsert.of(ImmutableMap.of( + "string", "test value 1", + "int", 123, + "new_string", "test value 2" + )) + ); + final InsertAllRequest insertAllRequest = InsertAllRequest.of(insertTableId, rowsToInsert); + final AtomicBoolean tableUpdated = new AtomicBoolean(false); + when(evolvedTable.update()).then(invocation -> { + if (!tableUpdated.get()) { + tableUpdated.set(true); + return mock(Table.class); + } else { + throw new IllegalStateException("Table already updated"); + } + }); + when(bigQuery.insertAll(insertAllRequest)).then(invocation -> { + if (tableUpdated.get()) { + return mock(InsertAllResponse.class); + } else { + throw new BigQueryException(400, "Missing column", new BigQueryError("invalid", "new_string", "Missing column")); + } + }); + + when(bigQuery.getTable(tableId)).thenReturn(existingTable); + + final CommitCallback commitCallback = mock(CommitCallback.class); + final CountDownLatch latch = new CountDownLatch(1); + doAnswer(invocation -> { + latch.countDown(); + return null; + }).when(commitCallback).commited(); + + + batchCommitter.commit(rowsToInsert, destination.toString(), ImmutableList.of(), ImmutableList.of( + new DatastreamRecordMetadata("test", "test", 0) + ), + ImmutableList.of(), commitCallback); + + latch.await(1, TimeUnit.SECONDS); + + verify(bigQuery).getTable(tableId); + verify(bigQuery, never()).create(any(TableInfo.class)); + verify(tableBuilder).setDefinition(evolvedTableDefinition); + verify(tableBuilder).setLabels(labelsMap); + verify(evolvedTable).update(); + + verify(bigQuery).insertAll(insertAllRequest); + } + + @Test + public void testSchemaEvolvedWhenAutoCreateIsDisabled() throws InterruptedException { + final TableId tableId = TableId.of("project_name", "dataset_name", "table_name"); + final BigqueryDatastreamDestination destination = new BigqueryDatastreamDestination(tableId.getProject(), tableId.getDataset(), tableId.getTable()); + destinationConfiguraitons.put(destination, BigqueryDatastreamConfiguration.builder(destination, schemaEvolver, false, mock(Deserializer.class), mock(Serializer.class)).build()); + + final TableId insertTableId = TableId.of(tableId.getProject(), tableId.getDataset(), + String.format("%s$%s", tableId.getTable(), LocalDate.now(ZoneOffset.UTC).format(DateTimeFormatter.ofPattern("yyyyMMdd")))); + final ImmutableList rowsToInsert = ImmutableList.of( + InsertAllRequest.RowToInsert.of(ImmutableMap.of( + "string", "test value 1", + "int", 123, + "missing", "test value 2" + )) + ); + final InsertAllRequest insertAllRequest = InsertAllRequest.of(insertTableId, rowsToInsert); + + final InsertAllResponse response = mock(InsertAllResponse.class); + final Map> insertErrors = ImmutableMap.of(0L, ImmutableList.of(new BigQueryError("invalid", "missing", "Missing column"))); + when(response.getInsertErrors()).thenReturn(insertErrors); + when(response.hasErrors()).thenReturn(!insertErrors.isEmpty()); + when(bigQuery.insertAll(insertAllRequest)).thenReturn(response); + + final CommitCallback commitCallback = mock(CommitCallback.class); + final CountDownLatch latch = new CountDownLatch(1); + doAnswer(invocation -> { + latch.countDown(); + return null; + }).when(commitCallback).commited(); + + final List capturedRecordMetadata = new LinkedList<>(); + final List capturedExceptions = new LinkedList<>(); + final SendCallback rowCallback = ((metadata, exception) -> { + capturedRecordMetadata.add(metadata); + Optional.ofNullable(exception).ifPresent(capturedExceptions::add); + }); + final ImmutableList callbacks = ImmutableList.of(rowCallback); + final ImmutableList metadata = ImmutableList.of(new DatastreamRecordMetadata("test", "test", 0)); + final ImmutableList timestamps = ImmutableList.of(System.currentTimeMillis()); + batchCommitter.commit(rowsToInsert, destination.toString(), callbacks, metadata, timestamps, commitCallback); + + latch.await(1, TimeUnit.SECONDS); + + verify(bigQuery, atLeastOnce()).getTable(tableId); + verify(bigQuery, never()).create(any(TableInfo.class)); + verify(bigQuery, atLeastOnce()).insertAll(any(InsertAllRequest.class)); + assertEquals(capturedRecordMetadata, metadata); + assertFalse(capturedExceptions.isEmpty()); + capturedExceptions.forEach(e -> { + assertTrue(e instanceof TransientStreamingInsertException); + assertEquals(e.getMessage(), ImmutableList.of(new BigQueryError("invalid", "missing", "Missing column")).toString()); + }); + } + + @Test + public void testEvolveTableSchemaFailure() throws InterruptedException { + final TableId tableId = TableId.of("project_name", "dataset_name", "table_name"); + final BigqueryDatastreamDestination destination = new BigqueryDatastreamDestination(tableId.getProject(), tableId.getDataset(), tableId.getTable()); + destinationConfiguraitons.put(destination, BigqueryDatastreamConfiguration.builder(destination, schemaEvolver, true, mock(Deserializer.class), mock(Serializer.class)).build()); + final Schema newSchema = Schema.of( + Field.of("string", StandardSQLTypeName.STRING), + Field.of("int", StandardSQLTypeName.INT64), + Field.of("new_string", StandardSQLTypeName.STRING) + ); + batchCommitter.setDestTableSchema(destination, newSchema); + + final Schema existingSchema = Schema.of( + Field.of("string", StandardSQLTypeName.STRING), + Field.of("int", StandardSQLTypeName.INT64) + ); + final TableDefinition existingTableDefinition = StandardTableDefinition.of(existingSchema); + final Table existingTable = mock(Table.class); + + when(existingTable.getDefinition()).thenReturn(existingTableDefinition); + + final Table.Builder tableBuilder = mock(Table.Builder.class); + final Schema evolvedSchema = schemaEvolver.evolveSchema(existingSchema, newSchema); + final TableDefinition evolvedTableDefinition = StandardTableDefinition.newBuilder() + .setSchema(evolvedSchema) + .setTimePartitioning(TimePartitioning.of(TimePartitioning.Type.DAY)) + .build(); + when(tableBuilder.setDefinition(evolvedTableDefinition)).thenReturn(tableBuilder); + final Table evolvedTable = mock(Table.class); + when(tableBuilder.build()).thenReturn(evolvedTable); + when(existingTable.toBuilder()).thenReturn(tableBuilder); + + when(bigQuery.getTable(tableId)).thenReturn(existingTable); + + final CommitCallback commitCallback = mock(CommitCallback.class); + final CountDownLatch latch = new CountDownLatch(1); + doAnswer(invocation -> { + latch.countDown(); + return null; + }).when(commitCallback).commited(); + + final ImmutableList rowsToInsert = ImmutableList.of( + InsertAllRequest.RowToInsert.of(ImmutableMap.of( + "string", "test", + "int", 123 + )) + ); + + final BigQueryException exception = new BigQueryException(400, "Test update table failure"); + when(evolvedTable.update()).thenThrow(exception); + + final SendCallback sendCallback = mock(SendCallback.class); + final DatastreamRecordMetadata metadata = new DatastreamRecordMetadata("test", "test", 0); + + batchCommitter.commit(rowsToInsert, destination.toString(), ImmutableList.of(sendCallback), ImmutableList.of(metadata), + ImmutableList.of(), commitCallback); + + latch.await(1, TimeUnit.SECONDS); + + verify(bigQuery, times(4)).getTable(tableId); + verify(bigQuery, never()).create(any(TableInfo.class)); + verify(tableBuilder, times(2)).setDefinition(evolvedTableDefinition); + verify(bigQuery).insertAll(any(InsertAllRequest.class)); + + final ArgumentCaptor exceptionArgumentCaptor = ArgumentCaptor.forClass(Exception.class); + verify(sendCallback).onCompletion(eq(metadata), exceptionArgumentCaptor.capture()); + final Exception actualException = exceptionArgumentCaptor.getValue(); + assertEquals(actualException, exception); + } + + + @Test + public void testEvolveTableSchemaConcurrencyFailure() throws InterruptedException { + final TableId tableId = TableId.of("project_name", "dataset_name", "table_name"); + final BigqueryDatastreamDestination destination = new BigqueryDatastreamDestination(tableId.getProject(), tableId.getDataset(), tableId.getTable()); + destinationConfiguraitons.put(destination, BigqueryDatastreamConfiguration.builder(destination, schemaEvolver, true, mock(Deserializer.class), mock(Serializer.class)).build()); + final Schema newSchema = Schema.of( + Field.of("string", StandardSQLTypeName.STRING), + Field.of("int", StandardSQLTypeName.INT64), + Field.of("new_string", StandardSQLTypeName.STRING) + ); + batchCommitter.setDestTableSchema(destination, newSchema); + + final Schema existingSchema = Schema.of( + Field.of("string", StandardSQLTypeName.STRING), + Field.of("int", StandardSQLTypeName.INT64) + ); + final TableDefinition existingTableDefinition = StandardTableDefinition.of(existingSchema); + final Table existingTable = mock(Table.class); + + when(existingTable.getDefinition()).thenReturn(existingTableDefinition); + + final Schema newBaseSchema = Schema.of( + Field.of("string", StandardSQLTypeName.STRING), + Field.of("int", StandardSQLTypeName.INT64), + Field.newBuilder("new_int", StandardSQLTypeName.INT64).setMode(Field.Mode.NULLABLE).build() + ); + final TableDefinition newBaseTableDefinition = StandardTableDefinition.of(newBaseSchema); + final Table newBaseTable = mock(Table.class); + + when(newBaseTable.getDefinition()).thenReturn(newBaseTableDefinition); + + when(bigQuery.getTable(tableId)).thenReturn(existingTable).thenReturn(newBaseTable); + + final Table.Builder tableBuilder1 = mock(Table.Builder.class); + final Schema evolvedSchema1 = schemaEvolver.evolveSchema(existingSchema, newSchema); + final TableDefinition evolvedTableDefinition1 = StandardTableDefinition.newBuilder() + .setSchema(evolvedSchema1) + .setTimePartitioning(TimePartitioning.of(TimePartitioning.Type.DAY)) + .build(); + when(tableBuilder1.setDefinition(evolvedTableDefinition1)).thenReturn(tableBuilder1); + final Table evolvedTable1 = mock(Table.class); + when(tableBuilder1.build()).thenReturn(evolvedTable1); + when(existingTable.toBuilder()).thenReturn(tableBuilder1); + + final BigQueryException exception = new BigQueryException(400, "Test update table failure"); + when(evolvedTable1.update()).thenThrow(exception); + + final Table.Builder tableBuilder2 = mock(Table.Builder.class); + final Schema evolvedSchema2 = schemaEvolver.evolveSchema(newBaseSchema, newSchema); + final TableDefinition evolvedTableDefinition2 = StandardTableDefinition.newBuilder() + .setSchema(evolvedSchema2) + .setTimePartitioning(TimePartitioning.of(TimePartitioning.Type.DAY)) + .build(); + when(tableBuilder2.setDefinition(evolvedTableDefinition2)).thenReturn(tableBuilder2); + final Table evolvedTable2 = mock(Table.class); + when(tableBuilder2.build()).thenReturn(evolvedTable2); + when(newBaseTable.toBuilder()).thenReturn(tableBuilder2); + + + final CommitCallback commitCallback = mock(CommitCallback.class); + final CountDownLatch latch = new CountDownLatch(1); + doAnswer(invocation -> { + latch.countDown(); + return null; + }).when(commitCallback).commited(); + + final ImmutableList rowsToInsert = ImmutableList.of( + InsertAllRequest.RowToInsert.of(ImmutableMap.of( + "string", "test", + "int", 123 + )) + ); + final TableId insertTableId = TableId.of(tableId.getProject(), tableId.getDataset(), + String.format("%s$%s", tableId.getTable(), LocalDate.now(ZoneOffset.UTC).format(DateTimeFormatter.ofPattern("yyyyMMdd")))); + final InsertAllRequest insertAllRequest = InsertAllRequest.of(insertTableId, rowsToInsert); + + final AtomicBoolean tableUpdated = new AtomicBoolean(false); + when(evolvedTable2.update()).then(invocation -> { + if (!tableUpdated.get()) { + tableUpdated.set(true); + return mock(Table.class); + } else { + throw new IllegalStateException("Table already updated"); + } + }); + + when(bigQuery.insertAll(insertAllRequest)).then(invocation -> { + if (tableUpdated.get()) { + return mock(InsertAllResponse.class); + } else { + throw new BigQueryException(400, "Missing column", new BigQueryError("invalid", "new_string", "Missing column")); + } + }); + + batchCommitter.commit(rowsToInsert, destination.toString(), ImmutableList.of(), ImmutableList.of( + new DatastreamRecordMetadata("test", "test", 0) + ), + ImmutableList.of(), commitCallback); + + latch.await(1, TimeUnit.SECONDS); + + verify(bigQuery, times(2)).getTable(tableId); + verify(bigQuery, never()).create(any(TableInfo.class)); + + verify(tableBuilder1).setDefinition(evolvedTableDefinition1); + verify(evolvedTable1).update(); + + verify(tableBuilder2).setDefinition(evolvedTableDefinition2); + verify(evolvedTable2).update(); + + verify(bigQuery).insertAll(insertAllRequest); + } + + @Test + public void testPayloadSizeLimitRetry() { + final TableId tableId = TableId.of("dataset", "table"); + final List batch = IntStream.range(0, 10) + .mapToObj(i -> InsertAllRequest.RowToInsert.of(ImmutableMap.of("f1", "test" + i))) + .collect(Collectors.toList()); + final InsertAllResponse response = mock(InsertAllResponse.class); + when(response.hasErrors()).thenReturn(false); + when(response.getInsertErrors()).thenReturn(Collections.emptyMap()); + when(bigQuery.insertAll(any(InsertAllRequest.class))) + .thenThrow(new BigQueryException(400, "Request payload size exceeds the limit")) + .thenReturn(response); + final Map insertErrors = batchCommitter.insertRowsAndMapErrorsWithRetry(tableId, batch); + verify(bigQuery).insertAll(InsertAllRequest.of(tableId, batch)); + verify(bigQuery).insertAll(InsertAllRequest.of(tableId, batch.subList(0, batch.size() / 2))); + verify(bigQuery).insertAll(InsertAllRequest.of(tableId, batch.subList(batch.size() / 2, batch.size()))); + assertTrue(insertErrors.isEmpty()); + } + + @Test + public void testBatchSizeLimitRetry() { + final TableId tableId = TableId.of("dataset", "table"); + final List batch = IntStream.range(0, 10) + .mapToObj(i -> InsertAllRequest.RowToInsert.of(ImmutableMap.of("f1", "test" + i))) + .collect(Collectors.toList()); + final InsertAllResponse response = mock(InsertAllResponse.class); + when(response.hasErrors()).thenReturn(false); + when(response.getInsertErrors()).thenReturn(Collections.emptyMap()); + when(bigQuery.insertAll(any(InsertAllRequest.class))) + .thenThrow(new BigQueryException(400, "too many rows present in the request")) + .thenReturn(response); + final Map insertErrors = batchCommitter.insertRowsAndMapErrorsWithRetry(tableId, batch); + verify(bigQuery).insertAll(InsertAllRequest.of(tableId, batch)); + verify(bigQuery).insertAll(InsertAllRequest.of(tableId, batch.subList(0, batch.size() / 2))); + verify(bigQuery).insertAll(InsertAllRequest.of(tableId, batch.subList(batch.size() / 2, batch.size()))); + assertTrue(insertErrors.isEmpty()); + } + + @Test + public void testBatchSizeLimitRetryRemappedRowErrorIndex() { + final TableId tableId = TableId.of("dataset", "table"); + final List batch = IntStream.range(0, 10) + .mapToObj(i -> InsertAllRequest.RowToInsert.of(ImmutableMap.of("f1", "test" + i))) + .collect(Collectors.toList()); + final InsertAllResponse response = mock(InsertAllResponse.class); + when(response.hasErrors()).thenReturn(false); + when(response.getInsertErrors()).thenReturn(Collections.emptyMap()); + final BigQueryException batchException = new BigQueryException(400, "too many rows present in the request"); + when(bigQuery.insertAll(any(InsertAllRequest.class))) + .thenThrow(batchException, batchException) + .thenThrow(new RuntimeException("non-batch size limit exception")); + final Map insertErrors = batchCommitter.insertRowsAndMapErrorsWithRetry(tableId, batch); + verify(bigQuery).insertAll(InsertAllRequest.of(tableId, batch)); + verify(bigQuery).insertAll(InsertAllRequest.of(tableId, batch.subList(0, batch.size() / 2))); + verify(bigQuery).insertAll(InsertAllRequest.of(tableId, batch.subList(0, batch.size() / 4))); + verify(bigQuery).insertAll(InsertAllRequest.of(tableId, batch.subList(batch.size() / 4, batch.size() / 2))); + verify(bigQuery).insertAll(InsertAllRequest.of(tableId, batch.subList(batch.size() / 2, batch.size()))); + assertEquals(insertErrors.keySet(), IntStream.range(0, batch.size()).boxed().collect(Collectors.toSet())); + } + + @Test + public void testBatchSizeLimitRetryMultipleTimes() { + final TableId tableId = TableId.of("dataset", "table"); + final List batch = IntStream.range(0, 10) + .mapToObj(i -> InsertAllRequest.RowToInsert.of(ImmutableMap.of("f1", "test" + i))) + .collect(Collectors.toList()); + final InsertAllResponse response = mock(InsertAllResponse.class); + when(response.hasErrors()).thenReturn(false); + when(response.getInsertErrors()).thenReturn(Collections.emptyMap()); + final Exception exception = new BigQueryException(400, "too many rows present in the request"); + when(bigQuery.insertAll(any(InsertAllRequest.class))) + .thenThrow(exception, exception) + .thenReturn(response); + final Map insertErrors = batchCommitter.insertRowsAndMapErrorsWithRetry(tableId, batch); + verify(bigQuery).insertAll(InsertAllRequest.of(tableId, batch)); + verify(bigQuery).insertAll(InsertAllRequest.of(tableId, batch.subList(0, batch.size() / 2))); + verify(bigQuery).insertAll(InsertAllRequest.of(tableId, batch.subList(0, batch.size() / 4))); + verify(bigQuery).insertAll(InsertAllRequest.of(tableId, batch.subList(batch.size() / 4, batch.size() / 2))); + verify(bigQuery).insertAll(InsertAllRequest.of(tableId, batch.subList(batch.size() / 2, batch.size()))); + assertTrue(insertErrors.isEmpty()); + } + + @Test + public void testBatchSizeLimitRetryInfiniteRecursionExit() { + final TableId tableId = TableId.of("dataset", "table"); + final List batch = IntStream.range(0, 10) + .mapToObj(i -> InsertAllRequest.RowToInsert.of(ImmutableMap.of("f1", "test" + i))) + .collect(Collectors.toList()); + final InsertAllResponse response = mock(InsertAllResponse.class); + when(response.hasErrors()).thenReturn(false); + when(response.getInsertErrors()).thenReturn(Collections.emptyMap()); + final Exception exception = new BigQueryException(400, "too many rows present in the request"); + when(bigQuery.insertAll(any(InsertAllRequest.class))) + .thenThrow(exception); + final Map insertErrors = batchCommitter.insertRowsAndMapErrorsWithRetry(tableId, batch); + verify(bigQuery, times(19)).insertAll(any(InsertAllRequest.class)); + assertEquals(insertErrors.keySet(), IntStream.range(0, batch.size()).boxed().collect(Collectors.toSet())); + insertErrors.values().stream().map(Exception::getCause).forEach(e -> assertEquals(e, exception)); + } + +} diff --git a/datastream-bigquery/src/test/java/com/linkedin/datastream/bigquery/BigqueryCachedSchemaRegistryClientTests.java b/datastream-bigquery/src/test/java/com/linkedin/datastream/bigquery/BigqueryCachedSchemaRegistryClientTests.java new file mode 100644 index 000000000..7ecfbad21 --- /dev/null +++ b/datastream-bigquery/src/test/java/com/linkedin/datastream/bigquery/BigqueryCachedSchemaRegistryClientTests.java @@ -0,0 +1,161 @@ +/* + * Copyright 2020 Wayfair LLC. All rights reserved. + * Licensed under the BSD 2-Clause License. See the LICENSE file in the project root for license information. + * See the NOTICE file in the project root for additional information regarding copyright ownership. + */ + +package com.linkedin.datastream.bigquery; + +import java.io.IOException; +import java.io.InputStream; +import java.util.Map; + +import org.apache.avro.AvroTypeException; +import org.apache.avro.Schema; +import org.apache.avro.SchemaParseException; +import org.testng.annotations.Test; +import org.testng.reporters.Files; + +import com.google.common.collect.ImmutableMap; +import io.confluent.kafka.schemaregistry.client.rest.RestService; +import io.confluent.kafka.schemaregistry.client.rest.entities.SchemaString; +import io.confluent.kafka.schemaregistry.client.rest.exceptions.RestClientException; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertTrue; +import static org.testng.FileAssert.fail; + +public class BigqueryCachedSchemaRegistryClientTests { + + @Test + public void testParseSchemasWithDefaultConfigs() throws IOException, RestClientException { + final RestService restService = mock(RestService.class); + final BigqueryCachedSchemaRegistryClient srClient = new BigqueryCachedSchemaRegistryClient(restService, 2); + + final int invalidDefaultsSchemaId = 1; + String invalidDefaultsSchemaString; + try (final InputStream avroFileInputStream = getClass().getClassLoader().getResourceAsStream("invalid_defaults_schema.avsc")) { + invalidDefaultsSchemaString = Files.readFile(avroFileInputStream); + } + when(restService.getId(invalidDefaultsSchemaId)).thenReturn(new SchemaString(invalidDefaultsSchemaString)); + + final int invalidFieldNamesSchemaId = 2; + String invalidFieldNamesSchemaString; + try (final InputStream avroFileInputStream = getClass().getClassLoader().getResourceAsStream("invalid_field_name_schema.avsc")) { + invalidFieldNamesSchemaString = Files.readFile(avroFileInputStream); + } + when(restService.getId(invalidFieldNamesSchemaId)).thenReturn(new SchemaString(invalidFieldNamesSchemaString)); + + try { + srClient.getById(invalidDefaultsSchemaId); + fail(); + } catch (final AvroTypeException e) { + assertTrue(e.getMessage().startsWith("Invalid default for field ")); + } + try { + srClient.getByID(invalidFieldNamesSchemaId); + fail(); + } catch (final SchemaParseException e) { + assertTrue(e.getMessage().startsWith("Illegal initial character: ")); + } + } + + @Test + public void testParseSchemasWithoutValidation() throws IOException, RestClientException { + final RestService restService = mock(RestService.class); + final BigqueryCachedSchemaRegistryClient srClient = new BigqueryCachedSchemaRegistryClient(restService, 2, + ImmutableMap.of( + BigquerySchemaRegistryClientConfig.SCHEMA_REGISTRY_PARSER_VALIDATE_DEFAULTS, false, + BigquerySchemaRegistryClientConfig.SCHEMA_REGISTRY_PARSER_VALIDATE_FIELD_NAMES, false + )); + final Schema.Parser parser = new Schema.Parser().setValidate(false).setValidateDefaults(false); + + final int invalidDefaultsSchemaId = 1; + String invalidDefaultsSchemaString; + try (final InputStream avroFileInputStream = getClass().getClassLoader().getResourceAsStream("invalid_defaults_schema.avsc")) { + invalidDefaultsSchemaString = Files.readFile(avroFileInputStream); + } + when(restService.getId(invalidDefaultsSchemaId)).thenReturn(new SchemaString(invalidDefaultsSchemaString)); + + final int invalidFieldNamesSchemaId = 2; + String invalidFieldNamesSchemaString; + try (final InputStream avroFileInputStream = getClass().getClassLoader().getResourceAsStream("invalid_field_name_schema.avsc")) { + invalidFieldNamesSchemaString = Files.readFile(avroFileInputStream); + } + when(restService.getId(invalidFieldNamesSchemaId)).thenReturn(new SchemaString(invalidFieldNamesSchemaString)); + + assertEquals(srClient.getById(invalidDefaultsSchemaId), parser.parse(invalidDefaultsSchemaString)); + assertEquals(srClient.getByID(invalidFieldNamesSchemaId), parser.parse(invalidFieldNamesSchemaString)); + } + + @Test + public void testParseSchemaWithInvalidDefaultsValidationDisabled() throws IOException, RestClientException { + final RestService restService = mock(RestService.class); + final BigqueryCachedSchemaRegistryClient srClient = new BigqueryCachedSchemaRegistryClient(restService, 1, + ImmutableMap.of( + BigquerySchemaRegistryClientConfig.SCHEMA_REGISTRY_PARSER_VALIDATE_DEFAULTS, false + )); + final int schemaId = 12345; + String schemaString; + try (final InputStream avroFileInputStream = getClass().getClassLoader().getResourceAsStream("invalid_defaults_schema.avsc")) { + schemaString = Files.readFile(avroFileInputStream); + } + final Schema schema = new Schema.Parser().setValidateDefaults(false).parse(schemaString); + when(restService.getId(schemaId)).thenReturn(new SchemaString(schemaString)); + final Schema fetchedSchema = srClient.getById(schemaId); + assertEquals(fetchedSchema, schema); + } + + @Test(expectedExceptions = AvroTypeException.class, expectedExceptionsMessageRegExp = "Invalid default for field .+") + public void testParseSchemaWithInvalidDefaultsValidationEnabled() throws IOException, RestClientException { + final RestService restService = mock(RestService.class); + final Map configs = ImmutableMap.of( + BigquerySchemaRegistryClientConfig.SCHEMA_REGISTRY_PARSER_VALIDATE_DEFAULTS, true + ); + final BigqueryCachedSchemaRegistryClient srClient = new BigqueryCachedSchemaRegistryClient(restService, 1, configs); + final int schemaId = 12345; + String schemaString; + try (final InputStream avroFileInputStream = getClass().getClassLoader().getResourceAsStream("invalid_defaults_schema.avsc")) { + schemaString = Files.readFile(avroFileInputStream); + } + when(restService.getId(schemaId)).thenReturn(new SchemaString(schemaString)); + srClient.getById(schemaId); + } + + @Test + public void testParseSchemaWithInvalidFieldNamesValidationDisabled() throws IOException, RestClientException { + final RestService restService = mock(RestService.class); + final BigqueryCachedSchemaRegistryClient srClient = new BigqueryCachedSchemaRegistryClient(restService, 1, + ImmutableMap.of( + BigquerySchemaRegistryClientConfig.SCHEMA_REGISTRY_PARSER_VALIDATE_FIELD_NAMES, false + )); + final int schemaId = 12345; + String schemaString; + try (final InputStream avroFileInputStream = getClass().getClassLoader().getResourceAsStream("invalid_field_name_schema.avsc")) { + schemaString = Files.readFile(avroFileInputStream); + } + final Schema schema = new Schema.Parser().setValidate(false).parse(schemaString); + when(restService.getId(schemaId)).thenReturn(new SchemaString(schemaString)); + final Schema fetchedSchema = srClient.getById(schemaId); + assertEquals(fetchedSchema, schema); + } + + @Test(expectedExceptions = SchemaParseException.class, expectedExceptionsMessageRegExp = "Illegal initial character: .+") + public void testParseSchemaWithInvalidFieldNamesValidationEnabled() throws IOException, RestClientException { + final RestService restService = mock(RestService.class); + final Map configs = ImmutableMap.of( + BigquerySchemaRegistryClientConfig.SCHEMA_REGISTRY_PARSER_VALIDATE_FIELD_NAMES, true + ); + final BigqueryCachedSchemaRegistryClient srClient = new BigqueryCachedSchemaRegistryClient(restService, 1, configs); + final int schemaId = 12345; + String schemaString; + try (final InputStream avroFileInputStream = getClass().getClassLoader().getResourceAsStream("invalid_field_name_schema.avsc")) { + schemaString = Files.readFile(avroFileInputStream); + } + when(restService.getId(schemaId)).thenReturn(new SchemaString(schemaString)); + srClient.getById(schemaId); + } + +} diff --git a/datastream-bigquery/src/test/java/com/linkedin/datastream/bigquery/BigqueryDatastreamConfigurationTests.java b/datastream-bigquery/src/test/java/com/linkedin/datastream/bigquery/BigqueryDatastreamConfigurationTests.java new file mode 100644 index 000000000..f993297ce --- /dev/null +++ b/datastream-bigquery/src/test/java/com/linkedin/datastream/bigquery/BigqueryDatastreamConfigurationTests.java @@ -0,0 +1,90 @@ +/* + * Copyright 2020 Wayfair LLC. All rights reserved. + * Licensed under the BSD 2-Clause License. See the LICENSE file in the project root for license information. + * See the NOTICE file in the project root for additional information regarding copyright ownership. + */ + +package com.linkedin.datastream.bigquery; + +import java.util.List; + +import org.apache.avro.Schema; +import org.testng.annotations.Test; + +import com.google.common.collect.ImmutableList; + +import com.linkedin.datastream.bigquery.schema.BigquerySchemaEvolver; +import com.linkedin.datastream.serde.Deserializer; + + +import static org.mockito.Mockito.mock; +import static org.testng.Assert.assertEquals; + +public class BigqueryDatastreamConfigurationTests { + + @Test + public void testEquals() { + final BigqueryDatastreamDestination destination = new BigqueryDatastreamDestination("project", "dataset", "destination"); + final BigquerySchemaEvolver schemaEvolver = mock(BigquerySchemaEvolver.class); + final boolean createDestinationTable = true; + final Deserializer deserializer = mock(Deserializer.class); + final Serializer serializer = mock(Serializer.class); + final long partitionExpirationDays = 5; + final List labels = ImmutableList.of(BigqueryLabel.of("test")); + final Schema fixedSchema = mock(Schema.class); + final BigqueryDatastreamDestination deadLetterTableDestination = new BigqueryDatastreamDestination("project", "dataset", "dlq"); + final BigqueryDatastreamConfiguration deadLetterTableConfig = BigqueryDatastreamConfiguration + .builder(deadLetterTableDestination, schemaEvolver, createDestinationTable, deserializer, serializer) + .build(); + final BigqueryDatastreamConfiguration config = BigqueryDatastreamConfiguration.builder(destination, schemaEvolver, createDestinationTable, deserializer, + serializer) + .withPartitionExpirationDays(partitionExpirationDays) + .withLabels(labels) + .withFixedSchema(fixedSchema) + .withDeadLetterTableConfiguration(deadLetterTableConfig) + .build(); + + assertEquals(BigqueryDatastreamConfiguration.builder(destination, schemaEvolver, createDestinationTable, deserializer, + serializer) + .withPartitionExpirationDays(partitionExpirationDays) + .withLabels(labels) + .withFixedSchema(fixedSchema) + .withDeadLetterTableConfiguration(BigqueryDatastreamConfiguration + .builder(deadLetterTableDestination, schemaEvolver, createDestinationTable, deserializer, serializer) + .build()) + .build(), config); + } + + @Test + public void testHashCode() { + final BigqueryDatastreamDestination destination = new BigqueryDatastreamDestination("project", "dataset", "destination"); + final BigquerySchemaEvolver schemaEvolver = mock(BigquerySchemaEvolver.class); + final boolean createDestinationTable = true; + final Deserializer deserializer = mock(Deserializer.class); + final Serializer serializer = mock(Serializer.class); + final long partitionExpirationDays = 5; + final List labels = ImmutableList.of(BigqueryLabel.of("test")); + final Schema fixedSchema = mock(Schema.class); + final BigqueryDatastreamDestination deadLetterTableDestination = new BigqueryDatastreamDestination("project", "dataset", "dlq"); + final BigqueryDatastreamConfiguration deadLetterTableConfig = BigqueryDatastreamConfiguration + .builder(deadLetterTableDestination, schemaEvolver, createDestinationTable, deserializer, serializer) + .build(); + final BigqueryDatastreamConfiguration config = BigqueryDatastreamConfiguration.builder(destination, schemaEvolver, createDestinationTable, deserializer, + serializer) + .withPartitionExpirationDays(partitionExpirationDays) + .withLabels(labels) + .withFixedSchema(fixedSchema) + .withDeadLetterTableConfiguration(deadLetterTableConfig) + .build(); + + assertEquals(BigqueryDatastreamConfiguration.builder(destination, schemaEvolver, createDestinationTable, deserializer, + serializer) + .withPartitionExpirationDays(partitionExpirationDays) + .withLabels(labels) + .withFixedSchema(fixedSchema) + .withDeadLetterTableConfiguration(BigqueryDatastreamConfiguration + .builder(deadLetterTableDestination, schemaEvolver, createDestinationTable, deserializer, serializer) + .build()) + .build().hashCode(), config.hashCode()); + } +} diff --git a/datastream-bigquery/src/test/java/com/linkedin/datastream/bigquery/BigqueryDatastreamDestinationTests.java b/datastream-bigquery/src/test/java/com/linkedin/datastream/bigquery/BigqueryDatastreamDestinationTests.java new file mode 100644 index 000000000..960ad3263 --- /dev/null +++ b/datastream-bigquery/src/test/java/com/linkedin/datastream/bigquery/BigqueryDatastreamDestinationTests.java @@ -0,0 +1,67 @@ +/** + * Copyright 2020 Wayfair LLC. All rights reserved. + * Licensed under the BSD 2-Clause License. See the LICENSE file in the project root for license information. + * See the NOTICE file in the project root for additional information regarding copyright ownership. + */ +package com.linkedin.datastream.bigquery; + +import java.net.URI; +import java.util.Objects; + +import org.testng.annotations.Test; + +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertTrue; + +public class BigqueryDatastreamDestinationTests { + + @Test + public void testToUri() { + final BigqueryDatastreamDestination destination = new BigqueryDatastreamDestination("project_name", "dataset", "*"); + final URI uri = destination.getUri(); + assertEquals(uri.toString(), "brooklin-bigquery://project_name.dataset.*"); + } + + @Test + public void testParse() { + final BigqueryDatastreamDestination destination = new BigqueryDatastreamDestination("project_name", "dataset", "*"); + assertEquals( + BigqueryDatastreamDestination.parse("brooklin-bigquery://project_name.dataset.*"), + destination + ); + } + + @Test + public void testParseDestinationWithPeriod() { + final BigqueryDatastreamDestination destination = new BigqueryDatastreamDestination("project_name", "dataset", "test.topic.with.period"); + assertEquals( + BigqueryDatastreamDestination.parse("brooklin-bigquery://project_name.dataset.test.topic.with.period"), + destination + ); + } + + @Test + public void testEquals() { + final String project = "project"; + final String dataset = "dataset"; + final String destinationName = "destination"; + final BigqueryDatastreamDestination destination = new BigqueryDatastreamDestination(project, dataset, destinationName); + final String url = destination.toString(); + + assertEquals(new BigqueryDatastreamDestination(project, dataset, destinationName), destination); + assertEquals(BigqueryDatastreamDestination.parse(url), destination); + } + + @Test + public void testHashCode() { + final String project = "project"; + final String dataset = "dataset"; + final String destinationName = "destination"; + final BigqueryDatastreamDestination destination = new BigqueryDatastreamDestination(project, dataset, destinationName); + final String url = destination.toString(); + + assertEquals(new BigqueryDatastreamDestination(project, dataset, destinationName).hashCode(), destination.hashCode()); + assertEquals(BigqueryDatastreamDestination.parse(url).hashCode(), destination.hashCode()); + } + +} diff --git a/datastream-bigquery/src/test/java/com/linkedin/datastream/bigquery/BigqueryTransportProviderAdminTests.java b/datastream-bigquery/src/test/java/com/linkedin/datastream/bigquery/BigqueryTransportProviderAdminTests.java new file mode 100644 index 000000000..497e77ab0 --- /dev/null +++ b/datastream-bigquery/src/test/java/com/linkedin/datastream/bigquery/BigqueryTransportProviderAdminTests.java @@ -0,0 +1,526 @@ +/* + * Copyright 2020 Wayfair LLC. All rights reserved. + * Licensed under the BSD 2-Clause License. See the LICENSE file in the project root for license information. + * See the NOTICE file in the project root for additional information regarding copyright ownership. + */ + +package com.linkedin.datastream.bigquery; + +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +import org.apache.avro.Schema; +import org.mockito.Mockito; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Test; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; + +import com.linkedin.data.template.StringMap; +import com.linkedin.datastream.bigquery.schema.BigquerySchemaEvolver; +import com.linkedin.datastream.bigquery.schema.BigquerySchemaEvolverFactory; +import com.linkedin.datastream.bigquery.schema.BigquerySchemaEvolverType; +import com.linkedin.datastream.common.Datastream; +import com.linkedin.datastream.serde.Deserializer; +import com.linkedin.datastream.server.DatastreamTask; +import com.linkedin.datastream.server.DatastreamTaskImpl; +import com.linkedin.datastream.server.api.transport.TransportProvider; +import com.linkedin.datastream.testutil.DatastreamTestUtils; + +import static com.linkedin.datastream.bigquery.BigqueryTransportProviderAdmin.METADATA_AUTO_CREATE_TABLE_KEY; +import static com.linkedin.datastream.bigquery.BigqueryTransportProviderAdmin.METADATA_DEAD_LETTER_TABLE_KEY; +import static com.linkedin.datastream.bigquery.BigqueryTransportProviderAdmin.METADATA_DISABLE_DEAD_LETTER_TABLE_KEY; +import static com.linkedin.datastream.bigquery.BigqueryTransportProviderAdmin.METADATA_LABELS_KEY; +import static com.linkedin.datastream.bigquery.BigqueryTransportProviderAdmin.METADATA_PARTITION_EXPIRATION_DAYS_KEY; +import static com.linkedin.datastream.bigquery.BigqueryTransportProviderAdmin.METADATA_RELAX_AVRO_SCHEMA_VALIDATION; +import static com.linkedin.datastream.bigquery.BigqueryTransportProviderAdmin.METADATA_SCHEMA_EVOLUTION_MODE_KEY; +import static com.linkedin.datastream.bigquery.BigqueryTransportProviderAdmin.METADATA_SCHEMA_ID_KEY; +import static com.linkedin.datastream.bigquery.BigqueryTransportProviderAdmin.METADATA_SCHEMA_REGISTRY_LOCATION_KEY; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertTrue; + +/** + * Tests for BigqueryTransportProviderAdmin. + */ +public class BigqueryTransportProviderAdminTests { + + private BigqueryBufferedTransportProvider bufferedTransportProvider; + private Serializer serializer; + private Deserializer deserializer; + private String defaultProjectId; + private String defaultSchemaRegistryUrl; + private BigquerySchemaEvolver defaultSchemaEvolver; + private ConcurrentMap datastreamConfigByDestination; + private Map bigquerySchemaEvolverMap; + private BigqueryTransportProviderFactory bigqueryTransportProviderFactory; + private BigqueryDatastreamConfigurationFactory bigqueryDatastreamConfigurationFactory; + + @BeforeMethod + public void beforeTest() { + bufferedTransportProvider = mock(BigqueryBufferedTransportProvider.class); + serializer = mock(Serializer.class); + deserializer = mock(Deserializer.class); + defaultProjectId = "projectId"; + defaultSchemaRegistryUrl = "https://schema-registry"; + defaultSchemaEvolver = BigquerySchemaEvolverFactory.createBigquerySchemaEvolver(BigquerySchemaEvolverType.dynamic); + datastreamConfigByDestination = new ConcurrentHashMap<>(); + bigquerySchemaEvolverMap = Arrays.stream(BigquerySchemaEvolverType.values()) + .collect(Collectors.toMap(BigquerySchemaEvolverType::getModeName, BigquerySchemaEvolverFactory::createBigquerySchemaEvolver)); + bigqueryTransportProviderFactory = mock(BigqueryTransportProviderFactory.class); + bigqueryDatastreamConfigurationFactory = mock(BigqueryDatastreamConfigurationFactory.class); + } + + @Test + public void testAssignTask() { + final BigqueryTransportProviderAdmin admin = new BigqueryTransportProviderAdmin( + bufferedTransportProvider, + defaultSchemaEvolver, + datastreamConfigByDestination, + bigquerySchemaEvolverMap, + bigqueryTransportProviderFactory, + bigqueryDatastreamConfigurationFactory, + defaultProjectId, + defaultSchemaRegistryUrl + ); + final String datastreamName = "test"; + final String schemaRegistryLocation = "https://schema-registry"; + final BigqueryDatastreamDestination destination = new BigqueryDatastreamDestination("project", "dataset", "table"); + final Datastream datastream = DatastreamTestUtils.createDatastream("connector", datastreamName, "source", destination.toString(), 1); + datastream.getMetadata().put(METADATA_SCHEMA_REGISTRY_LOCATION_KEY, schemaRegistryLocation); + final DatastreamTask task = new DatastreamTaskImpl(Collections.singletonList(datastream)); + final BigqueryDatastreamConfiguration config = BigqueryDatastreamConfiguration.builder( + destination, + defaultSchemaEvolver, + true, + deserializer, + serializer + ).build(); + when(bigqueryDatastreamConfigurationFactory.createBigqueryDatastreamConfiguration( + destination, + datastreamName, + schemaRegistryLocation, + config.getSchemaEvolver(), + config.isCreateDestinationTableEnabled(), + null, + null, + null, + null, + null + )).thenReturn(config); + final BigqueryTransportProvider bigqueryTransportProvider = mock(BigqueryTransportProvider.class); + + when(bigqueryTransportProviderFactory.createTransportProvider(bufferedTransportProvider, serializer, deserializer, config, datastreamConfigByDestination + )).thenReturn(bigqueryTransportProvider); + + final TransportProvider transportProvider = admin.assignTransportProvider(task); + + assertEquals(transportProvider, bigqueryTransportProvider); + assertEquals(admin.getDatastreamTransportProviders().get(datastream), transportProvider); + assertEquals(admin.getTransportProviderTasks().get(transportProvider), Collections.singleton(task)); + } + + @Test + public void testUnassignTask() { + final BigqueryTransportProviderAdmin admin = new BigqueryTransportProviderAdmin( + bufferedTransportProvider, + defaultSchemaEvolver, + datastreamConfigByDestination, + bigquerySchemaEvolverMap, + bigqueryTransportProviderFactory, + bigqueryDatastreamConfigurationFactory, + defaultProjectId, + defaultSchemaRegistryUrl + ); + final String datastreamName = "test"; + final String schemaRegistryLocation = "https://schema-registry"; + final BigqueryDatastreamDestination destination = new BigqueryDatastreamDestination("project", "dataset", "table"); + final Datastream datastream = DatastreamTestUtils.createDatastream("connector", datastreamName, "source", destination.toString(), 1); + datastream.getMetadata().put(METADATA_SCHEMA_REGISTRY_LOCATION_KEY, schemaRegistryLocation); + final DatastreamTask task = new DatastreamTaskImpl(Collections.singletonList(datastream)); + final BigqueryDatastreamConfiguration config = BigqueryDatastreamConfiguration.builder( + destination, + defaultSchemaEvolver, + true, + deserializer, + serializer + ).build(); + when(bigqueryDatastreamConfigurationFactory.createBigqueryDatastreamConfiguration( + destination, + datastreamName, + schemaRegistryLocation, + config.getSchemaEvolver(), + config.isCreateDestinationTableEnabled(), + null, + null, + null, + null, + null + )).thenReturn(config); + final BigqueryTransportProvider bigqueryTransportProvider = mock(BigqueryTransportProvider.class); + + when(bigqueryTransportProviderFactory.createTransportProvider(bufferedTransportProvider, serializer, deserializer, config, datastreamConfigByDestination + )).thenReturn(bigqueryTransportProvider); + + final TransportProvider transportProvider = admin.assignTransportProvider(task); + datastreamConfigByDestination.put(destination, config); + + assertEquals(transportProvider, bigqueryTransportProvider); + assertEquals(admin.getDatastreamTransportProviders().get(datastream), transportProvider); + assertEquals(admin.getTransportProviderTasks().get(transportProvider), Collections.singleton(task)); + + admin.unassignTransportProvider(task); + + assertFalse(admin.getDatastreamTransportProviders().containsKey(datastream)); + assertFalse(admin.getTransportProviderTasks().containsKey(transportProvider)); + verify(bigqueryTransportProvider).close(); + assertEquals(datastreamConfigByDestination.size(), 1); + } + + @Test + public void testParallelAssignAndUnassign() { + final BigqueryTransportProviderAdmin admin = new BigqueryTransportProviderAdmin( + bufferedTransportProvider, + defaultSchemaEvolver, + datastreamConfigByDestination, + bigquerySchemaEvolverMap, + bigqueryTransportProviderFactory, + bigqueryDatastreamConfigurationFactory, + defaultProjectId, + defaultSchemaRegistryUrl + ); + final String datastreamName = "test"; + final String schemaRegistryLocation = "https://schema-registry"; + final BigqueryDatastreamDestination destination = new BigqueryDatastreamDestination("project", "dataset", "table"); + final Datastream datastream = DatastreamTestUtils.createDatastream("connector", datastreamName, "source", destination.toString(), 1); + datastream.getMetadata().put(METADATA_SCHEMA_REGISTRY_LOCATION_KEY, schemaRegistryLocation); + final BigqueryDatastreamConfiguration config = BigqueryDatastreamConfiguration.builder( + destination, + defaultSchemaEvolver, + true, + deserializer, + serializer + ).build(); + when(bigqueryDatastreamConfigurationFactory.createBigqueryDatastreamConfiguration( + destination, + datastreamName, + schemaRegistryLocation, + config.getSchemaEvolver(), + config.isCreateDestinationTableEnabled(), + null, + null, + null, + null, + null + )).thenReturn(config); + final BigqueryTransportProvider bigqueryTransportProvider = mock(BigqueryTransportProvider.class); + + when(bigqueryTransportProviderFactory.createTransportProvider(bufferedTransportProvider, serializer, deserializer, config, datastreamConfigByDestination + )).thenReturn(bigqueryTransportProvider); + when(bigqueryTransportProvider.getDestinations()).thenReturn(ImmutableSet.of(destination)); + + final DatastreamTask task = new DatastreamTaskImpl(Collections.singletonList(datastream)); + + admin.assignTransportProvider(task); + + final ExecutorService executor = Executors.newCachedThreadPool(); + + final List> results = IntStream.range(0, 100).boxed().map(i -> executor.submit(() -> { + admin.unassignTransportProvider(task); + admin.assignTransportProvider(task); + datastreamConfigByDestination.put(destination, config); + })).collect(Collectors.toList()); + + results.forEach(r -> { + try { + r.get(); + } catch (final Exception e) { + throw new RuntimeException(e); + } + }); + + assertEquals(datastreamConfigByDestination.size(), 1); + } + + + @Test + public void testAssignMultipleTasksPerDatastream() { + final BigqueryTransportProviderAdmin admin = new BigqueryTransportProviderAdmin( + bufferedTransportProvider, + defaultSchemaEvolver, + datastreamConfigByDestination, + bigquerySchemaEvolverMap, + bigqueryTransportProviderFactory, + bigqueryDatastreamConfigurationFactory, + defaultProjectId, + defaultSchemaRegistryUrl + ); + final String datastreamName = "test"; + final String schemaRegistryLocation = "https://schema-registry"; + final BigqueryDatastreamDestination destination = new BigqueryDatastreamDestination("project", "dataset", "table"); + final Datastream datastream = DatastreamTestUtils.createDatastream("connector", datastreamName, "source", destination.toString(), 1); + datastream.getMetadata().put(METADATA_SCHEMA_REGISTRY_LOCATION_KEY, schemaRegistryLocation); + final BigqueryDatastreamConfiguration config = BigqueryDatastreamConfiguration.builder( + destination, + defaultSchemaEvolver, + true, + deserializer, + serializer + ).build(); + when(bigqueryDatastreamConfigurationFactory.createBigqueryDatastreamConfiguration( + destination, + datastreamName, + schemaRegistryLocation, + config.getSchemaEvolver(), + config.isCreateDestinationTableEnabled(), + null, + null, + null, + null, + null + )).thenReturn(config); + final BigqueryTransportProvider bigqueryTransportProvider = mock(BigqueryTransportProvider.class); + + when(bigqueryTransportProviderFactory.createTransportProvider(bufferedTransportProvider, serializer, deserializer, config, datastreamConfigByDestination + )).thenReturn(bigqueryTransportProvider); + when(bigqueryTransportProvider.getDestinations()).thenReturn(ImmutableSet.of(destination)); + + final DatastreamTask firstTask = new DatastreamTaskImpl(Collections.singletonList(datastream)); + + admin.assignTransportProvider(firstTask); + datastreamConfigByDestination.put(destination, config); + + final DatastreamTask secondTask = new DatastreamTaskImpl(Collections.singletonList(datastream)); + admin.assignTransportProvider(secondTask); + + admin.unassignTransportProvider(firstTask); + + assertEquals(datastreamConfigByDestination.size(), 1); + verify(bigqueryTransportProvider, never()).close(); + } + + @DataProvider(name = "datastream config test cases") + public Object[][] datastreamConfigTestCases() { + try { + return new Object[][]{ + { + new BigqueryDatastreamDestination("project", "dataset", "dest"), + ImmutableMap.of( + METADATA_SCHEMA_REGISTRY_LOCATION_KEY, "https://schema-registry", + METADATA_SCHEMA_ID_KEY, "1", + METADATA_SCHEMA_EVOLUTION_MODE_KEY, BigquerySchemaEvolverType.fixed.getModeName() + ), + BigqueryDatastreamConfiguration.builder(new BigqueryDatastreamDestination("project", "dataset", "dest"), + BigquerySchemaEvolverFactory.createBigquerySchemaEvolver(BigquerySchemaEvolverType.fixed), true, deserializer, serializer) + .withFixedSchema(mock(Schema.class)) + .withDeadLetterTableConfiguration(BigqueryDatastreamConfiguration + .builder(new BigqueryDatastreamDestination("project", "dataset", "dest_exceptions"), + BigquerySchemaEvolverFactory.createBigquerySchemaEvolver(BigquerySchemaEvolverType.dynamic), + true, + deserializer, serializer).build()) + .build() + }, + { + new BigqueryDatastreamDestination("project", "dataset", "dest"), + ImmutableMap.of( + METADATA_SCHEMA_REGISTRY_LOCATION_KEY, "https://schema-registry", + METADATA_SCHEMA_EVOLUTION_MODE_KEY, BigquerySchemaEvolverType.dynamic.getModeName() + ), + BigqueryDatastreamConfiguration.builder(new BigqueryDatastreamDestination("project", "dataset", "dest"), + BigquerySchemaEvolverFactory.createBigquerySchemaEvolver(BigquerySchemaEvolverType.dynamic), true, deserializer, serializer) + .withDeadLetterTableConfiguration(BigqueryDatastreamConfiguration + .builder(new BigqueryDatastreamDestination("project", "dataset", "dest_exceptions"), + BigquerySchemaEvolverFactory.createBigquerySchemaEvolver(BigquerySchemaEvolverType.dynamic), + true, + deserializer, serializer).build()) + .build() + }, + { + new BigqueryDatastreamDestination("project", "dataset", "dest"), + ImmutableMap.of( + METADATA_SCHEMA_REGISTRY_LOCATION_KEY, "https://schema-registry", + METADATA_SCHEMA_EVOLUTION_MODE_KEY, BigquerySchemaEvolverType.noop.getModeName() + ), + BigqueryDatastreamConfiguration.builder(new BigqueryDatastreamDestination("project", "dataset", "dest"), + BigquerySchemaEvolverFactory.createBigquerySchemaEvolver(BigquerySchemaEvolverType.noop), true, deserializer, serializer) + .withDeadLetterTableConfiguration(BigqueryDatastreamConfiguration + .builder(new BigqueryDatastreamDestination("project", "dataset", "dest_exceptions"), + BigquerySchemaEvolverFactory.createBigquerySchemaEvolver(BigquerySchemaEvolverType.dynamic), + true, + deserializer, serializer).build()) + .build() + }, + { + new BigqueryDatastreamDestination("project", "dataset", "dest"), + ImmutableMap.of( + METADATA_SCHEMA_REGISTRY_LOCATION_KEY, "https://schema-registry", + METADATA_SCHEMA_EVOLUTION_MODE_KEY, BigquerySchemaEvolverType.dynamic.getModeName(), + METADATA_RELAX_AVRO_SCHEMA_VALIDATION, Boolean.TRUE.toString() + ), + BigqueryDatastreamConfiguration.builder(new BigqueryDatastreamDestination("project", "dataset", "dest"), + BigquerySchemaEvolverFactory.createBigquerySchemaEvolver(BigquerySchemaEvolverType.dynamic), true, deserializer, serializer) + .withDeadLetterTableConfiguration(BigqueryDatastreamConfiguration + .builder(new BigqueryDatastreamDestination("project", "dataset", "dest_exceptions"), + BigquerySchemaEvolverFactory.createBigquerySchemaEvolver(BigquerySchemaEvolverType.dynamic), + true, + deserializer, serializer).build()) + .build() + }, + { + new BigqueryDatastreamDestination("project", "dataset", "dest"), + ImmutableMap.of( + METADATA_SCHEMA_REGISTRY_LOCATION_KEY, "https://schema-registry", + METADATA_SCHEMA_EVOLUTION_MODE_KEY, BigquerySchemaEvolverType.dynamic.getModeName(), + METADATA_RELAX_AVRO_SCHEMA_VALIDATION, Boolean.FALSE.toString() + ), + BigqueryDatastreamConfiguration.builder(new BigqueryDatastreamDestination("project", "dataset", "dest"), + BigquerySchemaEvolverFactory.createBigquerySchemaEvolver(BigquerySchemaEvolverType.dynamic), true, deserializer, serializer) + .withDeadLetterTableConfiguration(BigqueryDatastreamConfiguration + .builder(new BigqueryDatastreamDestination("project", "dataset", "dest_exceptions"), + BigquerySchemaEvolverFactory.createBigquerySchemaEvolver(BigquerySchemaEvolverType.dynamic), + true, + deserializer, serializer).build()) + .build() + }, + { + new BigqueryDatastreamDestination("project", "dataset", "dest"), + ImmutableMap.of( + METADATA_SCHEMA_REGISTRY_LOCATION_KEY, "https://schema-registry", + METADATA_SCHEMA_EVOLUTION_MODE_KEY, BigquerySchemaEvolverType.dynamic.getModeName(), + METADATA_LABELS_KEY, "test,name:value" + ), + BigqueryDatastreamConfiguration.builder(new BigqueryDatastreamDestination("project", "dataset", "dest"), + BigquerySchemaEvolverFactory.createBigquerySchemaEvolver(BigquerySchemaEvolverType.dynamic), true, deserializer, serializer) + .withDeadLetterTableConfiguration(BigqueryDatastreamConfiguration + .builder(new BigqueryDatastreamDestination("project", "dataset", "dest_exceptions"), + BigquerySchemaEvolverFactory.createBigquerySchemaEvolver(BigquerySchemaEvolverType.dynamic), + true, + deserializer, serializer).build()) + .withLabels(ImmutableList.of(BigqueryLabel.of("test"), BigqueryLabel.of("name", "value"))).build() + }, + { + new BigqueryDatastreamDestination("project", "dataset", "dest"), + ImmutableMap.of( + METADATA_SCHEMA_REGISTRY_LOCATION_KEY, "https://schema-registry", + METADATA_SCHEMA_EVOLUTION_MODE_KEY, BigquerySchemaEvolverType.dynamic.getModeName(), + METADATA_DEAD_LETTER_TABLE_KEY, new BigqueryDatastreamDestination("project", "dataset", "deadLetterTable").toString() + ), + BigqueryDatastreamConfiguration.builder(new BigqueryDatastreamDestination("project", "dataset", "dest"), + BigquerySchemaEvolverFactory.createBigquerySchemaEvolver(BigquerySchemaEvolverType.dynamic), true, deserializer, serializer) + .withDeadLetterTableConfiguration(BigqueryDatastreamConfiguration.builder( + new BigqueryDatastreamDestination("project", "dataset", "deadLetterTable"), + BigquerySchemaEvolverFactory.createBigquerySchemaEvolver(BigquerySchemaEvolverType.dynamic), true, deserializer, serializer).build()).build() + }, + { + new BigqueryDatastreamDestination("project", "dataset", "dest"), + ImmutableMap.of( + METADATA_SCHEMA_REGISTRY_LOCATION_KEY, "https://schema-registry", + METADATA_SCHEMA_EVOLUTION_MODE_KEY, BigquerySchemaEvolverType.dynamic.getModeName(), + METADATA_DEAD_LETTER_TABLE_KEY, new BigqueryDatastreamDestination("project", "dataset", "deadLetterTable").toString(), + METADATA_DISABLE_DEAD_LETTER_TABLE_KEY, "true" + ), + BigqueryDatastreamConfiguration.builder(new BigqueryDatastreamDestination("project", "dataset", "dest"), + BigquerySchemaEvolverFactory.createBigquerySchemaEvolver(BigquerySchemaEvolverType.dynamic), true, deserializer, serializer) + .build() + }, + { + new BigqueryDatastreamDestination("project", "dataset", "dest"), + ImmutableMap.of( + METADATA_SCHEMA_REGISTRY_LOCATION_KEY, "https://schema-registry", + METADATA_SCHEMA_EVOLUTION_MODE_KEY, BigquerySchemaEvolverType.dynamic.getModeName(), + METADATA_DEAD_LETTER_TABLE_KEY, new BigqueryDatastreamDestination("project", "dataset", "deadLetterTable").toString(), + METADATA_DISABLE_DEAD_LETTER_TABLE_KEY, "false" + ), + BigqueryDatastreamConfiguration.builder(new BigqueryDatastreamDestination("project", "dataset", "dest"), + BigquerySchemaEvolverFactory.createBigquerySchemaEvolver(BigquerySchemaEvolverType.dynamic), true, deserializer, serializer) + .withDeadLetterTableConfiguration(BigqueryDatastreamConfiguration.builder( + new BigqueryDatastreamDestination("project", "dataset", "deadLetterTable"), + BigquerySchemaEvolverFactory.createBigquerySchemaEvolver(BigquerySchemaEvolverType.dynamic), true, deserializer, serializer).build()).build() + } + }; + } catch (Exception e) { + e.printStackTrace(); + throw e; + } + } + + @Test(dataProvider = "datastream config test cases") + public void testGetConfigurationFromDatastream( + final BigqueryDatastreamDestination destination, + final Map datastreamMetadata, + final BigqueryDatastreamConfiguration expectedConfiguration + ) { + final BigqueryTransportProviderAdmin admin = new BigqueryTransportProviderAdmin( + bufferedTransportProvider, + defaultSchemaEvolver, + datastreamConfigByDestination, + bigquerySchemaEvolverMap, + bigqueryTransportProviderFactory, + bigqueryDatastreamConfigurationFactory, + defaultProjectId, + defaultSchemaRegistryUrl + ); + final String datastreamName = "test"; + final Datastream datastream = DatastreamTestUtils.createDatastream("connector", datastreamName, "source", destination.toString(), 1); + datastream.setMetadata(new StringMap(datastreamMetadata)); + final BigqueryDatastreamDestination deadLetterTable = BigqueryDatastreamDestination.parse(Optional.ofNullable(datastreamMetadata.get(METADATA_DEAD_LETTER_TABLE_KEY)) + .orElseGet(() -> new BigqueryDatastreamDestination(destination.getProjectId(), destination.getDatasetId(), destination.getDestinatonName() + "_exceptions").toString())); + final String schemaRegistryUrl = datastreamMetadata.getOrDefault(METADATA_SCHEMA_REGISTRY_LOCATION_KEY, defaultSchemaRegistryUrl); + final BigquerySchemaEvolver schemaEvolver = bigquerySchemaEvolverMap.getOrDefault(datastreamMetadata.getOrDefault(METADATA_SCHEMA_EVOLUTION_MODE_KEY, ""), defaultSchemaEvolver); + final boolean autoCreateTable = Boolean.parseBoolean(datastreamMetadata.getOrDefault(METADATA_AUTO_CREATE_TABLE_KEY, Boolean.TRUE.toString())); + final Long partitionExpirationDays = Optional.ofNullable(datastreamMetadata.get(METADATA_PARTITION_EXPIRATION_DAYS_KEY)).map(Long::getLong).orElse(null); + + final List labels = Optional.ofNullable(datastreamMetadata.get(METADATA_LABELS_KEY)) + .map(admin::parseLabelsString).orElse(null); + final Integer schemaId = Optional.ofNullable(datastreamMetadata.get(METADATA_SCHEMA_ID_KEY)).map(Integer::valueOf).orElse(null); + final Boolean relaxedAvroSchemaValidation = Optional.ofNullable(datastreamMetadata.get(METADATA_RELAX_AVRO_SCHEMA_VALIDATION)) + .map(Boolean::valueOf).orElse(null); + + when(bigqueryDatastreamConfigurationFactory.createBigqueryDatastreamConfiguration( + deadLetterTable, + datastreamName, + schemaRegistryUrl, + BigquerySchemaEvolverFactory.createBigquerySchemaEvolver(BigquerySchemaEvolverType.dynamic), + true, + null, + null, + null, + null, + null + )).thenReturn(expectedConfiguration.getDeadLetterTableConfiguration().orElse(null)); + when(bigqueryDatastreamConfigurationFactory.createBigqueryDatastreamConfiguration( + destination, + datastreamName, + schemaRegistryUrl, + schemaEvolver, + autoCreateTable, + partitionExpirationDays, + expectedConfiguration.getDeadLetterTableConfiguration().orElse(null), + labels, + schemaId, + relaxedAvroSchemaValidation + )).thenReturn(expectedConfiguration); + final BigqueryDatastreamConfiguration config = admin.getConfigurationFromDatastream(datastream); + assertEquals(config, expectedConfiguration); + } + +} \ No newline at end of file diff --git a/datastream-bigquery/src/test/java/com/linkedin/datastream/bigquery/BigqueryTransportProviderTests.java b/datastream-bigquery/src/test/java/com/linkedin/datastream/bigquery/BigqueryTransportProviderTests.java new file mode 100644 index 000000000..5fc7448d7 --- /dev/null +++ b/datastream-bigquery/src/test/java/com/linkedin/datastream/bigquery/BigqueryTransportProviderTests.java @@ -0,0 +1,749 @@ +/** + * Copyright 2020 Wayfair LLC. All rights reserved. + * Licensed under the BSD 2-Clause License. See the LICENSE file in the project root for license information. + * See the NOTICE file in the project root for additional information regarding copyright ownership. + */ +package com.linkedin.datastream.bigquery; + +import com.codahale.metrics.MetricRegistry; +import com.google.api.client.util.DateTime; +import com.google.api.services.bigquery.model.TableRow; +import com.google.cloud.bigquery.BigQuery; +import com.google.cloud.bigquery.BigQueryError; +import com.google.cloud.bigquery.BigQueryException; +import com.google.cloud.bigquery.InsertAllRequest; +import com.google.cloud.bigquery.InsertAllResponse; +import com.google.cloud.bigquery.StandardTableDefinition; +import com.google.cloud.bigquery.Table; +import com.google.cloud.bigquery.TableDefinition; +import com.google.cloud.bigquery.TableId; +import com.google.cloud.bigquery.TableInfo; +import com.google.cloud.bigquery.TimePartitioning; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; + +import com.linkedin.datastream.bigquery.schema.BigquerySchemaEvolverFactory; +import com.linkedin.datastream.bigquery.schema.BigquerySchemaEvolverType; +import com.linkedin.datastream.bigquery.translator.RecordTranslator; +import com.linkedin.datastream.bigquery.translator.SchemaTranslator; +import com.linkedin.datastream.common.BrooklinEnvelope; +import com.linkedin.datastream.common.BrooklinEnvelopeMetadataConstants; +import com.linkedin.datastream.common.DatastreamRecordMetadata; +import com.linkedin.datastream.common.SendCallback; +import com.linkedin.datastream.metrics.DynamicMetricsManager; +import com.linkedin.datastream.serde.Deserializer; +import com.linkedin.datastream.server.DatastreamProducerRecord; +import com.linkedin.datastream.server.DatastreamProducerRecordBuilder; +import com.linkedin.datastream.server.Pair; + +import io.confluent.kafka.schemaregistry.client.MockSchemaRegistryClient; +import io.confluent.kafka.serializers.KafkaAvroDeserializer; +import io.confluent.kafka.serializers.KafkaAvroSerializer; +import io.confluent.kafka.serializers.KafkaAvroSerializerConfig; + +import org.apache.avro.Schema; +import org.apache.avro.SchemaBuilder; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.GenericRecordBuilder; +import org.mockito.ArgumentCaptor; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +import java.time.LocalDate; +import java.time.ZoneOffset; +import java.time.format.DateTimeFormatter; +import java.util.Collections; +import java.util.Date; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import java.util.stream.Stream; + +import static org.mockito.Matchers.any; +import static com.linkedin.datastream.server.api.transport.buffered.AbstractBufferedTransportProvider.KAFKA_ORIGIN_CLUSTER; +import static com.linkedin.datastream.server.api.transport.buffered.AbstractBufferedTransportProvider.KAFKA_ORIGIN_OFFSET; +import static com.linkedin.datastream.server.api.transport.buffered.AbstractBufferedTransportProvider.KAFKA_ORIGIN_PARTITION; +import static com.linkedin.datastream.server.api.transport.buffered.AbstractBufferedTransportProvider.KAFKA_ORIGIN_TOPIC; +import static org.mockito.Mockito.atLeastOnce; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; +import static org.testng.Assert.assertEquals; + +public class BigqueryTransportProviderTests { + + private static final AtomicInteger TOPIC_COUNTER = new AtomicInteger(); + private static final AtomicInteger DATASET_COUNTER = new AtomicInteger(); + + @BeforeClass + public static void setup() { + DynamicMetricsManager.createInstance(new MetricRegistry(), BigqueryTransportProviderTests.class.getSimpleName()); + } + + @Test + public void testSendHappyPath() { + final int maxBatchAge = 10; + final int maxBatchSize = 10; + final int maxInflightCommits = 10; + final int queueSize = 10; + final int totalEvents = 25; + + final Schema schema = SchemaBuilder.builder("com.linkedin").record("test_message") + .fields().name("message").type("string").noDefault().endRecord(); + final List> data = IntStream.range(0, totalEvents) + .mapToObj(i -> ImmutableMap.of("message", "payload " + i)) + .collect(Collectors.toList()); + + final List events = data.stream() + .map(recordData -> { + final GenericRecordBuilder builder = new GenericRecordBuilder(schema); + recordData.forEach(builder::set); + return builder.build(); + }).collect(Collectors.toList()); + + + final String projectId = "projectId"; + final String datasetName = getUniqueDatasetName(); + final String cluster = "kafka://test"; + final String topicName = getUniqueTopicName(); + final String tableName = BigqueryBatchCommitter.sanitizeTableName(topicName); + final BigqueryDatastreamDestination destination = new BigqueryDatastreamDestination(projectId, datasetName, tableName); + final ConcurrentMap destConfigs = new ConcurrentHashMap<>(); + final BigqueryDatastreamConfiguration config = BigqueryDatastreamConfiguration.builder( + destination, BigquerySchemaEvolverFactory.createBigquerySchemaEvolver(BigquerySchemaEvolverType.dynamic), true, + valueDeserializer, valueSerializer).build(); + destConfigs.put(destination, config); + + final BigqueryBatchCommitter committer = new BigqueryBatchCommitter(bigQuery, 1, destConfigs); + final BatchBuilder batchBuilder = new BatchBuilder( + maxBatchSize, maxBatchAge, maxInflightCommits, committer, queueSize, destConfigs + ); + final List batchBuilders = ImmutableList.of(batchBuilder); + + final TableId tableId = TableId.of(projectId, datasetName, tableName); + final TableId insertTableId = TableId.of(projectId, tableId.getDataset(), + String.format("%s$%s", tableId.getTable(), LocalDate.now(ZoneOffset.UTC).format(DateTimeFormatter.ofPattern("yyyyMMdd")))); + + final InsertAllResponse insertAllResponse = mock(InsertAllResponse.class); + when(insertAllResponse.getInsertErrors()).thenReturn(ImmutableMap.of()); + when(bigQuery.insertAll(any(InsertAllRequest.class))).thenReturn(insertAllResponse); + + final DateTime eventTimestamp = new DateTime(new Date(System.currentTimeMillis())); + + try (final BigqueryTransportProvider transportProvider = new BigqueryTransportProvider( + new BigqueryBufferedTransportProvider("test", batchBuilders, committer, maxBatchAge), + valueSerializer, valueDeserializer, config, destConfigs)) { + sendEvents(transportProvider, destination.toString(), cluster, topicName, 0, events, eventTimestamp, null); + } + + final ArgumentCaptor requestArgumentCaptor = ArgumentCaptor.forClass(InsertAllRequest.class); + verify(bigQuery, atLeastOnce()).insertAll(requestArgumentCaptor.capture()); + final List capturedRequests = requestArgumentCaptor.getAllValues(); + + capturedRequests.forEach(request -> assertEquals(request.getTable(), insertTableId)); + final List> insertedData = capturedRequests.stream() + .flatMap(request -> request.getRows().stream().map(row -> row.getContent().entrySet().stream() + .filter(entry -> !"__metadata".equals(entry.getKey())) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)) + )).collect(Collectors.toList()); + + assertEquals(insertedData, data); + } + + private static InsertAllRequest.RowToInsert getExpectedRowToInsert(final InsertAllRequest.RowToInsert actualRow, final GenericRecord expectedRecord) { + return RecordTranslator.translate(expectedRecord, (TableRow) actualRow.getContent().get("__metadata")); + } + + private static InsertAllRequest getExpectedRequest(final InsertAllRequest actualRequest, final TableId insertTableId, + final List expectedRecords) { + assertEquals(actualRequest.getRows().size(), expectedRecords.size()); + return InsertAllRequest.of(insertTableId, IntStream.range(0, expectedRecords.size()) + .mapToObj(index -> getExpectedRowToInsert(actualRequest.getRows().get(index), expectedRecords.get(index))) + .collect(Collectors.toList())); + } + + @Test + public void testSendAndEvolveSchema() { + final int maxBatchAge = 10; + final int maxBatchSize = 10; + final int maxInflightCommits = 10; + final int queueSize = 10; + + final Schema schema1 = SchemaBuilder.builder("com.linkedin").record("test_message") + .fields().name("message").type("string").noDefault().endRecord(); + + final List> data1 = IntStream.range(0, 15) + .mapToObj(i -> ImmutableMap.of("message", "payload " + i)) + .collect(Collectors.toList()); + final List events1 = data1.stream() + .map(recordData -> { + final GenericRecordBuilder builder = new GenericRecordBuilder(schema1); + recordData.forEach(builder::set); + return builder.build(); + }).collect(Collectors.toList()); + + + final Schema schema2 = SchemaBuilder.builder("com.linkedin").record("test_message") + .fields() + .name("message").type("string").noDefault() + .name("new_message").type("string").noDefault() + .endRecord(); + final List> data2 = IntStream.range(0, 15) + .mapToObj(i -> ImmutableMap.of("message", "payload " + i, "new_message", "new payload " + i)) + .collect(Collectors.toList()); + final List events2 = data2.stream() + .map(recordData -> { + final GenericRecordBuilder builder = new GenericRecordBuilder(schema2); + recordData.forEach(builder::set); + return builder.build(); + }).collect(Collectors.toList()); + + final String projectId = "projectId"; + final String datasetName = getUniqueDatasetName(); + final String cluster = "kafka://test"; + final String topicName = getUniqueTopicName(); + final String tableName = BigqueryBatchCommitter.sanitizeTableName(topicName); + final BigqueryDatastreamDestination destination = new BigqueryDatastreamDestination(projectId, datasetName, tableName); + final BigqueryDatastreamConfiguration config = BigqueryDatastreamConfiguration.builder( + destination, BigquerySchemaEvolverFactory.createBigquerySchemaEvolver(BigquerySchemaEvolverType.dynamic), true, + valueDeserializer, valueSerializer).build(); + final ConcurrentMap destConfigs = new ConcurrentHashMap<>(); + destConfigs.put(destination, config); + + final BigqueryBatchCommitter committer = new BigqueryBatchCommitter(bigQuery, 1, destConfigs); + final BatchBuilder batchBuilder = new BatchBuilder( + maxBatchSize, maxBatchAge, maxInflightCommits, committer, queueSize, destConfigs + ); + final List batchBuilders = ImmutableList.of(batchBuilder); + final TableId tableId = TableId.of(projectId, datasetName, topicName); + + final com.google.cloud.bigquery.Schema firstSchema = SchemaTranslator.translate(schema1); + final TableDefinition tableDefinition = StandardTableDefinition.newBuilder() + .setSchema(firstSchema) + .setTimePartitioning(TimePartitioning.of(TimePartitioning.Type.DAY)) + .build(); + + final Table mockedTable = mock(Table.class); + when(mockedTable.getDefinition()).thenReturn(tableDefinition); + + final Table.Builder tableBuilder = mock(Table.Builder.class); + when(mockedTable.toBuilder()).thenReturn(tableBuilder); + final TableDefinition evolvedTableDefinition = StandardTableDefinition.newBuilder() + .setSchema(config.getSchemaEvolver().evolveSchema(firstSchema, SchemaTranslator.translate(schema2))) + .setTimePartitioning(TimePartitioning.of(TimePartitioning.Type.DAY)) + .build(); + when(tableBuilder.setDefinition(evolvedTableDefinition)).thenReturn(tableBuilder); + final Table evolvedTable = mock(Table.class); + when(evolvedTable.getDefinition()).thenReturn(evolvedTableDefinition); + when(tableBuilder.build()).thenReturn(evolvedTable); + final AtomicBoolean tableUpdated = new AtomicBoolean(false); + when(bigQuery.getTable(tableId)).thenReturn(mockedTable); + final List> insertedData = new LinkedList<>(); + when(bigQuery.insertAll(any(InsertAllRequest.class))).then(invocation -> { + final InsertAllRequest request = invocation.getArgumentAt(0, InsertAllRequest.class); + if (request.getRows().stream().anyMatch(row -> row.getContent().containsKey("new_message"))) { + if (tableUpdated.get()) { + request.getRows().forEach(row -> insertedData.add(row.getContent().entrySet().stream() + .filter(entry -> !"__metadata".equals(entry.getKey())) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)))); + final InsertAllResponse response = mock(InsertAllResponse.class); + when(response.getInsertErrors()).thenReturn(ImmutableMap.of()); + return response; + } else { + throw new BigQueryException(400, "Table missing column", new BigQueryError("invalid", "new_message", "Table missing column")); + } + } else { + request.getRows().forEach(row -> insertedData.add(row.getContent().entrySet().stream() + .filter(entry -> !"__metadata".equals(entry.getKey())) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)))); + final InsertAllResponse response = mock(InsertAllResponse.class); + when(response.getInsertErrors()).thenReturn(ImmutableMap.of()); + return response; + } + }); + when(evolvedTable.update()).then(invocation -> { + if (!tableUpdated.get()) { + tableUpdated.set(true); + return mock(Table.class); + } else { + throw new IllegalStateException("Table already updated"); + } + }); + + try (final BigqueryTransportProvider transportProvider = new BigqueryTransportProvider( + new BigqueryBufferedTransportProvider("test", batchBuilders, committer, maxBatchAge), + valueSerializer, valueDeserializer, config, destConfigs)) { + final DateTime eventTimestamp1 = new DateTime(new Date(System.currentTimeMillis())); + sendEvents(transportProvider, destination.toString(), cluster, topicName, 0, events1, eventTimestamp1, null); + + verify(bigQuery, atLeastOnce()).insertAll(any(InsertAllRequest.class)); + + final DateTime eventTimestamp2 = new DateTime(new Date(System.currentTimeMillis())); + sendEvents(transportProvider, destination.toString(), cluster, topicName, 0, events2, eventTimestamp2, null); + + verify(evolvedTable).update(); + verify(bigQuery, atLeastOnce()).insertAll(any(InsertAllRequest.class)); + + assertEquals(insertedData, Stream.concat(data1.stream(), data2.stream()).collect(Collectors.toList())); + } + } + + @Test + public void testInsertRecordExceptionHandling() { + final int maxBatchAge = 10; + final int maxBatchSize = 10; + final int maxInflightCommits = 10; + final int queueSize = 10; + final int totalEvents = 15; + + final Schema schema = SchemaBuilder.builder("com.linkedin").record("test_message") + .fields().name("message").type("string").noDefault().endRecord(); + final List events = IntStream.range(0, totalEvents) + .mapToObj(i -> new GenericRecordBuilder(schema).set("message", "payload " + i).build()) + .collect(Collectors.toList()); + + final String projectId = "projectId"; + final String datasetName = getUniqueDatasetName(); + final String cluster = "kafka://test"; + final String topicName = getUniqueTopicName(); + final String tableName = BigqueryBatchCommitter.sanitizeTableName(topicName); + final BigqueryDatastreamDestination destination = new BigqueryDatastreamDestination(projectId, datasetName, tableName); + final ConcurrentMap destConfigs = new ConcurrentHashMap<>(); + final BigqueryDatastreamConfiguration config = BigqueryDatastreamConfiguration.builder( + destination, BigquerySchemaEvolverFactory.createBigquerySchemaEvolver(BigquerySchemaEvolverType.dynamic), true, + valueDeserializer, valueSerializer).build(); + destConfigs.put(destination, config); + + final BigqueryBatchCommitter committer = new BigqueryBatchCommitter(bigQuery, 1, destConfigs); + final BatchBuilder batchBuilder = new BatchBuilder( + maxBatchSize, maxBatchAge, maxInflightCommits, committer, queueSize, destConfigs + ); + final List batchBuilders = ImmutableList.of(batchBuilder); + + final TableId tableId = TableId.of(projectId, datasetName, topicName); + final TableId insertTableId = TableId.of(tableId.getProject(), tableId.getDataset(), + String.format("%s$%s", tableId.getTable(), LocalDate.now(ZoneOffset.UTC).format(DateTimeFormatter.ofPattern("yyyyMMdd")))); + + final TableDefinition tableDefinition = StandardTableDefinition.newBuilder() + .setSchema(SchemaTranslator.translate(schema)) + .setTimePartitioning(TimePartitioning.of(TimePartitioning.Type.DAY)) + .build(); + final TableInfo tableInfo = TableInfo.newBuilder(tableId, tableDefinition).build(); + final Table mockedTable = mock(Table.class); + when(mockedTable.getDefinition()).thenReturn(tableDefinition); + when(bigQuery.getTable(tableId)).thenReturn(mockedTable); + + final TableId exceptionsTableId = TableId.of(projectId, datasetName, topicName + "_exceptions"); + + final TableDefinition exceptionsTableDefinition = StandardTableDefinition.newBuilder() + .setSchema(SchemaTranslator.translate(BigqueryTransportProvider.EXCEPTION_RECORD_SCHEMA)) + .setTimePartitioning(TimePartitioning.of(TimePartitioning.Type.DAY)) + .build(); + final TableInfo exceptionsTableInfo = TableInfo.newBuilder(exceptionsTableId, exceptionsTableDefinition).build(); + final Table mockedExceptionsTable = mock(Table.class); + when(mockedExceptionsTable.getDefinition()).thenReturn(exceptionsTableDefinition); + when(bigQuery.getTable(exceptionsTableId)).thenReturn(mockedExceptionsTable); + + final Exception testException = new RuntimeException("Test insert failure"); + when(bigQuery.insertAll(any(InsertAllRequest.class))) + .thenThrow(testException); + + final DateTime eventTimestamp = new DateTime(new Date(System.currentTimeMillis())); + final List callbackExceptions = new LinkedList<>(); + + try (final BigqueryTransportProvider transportProvider = new BigqueryTransportProvider( + new BigqueryBufferedTransportProvider("test", batchBuilders, committer, maxBatchAge), + valueSerializer, valueDeserializer, config, destConfigs)) { + sendEvents(transportProvider, destination.toString(), cluster, topicName, 0, events, eventTimestamp, (metadata, exception) -> { + if (exception != null) { + callbackExceptions.add(exception); + } + }); + } + + verify(bigQuery, never()).create(tableInfo); + verify(bigQuery, never()).create(exceptionsTableInfo); + + final ArgumentCaptor requestArgumentCaptor = ArgumentCaptor.forClass(InsertAllRequest.class); + verify(bigQuery, atLeastOnce()).insertAll(requestArgumentCaptor.capture()); + final List capturedRequests = requestArgumentCaptor.getAllValues(); + + int rowsChecked = 0; + for (final InsertAllRequest request : capturedRequests) { + final int actualRequestRowCount = request.getRows().size(); + assertEquals(request, getExpectedRequest(request, insertTableId, events.subList(rowsChecked, rowsChecked + actualRequestRowCount))); + rowsChecked += actualRequestRowCount; + } + assertEquals(rowsChecked, events.size()); + assertEquals(callbackExceptions.size(), totalEvents); + } + + + @Test + public void testSchemaEvolutionExceptionHandling() { + final int maxBatchAge = 10; + final int maxBatchSize = 10; + final int maxInflightCommits = 10; + final int queueSize = 10; + + final Schema currentSchema = SchemaBuilder.builder("com.linkedin").record("test_message") + .fields().requiredInt("message").endRecord(); + + final Schema incompatibleEvolvedSchema = SchemaBuilder.builder("com.linkedin").record("test_message") + .fields() + .requiredString("message") + .endRecord(); + final List incompatibleEvents = IntStream.range(0, 15) + .mapToObj(i -> new GenericRecordBuilder(incompatibleEvolvedSchema) + .set("message", "payload " + i).build()) + .collect(Collectors.toList()); + + final String projectId = "projectId"; + final String datasetName = getUniqueDatasetName(); + final String cluster = "kafka://test"; + final String topicName = getUniqueTopicName(); + final String tableName = BigqueryBatchCommitter.sanitizeTableName(topicName); + final BigqueryDatastreamDestination destination = new BigqueryDatastreamDestination(projectId, datasetName, tableName); + final TableId exceptionsTableId = TableId.of(projectId, datasetName, topicName + "_exceptions"); + final BigqueryDatastreamDestination deadLetterTableDestination = new BigqueryDatastreamDestination(projectId, datasetName, exceptionsTableId.getTable()); + final ConcurrentMap destConfigs = new ConcurrentHashMap<>(); + final BigqueryDatastreamConfiguration config = BigqueryDatastreamConfiguration.builder( + destination, BigquerySchemaEvolverFactory.createBigquerySchemaEvolver(BigquerySchemaEvolverType.dynamic), true, + valueDeserializer, valueSerializer) + .withDeadLetterTableConfiguration( + BigqueryDatastreamConfiguration.builder( + deadLetterTableDestination, BigquerySchemaEvolverFactory.createBigquerySchemaEvolver(BigquerySchemaEvolverType.dynamic), true, + valueDeserializer, valueSerializer).build()) + .build(); + + destConfigs.put(destination, config); + + final BigqueryBatchCommitter committer = new BigqueryBatchCommitter(bigQuery, 1, destConfigs); + final BatchBuilder batchBuilder = new BatchBuilder( + maxBatchSize, maxBatchAge, maxInflightCommits, committer, queueSize, destConfigs + ); + final List batchBuilders = ImmutableList.of(batchBuilder); + + final TableId tableId = TableId.of(projectId, datasetName, topicName); + final TableId insertTableId = TableId.of(tableId.getProject(), tableId.getDataset(), + String.format("%s$%s", tableId.getTable(), LocalDate.now(ZoneOffset.UTC).format(DateTimeFormatter.ofPattern("yyyyMMdd")))); + + final com.google.cloud.bigquery.Schema firstSchema = SchemaTranslator.translate(currentSchema); + final TableDefinition tableDefinition = StandardTableDefinition.newBuilder() + .setSchema(firstSchema) + .setTimePartitioning(TimePartitioning.of(TimePartitioning.Type.DAY)) + .build(); + final TableInfo tableInfo = TableInfo.newBuilder(tableId, tableDefinition).build(); + + final Table mockedTable = mock(Table.class); + when(mockedTable.getDefinition()).thenReturn(tableDefinition); + when(bigQuery.getTable(tableId)).thenReturn(mockedTable); + + + final TableId insertExceptionsTableId = TableId.of(tableId.getProject(), tableId.getDataset(), + String.format("%s$%s", exceptionsTableId.getTable(), LocalDate.now(ZoneOffset.UTC).format(DateTimeFormatter.ofPattern("yyyyMMdd")))); + + final TableDefinition exceptionsTableDefinition = StandardTableDefinition.newBuilder() + .setSchema(SchemaTranslator.translate(BigqueryTransportProvider.EXCEPTION_RECORD_SCHEMA)) + .setTimePartitioning(TimePartitioning.of(TimePartitioning.Type.DAY)) + .build(); + final TableInfo exceptionsTableInfo = TableInfo.newBuilder(exceptionsTableId, exceptionsTableDefinition).build(); + final Table mockedExceptionsTable = mock(Table.class); + when(mockedExceptionsTable.getDefinition()).thenReturn(exceptionsTableDefinition); + when(bigQuery.getTable(exceptionsTableId)).thenReturn(mockedExceptionsTable); + + final List> insertedErrors = new LinkedList<>(); + when(bigQuery.insertAll(any(InsertAllRequest.class))).then(invocation -> { + final InsertAllRequest request = invocation.getArgumentAt(0, InsertAllRequest.class); + final InsertAllResponse response = mock(InsertAllResponse.class); + final Map> insertErrors; + if (request.getTable().equals(insertTableId)) { + insertErrors = IntStream.range(0, request.getRows().size()).boxed() + .flatMap(i -> { + final InsertAllRequest.RowToInsert row = request.getRows().get(i); + final Optional optionalError; + if (row.getContent().containsKey("message")) { + final Object message = row.getContent().get("message"); + if (message instanceof Integer) { + optionalError = Optional.empty(); + } else { + optionalError = Optional.of(new BigQueryError("invalid", "message", "Cannot convert value to an int")); + } + } else { + throw new IllegalStateException("Missing required column: message"); + } + return optionalError.map(error -> Stream.of(Pair.of(i.longValue(), Collections.singletonList(error)))).orElse(Stream.empty()); + }).collect(Collectors.toMap(Pair::getKey, Pair::getValue)); + + } else if (request.getTable().equals(insertExceptionsTableId)) { + request.getRows().forEach(row -> insertedErrors.add(row.getContent())); + insertErrors = Collections.emptyMap(); + } else { + throw new IllegalStateException("Unexpected table id: " + request.getTable()); + } + when(response.getInsertErrors()).thenReturn(insertErrors); + return response; + }); + + final List callbackMetadata = new LinkedList<>(); + final List callbackExceptions = new LinkedList<>(); + + + try (final BigqueryTransportProvider transportProvider = new BigqueryTransportProvider( + new BigqueryBufferedTransportProvider("test", batchBuilders, committer, maxBatchAge), + valueSerializer, valueDeserializer, config, destConfigs)) { + + final DateTime eventTimestamp = new DateTime(new Date(System.currentTimeMillis())); + final String sourceCheckpoint = "testCheckpoint 1"; + final List records = incompatibleEvents.stream() + .map(event -> createRecord(cluster, topicName, offsetIncrement.getAndIncrement(), 0, event, eventTimestamp, sourceCheckpoint)) + .collect(Collectors.toList()); + sendEvents(transportProvider, destination.toString(), records, (metadata, exception) -> { + callbackMetadata.add(metadata); + if (exception != null) { + callbackExceptions.add(exception); + } + }); + final List expectedMetadata = records.stream() + .map(record -> new DatastreamRecordMetadata(sourceCheckpoint, topicName, 0)) + .collect(Collectors.toList()); + assertEquals( + callbackMetadata, + expectedMetadata + ); + verify(bigQuery, never()).create(tableInfo); + verify(bigQuery, atLeastOnce()).getTable(tableId); + verify(bigQuery, never()).create(exceptionsTableInfo); + verify(bigQuery).getTable(exceptionsTableId); + assertEquals(insertedErrors.size(), incompatibleEvents.size()); + assertEquals(callbackExceptions.size(), 0); + } + } + + + @Test + public void testInsertExceptionRecordFailure() { + final int maxBatchAge = 10; + final int maxBatchSize = 10; + final int maxInflightCommits = 10; + final int queueSize = 10; + + final Schema currentSchema = SchemaBuilder.builder("com.linkedin").record("test_message") + .fields().requiredInt("message").endRecord(); + + final Schema incompatibleEvolvedSchema = SchemaBuilder.builder("com.linkedin").record("test_message") + .fields() + .requiredString("message") + .endRecord(); + final List incompatibleEvents = IntStream.range(0, 15) + .mapToObj(i -> new GenericRecordBuilder(incompatibleEvolvedSchema) + .set("message", "payload " + i).build()) + .collect(Collectors.toList()); + + final String projectId = "projectId"; + final String datasetName = getUniqueDatasetName(); + final String cluster = "kafka://test"; + final String topicName = getUniqueTopicName(); + final String tableName = BigqueryBatchCommitter.sanitizeTableName(topicName); + final BigqueryDatastreamDestination destination = new BigqueryDatastreamDestination(projectId, datasetName, tableName); + final TableId exceptionsTableId = TableId.of(projectId, datasetName, topicName + "_exceptions"); + final BigqueryDatastreamDestination deadLetterTableDestination = new BigqueryDatastreamDestination(projectId, datasetName, exceptionsTableId.getTable()); + final ConcurrentMap destConfigs = new ConcurrentHashMap<>(); + final BigqueryDatastreamConfiguration config = BigqueryDatastreamConfiguration.builder( + destination, BigquerySchemaEvolverFactory.createBigquerySchemaEvolver(BigquerySchemaEvolverType.dynamic), true, + valueDeserializer, valueSerializer + ).withDeadLetterTableConfiguration( + BigqueryDatastreamConfiguration.builder( + deadLetterTableDestination, BigquerySchemaEvolverFactory.createBigquerySchemaEvolver(BigquerySchemaEvolverType.dynamic), true, + valueDeserializer, valueSerializer).build() + ).build(); + destConfigs.put(destination, config); + + final BigqueryBatchCommitter committer = new BigqueryBatchCommitter(bigQuery, 1, destConfigs); + final BatchBuilder batchBuilder = new BatchBuilder( + maxBatchSize, maxBatchAge, maxInflightCommits, committer, queueSize, destConfigs + ); + final List batchBuilders = ImmutableList.of(batchBuilder); + + final TableId tableId = TableId.of(projectId, datasetName, topicName); + final TableId insertTableId = TableId.of(tableId.getProject(), tableId.getDataset(), + String.format("%s$%s", tableId.getTable(), LocalDate.now(ZoneOffset.UTC).format(DateTimeFormatter.ofPattern("yyyyMMdd")))); + + final com.google.cloud.bigquery.Schema firstSchema = SchemaTranslator.translate(currentSchema); + final TableDefinition tableDefinition = StandardTableDefinition.newBuilder() + .setSchema(firstSchema) + .setTimePartitioning(TimePartitioning.of(TimePartitioning.Type.DAY)) + .build(); + final TableInfo tableInfo = TableInfo.newBuilder(tableId, tableDefinition).build(); + + final Table mockedTable = mock(Table.class); + when(mockedTable.getDefinition()).thenReturn(tableDefinition); + when(bigQuery.getTable(tableId)).thenReturn(mockedTable); + + final TableId insertExceptionsTableId = TableId.of(tableId.getProject(), tableId.getDataset(), + String.format("%s$%s", exceptionsTableId.getTable(), LocalDate.now(ZoneOffset.UTC).format(DateTimeFormatter.ofPattern("yyyyMMdd")))); + + final TableDefinition exceptionsTableDefinition = StandardTableDefinition.newBuilder() + .setSchema(SchemaTranslator.translate(BigqueryTransportProvider.EXCEPTION_RECORD_SCHEMA)) + .setTimePartitioning(TimePartitioning.of(TimePartitioning.Type.DAY)) + .build(); + final TableInfo exceptionsTableInfo = TableInfo.newBuilder(exceptionsTableId, exceptionsTableDefinition).build(); + final Table mockedExceptionsTable = mock(Table.class); + when(mockedExceptionsTable.getDefinition()).thenReturn(exceptionsTableDefinition); + when(bigQuery.getTable(exceptionsTableId)).thenReturn(mockedExceptionsTable); + + when(bigQuery.insertAll(any(InsertAllRequest.class))).then(invocation -> { + final InsertAllRequest request = invocation.getArgumentAt(0, InsertAllRequest.class); + final InsertAllResponse response = mock(InsertAllResponse.class); + final Map> insertErrors; + if (request.getTable().equals(insertTableId)) { + insertErrors = IntStream.range(0, request.getRows().size()).boxed() + .flatMap(i -> { + final InsertAllRequest.RowToInsert row = request.getRows().get(i); + final Optional optionalError; + if (row.getContent().containsKey("message")) { + final Object message = row.getContent().get("message"); + if (message instanceof Integer) { + optionalError = Optional.empty(); + } else { + optionalError = Optional.of(new BigQueryError("invalid", "message", "Cannot convert value to an int")); + } + } else { + throw new IllegalStateException("Missing required column: message"); + } + return optionalError.map(error -> Stream.of(Pair.of(i.longValue(), Collections.singletonList(error)))).orElse(Stream.empty()); + }).collect(Collectors.toMap(Pair::getKey, Pair::getValue)); + + } else if (request.getTable().equals(insertExceptionsTableId)) { + insertErrors = IntStream.range(0, request.getRows().size()).boxed() + .collect(Collectors.toMap(Integer::longValue, i -> + Collections.singletonList(new BigQueryError("error", null, "Unexpected error")))); + } else { + throw new IllegalStateException("Unexpected table id: " + request.getTable()); + } + when(response.getInsertErrors()).thenReturn(insertErrors); + return response; + }); + + final List callbackMetadata = new LinkedList<>(); + final List callbackExceptions = new LinkedList<>(); + + + try (final BigqueryTransportProvider transportProvider = new BigqueryTransportProvider( + new BigqueryBufferedTransportProvider("test", batchBuilders, committer, maxBatchAge), + valueSerializer, valueDeserializer, config, destConfigs)) { + + final DateTime eventTimestamp = new DateTime(new Date(System.currentTimeMillis())); + final String sourceCheckpoint = "testCheckpoint 1"; + final List records = incompatibleEvents.stream() + .map(event -> createRecord(cluster, topicName, offsetIncrement.getAndIncrement(), 0, event, eventTimestamp, sourceCheckpoint)) + .collect(Collectors.toList()); + sendEvents(transportProvider, destination.toString(), records, (metadata, exception) -> { + callbackMetadata.add(metadata); + if (exception != null) { + callbackExceptions.add(exception); + } + }); + final List expectedMetadata = records.stream() + .map(record -> new DatastreamRecordMetadata(sourceCheckpoint, topicName, 0)) + .collect(Collectors.toList()); + assertEquals( + callbackMetadata, + expectedMetadata + ); + verify(bigQuery, never()).create(tableInfo); + verify(bigQuery, atLeastOnce()).getTable(tableId); + verify(bigQuery, never()).create(exceptionsTableInfo); + verify(bigQuery, atLeastOnce()).getTable(exceptionsTableId); + assertEquals(callbackExceptions.size(), incompatibleEvents.size()); + } + } + + private KafkaAvroSerializer serializer; + private Deserializer valueDeserializer; + private Serializer valueSerializer; + private BigQuery bigQuery; + + @BeforeMethod + void beforeTest() { + final MockSchemaRegistryClient schemaRegistryClient = new MockSchemaRegistryClient(); + final String schemaRegistryUrl = "http://schema-registry/"; + serializer = new KafkaAvroSerializer(schemaRegistryClient, ImmutableMap.of(KafkaAvroSerializerConfig.SCHEMA_REGISTRY_URL_CONFIG, schemaRegistryUrl)); + serializer.configure(Collections.singletonMap("schema.registry.url", "http://schema-registry"), false); + valueDeserializer = new KafkaDeserializer(new KafkaAvroDeserializer(schemaRegistryClient)); + valueSerializer = new KafkaSerializer(serializer); + bigQuery = mock(BigQuery.class); + } + + private void sendEvents(final BigqueryTransportProvider transportProvider, final String destination, + final String cluster, final String topicName, final int partition, + final List events, final DateTime eventTimestamp, + final SendCallback onComplete) { + sendEvents(transportProvider, destination, + events.stream().map(event -> createRecord(cluster, topicName, offsetIncrement.getAndIncrement(), partition, event, eventTimestamp, null)) + .collect(Collectors.toList()), + onComplete); + } + + + private void sendEvents(final BigqueryTransportProvider transportProvider, final String destination, + final List records, + final SendCallback onComplete) { + final CountDownLatch latch = new CountDownLatch(records.size()); + records.stream().forEachOrdered(record -> transportProvider.send(destination, record, ((metadata, exception) -> { + if (onComplete != null) { + onComplete.onCompletion(metadata, exception); + } + latch.countDown(); + }))); + try { + while (!latch.await(500, TimeUnit.MILLISECONDS)) { + transportProvider.flush(); + } + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + } + + private final AtomicLong offsetIncrement = new AtomicLong(); + + private DatastreamProducerRecord createRecord(final String cluster, final String topicName, + final long offset, final int partition, final GenericRecord event, + final DateTime eventTimestamp, final String sourceCheckpoint) { + final DatastreamProducerRecordBuilder builder = new DatastreamProducerRecordBuilder(); + builder.setEventsSourceTimestamp(eventTimestamp.getValue()); + if (sourceCheckpoint != null) { + builder.setSourceCheckpoint(sourceCheckpoint); + } + + final BrooklinEnvelope envelope = new BrooklinEnvelope(null, serializer.serialize(topicName, event), null, ImmutableMap.of( + KAFKA_ORIGIN_CLUSTER, cluster, + KAFKA_ORIGIN_TOPIC, topicName, + KAFKA_ORIGIN_PARTITION, String.valueOf(partition), + KAFKA_ORIGIN_OFFSET, String.valueOf(offset), + BrooklinEnvelopeMetadataConstants.EVENT_TIMESTAMP, String.valueOf(eventTimestamp.getValue()) + )); + builder.addEvent(envelope); + return builder.build(); + } + + private String getUniqueDatasetName() { + return "testDataset_" + DATASET_COUNTER.incrementAndGet(); + } + + private String getUniqueTopicName() { + return "testTopic_" + TOPIC_COUNTER.incrementAndGet(); + } +} diff --git a/datastream-bigquery/src/test/java/com/linkedin/datastream/bigquery/schema/DynamicBigquerySchemaEvolverTests.java b/datastream-bigquery/src/test/java/com/linkedin/datastream/bigquery/schema/DynamicBigquerySchemaEvolverTests.java new file mode 100644 index 000000000..ad59c918b --- /dev/null +++ b/datastream-bigquery/src/test/java/com/linkedin/datastream/bigquery/schema/DynamicBigquerySchemaEvolverTests.java @@ -0,0 +1,506 @@ +/** + * Copyright 2020 Wayfair LLC. All rights reserved. + * Licensed under the BSD 2-Clause License. See the LICENSE file in the project root for license information. + * See the NOTICE file in the project root for additional information regarding copyright ownership. + */ +package com.linkedin.datastream.bigquery.schema; + +import com.google.cloud.bigquery.Field; +import com.google.cloud.bigquery.Schema; +import com.google.cloud.bigquery.StandardSQLTypeName; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Test; + +import java.io.IOException; +import java.io.InputStream; +import java.util.Optional; + +import com.linkedin.datastream.bigquery.translator.SchemaTranslator; + +import static org.testng.Assert.assertEquals; + +@Test +public class DynamicBigquerySchemaEvolverTests { + + private static DynamicBigquerySchemaEvolver schemaEvolver = null; + + @BeforeClass + public void beforeTests() { + schemaEvolver = new DynamicBigquerySchemaEvolver(); + } + + + @Test + public void testMergeSameFields() { + final Field baseField = requiredFieldOf("string", StandardSQLTypeName.STRING); + final Field newField = baseField.toBuilder().build(); + assertEquals(baseField, schemaEvolver.mergeFields(baseField, newField)); + } + + @Test + public void testMergeSameFieldWithModeChangedToNullable() { + final Field baseField = requiredFieldOf("string", StandardSQLTypeName.STRING); + final Field newField = baseField.toBuilder().setMode(Field.Mode.NULLABLE).build(); + assertEquals(schemaEvolver.mergeFields(baseField, newField), newField); + } + + @Test + public void testMergeSameFieldsWithModeKeptAsNullable() { + final Field baseField = Field.newBuilder("string", StandardSQLTypeName.STRING).setMode(Field.Mode.NULLABLE).build(); + final Field newField = requiredFieldOf(baseField.getName(), baseField.getType().getStandardType()); + assertEquals(schemaEvolver.mergeFields(baseField, newField), baseField); + } + + @DataProvider(name = "compatible type coercions") + public Object[][] compatibleTypeCoercionsDataProvider() { + return new Object[][] { + {StandardSQLTypeName.INT64, StandardSQLTypeName.NUMERIC}, + {StandardSQLTypeName.INT64, StandardSQLTypeName.FLOAT64}, + {StandardSQLTypeName.NUMERIC, StandardSQLTypeName.FLOAT64}, + }; + } + + @Test(dataProvider = "compatible type coercions") + public void testCompatibleTypeCoercions(final StandardSQLTypeName newType, final StandardSQLTypeName baseType) { + final Field newField = Field.of("field", newType); + final Field baseField = Field.of("field", baseType); + assertEquals(schemaEvolver.mergeFields(baseField, newField), baseField); + } + + + @DataProvider(name = "incompatible type coercions") + public Object[][] incompatibleTypeCoercionsDataProvider() { + return new Object[][] { + {StandardSQLTypeName.INT64, StandardSQLTypeName.STRING}, + {StandardSQLTypeName.FLOAT64, StandardSQLTypeName.INT64}, + {StandardSQLTypeName.FLOAT64, StandardSQLTypeName.NUMERIC}, + {StandardSQLTypeName.FLOAT64, StandardSQLTypeName.STRING}, + {StandardSQLTypeName.NUMERIC, StandardSQLTypeName.INT64}, + {StandardSQLTypeName.NUMERIC, StandardSQLTypeName.STRING}, + }; + } + + @Test(dataProvider = "incompatible type coercions", expectedExceptions = IncompatibleSchemaEvolutionException.class, expectedExceptionsMessageRegExp = "Cannot coerce type .+ into .+") + public void testIncompatibleTypeCoercions(final StandardSQLTypeName newType, final StandardSQLTypeName baseType) { + final Field newField = Field.of("field", newType); + final Field baseField = Field.of("field", baseType); + schemaEvolver.mergeFields(baseField, newField); + } + + @DataProvider(name = "compatible mode evolutions") + public Object[][] compatibleModeEvolutionDataProvider() { + return new Object[][] { + {null, null, null}, + {null, Field.Mode.NULLABLE, null}, + {null, Field.Mode.REQUIRED, null}, + {Field.Mode.NULLABLE, Field.Mode.REQUIRED, Field.Mode.NULLABLE}, + {Field.Mode.NULLABLE, null, Field.Mode.NULLABLE}, + {Field.Mode.REQUIRED, Field.Mode.REQUIRED, Field.Mode.REQUIRED}, + {Field.Mode.REQUIRED, Field.Mode.NULLABLE, Field.Mode.NULLABLE}, + {Field.Mode.REQUIRED, null, Field.Mode.NULLABLE}, + {Field.Mode.REPEATED, Field.Mode.REPEATED, Field.Mode.REPEATED}, + {Field.Mode.REPEATED, Field.Mode.NULLABLE, Field.Mode.REPEATED}, + {Field.Mode.REPEATED, null, Field.Mode.REPEATED}, + }; + } + + @Test(dataProvider = "compatible mode evolutions") + public void compatibleModeEvolutions(final Field.Mode baseMode, final Field.Mode newMode, final Field.Mode expectedMode) { + final Field.Builder baseFieldBuilder = Field.newBuilder("field", StandardSQLTypeName.STRING); + Optional.ofNullable(baseMode).ifPresent(baseFieldBuilder::setMode); + final Field baseField = baseFieldBuilder.build(); + + final Field.Builder newFieldBuilder = Field.newBuilder("field", StandardSQLTypeName.STRING); + Optional.ofNullable(newMode).ifPresent(newFieldBuilder::setMode); + final Field newField = newFieldBuilder.build(); + + assertEquals(schemaEvolver.mergeFields(baseField, newField).getMode(), expectedMode); + } + + @DataProvider(name = "incompatible mode evolutions") + public Object[][] incompatibleModeEvolutionDataProvider() { + return new Object[][] { + {null, Field.Mode.REPEATED}, + {Field.Mode.NULLABLE, Field.Mode.REPEATED}, + {Field.Mode.REQUIRED, Field.Mode.REPEATED}, + }; + } + + @Test(dataProvider = "incompatible mode evolutions", expectedExceptions = IncompatibleSchemaEvolutionException.class, expectedExceptionsMessageRegExp = "Cannot change field mode from .+ to .+") + public void incompatibleModeEvolutions(final Field.Mode baseMode, final Field.Mode newMode) { + final Field.Builder baseFieldBuilder = Field.newBuilder("field", StandardSQLTypeName.STRING); + Optional.ofNullable(baseMode).ifPresent(baseFieldBuilder::setMode); + final Field baseField = baseFieldBuilder.build(); + + final Field.Builder newFieldBuilder = Field.newBuilder("field", StandardSQLTypeName.STRING); + Optional.ofNullable(newMode).ifPresent(newFieldBuilder::setMode); + final Field newField = newFieldBuilder.build(); + + schemaEvolver.mergeFields(baseField, newField); + } + + @Test + public void testMergeNewField() { + final Field newField = Field.of("string", StandardSQLTypeName.STRING); + assertEquals(newField.toBuilder().setMode(Field.Mode.NULLABLE).build(), + schemaEvolver.mergeFields(null, newField)); + } + + @Test + public void testRemoveField() { + final Field baseField = Field.of("string", StandardSQLTypeName.STRING); + assertEquals(baseField.toBuilder().setMode(Field.Mode.NULLABLE).build(), + schemaEvolver.mergeFields(baseField, null)); + } + + @Test(expectedExceptions = IllegalArgumentException.class, expectedExceptionsMessageRegExp = "baseField and newField cannot both be null") + public void testNullFieldsException() { + schemaEvolver.mergeFields((Field)null, null); + } + + @Test(expectedExceptions = IncompatibleSchemaEvolutionException.class, expectedExceptionsMessageRegExp = "Cannot coerce type .+ into .+") + public void testFieldTypeMissmatchException() { + final Field baseField = Field.of("string", StandardSQLTypeName.STRING); + final Field newField = baseField.toBuilder().setType(StandardSQLTypeName.INT64).build(); + assertEquals(baseField, schemaEvolver.mergeFields(baseField, newField)); + } + + @Test(expectedExceptions = IllegalArgumentException.class, expectedExceptionsMessageRegExp = ".+ and .+ names do not match") + public void testFieldNameMissmatchException() { + final Field baseField = Field.of("string1", StandardSQLTypeName.STRING); + final Field newField = baseField.toBuilder().setName("string2").build(); + schemaEvolver.mergeFields(baseField, newField); + } + + @Test + public void testAddNewSchemaField() { + final Schema baseSchema = Schema.of( + Field.newBuilder("string", StandardSQLTypeName.STRING).setMode(Field.Mode.REQUIRED).build(), + Field.newBuilder("int", StandardSQLTypeName.INT64).setMode(Field.Mode.REQUIRED).build() + ); + final Schema newSchema = Schema.of( + Field.newBuilder("string", StandardSQLTypeName.STRING).setMode(Field.Mode.REQUIRED).build(), + Field.newBuilder("int", StandardSQLTypeName.INT64).setMode(Field.Mode.REQUIRED).build(), + Field.newBuilder("new_string", StandardSQLTypeName.STRING).setMode(Field.Mode.REQUIRED).build() + ); + final Schema evolvedSchema = schemaEvolver.evolveSchema(baseSchema, newSchema); + assertEquals(Schema.of( + Field.newBuilder("string", StandardSQLTypeName.STRING).setMode(Field.Mode.REQUIRED).build(), + Field.newBuilder("int", StandardSQLTypeName.INT64).setMode(Field.Mode.REQUIRED).build(), + Field.newBuilder("new_string", StandardSQLTypeName.STRING).setMode(Field.Mode.NULLABLE).build() + ), evolvedSchema); + } + + @Test + public void testAddNewSchemaFieldWithRequiredNestedStructure() { + final Schema baseSchema = Schema.of( + Field.newBuilder("string", StandardSQLTypeName.STRING).setMode(Field.Mode.REQUIRED).build(), + Field.newBuilder("int", StandardSQLTypeName.INT64).setMode(Field.Mode.REQUIRED).build() + ); + final Schema newSchema = Schema.of( + Field.newBuilder("string", StandardSQLTypeName.STRING).setMode(Field.Mode.REQUIRED).build(), + Field.newBuilder("int", StandardSQLTypeName.INT64).setMode(Field.Mode.REQUIRED).build(), + Field.newBuilder("new_nested", StandardSQLTypeName.STRUCT, + Field.newBuilder("nested_string", StandardSQLTypeName.STRING).setMode(Field.Mode.REQUIRED).build() + ).setMode(Field.Mode.REQUIRED).build() + ); + final Schema evolvedSchema = schemaEvolver.evolveSchema(baseSchema, newSchema); + assertEquals(Schema.of( + Field.newBuilder("string", StandardSQLTypeName.STRING).setMode(Field.Mode.REQUIRED).build(), + Field.newBuilder("int", StandardSQLTypeName.INT64).setMode(Field.Mode.REQUIRED).build(), + Field.newBuilder("new_nested", StandardSQLTypeName.STRUCT, + Field.newBuilder("nested_string", StandardSQLTypeName.STRING).setMode(Field.Mode.REQUIRED).build() + ).setMode(Field.Mode.NULLABLE).build() + ), evolvedSchema); + } + + @Test + public void testAddNewNestedSchemaField() { + final Schema baseSchema = Schema.of( + Field.newBuilder("string", StandardSQLTypeName.STRING).setMode(Field.Mode.REQUIRED).build(), + Field.newBuilder("int", StandardSQLTypeName.INT64).setMode(Field.Mode.REQUIRED).build(), + Field.newBuilder("nested", StandardSQLTypeName.STRUCT, + Field.newBuilder("nested_string", StandardSQLTypeName.STRING).setMode(Field.Mode.REQUIRED).build() + ).setMode(Field.Mode.REQUIRED).build() + ); + final Schema newSchema = Schema.of( + Field.newBuilder("string", StandardSQLTypeName.STRING).setMode(Field.Mode.REQUIRED).build(), + Field.newBuilder("int", StandardSQLTypeName.INT64).setMode(Field.Mode.REQUIRED).build(), + Field.newBuilder("nested", StandardSQLTypeName.STRUCT, + Field.newBuilder("nested_string", StandardSQLTypeName.STRING).setMode(Field.Mode.REQUIRED).build(), + Field.newBuilder("new_nested_int", StandardSQLTypeName.INT64).setMode(Field.Mode.REQUIRED).build() + ).setMode(Field.Mode.REQUIRED).build() + ); + final Schema evolvedSchema = schemaEvolver.evolveSchema(baseSchema, newSchema); + assertEquals(evolvedSchema, Schema.of( + Field.newBuilder("string", StandardSQLTypeName.STRING).setMode(Field.Mode.REQUIRED).build(), + Field.newBuilder("int", StandardSQLTypeName.INT64).setMode(Field.Mode.REQUIRED).build(), + Field.newBuilder("nested", StandardSQLTypeName.STRUCT, + Field.newBuilder("nested_string", StandardSQLTypeName.STRING).setMode(Field.Mode.REQUIRED).build(), + Field.newBuilder("new_nested_int", StandardSQLTypeName.INT64).setMode(Field.Mode.NULLABLE).build() + ).setMode(Field.Mode.REQUIRED).build() + )); + } + + @Test + public void testRemoveSchemaField() { + final Schema baseSchema = Schema.of( + Field.newBuilder("string", StandardSQLTypeName.STRING).setMode(Field.Mode.REQUIRED).build(), + Field.newBuilder("int", StandardSQLTypeName.INT64).setMode(Field.Mode.REQUIRED).build(), + Field.newBuilder("old_string", StandardSQLTypeName.STRING).setMode(Field.Mode.REQUIRED).build() + ); + final Schema newSchema = Schema.of( + Field.newBuilder("string", StandardSQLTypeName.STRING).setMode(Field.Mode.REQUIRED).build(), + Field.newBuilder("int", StandardSQLTypeName.INT64).setMode(Field.Mode.REQUIRED).build() + ); + final Schema evolvedSchema = schemaEvolver.evolveSchema(baseSchema, newSchema); + assertEquals(evolvedSchema, Schema.of( + Field.newBuilder("string", StandardSQLTypeName.STRING).setMode(Field.Mode.REQUIRED).build(), + Field.newBuilder("int", StandardSQLTypeName.INT64).setMode(Field.Mode.REQUIRED).build(), + Field.newBuilder("old_string", StandardSQLTypeName.STRING).setMode(Field.Mode.NULLABLE).build() + )); + } + + @Test + public void testRemoveMiddleSchemaField() { + final Schema baseSchema = Schema.of( + Field.newBuilder("string", StandardSQLTypeName.STRING).setMode(Field.Mode.REQUIRED).build(), + Field.newBuilder("int", StandardSQLTypeName.INT64).setMode(Field.Mode.REQUIRED).build(), + Field.newBuilder("old_string", StandardSQLTypeName.STRING).setMode(Field.Mode.REQUIRED).build() + ); + final Schema newSchema = Schema.of( + Field.newBuilder("string", StandardSQLTypeName.STRING).setMode(Field.Mode.REQUIRED).build(), + Field.newBuilder("old_string", StandardSQLTypeName.STRING).setMode(Field.Mode.REQUIRED).build() + ); + final Schema evolvedSchema = schemaEvolver.evolveSchema(baseSchema, newSchema); + assertEquals(evolvedSchema, Schema.of( + Field.newBuilder("string", StandardSQLTypeName.STRING).setMode(Field.Mode.REQUIRED).build(), + Field.newBuilder("int", StandardSQLTypeName.INT64).setMode(Field.Mode.NULLABLE).build(), + Field.newBuilder("old_string", StandardSQLTypeName.STRING).setMode(Field.Mode.REQUIRED).build() + )); + } + + @Test + public void testRemoveSchemaFieldWithRequiredNestedStructure() { + final Schema baseSchema = Schema.of( + Field.newBuilder("string", StandardSQLTypeName.STRING).setMode(Field.Mode.REQUIRED).build(), + Field.newBuilder("int", StandardSQLTypeName.INT64).setMode(Field.Mode.REQUIRED).build(), + Field.newBuilder("nested", StandardSQLTypeName.STRUCT, + Field.newBuilder("nested_string", StandardSQLTypeName.STRING).setMode(Field.Mode.REQUIRED).build(), + Field.newBuilder("nested_int", StandardSQLTypeName.INT64).setMode(Field.Mode.REQUIRED).build() + ).setMode(Field.Mode.REQUIRED).build() + ); + final Schema newSchema = Schema.of( + Field.newBuilder("string", StandardSQLTypeName.STRING).setMode(Field.Mode.REQUIRED).build(), + Field.newBuilder("int", StandardSQLTypeName.INT64).setMode(Field.Mode.REQUIRED).build() + ); + final Schema evolvedSchema = schemaEvolver.evolveSchema(baseSchema, newSchema); + assertEquals(evolvedSchema, Schema.of( + Field.newBuilder("string", StandardSQLTypeName.STRING).setMode(Field.Mode.REQUIRED).build(), + Field.newBuilder("int", StandardSQLTypeName.INT64).setMode(Field.Mode.REQUIRED).build(), + Field.newBuilder("nested", StandardSQLTypeName.STRUCT, + Field.newBuilder("nested_string", StandardSQLTypeName.STRING).setMode(Field.Mode.REQUIRED).build(), + Field.newBuilder("nested_int", StandardSQLTypeName.INT64).setMode(Field.Mode.REQUIRED).build() + ).setMode(Field.Mode.NULLABLE).build() + )); + } + + @Test + public void testRemoveNestedSchemaField() { + final Schema baseSchema = Schema.of( + Field.newBuilder("string", StandardSQLTypeName.STRING).setMode(Field.Mode.REQUIRED).build(), + Field.newBuilder("int", StandardSQLTypeName.INT64).setMode(Field.Mode.REQUIRED).build(), + Field.newBuilder("nested", StandardSQLTypeName.STRUCT, + Field.newBuilder("nested_string", StandardSQLTypeName.STRING).setMode(Field.Mode.REQUIRED).build(), + Field.newBuilder("nested_int", StandardSQLTypeName.INT64).setMode(Field.Mode.REQUIRED).build() + ).setMode(Field.Mode.REQUIRED).build() + ); + final Schema newSchema = Schema.of( + Field.newBuilder("string", StandardSQLTypeName.STRING).setMode(Field.Mode.REQUIRED).build(), + Field.newBuilder("int", StandardSQLTypeName.INT64).setMode(Field.Mode.REQUIRED).build(), + Field.newBuilder("nested", StandardSQLTypeName.STRUCT, + Field.newBuilder("nested_int", StandardSQLTypeName.INT64).setMode(Field.Mode.REQUIRED).build() + ).setMode(Field.Mode.REQUIRED).build() + ); + final Schema evolvedSchema = schemaEvolver.evolveSchema(baseSchema, newSchema); + assertEquals(evolvedSchema, Schema.of( + Field.newBuilder("string", StandardSQLTypeName.STRING).setMode(Field.Mode.REQUIRED).build(), + Field.newBuilder("int", StandardSQLTypeName.INT64).setMode(Field.Mode.REQUIRED).build(), + Field.newBuilder("nested", StandardSQLTypeName.STRUCT, + Field.newBuilder("nested_string", StandardSQLTypeName.STRING).setMode(Field.Mode.NULLABLE).build(), + Field.newBuilder("nested_int", StandardSQLTypeName.INT64).setMode(Field.Mode.REQUIRED).build() + ).setMode(Field.Mode.REQUIRED).build() + )); + } + + @Test + public void testSchemaFieldModeChangedToNullable() { + final Schema baseSchema = Schema.of( + Field.newBuilder("string", StandardSQLTypeName.STRING).setMode(Field.Mode.REQUIRED).build(), + Field.newBuilder("int", StandardSQLTypeName.INT64).setMode(Field.Mode.REQUIRED).build() + ); + final Schema newSchema = Schema.of( + Field.newBuilder("string", StandardSQLTypeName.STRING).setMode(Field.Mode.REQUIRED).build(), + Field.newBuilder("int", StandardSQLTypeName.INT64).setMode(Field.Mode.NULLABLE).build() + ); + final Schema evolvedSchema = schemaEvolver.evolveSchema(baseSchema, newSchema); + assertEquals(evolvedSchema, newSchema); + } + + @Test + public void testSchemaFieldModeChangedToRequiredIsIgnored() { + final Schema baseSchema = Schema.of( + Field.of("string", StandardSQLTypeName.STRING), + Field.newBuilder("int", StandardSQLTypeName.INT64).setMode(Field.Mode.NULLABLE).build() + ); + final Schema newSchema = Schema.of( + Field.of("string", StandardSQLTypeName.STRING), + Field.of("int", StandardSQLTypeName.INT64) + ); + final Schema evolvedSchema = schemaEvolver.evolveSchema(baseSchema, newSchema); + assertEquals(baseSchema, evolvedSchema); + } + + @Test + public void testSchemaFieldOrderingNotChangedOnAdd() { + final Schema baseSchema = Schema.of( + Field.of("int", StandardSQLTypeName.INT64), + Field.of("string", StandardSQLTypeName.STRING) + ); + final Schema newSchema = Schema.of( + Field.of("string", StandardSQLTypeName.STRING), + Field.of("newInt", StandardSQLTypeName.INT64), + Field.of("int", StandardSQLTypeName.INT64) + ); + final Schema evolvedSchema = schemaEvolver.evolveSchema(baseSchema, newSchema); + assertEquals(evolvedSchema, Schema.of( + Field.of("int", StandardSQLTypeName.INT64), + Field.of("string", StandardSQLTypeName.STRING), + Field.newBuilder("newInt", StandardSQLTypeName.INT64).setMode(Field.Mode.NULLABLE).build() + )); + } + + @Test + public void testSchemaFieldOrderingNotChangedOnAddNested() { + final Schema baseSchema = Schema.of( + Field.of("int", StandardSQLTypeName.INT64), + Field.of("nested", StandardSQLTypeName.STRUCT, + Field.of("nested_int", StandardSQLTypeName.INT64), + Field.of("nested_string", StandardSQLTypeName.STRING) + ), + Field.of("string", StandardSQLTypeName.STRING) + ); + final Schema newSchema = Schema.of( + Field.of("string", StandardSQLTypeName.STRING), + Field.of("newInt", StandardSQLTypeName.INT64), + Field.of("nested", StandardSQLTypeName.STRUCT, + Field.of("nested_int", StandardSQLTypeName.INT64), + Field.of("new_nested_string", StandardSQLTypeName.STRING), + Field.of("nested_string", StandardSQLTypeName.STRING) + ), + Field.of("int", StandardSQLTypeName.INT64) + ); + final Schema evolvedSchema = schemaEvolver.evolveSchema(baseSchema, newSchema); + assertEquals(evolvedSchema, Schema.of( + Field.of("int", StandardSQLTypeName.INT64), + Field.of("nested", StandardSQLTypeName.STRUCT, + Field.of("nested_int", StandardSQLTypeName.INT64), + Field.of("nested_string", StandardSQLTypeName.STRING), + Field.newBuilder("new_nested_string", StandardSQLTypeName.STRING).setMode(Field.Mode.NULLABLE).build() + ), + Field.of("string", StandardSQLTypeName.STRING), + Field.newBuilder("newInt", StandardSQLTypeName.INT64).setMode(Field.Mode.NULLABLE).build() + )); + } + + @Test + public void testSchemaFieldOrderingNotChangedOnRemove() { + final Schema baseSchema = Schema.of( + Field.newBuilder("int", StandardSQLTypeName.INT64).setMode(Field.Mode.REQUIRED).build(), + Field.newBuilder("old_string", StandardSQLTypeName.STRING).setMode(Field.Mode.REQUIRED).build(), + Field.newBuilder("string", StandardSQLTypeName.STRING).setMode(Field.Mode.REQUIRED).build() + ); + final Schema newSchema = Schema.of( + Field.newBuilder("string", StandardSQLTypeName.STRING).setMode(Field.Mode.REQUIRED).build(), + Field.newBuilder("int", StandardSQLTypeName.INT64).setMode(Field.Mode.REQUIRED).build() + ); + final Schema evolvedSchema = schemaEvolver.evolveSchema(baseSchema, newSchema); + assertEquals(evolvedSchema, Schema.of( + Field.newBuilder("int", StandardSQLTypeName.INT64).setMode(Field.Mode.REQUIRED).build(), + Field.newBuilder("old_string", StandardSQLTypeName.STRING).setMode(Field.Mode.NULLABLE).build(), + Field.newBuilder("string", StandardSQLTypeName.STRING).setMode(Field.Mode.REQUIRED).build() + )); + } + + @Test + public void testSchemaFieldOrderingNotChangedOnRemoveNested() { + final Schema baseSchema = Schema.of( + Field.newBuilder("int", StandardSQLTypeName.INT64).setMode(Field.Mode.REQUIRED).build(), + Field.newBuilder("nested", StandardSQLTypeName.STRUCT, + Field.newBuilder("nested_string", StandardSQLTypeName.STRING).setMode(Field.Mode.REQUIRED).build(), + Field.newBuilder("nested_int", StandardSQLTypeName.INT64).setMode(Field.Mode.REQUIRED).build(), + Field.newBuilder("old_nested_string", StandardSQLTypeName.STRING).setMode(Field.Mode.REQUIRED).build() + ).setMode(Field.Mode.REQUIRED).build(), + Field.newBuilder("string", StandardSQLTypeName.STRING).setMode(Field.Mode.REQUIRED).build() + ); + final Schema newSchema = Schema.of( + Field.newBuilder("nested", StandardSQLTypeName.STRUCT, + Field.newBuilder("nested_int", StandardSQLTypeName.INT64).setMode(Field.Mode.REQUIRED).build(), + Field.newBuilder("nested_string", StandardSQLTypeName.STRING).setMode(Field.Mode.REQUIRED).build() + ).setMode(Field.Mode.REQUIRED).build(), + Field.newBuilder("string", StandardSQLTypeName.STRING).setMode(Field.Mode.REQUIRED).build(), + Field.newBuilder("int", StandardSQLTypeName.INT64).setMode(Field.Mode.REQUIRED).build() + ); + final Schema evolvedSchema = schemaEvolver.evolveSchema(baseSchema, newSchema); + assertEquals(evolvedSchema, Schema.of( + Field.newBuilder("int", StandardSQLTypeName.INT64).setMode(Field.Mode.REQUIRED).build(), + Field.newBuilder("nested", StandardSQLTypeName.STRUCT, + Field.newBuilder("nested_string", StandardSQLTypeName.STRING).setMode(Field.Mode.REQUIRED).build(), + Field.newBuilder("nested_int", StandardSQLTypeName.INT64).setMode(Field.Mode.REQUIRED).build(), + Field.newBuilder("old_nested_string", StandardSQLTypeName.STRING).setMode(Field.Mode.NULLABLE).build() + ).setMode(Field.Mode.REQUIRED).build(), + Field.newBuilder("string", StandardSQLTypeName.STRING).setMode(Field.Mode.REQUIRED).build() + )); + } + + @Test + public void testEvolveComplexSchema() throws IOException { + org.apache.avro.Schema avroSchema; + try (final InputStream avroFileInputStream = getClass().getClassLoader().getResourceAsStream("complex_avro_schema.avsc")) { + avroSchema = new org.apache.avro.Schema.Parser().parse(avroFileInputStream); + } + final Schema baseSchema = SchemaTranslator.translate(avroSchema); + final Schema newSchema = Schema.of( + Field.newBuilder("test", StandardSQLTypeName.STRING).setMode(Field.Mode.REQUIRED).build() + ); + final Schema evolvedSchema = schemaEvolver.evolveSchema(baseSchema, newSchema); + assertEquals(evolvedSchema.getFields().size(), baseSchema.getFields().size() + 1); + for (int i = 0; i < baseSchema.getFields().size(); i++) { + final Field baseField = baseSchema.getFields().get(i); + final Field evolvedField = evolvedSchema.getFields().get(i); + assertFieldsEqual(baseField, evolvedField); + } + assertEquals( + evolvedSchema.getFields().get(evolvedSchema.getFields().size() - 1), + Field.newBuilder("test", StandardSQLTypeName.STRING).setMode(Field.Mode.NULLABLE).build() + ); + } + + private static void assertFieldsEqual(final Field baseField, final Field evolvedField) { + assertEquals(evolvedField.getName(), baseField.getName()); + assertEquals(evolvedField.getType(), baseField.getType()); + if (baseField.getSubFields() != null && !baseField.getSubFields().isEmpty()) { + assertEquals(evolvedField.getSubFields().size(), baseField.getSubFields().size()); + for (int i = 0; i < baseField.getSubFields().size(); i++) { + final Field baseSubField = baseField.getSubFields().get(i); + final Field evolvedSubField = evolvedField.getSubFields().get(i); + assertFieldsEqual(baseSubField, evolvedSubField); + } + } + } + + private static Field requiredFieldOf(final String name, final StandardSQLTypeName type) { + return Field.newBuilder(name, type).setMode(Field.Mode.REQUIRED).build(); + } +} diff --git a/datastream-bigquery/src/test/java/com/linkedin/datastream/bigquery/translator/SchemaTranslatorTests.java b/datastream-bigquery/src/test/java/com/linkedin/datastream/bigquery/translator/SchemaTranslatorTests.java new file mode 100644 index 000000000..9cf3dc1c7 --- /dev/null +++ b/datastream-bigquery/src/test/java/com/linkedin/datastream/bigquery/translator/SchemaTranslatorTests.java @@ -0,0 +1,48 @@ +/* + * Copyright 2020 Wayfair LLC. All rights reserved. + * Licensed under the BSD 2-Clause License. See the LICENSE file in the project root for license information. + * See the NOTICE file in the project root for additional information regarding copyright ownership. + */ + +package com.linkedin.datastream.bigquery.translator; + +import java.io.IOException; +import java.io.InputStream; + +import org.testng.annotations.Test; + +import static org.testng.Assert.assertNotNull; + +@Test +public class SchemaTranslatorTests { + + @Test + public void testLoadComplexSchema() throws IOException { + org.apache.avro.Schema avroSchema; + try (final InputStream avroFileInputStream = getClass().getClassLoader().getResourceAsStream("complex_avro_schema.avsc")) { + avroSchema = new org.apache.avro.Schema.Parser().parse(avroFileInputStream); + } + final com.google.cloud.bigquery.Schema bqSchema = SchemaTranslator.translate(avroSchema); + assertNotNull(bqSchema); + } + + @Test(expectedExceptions = StackOverflowError.class) + public void testRecursiveSchemaStackOverflowException() { + final String recursiveSchema = "{" + + " \"type\": \"record\"," + + " \"name\": \"TreeNode\"," + + " \"fields\": [" + + " {" + + " \"name\": \"value\"," + + " \"type\": \"long\"" + + " }," + + " {" + + " \"name\": \"children\"," + + " \"type\": { \"type\": \"array\", \"items\": \"TreeNode\" }" + + " }" + + " ]" + + "}"; + final org.apache.avro.Schema avroSchema = new org.apache.avro.Schema.Parser().parse(recursiveSchema); + SchemaTranslator.translate(avroSchema); + } +} diff --git a/datastream-bigquery/src/test/resources/complex_avro_schema.avsc b/datastream-bigquery/src/test/resources/complex_avro_schema.avsc new file mode 100644 index 000000000..bfffcab13 --- /dev/null +++ b/datastream-bigquery/src/test/resources/complex_avro_schema.avsc @@ -0,0 +1,1116 @@ +{ + "type": "record", + "name": "unionTypeSchemaCheck", + "fields": [ + { + "name": "col_null", + "type": "null" + }, + { + "name": "col_boolean", + "type": "boolean", + "default": false + }, + { + "name": "col_int", + "type": "int", + "default": 10 + }, + { + "name": "col_long", + "type": "long", + "default": 100 + }, + { + "name": "col_float", + "type": "float", + "default": 10.1 + }, + { + "name": "col_double", + "type": "double", + "default": 100.1 + }, + { + "name": "col_bytes", + "type": "bytes", + "default": "abc" + }, + { + "name": "col_string", + "type": "string", + "default": "str" + }, + { + "name": "col_enum", + "type": { + "type": "enum", + "name": "gender", + "symbols": [ + "MALE", + "FEMALE" + ] + }, + "default": "MALE" + }, + { + "name": "col_fixed", + "type": { + "type": "fixed", + "name": "fixed_name", + "size": 2 + } + }, + { + "name": "col_array_boolean", + "type": { + "type": "array", + "items": "boolean" + }, + "default": [ + true, + false + ] + }, + { + "name": "col_array_int", + "type": { + "type": "array", + "items": "int" + }, + "default": [ + 1, + 2 + ] + }, + { + "name": "col_array_long", + "type": { + "type": "array", + "items": "long" + }, + "default": [ + 1, + 2 + ] + }, + { + "name": "col_array_float", + "type": { + "type": "array", + "items": "float" + }, + "default": [ + 1.1, + 2.2 + ] + }, + { + "name": "col_array_double", + "type": { + "type": "array", + "items": "double" + }, + "default": [ + 1.1, + 2.2 + ] + }, + { + "name": "col_array_bytes", + "type": { + "type": "array", + "items": "bytes" + } + }, + { + "name": "col_array_string", + "type": { + "type": "array", + "items": "string" + }, + "default": [ + "abc", + "bcd" + ] + }, + { + "name": "col_array_record", + "type": { + "type": "array", + "items": { + "type": "record", + "name": "inner_record", + "fields": [ + { + "name": "name", + "type": "string" + }, + { + "name": "surname", + "type": "string" + } + ] + } + } + }, + { + "name": "col_array_enum", + "type": { + "type": "array", + "items": { + "type": "enum", + "name": "enum_fruit", + "symbols": [ + "ORANGE", + "APPLE", + "PINEAPPLE" + ] + } + } + }, + { + "name": "col_array_map", + "type": { + "type": "array", + "items": { + "type": "map", + "values": "string" + } + } + }, + { + "name": "col_array_union", + "type": { + "type": "array", + "items": [ + "int", + "string" + ] + } + }, + { + "name": "col_array_opt_union", + "type": { + "type": "array", + "items": [ + "null", + "string" + ] + } + }, + { + "name": "col_array_fixed", + "type": { + "type": "array", + "items": { + "type": "fixed", + "name": "arr_fixed_name", + "size": 5 + } + } + }, + { + "name": "col_record_boolean", + "type": { + "type": "record", + "name": "record_bool", + "fields": [ + { + "name": "a_bool", + "type": "boolean", + "default": true + }, + { + "name": "b_bool", + "type": "boolean", + "default": true + } + ] + } + }, + { + "name": "col_record_int", + "type": { + "type": "record", + "name": "record_int", + "fields": [ + { + "name": "a_int", + "type": "int" + }, + { + "name": "b_int", + "type": "int" + } + ] + } + }, + { + "name": "col_record_long", + "type": { + "type": "record", + "name": "record_long", + "fields": [ + { + "name": "a_long", + "type": "long" + }, + { + "name": "b_long", + "type": "long" + } + ] + } + }, + { + "name": "col_record_float", + "type": { + "type": "record", + "name": "record_float", + "fields": [ + { + "name": "a_float", + "type": "float" + }, + { + "name": "b_float", + "type": "float" + } + ] + } + }, + { + "name": "col_record_double", + "type": { + "type": "record", + "name": "record_double", + "fields": [ + { + "name": "a_double", + "type": "double" + }, + { + "name": "b_double", + "type": "double" + } + ] + } + }, + { + "name": "col_record_bytes", + "type": { + "type": "record", + "name": "record_bytes", + "fields": [ + { + "name": "a_bytes", + "type": "bytes" + }, + { + "name": "b_bytes", + "type": "bytes" + } + ] + } + }, + { + "name": "col_record_string", + "type": { + "type": "record", + "name": "record_string", + "fields": [ + { + "name": "a_string", + "type": "string" + }, + { + "name": "b_string", + "type": "string" + } + ] + } + }, + { + "name": "col_record_record", + "type": { + "type": "record", + "name": "record_record", + "fields": [ + { + "name": "a_record", + "type": { + "type": "record", + "name": "a_record_record", + "fields": [ + { + "name": "a_record_record_string", + "type": "string" + }, + { + "name": "a_record_record_int", + "type": "int" + } + ] + } + }, + { + "name": "b_record", + "type": { + "type": "record", + "name": "b_record_record", + "fields": [ + { + "name": "b_record_record_string", + "type": "string" + }, + { + "name": "b_record_record_int", + "type": "int" + } + ] + } + } + ] + } + }, + { + "name": "col_record_enum", + "type": { + "type": "record", + "name": "record_enum", + "fields": [ + { + "name": "a_enum", + "type": { + "type": "enum", + "name": "enum_gender", + "symbols": [ + "MALE", + "FEMALE" + ] + } + }, + { + "name": "b_enum", + "type": { + "type": "enum", + "name": "enum_answer", + "symbols": [ + "YES", + "NO" + ] + } + } + ] + } + }, + { + "name": "col_record_array", + "type": { + "type": "record", + "name": "record_array", + "fields": [ + { + "name": "a_array", + "type": { + "type": "array", + "items": "int" + } + }, + { + "name": "b_array", + "type": { + "type": "array", + "items": "boolean" + } + } + ] + } + }, + { + "name": "col_record_map", + "type": { + "type": "record", + "name": "record_map", + "fields": [ + { + "name": "a_map", + "type": { + "type": "map", + "values": "int" + } + }, + { + "name": "b_map", + "type": { + "type": "map", + "values": "string" + } + } + ] + } + }, + { + "name": "col_record_union", + "type": { + "type": "record", + "name": "record_union", + "fields": [ + { + "name": "a_union", + "type": [ + "int", + "string" + ] + }, + { + "name": "b_union", + "type": [ + "boolean", + "string" + ] + } + ] + } + }, + { + "name": "col_record_fixed", + "type": { + "type": "record", + "name": "record_fixed", + "fields": [ + { + "name": "a_fixed", + "type": { + "type": "fixed", + "name": "a_fixed_name", + "size": 2 + } + } + ] + } + }, + { + "name": "col_map_boolean", + "type": { + "type": "map", + "values": "boolean" + } + }, + { + "name": "col_map_int", + "type": { + "type": "map", + "values": "int" + } + }, + { + "name": "col_map_long", + "type": { + "type": "map", + "values": "long" + } + }, + { + "name": "col_map_float", + "type": { + "type": "map", + "values": "float" + } + }, + { + "name": "col_map_double", + "type": { + "type": "map", + "values": "double" + } + }, + { + "name": "col_map_bytes", + "type": { + "type": "map", + "values": "bytes" + } + }, + { + "name": "col_map_string", + "type": { + "type": "map", + "values": "string" + } + }, + { + "name": "col_map_record", + "type": { + "type": "map", + "values": { + "type": "record", + "name": "map_record_name", + "fields": [ + { + "name": "map_record_int", + "type": "int" + }, + { + "name": "map_record_string", + "type": "string" + } + ] + } + } + }, + { + "name": "col_map_enum", + "type": { + "type": "map", + "values": { + "type": "enum", + "name": "map_enum_name", + "symbols": [ + "KID", + "ADULT", + "SENIOR" + ] + } + } + }, + { + "name": "col_map_array", + "type": { + "type": "map", + "values": { + "type": "array", + "items": "int" + } + } + }, + { + "name": "col_map_map", + "type": { + "type": "map", + "values": { + "type": "map", + "values": "int" + } + } + }, + { + "name": "col_map_union", + "type": { + "type": "map", + "values": [ + "string", + "boolean" + ] + } + }, + { + "name": "col_map_fixed", + "type": { + "type": "map", + "values": { + "type": "fixed", + "name": "map_fixed_name", + "size": 3 + } + } + }, + { + "name": "col_union_opt_bool", + "type": [ + "null", + "boolean" + ] + }, + { + "name": "col_union_opt_int", + "type": [ + "null", + "int" + ] + }, + { + "name": "col_union_opt_long", + "type": [ + "null", + "long" + ] + }, + { + "name": "col_union_opt_float", + "type": [ + "null", + "float" + ] + }, + { + "name": "col_union_opt_double", + "type": [ + "null", + "double" + ] + }, + { + "name": "col_union_opt_bytes", + "type": [ + "null", + "bytes" + ] + }, + { + "name": "col_union_opt_string", + "type": [ + "null", + "string" + ] + }, + { + "name": "col_union_opt_record", + "type": [ + "null", + { + "type": "record", + "name": "union_opt_record_name", + "fields": [ + { + "name": "rec_string", + "type": "string" + } + ] + } + ] + }, + { + "name": "col_union_opt_enum", + "type": [ + "null", + { + "type": "enum", + "name": "union_opt_enum_name", + "symbols": [ + "RED", + "GREEN", + "YELLOW" + ] + } + ] + }, + { + "name": "col_union_opt_array", + "type": [ + "null", + { + "type": "array", + "items": "int" + } + ] + }, + { + "name": "col_union_opt_map", + "type": [ + "null", + { + "type": "map", + "values": "string" + } + ] + }, + { + "name": "col_union_opt_fixed", + "type": [ + "null", + { + "type": "fixed", + "name": "union_opt_fixed_name", + "size": 5 + } + ] + }, + { + "name": "col_union_mult_primitive", + "type": [ + "boolean", + "int", + "long", + "float", + "double", + "bytes", + "string", + { + "type": "enum", + "name": "union_mult_enum_name", + "symbols": [ + "RED", + "GREEN", + "YELLOW" + ] + }, + { + "type": "fixed", + "name": "union_mult_fixed", + "size": 3 + } + ] + }, + { + "name": "col_union_map_array", + "type": [ + { + "type": "map", + "values": { + "type": "int", + "logicalType": "date" + } + }, + { + "type": "array", + "items": { + "type": "int", + "logicalType": "date" + } + } + ] + }, + { + "name": "col_union_array_record", + "type": [ + { + "type": "array", + "items": { + "type": "int", + "logicalType": "date" + } + }, + { + "type": "record", + "name": "union_record_name", + "fields": [ + { + "name": "union_rec_string", + "type": "string" + }, + { + "name": "nested_record", + "type": { + "type": "record", + "name": "union_nested2", + "fields": [ + { + "name": "nested2_record_double", + "type": "double" + }, + { + "name": "nested2_record_string", + "type": "string" + }, + { + "name": "nested_rec2", + "type": { + "type": "record", + "name": "union_nested", + "fields": [ + { + "name": "nested_record_double", + "type": "double" + }, + { + "name": "nested_record_string", + "type": "string" + } + ] + } + } + ] + } + } + ] + } + ] + }, + { + "name": "col_union_multi_fields", + "type": [ + { + "type": "map", + "values": { + "type": "int", + "logicalType": "date" + } + }, + { + "type": "array", + "items": { + "type": "int", + "logicalType": "date" + } + }, + { + "type": "record", + "name": "union_multi_record_name", + "fields": [ + { + "name": "union_multi_rec_string", + "type": "string" + }, + { + "name": "multi_nested_record", + "type": { + "type": "record", + "name": "nested2", + "fields": [ + { + "name": "nested2_record_double", + "type": "double" + }, + { + "name": "nested2_record_string", + "type": "string" + }, + { + "name": "nested_rec2", + "type": { + "type": "record", + "name": "nested", + "fields": [ + { + "name": "nested_record_double", + "type": "double" + }, + { + "name": "nested_record_string", + "type": "string" + } + ] + } + } + ] + } + } + ] + }, + { + "type": "string", + "logicalType": "uuid" + }, + { + "type": "int", + "logicalType": "date" + }, + { + "type": "long", + "logicalType": "timestamp-micros" + } + ] + }, + { + "name": "col_union_2_records", + "type": [ + { + "type": "record", + "name": "union_1_record_name", + "fields": [ + { + "name": "union_rec_1_string", + "type": "string" + } + ] + }, + { + "type": "record", + "name": "union_2_record_name", + "fields": [ + { + "name": "union_rec_2_string", + "type": "string" + } + ] + } + ] + }, + { + "name": "col_union_multi_records", + "type": [ + { + "type": "record", + "name": "union_m1_record_name", + "fields": [ + { + "name": "union_rec_m1_string", + "type": "string" + }, + { + "name": "union_rec_m1_enum", + "type": { + "type": "enum", + "name": "union_rec_m1_enum_name", + "symbols": [ + "RED", + "YELLOW", + "GREEN" + ] + } + } + ] + }, + { + "type": "record", + "name": "union_m2_record_name", + "fields": [ + { + "name": "union_rec_2_string", + "type": "string" + }, + { + "name": "union_rec_m2_enum", + "type": { + "type": "enum", + "name": "union_rec_m2_enum_name", + "symbols": [ + "RED", + "YELLOW", + "GREEN" + ] + } + } + ] + } + ] + }, + { + "name": "col_union_multi_enums_string", + "type": [ + { + "type": "enum", + "name": "union_enum_1_name", + "symbols": [ + "BLACK", + "WHITE" + ] + }, + { + "type": "enum", + "name": "union_enum_2_name", + "symbols": [ + "BLACK", + "WHITE" + ] + }, + { + "type": "enum", + "name": "union_enum_3_name", + "symbols": [ + "SILVER", + "BLACK", + "WHITE" + ] + }, + "string" + ] + }, + { + "name": "col_union_enum_record", + "type": [ + { + "type": "enum", + "name": "union_enum_name", + "symbols": [ + "BLACK", + "WHITE" + ] + }, + { + "type": "record", + "name": "union_record_a_name", + "fields": [ + { + "name": "union_rec_a_string", + "type": "string" + } + ] + } + ] + }, + { + "name": "col_union_opt_timestampmilli", + "type": [ + "int", + { + "type": "long", + "logicalType": "timestamp-millis" + } + ] + }, + { + "name": "date_with_logical_type", + "type": { + "type": "int", + "logicalType": "date" + } + }, + { + "name": "timemillis_with_logical_type", + "type": { + "type": "int", + "logicalType": "time-millis" + } + }, + { + "name": "timemicros_with_logical_type", + "type": { + "type": "long", + "logicalType": "time-micros" + } + }, + { + "name": "timestampmillis_with_logical_type", + "type": { + "type": "long", + "logicalType": "timestamp-millis" + } + }, + { + "name": "timestampmicros_with_logical_type", + "type": { + "type": "long", + "logicalType": "timestamp-micros" + } + }, + { + "name": "uuid_logical_type", + "type": { + "type": "string", + "logicalType": "uuid" + } + }, + { + "name": "timestamp_no_logical_type", + "type": "long" + }, + { + "name": "union_multi_logical", + "type": [ + "null", + { + "type": "int", + "logicalType": "time-millis" + }, + { + "type": "long", + "logicalType": "timestamp-millis" + } + ] + }, + { + "name": "union_multi_logical2", + "type": [ + "null", + { + "type": "long", + "logicalType": "time-micros" + }, + "string" + ] + } + ] +} diff --git a/datastream-bigquery/src/test/resources/invalid_defaults_schema.avsc b/datastream-bigquery/src/test/resources/invalid_defaults_schema.avsc new file mode 100644 index 000000000..938493588 --- /dev/null +++ b/datastream-bigquery/src/test/resources/invalid_defaults_schema.avsc @@ -0,0 +1,15 @@ +{ + "type": "record", + "name": "invalid_defaults", + "namespace": "test", + "fields": [ + { + "name": "amend_operation", + "type": [ + "null", + "string" + ], + "default": "null" + } + ] +} \ No newline at end of file diff --git a/datastream-bigquery/src/test/resources/invalid_field_name_schema.avsc b/datastream-bigquery/src/test/resources/invalid_field_name_schema.avsc new file mode 100644 index 000000000..383ae0d75 --- /dev/null +++ b/datastream-bigquery/src/test/resources/invalid_field_name_schema.avsc @@ -0,0 +1,14 @@ +{ + "type": "record", + "name": "invalid_field_names", + "namespace": "test", + "fields": [ + { + "name": "42_invalid_names", + "type": [ + "string" + ], + "default": "" + } + ] +} \ No newline at end of file diff --git a/datastream-client/src/main/java/com/linkedin/datastream/DatastreamRestClient.java b/datastream-client/src/main/java/com/linkedin/datastream/DatastreamRestClient.java index 41bca3ad6..eb8b91648 100644 --- a/datastream-client/src/main/java/com/linkedin/datastream/DatastreamRestClient.java +++ b/datastream-client/src/main/java/com/linkedin/datastream/DatastreamRestClient.java @@ -57,7 +57,7 @@ * Datastream REST Client */ public class DatastreamRestClient { - private static final Logger LOG = LoggerFactory.getLogger(DatastreamRestClient.class); + private static final Logger _logger = LoggerFactory.getLogger(DatastreamRestClient.class); @VisibleForTesting protected static final String DATASTREAM_UUID = "datastreamUUID"; @@ -101,7 +101,7 @@ public DatastreamRestClient(RestClient restClient, Properties config) { Validate.notNull(restClient, "null restClient"); _builders = new DatastreamRequestBuilders(); _restClient = restClient; - LOG.info("DatastreamRestClient created with retryPeriodMs={} retryTimeoutMs={}", _retryPeriodMs, _retryTimeoutMs); + _logger.info("DatastreamRestClient created with retryPeriodMs={} retryTimeoutMs={}", _retryPeriodMs, _retryTimeoutMs); } private long getRetryPeriodMs() { @@ -144,20 +144,20 @@ public Datastream getDatastream(String datastreamName) { } catch (RemoteInvocationException e) { // instanceof works for null as well if (ExceptionUtils.getRootCause(e) instanceof TimeoutException) { - LOG.warn("Timeout: getDatastream. May retry...", e); + _logger.warn("Timeout: getDatastream. May retry...", e); return null; } if (isNotFoundHttpStatus(e)) { - LOG.warn(String.format("Datastream {%s} is not found", datastreamName), e); + _logger.warn(String.format("Datastream {%s} is not found", datastreamName), e); throw new DatastreamNotFoundException(datastreamName, e); } else { String errorMessage = String.format("Get Datastream {%s} failed with error.", datastreamName); - ErrorLogger.logAndThrowDatastreamRuntimeException(LOG, errorMessage, e); + ErrorLogger.logAndThrowDatastreamRuntimeException(_logger, errorMessage, e); return null; // not reachable; Meltdown hack goes here... } } }, Objects::nonNull, getRetryPeriodMs(), getRetryTimeoutMs()).orElseThrow(RetriesExhaustedException::new); - LOG.info("getDatastream for datastream {} took {} ms", datastreamName, + _logger.info("getDatastream for datastream {} took {} ms", datastreamName, Duration.between(startTime, Instant.now()).toMillis()); return datastream; } @@ -185,7 +185,7 @@ public Datastream waitTillDatastreamIsInitialized(String datastreamName, long ti } String errorMessage = String.format("Datastream was not initialized before the timeout %s", timeoutMs); - LOG.error(errorMessage); + _logger.error(errorMessage); throw new DatastreamRuntimeException(errorMessage); } @@ -208,7 +208,7 @@ public void waitTillDatastreamIsDeleted(String datastreamName, long timeoutMs) t } String errorMessage = String.format("Datastream was not deleted before the timeout %s", timeoutMs); - LOG.error(errorMessage); + _logger.error(errorMessage); throw new DatastreamRuntimeException(errorMessage); } @@ -220,15 +220,15 @@ private List getAllDatastreams(GetAllRequest request) { return datastreamResponseFuture.getResponse().getEntity().getElements(); } catch (RemoteInvocationException e) { if (ExceptionUtils.getRootCause(e) instanceof TimeoutException) { - LOG.warn("Timeout: getAllDatastreams. May retry...", e); + _logger.warn("Timeout: getAllDatastreams. May retry...", e); return null; } String errorMessage = "Get All Datastreams failed with error."; - ErrorLogger.logAndThrowDatastreamRuntimeException(LOG, errorMessage, e); + ErrorLogger.logAndThrowDatastreamRuntimeException(_logger, errorMessage, e); return null; // not reachable } }, Objects::nonNull, getRetryPeriodMs(), getRetryTimeoutMs()).orElseThrow(RetriesExhaustedException::new); - LOG.info("getAllDatastreams took {} ms", Duration.between(startTime, Instant.now()).toMillis()); + _logger.info("getAllDatastreams took {} ms", Duration.between(startTime, Instant.now()).toMillis()); return datastreams; } @@ -276,7 +276,7 @@ public void createDatastream(Datastream datastream) { return datastreamResponseFuture.getResponse(); } catch (RemoteInvocationException e) { if (ExceptionUtils.getRootCause(e) instanceof TimeoutException) { - LOG.warn("Timeout: createDatastream. May retry...", e); + _logger.warn("Timeout: createDatastream. May retry...", e); return null; } @@ -293,21 +293,21 @@ public void createDatastream(Datastream datastream) { String msg = String.format("Datastream %s exists with the same name. Requested datastream %s", existingDatastream.toString(), datastream.toString()); - LOG.warn(msg, e); + _logger.warn(msg, e); throw new DatastreamAlreadyExistsException(msg); } else if (errorCode == HttpStatus.S_403_FORBIDDEN.getCode()) { // Handle any DMS REST authorization failure for the caller principal String msg = "Client is not authorized to invoke Datastream-CREATE, stream=" + datastream.getName(); - ErrorLogger.logAndThrowDatastreamRuntimeException(LOG, msg, e); + ErrorLogger.logAndThrowDatastreamRuntimeException(_logger, msg, e); } } String errorMessage = String.format("Create Datastream %s failed with error.", datastream); - ErrorLogger.logAndThrowDatastreamRuntimeException(LOG, errorMessage, e); + ErrorLogger.logAndThrowDatastreamRuntimeException(_logger, errorMessage, e); return null; // unreachable } }, Objects::nonNull, getRetryPeriodMs(), getRetryTimeoutMs()).orElseThrow(RetriesExhaustedException::new); - LOG.info("createDatastream for datastream {} took {} ms", datastream, + _logger.info("createDatastream for datastream {} took {} ms", datastream, Duration.between(startTime, Instant.now()).toMillis()); } @@ -338,15 +338,15 @@ public void updateDatastream(List datastreams) { return datastreamResponseFuture.getResponse(); } catch (RemoteInvocationException e) { if (ExceptionUtils.getRootCause(e) instanceof TimeoutException) { - LOG.warn("Timeout: updateDatastream. May retry...", e); + _logger.warn("Timeout: updateDatastream. May retry...", e); return null; } - ErrorLogger.logAndThrowDatastreamRuntimeException(LOG, "Failed to update datastreams", e); + ErrorLogger.logAndThrowDatastreamRuntimeException(_logger, "Failed to update datastreams", e); return null; // not reachable } }, Objects::nonNull, getRetryPeriodMs(), getRetryTimeoutMs()).orElseThrow(RetriesExhaustedException::new); - LOG.info("updateDatastreams for datastreams {} took {} ms", datastreams, + _logger.info("updateDatastreams for datastreams {} took {} ms", datastreams, Duration.between(startTime, Instant.now()).toMillis()); } @@ -368,21 +368,21 @@ public void deleteDatastream(String datastreamName) { return response.getResponse(); } catch (RemoteInvocationException e) { if (ExceptionUtils.getRootCause(e) instanceof TimeoutException) { - LOG.warn("Timeout: deleteDatastream. May retry...", e); + _logger.warn("Timeout: deleteDatastream. May retry...", e); return null; } String errorMessage = String.format("Delete Datastream %s failed with error.", datastreamName); if (isNotFoundHttpStatus(e)) { - LOG.error(errorMessage, e); + _logger.error(errorMessage, e); throw new DatastreamNotFoundException(datastreamName, e); } else { - ErrorLogger.logAndThrowDatastreamRuntimeException(LOG, errorMessage, e); + ErrorLogger.logAndThrowDatastreamRuntimeException(_logger, errorMessage, e); } return null; // not reachable } }, Objects::nonNull, getRetryPeriodMs(), getRetryTimeoutMs()).orElseThrow(RetriesExhaustedException::new); - LOG.info("deleteDatastream for datastream {} took {} ms", datastreamName, + _logger.info("deleteDatastream for datastream {} took {} ms", datastreamName, Duration.between(startTime, Instant.now()).toMillis()); } @@ -398,14 +398,14 @@ public boolean datastreamExists(String datastreamName) { return Boolean.TRUE; } catch (RemoteInvocationException e) { if (ExceptionUtils.getRootCause(e) instanceof TimeoutException) { - LOG.warn("Timeout: datastreamExists. May retry...", e); + _logger.warn("Timeout: datastreamExists. May retry...", e); return null; } if (isNotFoundHttpStatus(e)) { - LOG.debug("Datastream {} is not found", datastreamName); + _logger.debug("Datastream {} is not found", datastreamName); } else { String errorMessage = String.format("Get Datastream %s failed with error.", datastreamName); - ErrorLogger.logAndThrowDatastreamRuntimeException(LOG, errorMessage, e); + ErrorLogger.logAndThrowDatastreamRuntimeException(_logger, errorMessage, e); } } return Boolean.FALSE; @@ -467,20 +467,20 @@ private void pauseOrStop(String datastreamName, boolean force, boolean isPaused) return datastreamResponseFuture.getResponse(); } catch (RemoteInvocationException e) { if (ExceptionUtils.getRootCause(e) instanceof TimeoutException) { - LOG.warn("Timeout: " + action + ". May retry...", e); + _logger.warn("Timeout: " + action + ". May retry...", e); return null; } if (isNotFoundHttpStatus(e)) { - LOG.warn(String.format("Datastream {%s} is not found", datastreamName), e); + _logger.warn(String.format("Datastream {%s} is not found", datastreamName), e); throw new DatastreamNotFoundException(datastreamName, e); } else { String errorMessage = String.format("Pause Datastream {%s} failed with error.", datastreamName); - ErrorLogger.logAndThrowDatastreamRuntimeException(LOG, errorMessage, e); + ErrorLogger.logAndThrowDatastreamRuntimeException(_logger, errorMessage, e); } return null; // not reachable } }, Objects::nonNull, getRetryPeriodMs(), getRetryTimeoutMs()).orElseThrow(RetriesExhaustedException::new); - LOG.info(action + " for datastream {} took {} ms", datastreamName, + _logger.info(action + " for datastream {} took {} ms", datastreamName, Duration.between(startTime, Instant.now()).toMillis()); } @@ -513,20 +513,20 @@ public void resume(String datastreamName, boolean force) throws RemoteInvocation return datastreamResponseFuture.getResponse(); } catch (RemoteInvocationException e) { if (ExceptionUtils.getRootCause(e) instanceof TimeoutException) { - LOG.warn("Timeout: pause. May retry...", e); + _logger.warn("Timeout: pause. May retry...", e); return null; } if (isNotFoundHttpStatus(e)) { - LOG.warn(String.format("Datastream {%s} is not found", datastreamName), e); + _logger.warn(String.format("Datastream {%s} is not found", datastreamName), e); throw new DatastreamNotFoundException(datastreamName, e); } else { String errorMessage = String.format("Resume Datastream {%s} failed with error.", datastreamName); - ErrorLogger.logAndThrowDatastreamRuntimeException(LOG, errorMessage, e); + ErrorLogger.logAndThrowDatastreamRuntimeException(_logger, errorMessage, e); } return null; // not reachable } }, Objects::nonNull, getRetryPeriodMs(), getRetryTimeoutMs()).orElseThrow(RetriesExhaustedException::new); - LOG.info("resume for datastream {} took {} ms", datastreamName, + _logger.info("resume for datastream {} took {} ms", datastreamName, Duration.between(startTime, Instant.now()).toMillis()); } @@ -550,20 +550,20 @@ public void movePartitions(String datastreamName, String partitions, String targ return datastreamResponseFuture.getResponse(); } catch (RemoteInvocationException e) { if (ExceptionUtils.getRootCause(e) instanceof TimeoutException) { - LOG.warn("Timeout: movePartitions. May retry...", e); + _logger.warn("Timeout: movePartitions. May retry...", e); return null; } if (isNotFoundHttpStatus(e)) { - LOG.warn(String.format("Datastream {%s} is not found", datastreamName), e); + _logger.warn(String.format("Datastream {%s} is not found", datastreamName), e); throw new DatastreamNotFoundException(datastreamName, e); } else { String errorMessage = String.format("move partitions failed with error, Datastream {%s}.", datastreamName); - ErrorLogger.logAndThrowDatastreamRuntimeException(LOG, errorMessage, e); + ErrorLogger.logAndThrowDatastreamRuntimeException(_logger, errorMessage, e); } return null; // not reachable } }, Objects::nonNull, getRetryPeriodMs(), getRetryTimeoutMs()).orElseThrow(RetriesExhaustedException::new); - LOG.info("move partitions for datastream {} took {} ms", datastreamName, Duration.between(startTime, Instant.now()).toMillis()); + _logger.info("move partitions for datastream {} took {} ms", datastreamName, Duration.between(startTime, Instant.now()).toMillis()); } /** * Get all the datastream objects that are in the same group as the datastream whose name is "datastreamName". @@ -580,10 +580,10 @@ public List findGroup(String datastreamName) { return _restClient.sendRequest(request).getResponse().getEntity().getElements(); } catch (RemoteInvocationException e) { if (ExceptionUtils.getRootCause(e) instanceof TimeoutException) { - LOG.warn("Timeout: findGroup. May retry...", e); + _logger.warn("Timeout: findGroup. May retry...", e); return null; } - ErrorLogger.logAndThrowDatastreamRuntimeException(LOG, "findGroup failed with error.", e); + ErrorLogger.logAndThrowDatastreamRuntimeException(_logger, "findGroup failed with error.", e); } return null; // not reachable }, Objects::nonNull, getRetryPeriodMs(), getRetryTimeoutMs()).orElseThrow(RetriesExhaustedException::new); @@ -609,22 +609,22 @@ public void pauseSourcePartitions(String datastreamName, StringMap sourcePartiti return datastreamResponseFuture.getResponse(); } catch (RemoteInvocationException e) { if (ExceptionUtils.getRootCause(e) instanceof TimeoutException) { - LOG.warn("Timeout: pauseSourcePartitions. May retry...", e); + _logger.warn("Timeout: pauseSourcePartitions. May retry...", e); return null; } if (isNotFoundHttpStatus(e)) { - LOG.warn(String.format("Datastream {%s} is not found", datastreamName), e); + _logger.warn(String.format("Datastream {%s} is not found", datastreamName), e); throw new DatastreamNotFoundException(datastreamName, e); } else { String errorMessage = String.format("Pause Datastream partitions failed with error. Datastream: {%s}, Partitions: {%s}", datastreamName, sourcePartitions); - ErrorLogger.logAndThrowDatastreamRuntimeException(LOG, errorMessage, e); + ErrorLogger.logAndThrowDatastreamRuntimeException(_logger, errorMessage, e); } return null; // not reachable } }, Objects::nonNull, getRetryPeriodMs(), getRetryTimeoutMs()).orElseThrow(RetriesExhaustedException::new); - LOG.info("pauseSourcePartitions for datastream {}, sourcePartitions {} took {} ms", datastreamName, sourcePartitions, + _logger.info("pauseSourcePartitions for datastream {}, sourcePartitions {} took {} ms", datastreamName, sourcePartitions, Duration.between(startTime, Instant.now()).toMillis()); } @@ -648,22 +648,22 @@ public void resumeSourcePartitions(String datastreamName, StringMap sourcePartit return datastreamResponseFuture.getResponse(); } catch (RemoteInvocationException e) { if (ExceptionUtils.getRootCause(e) instanceof TimeoutException) { - LOG.warn("Timeout: pauseSourcePartitions. May retry...", e); + _logger.warn("Timeout: pauseSourcePartitions. May retry...", e); return null; } if (isNotFoundHttpStatus(e)) { - LOG.warn(String.format("Datastream {%s} is not found", datastreamName), e); + _logger.warn(String.format("Datastream {%s} is not found", datastreamName), e); throw new DatastreamNotFoundException(datastreamName, e); } else { String errorMessage = String.format("Resume Datastream partitions failed with error. Datastream: {%s}, Partitions: {%s}", datastreamName, sourcePartitions); - ErrorLogger.logAndThrowDatastreamRuntimeException(LOG, errorMessage, e); + ErrorLogger.logAndThrowDatastreamRuntimeException(_logger, errorMessage, e); } return null; // not reachable } }, Objects::nonNull, getRetryPeriodMs(), getRetryTimeoutMs()).orElseThrow(RetriesExhaustedException::new); - LOG.info("resumeSourcePartitions for datastream {}, sourcePartitions {} took {} ms", datastreamName, sourcePartitions, + _logger.info("resumeSourcePartitions for datastream {}, sourcePartitions {} took {} ms", datastreamName, sourcePartitions, Duration.between(startTime, Instant.now()).toMillis()); } } diff --git a/datastream-client/src/main/java/com/linkedin/datastream/DatastreamRestClientFactory.java b/datastream-client/src/main/java/com/linkedin/datastream/DatastreamRestClientFactory.java index 9ddf7b96b..4ff221313 100644 --- a/datastream-client/src/main/java/com/linkedin/datastream/DatastreamRestClientFactory.java +++ b/datastream-client/src/main/java/com/linkedin/datastream/DatastreamRestClientFactory.java @@ -23,9 +23,9 @@ * DatastreamRestClient, without doing a major refactoring of the code. */ public final class DatastreamRestClientFactory { - private static final Logger LOG = LoggerFactory.getLogger(DatastreamRestClientFactory.class); + private static final Logger _logger = LoggerFactory.getLogger(DatastreamRestClientFactory.class); private static final BaseRestClientFactory FACTORY = - new BaseRestClientFactory<>(DatastreamRestClient.class, LOG); + new BaseRestClientFactory<>(DatastreamRestClient.class, _logger); /** * Get a DatastreamRestClient with default HTTP client diff --git a/datastream-client/src/main/java/com/linkedin/diagnostics/ServerComponentHealthRestClient.java b/datastream-client/src/main/java/com/linkedin/diagnostics/ServerComponentHealthRestClient.java index d717a2b86..25d5fcca1 100644 --- a/datastream-client/src/main/java/com/linkedin/diagnostics/ServerComponentHealthRestClient.java +++ b/datastream-client/src/main/java/com/linkedin/diagnostics/ServerComponentHealthRestClient.java @@ -23,7 +23,7 @@ */ public class ServerComponentHealthRestClient { - private static final Logger LOG = LoggerFactory.getLogger(ServerComponentHealthRestClient.class); + private static final Logger _logger = LoggerFactory.getLogger(ServerComponentHealthRestClient.class); private final DiagRequestBuilders _builders; private final RestClient _restClient; @@ -55,7 +55,7 @@ public ServerComponentHealth getStatus(String type, String scope, String content if (response != null && !response.isEmpty()) { return response.get(0); } else { - LOG.error("ServerComponentHealth getStatus {} {} failed with empty response.", type, scope); + _logger.error("ServerComponentHealth getStatus {} {} failed with empty response.", type, scope); return null; } } @@ -76,7 +76,7 @@ public List getServerComponentHealthStatues(String type, FindRequest request = _builders.findByStatus().typeParam(type).scopeParam(scope).contentParam(content).build(); return _restClient.sendRequest(request).getResponse().getEntity().getElements(); } catch (RemoteInvocationException e) { - LOG.error("Get serverComponentHealthStatus {} {} failed with error.", type, scope); + _logger.error("Get serverComponentHealthStatus {} {} failed with error.", type, scope); return null; } } @@ -98,7 +98,7 @@ public ServerComponentHealth getAllStatus(String type, String scope, String cont if (response != null && !response.isEmpty()) { return response.get(0); } else { - LOG.error("ServerComponentHealth getAllStatus {} {} failed with empty response.", type, scope); + _logger.error("ServerComponentHealth getAllStatus {} {} failed with empty response.", type, scope); return null; } } @@ -119,7 +119,7 @@ public List getServerComponentHealthAllStatus(String type FindRequest request = _builders.findByAllStatus().typeParam(type).scopeParam(scope).contentParam(content).build(); return _restClient.sendRequest(request).getResponse().getEntity().getElements(); } catch (RemoteInvocationException e) { - LOG.error("Get serverComponentHealthAllStatus {} {} failed with error.", type, scope); + _logger.error("Get serverComponentHealthAllStatus {} {} failed with error.", type, scope); return null; } } diff --git a/datastream-client/src/main/java/com/linkedin/diagnostics/ServerComponentHealthRestClientFactory.java b/datastream-client/src/main/java/com/linkedin/diagnostics/ServerComponentHealthRestClientFactory.java index c7775088d..c1911cbb8 100644 --- a/datastream-client/src/main/java/com/linkedin/diagnostics/ServerComponentHealthRestClientFactory.java +++ b/datastream-client/src/main/java/com/linkedin/diagnostics/ServerComponentHealthRestClientFactory.java @@ -21,9 +21,9 @@ * Factory class for obtaining {@link ServerComponentHealthRestClient} objects. */ public final class ServerComponentHealthRestClientFactory { - private static final Logger LOG = LoggerFactory.getLogger(DatastreamRestClientFactory.class); + private static final Logger _logger = LoggerFactory.getLogger(DatastreamRestClientFactory.class); private static final BaseRestClientFactory FACTORY = - new BaseRestClientFactory<>(ServerComponentHealthRestClient.class, LOG); + new BaseRestClientFactory<>(ServerComponentHealthRestClient.class, _logger); /** * Get a ServerComponentHealthRestClient with default HTTP client diff --git a/datastream-client/src/test/java/com/linkedin/datastream/TestDatastreamRestClient.java b/datastream-client/src/test/java/com/linkedin/datastream/TestDatastreamRestClient.java index fbbb9ea62..b8602bfae 100644 --- a/datastream-client/src/test/java/com/linkedin/datastream/TestDatastreamRestClient.java +++ b/datastream-client/src/test/java/com/linkedin/datastream/TestDatastreamRestClient.java @@ -59,7 +59,7 @@ */ @Test(singleThreaded = true) public class TestDatastreamRestClient extends TestRestliClientBase { - private static final Logger LOG = LoggerFactory.getLogger(TestDatastreamRestClient.class); + private static final Logger _logger = LoggerFactory.getLogger(TestDatastreamRestClient.class); /** * Generate a datastream @@ -102,11 +102,11 @@ private DatastreamRestClient createRestClient() { @Test public void testCreateTwoDatastreams() throws Exception { Datastream datastream = generateDatastream(6); - LOG.info("Datastream : {}", datastream); + _logger.info("Datastream : {}", datastream); DatastreamRestClient restClient = createRestClient(); restClient.createDatastream(datastream); Datastream createdDatastream = restClient.waitTillDatastreamIsInitialized(datastream.getName(), WAIT_TIMEOUT_MS); - LOG.info("Created Datastream : {}", createdDatastream); + _logger.info("Created Datastream : {}", createdDatastream); datastream.setDestination(new DatastreamDestination()); // server might have already set the destination so we need to unset it for comparison @@ -115,10 +115,10 @@ public void testCreateTwoDatastreams() throws Exception { Assert.assertEquals(createdDatastream, datastream); datastream = generateDatastream(7); - LOG.info("Datastream : {}", datastream); + _logger.info("Datastream : {}", datastream); restClient.createDatastream(datastream); createdDatastream = restClient.waitTillDatastreamIsInitialized(datastream.getName(), WAIT_TIMEOUT_MS); - LOG.info("Created Datastream : {}", createdDatastream); + _logger.info("Created Datastream : {}", createdDatastream); datastream.setDestination(new DatastreamDestination()); // server might have already set the destination so we need to unset it for comparison @@ -133,13 +133,13 @@ public void testCreateDatastreamToNonLeader() throws Exception { _datastreamCluster.startupServer(1); Datastream datastream = generateDatastream(5); - LOG.info("Datastream : {}", datastream); + _logger.info("Datastream : {}", datastream); int followerDmsPort = _datastreamCluster.getDatastreamPorts().get(1); DatastreamRestClient restClient = DatastreamRestClientFactory.getClient("http://localhost:" + followerDmsPort); restClient.createDatastream(datastream); Datastream createdDatastream = restClient.waitTillDatastreamIsInitialized(datastream.getName(), WAIT_TIMEOUT_MS); - LOG.info("Created Datastream : {}", createdDatastream); + _logger.info("Created Datastream : {}", createdDatastream); datastream.setDestination(new DatastreamDestination()); // server might have already set the destination so we need to unset it for comparison clearDatastreamDestination(Collections.singletonList(createdDatastream)); @@ -150,7 +150,7 @@ public void testCreateDatastreamToNonLeader() throws Exception { @Test(expectedExceptions = DatastreamAlreadyExistsException.class) public void testCreateDatastreamThatAlreadyExists() throws Exception { Datastream datastream = generateDatastream(1); - LOG.info("Datastream : {}", datastream); + _logger.info("Datastream : {}", datastream); DatastreamRestClient restClient = createRestClient(); restClient.createDatastream(datastream); restClient.createDatastream(datastream); @@ -159,11 +159,11 @@ public void testCreateDatastreamThatAlreadyExists() throws Exception { @Test public void testWaitTillDatastreamIsInitializedReturnsInitializedDatastream() throws Exception { Datastream datastream = generateDatastream(11); - LOG.info("Datastream : {}", datastream); + _logger.info("Datastream : {}", datastream); DatastreamRestClient restClient = createRestClient(); restClient.createDatastream(datastream); Datastream initializedDatastream = restClient.waitTillDatastreamIsInitialized(datastream.getName(), 60000); - LOG.info("Initialized Datastream : {}", initializedDatastream); + _logger.info("Initialized Datastream : {}", initializedDatastream); Assert.assertNotEquals(initializedDatastream.getDestination().getConnectionString(), ""); Assert.assertEquals(initializedDatastream.getDestination().getPartitions().intValue(), 1); } @@ -192,7 +192,7 @@ private void clearDynamicMetadata(Collection datastreams) { public void testGetAllDatastreams() { List datastreams = IntStream.range(100, 110).mapToObj(TestDatastreamRestClient::generateDatastream).collect(Collectors.toList()); - LOG.info("Datastreams : {}", datastreams); + _logger.info("Datastreams : {}", datastreams); DatastreamRestClient restClient = createRestClient(); int initialSize = restClient.getAllDatastreams().size(); @@ -209,7 +209,7 @@ public void testGetAllDatastreams() { Assert.assertTrue(result.isPresent()); List createdDatastreams = result.get(); - LOG.info("Created Datastreams : {}", createdDatastreams); + _logger.info("Created Datastreams : {}", createdDatastreams); clearDatastreamDestination(datastreams); clearDatastreamDestination(createdDatastreams); @@ -221,7 +221,7 @@ public void testGetAllDatastreams() { int skip = 2; int count = 5; List paginatedCreatedDatastreams = restClient.getAllDatastreams(2, 5); - LOG.info("Paginated Datastreams : {}", paginatedCreatedDatastreams); + _logger.info("Paginated Datastreams : {}", paginatedCreatedDatastreams); Assert.assertEquals(paginatedCreatedDatastreams.size(), count); @@ -235,7 +235,7 @@ public void testGetAllDatastreams() { @Test(expectedExceptions = DatastreamNotFoundException.class) public void testDeleteDatastream() throws Exception { Datastream datastream = generateDatastream(2); - LOG.info("Datastream : {}", datastream); + _logger.info("Datastream : {}", datastream); DatastreamRestClient restClient = createRestClient(); restClient.createDatastream(datastream); restClient.waitTillDatastreamIsInitialized(datastream.getName(), Duration.ofMinutes(2).toMillis()); diff --git a/datastream-client/src/test/java/com/linkedin/diagnostics/TestServerComponentHealthRestClient.java b/datastream-client/src/test/java/com/linkedin/diagnostics/TestServerComponentHealthRestClient.java index 7a20f1c35..2dc4e9bc5 100644 --- a/datastream-client/src/test/java/com/linkedin/diagnostics/TestServerComponentHealthRestClient.java +++ b/datastream-client/src/test/java/com/linkedin/diagnostics/TestServerComponentHealthRestClient.java @@ -23,7 +23,7 @@ @Test(singleThreaded = true) public class TestServerComponentHealthRestClient extends TestRestliClientBase { - private static final Logger LOG = LoggerFactory.getLogger(TestServerComponentHealthRestClient.class); + private static final Logger _logger = LoggerFactory.getLogger(TestServerComponentHealthRestClient.class); @BeforeTest public void setUp() throws Exception { diff --git a/datastream-cloud-storage/src/main/java/com/linkedin/datastream/cloud/storage/CloudStorageTransportProvider.java b/datastream-cloud-storage/src/main/java/com/linkedin/datastream/cloud/storage/CloudStorageTransportProvider.java index b40612507..9600aec0f 100644 --- a/datastream-cloud-storage/src/main/java/com/linkedin/datastream/cloud/storage/CloudStorageTransportProvider.java +++ b/datastream-cloud-storage/src/main/java/com/linkedin/datastream/cloud/storage/CloudStorageTransportProvider.java @@ -36,7 +36,7 @@ */ public class CloudStorageTransportProvider implements TransportProvider { - private static final Logger LOG = LoggerFactory.getLogger(CloudStorageTransportProvider.class.getName()); + private static final Logger _logger = LoggerFactory.getLogger(CloudStorageTransportProvider.class.getName()); private static final String KAFKA_ORIGIN_TOPIC = "kafka-origin-topic"; private static final String KAFKA_ORIGIN_PARTITION = "kafka-origin-partition"; @@ -52,18 +52,18 @@ public class CloudStorageTransportProvider implements TransportProvider { private static void initializeLocalDirectory(String localDirectory) { java.io.File localDir = new java.io.File(localDirectory); if (!localDir.exists()) { - LOG.info("Creating directory {}.", localDirectory); + _logger.info("Creating directory {}.", localDirectory); if (!localDir.mkdirs()) { - LOG.error("Unable to create the IO directory {}", localDir); + _logger.error("Unable to create the IO directory {}", localDir); throw new RuntimeException("Unable to create the IO directory"); } } try { - LOG.info("Cleaning the directory {}.", localDirectory); + _logger.info("Cleaning the directory {}.", localDirectory); FileUtils.cleanDirectory(localDir); } catch (IOException e) { - LOG.error("Unable to clear the IO directory {}", localDir); + _logger.error("Unable to clear the IO directory {}", localDir); throw new RuntimeException(e); } } @@ -96,7 +96,7 @@ private CloudStorageTransportProvider(CloudStorageTransportProviderBuilder build _scheduler.scheduleAtFixedRate( () -> { for (ObjectBuilder objectBuilder: _objectBuilders) { - LOG.info("Try flush signal sent."); + _logger.info("Try flush signal sent."); objectBuilder.assign(new com.linkedin.datastream.common.Package.PackageBuilder().buildTryFlushSignalPackage()); } }, @@ -129,12 +129,12 @@ public void send(String destination, DatastreamProducerRecord record, SendCallba @Override public synchronized void close() { if (_isClosed) { - LOG.info("Transport provider {} is already closed.", _transportProviderName); + _logger.info("Transport provider {} is already closed.", _transportProviderName); return; } try { - LOG.info("Closing the transport provider {}", _transportProviderName); + _logger.info("Closing the transport provider {}", _transportProviderName); for (ObjectBuilder objectBuilder : _objectBuilders) { objectBuilder.shutdown(); } @@ -143,7 +143,7 @@ public synchronized void close() { try { objectBuilder.join(); } catch (InterruptedException e) { - LOG.warn("An interrupt was raised during join() call on a Object Builder"); + _logger.warn("An interrupt was raised during join() call on a Object Builder"); Thread.currentThread().interrupt(); } } @@ -156,7 +156,7 @@ public synchronized void close() { @Override public void flush() { - LOG.info("Forcing flush on object builders."); + _logger.info("Forcing flush on object builders."); List flushSignalPackages = new ArrayList<>(); for (final ObjectBuilder objectBuilder : _objectBuilders) { final com.linkedin.datastream.common.Package aPackage = new com.linkedin.datastream.common.Package.PackageBuilder().buildFroceFlushSignalPackage(); diff --git a/datastream-cloud-storage/src/main/java/com/linkedin/datastream/cloud/storage/CloudStorageTransportProviderAdmin.java b/datastream-cloud-storage/src/main/java/com/linkedin/datastream/cloud/storage/CloudStorageTransportProviderAdmin.java index 561f549d0..20d1e72eb 100644 --- a/datastream-cloud-storage/src/main/java/com/linkedin/datastream/cloud/storage/CloudStorageTransportProviderAdmin.java +++ b/datastream-cloud-storage/src/main/java/com/linkedin/datastream/cloud/storage/CloudStorageTransportProviderAdmin.java @@ -34,7 +34,7 @@ */ public class CloudStorageTransportProviderAdmin implements TransportProviderAdmin { - public static final Logger LOG = LoggerFactory.getLogger(CloudStorageTransportProviderAdmin.class); + public static final Logger _logger = LoggerFactory.getLogger(CloudStorageTransportProviderAdmin.class); public static final String CONFIG_OBJECTBUILDER_QUEUE_SIZE = "objectBuilderQueueSize"; public static final String CONFIG_OBJECTBUILDER_THREAD_COUNT = "objectBuilderThreadCount"; diff --git a/datastream-cloud-storage/src/main/java/com/linkedin/datastream/cloud/storage/ObjectBuilder.java b/datastream-cloud-storage/src/main/java/com/linkedin/datastream/cloud/storage/ObjectBuilder.java index c79c351cf..7de42ff65 100644 --- a/datastream-cloud-storage/src/main/java/com/linkedin/datastream/cloud/storage/ObjectBuilder.java +++ b/datastream-cloud-storage/src/main/java/com/linkedin/datastream/cloud/storage/ObjectBuilder.java @@ -29,7 +29,7 @@ */ public class ObjectBuilder extends Thread { - private static final Logger LOG = LoggerFactory.getLogger(CloudStorageTransportProvider.class.getName()); + private static final Logger _logger = LoggerFactory.getLogger(CloudStorageTransportProvider.class.getName()); private final Map _registry; private final BlockingQueue _packageQueue; @@ -87,7 +87,7 @@ public void assign(Package aPackage) { try { _packageQueue.put(aPackage); } catch (InterruptedException e) { - LOG.warn("Assign is interrupted. {}", e); + _logger.warn("Assign is interrupted. {}", e); Thread.currentThread().interrupt(); } } @@ -133,15 +133,15 @@ public void run() { Thread.currentThread().interrupt(); break; } catch (IOException e) { - LOG.error("Unable to write to WriteLog {}", e); + _logger.error("Unable to write to WriteLog {}", e); aPackage.getAckCallback().onCompletion(new DatastreamRecordMetadata( aPackage.getCheckpoint(), aPackage.getTopic(), aPackage.getPartition()), new DatastreamTransientException(e)); } catch (IllegalStateException e) { - LOG.error("Unable to write to WriteLog {}", e); + _logger.error("Unable to write to WriteLog {}", e); aPackage.getAckCallback().onCompletion(new DatastreamRecordMetadata( aPackage.getCheckpoint(), aPackage.getTopic(), aPackage.getPartition()), e); } } - LOG.info("Object builder stopped."); + _logger.info("Object builder stopped."); } } diff --git a/datastream-cloud-storage/src/main/java/com/linkedin/datastream/cloud/storage/WriteLog.java b/datastream-cloud-storage/src/main/java/com/linkedin/datastream/cloud/storage/WriteLog.java index 55c1ab84e..f7f5b9206 100644 --- a/datastream-cloud-storage/src/main/java/com/linkedin/datastream/cloud/storage/WriteLog.java +++ b/datastream-cloud-storage/src/main/java/com/linkedin/datastream/cloud/storage/WriteLog.java @@ -30,7 +30,7 @@ * This class writes the messages to local file. Which can be committed to cloud storage later. */ public class WriteLog { - private static final Logger LOG = LoggerFactory.getLogger(WriteLog.class.getName()); + private static final Logger _logger = LoggerFactory.getLogger(WriteLog.class.getName()); private final String _ioClass; private final VerifiableProperties _ioProperties; @@ -109,7 +109,7 @@ private String getFilePath(String topic, String partition) { private void waitForRoomInCommitBacklog() throws InterruptedException { synchronized (_counterLock) { if (_inflightWriteLogCommits >= _maxInflightWriteLogCommits) { - LOG.info("Waiting for room in commit backlog, current inflight commits {} ", + _logger.info("Waiting for room in commit backlog, current inflight commits {} ", _inflightWriteLogCommits); } while (_inflightWriteLogCommits >= _maxInflightWriteLogCommits) { @@ -126,7 +126,7 @@ private void waitForRoomInCommitBacklog() throws InterruptedException { private void waitForCommitBacklogToClear() throws InterruptedException { synchronized (_counterLock) { if (_inflightWriteLogCommits > 0) { - LOG.info("Waiting for the commit backlog to clear."); + _logger.info("Waiting for the commit backlog to clear."); } while (_inflightWriteLogCommits > 0) { try { @@ -188,7 +188,7 @@ public void write(Package aPackage) throws IOException, InterruptedException { aPackage.getPartition())); } else if (aPackage.isTryFlushSignal() || aPackage.isForceFlushSignal()) { if (_file == null) { - LOG.debug("Nothing to flush."); + _logger.debug("Nothing to flush."); return; } } diff --git a/datastream-cloud-storage/src/main/java/com/linkedin/datastream/cloud/storage/committer/GCSObjectCommitter.java b/datastream-cloud-storage/src/main/java/com/linkedin/datastream/cloud/storage/committer/GCSObjectCommitter.java index cddcb7766..355c4b358 100644 --- a/datastream-cloud-storage/src/main/java/com/linkedin/datastream/cloud/storage/committer/GCSObjectCommitter.java +++ b/datastream-cloud-storage/src/main/java/com/linkedin/datastream/cloud/storage/committer/GCSObjectCommitter.java @@ -50,7 +50,7 @@ */ public class GCSObjectCommitter implements ObjectCommitter { - private static final Logger LOG = LoggerFactory.getLogger(GCSObjectCommitter.class.getName()); + private static final Logger _logger = LoggerFactory.getLogger(GCSObjectCommitter.class.getName()); private static final String CONFIG_THREADS = "threads"; private static final String CONFIG_WRITEATONCE_MAX_FILE_SIZE = "writeAtOnceMaxFileSize"; @@ -74,10 +74,10 @@ public GCSObjectCommitter(VerifiableProperties properties) { this._storage = StorageOptions.newBuilder().setCredentials(credentials) .build().getService(); } catch (FileNotFoundException e) { - LOG.error("Credentials path {} does not exist", credentialsPath); + _logger.error("Credentials path {} does not exist", credentialsPath); throw new RuntimeException(e); } catch (IOException e) { - LOG.error("Unable to read credentials: {}", credentialsPath); + _logger.error("Unable to read credentials: {}", credentialsPath); throw new RuntimeException(e); } @@ -89,16 +89,16 @@ public GCSObjectCommitter(VerifiableProperties properties) { } private static void deleteFile(File file) { - LOG.info("Deleting file {}", file.toPath()); + _logger.info("Deleting file {}", file.toPath()); if (!file.delete()) { - LOG.warn("Failed to delete file {}.", file.toPath()); + _logger.warn("Failed to delete file {}.", file.toPath()); } // clean crc files final File crcFile = new File(file.getParent() + "/." + file.getName() + ".crc"); if (crcFile.exists() && crcFile.isFile()) { if (!crcFile.delete()) { - LOG.warn("Failed to delete crc file {}.", crcFile.toPath()); + _logger.warn("Failed to delete crc file {}.", crcFile.toPath()); } } } @@ -168,11 +168,11 @@ public void commit(final String filePath, .newBuilder(BlobId.of(getBucketName(destination), objectName)) .setContentType(Files.probeContentType(file.toPath())) .build(); - LOG.info("Committing Object {}", objectName); + _logger.info("Committing Object {}", objectName); if (file.getTotalSpace() <= _writeAtOnceMaxFileSize) { Blob blob = _storage.create(sourceBlob, Files.readAllBytes(file.toPath())); } else { - LOG.info("Using writer channel to write {}", objectName); + _logger.info("Using writer channel to write {}", objectName); writerChannelUsed = true; try (WriteChannel writer = _storage.writer(sourceBlob)) { byte[] buffer = new byte[256 * 1024]; @@ -186,9 +186,9 @@ public void commit(final String filePath, } } _uploadRateMeter.mark(ackCallbacks.size()); - LOG.info("Successfully created object {}", objectName); + _logger.info("Successfully created object {}", objectName); } catch (Exception e) { - LOG.error("Failed to commit file {} for offsets {} to {} : {}", filePath, minOffset, maxOffset, e); + _logger.error("Failed to commit file {} for offsets {} to {} : {}", filePath, minOffset, maxOffset, e); exception = new DatastreamTransientException(e); } @@ -199,22 +199,22 @@ public void commit(final String filePath, // Delete the partially written object through WriteChannel if (exception != null && writerChannelUsed) { try { - LOG.info("Delete partially written object {} in the bucket {} with chunks count {}", + _logger.info("Delete partially written object {} in the bucket {} with chunks count {}", objectName, getBucketName(destination), chunkCount); if (_storage.delete(BlobId.of(getBucketName(destination), objectName))) { - LOG.info("Successfully deleted partially written object {} in the bucket {}", + _logger.info("Successfully deleted partially written object {} in the bucket {}", objectName, getBucketName(destination)); } else { - LOG.warn("Failed to delete partially written object {} in the bucket {} with chunks count {} - Object was not found", + _logger.warn("Failed to delete partially written object {} in the bucket {} with chunks count {} - Object was not found", objectName, getBucketName(destination), chunkCount); } } catch (StorageException e) { - LOG.warn("Failed to delete partially written object {} in the bucket {} with chunks count {} - {}", + _logger.warn("Failed to delete partially written object {} in the bucket {} with chunks count {} - {}", objectName, getBucketName(destination), chunkCount, e); } } - LOG.info("Deleting local file {}", file.getAbsolutePath()); + _logger.info("Deleting local file {}", file.getAbsolutePath()); deleteFile(file); callback.commited(); @@ -227,12 +227,12 @@ public void shutdown() { _executor.shutdown(); try { if (!_executor.awaitTermination(5, TimeUnit.SECONDS)) { - LOG.warn("Committer shutdown timed out."); + _logger.warn("Committer shutdown timed out."); } } catch (InterruptedException e) { - LOG.warn("Interrupted while awaiting committer termination."); + _logger.warn("Interrupted while awaiting committer termination."); Thread.currentThread().interrupt(); } - LOG.info("Object committer stopped."); + _logger.info("Object committer stopped."); } } diff --git a/datastream-cloud-storage/src/main/java/com/linkedin/datastream/cloud/storage/io/AvroParquetFile.java b/datastream-cloud-storage/src/main/java/com/linkedin/datastream/cloud/storage/io/AvroParquetFile.java index cb70d6583..9a161c107 100644 --- a/datastream-cloud-storage/src/main/java/com/linkedin/datastream/cloud/storage/io/AvroParquetFile.java +++ b/datastream-cloud-storage/src/main/java/com/linkedin/datastream/cloud/storage/io/AvroParquetFile.java @@ -32,7 +32,7 @@ * Implementation of {@link com.linkedin.datastream.cloud.storage.io.File} to support Parquet file format */ public class AvroParquetFile implements com.linkedin.datastream.cloud.storage.io.File { - private static final Logger LOG = LoggerFactory.getLogger(AvroParquetFile.class); + private static final Logger _logger = LoggerFactory.getLogger(AvroParquetFile.class); private final static Map SCHEMAS = new ConcurrentHashMap<>(); @@ -78,7 +78,7 @@ private Schema getSchemaByTopic(String topic) { String schemaName = topic + _schemaNameSuffix; return new Schema.Parser().parse(_schemaRegistryClient.getLatestSchemaMetadata(schemaName).getSchema()); } catch (Exception e) { - LOG.error("Unable to find schema for {} - {}", key, e); + _logger.error("Unable to find schema for {} - {}", key, e); return null; } }); diff --git a/datastream-cloud-storage/src/main/java/com/linkedin/datastream/cloud/storage/io/ScribeParquetFile.java b/datastream-cloud-storage/src/main/java/com/linkedin/datastream/cloud/storage/io/ScribeParquetFile.java index 4c9e9e307..a53bbb18c 100644 --- a/datastream-cloud-storage/src/main/java/com/linkedin/datastream/cloud/storage/io/ScribeParquetFile.java +++ b/datastream-cloud-storage/src/main/java/com/linkedin/datastream/cloud/storage/io/ScribeParquetFile.java @@ -27,7 +27,7 @@ * Implementation of {@link com.linkedin.datastream.cloud.storage.io.File} to support parquet file format for scribe */ public class ScribeParquetFile implements com.linkedin.datastream.cloud.storage.io.File { - private static final Logger LOG = LoggerFactory.getLogger(AvroParquetFile.class); + private static final Logger _logger = LoggerFactory.getLogger(AvroParquetFile.class); private static final String CONFIG_PAGE_SIZE = "pageSize"; private static final int DEFAULT_PAGE_SIZE = 64 * 1024; private static final CompressionCodecName COMPRESSION_TYPE = CompressionCodecName.SNAPPY; diff --git a/datastream-common/src/main/idl/com.linkedin.datastream.server.dms.datastream.restspec.json b/datastream-common/src/main/idl/com.linkedin.datastream.server.dms.datastream.restspec.json index 0c2a2e9cb..cbd26b479 100644 --- a/datastream-common/src/main/idl/com.linkedin.datastream.server.dms.datastream.restspec.json +++ b/datastream-common/src/main/idl/com.linkedin.datastream.server.dms.datastream.restspec.json @@ -93,6 +93,26 @@ "default" : "false", "doc" : "whether or not to resume all datastreams within the given datastream's group" } ] + } ], + "subresources" : [ { + "name" : "checkpoint", + "namespace" : "com.linkedin.datastream.server.dms", + "path" : "/datastream/{datastreamId}/checkpoint", + "schema" : "com.linkedin.datastream.diagnostics.ConnectorHealth", + "doc" : "The Restli resource of {@link ConnectorHealth}. Used for collecting datastream checkpoint info\n about a {@link Datastream}.\n\ngenerated from: com.linkedin.datastream.server.dms.DatastreamSourceCheckpointResources", + "simple" : { + "supports" : [ "get", "update" ], + "methods" : [ { + "method" : "get", + "doc" : "Get source checkpoints of the parent datastream for each tasks" + }, { + "method" : "update", + "doc" : "Update source checkpoint on a given datastream" + } ], + "entity" : { + "path" : "/datastream/{datastreamId}/checkpoint" + } + } } ] } } diff --git a/datastream-common/src/main/java/com/linkedin/datastream/common/DatastreamRecordMetadata.java b/datastream-common/src/main/java/com/linkedin/datastream/common/DatastreamRecordMetadata.java index c25f76c53..ca8bbe9d4 100644 --- a/datastream-common/src/main/java/com/linkedin/datastream/common/DatastreamRecordMetadata.java +++ b/datastream-common/src/main/java/com/linkedin/datastream/common/DatastreamRecordMetadata.java @@ -5,6 +5,8 @@ */ package com.linkedin.datastream.common; +import java.util.Objects; + /** * Metadata of the successfully produced datastream record */ @@ -51,4 +53,23 @@ public String getTopic() { public String toString() { return String.format("Checkpoint: %s, Topic: %s, Partition: %d", _checkpoint, _topic, _partition); } + + @Override + public boolean equals(final Object other) { + if (this == other) { + return true; + } else if (other == null || getClass() != other.getClass()) { + return false; + } else { + final DatastreamRecordMetadata otherMetadata = (DatastreamRecordMetadata) other; + return _partition == otherMetadata._partition && + Objects.equals(_topic, otherMetadata._topic) && + Objects.equals(_checkpoint, otherMetadata._checkpoint); + } + } + + @Override + public int hashCode() { + return Objects.hash(_topic, _partition, _checkpoint); + } } diff --git a/datastream-common/src/main/java/com/linkedin/datastream/common/DurableScheduledService.java b/datastream-common/src/main/java/com/linkedin/datastream/common/DurableScheduledService.java index da4edd7a1..1ca9172ed 100644 --- a/datastream-common/src/main/java/com/linkedin/datastream/common/DurableScheduledService.java +++ b/datastream-common/src/main/java/com/linkedin/datastream/common/DurableScheduledService.java @@ -41,7 +41,7 @@ */ public abstract class DurableScheduledService implements Service { - private static final Logger LOG = LoggerFactory.getLogger(DurableScheduledService.class); + private static final Logger _logger = LoggerFactory.getLogger(DurableScheduledService.class); /** * Counts instantiations of this class. Used to ensure the service and thread names are unique and monotonically @@ -223,13 +223,13 @@ protected String serviceName() { protected void startUp() { _started = Instant.now(); _lastSuccessfulRun = Instant.now(); - LOG.info("Starting the DurableScheduledService for {}", _serviceName); + _logger.info("Starting the DurableScheduledService for {}", _serviceName); } @Override protected void shutDown() { _stopped = Instant.now(); - LOG.info("Stopping the DurableScheduledService for {}", _serviceName); + _logger.info("Stopping the DurableScheduledService for {}", _serviceName); stopTask(); if (_taskService != null && _taskService.isRunning()) { _taskService.stopAsync().awaitTerminated(); @@ -241,7 +241,7 @@ protected void runOneIteration() { try { // If this service has leaked, we need to terminate ourself and the periodic task. if (hasLeaked()) { - LOG.warn("Detected that this service {} has leaked. Shutting the service down.", _serviceName); + _logger.warn("Detected that this service {} has leaked. Shutting the service down.", _serviceName); stopTask(); super.stopAsync(); return; @@ -250,7 +250,7 @@ protected void runOneIteration() { // If the task hasn't successfully completed in awhile, it is stuck. So, restart it. final Instant taskRunTimeoutTime = _lastSuccessfulRun.plus(_timeout); if (Instant.now().isAfter(taskRunTimeoutTime)) { - LOG.warn("Detected that the periodic task {} has not succeeded for an extended time - " + _logger.warn("Detected that the periodic task {} has not succeeded for an extended time - " + "terminating and restarting it", _taskService); stopTask(); startTask(); @@ -259,24 +259,24 @@ protected void runOneIteration() { // The task might still be starting, but not timed out yet. if (_taskService != null && (_taskService.state() == State.NEW || _taskService.state() == State.STARTING)) { - LOG.debug("Detected that the periodic task {} is still starting", _taskService); + _logger.debug("Detected that the periodic task {} is still starting", _taskService); return; } // If the task isn't running (and isn't starting up), we need to restart it. if (_taskService == null || !_taskService.isRunning()) { - LOG.warn("Detected that the periodic task {} is not running - starting it", _taskService); + _logger.warn("Detected that the periodic task {} is not running - starting it", _taskService); stopTask(); startTask(); return; } // No issues detected with the task - LOG.debug("Detected no issues with the periodic task {}. Last successful run was {}.", _taskService, + _logger.debug("Detected no issues with the periodic task {}. Last successful run was {}.", _taskService, _lastSuccessfulRun); } catch (Exception e) { _stopped = Instant.now(); - LOG.error("Error encountered in the thread watching (monitoring) the periodic task {}. " + _logger.error("Error encountered in the thread watching (monitoring) the periodic task {}. " + "The periodic task can no longer be monitored or controlled, so will terminate.", _taskService, e); stopTask(); throw e; @@ -293,11 +293,11 @@ private void startTask() { _taskThread = null; _taskService = createPeriodicTaskService(); _taskService.startAsync(); - LOG.debug("Starting task {}", _taskService); + _logger.debug("Starting task {}", _taskService); } private void stopTask() { - LOG.debug("Stopping task {}", _taskService); + _logger.debug("Stopping task {}", _taskService); // Flag the task to be stopped if (_taskService != null && _taskService.isRunning()) { @@ -320,7 +320,7 @@ private void stopTask() { // Best faith effort } - LOG.debug("Task {} stopped {}", _taskService, + _logger.debug("Task {} stopped {}", _taskService, Optional.ofNullable(_taskThread).map(Thread::isAlive).orElse(false) ? "unsuccessfully" : "successfully"); } }; diff --git a/datastream-common/src/main/java/com/linkedin/datastream/common/JsonUtils.java b/datastream-common/src/main/java/com/linkedin/datastream/common/JsonUtils.java index b7aaeed69..809b016d2 100644 --- a/datastream-common/src/main/java/com/linkedin/datastream/common/JsonUtils.java +++ b/datastream-common/src/main/java/com/linkedin/datastream/common/JsonUtils.java @@ -30,7 +30,7 @@ * for checking the */ public final class JsonUtils { - private static final Logger LOG = LoggerFactory.getLogger(JsonUtils.class.getName()); + private static final Logger _logger = LoggerFactory.getLogger(JsonUtils.class.getName()); private static final ObjectMapper MAPPER = new ObjectMapper(); static { @@ -69,7 +69,7 @@ public static T fromJson(String json, Class clazz, ObjectMapper mapper) { object = mapper.readValue(json, clazz); } catch (IOException e) { String errorMessage = "Failed to parse json: " + json; - ErrorLogger.logAndThrowDatastreamRuntimeException(LOG, errorMessage, e); + ErrorLogger.logAndThrowDatastreamRuntimeException(_logger, errorMessage, e); } return object; } @@ -91,7 +91,7 @@ public static T fromJson(String json, TypeReference typeRef) { object = MAPPER.readValue(json, typeRef); } catch (IOException e) { String errorMessage = "Failed to parse json: " + json; - ErrorLogger.logAndThrowDatastreamRuntimeException(LOG, errorMessage, e); + ErrorLogger.logAndThrowDatastreamRuntimeException(_logger, errorMessage, e); } return object; } @@ -109,7 +109,7 @@ public static String toJson(T object) { MAPPER.writeValue(out, object); } catch (IOException e) { String errorMessage = "Failed to serialize object: " + object; - ErrorLogger.logAndThrowDatastreamRuntimeException(LOG, errorMessage, e); + ErrorLogger.logAndThrowDatastreamRuntimeException(_logger, errorMessage, e); } return out.toString(); } diff --git a/datastream-common/src/main/java/com/linkedin/datastream/common/ReflectionUtils.java b/datastream-common/src/main/java/com/linkedin/datastream/common/ReflectionUtils.java index fd53d1b7c..8d0eed854 100644 --- a/datastream-common/src/main/java/com/linkedin/datastream/common/ReflectionUtils.java +++ b/datastream-common/src/main/java/com/linkedin/datastream/common/ReflectionUtils.java @@ -19,7 +19,7 @@ * Utility class to simplify usage of Java reflection. */ public class ReflectionUtils { - private static final Logger LOG = LoggerFactory.getLogger(ReflectionUtils.class); + private static final Logger _logger = LoggerFactory.getLogger(ReflectionUtils.class); private static final Class[][] COMPATIBLE_TYPES = { { Character.TYPE, Character.class }, @@ -47,7 +47,7 @@ public static T createInstance(String className, Object... args) { try { clazz = (Class) Class.forName(className); } catch (Exception e) { - LOG.warn("Failed to find class with name: " + className, e); + _logger.warn("Failed to find class with name: " + className, e); return null; } return createInstance(clazz, args); @@ -70,7 +70,7 @@ public static T createInstance(Class clazz, Object... args) { Constructor ctor = clazz.getDeclaredConstructor(argTypes); return ctor.newInstance(args); } catch (Exception e) { - LOG.warn("Failed to create instance for: " + clazz, e); + _logger.warn("Failed to create instance for: " + clazz, e); return null; } } @@ -93,7 +93,7 @@ public static T setField(Object object, String field, T value) throws Except fieldObj.set(object, value); return value; } catch (Exception e) { - LOG.warn(String.format("Failed to set field, object = %s field = %s value = %s", object, field, value), e); + _logger.warn(String.format("Failed to set field, object = %s field = %s value = %s", object, field, value), e); return null; } } @@ -115,7 +115,7 @@ public static T getField(Object object, String field) throws Exception { fieldObj.setAccessible(true); return (T) fieldObj.get(object); } catch (Exception e) { - LOG.warn(String.format("Failed to get field, object = %s field = %s", object, field), e); + _logger.warn(String.format("Failed to get field, object = %s field = %s", object, field), e); return null; } } @@ -192,7 +192,7 @@ public static T callMethod(Object object, String methodName, Object... args) method.setAccessible(true); return (T) method.invoke(object, args); } catch (Exception e) { - LOG.warn("Failed to invoke method: " + methodName, e); + _logger.warn("Failed to invoke method: " + methodName, e); throw e; } finally { if (method != null) { diff --git a/datastream-common/src/main/java/com/linkedin/datastream/common/VerifiableProperties.java b/datastream-common/src/main/java/com/linkedin/datastream/common/VerifiableProperties.java index 065cc5015..1380d63b4 100644 --- a/datastream-common/src/main/java/com/linkedin/datastream/common/VerifiableProperties.java +++ b/datastream-common/src/main/java/com/linkedin/datastream/common/VerifiableProperties.java @@ -23,7 +23,7 @@ */ public class VerifiableProperties { - protected static final Logger LOG = LoggerFactory.getLogger(VerifiableProperties.class); + protected static final Logger _logger = LoggerFactory.getLogger(VerifiableProperties.class); private final HashSet _referenceSet = new HashSet<>(); private final Properties _props; @@ -351,14 +351,14 @@ public List getStringList(String name, List defaultValue) { * a defensive measure against cases of extraneous or misspelled properties. */ public void verify() { - LOG.info("Verifying properties"); + _logger.info("Verifying properties"); Enumeration keys = _props.propertyNames(); while (keys.hasMoreElements()) { Object key = keys.nextElement(); if (!_referenceSet.contains(key)) { - LOG.warn("Property {} is not valid", key); + _logger.warn("Property {} is not valid", key); } else { - LOG.info("Property {} is overridden to {}", key, _props.getProperty(key.toString())); + _logger.info("Property {} is overridden to {}", key, _props.getProperty(key.toString())); } } } diff --git a/datastream-common/src/main/java/com/linkedin/datastream/common/databases/dbreader/DatabaseChunkedReader.java b/datastream-common/src/main/java/com/linkedin/datastream/common/databases/dbreader/DatabaseChunkedReader.java index 68591a34c..fc74fb9d0 100644 --- a/datastream-common/src/main/java/com/linkedin/datastream/common/databases/dbreader/DatabaseChunkedReader.java +++ b/datastream-common/src/main/java/com/linkedin/datastream/common/databases/dbreader/DatabaseChunkedReader.java @@ -59,7 +59,7 @@ * to ensure the indexes are being used for the queries and hence will be more performant. */ public class DatabaseChunkedReader implements Closeable { - private static final Logger LOG = LoggerFactory.getLogger(DatabaseChunkedReader.class); + private static final Logger _logger = LoggerFactory.getLogger(DatabaseChunkedReader.class); private final DatabaseSource _databaseSource; private final Connection _connection; @@ -128,7 +128,7 @@ public DatabaseChunkedReader(Properties props, DataSource source, String sourceQ if (StringUtils.isBlank(db)) { _database = _connection.getMetaData().getUserName(); - LOG.warn("Database name not specified. Using name derived from connection's username {}", + _logger.warn("Database name not specified. Using name derived from connection's username {}", _database); } else { _database = db; @@ -153,7 +153,7 @@ private void initializeDatabaseMetadata(List partitions) throws SQLExce _tableSchema = _databaseSource.getTableSchema(_table); if (_tableSchema == null) { - ErrorLogger.logAndThrowDatastreamRuntimeException(LOG, "Failed to get schema for table " + _table); + ErrorLogger.logAndThrowDatastreamRuntimeException(_logger, "Failed to get schema for table " + _table); } } @@ -207,13 +207,13 @@ private void executeNextChunkedQuery() throws SQLException { } private void releaseResources(String msg) { - LOG.info(msg); - LOG.info("Releasing resources"); + _logger.info(msg); + _logger.info("Releasing resources"); if (_queryResultSet != null) { try { _queryResultSet.close(); } catch (SQLException e) { - LOG.warn("Failed to close ResultSet for reader {}. Might cause resource leak", _readerId, e); + _logger.warn("Failed to close ResultSet for reader {}. Might cause resource leak", _readerId, e); } } @@ -221,7 +221,7 @@ private void releaseResources(String msg) { try { _firstStmt.close(); } catch (SQLException e) { - LOG.warn("Failed to close Statement for reader {}. Might cause resource leak", _readerId, e); + _logger.warn("Failed to close Statement for reader {}. Might cause resource leak", _readerId, e); } } @@ -229,7 +229,7 @@ private void releaseResources(String msg) { try { _queryStmt.close(); } catch (SQLException e) { - LOG.warn("Failed to close PreparedStatement for reader {}. Might cause resource leak", _readerId, e); + _logger.warn("Failed to close PreparedStatement for reader {}. Might cause resource leak", _readerId, e); } } @@ -262,7 +262,7 @@ public void subscribe(List partitions, Map checkpoint) private void loadCheckpoint(Map checkpoint) { if (checkpoint == null || checkpoint.isEmpty()) { - LOG.warn("No checkpoints supplied. Skipping checkpoint load"); + _logger.warn("No checkpoints supplied. Skipping checkpoint load"); return; } @@ -289,7 +289,7 @@ private DatabaseRow getNextRow() throws SQLException { // If column is one of the key values, save the result from query to perform chunking query in the future if (_chunkingKeys.containsKey(columnName)) { if (columnValue == null) { - ErrorLogger.logAndThrowDatastreamRuntimeException(LOG, columnName + " field is not expected to be null"); + ErrorLogger.logAndThrowDatastreamRuntimeException(_logger, columnName + " field is not expected to be null"); } _chunkingKeys.put(columnName, columnValue); } @@ -299,10 +299,10 @@ private DatabaseRow getNextRow() throws SQLException { _metrics.updateErrorRate(); if (_skipBadMessagesEnabled) { - LOG.warn("Skipping row due to SQL exception", e); + _logger.warn("Skipping row due to SQL exception", e); _metrics.updateSkipBadMessagesRate(); } else { - ErrorLogger.logAndThrowDatastreamRuntimeException(LOG, "Failed to interpret row and skipBadMessage not enabled", e); + ErrorLogger.logAndThrowDatastreamRuntimeException(_logger, "Failed to interpret row and skipBadMessage not enabled", e); } return null; } diff --git a/datastream-common/src/main/java/com/linkedin/datastream/common/databases/dbreader/DatabaseChunkedReaderConfig.java b/datastream-common/src/main/java/com/linkedin/datastream/common/databases/dbreader/DatabaseChunkedReaderConfig.java index 84fbebfd6..c76a7cd34 100644 --- a/datastream-common/src/main/java/com/linkedin/datastream/common/databases/dbreader/DatabaseChunkedReaderConfig.java +++ b/datastream-common/src/main/java/com/linkedin/datastream/common/databases/dbreader/DatabaseChunkedReaderConfig.java @@ -33,7 +33,7 @@ public class DatabaseChunkedReaderConfig { // only look for first 1000 matches that match the specified constraints and will do a full row fetch only for these. public static final String ROW_COUNT_LIMIT = "chunk.rowCountLimit"; public static final String DATABASE_QUERY_MANAGER_CLASS_NAME = "database.queryManager"; - private static final Logger LOG = LoggerFactory.getLogger(DatabaseChunkedReaderConfig.class); + private static final Logger _logger = LoggerFactory.getLogger(DatabaseChunkedReaderConfig.class); private static final int DEFAULT_QUERY_TIMEOUT_SECS = 0; private static final int DEFAULT_FETCH_SIZE = 10000; private static final long DEFAULT_ROW_COUNT_LIMIT = 50000; @@ -65,7 +65,7 @@ public DatabaseChunkedReaderConfig(Properties properties) { String queryManagerClass = verifiableProperties.getString(DATABASE_QUERY_MANAGER_CLASS_NAME); if (StringUtils.isBlank(queryManagerClass)) { String msg = "Database query manager class name is not set or is blank"; - LOG.error(msg); + _logger.error(msg); throw new DatastreamRuntimeException(msg); } _chunkedQueryManager = ReflectionUtils.createInstance(queryManagerClass); diff --git a/datastream-common/src/main/java/com/linkedin/datastream/common/translator/GenericRecordTranslator.java b/datastream-common/src/main/java/com/linkedin/datastream/common/translator/GenericRecordTranslator.java index b03fc33bf..da9ba8482 100644 --- a/datastream-common/src/main/java/com/linkedin/datastream/common/translator/GenericRecordTranslator.java +++ b/datastream-common/src/main/java/com/linkedin/datastream/common/translator/GenericRecordTranslator.java @@ -15,7 +15,7 @@ */ public class GenericRecordTranslator implements RecordTranslator, SchemaTranslator { - private static final Logger LOG = LoggerFactory.getLogger(GenericRecordTranslator.class.getSimpleName()); + private static final Logger _logger = LoggerFactory.getLogger(GenericRecordTranslator.class.getSimpleName()); /** * Translates values of record into the internal format * diff --git a/datastream-common/src/main/java/com/linkedin/datastream/common/translator/JsonTranslator.java b/datastream-common/src/main/java/com/linkedin/datastream/common/translator/JsonTranslator.java index e33b59741..ecd96ab0b 100644 --- a/datastream-common/src/main/java/com/linkedin/datastream/common/translator/JsonTranslator.java +++ b/datastream-common/src/main/java/com/linkedin/datastream/common/translator/JsonTranslator.java @@ -21,7 +21,7 @@ * Connector that implements RecordTranslator and SchemaTranslator to support Json. */ public class JsonTranslator implements RecordTranslator, SchemaTranslator { - private static final Logger LOG = LoggerFactory.getLogger(JsonTranslator.class.getSimpleName()); + private static final Logger _logger = LoggerFactory.getLogger(JsonTranslator.class.getSimpleName()); private static final String SCHEMA_FIELD_NAME = "schema"; private static final String PAYLOAD_FIELD_NAME = "payload"; diff --git a/datastream-common/src/main/java/com/linkedin/datastream/common/translator/ResultSetTranslator.java b/datastream-common/src/main/java/com/linkedin/datastream/common/translator/ResultSetTranslator.java index 3221c3258..afe1be576 100644 --- a/datastream-common/src/main/java/com/linkedin/datastream/common/translator/ResultSetTranslator.java +++ b/datastream-common/src/main/java/com/linkedin/datastream/common/translator/ResultSetTranslator.java @@ -64,7 +64,7 @@ public class ResultSetTranslator implements RecordTranslator>, SchemaTranslator { private static final String CLASS_NAME = ResultSetTranslator.class.getSimpleName(); - private static final Logger LOG = LoggerFactory.getLogger(CLASS_NAME); + private static final Logger _logger = LoggerFactory.getLogger(CLASS_NAME); public static final int MAX_DIGITS_IN_BIGINT = 19; public static final int MAX_DIGITS_IN_INT = 9; @@ -352,7 +352,7 @@ public Schema translateSchemaToInternalFormat(ResultSet rs) throws SQLException try { sqlType = meta.getColumnTypeName(i); } catch (SQLException ex) { - LOG.warn("Column type name not found for value {}", meta.getColumnType(i)); + _logger.warn("Column type name not found for value {}", meta.getColumnType(i)); } final Schema timestampMilliType = LogicalTypes.timestampMillis().addToSchema(SchemaBuilder.builder().longType()); if (sqlType != null) { diff --git a/datastream-common/src/main/java/com/linkedin/datastream/common/translator/TranslatorConstants.java b/datastream-common/src/main/java/com/linkedin/datastream/common/translator/TranslatorConstants.java index bc58e74df..8ac450721 100644 --- a/datastream-common/src/main/java/com/linkedin/datastream/common/translator/TranslatorConstants.java +++ b/datastream-common/src/main/java/com/linkedin/datastream/common/translator/TranslatorConstants.java @@ -19,4 +19,5 @@ public class TranslatorConstants { * Represents the record name in the Avro schema */ protected static final String AVRO_SCHEMA_RECORD_NAME = "sqlRecord"; + public static final String AVRO_FIELD_SEQ_NO = "seq_no"; } diff --git a/datastream-common/src/main/java/com/linkedin/datastream/findbug/SuppressFBWarnings.java b/datastream-common/src/main/java/com/linkedin/datastream/findbug/SuppressFBWarnings.java new file mode 100644 index 000000000..0e4be533f --- /dev/null +++ b/datastream-common/src/main/java/com/linkedin/datastream/findbug/SuppressFBWarnings.java @@ -0,0 +1,27 @@ +/** + * Copyright 2020 Wayfair LLC. All rights reserved. + * Licensed under the BSD 2-Clause License. See the LICENSE file in the project root for license information. + * See the NOTICE file in the project root for additional information regarding copyright ownership. + */ +package com.linkedin.datastream.findbug; + +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; + +/** + * Annotation to suppress findBug warnings. + */ +@Retention(RetentionPolicy.CLASS) +public @interface SuppressFBWarnings { + /** + * The set of FindBugs warnings that are to be suppressed in + * annotated element. The value can be a bug category, kind or pattern. + * + */ + String[] value() default {}; + + /** + * Optional documentation of the reason why the warning is suppressed + */ + String justification() default ""; +} \ No newline at end of file diff --git a/datastream-common/src/main/java/com/linkedin/datastream/metrics/DynamicMetricsManager.java b/datastream-common/src/main/java/com/linkedin/datastream/metrics/DynamicMetricsManager.java index 259709d17..48ce7ba18 100644 --- a/datastream-common/src/main/java/com/linkedin/datastream/metrics/DynamicMetricsManager.java +++ b/datastream-common/src/main/java/com/linkedin/datastream/metrics/DynamicMetricsManager.java @@ -30,7 +30,7 @@ */ public class DynamicMetricsManager { static final String NO_KEY_PLACEHOLDER = "NO_KEY"; - private static final Logger LOG = LoggerFactory.getLogger(DynamicMetricsManager.class); + private static final Logger _logger = LoggerFactory.getLogger(DynamicMetricsManager.class); private static DynamicMetricsManager _instance = null; // Metrics indexed by simple class name, key (if exists), and metric name // Simple class name -> key -> metric name -> Metric object @@ -79,7 +79,7 @@ public static DynamicMetricsManager createInstance(MetricRegistry metricRegistry _instance._metricRegistry = metricRegistry; _instance._indexedMetrics.clear(); - LOG.info("Returning the instance for unit test {}.", testName); + _logger.info("Returning the instance for unit test {}.", testName); } return _instance; diff --git a/datastream-common/src/main/snapshot/com.linkedin.datastream.server.dms.datastream.snapshot.json b/datastream-common/src/main/snapshot/com.linkedin.datastream.server.dms.datastream.snapshot.json index 87690148b..2d52d4c83 100644 --- a/datastream-common/src/main/snapshot/com.linkedin.datastream.server.dms.datastream.snapshot.json +++ b/datastream-common/src/main/snapshot/com.linkedin.datastream.server.dms.datastream.snapshot.json @@ -87,6 +87,65 @@ "doc" : "Generic metadata for Datastream (e.g. owner, expiration, etc). Metadata is stored as user defined name/value pair.", "optional" : true } ] + }, { + "type" : "record", + "name" : "TaskHealth", + "namespace" : "com.linkedin.datastream.diagnostics", + "doc" : "Datastream task health", + "fields" : [ { + "name" : "name", + "type" : "string", + "doc" : "name of the task." + }, { + "name" : "datastreams", + "type" : "string", + "doc" : "Name of the datastreams associated with the task." + }, { + "name" : "partitions", + "type" : "string", + "doc" : "Partitions associated with the task." + }, { + "name" : "source", + "type" : "string", + "doc" : "Source of the datastream." + }, { + "name" : "destination", + "type" : "string", + "doc" : "Destination of the datastream." + }, { + "name" : "statusCode", + "type" : "string", + "doc" : "Status code of the datastream task." + }, { + "name" : "statusMessage", + "type" : "string", + "doc" : "Status message of the datastream task." + }, { + "name" : "sourceCheckpoint", + "type" : "string", + "doc" : "Source checkpoint." + } ] + }, { + "type" : "record", + "name" : "ConnectorHealth", + "namespace" : "com.linkedin.datastream.diagnostics", + "doc" : "Datastream connector health", + "fields" : [ { + "name" : "connectorName", + "type" : "string", + "doc" : "name of the connector." + }, { + "name" : "strategy", + "type" : "string", + "doc" : "Strategy used by the connector." + }, { + "name" : "tasks", + "type" : { + "type" : "array", + "items" : "TaskHealth" + }, + "doc" : "tasks assigned to the connector Instance." + } ] } ], "schema" : { "name" : "datastream", @@ -183,6 +242,26 @@ "default" : "false", "doc" : "whether or not to resume all datastreams within the given datastream's group" } ] + } ], + "subresources" : [ { + "name" : "checkpoint", + "namespace" : "com.linkedin.datastream.server.dms", + "path" : "/datastream/{datastreamId}/checkpoint", + "schema" : "com.linkedin.datastream.diagnostics.ConnectorHealth", + "doc" : "The Restli resource of {@link ConnectorHealth}. Used for collecting datastream checkpoint info\n about a {@link Datastream}.\n\ngenerated from: com.linkedin.datastream.server.dms.DatastreamSourceCheckpointResources", + "simple" : { + "supports" : [ "get", "update" ], + "methods" : [ { + "method" : "get", + "doc" : "Get source checkpoints of the parent datastream for each tasks" + }, { + "method" : "update", + "doc" : "Update source checkpoint on a given datastream" + } ], + "entity" : { + "path" : "/datastream/{datastreamId}/checkpoint" + } + } } ] } } diff --git a/datastream-common/src/test/java/com/linkedin/datastream/common/TestDurableScheduledService.java b/datastream-common/src/test/java/com/linkedin/datastream/common/TestDurableScheduledService.java index 321a6b169..81e58f9c4 100644 --- a/datastream-common/src/test/java/com/linkedin/datastream/common/TestDurableScheduledService.java +++ b/datastream-common/src/test/java/com/linkedin/datastream/common/TestDurableScheduledService.java @@ -27,7 +27,7 @@ @Test public class TestDurableScheduledService { - private static final Logger LOG = LoggerFactory.getLogger(TestDurableScheduledService.class); + private static final Logger _logger = LoggerFactory.getLogger(TestDurableScheduledService.class); private static final Duration RUN_FREQUENCY = Duration.ofMillis(10); private static final Duration RUN_TIMEOUT = Duration.ofMillis(100); @@ -41,12 +41,12 @@ public void testAlwaysHealthy() throws Exception { new DurableScheduledService("testAlwaysHealthy", RUN_FREQUENCY, RUN_TIMEOUT, RUN_TIMEOUT, RESTART_TIMEOUT) { @Override protected void startUp() { - LOG.info(Thread.currentThread().getName() + " startUp"); + _logger.info(Thread.currentThread().getName() + " startUp"); } @Override protected void runOneIteration() { - LOG.info(Thread.currentThread().getName() + " runOneIteration"); + _logger.info(Thread.currentThread().getName() + " runOneIteration"); synchronized (taskThreads) { if (!taskThreads.contains(Thread.currentThread())) { taskThreads.add(Thread.currentThread()); @@ -57,7 +57,7 @@ protected void runOneIteration() { @Override protected void shutDown() { - LOG.info(Thread.currentThread().getName() + " shutDown"); + _logger.info(Thread.currentThread().getName() + " shutDown"); } }; service.startAsync().awaitRunning(); @@ -91,13 +91,13 @@ public void testAlwaysThrows() throws Exception { new DurableScheduledService("testAlwaysThrows", RUN_FREQUENCY, RUN_TIMEOUT, RUN_TIMEOUT, RESTART_TIMEOUT) { @Override protected void startUp() { - LOG.info(Thread.currentThread().getName() + " startUp"); + _logger.info(Thread.currentThread().getName() + " startUp"); } @Override protected void runOneIteration() throws Exception { try { - LOG.info(Thread.currentThread().getName() + " runOneIteration"); + _logger.info(Thread.currentThread().getName() + " runOneIteration"); synchronized (taskThreads) { if (!taskThreads.contains(Thread.currentThread())) { taskThreads.add(Thread.currentThread()); @@ -112,7 +112,7 @@ protected void runOneIteration() throws Exception { @Override protected void shutDown() { - LOG.info(Thread.currentThread().getName() + " shutDown"); + _logger.info(Thread.currentThread().getName() + " shutDown"); } }; service.startAsync().awaitRunning(); @@ -146,12 +146,12 @@ public void testAlwaysBlock() throws Exception { new DurableScheduledService("testAlwaysBlocks", RUN_FREQUENCY, RUN_TIMEOUT, RUN_TIMEOUT, RESTART_TIMEOUT) { @Override protected void startUp() { - LOG.info(Thread.currentThread().getName() + " startUp"); + _logger.info(Thread.currentThread().getName() + " startUp"); } @Override protected void runOneIteration() throws Exception { - LOG.info(Thread.currentThread().getName() + " runOneIteration"); + _logger.info(Thread.currentThread().getName() + " runOneIteration"); runsStarted.incrementAndGet(); synchronized (taskThreads) { if (!taskThreads.contains(Thread.currentThread())) { @@ -165,7 +165,7 @@ protected void runOneIteration() throws Exception { @Override protected void signalShutdown(@Nullable final Thread taskThread) { - LOG.info(Thread.currentThread().getName() + " signalShutdown"); + _logger.info(Thread.currentThread().getName() + " signalShutdown"); if (taskThread != null && taskThread.isAlive()) { taskThread.interrupt(); } @@ -173,7 +173,7 @@ protected void signalShutdown(@Nullable final Thread taskThread) { @Override protected void shutDown() { - LOG.info(Thread.currentThread().getName() + " shutDown"); + _logger.info(Thread.currentThread().getName() + " shutDown"); } }; @@ -212,12 +212,12 @@ public void testExtendedRun() throws Exception { @Override protected void startUp() { - LOG.info(Thread.currentThread().getName() + " startUp"); + _logger.info(Thread.currentThread().getName() + " startUp"); } @Override protected void runOneIteration() throws Exception { - LOG.info(Thread.currentThread().getName() + " runOneIteration"); + _logger.info(Thread.currentThread().getName() + " runOneIteration"); runsStarted.incrementAndGet(); synchronized (taskThreads) { if (!taskThreads.contains(Thread.currentThread())) { @@ -243,7 +243,7 @@ protected void runOneIteration() throws Exception { @Override protected void signalShutdown(@Nullable final Thread taskThread) { - LOG.info(Thread.currentThread().getName() + " signalShutdown"); + _logger.info(Thread.currentThread().getName() + " signalShutdown"); if (taskThread != null && taskThread.isAlive()) { taskThread.interrupt(); } @@ -251,7 +251,7 @@ protected void signalShutdown(@Nullable final Thread taskThread) { @Override protected void shutDown() { - LOG.info(Thread.currentThread().getName() + " shutDown"); + _logger.info(Thread.currentThread().getName() + " shutDown"); } }; @@ -287,18 +287,18 @@ public void testStartWithLeak() throws Exception { new DurableScheduledService("testStartWithLeak", RUN_FREQUENCY, RUN_TIMEOUT, RUN_TIMEOUT, RESTART_TIMEOUT) { @Override protected void startUp() { - LOG.info(Thread.currentThread().getName() + " startUp"); + _logger.info(Thread.currentThread().getName() + " startUp"); } @Override protected void runOneIteration() { - LOG.info(Thread.currentThread().getName() + " runOneIteration"); + _logger.info(Thread.currentThread().getName() + " runOneIteration"); runsStarted.incrementAndGet(); } @Override protected void shutDown() { - LOG.info(Thread.currentThread().getName() + " shutDown"); + _logger.info(Thread.currentThread().getName() + " shutDown"); } @Override @@ -332,12 +332,12 @@ public void testLeak() throws Exception { new DurableScheduledService("testStartWithLeak", RUN_FREQUENCY, RUN_TIMEOUT, RUN_TIMEOUT, RESTART_TIMEOUT) { @Override protected void startUp() { - LOG.info(Thread.currentThread().getName() + " startUp"); + _logger.info(Thread.currentThread().getName() + " startUp"); } @Override protected void runOneIteration() { - LOG.info(Thread.currentThread().getName() + " runOneIteration"); + _logger.info(Thread.currentThread().getName() + " runOneIteration"); runsStarted.incrementAndGet(); synchronized (taskThreads) { if (!taskThreads.contains(Thread.currentThread())) { @@ -348,7 +348,7 @@ protected void runOneIteration() { @Override protected void shutDown() { - LOG.info(Thread.currentThread().getName() + " shutDown"); + _logger.info(Thread.currentThread().getName() + " shutDown"); } @Override diff --git a/datastream-common/src/test/java/com/linkedin/datastream/connectors/TestCommonConnectorMetrics.java b/datastream-common/src/test/java/com/linkedin/datastream/connectors/TestCommonConnectorMetrics.java index febb3da07..0a2e0a02e 100644 --- a/datastream-common/src/test/java/com/linkedin/datastream/connectors/TestCommonConnectorMetrics.java +++ b/datastream-common/src/test/java/com/linkedin/datastream/connectors/TestCommonConnectorMetrics.java @@ -27,7 +27,7 @@ * Tests for {@link CommonConnectorMetrics}. */ public class TestCommonConnectorMetrics { - private static final Logger LOG = LoggerFactory.getLogger(TestCommonConnectorMetrics.class); + private static final Logger _logger = LoggerFactory.getLogger(TestCommonConnectorMetrics.class); private static final String CONSUMER1_NAME = "CONNECTOR_CONSUMER1"; private static final String DELIMITED_CONSUMER1_NAME = "." + CONSUMER1_NAME + "."; private static final String CONSUMER2_NAME = "CONNECTOR_CONSUMER2"; @@ -45,8 +45,8 @@ public void setup(Method method) { @Test public void testConnectorEventProcessingMetrics() { - CommonConnectorMetrics connectorConsumer1 = new CommonConnectorMetrics(CLASS_NAME, CONSUMER1_NAME, LOG); - CommonConnectorMetrics connectorConsumer2 = new CommonConnectorMetrics(CLASS_NAME, CONSUMER2_NAME, LOG); + CommonConnectorMetrics connectorConsumer1 = new CommonConnectorMetrics(CLASS_NAME, CONSUMER1_NAME, _logger); + CommonConnectorMetrics connectorConsumer2 = new CommonConnectorMetrics(CLASS_NAME, CONSUMER2_NAME, _logger); connectorConsumer1.createEventProcessingMetrics(); connectorConsumer2.createEventProcessingMetrics(); @@ -101,7 +101,7 @@ public void testConnectorEventProcessingMetrics() { try { Thread.sleep(sleepTimeMS); } catch (InterruptedException e) { - LOG.info("Thread interrupted while sleeping. Exiting test testCommonConnectorMetrics"); + _logger.info("Thread interrupted while sleeping. Exiting test testCommonConnectorMetrics"); return; } @@ -115,8 +115,8 @@ public void testConnectorEventProcessingMetrics() { @Test public void testConnectorPollMetrics() { - CommonConnectorMetrics connectorConsumer1 = new CommonConnectorMetrics(CLASS_NAME, CONSUMER1_NAME, LOG); - CommonConnectorMetrics connectorConsumer2 = new CommonConnectorMetrics(CLASS_NAME, CONSUMER2_NAME, LOG); + CommonConnectorMetrics connectorConsumer1 = new CommonConnectorMetrics(CLASS_NAME, CONSUMER1_NAME, _logger); + CommonConnectorMetrics connectorConsumer2 = new CommonConnectorMetrics(CLASS_NAME, CONSUMER2_NAME, _logger); connectorConsumer1.createPollMetrics(); connectorConsumer2.createPollMetrics(); @@ -150,8 +150,8 @@ public void testConnectorPollMetrics() { @Test public void testPConnectorPartitionMetrics() { - CommonConnectorMetrics connectorConsumer1 = new CommonConnectorMetrics(CLASS_NAME, CONSUMER1_NAME, LOG); - CommonConnectorMetrics connectorConsumer2 = new CommonConnectorMetrics(CLASS_NAME, CONSUMER2_NAME, LOG); + CommonConnectorMetrics connectorConsumer1 = new CommonConnectorMetrics(CLASS_NAME, CONSUMER1_NAME, _logger); + CommonConnectorMetrics connectorConsumer2 = new CommonConnectorMetrics(CLASS_NAME, CONSUMER2_NAME, _logger); connectorConsumer1.createPartitionMetrics(); connectorConsumer2.createPartitionMetrics(); @@ -213,8 +213,8 @@ public void testPConnectorPartitionMetrics() { @Test public void testDeregisterMetrics() { - CommonConnectorMetrics metrics1 = new CommonConnectorMetrics(CLASS_NAME, CONSUMER1_NAME, LOG); - CommonConnectorMetrics metrics2 = new CommonConnectorMetrics(CLASS_NAME, CONSUMER2_NAME, LOG); + CommonConnectorMetrics metrics1 = new CommonConnectorMetrics(CLASS_NAME, CONSUMER1_NAME, _logger); + CommonConnectorMetrics metrics2 = new CommonConnectorMetrics(CLASS_NAME, CONSUMER2_NAME, _logger); metrics1.createPartitionMetrics(); metrics2.createPartitionMetrics(); @@ -250,7 +250,7 @@ public void testDeregisterMetrics() { Assert.assertNull(_metricsManager.getMetric(String.join(".", CLASS_NAME, "aggregate", "stuckPartitions"))); Assert.assertNull(_metricsManager.getMetric(String.join(".", CLASS_NAME, CONSUMER2_NAME, "stuckPartitions"))); - CommonConnectorMetrics metrics3 = new CommonConnectorMetrics(CLASS_NAME, CONSUMER3_NAME, LOG); + CommonConnectorMetrics metrics3 = new CommonConnectorMetrics(CLASS_NAME, CONSUMER3_NAME, _logger); metrics3.createPartitionMetrics(); Assert.assertNotNull(_metricsManager.getMetric(String.join(".", CLASS_NAME, "aggregate", "stuckPartitions"))); diff --git a/datastream-directory/src/main/java/com/linkedin/datastream/connectors/directory/DirectoryChangeProcessor.java b/datastream-directory/src/main/java/com/linkedin/datastream/connectors/directory/DirectoryChangeProcessor.java index 1b8975928..79e22b4b7 100644 --- a/datastream-directory/src/main/java/com/linkedin/datastream/connectors/directory/DirectoryChangeProcessor.java +++ b/datastream-directory/src/main/java/com/linkedin/datastream/connectors/directory/DirectoryChangeProcessor.java @@ -40,7 +40,7 @@ * in the file system, and propagating them to a {@link DatastreamEventProducer}. */ public class DirectoryChangeProcessor implements Runnable, AutoCloseable { - private static final Logger LOG = LoggerFactory.getLogger(DirectoryChangeProcessor.class); + private static final Logger _logger = LoggerFactory.getLogger(DirectoryChangeProcessor.class); private static final Duration ACQUIRE_TIMEOUT = Duration.ofMinutes(5); private final DatastreamTask _task; @@ -108,7 +108,7 @@ public void run() { try { _task.acquire(ACQUIRE_TIMEOUT); - LOG.info("Started watching directory {}", _dirPath); + _logger.info("Started watching directory {}", _dirPath); boolean isWatchKeyValid = true; for (WatchKey key = pollWatchService(); isWatchKeyValid; key = pollWatchService()) { @@ -135,9 +135,9 @@ public void run() { _producer.send(builder.build(), ((metadata, exception) -> { if (exception == null) { - LOG.info("Sending event succeeded"); + _logger.info("Sending event succeeded"); } else { - LOG.error("Sending event failed", exception); + _logger.error("Sending event failed", exception); } })); } @@ -146,13 +146,13 @@ public void run() { } } - LOG.warn("Watch key no longer valid. Path {} might have been altered or removed.", _dirPath); + _logger.warn("Watch key no longer valid. Path {} might have been altered or removed.", _dirPath); } catch (InterruptedException e) { Thread.currentThread().interrupt(); } catch (ClosedWatchServiceException e) { - LOG.info("WatchService closed"); + _logger.info("WatchService closed"); } catch (Exception e) { - LOG.error("Unexpected exception", e); + _logger.error("Unexpected exception", e); } finally { _task.release(); } @@ -164,7 +164,7 @@ public void close() { try { _watchService.close(); } catch (IOException e) { - LOG.error(String.format("Encountered an error during closing watch service for path %s", _dirPath), e); + _logger.error(String.format("Encountered an error during closing watch service for path %s", _dirPath), e); } } diff --git a/datastream-directory/src/main/java/com/linkedin/datastream/connectors/directory/DirectoryConnector.java b/datastream-directory/src/main/java/com/linkedin/datastream/connectors/directory/DirectoryConnector.java index 0106d3322..2da6b7b8d 100644 --- a/datastream-directory/src/main/java/com/linkedin/datastream/connectors/directory/DirectoryConnector.java +++ b/datastream-directory/src/main/java/com/linkedin/datastream/connectors/directory/DirectoryConnector.java @@ -38,7 +38,7 @@ public class DirectoryConnector implements Connector { public static final String CONNECTOR_NAME = "directory"; - private static final Logger LOG = LoggerFactory.getLogger(DirectoryConnector.class); + private static final Logger _logger = LoggerFactory.getLogger(DirectoryConnector.class); private final ConcurrentMap _directoryProcessors; private final ExecutorService _executorService; @@ -57,7 +57,7 @@ public DirectoryConnector(int threadPoolSize) { @Override public void start(CheckpointProvider checkpointProvider) { - LOG.info("DirectoryConnector started"); + _logger.info("DirectoryConnector started"); } @Override @@ -69,7 +69,7 @@ public void stop() { public void onAssignmentChange(List tasks) { Validate.notNull(tasks); - LOG.info("onAssignmentChange called with datastream tasks {}", tasks); + _logger.info("onAssignmentChange called with datastream tasks {}", tasks); Set existingTasks = _directoryProcessors.keySet(); Set newTasks = new HashSet<>(tasks); @@ -86,7 +86,7 @@ public void initializeDatastream(Datastream stream, List allDatastre Validate.notNull(stream); Validate.notNull(allDatastreams); - LOG.info("validating datastream " + stream.toString()); + _logger.info("validating datastream " + stream.toString()); String sourceDirectoryPath = stream.getSource().getConnectionString(); validateDirectoryPath(sourceDirectoryPath); @@ -113,7 +113,7 @@ private void startTaskProcessors(Iterable datastreamTasks) { _directoryProcessors.put(datastreamTask, processor); _executorService.submit(processor); } catch (IOException ex) { - LOG.error("Encountered an error while attempting to process {}", datastreamTask); + _logger.error("Encountered an error while attempting to process {}", datastreamTask); } } } @@ -124,7 +124,7 @@ private void stopTaskProcessors(Iterable datastreamTasks) { if (processor != null) { processor.close(); } - LOG.info("Processor stopped for task: " + datastreamTask); + _logger.info("Processor stopped for task: " + datastreamTask); } } diff --git a/datastream-directory/src/main/java/com/linkedin/datastream/server/DirectoryTransportProvider.java b/datastream-directory/src/main/java/com/linkedin/datastream/server/DirectoryTransportProvider.java index 90ad70102..f5b202c54 100644 --- a/datastream-directory/src/main/java/com/linkedin/datastream/server/DirectoryTransportProvider.java +++ b/datastream-directory/src/main/java/com/linkedin/datastream/server/DirectoryTransportProvider.java @@ -34,7 +34,7 @@ * This transport provider is for demonstration purposes only. */ public class DirectoryTransportProvider implements TransportProvider { - private static final Logger LOG = LoggerFactory.getLogger(DirectoryTransportProvider.class); + private static final Logger _logger = LoggerFactory.getLogger(DirectoryTransportProvider.class); @Override public void send(String destination, DatastreamProducerRecord record, SendCallback onComplete) { @@ -47,7 +47,7 @@ public void send(String destination, DatastreamProducerRecord record, SendCallba for (BrooklinEnvelope envelope : record.getEvents()) { Path sourcePath = (Path) envelope.key().get(); DirectoryEvent changeEvent = (DirectoryEvent) envelope.value().get(); - LOG.info("Received change event {} in path {}", changeEvent, sourcePath); + _logger.info("Received change event {} in path {}", changeEvent, sourcePath); switch (changeEvent) { case ENTRY_CREATED: @@ -75,18 +75,18 @@ private static void copyPathToDir(Path sourcePath, Path destinationDir) { try { copyFn.apply(sourcePath.toFile(), destinationDir.toFile()); - LOG.info("Successfully copied {} to {}", sourcePath, destinationDir); + _logger.info("Successfully copied {} to {}", sourcePath, destinationDir); } catch (IOException e) { - LOG.error("Encountered an error while copying {} to {}: {}", sourcePath, destinationDir, e.getMessage()); + _logger.error("Encountered an error while copying {} to {}: {}", sourcePath, destinationDir, e.getMessage()); } } private void deleteSubPath(Path parentPath, Path subPath) { Path deletePath = parentPath.resolve(subPath); if (FileUtils.deleteQuietly(deletePath.toFile())) { - LOG.info("Successfully deleted {}", deletePath); + _logger.info("Successfully deleted {}", deletePath); } else { - LOG.warn("Path '{}' did not exist in destination directory", deletePath); + _logger.warn("Path '{}' did not exist in destination directory", deletePath); } } diff --git a/datastream-file-connector/src/main/java/com/linkedin/datastream/connectors/file/FileConnector.java b/datastream-file-connector/src/main/java/com/linkedin/datastream/connectors/file/FileConnector.java index d3d7c5477..6bcaf7eb2 100644 --- a/datastream-file-connector/src/main/java/com/linkedin/datastream/connectors/file/FileConnector.java +++ b/datastream-file-connector/src/main/java/com/linkedin/datastream/connectors/file/FileConnector.java @@ -46,7 +46,7 @@ public class FileConnector implements Connector, DiagnosticsAware { public static final String CFG_MAX_EXEC_PROCS = "maxExecProcessors"; public static final String CFG_NUM_PARTITIONS = "numPartitions"; - private static final Logger LOG = LoggerFactory.getLogger(FileConnector.class); + private static final Logger _logger = LoggerFactory.getLogger(FileConnector.class); private static final String DEFAULT_MAX_EXEC_PROCS = "5"; private static final Duration SHUTDOWN_TIMEOUT = Duration.ofMillis(5000); @@ -72,17 +72,17 @@ public FileConnector(Properties config) { @Override public void start(CheckpointProvider checkpointProvider) { - LOG.info("FileConnector started"); + _logger.info("FileConnector started"); } @Override public synchronized void stop() { // Stop all current processors stopProcessorForTasks(_fileProcessors.keySet()); - if (!ThreadUtils.shutdownExecutor(_executorService, SHUTDOWN_TIMEOUT, LOG)) { - LOG.warn("Failed shut down cleanly."); + if (!ThreadUtils.shutdownExecutor(_executorService, SHUTDOWN_TIMEOUT, _logger)) { + _logger.warn("Failed shut down cleanly."); } - LOG.info("FileConnector is stopped."); + _logger.info("FileConnector is stopped."); } private void stopProcessorForTasks(Set unassigned) { @@ -101,13 +101,13 @@ private void stopProcessorForTasks(Set unassigned) { throw new RuntimeException("Failed to stop processor for " + task); } _fileProcessors.remove(task); - LOG.info("Processor stopped for task: " + task); + _logger.info("Processor stopped for task: " + task); } } @Override public synchronized void onAssignmentChange(List tasks) { - LOG.info("onAssignmentChange called with datastream tasks {}", tasks); + _logger.info("onAssignmentChange called with datastream tasks {}", tasks); Set unassigned = new HashSet<>(_fileProcessors.keySet()); unassigned.removeAll(tasks); @@ -117,7 +117,7 @@ public synchronized void onAssignmentChange(List tasks) { for (DatastreamTask task : tasks) { if (!_fileProcessors.containsKey(task)) { try { - LOG.info("Creating file processor for " + task); + _logger.info("Creating file processor for " + task); FileProcessor processor = new FileProcessor(task, task.getEventProducer()); _fileProcessors.put(task, processor); _executorService.submit(processor); @@ -131,7 +131,7 @@ public synchronized void onAssignmentChange(List tasks) { @Override public void initializeDatastream(Datastream stream, List allDatastreams) throws DatastreamValidationException { - LOG.info("validating datastream " + stream.toString()); + _logger.info("validating datastream " + stream.toString()); File streamFile = new File(stream.getSource().getConnectionString()); if (!streamFile.exists() || !streamFile.isFile()) { throw new DatastreamValidationException(String.format("File %s doesn't exists", streamFile.getAbsolutePath())); @@ -147,18 +147,18 @@ public void initializeDatastream(Datastream stream, List allDatastre */ @Override public String process(String query) { - LOG.info("Processing query: {}", query); + _logger.info("Processing query: {}", query); try { - String path = getPath(query, LOG); + String path = getPath(query, _logger); if (path != null && path.equalsIgnoreCase(DiagnosticsRequestType.POSITION.toString())) { final String response = processPositionRequest(); - LOG.trace("Query: {} returns response: {}", query, response); + _logger.trace("Query: {} returns response: {}", query, response); return response; } else { - LOG.warn("Could not process query {} with path {}", query, path); + _logger.warn("Could not process query {} with path {}", query, path); } } catch (Exception e) { - LOG.warn("Failed to process query {}", query, e); + _logger.warn("Failed to process query {}", query, e); throw new DatastreamRuntimeException(e); } return null; @@ -193,14 +193,14 @@ private String processPositionRequest() { */ @Override public String reduce(String query, Map responses) { - LOG.info("Reducing query {} with responses from {}.", query, responses.keySet()); + _logger.info("Reducing query {} with responses from {}.", query, responses.keySet()); try { - String path = getPath(query, LOG); + String path = getPath(query, _logger); if (path != null && path.equalsIgnoreCase(DiagnosticsRequestType.POSITION.toString())) { return JsonUtils.toJson(responses); } } catch (Exception e) { - LOG.warn("Failed to reduce responses {}", responses, e); + _logger.warn("Failed to reduce responses {}", responses, e); throw new DatastreamRuntimeException(e); } return null; diff --git a/datastream-file-connector/src/main/java/com/linkedin/datastream/connectors/file/FileProcessor.java b/datastream-file-connector/src/main/java/com/linkedin/datastream/connectors/file/FileProcessor.java index 54f7d3527..f3a3e67fe 100644 --- a/datastream-file-connector/src/main/java/com/linkedin/datastream/connectors/file/FileProcessor.java +++ b/datastream-file-connector/src/main/java/com/linkedin/datastream/connectors/file/FileProcessor.java @@ -33,7 +33,7 @@ class FileProcessor implements Runnable { - private static final Logger LOG = LoggerFactory.getLogger(FileConnector.class); + private static final Logger _logger = LoggerFactory.getLogger(FileConnector.class); private static final int PARTITION = 0; private static final int POLL_WAIT_MS = 100; @@ -66,7 +66,7 @@ public FileProcessor(DatastreamTask datastreamTask, DatastreamEventProducer prod _producer = producer; _isStopped = false; _cancelRequested = false; - LOG.info("Created FileProcessor for " + datastreamTask); + _logger.info("Created FileProcessor for " + datastreamTask); } private int loadCheckpoint() throws IOException { @@ -83,9 +83,9 @@ private int loadCheckpoint() throws IOException { _fileReader.readLine(); } - LOG.info("Resumed from line " + lineNo); + _logger.info("Resumed from line " + lineNo); } else { - LOG.info("Resumed from beginning"); + _logger.info("Resumed from beginning"); } return lineNo + 1; @@ -116,7 +116,7 @@ public void run() { new BrooklinEnvelope(_lineNo.toString().getBytes(StandardCharsets.UTF_8), text.getBytes(StandardCharsets.UTF_8), null, eventMetadata); - LOG.info("sending event " + text); + _logger.info("sending event " + text); DatastreamProducerRecordBuilder builder = new DatastreamProducerRecordBuilder(); builder.addEvent(event); builder.setEventsSourceTimestamp(currentTimeMillis); @@ -130,9 +130,9 @@ public void run() { builder.setSourceCheckpoint(_lineNo.toString()); _producer.send(builder.build(), (metadata, exception) -> { if (exception == null) { - LOG.info("Sending event:{} succeeded, metadata:{}", text, metadata); + _logger.info("Sending event:{} succeeded, metadata:{}", text, metadata); } else { - LOG.error(String.format("Sending event:{%s} failed, metadata:{%s}", text, metadata), exception); + _logger.error(String.format("Sending event:{%s} failed, metadata:{%s}", text, metadata), exception); } }); _positionValue.setLinesRead((long) _lineNo.incrementAndGet()); @@ -141,7 +141,7 @@ public void run() { // Wait for new data Thread.sleep(POLL_WAIT_MS); } catch (InterruptedException e) { - LOG.info("Interrupted"); + _logger.info("Interrupted"); break; } } @@ -149,9 +149,9 @@ public void run() { _task.release(); _isStopped = true; - LOG.info("Stopped at line " + _lineNo + " task=" + _task); + _logger.info("Stopped at line " + _lineNo + " task=" + _task); } catch (Throwable e) { - LOG.error("File processor is quitting with exception, task=" + _task, e); + _logger.error("File processor is quitting with exception, task=" + _task, e); } } diff --git a/datastream-jdbc-connector/src/main/java/com/linkedin/datastream/connectors/jdbc/DBFetchSize.java b/datastream-jdbc-connector/src/main/java/com/linkedin/datastream/connectors/jdbc/DBFetchSize.java new file mode 100644 index 000000000..5cf91a46e --- /dev/null +++ b/datastream-jdbc-connector/src/main/java/com/linkedin/datastream/connectors/jdbc/DBFetchSize.java @@ -0,0 +1,47 @@ +/** + * Copyright 2020 Wayfair LLC. All rights reserved. + * Licensed under the BSD 2-Clause License. See the LICENSE file in the project root for license information. + * See the NOTICE file in the project root for additional information regarding copyright ownership. + */ +package com.linkedin.datastream.connectors.jdbc; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * The fetch size starts from 1 and gets increase exponentially until reach the max limit. + */ +public class DBFetchSize { + private final static Logger _logger = LoggerFactory.getLogger(DBFetchSize.class); + int _maxFetchSize = 1; + int _curFetchSize = 1; + + /** + * Constructor with configured max fetch size. + * @param maxFetchSize + */ + public DBFetchSize(int maxFetchSize) { + _maxFetchSize = maxFetchSize; + } + + /** + * Return current fetch size + * @return + */ + public int get() { + _curFetchSize = _curFetchSize == _maxFetchSize ? + _curFetchSize : + Math.min(_curFetchSize * 2, _maxFetchSize); + if (_logger.isTraceEnabled()) { + _logger.trace("DB fetch size is {}", _curFetchSize); + } + return _curFetchSize; + } + + /** + * Reset fetch size to 1. + */ + public void reset() { + _curFetchSize = 1; + } +} diff --git a/datastream-jdbc-connector/src/main/java/com/linkedin/datastream/connectors/jdbc/JDBCColumn.java b/datastream-jdbc-connector/src/main/java/com/linkedin/datastream/connectors/jdbc/JDBCColumn.java new file mode 100644 index 000000000..d416e9d00 --- /dev/null +++ b/datastream-jdbc-connector/src/main/java/com/linkedin/datastream/connectors/jdbc/JDBCColumn.java @@ -0,0 +1,207 @@ +/** + * Copyright 2020 Wayfair LLC. All rights reserved. + * Licensed under the BSD 2-Clause License. See the LICENSE file in the project root for license information. + * See the NOTICE file in the project root for additional information regarding copyright ownership. + */ +package com.linkedin.datastream.connectors.jdbc; + +/** + * Represent a DB column + */ +public class JDBCColumn { + + private String name; + private int schemaSize; + private int size; + private int sqlType; + + @Override + public String toString() { + return "JDBCColumn{" + + "name='" + name + '\'' + + ", schemaSize=" + schemaSize + + ", size=" + size + + ", sqlType=" + sqlType + + ", value=" + value + + ", precision=" + precision + + ", scale=" + scale + + ", optional=" + optional + + '}'; + } + + private Object value; + private int precision; + private int scale; + + private boolean optional = true; + + private JDBCColumn(String name, int schemaSize, int sqlType, Object value, int precision, int scale, boolean optional) { + this.name = name; + this.schemaSize = schemaSize; + this.sqlType = sqlType; + this.value = value; + this.precision = precision; + this.scale = scale; + this.optional = optional; + } + + /** + * the column name + * @return + */ + public String name() { + return name; + } + + /** + * the column size defined in schema + * @return + */ + public int schemaSize() { + return schemaSize; + } + + /** + * sql type of the column + * @return + */ + public int sqlType() { + return sqlType; + } + + /** + * The value of the column + * @return + */ + public Object value() { + return value; + } + + /** + * The precision of the column + * @return + */ + public int precision() { + return precision; + } + + /** + * The scale of the column + * @return + */ + public int scale() { + return scale; + } + + /** + * Indicate the column is optional or not + * @return + */ + public boolean optional() { + return optional; + } + + /** + * Builder class + */ + public static class JDBCColumnBuilder { + + private String name; + private int schemaSize; + private int sqlType; + private Object value; + private int precision; + private int scale; + private boolean optional = true; + + private JDBCColumnBuilder() { + + } + + /** + * + * @return + */ + public static JDBCColumnBuilder builder() { + return new JDBCColumnBuilder(); + } + + /** + * + * @param name + * @return + */ + public JDBCColumnBuilder name(String name) { + this.name = name; + return this; + } + + /** + * + * @param schemaSize + * @return + */ + public JDBCColumnBuilder schemaSize(int schemaSize) { + this.schemaSize = schemaSize; + return this; + } + + /** + * + * @param sqlType + * @return + */ + public JDBCColumnBuilder sqlType(int sqlType) { + this.sqlType = sqlType; + return this; + } + + /** + * + * @param value + * @return + */ + public JDBCColumnBuilder value(Object value) { + this.value = value; + return this; + } + + /** + * + * @param precision + * @return + */ + public JDBCColumnBuilder precision(int precision) { + this.precision = precision; + return this; + } + + /** + * + * @param scale + * @return + */ + public JDBCColumnBuilder scale(int scale) { + this.scale = scale; + return this; + } + + /** + * + * @param opt + * @return + */ + public JDBCColumnBuilder optional(boolean opt) { + this.optional = opt; + return this; + } + + /** + * + * @return + */ + public JDBCColumn build() { + return new JDBCColumn(name, schemaSize, sqlType, value, precision, scale, optional); + } + } +} diff --git a/datastream-jdbc-connector/src/main/java/com/linkedin/datastream/connectors/jdbc/JDBCConnector.java b/datastream-jdbc-connector/src/main/java/com/linkedin/datastream/connectors/jdbc/JDBCConnector.java index c31d36e65..53b312d22 100644 --- a/datastream-jdbc-connector/src/main/java/com/linkedin/datastream/connectors/jdbc/JDBCConnector.java +++ b/datastream-jdbc-connector/src/main/java/com/linkedin/datastream/connectors/jdbc/JDBCConnector.java @@ -38,7 +38,7 @@ */ public class JDBCConnector implements Connector { - private static final Logger LOG = LoggerFactory.getLogger(JDBCConnector.class); + private static final Logger _logger = LoggerFactory.getLogger(JDBCConnector.class); private static final String CONFIG_JDBC_USER = "user"; private static final String CONFIG_JDBC_CREDENTIAL_NAME = "credentialName"; @@ -93,7 +93,7 @@ public JDBCConnector(VerifiableProperties config) { try { _jdbcUserPassword = Passwords.get(config.getString(CONFIG_JDBC_CREDENTIAL_NAME)); } catch (IOException e) { - LOG.error("Unable to decrypt password."); + _logger.error("Unable to decrypt password."); throw new RuntimeException(e); } @@ -107,7 +107,7 @@ public void start(CheckpointProvider checkpointProvider) { @Override public synchronized void stop() { - LOG.info("Stopping connector tasks..."); + _logger.info("Stopping connector tasks..."); _jdbcConnectorTasks.forEach((k, v) -> v.stop()); } @@ -118,7 +118,7 @@ public String getDestinationName(Datastream stream) { @Override public synchronized void onAssignmentChange(List tasks) { - LOG.info("onAssignmentChange called with datastream tasks {}", tasks); + _logger.info("onAssignmentChange called with datastream tasks {}", tasks); Set unassigned = new HashSet<>(_jdbcConnectorTasks.keySet()); unassigned.removeAll(tasks); @@ -130,7 +130,7 @@ public synchronized void onAssignmentChange(List tasks) { for (DatastreamTask task : tasks) { if (!_jdbcConnectorTasks.containsKey(task)) { - LOG.info("Creating JDBC connector task for " + task); + _logger.info("Creating JDBC connector task for " + task); String connString = task.getDatastreamSource().getConnectionString(); @@ -177,7 +177,7 @@ public synchronized void onAssignmentChange(List tasks) { try { builder.setMaxPollRows(Integer.parseInt(metadata.get(DS_CONFIG_MAX_POLL_ROWS))); } catch (NumberFormatException e) { - LOG.warn(DS_CONFIG_MAX_POLL_ROWS + " config value is not a valid number. Using the default value " + DEFAULT_MAX_POLL_RECORDS); + _logger.warn(DS_CONFIG_MAX_POLL_ROWS + " config value is not a valid number. Using the default value " + DEFAULT_MAX_POLL_RECORDS); builder.setMaxPollRows(DEFAULT_MAX_POLL_RECORDS); } } else { @@ -188,7 +188,7 @@ public synchronized void onAssignmentChange(List tasks) { try { builder.setMaxFetchSize(Integer.parseInt(metadata.get(DS_CONFIG_MAX_FETCH_SIZE))); } catch (NumberFormatException e) { - LOG.warn(DS_CONFIG_MAX_FETCH_SIZE + " config value is not a valid number. Using the default value " + DEFAULT_MAX_FETCH_SIZE); + _logger.warn(DS_CONFIG_MAX_FETCH_SIZE + " config value is not a valid number. Using the default value " + DEFAULT_MAX_FETCH_SIZE); builder.setMaxFetchSize(DEFAULT_MAX_FETCH_SIZE); } } else { diff --git a/datastream-jdbc-connector/src/main/java/com/linkedin/datastream/connectors/jdbc/JDBCConnectorTask.java b/datastream-jdbc-connector/src/main/java/com/linkedin/datastream/connectors/jdbc/JDBCConnectorTask.java index efd1e7486..8ed6a58ae 100644 --- a/datastream-jdbc-connector/src/main/java/com/linkedin/datastream/connectors/jdbc/JDBCConnectorTask.java +++ b/datastream-jdbc-connector/src/main/java/com/linkedin/datastream/connectors/jdbc/JDBCConnectorTask.java @@ -5,7 +5,6 @@ */ package com.linkedin.datastream.connectors.jdbc; - import java.io.IOException; import java.sql.Connection; import java.sql.PreparedStatement; @@ -33,14 +32,16 @@ import com.linkedin.datastream.server.DatastreamProducerRecordBuilder; import com.linkedin.datastream.server.FlushlessEventProducerHandler; import com.linkedin.datastream.server.providers.CustomCheckpointProvider; +import com.linkedin.datastream.server.providers.JDBCCheckpoint; import com.linkedin.datastream.server.providers.KafkaCustomCheckpointProvider; + /** * Connector task that reads data from SQL server and streams to the specified transportprovider. */ public class JDBCConnectorTask { - private static final Logger LOG = LoggerFactory.getLogger(JDBCConnectorTask.class); + private static final Logger _logger = LoggerFactory.getLogger(JDBCConnectorTask.class); private final ScheduledExecutorService _scheduler = Executors.newScheduledThreadPool(1); private final String _datastreamName; @@ -53,13 +54,14 @@ public class JDBCConnectorTask { private final String _checkpointStorerURL; private final String _checkpointStoreTopic; private final String _destinationTopic; - private final FlushlessEventProducerHandler _flushlessProducer; + private final FlushlessEventProducerHandler _flushlessProducer; private final int _maxPollRows; private final int _maxFetchSize; private String _id; - private CustomCheckpointProvider _checkpointProvider; + private CustomCheckpointProvider _checkpointProvider; private AtomicBoolean _resetToSafeCommit; + private JDBCCheckpoint _committingCheckpoint; private JDBCConnectorTask(JDBCConnectorTaskBuilder builder) { this._datastreamName = builder._datastreamName; @@ -92,14 +94,17 @@ private void processResults(ResultSet resultSet) throws SQLException, IOExceptio ArrayList arrayRecords = resultSetTranslator.translateToInternalFormat(resultSet); for (GenericRecord record : arrayRecords) { - Long checkpoint = (record.get(_incrementingColumnName) instanceof Integer) ? + Long checkpointVal = (record.get(_incrementingColumnName) instanceof Integer) ? Long.valueOf((Integer) record.get(_incrementingColumnName)) : (Long) record.get(_incrementingColumnName); - if (checkpoint == null) { - LOG.error("failed to send row because checkpoint is null in datastream: {}", _datastreamName); + if (checkpointVal == null) { + _logger.error("failed to send row because checkpoint is null in datastream: {}", _datastreamName); return; } - GenericRecord checkpointRecord = longTranslator.translateToInternalFormat(checkpoint); + + JDBCCheckpoint checkpoint = new JDBCCheckpoint(checkpointVal); + + GenericRecord checkpointRecord = longTranslator.translateToInternalFormat(checkpoint.offset()); HashMap meta = new HashMap<>(); meta.put(BrooklinEnvelopeMetadataConstants.EVENT_TIMESTAMP, String.valueOf(System.currentTimeMillis())); BrooklinEnvelope envelope = new BrooklinEnvelope(checkpointRecord, record, null, meta); @@ -111,22 +116,22 @@ private void processResults(ResultSet resultSet) throws SQLException, IOExceptio (DatastreamRecordMetadata metadata, Exception exception) -> { if (exception != null) { _resetToSafeCommit.set(true); - LOG.warn("failed to send row {}. {}", checkpoint, exception); + _logger.warn("failed to send row {}. {}", checkpoint, exception); } }); - _checkpointProvider.updateCheckpoint(checkpoint); + _committingCheckpoint = checkpoint; } - } private synchronized void mayCommitCheckpoint() { - Optional safeCheckpoint = _flushlessProducer.getAckCheckpoint(_id, 0); + Optional safeCheckpoint = _flushlessProducer.getAckCheckpoint(_id, 0); if (safeCheckpoint.isPresent()) { - if (_resetToSafeCommit.get()) { - _checkpointProvider.rewindTo(safeCheckpoint.get()); + _logger.debug("New safe checkpoint is {}", safeCheckpoint.get()); + _checkpointProvider.commit(safeCheckpoint.get()); + + if (_resetToSafeCommit.get()) { // Something wrong, rewind committingCheckpoint + _committingCheckpoint = safeCheckpoint.get(); _resetToSafeCommit.set(false); - } else { - _checkpointProvider.commit(safeCheckpoint.get()); } } } @@ -143,32 +148,35 @@ private String generateStatement() { } private void poll() { - Long checkpoint = null; - LOG.info("poll initiated for {}", _datastreamName); + _logger.info("poll initiated for {}", _datastreamName); mayCommitCheckpoint(); - try { - checkpoint = _checkpointProvider.getSafeCheckpoint(); - } catch (Exception e) { - LOG.error("Ignoring this poll because of exception caught {}", e); - return; + + if (_committingCheckpoint == null) { + // try getting checkpoint from CheckpointProvider. + _committingCheckpoint = _checkpointProvider.getSafeCheckpoint( + () -> new JDBCCheckpoint.Deserializer(), + JDBCCheckpoint.class); + } + + if (_committingCheckpoint == null) { + _committingCheckpoint = new JDBCCheckpoint(getInitialCheckpoint()); } - checkpoint = (checkpoint == null) ? getInitialCheckpoint() : checkpoint; - LOG.info("start checkpoint for datastream:{} is {} with destination {}", _datastreamName, checkpoint, _destinationTopic); + _logger.info("start checkpoint for datastream:{} is {} with destination {}", _datastreamName, _committingCheckpoint, _destinationTopic); try (Connection conn = _dataSource.getConnection()) { try (PreparedStatement preparedStatement = conn.prepareStatement(generateStatement())) { preparedStatement.setMaxRows(_maxPollRows); preparedStatement.setFetchSize(_maxFetchSize); - preparedStatement.setLong(1, checkpoint); + preparedStatement.setLong(1, _committingCheckpoint.offset()); // Todo: should +1 ? try (ResultSet rs = preparedStatement.executeQuery()) { this.processResults(rs); } } } catch (SQLException | IOException e) { - LOG.warn("Failed to poll for datastream {} {}", _datastreamName, e); + _logger.warn("Failed to poll for datastream {} {}", _datastreamName, e); } } @@ -177,12 +185,12 @@ private void poll() { */ public void start() { _id = generateID(); - this._checkpointProvider = new KafkaCustomCheckpointProvider(_id, _checkpointStorerURL, _checkpointStoreTopic); + this._checkpointProvider = new KafkaCustomCheckpointProvider<>(_id, _checkpointStorerURL, _checkpointStoreTopic); _scheduler.scheduleWithFixedDelay(() -> { try { this.poll(); } catch (Exception e) { - LOG.warn("Failed poll. {}", e); + _logger.warn("Failed poll. {}", e); } }, _pollFrequencyMS, @@ -194,17 +202,17 @@ public void start() { * stop the task */ public void stop() { - LOG.info("Stopping datastream {}...", _datastreamName); + _logger.info("Stopping datastream {}...", _datastreamName); _scheduler.shutdownNow(); try { if (!_scheduler.awaitTermination(5, TimeUnit.SECONDS)) { - LOG.warn("Task scheduler shutdown timed out."); + _logger.warn("Task scheduler shutdown timed out."); } mayCommitCheckpoint(); _checkpointProvider.close(); } catch (InterruptedException e) { - LOG.warn("Interrupted while awaiting task scheduler termination."); + _logger.warn("Interrupted while awaiting task scheduler termination."); Thread.currentThread().interrupt(); } } diff --git a/datastream-jdbc-connector/src/main/java/com/linkedin/datastream/connectors/jdbc/cdc/CDCEventTranslator.java b/datastream-jdbc-connector/src/main/java/com/linkedin/datastream/connectors/jdbc/cdc/CDCEventTranslator.java new file mode 100644 index 000000000..67c8716d6 --- /dev/null +++ b/datastream-jdbc-connector/src/main/java/com/linkedin/datastream/connectors/jdbc/cdc/CDCEventTranslator.java @@ -0,0 +1,150 @@ +/** + * Copyright 2020 Wayfair LLC. All rights reserved. + * Licensed under the BSD 2-Clause License. See the LICENSE file in the project root for license information. + * See the NOTICE file in the project root for additional information regarding copyright ownership. + */ +package com.linkedin.datastream.connectors.jdbc.cdc; + +import java.sql.Types; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericRecord; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.linkedin.datastream.common.DatastreamRuntimeException; +import com.linkedin.datastream.common.translator.RecordTranslator; +import com.linkedin.datastream.connectors.jdbc.JDBCColumn; +import com.linkedin.datastream.connectors.jdbc.translator.ColumnConverter; + +import static com.linkedin.datastream.connectors.jdbc.JDBCColumn.JDBCColumnBuilder; +import static com.linkedin.datastream.connectors.jdbc.translator.TranslatorConstants.AVRO_FIELD_CHANGE_LSN; +import static com.linkedin.datastream.connectors.jdbc.translator.TranslatorConstants.AVRO_FIELD_COMMIT_LSN; +import static com.linkedin.datastream.connectors.jdbc.translator.TranslatorConstants.AVRO_FIELD_EVENT_SERIAL_NO; +import static com.linkedin.datastream.connectors.jdbc.translator.TranslatorConstants.AVRO_FIELD_TS_MS; +import static com.linkedin.datastream.connectors.jdbc.translator.TranslatorConstants.DECIMAL_BYTES_TYPE; + + +/** + * Translate a CDC event to Avro record + */ +public class CDCEventTranslator implements RecordTranslator { + private static final Logger _logger = LoggerFactory.getLogger(CDCEventTranslator.class); + + private final String _captureInstName; + private Schema _avroSchema; + private Schema _valueSchema; + private Schema _sourceSchema; + private final ColumnConverter _colConverter; + + /** + * constructor + * @param captureInstName + */ + public CDCEventTranslator(String captureInstName) { + _captureInstName = captureInstName; + _colConverter = new ColumnConverter(); + } + + /** + * Translate an event to Avro record + * @param event + * @return + */ + @Override + public GenericRecord translateToInternalFormat(ChangeEvent event) { + ensureSchema(event); + + GenericRecord rootRecord = new GenericData.Record(_avroSchema); + GenericRecord beforeRecord = buildValueRecord(event.getDataBefore()); + GenericRecord afterRecord = buildValueRecord(event.getDataAfter()); + GenericRecord sourceRecord = buildSourceRecord(event); + + rootRecord.put("before", beforeRecord); + rootRecord.put("after", afterRecord); + rootRecord.put("source", sourceRecord); + rootRecord.put("op", opStr(event.getOp()));; + rootRecord.put(AVRO_FIELD_TS_MS, event.txCommitTime().getTime()); + + return rootRecord; + } + + private GenericRecord buildValueRecord(JDBCColumn[] data) { + if (data == null) { + return null; + } + + GenericRecord record = new GenericData.Record(_valueSchema); + for (int i = 0; i < data.length; i++) { + if (data[i] != null) { + record.put(i, _colConverter.convert(data[i])); + if (_logger.isTraceEnabled()) { + _logger.trace("Column before conversion = {}, after {}", data[i], _colConverter.convert(data[i])); + } + } + } + + return record; + } + + private GenericRecord buildSourceRecord(ChangeEvent event) { + GenericRecord record = new GenericData.Record(_sourceSchema); + + Object commitLsnVal = _colConverter.convert(JDBCColumnBuilder.builder() + .sqlType(DECIMAL_BYTES_TYPE) + .value(event.getCheckpoint().lsnBytes()) + .build()); + record.put(AVRO_FIELD_COMMIT_LSN, commitLsnVal); + + Object changeLsnVal = _colConverter.convert(JDBCColumnBuilder.builder() + .sqlType(DECIMAL_BYTES_TYPE) + .value(event.getSeqVal()) + .build()); + record.put(AVRO_FIELD_CHANGE_LSN, changeLsnVal); + + Object eventSn = _colConverter.convert(JDBCColumnBuilder.builder() + .sqlType(Types.INTEGER) + .value(event.offset()) + .build()); + record.put(AVRO_FIELD_EVENT_SERIAL_NO, eventSn); + + Object txCommitTime = _colConverter.convert(JDBCColumnBuilder.builder() + .sqlType(Types.TIMESTAMP) + .value(event.txCommitTime()) + .build()); + record.put(AVRO_FIELD_TS_MS, txCommitTime); + + return record; + } + + private void ensureSchema(ChangeEvent event) { + // Don't cache schema for schema change. + CDCSchemaTranslator schemaTranslator = new CDCSchemaTranslator(_captureInstName); + _avroSchema = schemaTranslator.translateSchemaToInternalFormat(event); + _valueSchema = CDCSchemaTranslator.getValueSchema(_avroSchema); + _sourceSchema = CDCSchemaTranslator.getSourceSchema(_avroSchema); + if (_logger.isTraceEnabled()) { + _logger.trace("CDC event avro schema is {}", _avroSchema); + } + } + + private static String opStr(int op) { + String val = null; + switch (op) { + case 1: + val = "d"; + break; + case 2: + val = "i"; + break; + case 3: + case 4: + val = "u"; + break; + default: + throw new DatastreamRuntimeException("Invalid operation: " + op); + } + return val; + } +} diff --git a/datastream-jdbc-connector/src/main/java/com/linkedin/datastream/connectors/jdbc/cdc/CDCQueryBuilder.java b/datastream-jdbc-connector/src/main/java/com/linkedin/datastream/connectors/jdbc/cdc/CDCQueryBuilder.java new file mode 100644 index 000000000..bc7df050c --- /dev/null +++ b/datastream-jdbc-connector/src/main/java/com/linkedin/datastream/connectors/jdbc/cdc/CDCQueryBuilder.java @@ -0,0 +1,27 @@ +/** + * Copyright 2020 Wayfair LLC. All rights reserved. + * Licensed under the BSD 2-Clause License. See the LICENSE file in the project root for license information. + * See the NOTICE file in the project root for additional information regarding copyright ownership. + */ +package com.linkedin.datastream.connectors.jdbc.cdc; + + +/** + * To build a query to change table + */ +public interface CDCQueryBuilder { + final String PLACE_HOLDER = "#"; + int DATA_COLUMN_START_INDEX = 7; + int META_COLUMN_NUM = 7; + + int OP_INDEX = 5; + int SEQVAL_INDEX = 4; + int LSN_INDEX = 2; + int TRANSACTION_TIME_INDEX = 1; + + /** + * Generate a query string + * @return + */ + String generate(); +} diff --git a/datastream-jdbc-connector/src/main/java/com/linkedin/datastream/connectors/jdbc/cdc/CDCQueryBuilderWithOffset.java b/datastream-jdbc-connector/src/main/java/com/linkedin/datastream/connectors/jdbc/cdc/CDCQueryBuilderWithOffset.java new file mode 100644 index 000000000..3c1c4bb5e --- /dev/null +++ b/datastream-jdbc-connector/src/main/java/com/linkedin/datastream/connectors/jdbc/cdc/CDCQueryBuilderWithOffset.java @@ -0,0 +1,46 @@ +/** + * Copyright 2020 Wayfair LLC. All rights reserved. + * Licensed under the BSD 2-Clause License. See the LICENSE file in the project root for license information. + * See the NOTICE file in the project root for additional information regarding copyright ownership. + */ +package com.linkedin.datastream.connectors.jdbc.cdc; + + +/** + * Build a CDC query using Offset clause to implement pagination + */ +public class CDCQueryBuilderWithOffset implements CDCQueryBuilder { + private static final String QUERY = + "SELECT * FROM cdc.#_CT WITH (NOLOCK)" + + "WHERE __$start_lsn >= ? " + + "ORDER BY __$start_lsn, __$command_id, __$seqval, __$operation " + + "OFFSET ? ROWS " + + "FETCH NEXT ? ROWS ONLY "; + + private static final String QUERY_WITH_TS = + "SELECT sys.fn_cdc_map_lsn_to_time( __$start_lsn) as lsn_ts, *" + + "FROM cdc.#_CT WITH (NOLOCK)" + + "WHERE __$start_lsn >= ? " + + "ORDER BY __$start_lsn, __$command_id, __$seqval, __$operation " + + "OFFSET ? ROWS " + + "FETCH NEXT ? ROWS ONLY "; + + private String captureInstanceName; + private boolean withTimestamp; + + /** + * Constructor + * @param captureInstanceName + * @param withTimestamp + */ + public CDCQueryBuilderWithOffset(String captureInstanceName, boolean withTimestamp) { + this.captureInstanceName = captureInstanceName; + this.withTimestamp = withTimestamp; + } + + @Override + public String generate() { + String q = withTimestamp ? QUERY_WITH_TS : QUERY; + return q.replace(PLACE_HOLDER, captureInstanceName); + } +} diff --git a/datastream-jdbc-connector/src/main/java/com/linkedin/datastream/connectors/jdbc/cdc/CDCRowReader.java b/datastream-jdbc-connector/src/main/java/com/linkedin/datastream/connectors/jdbc/cdc/CDCRowReader.java new file mode 100644 index 000000000..9c782152b --- /dev/null +++ b/datastream-jdbc-connector/src/main/java/com/linkedin/datastream/connectors/jdbc/cdc/CDCRowReader.java @@ -0,0 +1,123 @@ +/** + * Copyright 2020 Wayfair LLC. All rights reserved. + * Licensed under the BSD 2-Clause License. See the LICENSE file in the project root for license information. + * See the NOTICE file in the project root for additional information regarding copyright ownership. + */ +package com.linkedin.datastream.connectors.jdbc.cdc; + +import java.sql.ResultSet; +import java.sql.ResultSetMetaData; +import java.sql.SQLException; +import java.sql.Timestamp; + +import com.linkedin.datastream.common.DatastreamRuntimeException; +import com.linkedin.datastream.connectors.jdbc.JDBCColumn; + + +/** + * Util class to read data from CDC table + */ +public class CDCRowReader { + ResultSet _row; + ResultSetMetaData _schema; + + /** + * constructor + */ + public CDCRowReader() { + } + + /** + * Fetch the next row from DB. Must be called before calling any read methods. + * @param resultSet + * @return + */ + public boolean next(ResultSet resultSet) { + boolean hasNext; + try { + hasNext = resultSet.next(); + _row = resultSet; + _schema = resultSet.getMetaData(); + } catch (SQLException e) { + throw new DatastreamRuntimeException(e); + } + + return hasNext; + } + + /** + * Read transaction commit time + * @return + */ + public Timestamp readTransactionTime() { + try { + Timestamp ts = _row.getTimestamp(CDCQueryBuilder.TRANSACTION_TIME_INDEX); + return ts; + } catch (SQLException e) { + throw new DatastreamRuntimeException(e); + } + } + + /** + * Read __$operation column + * @return + */ + public int readOperation() { + try { + return _row.getInt(CDCQueryBuilder.OP_INDEX); + } catch (SQLException e) { + throw new DatastreamRuntimeException(e); + } + } + + /** + * read __$seqVal + * @return + */ + public byte[] readSeqVal() { + try { + return _row.getBytes(CDCQueryBuilder.SEQVAL_INDEX); + } catch (SQLException e) { + throw new DatastreamRuntimeException(e); + } + } + + /** + * Read data columns + * @return + */ + public JDBCColumn[] readDataColumns() { + try { + int numCols = _row.getMetaData().getColumnCount(); + int size = numCols - CDCQueryBuilder.META_COLUMN_NUM; + JDBCColumn[] columns = new JDBCColumn[size]; // exclude 5 meta columns + // read columns from 6 to last-1. + for (int i = CDCQueryBuilder.DATA_COLUMN_START_INDEX; i < numCols; i++) { + columns[i - CDCQueryBuilder.DATA_COLUMN_START_INDEX] = + JDBCColumn.JDBCColumnBuilder.builder() + .name(_schema.getColumnName(i)) + .sqlType(_schema.getColumnType(i)) + .value(_row.getObject(i)) + .precision(_schema.getPrecision(i)) + .scale(_schema.getScale(i)) + .optional(_schema.isNullable(i) == 1) + .build(); + } + return columns; + } catch (SQLException e) { + throw new DatastreamRuntimeException("Failed to read data columns in each row", e); + } + } + + /** + * Read __$command_id + * @return + */ + public int readCommandId() { + try { + return _row.getInt("__$command_id"); + } catch (SQLException e) { + throw new DatastreamRuntimeException(e); + } + } +} diff --git a/datastream-jdbc-connector/src/main/java/com/linkedin/datastream/connectors/jdbc/cdc/CDCSchemaTranslator.java b/datastream-jdbc-connector/src/main/java/com/linkedin/datastream/connectors/jdbc/cdc/CDCSchemaTranslator.java new file mode 100644 index 000000000..e9ea9c9e3 --- /dev/null +++ b/datastream-jdbc-connector/src/main/java/com/linkedin/datastream/connectors/jdbc/cdc/CDCSchemaTranslator.java @@ -0,0 +1,122 @@ +/** + * Copyright 2020 Wayfair LLC. All rights reserved. + * Licensed under the BSD 2-Clause License. See the LICENSE file in the project root for license information. + * See the NOTICE file in the project root for additional information regarding copyright ownership. + */ +package com.linkedin.datastream.connectors.jdbc.cdc; + +import static com.linkedin.datastream.connectors.jdbc.translator.TranslatorConstants.AVRO_FIELD_CHANGE_LSN; +import static com.linkedin.datastream.connectors.jdbc.translator.TranslatorConstants.AVRO_FIELD_COMMIT_LSN; +import static com.linkedin.datastream.connectors.jdbc.translator.TranslatorConstants.AVRO_FIELD_EVENT_SERIAL_NO; +import static com.linkedin.datastream.connectors.jdbc.translator.TranslatorConstants.AVRO_FIELD_TS_MS; +import static com.linkedin.datastream.connectors.jdbc.translator.TranslatorConstants.AVRO_SCHEMA_NAMESPACE; +import static com.linkedin.datastream.connectors.jdbc.translator.TranslatorConstants.DECIMAL_BYTES_TYPE; +import static java.sql.Types.BIGINT; +import static java.sql.Types.INTEGER; +import static java.sql.Types.TIMESTAMP; + +import org.apache.avro.Schema; +import org.apache.avro.SchemaBuilder; + +import com.linkedin.datastream.common.translator.SchemaTranslator; +import com.linkedin.datastream.connectors.jdbc.JDBCColumn; +import com.linkedin.datastream.connectors.jdbc.translator.AvroFieldSchemaBuilder; + + +/** + * Translate a change table schema to Avro schema + */ +public class CDCSchemaTranslator implements SchemaTranslator { + String _cdcInstanceName; + + AvroFieldSchemaBuilder _fieldSchemaBuilder = new AvroFieldSchemaBuilder(); + + /** + * Constructor + * @param cdcInstanceName + */ + public CDCSchemaTranslator(String cdcInstanceName) { + this._cdcInstanceName = cdcInstanceName; + } + + @Override + public Schema translateSchemaToInternalFormat(ChangeEvent event) { + SchemaBuilder.FieldAssembler rootAssembler = + SchemaBuilder.record(_cdcInstanceName).namespace(AVRO_SCHEMA_NAMESPACE).fields(); + + Schema valueSchema = buildValueSchema(event); + Schema sourceSchema = buildSourceSchema(event); + + return rootAssembler + .name("before").type().optional().type(valueSchema) + .name("after").type().optional().type(valueSchema) + .name("source").type(sourceSchema).noDefault() + .name("op").type().stringType().noDefault() + .name("ts_ms").type().longType().noDefault() + .endRecord(); + } + + private Schema buildSourceSchema(ChangeEvent event) { + SchemaBuilder.FieldAssembler assembler = SchemaBuilder.record("Source").fields(); + + JDBCColumn commitLsn = JDBCColumn.JDBCColumnBuilder.builder() + .name(AVRO_FIELD_COMMIT_LSN) + .sqlType(DECIMAL_BYTES_TYPE).precision(24).scale(0) + .optional(false) + .build(); + _fieldSchemaBuilder.build(commitLsn, assembler); + + JDBCColumn changeLsn = JDBCColumn.JDBCColumnBuilder.builder() + .name(AVRO_FIELD_CHANGE_LSN) + .sqlType(DECIMAL_BYTES_TYPE).precision(24).scale(0) + .optional(false) + .build(); + _fieldSchemaBuilder.build(changeLsn, assembler); + + JDBCColumn eventSeqNo = JDBCColumn.JDBCColumnBuilder.builder() + .name(AVRO_FIELD_EVENT_SERIAL_NO) + .sqlType(INTEGER) + .optional(false) + .build(); + _fieldSchemaBuilder.build(eventSeqNo, assembler); + + JDBCColumn tsMs = JDBCColumn.JDBCColumnBuilder.builder() + .name(AVRO_FIELD_TS_MS) + .sqlType(BIGINT) + .optional(false) + .build(); + _fieldSchemaBuilder.build(tsMs, assembler); + + return assembler.endRecord(); + } + + private Schema buildValueSchema(ChangeEvent event) { + SchemaBuilder.FieldAssembler assembler = SchemaBuilder.record("Value").fields(); + + // DataAfter should always have data + for (JDBCColumn column: event.getDataAfter()) { + _fieldSchemaBuilder.build(column, assembler); + } + + return assembler.endRecord(); + } + + /** + * Get nested data schema of root + * @param root + * @return + */ + public static Schema getValueSchema(Schema root) { + // before record is union. + return root.getField("before").schema().getTypes().get(1); + } + + /** + * Get nested source schema of root + * @param root + * @return + */ + public static Schema getSourceSchema(Schema root) { + return root.getField("source").schema(); + } +} diff --git a/datastream-jdbc-connector/src/main/java/com/linkedin/datastream/connectors/jdbc/cdc/ChangeEvent.java b/datastream-jdbc-connector/src/main/java/com/linkedin/datastream/connectors/jdbc/cdc/ChangeEvent.java new file mode 100644 index 000000000..f116dd989 --- /dev/null +++ b/datastream-jdbc-connector/src/main/java/com/linkedin/datastream/connectors/jdbc/cdc/ChangeEvent.java @@ -0,0 +1,168 @@ +/** + * Copyright 2020 Wayfair LLC. All rights reserved. + * Licensed under the BSD 2-Clause License. See the LICENSE file in the project root for license information. + * See the NOTICE file in the project root for additional information regarding copyright ownership. + */ +package com.linkedin.datastream.connectors.jdbc.cdc; + +import java.sql.Timestamp; + +import com.linkedin.datastream.connectors.jdbc.JDBCColumn; +import com.linkedin.datastream.findbug.SuppressFBWarnings; +import com.linkedin.datastream.server.providers.CDCCheckPoint; +import com.linkedin.datastream.server.providers.Utils; + + +/** + * Represent a change generated from change table. A change could be one of including insert, update and delete + * Insert event only has valid dataAfter. Delete event only has valid dataAfter, and Update event has both dataBefore + * and dataAfter. + */ +public class ChangeEvent { + + // meta columns + private Timestamp _txStartTime; + private int _op; + private byte[] _seqVal; + private CDCCheckPoint _checkpoint; + private int _cmdId; + + private boolean _isCompleted = false; + + // data columns + private JDBCColumn[] _dataBefore; + private JDBCColumn[] _dataAfter; + + /** + * Constructor + * @param checkpoint + * @param sqlVal + * @param op + * @param cmdId + * @param txStartTime + */ + @SuppressFBWarnings(value = "EI_EXPOSE_REP2") + public ChangeEvent(CDCCheckPoint checkpoint, byte[] sqlVal, int op, int cmdId, Timestamp txStartTime) { + _checkpoint = checkpoint; + _op = op; + _seqVal = sqlVal; + _cmdId = cmdId; + _txStartTime = txStartTime; + } + + /** + * Get the checkpoint of an event. + * @return + */ + @SuppressFBWarnings(value = "EI_EXPOSE_REP") + public CDCCheckPoint getCheckpoint() { + return _checkpoint; + } + + /** + * Get operation type + * @return + */ + public int getOp() { + return _op; + } + + /** + * Get the sequence value of an event + * @return + */ + @SuppressFBWarnings(value = "EI_EXPOSE_REP") + public byte[] getSeqVal() { + return _seqVal; + } + + /** + * Get the offset + * @return + */ + public int offset() { + return _checkpoint.offset(); + } + + /** + * Get the transaction commit time (tx_end_time in db) of an event + * @return + */ + @SuppressFBWarnings(value = "EI_EXPOSE_REP") + public Timestamp txCommitTime() { + return _txStartTime; + } + + /** + * Get data before change + * @return + */ + @SuppressFBWarnings(value = "EI_EXPOSE_REP") + public JDBCColumn[] getDataBefore() { + return _dataBefore; + } + + /** + * Get data after change + * @return + */ + @SuppressFBWarnings(value = "EI_EXPOSE_REP") + public JDBCColumn[] getDataAfter() { + return _dataAfter; + } + + /** + * Set data before to event + * @param dataBefore + */ + @SuppressFBWarnings(value = "EI_EXPOSE_REP2") + public void setUpdateBefore(JDBCColumn[] dataBefore) { + this._dataBefore = dataBefore; + _isCompleted = false; + } + + /** + * Set data after to event + * @param dataAfter + */ + @SuppressFBWarnings(value = "EI_EXPOSE_REP2") + public void setUpdateAfter(JDBCColumn[] dataAfter) { + this._dataAfter = dataAfter; + _isCompleted = true; + } + + /** + * Complete a update evnt with data after and new checkpoint + * @param dataAfter + * @param seqVal + * @param checkpoint + */ + @SuppressFBWarnings(value = "EI_EXPOSE_REP2") + public void completeUpdate(JDBCColumn[] dataAfter, byte[] seqVal, CDCCheckPoint checkpoint) { + this._checkpoint = checkpoint; + this._dataAfter = dataAfter; + this._seqVal = seqVal; + this._isCompleted = true; + this._op = 4; + } + + /** + * Indicate the event is completed. + * @return + */ + public boolean isComplete() { + return _isCompleted; + } + + @Override + public String toString() { + return "ChangeEvent{" + + "txStartTime=" + _txStartTime + + ", op=" + _op + + ", seqVal=" + Utils.bytesToHex(_seqVal) + + ", checkpoint=" + _checkpoint + + ", cmdId=" + _cmdId + + ", isCompleted=" + _isCompleted + + '}'; + } +} \ No newline at end of file diff --git a/datastream-jdbc-connector/src/main/java/com/linkedin/datastream/connectors/jdbc/cdc/SQLServerCDCConnector.java b/datastream-jdbc-connector/src/main/java/com/linkedin/datastream/connectors/jdbc/cdc/SQLServerCDCConnector.java new file mode 100644 index 000000000..a68db731b --- /dev/null +++ b/datastream-jdbc-connector/src/main/java/com/linkedin/datastream/connectors/jdbc/cdc/SQLServerCDCConnector.java @@ -0,0 +1,235 @@ +/** + * Copyright 2020 Wayfair LLC. All rights reserved. + * Licensed under the BSD 2-Clause License. See the LICENSE file in the project root for license information. + * See the NOTICE file in the project root for additional information regarding copyright ownership. + */ +package com.linkedin.datastream.connectors.jdbc.cdc; + +import java.io.IOException; +import java.util.Arrays; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import javax.sql.DataSource; + +import com.linkedin.datastream.common.DatastreamRuntimeException; +import org.apache.commons.dbcp.BasicDataSource; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.wayfair.crypto.Passwords; + +import com.linkedin.data.template.StringMap; +import com.linkedin.datastream.common.Datastream; +import com.linkedin.datastream.common.DatastreamMetadataConstants; +import com.linkedin.datastream.common.VerifiableProperties; +import com.linkedin.datastream.server.DatastreamTask; +import com.linkedin.datastream.server.api.connector.Connector; +import com.linkedin.datastream.server.api.connector.DatastreamValidationException; +import com.linkedin.datastream.server.providers.CheckpointProvider; + + + +/** + * implementation of {@link Connector} + * JDBC connector to stream data from SQL tables + */ +public class SQLServerCDCConnector implements Connector { + + private static final Logger _logger = LoggerFactory.getLogger(SQLServerCDCConnector.class); + + private static final String CONFIG_JDBC_USER = "user"; + private static final String CONFIG_JDBC_CREDENTIAL_NAME = "credentialName"; + private static final String CONFIG_JDBC_PASSWORD = "password"; + private static final String CONFIG_CP_MIN_IDLE = "cpMinIdle"; + private static final String CONFIG_CP_MAX_IDLE = "cpMaxIdle"; + private static final String CONFIG_CHECKPOINT_STORE_URL = "checkpointStoreURL"; + private static final String CONFIG_CHECKPOINT_STORE_TOPIC = "checkpointStoreTopic"; + + private static final String DS_CONFIG_MAX_TASKS = "maxTasks"; + private static final String DS_CONFIG_TABLE = "table"; + private static final String DS_CONFIG_POLL_FREQUENCY_MS = "pollFrequencyMS"; + private static final String DS_CONFIG_MAX_POLL_ROWS = "maxPollRows"; + private static final String DS_CONFIG_MAX_FETCH_SIZE = "maxFetchSize"; + private static final String DS_CONFIG_DESTINATION_TOPIC = "destinationTopic"; + + private static final int DEFAULT_MAX_POLL_RECORDS = 10000; + private static final int DEFAULT_MAX_FETCH_SIZE = 1000; + + private final ConcurrentMap _datasources; + private final Map _runningTasks; + + private final String _jdbcUser; + private final String _jdbcUserPassword; + private final int _cpMinIdle; + private final int _cpMaxIdle; + private final String _checkpointStoreUrl; + private final String _checkpointStoreTopic; + + /** + * constructor for JDBCConnector + * @param config configuration options + */ + public SQLServerCDCConnector(VerifiableProperties config) { + _datasources = new ConcurrentHashMap<>(); + _runningTasks = new HashMap<>(); + + if (!config.containsKey(CONFIG_JDBC_USER) + || !config.containsKey(CONFIG_JDBC_CREDENTIAL_NAME) && !config.containsKey(CONFIG_JDBC_PASSWORD)) { + throw new RuntimeException("Config options user or password is missing"); + } + + if (!config.containsKey(CONFIG_CHECKPOINT_STORE_URL) || !config.containsKey(CONFIG_CHECKPOINT_STORE_TOPIC)) { + throw new RuntimeException("Config options checkpointStoreURL or checkpointStoreTopic is missing"); + } + + _checkpointStoreUrl = config.getProperty(CONFIG_CHECKPOINT_STORE_URL); + _checkpointStoreTopic = config.getProperty(CONFIG_CHECKPOINT_STORE_TOPIC); + + _jdbcUser = config.getString(CONFIG_JDBC_USER); + _jdbcUserPassword = getJDBCPassword(config); + + _cpMinIdle = config.getInt(CONFIG_CP_MIN_IDLE, 1); + _cpMaxIdle = config.getInt(CONFIG_CP_MAX_IDLE, 5); + } + + private String getJDBCPassword(VerifiableProperties config) { + if (config.containsKey(CONFIG_JDBC_CREDENTIAL_NAME)) { + try { + return Passwords.get(config.getString(CONFIG_JDBC_CREDENTIAL_NAME)); + } catch (IOException e) { + _logger.error("Unable to decrypt password."); + throw new DatastreamRuntimeException(e); + } + } else { + return config.getString(CONFIG_JDBC_PASSWORD, "password_is_required"); + } + } + + @Override + public void start(CheckpointProvider checkpointProvider) { + _logger.info("SQLServerCDCConnector started..."); + } + + @Override + public synchronized void stop() { + _logger.info("Stopping connector tasks..."); + _runningTasks.forEach((k, v) -> v.stop()); + } + + @Override + public String getDestinationName(Datastream stream) { + return stream.getMetadata().get(DS_CONFIG_DESTINATION_TOPIC); + } + + @Override + public synchronized void onAssignmentChange(List tasks) { + _logger.info("onAssignmentChange called with datastream tasks {}", tasks); + Set unassigned = new HashSet<>(_runningTasks.keySet()); + unassigned.removeAll(tasks); + + // Stop any unassigned tasks + unassigned.forEach(t -> { + _runningTasks.get(t).stop(); + _runningTasks.remove(t); + }); + + for (DatastreamTask task : tasks) { + if (!_runningTasks.containsKey(task)) { + _logger.info("Creating CDC connector task for " + task); + + String connString = task.getDatastreamSource().getConnectionString(); + + DataSource dataSource = _datasources.computeIfAbsent( + connString, + k -> { + BasicDataSource ds = new BasicDataSource(); + ds.setUrl(connString); + ds.setUsername(_jdbcUser); + ds.setPassword(_jdbcUserPassword); + ds.setMinIdle(_cpMinIdle); + ds.setMaxIdle(_cpMaxIdle); + ds.setInitialSize(50); + ds.setMaxActive(100); + return ds; + } + ); + + StringMap metadata = task.getDatastreams().get(0).getMetadata(); + SQLServerCDCConnectorTask.SQLServerCDCConnectorTaskBuilder builder = new SQLServerCDCConnectorTask.SQLServerCDCConnectorTaskBuilder(); + builder.setDatastreamName(task.getDatastreams().get(0).getName()) + .setEventProducer(task.getEventProducer()) + .setDataSource(dataSource) + .setPollFrequencyMS(Long.parseLong(metadata.get(DS_CONFIG_POLL_FREQUENCY_MS))) + .setDestinationTopic(metadata.get(DS_CONFIG_DESTINATION_TOPIC)) + .setCheckpointStoreURL(_checkpointStoreUrl) + .setCheckpointStoreTopic(_checkpointStoreTopic); + + if (metadata.containsKey(DS_CONFIG_TABLE)) { + builder.setTable(metadata.get(DS_CONFIG_TABLE)); + } + + if (metadata.containsKey(DS_CONFIG_MAX_POLL_ROWS)) { + try { + builder.setMaxPollRows(Integer.parseInt(metadata.get(DS_CONFIG_MAX_POLL_ROWS))); + } catch (NumberFormatException e) { + _logger.warn(DS_CONFIG_MAX_POLL_ROWS + " config value is not a valid number. Using the default value " + DEFAULT_MAX_POLL_RECORDS); + builder.setMaxPollRows(DEFAULT_MAX_POLL_RECORDS); + } + } else { + builder.setMaxPollRows(DEFAULT_MAX_POLL_RECORDS); + } + + if (metadata.containsKey(DS_CONFIG_MAX_FETCH_SIZE)) { + try { + builder.setMaxFetchSize(Integer.parseInt(metadata.get(DS_CONFIG_MAX_FETCH_SIZE))); + } catch (NumberFormatException e) { + _logger.warn(DS_CONFIG_MAX_FETCH_SIZE + " config value is not a valid number. Using the default value " + DEFAULT_MAX_FETCH_SIZE); + builder.setMaxFetchSize(DEFAULT_MAX_FETCH_SIZE); + } + } else { + builder.setMaxFetchSize(DEFAULT_MAX_FETCH_SIZE); + } + + builder.setDatastreamTask(task); + + SQLServerCDCConnectorTask connectorTask = builder.build(); + _runningTasks.put(task, connectorTask); + connectorTask.start(); + } + } + } + + @Override + public void initializeDatastream(Datastream stream, List allDatastreams) throws DatastreamValidationException { + StringMap metadata = stream.getMetadata(); + if (metadata.containsKey(DS_CONFIG_MAX_TASKS)) { + if (Integer.parseInt(stream.getMetadata().get(DS_CONFIG_MAX_TASKS)) != 1) { + throw new DatastreamValidationException("maxTasks other than value 1 is not supported."); + } + + } else { + metadata.put(DS_CONFIG_MAX_TASKS, "1"); + } + + for (String param : Arrays.asList(DS_CONFIG_POLL_FREQUENCY_MS, + DS_CONFIG_DESTINATION_TOPIC)) { + if (!metadata.containsKey(param)) { + throw new DatastreamValidationException(param + " is missing in the config"); + } + } + + if (!metadata.containsKey(DS_CONFIG_TABLE) || metadata.get(DS_CONFIG_TABLE).length() == 0) { + throw new DatastreamValidationException("Config options " + DS_CONFIG_TABLE + " must be provided."); + } + + metadata.put(DatastreamMetadataConstants.IS_CONNECTOR_MANAGED_DESTINATION_KEY, Boolean.TRUE.toString()); + + stream.setMetadata(metadata); + } + +} \ No newline at end of file diff --git a/datastream-jdbc-connector/src/main/java/com/linkedin/datastream/connectors/jdbc/cdc/SQLServerCDCConnectorFactory.java b/datastream-jdbc-connector/src/main/java/com/linkedin/datastream/connectors/jdbc/cdc/SQLServerCDCConnectorFactory.java new file mode 100644 index 000000000..717e83bba --- /dev/null +++ b/datastream-jdbc-connector/src/main/java/com/linkedin/datastream/connectors/jdbc/cdc/SQLServerCDCConnectorFactory.java @@ -0,0 +1,22 @@ +/** + * Copyright 2020 Wayfair LLC. All rights reserved. + * Licensed under the BSD 2-Clause License. See the LICENSE file in the project root for license information. + * See the NOTICE file in the project root for additional information regarding copyright ownership. + */ +package com.linkedin.datastream.connectors.jdbc.cdc; + +import java.util.Properties; + +import com.linkedin.datastream.common.VerifiableProperties; +import com.linkedin.datastream.server.api.connector.ConnectorFactory; + + +/** + * implementation of {@link ConnectorFactory} for {@link SQLServerCDCConnector} + */ +public class SQLServerCDCConnectorFactory implements ConnectorFactory { + @Override + public SQLServerCDCConnector createConnector(String connectorName, Properties config, String clusterName) { + return new SQLServerCDCConnector(new VerifiableProperties(config)); + } +} diff --git a/datastream-jdbc-connector/src/main/java/com/linkedin/datastream/connectors/jdbc/cdc/SQLServerCDCConnectorTask.java b/datastream-jdbc-connector/src/main/java/com/linkedin/datastream/connectors/jdbc/cdc/SQLServerCDCConnectorTask.java new file mode 100644 index 000000000..c5603d9c5 --- /dev/null +++ b/datastream-jdbc-connector/src/main/java/com/linkedin/datastream/connectors/jdbc/cdc/SQLServerCDCConnectorTask.java @@ -0,0 +1,591 @@ +/** + * Copyright 2020 Wayfair LLC. All rights reserved. + * Licensed under the BSD 2-Clause License. See the LICENSE file in the project root for license information. + * See the NOTICE file in the project root for additional information regarding copyright ownership. + */ +package com.linkedin.datastream.connectors.jdbc.cdc; + +import java.sql.Connection; +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; +import java.sql.Timestamp; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Optional; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import javax.sql.DataSource; + +import com.linkedin.datastream.common.DatastreamTransientException; +import org.apache.avro.generic.GenericRecord; +import org.apache.kafka.common.errors.RecordTooLargeException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.common.base.Preconditions; + +import com.linkedin.datastream.common.BrooklinEnvelope; +import com.linkedin.datastream.common.BrooklinEnvelopeMetadataConstants; +import com.linkedin.datastream.common.DatastreamRecordMetadata; +import com.linkedin.datastream.common.DatastreamRuntimeException; +import com.linkedin.datastream.connectors.jdbc.DBFetchSize; +import com.linkedin.datastream.connectors.jdbc.JDBCColumn; +import com.linkedin.datastream.server.DatastreamEventProducer; +import com.linkedin.datastream.server.DatastreamProducerRecord; +import com.linkedin.datastream.server.DatastreamProducerRecordBuilder; +import com.linkedin.datastream.server.DatastreamTask; +import com.linkedin.datastream.server.DatastreamTaskStatus; +import com.linkedin.datastream.server.FlushlessEventProducerHandler; +import com.linkedin.datastream.server.providers.CDCCheckPoint; +import com.linkedin.datastream.server.providers.CustomCheckpointProvider; +import com.linkedin.datastream.server.providers.KafkaCustomCheckpointProvider; + + +/** + * Connector task that reads data from SQL server and streams to the specified transportprovider. + */ +public class SQLServerCDCConnectorTask { + + private static final Logger _logger = LoggerFactory.getLogger(SQLServerCDCConnectorTask.class); + + private String _id; + private final String _datastreamName; + private final DatastreamTask _datastreamTask; + + private final String _sourceTable; + private String _captureInstName; + + private final String _checkpointStorerURL; + private final String _checkpointStoreTopic; + private final String _destinationTopic; + + private final long _pollFrequencyMS; + private final int _maxPollRows; + + private final ScheduledExecutorService _scheduler = Executors.newScheduledThreadPool(1); + private final DataSource _dataSource; + private final FlushlessEventProducerHandler _flushlessProducer; + private CDCRowReader _rowReader; + + private static final String GET_MIN_LSN = "SELECT sys.fn_cdc_get_min_lsn('#')"; + private static final String GET_CAPTURE_INST_SQL = + "select capture_instance from cdc.change_tables where source_object_id = OBJECT_ID('#')"; + + + private String _cdcQuery; + + private CDCCheckPoint _committingCheckpoint; + private CustomCheckpointProvider _checkpointProvider; + private AtomicBoolean _resetToSafeCommit; + + DBFetchSize _dbFetchSize; + + // Non-null pendingEvent means Connector received updateBefore only but has not got updateAfter + ChangeEvent _pendingEvent = null; + + CDCEventTranslator _eventTranslator; + + // How many extra rounds last poll job runs + // 0 means last poll is done within one round and no extra round needed. + private int _pollExecRounds = 0; + + private SQLServerCDCConnectorTask(SQLServerCDCConnectorTaskBuilder builder) { + _datastreamName = builder._datastreamName; + _datastreamTask = builder._datastreamTask; + _dataSource = builder._dataSource; + _sourceTable = builder._table; + _pollFrequencyMS = builder._pollFrequencyMS; + _flushlessProducer = new FlushlessEventProducerHandler<>(builder._producer); + _checkpointStorerURL = builder._checkpointStorerURL; + _checkpointStoreTopic = builder._checkpointStoreTopic; + _destinationTopic = builder._destinationTopic; + + _checkpointProvider = null; + _resetToSafeCommit = new AtomicBoolean(false); + + _rowReader = new CDCRowReader(); + + _maxPollRows = builder._maxPollRows; + _dbFetchSize = new DBFetchSize(builder._maxFetchSize); + } + + /** + * start the task + */ + public void start() { + try { + initTask(); + } catch (Exception e) { + _logger.error("Failed to start the task {}", _id, e); + return; + } + + _checkpointProvider = new KafkaCustomCheckpointProvider<>(_id, _checkpointStorerURL, _checkpointStoreTopic); + _scheduler.scheduleAtFixedRate(() -> { + try { + execute(); + } catch (DatastreamTransientException e) { + _logger.warn("Transient error in poll, retry needed", e); + } catch (Exception e) { + _logger.error("Failed poll.", e); + // Todo: how to handle error + } + }, + 1, + _pollFrequencyMS, TimeUnit.MILLISECONDS); + _logger.info("CDC task {} started at {} interval", _id, _pollFrequencyMS); + } + + private void initTask() { + _id = generateID(); + queryCaptureInstanceName(); + generateQuery(); + _eventTranslator = new CDCEventTranslator(_captureInstName); + + _logger.info("init task {} successfully", _id); + } + + /** + * stop the task + */ + public void stop() { + _logger.info("Stopping connector {}...", _id); + + _scheduler.shutdownNow(); + try { + if (!_scheduler.awaitTermination(5, TimeUnit.SECONDS)) { + _logger.warn("Task scheduler shutdown timed out."); + } + + // Store checkpoint + Optional safeCheckpoint = _flushlessProducer.getAckCheckpoint(_id, 0); + if (safeCheckpoint.isPresent()) { + _checkpointProvider.commit(safeCheckpoint.get()); + } + _checkpointProvider.close(); + } catch (InterruptedException e) { + _logger.warn("Interrupted while awaiting task scheduler termination."); + Thread.currentThread().interrupt(); + } + } + + protected String generateID() { + String idString = _datastreamName + "_" + _destinationTopic; + long hash = idString.hashCode(); + return String.valueOf(hash > 0 ? hash : -hash); + } + + protected void processCheckpoint() { + Optional safeCheckpoint = _flushlessProducer.getAckCheckpoint(_id, 0); + + // Commit checkpoint to CheckpointProvider if there's new checkpoint + if (safeCheckpoint.isPresent()) { + _checkpointProvider.commit(safeCheckpoint.get()); + + if (_resetToSafeCommit.get()) { // Failure happened, rewind committingCheckpoint + _committingCheckpoint = safeCheckpoint.get(); + _resetToSafeCommit.set(false); + _logger.info("Rewind the committing point to {} ", _committingCheckpoint); + } + if (_logger.isDebugEnabled()) { + _logger.debug("Got committing checkpoint from acknowledged checkpoint: {}", _committingCheckpoint); + } + } + + if (_committingCheckpoint == null) { + // try getting checkpoint from CheckpointProvider. + _committingCheckpoint = _checkpointProvider.getSafeCheckpoint( + () -> new CDCCheckPoint.Deserializer(), + CDCCheckPoint.class); + if (_logger.isDebugEnabled()) { + _logger.debug("Got committing checkpoint from persistent provider: {}", _committingCheckpoint); + } + } + + if (_committingCheckpoint == null) { + // try getting initial checkpoint from DB + _committingCheckpoint = new CDCCheckPoint(getMinLSN(), -1); + if (_logger.isDebugEnabled()) { + _logger.debug("Got committing checkpoint from DB: {}", _committingCheckpoint); + } + } + } + + protected void execute() { + _logger.info("CDC poll starts. capture instance = {}", _captureInstName); + + if (_pollExecRounds > 0) { + _pollExecRounds++; + _logger.warn("Last poll is not done yet. It would take at least {} rounds for execution", _pollExecRounds); + return; + } + + // Called right before a poll starts + processCheckpoint(); + + try (Connection conn = _dataSource.getConnection(); + PreparedStatement preparedStatement = + buildCDCStatement(conn, _committingCheckpoint.lsnBytes(), _committingCheckpoint.offset() + 1); + ResultSet rs = preparedStatement.executeQuery()) { + processChanges(rs); + } catch (SQLException e) { + _logger.error("Failed to poll CDC table {}", _captureInstName, e); + throw new DatastreamRuntimeException("Error in execute polling CDC table.", e); + } catch (OutOfMemoryError e) { + _logger.error("Failed to poll CDC table {} due to OOM", _captureInstName, e); + // One big reason to get this error is huge row. We want to reduce the fetch size to get it through. + _dbFetchSize.reset(); + } finally { + _pollExecRounds = 0; + } + } + + private void queryCaptureInstanceName() { + if (_captureInstName != null && !_captureInstName.isEmpty()) { + return; + } + + String query = GET_CAPTURE_INST_SQL.replace(CDCQueryBuilder.PLACE_HOLDER, _sourceTable); + try (Connection conn = _dataSource.getConnection(); + Statement stmt = conn.createStatement(); + ResultSet rs = stmt.executeQuery(query)) { + + if (rs.next()) { + _captureInstName = rs.getString(1); + } else { + throw new DatastreamRuntimeException("Capture instance for the table " + _sourceTable + " is not available"); + } + + _logger.info("Capture Instance of the table {} is DB: {}", _captureInstName); + } catch (SQLException e) { + throw new DatastreamRuntimeException("Failed to query Capture instance for the table " + _sourceTable, e); + } + } + + private byte[] getMinLSN() { + String query = GET_MIN_LSN.replace(CDCQueryBuilder.PLACE_HOLDER, _captureInstName); + try (Connection conn = _dataSource.getConnection(); + PreparedStatement preparedStatement = conn.prepareStatement(query); + ResultSet rs = preparedStatement.executeQuery()) { + + return rs.next() + ? rs.getBytes(1) + : null; + } catch (SQLException e) { + throw new DatastreamRuntimeException("Failed to get MinLsn from DB for capture instnce " + _captureInstName, e); + } + } + + private PreparedStatement buildCDCStatement(Connection conn, byte[] fromLsn, int offset) { + PreparedStatement preparedStatement = null; + try { + preparedStatement = conn.prepareStatement(_cdcQuery); + + preparedStatement.setBytes(1, fromLsn); + preparedStatement.setInt(2, offset); + preparedStatement.setInt(3, _maxPollRows); + + preparedStatement.setFetchSize(_dbFetchSize.get()); + } catch (SQLException e) { + _logger.error("Error in build CDC statement", e); + throw new DatastreamRuntimeException("Error in build CDC statement", e); + } + + return preparedStatement; + } + + private void generateQuery() { + _cdcQuery = new CDCQueryBuilderWithOffset(_captureInstName, true).generate(); + } + + private void processChanges(ResultSet resultSet) { + int resultSetSize = 0; + + while (_rowReader.next(resultSet)) { + long processStartTime = System.currentTimeMillis(); + resultSetSize++; + + // create checkpoint from current row and update committingCheckpoint after the row is + // sent to TransportProvider + CDCCheckPoint checkpoint = newCheckpointForRow(resultSet); + + // Create an event object for each change event + // Each insert or delete has 1 event object. + // Each update needs to merge 2 rows into one event object. + int op = _rowReader.readOperation(); + ChangeEvent curEvent = null; + if (op == 3) { // the row before update + // create a partial event + curEvent = buildEventWithUpdateBefore(checkpoint); + } else if (op == 4) { // the row after update + // complete event + curEvent = completeEventWithUpdateAfter(_pendingEvent, checkpoint); + } else { // insert or delete + curEvent = buildEventWithInsertOrDelete(op, checkpoint); + } + + if (_logger.isTraceEnabled()) { + _logger.trace("Got an event {}", curEvent); + } + + if (!curEvent.isComplete()) { // This is update before, wait for next row. + _pendingEvent = curEvent; + continue; + } else { + _pendingEvent = null; + } + + // Transform to Avro + GenericRecord record = _eventTranslator.translateToInternalFormat(curEvent); + + // Send to KafkaTransportProvider + HashMap meta = new HashMap<>(); + meta.put(BrooklinEnvelopeMetadataConstants.EVENT_TIMESTAMP, String.valueOf(System.currentTimeMillis())); + + BrooklinEnvelope envelope = new BrooklinEnvelope(checkpoint.serialize(), record, null, meta); + + DatastreamProducerRecordBuilder builder = new DatastreamProducerRecordBuilder(); + builder.addEvent(envelope); + builder.setEventsSourceTimestamp(processStartTime); + builder.setEventsProduceTimestamp(Optional.of(curEvent.txCommitTime().getTime())); + DatastreamProducerRecord producerRecord = builder.build(); + + _flushlessProducer.send(producerRecord, _id, 0, checkpoint, + (DatastreamRecordMetadata dsMetadata, Exception exception) -> { + if (exception == null) { + if (_logger.isTraceEnabled()) { + _logger.trace("sent record {} successfully", dsMetadata.getCheckpoint()); + } + } else { + onSendError(dsMetadata, exception); + } + }); + + // Here the row has been sent to TP, update committingCheckpoint + _committingCheckpoint = checkpoint; + } + _logger.info("CDC poll ends: handled {} cdc events for capture instance = {}. Committing checkpoint is {}", + resultSetSize, _captureInstName, _committingCheckpoint); + } + + private void onSendError(DatastreamRecordMetadata dsMetadata, Exception ex) { + _resetToSafeCommit.set(true); + _logger.warn("failed to send record {}.", dsMetadata.getCheckpoint(), ex); + + if (ex instanceof RecordTooLargeException) { + String msg = String.format("The record exceeds the limit of Kafka. table = %s, checkpoint = %s", + dsMetadata.getTopic(), dsMetadata.getCheckpoint()); + _datastreamTask.setStatus(DatastreamTaskStatus.error(msg)); + throw new DatastreamRuntimeException(msg); + } + } + + private CDCCheckPoint newCheckpointForRow(ResultSet row) { + try { + byte[] lsn = row.getBytes(CDCQueryBuilder.LSN_INDEX); + int offset = 0; + if (Arrays.equals(lsn, _committingCheckpoint.lsnBytes())) { + // updateBefore row will not increment committingCheckpoint. So updateAfter row should increment by 2 + offset = _pendingEvent == null ? + _committingCheckpoint.offset() + 1 : + _committingCheckpoint.offset() + 2; + } else if (_pendingEvent != null) { + // for first row in first poll where committingCheckpoint == minLSN + offset = 1; + } + return new CDCCheckPoint(lsn, offset); + } catch (SQLException e) { + throw new DatastreamRuntimeException(e); + } + } + + private ChangeEvent buildEventWithInsertOrDelete(int op, CDCCheckPoint checkpoint) { + Timestamp txTs = _rowReader.readTransactionTime(); + int cmdId = _rowReader.readCommandId(); + byte[] seqVal = _rowReader.readSeqVal(); + JDBCColumn[] dataAfter = _rowReader.readDataColumns(); + + ChangeEvent event = new ChangeEvent(checkpoint, seqVal, op, cmdId, txTs); + event.setUpdateAfter(dataAfter); + + checkEvent(event); + return event; + } + + private void checkEvent(ChangeEvent e) { + if (e.txCommitTime() == null) { + throw new DatastreamTransientException("tx_commit_time not available"); + } + } + private ChangeEvent buildEventWithUpdateBefore(CDCCheckPoint checkpoint) { + Timestamp txTs = _rowReader.readTransactionTime(); + int cmdId = _rowReader.readCommandId(); + byte[] seqVal = _rowReader.readSeqVal(); + JDBCColumn[] dataBefore = _rowReader.readDataColumns(); + + ChangeEvent event = new ChangeEvent(checkpoint, seqVal, 3, cmdId, txTs); + event.setUpdateBefore(dataBefore); + checkEvent(event); + return event; + } + + private ChangeEvent completeEventWithUpdateAfter(ChangeEvent updateBefore, CDCCheckPoint checkpoint) { + byte[] seqVal = _rowReader.readSeqVal(); + JDBCColumn[] dataAfter = _rowReader.readDataColumns(); + + String msg = String.format("updateBefore is NULL. checkpoint = %s", checkpoint); + Preconditions.checkNotNull(updateBefore, msg); + + updateBefore.completeUpdate(dataAfter, seqVal, checkpoint); + checkEvent(updateBefore); + return updateBefore; + } + + + /** + * builder class for SQLServerCDCConnectorTask + */ + public static class SQLServerCDCConnectorTaskBuilder { + private String _datastreamName; + private DatastreamTask _datastreamTask; + + private DatastreamEventProducer _producer; + + private DataSource _dataSource; + private String _table = null; + private long _pollFrequencyMS = 60000; + private int _maxPollRows; + private int _maxFetchSize; + + private String _checkpointStorerURL; + private String _checkpointStoreTopic; + private String _destinationTopic; + + /** + * set the datastream name + * + * @param datastreamName datastream name + * @return instance of SQLServerCDCConnectorTaskBuilder + */ + public SQLServerCDCConnectorTask.SQLServerCDCConnectorTaskBuilder setDatastreamName(String datastreamName) { + this._datastreamName = datastreamName; + return this; + } + + public void setDatastreamTask(DatastreamTask datastreamTask) { + _datastreamTask = datastreamTask; + } + + /** + * set the sql data source + * + * @param dataSource datasource + * @return instance of SQLServerCDCConnectorTaskBuilder + */ + public SQLServerCDCConnectorTask.SQLServerCDCConnectorTaskBuilder setDataSource(DataSource dataSource) { + this._dataSource = dataSource; + return this; + } + + /** + * set the table to query + * + * @param table table name + * @return instance of SQLServerCDCConnectorTaskBuilder + */ + public SQLServerCDCConnectorTask.SQLServerCDCConnectorTaskBuilder setTable(String table) { + this._table = table; + return this; + } + + /** + * set the poll frequency + * + * @param pollFrequencyMS poll frequency in milli seconds + * @return instance of SQLServerCDCConnectorTaskBuilder + */ + public SQLServerCDCConnectorTask.SQLServerCDCConnectorTaskBuilder setPollFrequencyMS(long pollFrequencyMS) { + this._pollFrequencyMS = pollFrequencyMS; + return this; + } + + /** + * set the event producer + * + * @param producer event producer + * @return instance of SQLServerCDCConnectorTaskBuilder + */ + public SQLServerCDCConnectorTask.SQLServerCDCConnectorTaskBuilder setEventProducer(DatastreamEventProducer producer) { + this._producer = producer; + return this; + } + + /** + * set the checkpoint store url + * + * @param checkpointStoreURL checkpoint store url + * @return instance of SQLServerCDCConnectorTaskBuilder + */ + public SQLServerCDCConnectorTask.SQLServerCDCConnectorTaskBuilder setCheckpointStoreURL(String checkpointStoreURL) { + this._checkpointStorerURL = checkpointStoreURL; + return this; + } + + /** + * set the checkpoint store topic + * + * @param checkpointStoreTopic topic name + * @return instance of SQLServerCDCConnectorTaskBuilder + */ + public SQLServerCDCConnectorTask.SQLServerCDCConnectorTaskBuilder setCheckpointStoreTopic(String checkpointStoreTopic) { + this._checkpointStoreTopic = checkpointStoreTopic; + return this; + } + + /** + * set the destination topic + * + * @param destinationTopic topic to produce to + * @return instance of SQLServerCDCConnectorTaskBuilder + */ + public SQLServerCDCConnectorTask.SQLServerCDCConnectorTaskBuilder setDestinationTopic(String destinationTopic) { + this._destinationTopic = destinationTopic; + return this; + } + + /** + * set max rows to poll + * + * @param maxPollRows max number of rows to poll + * @return instance of SQLServerCDCConnectorTaskBuilder + */ + public SQLServerCDCConnectorTask.SQLServerCDCConnectorTaskBuilder setMaxPollRows(int maxPollRows) { + this._maxPollRows = maxPollRows; + return this; + } + + /** + * set max fetch size + * + * @param maxFetchSize max rows to fetch in each network call + * @return instance of SQLServerCDCConnectorTaskBuilder + */ + public SQLServerCDCConnectorTask.SQLServerCDCConnectorTaskBuilder setMaxFetchSize(int maxFetchSize) { + this._maxFetchSize = maxFetchSize; + return this; + } + + /** + * build connector task + * + * @return instance of SQLServerCDCConnectorTask + */ + public SQLServerCDCConnectorTask build() { + return new SQLServerCDCConnectorTask(this); + } + } +} diff --git a/datastream-jdbc-connector/src/main/java/com/linkedin/datastream/connectors/jdbc/translator/AvroFieldSchemaBuilder.java b/datastream-jdbc-connector/src/main/java/com/linkedin/datastream/connectors/jdbc/translator/AvroFieldSchemaBuilder.java new file mode 100644 index 000000000..bd05d0ba1 --- /dev/null +++ b/datastream-jdbc-connector/src/main/java/com/linkedin/datastream/connectors/jdbc/translator/AvroFieldSchemaBuilder.java @@ -0,0 +1,144 @@ +/** + * Copyright 2020 Wayfair LLC. All rights reserved. + * Licensed under the BSD 2-Clause License. See the LICENSE file in the project root for license information. + * See the NOTICE file in the project root for additional information regarding copyright ownership. + */ +package com.linkedin.datastream.connectors.jdbc.translator; + +import static com.linkedin.datastream.connectors.jdbc.translator.TranslatorConstants.AVRO_SCHEMA_RECORD_NAME; +import static com.linkedin.datastream.connectors.jdbc.translator.TranslatorConstants.DECIMAL_BYTES_TYPE; +import static java.sql.Types.ARRAY; +import static java.sql.Types.BIGINT; +import static java.sql.Types.BINARY; +import static java.sql.Types.BIT; +import static java.sql.Types.BLOB; +import static java.sql.Types.BOOLEAN; +import static java.sql.Types.CHAR; +import static java.sql.Types.CLOB; +import static java.sql.Types.DATE; +import static java.sql.Types.DECIMAL; +import static java.sql.Types.DOUBLE; +import static java.sql.Types.FLOAT; +import static java.sql.Types.INTEGER; +import static java.sql.Types.LONGNVARCHAR; +import static java.sql.Types.LONGVARBINARY; +import static java.sql.Types.LONGVARCHAR; +import static java.sql.Types.NCHAR; +import static java.sql.Types.NCLOB; +import static java.sql.Types.NUMERIC; +import static java.sql.Types.NVARCHAR; +import static java.sql.Types.OTHER; +import static java.sql.Types.REAL; +import static java.sql.Types.SMALLINT; +import static java.sql.Types.SQLXML; +import static java.sql.Types.TIME; +import static java.sql.Types.TIMESTAMP; +import static java.sql.Types.TIMESTAMP_WITH_TIMEZONE; +import static java.sql.Types.TINYINT; +import static java.sql.Types.VARBINARY; +import static java.sql.Types.VARCHAR; + +import org.apache.avro.LogicalTypes; +import org.apache.avro.Schema; +import org.apache.avro.SchemaBuilder; + +import com.linkedin.datastream.connectors.jdbc.JDBCColumn; + + +/** + * Builder class to build a schema for Avro field + */ +public class AvroFieldSchemaBuilder { + + /** + * Build schema for a field + * @param column + * @param fieldAssembler + */ + public void build(JDBCColumn column, SchemaBuilder.FieldAssembler fieldAssembler) { + Schema fieldSchema = fieldSchema(column); + Schema unioned = unionIfNullable(fieldSchema, column.optional()); + fieldAssembler.name(column.name()).type(unioned).noDefault(); + } + + private Schema unionIfNullable(Schema fieldSchema, boolean optional) { + return optional ? + SchemaBuilder.builder().unionOf().nullType().and().type(fieldSchema).endUnion() : + fieldSchema; + } + + private Schema fieldSchema(JDBCColumn column) { + + switch (column.sqlType()) { + case CHAR: + case LONGNVARCHAR: + case LONGVARCHAR: + case NCHAR: + case NVARCHAR: + case VARCHAR: + case CLOB: + case NCLOB: + case OTHER: + case SQLXML: + return SchemaBuilder.builder().stringType(); + + case BINARY: + case VARBINARY: + case LONGVARBINARY: + case ARRAY: + case BLOB: + return SchemaBuilder.builder().bytesType(); + + case BIT: + case BOOLEAN: + return SchemaBuilder.builder().booleanType(); + + case INTEGER: + case SMALLINT: + case TINYINT: + return SchemaBuilder.builder().intType(); + + case BIGINT: + return SchemaBuilder.builder().longType(); + + case REAL: + return SchemaBuilder.builder().floatType(); + + case FLOAT: + if (column.scale() <= 7) { + return SchemaBuilder.builder().floatType(); + } else { + return SchemaBuilder.builder().doubleType(); + } + + case DOUBLE: + return SchemaBuilder.builder().doubleType(); + + // Since Avro 1.8, LogicalType is supported. + case DECIMAL: + case NUMERIC: + case DECIMAL_BYTES_TYPE: + return LogicalTypes.decimal(column.precision(), column.scale()) + .addToSchema(SchemaBuilder.builder().bytesType()); + + case DATE: + return LogicalTypes.date().addToSchema(SchemaBuilder.builder().intType()); + + case TIME: + return LogicalTypes.timeMillis().addToSchema(SchemaBuilder.builder().intType()); + + case TIMESTAMP: // SQL Datetime + case microsoft.sql.Types.SMALLDATETIME: // -150 represents TSQL smalldatetime type + case microsoft.sql.Types.DATETIME: // -151 represents TSQL datetime type + return CustomLogicalTypes.datetime().addToSchema(SchemaBuilder.builder().longType()); + + case TIMESTAMP_WITH_TIMEZONE: + case microsoft.sql.Types.DATETIMEOFFSET: // -155 represents TSQL DATETIMEOFFSET type + return CustomLogicalTypes.zonedTimestamp().addToSchema(SchemaBuilder.builder().stringType()); + + default: + throw new IllegalArgumentException("createSchema: Unknown SQL type " + column.sqlType() + " / " + column.name() + + " (table: " + AVRO_SCHEMA_RECORD_NAME + ", column: " + column.name() + ") cannot be converted to Avro type"); + } + } +} diff --git a/datastream-jdbc-connector/src/main/java/com/linkedin/datastream/connectors/jdbc/translator/BrooklinDatetime.java b/datastream-jdbc-connector/src/main/java/com/linkedin/datastream/connectors/jdbc/translator/BrooklinDatetime.java new file mode 100644 index 000000000..9406076e2 --- /dev/null +++ b/datastream-jdbc-connector/src/main/java/com/linkedin/datastream/connectors/jdbc/translator/BrooklinDatetime.java @@ -0,0 +1,36 @@ +/** + * Copyright 2020 Wayfair LLC. All rights reserved. + * Licensed under the BSD 2-Clause License. See the LICENSE file in the project root for license information. + * See the NOTICE file in the project root for additional information regarding copyright ownership. + */ +package com.linkedin.datastream.connectors.jdbc.translator; + + +import org.apache.avro.LogicalType; +import org.apache.avro.Schema; + +/** + * Brooklin Datetime type + */ +public class BrooklinDatetime extends LogicalType { + /** + * Constructor + * @param logicalTypeName + */ + public BrooklinDatetime(String logicalTypeName) { + super(logicalTypeName); + } + + @Override + public Schema addToSchema(Schema schema) { + Schema newSchema = super.addToSchema(schema); + // Decoder will use this property to know this is zoned timestamp + newSchema.addProp(CustomLogicalTypes.LOGICAL_TYPE_KEY, CustomLogicalTypes.BROOKLIN_DATETIME); + return newSchema; + } + + @Override + public void validate(Schema schema) { + super.validate(schema); + } +} diff --git a/datastream-jdbc-connector/src/main/java/com/linkedin/datastream/connectors/jdbc/translator/ColumnConverter.java b/datastream-jdbc-connector/src/main/java/com/linkedin/datastream/connectors/jdbc/translator/ColumnConverter.java new file mode 100644 index 000000000..14db38a90 --- /dev/null +++ b/datastream-jdbc-connector/src/main/java/com/linkedin/datastream/connectors/jdbc/translator/ColumnConverter.java @@ -0,0 +1,141 @@ +/** + * Copyright 2020 Wayfair LLC. All rights reserved. + * Licensed under the BSD 2-Clause License. See the LICENSE file in the project root for license information. + * See the NOTICE file in the project root for additional information regarding copyright ownership. + */ +package com.linkedin.datastream.connectors.jdbc.translator; + +import java.math.BigDecimal; +import java.nio.ByteBuffer; +import java.sql.Date; +import java.sql.Time; +import java.sql.Timestamp; +import java.time.LocalDate; +import java.time.temporal.ChronoUnit; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import microsoft.sql.DateTimeOffset; + +import com.linkedin.datastream.connectors.jdbc.JDBCColumn; + +import static com.linkedin.datastream.connectors.jdbc.translator.TranslatorConstants.DECIMAL_BYTES_TYPE; +import static java.sql.Types.ARRAY; +import static java.sql.Types.BINARY; +import static java.sql.Types.DATE; +import static java.sql.Types.DECIMAL; +import static java.sql.Types.NUMERIC; +import static java.sql.Types.SMALLINT; +import static java.sql.Types.TIME; +import static java.sql.Types.TIMESTAMP; +import static java.sql.Types.TIMESTAMP_WITH_TIMEZONE; +import static java.sql.Types.TINYINT; +import static java.sql.Types.VARBINARY; +import static java.time.format.DateTimeFormatter.ISO_OFFSET_DATE_TIME; + + +/** + * Convert a JDBC column to Avro field + */ +public class ColumnConverter { + private static final Logger _logger = LoggerFactory.getLogger(ColumnConverter.class); + + /** + * convert a column to an Avro field + * @param column + * @return + */ + public Object convert(JDBCColumn column) { + Object value = null; + if (column == null || column.value() == null) { + return null; + } + + if (_logger.isTraceEnabled()) { + _logger.trace("Converting column: {}", column); + } + + switch (column.sqlType()) { + case DATE: + value = convertToDate((Date) column.value()); + break; + + case TIME: + value = convertToTime((Time) column.value()); + break; + + case TIMESTAMP: // 93 + case microsoft.sql.Types.SMALLDATETIME: // -150 represents TSQL smalldatetime type + case microsoft.sql.Types.DATETIME: // -151 represents TSQL datetime type + value = convertToTimestamp((Timestamp) column.value()); + break; + + case TIMESTAMP_WITH_TIMEZONE: // 2014 + case microsoft.sql.Types.DATETIMEOFFSET: // -155 represents TSQL Datetimeoffset type + value = convertToZonedTimestamp((DateTimeOffset) column.value()); + break; + + case BINARY: + case VARBINARY: + case ARRAY: + value = convertToBytes((byte[]) column.value()); + break; + + case DECIMAL_BYTES_TYPE: // custom types for LSN + value = convertToDecimal((byte[]) column.value()); + break; + + case NUMERIC: + case DECIMAL: + value = convertToDecimal((BigDecimal) column.value(), column.scale()); + break; + + case TINYINT: + case SMALLINT: + // Avro doesn't have type for short + value = ((Short) column.value()).intValue(); + break; + + default: // for all other types + value = column.value(); + break; + } + + return value; + } + + private Object convertToZonedTimestamp(DateTimeOffset value) { + return value.getOffsetDateTime().format(ISO_OFFSET_DATE_TIME); + } + + private Object convertToTimestamp(Timestamp timestamp) { + // milli second should be enough. + return timestamp.getTime(); + } + + private Object convertToTime(Time time) { + // Avro requires int type and should be enough to hold milli second + return (int) time.getTime(); + } + + private Object convertToDate(Date date) { + // Avro requires int + return (int) ChronoUnit.DAYS.between(LocalDate.ofEpochDay(0), date.toLocalDate()); + } + + private Object convertToDecimal(BigDecimal decimal, int scale) { + BigDecimal unscaled = decimal.multiply(new BigDecimal(Math.pow(10, scale))); + return ByteBuffer.wrap(unscaled.toBigInteger().toByteArray()); + } + + private Object convertToDecimal(byte[] bytes) { + // scale is always 0 + return ByteBuffer.wrap(bytes); + } + + private Object convertToBytes(byte[] bytes) { + // Kafka Avro Serializer knows ByteBuffer rather than byte[] + return ByteBuffer.wrap(bytes); + } +} diff --git a/datastream-jdbc-connector/src/main/java/com/linkedin/datastream/connectors/jdbc/translator/CustomLogicalTypes.java b/datastream-jdbc-connector/src/main/java/com/linkedin/datastream/connectors/jdbc/translator/CustomLogicalTypes.java new file mode 100644 index 000000000..9673b115c --- /dev/null +++ b/datastream-jdbc-connector/src/main/java/com/linkedin/datastream/connectors/jdbc/translator/CustomLogicalTypes.java @@ -0,0 +1,32 @@ +/** + * Copyright 2020 Wayfair LLC. All rights reserved. + * Licensed under the BSD 2-Clause License. See the LICENSE file in the project root for license information. + * See the NOTICE file in the project root for additional information regarding copyright ownership. + */ +package com.linkedin.datastream.connectors.jdbc.translator; + + +/** + * Wrapper class for all customized logical types + */ +public class CustomLogicalTypes { + public final static String LOGICAL_TYPE_KEY = "brooklin_type_key"; + public final static String BROOKLIN_ZONED_TIMESTAMP = "brooklin_zoned_timestamp"; + public final static String BROOKLIN_DATETIME = "brooklin_datetime"; + + /** + * Factory method to create a ZonedTimestamp instance. + * @return + */ + public static ZonedTimestamp zonedTimestamp() { + return new ZonedTimestamp(BROOKLIN_ZONED_TIMESTAMP); + } + + /** + * Factory method to create a BrooklinDatetime + * @return + */ + public static BrooklinDatetime datetime() { + return new BrooklinDatetime(BROOKLIN_DATETIME); + } +} diff --git a/datastream-jdbc-connector/src/main/java/com/linkedin/datastream/connectors/jdbc/translator/TranslatorConstants.java b/datastream-jdbc-connector/src/main/java/com/linkedin/datastream/connectors/jdbc/translator/TranslatorConstants.java new file mode 100644 index 000000000..3d18c6da2 --- /dev/null +++ b/datastream-jdbc-connector/src/main/java/com/linkedin/datastream/connectors/jdbc/translator/TranslatorConstants.java @@ -0,0 +1,30 @@ +/** + * Copyright 2020 Wayfair LLC. All rights reserved. + * Licensed under the BSD 2-Clause License. See the LICENSE file in the project root for license information. + * See the NOTICE file in the project root for additional information regarding copyright ownership. + */ +package com.linkedin.datastream.connectors.jdbc.translator; + + +/** + * Constants used for translators + */ +public class TranslatorConstants { + public static final String AVRO_SCHEMA_RECORD_NAME = "sqlRecord"; + public static final String SOURCE_SQL_DATA_TYPE = "source.sql.datetype"; + + public static final int MAX_DIGITS_IN_BIGINT = 19; + public static final int MAX_DIGITS_IN_INT = 9; + + public static final int DEFAULT_PRECISION_VALUE = 10; + public static final int DEFAULT_SCALE_VALUE = 0; + public static final String AVRO_SCHEMA_NAMESPACE = "com.wayfair.brooklin"; + + public static final String AVRO_FIELD_COMMIT_LSN = "commit_lsn"; + public static final String AVRO_FIELD_CHANGE_LSN = "change_lsn"; + public static final String AVRO_FIELD_EVENT_SERIAL_NO = "event_serial_no"; + public static final String AVRO_FIELD_TS_MS = "ts_ms"; + + // used by CDC + public static final int DECIMAL_BYTES_TYPE = 1000; +} diff --git a/datastream-jdbc-connector/src/main/java/com/linkedin/datastream/connectors/jdbc/translator/ZonedTimestamp.java b/datastream-jdbc-connector/src/main/java/com/linkedin/datastream/connectors/jdbc/translator/ZonedTimestamp.java new file mode 100644 index 000000000..d3c0c7c44 --- /dev/null +++ b/datastream-jdbc-connector/src/main/java/com/linkedin/datastream/connectors/jdbc/translator/ZonedTimestamp.java @@ -0,0 +1,37 @@ +/** + * Copyright 2020 Wayfair LLC. All rights reserved. + * Licensed under the BSD 2-Clause License. See the LICENSE file in the project root for license information. + * See the NOTICE file in the project root for additional information regarding copyright ownership. + */ +package com.linkedin.datastream.connectors.jdbc.translator; + +import org.apache.avro.LogicalType; +import org.apache.avro.Schema; + + +/** + * Avro custom Logical type represents a timestamp with zone + */ +public class ZonedTimestamp extends LogicalType { + + /** + * Constructor + * @param logicalTypeName + */ + public ZonedTimestamp(String logicalTypeName) { + super(logicalTypeName); + } + + @Override + public Schema addToSchema(Schema schema) { + Schema newSchema = super.addToSchema(schema); + // Decoder will use this property to know this is zoned timestamp + newSchema.addProp(CustomLogicalTypes.LOGICAL_TYPE_KEY, CustomLogicalTypes.BROOKLIN_ZONED_TIMESTAMP); + return newSchema; + } + + @Override + public void validate(Schema schema) { + super.validate(schema); + } +} \ No newline at end of file diff --git a/datastream-kafka-connector/src/main/java/com/linkedin/datastream/connectors/kafka/AbstractKafkaBasedConnectorTask.java b/datastream-kafka-connector/src/main/java/com/linkedin/datastream/connectors/kafka/AbstractKafkaBasedConnectorTask.java index 92d8fedbe..b6fc98132 100644 --- a/datastream-kafka-connector/src/main/java/com/linkedin/datastream/connectors/kafka/AbstractKafkaBasedConnectorTask.java +++ b/datastream-kafka-connector/src/main/java/com/linkedin/datastream/connectors/kafka/AbstractKafkaBasedConnectorTask.java @@ -429,7 +429,10 @@ protected void trackEventsProcessedProgress(int recordCount) { try { long curPollTime = System.currentTimeMillis(); _lastPolledTimeMillis = curPollTime; + _logger.info("consumer {} start polling, interval = {}", _taskName, pollInterval); records = consumerPoll(pollInterval); + _logger.info("consumer {} consuming {} records", _taskName, records.count()); + long pollDurationMillis = System.currentTimeMillis() - curPollTime; if (pollDurationMillis > pollInterval + POLL_BUFFER_TIME_MILLIS) { // record poll time exceeding client poll timeout diff --git a/datastream-kafka-connector/src/main/java/com/linkedin/datastream/connectors/kafka/KafkaConnector.java b/datastream-kafka-connector/src/main/java/com/linkedin/datastream/connectors/kafka/KafkaConnector.java index ef99cb3d3..302c1f955 100644 --- a/datastream-kafka-connector/src/main/java/com/linkedin/datastream/connectors/kafka/KafkaConnector.java +++ b/datastream-kafka-connector/src/main/java/com/linkedin/datastream/connectors/kafka/KafkaConnector.java @@ -38,7 +38,7 @@ public class KafkaConnector extends AbstractKafkaConnector { public static final String CONFIG_WHITE_LISTED_CLUSTERS = "whiteListedClusters"; - private static final Logger LOG = LoggerFactory.getLogger(KafkaConnector.class); + private static final Logger _logger = LoggerFactory.getLogger(KafkaConnector.class); private final Set _whiteListedBrokers; @@ -50,7 +50,7 @@ public class KafkaConnector extends AbstractKafkaConnector { public KafkaConnector(String connectorName, Properties config, String clusterName) { super(connectorName, config, new KafkaGroupIdConstructor( Boolean.parseBoolean(config.getProperty(IS_GROUP_ID_HASHING_ENABLED, Boolean.FALSE.toString())), clusterName), - clusterName, LOG); + clusterName, _logger); VerifiableProperties verifiableProperties = new VerifiableProperties(config); List brokers = @@ -63,7 +63,7 @@ public KafkaConnector(String connectorName, Properties config, String clusterNam @Override public void initializeDatastream(Datastream stream, List allDatastreams) throws DatastreamValidationException { - LOG.info("Initialize datastream {}", stream); + _logger.info("Initialize datastream {}", stream); DatastreamSource source = stream.getSource(); String connectionString = source.getConnectionString(); @@ -85,7 +85,7 @@ public void initializeDatastream(Datastream stream, List allDatastre String msg = String.format("Kafka connector is not white-listed for the cluster %s. Current white-listed clusters %s.", connectionString, _whiteListedBrokers); - LOG.error(msg); + _logger.error(msg); throw new DatastreamValidationException(msg); } try (Consumer consumer = KafkaConnectorTask.createConsumer(_config.getConsumerFactory(), @@ -98,14 +98,14 @@ public void initializeDatastream(Datastream stream, List allDatastre } int numPartitions = partitionInfos.size(); if (!source.hasPartitions()) { - LOG.info("Kafka source {} has {} partitions.", parsed, numPartitions); + _logger.info("Kafka source {} has {} partitions.", parsed, numPartitions); source.setPartitions(numPartitions); } else { if (source.getPartitions() != numPartitions) { String msg = String.format("Source is configured with %d partitions, But the topic %s actually has %d partitions", source.getPartitions(), parsed.getTopicName(), numPartitions); - LOG.error(msg); + _logger.error(msg); throw new DatastreamValidationException(msg); } } @@ -120,7 +120,7 @@ public void initializeDatastream(Datastream stream, List allDatastre .anyMatch(x -> !offsetMap.containsKey(x))) { String msg = String.format("Missing partitions starting offset for datastream %s, json value %s", stream, json); - LOG.warn(msg); + _logger.warn(msg); throw new DatastreamValidationException(msg); } } @@ -128,7 +128,7 @@ public void initializeDatastream(Datastream stream, List allDatastre } catch (DatastreamValidationException e) { throw e; } catch (Exception e) { - LOG.error("Initialization threw an exception.", e); + _logger.error("Initialization threw an exception.", e); throw new DatastreamValidationException(e); } } @@ -150,6 +150,6 @@ protected AbstractKafkaBasedConnectorTask createKafkaBasedConnectorTask(Datastre @Override public void postDatastreamInitialize(Datastream datastream, List allDatastreams) throws DatastreamValidationException { - _groupIdConstructor.populateDatastreamGroupIdInMetadata(datastream, allDatastreams, Optional.of(LOG)); + _groupIdConstructor.populateDatastreamGroupIdInMetadata(datastream, allDatastreams, Optional.of(_logger)); } } diff --git a/datastream-kafka-connector/src/main/java/com/linkedin/datastream/connectors/kafka/KafkaConnectorTask.java b/datastream-kafka-connector/src/main/java/com/linkedin/datastream/connectors/kafka/KafkaConnectorTask.java index c276a490b..b0cc862aa 100644 --- a/datastream-kafka-connector/src/main/java/com/linkedin/datastream/connectors/kafka/KafkaConnectorTask.java +++ b/datastream-kafka-connector/src/main/java/com/linkedin/datastream/connectors/kafka/KafkaConnectorTask.java @@ -39,7 +39,7 @@ public class KafkaConnectorTask extends AbstractKafkaBasedConnectorTask { private static final String CLASS_NAME = KafkaConnectorTask.class.getSimpleName(); - private static final Logger LOG = LoggerFactory.getLogger(KafkaConnectorTask.class); + private static final Logger _logger = LoggerFactory.getLogger(KafkaConnectorTask.class); private final KafkaConnectionString _srcConnString = KafkaConnectionString.valueOf(_datastreamTask.getDatastreamSource().getConnectionString()); @@ -57,7 +57,7 @@ public class KafkaConnectorTask extends AbstractKafkaBasedConnectorTask { */ public KafkaConnectorTask(KafkaBasedConnectorConfig config, DatastreamTask task, String connectorName, GroupIdConstructor groupIdConstructor) { - super(config, task, LOG, generateMetricsPrefix(connectorName, CLASS_NAME)); + super(config, task, _logger, generateMetricsPrefix(connectorName, CLASS_NAME)); _consumerFactory = config.getConsumerFactory(); _groupIdConstructor = groupIdConstructor; } @@ -109,7 +109,7 @@ protected void consumerSubscribe() { @Override protected Consumer createKafkaConsumer(Properties consumerProps) { - return createConsumer(_consumerFactory, consumerProps, getKafkaGroupId(_datastreamTask, _groupIdConstructor, _consumerMetrics, LOG), + return createConsumer(_consumerFactory, consumerProps, getKafkaGroupId(_datastreamTask, _groupIdConstructor, _consumerMetrics, _logger), _srcConnString); } diff --git a/datastream-kafka-connector/src/main/java/com/linkedin/datastream/connectors/kafka/KafkaPositionTracker.java b/datastream-kafka-connector/src/main/java/com/linkedin/datastream/connectors/kafka/KafkaPositionTracker.java index 5f5c19af8..f53cc21b6 100644 --- a/datastream-kafka-connector/src/main/java/com/linkedin/datastream/connectors/kafka/KafkaPositionTracker.java +++ b/datastream-kafka-connector/src/main/java/com/linkedin/datastream/connectors/kafka/KafkaPositionTracker.java @@ -54,7 +54,7 @@ */ public class KafkaPositionTracker implements Closeable { - private static final Logger LOG = LoggerFactory.getLogger(KafkaPositionTracker.class); + private static final Logger _logger = LoggerFactory.getLogger(KafkaPositionTracker.class); /** * The suffix for Kafka consumer metrics that record records lag. @@ -330,7 +330,7 @@ private Optional tryCreateMetricName(@NotNull TopicPartition topicPa @Nullable final ConsumerMetricsFormatSupport consumerMetricsSupport = _consumerMetricsSupport; if (clientId == null || consumerMetricsSupport == null) { // Client metric support is unimplemented in the current consumer - LOG.trace("The current consumer does not seem to have metric support for record lag."); + _logger.trace("The current consumer does not seem to have metric support for record lag."); return Optional.empty(); } @@ -353,7 +353,7 @@ private Optional tryCreateMetricName(@NotNull TopicPartition topicPa } default: { // Client metric support is unimplemented in the current consumer - LOG.trace("The current consumer does not seem to have metric support for record lag."); + _logger.trace("The current consumer does not seem to have metric support for record lag."); return Optional.empty(); } } @@ -445,16 +445,16 @@ public static Builder builder() { void queryBrokerForLatestOffsets(@NotNull Consumer consumer, @NotNull Set partitions, @NotNull Duration requestTimeout) { for (final List batch : Iterables.partition(partitions, _brokerOffsetsFetchSize)) { - LOG.trace("Fetching the latest offsets for partitions: {}", batch); + _logger.trace("Fetching the latest offsets for partitions: {}", batch); final Instant queryTime = Instant.now(); final Map offsets; try { offsets = consumer.endOffsets(batch, requestTimeout); } catch (Exception e) { - LOG.trace("Unable to fetch latest offsets for partitions: {}", batch, e); + _logger.trace("Unable to fetch latest offsets for partitions: {}", batch, e); continue; } finally { - LOG.trace("Finished fetching the latest offsets for partitions {} in {} ms", batch, + _logger.trace("Finished fetching the latest offsets for partitions {} in {} ms", batch, Duration.between(queryTime, Instant.now()).toMillis()); } offsets.forEach((topicPartition, offset) -> { @@ -595,8 +595,8 @@ protected void runOneIteration() { } } } catch (Exception e) { - LOG.warn("Failed to update the partition leadership map. Using stale leadership data. Reason: {}", e.getMessage()); - LOG.debug("Failed to update the partition leadership map. Using stale leadership data.", e); + _logger.warn("Failed to update the partition leadership map. Using stale leadership data. Reason: {}", e.getMessage()); + _logger.debug("Failed to update the partition leadership map. Using stale leadership data.", e); } // Query the broker for its offsets for those partitions @@ -604,8 +604,8 @@ protected void runOneIteration() { try { _kafkaPositionTracker.queryBrokerForLatestOffsets(_consumer, partitionLeaderBatch, _consumerRequestTimeout); } catch (Exception e) { - LOG.warn("Failed to query latest broker offsets for this leader batch via endOffsets() RPC. Reason: {}", e.getMessage()); - LOG.debug("Failed to query latest broker offsets for this leader batch via endOffsets() RPC", e); + _logger.warn("Failed to query latest broker offsets for this leader batch via endOffsets() RPC. Reason: {}", e.getMessage()); + _logger.debug("Failed to query latest broker offsets for this leader batch via endOffsets() RPC", e); } }); } @@ -614,7 +614,7 @@ protected void runOneIteration() { * Queries a Kafka broker for the leader of each partition and caches that information. */ private void updatePartitionLeadershipMap() { - LOG.debug("Updating partition leadership map"); + _logger.debug("Updating partition leadership map"); Optional.ofNullable(_consumer.listTopics(_consumerRequestTimeout)).ifPresent(response -> { Map updateMap = response.values().stream() .filter(Objects::nonNull) @@ -637,7 +637,7 @@ private void updatePartitionLeadershipMap() { private List> batchPartitionsByLeader(@NotNull Set topicPartitions) { if (!_enablePartitionLeadershipCalculation || _partitionLeadershipMap.isEmpty()) { if (_partitionLeadershipMap.isEmpty()) { - LOG.debug("Leadership unknown for all topic partitions"); + _logger.debug("Leadership unknown for all topic partitions"); } return Collections.singletonList(topicPartitions); } @@ -648,10 +648,10 @@ private List> batchPartitionsByLeader(@NotNull Set { @Nullable final Node leader = _partitionLeadershipMap.get(topicPartition); if (leader == null) { - LOG.debug("Leader unknown for topic partition {}", topicPartition); + _logger.debug("Leader unknown for topic partition {}", topicPartition); unassignedPartitions.add(topicPartition); } else { - LOG.trace("Leader for topic partition {} is {}", topicPartition, leader); + _logger.trace("Leader for topic partition {} is {}", topicPartition, leader); assignedPartitions.computeIfAbsent(leader, s -> new HashSet<>()).add(topicPartition); } }); diff --git a/datastream-kafka-connector/src/main/java/com/linkedin/datastream/connectors/kafka/mirrormaker/KafkaMirrorMakerConnector.java b/datastream-kafka-connector/src/main/java/com/linkedin/datastream/connectors/kafka/mirrormaker/KafkaMirrorMakerConnector.java index 89c21b026..da6ad4edb 100644 --- a/datastream-kafka-connector/src/main/java/com/linkedin/datastream/connectors/kafka/mirrormaker/KafkaMirrorMakerConnector.java +++ b/datastream-kafka-connector/src/main/java/com/linkedin/datastream/connectors/kafka/mirrormaker/KafkaMirrorMakerConnector.java @@ -63,7 +63,7 @@ public class KafkaMirrorMakerConnector extends AbstractKafkaConnector { protected static final String PARTITION_FETCH_INTERVAL = "PartitionFetchIntervalMs"; protected static final String MM_TOPIC_PLACEHOLDER = "*"; - private static final Logger LOG = LoggerFactory.getLogger(KafkaMirrorMakerConnector.class); + private static final Logger _logger = LoggerFactory.getLogger(KafkaMirrorMakerConnector.class); private static final String DEST_CONSUMER_GROUP_ID_SUFFIX = "-topic-partition-listener"; private static final long DEFAULT_PARTITION_FETCH_INTERVAL = Duration.ofSeconds(30).toMillis(); private static final String NUM_PARTITION_FETCH_ERRORS = "numPartitionFetchErrors"; @@ -89,7 +89,7 @@ public class KafkaMirrorMakerConnector extends AbstractKafkaConnector { public KafkaMirrorMakerConnector(String connectorName, Properties config, String clusterName) { super(connectorName, config, new KafkaMirrorMakerGroupIdConstructor( Boolean.parseBoolean(config.getProperty(IS_GROUP_ID_HASHING_ENABLED, Boolean.FALSE.toString())), clusterName), - clusterName, LOG); + clusterName, _logger); _isFlushlessModeEnabled = Boolean.parseBoolean(config.getProperty(IS_FLUSHLESS_MODE_ENABLED, Boolean.FALSE.toString())); _partitionFetchIntervalMs = Long.parseLong(config.getProperty(PARTITION_FETCH_INTERVAL, @@ -101,7 +101,7 @@ public KafkaMirrorMakerConnector(String connectorName, Properties config, String _dynamicMetricsManager = DynamicMetricsManager.getInstance(); _shutdown = false; if (_enablePartitionAssignment) { - LOG.info("PartitionAssignment enabled for KafkaMirrorConnector"); + _logger.info("PartitionAssignment enabled for KafkaMirrorConnector"); } } @@ -124,20 +124,35 @@ public void initializeDatastream(Datastream stream, List allDatastre // verify that BYOT is not used if (DatastreamUtils.isUserManagedDestination(stream)) { - throw new DatastreamValidationException( - String.format("BYOT is not allowed for connector %s. Datastream: %s", stream.getConnectorName(), stream)); - } - - if (!DatastreamUtils.isConnectorManagedDestination(stream)) { - stream.getMetadata() - .put(DatastreamMetadataConstants.IS_CONNECTOR_MANAGED_DESTINATION_KEY, Boolean.TRUE.toString()); + // This should always be true, but checking just to be safe... + if (stream.hasDestination() && stream.getDestination().hasConnectionString()) { + final String destinationConnectionString = stream.getDestination().getConnectionString(); + final int placeholderIndex = destinationConnectionString.indexOf(MM_TOPIC_PLACEHOLDER); + if (placeholderIndex != -1 && placeholderIndex == destinationConnectionString.lastIndexOf(MM_TOPIC_PLACEHOLDER)) { + _logger.info("Allowing user managed datastream destination with connector {} for datastream {}", stream.getConnectorName(), stream.getName()); + } else { + // TODO: Implement validation on the source to confirm that it is not a regex/wildcard source + _logger.warn("Allowing explicit user managed datastream destination with connector {} for datastream {}", + stream.getConnectorName(), stream.getName()); +// _logger.error("User managed datastream destination format is invalid for connector {} and datastream {}", +// stream.getConnectorName(), stream.getName()); +// throw new DatastreamValidationException( +// String.format("Datastream destination format is invalid for connector %s. Datastream: %s", stream.getConnectorName(), stream) +// ); + } + } else { + throw new DatastreamValidationException( + String.format("BYOT is not allowed for connector %s. Datastream: %s", stream.getConnectorName(), stream)); + } + } else if (!DatastreamUtils.isConnectorManagedDestination(stream)) { + stream.getMetadata().put(DatastreamMetadataConstants.IS_CONNECTOR_MANAGED_DESTINATION_KEY, Boolean.TRUE.toString()); } // verify that the source regular expression can be compiled KafkaConnectionString connectionString = KafkaConnectionString.valueOf(stream.getSource().getConnectionString()); try { Pattern pattern = Pattern.compile(connectionString.getTopicName()); - LOG.info("Successfully compiled topic name pattern {}", pattern); + _logger.info("Successfully compiled topic name pattern {}", pattern); } catch (PatternSyntaxException e) { throw new DatastreamValidationException( String.format("Regular expression in Datastream source connection string (%s) is ill-formatted.", @@ -159,7 +174,7 @@ public List getMetricInfos() { @Override public void postDatastreamInitialize(Datastream datastream, List allDatastreams) throws DatastreamValidationException { - _groupIdConstructor.populateDatastreamGroupIdInMetadata(datastream, allDatastreams, Optional.of(LOG)); + _groupIdConstructor.populateDatastreamGroupIdInMetadata(datastream, allDatastreams, Optional.of(_logger)); } @Override @@ -219,7 +234,7 @@ public void handleDatastream(List datastreamGroups) { throw new DatastreamRuntimeException("Partition change callback is not defined"); } - LOG.info("handleDatastream: original datastream groups: {}, received datastream group {}", + _logger.info("handleDatastream: original datastream groups: {}, received datastream group {}", _partitionDiscoveryThreadMap.keySet(), datastreamGroups); List dgNames = datastreamGroups.stream().map(DatastreamGroup::getName).collect(Collectors.toList()); @@ -237,10 +252,10 @@ public void handleDatastream(List datastreamGroups) { new PartitionDiscoveryThread(datastreamGroup); partitionDiscoveryThread.start(); _partitionDiscoveryThreadMap.put(datastreamGroupName, partitionDiscoveryThread); - LOG.info("DatastreamChangeListener for {} registered", datastreamGroupName); + _logger.info("DatastreamChangeListener for {} registered", datastreamGroupName); } }); - LOG.info("handleDatastream: new datastream groups: {}", _partitionDiscoveryThreadMap.keySet()); + _logger.info("handleDatastream: new datastream groups: {}", _partitionDiscoveryThreadMap.keySet()); } @@ -307,16 +322,16 @@ public void run() { Consumer consumer = createConsumer(_consumerProperties, bootstrapValue, _groupIdConstructor.constructGroupId(datastream) + DEST_CONSUMER_GROUP_ID_SUFFIX); - LOG.info("Fetch thread for {} started", _datastreamGroup.getName()); + _logger.info("Fetch thread for {} started", _datastreamGroup.getName()); while (!isInterrupted() && !_shutdown) { try { List newPartitionInfo = getPartitionsInfo(consumer); - LOG.debug("Fetch partition info for {}, oldPartitionInfo: {}, new Partition info: {}" - , datastream.getName(), _subscribedPartitions, newPartitionInfo); + _logger.debug("Fetch partition info for {}, oldPartitionInfo: {}, new Partition info: {}", + datastream.getName(), _subscribedPartitions, newPartitionInfo); if (!ListUtils.isEqualList(newPartitionInfo, _subscribedPartitions)) { - LOG.info("get updated partition info for {}, oldPartitionInfo: {}, new Partition info: {}" - , datastream.getName(), _subscribedPartitions, newPartitionInfo); + _logger.info("get updated partition info for {}, oldPartitionInfo: {}, new Partition info: {}", + datastream.getName(), _subscribedPartitions, newPartitionInfo); _subscribedPartitions = Collections.synchronizedList(newPartitionInfo); _initialized = true; @@ -326,7 +341,7 @@ public void run() { } catch (Throwable t) { // If the Broker goes down, the consumer will receive an exception. However, there is no need to // re-initiate the consumer when the Broker comes back. Kafka consumer will automatic reconnect - LOG.warn("detect error for thread " + _datastreamGroup.getName() + ", ex: ", t); + _logger.warn("detect error for thread " + _datastreamGroup.getName() + ", ex: ", t); _dynamicMetricsManager.createOrUpdateMeter(MODULE, _datastreamGroup.getName(), NUM_PARTITION_FETCH_ERRORS, 1); } } @@ -336,7 +351,7 @@ public void run() { } consumer = null; - LOG.info("PartitionDiscoveryThread for {} stopped", _datastreamGroup.getName()); + _logger.info("PartitionDiscoveryThread for {} stopped", _datastreamGroup.getName()); } /** @@ -344,7 +359,7 @@ public void run() { */ public void shutdown() { this.interrupt(); - LOG.info("PartitionListenerThread Shutdown called for datastreamgroup {}", _datastreamGroup.getName()); + _logger.info("PartitionListenerThread Shutdown called for datastreamgroup {}", _datastreamGroup.getName()); } public List getSubscribedPartitions() { diff --git a/datastream-kafka-connector/src/main/java/com/linkedin/datastream/connectors/kafka/mirrormaker/KafkaMirrorMakerConnectorTask.java b/datastream-kafka-connector/src/main/java/com/linkedin/datastream/connectors/kafka/mirrormaker/KafkaMirrorMakerConnectorTask.java index d85d556ea..cd7989afe 100644 --- a/datastream-kafka-connector/src/main/java/com/linkedin/datastream/connectors/kafka/mirrormaker/KafkaMirrorMakerConnectorTask.java +++ b/datastream-kafka-connector/src/main/java/com/linkedin/datastream/connectors/kafka/mirrormaker/KafkaMirrorMakerConnectorTask.java @@ -75,7 +75,7 @@ */ public class KafkaMirrorMakerConnectorTask extends AbstractKafkaBasedConnectorTask { - private static final Logger LOG = LoggerFactory.getLogger(KafkaMirrorMakerConnectorTask.class.getName()); + private static final Logger _logger = LoggerFactory.getLogger(KafkaMirrorMakerConnectorTask.class.getName()); private static final String CLASS_NAME = KafkaMirrorMakerConnectorTask.class.getSimpleName(); private static final String KAFKA_ORIGIN_CLUSTER = "kafka-origin-cluster"; @@ -131,7 +131,7 @@ public class KafkaMirrorMakerConnectorTask extends AbstractKafkaBasedConnectorTa */ public KafkaMirrorMakerConnectorTask(KafkaBasedConnectorConfig config, DatastreamTask task, String connectorName, boolean isFlushlessModeEnabled, GroupIdConstructor groupIdConstructor) { - super(config, task, LOG, generateMetricsPrefix(connectorName, CLASS_NAME)); + super(config, task, _logger, generateMetricsPrefix(connectorName, CLASS_NAME)); _consumerFactory = config.getConsumerFactory(); _mirrorMakerSource = KafkaConnectionString.valueOf(_datastreamTask.getDatastreamSource().getConnectionString()); @@ -142,7 +142,7 @@ public KafkaMirrorMakerConnectorTask(KafkaBasedConnectorConfig config, Datastrea _dynamicMetricsManager = DynamicMetricsManager.getInstance(); if (_enablePartitionAssignment) { - LOG.info("Enable Brooklin partition assignment"); + _logger.info("Enable Brooklin partition assignment"); } if (_isFlushlessModeEnabled) { @@ -152,7 +152,7 @@ public KafkaMirrorMakerConnectorTask(KafkaBasedConnectorConfig config, Datastrea config.getConnectorProps().getLong(CONFIG_MAX_IN_FLIGHT_MSGS_THRESHOLD, DEFAULT_MAX_IN_FLIGHT_MSGS_THRESHOLD); _minInFlightMessagesThreshold = config.getConnectorProps().getLong(CONFIG_MIN_IN_FLIGHT_MSGS_THRESHOLD, DEFAULT_MIN_IN_FLIGHT_MSGS_THRESHOLD); - LOG.info("Flushless mode is enabled for task: {}, with flowControlEnabled={}, minInFlightMessagesThreshold={}, " + _logger.info("Flushless mode is enabled for task: {}, with flowControlEnabled={}, minInFlightMessagesThreshold={}, " + "maxInFlightMessagesThreshold={}", task, _flowControlEnabled, _minInFlightMessagesThreshold, _maxInFlightMessagesThreshold); } @@ -186,8 +186,8 @@ public KafkaMirrorMakerConnectorTask(KafkaBasedConnectorConfig config, Datastrea Boolean.FALSE.toString()); // auto-commits are unsafe properties.putIfAbsent(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, CONSUMER_AUTO_OFFSET_RESET_CONFIG_EARLIEST); properties.putIfAbsent(ConsumerConfig.GROUP_ID_CONFIG, - getMirrorMakerGroupId(_datastreamTask, _groupIdConstructor, _consumerMetrics, LOG)); - LOG.info("Creating Kafka consumer for task {} with properties {}", _datastreamTask, properties); + getMirrorMakerGroupId(_datastreamTask, _groupIdConstructor, _consumerMetrics, _logger)); + _logger.info("Creating Kafka consumer for task {} with properties {}", _datastreamTask, properties); return _consumerFactory.createConsumer(properties); } @@ -203,7 +203,7 @@ protected void consumerSubscribe() { // Invoke topic manager handleTopicMangerPartitionAssignment(topicPartition); } else { - LOG.info("About to subscribe to source: {}", _mirrorMakerSource.getTopicName()); + _logger.info("About to subscribe to source: {}", _mirrorMakerSource.getTopicName()); _consumer.subscribe(Pattern.compile(_mirrorMakerSource.getTopicName()), this); } } @@ -269,7 +269,7 @@ protected void sendDatastreamProducerRecord(DatastreamProducerRecord datastreamP long inFlightMessageCount = _flushlessProducer.getInFlightCount(topic, partition); if (inFlightMessageCount > _maxInFlightMessagesThreshold) { // add the partition to the pause list - LOG.warn( + _logger.warn( "In-flight message count of {} for topic partition {} exceeded maxInFlightMessagesThreshold of {}. Will pause partition.", inFlightMessageCount, tp, _maxInFlightMessagesThreshold); _autoPausedSourcePartitions.put(tp, new PausedSourcePartitionMetadata( @@ -309,7 +309,7 @@ public void run() { try { _datastreamTask.acquire(LOCK_ACQUIRE_TIMEOUT); } catch (DatastreamRuntimeException ex) { - LOG.error("Failed to acquire lock for datastreamTask {}", _datastreamTask); + _logger.error("Failed to acquire lock for datastreamTask {}", _datastreamTask); _dynamicMetricsManager.createOrUpdateMeter(CLASS_NAME, NUM_LOCK_FAILS, 1); throw ex; } @@ -333,7 +333,7 @@ public void onPartitionsRevoked(Collection partitions) { } private void commitSafeOffsets(Consumer consumer) { - LOG.info("Trying to commit safe offsets."); + _logger.info("Trying to commit safe offsets."); Map offsets = new HashMap<>(); for (TopicPartition tp : consumer.assignment()) { // add 1 to the last acked checkpoint to set to the offset of the next message to consume @@ -349,7 +349,7 @@ protected void maybeCommitOffsets(Consumer consumer, boolean hardCommit) { boolean isTimeToCommit = System.currentTimeMillis() - _lastCommittedTime > _offsetCommitInterval; if (_isFlushlessModeEnabled) { if (hardCommit) { // hard commit (flush and commit checkpoints) - LOG.info("Calling flush on the producer."); + _logger.info("Calling flush on the producer."); _datastreamTask.getEventProducer().flush(); commitSafeOffsets(consumer); diff --git a/datastream-kafka-connector/src/test/java/com/linkedin/datastream/connectors/kafka/MockDatastreamEventProducer.java b/datastream-kafka-connector/src/test/java/com/linkedin/datastream/connectors/kafka/MockDatastreamEventProducer.java index 162f845fb..47caa00f0 100644 --- a/datastream-kafka-connector/src/test/java/com/linkedin/datastream/connectors/kafka/MockDatastreamEventProducer.java +++ b/datastream-kafka-connector/src/test/java/com/linkedin/datastream/connectors/kafka/MockDatastreamEventProducer.java @@ -28,7 +28,7 @@ */ public class MockDatastreamEventProducer implements DatastreamEventProducer { - private static final Logger LOG = LoggerFactory.getLogger(MockDatastreamEventProducer.class); + private static final Logger _logger = LoggerFactory.getLogger(MockDatastreamEventProducer.class); private final List _events = Collections.synchronizedList(new ArrayList<>()); private int _numFlushes = 0; private final ExecutorService _executorService = Executors.newFixedThreadPool(1); @@ -96,7 +96,7 @@ public void send(DatastreamProducerRecord event, SendCallback callback) { } sendEvent(event, callback); } catch (InterruptedException e) { - LOG.error("Sleep was interrupted while throttling send callback"); + _logger.error("Sleep was interrupted while throttling send callback"); throw new DatastreamRuntimeException("Sleep was interrupted", e); } }); @@ -107,7 +107,7 @@ public void send(DatastreamProducerRecord event, SendCallback callback) { private void sendEvent(DatastreamProducerRecord event, SendCallback callback) { _events.add(event); - LOG.info("sent event {}, total _events {}", event, _events.size()); + _logger.info("sent event {}, total _events {}", event, _events.size()); DatastreamRecordMetadata md = new DatastreamRecordMetadata(event.getCheckpoint(), "mock topic", 666); if (callback != null) { callback.onCompletion(md, null); @@ -120,7 +120,7 @@ public void flush() { try { Thread.sleep(_flushDuration.toMillis()); } catch (InterruptedException e) { - LOG.info("Flush interrupted"); + _logger.info("Flush interrupted"); return; } } diff --git a/datastream-kafka-connector/src/test/java/com/linkedin/datastream/connectors/kafka/TestAbstractKafkaConnector.java b/datastream-kafka-connector/src/test/java/com/linkedin/datastream/connectors/kafka/TestAbstractKafkaConnector.java index 30c95a7d6..33759524c 100644 --- a/datastream-kafka-connector/src/test/java/com/linkedin/datastream/connectors/kafka/TestAbstractKafkaConnector.java +++ b/datastream-kafka-connector/src/test/java/com/linkedin/datastream/connectors/kafka/TestAbstractKafkaConnector.java @@ -36,7 +36,7 @@ */ public class TestAbstractKafkaConnector { - private static final Logger LOG = LoggerFactory.getLogger(TestAbstractKafkaConnector.class); + private static final Logger _logger = LoggerFactory.getLogger(TestAbstractKafkaConnector.class); @Test public void testConnectorRestartCalled() { @@ -125,7 +125,7 @@ public class TestKafkaConnector extends AbstractKafkaConnector { public TestKafkaConnector(boolean restartThrows, Properties props) { super("test", props, new KafkaGroupIdConstructor( Boolean.parseBoolean(props.getProperty(IS_GROUP_ID_HASHING_ENABLED, Boolean.FALSE.toString())), - "TestkafkaConnectorCluster"), "TestkafkaConnectorCluster", LOG); + "TestkafkaConnectorCluster"), "TestkafkaConnectorCluster", _logger); _restartThrows = restartThrows; } diff --git a/datastream-kafka-connector/src/test/java/com/linkedin/datastream/connectors/kafka/TestKafkaBasedConnectorTaskMetrics.java b/datastream-kafka-connector/src/test/java/com/linkedin/datastream/connectors/kafka/TestKafkaBasedConnectorTaskMetrics.java index 2fa8490f1..642281d86 100644 --- a/datastream-kafka-connector/src/test/java/com/linkedin/datastream/connectors/kafka/TestKafkaBasedConnectorTaskMetrics.java +++ b/datastream-kafka-connector/src/test/java/com/linkedin/datastream/connectors/kafka/TestKafkaBasedConnectorTaskMetrics.java @@ -27,7 +27,7 @@ */ public class TestKafkaBasedConnectorTaskMetrics { - private static final Logger LOG = LoggerFactory.getLogger(TestKafkaBasedConnectorTaskMetrics.class); + private static final Logger _logger = LoggerFactory.getLogger(TestKafkaBasedConnectorTaskMetrics.class); private static final String CONSUMER1_NAME = "CONNECTOR_CONSUMER1"; private static final String DELIMITED_CONSUMER1_NAME = "." + CONSUMER1_NAME + "."; private static final String CONSUMER2_NAME = "CONNECTOR_CONSUMER2"; @@ -45,9 +45,9 @@ public void setup(Method method) { @Test public void testConnectorPollMetrics() { KafkaBasedConnectorTaskMetrics connectorConsumer1 = - new KafkaBasedConnectorTaskMetrics(CLASS_NAME, CONSUMER1_NAME, LOG); + new KafkaBasedConnectorTaskMetrics(CLASS_NAME, CONSUMER1_NAME, _logger); KafkaBasedConnectorTaskMetrics connectorConsumer2 = - new KafkaBasedConnectorTaskMetrics(CLASS_NAME, CONSUMER2_NAME, LOG); + new KafkaBasedConnectorTaskMetrics(CLASS_NAME, CONSUMER2_NAME, _logger); connectorConsumer1.createPollMetrics(); connectorConsumer2.createPollMetrics(); @@ -84,9 +84,9 @@ public void testConnectorPollMetrics() { @Test public void testConnectorPartitionMetrics() { KafkaBasedConnectorTaskMetrics connectorConsumer1 = - new KafkaBasedConnectorTaskMetrics(CLASS_NAME, CONSUMER1_NAME, LOG); + new KafkaBasedConnectorTaskMetrics(CLASS_NAME, CONSUMER1_NAME, _logger); KafkaBasedConnectorTaskMetrics connectorConsumer2 = - new KafkaBasedConnectorTaskMetrics(CLASS_NAME, CONSUMER2_NAME, LOG); + new KafkaBasedConnectorTaskMetrics(CLASS_NAME, CONSUMER2_NAME, _logger); connectorConsumer1.createPartitionMetrics(); connectorConsumer2.createPartitionMetrics(); diff --git a/datastream-kafka-connector/src/test/java/com/linkedin/datastream/connectors/kafka/TestKafkaConnector.java b/datastream-kafka-connector/src/test/java/com/linkedin/datastream/connectors/kafka/TestKafkaConnector.java index 5af6e4125..f9888ec41 100644 --- a/datastream-kafka-connector/src/test/java/com/linkedin/datastream/connectors/kafka/TestKafkaConnector.java +++ b/datastream-kafka-connector/src/test/java/com/linkedin/datastream/connectors/kafka/TestKafkaConnector.java @@ -37,7 +37,7 @@ @Test public class TestKafkaConnector extends BaseKafkaZkTest { - private static final Logger LOG = LoggerFactory.getLogger(TestKafkaConnector.class); + private static final Logger _logger = LoggerFactory.getLogger(TestKafkaConnector.class); static Properties getDefaultConfig(Properties override) { Properties config = new Properties(); @@ -168,8 +168,8 @@ private void executeTestGroupIdAssignment(boolean isGroupIdHashingEnabled) throw datastream1.setTransportProviderName(transportProviderName); coordinator.initializeDatastream(datastream1); DatastreamTestUtils.storeDatastreams(_zkClient, clusterName, datastream1); - LOG.info("datastream1 groupID: {}", datastream1.getMetadata().get(DatastreamMetadataConstants.GROUP_ID)); - LOG.info("datastream1 task prefix:: {}", datastream1.getMetadata().get(DatastreamMetadataConstants.TASK_PREFIX)); + _logger.info("datastream1 groupID: {}", datastream1.getMetadata().get(DatastreamMetadataConstants.GROUP_ID)); + _logger.info("datastream1 task prefix:: {}", datastream1.getMetadata().get(DatastreamMetadataConstants.TASK_PREFIX)); // create datastream without any group ID override, but same topic as datastream1 // - datastream1's group ID should get copied @@ -177,8 +177,8 @@ private void executeTestGroupIdAssignment(boolean isGroupIdHashingEnabled) throw datastream2.setTransportProviderName(transportProviderName); coordinator.initializeDatastream(datastream2); DatastreamTestUtils.storeDatastreams(_zkClient, clusterName, datastream2); - LOG.info("datastream2 groupID: {}", datastream2.getMetadata().get(DatastreamMetadataConstants.GROUP_ID)); - LOG.info("datastream2 task prefix:: {}", datastream2.getMetadata().get(DatastreamMetadataConstants.TASK_PREFIX)); + _logger.info("datastream2 groupID: {}", datastream2.getMetadata().get(DatastreamMetadataConstants.GROUP_ID)); + _logger.info("datastream2 task prefix:: {}", datastream2.getMetadata().get(DatastreamMetadataConstants.TASK_PREFIX)); // create datastream with group ID override and same source as datastream 1 & 2 // - group ID override should take precedence @@ -187,8 +187,8 @@ private void executeTestGroupIdAssignment(boolean isGroupIdHashingEnabled) throw datastream3.setTransportProviderName(transportProviderName); coordinator.initializeDatastream(datastream3); DatastreamTestUtils.storeDatastreams(_zkClient, clusterName, datastream3); - LOG.info("datastream3 groupID: {}", datastream3.getMetadata().get(DatastreamMetadataConstants.GROUP_ID)); - LOG.info("datastream3 task prefix:: {}", datastream3.getMetadata().get(DatastreamMetadataConstants.TASK_PREFIX)); + _logger.info("datastream3 groupID: {}", datastream3.getMetadata().get(DatastreamMetadataConstants.GROUP_ID)); + _logger.info("datastream3 task prefix:: {}", datastream3.getMetadata().get(DatastreamMetadataConstants.TASK_PREFIX)); // create a datastream with different source than datastream1/2/3 and overridden group ID // - overridden group ID should be used. @@ -197,8 +197,8 @@ private void executeTestGroupIdAssignment(boolean isGroupIdHashingEnabled) throw datastream4.setTransportProviderName(transportProviderName); coordinator.initializeDatastream(datastream4); DatastreamTestUtils.storeDatastreams(_zkClient, clusterName, datastream4); - LOG.info("datastream4 groupID: {}", datastream4.getMetadata().get(DatastreamMetadataConstants.GROUP_ID)); - LOG.info("datastream4 task prefix:: {}", datastream4.getMetadata().get(DatastreamMetadataConstants.TASK_PREFIX)); + _logger.info("datastream4 groupID: {}", datastream4.getMetadata().get(DatastreamMetadataConstants.GROUP_ID)); + _logger.info("datastream4 task prefix:: {}", datastream4.getMetadata().get(DatastreamMetadataConstants.TASK_PREFIX)); // create a datastream with same source as datastream4 - it should use same overridden group ID from datastream4 Datastream datastream5 = createDatastream("datastream5", topicName2); @@ -206,16 +206,16 @@ private void executeTestGroupIdAssignment(boolean isGroupIdHashingEnabled) throw datastream5.setTransportProviderName(transportProviderName); coordinator.initializeDatastream(datastream5); DatastreamTestUtils.storeDatastreams(_zkClient, clusterName, datastream5); - LOG.info("datastream5 groupID: {}", datastream5.getMetadata().get(DatastreamMetadataConstants.GROUP_ID)); - LOG.info("datastream5 task prefix:: {}", datastream5.getMetadata().get(DatastreamMetadataConstants.TASK_PREFIX)); + _logger.info("datastream5 groupID: {}", datastream5.getMetadata().get(DatastreamMetadataConstants.GROUP_ID)); + _logger.info("datastream5 task prefix:: {}", datastream5.getMetadata().get(DatastreamMetadataConstants.TASK_PREFIX)); // create datastream with a different source than any of the above datastreams - group ID should be constructed Datastream datastream6 = createDatastream("datastream6", topicName3); datastream6.setTransportProviderName(transportProviderName); coordinator.initializeDatastream(datastream6); DatastreamTestUtils.storeDatastreams(_zkClient, clusterName, datastream6); - LOG.info("datastream6 groupID: {}", datastream6.getMetadata().get(DatastreamMetadataConstants.GROUP_ID)); - LOG.info("datastream6 task prefix: {}", datastream5.getMetadata().get(DatastreamMetadataConstants.TASK_PREFIX)); + _logger.info("datastream6 groupID: {}", datastream6.getMetadata().get(DatastreamMetadataConstants.GROUP_ID)); + _logger.info("datastream6 task prefix: {}", datastream5.getMetadata().get(DatastreamMetadataConstants.TASK_PREFIX)); Assert.assertEquals(datastream1.getMetadata().get(DatastreamMetadataConstants.GROUP_ID), groupIdConstructor.constructGroupId(datastream1)); diff --git a/datastream-kafka-connector/src/test/java/com/linkedin/datastream/connectors/kafka/TestKafkaConnectorTask.java b/datastream-kafka-connector/src/test/java/com/linkedin/datastream/connectors/kafka/TestKafkaConnectorTask.java index f2ae2d994..61a690506 100644 --- a/datastream-kafka-connector/src/test/java/com/linkedin/datastream/connectors/kafka/TestKafkaConnectorTask.java +++ b/datastream-kafka-connector/src/test/java/com/linkedin/datastream/connectors/kafka/TestKafkaConnectorTask.java @@ -73,7 +73,7 @@ */ public class TestKafkaConnectorTask extends BaseKafkaZkTest { - private static final Logger LOG = LoggerFactory.getLogger(TestKafkaConnectorTask.class); + private static final Logger _logger = LoggerFactory.getLogger(TestKafkaConnectorTask.class); private static final int POLL_TIMEOUT_MS = 25000; private static final long CONNECTOR_AWAIT_STOP_TIMEOUT_MS = 30000; @@ -98,7 +98,7 @@ protected static void produceEvents(DatastreamEmbeddedZookeeperKafkaCluster clus StandardCharsets.UTF_8)), (metadata, exception) -> { if (exception == null) { - LOG.info("send completed for event {} at offset {}", finalIndex, metadata.offset()); + _logger.info("send completed for event {} at offset {}", finalIndex, metadata.offset()); } else { throw new RuntimeException("Failed to send message.", exception); } @@ -118,28 +118,28 @@ public void testKafkaGroupId() throws Exception { DatastreamTaskImpl task = new DatastreamTaskImpl(Arrays.asList(datastream1, datastream2)); KafkaBasedConnectorTaskMetrics consumerMetrics = - new KafkaBasedConnectorTaskMetrics(TestKafkaConnectorTask.class.getName(), "testConsumer", LOG); + new KafkaBasedConnectorTaskMetrics(TestKafkaConnectorTask.class.getName(), "testConsumer", _logger); consumerMetrics.createEventProcessingMetrics(); String defaultGrpId = datastream1.getSource().getConnectionString() + "-to-" + datastream1.getDestination().getConnectionString(); // Testing with default group id - Assert.assertEquals(KafkaConnectorTask.getKafkaGroupId(task, groupIdConstructor, consumerMetrics, LOG), defaultGrpId); + Assert.assertEquals(KafkaConnectorTask.getKafkaGroupId(task, groupIdConstructor, consumerMetrics, _logger), defaultGrpId); // Test with setting explicit group id in one datastream datastream1.getMetadata().put(ConsumerConfig.GROUP_ID_CONFIG, "MyGroupId"); - Assert.assertEquals(KafkaConnectorTask.getKafkaGroupId(task, groupIdConstructor, consumerMetrics, LOG), "MyGroupId"); + Assert.assertEquals(KafkaConnectorTask.getKafkaGroupId(task, groupIdConstructor, consumerMetrics, _logger), "MyGroupId"); // Test with explicitly setting group id in both datastream datastream2.getMetadata().put(ConsumerConfig.GROUP_ID_CONFIG, "MyGroupId"); - Assert.assertEquals(KafkaConnectorTask.getKafkaGroupId(task, groupIdConstructor, consumerMetrics, LOG), "MyGroupId"); + Assert.assertEquals(KafkaConnectorTask.getKafkaGroupId(task, groupIdConstructor, consumerMetrics, _logger), "MyGroupId"); // now set different group ids in 2 datastreams and make sure validation fails datastream2.getMetadata().put(ConsumerConfig.GROUP_ID_CONFIG, "invalidGroupId"); boolean exceptionSeen = false; try { - KafkaConnectorTask.getKafkaGroupId(task, groupIdConstructor, consumerMetrics, LOG); + KafkaConnectorTask.getKafkaGroupId(task, groupIdConstructor, consumerMetrics, _logger); } catch (DatastreamRuntimeException e) { exceptionSeen = true; } @@ -151,13 +151,13 @@ public void testConsumeWithStartingOffset() throws Exception { String topic = "pizza1"; createTopic(_zkUtils, topic); - LOG.info("Sending first set of events"); + _logger.info("Sending first set of events"); //produce 100 msgs to topic before start produceEvents(_kafkaCluster, _zkUtils, topic, 0, 100); Map startOffsets = Collections.singletonMap(0, 100L); - LOG.info("Sending second set of events"); + _logger.info("Sending second set of events"); //produce 100 msgs to topic before start produceEvents(_kafkaCluster, _zkUtils, topic, 100, 100); @@ -173,7 +173,7 @@ public void testConsumeWithStartingOffset() throws Exception { KafkaConnectorTask connectorTask = createKafkaConnectorTask(task); - LOG.info("Sending third set of events"); + _logger.info("Sending third set of events"); //send 100 more msgs produceEvents(_kafkaCluster, _zkUtils, topic, 1000, 100); @@ -205,8 +205,10 @@ public void testCommittingOffsetRegularly() throws Exception { @Override public Consumer createConsumer(Properties properties) { Consumer result = spy(super.createConsumer(properties)); - doAnswer(invocation -> { remainingCommitSyncCalls.countDown(); return null; }) - .when(result).commitSync(any(Duration.class)); + doAnswer(invocation -> { + remainingCommitSyncCalls.countDown(); + return null; + }).when(result).commitSync(any(Duration.class)); return result; } }; @@ -228,10 +230,10 @@ public void testConsumerBaseCase() throws Exception { String topic = "Pizza2"; createTopic(_zkUtils, topic); - LOG.info("Sending first event, to avoid an empty topic."); + _logger.info("Sending first event, to avoid an empty topic."); produceEvents(_kafkaCluster, _zkUtils, topic, 0, 1); - LOG.info("Creating and Starting KafkaConnectorTask"); + _logger.info("Creating and Starting KafkaConnectorTask"); Datastream datastream = getDatastream(_broker, topic); DatastreamTaskImpl task = new DatastreamTaskImpl(Collections.singletonList(datastream)); MockDatastreamEventProducer datastreamProducer = new MockDatastreamEventProducer(); @@ -239,7 +241,7 @@ public void testConsumerBaseCase() throws Exception { KafkaConnectorTask connectorTask = createKafkaConnectorTask(task); - LOG.info("Producing 100 msgs to topic: " + topic); + _logger.info("Producing 100 msgs to topic: " + topic); produceEvents(_kafkaCluster, _zkUtils, topic, 1000, 100); if (!PollUtils.poll(() -> datastreamProducer.getEvents().size() == 100, 100, POLL_TIMEOUT_MS)) { @@ -258,10 +260,10 @@ public void testConsumerPositionTracking() throws Exception { final String topic = "ChicagoStylePizza"; createTopic(_zkUtils, topic); - LOG.info("Sending first event, to avoid an empty topic."); + _logger.info("Sending first event, to avoid an empty topic."); produceEvents(_kafkaCluster, _zkUtils, topic, 0, 1); - LOG.info("Creating and Starting KafkaConnectorTask"); + _logger.info("Creating and Starting KafkaConnectorTask"); final Datastream datastream = getDatastream(_broker, topic); final DatastreamTaskImpl task = new DatastreamTaskImpl(Collections.singletonList(datastream)); final MockDatastreamEventProducer datastreamProducer = new MockDatastreamEventProducer(); @@ -269,7 +271,7 @@ public void testConsumerPositionTracking() throws Exception { final KafkaConnectorTask connectorTask = createKafkaConnectorTask(task, config); - LOG.info("Producing 100 msgs to topic: " + topic); + _logger.info("Producing 100 msgs to topic: " + topic); produceEvents(_kafkaCluster, _zkUtils, topic, 1000, 100); if (!PollUtils.poll(() -> datastreamProducer.getEvents().size() == 100, 100, POLL_TIMEOUT_MS)) { @@ -389,7 +391,7 @@ public void testFlakyProducer() throws Exception { String topic = "pizza3"; createTopic(_zkUtils, topic); - LOG.info("Sending first event, to avoid an empty topic."); + _logger.info("Sending first event, to avoid an empty topic."); produceEvents(_kafkaCluster, _zkUtils, topic, 0, 1); class State { @@ -408,14 +410,14 @@ class State { return null; }).when(datastreamProducer).send(any(), any()); - LOG.info("Creating and Starting KafkaConnectorTask"); + _logger.info("Creating and Starting KafkaConnectorTask"); Datastream datastream = getDatastream(_broker, topic); DatastreamTaskImpl task = new DatastreamTaskImpl(Collections.singletonList(datastream)); task.setEventProducer(datastreamProducer); KafkaConnectorTask connectorTask = createKafkaConnectorTask(task); - LOG.info("Producing 100 msgs to topic: " + topic); + _logger.info("Producing 100 msgs to topic: " + topic); produceEvents(_kafkaCluster, _zkUtils, topic, 1000, 100); if (!PollUtils.poll(() -> state.messagesProcessed >= 100, 100, POLL_TIMEOUT_MS)) { @@ -433,10 +435,10 @@ public void testFlakyConsumer() throws Exception { String topic = "Pizza2"; createTopic(_zkUtils, topic); - LOG.info("Sending first event, to avoid an empty topic."); + _logger.info("Sending first event, to avoid an empty topic."); produceEvents(_kafkaCluster, _zkUtils, topic, 0, 1); - LOG.info("Creating and Starting KafkaConnectorTask"); + _logger.info("Creating and Starting KafkaConnectorTask"); Datastream datastream = getDatastream(_broker, topic); DatastreamTaskImpl task = new DatastreamTaskImpl(Collections.singletonList(datastream)); task.setZkAdapter(Mockito.mock(ZkAdapter.class)); diff --git a/datastream-kafka-connector/src/test/java/com/linkedin/datastream/connectors/kafka/TestKafkaPositionTracker.java b/datastream-kafka-connector/src/test/java/com/linkedin/datastream/connectors/kafka/TestKafkaPositionTracker.java index a32230c7e..2fd0abaaa 100644 --- a/datastream-kafka-connector/src/test/java/com/linkedin/datastream/connectors/kafka/TestKafkaPositionTracker.java +++ b/datastream-kafka-connector/src/test/java/com/linkedin/datastream/connectors/kafka/TestKafkaPositionTracker.java @@ -69,7 +69,7 @@ */ @Test(singleThreaded = true) public class TestKafkaPositionTracker { - private static final Logger LOG = LoggerFactory.getLogger(TestKafkaPositionTracker.class); + private static final Logger _logger = LoggerFactory.getLogger(TestKafkaPositionTracker.class); private static final long POLL_PERIOD_MS = Duration.ofMillis(250).toMillis(); private static final long POLL_TIMEOUT_MS = Duration.ofMillis(1_000).toMillis(); @@ -326,13 +326,13 @@ private void produceAndPollEvents(Map events) { .anyMatch(event -> event.getMetadata().get("kafka-origin-partition").equals(String.valueOf(topicPartition.partition())) && event.getMetadata().get("SourceTimestamp").equals(String.valueOf(timestamp.toEpochMilli()))); - LOG.info("Checking for event with partition {} with timestamp {}", topicPartition, timestamp); + _logger.info("Checking for event with partition {} with timestamp {}", topicPartition, timestamp); if (!eventProduced) { - LOG.info("Event with for partition {} with timestamp {} has not been produced yet", topicPartition, timestamp); + _logger.info("Event with for partition {} with timestamp {} has not been produced yet", topicPartition, timestamp); return false; } } - LOG.info("All events successfully produced"); + _logger.info("All events successfully produced"); return true; }, POLL_PERIOD_MS, POLL_TIMEOUT_MS); } @@ -361,9 +361,9 @@ private void testConsumerPositionData(TopicPartition topicPartition, long expect && positionData.get().getConsumerOffset() != null && (expectedRecordTimestamp == null || positionData.get().getLastRecordReceivedTimestamp() != null); if (!available) { - LOG.info("Waiting for consumer position data to be set/available"); + _logger.info("Waiting for consumer position data to be set/available"); } else { - LOG.info("Consumer position data is set/available"); + _logger.info("Consumer position data is set/available"); } return available; }, POLL_PERIOD_MS, POLL_TIMEOUT_MS); @@ -390,9 +390,9 @@ private void testBrokerPositionData(TopicPartition topicPartition, long expected Optional positionData = getPositionData(topicPartition); boolean available = positionData.isPresent() && positionData.get().getBrokerOffset() != null; if (!available) { - LOG.info("Waiting for broker position data to be set/available"); + _logger.info("Waiting for broker position data to be set/available"); } else { - LOG.info("Broker position data is set/available"); + _logger.info("Broker position data is set/available"); } return available; }, POLL_PERIOD_MS, POLL_TIMEOUT_MS); @@ -416,7 +416,7 @@ private KafkaConnectorTask createKafkaConnectorTask() { new KafkaGroupIdConstructor(false, "testCluster")); final Thread consumerThread = new Thread(connectorTask, "Consumer Thread"); consumerThread.setDaemon(true); - consumerThread.setUncaughtExceptionHandler((t, e) -> LOG.error("Got uncaught exception in consumer thread", e)); + consumerThread.setUncaughtExceptionHandler((t, e) -> _logger.error("Got uncaught exception in consumer thread", e)); consumerThread.start(); return connectorTask; } @@ -557,8 +557,8 @@ synchronized void push(TopicPartition topicPartition, Instant timestamp) { List> replacementList = new ArrayList<>(_recordsOnBroker.get(topicPartition)); replacementList.add(consumerRecord); _recordsOnBroker.put(topicPartition, replacementList); - LOG.info("Pushing an event with timestamp " + timestamp + " to " + topicPartition); - LOG.info("Current broker offsets: " + getBrokerOffsets(getAllTopicPartitions())); + _logger.info("Pushing an event with timestamp " + timestamp + " to " + topicPartition); + _logger.info("Current broker offsets: " + getBrokerOffsets(getAllTopicPartitions())); } /** @@ -600,7 +600,7 @@ private synchronized ConsumerRecords pollImpl(Map resultCount = results.entrySet().stream().collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().size())); - LOG.info("Tried to poll for " + fromOffsets.keySet() + " and got " + resultCount.toString() + " results"); + _logger.info("Tried to poll for " + fromOffsets.keySet() + " and got " + resultCount.toString() + " results"); return new ConsumerRecords<>(results.entrySet() .stream() .filter(e -> !e.getValue().isEmpty()) diff --git a/datastream-kafka-connector/src/test/java/com/linkedin/datastream/connectors/kafka/TestPauseResumePartitions.java b/datastream-kafka-connector/src/test/java/com/linkedin/datastream/connectors/kafka/TestPauseResumePartitions.java index d383c5ff0..8df38e11e 100644 --- a/datastream-kafka-connector/src/test/java/com/linkedin/datastream/connectors/kafka/TestPauseResumePartitions.java +++ b/datastream-kafka-connector/src/test/java/com/linkedin/datastream/connectors/kafka/TestPauseResumePartitions.java @@ -37,7 +37,7 @@ */ public class TestPauseResumePartitions { - private static final Logger LOG = LoggerFactory.getLogger(TestPauseResumePartitions.class.getName()); + private static final Logger _logger = LoggerFactory.getLogger(TestPauseResumePartitions.class.getName()); @Test public void testPausePartitions() { @@ -55,7 +55,7 @@ public void testPausePartitions() { pausedSourcePartitionsConfig.put(topic, partitions); Set partitionsToPause = KafkaConnectorTask.determinePartitionsToPause(assignedPartitions, pausedSourcePartitionsConfig, - autoPausedPartitions, LOG); + autoPausedPartitions, _logger); Assert.assertEquals(partitionsToPause, assignedPartitions); // resume some partitions by removing them from config @@ -63,7 +63,7 @@ public void testPausePartitions() { Set expectedPartitionsToPause = new HashSet<>(partitionsToPause); expectedPartitionsToPause.removeAll(Arrays.asList(new TopicPartition(topic, 2), new TopicPartition(topic, 5))); partitionsToPause = KafkaConnectorTask.determinePartitionsToPause(assignedPartitions, pausedSourcePartitionsConfig, - autoPausedPartitions, LOG); + autoPausedPartitions, _logger); Assert.assertEquals(partitionsToPause, expectedPartitionsToPause); } @@ -91,13 +91,13 @@ public void testAutoPausePartitions() { // verify that 2 of the partitions were designated for pause Set partitionsToPause = KafkaConnectorTask.determinePartitionsToPause(assignedPartitions, pausedSourcePartitionsConfig, - autoPausePartitions, LOG); + autoPausePartitions, _logger); Assert.assertEquals(partitionsToPause, autoPausePartitions.keySet()); // auto-resume one of the partitions autoPausePartitions.remove(partition4); partitionsToPause = KafkaConnectorTask.determinePartitionsToPause(assignedPartitions, pausedSourcePartitionsConfig, - autoPausePartitions, LOG); + autoPausePartitions, _logger); Assert.assertEquals(partitionsToPause, autoPausePartitions.keySet()); } @@ -126,7 +126,7 @@ public void testConfigAndAutoPausePartitions() { // verify that 6 of the partitions are designated for pause Set partitionsToPause = KafkaConnectorTask.determinePartitionsToPause(assignedPartitions, pausedSourcePartitionsConfig, - autoPausedPartitions, LOG); + autoPausedPartitions, _logger); Assert.assertEquals(partitionsToPause, IntStream.range(0, 8) .filter(i -> i != 4 && i != 6) .mapToObj(p -> new TopicPartition(topic, p)) @@ -137,7 +137,7 @@ public void testConfigAndAutoPausePartitions() { pausedSourcePartitionsConfig.get(topic).add("7"); // verify that partitionsToPause is the same partitionsToPause = KafkaConnectorTask.determinePartitionsToPause(assignedPartitions, pausedSourcePartitionsConfig, - autoPausedPartitions, LOG); + autoPausedPartitions, _logger); Assert.assertEquals(partitionsToPause, IntStream.range(0, 8) .filter(i -> i != 4 && i != 6) .mapToObj(p -> new TopicPartition(topic, p)) @@ -149,7 +149,7 @@ public void testConfigAndAutoPausePartitions() { // now resume partition 7, which was auto-paused then configured for pause pausedSourcePartitionsConfig.get(topic).remove("7"); partitionsToPause = KafkaConnectorTask.determinePartitionsToPause(assignedPartitions, pausedSourcePartitionsConfig, - autoPausedPartitions, LOG); + autoPausedPartitions, _logger); Assert.assertEquals(partitionsToPause, IntStream.range(0, 8) .filter(i -> i != 4 && i != 6 && i != 7) .mapToObj(p -> new TopicPartition(topic, p)) @@ -178,7 +178,7 @@ public void testUnassignedPartitionsRemovedFromPauseSets() { new PausedSourcePartitionMetadata(() -> false, PausedSourcePartitionMetadata.Reason.SEND_ERROR)); Set partitionsToPause = KafkaConnectorTask.determinePartitionsToPause(assignedPartitions, - pausedSourcePartitionsConfig, autoPausedPartitions, LOG); + pausedSourcePartitionsConfig, autoPausedPartitions, _logger); // verify that partition 9 was removed from auto-pause list since it's no longer assigned Assert.assertEquals(partitionsToPause, Sets.newHashSet(partition5), "Partition should have been removed from auto-pause set, since it is not in the assignment"); diff --git a/datastream-kafka-connector/src/test/java/com/linkedin/datastream/connectors/kafka/mirrormaker/TestKafkaMirrorMakerConnector.java b/datastream-kafka-connector/src/test/java/com/linkedin/datastream/connectors/kafka/mirrormaker/TestKafkaMirrorMakerConnector.java index 1c8c0c316..0039fae3c 100644 --- a/datastream-kafka-connector/src/test/java/com/linkedin/datastream/connectors/kafka/mirrormaker/TestKafkaMirrorMakerConnector.java +++ b/datastream-kafka-connector/src/test/java/com/linkedin/datastream/connectors/kafka/mirrormaker/TestKafkaMirrorMakerConnector.java @@ -70,7 +70,7 @@ @Test public class TestKafkaMirrorMakerConnector extends BaseKafkaZkTest { - private static final Logger LOG = LoggerFactory.getLogger(TestKafkaMirrorMakerConnector.class); + private static final Logger _logger = LoggerFactory.getLogger(TestKafkaMirrorMakerConnector.class); private static final String DATASTREAM_STATE_QUERY = "/datastream_state?datastream="; @@ -767,8 +767,8 @@ private void executeGroupIdAssignment(boolean groupIdHashingEnabled) throws Exce Assert.assertEquals(datastream1.getMetadata().get(DatastreamMetadataConstants.GROUP_ID), new KafkaMirrorMakerGroupIdConstructor(groupIdHashingEnabled, "testGroupIdAssignment").constructGroupId( datastream1)); - LOG.info("datastream1: {}", datastream1.getMetadata().get(DatastreamMetadataConstants.GROUP_ID)); - LOG.info("datastream1: {}", datastream1.getMetadata().get(DatastreamMetadataConstants.TASK_PREFIX)); + _logger.info("datastream1: {}", datastream1.getMetadata().get(DatastreamMetadataConstants.GROUP_ID)); + _logger.info("datastream1: {}", datastream1.getMetadata().get(DatastreamMetadataConstants.TASK_PREFIX)); // create datastream with group ID specified in metadata, expect that group ID being used as it is. StringMap metadata2 = new StringMap(); @@ -780,8 +780,8 @@ private void executeGroupIdAssignment(boolean groupIdHashingEnabled) throws Exce coordinator.initializeDatastream(datastream2); DatastreamTestUtils.storeDatastreams(_zkClient, clusterName, datastream2); Assert.assertEquals(datastream2.getMetadata().get(DatastreamMetadataConstants.GROUP_ID), "randomId"); - LOG.info("datastream2: {}", datastream2.getMetadata().get(DatastreamMetadataConstants.GROUP_ID)); - LOG.info("datastream2: {}", datastream2.getMetadata().get(DatastreamMetadataConstants.TASK_PREFIX)); + _logger.info("datastream2: {}", datastream2.getMetadata().get(DatastreamMetadataConstants.GROUP_ID)); + _logger.info("datastream2: {}", datastream2.getMetadata().get(DatastreamMetadataConstants.TASK_PREFIX)); coordinator.stop(); } @@ -795,7 +795,7 @@ private void verifyPausedPartitions(Connector connector, Datastream datastream, connector.validateUpdateDatastreams(Collections.singletonList(datastream), Collections.singletonList(datastream)); return expectedPartitions.equals(DatastreamUtils.getDatastreamSourcePartitions(datastream)); } catch (Exception e) { - LOG.warn("validateUpdateDatastreams failed with error: " + e); + _logger.warn("validateUpdateDatastreams failed with error: " + e); return false; } }, POLL_PERIOD_MS, POLL_TIMEOUT_MS); diff --git a/datastream-kafka-connector/src/test/java/com/linkedin/datastream/connectors/kafka/mirrormaker/TestKafkaMirrorMakerConnectorTask.java b/datastream-kafka-connector/src/test/java/com/linkedin/datastream/connectors/kafka/mirrormaker/TestKafkaMirrorMakerConnectorTask.java index bd5f89cc6..1cb57190b 100644 --- a/datastream-kafka-connector/src/test/java/com/linkedin/datastream/connectors/kafka/mirrormaker/TestKafkaMirrorMakerConnectorTask.java +++ b/datastream-kafka-connector/src/test/java/com/linkedin/datastream/connectors/kafka/mirrormaker/TestKafkaMirrorMakerConnectorTask.java @@ -73,7 +73,7 @@ public class TestKafkaMirrorMakerConnectorTask extends BaseKafkaZkTest { private static final long CONNECTOR_AWAIT_STOP_TIMEOUT_MS = 30000; - private static final Logger LOG = LoggerFactory.getLogger(TestKafkaMirrorMakerConnectorTask.class); + private static final Logger _logger = LoggerFactory.getLogger(TestKafkaMirrorMakerConnectorTask.class); @Test public void testConsumeFromMultipleTopics() throws Exception { @@ -224,8 +224,10 @@ public void testRegularCommitWithFlushlessProducer() throws Exception { @Override public Consumer createConsumer(Properties properties) { Consumer result = spy(super.createConsumer(properties)); - doAnswer(invocation -> { remainingCommitSyncCalls.countDown(); return null; }) - .when(result).commitSync(anyMapOf(TopicPartition.class, OffsetAndMetadata.class), any(Duration.class)); + doAnswer(invocation -> { + remainingCommitSyncCalls.countDown(); + return null; + }).when(result).commitSync(anyMapOf(TopicPartition.class, OffsetAndMetadata.class), any(Duration.class)); return result; } }; @@ -698,33 +700,33 @@ public void testMirrorMakerGroupId() throws Exception { // Creating the task with multiple datastreams strictly for testing purposes. DatastreamTaskImpl task = new DatastreamTaskImpl(Arrays.asList(datastream1, datastream2)); CommonConnectorMetrics consumerMetrics = - new CommonConnectorMetrics(TestKafkaMirrorMakerConnectorTask.class.getName(), "testConsumer", LOG); + new CommonConnectorMetrics(TestKafkaMirrorMakerConnectorTask.class.getName(), "testConsumer", _logger); consumerMetrics.createEventProcessingMetrics(); String defaultGrpId = datastream1.getName(); // Testing with default group id Assert.assertEquals( - KafkaMirrorMakerConnectorTask.getMirrorMakerGroupId(task, groupIdConstructor, consumerMetrics, LOG), + KafkaMirrorMakerConnectorTask.getMirrorMakerGroupId(task, groupIdConstructor, consumerMetrics, _logger), defaultGrpId); // Test with setting explicit group id in one datastream datastream1.getMetadata().put(ConsumerConfig.GROUP_ID_CONFIG, "MyGroupId"); Assert.assertEquals( - KafkaMirrorMakerConnectorTask.getMirrorMakerGroupId(task, groupIdConstructor, consumerMetrics, LOG), + KafkaMirrorMakerConnectorTask.getMirrorMakerGroupId(task, groupIdConstructor, consumerMetrics, _logger), "MyGroupId"); // Test with explicitly setting group id in both datastream datastream2.getMetadata().put(ConsumerConfig.GROUP_ID_CONFIG, "MyGroupId"); Assert.assertEquals( - KafkaMirrorMakerConnectorTask.getMirrorMakerGroupId(task, groupIdConstructor, consumerMetrics, LOG), + KafkaMirrorMakerConnectorTask.getMirrorMakerGroupId(task, groupIdConstructor, consumerMetrics, _logger), "MyGroupId"); // now set different group ids in 2 datastreams and make sure validation fails datastream2.getMetadata().put(ConsumerConfig.GROUP_ID_CONFIG, "invalidGroupId"); boolean exceptionSeen = false; try { - KafkaMirrorMakerConnectorTask.getMirrorMakerGroupId(task, groupIdConstructor, consumerMetrics, LOG); + KafkaMirrorMakerConnectorTask.getMirrorMakerGroupId(task, groupIdConstructor, consumerMetrics, _logger); } catch (DatastreamRuntimeException e) { exceptionSeen = true; } diff --git a/datastream-kafka/src/main/java/com/linkedin/datastream/kafka/KafkaProducerWrapper.java b/datastream-kafka/src/main/java/com/linkedin/datastream/kafka/KafkaProducerWrapper.java index b48e1d208..1eab067aa 100644 --- a/datastream-kafka/src/main/java/com/linkedin/datastream/kafka/KafkaProducerWrapper.java +++ b/datastream-kafka/src/main/java/com/linkedin/datastream/kafka/KafkaProducerWrapper.java @@ -191,7 +191,9 @@ void send(DatastreamTask task, ProducerRecord producerRecord, Callback onC while (retry) { try { ++numberOfAttempt; - maybeGetKafkaProducer(task).ifPresent(p -> p.send(producerRecord, (metadata, exception) -> { + Optional> producer = maybeGetKafkaProducer(task); + + producer.ifPresent(p -> p.send(producerRecord, (metadata, exception) -> { if (exception == null) { onComplete.onCompletion(metadata, null); } else { diff --git a/datastream-kafka/src/main/java/com/linkedin/datastream/kafka/KafkaTransportProvider.java b/datastream-kafka/src/main/java/com/linkedin/datastream/kafka/KafkaTransportProvider.java index 672d4439e..bced29c43 100644 --- a/datastream-kafka/src/main/java/com/linkedin/datastream/kafka/KafkaTransportProvider.java +++ b/datastream-kafka/src/main/java/com/linkedin/datastream/kafka/KafkaTransportProvider.java @@ -48,7 +48,7 @@ */ public class KafkaTransportProvider implements TransportProvider { private static final String CLASS_NAME = KafkaTransportProvider.class.getSimpleName(); - private static final Logger LOG = LoggerFactory.getLogger(CLASS_NAME); + private static final Logger _logger = LoggerFactory.getLogger(CLASS_NAME); static final String AGGREGATE = "aggregate"; static final String EVENT_WRITE_RATE = "eventWriteRate"; @@ -85,10 +85,10 @@ public KafkaTransportProvider(DatastreamTask datastreamTask, List outgoing = convertToProducerRecord(topicName, record, event); @@ -170,7 +167,7 @@ public void send(String destinationUri, DatastreamProducerRecord record, SendCal producer.send(_datastreamTask, outgoing, (metadata, exception) -> { int partition = metadata != null ? metadata.partition() : -1; if (exception != null) { - LOG.error("Sending a message with source checkpoint {} to topic {} partition {} for datastream task {} " + _logger.error("Sending a message with source checkpoint {} to topic {} partition {} for datastream task {} " + "threw an exception.", record.getCheckpoint(), topicName, partition, _datastreamTask, exception); } doOnSendCallback(record, onSendComplete, metadata, exception); @@ -183,13 +180,15 @@ public void send(String destinationUri, DatastreamProducerRecord record, SendCal _eventTransportErrorRate.mark(); _dynamicMetricsManager.createOrUpdateMeter(_metricsNamesPrefix, topicName, EVENT_TRANSPORT_ERROR_RATE, 1); String errorMessage = String.format( - "Sending DatastreamRecord (%s) to topic %s, partition %s, Kafka cluster %s failed with exception.", record, - topicName, record.getPartition().orElse(-1), destinationUri); + "Sending Datastream Record (%s) to topic %s, partition %s, Kafka cluster %s failed with exception.", + record.getCheckpoint(), topicName, record.getPartition().orElse(-1), destinationUri); - ErrorLogger.logAndThrowDatastreamRuntimeException(LOG, errorMessage, e); + ErrorLogger.logAndThrowDatastreamRuntimeException(_logger, errorMessage, e); } - LOG.debug("Done sending Datastream event record: {}", record); + if (_logger.isTraceEnabled()) { + _logger.trace("Done sending Datastream event record: {}", record.getCheckpoint()); + } } @Override diff --git a/datastream-kafka/src/main/java/com/linkedin/datastream/kafka/KafkaTransportProviderAdmin.java b/datastream-kafka/src/main/java/com/linkedin/datastream/kafka/KafkaTransportProviderAdmin.java index 74a58c77f..6651f5233 100644 --- a/datastream-kafka/src/main/java/com/linkedin/datastream/kafka/KafkaTransportProviderAdmin.java +++ b/datastream-kafka/src/main/java/com/linkedin/datastream/kafka/KafkaTransportProviderAdmin.java @@ -57,7 +57,7 @@ * @param type of the value */ public class KafkaTransportProviderAdmin implements TransportProviderAdmin { - public static final Logger LOG = LoggerFactory.getLogger(KafkaTransportProviderAdmin.class); + public static final Logger _logger = LoggerFactory.getLogger(KafkaTransportProviderAdmin.class); public static final int DEFAULT_PRODUCERS_PER_CONNECTOR = 10; public static final String DEFAULT_REPLICATION_FACTOR = "1"; public static final String DEFAULT_TRANSLATOR = "com.linkedin.datastream.common.translator.GenericRecordTranslator"; @@ -149,7 +149,7 @@ public TransportProvider assignTransportProvider(DatastreamTask task) { new KafkaTransportProvider<>(task, producers, transportProviderProperties, _transportProviderMetricsNamesPrefix)); producers.forEach(p -> p.assignTask(task)); } else { - LOG.warn("Trying to assign transport provider to task {} which is already assigned.", task); + _logger.warn("Trying to assign transport provider to task {} which is already assigned.", task); } return _transportProviders.get(task); @@ -175,7 +175,7 @@ public void unassignTransportProvider(DatastreamTask task) { KafkaTransportProvider transportProvider = _transportProviders.remove(task); transportProvider.getProducers().forEach(p -> p.unassignTask(task)); } else { - LOG.warn("Trying to unassign already unassigned transport provider."); + _logger.warn("Trying to unassign already unassigned transport provider."); } } @@ -205,7 +205,7 @@ public void initializeDestinationForDatastream(Datastream datastream, String des if (source.hasPartitions()) { destination.setPartitions(source.getPartitions()); } else { - LOG.warn("Unable to set the number of partitions in a destination, set to default {}", DEFAULT_NUMBER_PARTITIONS); + _logger.warn("Unable to set the number of partitions in a destination, set to default {}", DEFAULT_NUMBER_PARTITIONS); destination.setPartitions(DEFAULT_NUMBER_PARTITIONS); } } @@ -220,7 +220,7 @@ public void createDestination(Datastream datastream) { @Override public void dropDestination(Datastream datastream) { - LOG.info("Drop destination called for datastream {}. Ignoring it.", datastream); + _logger.info("Drop destination called for datastream {}. Ignoring it.", datastream); } /** @@ -261,15 +261,15 @@ public void createTopic(String connectionString, int numberOfPartitions, Propert // Create only if it doesn't exist. if (!AdminUtils.topicExists(_zkUtils.get(), topicName)) { int replicationFactor = Integer.parseInt(topicConfig.getProperty("replicationFactor", DEFAULT_REPLICATION_FACTOR)); - LOG.info("Creating topic with name {} partitions={} with properties {}", topicName, numberOfPartitions, + _logger.info("Creating topic with name {} partitions={} with properties {}", topicName, numberOfPartitions, topicConfig); AdminUtils.createTopic(_zkUtils.get(), topicName, numberOfPartitions, replicationFactor, topicConfig, RackAwareMode.Disabled$.MODULE$); } else { - LOG.warn("Topic with name {} already exists", topicName); + _logger.warn("Topic with name {} already exists", topicName); } } catch (Throwable e) { - LOG.error("Creating topic {} failed with exception {}", topicName, e); + _logger.error("Creating topic {} failed with exception {}", topicName, e); throw e; } } diff --git a/datastream-kafka/src/main/java/com/linkedin/datastream/kafka/KafkaTransportProviderUtils.java b/datastream-kafka/src/main/java/com/linkedin/datastream/kafka/KafkaTransportProviderUtils.java index 543296c71..47798d490 100644 --- a/datastream-kafka/src/main/java/com/linkedin/datastream/kafka/KafkaTransportProviderUtils.java +++ b/datastream-kafka/src/main/java/com/linkedin/datastream/kafka/KafkaTransportProviderUtils.java @@ -18,7 +18,7 @@ * Utility methods for {@link KafkaTransportProvider} */ public class KafkaTransportProviderUtils { - private static final Logger LOG = LoggerFactory.getLogger(KafkaTransportProviderUtils.class.getName()); + private static final Logger _logger = LoggerFactory.getLogger(KafkaTransportProviderUtils.class.getName()); // Mapping destination URI to topic name private final static Map URI_TOPICS = new ConcurrentHashMap<>(); diff --git a/datastream-kafka/src/test/java/com/linkedin/datastream/kafka/TestKafkaTransportProvider.java b/datastream-kafka/src/test/java/com/linkedin/datastream/kafka/TestKafkaTransportProvider.java index 1603b1830..2ea79a860 100644 --- a/datastream-kafka/src/test/java/com/linkedin/datastream/kafka/TestKafkaTransportProvider.java +++ b/datastream-kafka/src/test/java/com/linkedin/datastream/kafka/TestKafkaTransportProvider.java @@ -52,7 +52,7 @@ @Test public class TestKafkaTransportProvider extends BaseKafkaZkTest { private static final Integer NUM_PARTITIONS = 10; - private static final Logger LOG = LoggerFactory.getLogger(TestKafkaTransportProvider.class); + private static final Logger _logger = LoggerFactory.getLogger(TestKafkaTransportProvider.class); private static final AtomicInteger TOPIC_COUNTER = new AtomicInteger(); private Properties _transportProviderProperties; @@ -228,12 +228,12 @@ private void testEventSend(int numberOfEvents, int numberOfPartitions, int parti KafkaTestUtils.waitForTopicCreation(_zkUtils, topicName, _kafkaCluster.getBrokers()); - LOG.info(String.format("Topic %s created with %d partitions and topic properties %s", topicName, numberOfPartitions, + _logger.info(String.format("Topic %s created with %d partitions and topic properties %s", topicName, numberOfPartitions, new Properties())); List datastreamEvents = createEvents(topicName, partition, numberOfEvents, includeKey, includeValue); - LOG.info(String.format("Trying to send %d events to topic %s", datastreamEvents.size(), topicName)); + _logger.info(String.format("Trying to send %d events to topic %s", datastreamEvents.size(), topicName)); final Integer[] callbackCalled = {0}; for (DatastreamProducerRecord event : datastreamEvents) { @@ -244,7 +244,7 @@ private void testEventSend(int numberOfEvents, int numberOfPartitions, int parti Assert.assertTrue(PollUtils.poll(() -> callbackCalled[0] == datastreamEvents.size(), 1000, 10000), "Send callback was not called; likely topic was not created in time"); - LOG.info(String.format("Trying to read events from the topicName %s partition %d", topicName, partition)); + _logger.info(String.format("Trying to read events from the topicName %s partition %d", topicName, partition)); Map events = new HashMap<>(); KafkaTestUtils.readTopic(topicName, partition, _kafkaCluster.getBrokers(), (key, value) -> { diff --git a/datastream-server-api/src/main/java/com/linkedin/datastream/server/DatastreamGroup.java b/datastream-server-api/src/main/java/com/linkedin/datastream/server/DatastreamGroup.java index 1b1b20a60..aa4c6578b 100644 --- a/datastream-server-api/src/main/java/com/linkedin/datastream/server/DatastreamGroup.java +++ b/datastream-server-api/src/main/java/com/linkedin/datastream/server/DatastreamGroup.java @@ -25,7 +25,7 @@ */ public class DatastreamGroup { - private static final Logger LOG = LoggerFactory.getLogger(DatastreamGroup.class.getName()); + private static final Logger _logger = LoggerFactory.getLogger(DatastreamGroup.class.getName()); private final String _taskPrefix; private final List _datastreams; @@ -44,7 +44,7 @@ public DatastreamGroup(List datastreams) { String msg = String.format("Datastreams within the group {%s} doesn't share the common connector name and task prefix", datastreams); - LOG.error(msg); + _logger.error(msg); throw new DatastreamRuntimeException(msg); } @@ -91,7 +91,7 @@ public boolean isPaused() { .map(ds -> ds.getName() + ds.getStatus()) .collect(Collectors.toList()); - LOG.warn("Some datastreams are paused in a group, while others are not. " + "Datastreams: " + streamsWithStatus); + _logger.warn("Some datastreams are paused in a group, while others are not. " + "Datastreams: " + streamsWithStatus); } return allPaused; } diff --git a/datastream-server-api/src/main/java/com/linkedin/datastream/server/DatastreamProducerRecord.java b/datastream-server-api/src/main/java/com/linkedin/datastream/server/DatastreamProducerRecord.java index 7e100f306..bf55caf28 100644 --- a/datastream-server-api/src/main/java/com/linkedin/datastream/server/DatastreamProducerRecord.java +++ b/datastream-server-api/src/main/java/com/linkedin/datastream/server/DatastreamProducerRecord.java @@ -29,15 +29,25 @@ public class DatastreamProducerRecord { private final List _events; // timestamp of when the record was sent to transport provider - private Optional _eventsSendTimestamp = Optional.empty(); + private Optional _eventsSendTimestamp; + + // timestamp when the record was landed on db, used by CDC + private Optional _eventsProduceTimestamp; DatastreamProducerRecord(List events, Optional partition, Optional partitionKey, - String checkpoint, long eventsSourceTimestamp) { - this(events, partition, partitionKey, Optional.empty(), checkpoint, eventsSourceTimestamp); + String checkpoint, long eventsSourceTimestamp, Optional eventSendTimestamp) { + this(events, partition, partitionKey, Optional.empty(), checkpoint, eventsSourceTimestamp, + eventSendTimestamp, Optional.empty()); } - DatastreamProducerRecord(List events, Optional partition, Optional partitionKey, - Optional destination, String checkpoint, long eventsSourceTimestamp) { + DatastreamProducerRecord(List events, + Optional partition, + Optional partitionKey, + Optional destination, + String checkpoint, + long eventsSourceTimestamp, + Optional eventSendTimestamp, + Optional eventsProduceTimestamp) { Validate.notNull(events, "null event"); events.forEach((e) -> Validate.notNull(e, "null event")); Validate.isTrue(eventsSourceTimestamp > 0, "events source timestamp is invalid"); @@ -47,7 +57,9 @@ public class DatastreamProducerRecord { _partitionKey = partitionKey; _checkpoint = checkpoint; _eventsSourceTimestamp = eventsSourceTimestamp; + _eventsProduceTimestamp = eventsProduceTimestamp; _destination = destination; + _eventsSendTimestamp = eventSendTimestamp; } /** @@ -78,6 +90,10 @@ public synchronized List getEvents() { return Collections.unmodifiableList(_events); } + public Optional getEventsProduceTimestamp() { + return _eventsProduceTimestamp; + } + /** * Get timestamp in Epoch-millis when the events were produced onto the source */ @@ -149,4 +165,5 @@ public Optional getPartitionKey() { public Optional getDestination() { return _destination; } + } diff --git a/datastream-server-api/src/main/java/com/linkedin/datastream/server/DatastreamProducerRecordBuilder.java b/datastream-server-api/src/main/java/com/linkedin/datastream/server/DatastreamProducerRecordBuilder.java index f53087f79..416965ad7 100644 --- a/datastream-server-api/src/main/java/com/linkedin/datastream/server/DatastreamProducerRecordBuilder.java +++ b/datastream-server-api/src/main/java/com/linkedin/datastream/server/DatastreamProducerRecordBuilder.java @@ -5,6 +5,7 @@ */ package com.linkedin.datastream.server; +import java.sql.Timestamp; import java.util.ArrayList; import java.util.List; import java.util.Map; @@ -22,7 +23,7 @@ */ public class DatastreamProducerRecordBuilder { - private static final Logger LOG = LoggerFactory.getLogger(DatastreamProducerRecordBuilder.class.getName()); + private static final Logger _logger = LoggerFactory.getLogger(DatastreamProducerRecordBuilder.class.getName()); private Optional _partition = Optional.empty(); private String _sourceCheckpoint = ""; @@ -30,6 +31,8 @@ public class DatastreamProducerRecordBuilder { private long _eventsSourceTimestamp; private Optional _partitionKey = Optional.empty(); private Optional _destination = Optional.empty(); + private Optional _eventSendTimestamp = Optional.empty(); + private Optional _eventsProduceTimestamp = Optional.empty(); /** * Partition to which this DatastreamProducerRecord should be produced. If the partition is not set, TransportProvider @@ -84,6 +87,18 @@ public void setEventsSourceTimestamp(long eventsSourceTimestamp) { _eventsSourceTimestamp = eventsSourceTimestamp; } + public void setEventsProduceTimestamp(Optional eventsProduceTimestamp) { + _eventsProduceTimestamp = eventsProduceTimestamp; + } + + /** + * Set the event send timestamp. + * @param eventSendTimestamp the timestamp + */ + public void setEventsSendTimestamp(final long eventSendTimestamp) { + _eventSendTimestamp = Optional.of(eventSendTimestamp); + } + /** * Build the DatastreamProducerRecord. * @return @@ -91,6 +106,6 @@ public void setEventsSourceTimestamp(long eventsSourceTimestamp) { */ public DatastreamProducerRecord build() { return new DatastreamProducerRecord(_events, _partition, _partitionKey, _destination, _sourceCheckpoint, - _eventsSourceTimestamp); + _eventsSourceTimestamp, _eventSendTimestamp, _eventsProduceTimestamp); } } diff --git a/datastream-server-api/src/main/java/com/linkedin/datastream/server/DatastreamTaskStatus.java b/datastream-server-api/src/main/java/com/linkedin/datastream/server/DatastreamTaskStatus.java index 892b94b7b..199694cd6 100644 --- a/datastream-server-api/src/main/java/com/linkedin/datastream/server/DatastreamTaskStatus.java +++ b/datastream-server-api/src/main/java/com/linkedin/datastream/server/DatastreamTaskStatus.java @@ -18,7 +18,7 @@ * Represent the status of a DatastreamTask with a code and message. */ public class DatastreamTaskStatus { - private static final Logger LOG = LoggerFactory.getLogger(DatastreamTaskStatus.class); + private static final Logger _logger = LoggerFactory.getLogger(DatastreamTaskStatus.class); /** * DatastreamTask status code @@ -94,7 +94,7 @@ private String localHostName() { try { return InetAddress.getLocalHost().getHostName(); } catch (UnknownHostException e) { - LOG.warn("Couldn't get the hostname"); + _logger.warn("Couldn't get the hostname"); return "unknown"; } } diff --git a/datastream-server-api/src/main/java/com/linkedin/datastream/server/FlushlessEventProducerHandler.java b/datastream-server-api/src/main/java/com/linkedin/datastream/server/FlushlessEventProducerHandler.java index 829d6c3f3..bea07d42d 100644 --- a/datastream-server-api/src/main/java/com/linkedin/datastream/server/FlushlessEventProducerHandler.java +++ b/datastream-server-api/src/main/java/com/linkedin/datastream/server/FlushlessEventProducerHandler.java @@ -34,7 +34,7 @@ */ public class FlushlessEventProducerHandler> { - private static final Logger LOG = LoggerFactory.getLogger(FlushlessEventProducerHandler.class); + private static final Logger _logger = LoggerFactory.getLogger(FlushlessEventProducerHandler.class); private final DatastreamEventProducer _eventProducer; private final ConcurrentHashMap _callbackStatusMap = new ConcurrentHashMap<>(); @@ -69,7 +69,7 @@ public void send(DatastreamProducerRecord record, String source, int sourceParti status.register(sourceCheckpoint); _eventProducer.send(record, ((metadata, exception) -> { if (exception != null) { - LOG.error("Failed to send datastream record: " + metadata, exception); + _logger.error("Failed to send datastream record: " + metadata, exception); } else { status.ack(sourceCheckpoint); } @@ -181,6 +181,7 @@ public long getInFlightCount() { * @param checkpoint the checkpoint to register */ public synchronized void register(T checkpoint) { + _logger.debug("register checkpoint {}", checkpoint); _inFlight.add(checkpoint); } @@ -190,8 +191,10 @@ public synchronized void register(T checkpoint) { * been received. */ public synchronized void ack(T checkpoint) { + _logger.debug("removing checkpoint {}, _inFlight size = {}", checkpoint, _inFlight.size()); + if (!_inFlight.remove(checkpoint)) { - LOG.error("Internal state error; could not remove checkpoint {}", checkpoint); + _logger.error("Internal state error; could not remove checkpoint {}", checkpoint); } _acked.add(checkpoint); @@ -213,7 +216,7 @@ public synchronized void ack(T checkpoint) { if (max != null) { if (_currentCheckpoint != null && max.compareTo(_currentCheckpoint) < 0) { // max is less than current checkpoint, should not happen - LOG.error( + _logger.error( "Internal error: checkpoints should progress in increasing order. Resolved checkpoint as {} which is " + "less than current checkpoint of {}", max, _currentCheckpoint); diff --git a/datastream-server-api/src/main/java/com/linkedin/datastream/server/NoOpConnectorFactory.java b/datastream-server-api/src/main/java/com/linkedin/datastream/server/NoOpConnectorFactory.java index 2724acff5..77dd361a8 100644 --- a/datastream-server-api/src/main/java/com/linkedin/datastream/server/NoOpConnectorFactory.java +++ b/datastream-server-api/src/main/java/com/linkedin/datastream/server/NoOpConnectorFactory.java @@ -33,7 +33,7 @@ public NoOpConnector createConnector(String connectorName, Properties config, St * A {@link Connector} implementation that does nothing. */ public static class NoOpConnector implements Connector { - private static final Logger LOG = LoggerFactory.getLogger(NoOpConnector.class.getName()); + private static final Logger _logger = LoggerFactory.getLogger(NoOpConnector.class.getName()); @Override public void start(CheckpointProvider checkpointProvider) { } @@ -83,7 +83,7 @@ public void validateUpdateDatastreams(List datastreams, List= _maxInflightWriteLogCommits) { - LOG.info("Waiting for room in commit backlog, current inflight commits {} ", + _logger.info("Waiting for room in commit backlog, current inflight commits {} ", _inflightWriteLogCommits); } while (_inflightWriteLogCommits >= _maxInflightWriteLogCommits) { @@ -60,7 +60,7 @@ protected void waitForRoomInCommitBacklog() throws InterruptedException { protected void waitForCommitBacklogToClear() throws InterruptedException { synchronized (_counterLock) { if (_inflightWriteLogCommits > 0) { - LOG.info("Waiting for the commit backlog to clear."); + _logger.info("Waiting for the commit backlog to clear."); } while (_inflightWriteLogCommits > 0) { try { diff --git a/datastream-server-api/src/main/java/com/linkedin/datastream/server/api/transport/buffered/AbstractBatchBuilder.java b/datastream-server-api/src/main/java/com/linkedin/datastream/server/api/transport/buffered/AbstractBatchBuilder.java index 43a4b42da..f0df2f06e 100644 --- a/datastream-server-api/src/main/java/com/linkedin/datastream/server/api/transport/buffered/AbstractBatchBuilder.java +++ b/datastream-server-api/src/main/java/com/linkedin/datastream/server/api/transport/buffered/AbstractBatchBuilder.java @@ -20,7 +20,7 @@ */ public abstract class AbstractBatchBuilder extends Thread { - protected static final Logger LOG = LoggerFactory.getLogger(AbstractBatchBuilder.class.getName()); + protected static final Logger _logger = LoggerFactory.getLogger(AbstractBatchBuilder.class.getName()); protected final Map _registry; @@ -46,7 +46,7 @@ public void assign(Package aPackage) { try { _packageQueue.put(aPackage); } catch (InterruptedException e) { - LOG.warn("Assign is interrupted. {}", e); + _logger.warn("Assign is interrupted. {}", e); Thread.currentThread().interrupt(); } } diff --git a/datastream-server-api/src/main/java/com/linkedin/datastream/server/api/transport/buffered/AbstractBufferedTransportProvider.java b/datastream-server-api/src/main/java/com/linkedin/datastream/server/api/transport/buffered/AbstractBufferedTransportProvider.java index 5664d382a..5b5e5046d 100644 --- a/datastream-server-api/src/main/java/com/linkedin/datastream/server/api/transport/buffered/AbstractBufferedTransportProvider.java +++ b/datastream-server-api/src/main/java/com/linkedin/datastream/server/api/transport/buffered/AbstractBufferedTransportProvider.java @@ -1,20 +1,27 @@ -/** - * Copyright 2020 Wayfair LLC. All rights reserved. - * Licensed under the BSD 2-Clause License. See the LICENSE file in the project root for license information. - * See the NOTICE file in the project root for additional information regarding copyright ownership. +/* + * Copyright 2020 Wayfair LLC. All rights reserved. + * Licensed under the BSD 2-Clause License. See the LICENSE file in the project root for license information. + * See the NOTICE file in the project root for additional information regarding copyright ownership. */ package com.linkedin.datastream.server.api.transport.buffered; import java.util.ArrayList; import java.util.List; -import java.util.concurrent.CopyOnWriteArrayList; +import java.util.Objects; +import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ScheduledThreadPoolExecutor; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; +import java.util.stream.IntStream; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import com.google.common.collect.ImmutableList; + import com.linkedin.datastream.common.BrooklinEnvelope; +import com.linkedin.datastream.common.DatastreamRecordMetadata; import com.linkedin.datastream.common.Package; import com.linkedin.datastream.common.Record; import com.linkedin.datastream.common.SendCallback; @@ -24,54 +31,101 @@ /** * Extend this abstract class to implement buffered writes to the destination. */ -public abstract class AbstractBufferedTransportProvider implements TransportProvider { - protected static final Logger LOG = LoggerFactory.getLogger(AbstractBufferedTransportProvider.class.getName()); +public abstract class AbstractBufferedTransportProvider implements TransportProvider { + protected static final Logger _logger = LoggerFactory.getLogger(AbstractBufferedTransportProvider.class.getName()); - protected static final String KAFKA_ORIGIN_TOPIC = "kafka-origin-topic"; - protected static final String KAFKA_ORIGIN_PARTITION = "kafka-origin-partition"; - protected static final String KAFKA_ORIGIN_OFFSET = "kafka-origin-offset"; + public static final String KAFKA_ORIGIN_CLUSTER = "kafka-origin-cluster"; + public static final String KAFKA_ORIGIN_TOPIC = "kafka-origin-topic"; + public static final String KAFKA_ORIGIN_PARTITION = "kafka-origin-partition"; + public static final String KAFKA_ORIGIN_OFFSET = "kafka-origin-offset"; protected final String _transportProviderName; protected final ScheduledExecutorService _scheduler = new ScheduledThreadPoolExecutor(1); - protected CopyOnWriteArrayList _batchBuilders = new CopyOnWriteArrayList<>(); + final protected ImmutableList _batchBuilders; + + static class DelayedCallback { + final SendCallback callback; + final DatastreamRecordMetadata metadata; + final Exception exception; + + DelayedCallback(final SendCallback callback, final DatastreamRecordMetadata metadata, final Exception exception) { + this.callback = callback; + this.metadata = metadata; + this.exception = exception; + } + } + final ImmutableList> delayedCallbackQueues; protected volatile boolean _isClosed; - protected AbstractBufferedTransportProvider(String transportProviderName) { + protected AbstractBufferedTransportProvider(final String transportProviderName, final List batchBuilders) { this._isClosed = false; this._transportProviderName = transportProviderName; + _batchBuilders = ImmutableList.copyOf(batchBuilders); + delayedCallbackQueues = ImmutableList.copyOf(IntStream.range(0, batchBuilders.size()).mapToObj( + i -> new ConcurrentLinkedQueue() + ).collect(Collectors.toList())); } private void delegate(final com.linkedin.datastream.common.Package aPackage) { this._batchBuilders.get(Math.abs(aPackage.hashCode() % _batchBuilders.size())).assign(aPackage); } + void fireDelayedCallbacks() { + delayedCallbackQueues.forEach(queue -> { + DelayedCallback delayedCallback; + while ((delayedCallback = queue.poll()) != null) { + delayedCallback.callback.onCompletion(delayedCallback.metadata, delayedCallback.exception); + } + }); + } + + void addDelayedCallback(final String topic, final String partition, final DelayedCallback delayedCallback) { + delayedCallbackQueues.get(Math.abs(Objects.hash(topic, partition) % delayedCallbackQueues.size())) + .add(delayedCallback); + } + @Override public void send(String destination, DatastreamProducerRecord record, SendCallback onComplete) { - for (final BrooklinEnvelope env : record.getEvents()) { + for (final BrooklinEnvelope env : record.getEvents()) { + final String topic = env.getMetadata().get(KAFKA_ORIGIN_TOPIC); + final String partition = env.getMetadata().get(KAFKA_ORIGIN_PARTITION); + final String offset = env.getMetadata().get(KAFKA_ORIGIN_OFFSET); final Package aPackage = new Package.PackageBuilder() .setRecord(new Record(env.getKey(), env.getValue())) - .setTopic(env.getMetadata().get(KAFKA_ORIGIN_TOPIC)) - .setPartition(env.getMetadata().get(KAFKA_ORIGIN_PARTITION)) - .setOffset(env.getMetadata().get(KAFKA_ORIGIN_OFFSET)) + .setTopic(topic) + .setPartition(partition) + .setOffset(offset) .setTimestamp(record.getEventsSourceTimestamp()) .setDestination(destination) - .setAckCallBack(onComplete) + .setAckCallBack((final DatastreamRecordMetadata metadata, final Exception exception) -> { + addDelayedCallback(topic, partition, new DelayedCallback(onComplete, metadata, exception)); + }) .setCheckpoint(record.getCheckpoint()) .build(); delegate(aPackage); } + fireDelayedCallbacks(); } @Override public synchronized void close() { if (_isClosed) { - LOG.info("Transport provider {} is already closed.", _transportProviderName); + _logger.info("Transport provider {} is already closed.", _transportProviderName); return; } try { - LOG.info("Closing the transport provider {}", _transportProviderName); + _logger.info("Closing the transport provider {}", _transportProviderName); + + _scheduler.shutdown(); + try { + _scheduler.awaitTermination(3, TimeUnit.SECONDS); + } catch (final InterruptedException e) { + _logger.warn("An interrupt was raised during awaitTermination() call on a ScheduledExecutorService"); + Thread.currentThread().interrupt(); + } + for (AbstractBatchBuilder objectBuilder : _batchBuilders) { objectBuilder.shutdown(); } @@ -80,12 +134,14 @@ public synchronized void close() { try { objectBuilder.join(); } catch (InterruptedException e) { - LOG.warn("An interrupt was raised during join() call on a Batch Builder"); + _logger.warn("An interrupt was raised during join() call on a Batch Builder"); Thread.currentThread().interrupt(); } } shutdownCommitter(); + + fireDelayedCallbacks(); } finally { _isClosed = true; } @@ -93,7 +149,7 @@ public synchronized void close() { @Override public void flush() { - LOG.info("Forcing flush on batch builders."); + _logger.info("Forcing flush on batch builders."); List flushSignalPackages = new ArrayList<>(); for (final AbstractBatchBuilder objectBuilder : _batchBuilders) { final Package aPackage = new Package.PackageBuilder().buildFroceFlushSignalPackage(); @@ -103,6 +159,7 @@ public void flush() { for (final Package aPackage : flushSignalPackages) { aPackage.waitUntilDelivered(); } + fireDelayedCallbacks(); } protected abstract void shutdownCommitter(); diff --git a/datastream-server-api/src/main/java/com/linkedin/datastream/server/providers/CDCCheckPoint.java b/datastream-server-api/src/main/java/com/linkedin/datastream/server/providers/CDCCheckPoint.java new file mode 100644 index 000000000..e9a48815c --- /dev/null +++ b/datastream-server-api/src/main/java/com/linkedin/datastream/server/providers/CDCCheckPoint.java @@ -0,0 +1,154 @@ +/** + * Copyright 2020 Wayfair LLC. All rights reserved. + * Licensed under the BSD 2-Clause License. See the LICENSE file in the project root for license information. + * See the NOTICE file in the project root for additional information regarding copyright ownership. + */ +package com.linkedin.datastream.server.providers; + +import java.util.Arrays; + +import org.jetbrains.annotations.NotNull; + +import com.linkedin.datastream.findbug.SuppressFBWarnings; + + +/** + * Represent the pointer all cdc rows with less and equal values already streamed to target. + */ +public class CDCCheckPoint implements PersistableCheckpoint, Comparable { + + private byte[] _lsn; + private int _offset; + + private int[] _lsnUnsignedBinary; + + /** + * Constructor + * @param lsn + * @param offset + */ + @SuppressFBWarnings(value = "EI_EXPOSE_REP2") + public CDCCheckPoint(byte[] lsn, int offset) { + this._lsn = lsn; + this._offset = offset; + } + + /** + * offset within the transaction + * @return + */ + public int offset() { + return _offset; + } + + /** + * Serialize CDC checkpoint to byte array + * format: [lsn (10 bytes) + offset (4 bytes)] + * @return + */ + @Override + public byte[] serialize() { + // Assume lsn is never null + byte[] buf = Arrays.copyOf(_lsn, 14); + System.arraycopy(Utils.intToBytes(_offset), 0, buf, 10, 4); + return buf; + } + + @Override + public int compareTo(@NotNull CDCCheckPoint checkpoint) { + return compare(checkpoint._lsn, checkpoint._offset); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj == null) { + return false; + } + if (getClass() != obj.getClass()) { + return false; + } + + return compareTo((CDCCheckPoint) obj) == 0; + } + + @Override + public int hashCode() { + final int prime = 31; + int result = 1; + result = prime * result + Arrays.hashCode(_lsn); + result = prime * result + _offset; + return result; + } + + @Override + public String toString() { + String lsnStr = (_lsn == null) ? "null" : Utils.bytesToHex(_lsn); + return lsnStr + ":" + _offset; + } + + private int compare(byte[] lsn2, int offset2) { + int lsnCmp = compareLsn(lsn2); + + // LSN not equal, return + if (lsnCmp != 0) { + return lsnCmp; + } + + return _offset - offset2; + } + + private int compareLsn(byte[] lsn2) { + final int[] thisU = lsnUnsignedBinary(); + final int[] thatU = getUnsignedBinary(lsn2); + for (int i = 0; i < thisU.length; i++) { + final int diff = thisU[i] - thatU[i]; + if (diff != 0) { + return diff; + } + } + return 0; + } + + /** + * Get the LSN in bytes + * @return + */ + @SuppressFBWarnings(value = "EI_EXPOSE_REP") + public byte[] lsnBytes() { + return _lsn; + } + + private int[] lsnUnsignedBinary() { + if (_lsnUnsignedBinary != null || _lsn == null) { + return _lsnUnsignedBinary; + } + _lsnUnsignedBinary = getUnsignedBinary(_lsn); + return _lsnUnsignedBinary; + } + + private static int[] getUnsignedBinary(byte[] binary) { + int[] unsignedBinary = new int[binary.length]; + for (int i = 0; i < binary.length; i++) { + unsignedBinary[i] = Byte.toUnsignedInt(binary[i]); + } + return unsignedBinary; + } + + /** + * Deserializer for CDC checkpoint + */ + public static class Deserializer implements PersistableCheckpoint.Deserializer { + + @Override + public T deserialize(byte[] value, Class checkpointClass) { + // Assume the length is 14 + byte[] lsn = Arrays.copyOfRange(value, 0, 10); + int offset = Utils.bytesToInt(value, 10); + + return checkpointClass.cast(new CDCCheckPoint(lsn, offset)); + } + } +} diff --git a/datastream-server-api/src/main/java/com/linkedin/datastream/server/providers/CustomCheckpointProvider.java b/datastream-server-api/src/main/java/com/linkedin/datastream/server/providers/CustomCheckpointProvider.java index bb9fd10ec..849009c1b 100644 --- a/datastream-server-api/src/main/java/com/linkedin/datastream/server/providers/CustomCheckpointProvider.java +++ b/datastream-server-api/src/main/java/com/linkedin/datastream/server/providers/CustomCheckpointProvider.java @@ -5,28 +5,18 @@ */ package com.linkedin.datastream.server.providers; +import java.util.function.Supplier; + /** * An abstraction for the connector to maintain information about the progress made * in processing {@link com.linkedin.datastream.server.DatastreamTask}s, e.g. checkpoints/offsets. * Use the implementation of this interface to support custom checkpointing by the connector. * * @param checkpoint value type + * @param Deserializer of Checkpoint */ -public interface CustomCheckpointProvider { - - /** - * Update the checkpoint to the given checkpoint. - * This checkpoint may not be persisted to underlying checkpoint store. - * @param checkpoint new checkpoint - */ - void updateCheckpoint(T checkpoint); - - /** - * Rewind the checkpoint to last know safe checkpoint. - * The safe checkpoint must be provided by the caller. - * @param checkpoint checkpoint to rewind to - */ - void rewindTo(T checkpoint); +public interface CustomCheckpointProvider { /** * Persist the safe checkpoint to the underlying checkpoint store @@ -40,14 +30,10 @@ public interface CustomCheckpointProvider { void close(); /** - * get safe checkpoint - * @return last known safe checkpoint - */ - T getSafeCheckpoint() throws Exception; - - /** - * get last committed checkpoint - * @return last committed checkpoint + * Get safe checkpoint + * @param deserSupplier + * @param checkpointClass + * @return */ - T getCommitted() throws Exception; + T getSafeCheckpoint(Supplier deserSupplier, Class checkpointClass); } diff --git a/datastream-server-api/src/main/java/com/linkedin/datastream/server/providers/JDBCCheckpoint.java b/datastream-server-api/src/main/java/com/linkedin/datastream/server/providers/JDBCCheckpoint.java new file mode 100644 index 000000000..2fc3d4130 --- /dev/null +++ b/datastream-server-api/src/main/java/com/linkedin/datastream/server/providers/JDBCCheckpoint.java @@ -0,0 +1,93 @@ +/** + * Copyright 2020 Wayfair LLC. All rights reserved. + * Licensed under the BSD 2-Clause License. See the LICENSE file in the project root for license information. + * See the NOTICE file in the project root for additional information regarding copyright ownership. + */ +package com.linkedin.datastream.server.providers; + +import java.nio.charset.UnsupportedCharsetException; +import java.util.Arrays; + +import org.jetbrains.annotations.NotNull; + +import com.google.common.primitives.Longs; + +import com.linkedin.datastream.common.DatastreamRuntimeException; + + +/** + * Represent a checkpoint in JDBC connector + */ +public class JDBCCheckpoint implements PersistableCheckpoint, Comparable { + long _offset = 1; + + /** + * Constructor + * @param offset + */ + public JDBCCheckpoint(long offset) { + _offset = offset; + } + + @Override + public byte[] serialize() { + return Longs.toByteArray(_offset); + } + + @Override + public int compareTo(@NotNull JDBCCheckpoint o) { + return Long.compare(this._offset, o._offset); + } + + /** + * Get offset + * @return + */ + public Long offset() { + return _offset; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj == null) { + return false; + } + if (getClass() != obj.getClass()) { + return false; + } + + return compareTo((JDBCCheckpoint) obj) == 0; + } + + @Override + public int hashCode() { + return Long.hashCode(_offset); + } + + @Override + public String toString() { + return String.valueOf(_offset); + } + + /** + * JDBC checkpoint deserializer + */ + public static class Deserializer implements PersistableCheckpoint.Deserializer { + + @Override + public T deserialize(byte[] value, Class checkpointClass) { + long offset = -1; + try { + // Old version of checkpoint uses the string representation of Long. To be compatible still use same format. + offset = Longs.fromByteArray(value); + } catch (NumberFormatException | UnsupportedCharsetException e) { + throw new DatastreamRuntimeException("Invalid CDC checkpoint offset " + Arrays.toString(value)); + } + + return checkpointClass.cast(new JDBCCheckpoint(offset)); + } + } +} diff --git a/datastream-server-api/src/main/java/com/linkedin/datastream/server/providers/PersistableCheckpoint.java b/datastream-server-api/src/main/java/com/linkedin/datastream/server/providers/PersistableCheckpoint.java new file mode 100644 index 000000000..c8e6f73ed --- /dev/null +++ b/datastream-server-api/src/main/java/com/linkedin/datastream/server/providers/PersistableCheckpoint.java @@ -0,0 +1,32 @@ +/** + * Copyright 2020 Wayfair LLC. All rights reserved. + * Licensed under the BSD 2-Clause License. See the LICENSE file in the project root for license information. + * See the NOTICE file in the project root for additional information regarding copyright ownership. + */ +package com.linkedin.datastream.server.providers; + + +/** + * Represent a checkpoint which could be persisted somewhere. + */ +public interface PersistableCheckpoint { + /** + * Serialize a checkpoint into String representation so the checkpoint could be stored into somewhere like Kafka + * @return + */ + byte[] serialize(); + + /** + * Deserializer of Checkpoint + */ + interface Deserializer { + + /** + * Deserialize a value represented as String and return an instance of PersistableCheckpoint + * @param value + * @param + * @return + */ + T deserialize(byte[] value, Class checkpointClass); + } +} diff --git a/datastream-server-api/src/main/java/com/linkedin/datastream/server/providers/Utils.java b/datastream-server-api/src/main/java/com/linkedin/datastream/server/providers/Utils.java new file mode 100644 index 000000000..74d8db8bc --- /dev/null +++ b/datastream-server-api/src/main/java/com/linkedin/datastream/server/providers/Utils.java @@ -0,0 +1,68 @@ +/** + * Copyright 2020 Wayfair LLC. All rights reserved. + * Licensed under the BSD 2-Clause License. See the LICENSE file in the project root for license information. + * See the NOTICE file in the project root for additional information regarding copyright ownership. + */ +package com.linkedin.datastream.server.providers; + + +/** + * Encoding Utility class + */ +public class Utils { + + private final static char[] HEX_CHARS = "0123456789ABCDEF".toCharArray(); + + /** + * Convert a binary array to hex string + * @param bytes + * @return + */ + public static String bytesToHex(byte[] bytes) { + char[] hexChars = new char[bytes.length * 2]; + for (int j = 0; j < bytes.length; j++) { + int v = bytes[j] & 0xFF; + hexChars[j * 2] = HEX_CHARS[v >>> 4]; + hexChars[j * 2 + 1] = HEX_CHARS[v & 0x0F]; + } + return String.valueOf(hexChars); + } + + /** + * Convert a integer to byte array + * @param l + * @return + */ + public static byte[] intToBytes(int l) { + byte[] result = new byte[4]; + for (int i = 3; i >= 0; i--) { + result[i] = (byte) (l & 0xFF); + l >>= 4; + } + return result; + } + + /** + * byte array to int + * @param b + * @return + */ + public static int bytesToInt(final byte[] b) { + return bytesToInt(b, 0); + } + + /** + * byte array to int + * @param b + * @param from + * @return + */ + public static int bytesToInt(final byte[] b, final int from) { + int result = 0; + for (int i = from; i < from + 4; i++) { + result <<= 4; + result |= (b[i] & 0xFF); + } + return result; + } +} diff --git a/datastream-server-api/src/test/java/com/linkedin/datastream/server/api/transport/buffered/TestAbstractBufferedTransportProvider.java b/datastream-server-api/src/test/java/com/linkedin/datastream/server/api/transport/buffered/TestAbstractBufferedTransportProvider.java new file mode 100644 index 000000000..6d205218b --- /dev/null +++ b/datastream-server-api/src/test/java/com/linkedin/datastream/server/api/transport/buffered/TestAbstractBufferedTransportProvider.java @@ -0,0 +1,125 @@ +/** + * Copyright 2020 Wayfair LLC. All rights reserved. + * Licensed under the BSD 2-Clause License. See the LICENSE file in the project root for license information. + * See the NOTICE file in the project root for additional information regarding copyright ownership. + */ +package com.linkedin.datastream.server.api.transport.buffered; + +import java.time.Instant; +import java.util.List; +import java.util.Map; + +import org.testng.annotations.Test; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; + +import com.linkedin.datastream.common.BrooklinEnvelope; +import com.linkedin.datastream.common.DatastreamRecordMetadata; +import com.linkedin.datastream.common.Package; +import com.linkedin.datastream.common.SendCallback; +import com.linkedin.datastream.server.DatastreamProducerRecordBuilder; + +import static com.linkedin.datastream.server.api.transport.buffered.AbstractBufferedTransportProvider.KAFKA_ORIGIN_OFFSET; +import static com.linkedin.datastream.server.api.transport.buffered.AbstractBufferedTransportProvider.KAFKA_ORIGIN_PARTITION; +import static com.linkedin.datastream.server.api.transport.buffered.AbstractBufferedTransportProvider.KAFKA_ORIGIN_TOPIC; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyZeroInteractions; + +/** + * Test AbstractBufferedTransportProvider. + */ +public class TestAbstractBufferedTransportProvider { + + private static class TestBufferedTransportProvider extends AbstractBufferedTransportProvider { + + TestBufferedTransportProvider(final String transportProviderName, final List batchBuilders) { + super(transportProviderName, batchBuilders); + } + + @Override + protected void shutdownCommitter() { + + } + } + + private static class TestBatchBuilder extends AbstractBatchBuilder { + TestBatchBuilder(final int queueSize) { + super(queueSize); + } + + @Override + public void assign(Package aPackage) { + super.assign(aPackage); + aPackage.markAsDelivered(); + } + } + + @Test + public void testDelayedCallbackOnFlush() throws InterruptedException { + final AbstractBatchBuilder batchBuilder = new TestBatchBuilder(2); + final AbstractBufferedTransportProvider transportProvider = new TestBufferedTransportProvider("test", ImmutableList.of(batchBuilder)); + final String destination = "test"; + + final Object key = null; + final Object value = null; + final Map metadata = ImmutableMap.of( + KAFKA_ORIGIN_TOPIC, "topic", + KAFKA_ORIGIN_PARTITION, "0", + KAFKA_ORIGIN_OFFSET, "0" + ); + final BrooklinEnvelope envelope = new BrooklinEnvelope(key, value, metadata); + final long eventSourceTimestamp = Instant.now().toEpochMilli(); + final DatastreamProducerRecordBuilder recordBuilder = new DatastreamProducerRecordBuilder(); + recordBuilder.addEvent(envelope); + recordBuilder.setEventsSourceTimestamp(eventSourceTimestamp); + final SendCallback onComplete = mock(SendCallback.class); + transportProvider.send(destination, recordBuilder.build(), onComplete); + final Package aPackage = batchBuilder.getNextPackage(); + final DatastreamRecordMetadata recordMetadata = mock(DatastreamRecordMetadata.class); + final Exception exception = mock(Exception.class); + aPackage.getAckCallback().onCompletion(recordMetadata, exception); + verifyZeroInteractions(onComplete); + transportProvider.flush(); + verify(onComplete).onCompletion(recordMetadata, exception); + } + + @Test + public void testDelayedCallbackOnNextSend() throws InterruptedException { + final AbstractBatchBuilder batchBuilder = new TestBatchBuilder(2); + final AbstractBufferedTransportProvider transportProvider = new TestBufferedTransportProvider("test", ImmutableList.of(batchBuilder)); + final String destination = "test"; + + final DatastreamProducerRecordBuilder recordBuilder1 = new DatastreamProducerRecordBuilder(); + recordBuilder1.addEvent(new BrooklinEnvelope(null, null, ImmutableMap.of( + KAFKA_ORIGIN_TOPIC, "topic", + KAFKA_ORIGIN_PARTITION, "0", + KAFKA_ORIGIN_OFFSET, "0" + ))); + recordBuilder1.setEventsSourceTimestamp(Instant.now().toEpochMilli()); + final SendCallback onComplete1 = mock(SendCallback.class); + transportProvider.send(destination, recordBuilder1.build(), onComplete1); + + final Package aPackage = batchBuilder.getNextPackage(); + final DatastreamRecordMetadata recordMetadata = mock(DatastreamRecordMetadata.class); + final Exception exception = mock(Exception.class); + aPackage.getAckCallback().onCompletion(recordMetadata, exception); + + verifyZeroInteractions(onComplete1); + + final DatastreamProducerRecordBuilder recordBuilder2 = new DatastreamProducerRecordBuilder(); + recordBuilder2.addEvent(new BrooklinEnvelope(null, null, ImmutableMap.of( + KAFKA_ORIGIN_TOPIC, "topic", + KAFKA_ORIGIN_PARTITION, "0", + KAFKA_ORIGIN_OFFSET, "1" + ))); + recordBuilder2.setEventsSourceTimestamp(Instant.now().toEpochMilli()); + final SendCallback onCallback2 = mock(SendCallback.class); + transportProvider.send(destination, recordBuilder2.build(), onCallback2); + verify(onComplete1).onCompletion(recordMetadata, exception); + verifyZeroInteractions(onCallback2); + } + +} diff --git a/datastream-server-restli/src/main/java/com/linkedin/datastream/server/DatastreamJettyStandaloneLauncher.java b/datastream-server-restli/src/main/java/com/linkedin/datastream/server/DatastreamJettyStandaloneLauncher.java index aacf2c0fb..d8f7026fb 100644 --- a/datastream-server-restli/src/main/java/com/linkedin/datastream/server/DatastreamJettyStandaloneLauncher.java +++ b/datastream-server-restli/src/main/java/com/linkedin/datastream/server/DatastreamJettyStandaloneLauncher.java @@ -33,7 +33,7 @@ * the datastream Rest.li resources. */ public class DatastreamJettyStandaloneLauncher { - private static final Logger LOG = LoggerFactory.getLogger(DatastreamJettyStandaloneLauncher.class.getName()); + private static final Logger _logger = LoggerFactory.getLogger(DatastreamJettyStandaloneLauncher.class.getName()); private final String[] _packages; private final HttpJettyServer _httpJettyServer; @@ -94,7 +94,7 @@ public void start() throws Exception { _port = jettyServer.getURI().getPort(); } - LOG.info("Embedded Jetty started with port: " + _port); + _logger.info("Embedded Jetty started with port: " + _port); } /** diff --git a/datastream-server-restli/src/main/java/com/linkedin/datastream/server/DatastreamServer.java b/datastream-server-restli/src/main/java/com/linkedin/datastream/server/DatastreamServer.java index 5cb63f274..46f3299c9 100644 --- a/datastream-server-restli/src/main/java/com/linkedin/datastream/server/DatastreamServer.java +++ b/datastream-server-restli/src/main/java/com/linkedin/datastream/server/DatastreamServer.java @@ -30,6 +30,7 @@ import com.codahale.metrics.MetricRegistry; import com.codahale.metrics.jmx.JmxReporter; +import com.linkedin.datastream.common.Datastream; import com.linkedin.datastream.common.DatastreamException; import com.linkedin.datastream.common.DatastreamRuntimeException; import com.linkedin.datastream.common.ErrorLogger; @@ -44,6 +45,7 @@ import com.linkedin.datastream.server.api.connector.ConnectorFactory; import com.linkedin.datastream.server.api.connector.DatastreamDeduper; import com.linkedin.datastream.server.api.connector.DatastreamDeduperFactory; +import com.linkedin.datastream.server.api.connector.DatastreamValidationException; import com.linkedin.datastream.server.api.serde.SerdeAdmin; import com.linkedin.datastream.server.api.serde.SerdeAdminFactory; import com.linkedin.datastream.server.api.strategy.AssignmentStrategy; @@ -55,6 +57,9 @@ import com.linkedin.datastream.server.dms.DatastreamResources; import com.linkedin.datastream.server.dms.DatastreamStore; import com.linkedin.datastream.server.dms.ZookeeperBackedDatastreamStore; +import com.linkedin.datastream.server.providers.CustomCheckpointProvider; +import com.linkedin.datastream.server.providers.JDBCCheckpoint; +import com.linkedin.datastream.server.providers.KafkaCustomCheckpointProvider; import static com.linkedin.datastream.server.DatastreamServerConfigurationConstants.CONFIG_CONNECTOR_ASSIGNMENT_STRATEGY_FACTORY; import static com.linkedin.datastream.server.DatastreamServerConfigurationConstants.CONFIG_CONNECTOR_AUTHORIZER_NAME; @@ -86,12 +91,15 @@ * datastream services including the REST API service, the coordinator and so on. */ public class DatastreamServer { - private static final Logger LOG = LoggerFactory.getLogger(DatastreamServer.class); + private static final Logger _logger = LoggerFactory.getLogger(DatastreamServer.class); private static final MetricRegistry METRIC_REGISTRY = new MetricRegistry(); private static final List METRIC_INFOS = new ArrayList<>(); + private static final String CONFIG_CHECKPOINT_STORE_URL = "checkpointStoreURL"; + private static final String CONFIG_CHECKPOINT_STORE_TOPIC = "checkpointStoreTopic"; private final String _csvMetricsDir; private final Map _bootstrapConnectors; + private final Properties _properties; private Coordinator _coordinator; private DatastreamStore _datastreamStore; @@ -128,15 +136,16 @@ public class DatastreamServer { * */ public DatastreamServer(Properties properties) throws DatastreamException { - LOG.info("Start to initialize DatastreamServer. Properties: " + properties); - LOG.info("Creating coordinator."); - VerifiableProperties verifiableProperties = new VerifiableProperties(properties); + _properties = properties; + _logger.info("Start to initialize DatastreamServer. Properties: " + _properties); + _logger.info("Creating coordinator."); + VerifiableProperties verifiableProperties = new VerifiableProperties(_properties); HashSet connectorTypes = new HashSet<>(verifiableProperties.getStringList(CONFIG_CONNECTOR_NAMES, Collections.emptyList())); if (connectorTypes.size() == 0) { String errorMessage = "No connectors specified in connectorTypes"; - LOG.error(errorMessage); + _logger.error(errorMessage); throw new DatastreamRuntimeException(errorMessage); } @@ -144,19 +153,19 @@ public DatastreamServer(Properties properties) throws DatastreamException { new HashSet<>(verifiableProperties.getStringList(CONFIG_TRANSPORT_PROVIDER_NAMES, Collections.emptyList())); if (transportProviderNames.size() == 0) { String errorMessage = "No transport providers specified in config: " + CONFIG_TRANSPORT_PROVIDER_NAMES; - LOG.error(errorMessage); + _logger.error(errorMessage); throw new DatastreamRuntimeException(errorMessage); } - CoordinatorConfig coordinatorConfig = new CoordinatorConfig(properties); + CoordinatorConfig coordinatorConfig = new CoordinatorConfig(_properties); - LOG.info("Setting up DMS endpoint server."); + _logger.info("Setting up DMS endpoint server."); ZkClient zkClient = new ZkClient(coordinatorConfig.getZkAddress(), coordinatorConfig.getZkSessionTimeout(), coordinatorConfig.getZkConnectionTimeout()); CachedDatastreamReader datastreamCache = new CachedDatastreamReader(zkClient, coordinatorConfig.getCluster()); _coordinator = new Coordinator(datastreamCache, coordinatorConfig); - LOG.info("Loading connectors {}", connectorTypes); + _logger.info("Loading connectors {}", connectorTypes); _bootstrapConnectors = new HashMap<>(); for (String connectorStr : connectorTypes) { initializeConnector(connectorStr, @@ -164,14 +173,14 @@ public DatastreamServer(Properties properties) throws DatastreamException { coordinatorConfig.getCluster()); } - LOG.info("Loading Transport providers {}", transportProviderNames); + _logger.info("Loading Transport providers {}", transportProviderNames); for (String tpName : transportProviderNames) { initializeTransportProvider(tpName, verifiableProperties.getDomainProperties(CONFIG_TRANSPORT_PROVIDER_PREFIX + tpName)); } Set serdeNames = new HashSet<>(verifiableProperties.getStringList(CONFIG_SERDE_NAMES, Collections.emptyList())); - LOG.info("Loading Serdes {} ", serdeNames); + _logger.info("Loading Serdes {} ", serdeNames); for (String serde : serdeNames) { initializeSerde(serde, verifiableProperties.getDomainProperties(CONFIG_SERDE_PREFIX + serde)); } @@ -203,7 +212,7 @@ public DatastreamServer(Properties properties) throws DatastreamException { _isInitialized = true; - LOG.info("DatastreamServer initialized successfully."); + _logger.info("DatastreamServer initialized successfully."); } public synchronized boolean isInitialized() { @@ -234,20 +243,55 @@ public ServerComponentHealthAggregator getServerComponentHealthAggregator() { return _serverComponentHealthAggregator; } + /** + * Get instance of CustomCheckpointProvider + * @param datastream + * @return CustomCheckpointProvider + */ + public CustomCheckpointProvider + getCustomCheckpointProvider(Datastream datastream) throws DatastreamException { + if (datastream.getName().isEmpty()) { + String msg = "Datastream name is empty"; + _logger.error(msg); + throw new DatastreamValidationException(msg); + } + if (datastream.getMetadata().get("incrementingColumnName").isEmpty()) { + String msg = "Datastream incrementingColumnName is empty"; + _logger.error(msg); + throw new DatastreamValidationException(msg); + } + if (datastream.getMetadata().get("destinationTopic").isEmpty()) { + String msg = "Datastream destinationTopic is empty"; + _logger.error(msg); + throw new DatastreamValidationException(msg); + } + VerifiableProperties verifiableProperties = new VerifiableProperties(_properties); + Properties connectorProperties = verifiableProperties.getDomainProperties(CONFIG_CONNECTOR_PREFIX + datastream.getConnectorName()); + String checkpointStoreUrl = connectorProperties.getProperty(CONFIG_CHECKPOINT_STORE_URL); + String checkpointStoreTopic = connectorProperties.getProperty(CONFIG_CHECKPOINT_STORE_TOPIC); + String jdbcIdCheckpointId; + String idString = + datastream.getName() + " " + datastream.getMetadata().get("incrementingColumnName") + " " + datastream.getMetadata().get("destinationTopic"); + long hash = idString.hashCode(); + jdbcIdCheckpointId = String.valueOf(hash > 0 ? hash : -hash); + return new KafkaCustomCheckpointProvider( + jdbcIdCheckpointId, checkpointStoreUrl, checkpointStoreTopic); + } + private void initializeSerde(String serdeName, Properties serdeConfig) { - LOG.info("Starting to load the serde:{} with config: {} ", serdeName, serdeConfig); + _logger.info("Starting to load the serde:{} with config: {} ", serdeName, serdeConfig); String factoryClassName = serdeConfig.getProperty(CONFIG_FACTORY_CLASS_NAME, ""); if (StringUtils.isBlank(factoryClassName)) { String msg = "Factory class name is not set or empty for serde: " + serdeName; - LOG.error(msg); + _logger.error(msg); throw new DatastreamRuntimeException(msg); } SerdeAdminFactory factory = ReflectionUtils.createInstance(factoryClassName); if (factory == null) { String msg = "Invalid class name or no parameter-less constructor, class=" + factoryClassName; - LOG.error(msg); + _logger.error(msg); throw new DatastreamRuntimeException(msg); } @@ -256,19 +300,19 @@ private void initializeSerde(String serdeName, Properties serdeConfig) { } private void initializeTransportProvider(String transportProviderName, Properties transportProviderConfig) { - LOG.info("Starting to load the transport provider: " + transportProviderName); + _logger.info("Starting to load the transport provider: " + transportProviderName); String factoryClassName = transportProviderConfig.getProperty(CONFIG_FACTORY_CLASS_NAME, ""); if (StringUtils.isBlank(factoryClassName)) { String msg = "Factory class name is not set or empty for transport provider: " + transportProviderName; - LOG.error(msg); + _logger.error(msg); throw new DatastreamRuntimeException(msg); } TransportProviderAdminFactory factory = ReflectionUtils.createInstance(factoryClassName); if (factory == null) { String msg = "Invalid class name or no parameter-less constructor, class=" + factoryClassName; - LOG.error(msg); + _logger.error(msg); throw new DatastreamRuntimeException(msg); } @@ -277,7 +321,7 @@ private void initializeTransportProvider(String transportProviderName, Propertie } private void initializeConnector(String connectorName, Properties connectorProperties, String clusterName) { - LOG.info("Starting to load connector: " + connectorName); + _logger.info("Starting to load connector: " + connectorName); VerifiableProperties connectorProps = new VerifiableProperties(connectorProperties); @@ -285,13 +329,13 @@ private void initializeConnector(String connectorName, Properties connectorPrope String className = connectorProperties.getProperty(CONFIG_FACTORY_CLASS_NAME, ""); if (StringUtils.isBlank(className)) { String errorMessage = "Factory className is empty for connector " + connectorName; - ErrorLogger.logAndThrowDatastreamRuntimeException(LOG, errorMessage, null); + ErrorLogger.logAndThrowDatastreamRuntimeException(_logger, errorMessage, null); } ConnectorFactory connectorFactoryInstance = ReflectionUtils.createInstance(className); if (connectorFactoryInstance == null) { String msg = "Invalid class name or no parameter-less constructor, class=" + className; - ErrorLogger.logAndThrowDatastreamRuntimeException(LOG, msg, null); + ErrorLogger.logAndThrowDatastreamRuntimeException(_logger, msg, null); } Connector connectorInstance = @@ -312,7 +356,7 @@ private void initializeConnector(String connectorName, Properties connectorPrope if (assignmentStrategyFactoryInstance == null) { String errorMessage = "Invalid strategy factory class: " + strategyFactory; - ErrorLogger.logAndThrowDatastreamRuntimeException(LOG, errorMessage, null); + ErrorLogger.logAndThrowDatastreamRuntimeException(_logger, errorMessage, null); } Properties strategyProps = connectorProps.getDomainProperties(STRATEGY_DOMAIN); @@ -327,7 +371,7 @@ private void initializeConnector(String connectorName, Properties connectorPrope if (deduperFactoryInstance == null) { String errorMessage = "Invalid de-duper factory class: " + deduperFactory; - ErrorLogger.logAndThrowDatastreamRuntimeException(LOG, errorMessage, null); + ErrorLogger.logAndThrowDatastreamRuntimeException(_logger, errorMessage, null); } Properties deduperProps = connectorProps.getDomainProperties(DOMAIN_DEDUPER); @@ -341,7 +385,7 @@ private void initializeConnector(String connectorName, Properties connectorPrope _coordinator.addConnector(connectorName, connectorInstance, assignmentStrategy, customCheckpointing, deduper, authorizerName); - LOG.info("Connector loaded successfully. Type: " + connectorName); + _logger.info("Connector loaded successfully. Type: " + connectorName); } private void initializeMetrics() { @@ -352,12 +396,12 @@ private void initializeMetrics() { _jmxReporter = JmxReporterFactory.createJmxReporter(METRIC_REGISTRY); if (StringUtils.isNotEmpty(_csvMetricsDir)) { - LOG.info("Starting CsvReporter in " + _csvMetricsDir); + _logger.info("Starting CsvReporter in " + _csvMetricsDir); File csvDir = new File(_csvMetricsDir); if (!csvDir.exists()) { - LOG.info("csvMetricsDir {} doesn't exist, creating it.", _csvMetricsDir); + _logger.info("csvMetricsDir {} doesn't exist, creating it.", _csvMetricsDir); if (!csvDir.mkdirs()) { - LOG.warn("failed to created csvMetricsDir"); + _logger.warn("failed to created csvMetricsDir"); } } @@ -394,7 +438,7 @@ public synchronized void startup() throws DatastreamException { _serverComponentHealthAggregator.setPort(_httpPort); _isStarted = true; } catch (Exception ex) { - ErrorLogger.logAndThrowDatastreamRuntimeException(LOG, "Failed to start embedded Jetty.", ex); + ErrorLogger.logAndThrowDatastreamRuntimeException(_logger, "Failed to start embedded Jetty.", ex); } } @@ -412,7 +456,7 @@ public synchronized void shutdown() { try { _jettyLauncher.stop(); } catch (Exception e) { - LOG.error("Failed to shutdown embedded Jetty.", e); + _logger.error("Failed to shutdown embedded Jetty.", e); } _jettyLauncher = null; } @@ -438,7 +482,7 @@ public static void main(String[] args) throws Exception { // attach shutdown handler to catch control-c Runtime.getRuntime().addShutdownHook(new Thread(() -> { lock.lock(); - LOG.info("Starting the shutdown process.."); + _logger.info("Starting the shutdown process.."); server.shutdown(); shutdownCondition.signalAll(); })); @@ -446,7 +490,7 @@ public static void main(String[] args) throws Exception { lock.lock(); server.startup(); shutdownCondition.await(); - LOG.info("Main thread is exiting..."); + _logger.info("Main thread is exiting..."); } private static Properties getServerProperties(String[] args) throws IOException { @@ -472,4 +516,15 @@ public static Properties loadProps(String filename) throws IOException { return props; } + + /** + * Check if connector name uses custom checkpointing + * @param connectorName + * @return + */ + public boolean isCustomCheckpointing(String connectorName) { + VerifiableProperties verifiableProperties = new VerifiableProperties(_properties); + Properties connectorProperties = verifiableProperties.getDomainProperties(CONFIG_CONNECTOR_PREFIX + connectorName); + return Boolean.parseBoolean(connectorProperties.getProperty(CONFIG_CONNECTOR_CUSTOM_CHECKPOINTING, "false")); + } } diff --git a/datastream-server-restli/src/main/java/com/linkedin/datastream/server/diagnostics/ServerComponentHealthAggregator.java b/datastream-server-restli/src/main/java/com/linkedin/datastream/server/diagnostics/ServerComponentHealthAggregator.java index 0579321e0..2c3339195 100644 --- a/datastream-server-restli/src/main/java/com/linkedin/datastream/server/diagnostics/ServerComponentHealthAggregator.java +++ b/datastream-server-restli/src/main/java/com/linkedin/datastream/server/diagnostics/ServerComponentHealthAggregator.java @@ -31,7 +31,7 @@ */ public class ServerComponentHealthAggregator { - private static final Logger LOG = LoggerFactory.getLogger(ServerComponentHealthAggregator.class.getName()); + private static final Logger _logger = LoggerFactory.getLogger(ServerComponentHealthAggregator.class.getName()); private final ZkClient _zkClient; private final String _cluster; @@ -73,7 +73,7 @@ public List getResponses(String componentType, String com hosts.parallelStream().forEach(hostName -> { // Send requests to all the server live instances String dmsUri = getDmsUri(hostName); - LOG.info("Send restli status request to " + dmsUri); + _logger.info("Send restli status request to " + dmsUri); ServerComponentHealthRestClient restClient = ServerComponentHealthRestClientFactory.getClient(dmsUri); ServerComponentHealth response = null; @@ -82,18 +82,18 @@ public List getResponses(String componentType, String com response = restClient.getStatus(componentType, componentScope, componentInputs); } catch (Exception e) { errorMessage = "Received REST exception: " + e.toString() + " from the host: " + dmsUri; - LOG.error("Received REST exception from the host: {}", dmsUri, e); + _logger.error("Received REST exception from the host: {}", dmsUri, e); } finally { // No response received from a host, set error message if (response == null && errorMessage.isEmpty()) { errorMessage = "Failed to receive REST response from the host: " + dmsUri; - LOG.error(errorMessage); + _logger.error(errorMessage); } if (!errorMessage.isEmpty()) { errorResponses.put(hostName, errorMessage); } else { String message = "Received REST response from the host: " + dmsUri + " with status: " + response.getStatus(); - LOG.info(message); + _logger.info(message); responses.put(hostName, response.getStatus()); } } @@ -108,7 +108,7 @@ public List getResponses(String componentType, String com try { localhostName = InetAddress.getLocalHost().getHostName(); } catch (UnknownHostException uhe) { - LOG.error("Could not get localhost Name {}", uhe.getMessage()); + _logger.error("Could not get localhost Name {}", uhe.getMessage()); } serverComponentHealth.setInstanceName(localhostName); serverComponentHealth.setErrorMessages(errorResponses.toString()); diff --git a/datastream-server-restli/src/main/java/com/linkedin/datastream/server/diagnostics/ServerComponentHealthResources.java b/datastream-server-restli/src/main/java/com/linkedin/datastream/server/diagnostics/ServerComponentHealthResources.java index 9bd9bcb03..cf21eeeab 100644 --- a/datastream-server-restli/src/main/java/com/linkedin/datastream/server/diagnostics/ServerComponentHealthResources.java +++ b/datastream-server-restli/src/main/java/com/linkedin/datastream/server/diagnostics/ServerComponentHealthResources.java @@ -47,7 +47,7 @@ public class ServerComponentHealthResources extends CollectionResourceTemplate { public static final String CONNECTOR_NAME = "connector"; - private static final Logger LOG = LoggerFactory.getLogger(ServerComponentHealthResources.class); + private static final Logger _logger = LoggerFactory.getLogger(ServerComponentHealthResources.class); private final ServerComponentHealthAggregator _aggregator; private final DatastreamServer _server; private final Coordinator _coordinator; @@ -60,7 +60,7 @@ public ServerComponentHealthResources(DatastreamServer datastreamServer) { _aggregator = datastreamServer.getServerComponentHealthAggregator(); _server = datastreamServer; _coordinator = datastreamServer.getCoordinator(); - _errorLogger = new ErrorLogger(LOG, _coordinator.getInstanceName()); + _errorLogger = new ErrorLogger(_logger, _coordinator.getInstanceName()); } /** @@ -73,7 +73,7 @@ public List getAllStatus(@PagingContextParam PagingContex @QueryParam("scope") String componentScope, // Espresso, EspressoBootstrap and etc. @QueryParam("content") @Optional String componentInputs) { - LOG.info("Restli getAllStatus request with name: {}, type: {} and content: {}.", componentType, componentScope, + _logger.info("Restli getAllStatus request with name: {}, type: {} and content: {}.", componentType, componentScope, componentInputs); DiagnosticsAware component = getComponent(componentType, componentScope); if (component != null) { @@ -94,7 +94,7 @@ public List getStatus(@PagingContextParam PagingContext c @QueryParam("scope") String componentScope, @QueryParam("content") @Optional String componentInputs) { - LOG.info("Restli getStatus request with name: {}, type: {} and content: {}.", componentType, componentScope, + _logger.info("Restli getStatus request with name: {}, type: {} and content: {}.", componentType, componentScope, componentInputs); ServerComponentHealth serverComponentHealth = new ServerComponentHealth(); @@ -116,7 +116,7 @@ public List getStatus(@PagingContextParam PagingContext c try { localhostName = InetAddress.getLocalHost().getHostName(); } catch (UnknownHostException uhe) { - LOG.error("Could not get localhost Name {}", uhe.getMessage()); + _logger.error("Could not get localhost Name {}", uhe.getMessage()); } serverComponentHealth.setInstanceName(localhostName); return Collections.singletonList(serverComponentHealth); diff --git a/datastream-server-restli/src/main/java/com/linkedin/datastream/server/diagnostics/ServerHealthResources.java b/datastream-server-restli/src/main/java/com/linkedin/datastream/server/diagnostics/ServerHealthResources.java index bd471db5a..b508da90f 100644 --- a/datastream-server-restli/src/main/java/com/linkedin/datastream/server/diagnostics/ServerHealthResources.java +++ b/datastream-server-restli/src/main/java/com/linkedin/datastream/server/diagnostics/ServerHealthResources.java @@ -13,6 +13,7 @@ import org.slf4j.LoggerFactory; import com.linkedin.datastream.common.Datastream; +import com.linkedin.datastream.common.DatastreamException; import com.linkedin.datastream.diagnostics.ConnectorHealth; import com.linkedin.datastream.diagnostics.ConnectorHealthArray; import com.linkedin.datastream.diagnostics.ServerHealth; @@ -21,6 +22,11 @@ import com.linkedin.datastream.server.Coordinator; import com.linkedin.datastream.server.DatastreamServer; import com.linkedin.datastream.server.DatastreamTask; +import com.linkedin.datastream.server.ErrorLogger; +import com.linkedin.datastream.server.dms.DatastreamSourceCheckpointResources; +import com.linkedin.datastream.server.providers.CustomCheckpointProvider; +import com.linkedin.datastream.server.providers.JDBCCheckpoint; +import com.linkedin.restli.common.HttpStatus; import com.linkedin.restli.server.annotations.RestLiSimpleResource; import com.linkedin.restli.server.resources.SimpleResourceTemplate; @@ -31,10 +37,12 @@ @RestLiSimpleResource(name = "health", namespace = "com.linkedin.datastream.server.diagnostics") public class ServerHealthResources extends SimpleResourceTemplate { - private static final Logger LOG = LoggerFactory.getLogger(ServerHealthResources.class); + private static final Logger _logger = LoggerFactory.getLogger(ServerHealthResources.class); private final DatastreamServer _server; private final Coordinator _coordinator; + private final DatastreamSourceCheckpointResources _checkpointResources; + private final ErrorLogger _errorLogger; /** * Construct an instance of ServerHealthResources @@ -43,13 +51,16 @@ public class ServerHealthResources extends SimpleResourceTemplate public ServerHealthResources(DatastreamServer datastreamServer) { _server = datastreamServer; _coordinator = datastreamServer.getCoordinator(); + _checkpointResources = new DatastreamSourceCheckpointResources(datastreamServer); + _errorLogger = new ErrorLogger(_logger, _coordinator.getInstanceName()); + } @Override public ServerHealth get() { - LOG.info("Get request for serverHealth"); + _logger.info("Get request for serverHealth"); ServerHealth health = buildServerHealth(); - LOG.info("Server Health: " + health.toString()); + _logger.info("Server Health: " + health.toString()); return health; } @@ -106,7 +117,30 @@ private TaskHealthArray buildTasksHealthForConnectorType(String connectorType) { taskHealth.setName(task.getDatastreamTaskName()); taskHealth.setPartitions(task.getPartitions().toString()); - taskHealth.setSourceCheckpoint(task.getCheckpoints().toString()); + if (_server.isCustomCheckpointing(connectorType) && connectorType.equals("jdbcConnector")) { + CustomCheckpointProvider checkpointProvider = null; + try { + checkpointProvider = _server.getCustomCheckpointProvider(task.getDatastreams().get(0)); + JDBCCheckpoint checkpoint = checkpointProvider.getSafeCheckpoint( + () -> new JDBCCheckpoint.Deserializer(), + JDBCCheckpoint.class + ); + + taskHealth.setSourceCheckpoint(checkpoint.toString()); + } catch (DatastreamException e) { + _errorLogger.logAndThrowRestLiServiceException(HttpStatus.S_400_BAD_REQUEST, + "Failed to get checkpoints." + task.getDatastreams().get(0), e); + } catch (Exception e) { + _errorLogger.logAndThrowRestLiServiceException(HttpStatus.S_500_INTERNAL_SERVER_ERROR, + "Failed to get checkpoints.", e); + } finally { + if (checkpointProvider != null) { + checkpointProvider.close(); + } + } + } else { + taskHealth.setSourceCheckpoint(task.getCheckpoints().toString()); + } allTasksHealth.add(taskHealth); }); diff --git a/datastream-server-restli/src/main/java/com/linkedin/datastream/server/dms/DatastreamResources.java b/datastream-server-restli/src/main/java/com/linkedin/datastream/server/dms/DatastreamResources.java index ad0362a42..412bd74f7 100644 --- a/datastream-server-restli/src/main/java/com/linkedin/datastream/server/dms/DatastreamResources.java +++ b/datastream-server-restli/src/main/java/com/linkedin/datastream/server/dms/DatastreamResources.java @@ -77,7 +77,7 @@ @RestLiCollection(name = "datastream", keyName = DatastreamResources.KEY_NAME, namespace = "com.linkedin.datastream.server.dms") public class DatastreamResources extends CollectionResourceTemplate { public static final String KEY_NAME = "datastreamId"; - private static final Logger LOG = LoggerFactory.getLogger(DatastreamResources.class); + private static final Logger _logger = LoggerFactory.getLogger(DatastreamResources.class); private static final String CLASS_NAME = DatastreamResources.class.getSimpleName(); private static final String UPDATE_CALL = "updateCall"; @@ -117,7 +117,7 @@ public DatastreamResources(DatastreamServer datastreamServer) { public DatastreamResources(DatastreamStore store, Coordinator coordinator) { _store = store; _coordinator = coordinator; - _errorLogger = new ErrorLogger(LOG, _coordinator.getInstanceName()); + _errorLogger = new ErrorLogger(_logger, _coordinator.getInstanceName()); _dynamicMetricsManager = DynamicMetricsManager.getInstance(); _dynamicMetricsManager.registerGauge(CLASS_NAME, CREATE_CALL_LATENCY_MS_STRING, CREATE_CALL_LATENCY_MS_SUPPLIER); @@ -136,10 +136,10 @@ public DatastreamResources(DatastreamStore store, Coordinator coordinator) { * */ private void doUpdateDatastreams(Map datastreamMap) { - LOG.info("Update datastream call with request: {}", datastreamMap); + _logger.info("Update datastream call with request: {}", datastreamMap); _dynamicMetricsManager.createOrUpdateMeter(CLASS_NAME, UPDATE_CALL, 1); if (datastreamMap.isEmpty()) { - LOG.warn("Update datastream call with empty input."); + _logger.warn("Update datastream call with empty input."); return; } @@ -288,7 +288,7 @@ public ActionResult pause(@PathKeysParam PathKeys pathKeys, String datastreamName = pathKeys.getAsString(KEY_NAME); Datastream datastream = _store.getDatastream(datastreamName); - LOG.info("Received request to pause datastream {}", datastream); + _logger.info("Received request to pause datastream {}", datastream); if (datastream == null) { _errorLogger.logAndThrowRestLiServiceException(HttpStatus.S_404_NOT_FOUND, @@ -302,14 +302,14 @@ public ActionResult pause(@PathKeysParam PathKeys pathKeys, List datastreamsToPause = force ? getGroupedDatastreams(datastream) : Collections.singletonList(datastream); - LOG.info("Pausing datastreams {}", datastreamsToPause); + _logger.info("Pausing datastreams {}", datastreamsToPause); for (Datastream d : datastreamsToPause) { try { if (DatastreamStatus.READY.equals(datastream.getStatus())) { d.setStatus(DatastreamStatus.PAUSED); _store.updateDatastream(d.getName(), d, true); } else { - LOG.warn("Cannot pause datastream {}, as it is not in READY state. State: {}", d, datastream.getStatus()); + _logger.warn("Cannot pause datastream {}, as it is not in READY state. State: {}", d, datastream.getStatus()); } } catch (DatastreamException e) { _errorLogger.logAndThrowRestLiServiceException(HttpStatus.S_500_INTERNAL_SERVER_ERROR, @@ -317,7 +317,7 @@ public ActionResult pause(@PathKeysParam PathKeys pathKeys, } } - LOG.info("Completed request for pausing datastream {}", datastream); + _logger.info("Completed request for pausing datastream {}", datastream); return new ActionResult<>(HttpStatus.S_200_OK); } @@ -339,7 +339,7 @@ public ActionResult movePartitions(@PathKeysParam PathKeys pathKeys, String datastreamName = pathKeys.getAsString(KEY_NAME); Datastream datastream = _store.getDatastream(datastreamName); - LOG.info("Received request to move datastream: {}, partitions: {}, targetHost: {}, notify: {}", + _logger.info("Received request to move datastream: {}, partitions: {}, targetHost: {}, notify: {}", datastream, partitions, targetHost, notify); if (datastream == null) { @@ -368,11 +368,11 @@ public ActionResult movePartitions(@PathKeysParam PathKeys pathKeys, try { _store.updatePartitionAssignments(datastream.getName(), datastream, targetAssignment, notify); } catch (Exception ex) { - LOG.error("Error to move partitions", ex); + _logger.error("Error to move partitions", ex); _errorLogger.logAndThrowRestLiServiceException(HttpStatus.S_500_INTERNAL_SERVER_ERROR, "ERROR"); } - LOG.info("Completed request for moving datastream {}", datastream); + _logger.info("Completed request for moving datastream {}", datastream); return new ActionResult<>(HttpStatus.S_200_OK); } @@ -389,7 +389,7 @@ public ActionResult stop(@PathKeysParam PathKeys pathKeys, String datastreamName = pathKeys.getAsString(KEY_NAME); Datastream datastream = _store.getDatastream(datastreamName); - LOG.info("Received request to stop datastream {}", datastream); + _logger.info("Received request to stop datastream {}", datastream); if (datastream == null) { _errorLogger.logAndThrowRestLiServiceException(HttpStatus.S_404_NOT_FOUND, @@ -403,14 +403,14 @@ public ActionResult stop(@PathKeysParam PathKeys pathKeys, List datastreamsToStop = force ? getGroupedDatastreams(datastream) : Collections.singletonList(datastream); - LOG.info("Stop datastreams {}", datastreamsToStop); + _logger.info("Stop datastreams {}", datastreamsToStop); for (Datastream d : datastreamsToStop) { try { if (DatastreamStatus.READY.equals(datastream.getStatus()) || DatastreamStatus.PAUSED.equals(datastream.getStatus())) { d.setStatus(DatastreamStatus.STOPPED); _store.updateDatastream(d.getName(), d, true); } else { - LOG.warn("Cannot stop datastream {}, as it is not in READY/PAUSED state. State: {}", d, datastream.getStatus()); + _logger.warn("Cannot stop datastream {}, as it is not in READY/PAUSED state. State: {}", d, datastream.getStatus()); } } catch (DatastreamException e) { _errorLogger.logAndThrowRestLiServiceException(HttpStatus.S_500_INTERNAL_SERVER_ERROR, @@ -418,7 +418,7 @@ public ActionResult stop(@PathKeysParam PathKeys pathKeys, } } - LOG.info("Completed request for stopping datastream {}", datastream); + _logger.info("Completed request for stopping datastream {}", datastream); return new ActionResult<>(HttpStatus.S_200_OK); } @@ -435,7 +435,7 @@ public ActionResult resume(@PathKeysParam PathKeys pathKeys, String datastreamName = pathKeys.getAsString(KEY_NAME); Datastream datastream = _store.getDatastream(datastreamName); - LOG.info("Received request to resume datastream {}", datastream); + _logger.info("Received request to resume datastream {}", datastream); if (datastream == null) { _errorLogger.logAndThrowRestLiServiceException(HttpStatus.S_404_NOT_FOUND, @@ -450,7 +450,7 @@ public ActionResult resume(@PathKeysParam PathKeys pathKeys, List datastreamsToResume = force ? getGroupedDatastreams(datastream) : Collections.singletonList(datastream); - LOG.info("Resuming datastreams {}", datastreamsToResume); + _logger.info("Resuming datastreams {}", datastreamsToResume); for (Datastream d : datastreamsToResume) { try { if (DatastreamStatus.PAUSED.equals(datastream.getStatus()) || @@ -458,7 +458,7 @@ public ActionResult resume(@PathKeysParam PathKeys pathKeys, d.setStatus(DatastreamStatus.READY); _store.updateDatastream(d.getName(), d, true); } else { - LOG.warn("Will not resume datastream {}, as it is not already in PAUSED/STOPPED state", d); + _logger.warn("Will not resume datastream {}, as it is not already in PAUSED/STOPPED state", d); } } catch (DatastreamException e) { _errorLogger.logAndThrowRestLiServiceException(HttpStatus.S_500_INTERNAL_SERVER_ERROR, @@ -466,7 +466,7 @@ public ActionResult resume(@PathKeysParam PathKeys pathKeys, } } - LOG.info("Completed request for resuming datastream {}", datastream); + _logger.info("Completed request for resuming datastream {}", datastream); return new ActionResult<>(HttpStatus.S_200_OK); } @@ -483,7 +483,7 @@ public ActionResult pauseSourcePartitions(@PathKeysParam PathKeys pathKeys // Get datastream. String datastreamName = pathKeys.getAsString(KEY_NAME); // Log for debugging purposes. - LOG.info("pauseSourcePartitions called for datastream: {}, with partitions: {}", datastreamName, sourcePartitions); + _logger.info("pauseSourcePartitions called for datastream: {}, with partitions: {}", datastreamName, sourcePartitions); Datastream datastream = _store.getDatastream(datastreamName); if (datastream == null) { @@ -544,7 +544,7 @@ public ActionResult pauseSourcePartitions(@PathKeysParam PathKeys pathKeys "Could not update datastream's paused partitions: " + datastream.getName(), e); } - LOG.info("Completed request to pause datastream: {}, source partitions: {}", datastreamName, sourcePartitions); + _logger.info("Completed request to pause datastream: {}, source partitions: {}", datastreamName, sourcePartitions); return new ActionResult<>(HttpStatus.S_200_OK); } @@ -561,7 +561,7 @@ public ActionResult resumeSourcePartitions(@PathKeysParam PathKeys pathKey // Get datastream. String datastreamName = pathKeys.getAsString(KEY_NAME); // Log for debugging purposes. - LOG.info("resumeSourcePartitions called for datastream: {}, with partitions: {}", datastreamName, + _logger.info("resumeSourcePartitions called for datastream: {}, with partitions: {}", datastreamName, sourcePartitions); Datastream datastream = _store.getDatastream(datastreamName); @@ -635,7 +635,7 @@ public ActionResult resumeSourcePartitions(@PathKeysParam PathKeys pathKey "Could not update datastream's paused partitions: " + datastream.getName(), e); } - LOG.info("Completed request to resume datastream: {}, source partitions: {}", datastreamName, sourcePartitions); + _logger.info("Completed request to resume datastream: {}, source partitions: {}", datastreamName, sourcePartitions); return new ActionResult<>(HttpStatus.S_200_OK); } @@ -647,7 +647,7 @@ public UpdateResponse delete(String datastreamName) { } try { - LOG.info("Delete datastream called for datastream " + datastreamName); + _logger.info("Delete datastream called for datastream " + datastreamName); _dynamicMetricsManager.createOrUpdateMeter(CLASS_NAME, DELETE_CALL, 1); Instant startTime = Instant.now(); @@ -667,11 +667,11 @@ public UpdateResponse delete(String datastreamName) { @Override public Datastream get(String name) { try { - LOG.info("Get datastream called for datastream {}", name); + _logger.info("Get datastream called for datastream {}", name); _dynamicMetricsManager.createOrUpdateMeter(CLASS_NAME, GET_CALL, 1); Instant startTime = Instant.now(); Datastream stream = _store.getDatastream(name); - LOG.info("Get datastream call took {} ms", Duration.between(startTime, Instant.now()).toMillis()); + _logger.info("Get datastream call took {} ms", Duration.between(startTime, Instant.now()).toMillis()); return stream; } catch (Exception e) { _dynamicMetricsManager.createOrUpdateMeter(CLASS_NAME, CALL_ERROR, 1); @@ -687,15 +687,15 @@ public Datastream get(String name) { @Override public List getAll(@Context PagingContext pagingContext) { try { - LOG.info("Get all datastreams called with paging context {}", pagingContext); + _logger.info("Get all datastreams called with paging context {}", pagingContext); _dynamicMetricsManager.createOrUpdateMeter(CLASS_NAME, GET_ALL_CALL, 1); Instant startTime = Instant.now(); List ret = RestliUtils.withPaging(_store.getAllDatastreams(), pagingContext) .map(_store::getDatastream) .filter(Objects::nonNull) .collect(Collectors.toList()); - LOG.info("Get all datastreams call took {} ms", Duration.between(startTime, Instant.now()).toMillis()); - LOG.debug("Result collected for getAll {}", ret); + _logger.info("Get all datastreams call took {} ms", Duration.between(startTime, Instant.now()).toMillis()); + _logger.debug("Result collected for getAll {}", ret); return ret; } catch (Exception e) { _dynamicMetricsManager.createOrUpdateMeter(CLASS_NAME, CALL_ERROR, 1); @@ -715,7 +715,7 @@ public List getAll(@Context PagingContext pagingContext) { public List findGroup(@Context PagingContext pagingContext, @QueryParam("datastreamName") String datastreamName) { try { - LOG.info("findDuplicates called with paging context {}", pagingContext); + _logger.info("findDuplicates called with paging context {}", pagingContext); _dynamicMetricsManager.createOrUpdateMeter(CLASS_NAME, FINDER_CALL, 1); Datastream datastream = _store.getDatastream(datastreamName); if (datastream == null) { @@ -725,7 +725,7 @@ public List findGroup(@Context PagingContext pagingContext, List ret = RestliUtils.withPaging(getGroupedDatastreams(datastream).stream(), pagingContext) .filter(Objects::nonNull) .collect(Collectors.toList()); - LOG.debug("Result collected for findDuplicates: {}", ret); + _logger.debug("Result collected for findDuplicates: {}", ret); return ret; } catch (Exception e) { _dynamicMetricsManager.createOrUpdateMeter(CLASS_NAME, CALL_ERROR, 1); @@ -738,9 +738,9 @@ public List findGroup(@Context PagingContext pagingContext, @Override public CreateResponse create(Datastream datastream) { try { - LOG.info("Create datastream called with datastream {}", datastream); - if (LOG.isDebugEnabled()) { - LOG.debug("Handling request on object: {} thread: {}", this, Thread.currentThread()); + _logger.info("Create datastream called with datastream {}", datastream); + if (_logger.isDebugEnabled()) { + _logger.debug("Handling request on object: {} thread: {}", this, Thread.currentThread()); } _dynamicMetricsManager.createOrUpdateMeter(CLASS_NAME, CREATE_CALL, 1); @@ -763,7 +763,7 @@ public CreateResponse create(Datastream datastream) { Instant startTime = Instant.now(); - LOG.debug("Sanity check is finished, initializing datastream"); + _logger.debug("Sanity check is finished, initializing datastream"); // Before the initializeDatastream (which could be heavy depends on the types of datastreams), // quickly check whether the datastream has already existed. @@ -773,14 +773,14 @@ public CreateResponse create(Datastream datastream) { _coordinator.initializeDatastream(datastream); - LOG.debug("Persisting initialized datastream to zookeeper: {}", datastream); + _logger.debug("Persisting initialized datastream to zookeeper: {}", datastream); _store.createDatastream(datastream.getName(), datastream); Duration delta = Duration.between(startTime, Instant.now()); CREATE_CALL_LATENCY_MS.set(delta.toMillis()); - LOG.info("Datastream persisted to zookeeper, total time used: {} ms", delta.toMillis()); + _logger.info("Datastream persisted to zookeeper, total time used: {} ms", delta.toMillis()); return new CreateResponse(datastream.getName(), HttpStatus.S_201_CREATED); } catch (IllegalArgumentException e) { _dynamicMetricsManager.createOrUpdateMeter(CLASS_NAME, CALL_ERROR, 1); diff --git a/datastream-server-restli/src/main/java/com/linkedin/datastream/server/dms/DatastreamSourceCheckpointResources.java b/datastream-server-restli/src/main/java/com/linkedin/datastream/server/dms/DatastreamSourceCheckpointResources.java new file mode 100644 index 000000000..8e3f07e46 --- /dev/null +++ b/datastream-server-restli/src/main/java/com/linkedin/datastream/server/dms/DatastreamSourceCheckpointResources.java @@ -0,0 +1,167 @@ +/** + * Copyright 2021 LinkedIn Corporation. All rights reserved. + * Licensed under the BSD 2-Clause License. See the LICENSE file in the project root for license information. + * See the NOTICE file in the project root for additional information regarding copyright ownership. + */ +package com.linkedin.datastream.server.dms; + + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.linkedin.datastream.common.Datastream; +import com.linkedin.datastream.common.DatastreamStatus; +import com.linkedin.datastream.diagnostics.ConnectorHealth; +import com.linkedin.datastream.diagnostics.TaskHealth; +import com.linkedin.datastream.diagnostics.TaskHealthArray; +import com.linkedin.datastream.server.Coordinator; +import com.linkedin.datastream.server.DatastreamServer; +import com.linkedin.datastream.server.ErrorLogger; +import com.linkedin.datastream.server.providers.CustomCheckpointProvider; +import com.linkedin.datastream.server.providers.JDBCCheckpoint; +import com.linkedin.restli.common.HttpStatus; +import com.linkedin.restli.server.UpdateResponse; +import com.linkedin.restli.server.annotations.RestLiSimpleResource; +import com.linkedin.restli.server.resources.SimpleResourceTemplate; + +/** + * The Restli resource of {@link ConnectorHealth}. Used for collecting datastream checkpoint info + * about a {@link Datastream}. + */ +@RestLiSimpleResource(name = "checkpoint", namespace = "com.linkedin.datastream.server.dms", parent = DatastreamResources.class) +public class DatastreamSourceCheckpointResources extends SimpleResourceTemplate { + private static final Logger _logger = LoggerFactory.getLogger(DatastreamSourceCheckpointResources.class); + private static final String CONFIG_CHECKPOINT_STORE_URL = "checkpointStoreURL"; + private static final String CONFIG_CHECKPOINT_STORE_TOPIC = "checkpointStoreTopic"; + public final DatastreamServer _server; + private final DatastreamStore _store; + private final Coordinator _coordinator; + private final ErrorLogger _errorLogger; + + + /** + * Construct an instance of DatastreamCheckpointResources + * @param datastreamServer Datastream server for which health data is retrieved + */ + public DatastreamSourceCheckpointResources(DatastreamServer datastreamServer) { + _server = datastreamServer; + _store = _server.getDatastreamStore(); + _coordinator = _server.getCoordinator(); + _errorLogger = new ErrorLogger(_logger, _coordinator.getInstanceName()); + } + + /** + * Get source checkpoints of the parent datastream for each tasks + * @return current checkpoint + */ + @Override + public ConnectorHealth get() { + String datastreamName = getParentKey(); + try { + Datastream datastream = _store.getDatastream(datastreamName); + if (datastream == null) { + _errorLogger.logAndThrowRestLiServiceException(HttpStatus.S_404_NOT_FOUND, + "Datastream does not exist: " + datastreamName); + + } + return buildSourceCheckpoint(datastream); + } catch (Exception e) { + _errorLogger.logAndThrowRestLiServiceException(HttpStatus.S_500_INTERNAL_SERVER_ERROR, + "Get datastream checkpoint failed for datastream: " + datastreamName, e); + } + // Returning null will automatically trigger a 404 Not Found response + return null; + } + + /** + * Update source checkpoint on a given datastream + * @param input desired checkpoint value + * @return + */ + @Override + public UpdateResponse update(ConnectorHealth input) { + String datastreamName = getParentKey(); + Datastream datastream = _store.getDatastream(datastreamName); + if (datastream == null) { + _errorLogger.logAndThrowRestLiServiceException(HttpStatus.S_404_NOT_FOUND, + "Datastream does not exist: " + datastreamName); + } + // Currently only KafkaCustomCheckpointProvider supports rewinding the checkpoint + // Todo: only JDBC checkpoint is updatable + if (!_server.isCustomCheckpointing(datastream.getConnectorName()) || + !datastream.getConnectorName().equals("jdbcConnector")) { + _errorLogger.logAndThrowRestLiServiceException(HttpStatus.S_501_NOT_IMPLEMENTED, + "This datastream doesn't support checkpoints to be updated."); + } + // Only paused datastreams can be updated + if (!DatastreamStatus.PAUSED.equals(datastream.getStatus())) { + _errorLogger.logAndThrowRestLiServiceException(HttpStatus.S_405_METHOD_NOT_ALLOWED, + "Datastream must be in PAUSED state before updating checkpoint"); + } + CustomCheckpointProvider customCheckpointProvider = null; + try { + // Sample SourceCheckpoint resource {"tasks":["sourceCheckpoint":"1234"]} + final long newCheckpointVal = Long.parseLong(input.getTasks().get(0).getSourceCheckpoint()); + JDBCCheckpoint newCheckpoint = new JDBCCheckpoint(newCheckpointVal); + customCheckpointProvider = _server.getCustomCheckpointProvider(datastream); + customCheckpointProvider.commit(newCheckpoint); + } catch (Exception e) { + _errorLogger.logAndThrowRestLiServiceException(HttpStatus.S_400_BAD_REQUEST, + "Could not complete datastream checkpoint update.", e); + } finally { + if (customCheckpointProvider != null) { + customCheckpointProvider.close(); + } + } + return new UpdateResponse(HttpStatus.S_200_OK); + } + + private ConnectorHealth buildSourceCheckpoint(Datastream datastream) { + ConnectorHealth sourceCheckpointDetail = new ConnectorHealth(); + sourceCheckpointDetail.setTasks(buildTaskSourceCheckpoint(datastream)); + + return sourceCheckpointDetail; + } + + private TaskHealthArray buildTaskSourceCheckpoint(Datastream datastream) { + TaskHealthArray tasksSourceCheckpoint = new TaskHealthArray(); + _coordinator.getDatastreamTasks().stream() + .filter(t -> t.getDatastreams().get(0).getName().equals(datastream.getName())).forEach(task -> { + TaskHealth taskHealth = new TaskHealth(); + taskHealth.setName(task.getDatastreamTaskName()); + // Todo: only returns JDBC checkpoint. Will change in future + if (_server.isCustomCheckpointing(datastream.getConnectorName())) { + if (datastream.getConnectorName().equals("jdbcConnector")) { // todo: hardcode for now + CustomCheckpointProvider checkpointProvider = null; + try { + checkpointProvider = _server.getCustomCheckpointProvider(task.getDatastreams().get(0)); + JDBCCheckpoint checkpoint = checkpointProvider.getSafeCheckpoint( + () -> new JDBCCheckpoint.Deserializer(), + JDBCCheckpoint.class + ); + taskHealth.setSourceCheckpoint(checkpoint.toString()); + } catch (Exception e) { + e.printStackTrace(); + } finally { + if (checkpointProvider != null) { + checkpointProvider.close(); + } + } + } else { + taskHealth.setSourceCheckpoint("Accessing checkpoint of the connector is not supported yet"); + } + } else { + taskHealth.setSourceCheckpoint(task.getCheckpoints().toString()); + } + tasksSourceCheckpoint.add(taskHealth); + }); + + return tasksSourceCheckpoint; + } + + + private String getParentKey() { + return getContext().getPathKeys().getAsString(DatastreamResources.KEY_NAME); + } + +} diff --git a/datastream-server-restli/src/main/java/com/linkedin/datastream/server/dms/ZookeeperBackedDatastreamStore.java b/datastream-server-restli/src/main/java/com/linkedin/datastream/server/dms/ZookeeperBackedDatastreamStore.java index 0ebdc777a..358977a0f 100644 --- a/datastream-server-restli/src/main/java/com/linkedin/datastream/server/dms/ZookeeperBackedDatastreamStore.java +++ b/datastream-server-restli/src/main/java/com/linkedin/datastream/server/dms/ZookeeperBackedDatastreamStore.java @@ -34,7 +34,7 @@ */ public class ZookeeperBackedDatastreamStore implements DatastreamStore { - private static final Logger LOG = LoggerFactory.getLogger(ZookeeperBackedDatastreamStore.class.getName()); + private static final Logger _logger = LoggerFactory.getLogger(ZookeeperBackedDatastreamStore.class.getName()); private final ZkClient _zkClient; private final String _cluster; @@ -113,7 +113,7 @@ public void createDatastream(String key, Datastream datastream) { if (_zkClient.exists(path)) { String content = _zkClient.ensureReadData(path); String errorMessage = String.format("Datastream already exists: path=%s, content=%s", key, content); - LOG.warn(errorMessage); + _logger.warn(errorMessage); throw new DatastreamAlreadyExistsException(errorMessage); } _zkClient.ensurePath(path); @@ -151,7 +151,7 @@ public void updatePartitionAssignments(String key, Datastream datastream, HostTa _zkClient.writeData(KeyBuilder.getTargetAssignmentBase(_cluster, datastream.getConnectorName()), String.valueOf(System.currentTimeMillis())); } catch (Exception e) { - LOG.warn("Failed to touch the assignment update", e); + _logger.warn("Failed to touch the assignment update", e); throw new DatastreamException(e); } } @@ -167,17 +167,17 @@ private void verifyHostname(String hostname) throws DatastreamException { try { return ZkAdapter.parseHostnameFromZkInstance(s); } catch (Exception ex) { - LOG.error("Fails to parse instance: " + s, ex); + _logger.error("Fails to parse instance: " + s, ex); return null; } }).filter(Objects::nonNull).collect(Collectors.toSet()); if (!hostnames.contains(hostname)) { String msg = "Hostname " + hostname + " is not valid"; - LOG.error(msg); + _logger.error(msg); throw new DatastreamException(msg); } } catch (Exception ex) { - LOG.error("Fail to verify the hostname", ex); + _logger.error("Fail to verify the hostname", ex); throw new DatastreamException(ex); } diff --git a/datastream-server-restli/src/test/java/com/linkedin/datastream/server/TestCoordinator.java b/datastream-server-restli/src/test/java/com/linkedin/datastream/server/TestCoordinator.java index 585395b94..063d3aedd 100644 --- a/datastream-server-restli/src/test/java/com/linkedin/datastream/server/TestCoordinator.java +++ b/datastream-server-restli/src/test/java/com/linkedin/datastream/server/TestCoordinator.java @@ -92,7 +92,7 @@ * Tests for {@link Coordinator} */ public class TestCoordinator { - private static final Logger LOG = LoggerFactory.getLogger(TestCoordinator.class); + private static final Logger _logger = LoggerFactory.getLogger(TestCoordinator.class); private static final long WAIT_DURATION_FOR_ZK = Duration.ofMinutes(1).toMillis(); private static final int WAIT_TIMEOUT_MS = 60000; @@ -761,7 +761,7 @@ private Map> collectDatastreamPartitions(List { String datastream = task.getDatastreams().get(0).getName(); datastreamMap.putIfAbsent(datastream, new ArrayList<>()); - LOG.info("{}", task); + _logger.info("{}", task); datastreamMap.get(datastream).addAll(task.getPartitionsV2()); }); } @@ -980,7 +980,7 @@ public void testValidatePartitionAssignmentSupported() throws Exception { coordinator.validatePartitionAssignmentSupported(datastream2[0]); Assert.fail("Should fail validation when partition assignment is not supported"); } catch (DatastreamValidationException e) { - LOG.info("Caught exception as partition assignment is not supported"); + _logger.info("Caught exception as partition assignment is not supported"); } } @@ -1022,7 +1022,7 @@ public void testValidateDatastreamsUpdate() throws Exception { resource.batchUpdate(batchRequest); Assert.fail("Should fail validation when there are multiple connector types"); } catch (RestLiServiceException e) { - LOG.info("Caught exception as expected on update of multiple connector types in one batch"); + _logger.info("Caught exception as expected on update of multiple connector types in one batch"); } connector1._allowDatastreamUpdate = false; @@ -1031,7 +1031,7 @@ public void testValidateDatastreamsUpdate() throws Exception { coordinator.validateDatastreamsUpdate(Arrays.asList(datastreams.get(0), datastreams.get(1))); Assert.fail("Should fail validation when update is not allowed"); } catch (DatastreamValidationException e) { - LOG.info("Caught exception as expected on update of connector type that doesn't support update"); + _logger.info("Caught exception as expected on update of connector type that doesn't support update"); } coordinator.validateDatastreamsUpdate(Arrays.asList(datastreams.get(2), datastreams.get(3))); @@ -1061,7 +1061,7 @@ public void testCoordinatorHandleUpdateDatastream() throws Exception { Datastream[] list = DatastreamTestUtils.createAndStoreDatastreams(zkClient, testCluster, connectorType, "datastream1"); Datastream datastream = list[0]; - LOG.info("Created datastream: {}", datastream); + _logger.info("Created datastream: {}", datastream); // wait for datastream to be READY PollUtils.poll(() -> DatastreamTestUtils.getDatastream(zkClient, testCluster, "datastream1") @@ -1075,7 +1075,7 @@ public void testCoordinatorHandleUpdateDatastream() throws Exception { datastream.getMetadata().put("key", "value"); datastream.getSource().setConnectionString("newSource"); - LOG.info("Updating datastream: {}", datastream); + _logger.info("Updating datastream: {}", datastream); CachedDatastreamReader datastreamCache = new CachedDatastreamReader(zkClient, testCluster); ZookeeperBackedDatastreamStore dsStore = new ZookeeperBackedDatastreamStore(datastreamCache, zkClient, testCluster); DatastreamResources datastreamResources = new DatastreamResources(dsStore, coordinator1); @@ -1092,7 +1092,7 @@ public void testCoordinationMultipleConnectorTypesForBroadcastStrategy() throws String connectorType1 = "connectorType1"; String connectorType2 = "connectorType2"; - LOG.info("create two coordinators and two connector instances per coordinator of broadcast strategy"); + _logger.info("create two coordinators and two connector instances per coordinator of broadcast strategy"); // // create two live instances, each handle two different types of connectors @@ -1119,14 +1119,14 @@ public void testCoordinationMultipleConnectorTypesForBroadcastStrategy() throws ZkClient zkClient = new ZkClient(_zkConnectionString); - LOG.info("Create a datastream of connectorType1"); + _logger.info("Create a datastream of connectorType1"); // // create a new datastream for connectorType1 // DatastreamTestUtils.createAndStoreDatastreams(zkClient, testCluster, connectorType1, "datastream1"); - LOG.info("Verify whether the datastream is assigned to connector instances on both the coordinator"); + _logger.info("Verify whether the datastream is assigned to connector instances on both the coordinator"); // // verify both live instances have tasks assigned for connector type 1 only @@ -1137,14 +1137,14 @@ public void testCoordinationMultipleConnectorTypesForBroadcastStrategy() throws assertConnectorAssignment(connector21, WAIT_TIMEOUT_MS, "datastream1"); Assert.assertTrue(connector22.getTasks().isEmpty()); - LOG.info("Create a datastream of connectorType2"); + _logger.info("Create a datastream of connectorType2"); // // create a new datastream for connectorType2 // DatastreamTestUtils.createAndStoreDatastreams(zkClient, testCluster, connectorType2, "datastream2"); - LOG.info("Verify the assignment"); + _logger.info("Verify the assignment"); // // verify both live instances have tasks assigned for both connector types @@ -1184,7 +1184,7 @@ public void testStressLargeNumberOfLiveInstances() throws Exception { Thread.sleep(duration); instance.stop(); } catch (Exception ex) { - LOG.error("Failed to launch coordinator", ex); + _logger.error("Failed to launch coordinator", ex); Assert.fail(); } }; @@ -1204,7 +1204,7 @@ public void testStressLargeNumberOfLiveInstances() throws Exception { List sortedList = new ArrayList<>(instances); Collections.sort(sortedList); - LOG.info("Live instances {}", sortedList); + _logger.info("Live instances {}", sortedList); Assert.assertEquals(instances.size(), concurrencyLevel, String.format("Live instances %s", sortedList)); zkClient.close(); @@ -1220,7 +1220,7 @@ public void testStressLargeNumberOfDatastreams() throws Exception { String datastreamName = "datastream"; ZkClient zkClient = new ZkClient(_zkConnectionString); - LOG.info("Create two coordinator and two connector instances of broadcast strategy"); + _logger.info("Create two coordinator and two connector instances of broadcast strategy"); // // create 1 live instance and start it @@ -1240,7 +1240,7 @@ public void testStressLargeNumberOfDatastreams() throws Exception { String[] datastreamNames = new String[concurrencyLevel]; - LOG.info("Create 10 datastreams"); + _logger.info("Create 10 datastreams"); // // create large number of datastreams @@ -1250,7 +1250,7 @@ public void testStressLargeNumberOfDatastreams() throws Exception { datastreamNames[i] = datastreamName + i; } - LOG.info("validate whether all the datastreams are assigned to all the instances"); + _logger.info("validate whether all the datastreams are assigned to all the instances"); assertConnectorAssignment(connector1, WAIT_TIMEOUT_MS, datastreamNames); assertConnectorAssignment(connector2, WAIT_TIMEOUT_MS, datastreamNames); @@ -1270,7 +1270,7 @@ public void testSimpleAssignmentReassignWithNewInstances() throws Exception { String testConnectorType = "testConnectorType"; ZkClient zkClient = new ZkClient(_zkConnectionString); - LOG.info("Creating the first coordinator and connector instance"); + _logger.info("Creating the first coordinator and connector instance"); // // create 1 instance // @@ -1280,7 +1280,7 @@ public void testSimpleAssignmentReassignWithNewInstances() throws Exception { null); instance1.start(); - LOG.info("Creating two datastream"); + _logger.info("Creating two datastream"); // // create 2 datastreams, [datastream0, datastream1] @@ -1293,7 +1293,7 @@ public void testSimpleAssignmentReassignWithNewInstances() throws Exception { // assertConnectorAssignment(connector1, WAIT_TIMEOUT_MS, "datastream0", "datastream1"); - LOG.info("Creating the second coordinator and connector instance"); + _logger.info("Creating the second coordinator and connector instance"); // // add a new live instance instance2 @@ -1310,7 +1310,7 @@ public void testSimpleAssignmentReassignWithNewInstances() throws Exception { assertConnectorAssignment(connector1, WAIT_TIMEOUT_MS, "datastream0"); assertConnectorAssignment(connector2, WAIT_TIMEOUT_MS, "datastream1"); - LOG.info("Creating the third coordinator and connector instance"); + _logger.info("Creating the third coordinator and connector instance"); // // add instance3 @@ -1388,7 +1388,7 @@ public void testSimpleAssignmentReassignAfterDeath() throws Exception { String datastreamName = "datastream"; ZkClient zkClient = new ZkClient(_zkConnectionString); - LOG.info("Create two coordinators and two connector instances"); + _logger.info("Create two coordinators and two connector instances"); // // setup a cluster with 2 live instances with simple assignment strategy @@ -1406,7 +1406,7 @@ public void testSimpleAssignmentReassignAfterDeath() throws Exception { null); instance2.start(); - LOG.info("Create four datastreams"); + _logger.info("Create four datastreams"); // // create 4 datastreams, [datastream0, datastream1, datastream2, datastream3] @@ -1415,7 +1415,7 @@ public void testSimpleAssignmentReassignAfterDeath() throws Exception { DatastreamTestUtils.createAndStoreDatastreams(zkClient, testCluster, testConnectorType, datastreamName + i); } - LOG.info("Verify that the datastreams are assigned across two connectors"); + _logger.info("Verify that the datastreams are assigned across two connectors"); waitTillAssignmentIsComplete(4, WAIT_TIMEOUT_MS, connector1, connector2); // @@ -1424,14 +1424,14 @@ public void testSimpleAssignmentReassignAfterDeath() throws Exception { assertConnectorAssignment(connector1, WAIT_DURATION_FOR_ZK, "datastream0", "datastream2"); assertConnectorAssignment(connector2, WAIT_DURATION_FOR_ZK, "datastream1", "datastream3"); - LOG.info("Tasks assigned to instance1: " + connector1.getTasks().toString()); - LOG.info("Tasks assigned to instance2: " + connector2.getTasks().toString()); + _logger.info("Tasks assigned to instance1: " + connector1.getTasks().toString()); + _logger.info("Tasks assigned to instance2: " + connector2.getTasks().toString()); List tasks1 = new ArrayList<>(connector1.getTasks()); tasks1.addAll(connector2.getTasks()); tasks1.sort(Comparator.comparing(DatastreamTask::getDatastreamTaskName)); - LOG.info("Take the instance2 offline"); + _logger.info("Take the instance2 offline"); // // take instance2 offline @@ -1439,7 +1439,7 @@ public void testSimpleAssignmentReassignAfterDeath() throws Exception { instance2.stop(); deleteLiveInstanceNode(zkClient, testCluster, instance2); - LOG.info("verify that the four datastreams are assigned to the instance1"); + _logger.info("verify that the four datastreams are assigned to the instance1"); // // verify all 4 datastreams are assigned to instance1 @@ -1450,8 +1450,8 @@ public void testSimpleAssignmentReassignAfterDeath() throws Exception { List tasks2 = new ArrayList<>(connector1.getTasks()); tasks2.sort(Comparator.comparing(DatastreamTask::getDatastreamTaskName)); - LOG.info("Tasks1: " + tasks1.toString()); - LOG.info("Tasks2: " + tasks2.toString()); + _logger.info("Tasks1: " + tasks1.toString()); + _logger.info("Tasks2: " + tasks2.toString()); Assert.assertEquals(tasks1, tasks2); @@ -1469,7 +1469,7 @@ public void testBroadcastAssignmentReassignAfterDeath() throws Exception { String datastreamName = "datastream"; ZkClient zkClient = new ZkClient(_zkConnectionString); - LOG.info("Creating two coordinator and two connector instances"); + _logger.info("Creating two coordinator and two connector instances"); // // setup a cluster with 2 live instances with simple assignment strategy @@ -1487,7 +1487,7 @@ public void testBroadcastAssignmentReassignAfterDeath() throws Exception { new SourceBasedDeduper(), null); instance2.start(); - LOG.info("Create two datastreams"); + _logger.info("Create two datastreams"); // // create 2 datastreams, [datastream0, datastream1] @@ -1496,7 +1496,7 @@ public void testBroadcastAssignmentReassignAfterDeath() throws Exception { DatastreamTestUtils.createAndStoreDatastreams(zkClient, testCluster, testConnectorType, datastreamName + i); } - LOG.info("Validate the broadcast assignment"); + _logger.info("Validate the broadcast assignment"); // // verify assignment, instance1: [datastream0, datastream1], instance2:[datastream0, datastream1] @@ -1506,7 +1506,7 @@ public void testBroadcastAssignmentReassignAfterDeath() throws Exception { List tasks2 = new ArrayList<>(connector2.getTasks()); - LOG.info("Take the instance2 offline"); + _logger.info("Take the instance2 offline"); // // take instance2 offline @@ -1514,13 +1514,13 @@ public void testBroadcastAssignmentReassignAfterDeath() throws Exception { instance2.stop(); deleteLiveInstanceNode(zkClient, testCluster, instance2); - LOG.info("Verify whether the live instance assignment has been removed."); + _logger.info("Verify whether the live instance assignment has been removed."); // Verify dead instance assignments have been removed String instancePath = KeyBuilder.instanceAssignments(testCluster, instance2.getInstanceName()); Assert.assertTrue(PollUtils.poll(() -> !zkClient.exists(instancePath), 200, WAIT_TIMEOUT_MS)); - LOG.info("Verify instance1 still has two datastreams"); + _logger.info("Verify instance1 still has two datastreams"); // // verify instance1 still has 2 datastreams @@ -1530,7 +1530,7 @@ public void testBroadcastAssignmentReassignAfterDeath() throws Exception { // Make sure Coordinator has removed deprecated connector tasks of instance2 for (DatastreamTask task : tasks2) { String path = KeyBuilder.connectorTask(testCluster, task.getConnectorType(), task.getDatastreamTaskName()); - LOG.info("Checking whether the path doesn't exist anymore: " + path); + _logger.info("Checking whether the path doesn't exist anymore: " + path); Assert.assertTrue(PollUtils.poll(() -> !zkClient.exists(path), 200, WAIT_TIMEOUT_MS)); } @@ -1552,7 +1552,7 @@ public void testSimpleAssignmentReassignAfterLeaderDeath() throws Exception { String datastreamName = "datastream"; ZkClient zkClient = new ZkClient(_zkConnectionString); - LOG.info("Creating three coordinator and connector instances "); + _logger.info("Creating three coordinator and connector instances "); // // setup a cluster with 3 live instances with simple assignment strategy // @@ -1574,7 +1574,7 @@ public void testSimpleAssignmentReassignAfterLeaderDeath() throws Exception { null); instance3.start(); - LOG.info("Creating six datastreams"); + _logger.info("Creating six datastreams"); // // create 6 datastreams, [datastream0, ..., datastream5] // @@ -1582,7 +1582,7 @@ public void testSimpleAssignmentReassignAfterLeaderDeath() throws Exception { DatastreamTestUtils.createAndStoreDatastreams(zkClient, testCluster, testConnectorType, datastreamName + i); } - LOG.info("Verify whether the six datastreams are assigned to the three connector instances"); + _logger.info("Verify whether the six datastreams are assigned to the three connector instances"); // // verify assignment, instance1: [datastream0, datastream2], instance2:[datastream1, datastream3] @@ -1596,7 +1596,7 @@ public void testSimpleAssignmentReassignAfterLeaderDeath() throws Exception { tasks1.addAll(connector3.getTasks()); tasks1.sort(Comparator.comparing(DatastreamTask::getDatastreamTaskName)); - LOG.info("Stop the instance1 and delete the live instance"); + _logger.info("Stop the instance1 and delete the live instance"); // // take current leader instance1 offline @@ -1604,7 +1604,7 @@ public void testSimpleAssignmentReassignAfterLeaderDeath() throws Exception { instance1.stop(); deleteLiveInstanceNode(zkClient, testCluster, instance1); - LOG.info("Verify that the 6 datastreams are assigned to the remaining two instances"); + _logger.info("Verify that the 6 datastreams are assigned to the remaining two instances"); // // verify all 6 datastreams are assigned to instance2 and instance3 @@ -1612,7 +1612,7 @@ public void testSimpleAssignmentReassignAfterLeaderDeath() throws Exception { assertConnectorAssignment(connector2, WAIT_TIMEOUT_MS, "datastream0", "datastream2", "datastream4"); assertConnectorAssignment(connector3, WAIT_TIMEOUT_MS, "datastream1", "datastream3", "datastream5"); - LOG.info("Stop the instance2 and delete the live instance"); + _logger.info("Stop the instance2 and delete the live instance"); // // take current leader instance2 offline @@ -1620,21 +1620,21 @@ public void testSimpleAssignmentReassignAfterLeaderDeath() throws Exception { instance2.stop(); deleteLiveInstanceNode(zkClient, testCluster, instance2); - LOG.info("Verify that the 6 datastreams are assigned to remaining one instance"); + _logger.info("Verify that the 6 datastreams are assigned to remaining one instance"); // // verify all tasks assigned to instance3 assertConnectorAssignment(connector3, WAIT_TIMEOUT_MS, "datastream0", "datastream2", "datastream4", "datastream1", "datastream3", "datastream5"); - LOG.info("Make sure strategy reused all the tasks as opposed to creating new ones"); + _logger.info("Make sure strategy reused all the tasks as opposed to creating new ones"); // Make sure strategy reused all tasks as opposed to creating new ones List tasks2 = new ArrayList<>(connector3.getTasks()); tasks2.sort(Comparator.comparing(DatastreamTask::getDatastreamTaskName)); - LOG.info("Tasks1: " + tasks1.toString()); - LOG.info("Tasks2: " + tasks2.toString()); + _logger.info("Tasks1: " + tasks1.toString()); + _logger.info("Tasks2: " + tasks2.toString()); Assert.assertEquals(tasks1, tasks2); @@ -1659,7 +1659,7 @@ public void testMultipleInstanceDeath() throws Exception { String datastreamName = "datastream"; ZkClient zkClient = new ZkClient(_zkConnectionString); - LOG.info("Create four instances"); + _logger.info("Create four instances"); // // create a list of instances // @@ -1674,7 +1674,7 @@ public void testMultipleInstanceDeath() throws Exception { coordinators[i].start(); } - LOG.info("Create four datastreams"); + _logger.info("Create four datastreams"); // // create 1 datastream per instance @@ -1683,14 +1683,14 @@ public void testMultipleInstanceDeath() throws Exception { DatastreamTestUtils.createAndStoreDatastreams(zkClient, testCluster, testConnectorType, datastreamName + i); } - LOG.info("Validate whether the four datastreams are assigned to four instances"); + _logger.info("Validate whether the four datastreams are assigned to four instances"); // // wait until the last instance was assigned the last datastream, by now all datastream should be assigned // assertConnectorAssignment(connectors[count - 1], WAIT_TIMEOUT_MS, "datastream" + (count - 1)); - LOG.info("kill three instances except for the leader"); + _logger.info("kill three instances except for the leader"); // // kill all instances except the current leader @@ -1700,7 +1700,7 @@ public void testMultipleInstanceDeath() throws Exception { deleteLiveInstanceNode(zkClient, testCluster, coordinators[i]); } - LOG.info("Check whether all the instances are assigned to the only remaining instance."); + _logger.info("Check whether all the instances are assigned to the only remaining instance."); // // validate all datastream tasks are assigned to the leader now @@ -1730,7 +1730,7 @@ public void testSimpleAssignmentRebalancing() throws Exception { String testConnectorType = "testConnectorType"; ZkClient zkClient = new ZkClient(_zkConnectionString); - LOG.info("Create two coordinators and connector instances"); + _logger.info("Create two coordinators and connector instances"); // // setup a cluster with 2 live instances with simple assignment strategy @@ -1747,7 +1747,7 @@ public void testSimpleAssignmentRebalancing() throws Exception { null); instance2.start(); - LOG.info("Create two datastreams."); + _logger.info("Create two datastreams."); // // create 2 datastreams [datastream1, datastream2] @@ -1755,7 +1755,7 @@ public void testSimpleAssignmentRebalancing() throws Exception { DatastreamTestUtils.createAndStoreDatastreams(zkClient, testCluster, testConnectorType, "datastream1"); DatastreamTestUtils.createAndStoreDatastreams(zkClient, testCluster, testConnectorType, "datastream2"); - LOG.info("verify that two datastreams are assigned to two instances"); + _logger.info("verify that two datastreams are assigned to two instances"); // // verify assignment instance1: [datastream1], instance2:[datastream2] @@ -1763,13 +1763,13 @@ public void testSimpleAssignmentRebalancing() throws Exception { assertConnectorAssignment(connector1, WAIT_TIMEOUT_MS, "datastream1"); assertConnectorAssignment(connector2, WAIT_TIMEOUT_MS, "datastream2"); - LOG.info("create a third datastream but which is smaller than earlier two"); + _logger.info("create a third datastream but which is smaller than earlier two"); // // create 1 new datastream "datastream0", which has the smallest lexicographical order // DatastreamTestUtils.createAndStoreDatastreams(zkClient, testCluster, testConnectorType, "datastream0"); - LOG.info("Verify that the new datastream is assigned to first instance."); + _logger.info("Verify that the new datastream is assigned to first instance."); // // verify assignment instance1:[datastream0, datastream2], instance2:[datastream1] @@ -1799,7 +1799,7 @@ public void testSimpleAssignmentStrategyIndependent() throws Exception { String connectorType2 = "ConnectorType2"; ZkClient zkClient = new ZkClient(_zkConnectionString); - LOG.info( + _logger.info( "Create two coordinator with two connector types (one simple and one broadcast) in each and create a connector instance" + " of each connector type per coordinator"); @@ -1825,7 +1825,7 @@ public void testSimpleAssignmentStrategyIndependent() throws Exception { new SourceBasedDeduper(), null); instance2.start(); - LOG.info("Create three datastreams of connectorType1 and three datastreams of connectorType2"); + _logger.info("Create three datastreams of connectorType1 and three datastreams of connectorType2"); // // create 3 datastreams ["simple0", "simple1", "simple2"] for ConnectoryType1 @@ -1838,7 +1838,7 @@ public void testSimpleAssignmentStrategyIndependent() throws Exception { DatastreamTestUtils.createAndStoreDatastreams(zkClient, testCluster, connectorType2, "broadcast0", "broadcast1", "broadcast2"); - LOG.info("verify that the datastreams are properly assigned based on simple or broadcast"); + _logger.info("verify that the datastreams are properly assigned based on simple or broadcast"); // // verify assignment: instance1.connector1: [datastream0], connector2:[datastream2, datastream4"] // instance2.connector1:[datastream1], connector2:[datastream3] @@ -2286,7 +2286,7 @@ public void testPauseResumeSourcePartitions() throws Exception { Datastream[] list = DatastreamTestUtils.createAndStoreDatastreams(zkClient, testCluster, mmConnectorType, "mmDatastream"); Datastream mmDatastream = list[0]; - LOG.info("Created datastream: {}", mmDatastream); + _logger.info("Created datastream: {}", mmDatastream); // wait for datastream to be READY PollUtils.poll(() -> DatastreamTestUtils.getDatastream(zkClient, testCluster, "mmDatastream") @@ -2306,7 +2306,7 @@ public void testPauseResumeSourcePartitions() throws Exception { PathKeys pathKey = Mockito.mock(PathKeys.class); Mockito.when(pathKey.getAsString(DatastreamResources.KEY_NAME)).thenReturn(mmDatastream.getName()); - LOG.info("calling pause on mm datastream: {}", mmDatastream); + _logger.info("calling pause on mm datastream: {}", mmDatastream); CachedDatastreamReader datastreamCache = new CachedDatastreamReader(zkClient, testCluster); ZookeeperBackedDatastreamStore dsStore = new ZookeeperBackedDatastreamStore(datastreamCache, zkClient, testCluster); DatastreamResources datastreamResources = new DatastreamResources(dsStore, mmCoordinator); @@ -2345,7 +2345,7 @@ public void testPauseResumeSourcePartitionsThrowsErrorForNonMMConnectors() throw Datastream[] list = DatastreamTestUtils.createAndStoreDatastreams(zkClient, testCluster, nonMmConnectorType, "nonMmDatastream"); Datastream nonMmDatastream = list[0]; - LOG.info("Created datastream: {}", nonMmDatastream); + _logger.info("Created datastream: {}", nonMmDatastream); // wait for datastream to be READY PollUtils.poll(() -> DatastreamTestUtils.getDatastream(zkClient, testCluster, "nonMmDatastream") @@ -2372,7 +2372,7 @@ public void testPauseResumeSourcePartitionsThrowsErrorForNonMMConnectors() throw // For non-mm, should receive an error boolean exceptionReceived = false; - LOG.info("calling non mm datastream: {}", nonMmDatastream); + _logger.info("calling non mm datastream: {}", nonMmDatastream); datastreamCache = new CachedDatastreamReader(zkClient, testCluster); dsStore = new ZookeeperBackedDatastreamStore(datastreamCache, zkClient, testCluster); datastreamResources = new DatastreamResources(dsStore, nonMmCoordinator); @@ -2413,7 +2413,7 @@ public void testDatastreamDedupeMetadataCopy() throws Exception { PollUtils.poll(() -> DatastreamTestUtils.getDatastream(zkClient, testCluster, "stream1") .getStatus() .equals(DatastreamStatus.READY), 1000, WAIT_TIMEOUT_MS); - LOG.info("Created datastream: {}", datastreams[0]); + _logger.info("Created datastream: {}", datastreams[0]); // Trigger the code that does the dedupe and metadata copying datastreams[1].setSource(datastreams[0].getSource()); @@ -2453,7 +2453,7 @@ private void assertConnectorAssignment(TestHookConnector connector, long timeout boolean result = PollUtils.poll(() -> validateAssignment(connector.getTasks(), datastreamNames), interval, timeoutMs); - LOG.info( + _logger.info( String.format("assertConnectorAssignment. Connector: %s, Connector Tasks: %s, ASSERT: %s", connector.getName(), connector.getTasks(), result)); @@ -2473,7 +2473,7 @@ private void waitTillAssignmentIsComplete(int totalTasks, long timeoutMs, TestHo private boolean validateAssignment(List assignment, String... datastreamNames) { if (assignment.size() != datastreamNames.length) { - LOG.warn("Expected size: " + datastreamNames.length + ", Actual size: " + assignment.size()); + _logger.warn("Expected size: " + datastreamNames.length + ", Actual size: " + assignment.size()); return false; } @@ -2524,14 +2524,14 @@ public String getName() { * Get all datastream tasks */ public List getTasks() { - LOG.info(_name + ": getTasks. Instance: " + _instance + ", size: " + _tasks.size() + ", tasks: " + _tasks); + _logger.info(_name + ": getTasks. Instance: " + _instance + ", size: " + _tasks.size() + ", tasks: " + _tasks); return _tasks; } @Override public void start(CheckpointProvider checkpointProvider) { _isStarted = true; - LOG.info("Connector " + _name + " started"); + _logger.info("Connector " + _name + " started"); } @Override @@ -2542,7 +2542,7 @@ public void stop() { @Override public void onAssignmentChange(List tasks) { - LOG.info("START: onAssignmentChange. Name: " + _name + ", ConnectorType: " + _connectorType + _logger.info("START: onAssignmentChange. Name: " + _name + ", ConnectorType: " + _connectorType + ", Number of assignments: " + tasks.size() + ", tasks: " + tasks); _tasks = tasks; @@ -2552,7 +2552,7 @@ public void onAssignmentChange(List tasks) { } } - LOG.info("END: onAssignmentChange"); + _logger.info("END: onAssignmentChange"); } @Override diff --git a/datastream-server-restli/src/test/java/com/linkedin/datastream/server/TestDatastreamServer.java b/datastream-server-restli/src/test/java/com/linkedin/datastream/server/TestDatastreamServer.java index 0406bbbee..ce345fb7d 100644 --- a/datastream-server-restli/src/test/java/com/linkedin/datastream/server/TestDatastreamServer.java +++ b/datastream-server-restli/src/test/java/com/linkedin/datastream/server/TestDatastreamServer.java @@ -80,7 +80,7 @@ public class TestDatastreamServer { public static final String BROKEN_CONNECTOR = BrokenConnector.CONNECTOR_TYPE; public static final String FILE_CONNECTOR = FileConnector.CONNECTOR_NAME; - private static final Logger LOG = LoggerFactory.getLogger(TestDatastreamServer.class.getName()); + private static final Logger _logger = LoggerFactory.getLogger(TestDatastreamServer.class.getName()); private EmbeddedDatastreamCluster _datastreamCluster; @@ -208,8 +208,8 @@ public void testCreateTwoDatastreamOfFileConnectorProduceEventsReceiveEvents() t Collection eventsReceived1 = readFileDatastreamEvents(fileDatastream1, totalEvents); - LOG.info("Events Received " + eventsReceived1); - LOG.info("Events Written to file " + eventsWritten1); + _logger.info("Events Received " + eventsReceived1); + _logger.info("Events Written to file " + eventsWritten1); Assert.assertTrue(eventsReceived1.containsAll(eventsWritten1)); @@ -223,8 +223,8 @@ public void testCreateTwoDatastreamOfFileConnectorProduceEventsReceiveEvents() t Collection eventsReceived2 = readFileDatastreamEvents(fileDatastream2, totalEvents); - LOG.info("Events Received " + eventsReceived2); - LOG.info("Events Written to file " + eventsWritten2); + _logger.info("Events Received " + eventsReceived2); + _logger.info("Events Written to file " + eventsWritten2); Assert.assertTrue(eventsReceived2.containsAll(eventsWritten2)); @@ -258,8 +258,8 @@ public void testUserManagedDestination() throws Exception { Collection eventsReceived2 = readFileDatastreamEvents(fileDatastream1, 1, 6); eventsReceived1.addAll(eventsReceived2); - LOG.info("Events Received " + eventsReceived1); - LOG.info("Events Written to file " + eventsWritten1); + _logger.info("Events Received " + eventsReceived1); + _logger.info("Events Written to file " + eventsWritten1); Assert.assertTrue(eventsReceived1.containsAll(eventsWritten1)); } @@ -281,46 +281,46 @@ public void testDeleteDatastreamAndRecreateDatastream() throws Exception { Path tempFile1 = Files.createTempFile("testFile1", ""); String fileName1 = tempFile1.toAbsolutePath().toString(); - LOG.info("Creating the file datastream " + fileName1); + _logger.info("Creating the file datastream " + fileName1); Datastream fileDatastream1 = createFileDatastream(fileName1, 1); - LOG.info("Writing events to the file"); + _logger.info("Writing events to the file"); Collection eventsWritten1 = TestUtils.generateStrings(totalEvents); FileUtils.writeLines(new File(fileName1), eventsWritten1); - LOG.info("Reading events from file datastream " + fileDatastream1.getName()); + _logger.info("Reading events from file datastream " + fileDatastream1.getName()); Collection eventsReceived1 = readFileDatastreamEvents(fileDatastream1, totalEvents); - LOG.info("Events Received " + eventsReceived1); - LOG.info("Events Written to file " + eventsWritten1); + _logger.info("Events Received " + eventsReceived1); + _logger.info("Events Written to file " + eventsWritten1); Assert.assertTrue(eventsReceived1.containsAll(eventsWritten1)); - LOG.info("Deleting the datastream " + fileDatastream1.getName()); + _logger.info("Deleting the datastream " + fileDatastream1.getName()); DatastreamRestClient restClient = _datastreamCluster.createDatastreamRestClient(); restClient.deleteDatastream(fileDatastream1.getName()); // Adding a second sleep so that the new datastream will have a unique destination. Thread.sleep(Duration.ofSeconds(1).toMillis()); - LOG.info("Creating the datastream after deletion"); + _logger.info("Creating the datastream after deletion"); Datastream fileDatastream2 = createFileDatastream(fileName1, 1); - LOG.info("Destination for first datastream " + fileDatastream1.getDestination().toString()); - LOG.info("Destination for second datastream " + fileDatastream2.getDestination().toString()); + _logger.info("Destination for first datastream " + fileDatastream1.getDestination().toString()); + _logger.info("Destination for second datastream " + fileDatastream2.getDestination().toString()); Assert.assertNotEquals(fileDatastream1.getDestination().getConnectionString(), fileDatastream2.getDestination().getConnectionString()); - LOG.info("Writing events to the file " + fileName1); + _logger.info("Writing events to the file " + fileName1); eventsWritten1 = TestUtils.generateStrings(totalEvents); FileUtils.writeLines(new File(fileName1), eventsWritten1); - LOG.info("Reading events from file datastream " + fileDatastream2.getName()); + _logger.info("Reading events from file datastream " + fileDatastream2.getName()); eventsReceived1 = readFileDatastreamEvents(fileDatastream2, totalEvents); - LOG.info("Events Received " + eventsReceived1); - LOG.info("Events Written to file " + eventsWritten1); + _logger.info("Events Received " + eventsReceived1); + _logger.info("Events Written to file " + eventsWritten1); Assert.assertTrue(eventsReceived1.containsAll(eventsWritten1)); @@ -354,8 +354,8 @@ public void testNodeDownOneDatastreamSimpleStrategy() throws Exception { List eventsReceived1 = readFileDatastreamEvents(fileDatastream1, totalEvents); - LOG.info("Events Received " + eventsReceived1); - LOG.info("Events Written to file " + eventsWritten1); + _logger.info("Events Received " + eventsReceived1); + _logger.info("Events Written to file " + eventsWritten1); Assert.assertTrue(eventsReceived1.containsAll(eventsWritten1)); @@ -367,7 +367,7 @@ public void testNodeDownOneDatastreamSimpleStrategy() throws Exception { List assignments = zkclient.getChildren(assignmentPath); Assert.assertEquals(assignments.size(), 1); - LOG.info("Shutting down the server0."); + _logger.info("Shutting down the server0."); // Stop 1st instance and wait until its ZK node is gone _datastreamCluster.shutdownServer(0); String instancesPath = KeyBuilder.liveInstances(cluster); @@ -393,9 +393,9 @@ public void testNodeDownOneDatastreamSimpleStrategy() throws Exception { // KafkaTestUtils.readTopic always seeks to the beginning of the topic. List eventsReceived2 = readFileDatastreamEvents(fileDatastream1, totalEvents * 2); - LOG.info("Events Received " + eventsReceived2); - LOG.info("First set of events Written to file " + eventsWritten1); - LOG.info("Second set of events Written to file " + eventsWritten2); + _logger.info("Events Received " + eventsReceived2); + _logger.info("First set of events Written to file " + eventsWritten1); + _logger.info("Second set of events Written to file " + eventsWritten2); // If no duplicate events were produced eventsReceived2 should equal eventsWritten1 + eventsWritten2 // because KafkaTestUtils.readTopic always seeks to the beginning of the topic. @@ -426,8 +426,8 @@ public void testNodeDownOneDatastreamBroadcastStrategy() throws Exception { List eventsReceived1 = readFileDatastreamEvents(fileDatastream1, totalEvents * 2); - LOG.info("Events Received " + eventsReceived1); - LOG.info("Events Written to file " + eventsWritten1); + _logger.info("Events Received " + eventsReceived1); + _logger.info("Events Written to file " + eventsWritten1); // Expect two copies of eventsWritten1 given the two instances and BROADCAST strategy Map countMap = new HashMap<>(); @@ -474,8 +474,8 @@ public void testNodeDownOneDatastreamBroadcastStrategy() throws Exception { // KafkaTestUtils.readTopic always seeks to the beginning of the topic. List eventsReceived2 = readFileDatastreamEvents(fileDatastream1, totalEvents * 3); - LOG.info("Events Received " + eventsReceived2); - LOG.info("Events Written to file " + eventsWritten2); + _logger.info("Events Received " + eventsReceived2); + _logger.info("Events Written to file " + eventsWritten2); // Expect to see one copy of eventsWritten2 in eventsReceived2 Map countMap2 = new HashMap<>(); @@ -519,10 +519,10 @@ public void testNodeUpRebalanceTwoDatastreamsSimpleStrategy() throws Exception { List eventsReceived1 = readFileDatastreamEvents(fileDatastream1, totalEvents); List eventsReceived2 = readFileDatastreamEvents(fileDatastream2, totalEvents); - LOG.info("(1) Events Received " + eventsReceived1); - LOG.info("(1) Events Written to file " + eventsWritten1); - LOG.info("(2) Events Received " + eventsReceived2); - LOG.info("(2) Events Written to file " + eventsWritten2); + _logger.info("(1) Events Received " + eventsReceived1); + _logger.info("(1) Events Written to file " + eventsWritten1); + _logger.info("(2) Events Received " + eventsReceived2); + _logger.info("(2) Events Written to file " + eventsWritten2); Assert.assertTrue(eventsReceived1.containsAll(eventsWritten1)); Assert.assertTrue(eventsReceived2.containsAll(eventsWritten2)); @@ -545,12 +545,12 @@ public void testNodeUpRebalanceTwoDatastreamsSimpleStrategy() throws Exception { // Ensure each instance gets one task assignmentPath = KeyBuilder.instanceAssignments(cluster, instance1); Assert.assertTrue(PollUtils.poll((path) -> zkclient.getChildren(path).size() == 1, 100, 10000, assignmentPath)); - LOG.info("Instance1 got task: " + zkclient.getChildren(assignmentPath)); + _logger.info("Instance1 got task: " + zkclient.getChildren(assignmentPath)); String instance2 = server2.getCoordinator().getInstanceName(); assignmentPath = KeyBuilder.instanceAssignments(cluster, instance2); Assert.assertTrue(PollUtils.poll((path) -> zkclient.getChildren(path).size() == 1, 100, 10000, assignmentPath)); - LOG.info("Instance2 got task: " + zkclient.getChildren(assignmentPath)); + _logger.info("Instance2 got task: " + zkclient.getChildren(assignmentPath)); // Wait 3 seconds to allow the connectors to stop the handler and flush the checkpoints // Automatic flush period is 1 second by default. @@ -567,10 +567,10 @@ public void testNodeUpRebalanceTwoDatastreamsSimpleStrategy() throws Exception { eventsReceived1 = readFileDatastreamEvents(fileDatastream1, totalEvents * 2); eventsReceived2 = readFileDatastreamEvents(fileDatastream2, totalEvents * 2); - LOG.info("(1-NEW) Events Received " + eventsReceived1); - LOG.info("(1-NEW) Events Written to file " + eventsWritten1); - LOG.info("(2-NEW) Events Received " + eventsReceived2); - LOG.info("(2-NEW) Events Written to file " + eventsWritten2); + _logger.info("(1-NEW) Events Received " + eventsReceived1); + _logger.info("(1-NEW) Events Written to file " + eventsWritten1); + _logger.info("(2-NEW) Events Received " + eventsReceived2); + _logger.info("(2-NEW) Events Written to file " + eventsWritten2); Assert.assertTrue(eventsReceived1.containsAll(eventsWritten1)); Assert.assertTrue(eventsReceived2.containsAll(eventsWritten2)); @@ -589,7 +589,7 @@ private List readFileDatastreamEvents(Datastream datastream, int partiti _datastreamCluster.getKafkaCluster().getBrokers(), (key, value) -> { numberOfMessages[0]++; String eventValue = new String(value); - LOG.info("Read {}th of {} events, Event {} from datastream {}", numberOfMessages[0], totalEvents, eventValue, + _logger.info("Read {}th of {} events, Event {} from datastream {}", numberOfMessages[0], totalEvents, eventValue, datastream); eventsReceived.add(eventValue); return numberOfMessages[0] < totalEvents; diff --git a/datastream-server/src/main/java/com/linkedin/datastream/server/CachedDatastreamReader.java b/datastream-server/src/main/java/com/linkedin/datastream/server/CachedDatastreamReader.java index 8ef2ad490..545b12666 100644 --- a/datastream-server/src/main/java/com/linkedin/datastream/server/CachedDatastreamReader.java +++ b/datastream-server/src/main/java/com/linkedin/datastream/server/CachedDatastreamReader.java @@ -38,7 +38,7 @@ * update events such that any future datastream accesses will update the cached copies. */ public class CachedDatastreamReader { - private static final Logger LOG = LoggerFactory.getLogger(CachedDatastreamReader.class); + private static final Logger _logger = LoggerFactory.getLogger(CachedDatastreamReader.class); private final String _cluster; private final ZkClient _zkclient; @@ -59,24 +59,24 @@ public CachedDatastreamReader(ZkClient zkclient, String cluster) { _datastreamNames = fetchAllDatastreamNamesFromZk(); String path = KeyBuilder.datastreams(_cluster); - LOG.info("Subscribing to notification on zk path " + path); + _logger.info("Subscribing to notification on zk path " + path); // Be notified of changes to the children list in order to cache it. The listener creates a copy of the list // because other listeners could potentially get access to it and modify its contents. _zkclient.subscribeChildChanges(path, (parentPath, currentChildren) -> { synchronized (CachedDatastreamReader.this) { - LOG.debug( + _logger.debug( String.format("Received datastream add or delete notification. parentPath %s, children %s", parentPath, currentChildren)); _datastreamNames = new ArrayList<>(currentChildren); Set datastreamsRemoved = new HashSet<>(_datastreams.keySet()); datastreamsRemoved.removeAll(_datastreamNames); if (!datastreamsRemoved.isEmpty()) { - LOG.info("Removing the deleted datastreams {} from cache", datastreamsRemoved); + _logger.info("Removing the deleted datastreams {} from cache", datastreamsRemoved); _datastreams.keySet().removeAll(datastreamsRemoved); } - LOG.debug("New datastream list in the cache: {}", _datastreamNames); + _logger.debug("New datastream list in the cache: {}", _datastreamNames); } }); } @@ -135,7 +135,7 @@ public synchronized List getAllDatastreams(boolean flushCache) { * function would effectively make sure any following getDatastream calls get a newer copy of data. */ public synchronized void invalidateAllCache() { - LOG.info("About to invalidate all cache entries..."); + _logger.info("About to invalidate all cache entries..."); _datastreams.clear(); } @@ -152,9 +152,9 @@ Datastream getDatastream(String datastreamName, boolean flushCache) { ds = getDatastreamFromZk(datastreamName); if (ds == null) { - LOG.info("Datastream {} does not exist in cache/ZK.", datastreamName); + _logger.info("Datastream {} does not exist in cache/ZK.", datastreamName); } else if (!DatastreamUtils.hasValidDestination(ds)) { - LOG.info("Datastream {} does not have a valid destination yet and is not ready for use.", datastreamName); + _logger.info("Datastream {} does not have a valid destination yet and is not ready for use.", datastreamName); } else { _datastreams.put(datastreamName, ds); } @@ -178,7 +178,7 @@ private Datastream getDatastreamFromZk(String datastreamName) { } catch (ZkNoNodeException e) { // This can happen when the path still exists but later deleted // during ensureReadData - LOG.warn("Datastream {} is just deleted from ZK.", datastreamName); + _logger.warn("Datastream {} is just deleted from ZK.", datastreamName); } } return null; @@ -189,7 +189,7 @@ private List fetchAllDatastreamNamesFromZk() { if (_zkclient.exists(path)) { return _zkclient.getChildren(path, true); } else { - LOG.warn("Brooklin cluster '{}' is not initialize yet.", _cluster); + _logger.warn("Brooklin cluster '{}' is not initialize yet.", _cluster); return Collections.emptyList(); } } diff --git a/datastream-server/src/main/java/com/linkedin/datastream/server/CoordinatorEventBlockingQueue.java b/datastream-server/src/main/java/com/linkedin/datastream/server/CoordinatorEventBlockingQueue.java index 0068d8f3d..3662d09ee 100644 --- a/datastream-server/src/main/java/com/linkedin/datastream/server/CoordinatorEventBlockingQueue.java +++ b/datastream-server/src/main/java/com/linkedin/datastream/server/CoordinatorEventBlockingQueue.java @@ -20,7 +20,7 @@ */ public class CoordinatorEventBlockingQueue { - private static final Logger LOG = LoggerFactory.getLogger(CoordinatorEventBlockingQueue.class.getName()); + private static final Logger _logger = LoggerFactory.getLogger(CoordinatorEventBlockingQueue.class.getName()); private final Map _eventMap; private final Queue _eventQueue; @@ -37,7 +37,7 @@ public CoordinatorEventBlockingQueue() { * @param event CoordinatorEvent event to add to the queue */ public synchronized void put(CoordinatorEvent event) { - LOG.info("Queuing event {} to event queue", event.getType()); + _logger.info("Queuing event {} to event queue", event.getType()); if (!_eventMap.containsKey(event.getType())) { // only insert if there isn't an event present in the queue with the same name boolean result = _eventQueue.offer(event); @@ -51,7 +51,7 @@ public synchronized void put(CoordinatorEvent event) { _eventMap.put(event.getType(), event); } - LOG.debug("Event queue size {}", _eventQueue.size()); + _logger.debug("Event queue size {}", _eventQueue.size()); notify(); } @@ -76,8 +76,8 @@ public synchronized CoordinatorEvent take() throws InterruptedException { } if (queuedEvent != null) { - LOG.info("De-queuing event " + queuedEvent.getType()); - LOG.debug("Event queue size: {}", _eventQueue.size()); + _logger.info("De-queuing event " + queuedEvent.getType()); + _logger.debug("Event queue size: {}", _eventQueue.size()); _eventMap.remove(queuedEvent.getType()); return queuedEvent; } diff --git a/datastream-server/src/main/java/com/linkedin/datastream/server/DatastreamServerConfigurationConstants.java b/datastream-server/src/main/java/com/linkedin/datastream/server/DatastreamServerConfigurationConstants.java index 22c0bbf2b..067520fb9 100644 --- a/datastream-server/src/main/java/com/linkedin/datastream/server/DatastreamServerConfigurationConstants.java +++ b/datastream-server/src/main/java/com/linkedin/datastream/server/DatastreamServerConfigurationConstants.java @@ -35,4 +35,8 @@ public final class DatastreamServerConfigurationConstants { public static final String DOMAIN_DIAG = CONFIG_PREFIX + "diag"; public static final String CONFIG_DIAG_PORT = "port"; public static final String CONFIG_DIAG_PATH = "path"; + // Inflexdb Metrics config + public static final String CONFIG_TELEGRAF_HOST = CONFIG_PREFIX + "telegrafHost"; + public static final String CONFIG_TELEGRAF_PORT = CONFIG_PREFIX + "telegrafPort"; + public static final String CONFIG_METRICS_REPORT_INTERVAL_SEC = CONFIG_PREFIX + "metricsReportIntervalSec"; } diff --git a/datastream-server/src/main/java/com/linkedin/datastream/server/DatastreamTaskImpl.java b/datastream-server/src/main/java/com/linkedin/datastream/server/DatastreamTaskImpl.java index 0a733b3b8..bc927dcb2 100644 --- a/datastream-server/src/main/java/com/linkedin/datastream/server/DatastreamTaskImpl.java +++ b/datastream-server/src/main/java/com/linkedin/datastream/server/DatastreamTaskImpl.java @@ -54,7 +54,7 @@ */ public class DatastreamTaskImpl implements DatastreamTask { - private static final Logger LOG = LoggerFactory.getLogger(DatastreamTask.class.getName()); + private static final Logger _logger = LoggerFactory.getLogger(DatastreamTask.class.getName()); private static final String STATUS = "STATUS"; private volatile List _datastreams; @@ -143,7 +143,7 @@ public DatastreamTaskImpl(List datastreams, String id, List //partitionV2 doesn't require a default partition } } - LOG.info("Created new DatastreamTask " + this); + _logger.info("Created new DatastreamTask " + this); _dependencies = new ArrayList<>(); } @@ -191,7 +191,7 @@ public static String getTaskPrefix(Datastream datastream) { */ public static DatastreamTaskImpl fromJson(String json) { DatastreamTaskImpl task = JsonUtils.fromJson(json, DatastreamTaskImpl.class); - LOG.info("Loaded existing DatastreamTask: {}", task); + _logger.info("Loaded existing DatastreamTask: {}", task); return task; } @@ -308,7 +308,7 @@ public void acquire(Duration timeout) { _zkAdapter.acquireTask(this, timeout); } catch (Exception e) { - LOG.error("Failed to acquire task: " + this, e); + _logger.error("Failed to acquire task: " + this, e); setStatus(DatastreamTaskStatus.error("Acquire failed, exception: " + e)); throw e; } @@ -450,7 +450,7 @@ public void setZkAdapter(ZkAdapter adapter) { * @param checkpoint Checkpoint to update to. */ public void updateCheckpoint(int partition, String checkpoint) { - LOG.debug("Update checkpoint called for partition {} and checkpoint {}", partition, checkpoint); + _logger.debug("Update checkpoint called for partition {} and checkpoint {}", partition, checkpoint); _checkpoints.put(partition, checkpoint); } diff --git a/datastream-server/src/main/java/com/linkedin/datastream/server/EventProducer.java b/datastream-server/src/main/java/com/linkedin/datastream/server/EventProducer.java index 3af2eebfc..342f7a2e2 100644 --- a/datastream-server/src/main/java/com/linkedin/datastream/server/EventProducer.java +++ b/datastream-server/src/main/java/com/linkedin/datastream/server/EventProducer.java @@ -53,6 +53,7 @@ public class EventProducer implements DatastreamEventProducer { static final String EVENTS_LATENCY_MS_STRING = "eventsLatencyMs"; static final String EVENTS_SEND_LATENCY_MS_STRING = "eventsSendLatencyMs"; + static final String EVENTS_PRODUCE_LATENCY_MS_STRING = "eventsProduceLatencyMs"; private static final String MODULE = EventProducer.class.getSimpleName(); private static final String METRICS_PREFIX = MODULE + MetricsAware.KEY_REGEX; @@ -168,6 +169,9 @@ private void validateEventRecord(DatastreamProducerRecord record) { */ @Override public void send(DatastreamProducerRecord record, SendCallback sendCallback) { + _dynamicMetricsManager.createOrUpdateMeter(MODULE, _datastreamTask.getConnectorType(), "sendCall", 1); + + try { validateEventRecord(record); @@ -191,7 +195,9 @@ public void send(DatastreamProducerRecord record, SendCallback sendCallback) { record.getDestination().orElse(_datastreamTask.getDatastreamDestination().getConnectionString()); record.setEventsSendTimestamp(System.currentTimeMillis()); _transportProvider.send(destination, record, - (metadata, exception) -> onSendCallback(metadata, exception, sendCallback, record.getEventsSourceTimestamp(), + (metadata, exception) -> onSendCallback(metadata, exception, sendCallback, + record.getEventsProduceTimestamp().orElse(0L), + record.getEventsSourceTimestamp(), record.getEventsSendTimestamp().orElse(0L))); } catch (Exception e) { String errorMessage = String.format("Failed send the event %s exception %s", record, e); @@ -226,7 +232,10 @@ private void reportSLAMetrics(String topicOrDatastreamName, boolean isWithinSLA, * per DatastreamProducerRecord (i.e. by the number of events within the record), only increment all metrics by 1 * to avoid overcounting. */ - private void reportMetrics(DatastreamRecordMetadata metadata, long eventsSourceTimestamp, long eventsSendTimestamp) { + private void reportMetrics(DatastreamRecordMetadata metadata, + long eventProduceTimestamp, + long eventsSourceTimestamp, + long eventsSendTimestamp) { // If per-topic metrics are enabled, use topic as key for metrics; else, use datastream name as the key String datastreamName = getDatastreamName(); String topicOrDatastreamName = _enablePerTopicMetrics ? metadata.getTopic() : datastreamName; @@ -236,12 +245,12 @@ private void reportMetrics(DatastreamRecordMetadata metadata, long eventsSourceT long sourceToDestinationLatencyMs = System.currentTimeMillis() - eventsSourceTimestamp; // Using a time sliding window for reporting latency specifically. // Otherwise we report very stuck max value for slow source - _dynamicMetricsManager.createOrUpdateSlidingWindowHistogram(MODULE, topicOrDatastreamName, - EVENTS_LATENCY_MS_STRING, LATENCY_SLIDING_WINDOW_LENGTH_MS, sourceToDestinationLatencyMs); - _dynamicMetricsManager.createOrUpdateSlidingWindowHistogram(MODULE, AGGREGATE, EVENTS_LATENCY_MS_STRING, - LATENCY_SLIDING_WINDOW_LENGTH_MS, sourceToDestinationLatencyMs); - _dynamicMetricsManager.createOrUpdateSlidingWindowHistogram(MODULE, _datastreamTask.getConnectorType(), - EVENTS_LATENCY_MS_STRING, LATENCY_SLIDING_WINDOW_LENGTH_MS, sourceToDestinationLatencyMs); + _dynamicMetricsManager.createOrUpdateHistogram(MODULE, topicOrDatastreamName, + EVENTS_LATENCY_MS_STRING, sourceToDestinationLatencyMs); + _dynamicMetricsManager.createOrUpdateHistogram(MODULE, AGGREGATE, EVENTS_LATENCY_MS_STRING, + sourceToDestinationLatencyMs); + _dynamicMetricsManager.createOrUpdateHistogram(MODULE, _datastreamTask.getConnectorType(), + EVENTS_LATENCY_MS_STRING, sourceToDestinationLatencyMs); reportSLAMetrics(topicOrDatastreamName, sourceToDestinationLatencyMs <= _availabilityThresholdSlaMs, EVENTS_PRODUCED_WITHIN_SLA, EVENTS_PRODUCED_OUTSIDE_SLA); @@ -269,6 +278,16 @@ private void reportMetrics(DatastreamRecordMetadata metadata, long eventsSourceT 1); } + // CDC e2e latency + if (eventProduceTimestamp > 0) { + long produceLatency = System.currentTimeMillis() - eventProduceTimestamp; + _dynamicMetricsManager.createOrUpdateHistogram(MODULE, topicOrDatastreamName, EVENTS_PRODUCE_LATENCY_MS_STRING, + produceLatency); + _dynamicMetricsManager.createOrUpdateHistogram(MODULE, AGGREGATE, EVENTS_PRODUCE_LATENCY_MS_STRING, produceLatency); + _dynamicMetricsManager.createOrUpdateHistogram(MODULE, _datastreamTask.getConnectorType(), + EVENTS_PRODUCE_LATENCY_MS_STRING, produceLatency); + } + // Report the time it took to just send the events to destination if (eventsSendTimestamp > 0) { long sendLatency = System.currentTimeMillis() - eventsSendTimestamp; @@ -278,12 +297,16 @@ private void reportMetrics(DatastreamRecordMetadata metadata, long eventsSourceT _dynamicMetricsManager.createOrUpdateHistogram(MODULE, _datastreamTask.getConnectorType(), EVENTS_SEND_LATENCY_MS_STRING, sendLatency); } + _dynamicMetricsManager.createOrUpdateMeter(MODULE, AGGREGATE, EVENT_PRODUCE_RATE, 1); _dynamicMetricsManager.createOrUpdateMeter(MODULE, _datastreamTask.getConnectorType(), EVENT_PRODUCE_RATE, 1); + _dynamicMetricsManager.createOrUpdateMeter(MODULE, topicOrDatastreamName, EVENT_PRODUCE_RATE, 1); } private void onSendCallback(DatastreamRecordMetadata metadata, Exception exception, SendCallback sendCallback, - long eventSourceTimestamp, long eventSendTimestamp) { + long eventProduceTimestamp, + long eventSourceTimestamp, + long eventSendTimestamp) { SendFailedException sendFailedException = null; @@ -294,7 +317,7 @@ private void onSendCallback(DatastreamRecordMetadata metadata, Exception excepti } else { // Report metrics checkpoint(metadata.getPartition(), metadata.getCheckpoint()); - reportMetrics(metadata, eventSourceTimestamp, eventSendTimestamp); + reportMetrics(metadata, eventProduceTimestamp, eventSourceTimestamp, eventSendTimestamp); } // Inform the connector about the success or failure, In the case of failure, diff --git a/datastream-server/src/main/java/com/linkedin/datastream/server/SourceBasedDeduper.java b/datastream-server/src/main/java/com/linkedin/datastream/server/SourceBasedDeduper.java index f468b9ab7..d7c67bf1f 100644 --- a/datastream-server/src/main/java/com/linkedin/datastream/server/SourceBasedDeduper.java +++ b/datastream-server/src/main/java/com/linkedin/datastream/server/SourceBasedDeduper.java @@ -20,7 +20,7 @@ * Deduper that uses the source to figure out whether two datastreams can be deduped. */ public class SourceBasedDeduper extends AbstractDatastreamDeduper { - private static final Logger LOG = LoggerFactory.getLogger(SourceBasedDeduper.class); + private static final Logger _logger = LoggerFactory.getLogger(SourceBasedDeduper.class); @Override public Optional dedupeStreams(Datastream stream, List candidates) @@ -33,7 +33,7 @@ public Optional dedupeStreams(Datastream stream, List ca if (!duplicateDatastreams.isEmpty()) { reusedStream = Optional.of(duplicateDatastreams.get(0)); - LOG.info("Found duplicate datastreams {} for datastream {}, picked {}", + _logger.info("Found duplicate datastreams {} for datastream {}, picked {}", duplicateDatastreams, stream, reusedStream); } diff --git a/datastream-server/src/main/java/com/linkedin/datastream/server/assignment/BroadcastStrategy.java b/datastream-server/src/main/java/com/linkedin/datastream/server/assignment/BroadcastStrategy.java index 2c505ac87..703d11818 100644 --- a/datastream-server/src/main/java/com/linkedin/datastream/server/assignment/BroadcastStrategy.java +++ b/datastream-server/src/main/java/com/linkedin/datastream/server/assignment/BroadcastStrategy.java @@ -40,7 +40,7 @@ */ public class BroadcastStrategy implements AssignmentStrategy { - private static final Logger LOG = LoggerFactory.getLogger(BroadcastStrategy.class.getName()); + private static final Logger _logger = LoggerFactory.getLogger(BroadcastStrategy.class.getName()); private final Optional _maxTasks; private final Optional _maxTasksPerInstance; @@ -76,7 +76,7 @@ public Map> assign(List datastreams Map> currentAssignment) { int totalAssignedTasks = currentAssignment.values().stream().mapToInt(Set::size).sum(); - LOG.info("Assigning {} datastreams to {} instances with {} tasks", datastreams.size(), instances.size(), + _logger.info("Assigning {} datastreams to {} instances with {} tasks", datastreams.size(), instances.size(), totalAssignedTasks); // if there are no live instances, return empty assignment @@ -127,7 +127,7 @@ public Map> assign(List datastreams } } - LOG.info("New assignment is {}", newAssignment); + _logger.info("New assignment is {}", newAssignment); return newAssignment; } diff --git a/datastream-server/src/main/java/com/linkedin/datastream/server/assignment/LoadbalancingStrategy.java b/datastream-server/src/main/java/com/linkedin/datastream/server/assignment/LoadbalancingStrategy.java index ad416663d..40814597b 100644 --- a/datastream-server/src/main/java/com/linkedin/datastream/server/assignment/LoadbalancingStrategy.java +++ b/datastream-server/src/main/java/com/linkedin/datastream/server/assignment/LoadbalancingStrategy.java @@ -39,7 +39,7 @@ public class LoadbalancingStrategy implements AssignmentStrategy { public static final String CFG_MIN_TASKS = "TasksPerDatastream"; public static final int DEFAULT_MIN_TASKS = -1; - private static final Logger LOG = LoggerFactory.getLogger(LoadbalancingStrategy.class.getName()); + private static final Logger _logger = LoggerFactory.getLogger(LoadbalancingStrategy.class.getName()); // If the instances are down while the assignment is happening. We need to ensure that at least min tasks are created private final int minTasks; @@ -65,7 +65,7 @@ public LoadbalancingStrategy(Properties properties) { @Override public Map> assign(List datastreams, List instances, Map> currentAssignment) { - LOG.info("Assign called with datastreams: {}, instances: {}, currentAssignment: {}", datastreams, + _logger.info("Assign called with datastreams: {}, instances: {}, currentAssignment: {}", datastreams, instances, currentAssignment); // if there are no live instances, return empty assignment if (instances.size() == 0) { @@ -87,7 +87,7 @@ public Map> assign(List datastreams assignment.get(instanceName).add(datastreamTasks.get(i)); } - LOG.info("Datastream Groups: {}, instances: {}, currentAssignment: {}, NewAssignment: {}", + _logger.info("Datastream Groups: {}, instances: {}, currentAssignment: {}, NewAssignment: {}", datastreams.stream().map(DatastreamGroup::getTaskPrefix).collect(Collectors.toList()), instances, currentAssignment, assignment); @@ -121,7 +121,7 @@ private List getDatastreamTasks(List datastream assignedPartitions.addAll(task.getPartitions()); } if (count != numberOfDatastreamPartitions || assignedPartitions.size() != numberOfDatastreamPartitions) { - LOG.error( + _logger.error( "Corrupted partition information for datastream {}. Expected number of partitions {}, actual {}: {}", datastream.getName(), numberOfDatastreamPartitions, count, assignedPartitions); } diff --git a/datastream-server/src/main/java/com/linkedin/datastream/server/assignment/StickyMulticastStrategy.java b/datastream-server/src/main/java/com/linkedin/datastream/server/assignment/StickyMulticastStrategy.java index 7eff0c69a..a920bedfc 100644 --- a/datastream-server/src/main/java/com/linkedin/datastream/server/assignment/StickyMulticastStrategy.java +++ b/datastream-server/src/main/java/com/linkedin/datastream/server/assignment/StickyMulticastStrategy.java @@ -60,7 +60,7 @@ */ public class StickyMulticastStrategy implements AssignmentStrategy { - private static final Logger LOG = LoggerFactory.getLogger(StickyMulticastStrategy.class.getName()); + private static final Logger _logger = LoggerFactory.getLogger(StickyMulticastStrategy.class.getName()); private static final Integer DEFAULT_IMBALANCE_THRESHOLD = 1; private final Optional _maxTasks; @@ -91,7 +91,7 @@ public Map> assign(List datastreams Map> currentAssignment) { int totalAssignedTasks = currentAssignment.values().stream().mapToInt(Set::size).sum(); - LOG.info("Begin assign {} datastreams to {} instances with {} tasks", datastreams.size(), instances.size(), + _logger.info("Begin assign {} datastreams to {} instances with {} tasks", datastreams.size(), instances.size(), totalAssignedTasks); if (instances.isEmpty()) { @@ -199,8 +199,8 @@ public Map> assign(List datastreams } // STEP4: Format the result with the right data structure. - LOG.info("Assignment completed"); - LOG.debug("New assignment is {}", newAssignment); + _logger.info("Assignment completed"); + _logger.debug("New assignment is {}", newAssignment); // STEP5: Some Sanity Checks, to detect missing tasks. sanityChecks(datastreams, instances, newAssignment); diff --git a/datastream-server/src/main/java/com/linkedin/datastream/server/assignment/StickyPartitionAssignmentStrategy.java b/datastream-server/src/main/java/com/linkedin/datastream/server/assignment/StickyPartitionAssignmentStrategy.java index 1ab1d0071..729542aed 100644 --- a/datastream-server/src/main/java/com/linkedin/datastream/server/assignment/StickyPartitionAssignmentStrategy.java +++ b/datastream-server/src/main/java/com/linkedin/datastream/server/assignment/StickyPartitionAssignmentStrategy.java @@ -33,7 +33,7 @@ * The total number of tasks is also unchanged during this process. */ public class StickyPartitionAssignmentStrategy extends StickyMulticastStrategy { - private static final Logger LOG = LoggerFactory.getLogger(StickyPartitionAssignmentStrategy.class.getName()); + private static final Logger _logger = LoggerFactory.getLogger(StickyPartitionAssignmentStrategy.class.getName()); private final Integer _maxPartitionPerTask; /** @@ -65,7 +65,7 @@ public StickyPartitionAssignmentStrategy(Optional maxTasks, Optional> assignPartitions(Map> currentAssignment, DatastreamGroupPartitionsMetadata datastreamPartitions) { - LOG.info("old partition assignment info, assignment: {}", currentAssignment); + _logger.info("old partition assignment info, assignment: {}", currentAssignment); String dgName = datastreamPartitions.getDatastreamGroup().getName(); @@ -90,7 +90,7 @@ public Map> assignPartitions(Map> assignPartitions(Map> assignPartitions(Map> movePartitions(Map> currentAssignment, Map> targetAssignment, DatastreamGroupPartitionsMetadata partitionsMetadata) { - LOG.info("Move partition, current assignment: {}, target assignment: {}, all partitions: {}", currentAssignment, + _logger.info("Move partition, current assignment: {}, target assignment: {}, all partitions: {}", currentAssignment, targetAssignment, partitionsMetadata.getPartitions()); DatastreamGroup dg = partitionsMetadata.getDatastreamGroup(); @@ -213,7 +213,7 @@ public Map> movePartitions(Map 0 && dgTasks.isEmpty()) { String errorMsg = String.format("No task is available in the target instance %s", instance); - LOG.error(errorMsg); + _logger.error(errorMsg); throw new DatastreamRuntimeException(errorMsg); } @@ -256,7 +256,7 @@ public Map> movePartitions(Map> assignedTask if (total != allPartitions.getPartitions().size()) { String errorMsg = String.format("Validation failed after assignment, assigned partitions " + "size: %s is not equal to all partitions size: %s", total, allPartitions.getPartitions().size()); - LOG.error(errorMsg); + _logger.error(errorMsg); throw new DatastreamRuntimeException(errorMsg); } if (unassignedPartitions.size() > 0) { String errorMsg = String.format("Validation failed after assignment, " + "unassigned partition: %s", unassignedPartitions); - LOG.error(errorMsg); + _logger.error(errorMsg); throw new DatastreamRuntimeException(errorMsg); } } diff --git a/datastream-server/src/main/java/com/linkedin/datastream/server/providers/KafkaCustomCheckpointProvider.java b/datastream-server/src/main/java/com/linkedin/datastream/server/providers/KafkaCustomCheckpointProvider.java index fa57f6b77..d8370e1df 100644 --- a/datastream-server/src/main/java/com/linkedin/datastream/server/providers/KafkaCustomCheckpointProvider.java +++ b/datastream-server/src/main/java/com/linkedin/datastream/server/providers/KafkaCustomCheckpointProvider.java @@ -11,6 +11,7 @@ import java.util.Map; import java.util.Properties; import java.util.concurrent.ExecutionException; +import java.util.function.Supplier; import org.apache.kafka.clients.admin.AdminClient; import org.apache.kafka.clients.admin.CreateTopicsResult; @@ -26,6 +27,8 @@ import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.TopicExistsException; +import org.apache.kafka.common.serialization.ByteArrayDeserializer; +import org.apache.kafka.common.serialization.ByteArraySerializer; import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.common.serialization.StringSerializer; import org.slf4j.Logger; @@ -33,22 +36,24 @@ import static java.util.Collections.singletonList; + /** * implementation for {@link CustomCheckpointProvider} * uses kafka topic as the datastore + * @param + * @param */ -public class KafkaCustomCheckpointProvider implements CustomCheckpointProvider { - private static final Logger LOG = LoggerFactory.getLogger(KafkaCustomCheckpointProvider.class); +public class KafkaCustomCheckpointProvider + implements CustomCheckpointProvider { + + private static final Logger _logger = LoggerFactory.getLogger(KafkaCustomCheckpointProvider.class); private final String _taskId; private final String _topic; - private final Consumer _consumer; - private final Producer _producer; + private final Consumer _consumer; + private final Producer _producer; private final TopicPartition _topicPartition; - private Long _checkpoint; - private Long _previousCommittedCheckpoint; - /** * Constructor for KafkaCustomCheckpointProvider * @param taskId id that uniquely identifies a task @@ -65,7 +70,7 @@ public KafkaCustomCheckpointProvider(String taskId, consumerProperties.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers); consumerProperties.put(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, 1); consumerProperties.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName()); - consumerProperties.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName()); + consumerProperties.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName()); this._consumer = new KafkaConsumer<>(consumerProperties); Properties producerProperties = new Properties(); @@ -74,7 +79,7 @@ public KafkaCustomCheckpointProvider(String taskId, producerProperties.put(ProducerConfig.RETRIES_CONFIG, 5); producerProperties.put(ProducerConfig.BATCH_SIZE_CONFIG, 0); producerProperties.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName()); - producerProperties.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName()); + producerProperties.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName()); this._producer = new KafkaProducer<>(producerProperties); @@ -102,36 +107,18 @@ public KafkaCustomCheckpointProvider(String taskId, this._consumer.poll(Duration.ZERO); // poll the first time this._topicPartition = new TopicPartition(_topic, 0); - - this._checkpoint = null; - - this._previousCommittedCheckpoint = Long.MIN_VALUE; - } - - @Override - public void updateCheckpoint(Long checkpoint) { - _checkpoint = checkpoint; - } - - @Override - public void rewindTo(Long checkpoint) { - _checkpoint = checkpoint; - commit(checkpoint); } @Override - public void commit(Long checkpoint) { - if (!_previousCommittedCheckpoint.equals(checkpoint)) { - LOG.info("Commit call for task {} with checkpoint {}", _taskId, checkpoint); - try { - _producer.send(new ProducerRecord<>(_topic, _taskId, checkpoint.toString())).get(); - _producer.flush(); - _previousCommittedCheckpoint = checkpoint; - } catch (ExecutionException e) { - throw new RuntimeException(e); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - } + public void commit(T checkpoint) { + _logger.info("Committing checkpoint {} for task {}", checkpoint, _taskId); + try { + _producer.send(new ProducerRecord<>(_topic, _taskId, checkpoint.serialize())).get(); + _producer.flush(); + } catch (ExecutionException e) { + throw new RuntimeException(e); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); } } @@ -142,14 +129,8 @@ public void close() { } @Override - public Long getSafeCheckpoint() throws Exception { - _checkpoint = (_checkpoint == null) ? getCommitted() : _checkpoint; - return _checkpoint; - } - - @Override - public Long getCommitted() throws Exception { - Long checkpoint = null; + public T getSafeCheckpoint(Supplier deserSupplier, Class checkpointClass) { + T checkpoint = null; long endOffset = -1; Map endOffsets = _consumer.endOffsets(Collections.singletonList(_topicPartition)); @@ -158,18 +139,14 @@ public Long getCommitted() throws Exception { } _consumer.seekToBeginning(Collections.singletonList(new TopicPartition(_topic, 0))); - ConsumerRecords records = _consumer.poll(Duration.ofMillis(30)); + ConsumerRecords records = _consumer.poll(Duration.ofMillis(30)); long currentOffset = -1; while (currentOffset < endOffset - 1) { - for (ConsumerRecord record : records) { + for (ConsumerRecord record : records) { if (record.key().equals(_taskId)) { - try { - checkpoint = Long.parseLong(record.value()); - } catch (NumberFormatException n) { - LOG.error("corrupted checkpoint data was encountered in topic {}. {}", _topic, n); - throw n; - } + S deser = deserSupplier.get(); + checkpoint = deser.deserialize(record.value(), checkpointClass); } currentOffset = record.offset(); } diff --git a/datastream-server/src/main/java/com/linkedin/datastream/server/providers/ZookeeperCheckpointProvider.java b/datastream-server/src/main/java/com/linkedin/datastream/server/providers/ZookeeperCheckpointProvider.java index b9a9cb5bd..4911b229a 100644 --- a/datastream-server/src/main/java/com/linkedin/datastream/server/providers/ZookeeperCheckpointProvider.java +++ b/datastream-server/src/main/java/com/linkedin/datastream/server/providers/ZookeeperCheckpointProvider.java @@ -36,7 +36,7 @@ public class ZookeeperCheckpointProvider implements CheckpointProvider { public static final String CHECKPOINT_KEY_NAME = "sourceCheckpoint"; - private static final Logger LOG = LoggerFactory.getLogger(ZookeeperCheckpointProvider.class.getName()); + private static final Logger _logger = LoggerFactory.getLogger(ZookeeperCheckpointProvider.class.getName()); private static final String MODULE = ZookeeperCheckpointProvider.class.getSimpleName(); private static final String NUM_CHECKPOINT_COMMITS = "numCheckpointCommits"; private static final String CHECKPOINT_COMMIT_LATENCY_MS = "checkpointCommitLatencyMs"; @@ -148,7 +148,7 @@ private Map getCheckpoint(DatastreamTask task) { if (StringUtils.isNotBlank(checkpoint)) { return JsonUtils.fromJson(checkpoint, _hashMapTypeReference); } else { - LOG.info("Checkpoint doesn't exist for DatastreamTask " + task.toString()); + _logger.info("Checkpoint doesn't exist for DatastreamTask " + task.toString()); return new HashMap<>(); } } @@ -162,7 +162,7 @@ private Map getCheckpoint(DatastreamTask task) { public Map getCommitted(DatastreamTask datastreamTask) { Validate.notNull(datastreamTask, "datastreamTask should not be null"); Map checkpoints = getCheckpoint(datastreamTask); - LOG.info("GetCommitted returning the last committed checkpoints " + checkpoints.toString()); + _logger.info("GetCommitted returning the last committed checkpoints " + checkpoints.toString()); return checkpoints; } diff --git a/datastream-server/src/main/java/com/linkedin/datastream/server/zk/ZkAdapter.java b/datastream-server/src/main/java/com/linkedin/datastream/server/zk/ZkAdapter.java index 39ba37f81..1372a355b 100644 --- a/datastream-server/src/main/java/com/linkedin/datastream/server/zk/ZkAdapter.java +++ b/datastream-server/src/main/java/com/linkedin/datastream/server/zk/ZkAdapter.java @@ -105,7 +105,7 @@ * @see ZkClient */ public class ZkAdapter { - private static final Logger LOG = LoggerFactory.getLogger(ZkAdapter.class); + private static final Logger _logger = LoggerFactory.getLogger(ZkAdapter.class); private final String _defaultTransportProviderName; @@ -181,7 +181,7 @@ public void disconnect() { try { // remove the liveinstance node String liveInstancePath = KeyBuilder.liveInstance(_cluster, _liveInstanceName); - LOG.info("deleting live instance node: " + liveInstancePath); + _logger.info("deleting live instance node: " + liveInstancePath); _zkclient.delete(liveInstancePath); // NOTE: we should not delete the instance node which still holds the @@ -219,7 +219,7 @@ private void initializeSession() { // create a globally unique instance name and create a live instance node in ZooKeeper _instanceName = createLiveInstanceNode(); - LOG.info("Coordinator instance " + _instanceName + " is online"); + _logger.info("Coordinator instance " + _instanceName + " is online"); // both leader and follower needs to listen to its own instance change // under /{cluster}/instances/{instance} @@ -229,12 +229,12 @@ private void initializeSession() { onBecomeFollower(); joinLeaderElection(); - LOG.info("Instance " + _instanceName + " is ready."); + _logger.info("Instance " + _instanceName + " is ready."); // populate the instance name. } private void onBecomeLeader() { - LOG.info("Instance " + _instanceName + " becomes leader"); + _logger.info("Instance " + _instanceName + " becomes leader"); _datastreamList = new ZkBackedDMSDatastreamList(); _liveInstancesProvider = new ZkBackedLiveInstanceListProvider(); @@ -252,7 +252,7 @@ private void onBecomeLeader() { } private void onBecomeFollower() { - LOG.info("Instance " + _instanceName + " becomes follower"); + _logger.info("Instance " + _instanceName + " becomes follower"); if (_datastreamList != null) { _datastreamList.close(); @@ -316,7 +316,7 @@ private void joinLeaderElection() { if (index < 0) { // only when the ZooKeeper session already expired by the time this adapter joins for leader election. // mostly because the zkclient session expiration timeout. - LOG.error("Failed to join leader election. Let the state change handler handle the reconnect."); + _logger.error("Failed to join leader election. Let the state change handler handle the reconnect."); return; } @@ -369,7 +369,7 @@ private void joinLeaderElection() { public boolean updateDatastream(Datastream datastream) { String path = KeyBuilder.datastream(_cluster, datastream.getName()); if (!_zkclient.exists(path)) { - LOG.warn("trying to update znode of datastream that does not exist. Datastream name: " + datastream.getName()); + _logger.warn("trying to update znode of datastream that does not exist. Datastream name: " + datastream.getName()); return false; } @@ -396,7 +396,7 @@ public void deleteTasksWithPrefix(Set connectors, String taskPrefix) { List deadTasks = allTasks.stream().filter(tasksToDelete::contains).collect(Collectors.toList()); if (deadTasks.size() > 0) { - LOG.info("Cleaning up deprecated connector tasks: {} for connector: {}", deadTasks, connector); + _logger.info("Cleaning up deprecated connector tasks: {} for connector: {}", deadTasks, connector); for (String task : deadTasks) { deleteConnectorTask(connector, task); } @@ -405,11 +405,11 @@ public void deleteTasksWithPrefix(Set connectors, String taskPrefix) { } private void deleteConnectorTask(String connector, String taskName) { - LOG.info("Trying to delete task " + taskName); + _logger.info("Trying to delete task " + taskName); String path = KeyBuilder.connectorTask(_cluster, connector, taskName); if (_zkclient.exists(path) && !_zkclient.deleteRecursive(path)) { // Ignore such failure for now - LOG.warn("Failed to remove connector task: " + path); + _logger.warn("Failed to remove connector task: " + path); } } @@ -421,11 +421,11 @@ public void deleteDatastream(String datastreamName) { String path = KeyBuilder.datastream(_cluster, datastreamName); if (!_zkclient.exists(path)) { - LOG.warn("trying to delete znode of datastream that does not exist. Datastream name: " + datastreamName); + _logger.warn("trying to delete znode of datastream that does not exist. Datastream name: " + datastreamName); return; } - LOG.info("Deleting the zk path {} ", path); + _logger.info("Deleting the zk path {} ", path); // Pipeline could have created more nodes under datastream node. Delete all associated state with deleteRecursive _zkclient.deleteRecursive(path); } @@ -447,7 +447,7 @@ public List getAllInstances() { */ public void touchAllInstanceAssignments() { List allInstances = getAllInstances(); - LOG.info("About to touch all instances' assignments node. instances = {}", allInstances); + _logger.info("About to touch all instances' assignments node. instances = {}", allInstances); // since all the requests below talk to the same zk server, we don't benefit a lot from parallelism for (String instance : allInstances) { // Ensure that the instance and instance/Assignment paths are ready before writing the task @@ -459,7 +459,7 @@ public void touchAllInstanceAssignments() { } catch (Exception e) { // we don't need to do an atomic update; if the node gets update by others somehow or get deleted by // leader, it's ok to ignore the failure - LOG.warn("Failed to touch the assignment node for instance " + instance, e); + _logger.warn("Failed to touch the assignment node for instance " + instance, e); } } } @@ -521,7 +521,7 @@ private void loadAllDatastreamTasks() { * have been cleaned up after each task reassignment. */ public Map> getAllAssignedDatastreamTasks() { - LOG.info("All live tasks: " + _liveTaskMap); + _logger.info("All live tasks: " + _liveTaskMap); return new HashMap<>(_liveTaskMap); } @@ -551,7 +551,7 @@ public DatastreamTaskImpl getAssignedDatastreamTask(String instance, String task // instance. In this case, we would get such exception. This is tolerable // because we should be receiving another AssignmentChange event right after // then we can dispatch the tasks based on the latest assignment data. - LOG.warn("ZNode does not exist for instance={}, task={}, ignoring the task.", instance, taskName); + _logger.warn("ZNode does not exist for instance={}, task={}, ignoring the task.", instance, taskName); return null; } } @@ -570,7 +570,7 @@ private String parseTaskPrefix(String datastreamTaskName) { * If any one failed, RuntimeException will be thrown. */ private void addTaskNodes(String instance, DatastreamTaskImpl task) { - LOG.info("Adding Task Node: " + instance + ", task: " + task); + _logger.info("Adding Task Node: " + instance + ", task: " + task); String name = task.getDatastreamTaskName(); // Must add task node under connector first because as soon as we update the @@ -594,7 +594,7 @@ private void addTaskNodes(String instance, DatastreamTaskImpl task) { } catch (IOException e) { // This should never happen String errorMessage = "Failed to serialize task into JSON."; - ErrorLogger.logAndThrowDatastreamRuntimeException(LOG, errorMessage, e); + ErrorLogger.logAndThrowDatastreamRuntimeException(_logger, errorMessage, e); } // Ensure that the instance and instance/Assignment paths are ready before writing the task @@ -603,11 +603,11 @@ private void addTaskNodes(String instance, DatastreamTaskImpl task) { String created = _zkclient.create(instancePath, json, CreateMode.PERSISTENT); if (created != null && !created.isEmpty()) { - LOG.info("create zookeeper node: " + instancePath); + _logger.info("create zookeeper node: " + instancePath); } else { // FIXME: we should do some error handling String errorMessage = "failed to create zookeeper node: " + instancePath; - ErrorLogger.logAndThrowDatastreamRuntimeException(LOG, errorMessage, null); + ErrorLogger.logAndThrowDatastreamRuntimeException(_logger, errorMessage, null); } } @@ -620,7 +620,7 @@ private void addTaskNodes(String instance, DatastreamTaskImpl task) { * If either failed, a RuntimeException will be thrown. */ private void removeTaskNodes(String instance, String name) { - LOG.info("Removing Task Node: " + instance + ", task: " + name); + _logger.info("Removing Task Node: " + instance + ", task: " + name); String instancePath = KeyBuilder.instanceAssignment(_cluster, instance, name); // NOTE: we can't remove the connector task node since it has the state (checkpoint/lock). @@ -663,7 +663,7 @@ public void updateAllAssignments(Map> assignmentsBy for (String instance : nodesToAdd.keySet()) { Set added = nodesToAdd.get(instance); if (added.size() > 0) { - LOG.info("Instance: {}, adding assignments: {}", instance, added); + _logger.info("Instance: {}, adding assignments: {}", instance, added); for (String name : added) { addTaskNodes(instance, (DatastreamTaskImpl) assignmentsMap.get(name)); } @@ -674,7 +674,7 @@ public void updateAllAssignments(Map> assignmentsBy for (String instance : nodesToRemove.keySet()) { Set removed = nodesToRemove.get(instance); if (removed.size() > 0) { - LOG.info("Instance: {}, removing assignments: {}", instance, removed); + _logger.info("Instance: {}, removing assignments: {}", instance, removed); for (String name : removed) { removeTaskNodes(instance, name); } @@ -738,17 +738,17 @@ private String createLiveInstanceNode() { try { _hostname = InetAddress.getLocalHost().getHostName(); } catch (UnknownHostException uhe) { - LOG.error(uhe.getMessage()); + _logger.error(uhe.getMessage()); } // // create an ephemeral sequential node under /{cluster}/liveinstances for leader election // String electionPath = KeyBuilder.liveInstance(_cluster, ""); - LOG.info("Creating ephemeral node on path: {}", electionPath); + _logger.info("Creating ephemeral node on path: {}", electionPath); String liveInstancePath = _zkclient.create(electionPath, _hostname, CreateMode.EPHEMERAL_SEQUENTIAL); _liveInstanceName = liveInstancePath.substring(electionPath.length()); - LOG.info("Getting live instance name as: {}", _liveInstanceName); + _logger.info("Getting live instance name as: {}", _liveInstanceName); // // create instance node /{cluster}/instance/{instanceName} for keeping instance @@ -782,7 +782,7 @@ public void addConnectorType(String connectorType) { public void zkSaveInstanceError(String message) { String path = KeyBuilder.instanceErrors(_cluster, _instanceName); if (!_zkclient.exists(path)) { - LOG.warn("failed to persist instance error because znode does not exist:" + path); + _logger.warn("failed to persist instance error because znode does not exist:" + path); return; } @@ -795,9 +795,9 @@ public void zkSaveInstanceError(String message) { if (numberOfExistingErrors < 10) { try { String errorNode = _zkclient.createPersistentSequential(path + "/", message); - LOG.info("created error node at: " + errorNode); + _logger.info("created error node at: " + errorNode); } catch (RuntimeException ex) { - LOG.error("failed to create instance error node: " + path); + _logger.error("failed to create instance error node: " + path); } } } @@ -840,14 +840,14 @@ public void cleanupDeadInstanceAssignments(List instances) { List deadInstances = getAllInstances(); deadInstances.removeAll(instances); if (deadInstances.size() > 0) { - LOG.info("Cleaning up assignments for dead instances: " + deadInstances); + _logger.info("Cleaning up assignments for dead instances: " + deadInstances); for (String instance : deadInstances) { String path = KeyBuilder.instance(_cluster, instance); - LOG.info("Deleting zk path recursively: " + path); + _logger.info("Deleting zk path recursively: " + path); if (!_zkclient.deleteRecursive(path)) { // Ignore such failure for now - LOG.warn("Failed to remove zk path: {} Very likely that the zk node doesn't exist anymore", path); + _logger.warn("Failed to remove zk path: {} Very likely that the zk node doesn't exist anymore", path); } _liveTaskMap.remove(instance); @@ -870,7 +870,7 @@ public void cleanupOldUnusedTasks(Map> previousAssig previousAssignmentByInstance.values().stream().flatMap(Collection::stream).collect(Collectors.toSet()); List unusedTasks = oldTasks.stream().filter(x -> !newTasks.contains(x)).collect(Collectors.toList()); - LOG.info("Deleting the unused tasks {} ", unusedTasks); + _logger.info("Deleting the unused tasks {} ", unusedTasks); // Delete the connector tasks. unusedTasks.forEach(t -> deleteConnectorTask(t.getConnectorType(), t.getDatastreamTaskName())); @@ -895,7 +895,7 @@ private void waitForTaskRelease(DatastreamTask task, long timeoutMs, String lock busyLatch.await(timeoutMs, TimeUnit.MILLISECONDS); } catch (InterruptedException e) { String errorMsg = "Unexpectedly interrupted during task acquire."; - ErrorLogger.logAndThrowDatastreamRuntimeException(LOG, errorMsg, e); + ErrorLogger.logAndThrowDatastreamRuntimeException(_logger, errorMsg, e); } finally { _zkclient.unsubscribeChildChanges(lockRootPath, listener); } @@ -918,7 +918,7 @@ public void acquireTask(DatastreamTaskImpl task, Duration timeout) { if (_zkclient.exists(lockPath)) { owner = _zkclient.ensureReadData(lockPath); if (owner.equals(_instanceName)) { - LOG.info("{} already owns the lock on {}", _instanceName, task); + _logger.info("{} already owns the lock on {}", _instanceName, task); return; } @@ -927,11 +927,11 @@ public void acquireTask(DatastreamTaskImpl task, Duration timeout) { if (!_zkclient.exists(lockPath)) { _zkclient.createEphemeral(lockPath, _instanceName); - LOG.info("{} successfully acquired the lock on {}", _instanceName, task); + _logger.info("{} successfully acquired the lock on {}", _instanceName, task); } else { String msg = String.format("%s failed to acquire task %s in %dms, current owner: %s", _instanceName, task, timeout.toMillis(), owner); - ErrorLogger.logAndThrowDatastreamRuntimeException(LOG, msg, null); + ErrorLogger.logAndThrowDatastreamRuntimeException(_logger, msg, null); } } @@ -973,7 +973,7 @@ public void cleanUpPartitionMovement(String connectorType, String datastreamGrou } } } - LOG.info("clean up Target assignment info for {}", datastreamGroupName); + _logger.info("clean up Target assignment info for {}", datastreamGroupName); } /** @@ -1028,14 +1028,14 @@ public Map> getPartitionMovement(String connectorType, Strin // Map the hostname to correct Zookeeper instance name if (hostInstanceMap.containsKey(assignment.getTargetHost()) && assignment.getPartitionNames() != null) { - LOG.info("Added assignment {} {}", assignment.getTargetHost(), hostPartitions); + _logger.info("Added assignment {} {}", assignment.getTargetHost(), hostPartitions); if (result.containsKey(hostInstanceMap.get(assignment.getTargetHost()))) { result.get(hostInstanceMap.get(assignment.getTargetHost())).addAll(hostPartitions); } else { result.put(hostInstanceMap.get(assignment.getTargetHost()), new HashSet<>(hostPartitions)); } } else { - LOG.warn("assignment target host {} not found from the live instances {}", assignment.getTargetHost(), hostInstanceMap.keySet()); + _logger.warn("assignment target host {} not found from the live instances {}", assignment.getTargetHost(), hostInstanceMap.keySet()); } } } @@ -1050,18 +1050,18 @@ public Map> getPartitionMovement(String connectorType, Strin public void releaseTask(DatastreamTaskImpl task) { String lockPath = KeyBuilder.datastreamTaskLock(_cluster, task.getConnectorType(), task.getDatastreamTaskName()); if (!_zkclient.exists(lockPath)) { - LOG.info("There is no lock on {}", task); + _logger.info("There is no lock on {}", task); return; } String owner = _zkclient.ensureReadData(lockPath); if (!owner.equals(_instanceName)) { - LOG.warn("{} does not have the lock on {}", _instanceName, task); + _logger.warn("{} does not have the lock on {}", _instanceName, task); return; } _zkclient.delete(lockPath); - LOG.info("{} successfully released the lock on {}", _instanceName, task); + _logger.info("{} successfully released the lock on {}", _instanceName, task); } private Map getHostInstanceMap() { @@ -1144,7 +1144,7 @@ public class ZkBackedDMSDatastreamList implements IZkChildListener, IZkDataListe public ZkBackedDMSDatastreamList() { _path = KeyBuilder.datastreams(_cluster); _zkclient.ensurePath(KeyBuilder.datastreams(_cluster)); - LOG.info("ZkBackedDMSDatastreamList::Subscribing to the changes under the path " + _path); + _logger.info("ZkBackedDMSDatastreamList::Subscribing to the changes under the path " + _path); _zkclient.subscribeChildChanges(_path, this); _zkclient.subscribeDataChanges(_path, this); } @@ -1153,13 +1153,13 @@ public ZkBackedDMSDatastreamList() { * Unsubscribe from all datastream changes in the cluster */ public void close() { - LOG.info("ZkBackedDMSDatastreamList::Unsubscribing to the changes under the path " + _path); + _logger.info("ZkBackedDMSDatastreamList::Unsubscribing to the changes under the path " + _path); _zkclient.unsubscribeChildChanges(_path, this); } @Override public synchronized void handleChildChange(String parentPath, List currentChildren) throws Exception { - LOG.info(String.format("ZkBackedDMSDatastreamList::Received Child change notification on the datastream list" + _logger.info(String.format("ZkBackedDMSDatastreamList::Received Child change notification on the datastream list" + "parentPath %s,children %s", parentPath, currentChildren)); if (_listener != null && ZkAdapter.this.isLeader()) { _listener.onDatastreamAddOrDrop(); @@ -1170,7 +1170,7 @@ public synchronized void handleChildChange(String parentPath, List curre // due to datastream add or delete. @Override public void handleDataChange(String dataPath, Object data) throws Exception { - LOG.info("ZkBackedDMSDatastreamList::Received Data change notification on the path {}, data {}.", + _logger.info("ZkBackedDMSDatastreamList::Received Data change notification on the path {}, data {}.", dataPath, data.toString()); if (_listener != null) { _listener.onDatastreamAddOrDrop(); @@ -1212,7 +1212,7 @@ public class ZkBackedLiveInstanceListProvider implements IZkChildListener { public ZkBackedLiveInstanceListProvider() { _path = KeyBuilder.liveInstances(_cluster); _zkclient.ensurePath(_path); - LOG.info("ZkBackedLiveInstanceListProvider::Subscribing to the under the path " + _path); + _logger.info("ZkBackedLiveInstanceListProvider::Subscribing to the under the path " + _path); _zkclient.subscribeChildChanges(_path, this); _liveInstances = getLiveInstanceNames(_zkclient.getChildren(_path)); } @@ -1225,7 +1225,7 @@ private List getLiveInstanceNames(List nodes) { String hostname = _zkclient.ensureReadData(KeyBuilder.liveInstance(_cluster, n)); if (hostname != null) { // hostname can be null if a node dies immediately after reading all live instances - LOG.error("Node {} is dead. Likely cause it dies after reading list of nodes.", n); + _logger.error("Node {} is dead. Likely cause it dies after reading list of nodes.", n); liveInstances.add(formatZkInstance(hostname, n)); } } @@ -1236,7 +1236,7 @@ private List getLiveInstanceNames(List nodes) { * Unsubscribe from all live instance changes in the cluster */ public void close() { - LOG.info("ZkBackedLiveInstanceListProvider::Unsubscribing to the under the path " + _path); + _logger.info("ZkBackedLiveInstanceListProvider::Unsubscribing to the under the path " + _path); _zkclient.unsubscribeChildChanges(_path, this); } @@ -1246,7 +1246,7 @@ public List getLiveInstances() { @Override public void handleChildChange(String parentPath, List currentChildren) throws Exception { - LOG.info(String.format( + _logger.info(String.format( "ZkBackedLiveInstanceListProvider::Received Child change notification on the instances list " + "parentPath %s,children %s", parentPath, currentChildren)); @@ -1283,7 +1283,7 @@ public void handleStateChanged(Watcher.Event.KeeperState state) { @Override public void handleNewSession() { - LOG.info("ZkStateChangeListener::A new session established after the earlier session was expired."); + _logger.info("ZkStateChangeListener::A new session established after the earlier session was expired."); initializeSession(); } @@ -1306,7 +1306,7 @@ public class ZkBackedTaskListProvider implements IZkChildListener, IZkDataListen */ public ZkBackedTaskListProvider(String cluster, String instanceName) { _path = KeyBuilder.instanceAssignments(cluster, instanceName); - LOG.info("ZkBackedTaskListProvider::Subscribing to the changes under the path " + _path); + _logger.info("ZkBackedTaskListProvider::Subscribing to the changes under the path " + _path); _zkclient.subscribeChildChanges(_path, this); _zkclient.subscribeDataChanges(_path, this); } @@ -1315,13 +1315,13 @@ public ZkBackedTaskListProvider(String cluster, String instanceName) { * Unsubscribe to all changes to the task assignment for this instance. */ public void close() { - LOG.info("ZkBackedTaskListProvider::Unsubscribing to the changes under the path " + _path); + _logger.info("ZkBackedTaskListProvider::Unsubscribing to the changes under the path " + _path); _zkclient.unsubscribeChildChanges(KeyBuilder.instanceAssignments(_cluster, _instanceName), this); } @Override public synchronized void handleChildChange(String parentPath, List currentChildren) throws Exception { - LOG.info(String.format( + _logger.info(String.format( "ZkBackedTaskListProvider::Received Child change notification on the datastream task list " + "parentPath %s,children %s", parentPath, currentChildren)); if (_listener != null) { @@ -1333,7 +1333,7 @@ public synchronized void handleChildChange(String parentPath, List curre // updated, but the list of tasks may remain the same @Override public void handleDataChange(String dataPath, Object data) throws Exception { - LOG.info("ZkBackedTaskListProvider::Received Data change notification on the path {}, data {}.", dataPath, data); + _logger.info("ZkBackedTaskListProvider::Received Data change notification on the path {}, data {}.", dataPath, data); if (_listener != null && data != null && !data.toString().isEmpty()) { // only care about the data change when there is an update in the data node _listener.onDatastreamUpdate(); @@ -1358,7 +1358,7 @@ public ZkTargetAssignmentProvider(Set connectorTypes) { for (String connectorType : connectorTypes) { String path = KeyBuilder.getTargetAssignmentBase(_cluster, connectorType); _zkclient.subscribeDataChanges(path, this); - LOG.info("ZkTargetAssignmentProvider::Subscribing to the changes under the path " + path); + _logger.info("ZkTargetAssignmentProvider::Subscribing to the changes under the path " + path); } _listenedConnectors.addAll(connectorTypes); } @@ -1370,7 +1370,7 @@ public void addListener(String connectorType) { if (!_listenedConnectors.contains(connectorType)) { String path = KeyBuilder.getTargetAssignmentBase(_cluster, connectorType); _zkclient.subscribeDataChanges(path, this); - LOG.info("ZkTargetAssignmentProvider::Subscribing to the changes under the path " + path); + _logger.info("ZkTargetAssignmentProvider::Subscribing to the changes under the path " + path); _listenedConnectors.add(connectorType); } } @@ -1382,13 +1382,13 @@ public void close() { for (String connectorType : _listenedConnectors) { String path = KeyBuilder.getTargetAssignmentBase(_cluster, connectorType); _zkclient.unsubscribeDataChanges(path, this); - LOG.info("ZkTargetAssignmentProvider::Unsubscribing to the changes under the path " + path); + _logger.info("ZkTargetAssignmentProvider::Unsubscribing to the changes under the path " + path); } } @Override public void handleDataChange(String dataPath, Object data) throws Exception { - LOG.info("ZkTargetAssignmentProvider::Received Data change notification on the path {}, data {}.", dataPath, data); + _logger.info("ZkTargetAssignmentProvider::Received Data change notification on the path {}, data {}.", dataPath, data); if (_listener != null && data != null && !data.toString().isEmpty()) { // data consists of the timestamp when partition movement is triggered from the client _listener.onPartitionMovement(Long.valueOf(data.toString())); diff --git a/datastream-server/src/test/java/com/linkedin/datastream/server/assignment/TestBroadcastStrategy.java b/datastream-server/src/test/java/com/linkedin/datastream/server/assignment/TestBroadcastStrategy.java index b4f776eb5..4cf24ee60 100644 --- a/datastream-server/src/test/java/com/linkedin/datastream/server/assignment/TestBroadcastStrategy.java +++ b/datastream-server/src/test/java/com/linkedin/datastream/server/assignment/TestBroadcastStrategy.java @@ -37,7 +37,7 @@ */ public class TestBroadcastStrategy { - private static final Logger LOG = LoggerFactory.getLogger(TestBroadcastStrategy.class.getName()); + private static final Logger _logger = LoggerFactory.getLogger(TestBroadcastStrategy.class.getName()); @Test public void testCreatesAssignmentAcrossAllInstances() { @@ -167,8 +167,8 @@ public void testDontCreateNewTasksWhenCalledSecondTime() { Set oldAssignmentTasks = assignment.get(instance); Set newAssignmentTasks = newAssignment.get(instance); Assert.assertEquals(oldAssignmentTasks.size(), newAssignmentTasks.size()); - LOG.info("New assignment : " + newAssignmentTasks); - LOG.info("Old assignment : " + oldAssignmentTasks); + _logger.info("New assignment : " + newAssignmentTasks); + _logger.info("Old assignment : " + oldAssignmentTasks); Assert.assertTrue(newAssignmentTasks.containsAll(oldAssignmentTasks)); } @@ -181,8 +181,8 @@ public void testDontCreateNewTasksWhenCalledSecondTime() { Set oldAssignmentTasks = assignment.get(instance); Set newAssignmentTasks = newAssignment.get(instance); Assert.assertEquals(oldAssignmentTasks.size(), newAssignmentTasks.size()); - LOG.info("New assignment : " + newAssignmentTasks); - LOG.info("Old assignment : " + oldAssignmentTasks); + _logger.info("New assignment : " + newAssignmentTasks); + _logger.info("Old assignment : " + oldAssignmentTasks); Assert.assertTrue(newAssignmentTasks.containsAll(oldAssignmentTasks)); } } diff --git a/datastream-server/src/test/java/com/linkedin/datastream/server/assignment/TestStickyMulticastStrategy.java b/datastream-server/src/test/java/com/linkedin/datastream/server/assignment/TestStickyMulticastStrategy.java index fbb83ad20..50545ff79 100644 --- a/datastream-server/src/test/java/com/linkedin/datastream/server/assignment/TestStickyMulticastStrategy.java +++ b/datastream-server/src/test/java/com/linkedin/datastream/server/assignment/TestStickyMulticastStrategy.java @@ -40,7 +40,7 @@ */ public class TestStickyMulticastStrategy { - private static final Logger LOG = LoggerFactory.getLogger(TestBroadcastStrategy.class.getName()); + private static final Logger _logger = LoggerFactory.getLogger(TestBroadcastStrategy.class.getName()); @Test public void testCreateAssignmentAcrossAllInstances() { @@ -248,8 +248,8 @@ public void testDontCreateNewTasksWhenCalledSecondTime() { Set oldAssignmentTasks = assignment.get(instance); Set newAssignmentTasks = newAssignment.get(instance); Assert.assertEquals(oldAssignmentTasks.size(), newAssignmentTasks.size()); - LOG.info("New assignment : " + newAssignmentTasks); - LOG.info("Old assignment : " + oldAssignmentTasks); + _logger.info("New assignment : " + newAssignmentTasks); + _logger.info("Old assignment : " + oldAssignmentTasks); Assert.assertTrue(newAssignmentTasks.containsAll(oldAssignmentTasks)); } @@ -261,8 +261,8 @@ public void testDontCreateNewTasksWhenCalledSecondTime() { Set oldAssignmentTasks = assignment.get(instance); Set newAssignmentTasks = newAssignment.get(instance); Assert.assertEquals(oldAssignmentTasks.size(), newAssignmentTasks.size()); - LOG.info("New assignment : " + newAssignmentTasks); - LOG.info("Old assignment : " + oldAssignmentTasks); + _logger.info("New assignment : " + newAssignmentTasks); + _logger.info("Old assignment : " + oldAssignmentTasks); Assert.assertTrue(newAssignmentTasks.containsAll(oldAssignmentTasks)); } } diff --git a/datastream-server/src/test/java/com/linkedin/datastream/server/assignment/TestStickyPartitionAssignment.java b/datastream-server/src/test/java/com/linkedin/datastream/server/assignment/TestStickyPartitionAssignment.java index 3f0ada3b1..8411d74d6 100644 --- a/datastream-server/src/test/java/com/linkedin/datastream/server/assignment/TestStickyPartitionAssignment.java +++ b/datastream-server/src/test/java/com/linkedin/datastream/server/assignment/TestStickyPartitionAssignment.java @@ -43,7 +43,7 @@ */ public class TestStickyPartitionAssignment { - private static final Logger LOG = LoggerFactory.getLogger(TestStickyPartitionAssignment.class.getName()); + private static final Logger _logger = LoggerFactory.getLogger(TestStickyPartitionAssignment.class.getName()); @Test public void testCreateAssignmentAcrossAllTasks() { diff --git a/datastream-server/src/test/java/com/linkedin/datastream/server/zk/TestZkAdapter.java b/datastream-server/src/test/java/com/linkedin/datastream/server/zk/TestZkAdapter.java index 2d82def17..c13869452 100644 --- a/datastream-server/src/test/java/com/linkedin/datastream/server/zk/TestZkAdapter.java +++ b/datastream-server/src/test/java/com/linkedin/datastream/server/zk/TestZkAdapter.java @@ -44,7 +44,7 @@ * Tests for {@link ZkAdapter} */ public class TestZkAdapter { - private static final Logger LOG = LoggerFactory.getLogger(TestZkAdapter.class); + private static final Logger _logger = LoggerFactory.getLogger(TestZkAdapter.class); private static final int ZK_WAIT_IN_MS = 500; private final String defaultTransportProviderName = "test"; @@ -590,14 +590,14 @@ public void testTaskAcquireReleaseOwnerUncleanShutdown() throws Exception { ZkAdapter adapter2 = createZkAdapter(testCluster); adapter2.connect(); - LOG.info("Acquire from instance2 should fail"); + _logger.info("Acquire from instance2 should fail"); task.setZkAdapter(adapter2); Assert.assertTrue(expectException(() -> task.acquire(Duration.ofMillis(100)), true)); - LOG.info("Disconnecting instance1"); + _logger.info("Disconnecting instance1"); adapter1.disconnect(); - LOG.info("instance2 should be able to acquire after instance1's disconnection"); + _logger.info("instance2 should be able to acquire after instance1's disconnection"); Assert.assertTrue(expectException(() -> task.acquire(timeout), false)); } @@ -623,34 +623,34 @@ public void testTaskAcquireReleaseOwnerUncleanBounce() throws Exception { tasks.add(task); updateInstanceAssignment(adapter1, adapter1.getInstanceName(), tasks); - LOG.info("Acquire from instance1 should succeed"); + _logger.info("Acquire from instance1 should succeed"); Assert.assertTrue(expectException(() -> task.acquire(timeout), false)); ZkAdapter adapter2 = createZkAdapter(testCluster); adapter2.connect(); - LOG.info("Acquire from instance2 should fail"); + _logger.info("Acquire from instance2 should fail"); task.setZkAdapter(adapter2); Assert.assertTrue(expectException(() -> task.acquire(Duration.ofMillis(100)), true)); - LOG.info("Disconnecting instance1"); + _logger.info("Disconnecting instance1"); String instanceName1 = adapter1.getInstanceName(); adapter1.disconnect(); - LOG.info("Waiting up to 5 seconds for instance2 to become leader"); + _logger.info("Waiting up to 5 seconds for instance2 to become leader"); PollUtils.poll(adapter2::isLeader, 50, 5000); - LOG.info("Wait up to 5 seconds for the ephemeral node to be gone"); + _logger.info("Wait up to 5 seconds for the ephemeral node to be gone"); PollUtils.poll(() -> !adapter2.getLiveInstances().contains(instanceName1), 50, 5000); - LOG.info("Reconnecting instance1 should get a new instanceName"); + _logger.info("Reconnecting instance1 should get a new instanceName"); adapter1.connect(); Assert.assertNotEquals(instanceName1, adapter1.getInstanceName()); - LOG.info("instance2 should be able to acquire since old instance1 is dead"); + _logger.info("instance2 should be able to acquire since old instance1 is dead"); Assert.assertTrue(expectException(() -> task.acquire(timeout), false)); - LOG.info("Acquire from the new instance1 should fail"); + _logger.info("Acquire from the new instance1 should fail"); task.setZkAdapter(adapter1); Assert.assertTrue(expectException(() -> task.acquire(Duration.ofMillis(100)), true)); } @@ -676,7 +676,7 @@ public void testTaskAcquireWithDependencies() throws Exception { tasks.add(task1); updateInstanceAssignment(adapter1, adapter1.getInstanceName(), tasks); - LOG.info("Acquire from instance1 should succeed"); + _logger.info("Acquire from instance1 should succeed"); Assert.assertTrue(expectException(() -> task1.acquire(Duration.ofMillis(100)), false)); //The task2 cannot be acquired as the dependencies are not released diff --git a/datastream-testcommon/src/main/java/com/linkedin/datastream/connectors/TestEventProducingConnector.java b/datastream-testcommon/src/main/java/com/linkedin/datastream/connectors/TestEventProducingConnector.java index c819c7447..2627a94b7 100644 --- a/datastream-testcommon/src/main/java/com/linkedin/datastream/connectors/TestEventProducingConnector.java +++ b/datastream-testcommon/src/main/java/com/linkedin/datastream/connectors/TestEventProducingConnector.java @@ -51,7 +51,7 @@ public class TestEventProducingConnector implements Connector { public static final String CFG_SLEEP_BETWEEN_SEND_MS = "sleepBetweenSendMs"; public static final String CFG_NUM_PRODUCER_THREADS = "numProducerThreads"; - private static final Logger LOG = LoggerFactory.getLogger(TestEventProducingConnector.class); + private static final Logger _logger = LoggerFactory.getLogger(TestEventProducingConnector.class); private static final int DEFAULT_MESSAGE_SIZE_BYTES = 100; private static final long DEFAULT_SLEEP_BETWEEN_SEND_MS = 1000; private static final int DEFAULT_NUM_PRODUCER_THREADS = 10; @@ -79,19 +79,19 @@ public TestEventProducingConnector(Properties props) { try { _hostName = InetAddress.getLocalHost().getHostName(); } catch (UnknownHostException e) { - LOG.error("getLocalhost threw an exception", e); + _logger.error("getLocalhost threw an exception", e); throw new DatastreamRuntimeException(e); } } @Override public synchronized void start(CheckpointProvider checkpointProvider) { - LOG.info("Start called."); + _logger.info("Start called."); } @Override public synchronized void stop() { - LOG.info("Stop called."); + _logger.info("Stop called."); // Flush the producers so that the data is sent and water marks move forward. _tasksAssigned.keySet().forEach(x -> x.getEventProducer().flush()); @@ -104,7 +104,7 @@ public synchronized void stop() { @Override public synchronized void onAssignmentChange(List tasks) { - LOG.info("onAssignmentChange called with tasks {}, existing assignment {}", tasks, _tasksAssigned.keySet()); + _logger.info("onAssignmentChange called with tasks {}, existing assignment {}", tasks, _tasksAssigned.keySet()); for (DatastreamTask task : tasks) { if (_tasksAssigned.containsKey(task)) { continue; @@ -119,7 +119,7 @@ public synchronized void onAssignmentChange(List tasks) { _tasksAssigned.entrySet().stream().filter(x -> !tasks.contains(x.getKey())).collect(Collectors.toList()); tasksToRemove.forEach((x) -> { - LOG.info("Task {} is reassigned from the current instance, cancelling the producer", x.getKey()); + _logger.info("Task {} is reassigned from the current instance, cancelling the producer", x.getKey()); x.getValue().cancel(true); while (!x.getValue().isDone()) { Thread.yield(); @@ -127,7 +127,7 @@ public synchronized void onAssignmentChange(List tasks) { _tasksAssigned.remove(x.getKey()); - LOG.info("Producer corresponding to the task {} has been stopped", x.getKey()); + _logger.info("Producer corresponding to the task {} has been stopped", x.getKey()); }); } @@ -135,7 +135,7 @@ private void executeTask(DatastreamTask task) { int counter = 0; Instant startTime = Instant.now(); try { - LOG.info("Starting the producer for task " + task); + _logger.info("Starting the producer for task " + task); Datastream datastream = task.getDatastreams().get(0); int messageSize = _messageSize; long sleepBetweenSendMs = _sleepBetweenSendMs; @@ -143,7 +143,7 @@ private void executeTask(DatastreamTask task) { long index = Long.parseLong(StringUtils.isBlank(checkpoint) ? "0" : checkpoint) + 1; - LOG.info("Checkpoint string = " + checkpoint + " index = " + index); + _logger.info("Checkpoint string = " + checkpoint + " index = " + index); if (datastream.hasMetadata()) { StringMap dsMetadata = datastream.getMetadata(); @@ -160,8 +160,8 @@ private void executeTask(DatastreamTask task) { DatastreamProducerRecord record = createDatastreamEvent(index, messageSize, partition); task.getEventProducer().send(record, (metadata, exception) -> { if (exception != null) { - LOG.info("metadata is " + metadata.toString()); - LOG.error("Send failed for event " + metadata.getCheckpoint(), exception); + _logger.info("metadata is " + metadata.toString()); + _logger.error("Send failed for event " + metadata.getCheckpoint(), exception); } }); counter++; @@ -169,7 +169,7 @@ private void executeTask(DatastreamTask task) { Thread.sleep(sleepBetweenSendMs); } catch (InterruptedException e) { String msg = "Producer thread is interrupted. Stopping the producer for task " + task; - LOG.error(msg, e); + _logger.error(msg, e); throw new DatastreamRuntimeException(msg, e); } } @@ -177,15 +177,15 @@ private void executeTask(DatastreamTask task) { index++; } } catch (RuntimeException ex) { - LOG.error("Producer thread threw exception, Stopping event producer for task " + task, ex); + _logger.error("Producer thread threw exception, Stopping event producer for task " + task, ex); } catch (Exception ex) { - LOG.error("Producer thread threw exception, Stopping event producer for task " + task, ex); + _logger.error("Producer thread threw exception, Stopping event producer for task " + task, ex); } Duration elapseTime = Duration.between(startTime, Instant.now()); double elapseSeconds = ((double) elapseTime.toNanos()) / NANOSECONDS.convert(1, SECONDS); - LOG.info("XXXXXX {Task: " + task + "} Total Time: " + elapseSeconds + " total Messages: " + counter); - LOG.info("XXXXXX {Task: " + task + "} QPS: " + (counter / elapseSeconds)); + _logger.info("XXXXXX {Task: " + task + "} Total Time: " + elapseSeconds + " total Messages: " + counter); + _logger.info("XXXXXX {Task: " + task + "} QPS: " + (counter / elapseSeconds)); } private DatastreamProducerRecord createDatastreamEvent(long eventIndex, int messageSize, int partition) { @@ -208,7 +208,7 @@ private DatastreamProducerRecord createDatastreamEvent(long eventIndex, int mess @Override public void initializeDatastream(Datastream stream, List allDatastreams) throws DatastreamValidationException { - LOG.info("initialize called for datastream {} with datastreams {}", stream, allDatastreams); + _logger.info("initialize called for datastream {} with datastreams {}", stream, allDatastreams); } @Override diff --git a/datastream-testcommon/src/main/java/com/linkedin/datastream/server/EmbeddedDatastreamCluster.java b/datastream-testcommon/src/main/java/com/linkedin/datastream/server/EmbeddedDatastreamCluster.java index 84ddace8d..24cf3dec0 100644 --- a/datastream-testcommon/src/main/java/com/linkedin/datastream/server/EmbeddedDatastreamCluster.java +++ b/datastream-testcommon/src/main/java/com/linkedin/datastream/server/EmbeddedDatastreamCluster.java @@ -32,7 +32,7 @@ public class EmbeddedDatastreamCluster { public static final String CONFIG_ZK_CONNECT = "zookeeper.connect"; public static final String BOOTSTRAP_SERVERS_CONFIG = "bootstrap.servers"; - private static final Logger LOG = LoggerFactory.getLogger(EmbeddedDatastreamCluster.class); + private static final Logger _logger = LoggerFactory.getLogger(EmbeddedDatastreamCluster.class); private static final String KAFKA_TRANSPORT_FACTORY = "com.linkedin.datastream.kafka.KafkaTransportProviderAdminFactory"; private static final long SERVER_INIT_TIMEOUT_MS = 60000; // 1 minute @@ -235,7 +235,7 @@ private void prepareStartup() throws IOException { public void startupServer(int index) throws IOException, DatastreamException { Validate.isTrue(index >= 0, "Server index out of bound: " + index); if (index < _servers.size() && _servers.get(index) != null) { - LOG.warn("Server[{}] already exists, skipping.", index); + _logger.warn("Server[{}] already exists, skipping.", index); return; } @@ -248,7 +248,7 @@ public void startupServer(int index) throws IOException, DatastreamException { // Update HTTP port in case it is lazily bound _datastreamPorts.set(index, server.getHttpPort()); - LOG.info("DatastreamServer[{}] started at port={}.", index, server.getHttpPort()); + _logger.info("DatastreamServer[{}] started at port={}.", index, server.getHttpPort()); } /** @@ -271,7 +271,7 @@ public void startup() throws IOException, DatastreamException { public void shutdownServer(int index) { Validate.isTrue(index >= 0 && index < _servers.size(), "Server index out of bound: " + index); if (_servers.get(index) == null) { - LOG.warn("Server[{}] has not been initialized, skipping.", index); + _logger.warn("Server[{}] has not been initialized, skipping.", index); return; } _servers.get(index).shutdown(); diff --git a/datastream-testcommon/src/main/java/com/linkedin/datastream/server/InMemoryTransportProvider.java b/datastream-testcommon/src/main/java/com/linkedin/datastream/server/InMemoryTransportProvider.java index 06cca06e0..a8be5903c 100644 --- a/datastream-testcommon/src/main/java/com/linkedin/datastream/server/InMemoryTransportProvider.java +++ b/datastream-testcommon/src/main/java/com/linkedin/datastream/server/InMemoryTransportProvider.java @@ -24,7 +24,7 @@ * {@link InMemoryTransportProvider#send(String, DatastreamProducerRecord, SendCallback)} method */ public class InMemoryTransportProvider implements TransportProvider { - private static final Logger LOG = LoggerFactory.getLogger(InMemoryTransportProvider.class); + private static final Logger _logger = LoggerFactory.getLogger(InMemoryTransportProvider.class); private final HashMap _topics = new HashMap<>(); @@ -50,7 +50,7 @@ public synchronized void send(String connectionString, DatastreamProducerRecord String topicName = getTopicName(connectionString); if (!_topics.containsKey(topicName)) { String msg = String.format("Topic %s doesn't exist", topicName); - LOG.error(msg); + _logger.error(msg); throw new DatastreamRuntimeException(msg); } @@ -58,7 +58,7 @@ public synchronized void send(String connectionString, DatastreamProducerRecord _recordsReceived.put(connectionString, new ArrayList<>()); } - LOG.info("Adding record with {} events to topic {}", record.getEvents().size(), topicName); + _logger.info("Adding record with {} events to topic {}", record.getEvents().size(), topicName); _recordsReceived.get(connectionString).add(record); } @@ -90,7 +90,7 @@ public long getTotalEventsReceived(String connectionString) { long totalEventsReceived = _recordsReceived.get(connectionString).stream().mapToInt(r -> r.getEvents().size()).sum(); - LOG.info( + _logger.info( String.format("Total events received for the destination %s is %d", connectionString, totalEventsReceived)); return totalEventsReceived; } diff --git a/datastream-testcommon/src/main/java/com/linkedin/datastream/server/InMemoryTransportProviderAdmin.java b/datastream-testcommon/src/main/java/com/linkedin/datastream/server/InMemoryTransportProviderAdmin.java index 3e479e51a..17a0b64a7 100644 --- a/datastream-testcommon/src/main/java/com/linkedin/datastream/server/InMemoryTransportProviderAdmin.java +++ b/datastream-testcommon/src/main/java/com/linkedin/datastream/server/InMemoryTransportProviderAdmin.java @@ -24,7 +24,7 @@ */ public class InMemoryTransportProviderAdmin implements TransportProviderAdmin { - private static final Logger LOG = LoggerFactory.getLogger(InMemoryTransportProviderAdminFactory.class); + private static final Logger _logger = LoggerFactory.getLogger(InMemoryTransportProviderAdminFactory.class); private static final int DEFAULT_NUMBER_PARTITIONS = 1; private static final InMemoryTransportProvider TRANSPORT_PROVIDER = new InMemoryTransportProvider(); @@ -45,7 +45,7 @@ public synchronized void createDestination(Datastream datastream) { String topicName = datastream.getDestination().getConnectionString(); int numberOfPartitions = datastream.getDestination().getPartitions(); if (TRANSPORT_PROVIDER.getTopics().containsKey(topicName)) { - LOG.warn("Topic {} already exists", topicName); + _logger.warn("Topic {} already exists", topicName); } TRANSPORT_PROVIDER.addTopic(topicName, numberOfPartitions); } @@ -56,7 +56,7 @@ public synchronized void dropDestination(Datastream datastream) { String topicName = datastream.getDestination().getConnectionString(); if (!TRANSPORT_PROVIDER.getTopics().containsKey(topicName)) { String msg = String.format("Topic %s doesn't exist", topicName); - LOG.error(msg); + _logger.error(msg); throw new DatastreamRuntimeException(msg); } } diff --git a/datastream-testcommon/src/main/java/com/linkedin/datastream/server/ListBackedTransportProvider.java b/datastream-testcommon/src/main/java/com/linkedin/datastream/server/ListBackedTransportProvider.java index 5fe2925a2..f64562ef1 100644 --- a/datastream-testcommon/src/main/java/com/linkedin/datastream/server/ListBackedTransportProvider.java +++ b/datastream-testcommon/src/main/java/com/linkedin/datastream/server/ListBackedTransportProvider.java @@ -23,7 +23,7 @@ */ public class ListBackedTransportProvider implements TransportProvider { - private static final Logger LOG = LoggerFactory.getLogger(ListBackedTransportProvider.class); + private static final Logger _logger = LoggerFactory.getLogger(ListBackedTransportProvider.class); /** * Each ProducerRecord holds a list of events of to the same transaction. * This is a list of the event lists of all transactions. @@ -33,7 +33,7 @@ public class ListBackedTransportProvider implements TransportProvider { @Override public synchronized void send(String destination, DatastreamProducerRecord record, SendCallback onComplete) { - LOG.info("send called on destination {} with {} events", destination, record.getEvents().size()); + _logger.info("send called on destination {} with {} events", destination, record.getEvents().size()); _allEvents.add(record.getEvents()); _allRecords.add(record); } diff --git a/datastream-testcommon/src/main/java/com/linkedin/datastream/testutil/BaseKafkaZkTest.java b/datastream-testcommon/src/main/java/com/linkedin/datastream/testutil/BaseKafkaZkTest.java index 3a01eada3..da2429293 100644 --- a/datastream-testcommon/src/main/java/com/linkedin/datastream/testutil/BaseKafkaZkTest.java +++ b/datastream-testcommon/src/main/java/com/linkedin/datastream/testutil/BaseKafkaZkTest.java @@ -13,13 +13,11 @@ import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; -import com.codahale.metrics.MetricRegistry; import kafka.admin.AdminUtils; import kafka.utils.ZkUtils; import com.linkedin.datastream.common.PollUtils; import com.linkedin.datastream.common.zk.ZkClient; -import com.linkedin.datastream.metrics.DynamicMetricsManager; /** @@ -36,7 +34,8 @@ public abstract class BaseKafkaZkTest { @BeforeMethod(alwaysRun = true) public void beforeMethodSetup() throws Exception { - DynamicMetricsManager.createInstance(new MetricRegistry(), getClass().getSimpleName()); + // todo: seems no one uses this metrics manager. + // DynamicMetricsManager.createInstance(new MetricRegistry(), getClass().getSimpleName()); Properties kafkaConfig = new Properties(); // we will disable auto topic creation for tests kafkaConfig.setProperty("auto.create.topics.enable", Boolean.FALSE.toString()); diff --git a/datastream-testcommon/src/main/java/com/linkedin/datastream/testutil/EmbeddedZookeeper.java b/datastream-testcommon/src/main/java/com/linkedin/datastream/testutil/EmbeddedZookeeper.java index 865e64915..873cbfaf4 100644 --- a/datastream-testcommon/src/main/java/com/linkedin/datastream/testutil/EmbeddedZookeeper.java +++ b/datastream-testcommon/src/main/java/com/linkedin/datastream/testutil/EmbeddedZookeeper.java @@ -27,7 +27,7 @@ * Encapsulates a simple standalone ZooKeeper server */ public class EmbeddedZookeeper { - private static final Logger LOG = LoggerFactory.getLogger(EmbeddedZookeeper.class); + private static final Logger _logger = LoggerFactory.getLogger(EmbeddedZookeeper.class); private int _port = -1; private int _tickTime = 500; @@ -82,7 +82,7 @@ public EmbeddedZookeeper(int port, String snapshotDirPath, String logDirPath) th */ public void startup() throws IOException { Validate.isTrue(this._port > 0, "Failed to reserve port for zookeeper server."); - LOG.info("Starting Zookeeper Cluster"); + _logger.info("Starting Zookeeper Cluster"); if (this._snapshotDirPath == null) { this._snapshotDir = FileUtils.constructRandomDirectoryInTempDir("embedded-zk/snapshot-" + this._port); } else { @@ -101,7 +101,7 @@ public void startup() throws IOException { } catch (InterruptedException e) { throw new IOException(e); } - LOG.info("Zookeeper started with ..." + + _logger.info("Zookeeper started with ..." + "\n Port: " + this._port + "\n Snapshot Dir Path: " + this._snapshotDirPath + "\n Log Dir Path: " + this._logDirPath); @@ -190,7 +190,7 @@ private static CommandLine parseArgs(String[] args) { commandLine = parser.parse(options, args); } catch (Exception e) { commandLine = null; - LOG.error(e.getMessage()); + _logger.error(e.getMessage()); } return commandLine; } diff --git a/datastream-testcommon/src/main/java/com/linkedin/datastream/testutil/InMemoryCheckpointProvider.java b/datastream-testcommon/src/main/java/com/linkedin/datastream/testutil/InMemoryCheckpointProvider.java index 64252a92e..ea96bd48b 100644 --- a/datastream-testcommon/src/main/java/com/linkedin/datastream/testutil/InMemoryCheckpointProvider.java +++ b/datastream-testcommon/src/main/java/com/linkedin/datastream/testutil/InMemoryCheckpointProvider.java @@ -22,7 +22,7 @@ */ public class InMemoryCheckpointProvider implements CheckpointProvider { - private static final Logger LOG = LoggerFactory.getLogger(InMemoryCheckpointProvider.class); + private static final Logger _logger = LoggerFactory.getLogger(InMemoryCheckpointProvider.class); private final Map> _cpMap = new HashMap<>(); diff --git a/datastream-utils/src/main/java/com/linkedin/datastream/common/FileUtils.java b/datastream-utils/src/main/java/com/linkedin/datastream/common/FileUtils.java index 8bc71c9ad..3711b4f6d 100644 --- a/datastream-utils/src/main/java/com/linkedin/datastream/common/FileUtils.java +++ b/datastream-utils/src/main/java/com/linkedin/datastream/common/FileUtils.java @@ -20,7 +20,7 @@ public class FileUtils { private static final Random RANDOM = new Random(); - private static final Logger LOG = LoggerFactory.getLogger(FileUtils.class.getName()); + private static final Logger _logger = LoggerFactory.getLogger(FileUtils.class.getName()); /** * Constructs a random directory with the prefix in the temp folder @@ -34,7 +34,7 @@ public static File constructRandomDirectoryInTempDir(String dirPrefix) { File file = new File(System.getProperty("java.io.tmpdir"), dirPrefix + RANDOM.nextInt(10000000)); if (!file.mkdirs()) { String errorMessage = "could not create temp directory: " + file.getAbsolutePath(); - ErrorLogger.logAndThrowDatastreamRuntimeException(LOG, errorMessage, null); + ErrorLogger.logAndThrowDatastreamRuntimeException(_logger, errorMessage, null); } file.deleteOnExit(); @@ -53,7 +53,7 @@ public static File constructDirectoryInTempDir(String dirName) { File file = new File(System.getProperty("java.io.tmpdir"), dirName); if (!file.mkdirs()) { String errorMessage = "could not create temp directory: " + file.getAbsolutePath(); - LOG.info(errorMessage); + _logger.info(errorMessage); } return file; diff --git a/datastream-utils/src/main/java/com/linkedin/datastream/common/LogUtils.java b/datastream-utils/src/main/java/com/linkedin/datastream/common/LogUtils.java index 1d0764a97..44f4e22bf 100644 --- a/datastream-utils/src/main/java/com/linkedin/datastream/common/LogUtils.java +++ b/datastream-utils/src/main/java/com/linkedin/datastream/common/LogUtils.java @@ -16,7 +16,7 @@ * Util class for logging-related methods */ public class LogUtils { - private static final Logger LOG = LoggerFactory.getLogger(LogUtils.class.getName()); + private static final Logger _logger = LoggerFactory.getLogger(LogUtils.class.getName()); private static void printNumberRange(StringBuilder stringBuilder, int start, int tail) { if (start == tail) { @@ -58,7 +58,7 @@ public static String logNumberArrayInRange(List list) { stringBuilder.append("]"); return stringBuilder.toString(); } catch (Exception e) { - LOG.error("Failed to generate string for the int list in range", e); + _logger.error("Failed to generate string for the int list in range", e); return list.toString(); } } diff --git a/datastream-utils/src/main/java/com/linkedin/datastream/common/RetryUtils.java b/datastream-utils/src/main/java/com/linkedin/datastream/common/RetryUtils.java index b6a4d3bac..079fa5ee9 100644 --- a/datastream-utils/src/main/java/com/linkedin/datastream/common/RetryUtils.java +++ b/datastream-utils/src/main/java/com/linkedin/datastream/common/RetryUtils.java @@ -17,7 +17,7 @@ */ public class RetryUtils { - private static final Logger LOG = LoggerFactory.getLogger(RetryUtils.class.getName()); + private static final Logger _logger = LoggerFactory.getLogger(RetryUtils.class.getName()); /** * Keep invoking a function until it succeeds or a timeout period elapses @@ -31,7 +31,7 @@ public static T retry(Supplier func, Duration period, Duration timeout) { boolean result = PollUtils.poll(supplier, period.toMillis(), timeout.toMillis()); if (!result) { - LOG.error("Retries exhausted.", supplier.getLastException()); + _logger.error("Retries exhausted.", supplier.getLastException()); throw new RetriesExhaustedException(supplier.getLastException()); } @@ -39,7 +39,7 @@ public static T retry(Supplier func, Duration period, Duration timeout) { } private static class ExceptionTrackingMethodCaller implements BooleanSupplier { - private static final Logger LOG = LoggerFactory.getLogger(ExceptionTrackingMethodCaller.class.getName()); + private static final Logger _logger = LoggerFactory.getLogger(ExceptionTrackingMethodCaller.class.getName()); private final Supplier _func; private U _value; @@ -64,7 +64,7 @@ public boolean getAsBoolean() { } catch (Exception e) { if (e instanceof RetriableException) { _lastException = e; - LOG.info("Method threw a retriable exception.", e); + _logger.info("Method threw a retriable exception.", e); return false; } throw e; diff --git a/datastream-utils/src/main/java/com/linkedin/datastream/common/ThreadTerminationMonitor.java b/datastream-utils/src/main/java/com/linkedin/datastream/common/ThreadTerminationMonitor.java index 14632db87..cb8b697a6 100644 --- a/datastream-utils/src/main/java/com/linkedin/datastream/common/ThreadTerminationMonitor.java +++ b/datastream-utils/src/main/java/com/linkedin/datastream/common/ThreadTerminationMonitor.java @@ -29,7 +29,7 @@ public class ThreadTerminationMonitor { private static final String ABNORMAL_TERMINATIONS = "abnormalTerminations"; private static final String MODULE = ThreadTerminationMonitor.class.getSimpleName(); - private static final Logger LOG = LoggerFactory.getLogger(ThreadTerminationMonitor.class); + private static final Logger _logger = LoggerFactory.getLogger(ThreadTerminationMonitor.class); private static final Thread.UncaughtExceptionHandler OLD_HANDLER; @@ -37,7 +37,7 @@ public class ThreadTerminationMonitor { // Replace the default uncaught exception handler OLD_HANDLER = Thread.getDefaultUncaughtExceptionHandler(); Thread.setDefaultUncaughtExceptionHandler((Thread t, Throwable e) -> { - LOG.error(String.format("Thread %s terminated abnormally", t.getName()), e); + _logger.error(String.format("Thread %s terminated abnormally", t.getName()), e); DynamicMetricsManager.getInstance().createOrUpdateMeter(MODULE, ABNORMAL_TERMINATIONS, 1); // Resume the old behavior if (OLD_HANDLER != null) { diff --git a/datastream-utils/src/main/java/com/linkedin/datastream/common/zk/ZkClient.java b/datastream-utils/src/main/java/com/linkedin/datastream/common/zk/ZkClient.java index 34f2c1053..acf8f1e78 100644 --- a/datastream-utils/src/main/java/com/linkedin/datastream/common/zk/ZkClient.java +++ b/datastream-utils/src/main/java/com/linkedin/datastream/common/zk/ZkClient.java @@ -37,7 +37,7 @@ public class ZkClient extends org.I0Itec.zkclient.ZkClient { public static final int DEFAULT_CONNECTION_TIMEOUT = 60 * 1000; public static final int DEFAULT_SESSION_TIMEOUT = 30 * 1000; - private static final Logger LOG = LoggerFactory.getLogger(ZkClient.class); + private static final Logger _logger = LoggerFactory.getLogger(ZkClient.class); private final ZkSerializer _zkSerializer = new ZKStringSerializer(); private int _zkSessionTimeoutMs = DEFAULT_SESSION_TIMEOUT; @@ -73,16 +73,16 @@ public ZkClient(String zkServers) { @Override public void close() throws ZkInterruptedException { - if (LOG.isTraceEnabled()) { + if (_logger.isTraceEnabled()) { StackTraceElement[] calls = Thread.currentThread().getStackTrace(); - LOG.trace("closing zkclient. callStack: {}", Arrays.asList(calls)); + _logger.trace("closing zkclient. callStack: {}", Arrays.asList(calls)); } getEventLock().lock(); try { if (_connection == null) { return; } - LOG.info("closing zkclient: {}", ((ZkConnection) _connection).getZookeeper()); + _logger.info("closing zkclient: {}", ((ZkConnection) _connection).getZookeeper()); super.close(); } catch (ZkInterruptedException e) { /* @@ -106,7 +106,7 @@ public void close() throws ZkInterruptedException { } } finally { getEventLock().unlock(); - LOG.info("closed zkclient"); + _logger.info("closed zkclient"); } } @@ -118,8 +118,8 @@ public boolean exists(final String path, final boolean watch) { return retryUntilConnected(() -> _connection.exists(path, watch)); } finally { long endT = System.nanoTime(); - if (LOG.isTraceEnabled()) { - LOG.trace("exists, path: {}, time: {} ns", path, (endT - startT)); + if (_logger.isTraceEnabled()) { + _logger.trace("exists, path: {}, time: {} ns", path, (endT - startT)); } } } @@ -132,8 +132,8 @@ public List getChildren(final String path, final boolean watch) { return retryUntilConnected(() -> _connection.getChildren(path, watch)); } finally { long endT = System.nanoTime(); - if (LOG.isTraceEnabled()) { - LOG.trace("getChildren, path: {}, time: {} ns", path, (endT - startT)); + if (_logger.isTraceEnabled()) { + _logger.trace("getChildren, path: {}, time: {} ns", path, (endT - startT)); } } } @@ -173,14 +173,14 @@ public String ensureReadData(final String path, final long timeout) { try { Thread.sleep(nextWait); } catch (InterruptedException e) { - LOG.error("Failed to sleep at retry: {}", counter, e); + _logger.error("Failed to sleep at retry: {}", counter, e); } totalWait += nextWait; content = super.readData(path, false); } if (content == null) { - LOG.warn("Failed to read znode data for path {} within timeout of {} milliseconds", path, timeout); + _logger.warn("Failed to read znode data for path {} within timeout of {} milliseconds", path, timeout); } return content; @@ -206,8 +206,8 @@ protected T readData(final String path, final Stat stat, fina return (T) deserialize(data); } finally { long endT = System.nanoTime(); - if (LOG.isTraceEnabled()) { - LOG.trace("readData, path: {}, time: {} ns", path, (endT - startT)); + if (_logger.isTraceEnabled()) { + _logger.trace("readData, path: {}, time: {} ns", path, (endT - startT)); } } } @@ -224,8 +224,8 @@ public void writeData(final String path, Object data, final int expectedVersion) }); } finally { long endT = System.nanoTime(); - if (LOG.isTraceEnabled()) { - LOG.trace("writeData, path: {}, time: {} ns", path, (endT - startT)); + if (_logger.isTraceEnabled()) { + _logger.trace("writeData, path: {}, time: {} ns", path, (endT - startT)); } } } @@ -243,8 +243,8 @@ public String create(final String path, Object data, final CreateMode mode) thro return retryUntilConnected(() -> _connection.create(path, bytes, mode)); } finally { long endT = System.nanoTime(); - if (LOG.isTraceEnabled()) { - LOG.trace("create, path: {}, time: {} ns", path, (endT - startT)); + if (_logger.isTraceEnabled()) { + _logger.trace("create, path: {}, time: {} ns", path, (endT - startT)); } } } @@ -265,8 +265,8 @@ public boolean delete(final String path) { } } finally { long endT = System.nanoTime(); - if (LOG.isTraceEnabled()) { - LOG.trace("delete, path: {}, time: {} ns", path, (endT - startT)); + if (_logger.isTraceEnabled()) { + _logger.trace("delete, path: {}, time: {} ns", path, (endT - startT)); } } } @@ -305,11 +305,11 @@ public void ensurePath(String path) { continue; } - LOG.info("creating path in zookeeper: {}", p); + _logger.info("creating path in zookeeper: {}", p); try { this.createPersistent(p); } catch (ZkNodeExistsException e) { - LOG.info(e.getMessage()); + _logger.info(e.getMessage()); } } } diff --git a/gradle/dependency-versions.gradle b/gradle/dependency-versions.gradle index 9610b6ee8..5d5983721 100644 --- a/gradle/dependency-versions.gradle +++ b/gradle/dependency-versions.gradle @@ -9,19 +9,18 @@ ext { commonsValidatorVersion = "1.5.1" dbcpVersion = "1.2.2" gelfJVersion = "1.1.16" - googleCloudBigqueryVersion = "1.116.6" - googleCloudStorageVersion = "1.107.0" + googleCloudBigqueryVersion = "1.127.11" + googleCloudStorageVersion = "1.113.12" guavaVersion = "25.0-jre" hadoopCommonVersion= "0.22.0" intellijAnnotationsVersion = "12.0" jacksonVersion = "1.8.5" json2avroVersion = "0.2.9" kafkaClientsVersion = "2.2.1" - kafkaSchemaRegistryVersion = "5.2.1" kafkaSchemaRegistryClientVersion = "5.3.0" kafkaVersion = "2.0.0" log4jVersion = "1.2.17" - metricsCoreVersion = "4.1.0" + metricsCoreVersion = "4.1.17" mockitoVersion = "1.10.19" mssqlJDBCVersion = "8.1.1.jre8-preview" nifiVersion = "1.11.4" @@ -36,4 +35,5 @@ ext { wayfairCryptoVersion = "3.2.0" zkclientVersion = "0.11" zookeeperVersion = "3.4.13" + wayfairMetricsVersion = "5.6.0" } diff --git a/gradle/maven.gradle b/gradle/maven.gradle index cbc21f716..4a3359403 100644 --- a/gradle/maven.gradle +++ b/gradle/maven.gradle @@ -1,5 +1,5 @@ allprojects { - version = "1.0.2-18-SNAPSHOT" + version = "1.0.2-21-SNAPSHOT" } subprojects { diff --git a/settings.gradle b/settings.gradle index 4d3611a06..b6306d63e 100644 --- a/settings.gradle +++ b/settings.gradle @@ -17,4 +17,3 @@ include 'datastream-tools' include 'datastream-utils' include 'datastream-bigquery' include 'datastream-jdbc-connector' -