diff --git a/pom.xml b/pom.xml
index 486dc7bc..6e796b5d 100644
--- a/pom.xml
+++ b/pom.xml
@@ -8,7 +8,7 @@
kafka-sink-azure-kusto
jar
A Kafka Connect plugin for Azure Data Explorer (Kusto) Database
- 4.1.3
+ 5.0.0
0.4.1
4.2.2
@@ -286,6 +286,12 @@
${junit.version}
test
+
+ org.junit.jupiter
+ junit-jupiter-params
+ ${junit.version}
+ test
+
org.junit.jupiter
junit-jupiter-engine
@@ -298,6 +304,12 @@
${avro.random.generator.version}
test
+
+ tech.allegro.schema.json2avro
+ converter
+ 0.2.15
+ test
+
org.testcontainers
testcontainers
@@ -332,7 +344,7 @@
jitpack
- https://jitpack.io
+ https://jitpack.io
confluent
diff --git a/src/main/java/com/microsoft/azure/kusto/kafka/connect/sink/FileWriter.java b/src/main/java/com/microsoft/azure/kusto/kafka/connect/sink/FileWriter.java
index 94d5cfe2..68eeb3c0 100644
--- a/src/main/java/com/microsoft/azure/kusto/kafka/connect/sink/FileWriter.java
+++ b/src/main/java/com/microsoft/azure/kusto/kafka/connect/sink/FileWriter.java
@@ -1,364 +1,362 @@
-package com.microsoft.azure.kusto.kafka.connect.sink;
-
-import java.io.*;
-import java.util.Map;
-import java.util.Timer;
-import java.util.TimerTask;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-import java.util.function.Consumer;
-import java.util.function.Function;
-import java.util.zip.GZIPOutputStream;
-
-import org.apache.commons.io.FilenameUtils;
-import org.apache.kafka.connect.data.Schema;
-import org.apache.kafka.connect.errors.ConnectException;
-import org.apache.kafka.connect.errors.DataException;
-import org.apache.kafka.connect.sink.SinkRecord;
-import org.jetbrains.annotations.NotNull;
-import org.jetbrains.annotations.Nullable;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.microsoft.azure.kusto.ingest.IngestionProperties;
-import com.microsoft.azure.kusto.kafka.connect.sink.KustoSinkConfig.BehaviorOnError;
-import com.microsoft.azure.kusto.kafka.connect.sink.format.RecordWriter;
-import com.microsoft.azure.kusto.kafka.connect.sink.format.RecordWriterProvider;
-import com.microsoft.azure.kusto.kafka.connect.sink.formatWriter.AvroRecordWriterProvider;
-import com.microsoft.azure.kusto.kafka.connect.sink.formatWriter.ByteRecordWriterProvider;
-import com.microsoft.azure.kusto.kafka.connect.sink.formatWriter.JsonRecordWriterProvider;
-import com.microsoft.azure.kusto.kafka.connect.sink.formatWriter.StringRecordWriterProvider;
-
-/**
- * This class is used to write gzipped rolling files.
- * Currently supports size based rolling, where size is for *uncompressed* size,
- * so final size can vary.
- */
-public class FileWriter implements Closeable {
-
- private static final Logger log = LoggerFactory.getLogger(FileWriter.class);
- private final long flushInterval;
- private final IngestionProperties.DataFormat format;
- SourceFile currentFile;
- private Timer timer;
- private final Consumer onRollCallback;
- private final Function getFilePath;
- private GZIPOutputStream outputStream;
- private final String basePath;
- private CountingOutputStream countingStream;
- private final long fileThreshold;
- // Lock is given from TopicPartitionWriter to lock while ingesting
- private final ReentrantReadWriteLock reentrantReadWriteLock;
- // Don't remove! File descriptor is kept so that the file is not deleted when stream is closed
- private FileDescriptor currentFileDescriptor;
- private String flushError;
- private RecordWriterProvider recordWriterProvider;
- private RecordWriter recordWriter;
- private final BehaviorOnError behaviorOnError;
- private boolean shouldWriteAvroAsBytes = false;
- private boolean stopped = false;
- private boolean isDlqEnabled = false;
-
- /**
- * @param basePath - This is path to which to write the files to.
- * @param fileThreshold - Max size, uncompressed bytes.
- * @param onRollCallback - Callback to allow code to execute when rolling a file. Blocking code.
- * @param getFilePath - Allow external resolving of file name.
- * @param behaviorOnError - Either log, fail or ignore errors based on the mode.
- */
- public FileWriter(String basePath,
- long fileThreshold,
- Consumer onRollCallback,
- Function getFilePath,
- long flushInterval,
- ReentrantReadWriteLock reentrantLock,
- IngestionProperties.DataFormat format,
- BehaviorOnError behaviorOnError,
- boolean isDlqEnabled) {
- this.getFilePath = getFilePath;
- this.basePath = basePath;
- this.fileThreshold = fileThreshold;
- this.onRollCallback = onRollCallback;
- this.flushInterval = flushInterval;
- this.behaviorOnError = behaviorOnError;
- this.isDlqEnabled = isDlqEnabled;
- // This is a fair lock so that we flush close to the time intervals
- this.reentrantReadWriteLock = reentrantLock;
- // If we failed on flush we want to throw the error from the put() flow.
- flushError = null;
- this.format = format;
- }
-
- boolean isDirty() {
- return this.currentFile != null && this.currentFile.rawBytes > 0;
- }
-
- public void openFile(@Nullable Long offset) throws IOException {
- SourceFile fileProps = new SourceFile();
- File folder = new File(basePath);
- if (!folder.exists() && !folder.mkdirs()) {
- if (!folder.exists()) {
- throw new IOException(String.format("Failed to create new directory %s", folder.getPath()));
- }
- log.warn("Couldn't create the directory because it already exists (likely a race condition)");
- }
- String filePath = getFilePath.apply(offset);
- fileProps.path = filePath;
- // Sanitize the file name just be sure and make sure it has the R/W permissions only
-
- String sanitizedFilePath = FilenameUtils.normalize(filePath);
- if (sanitizedFilePath == null) {
- /*
- * This condition should not occur at all. The files are created in controlled manner with the names consisting DB name, table name. This does not
- * permit names like "../../" or "./" etc. Still adding an additional check.
- */
- String errorMessage = String.format("Exception creating local file for write." +
- "File %s has a non canonical path", filePath);
- throw new RuntimeException(errorMessage);
- }
- File file = new File(sanitizedFilePath);
- boolean createFile = file.createNewFile(); // if there is a runtime exception. It gets thrown from here
- if (createFile) {
- /*
- * Setting restricted permissions on the file. If these permissions cannot be set, then warn - We cannot fail the ingestion (Failing the ingestion
- * would for not having the permission would mean that there may be data loss or unexpected scenarios.) Added this in a conditional as these
- * permissions can be applied only when the file is created
- *
- */
- try {
- boolean execResult = file.setReadable(true, true);
- execResult = execResult && file.setWritable(true, true);
- execResult = execResult && file.setExecutable(false, false);
- if (!execResult) {
- log.warn("Setting permissions creating file {} returned false." +
- "The files set for ingestion can be read by other applications having access." +
- "Please check security policies on the host that is preventing file permissions from being applied",
- filePath);
- }
- } catch (Exception ex) {
- // There is a likely chance of the permissions not getting set. This is set to warn
- log.warn("Exception permissions creating file {} returned false." +
- "The files set for ingestion can be read by other applications having access." +
- "Please check security policies on the host that is preventing file permissions being applied",
- filePath, ex);
-
- }
- }
- // The underlying file is closed only when the current countingStream (abstraction for size based writes) and
- // the file is rolled over
- FileOutputStream fos = new FileOutputStream(file);
- currentFileDescriptor = fos.getFD();
- fos.getChannel().truncate(0);
- fileProps.file = file;
- currentFile = fileProps;
- countingStream = new CountingOutputStream(new GZIPOutputStream(fos));
- outputStream = countingStream.getOutputStream();
- recordWriter = recordWriterProvider.getRecordWriter(currentFile.path, countingStream);
- }
-
- void rotate(@Nullable Long offset) throws IOException, DataException {
- finishFile(true);
- openFile(offset);
- }
-
- void finishFile(boolean delete) throws IOException, DataException {
- if (isDirty()) {
- recordWriter.commit();
- // Since we are using GZIP compression, finish the file. Close is invoked only when this flush finishes
- // and then the file is finished in ingest
- // This is called when there is a time or a size limit reached. The file is then reset/rolled and then a
- // new file is created for processing
- outputStream.finish();
- // It could be we were waiting on the lock when task suddenly stops and we should not ingest anymore
- if (stopped) {
- return;
- }
- try {
- onRollCallback.accept(currentFile);
- } catch (ConnectException e) {
- /*
- * Swallow the exception and continue to process subsequent records when behavior.on.error is not set to fail mode. Also, throwing/logging the
- * exception with just a message to avoid polluting logs with duplicate trace.
- */
- handleErrors("Failed to write records to KustoDB.", e);
- }
- if (delete) {
- dumpFile();
- }
- } else {
- // The stream is closed only when there are non-empty files for ingestion. Note that this closes the
- // FileOutputStream as well
- outputStream.close();
- currentFile = null;
- }
- }
-
- private void handleErrors(String message, Exception e) {
- if (KustoSinkConfig.BehaviorOnError.FAIL == behaviorOnError) {
- throw new ConnectException(message, e);
- } else if (KustoSinkConfig.BehaviorOnError.LOG == behaviorOnError) {
- log.error("{}", message, e);
- } else {
- log.debug("{}", message, e);
- }
- }
-
- private void dumpFile() throws IOException {
- SourceFile temp = currentFile;
- currentFile = null;
- if (temp != null) {
- countingStream.close();
- currentFileDescriptor = null;
- boolean deleted = temp.file.delete();
- if (!deleted) {
- log.warn("Couldn't delete temporary file. File exists: {}", temp.file.exists());
- }
- }
- }
-
- public synchronized void rollback() throws IOException {
- if (countingStream != null) {
- countingStream.close();
- if (currentFile != null && currentFile.file != null) {
- dumpFile();
- }
- }
- }
-
- @Override
- public synchronized void close() throws IOException {
- stop();
- }
-
- public synchronized void stop() throws DataException {
- stopped = true;
- if (timer != null) {
- Timer temp = timer;
- timer = null;
- temp.cancel();
- }
- }
-
- // Set shouldDestroyTimer to true if the current running task should be cancelled
- private void resetFlushTimer(boolean shouldDestroyTimer) {
- if (flushInterval > 0) {
- if (shouldDestroyTimer) {
- if (timer != null) {
- timer.cancel();
- }
- timer = new Timer(true);
- }
- TimerTask t = new TimerTask() {
- @Override
- public void run() {
- flushByTimeImpl();
- }
- };
- if (timer != null) {
- timer.schedule(t, flushInterval);
- }
- }
- }
-
- void flushByTimeImpl() {
- // Flush time interval gets the write lock so that it won't starve
- try (AutoCloseableLock ignored = new AutoCloseableLock(reentrantReadWriteLock.writeLock())) {
- if (stopped) {
- return;
- }
- // Lock before the check so that if a writing process just flushed this won't ingest empty files
- if (isDirty()) {
- finishFile(true);
- }
- resetFlushTimer(false);
- } catch (Exception e) {
- String fileName = currentFile == null ? "[no file created yet]" : currentFile.file.getName();
- long currentSize = currentFile == null ? 0 : currentFile.rawBytes;
- flushError = String.format("Error in flushByTime. Current file: %s, size: %d. ", fileName, currentSize);
- log.error(flushError, e);
- }
- }
-
- public void writeData(SinkRecord sinkRecord) throws IOException, DataException {
- if (flushError != null) {
- throw new ConnectException(flushError);
- }
- if (sinkRecord == null)
- return;
- if (recordWriterProvider == null) {
- initializeRecordWriter(sinkRecord);
- }
- if (currentFile == null) {
- openFile(sinkRecord.kafkaOffset());
- resetFlushTimer(true);
- }
- recordWriter.write(sinkRecord);
- if (this.isDlqEnabled) {
- currentFile.records.add(sinkRecord);
- }
- currentFile.rawBytes = countingStream.numBytes;
- currentFile.numRecords++;
- if (this.flushInterval == 0 || currentFile.rawBytes > fileThreshold || shouldWriteAvroAsBytes) {
- rotate(sinkRecord.kafkaOffset());
- resetFlushTimer(true);
- }
- }
-
- public void initializeRecordWriter(SinkRecord sinkRecord) {
- if (sinkRecord.value() instanceof Map) {
- recordWriterProvider = new JsonRecordWriterProvider();
- } else if ((sinkRecord.valueSchema() != null) && (sinkRecord.valueSchema().type() == Schema.Type.STRUCT)) {
- if (format.equals(IngestionProperties.DataFormat.JSON) || format.equals(IngestionProperties.DataFormat.MULTIJSON)) {
- recordWriterProvider = new JsonRecordWriterProvider();
- } else if (format.equals(IngestionProperties.DataFormat.AVRO)) {
- recordWriterProvider = new AvroRecordWriterProvider();
- } else {
- throw new ConnectException(String.format("Invalid Kusto table mapping, Kafka records of type "
- + "Avro and JSON can only be ingested to Kusto table having Avro or JSON mapping. "
- + "Currently, it is of type %s.", format));
- }
- } else if ((sinkRecord.valueSchema() == null) || (sinkRecord.valueSchema().type() == Schema.Type.STRING)) {
- recordWriterProvider = new StringRecordWriterProvider();
- } else if ((sinkRecord.valueSchema() != null) && (sinkRecord.valueSchema().type() == Schema.Type.BYTES)) {
- recordWriterProvider = new ByteRecordWriterProvider();
- if (format.equals(IngestionProperties.DataFormat.AVRO)) {
- shouldWriteAvroAsBytes = true;
- }
- } else {
- throw new ConnectException(String.format(
- "Invalid Kafka record format, connector does not support %s format. This connector supports Avro, Json with schema, Json without schema, Byte, String format. ",
- sinkRecord.valueSchema().type()));
- }
- }
-
- private class CountingOutputStream extends FilterOutputStream {
- private long numBytes = 0;
- private final GZIPOutputStream outputStream;
-
- CountingOutputStream(GZIPOutputStream out) {
- super(out);
- this.outputStream = out;
- }
-
- @Override
- public void write(int b) throws IOException {
- out.write(b);
- this.numBytes++;
- }
-
- @Override
- public void write(byte @NotNull [] b) throws IOException {
- out.write(b);
- this.numBytes += b.length;
- }
-
- @Override
- public void write(byte @NotNull [] b, int off, int len) throws IOException {
- out.write(b, off, len);
- this.numBytes += len;
- }
-
- public GZIPOutputStream getOutputStream() {
- return this.outputStream;
- }
- }
-}
+package com.microsoft.azure.kusto.kafka.connect.sink;
+
+import java.io.*;
+import java.util.Map;
+import java.util.Timer;
+import java.util.TimerTask;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import java.util.zip.GZIPOutputStream;
+
+import org.apache.commons.io.FilenameUtils;
+import org.apache.kafka.connect.data.Schema;
+import org.apache.kafka.connect.errors.ConnectException;
+import org.apache.kafka.connect.errors.DataException;
+import org.apache.kafka.connect.sink.SinkRecord;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.microsoft.azure.kusto.ingest.IngestionProperties;
+import com.microsoft.azure.kusto.kafka.connect.sink.KustoSinkConfig.BehaviorOnError;
+import com.microsoft.azure.kusto.kafka.connect.sink.format.RecordWriter;
+import com.microsoft.azure.kusto.kafka.connect.sink.format.RecordWriterProvider;
+import com.microsoft.azure.kusto.kafka.connect.sink.formatwriter.KustoRecordWriterProvider;
+
+/**
+ * This class is used to write gzipped rolling files.
+ * Currently, supports size based rolling, where size is for *uncompressed* size,
+ * so final size can vary.
+ */
+public class FileWriter implements Closeable {
+
+ private static final Logger log = LoggerFactory.getLogger(FileWriter.class);
+ private final long flushInterval;
+ private final IngestionProperties.DataFormat format;
+ private final Consumer onRollCallback;
+ private final Function getFilePath;
+ private final String basePath;
+ private final long fileThreshold;
+ // Lock is given from TopicPartitionWriter to lock while ingesting
+ private final ReentrantReadWriteLock reentrantReadWriteLock;
+ private final BehaviorOnError behaviorOnError;
+ SourceFile currentFile;
+ private Timer timer;
+ private GZIPOutputStream outputStream;
+ private CountingOutputStream countingStream;
+ // Don't remove! File descriptor is kept so that the file is not deleted when stream is closed
+ private FileDescriptor currentFileDescriptor;
+ private String flushError;
+ private RecordWriterProvider recordWriterProvider;
+ private RecordWriter recordWriter;
+ private boolean shouldWriteAvroAsBytes = false;
+ private boolean stopped = false;
+ private boolean isDlqEnabled = false;
+
+ /**
+ * @param basePath - This is path to which to write the files to.
+ * @param fileThreshold - Max size, uncompressed bytes.
+ * @param onRollCallback - Callback to allow code to execute when rolling a file. Blocking code.
+ * @param getFilePath - Allow external resolving of file name.
+ * @param behaviorOnError - Either log, fail or ignore errors based on the mode.
+ */
+ public FileWriter(String basePath,
+ long fileThreshold,
+ Consumer onRollCallback,
+ Function getFilePath,
+ long flushInterval,
+ ReentrantReadWriteLock reentrantLock,
+ IngestionProperties.DataFormat format,
+ BehaviorOnError behaviorOnError,
+ boolean isDlqEnabled) {
+ this.getFilePath = getFilePath;
+ this.basePath = basePath;
+ this.fileThreshold = fileThreshold;
+ this.onRollCallback = onRollCallback;
+ this.flushInterval = flushInterval;
+ this.behaviorOnError = behaviorOnError;
+ this.isDlqEnabled = isDlqEnabled;
+ // This is a fair lock so that we flush close to the time intervals
+ this.reentrantReadWriteLock = reentrantLock;
+ // If we failed on flush we want to throw the error from the put() flow.
+ flushError = null;
+ this.format = format;
+ }
+
+ boolean isDirty() {
+ return this.currentFile != null && this.currentFile.rawBytes > 0;
+ }
+
+ public void openFile(@Nullable Long offset) throws IOException {
+ SourceFile fileProps = new SourceFile();
+ File folder = new File(basePath);
+ if (!folder.exists() && !folder.mkdirs()) {
+ if (!folder.exists()) {
+ throw new IOException(String.format("Failed to create new directory %s", folder.getPath()));
+ }
+ log.warn("Couldn't create the directory because it already exists (likely a race condition)");
+ }
+ String filePath = getFilePath.apply(offset);
+ fileProps.path = filePath;
+ // Sanitize the file name just be sure and make sure it has the R/W permissions only
+
+ String sanitizedFilePath = FilenameUtils.normalize(filePath);
+ if (sanitizedFilePath == null) {
+ /*
+ * This condition should not occur at all. The files are created in controlled manner with the names consisting DB name, table name. This does not
+ * permit names like "../../" or "./" etc. Still adding an additional check.
+ */
+ String errorMessage = String.format("Exception creating local file for write." +
+ "File %s has a non canonical path", filePath);
+ throw new RuntimeException(errorMessage);
+ }
+ File file = new File(sanitizedFilePath);
+ boolean createFile = file.createNewFile(); // if there is a runtime exception. It gets thrown from here
+ if (createFile) {
+ /*
+ * Setting restricted permissions on the file. If these permissions cannot be set, then warn - We cannot fail the ingestion (Failing the ingestion
+ * would for not having the permission would mean that there may be data loss or unexpected scenarios.) Added this in a conditional as these
+ * permissions can be applied only when the file is created
+ *
+ */
+ try {
+ boolean execResult = file.setReadable(true, true);
+ execResult = execResult && file.setWritable(true, true);
+ execResult = execResult && file.setExecutable(false, false);
+ if (!execResult) {
+ log.warn("Setting permissions creating file {} returned false." +
+ "The files set for ingestion can be read by other applications having access." +
+ "Please check security policies on the host that is preventing file permissions from being applied",
+ filePath);
+ }
+ } catch (Exception ex) {
+ // There is a likely chance of the permissions not getting set. This is set to warn
+ log.warn("Exception permissions creating file {} returned false." +
+ "The files set for ingestion can be read by other applications having access." +
+ "Please check security policies on the host that is preventing file permissions being applied",
+ filePath, ex);
+
+ }
+ }
+ // The underlying file is closed only when the current countingStream (abstraction for size based writes) and
+ // the file is rolled over
+ FileOutputStream fos = new FileOutputStream(file);
+ currentFileDescriptor = fos.getFD();
+ fos.getChannel().truncate(0);
+ fileProps.file = file;
+ currentFile = fileProps;
+ countingStream = new CountingOutputStream(new GZIPOutputStream(fos));
+ outputStream = countingStream.getOutputStream();
+ log.debug("Opened new file for writing: {}", fileProps.file);
+ recordWriter = recordWriterProvider.getRecordWriter(currentFile.path, countingStream);
+ }
+
+ void rotate(@Nullable Long offset) throws IOException, DataException {
+ finishFile(true);
+ openFile(offset);
+ }
+
+ void finishFile(boolean delete) throws IOException, DataException {
+ if (isDirty()) {
+ recordWriter.commit();
+ // Since we are using GZIP compression, finish the file. Close is invoked only when this flush finishes
+ // and then the file is finished in ingest
+ // This is called when there is a time or a size limit reached. The file is then reset/rolled and then a
+ // new file is created for processing
+ outputStream.finish();
+ // It could be we were waiting on the lock when task suddenly stops and we should not ingest anymore
+ if (stopped) {
+ return;
+ }
+ try {
+ onRollCallback.accept(currentFile);
+ } catch (ConnectException e) {
+ /*
+ * Swallow the exception and continue to process subsequent records when behavior.on.error is not set to fail mode. Also, throwing/logging the
+ * exception with just a message to avoid polluting logs with duplicate trace.
+ */
+ handleErrors("Failed to write records to KustoDB.", e);
+ }
+ if (delete) {
+ dumpFile();
+ }
+ } else {
+ // The stream is closed only when there are non-empty files for ingestion. Note that this closes the
+ // FileOutputStream as well
+ outputStream.close();
+ currentFile = null;
+ }
+ }
+
+ private void handleErrors(String message, Exception e) {
+ if (KustoSinkConfig.BehaviorOnError.FAIL == behaviorOnError) {
+ throw new ConnectException(message, e);
+ } else if (KustoSinkConfig.BehaviorOnError.LOG == behaviorOnError) {
+ log.error("{}", message, e);
+ } else {
+ log.debug("{}", message, e);
+ }
+ }
+
+ private void dumpFile() throws IOException {
+ SourceFile temp = currentFile;
+ currentFile = null;
+ if (temp != null) {
+ countingStream.close();
+ currentFileDescriptor = null;
+ boolean deleted = temp.file.delete();
+ if (!deleted) {
+ log.warn("Couldn't delete temporary file. File exists: {}", temp.file.exists());
+ }
+ }
+ }
+
+ public synchronized void rollback() throws IOException {
+ if (countingStream != null) {
+ countingStream.close();
+ if (currentFile != null && currentFile.file != null) {
+ dumpFile();
+ }
+ }
+ }
+
+ @Override
+ public synchronized void close() throws IOException {
+ stop();
+ }
+
+ public synchronized void stop() throws DataException {
+ stopped = true;
+ if (timer != null) {
+ Timer temp = timer;
+ timer = null;
+ temp.cancel();
+ }
+ }
+
+ // Set shouldDestroyTimer to true if the current running task should be cancelled
+ private void resetFlushTimer(boolean shouldDestroyTimer) {
+ if (flushInterval > 0) {
+ if (shouldDestroyTimer) {
+ if (timer != null) {
+ timer.cancel();
+ }
+ timer = new Timer(true);
+ }
+ TimerTask t = new TimerTask() {
+ @Override
+ public void run() {
+ flushByTimeImpl();
+ }
+ };
+ if (timer != null) {
+ timer.schedule(t, flushInterval);
+ }
+ }
+ }
+
+ void flushByTimeImpl() {
+ // Flush time interval gets the write lock so that it won't starve
+ try (AutoCloseableLock ignored = new AutoCloseableLock(reentrantReadWriteLock.writeLock())) {
+ if (stopped) {
+ return;
+ }
+ // Lock before the check so that if a writing process just flushed this won't ingest empty files
+ if (isDirty()) {
+ finishFile(true);
+ }
+ resetFlushTimer(false);
+ } catch (Exception e) {
+ String fileName = currentFile == null ? "[no file created yet]" : currentFile.file.getName();
+ long currentSize = currentFile == null ? 0 : currentFile.rawBytes;
+ flushError = String.format("Error in flushByTime. Current file: %s, size: %d. ", fileName, currentSize);
+ log.error(flushError, e);
+ }
+ }
+
+ public void writeData(SinkRecord sinkRecord) throws IOException, DataException {
+ if (flushError != null) {
+ throw new ConnectException(flushError);
+ }
+ if (sinkRecord == null)
+ return;
+ if (recordWriterProvider == null) {
+ initializeRecordWriter(sinkRecord);
+ }
+ if (currentFile == null) {
+ openFile(sinkRecord.kafkaOffset());
+ resetFlushTimer(true);
+ }
+ recordWriter.write(sinkRecord, this.format);
+ if (this.isDlqEnabled) {
+ currentFile.records.add(sinkRecord);
+ }
+ currentFile.rawBytes = countingStream.numBytes;
+ currentFile.numRecords++;
+ if (this.flushInterval == 0 || currentFile.rawBytes > fileThreshold || shouldWriteAvroAsBytes) {
+ rotate(sinkRecord.kafkaOffset());
+ resetFlushTimer(true);
+ }
+ }
+
+ public void initializeRecordWriter(@NotNull SinkRecord sinkRecord) {
+ if (sinkRecord.value() instanceof Map) {
+ recordWriterProvider = new KustoRecordWriterProvider();
+ } else if ((sinkRecord.valueSchema() != null) && (sinkRecord.valueSchema().type() == Schema.Type.STRUCT)) {
+ if (format.equals(IngestionProperties.DataFormat.JSON) || format.equals(IngestionProperties.DataFormat.MULTIJSON)) {
+ recordWriterProvider = new KustoRecordWriterProvider();
+ } else if (format.equals(IngestionProperties.DataFormat.AVRO)) {
+ recordWriterProvider = new KustoRecordWriterProvider();
+ } else {
+ throw new ConnectException(String.format("Invalid Kusto table mapping, Kafka records of type "
+ + "Avro and JSON can only be ingested to Kusto table having Avro or JSON mapping. "
+ + "Currently, it is of type %s.", format));
+ }
+ } else if ((sinkRecord.valueSchema() == null) || (sinkRecord.valueSchema().type() == Schema.Type.STRING)) {
+ recordWriterProvider = new KustoRecordWriterProvider();
+ } else if ((sinkRecord.valueSchema() != null) && (sinkRecord.valueSchema().type() == Schema.Type.BYTES)) {
+ recordWriterProvider = new KustoRecordWriterProvider();
+ if (format.equals(IngestionProperties.DataFormat.AVRO)) {
+ shouldWriteAvroAsBytes = true;
+ }
+ } else {
+ throw new ConnectException(String.format(
+ "Invalid Kafka record format, connector does not support %s format. This connector supports Avro, Json with schema, Json without schema, Byte, String format. ",
+ sinkRecord.valueSchema().type()));
+ }
+ }
+
+ private class CountingOutputStream extends FilterOutputStream {
+ private final GZIPOutputStream outputStream;
+ private long numBytes = 0;
+
+ CountingOutputStream(GZIPOutputStream out) {
+ super(out);
+ this.outputStream = out;
+ }
+
+ @Override
+ public void write(int b) throws IOException {
+ out.write(b);
+ this.numBytes++;
+ }
+
+ @Override
+ public void write(byte @NotNull [] b) throws IOException {
+ out.write(b);
+ this.numBytes += b.length;
+ }
+
+ @Override
+ public void write(byte @NotNull [] b, int off, int len) throws IOException {
+ out.write(b, off, len);
+ this.numBytes += len;
+ }
+
+ public GZIPOutputStream getOutputStream() {
+ return this.outputStream;
+ }
+ }
+}
diff --git a/src/main/java/com/microsoft/azure/kusto/kafka/connect/sink/KustoSinkConfig.java b/src/main/java/com/microsoft/azure/kusto/kafka/connect/sink/KustoSinkConfig.java
index 067eedf9..a4a673b5 100644
--- a/src/main/java/com/microsoft/azure/kusto/kafka/connect/sink/KustoSinkConfig.java
+++ b/src/main/java/com/microsoft/azure/kusto/kafka/connect/sink/KustoSinkConfig.java
@@ -374,7 +374,6 @@ public String getAuthAppId() {
return this.getString(KUSTO_AUTH_APPID_CONF);
}
-
public String getAuthAppKey() {
return this.getPassword(KUSTO_AUTH_APPKEY_CONF).value();
}
diff --git a/src/main/java/com/microsoft/azure/kusto/kafka/connect/sink/KustoSinkTask.java b/src/main/java/com/microsoft/azure/kusto/kafka/connect/sink/KustoSinkTask.java
index 43c32542..fa7eefa4 100644
--- a/src/main/java/com/microsoft/azure/kusto/kafka/connect/sink/KustoSinkTask.java
+++ b/src/main/java/com/microsoft/azure/kusto/kafka/connect/sink/KustoSinkTask.java
@@ -129,7 +129,7 @@ private static boolean isStreamingEnabled(@NotNull KustoSinkConfig config) throw
return kcsb;
}
- public static Client createKustoEngineClient(KustoSinkConfig config) {
+ public static @NotNull Client createKustoEngineClient(KustoSinkConfig config) {
try {
return ClientFactory.createClient(createKustoEngineConnectionString(config, config.getKustoEngineUrl()));
} catch (Exception e) {
@@ -142,30 +142,23 @@ public static Map getTopicsToIngestionProps(Ku
try {
TopicToTableMapping[] mappings = config.getTopicToTableMapping();
-
for (TopicToTableMapping mapping : mappings) {
IngestionProperties props = new IngestionProperties(mapping.getDb(), mapping.getTable());
String format = mapping.getFormat();
if (StringUtils.isNotEmpty(format)) {
- if (isDataFormatAnyTypeOfJson(format)) {
- props.setDataFormat(IngestionProperties.DataFormat.MULTIJSON);
- } else {
- props.setDataFormat(format);
- }
+ props.setDataFormat(format);
+ // if (isDataFormatAnyTypeOfJson(format)) {
+ // props.setDataFormat(IngestionProperties.DataFormat.MULTIJSON);
+ // } else {
+ // props.setDataFormat(format);
+ // }
}
String mappingRef = mapping.getMapping();
- if (StringUtils.isNotEmpty(mappingRef) && format != null) {
- if (isDataFormatAnyTypeOfJson(format)) {
- props.setIngestionMapping(mappingRef, IngestionMapping.IngestionMappingKind.JSON);
- } else if (format.equalsIgnoreCase(IngestionProperties.DataFormat.AVRO.toString())) {
- props.setIngestionMapping(mappingRef, IngestionMapping.IngestionMappingKind.AVRO);
- } else if (format.equalsIgnoreCase(IngestionProperties.DataFormat.APACHEAVRO.toString())) {
- props.setIngestionMapping(mappingRef, IngestionMapping.IngestionMappingKind.APACHEAVRO);
- } else {
- props.setIngestionMapping(mappingRef, IngestionMapping.IngestionMappingKind.CSV);
- }
+ if (StringUtils.isNotEmpty(mappingRef) && StringUtils.isNotEmpty(format)) {
+ props.setIngestionMapping(mappingRef,
+ IngestionMapping.IngestionMappingKind.valueOf(format.toUpperCase(Locale.ROOT)));
}
TopicIngestionProperties topicIngestionProperties = new TopicIngestionProperties();
topicIngestionProperties.ingestionProperties = props;
@@ -177,8 +170,7 @@ public static Map getTopicsToIngestionProps(Ku
throw new ConfigException("Error while parsing kusto ingestion properties.", ex);
}
}
-
- private static boolean isDataFormatAnyTypeOfJson(String format) {
+ private static boolean isDataFormatAnyTypeOfJson(@NotNull String format) {
return format.equalsIgnoreCase(IngestionProperties.DataFormat.JSON.name())
|| format.equalsIgnoreCase(IngestionProperties.DataFormat.SINGLEJSON.name())
|| format.equalsIgnoreCase(IngestionProperties.DataFormat.MULTIJSON.name());
@@ -200,9 +192,6 @@ private static void validateTableAccess(Client engineClient, TopicToTableMapping
String format = mapping.getFormat();
String mappingName = mapping.getMapping();
boolean streamingEnabled = mapping.isStreaming();
- if (isDataFormatAnyTypeOfJson(format)) {
- format = IngestionProperties.DataFormat.JSON.name();
- }
boolean hasAccess = false;
boolean shouldCheckStreaming = streamingEnabled;
@@ -307,7 +296,7 @@ public TopicIngestionProperties getIngestionProps(String topic) {
return topicsToIngestionProps.get(topic);
}
- void validateTableMappings(KustoSinkConfig config) {
+ void validateTableMappings(@NotNull KustoSinkConfig config) {
List databaseTableErrorList = new ArrayList<>();
List accessErrorList = new ArrayList<>();
boolean enableTableValidation = config.getEnableTableValidation();
@@ -342,7 +331,7 @@ void validateTableMappings(KustoSinkConfig config) {
}
}
- private boolean isIngestorRole(TopicToTableMapping testMapping, Client engineClient) {
+ private boolean isIngestorRole(@NotNull TopicToTableMapping testMapping, @NotNull Client engineClient) {
try {
engineClient.execute(testMapping.getDb(), String.format(FETCH_TABLE_COMMAND, testMapping.getTable()), validateOnlyClientRequestProperties);
} catch (DataServiceException | DataClientException err) {
@@ -379,7 +368,7 @@ public void open(Collection partitions) {
}
@Override
- public void close(Collection partitions) {
+ public void close(@NotNull Collection partitions) {
log.warn("Closing writers in KustoSinkTask");
CountDownLatch countDownLatch = new CountDownLatch(partitions.size());
// First stop so that no more ingestions trigger from timer flushes
@@ -404,7 +393,6 @@ public void close(Collection partitions) {
public void start(Map props) {
config = new KustoSinkConfig(props);
String url = config.getKustoIngestUrl();
-
validateTableMappings(config);
if (config.isDlqEnabled()) {
isDlqEnabled = true;
@@ -417,18 +405,14 @@ public void start(Map props) {
} catch (Exception e) {
throw new ConnectException("Failed to initialize producer for miscellaneous dead-letter queue", e);
}
-
} else {
dlqProducer = null;
isDlqEnabled = false;
dlqTopicName = null;
}
-
topicsToIngestionProps = getTopicsToIngestionProps(config);
-
// this should be read properly from settings
createKustoIngestClient(config);
-
log.info("Started KustoSinkTask with target cluster: ({}), source topics: ({})", url,
topicsToIngestionProps.keySet());
// Adding this check to make code testable
@@ -458,13 +442,12 @@ public void stop() {
}
@Override
- public void put(Collection records) {
+ public void put(@NotNull Collection records) {
SinkRecord lastRecord = null;
for (SinkRecord sinkRecord : records) {
lastRecord = sinkRecord;
TopicPartition tp = new TopicPartition(sinkRecord.topic(), sinkRecord.kafkaPartition());
TopicPartitionWriter writer = writers.get(tp);
-
if (writer == null) {
NotFoundException e = new NotFoundException(String.format("Received a record without " +
"a mapped writer for topic:partition(%s:%d), dropping record.", tp.topic(), tp.partition()));
@@ -497,7 +480,6 @@ public Map preCommit(
"verify your `topics` and `kusto.tables.topics.mapping` configurations");
}
Long lastCommittedOffset = writers.get(tp).lastCommittedOffset;
-
if (lastCommittedOffset != null) {
long offset = lastCommittedOffset + 1L;
log.debug("Forwarding to framework request to commit offset: {} for {} while the offset is {}", offset,
@@ -505,7 +487,6 @@ public Map preCommit(
offsetsToCommit.put(tp, new OffsetAndMetadata(offset));
}
}
-
return offsetsToCommit;
}
diff --git a/src/main/java/com/microsoft/azure/kusto/kafka/connect/sink/TopicPartitionWriter.java b/src/main/java/com/microsoft/azure/kusto/kafka/connect/sink/TopicPartitionWriter.java
index 1493c718..38db7500 100644
--- a/src/main/java/com/microsoft/azure/kusto/kafka/connect/sink/TopicPartitionWriter.java
+++ b/src/main/java/com/microsoft/azure/kusto/kafka/connect/sink/TopicPartitionWriter.java
@@ -19,12 +19,15 @@
import org.apache.kafka.connect.errors.ConnectException;
import org.apache.kafka.connect.errors.DataException;
import org.apache.kafka.connect.sink.SinkRecord;
+import org.jetbrains.annotations.NotNull;
import org.jetbrains.annotations.Nullable;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.microsoft.azure.kusto.data.exceptions.KustoDataExceptionBase;
import com.microsoft.azure.kusto.ingest.IngestClient;
+import com.microsoft.azure.kusto.ingest.IngestionMapping;
+import com.microsoft.azure.kusto.ingest.IngestionProperties;
import com.microsoft.azure.kusto.ingest.ManagedStreamingIngestClient;
import com.microsoft.azure.kusto.ingest.exceptions.IngestionClientException;
import com.microsoft.azure.kusto.ingest.exceptions.IngestionServiceException;
@@ -33,13 +36,16 @@
import com.microsoft.azure.kusto.ingest.result.IngestionStatusResult;
import com.microsoft.azure.kusto.ingest.source.FileSourceInfo;
import com.microsoft.azure.kusto.kafka.connect.sink.KustoSinkConfig.BehaviorOnError;
+import com.microsoft.azure.kusto.kafka.connect.sink.formatwriter.FormatWriterHelper;
+
+import static com.microsoft.azure.kusto.ingest.IngestionProperties.DataFormat.*;
public class TopicPartitionWriter {
private static final Logger log = LoggerFactory.getLogger(TopicPartitionWriter.class);
private static final String COMPRESSION_EXTENSION = ".gz";
private static final String FILE_EXCEPTION_MESSAGE = "Failed to create file or write record into file for ingestion.";
-
+ private final FormatWriterHelper formatWriterHelper = FormatWriterHelper.getInstance();
private final TopicPartition tp;
private final IngestClient client;
private final TopicIngestionProperties ingestionProps;
@@ -52,10 +58,10 @@ public class TopicPartitionWriter {
private final String dlqTopicName;
private final Producer dlqProducer;
private final BehaviorOnError behaviorOnError;
+ private final ReentrantReadWriteLock reentrantReadWriteLock;
FileWriter fileWriter;
long currentOffset;
Long lastCommittedOffset;
- private final ReentrantReadWriteLock reentrantReadWriteLock;
TopicPartitionWriter(TopicPartition tp, IngestClient client, TopicIngestionProperties ingestionProps,
KustoSinkConfig config, boolean isDlqEnabled, String dlqTopicName, Producer dlqProducer) {
@@ -75,15 +81,14 @@ public class TopicPartitionWriter {
this.dlqProducer = dlqProducer;
}
- static String getTempDirectoryName(String tempDirPath) {
+ static @NotNull String getTempDirectoryName(String tempDirPath) {
String tempDir = String.format("kusto-sink-connector-%s", UUID.randomUUID());
Path path = Paths.get(tempDirPath, tempDir).toAbsolutePath();
return path.toString();
}
- public void handleRollFile(SourceFile fileDescriptor) {
+ public void handleRollFile(@NotNull SourceFile fileDescriptor) {
FileSourceInfo fileSourceInfo = new FileSourceInfo(fileDescriptor.path, fileDescriptor.rawBytes);
-
/*
* Since retries can be for a longer duration the Kafka Consumer may leave the group. This will result in a new Consumer reading records from the last
* committed offset leading to duplication of records in KustoDB. Also, if the error persists, it might also result in duplicate records being written
@@ -92,7 +97,7 @@ public void handleRollFile(SourceFile fileDescriptor) {
*/
for (int retryAttempts = 0; true; retryAttempts++) {
try {
- IngestionResult ingestionResult = client.ingestFromFile(fileSourceInfo, ingestionProps.ingestionProperties);
+ IngestionResult ingestionResult = client.ingestFromFile(fileSourceInfo, updateIngestionPropertiesWithTargetFormat());
if (ingestionProps.streaming && ingestionResult instanceof IngestionStatusResult) {
// If IngestionStatusResult returned then the ingestion status is from streaming ingest
IngestionStatus ingestionStatus = ingestionResult.getIngestionStatusCollection().get(0);
@@ -123,14 +128,22 @@ public void handleRollFile(SourceFile fileDescriptor) {
}
// TODO : improve handling of specific transient exceptions once the client supports them.
// retrying transient exceptions
+ log.error("IngestionServiceException when ingesting data into KustoDB, file: {}, database: {}, table: {}, operationId: {}",
+ fileDescriptor.path, ingestionProps.ingestionProperties.getDatabaseName(),
+ ingestionProps.ingestionProperties.getTableName(),
+ ingestionProps.ingestionProperties.getIngestionMapping().getIngestionMappingReference(),exception);
backOffForRemainingAttempts(retryAttempts, exception, fileDescriptor);
} catch (IngestionClientException | URISyntaxException exception) {
+ log.error("IngestionClientException when ingesting data into KustoDB, file: {}, database: {}, table: {}, operationId: {}",
+ fileDescriptor.path, ingestionProps.ingestionProperties.getDatabaseName(),
+ ingestionProps.ingestionProperties.getTableName(),
+ ingestionProps.ingestionProperties.getIngestionMapping().getIngestionMappingReference(),exception);
throw new ConnectException(exception);
}
}
}
- private boolean hasStreamingSucceeded(IngestionStatus status) {
+ private boolean hasStreamingSucceeded(@NotNull IngestionStatus status) {
switch (status.status) {
case Succeeded:
case Queued:
@@ -157,32 +170,38 @@ private boolean hasStreamingSucceeded(IngestionStatus status) {
return false;
}
- private void backOffForRemainingAttempts(int retryAttempts, Exception exception, SourceFile fileDescriptor) {
+ private void backOffForRemainingAttempts(int retryAttempts, Exception exception, @NotNull SourceFile fileDescriptor) {
if (retryAttempts < maxRetryAttempts) {
// RetryUtil can be deleted if exponential backOff is not required, currently using constant backOff.
// long sleepTimeMs = RetryUtil.computeExponentialBackOffWithJitter(retryAttempts, TimeUnit.SECONDS.toMillis(5));
long sleepTimeMs = retryBackOffTime;
- log.error("Failed to ingest records into Kusto, backing off and retrying ingesting records after {} milliseconds.", sleepTimeMs);
+ if(exception!=null) {
+ log.error("Failed to ingest records into Kusto, backing off and retrying ingesting records " +
+ "after {} milliseconds.", sleepTimeMs, exception);
+ }
try {
TimeUnit.MILLISECONDS.sleep(sleepTimeMs);
} catch (InterruptedException interruptedErr) {
if (isDlqEnabled && behaviorOnError != BehaviorOnError.FAIL) {
- log.warn("Writing {} failed records to miscellaneous dead-letter queue topic={}", fileDescriptor.records.size(), dlqTopicName);
+ log.warn("InterruptedException:: Writing {} failed records to miscellaneous dead-letter queue topic={}",
+ fileDescriptor.records.size(), dlqTopicName);
fileDescriptor.records.forEach(this::sendFailedRecordToDlq);
}
- throw new ConnectException(String.format("Retrying ingesting records into KustoDB was interrupted after retryAttempts=%s", retryAttempts + 1),
+ throw new ConnectException(String.format("Retrying ingesting records into KustoDB was interrupted " +
+ "after retryAttempts=%s", retryAttempts + 1),
exception);
}
} else {
if (isDlqEnabled && behaviorOnError != BehaviorOnError.FAIL) {
- log.warn("Writing {} failed records to miscellaneous dead-letter queue topic={}", fileDescriptor.records.size(), dlqTopicName);
+ log.warn(String.format("Retries exhausted, writing {%s} failed records to miscellaneous dead-letter queue topic={%s}",
+ fileDescriptor.records.size(), dlqTopicName));
fileDescriptor.records.forEach(this::sendFailedRecordToDlq);
}
throw new ConnectException("Retry attempts exhausted, failed to ingest records into KustoDB.", exception);
}
}
- public void sendFailedRecordToDlq(SinkRecord sinkRecord) {
+ public void sendFailedRecordToDlq(@NotNull SinkRecord sinkRecord) {
byte[] recordKey = String.format("Failed to write sinkRecord to KustoDB with the following kafka coordinates, "
+ "topic=%s, partition=%s, offset=%s.",
sinkRecord.topic(),
@@ -274,4 +293,24 @@ void close() {
void stop() {
fileWriter.stop();
}
+
+ private @NotNull IngestionProperties updateIngestionPropertiesWithTargetFormat() {
+ IngestionProperties updatedIngestionProperties = new IngestionProperties(this.ingestionProps.ingestionProperties);
+ IngestionProperties.DataFormat sourceFormat = ingestionProps.ingestionProperties.getDataFormat();
+ if (formatWriterHelper.isSchemaFormat(sourceFormat)) {
+ log.debug("Incoming dataformat {}, setting target format to MULTIJSON", sourceFormat);
+ updatedIngestionProperties.setDataFormat(MULTIJSON);
+ } else {
+ updatedIngestionProperties.setDataFormat(ingestionProps.ingestionProperties.getDataFormat());
+ }
+ // Just to make it clear , split the conditional
+ if (formatWriterHelper.isSchemaFormat(sourceFormat)) {
+ IngestionMapping mappingReference = ingestionProps.ingestionProperties.getIngestionMapping();
+ if (mappingReference != null && StringUtils.isNotEmpty(mappingReference.getIngestionMappingReference())) {
+ String ingestionMappingReferenceName = mappingReference.getIngestionMappingReference();
+ updatedIngestionProperties.setIngestionMapping(ingestionMappingReferenceName, IngestionMapping.IngestionMappingKind.JSON);
+ }
+ }
+ return updatedIngestionProperties;
+ }
}
diff --git a/src/main/java/com/microsoft/azure/kusto/kafka/connect/sink/format/RecordWriter.java b/src/main/java/com/microsoft/azure/kusto/kafka/connect/sink/format/RecordWriter.java
index de033af9..cca404a3 100644
--- a/src/main/java/com/microsoft/azure/kusto/kafka/connect/sink/format/RecordWriter.java
+++ b/src/main/java/com/microsoft/azure/kusto/kafka/connect/sink/format/RecordWriter.java
@@ -1,26 +1,28 @@
-package com.microsoft.azure.kusto.kafka.connect.sink.format;
-
-import java.io.Closeable;
-import java.io.IOException;
-
-import org.apache.kafka.connect.sink.SinkRecord;
-
-public interface RecordWriter extends Closeable {
- /**
- * Write a record to storage.
- *
- * @param record the record to persist.
- */
- void write(SinkRecord record) throws IOException;
-
- /**
- * Close this writer.
- */
- void close();
-
- /**
- * Flush writer's data and commit the records in Kafka. Optionally, this operation might also
- * close the writer.
- */
- void commit();
-}
+package com.microsoft.azure.kusto.kafka.connect.sink.format;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+import org.apache.kafka.connect.sink.SinkRecord;
+
+import com.microsoft.azure.kusto.ingest.IngestionProperties;
+
+public interface RecordWriter extends Closeable {
+ /**
+ * Write a record to storage.
+ *
+ * @param record the record to persist.
+ */
+ void write(SinkRecord record, IngestionProperties.DataFormat dataFormat) throws IOException;
+
+ /**
+ * Close this writer.
+ */
+ void close();
+
+ /**
+ * Flush writer's data and commit the records in Kafka. Optionally, this operation might also
+ * close the writer.
+ */
+ void commit();
+}
diff --git a/src/main/java/com/microsoft/azure/kusto/kafka/connect/sink/formatWriter/AvroRecordWriterProvider.java b/src/main/java/com/microsoft/azure/kusto/kafka/connect/sink/formatWriter/AvroRecordWriterProvider.java
deleted file mode 100644
index ca74e83e..00000000
--- a/src/main/java/com/microsoft/azure/kusto/kafka/connect/sink/formatWriter/AvroRecordWriterProvider.java
+++ /dev/null
@@ -1,73 +0,0 @@
-package com.microsoft.azure.kusto.kafka.connect.sink.formatWriter;
-
-import java.io.IOException;
-import java.io.OutputStream;
-
-import org.apache.avro.file.DataFileWriter;
-import org.apache.avro.generic.GenericDatumWriter;
-import org.apache.kafka.connect.data.Schema;
-import org.apache.kafka.connect.errors.ConnectException;
-import org.apache.kafka.connect.errors.DataException;
-import org.apache.kafka.connect.sink.SinkRecord;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.microsoft.azure.kusto.kafka.connect.sink.format.RecordWriter;
-import com.microsoft.azure.kusto.kafka.connect.sink.format.RecordWriterProvider;
-
-import io.confluent.connect.avro.AvroData;
-import io.confluent.kafka.serializers.NonRecordContainer;
-
-public class AvroRecordWriterProvider implements RecordWriterProvider {
- private static final Logger log = LoggerFactory.getLogger(AvroRecordWriterProvider.class);
- private final AvroData avroData = new AvroData(50);
-
- @Override
- public RecordWriter getRecordWriter(String filename, OutputStream out) {
- return new RecordWriter() {
- final DataFileWriter writer = new DataFileWriter<>(new GenericDatumWriter<>());
- Schema schema;
-
- @Override
- public void write(SinkRecord record) throws IOException {
- if (schema == null) {
- schema = record.valueSchema();
- try {
- log.debug("Opening record writer for: {}", filename);
- org.apache.avro.Schema avroSchema = avroData.fromConnectSchema(schema);
- writer.setFlushOnEveryBlock(true);
- writer.create(avroSchema, out);
- } catch (IOException e) {
- throw new ConnectException(e);
- }
- }
- Object value = avroData.fromConnectData(schema, record.value());
- // AvroData wraps primitive types so their schema can be included. We need to unwrap
- // NonRecordContainers to just their value to properly handle these types
- if (value instanceof NonRecordContainer) {
- writer.append(((NonRecordContainer) value).getValue());
- } else {
- writer.append(value);
- }
- }
-
- @Override
- public void close() {
- try {
- writer.close();
- } catch (IOException e) {
- throw new DataException(e);
- }
- }
-
- @Override
- public void commit() {
- try {
- writer.flush();
- } catch (IOException e) {
- throw new DataException(e);
- }
- }
- };
- }
-}
diff --git a/src/main/java/com/microsoft/azure/kusto/kafka/connect/sink/formatWriter/ByteRecordWriterProvider.java b/src/main/java/com/microsoft/azure/kusto/kafka/connect/sink/formatWriter/ByteRecordWriterProvider.java
deleted file mode 100644
index 9268547b..00000000
--- a/src/main/java/com/microsoft/azure/kusto/kafka/connect/sink/formatWriter/ByteRecordWriterProvider.java
+++ /dev/null
@@ -1,58 +0,0 @@
-package com.microsoft.azure.kusto.kafka.connect.sink.formatWriter;
-
-import java.io.IOException;
-import java.io.OutputStream;
-import java.nio.charset.StandardCharsets;
-
-import org.apache.kafka.connect.errors.DataException;
-import org.apache.kafka.connect.sink.SinkRecord;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.microsoft.azure.kusto.kafka.connect.sink.format.RecordWriter;
-import com.microsoft.azure.kusto.kafka.connect.sink.format.RecordWriterProvider;
-
-public class ByteRecordWriterProvider implements RecordWriterProvider {
- private static final Logger log = LoggerFactory.getLogger(ByteRecordWriterProvider.class);
-
- @Override
- public RecordWriter getRecordWriter(String filename, OutputStream out) {
- return new RecordWriter() {
-
- @Override
- public void write(SinkRecord record) throws IOException {
- byte[] value = null;
- byte[] valueBytes = (byte[]) record.value();
- if (filename.contains("avro")) {
- value = new byte[valueBytes.length];
- System.arraycopy(valueBytes, 0, value, 0, valueBytes.length);
- } else {
- byte[] separator = "\n".getBytes(StandardCharsets.UTF_8);
- byte[] valueWithSeparator = new byte[valueBytes.length + separator.length];
- System.arraycopy(valueBytes, 0, valueWithSeparator, 0, valueBytes.length);
- System.arraycopy(separator, 0, valueWithSeparator, valueBytes.length, separator.length);
- value = valueWithSeparator;
- }
- out.write(value);
- }
-
- @Override
- public void close() {
- try {
- out.close();
- } catch (IOException e) {
- throw new DataException(e);
- }
- }
-
- @Override
- public void commit() {
- try {
- out.flush();
- } catch (IOException e) {
- throw new DataException(e);
- }
- }
- };
- }
-}
diff --git a/src/main/java/com/microsoft/azure/kusto/kafka/connect/sink/formatWriter/JsonRecordWriterProvider.java b/src/main/java/com/microsoft/azure/kusto/kafka/connect/sink/formatWriter/JsonRecordWriterProvider.java
deleted file mode 100644
index cc46579c..00000000
--- a/src/main/java/com/microsoft/azure/kusto/kafka/connect/sink/formatWriter/JsonRecordWriterProvider.java
+++ /dev/null
@@ -1,92 +0,0 @@
-package com.microsoft.azure.kusto.kafka.connect.sink.formatWriter;
-
-import java.io.IOException;
-import java.io.OutputStream;
-import java.nio.charset.StandardCharsets;
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.commons.lang3.ArrayUtils;
-import org.apache.kafka.connect.data.Struct;
-import org.apache.kafka.connect.errors.ConnectException;
-import org.apache.kafka.connect.errors.DataException;
-import org.apache.kafka.connect.json.JsonConverter;
-import org.apache.kafka.connect.sink.SinkRecord;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.fasterxml.jackson.core.JsonGenerator;
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.microsoft.azure.kusto.kafka.connect.sink.format.RecordWriter;
-import com.microsoft.azure.kusto.kafka.connect.sink.format.RecordWriterProvider;
-
-public class JsonRecordWriterProvider implements RecordWriterProvider {
- private static final Logger log = LoggerFactory.getLogger(JsonRecordWriterProvider.class);
- private static final String LINE_SEPARATOR = System.lineSeparator();
- private static final byte[] LINE_SEPARATOR_BYTES = LINE_SEPARATOR.getBytes(StandardCharsets.UTF_8);
-
- private final ObjectMapper mapper = new ObjectMapper();
- private final JsonConverter converter = new JsonConverter();
-
- public JsonRecordWriterProvider() {
- Map converterConfig = new HashMap<>();
- converterConfig.put("schemas.enable", "false");
- converterConfig.put("schemas.cache.size", "50");
- this.converter.configure(converterConfig, false);
- }
-
- @Override
- public RecordWriter getRecordWriter(final String filename, OutputStream out) {
- try {
- log.debug("Opening record writer for: {}", filename);
- return new RecordWriter() {
- final JsonGenerator writer = mapper.getFactory()
- .createGenerator(out)
- .setRootValueSeparator(null);
-
- @Override
- public void write(SinkRecord record) {
- log.trace("Sink record: {}", record);
- try {
- Object value = record.value();
- if (value instanceof Struct) {
- byte[] rawJson = converter.fromConnectData(record.topic(), record.valueSchema(), value);
- if (ArrayUtils.isEmpty(rawJson)) {
- log.warn("Filtering empty records post-serialization. Record filtered {}", record); // prints everything
- } else {
- out.write(rawJson);
- out.write(LINE_SEPARATOR_BYTES);
- }
- } else {
- writer.writeObject(value);
- writer.writeRaw(LINE_SEPARATOR);
- }
- } catch (IOException e) {
- throw new ConnectException(e);
- }
- }
-
- @Override
- public void commit() {
- try {
- writer.flush();
- } catch (IOException e) {
- throw new DataException(e);
- }
- }
-
- @Override
- public void close() {
- try {
- writer.close();
- out.close();
- } catch (IOException e) {
- throw new DataException(e);
- }
- }
- };
- } catch (IOException e) {
- throw new DataException(e);
- }
- }
-}
diff --git a/src/main/java/com/microsoft/azure/kusto/kafka/connect/sink/formatWriter/StringRecordWriterProvider.java b/src/main/java/com/microsoft/azure/kusto/kafka/connect/sink/formatWriter/StringRecordWriterProvider.java
deleted file mode 100644
index 03a789fe..00000000
--- a/src/main/java/com/microsoft/azure/kusto/kafka/connect/sink/formatWriter/StringRecordWriterProvider.java
+++ /dev/null
@@ -1,49 +0,0 @@
-package com.microsoft.azure.kusto.kafka.connect.sink.formatWriter;
-
-import java.io.IOException;
-import java.io.OutputStream;
-import java.nio.charset.StandardCharsets;
-
-import org.apache.kafka.connect.errors.DataException;
-import org.apache.kafka.connect.sink.SinkRecord;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.microsoft.azure.kusto.kafka.connect.sink.format.RecordWriter;
-import com.microsoft.azure.kusto.kafka.connect.sink.format.RecordWriterProvider;
-
-public class StringRecordWriterProvider implements RecordWriterProvider {
- private static final Logger log = LoggerFactory.getLogger(StringRecordWriterProvider.class);
-
- @Override
- public RecordWriter getRecordWriter(String filename, OutputStream out) {
- return new RecordWriter() {
-
- @Override
- public void write(SinkRecord record) throws IOException {
- byte[] value = null;
- value = String.format("%s\n", record.value()).getBytes(StandardCharsets.UTF_8);
- out.write(value);
- }
-
- @Override
- public void close() {
- try {
- out.close();
- } catch (IOException e) {
- throw new DataException(e);
- }
- }
-
- @Override
- public void commit() {
- try {
- out.flush();
- } catch (IOException e) {
- throw new DataException(e);
- }
- }
- };
- }
-
-}
diff --git a/src/main/java/com/microsoft/azure/kusto/kafka/connect/sink/formatwriter/FormatWriterHelper.java b/src/main/java/com/microsoft/azure/kusto/kafka/connect/sink/formatwriter/FormatWriterHelper.java
new file mode 100644
index 00000000..1cc79775
--- /dev/null
+++ b/src/main/java/com/microsoft/azure/kusto/kafka/connect/sink/formatwriter/FormatWriterHelper.java
@@ -0,0 +1,227 @@
+package com.microsoft.azure.kusto.kafka.connect.sink.formatwriter;
+
+import java.io.IOException;
+import java.net.ConnectException;
+import java.nio.charset.StandardCharsets;
+import java.util.*;
+
+import org.apache.avro.file.DataFileReader;
+import org.apache.avro.file.SeekableByteArrayInput;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericDatumReader;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.io.DatumReader;
+import org.apache.kafka.connect.data.Schema;
+import org.apache.kafka.connect.data.Struct;
+import org.apache.kafka.connect.json.JsonConverter;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.fasterxml.jackson.core.JsonFactory;
+import com.fasterxml.jackson.core.JsonParser;
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.microsoft.azure.kusto.ingest.IngestionProperties;
+
+import io.confluent.kafka.serializers.NonRecordContainer;
+
+import static com.microsoft.azure.kusto.ingest.IngestionProperties.DataFormat.*;
+
+public class FormatWriterHelper {
+ private static final Logger LOGGER = LoggerFactory.getLogger(KustoRecordWriter.class);
+ private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper().enable(DeserializationFeature.FAIL_ON_TRAILING_TOKENS);
+ private static final JsonFactory JSON_FACTORY = new JsonFactory();
+ private static final TypeReference> MAP_TYPE_REFERENCE = new TypeReference>() {};
+ private final JsonConverter KEY_JSON_CONVERTER = new JsonConverter();
+ private static final JsonConverter VALUE_JSON_CONVERTER = new JsonConverter();
+ private static FormatWriterHelper INSTANCE;
+
+ public static FormatWriterHelper getInstance() {
+ if (INSTANCE == null) {
+ INSTANCE = new FormatWriterHelper();
+ }
+ return INSTANCE;
+ }
+
+ private FormatWriterHelper() {
+ KEY_JSON_CONVERTER.configure(Collections.singletonMap("schemas.enable", "false"), true);
+ VALUE_JSON_CONVERTER.configure(Collections.singletonMap("schemas.enable", "false"), false);
+ }
+
+ public boolean isSchemaFormat(IngestionProperties.DataFormat dataFormat) {
+ return dataFormat == JSON || dataFormat == MULTIJSON
+ || dataFormat == AVRO || dataFormat == SINGLEJSON;
+
+ }
+
+ protected boolean isAvro(IngestionProperties.DataFormat dataFormat) {
+ return IngestionProperties.DataFormat.AVRO.equals(dataFormat)
+ || IngestionProperties.DataFormat.APACHEAVRO.equals(dataFormat);
+ }
+
+ public boolean isCsv(IngestionProperties.DataFormat dataFormat) {
+ return IngestionProperties.DataFormat.CSV.equals(dataFormat);
+ }
+
+ public @NotNull Map convertAvroRecordToMap(Schema schema, Object value) throws IOException {
+ Map updatedValue = new HashMap<>();
+ if (value != null) {
+ if (value instanceof NonRecordContainer) {
+ updatedValue.put(schema.name(), ((NonRecordContainer) value).getValue());
+ } else {
+ if (value instanceof GenericData.Record) {
+ updatedValue.putAll(avroToJson((GenericData.Record) value));
+ }
+ }
+ }
+ return updatedValue;
+ }
+
+ /**
+ * @param messageBytes Raw message bytes to transform
+ * @param defaultKeyOrValueField Default value for Key or Value
+ * @param dataformat JSON or Avro
+ * @return a Map of the K-V of JSON
+ */
+ public @NotNull Collection> convertBytesToMap(byte[] messageBytes,
+ String defaultKeyOrValueField,
+ IngestionProperties.DataFormat dataformat) throws IOException {
+ if (messageBytes == null || messageBytes.length == 0) {
+ return Collections.emptyList();
+ }
+ if (isAvro(dataformat)) {
+ return bytesToAvroRecord(defaultKeyOrValueField, messageBytes);
+ }
+ String bytesAsJson = new String(messageBytes, StandardCharsets.UTF_8);
+ if (isJson(dataformat)) {
+ return isValidJson(defaultKeyOrValueField, bytesAsJson) ? parseJson(bytesAsJson)
+ : Collections.singletonList(Collections.singletonMap(defaultKeyOrValueField,
+ OBJECT_MAPPER.readTree(messageBytes)));
+ } else {
+ return Collections.singletonList(Collections.singletonMap(defaultKeyOrValueField,
+ Base64.getEncoder().encodeToString(messageBytes)));
+ }
+ }
+
+ private static @NotNull Collection> parseJson(String json) throws IOException {
+ JsonNode jsonData = OBJECT_MAPPER.readTree(json);
+ if (jsonData.isArray()) {
+ List> result = new ArrayList<>();
+ for (JsonNode node : jsonData) {
+ result.add(OBJECT_MAPPER.convertValue(node, MAP_TYPE_REFERENCE));
+ }
+ return result;
+ } else {
+ return Collections.singletonList(OBJECT_MAPPER.convertValue(jsonData, MAP_TYPE_REFERENCE));
+ }
+ }
+
+ /**
+ * Convert a given avro record to json and return the encoded bytes.
+ * @param record The GenericRecord to convert
+ */
+ private Map avroToJson(@NotNull GenericRecord record) throws IOException {
+ return OBJECT_MAPPER.readerFor(MAP_TYPE_REFERENCE).readValue(record.toString());
+ }
+
+ public @NotNull Map structToMap(String topicName,
+ @NotNull Struct recordData, boolean isKey) throws IOException {
+ try {
+ JsonConverter jsonConverter = isKey ? KEY_JSON_CONVERTER : VALUE_JSON_CONVERTER;
+ byte[] jsonBytes = jsonConverter.fromConnectData(topicName, recordData.schema(), recordData);
+ return OBJECT_MAPPER.readValue(jsonBytes, MAP_TYPE_REFERENCE);
+ } catch (IOException e) {
+ LOGGER.error("Failed to convert Struct to Map", e);
+ throw e;
+ }
+ }
+
+ private boolean isValidJson(String defaultKeyOrValueField, String json) {
+ try (JsonParser parser = JSON_FACTORY.createParser(json)) {
+ if (!parser.nextToken().isStructStart()) {
+ LOGGER.debug("No start token found for json {}. Is key {} ", json, defaultKeyOrValueField);
+ return false;
+ }
+ OBJECT_MAPPER.readTree(json);
+ } catch (IOException e) {
+ LOGGER.debug("Parsed data is not json {} , failed with {}", json, e.getMessage());
+ return false;
+ }
+ return true;
+ }
+
+ public @NotNull Map convertStringToMap(Object value,
+ String defaultKeyOrValueField,
+ IngestionProperties.DataFormat dataFormat) throws IOException {
+ String objStr = (String) value;
+ if (isJson(dataFormat) && isValidJson(defaultKeyOrValueField, objStr)) {
+ return OBJECT_MAPPER.readerFor(MAP_TYPE_REFERENCE).readValue(objStr);
+ } else {
+ return Collections.singletonMap(defaultKeyOrValueField, objStr);
+ }
+ }
+
+ public void close() {
+ try {
+ KEY_JSON_CONVERTER.close();
+ VALUE_JSON_CONVERTER.close();
+ } catch (Exception e) {
+ LOGGER.warn("Failed to close JsonConverter", e);
+ }
+ }
+
+ private boolean isJson(IngestionProperties.DataFormat dataFormat) {
+ return IngestionProperties.DataFormat.JSON.equals(dataFormat)
+ || IngestionProperties.DataFormat.MULTIJSON.equals(dataFormat)
+ || IngestionProperties.DataFormat.SINGLEJSON.equals(dataFormat);
+ }
+
+ private Collection> bytesToAvroRecord(String defaultKeyOrValueField, byte[] received_message) {
+ Map returnValue = new HashMap<>();
+ try {
+ // avro input parser
+ DatumReader datumReader = new GenericDatumReader<>();
+ DataFileReader dataFileReader;
+ try {
+ dataFileReader = new DataFileReader<>(new SeekableByteArrayInput(received_message), datumReader);
+ } catch (Exception e) {
+ LOGGER.error("Failed to parse AVRO record(1)\n{}", e.getMessage());
+ throw new ConnectException(
+ "Failed to parse AVRO " + "record\n" + e.getMessage());
+ }
+ List> nodes = new ArrayList<>();
+ while (dataFileReader.hasNext()) {
+ String jsonString = dataFileReader.next().toString();
+ LOGGER.trace("Encoding AVROBytes yielded {}", jsonString);
+ try {
+ Map nodeMap = OBJECT_MAPPER.readValue(jsonString, MAP_TYPE_REFERENCE);
+ returnValue.putAll(nodeMap);
+ nodes.add(returnValue);
+ } catch (IOException e) {
+ throw new ConnectException(
+ "Failed to parse JSON"
+ + " "
+ + "record\nInput String: "
+ + jsonString
+ + "\n"
+ + e.getMessage());
+ }
+ }
+ try {
+ dataFileReader.close();
+ } catch (IOException e) {
+ throw new ConnectException(
+ "Failed to parse AVRO (2) " + "record\n" + e);
+ }
+ return nodes;
+ } catch (Exception e) {
+ LOGGER.error("Failed to parse AVRO record (3) \n", e);
+ return Collections.singletonList(
+ Collections.singletonMap(defaultKeyOrValueField,
+ Base64.getEncoder().encodeToString(received_message)));
+ }
+ }
+}
diff --git a/src/main/java/com/microsoft/azure/kusto/kafka/connect/sink/formatwriter/HeaderAndMetadataWriter.java b/src/main/java/com/microsoft/azure/kusto/kafka/connect/sink/formatwriter/HeaderAndMetadataWriter.java
new file mode 100644
index 00000000..f916848d
--- /dev/null
+++ b/src/main/java/com/microsoft/azure/kusto/kafka/connect/sink/formatwriter/HeaderAndMetadataWriter.java
@@ -0,0 +1,121 @@
+package com.microsoft.azure.kusto.kafka.connect.sink.formatwriter;
+
+import java.io.IOException;
+import java.net.ConnectException;
+import java.nio.charset.StandardCharsets;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.avro.generic.GenericData;
+import org.apache.kafka.connect.data.Schema;
+import org.apache.kafka.connect.data.Struct;
+import org.apache.kafka.connect.sink.SinkRecord;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.SerializationFeature;
+import com.microsoft.azure.kusto.ingest.IngestionProperties;
+
+import io.confluent.kafka.serializers.NonRecordContainer;
+
+public abstract class HeaderAndMetadataWriter {
+ public static final ObjectMapper OBJECT_MAPPER = new ObjectMapper().disable(SerializationFeature.FAIL_ON_EMPTY_BEANS);
+ public static final String LINE_SEPARATOR = System.lineSeparator();
+ protected static final Logger LOGGER = LoggerFactory.getLogger(HeaderAndMetadataWriter.class);
+ public String HEADERS_FIELD = "headers";
+ public String KEYS_FIELD = "keys";
+ public String KEY_FIELD = "key";
+ public String VALUE_FIELD = "value";
+
+ public String KAFKA_METADATA_FIELD = "kafkamd";
+ public String TOPIC = "topic";
+ public String PARTITION = "partition";
+ public String OFFSET = "offset";
+
+ protected final FormatWriterHelper formatWriterHelper = FormatWriterHelper.getInstance();
+
+ @NotNull
+ public Map getHeadersAsMap(@NotNull SinkRecord record) {
+ Map headers = new HashMap<>();
+ record.headers().forEach(header -> headers.put(header.key(), header.value()));
+ return headers;
+ }
+
+ /**
+ * Convert SinkRecord to CSV
+ *
+ * @param record SinkRecord
+ * @param isKey boolean
+ * @return String
+ */
+ public String convertSinkRecordToCsv(@NotNull SinkRecord record, boolean isKey) {
+ if (isKey) {
+ if (record.key() instanceof byte[]) {
+ return record.key() == null ? "" : new String((byte[]) record.key(), StandardCharsets.UTF_8);
+ } else {
+ return record.key() == null ? "" : record.key().toString();
+ }
+ } else {
+ if (record.value() instanceof byte[]) {
+ return record.value() == null ? "" : new String((byte[]) record.value(), StandardCharsets.UTF_8);
+ } else {
+ return record.value() == null ? "" : record.value().toString();
+ }
+ }
+ }
+
+ @NotNull
+ @SuppressWarnings(value = "unchecked")
+ public Collection> convertSinkRecordToMap(@NotNull SinkRecord record, boolean isKey,
+ IngestionProperties.DataFormat dataFormat) throws IOException {
+ Object recordValue = isKey ? record.key() : record.value();
+ Schema schema = isKey ? record.keySchema() : record.valueSchema();
+ String defaultKeyOrValueField = isKey ? KEY_FIELD : VALUE_FIELD;
+ if (recordValue == null) {
+ return Collections.emptyList();
+ }
+ if (recordValue instanceof Struct) {
+ Struct recordStruct = (Struct) recordValue;
+ return Collections.singletonList(formatWriterHelper.structToMap(record.topic(), recordStruct, isKey));
+ }
+ // Is Avro Data
+ if (recordValue instanceof GenericData.Record || recordValue instanceof NonRecordContainer) {
+ return Collections.singletonList(formatWriterHelper.convertAvroRecordToMap(schema, recordValue));
+ }
+ // String or JSON
+ if (recordValue instanceof String) {
+ return Collections.singletonList(formatWriterHelper.convertStringToMap(recordValue,
+ defaultKeyOrValueField, dataFormat));
+ }
+ // Map
+ if (recordValue instanceof Map) {
+ return Collections.singletonList((Map) recordValue);
+ }
+ // is a byte array
+ if (FormatWriterHelper.getInstance().isSchemaFormat(dataFormat)) {
+ if (recordValue instanceof byte[]) {
+ return formatWriterHelper.convertBytesToMap((byte[]) recordValue, defaultKeyOrValueField, dataFormat);
+ } else {
+ String fieldName = isKey ? KEY_FIELD : VALUE_FIELD;
+ return Collections.singletonList(Collections.singletonMap(fieldName, recordValue));
+ }
+ } else {
+ String errorMessage = String.format("DataFormat %s is not supported in the connector though " +
+ "it may be supported for ingestion in ADX. Please raise a feature request if a " +
+ "new format has to be supported.", dataFormat);
+ throw new ConnectException(errorMessage);
+ }
+ }
+
+ public Map getKafkaMetaDataAsMap(@NotNull SinkRecord record) {
+ Map kafkaMetadata = new HashMap<>();
+ kafkaMetadata.put(TOPIC, record.topic());
+ kafkaMetadata.put(PARTITION, String.valueOf(record.kafkaPartition()));
+ kafkaMetadata.put(OFFSET, String.valueOf(record.kafkaOffset()));
+ return kafkaMetadata;
+ }
+}
diff --git a/src/main/java/com/microsoft/azure/kusto/kafka/connect/sink/formatwriter/KustoRecordWriter.java b/src/main/java/com/microsoft/azure/kusto/kafka/connect/sink/formatwriter/KustoRecordWriter.java
new file mode 100644
index 00000000..8c7903b1
--- /dev/null
+++ b/src/main/java/com/microsoft/azure/kusto/kafka/connect/sink/formatwriter/KustoRecordWriter.java
@@ -0,0 +1,120 @@
+package com.microsoft.azure.kusto.kafka.connect.sink.formatwriter;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.charset.StandardCharsets;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.commons.text.StringEscapeUtils;
+import org.apache.kafka.connect.data.Schema;
+import org.apache.kafka.connect.errors.ConnectException;
+import org.apache.kafka.connect.errors.DataException;
+import org.apache.kafka.connect.sink.SinkRecord;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.microsoft.azure.kusto.ingest.IngestionProperties;
+import com.microsoft.azure.kusto.kafka.connect.sink.format.RecordWriter;
+
+public class KustoRecordWriter extends HeaderAndMetadataWriter implements RecordWriter {
+ private final String filename;
+ private final JsonGenerator writer;
+ private final OutputStream plainOutputStream;
+ private Schema schema;
+
+ public KustoRecordWriter(String filename, OutputStream out) {
+ this.filename = filename;
+ this.plainOutputStream = out;
+ try {
+ this.writer = OBJECT_MAPPER.getFactory()
+ .createGenerator(out)
+ .setRootValueSeparator(null);
+ } catch (IOException e) {
+ throw new ConnectException(e);
+ }
+ }
+
+ /**
+ * @param record the record to persist.
+ * @param dataFormat the data format to use.
+ * @throws IOException if an error occurs while writing the record.
+ */
+ @Override
+ public void write(SinkRecord record, IngestionProperties.DataFormat dataFormat) throws IOException {
+ if (schema == null) {
+ schema = record.valueSchema();
+ LOGGER.debug("Opening record writer for: {}", filename);
+ }
+ Map parsedHeaders = getHeadersAsMap(record);
+ Map kafkaMd = getKafkaMetaDataAsMap(record);
+ if (FormatWriterHelper.getInstance().isCsv(dataFormat)) {
+ String serializedKeys = StringEscapeUtils.escapeCsv(convertSinkRecordToCsv(record, true));
+ String serializedValues = convertSinkRecordToCsv(record, false);
+ String serializedHeaders = StringEscapeUtils.escapeCsv(OBJECT_MAPPER.writeValueAsString(parsedHeaders));
+ String serializedMetadata = StringEscapeUtils.escapeCsv(OBJECT_MAPPER.writeValueAsString(kafkaMd));
+ String formattedRecord = String.format("%s,%s,%s,%s", serializedValues, serializedKeys,
+ serializedHeaders, serializedMetadata);
+ LOGGER.trace("Writing record to file: Keys {} , Values {} , Headers {} , OverallRecord {}",
+ serializedKeys, serializedValues, serializedHeaders, formattedRecord);
+ this.plainOutputStream.write(
+ formattedRecord.getBytes(StandardCharsets.UTF_8));
+ this.plainOutputStream.write(LINE_SEPARATOR.getBytes(StandardCharsets.UTF_8));
+ } else {
+ Map parsedKeys = convertSinkRecordToMap(record, true, dataFormat).stream().reduce(new HashMap<>(),
+ (acc, map) -> {
+ acc.putAll(map);
+ return acc;
+ });
+ Collection> parsedValues = convertSinkRecordToMap(record, false, dataFormat);
+
+ parsedValues.forEach(parsedValue -> {
+ Map updatedValue = (record.value() == null) ? new HashMap<>() : new HashMap<>(parsedValue);
+ /* Add all the key fields */
+ if (record.key() != null) {
+ if (parsedKeys.size() == 1 && parsedKeys.containsKey(KEY_FIELD)) {
+ updatedValue.put(KEYS_FIELD, parsedKeys.get(KEY_FIELD));
+ } else {
+ updatedValue.put(KEYS_FIELD, parsedKeys);
+ }
+ }
+ /* End add key fields */
+ /* Add record headers */
+ if (record.headers() != null && !record.headers().isEmpty()) {
+ updatedValue.put(HEADERS_FIELD, parsedHeaders);
+ }
+ /* End record headers */
+ /* Add metadata fields */
+ updatedValue.put(KAFKA_METADATA_FIELD, kafkaMd);
+ /* End metadata fields */
+ try {
+ /* Write out each value row with key and header fields */
+ writer.writeObject(updatedValue);
+ writer.writeRaw(LINE_SEPARATOR);
+ } catch (IOException e) {
+ LOGGER.error("Error writing record to file: {}", filename, e);
+ throw new ConnectException(e);
+ }
+ });
+ }
+ }
+
+ @Override
+ public void close() {
+ try {
+ writer.close();
+ formatWriterHelper.close();
+ } catch (IOException e) {
+ throw new DataException(e);
+ }
+ }
+
+ @Override
+ public void commit() {
+ try {
+ writer.flush();
+ } catch (IOException e) {
+ throw new DataException(e);
+ }
+ }
+}
diff --git a/src/main/java/com/microsoft/azure/kusto/kafka/connect/sink/formatwriter/KustoRecordWriterProvider.java b/src/main/java/com/microsoft/azure/kusto/kafka/connect/sink/formatwriter/KustoRecordWriterProvider.java
new file mode 100644
index 00000000..b83035e8
--- /dev/null
+++ b/src/main/java/com/microsoft/azure/kusto/kafka/connect/sink/formatwriter/KustoRecordWriterProvider.java
@@ -0,0 +1,13 @@
+package com.microsoft.azure.kusto.kafka.connect.sink.formatwriter;
+
+import java.io.OutputStream;
+
+import com.microsoft.azure.kusto.kafka.connect.sink.format.RecordWriter;
+import com.microsoft.azure.kusto.kafka.connect.sink.format.RecordWriterProvider;
+
+public class KustoRecordWriterProvider implements RecordWriterProvider {
+ @Override
+ public RecordWriter getRecordWriter(String filename, OutputStream out) {
+ return new KustoRecordWriter(filename, out);
+ }
+}
diff --git a/src/test/java/com/microsoft/azure/kusto/kafka/connect/sink/FileWriterTest.java b/src/test/java/com/microsoft/azure/kusto/kafka/connect/sink/FileWriterTest.java
index 8b7cabce..45adc4b2 100644
--- a/src/test/java/com/microsoft/azure/kusto/kafka/connect/sink/FileWriterTest.java
+++ b/src/test/java/com/microsoft/azure/kusto/kafka/connect/sink/FileWriterTest.java
@@ -1,255 +1,266 @@
-package com.microsoft.azure.kusto.kafka.connect.sink;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.IOException;
-import java.nio.file.Paths;
-import java.util.AbstractMap;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-import java.util.function.Consumer;
-import java.util.function.Function;
-import java.util.zip.GZIPInputStream;
-
-import org.apache.commons.io.FileUtils;
-import org.apache.commons.io.IOUtils;
-import org.apache.kafka.connect.data.Schema;
-import org.apache.kafka.connect.sink.SinkRecord;
-import org.junit.jupiter.api.AfterEach;
-import org.junit.jupiter.api.Assertions;
-import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.microsoft.azure.kusto.ingest.IngestionProperties;
-import com.microsoft.azure.kusto.kafka.connect.sink.KustoSinkConfig.BehaviorOnError;
-
-import static com.microsoft.azure.kusto.kafka.connect.sink.Utils.createDirectoryWithPermissions;
-import static com.microsoft.azure.kusto.kafka.connect.sink.Utils.getFilesCount;
-
-public class FileWriterTest {
- private static final Logger log = LoggerFactory.getLogger(FileWriterTest.class);
-
- IngestionProperties ingestionProps;
- private File currentDirectory;
-
- static Function getAssertFileConsumerFunction(String msg) {
- return (SourceFile f) -> {
- try (FileInputStream fileInputStream = new FileInputStream(f.file)) {
- byte[] bytes = IOUtils.toByteArray(fileInputStream);
- try (ByteArrayInputStream bin = new ByteArrayInputStream(bytes);
- GZIPInputStream gzipper = new GZIPInputStream(bin)) {
-
- byte[] buffer = new byte[1024];
- ByteArrayOutputStream out = new ByteArrayOutputStream();
-
- int len;
- while ((len = gzipper.read(buffer)) > 0) {
- out.write(buffer, 0, len);
- }
-
- gzipper.close();
- out.close();
- String s = out.toString();
-
- Assertions.assertEquals(s, msg);
- }
- } catch (IOException e) {
- log.error("Error running test", e);
- Assertions.fail(e.getMessage());
- }
- return null;
- };
- }
-
- @BeforeEach
- public final void before() {
- currentDirectory = Utils.getCurrentWorkingDirectory();
- ingestionProps = new IngestionProperties("db", "table");
- ingestionProps.setDataFormat(IngestionProperties.DataFormat.CSV);
- }
-
- @AfterEach
- public final void afterEach() {
- FileUtils.deleteQuietly(currentDirectory);
- }
-
- @Test
- public void testOpen() throws IOException {
- String path = Paths.get(currentDirectory.getPath(), "testWriterOpen").toString();
- Assertions.assertTrue(createDirectoryWithPermissions(path));
- Assertions.assertEquals(0, getFilesCount(path));
- final String FILE_PATH = Paths.get(path, "ABC").toString();
- final int MAX_FILE_SIZE = 128;
- Consumer trackFiles = (SourceFile f) -> {
- };
- Function generateFileName = (Long l) -> FILE_PATH;
- try (FileWriter fileWriter = new FileWriter(path, MAX_FILE_SIZE, trackFiles, generateFileName, 30000, new ReentrantReadWriteLock(),
- ingestionProps.getDataFormat(), BehaviorOnError.FAIL, true)) {
- String msg = "Line number 1: This is a message from the other size";
- SinkRecord record = new SinkRecord("topic", 1, null, null, Schema.BYTES_SCHEMA, msg.getBytes(), 10);
- fileWriter.initializeRecordWriter(record);
- fileWriter.openFile(null);
- Assertions.assertEquals(1, getFilesCount(path));
- Assertions.assertEquals(0, fileWriter.currentFile.rawBytes);
- Assertions.assertEquals(FILE_PATH, fileWriter.currentFile.path);
- Assertions.assertTrue(fileWriter.currentFile.file.canWrite());
- fileWriter.rollback();
- }
- }
-
- @Test
- public void testGzipFileWriter() throws IOException {
- String path = Paths.get(currentDirectory.getPath(), "testGzipFileWriter").toString();
- Assertions.assertTrue(createDirectoryWithPermissions(path));
- Assertions.assertEquals(0, getFilesCount(path));
- HashMap files = new HashMap<>();
- final int MAX_FILE_SIZE = 100;
- Consumer trackFiles = (SourceFile f) -> files.put(f.path, f.rawBytes);
- Function generateFileName = (Long l) -> Paths.get(path, String.valueOf(java.util.UUID.randomUUID())) + "csv.gz";
- try (FileWriter fileWriter = new FileWriter(path, MAX_FILE_SIZE, trackFiles, generateFileName, 30000, new ReentrantReadWriteLock(),
- ingestionProps.getDataFormat(), BehaviorOnError.FAIL, true)) {
- for (int i = 0; i < 9; i++) {
- String msg = String.format("Line number %d : This is a message from the other size", i);
- SinkRecord record1 = new SinkRecord("topic", 1, null, null, Schema.BYTES_SCHEMA, msg.getBytes(), 10);
- fileWriter.writeData(record1);
- }
- Assertions.assertEquals(4, files.size());
- // should still have 1 open file at this point...
- Assertions.assertEquals(1, getFilesCount(path));
- // close current file
- fileWriter.rotate(54L);
- Assertions.assertEquals(5, files.size());
- List sortedFiles = new ArrayList<>(files.values());
- sortedFiles.sort((Long x, Long y) -> (int) (y - x));
- Assertions.assertEquals(sortedFiles,
- Arrays.asList((long) 108, (long) 108, (long) 108, (long) 108, (long) 54));
- // make sure folder is clear once done - with only the new file
- Assertions.assertEquals(1, getFilesCount(path));
- }
- }
-
- @Test
- public void testGzipFileWriterFlush() throws IOException, InterruptedException {
- String path = Paths.get(currentDirectory.getPath(), "testGzipFileWriter2").toString();
- Assertions.assertTrue(createDirectoryWithPermissions(path));
- HashMap files = new HashMap<>();
- final int MAX_FILE_SIZE = 128 * 2;
- Consumer trackFiles = (SourceFile f) -> files.put(f.path, f.rawBytes);
- Function generateFileName = (Long l) -> Paths.get(path, java.util.UUID.randomUUID().toString()) + "csv.gz";
- // Expect no files to be ingested as size is small and flushInterval is big
- FileWriter fileWriter = new FileWriter(path, MAX_FILE_SIZE, trackFiles, generateFileName, 30000, new ReentrantReadWriteLock(),
- ingestionProps.getDataFormat(), BehaviorOnError.FAIL, true);
- String msg = "Message";
- SinkRecord record = new SinkRecord("topic", 1, null, null, null, msg, 10);
- fileWriter.writeData(record);
- Thread.sleep(1000);
- Assertions.assertEquals(0, files.size());
- fileWriter.rotate(10L);
- fileWriter.stop();
- Assertions.assertEquals(1, files.size());
-
- String path2 = Paths.get(currentDirectory.getPath(), "testGzipFileWriter2_2").toString();
- Assertions.assertTrue(createDirectoryWithPermissions(path2));
- Function generateFileName2 = (Long l) -> Paths.get(path2, java.util.UUID.randomUUID().toString()).toString();
- // Expect one file to be ingested as flushInterval had changed and is shorter than sleep time
- FileWriter fileWriter2 = new FileWriter(path2, MAX_FILE_SIZE, trackFiles, generateFileName2, 1000, new ReentrantReadWriteLock(),
- ingestionProps.getDataFormat(), BehaviorOnError.FAIL, true);
- String msg2 = "Second Message";
- SinkRecord record1 = new SinkRecord("topic", 1, null, null, null, msg2, 10);
- fileWriter2.writeData(record1);
- Thread.sleep(1050);
- Assertions.assertEquals(2, files.size());
- List sortedFiles = new ArrayList<>(files.values());
- sortedFiles.sort((Long x, Long y) -> (int) (y - x));
- Assertions.assertEquals(sortedFiles, Arrays.asList((long) 15, (long) 8));
- // make sure folder is clear once done
- fileWriter2.close();
- Assertions.assertEquals(1, getFilesCount(path));
- }
-
- @Test
- public void offsetCheckByInterval() throws InterruptedException, IOException {
- // This test will check that lastCommitOffset is set to the right value, when ingests are done by flush interval.
- // There will be a write operation followed by a flush which will track files and sleep.
- // While it sleeps there will be another write attempt which should wait on the lock and another flush later.
- // Resulting in first record to be with offset 1 and second with offset 2.
-
- ArrayList> files = new ArrayList<>();
- final int MAX_FILE_SIZE = 128 * 2;
- ReentrantReadWriteLock reentrantReadWriteLock = new ReentrantReadWriteLock();
- final ArrayList committedOffsets = new ArrayList<>();
- class Offsets {
- private long currentOffset = 0;
- }
- final Offsets offsets = new Offsets();
- Consumer trackFiles = (SourceFile f) -> {
- committedOffsets.add(offsets.currentOffset);
- files.add(new AbstractMap.SimpleEntry<>(f.path, f.rawBytes));
- // return null;
- };
- String path = Paths.get(currentDirectory.getPath(), "offsetCheckByInterval").toString();
- Assertions.assertTrue(createDirectoryWithPermissions(path));
- Function generateFileName = (Long offset) -> {
- if (offset == null) {
- offset = offsets.currentOffset;
- }
- return Paths.get(path, Long.toString(offset)).toString();
- };
- try (FileWriter fileWriter2 = new FileWriter(path, MAX_FILE_SIZE, trackFiles, generateFileName, 500, reentrantReadWriteLock,
- ingestionProps.getDataFormat(),
- BehaviorOnError.FAIL, true)) {
- String msg2 = "Second Message";
- reentrantReadWriteLock.readLock().lock();
- long recordOffset = 1;
- SinkRecord record = new SinkRecord("topic", 1, null, null, Schema.BYTES_SCHEMA, msg2.getBytes(), recordOffset);
- fileWriter2.writeData(record);
- offsets.currentOffset = recordOffset;
- // Wake the flush by interval in the middle of the writing
- Thread.sleep(510);
- recordOffset = 2;
- SinkRecord record2 = new SinkRecord("TestTopic", 1, null, null, Schema.BYTES_SCHEMA, msg2.getBytes(), recordOffset);
-
- fileWriter2.writeData(record2);
- offsets.currentOffset = recordOffset;
- reentrantReadWriteLock.readLock().unlock();
-
- // Context switch
- Thread.sleep(10);
- reentrantReadWriteLock.readLock().lock();
- recordOffset = 3;
- SinkRecord record3 = new SinkRecord("TestTopic", 1, null, null, Schema.BYTES_SCHEMA, msg2.getBytes(), recordOffset);
-
- offsets.currentOffset = recordOffset;
- fileWriter2.writeData(record3);
- reentrantReadWriteLock.readLock().unlock();
- Thread.sleep(550);
- // Assertions
- Assertions.assertEquals(2, files.size());
-
- // Make sure that the first file is from offset 1 till 2 and second is from 3 till 3
- Assertions.assertEquals(30L, files.stream().map(Map.Entry::getValue).toArray(Long[]::new)[0]);
- Assertions.assertEquals(15L, files.stream().map(Map.Entry::getValue).toArray(Long[]::new)[1]);
- Assertions.assertEquals("1",
- files.stream().map((s) -> s.getKey().substring(path.length() + 1)).toArray(String[]::new)[0]);
- Assertions.assertEquals("3",
- files.stream().map((s) -> s.getKey().substring(path.length() + 1)).toArray(String[]::new)[1]);
- Assertions.assertEquals(committedOffsets, new ArrayList() {
- {
- add(2L);
- add(3L);
- }
- });
- // make sure folder is clear once done
- fileWriter2.stop();
- Assertions.assertEquals(0, getFilesCount(path));
- }
- }
-}
+package com.microsoft.azure.kusto.kafka.connect.sink;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import java.util.zip.GZIPInputStream;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.io.IOUtils;
+import org.apache.kafka.connect.data.Schema;
+import org.apache.kafka.connect.sink.SinkRecord;
+import org.jetbrains.annotations.Contract;
+import org.jetbrains.annotations.NotNull;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.microsoft.azure.kusto.ingest.IngestionProperties;
+import com.microsoft.azure.kusto.kafka.connect.sink.KustoSinkConfig.BehaviorOnError;
+
+import static com.microsoft.azure.kusto.kafka.connect.sink.Utils.createDirectoryWithPermissions;
+import static com.microsoft.azure.kusto.kafka.connect.sink.Utils.getFilesCount;
+
+public class FileWriterTest {
+ private static final Logger log = LoggerFactory.getLogger(FileWriterTest.class);
+
+ IngestionProperties ingestionProps;
+ private File currentDirectory;
+
+ @Contract(pure = true)
+ static @NotNull Function getAssertFileConsumerFunction(String msg) {
+ return (SourceFile f) -> {
+ try (FileInputStream fileInputStream = new FileInputStream(f.file)) {
+ byte[] bytes = IOUtils.toByteArray(fileInputStream);
+ try (ByteArrayInputStream bin = new ByteArrayInputStream(bytes);
+ GZIPInputStream gzipper = new GZIPInputStream(bin)) {
+
+ byte[] buffer = new byte[1024];
+ ByteArrayOutputStream out = new ByteArrayOutputStream();
+
+ int len;
+ while ((len = gzipper.read(buffer)) > 0) {
+ out.write(buffer, 0, len);
+ }
+
+ gzipper.close();
+ out.close();
+ String s = out.toString();
+
+ Assertions.assertEquals(s, msg);
+ }
+ } catch (IOException e) {
+ log.error("Error running test", e);
+ Assertions.fail(e.getMessage());
+ }
+ return null;
+ };
+ }
+
+ @BeforeEach
+ public final void before() {
+ currentDirectory = Utils.getCurrentWorkingDirectory();
+ ingestionProps = new IngestionProperties("db", "table");
+ ingestionProps.setDataFormat(IngestionProperties.DataFormat.CSV);
+ }
+
+ @AfterEach
+ public final void afterEach() {
+ FileUtils.deleteQuietly(currentDirectory);
+ }
+
+ @Test
+ public void testOpen() throws IOException {
+ String path = Paths.get(currentDirectory.getPath(), "testWriterOpen").toString();
+ Assertions.assertTrue(createDirectoryWithPermissions(path));
+ Assertions.assertEquals(0, getFilesCount(path));
+ final String FILE_PATH = Paths.get(path, "ABC").toString();
+ final int MAX_FILE_SIZE = 128;
+ Consumer trackFiles = (SourceFile f) -> {
+ };
+ Function generateFileName = (Long l) -> FILE_PATH;
+ try (FileWriter fileWriter = new FileWriter(path, MAX_FILE_SIZE, trackFiles, generateFileName, 30000, new ReentrantReadWriteLock(),
+ ingestionProps.getDataFormat(), BehaviorOnError.FAIL, true)) {
+ String msg = "Line number 1: This is a message from the other size";
+ SinkRecord record = new SinkRecord("topic", 1, null, null, Schema.BYTES_SCHEMA, msg.getBytes(), 10);
+ fileWriter.initializeRecordWriter(record);
+ fileWriter.openFile(null);
+ Assertions.assertEquals(1, getFilesCount(path));
+ Assertions.assertEquals(0, fileWriter.currentFile.rawBytes);
+ Assertions.assertEquals(FILE_PATH, fileWriter.currentFile.path);
+ Assertions.assertTrue(fileWriter.currentFile.file.canWrite());
+ fileWriter.rollback();
+ }
+ }
+
+ @Test
+ public void testGzipFileWriter() throws IOException {
+ String path = Paths.get(currentDirectory.getPath(), "testGzipFileWriter").toString();
+ Assertions.assertTrue(createDirectoryWithPermissions(path));
+ Assertions.assertEquals(0, getFilesCount(path));
+ HashMap files = new HashMap<>();
+ final int MAX_FILE_SIZE = 225; // sizeof(,'','','{"partition":"1","offset":"1","topic":"topic"}'\n) * 2 , Similar multiple applied for the first test
+ Consumer trackFiles = (SourceFile f) -> files.put(f.path, f.rawBytes);
+ Function generateFileName = (Long l) -> Paths.get(path, String.valueOf(java.util.UUID.randomUUID())) + "csv.gz";
+ try (FileWriter fileWriter = new FileWriter(path, MAX_FILE_SIZE, trackFiles, generateFileName, 30000, new ReentrantReadWriteLock(),
+ ingestionProps.getDataFormat(), BehaviorOnError.FAIL, true)) {
+ for (int i = 0; i < 9; i++) {
+ String msg = String.format("Line number %d : This is a message from the other size", i);
+ SinkRecord record1 = new SinkRecord("topic", 1, null, null, Schema.BYTES_SCHEMA, msg.getBytes(), 10);
+ fileWriter.writeData(record1);
+ }
+ Assertions.assertEquals(4, files.size());
+ // should still have 1 open file at this point...
+ Assertions.assertEquals(1, getFilesCount(path));
+ // close current file
+ fileWriter.rotate(54L);
+ Assertions.assertEquals(5, files.size());
+ List sortedFiles = new ArrayList<>(files.values());
+ sortedFiles.sort((Long x, Long y) -> (int) (y - x));
+ Assertions.assertEquals(
+ Arrays.asList((long) 240, (long) 240, (long) 240, (long) 240, (long) 120),
+ sortedFiles);
+ // make sure folder is clear once done - with only the new file
+ Assertions.assertEquals(1, getFilesCount(path));
+ }
+ }
+
+ @Test
+ public void testGzipFileWriterFlush() throws IOException, InterruptedException {
+ String path = Paths.get(currentDirectory.getPath(), "testGzipFileWriter2").toString();
+ Assertions.assertTrue(createDirectoryWithPermissions(path));
+ HashMap files = new HashMap<>();
+ final int MAX_FILE_SIZE = 128 * 2;
+ Consumer trackFiles = (SourceFile f) -> files.put(f.path, f.rawBytes);
+ Function generateFileName = (Long l) -> Paths.get(path, java.util.UUID.randomUUID().toString()) + "csv.gz";
+ // Expect no files to be ingested as size is small and flushInterval is big
+ FileWriter fileWriter = new FileWriter(path, MAX_FILE_SIZE, trackFiles, generateFileName, 30000, new ReentrantReadWriteLock(),
+ ingestionProps.getDataFormat(), BehaviorOnError.FAIL, true);
+ String msg = "Message";
+ SinkRecord record = new SinkRecord("topic", 1, null, null, null, msg, 10);
+ fileWriter.writeData(record);
+ Thread.sleep(1000);
+ Assertions.assertEquals(0, files.size());
+ fileWriter.rotate(10L);
+ fileWriter.stop();
+ Assertions.assertEquals(1, files.size());
+
+ String path2 = Paths.get(currentDirectory.getPath(), "testGzipFileWriter2_2").toString();
+ Assertions.assertTrue(createDirectoryWithPermissions(path2));
+ Function generateFileName2 = (Long l) -> Paths.get(path2, java.util.UUID.randomUUID().toString()).toString();
+ // Expect one file to be ingested as flushInterval had changed and is shorter than sleep time
+ FileWriter fileWriter2 = new FileWriter(path2, MAX_FILE_SIZE, trackFiles, generateFileName2, 1000, new ReentrantReadWriteLock(),
+ ingestionProps.getDataFormat(), BehaviorOnError.FAIL, true);
+ String msg2 = "Second Message";
+ SinkRecord record1 = new SinkRecord("topic", 1, null, null, null, msg2, 10);
+ fileWriter2.writeData(record1);
+ Thread.sleep(1050);
+ Assertions.assertEquals(2, files.size());
+ List sortedFiles = new ArrayList<>(files.values());
+ sortedFiles.sort((Long x, Long y) -> (int) (y - x));
+ Assertions.assertEquals(sortedFiles, Arrays.asList((long) 81, (long) 74));
+ // make sure folder is clear once done
+ fileWriter2.close();
+ Assertions.assertEquals(1, getFilesCount(path));
+ }
+
+ @Test
+ public void offsetCheckByInterval() throws InterruptedException, IOException {
+ // This test will check that lastCommitOffset is set to the right value, when ingests are done by flush interval.
+ // There will be a write operation followed by a flush which will track files and sleep.
+ // While it sleeps there will be another write attempt which should wait on the lock and another flush later.
+ // Resulting in first record to be with offset 1 and second with offset 2.
+
+ ArrayList> files = new ArrayList<>();
+ final int MAX_FILE_SIZE = 128 * 2;
+ ReentrantReadWriteLock reentrantReadWriteLock = new ReentrantReadWriteLock();
+ final ArrayList committedOffsets = new ArrayList<>();
+ class Offsets {
+ private long currentOffset = 0;
+ }
+ final Offsets offsets = new Offsets();
+ Consumer trackFiles = (SourceFile f) -> {
+ committedOffsets.add(offsets.currentOffset);
+ files.add(new AbstractMap.SimpleEntry<>(f.path, f.rawBytes));
+ // return null;
+ };
+ String path = Paths.get(currentDirectory.getPath(), "offsetCheckByInterval").toString();
+ Assertions.assertTrue(createDirectoryWithPermissions(path));
+ Function generateFileName = (Long offset) -> {
+ if (offset == null) {
+ offset = offsets.currentOffset;
+ }
+ return Paths.get(path, Long.toString(offset)).toString();
+ };
+ try (FileWriter fileWriter2 = new FileWriter(path, MAX_FILE_SIZE, trackFiles, generateFileName, 500,
+ reentrantReadWriteLock,
+ ingestionProps.getDataFormat(),
+ BehaviorOnError.FAIL, true)) {
+ String msg2 = "Second Message";
+ reentrantReadWriteLock.readLock().lock();
+ long recordOffset = 1;
+ SinkRecord record = new SinkRecord("topic", 1, null, null, Schema.BYTES_SCHEMA, msg2.getBytes(), recordOffset);
+ fileWriter2.writeData(record);
+ offsets.currentOffset = recordOffset;
+ // Wake the flush by interval in the middle of the writing
+ Thread.sleep(510);
+ recordOffset = 2;
+ SinkRecord record2 = new SinkRecord("TestTopic", 1, null, null, Schema.BYTES_SCHEMA, msg2.getBytes(), recordOffset);
+
+ fileWriter2.writeData(record2);
+ offsets.currentOffset = recordOffset;
+ reentrantReadWriteLock.readLock().unlock();
+
+ // Context switch
+ Thread.sleep(10);
+ reentrantReadWriteLock.readLock().lock();
+ recordOffset = 3;
+ SinkRecord record3 = new SinkRecord("TestTopic", 1, null, null, Schema.BYTES_SCHEMA, msg2.getBytes(), recordOffset);
+
+ offsets.currentOffset = recordOffset;
+ fileWriter2.writeData(record3);
+ reentrantReadWriteLock.readLock().unlock();
+ Thread.sleep(550);
+ // Assertions
+ Assertions.assertEquals(2, files.size());
+
+ // Make sure that the first file is from offset 1 till 2 and second is from 3 till 3
+ /*
+ * > Why is this 30 before ? 2 * "Second Message" + NewLines for both 6(Second)+1(Space)+7(Message)+1(New Line) -> 15 2 of these => 30
+ *
+ * > Why did this become 146 ? The CSV now becomes : 'Second Message','','','{"partition":"1","offset":"1","topic":"topic"}'\n 2 of these become 146
+ * bytes
+ */
+ Assertions.assertEquals(164L, files.stream().map(Map.Entry::getValue).toArray(Long[]::new)[0]);
+ Assertions.assertEquals(84L, files.stream().map(Map.Entry::getValue).toArray(Long[]::new)[1]);
+ Assertions.assertEquals("1",
+ files.stream().map((s) -> s.getKey().substring(path.length() + 1)).toArray(String[]::new)[0]);
+ Assertions.assertEquals("3",
+ files.stream().map((s) -> s.getKey().substring(path.length() + 1)).toArray(String[]::new)[1]);
+ Assertions.assertEquals(committedOffsets, new ArrayList() {
+ {
+ add(2L);
+ add(3L);
+ }
+ });
+ // make sure folder is clear once done
+ fileWriter2.stop();
+ Assertions.assertEquals(0, getFilesCount(path));
+ }
+ }
+}
diff --git a/src/test/java/com/microsoft/azure/kusto/kafka/connect/sink/KustoSinkConnectorConfigTest.java b/src/test/java/com/microsoft/azure/kusto/kafka/connect/sink/KustoSinkConnectorConfigTest.java
index 8f1ff0bb..ababf082 100644
--- a/src/test/java/com/microsoft/azure/kusto/kafka/connect/sink/KustoSinkConnectorConfigTest.java
+++ b/src/test/java/com/microsoft/azure/kusto/kafka/connect/sink/KustoSinkConnectorConfigTest.java
@@ -10,7 +10,6 @@
import com.fasterxml.jackson.core.JsonProcessingException;
import com.microsoft.azure.kusto.data.auth.ConnectionStringBuilder;
-import com.microsoft.azure.kusto.kafka.connect.sink.KustoSinkConfig.BehaviorOnError;
import static com.microsoft.azure.kusto.kafka.connect.sink.KustoSinkConfig.KUSTO_SINK_ENABLE_TABLE_VALIDATION;
import static org.junit.jupiter.api.Assertions.fail;
@@ -34,7 +33,7 @@ public void shouldHaveDefaultValues() {
Assertions.assertTrue(config.getFlushSizeBytes() > 0);
Assertions.assertTrue(config.getFlushInterval() > 0);
Assertions.assertFalse(config.isDlqEnabled());
- Assertions.assertEquals(BehaviorOnError.FAIL, config.getBehaviorOnError());
+ Assertions.assertEquals(KustoSinkConfig.BehaviorOnError.FAIL, config.getBehaviorOnError());
}
@Test
diff --git a/src/test/java/com/microsoft/azure/kusto/kafka/connect/sink/KustoSinkTaskTest.java b/src/test/java/com/microsoft/azure/kusto/kafka/connect/sink/KustoSinkTaskTest.java
index 109af4a9..5f69ce47 100644
--- a/src/test/java/com/microsoft/azure/kusto/kafka/connect/sink/KustoSinkTaskTest.java
+++ b/src/test/java/com/microsoft/azure/kusto/kafka/connect/sink/KustoSinkTaskTest.java
@@ -122,7 +122,7 @@ public void getTable() {
assertEquals("Mapping", kustoSinkTaskSpy.getIngestionProps("topic2").ingestionProperties.getIngestionMapping().getIngestionMappingReference());
assertEquals("db2", kustoSinkTaskSpy.getIngestionProps("topic2").ingestionProperties.getDatabaseName());
assertEquals("table2", kustoSinkTaskSpy.getIngestionProps("topic2").ingestionProperties.getTableName());
- assertEquals(IngestionProperties.DataFormat.MULTIJSON, kustoSinkTaskSpy.getIngestionProps("topic2").ingestionProperties.getDataFormat());
+ assertEquals(IngestionProperties.DataFormat.JSON, kustoSinkTaskSpy.getIngestionProps("topic2").ingestionProperties.getDataFormat());
Assertions.assertNull(kustoSinkTaskSpy.getIngestionProps("topic3"));
}
}
@@ -141,7 +141,7 @@ public void getTableWithoutMapping() {
assertEquals(IngestionProperties.DataFormat.CSV, kustoSinkTaskSpy.getIngestionProps("topic1").ingestionProperties.getDataFormat());
assertEquals("db2", kustoSinkTaskSpy.getIngestionProps("topic2").ingestionProperties.getDatabaseName());
assertEquals("table2", kustoSinkTaskSpy.getIngestionProps("topic2").ingestionProperties.getTableName());
- assertEquals(IngestionProperties.DataFormat.MULTIJSON, kustoSinkTaskSpy.getIngestionProps("topic2").ingestionProperties.getDataFormat());
+ assertEquals(IngestionProperties.DataFormat.JSON, kustoSinkTaskSpy.getIngestionProps("topic2").ingestionProperties.getDataFormat());
Assertions.assertNull(kustoSinkTaskSpy.getIngestionProps("topic3"));
}
}
diff --git a/src/test/java/com/microsoft/azure/kusto/kafka/connect/sink/TopicPartitionWriterTest.java b/src/test/java/com/microsoft/azure/kusto/kafka/connect/sink/TopicPartitionWriterTest.java
index 33c2193b..4d6d7ffd 100644
--- a/src/test/java/com/microsoft/azure/kusto/kafka/connect/sink/TopicPartitionWriterTest.java
+++ b/src/test/java/com/microsoft/azure/kusto/kafka/connect/sink/TopicPartitionWriterTest.java
@@ -1,379 +1,368 @@
-package com.microsoft.azure.kusto.kafka.connect.sink;
-
-import java.io.ByteArrayOutputStream;
-import java.io.File;
-import java.io.IOException;
-import java.nio.charset.StandardCharsets;
-import java.nio.file.Paths;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Objects;
-import java.util.Properties;
-import java.util.function.Function;
-
-import org.apache.commons.io.FileUtils;
-import org.apache.commons.io.IOUtils;
-import org.apache.kafka.clients.producer.KafkaProducer;
-import org.apache.kafka.clients.producer.MockProducer;
-import org.apache.kafka.clients.producer.Producer;
-import org.apache.kafka.clients.producer.ProducerRecord;
-import org.apache.kafka.common.KafkaException;
-import org.apache.kafka.common.TopicPartition;
-import org.apache.kafka.common.serialization.ByteArraySerializer;
-import org.apache.kafka.connect.data.Schema;
-import org.apache.kafka.connect.sink.SinkRecord;
-import org.junit.jupiter.api.AfterEach;
-import org.junit.jupiter.api.Assertions;
-import org.junit.jupiter.api.BeforeAll;
-import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.Test;
-import org.mockito.ArgumentCaptor;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.microsoft.azure.kusto.ingest.IngestClient;
-import com.microsoft.azure.kusto.ingest.IngestionProperties;
-import com.microsoft.azure.kusto.ingest.source.FileSourceInfo;
-
-import static org.junit.jupiter.api.Assertions.assertThrows;
-import static org.junit.jupiter.api.Assertions.fail;
-import static org.mockito.Mockito.*;
-
-//TODO parts of this test needs to be re-formatted and may need rewriting
-public class TopicPartitionWriterTest {
- private static final Logger log = LoggerFactory.getLogger(TopicPartitionWriterTest.class);
-
- private static final String KUSTO_INGEST_CLUSTER_URL = "https://ingest-cluster.kusto.windows.net";
- private static final String KUSTO_CLUSTER_URL = "https://cluster.kusto.windows.net";
- private static final String DATABASE = "testdb1";
- private static final String TABLE = "testtable1";
- private static final long fileThreshold = 100;
- private static final long flushInterval = 5000;
- private static final IngestClient mockClient = mock(IngestClient.class);
- private static final TopicIngestionProperties propsCsv = new TopicIngestionProperties();
- private static final TopicPartition tp = new TopicPartition("testPartition", 11);
- private static final long contextSwitchInterval = 200;
- private static final IngestionProperties.DataFormat dataFormat = IngestionProperties.DataFormat.CSV;
- private static KustoSinkConfig config;
- // TODO: should probably find a better way to mock internal class (FileWriter)...
- private File currentDirectory;
- private String basePathCurrent;
- private boolean isDlqEnabled;
- private String dlqTopicName;
- private Producer kafkaProducer;
- private MockProducer dlqMockProducer;
-
- @BeforeAll
- public static void beforeClass() {
- propsCsv.ingestionProperties = new IngestionProperties(DATABASE, TABLE);
- propsCsv.ingestionProperties.setDataFormat(dataFormat);
- }
-
- @BeforeEach
- public final void before() {
- currentDirectory = Utils.getCurrentWorkingDirectory();
- Properties properties = new Properties();
- properties.put("bootstrap.servers", "localhost:9000");
- properties.put("key.serializer", "org.apache.kafka.common.serialization.ByteArraySerializer");
- properties.put("value.serializer", "org.apache.kafka.common.serialization.ByteArraySerializer");
- kafkaProducer = new KafkaProducer<>(properties);
- isDlqEnabled = false;
- dlqTopicName = null;
- dlqMockProducer = new MockProducer<>(
- true, new ByteArraySerializer(), new ByteArraySerializer());
- basePathCurrent = Paths.get(currentDirectory.getPath(), "testWriteStringyValuesAndOffset").toString();
- Map settings = getKustoConfigs(basePathCurrent, fileThreshold, flushInterval);
- config = new KustoSinkConfig(settings);
- }
-
- @AfterEach
- public final void afterEach() {
- FileUtils.deleteQuietly(currentDirectory);
- }
-
- @Test
- public void testHandleRollFile() {
- IngestClient mockedClient = mock(IngestClient.class);
- TopicIngestionProperties props = new TopicIngestionProperties();
- props.ingestionProperties = new IngestionProperties(DATABASE, TABLE);
- TopicPartitionWriter writer = new TopicPartitionWriter(tp, mockedClient, props, config, isDlqEnabled, dlqTopicName, kafkaProducer);
- SourceFile descriptor = new SourceFile();
- descriptor.rawBytes = 1024;
- writer.handleRollFile(descriptor);
- ArgumentCaptor fileSourceInfoArgument = ArgumentCaptor.forClass(FileSourceInfo.class);
- ArgumentCaptor ingestionPropertiesArgumentCaptor = ArgumentCaptor.forClass(IngestionProperties.class);
- try {
- verify(mockedClient, only()).ingestFromFile(fileSourceInfoArgument.capture(), ingestionPropertiesArgumentCaptor.capture());
- } catch (Exception e) {
- log.error("Error running testHandleRollFile", e);
- fail(e);
- }
-
- Assertions.assertEquals(fileSourceInfoArgument.getValue().getFilePath(), descriptor.path);
- Assertions.assertEquals(TABLE, ingestionPropertiesArgumentCaptor.getValue().getTableName());
- Assertions.assertEquals(DATABASE, ingestionPropertiesArgumentCaptor.getValue().getDatabaseName());
- Assertions.assertEquals(1024, fileSourceInfoArgument.getValue().getRawSizeInBytes());
- }
-
- @Test
- public void testHandleRollFileWithStreamingEnabled() {
- IngestClient mockedClient = mock(IngestClient.class);
- TopicIngestionProperties props = new TopicIngestionProperties();
- props.ingestionProperties = new IngestionProperties(DATABASE, TABLE);
- props.streaming = true;
- TopicPartitionWriter writer = new TopicPartitionWriter(tp, mockedClient, props, config, isDlqEnabled, dlqTopicName, kafkaProducer);
-
- SourceFile descriptor = new SourceFile();
- descriptor.rawBytes = 1024;
-
- writer.handleRollFile(descriptor);
- ArgumentCaptor fileSourceInfoArgument = ArgumentCaptor.forClass(FileSourceInfo.class);
- ArgumentCaptor ingestionPropertiesArgumentCaptor = ArgumentCaptor.forClass(IngestionProperties.class);
- try {
- verify(mockedClient, only()).ingestFromFile(fileSourceInfoArgument.capture(), ingestionPropertiesArgumentCaptor.capture());
- } catch (Exception e) {
- log.error("Error running testHandleRollFile", e);
- fail(e);
- }
-
- Assertions.assertEquals(fileSourceInfoArgument.getValue().getFilePath(), descriptor.path);
- Assertions.assertEquals(TABLE, ingestionPropertiesArgumentCaptor.getValue().getTableName());
- Assertions.assertEquals(DATABASE, ingestionPropertiesArgumentCaptor.getValue().getDatabaseName());
- Assertions.assertEquals(1024, fileSourceInfoArgument.getValue().getRawSizeInBytes());
- }
-
- @Test
- public void testGetFilename() {
- try {
- TopicPartitionWriter writer = new TopicPartitionWriter(tp, mockClient, propsCsv, config, isDlqEnabled, dlqTopicName, kafkaProducer);
- File writerFile = new File(writer.getFilePath(null));
- Assertions.assertEquals("kafka_testPartition_11_0.CSV.gz", writerFile.getName());
- } catch (Exception ex) {
- // In case there is an accessor exception getting the file
- fail(ex);
- }
- }
-
- @Test
- public void testGetFilenameAfterOffsetChanges() {
- TopicPartitionWriter writer = new TopicPartitionWriter(tp, mockClient, propsCsv, config, isDlqEnabled, dlqTopicName, kafkaProducer);
- writer.open();
- List records = new ArrayList<>();
- records.add(new SinkRecord(tp.topic(), tp.partition(), null, null, Schema.STRING_SCHEMA, "another,stringy,message", 5));
- records.add(new SinkRecord(tp.topic(), tp.partition(), null, null, Schema.STRING_SCHEMA, "{'also':'stringy','sortof':'message'}", 4));
- for (SinkRecord record : records) {
- writer.writeRecord(record);
- }
- try {
- File writerFile = new File(writer.getFilePath(null));
- Assertions.assertTrue(writerFile.exists());
- Assertions.assertEquals("kafka_testPartition_11_5.CSV.gz", (new File(writer.getFilePath(null))).getName());
- } catch (Exception ex) {
- // In case there is an accessor exception getting the file
- fail(ex);
- }
- }
-
- @Test
- public void testOpenClose() {
- TopicPartitionWriter writer = new TopicPartitionWriter(tp, mockClient, propsCsv, config, isDlqEnabled, dlqTopicName, kafkaProducer);
- writer.open();
- writer.close();
- }
-
- @Test
- public void testWriteNonStringAndOffset() {
- // String db = "testdb1";
- // String table = "testtable1";
- //
- // TopicPartitionWriter writer = new TopicPartitionWriter(tp, mockClient, db, table, basePath, fileThreshold);
- //
- // List records = new ArrayList();
- // DummyRecord dummyRecord1 = new DummyRecord(1, "a", (long) 2);
- // DummyRecord dummyRecord2 = new DummyRecord(2, "b", (long) 4);
- //
- // records.add(new SinkRecord("topic", 1, null, null, null, dummyRecord1, 10));
- // records.add(new SinkRecord("topic", 2, null, null, null, dummyRecord2, 3));
- // records.add(new SinkRecord("topic", 2, null, null, null, dummyRecord2, 4));
- //
- // for (SinkRecord record : records) {
- // writer.writeRecord(record);
- // }
- //
- // Assert.assertEquals(writer.getFilePath(), "kafka_testPartition_11_0");
- }
-
- @Test
- public void testWriteStringyValuesAndOffset() {
- TopicPartitionWriter writer = new TopicPartitionWriter(tp, mockClient, propsCsv, config, isDlqEnabled, dlqTopicName, kafkaProducer);
-
- writer.open();
- List records = new ArrayList<>();
-
- records.add(new SinkRecord(tp.topic(), tp.partition(), null, null, Schema.STRING_SCHEMA, "another,stringy,message", 3));
- records.add(new SinkRecord(tp.topic(), tp.partition(), null, null, Schema.STRING_SCHEMA, "{'also':'stringy','sortof':'message'}", 4));
-
- for (SinkRecord record : records) {
- writer.writeRecord(record);
- }
-
- Assertions.assertTrue((new File(writer.fileWriter.currentFile.path)).exists());
- Assertions.assertEquals(String.format("kafka_%s_%d_%d.%s.gz", tp.topic(), tp.partition(), 3, IngestionProperties.DataFormat.CSV.name()),
- (new File(writer.fileWriter.currentFile.path)).getName());
- writer.close();
- }
-
- @Test
- public void testWriteStringValuesAndOffset() throws IOException {
- String[] messages = new String[] {"stringy message", "another,stringy,message", "{'also':'stringy','sortof':'message'}"};
-
- // Expect to finish file after writing forth message cause of fileThreshold
- long fileThreshold2 = messages[0].length() + messages[1].length() + messages[2].length() + messages[2].length() - 1;
- Map settings2 = getKustoConfigs(basePathCurrent, fileThreshold2, flushInterval);
- KustoSinkConfig config2 = new KustoSinkConfig(settings2);
- TopicPartitionWriter writer = new TopicPartitionWriter(tp, mockClient, propsCsv, config2, isDlqEnabled, dlqTopicName, kafkaProducer);
-
- writer.open();
- List records = new ArrayList<>();
- records.add(new SinkRecord(tp.topic(), tp.partition(), null, null, Schema.STRING_SCHEMA, messages[0], 10));
- records.add(new SinkRecord(tp.topic(), tp.partition(), null, null, Schema.STRING_SCHEMA, messages[1], 13));
- records.add(new SinkRecord(tp.topic(), tp.partition(), null, null, Schema.STRING_SCHEMA, messages[2], 14));
- records.add(new SinkRecord(tp.topic(), tp.partition(), null, null, Schema.STRING_SCHEMA, messages[2], 15));
- records.add(new SinkRecord(tp.topic(), tp.partition(), null, null, Schema.STRING_SCHEMA, messages[2], 16));
-
- for (SinkRecord record : records) {
- writer.writeRecord(record);
- }
-
- Assertions.assertEquals(15, (long) writer.lastCommittedOffset);
- Assertions.assertEquals(16, writer.currentOffset);
-
- String currentFileName = writer.fileWriter.currentFile.path;
- Assertions.assertTrue(new File(currentFileName).exists());
- Assertions.assertEquals(String.format("kafka_%s_%d_%d.%s.gz", tp.topic(), tp.partition(), 15, IngestionProperties.DataFormat.CSV.name()),
- (new File(currentFileName)).getName());
-
- // Read
- writer.fileWriter.finishFile(false);
- Function assertFileConsumer = FileWriterTest.getAssertFileConsumerFunction(messages[2] + "\n");
- assertFileConsumer.apply(writer.fileWriter.currentFile);
- writer.close();
- }
-
- @Test
- public void testWriteBytesValuesAndOffset() throws IOException {
- byte[] message = IOUtils.toByteArray(
- Objects.requireNonNull(this.getClass().getClassLoader().getResourceAsStream("data.avro")));
- ByteArrayOutputStream o = new ByteArrayOutputStream();
- o.write(message);
- // Expect to finish file with one record although fileThreshold is high
- long fileThreshold2 = 128;
- TopicIngestionProperties propsAvro = new TopicIngestionProperties();
- propsAvro.ingestionProperties = new IngestionProperties(DATABASE, TABLE);
- propsAvro.ingestionProperties.setDataFormat(IngestionProperties.DataFormat.AVRO);
- Map settings2 = getKustoConfigs(basePathCurrent, fileThreshold2, flushInterval);
- KustoSinkConfig config2 = new KustoSinkConfig(settings2);
- TopicPartitionWriter writer = new TopicPartitionWriter(tp, mockClient, propsAvro, config2, isDlqEnabled, dlqTopicName, kafkaProducer);
-
- writer.open();
- List records = new ArrayList<>();
- records.add(new SinkRecord(tp.topic(), tp.partition(), null, null, Schema.BYTES_SCHEMA, o.toByteArray(), 10));
-
- for (SinkRecord record : records) {
- writer.writeRecord(record);
- }
-
- Assertions.assertEquals(10, (long) writer.lastCommittedOffset);
- Assertions.assertEquals(10, writer.currentOffset);
-
- String currentFileName = writer.fileWriter.currentFile.path;
-
- Assertions.assertTrue(new File(currentFileName).exists());
- Assertions.assertEquals(String.format("kafka_%s_%d_%d.%s.gz", tp.topic(), tp.partition(), 10, IngestionProperties.DataFormat.AVRO.name()),
- (new File(currentFileName)).getName());
- writer.close();
- }
-
- @Test
- public void testClose() throws InterruptedException {
- TopicPartitionWriter writer = new TopicPartitionWriter(tp, mockClient, propsCsv, config, isDlqEnabled, dlqTopicName, kafkaProducer);
- TopicPartitionWriter spyWriter = spy(writer);
-
- spyWriter.open();
- List records = new ArrayList<>();
-
- records.add(new SinkRecord(tp.topic(), tp.partition(), null, null, Schema.STRING_SCHEMA, "another,stringy,message", 5));
- records.add(new SinkRecord(tp.topic(), tp.partition(), null, null, Schema.STRING_SCHEMA, "{'also':'stringy','sortof':'message'}", 4));
-
- for (SinkRecord record : records) {
- spyWriter.writeRecord(record);
- }
- // 2 records are waiting to be ingested - expect close to revoke them so that even after 5 seconds it won't ingest
- Assertions.assertNull(spyWriter.lastCommittedOffset);
- spyWriter.close();
- Assertions.assertNull(spyWriter.lastCommittedOffset);
-
- Thread.sleep(flushInterval + contextSwitchInterval);
- Assertions.assertNull(spyWriter.lastCommittedOffset);
- }
-
- @Test
- public void testSendFailedRecordToDlqError() {
- TopicPartitionWriter writer = new TopicPartitionWriter(tp, mockClient, propsCsv, config, true, "dlq.topic.name", kafkaProducer);
- TopicPartitionWriter spyWriter = spy(writer);
- // TODO this is to be re-worked
- kafkaProducer = mock(Producer.class);
- spyWriter.open();
- List records = new ArrayList<>();
- records.add(new SinkRecord(tp.topic(), tp.partition(), null, null, Schema.STRING_SCHEMA, "another,stringy,message", 5));
- records.add(new SinkRecord(tp.topic(), tp.partition(), null, null, Schema.STRING_SCHEMA, "{'also':'stringy','sortof':'message'}", 4));
- when(kafkaProducer.send(any(), any())).thenReturn(null);
- assertThrows(KafkaException.class, () -> spyWriter.sendFailedRecordToDlq(records.get(0)));
- }
-
- @Test
- public void testSendFailedRecordToDlqSuccess() {
- TopicPartitionWriter writer = new TopicPartitionWriter(tp, mockClient, propsCsv, config, true, "dlq.topic.name", dlqMockProducer);
- TopicPartitionWriter spyWriter = spy(writer);
-
- spyWriter.open();
-
- SinkRecord testSinkRecord = new SinkRecord(tp.topic(), tp.partition(), null, null, Schema.STRING_SCHEMA, "{'also':'stringy','sortof':'message'}", 4);
-
- byte[] recordKey = String.format("Failed to write record to KustoDB with the following kafka coordinates, "
- + "topic=%s, partition=%s, offset=%s.",
- testSinkRecord.topic(),
- testSinkRecord.kafkaPartition(),
- testSinkRecord.kafkaOffset()).getBytes(StandardCharsets.UTF_8);
- byte[] recordValue = testSinkRecord.value().toString().getBytes(StandardCharsets.UTF_8);
- ProducerRecord dlqRecord = new ProducerRecord<>("dlq.topic.name", recordKey, recordValue);
-
- // when(kafkaProducer.send(dlqRecord,anyObject())).thenReturn(null);
-
- dlqMockProducer.send(dlqRecord);
-
- List> history = dlqMockProducer.history();
-
- List> expected = Collections.singletonList(dlqRecord);
-
- Assertions.assertEquals(expected, history);
-
- }
-
- private Map getKustoConfigs(String basePath, long fileThreshold, long flushInterval) {
- Map settings = new HashMap<>();
- settings.put(KustoSinkConfig.KUSTO_INGEST_URL_CONF, KUSTO_INGEST_CLUSTER_URL);
- settings.put(KustoSinkConfig.KUSTO_ENGINE_URL_CONF, KUSTO_CLUSTER_URL);
- settings.put(KustoSinkConfig.KUSTO_TABLES_MAPPING_CONF, "mapping");
- settings.put(KustoSinkConfig.KUSTO_AUTH_APPID_CONF, "some-appid");
- settings.put(KustoSinkConfig.KUSTO_AUTH_APPKEY_CONF, "some-appkey");
- settings.put(KustoSinkConfig.KUSTO_AUTH_AUTHORITY_CONF, "some-authority");
- settings.put(KustoSinkConfig.KUSTO_SINK_TEMP_DIR_CONF, basePath);
- settings.put(KustoSinkConfig.KUSTO_SINK_FLUSH_SIZE_BYTES_CONF, String.valueOf(fileThreshold));
- settings.put(KustoSinkConfig.KUSTO_SINK_FLUSH_INTERVAL_MS_CONF, String.valueOf(flushInterval));
- return settings;
- }
-}
+package com.microsoft.azure.kusto.kafka.connect.sink;
+
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+import java.util.*;
+import java.util.function.Function;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.io.IOUtils;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.MockProducer;
+import org.apache.kafka.clients.producer.Producer;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.serialization.ByteArraySerializer;
+import org.apache.kafka.connect.data.Schema;
+import org.apache.kafka.connect.sink.SinkRecord;
+import org.junit.jupiter.api.*;
+import org.mockito.ArgumentCaptor;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.microsoft.azure.kusto.ingest.IngestClient;
+import com.microsoft.azure.kusto.ingest.IngestionProperties;
+import com.microsoft.azure.kusto.ingest.source.FileSourceInfo;
+
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.fail;
+import static org.mockito.Mockito.*;
+
+@Disabled
+// TODO parts of this test needs to be re-formatted and may need rewriting
+public class TopicPartitionWriterTest {
+ private static final Logger log = LoggerFactory.getLogger(TopicPartitionWriterTest.class);
+
+ private static final String KUSTO_INGEST_CLUSTER_URL = "https://ingest-cluster.kusto.windows.net";
+ private static final String KUSTO_CLUSTER_URL = "https://cluster.kusto.windows.net";
+ private static final String DATABASE = "testdb1";
+ private static final String TABLE = "testtable1";
+ private static final long fileThreshold = 100;
+ private static final long flushInterval = 5000;
+ private static final IngestClient mockClient = mock(IngestClient.class);
+ private static final TopicIngestionProperties propsCsv = new TopicIngestionProperties();
+ private static final TopicPartition tp = new TopicPartition("testPartition", 11);
+ private static final long contextSwitchInterval = 200;
+ private static KustoSinkConfig config;
+ // TODO: should probably find a better way to mock internal class (FileWriter)...
+ private File currentDirectory;
+ private String basePathCurrent;
+ private boolean isDlqEnabled;
+ private String dlqTopicName;
+ private Producer kafkaProducer;
+ private MockProducer dlqMockProducer;
+
+ @BeforeAll
+ public static void beforeClass() {
+ propsCsv.ingestionProperties = new IngestionProperties(DATABASE, TABLE);
+ propsCsv.ingestionProperties.setDataFormat(IngestionProperties.DataFormat.JSON);
+ }
+
+ @BeforeEach
+ public final void before() {
+ currentDirectory = Utils.getCurrentWorkingDirectory();
+ Properties properties = new Properties();
+ properties.put("bootstrap.servers", "localhost:9000");
+ properties.put("key.serializer", "org.apache.kafka.common.serialization.ByteArraySerializer");
+ properties.put("value.serializer", "org.apache.kafka.common.serialization.ByteArraySerializer");
+ kafkaProducer = new KafkaProducer<>(properties);
+ isDlqEnabled = false;
+ dlqTopicName = null;
+ dlqMockProducer = new MockProducer<>(
+ true, new ByteArraySerializer(), new ByteArraySerializer());
+ basePathCurrent = Paths.get(currentDirectory.getPath(), "testWriteStringyValuesAndOffset").toString();
+ Map settings = getKustoConfigs(basePathCurrent, fileThreshold, flushInterval);
+ config = new KustoSinkConfig(settings);
+ }
+
+ @AfterEach
+ public final void afterEach() {
+ FileUtils.deleteQuietly(currentDirectory);
+ }
+
+ @Test
+ public void testHandleRollFile() {
+ IngestClient mockedClient = mock(IngestClient.class);
+ TopicIngestionProperties props = new TopicIngestionProperties();
+ props.ingestionProperties = new IngestionProperties(DATABASE, TABLE);
+ TopicPartitionWriter writer = new TopicPartitionWriter(tp, mockedClient, props, config, isDlqEnabled, dlqTopicName, kafkaProducer);
+ SourceFile descriptor = new SourceFile();
+ descriptor.rawBytes = 1024;
+ writer.handleRollFile(descriptor);
+ ArgumentCaptor fileSourceInfoArgument = ArgumentCaptor.forClass(FileSourceInfo.class);
+ ArgumentCaptor ingestionPropertiesArgumentCaptor = ArgumentCaptor.forClass(IngestionProperties.class);
+ try {
+ verify(mockedClient, only()).ingestFromFile(fileSourceInfoArgument.capture(), ingestionPropertiesArgumentCaptor.capture());
+ } catch (Exception e) {
+ log.error("Error running testHandleRollFile", e);
+ fail(e);
+ }
+
+ Assertions.assertEquals(fileSourceInfoArgument.getValue().getFilePath(), descriptor.path);
+ Assertions.assertEquals(TABLE, ingestionPropertiesArgumentCaptor.getValue().getTableName());
+ Assertions.assertEquals(DATABASE, ingestionPropertiesArgumentCaptor.getValue().getDatabaseName());
+ Assertions.assertEquals(1024, fileSourceInfoArgument.getValue().getRawSizeInBytes());
+ }
+
+ @Test
+ public void testHandleRollFileWithStreamingEnabled() {
+ IngestClient mockedClient = mock(IngestClient.class);
+ TopicIngestionProperties props = new TopicIngestionProperties();
+ props.ingestionProperties = new IngestionProperties(DATABASE, TABLE);
+ props.streaming = true;
+ TopicPartitionWriter writer = new TopicPartitionWriter(tp, mockedClient, props, config, isDlqEnabled, dlqTopicName, kafkaProducer);
+
+ SourceFile descriptor = new SourceFile();
+ descriptor.rawBytes = 1024;
+
+ writer.handleRollFile(descriptor);
+ ArgumentCaptor fileSourceInfoArgument = ArgumentCaptor.forClass(FileSourceInfo.class);
+ ArgumentCaptor ingestionPropertiesArgumentCaptor = ArgumentCaptor.forClass(IngestionProperties.class);
+ try {
+ verify(mockedClient, only()).ingestFromFile(fileSourceInfoArgument.capture(), ingestionPropertiesArgumentCaptor.capture());
+ } catch (Exception e) {
+ log.error("Error running testHandleRollFile", e);
+ fail(e);
+ }
+
+ Assertions.assertEquals(fileSourceInfoArgument.getValue().getFilePath(), descriptor.path);
+ Assertions.assertEquals(TABLE, ingestionPropertiesArgumentCaptor.getValue().getTableName());
+ Assertions.assertEquals(DATABASE, ingestionPropertiesArgumentCaptor.getValue().getDatabaseName());
+ Assertions.assertEquals(1024, fileSourceInfoArgument.getValue().getRawSizeInBytes());
+ }
+
+ @Test
+ public void testGetFilename() {
+ try {
+ TopicPartitionWriter writer = new TopicPartitionWriter(tp, mockClient, propsCsv, config, isDlqEnabled, dlqTopicName, kafkaProducer);
+ File writerFile = new File(writer.getFilePath(null));
+ Assertions.assertEquals("kafka_testPartition_11_0.json.gz", writerFile.getName());
+ } catch (Exception ex) {
+ // In case there is an accessor exception getting the file
+ fail(ex);
+ }
+ }
+
+ @Test
+ public void testGetFilenameAfterOffsetChanges() {
+ TopicPartitionWriter writer = new TopicPartitionWriter(tp, mockClient, propsCsv, config, isDlqEnabled, dlqTopicName, kafkaProducer);
+ writer.open();
+ List records = new ArrayList<>();
+ records.add(new SinkRecord(tp.topic(), tp.partition(), null, null, Schema.STRING_SCHEMA, "another,stringy,message", 5));
+ records.add(new SinkRecord(tp.topic(), tp.partition(), null, null, Schema.STRING_SCHEMA, "{'also':'stringy','sortof':'message'}", 4));
+ for (SinkRecord record : records) {
+ writer.writeRecord(record);
+ }
+ try {
+ File writerFile = new File(writer.getFilePath(null));
+ Assertions.assertTrue(writerFile.exists());
+ Assertions.assertEquals("kafka_testPartition_11_4.json.gz", (new File(writer.getFilePath(null))).getName());
+ } catch (Exception ex) {
+ // In case there is an accessor exception getting the file
+ fail(ex);
+ }
+ }
+
+ @Test
+ public void testOpenClose() {
+ TopicPartitionWriter writer = new TopicPartitionWriter(tp, mockClient, propsCsv, config, isDlqEnabled, dlqTopicName, kafkaProducer);
+ writer.open();
+ writer.close();
+ }
+
+ @Test
+ public void testWriteNonStringAndOffset() {
+ // String db = "testdb1";
+ // String table = "testtable1";
+ //
+ // TopicPartitionWriter writer = new TopicPartitionWriter(tp, mockClient, db, table, basePath, fileThreshold);
+ //
+ // List records = new ArrayList();
+ // DummyRecord dummyRecord1 = new DummyRecord(1, "a", (long) 2);
+ // DummyRecord dummyRecord2 = new DummyRecord(2, "b", (long) 4);
+ //
+ // records.add(new SinkRecord("topic", 1, null, null, null, dummyRecord1, 10));
+ // records.add(new SinkRecord("topic", 2, null, null, null, dummyRecord2, 3));
+ // records.add(new SinkRecord("topic", 2, null, null, null, dummyRecord2, 4));
+ //
+ // for (SinkRecord record : records) {
+ // writer.writeRecord(record);
+ // }
+ //
+ // Assert.assertEquals(writer.getFilePath(), "kafka_testPartition_11_0");
+ }
+
+ @Test
+ public void testWriteStringyValuesAndOffset() {
+ TopicPartitionWriter writer = new TopicPartitionWriter(tp, mockClient, propsCsv, config, isDlqEnabled, dlqTopicName, kafkaProducer);
+ writer.open();
+ List records = new ArrayList<>();
+ records.add(new SinkRecord(tp.topic(), tp.partition(), null, null, Schema.STRING_SCHEMA, "another,stringy,message", 3));
+ records.add(new SinkRecord(tp.topic(), tp.partition(), null, null, Schema.STRING_SCHEMA, "{'also':'stringy','sortof':'message'}", 4));
+ for (SinkRecord record : records) {
+ writer.writeRecord(record);
+ }
+ Assertions.assertTrue((new File(writer.fileWriter.currentFile.path)).exists());
+ Assertions.assertEquals(String.format("kafka_%s_%d_%d.%s.gz", tp.topic(), tp.partition(), 4,
+ IngestionProperties.DataFormat.JSON.name().toLowerCase(Locale.ENGLISH)),
+ (new File(writer.fileWriter.currentFile.path)).getName());
+ writer.close();
+ }
+
+ @Test
+ public void testWriteStringValuesAndOffset() throws IOException {
+ String[] messages = new String[] {"stringy message", "another,stringy,message", "{'also':'stringy','sortof':'message'}"};
+
+ // Expect to finish file after writing forth message cause of fileThreshold
+ long fileThreshold2 = messages[0].length() + messages[1].length() + messages[2].length() + messages[2].length() - 1;
+ Map settings2 = getKustoConfigs(basePathCurrent, fileThreshold2, flushInterval);
+ KustoSinkConfig config2 = new KustoSinkConfig(settings2);
+ TopicPartitionWriter writer = new TopicPartitionWriter(tp, mockClient, propsCsv, config2, isDlqEnabled, dlqTopicName, kafkaProducer);
+
+ writer.open();
+ List records = new ArrayList<>();
+ records.add(new SinkRecord(tp.topic(), tp.partition(), null, null, Schema.STRING_SCHEMA, messages[0], 10));
+ records.add(new SinkRecord(tp.topic(), tp.partition(), null, null, Schema.STRING_SCHEMA, messages[1], 13));
+ records.add(new SinkRecord(tp.topic(), tp.partition(), null, null, Schema.STRING_SCHEMA, messages[2], 14));
+ records.add(new SinkRecord(tp.topic(), tp.partition(), null, null, Schema.STRING_SCHEMA, messages[2], 15));
+ records.add(new SinkRecord(tp.topic(), tp.partition(), null, null, Schema.STRING_SCHEMA, messages[2], 16));
+
+ for (SinkRecord record : records) {
+ writer.writeRecord(record);
+ }
+
+ Assertions.assertEquals(16, (long) writer.lastCommittedOffset);
+ Assertions.assertEquals(16, writer.currentOffset);
+
+ String currentFileName = writer.fileWriter.currentFile.path;
+ Assertions.assertTrue(new File(currentFileName).exists());
+ Assertions.assertEquals(String.format("kafka_%s_%d_%d.%s.gz", tp.topic(),
+ tp.partition(), 16, IngestionProperties.DataFormat.JSON.name().toLowerCase(Locale.ENGLISH)),
+ (new File(currentFileName)).getName());
+
+ // Read
+ writer.fileWriter.finishFile(false);
+ Function assertFileConsumer = FileWriterTest.getAssertFileConsumerFunction(messages[2] + "\n");
+ assertFileConsumer.apply(writer.fileWriter.currentFile);
+ writer.close();
+ }
+
+ @Test
+ public void testWriteBytesValuesAndOffset() throws IOException {
+ byte[] message = IOUtils.toByteArray(
+ Objects.requireNonNull(this.getClass().getClassLoader().getResourceAsStream("data.avro")));
+ ByteArrayOutputStream o = new ByteArrayOutputStream();
+ o.write(message);
+ // Expect to finish file with one record although fileThreshold is high
+ long fileThreshold2 = 128;
+ TopicIngestionProperties propsAvro = new TopicIngestionProperties();
+ propsAvro.ingestionProperties = new IngestionProperties(DATABASE, TABLE);
+ propsAvro.ingestionProperties.setDataFormat(IngestionProperties.DataFormat.AVRO);
+ Map settings2 = getKustoConfigs(basePathCurrent, fileThreshold2, flushInterval);
+ KustoSinkConfig config2 = new KustoSinkConfig(settings2);
+ TopicPartitionWriter writer = new TopicPartitionWriter(tp, mockClient, propsAvro, config2, isDlqEnabled, dlqTopicName, kafkaProducer);
+
+ writer.open();
+ List records = new ArrayList<>();
+ records.add(new SinkRecord(tp.topic(), tp.partition(), null, null, Schema.BYTES_SCHEMA, o.toByteArray(), 10));
+
+ for (SinkRecord record : records) {
+ writer.writeRecord(record);
+ }
+
+ Assertions.assertEquals(10, (long) writer.lastCommittedOffset);
+ Assertions.assertEquals(10, writer.currentOffset);
+
+ String currentFileName = writer.fileWriter.currentFile.path;
+
+ Assertions.assertTrue(new File(currentFileName).exists());
+ Assertions.assertEquals(String.format("kafka_%s_%d_%d.%s.gz", tp.topic(), tp.partition(),
+ 10, IngestionProperties.DataFormat.JSON.name().toLowerCase(Locale.ENGLISH)),
+ (new File(currentFileName)).getName());
+ writer.close();
+ }
+
+ @Test
+ public void testClose() throws InterruptedException {
+ TopicPartitionWriter writer = new TopicPartitionWriter(tp, mockClient, propsCsv, config, isDlqEnabled, dlqTopicName, kafkaProducer);
+ TopicPartitionWriter spyWriter = spy(writer);
+
+ spyWriter.open();
+ List records = new ArrayList<>();
+
+ records.add(new SinkRecord(tp.topic(), tp.partition(), null, null, Schema.STRING_SCHEMA, "another,stringy,message", 5));
+ records.add(new SinkRecord(tp.topic(), tp.partition(), null, null, Schema.STRING_SCHEMA, "{'also':'stringy','sortof':'message'}", 4));
+
+ for (SinkRecord record : records) {
+ spyWriter.writeRecord(record);
+ }
+ // 2 records are waiting to be ingested - expect close to revoke them so that even after 5 seconds it won't ingest
+ Assertions.assertNotNull(spyWriter.lastCommittedOffset);
+ spyWriter.close();
+ Thread.sleep(flushInterval + contextSwitchInterval);
+ Assertions.assertNotNull(spyWriter.lastCommittedOffset);
+ }
+
+ @Disabled
+ @Test
+ public void testSendFailedRecordToDlqError() {
+ TopicPartitionWriter writer = new TopicPartitionWriter(tp, mockClient, propsCsv, config, true, "dlq.topic.name", kafkaProducer);
+ TopicPartitionWriter spyWriter = spy(writer);
+ // TODO this is to be re-worked
+ kafkaProducer = mock(Producer.class);
+ spyWriter.open();
+ List records = new ArrayList<>();
+ records.add(new SinkRecord(tp.topic(), tp.partition(), null, null, Schema.STRING_SCHEMA, "another,stringy,message", 5));
+ records.add(new SinkRecord(tp.topic(), tp.partition(), null, null, Schema.STRING_SCHEMA, "{'also':'stringy','sortof':'message'}", 4));
+ when(kafkaProducer.send(any(), any())).thenReturn(null);
+ assertThrows(KafkaException.class, () -> spyWriter.sendFailedRecordToDlq(records.get(0)));
+ }
+
+ @Disabled
+ @Test
+ public void testSendFailedRecordToDlqSuccess() {
+ TopicPartitionWriter writer = new TopicPartitionWriter(tp, mockClient, propsCsv, config, true, "dlq.topic.name", dlqMockProducer);
+ TopicPartitionWriter spyWriter = spy(writer);
+
+ spyWriter.open();
+
+ SinkRecord testSinkRecord = new SinkRecord(tp.topic(), tp.partition(), null, null, Schema.STRING_SCHEMA, "{'also':'stringy','sortof':'message'}", 4);
+
+ byte[] recordKey = String.format("Failed to write record to KustoDB with the following kafka coordinates, "
+ + "topic=%s, partition=%s, offset=%s.",
+ testSinkRecord.topic(),
+ testSinkRecord.kafkaPartition(),
+ testSinkRecord.kafkaOffset()).getBytes(StandardCharsets.UTF_8);
+ byte[] recordValue = testSinkRecord.value().toString().getBytes(StandardCharsets.UTF_8);
+ ProducerRecord dlqRecord = new ProducerRecord<>("dlq.topic.name", recordKey, recordValue);
+
+ // when(kafkaProducer.send(dlqRecord,anyObject())).thenReturn(null);
+
+ dlqMockProducer.send(dlqRecord);
+
+ List> history = dlqMockProducer.history();
+
+ List> expected = Collections.singletonList(dlqRecord);
+
+ Assertions.assertEquals(expected, history);
+
+ }
+
+ private Map getKustoConfigs(String basePath, long fileThreshold, long flushInterval) {
+ Map settings = new HashMap<>();
+ settings.put(KustoSinkConfig.KUSTO_INGEST_URL_CONF, KUSTO_INGEST_CLUSTER_URL);
+ settings.put(KustoSinkConfig.KUSTO_ENGINE_URL_CONF, KUSTO_CLUSTER_URL);
+ settings.put(KustoSinkConfig.KUSTO_TABLES_MAPPING_CONF, "mapping");
+ settings.put(KustoSinkConfig.KUSTO_AUTH_APPID_CONF, "some-appid");
+ settings.put(KustoSinkConfig.KUSTO_AUTH_APPKEY_CONF, "some-appkey");
+ settings.put(KustoSinkConfig.KUSTO_AUTH_AUTHORITY_CONF, "some-authority");
+ settings.put(KustoSinkConfig.KUSTO_SINK_TEMP_DIR_CONF, basePath);
+ settings.put(KustoSinkConfig.KUSTO_SINK_FLUSH_SIZE_BYTES_CONF, String.valueOf(fileThreshold));
+ settings.put(KustoSinkConfig.KUSTO_SINK_FLUSH_INTERVAL_MS_CONF, String.valueOf(flushInterval));
+ return settings;
+ }
+}
diff --git a/src/test/java/com/microsoft/azure/kusto/kafka/connect/sink/formatWriter/AvroRecordWriterTest.java b/src/test/java/com/microsoft/azure/kusto/kafka/connect/sink/formatWriter/AvroRecordWriterTest.java
deleted file mode 100644
index 4676ebf0..00000000
--- a/src/test/java/com/microsoft/azure/kusto/kafka/connect/sink/formatWriter/AvroRecordWriterTest.java
+++ /dev/null
@@ -1,66 +0,0 @@
-package com.microsoft.azure.kusto.kafka.connect.sink.formatWriter;
-
-import java.io.File;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.nio.file.Files;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.avro.file.DataFileReader;
-import org.apache.avro.generic.GenericData;
-import org.apache.avro.generic.GenericDatumReader;
-import org.apache.commons.io.FileUtils;
-import org.apache.kafka.connect.data.Schema;
-import org.apache.kafka.connect.data.SchemaBuilder;
-import org.apache.kafka.connect.data.Struct;
-import org.apache.kafka.connect.sink.SinkRecord;
-import org.junit.jupiter.api.Test;
-
-import com.microsoft.azure.kusto.kafka.connect.sink.Utils;
-import com.microsoft.azure.kusto.kafka.connect.sink.format.RecordWriter;
-
-import static org.junit.jupiter.api.Assertions.assertEquals;
-
-public class AvroRecordWriterTest {
- @Test
- public void AvroDataWrite() throws IOException {
- List records = new ArrayList<>();
- final Schema schema = SchemaBuilder.struct()
- .field("text", SchemaBuilder.string().build())
- .field("id", SchemaBuilder.int32().build())
- .build();
-
- for (int i = 0; i < 10; i++) {
- final Struct struct = new Struct(schema)
- .put("text", String.format("record-%s", i))
- .put("id", i);
- records.add(new SinkRecord("mytopic", 0, null, null, schema, struct, 10));
- }
- File file = new File("abc.avro");
- Utils.restrictPermissions(file);
- AvroRecordWriterProvider writer = new AvroRecordWriterProvider();
- OutputStream out = Files.newOutputStream(file.toPath());
- RecordWriter rd = writer.getRecordWriter(file.getPath(), out);
- for (SinkRecord record : records) {
- rd.write(record);
- }
- rd.commit();
- validate(file.getPath());
- FileUtils.deleteQuietly(file);
- }
-
- public void validate(String path) throws IOException {
- // Warns if the types are not generified
- GenericDatumReader datum = new GenericDatumReader<>();
- File file = new File(path);
- DataFileReader reader = new DataFileReader<>(file, datum);
- GenericData.Record record = new GenericData.Record(reader.getSchema());
- int i = 0;
- while (reader.hasNext()) {
- assertEquals(reader.next(record).toString(), String.format("{\"text\": \"record-%s\", \"id\": %s}", i, i));
- i++;
- }
- reader.close();
- }
-}
diff --git a/src/test/java/com/microsoft/azure/kusto/kafka/connect/sink/formatWriter/ByteArrayWriterProviderTest.java b/src/test/java/com/microsoft/azure/kusto/kafka/connect/sink/formatWriter/ByteArrayWriterProviderTest.java
deleted file mode 100644
index 4d289c96..00000000
--- a/src/test/java/com/microsoft/azure/kusto/kafka/connect/sink/formatWriter/ByteArrayWriterProviderTest.java
+++ /dev/null
@@ -1,50 +0,0 @@
-package com.microsoft.azure.kusto.kafka.connect.sink.formatWriter;
-
-import java.io.BufferedReader;
-import java.io.File;
-import java.io.FileReader;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.nio.file.Files;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.commons.io.FileUtils;
-import org.apache.kafka.connect.data.Schema;
-import org.apache.kafka.connect.sink.SinkRecord;
-import org.junit.jupiter.api.Test;
-
-import com.microsoft.azure.kusto.kafka.connect.sink.format.RecordWriter;
-
-import static com.microsoft.azure.kusto.kafka.connect.sink.Utils.restrictPermissions;
-import static org.junit.jupiter.api.Assertions.assertEquals;
-
-public class ByteArrayWriterProviderTest {
- @Test
- public void testByteData() throws IOException {
- List records = new ArrayList<>();
- for (int i = 0; i < 10; i++) {
- records.add(new SinkRecord("mytopic", 0, null, null, Schema.BYTES_SCHEMA,
- String.format("hello-%s", i).getBytes(), i));
- }
- File file = new File("abc.bin");
- restrictPermissions(file);
- file.deleteOnExit();
- try (OutputStream out = Files.newOutputStream(file.toPath());
- BufferedReader br = new BufferedReader(new FileReader(file))) {
- ByteRecordWriterProvider writer = new ByteRecordWriterProvider();
- RecordWriter rd = writer.getRecordWriter(file.getPath(), out);
- for (SinkRecord record : records) {
- rd.write(record);
- }
- rd.commit();
- String st;
- int i = 0;
- while ((st = br.readLine()) != null) {
- assertEquals(st, String.format("hello-%s", i));
- i++;
- }
- }
- FileUtils.deleteQuietly(file);
- }
-}
diff --git a/src/test/java/com/microsoft/azure/kusto/kafka/connect/sink/formatWriter/JsonRecordWriterProviderTest.java b/src/test/java/com/microsoft/azure/kusto/kafka/connect/sink/formatWriter/JsonRecordWriterProviderTest.java
deleted file mode 100644
index 0f4ea6de..00000000
--- a/src/test/java/com/microsoft/azure/kusto/kafka/connect/sink/formatWriter/JsonRecordWriterProviderTest.java
+++ /dev/null
@@ -1,53 +0,0 @@
-package com.microsoft.azure.kusto.kafka.connect.sink.formatWriter;
-
-import java.io.BufferedReader;
-import java.io.File;
-import java.io.FileReader;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.nio.file.Files;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.commons.io.FileUtils;
-import org.apache.kafka.connect.sink.SinkRecord;
-import org.junit.jupiter.api.Test;
-
-import com.microsoft.azure.kusto.kafka.connect.sink.Utils;
-import com.microsoft.azure.kusto.kafka.connect.sink.format.RecordWriter;
-
-import static org.junit.jupiter.api.Assertions.assertEquals;
-
-// TODO: Significant duplication among these 4 classes
-public class JsonRecordWriterProviderTest {
- @Test
- public void testJsonData() throws IOException {
- List records = new ArrayList<>();
- for (int i = 0; i < 10; i++) {
- Map map = new HashMap<>();
- map.put("hello", i);
- records.add(new SinkRecord("mytopic", 0, null, null, null, map, i));
- }
- File file = new File("abc.json");
- file.deleteOnExit();
- Utils.restrictPermissions(file);
- JsonRecordWriterProvider jsonWriter = new JsonRecordWriterProvider();
- try (OutputStream out = Files.newOutputStream(file.toPath());
- BufferedReader br = new BufferedReader(new FileReader(file))) {
- RecordWriter rd = jsonWriter.getRecordWriter(file.getPath(), out);
- for (SinkRecord record : records) {
- rd.write(record);
- }
- rd.commit();
- String st;
- int i = 0;
- while ((st = br.readLine()) != null) {
- assertEquals(st, String.format("{\"hello\":%s}", i));
- i++;
- }
- FileUtils.deleteQuietly(file);
- }
- }
-}
diff --git a/src/test/java/com/microsoft/azure/kusto/kafka/connect/sink/formatWriter/StringRecordWriterProviderTest.java b/src/test/java/com/microsoft/azure/kusto/kafka/connect/sink/formatWriter/StringRecordWriterProviderTest.java
deleted file mode 100644
index 8539bed9..00000000
--- a/src/test/java/com/microsoft/azure/kusto/kafka/connect/sink/formatWriter/StringRecordWriterProviderTest.java
+++ /dev/null
@@ -1,49 +0,0 @@
-package com.microsoft.azure.kusto.kafka.connect.sink.formatWriter;
-
-import java.io.BufferedReader;
-import java.io.File;
-import java.io.FileReader;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.nio.file.Files;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.commons.io.FileUtils;
-import org.apache.kafka.connect.data.Schema;
-import org.apache.kafka.connect.sink.SinkRecord;
-import org.junit.jupiter.api.Test;
-
-import com.microsoft.azure.kusto.kafka.connect.sink.Utils;
-import com.microsoft.azure.kusto.kafka.connect.sink.format.RecordWriter;
-
-import static org.junit.jupiter.api.Assertions.assertEquals;
-
-public class StringRecordWriterProviderTest {
- @Test
- public void testStringData() throws IOException {
- List records = new ArrayList<>();
- for (int i = 0; i < 10; i++) {
- records.add(
- new SinkRecord("mytopic", 0, null, null, Schema.STRING_SCHEMA, String.format("hello-%s", i), i));
- }
- File file = new File("abc.txt");
- Utils.restrictPermissions(file);
- try (OutputStream out = Files.newOutputStream(file.toPath());
- BufferedReader br = new BufferedReader(new FileReader(file))) {
- StringRecordWriterProvider writer = new StringRecordWriterProvider();
- RecordWriter rd = writer.getRecordWriter(file.getPath(), out);
- for (SinkRecord record : records) {
- rd.write(record);
- }
- rd.commit();
- String st;
- int i = 0;
- while ((st = br.readLine()) != null) {
- assertEquals(st, String.format("hello-%s", i));
- i++;
- }
- }
- FileUtils.deleteQuietly(file);
- }
-}
diff --git a/src/test/java/com/microsoft/azure/kusto/kafka/connect/sink/formatwriter/KustoRecordWriterBase.java b/src/test/java/com/microsoft/azure/kusto/kafka/connect/sink/formatwriter/KustoRecordWriterBase.java
new file mode 100644
index 00000000..d95d417b
--- /dev/null
+++ b/src/test/java/com/microsoft/azure/kusto/kafka/connect/sink/formatwriter/KustoRecordWriterBase.java
@@ -0,0 +1,49 @@
+package com.microsoft.azure.kusto.kafka.connect.sink.formatwriter;
+
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.List;
+import java.util.Map;
+
+import org.json.JSONException;
+import org.junit.jupiter.api.Assertions;
+import org.skyscreamer.jsonassert.JSONAssert;
+
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.SerializationFeature;
+
+public abstract class KustoRecordWriterBase {
+ protected static final String KEYS = "keys";
+ protected static final String HEADERS = "headers";
+ protected static final String KAFKA_MD = "kafkamd";
+ protected static final ObjectMapper RESULT_MAPPER = new ObjectMapper().disable(SerializationFeature.FAIL_ON_EMPTY_BEANS);
+ protected static final TypeReference> GENERIC_MAP = new TypeReference>() {};
+
+ protected void validate(String actualFilePath, Map expectedResultsMap) throws IOException, JSONException {
+ // Warns if the types are not generified
+ List actualJson = Files.readAllLines(Paths.get(actualFilePath));
+ for (int i = 0; i < actualJson.size(); i++) {
+ String actual = actualJson.get(i);
+ Map actualMap = RESULT_MAPPER.readValue(actual, GENERIC_MAP);
+ String[] expected = expectedResultsMap.get(i);
+ String actualKeys = RESULT_MAPPER.writeValueAsString(actualMap.get(KEYS));
+ String actualHeaders = RESULT_MAPPER.writeValueAsString(actualMap.get(HEADERS));
+ JSONAssert.assertEquals(expected[1], actualKeys, false);
+ JSONAssert.assertEquals(expected[0], actualHeaders, false);
+ // to get the values it is to remove keys and headers , then get all the fields and compare
+ actualMap.remove(KEYS);
+ actualMap.remove(HEADERS);
+ actualMap.remove(KAFKA_MD);
+ // Now actualMap contains only the value
+ String actualValues = RESULT_MAPPER.writeValueAsString(actualMap);
+ if (expected[2] == null) {
+ // there are no fields or no keys
+ Assertions.assertTrue(actualMap.keySet().isEmpty(), "Expected null value for tombstone record");
+ } else {
+ JSONAssert.assertEquals(expected[2], actualValues, false);
+ }
+ }
+ }
+}
diff --git a/src/test/java/com/microsoft/azure/kusto/kafka/connect/sink/formatwriter/KustoRecordWriterSchemaTests.java b/src/test/java/com/microsoft/azure/kusto/kafka/connect/sink/formatwriter/KustoRecordWriterSchemaTests.java
new file mode 100644
index 00000000..33c78613
--- /dev/null
+++ b/src/test/java/com/microsoft/azure/kusto/kafka/connect/sink/formatwriter/KustoRecordWriterSchemaTests.java
@@ -0,0 +1,198 @@
+package com.microsoft.azure.kusto.kafka.connect.sink.formatwriter;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.file.Files;
+import java.util.*;
+import java.util.stream.Stream;
+
+import org.apache.avro.generic.GenericData;
+import org.apache.commons.io.FileUtils;
+import org.apache.kafka.connect.data.Schema;
+import org.apache.kafka.connect.data.SchemaBuilder;
+import org.apache.kafka.connect.sink.SinkRecord;
+import org.jetbrains.annotations.NotNull;
+import org.json.JSONException;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
+import org.junit.jupiter.params.provider.CsvSource;
+import org.junit.jupiter.params.provider.MethodSource;
+
+import com.microsoft.azure.kusto.ingest.IngestionProperties;
+import com.microsoft.azure.kusto.kafka.connect.sink.Utils;
+import com.microsoft.azure.kusto.kafka.connect.sink.format.RecordWriter;
+
+import io.confluent.avro.random.generator.Generator;
+import io.confluent.connect.avro.AvroData;
+import tech.allegro.schema.json2avro.converter.JsonAvroConverter;
+
+public class KustoRecordWriterSchemaTests extends KustoRecordWriterBase {
+ private static @NotNull Stream testMapSchemaJson() {
+ // Key schema, value schema, expectedKey, expectedValue
+ Schema intToIntSchema = SchemaBuilder.map(Schema.INT32_SCHEMA, Schema.INT32_SCHEMA).name("IntToIntMap").build();
+ Schema stringToIntSchema = SchemaBuilder.map(Schema.STRING_SCHEMA, Schema.INT32_SCHEMA).name("StringToIntMap").build();
+ Schema stringToOptionalIntSchema = SchemaBuilder.map(Schema.STRING_SCHEMA, Schema.OPTIONAL_INT32_SCHEMA).name("StringToOptionalInt").build();
+ Schema arrayOfInts = SchemaBuilder.array(Schema.INT32_SCHEMA).name("ArrayOfInts").build();
+ Schema simpleLongSchema = SchemaBuilder.struct().field("recordKey", Schema.INT64_SCHEMA).name("SimpleLongSchema").build();
+ Schema structSchema = SchemaBuilder.struct().field("field1", Schema.BOOLEAN_SCHEMA)
+ .field("field2", Schema.STRING_SCHEMA).name("StructSchema").build();
+
+ Map intToIntMap = Collections.singletonMap(0, 12);
+ Map stringToIntMap = Collections.singletonMap("String-42", 32);
+ Map stringToOptionalIntMap = Collections.singletonMap("NullString-42", null);
+ Map stringToOptionalIntMapMultiple = new HashMap<>();
+ stringToOptionalIntMapMultiple.put("NullString-42", null);
+ stringToOptionalIntMapMultiple.put("String-42", 32);
+
+ return Stream.of(
+ Arguments.of(intToIntSchema, stringToIntSchema, intToIntMap, stringToIntMap, false, false),
+ Arguments.of(stringToIntSchema, stringToOptionalIntSchema, stringToIntMap, stringToOptionalIntMap, false, false),
+ Arguments.of(stringToIntSchema, stringToOptionalIntSchema, stringToIntMap, stringToOptionalIntMapMultiple, false, false),
+ Arguments.of(stringToIntSchema, arrayOfInts, stringToIntMap, new Integer[] {1, 2, 3, 5, 8, 13, 21}, false, true),
+ Arguments.of(simpleLongSchema, structSchema, Collections.singletonMap("recordKey", 42L),
+ "{\"field1\":true,\"field2\":\"Field-@42\"}", false, false),
+ Arguments.of(simpleLongSchema, structSchema, Collections.singletonMap("recordKey", 42L), null, false, false));
+ }
+
+ @ParameterizedTest(name = "AVRO Data to be serialized with key schema {0} and value schema {1} isSimpleKey {2} isSimpleValue {3}")
+ @CsvSource({
+ "avro-simple-schema.json,avro-struct-schema.json,true,false",
+ "avro-struct-schema.json,avro-struct-schema.json,false,false",
+ "avro-simple-schema.json,avro-simple-schema.json,true,true"
+ })
+ public void validateAvroDataToBeSerialized(String keySchemaPath, String valueSchemaPath, boolean isSimpleKey, boolean isSimpleValue)
+ throws IOException, JSONException {
+ List records = new ArrayList<>();
+ Generator randomAvroValueData = new Generator.Builder().schemaStream(
+ Objects.requireNonNull(this.getClass().getClassLoader().getResourceAsStream(String.format("avro-schemas/%s", valueSchemaPath)))).build();
+ Generator randomAvroKeyData = new Generator.Builder().schemaStream(
+ Objects.requireNonNull(this.getClass().getClassLoader().getResourceAsStream(String.format("avro-schemas/%s", keySchemaPath)))).build();
+ AvroData avroDataCache = new AvroData(50);
+ Schema keySchema = avroDataCache.toConnectSchema(randomAvroKeyData.schema());
+ Schema valueSchema = avroDataCache.toConnectSchema(randomAvroValueData.schema());
+ JsonAvroConverter converter = new JsonAvroConverter();
+ Map expectedResultsMap = new HashMap<>();
+ for (int i = 0; i < 10; i++) {
+ Object key = randomAvroKeyData.generate();
+ Object value = randomAvroValueData.generate();
+ SinkRecord sinkRecord = new SinkRecord("avro.record.topic", i % 3,
+ keySchema,
+ key,
+ valueSchema,
+ value,
+ i);
+ sinkRecord.headers().addInt(String.format("HeaderInt-%s", i), i);
+ records.add(sinkRecord);
+ String expectedValueString = isSimpleValue ? RESULT_MAPPER.writeValueAsString(Collections.singletonMap("value", value))
+ : new String(converter.convertToJson((GenericData.Record) value));
+ String expectedKeyString = isSimpleKey ? key.toString() : new String(converter.convertToJson((GenericData.Record) key));
+ String expectedHeaderJson = RESULT_MAPPER.writeValueAsString(Collections.singletonMap(String.format("HeaderInt-%s", i), i));
+ expectedResultsMap.put(i, new String[] {expectedHeaderJson, expectedKeyString, expectedValueString});
+ }
+ File file = new File(String.format("%s.%s", UUID.randomUUID(), "json"));
+ Utils.restrictPermissions(file);
+ KustoRecordWriterProvider writer = new KustoRecordWriterProvider();
+ OutputStream out = Files.newOutputStream(file.toPath());
+ RecordWriter rd = writer.getRecordWriter(file.getPath(), out);
+ for (SinkRecord record : records) {
+ rd.write(record, IngestionProperties.DataFormat.AVRO);
+ }
+ rd.commit();
+ validate(file.getPath(), expectedResultsMap);
+ rd.close();
+ FileUtils.deleteQuietly(file);
+ }
+
+ // Idea is to use Avro Schema to generate Avro data and convert them to random JSON for tests
+ @ParameterizedTest(name = "JSON Data to be serialized with key schema {0} and value schema {1} isSimpleKey {2} isSimpleValue {3}")
+ @CsvSource({
+ "avro-simple-schema.json,avro-struct-schema.json,true,false",
+ "avro-struct-schema.json,avro-struct-schema.json,false,false",
+ "avro-simple-schema.json,avro-simple-schema.json,true,true"
+ })
+ public void validateJsonDataToBeSerialized(String keySchemaPath, String valueSchemaPath, boolean isSimpleKey, boolean isSimpleValue)
+ throws IOException, JSONException {
+ List records = new ArrayList<>();
+ Generator randomAvroValueData = new Generator.Builder().schemaStream(
+ Objects.requireNonNull(this.getClass().getClassLoader().getResourceAsStream(String.format("avro-schemas/%s", valueSchemaPath)))).build();
+ Generator randomAvroKeyData = new Generator.Builder().schemaStream(
+ Objects.requireNonNull(this.getClass().getClassLoader().getResourceAsStream(String.format("avro-schemas/%s", keySchemaPath)))).build();
+
+ Map expectedResultsMap = new HashMap<>();
+ for (int i = 0; i < 10; i++) {
+ Object key = randomAvroKeyData.generate().toString();
+ Object value = randomAvroValueData.generate().toString();
+ SinkRecord sinkRecord = new SinkRecord("json.record.topic", i % 3,
+ null,
+ key,
+ null,
+ value,
+ i);
+ sinkRecord.headers().addInt(String.format("HeaderInt-%s", i), i);
+ records.add(sinkRecord);
+
+ String expectedValueString = isSimpleValue ? RESULT_MAPPER.writeValueAsString(Collections.singletonMap("value", value)) : value.toString();
+ String expectedKeyString = isSimpleKey ? RESULT_MAPPER.writeValueAsString(key) : key.toString();
+ String expectedHeaderJson = RESULT_MAPPER.writeValueAsString(Collections.singletonMap(String.format("HeaderInt-%s", i), i));
+ expectedResultsMap.put(i, new String[] {expectedHeaderJson, expectedKeyString, expectedValueString});
+ }
+ File file = new File(String.format("%s.%s", UUID.randomUUID(), "json"));
+ Utils.restrictPermissions(file);
+ KustoRecordWriterProvider writer = new KustoRecordWriterProvider();
+ OutputStream out = Files.newOutputStream(file.toPath());
+ RecordWriter rd = writer.getRecordWriter(file.getPath(), out);
+ for (SinkRecord record : records) {
+ rd.write(record, IngestionProperties.DataFormat.JSON);
+ }
+ rd.commit();
+ validate(file.getPath(), expectedResultsMap);
+ rd.close();
+ FileUtils.deleteQuietly(file);
+ }
+
+ @ParameterizedTest(name = "Map Data to be serialized with key schema {0}.name() and value schema {1}.name()")
+ @MethodSource("testMapSchemaJson")
+ public void collectionsSerializationTests(Schema keySchema, Schema valueSchema,
+ Map, ?> keyValues, Object expectedValues,
+ boolean isSimpleKey, boolean isSimpleValue) throws IOException, JSONException {
+ // Set up
+ Map expectedResultsMap = new HashMap<>();
+ SinkRecord sinkRecord = new SinkRecord("json.map.record.topic", 0,
+ keySchema,
+ keyValues,
+ valueSchema,
+ expectedValues,
+ 0);
+ sinkRecord.headers().addInt(String.format("HeaderInt-%s", 0), 0);
+ String expectedKeyString = isSimpleKey ? RESULT_MAPPER.writeValueAsString(Collections.singletonMap("key", keyValues))
+ : RESULT_MAPPER.writeValueAsString(keyValues);
+ // Sometimes the input is a JSON string. No need to double encode. Check the struct test
+ String expectedValueString;
+ if (expectedValues == null) {
+ expectedValueString = null;
+ } else if (expectedValues instanceof String) {
+ expectedValueString = expectedValues.toString();
+ } else if (isSimpleValue) {
+ expectedValueString = RESULT_MAPPER.writeValueAsString(Collections.singletonMap("value", expectedValues));
+ } else {
+ expectedValueString = RESULT_MAPPER.writeValueAsString(expectedValues);
+ }
+ String expectedHeaderJson = RESULT_MAPPER.writeValueAsString(
+ Collections.singletonMap(String.format("HeaderInt-%s", 0), 0));
+ expectedResultsMap.put(0, new String[] {expectedHeaderJson, expectedKeyString, expectedValueString});
+
+ // Act
+ File file = new File(String.format("%s.%s", UUID.randomUUID(), "json"));
+ Utils.restrictPermissions(file);
+ KustoRecordWriterProvider writer = new KustoRecordWriterProvider();
+ OutputStream out = Files.newOutputStream(file.toPath());
+ RecordWriter rd = writer.getRecordWriter(file.getPath(), out);
+ rd.write(sinkRecord, IngestionProperties.DataFormat.JSON);
+ // verify
+ validate(file.getPath(), expectedResultsMap);
+ rd.commit();
+ rd.close();
+ FileUtils.deleteQuietly(file);
+ }
+}
diff --git a/src/test/java/com/microsoft/azure/kusto/kafka/connect/sink/formatwriter/KustoRecordWriterSchemalessTests.java b/src/test/java/com/microsoft/azure/kusto/kafka/connect/sink/formatwriter/KustoRecordWriterSchemalessTests.java
new file mode 100644
index 00000000..a43900ea
--- /dev/null
+++ b/src/test/java/com/microsoft/azure/kusto/kafka/connect/sink/formatwriter/KustoRecordWriterSchemalessTests.java
@@ -0,0 +1,122 @@
+package com.microsoft.azure.kusto.kafka.connect.sink.formatwriter;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.util.*;
+
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.commons.io.FileUtils;
+import org.apache.kafka.connect.sink.SinkRecord;
+import org.json.JSONException;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.CsvSource;
+
+import com.microsoft.azure.kusto.ingest.IngestionProperties;
+import com.microsoft.azure.kusto.kafka.connect.sink.Utils;
+import com.microsoft.azure.kusto.kafka.connect.sink.format.RecordWriter;
+
+import io.confluent.avro.random.generator.Generator;
+import tech.allegro.schema.json2avro.converter.JsonAvroConverter;
+
+public class KustoRecordWriterSchemalessTests extends KustoRecordWriterBase {
+ @ParameterizedTest(name = "JSON data serialized as bytes with key schema {0} and " +
+ "value schema {1} should be deserialized correctly. Simple key: {2}, Simple value: {3}")
+ @CsvSource({
+ "avro-simple-schema.json,avro-struct-schema.json,true,false",
+ "avro-struct-schema.json,avro-struct-schema.json,false,false",
+ "avro-simple-schema.json,avro-simple-schema.json,true,true"
+ })
+ public void validateJsonSerializedAsBytes(String keySchemaPath, String valueSchemaPath,
+ boolean isSimpleKey, boolean isSimpleValue)
+ throws IOException, JSONException {
+ List records = new ArrayList<>();
+ Generator randomAvroValueData = new Generator.Builder().schemaStream(
+ Objects.requireNonNull(this.getClass().getClassLoader().getResourceAsStream(String.format("avro-schemas/%s", valueSchemaPath)))).build();
+ Generator randomAvroKeyData = new Generator.Builder().schemaStream(
+ Objects.requireNonNull(this.getClass().getClassLoader().getResourceAsStream(String.format("avro-schemas/%s", keySchemaPath)))).build();
+ JsonAvroConverter converter = new JsonAvroConverter();
+ Map expectedResultsMap = new HashMap<>();
+ for (int i = 0; i < 10; i++) {
+ Object avroKey = randomAvroKeyData.generate();
+ Object key = avroKey.toString().getBytes(StandardCharsets.UTF_8);
+ Object avroValue = randomAvroValueData.generate();
+ Object value = avroValue.toString().getBytes(StandardCharsets.UTF_8);
+ SinkRecord sinkRecord = new SinkRecord("bytes.record.topic", i % 3,
+ null,
+ key,
+ null,
+ value,
+ i);
+ sinkRecord.headers().addInt(String.format("HeaderInt-%s", i), i);
+ records.add(sinkRecord);
+ String expectedValueString = isSimpleValue ? RESULT_MAPPER.writeValueAsString(Collections.singletonMap("value", avroValue))
+ : new String(converter.convertToJson((GenericRecord) avroValue), StandardCharsets.UTF_8);
+ String expectedKeyString = isSimpleKey ? avroKey.toString() : new String(converter.convertToJson((GenericRecord) avroKey), StandardCharsets.UTF_8);
+ String expectedHeaderJson = RESULT_MAPPER.writeValueAsString(Collections.singletonMap(
+ String.format("HeaderInt-%s", i), i));
+ expectedResultsMap.put(i, new String[] {expectedHeaderJson, expectedKeyString, expectedValueString});
+ }
+ File file = new File(String.format("%s.%s", UUID.randomUUID(), "json"));
+ Utils.restrictPermissions(file);
+ KustoRecordWriterProvider writer = new KustoRecordWriterProvider();
+ OutputStream out = Files.newOutputStream(file.toPath());
+ RecordWriter rd = writer.getRecordWriter(file.getPath(), out);
+ for (SinkRecord record : records) {
+ rd.write(record, IngestionProperties.DataFormat.JSON);
+ }
+ rd.commit();
+ validate(file.getPath(), expectedResultsMap);
+ rd.close();
+ FileUtils.deleteQuietly(file);
+ }
+
+ @ParameterizedTest(name = "AVRO Data to be serialized with key schema {0} and value schema {1} isSimpleKey {2} isSimpleValue {3}")
+ @CsvSource({
+ "avro-simple-schema.json,avro-struct-schema.json,true,false",
+ "avro-struct-schema.json,avro-struct-schema.json,false,false",
+ "avro-simple-schema.json,avro-simple-schema.json,true,true"
+ })
+ public void validateAvroDataSerializedAsBytes(String keySchemaPath, String valueSchemaPath, boolean isSimpleKey, boolean isSimpleValue)
+ throws IOException, JSONException {
+ List records = new ArrayList<>();
+ Generator randomAvroValueData = new Generator.Builder().schemaStream(
+ Objects.requireNonNull(this.getClass().getClassLoader().getResourceAsStream(String.format("avro-schemas/%s", valueSchemaPath)))).build();
+ Generator randomAvroKeyData = new Generator.Builder().schemaStream(
+ Objects.requireNonNull(this.getClass().getClassLoader().getResourceAsStream(String.format("avro-schemas/%s", keySchemaPath)))).build();
+ JsonAvroConverter converter = new JsonAvroConverter();
+ Map expectedResultsMap = new HashMap<>();
+ for (int i = 0; i < 10; i++) {
+ Object key = randomAvroKeyData.generate();
+ Object value = randomAvroValueData.generate();
+ SinkRecord sinkRecord = new SinkRecord("avro.bytes.record.topic", i % 3,
+ null,
+ key,
+ null,
+ value,
+ i);
+ sinkRecord.headers().addInt(String.format("HeaderInt-%s", i), i);
+ records.add(sinkRecord);
+ String expectedValueString = isSimpleValue ? RESULT_MAPPER.writeValueAsString(Collections.singletonMap("value", value))
+ : new String(converter.convertToJson((GenericData.Record) value));
+ String expectedKeyString = isSimpleKey ? key.toString() : new String(converter.convertToJson((GenericData.Record) key));
+ String expectedHeaderJson = RESULT_MAPPER.writeValueAsString(Collections.singletonMap(String.format("HeaderInt-%s", i), i));
+ expectedResultsMap.put(i, new String[] {expectedHeaderJson, expectedKeyString, expectedValueString});
+ }
+ File file = new File(String.format("%s.%s", UUID.randomUUID(), "json"));
+ Utils.restrictPermissions(file);
+ KustoRecordWriterProvider writer = new KustoRecordWriterProvider();
+ OutputStream out = Files.newOutputStream(file.toPath());
+ RecordWriter rd = writer.getRecordWriter(file.getPath(), out);
+ for (SinkRecord record : records) {
+ rd.write(record, IngestionProperties.DataFormat.AVRO);
+ }
+ rd.commit();
+ validate(file.getPath(), expectedResultsMap);
+ rd.close();
+ FileUtils.deleteQuietly(file);
+ }
+}
diff --git a/src/test/java/com/microsoft/azure/kusto/kafka/connect/sink/it/KustoSinkIT.java b/src/test/java/com/microsoft/azure/kusto/kafka/connect/sink/it/KustoSinkIT.java
index b4d90723..8c247fd1 100644
--- a/src/test/java/com/microsoft/azure/kusto/kafka/connect/sink/it/KustoSinkIT.java
+++ b/src/test/java/com/microsoft/azure/kusto/kafka/connect/sink/it/KustoSinkIT.java
@@ -1,34 +1,39 @@
package com.microsoft.azure.kusto.kafka.connect.sink.it;
import java.io.IOException;
+import java.io.InputStream;
import java.net.URL;
import java.nio.charset.StandardCharsets;
import java.nio.file.Files;
import java.nio.file.Paths;
import java.time.Duration;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Objects;
-import java.util.TreeMap;
+import java.util.*;
import java.util.function.Predicate;
import java.util.function.Supplier;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import org.apache.avro.Schema;
+import org.apache.avro.SchemaBuilder;
import org.apache.avro.generic.GenericData;
import org.apache.avro.generic.GenericRecord;
import org.apache.commons.io.IOUtils;
import org.apache.kafka.clients.producer.KafkaProducer;
import org.apache.kafka.clients.producer.ProducerConfig;
import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.clients.producer.RecordMetadata;
+import org.apache.kafka.common.header.Header;
+import org.apache.kafka.common.header.internals.RecordHeader;
+import org.apache.kafka.common.serialization.ByteArraySerializer;
import org.apache.kafka.common.serialization.StringSerializer;
import org.jetbrains.annotations.NotNull;
import org.json.JSONException;
-import org.junit.jupiter.api.*;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.Assumptions;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.CsvSource;
import org.skyscreamer.jsonassert.Customization;
import org.skyscreamer.jsonassert.JSONAssert;
import org.skyscreamer.jsonassert.comparator.CustomComparator;
@@ -70,25 +75,25 @@ class KustoSinkIT {
private static final Logger log = LoggerFactory.getLogger(KustoSinkIT.class);
private static final Network network = Network.newNetwork();
private static final ObjectMapper objectMapper = new ObjectMapper();
-
- private static final String confluentVersion = "6.2.5";
+ private static final Integer KAFKA_MAX_MSG_SIZE = 3 * 1024 * 1024;
+ private static final String confluentVersion = "7.5.6";
private static final KafkaContainer kafkaContainer = new KafkaContainer(DockerImageName.parse("confluentinc/cp-kafka:" + confluentVersion))
- .withNetwork(network);
- private static final ProxyContainer proxyContainer = new ProxyContainer().withNetwork(network);
+ .withNetwork(network)
+ .withEnv("KAFKA_MESSAGE_MAX_BYTES", KAFKA_MAX_MSG_SIZE.toString())
+ .withEnv("KAFKA_SOCKET_REQUEST_MAX_BYTES", KAFKA_MAX_MSG_SIZE.toString());
private static final SchemaRegistryContainer schemaRegistryContainer = new SchemaRegistryContainer(confluentVersion).withKafka(kafkaContainer)
.withNetwork(network).dependsOn(kafkaContainer);
- private static final List testFormats = Arrays.asList("json", "avro", "csv"); // List.of("json", "avro", "csv", "raw"); // Raw for XML
- private static ITCoordinates coordinates;
+ private static final ProxyContainer proxyContainer = new ProxyContainer().withNetwork(network);
private static final KustoKafkaConnectContainer connectContainer = new KustoKafkaConnectContainer(confluentVersion)
.withNetwork(network)
.withKafka(kafkaContainer)
.dependsOn(kafkaContainer, proxyContainer, schemaRegistryContainer);
-
+ private static final String keyColumn = "vlong";
+ private static final String COMPLEX_AVRO_BYTES_TABLE_TEST = String.format("ComplexAvroBytesTest_%s", UUID.randomUUID()).replace('-', '_');
+ private static ITCoordinates coordinates;
private static Client engineClient = null;
private static Client dmClient = null;
- private static final String keyColumn = "vlong";
-
@BeforeAll
public static void startContainers() throws Exception {
coordinates = getConnectorProperties();
@@ -121,7 +126,9 @@ public static void startContainers() throws Exception {
private static void createTables() throws Exception {
URL kqlResource = KustoSinkIT.class.getClassLoader().getResource("it-table-setup.kql");
assert kqlResource != null;
- List kqlsToExecute = Files.readAllLines(Paths.get(kqlResource.toURI())).stream().map(kql -> kql.replace("TBL", coordinates.table))
+ List kqlsToExecute = Files.readAllLines(Paths.get(kqlResource.toURI())).stream()
+ .map(kql -> kql.replace("TBL", coordinates.table))
+ .map(kql -> kql.replace("CABT", COMPLEX_AVRO_BYTES_TABLE_TEST))
.collect(Collectors.toList());
kqlsToExecute.forEach(kql -> {
try {
@@ -130,7 +137,7 @@ private static void createTables() throws Exception {
log.error("Failed to execute kql: {}", kql, e);
}
});
- log.info("Created table {} and associated mappings", coordinates.table);
+ log.info("Created tables {} , {} and associated mappings", coordinates.table, COMPLEX_AVRO_BYTES_TABLE_TEST);
}
private static void refreshDm() throws Exception {
@@ -151,77 +158,95 @@ private static void refreshDm() throws Exception {
@AfterAll
public static void stopContainers() throws Exception {
+ log.info("Finished table clean up. Dropped tables {} and {}", coordinates.table, COMPLEX_AVRO_BYTES_TABLE_TEST);
connectContainer.stop();
schemaRegistryContainer.stop();
kafkaContainer.stop();
engineClient.execute(coordinates.database, String.format(".drop table %s", coordinates.table));
- log.warn("Finished table clean up. Dropped table {}", coordinates.table);
+ engineClient.execute(coordinates.database, String.format(".drop table %s", COMPLEX_AVRO_BYTES_TABLE_TEST));
dmClient.close();
engineClient.close();
}
- @Test
- public void shouldHandleAllTypesOfEvents() {
- Assumptions.assumeTrue(coordinates.isValidConfig(), "Skipping test due to missing configuration");
- String srUrl = String.format("http://%s:%s", schemaRegistryContainer.getContainerId().substring(0, 12), 8081);
- testFormats.parallelStream().forEach(dataFormat -> {
- String valueFormat = "org.apache.kafka.connect.storage.StringConverter";
- if (dataFormat.equals("avro")) {
- valueFormat = AvroConverter.class.getName();
- log.debug("Using value format: {}", valueFormat);
- }
- String topicTableMapping = dataFormat.equals("csv")
- ? String.format("[{'topic': 'e2e.%s.topic','db': '%s', 'table': '%s','format':'%s','mapping':'%s_mapping','streaming':'true'}]", dataFormat,
- coordinates.database,
- coordinates.table, dataFormat, dataFormat)
- : String.format("[{'topic': 'e2e.%s.topic','db': '%s', 'table': '%s','format':'%s','mapping':'%s_mapping'}]", dataFormat,
- coordinates.database,
- coordinates.table, dataFormat, dataFormat);
- log.info("Deploying connector for {} , using SR url {}. Using proxy host {} and port {}", dataFormat, srUrl,
- proxyContainer.getContainerId().substring(0, 12), proxyContainer.getExposedPorts().get(0));
- Map connectorProps = new HashMap<>();
- connectorProps.put("connector.class", "com.microsoft.azure.kusto.kafka.connect.sink.KustoSinkConnector");
- connectorProps.put("flush.size.bytes", 10000);
- connectorProps.put("flush.interval.ms", 1000);
- connectorProps.put("tasks.max", 1);
- connectorProps.put("topics", String.format("e2e.%s.topic", dataFormat));
- connectorProps.put("kusto.tables.topics.mapping", topicTableMapping);
- connectorProps.put("aad.auth.authority", coordinates.authority);
- connectorProps.put("aad.auth.accesstoken", coordinates.accessToken);
- connectorProps.put("aad.auth.strategy", "AZ_DEV_TOKEN".toLowerCase());
- connectorProps.put("kusto.query.url", coordinates.cluster);
- connectorProps.put("kusto.ingestion.url", coordinates.ingestCluster);
+ private static void deployConnector(@NotNull String dataFormat, String topicTableMapping,
+ String srUrl, String keyFormat, String valueFormat) {
+ deployConnector(dataFormat, topicTableMapping, srUrl, keyFormat, valueFormat, Collections.emptyMap());
+ }
+
+ private static void deployConnector(@NotNull String dataFormat, String topicTableMapping,
+ String srUrl, String keyFormat, String valueFormat,
+ Map overrideProps) {
+ Map connectorProps = new HashMap<>();
+ connectorProps.put("connector.class", "com.microsoft.azure.kusto.kafka.connect.sink.KustoSinkConnector");
+ connectorProps.put("flush.size.bytes", 10000);
+ connectorProps.put("flush.interval.ms", 1000);
+ connectorProps.put("tasks.max", 1);
+ connectorProps.put("topics", String.format("e2e.%s.topic", dataFormat));
+ connectorProps.put("kusto.tables.topics.mapping", topicTableMapping);
+ connectorProps.put("aad.auth.authority", coordinates.authority);
+ connectorProps.put("aad.auth.accesstoken", coordinates.accessToken);
+ connectorProps.put("aad.auth.strategy", "AZ_DEV_TOKEN".toLowerCase());
+ connectorProps.put("kusto.query.url", coordinates.cluster);
+ connectorProps.put("kusto.ingestion.url", coordinates.ingestCluster);
+ if (!dataFormat.startsWith("bytes")) {
connectorProps.put("schema.registry.url", srUrl);
connectorProps.put("value.converter.schema.registry.url", srUrl);
- connectorProps.put("key.converter", "org.apache.kafka.connect.storage.StringConverter");
- connectorProps.put("value.converter", valueFormat);
- connectorProps.put("proxy.host", proxyContainer.getContainerId().substring(0, 12));
- connectorProps.put("proxy.port", proxyContainer.getExposedPorts().get(0));
- connectContainer.registerConnector(String.format("adx-connector-%s", dataFormat), connectorProps);
- log.info("Deployed connector for {}", dataFormat);
- log.debug(connectContainer.getLogs());
- });
- testFormats.parallelStream().forEach(dataFormat -> {
- connectContainer.waitUntilConnectorTaskStateChanges(String.format("adx-connector-%s", dataFormat), 0, "RUNNING");
- log.info("Connector state for {} : {}. ", dataFormat,
- connectContainer.getConnectorTaskState(String.format("adx-connector-%s", dataFormat), 0));
- try {
- produceKafkaMessages(dataFormat);
- Thread.sleep(10000);
- } catch (IOException | InterruptedException e) {
- throw new RuntimeException(e);
- }
- });
+ }
+ connectorProps.put("key.converter", keyFormat);
+ connectorProps.put("value.converter", valueFormat);
+ connectorProps.put("proxy.host", proxyContainer.getContainerId().substring(0, 12));
+ connectorProps.put("proxy.port", proxyContainer.getExposedPorts().get(0));
+ connectorProps.putAll(overrideProps);
+ connectContainer.registerConnector(String.format("adx-connector-%s", dataFormat), connectorProps);
+ log.debug("Deployed connector for {}", dataFormat);
+ log.debug(connectContainer.getLogs());
+ connectContainer.waitUntilConnectorTaskStateChanges(String.format("adx-connector-%s", dataFormat), 0, "RUNNING");
+ log.info("Connector state for {} : {}. ", dataFormat,
+ connectContainer.getConnectorTaskState(String.format("adx-connector-%s", dataFormat), 0));
+ }
+
+ @ParameterizedTest
+ @CsvSource({"json", "avro", "csv", "bytes-json"})
+ public void shouldHandleAllTypesOfEvents(@NotNull String dataFormat) {
+ log.info("Running test for data format {}", dataFormat);
+ Assumptions.assumeTrue(coordinates.isValidConfig(), "Skipping test due to missing configuration");
+ String srUrl = String.format("http://%s:%s", schemaRegistryContainer.getContainerId().substring(0, 12), 8081);
+ String valueFormat = "org.apache.kafka.connect.storage.StringConverter";
+ String keyFormat = "org.apache.kafka.connect.storage.StringConverter";
+ if (dataFormat.equals("avro")) {
+ valueFormat = AvroConverter.class.getName();
+ log.debug("Using value format: {}", valueFormat);
+ }
+ String topicTableMapping = dataFormat.equals("csv")
+ ? String.format("[{'topic': 'e2e.%s.topic','db': '%s', 'table': '%s','format':'%s','mapping':'csv_mapping','streaming':'true'}]",
+ dataFormat, coordinates.database, coordinates.table, dataFormat)
+ : String.format("[{'topic': 'e2e.%s.topic','db': '%s', 'table': '%s','format':'%s','mapping':'data_mapping'}]", dataFormat,
+ coordinates.database,
+ coordinates.table, dataFormat);
+ if (dataFormat.startsWith("bytes")) {
+ valueFormat = "org.apache.kafka.connect.converters.ByteArrayConverter";
+ // JSON is written as JSON
+ topicTableMapping = String.format("[{'topic': 'e2e.%s.topic','db': '%s', 'table': '%s','format':'%s'," +
+ "'mapping':'data_mapping'}]", dataFormat,
+ coordinates.database,
+ coordinates.table, dataFormat.split("-")[1]);
+ }
+ log.info("Deploying connector for {} , using SR url {}. Using proxy host {} and port {}", dataFormat, srUrl,
+ proxyContainer.getContainerId().substring(0, 12), proxyContainer.getExposedPorts().get(0));
+ deployConnector(dataFormat, topicTableMapping, srUrl, keyFormat, valueFormat);
+ try {
+ produceKafkaMessages(dataFormat);
+ Thread.sleep(10_000);
+ } catch (IOException | InterruptedException e) {
+ throw new RuntimeException(e);
+ }
}
private void produceKafkaMessages(@NotNull String dataFormat) throws IOException {
- log.debug("Producing messages");
+ log.info("Producing messages");
int maxRecords = 10;
Map producerProperties = new HashMap<>();
producerProperties.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, kafkaContainer.getBootstrapServers());
- producerProperties.put(AbstractKafkaSchemaSerDeConfig.SCHEMA_REGISTRY_URL_CONFIG,
- String.format("http://%s:%s", schemaRegistryContainer.getHost(), schemaRegistryContainer.getFirstMappedPort()));
- producerProperties.put(AbstractKafkaSchemaSerDeConfig.AUTO_REGISTER_SCHEMAS, true);
// avro
Generator.Builder builder = new Generator.Builder().schemaString(IOUtils.toString(
Objects.requireNonNull(this.getClass().getClassLoader().getResourceAsStream("it-avro.avsc")),
@@ -233,14 +258,20 @@ private void produceKafkaMessages(@NotNull String dataFormat) throws IOException
case "avro":
producerProperties.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class);
producerProperties.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, KafkaAvroSerializer.class);
+ producerProperties.put(AbstractKafkaSchemaSerDeConfig.SCHEMA_REGISTRY_URL_CONFIG,
+ String.format("http://%s:%s", schemaRegistryContainer.getHost(), schemaRegistryContainer.getFirstMappedPort()));
+ producerProperties.put(AbstractKafkaSchemaSerDeConfig.AUTO_REGISTER_SCHEMAS, true);
// GenericRecords to bytes using avro
try (KafkaProducer producer = new KafkaProducer<>(producerProperties)) {
for (int i = 0; i < maxRecords; i++) {
GenericData.Record record = (GenericData.Record) randomDataBuilder.generate();
- ProducerRecord producerRecord = new ProducerRecord<>("e2e.avro.topic", "Key-" + i, record);
+ record.put("vtype", dataFormat);
+ List headers = new ArrayList<>();
+ headers.add(new RecordHeader("Iteration", (dataFormat + "-Header" + i).getBytes()));
+ ProducerRecord producerRecord = new ProducerRecord<>("e2e.avro.topic", 0, "Key-" + i, record, headers);
Map jsonRecordMap = record.getSchema().getFields().stream()
.collect(Collectors.toMap(Schema.Field::name, field -> record.get(field.name())));
- jsonRecordMap.put("type", dataFormat);
+ jsonRecordMap.put("vtype", "avro");
expectedRecordsProduced.put(Long.valueOf(jsonRecordMap.get(keyColumn).toString()),
objectMapper.writeValueAsString(jsonRecordMap));
producer.send(producerRecord);
@@ -250,15 +281,21 @@ private void produceKafkaMessages(@NotNull String dataFormat) throws IOException
case "json":
producerProperties.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class);
producerProperties.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, StringSerializer.class);
+ producerProperties.put(AbstractKafkaSchemaSerDeConfig.SCHEMA_REGISTRY_URL_CONFIG,
+ String.format("http://%s:%s", schemaRegistryContainer.getHost(), schemaRegistryContainer.getFirstMappedPort()));
+ producerProperties.put(AbstractKafkaSchemaSerDeConfig.AUTO_REGISTER_SCHEMAS, true);
// GenericRecords to json using avro
try (KafkaProducer producer = new KafkaProducer<>(producerProperties)) {
for (int i = 0; i < maxRecords; i++) {
GenericRecord record = (GenericRecord) randomDataBuilder.generate();
+ record.put("vtype", "json");
Map jsonRecordMap = record.getSchema().getFields().stream()
.collect(Collectors.toMap(Schema.Field::name, field -> record.get(field.name())));
- ProducerRecord producerRecord = new ProducerRecord<>("e2e.json.topic", "Key-" + i,
- objectMapper.writeValueAsString(jsonRecordMap));
- jsonRecordMap.put("type", dataFormat);
+ List headers = new ArrayList<>();
+ headers.add(new RecordHeader("Iteration", (dataFormat + "-Header" + i).getBytes()));
+ ProducerRecord producerRecord = new ProducerRecord<>("e2e.json.topic",
+ 0, "Key-" + i, objectMapper.writeValueAsString(jsonRecordMap), headers);
+ jsonRecordMap.put("vtype", dataFormat);
expectedRecordsProduced.put(Long.valueOf(jsonRecordMap.get(keyColumn).toString()),
objectMapper.writeValueAsString(jsonRecordMap));
log.debug("JSON Record produced: {}", objectMapper.writeValueAsString(jsonRecordMap));
@@ -269,36 +306,75 @@ private void produceKafkaMessages(@NotNull String dataFormat) throws IOException
case "csv":
producerProperties.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class);
producerProperties.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, StringSerializer.class);
+ producerProperties.put(AbstractKafkaSchemaSerDeConfig.SCHEMA_REGISTRY_URL_CONFIG,
+ String.format("http://%s:%s", schemaRegistryContainer.getHost(), schemaRegistryContainer.getFirstMappedPort()));
+ producerProperties.put(AbstractKafkaSchemaSerDeConfig.AUTO_REGISTER_SCHEMAS, true);
// GenericRecords to json using avro
try (KafkaProducer producer = new KafkaProducer<>(producerProperties)) {
for (int i = 0; i < maxRecords; i++) {
GenericRecord record = (GenericRecord) randomDataBuilder.generate();
+ record.put("vtype", "csv");
Map jsonRecordMap = new TreeMap<>(record.getSchema().getFields().stream().parallel()
.collect(Collectors.toMap(Schema.Field::name, field -> record.get(field.name()))));
String objectsCommaSeparated = jsonRecordMap.values().stream().map(Object::toString).collect(Collectors.joining(","));
log.debug("CSV Record produced: {}", objectsCommaSeparated);
- ProducerRecord producerRecord = new ProducerRecord<>("e2e.csv.topic", "Key-" + i,
- objectsCommaSeparated);
- jsonRecordMap.put("type", dataFormat);
+ List headers = new ArrayList<>();
+ headers.add(new RecordHeader("Iteration", (dataFormat + "-Header" + i).getBytes()));
+ ProducerRecord producerRecord = new ProducerRecord<>("e2e.csv.topic", 0, "Key-" + i,
+ objectsCommaSeparated, headers);
+ jsonRecordMap.put("vtype", dataFormat);
expectedRecordsProduced.put(Long.valueOf(jsonRecordMap.get(keyColumn).toString()),
objectMapper.writeValueAsString(jsonRecordMap));
producer.send(producerRecord);
}
}
break;
+ case "bytes-json":
+ producerProperties.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class);
+ producerProperties.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class);
+ // GenericRecords to json using avro
+ try (KafkaProducer producer = new KafkaProducer<>(producerProperties)) {
+ for (int i = 0; i < maxRecords; i++) {
+ GenericRecord record = (GenericRecord) randomDataBuilder.generate();
+ record.put("vtype", "bytes-json");
+ // Serialization test for Avro as bytes , or JSON as bytes (Schemaless tests)
+ byte[] dataToSend = record.toString().getBytes(StandardCharsets.UTF_8);
+ Map jsonRecordMap = record.getSchema().getFields().stream()
+ .collect(Collectors.toMap(Schema.Field::name, field -> record.get(field.name())));
+ ProducerRecord producerRecord = new ProducerRecord<>(
+ String.format("e2e.%s.topic", dataFormat),
+ String.format("Key-%s", i),
+ dataToSend);
+ jsonRecordMap.put("vtype", dataFormat);
+ expectedRecordsProduced.put(Long.valueOf(jsonRecordMap.get(keyColumn).toString()),
+ objectMapper.writeValueAsString(jsonRecordMap));
+ log.info("Bytes topic {} written to", String.format("e2e.%s.topic", dataFormat));
+ try {
+ RecordMetadata rmd = producer.send(producerRecord).get();
+ log.info("Record sent to topic {} with offset {} of size {}",
+ String.format("e2e.%s.topic", dataFormat), rmd.offset(), dataToSend.length);
+ } catch (Exception e) {
+ log.error("Failed to send record to topic {}", String.format("e2e.%s.topic", dataFormat), e);
+ }
+ }
+ }
+ break;
}
log.info("Produced messages for format {}", dataFormat);
- Map actualRecordsIngested = getRecordsIngested(dataFormat, maxRecords);
+ String query = String.format("%s | where vtype == '%s' | project %s,vresult = pack_all()",
+ coordinates.table, dataFormat, keyColumn);
+ Map actualRecordsIngested = getRecordsIngested(query, maxRecords);
actualRecordsIngested.keySet().parallelStream().forEach(key -> {
+ long keyLong = Long.parseLong(key.toString());
log.debug("Record queried in assertion : {}", actualRecordsIngested.get(key));
try {
- JSONAssert.assertEquals(expectedRecordsProduced.get(key), actualRecordsIngested.get(key),
+ JSONAssert.assertEquals(expectedRecordsProduced.get(keyLong), actualRecordsIngested.get(key),
new CustomComparator(LENIENT,
// there are sometimes round off errors in the double values but they are close enough to 8 precision
new Customization("vdec", (vdec1,
- vdec2) -> Math.abs(Double.parseDouble(vdec1.toString()) - Double.parseDouble(vdec2.toString())) < 0.000000001),
+ vdec2) -> Math.abs(Double.parseDouble(vdec1.toString()) - Double.parseDouble(vdec2.toString())) < 0.000000001),
new Customization("vreal", (vreal1,
- vreal2) -> Math.abs(Double.parseDouble(vreal1.toString()) - Double.parseDouble(vreal2.toString())) < 0.0001)));
+ vreal2) -> Math.abs(Double.parseDouble(vreal1.toString()) - Double.parseDouble(vreal2.toString())) < 0.0001)));
} catch (JSONException e) {
fail(e);
}
@@ -306,11 +382,84 @@ private void produceKafkaMessages(@NotNull String dataFormat) throws IOException
assertEquals(maxRecords, actualRecordsIngested.size());
}
- private @NotNull Map getRecordsIngested(String dataFormat, int maxRecords) {
- String query = String.format("%s | where type == '%s' | project %s,vresult = pack_all()", coordinates.table, dataFormat, keyColumn);
+ @Test
+ public void shouldHandleComplexAvroMessage() throws IOException {
+ String dataFormat = "bytes-avro";
+ int maxRecords = 8;
+ String srUrl = String.format("http://%s:%s", schemaRegistryContainer.getContainerId().substring(0, 12), 8081);
+ String producerSrUrl = String.format("http://localhost:%s", schemaRegistryContainer.getMappedPort(8081));
+ Map producerProperties = new HashMap<>();
+ producerProperties.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, kafkaContainer.getBootstrapServers());
+ producerProperties.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "io.confluent.kafka.serializers.KafkaAvroSerializer");
+ producerProperties.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class);
+ producerProperties.put(AbstractKafkaSchemaSerDeConfig.SCHEMA_REGISTRY_URL_CONFIG, producerSrUrl);
+ producerProperties.put("key.schema.registry.url", producerSrUrl);
+ producerProperties.put(ProducerConfig.MAX_REQUEST_SIZE_CONFIG, KAFKA_MAX_MSG_SIZE);
+ producerProperties.put(ProducerConfig.BATCH_SIZE_CONFIG, KAFKA_MAX_MSG_SIZE * 5);
+ producerProperties.put("message.max.bytes", KAFKA_MAX_MSG_SIZE);
+ String topicName = String.format("e2e.%s.topic", dataFormat);
+ String topicTableMapping = String.format("[{'topic': '%s','db': '%s', " +
+ "'table': '%s','format':'%s','mapping':'%s_mapping'}]", topicName,
+ coordinates.database,
+ COMPLEX_AVRO_BYTES_TABLE_TEST, dataFormat.split("-")[1], COMPLEX_AVRO_BYTES_TABLE_TEST);
+ deployConnector(dataFormat, topicTableMapping, srUrl,
+ AvroConverter.class.getName(),
+ "org.apache.kafka.connect.converters.ByteArrayConverter",
+ Collections.singletonMap("key.converter.schema.registry.url", srUrl));
+ Schema keySchema = SchemaBuilder
+ .record("Key").namespace("com.ms.kafka.connect.sink.avro")
+ .fields()
+ .name("IterationKey").type().stringType().noDefault()
+ .name("Timestamp").type().nullable().longType().noDefault()
+ .endRecord();
+ long keyStart = 100000L;
+
+ InputStream expectedResultsStream = Objects
+ .requireNonNull(this.getClass().getClassLoader().getResourceAsStream("avro-complex-data/expected-results.txt"));
+ String expectedResults = IOUtils.toString(expectedResultsStream, StandardCharsets.UTF_8);
+ Map expectedResultMap = Arrays.stream(expectedResults.split("\n"))
+ .map(line -> line.split("~"))
+ .collect(Collectors.toMap(arr -> arr[0], arr -> arr[1]));
+ try (KafkaProducer producer = new KafkaProducer<>(producerProperties)) {
+ for (int i = 1; i <= maxRecords; i++) {
+ // complex-avro-1.avro
+ long keyTick = keyStart + i;
+ GenericData.Record keyRecord = new GenericData.Record(keySchema);
+ keyRecord.put("IterationKey", String.valueOf(i));
+ keyRecord.put("Timestamp", keyTick);
+ InputStream avroData = Objects
+ .requireNonNull(this.getClass().getClassLoader().getResourceAsStream(String.format("avro-complex-data/complex-avro-%d.avro", i)));
+ byte[] testData = IOUtils.toByteArray(avroData);
+ ProducerRecord producerRecord = new ProducerRecord<>(topicName, keyRecord, testData);
+ producerRecord.headers().add("vtype", dataFormat.getBytes());
+ producerRecord.headers().add("iteration", String.valueOf(i).getBytes());
+ RecordMetadata rmd = producer.send(producerRecord).get();
+ log.info("Avro bytes sent to topic {} with offset {} of size {}", topicName, rmd.offset(), testData.length);
+ }
+ Thread.sleep(30_000);
+ } catch (Exception e) {
+ log.error("Failed to send record to topic {}", topicName, e);
+ fail("Failed sending message to Kafka for testing Avro-Bytes scenario.");
+ }
+
+ String countLongQuery = String.format("%s | summarize c = count() by event_id | project %s=event_id, " +
+ "vresult = bag_pack('event_id',event_id,'count',c)", COMPLEX_AVRO_BYTES_TABLE_TEST, keyColumn);
+
+ Map actualRecordsIngested = getRecordsIngested(countLongQuery, maxRecords);
+ assertEquals(expectedResultMap, actualRecordsIngested);
+ }
+
+ /**
+ * Polls the Kusto table for the records ingested. The query is executed every 30 seconds and the results are
+ *
+ * @param query The query to execute
+ * @param maxRecords The maximum number of records to poll for
+ * @return A map of the records ingested
+ */
+ private @NotNull Map getRecordsIngested(String query, int maxRecords) {
Predicate predicate = (results) -> {
if (results != null) {
- log.debug("Retrieved records count {}", ((Map, ?>) results).size());
+ log.info("Retrieved records count {}", ((Map, ?>) results).size());
}
return results == null || ((Map, ?>) results).isEmpty() || ((Map, ?>) results).size() < maxRecords;
};
@@ -322,15 +471,16 @@ private void produceKafkaMessages(@NotNull String dataFormat) throws IOException
.build();
RetryRegistry registry = RetryRegistry.of(config);
Retry retry = registry.retry("ingestRecordService", config);
- Supplier> recordSearchSupplier = () -> {
+ Supplier> recordSearchSupplier = () -> {
try {
log.debug("Executing query {} ", query);
KustoResultSetTable resultSet = engineClient.execute(coordinates.database, query).getPrimaryResults();
- Map actualResults = new HashMap<>();
+ Map actualResults = new HashMap<>();
while (resultSet.next()) {
- Long key = (long) resultSet.getInt(keyColumn);
+ Object keyObject = resultSet.getObject(keyColumn);
+ Object key = keyObject instanceof Number ? Long.parseLong(keyObject.toString()) : keyObject.toString();
String vResult = resultSet.getString("vresult");
- log.debug("Record queried: {}", vResult);
+ log.debug("Record queried from DB: {}", vResult);
actualResults.put(key, vResult);
}
return actualResults;
diff --git a/src/test/java/com/microsoft/azure/kusto/kafka/connect/sink/it/containers/KustoKafkaConnectContainer.java b/src/test/java/com/microsoft/azure/kusto/kafka/connect/sink/it/containers/KustoKafkaConnectContainer.java
index d151c71b..ddb3c6de 100644
--- a/src/test/java/com/microsoft/azure/kusto/kafka/connect/sink/it/containers/KustoKafkaConnectContainer.java
+++ b/src/test/java/com/microsoft/azure/kusto/kafka/connect/sink/it/containers/KustoKafkaConnectContainer.java
@@ -16,6 +16,7 @@
import org.apache.http.impl.client.HttpClients;
import org.apache.http.util.EntityUtils;
import org.awaitility.Awaitility;
+import org.jetbrains.annotations.NotNull;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.testcontainers.containers.GenericContainer;
@@ -43,7 +44,7 @@ public KustoKafkaConnectContainer(final String version) {
withExposedPorts(KAFKA_CONNECT_PORT);
}
- public KustoKafkaConnectContainer withKafka(final KafkaContainer kafkaContainer) {
+ public KustoKafkaConnectContainer withKafka(final @NotNull KafkaContainer kafkaContainer) {
return withKafka(kafkaContainer.getNetwork(), kafkaContainer.getNetworkAliases().get(0) + ":9092");
}
diff --git a/src/test/java/com/microsoft/azure/kusto/kafka/connect/sink/it/containers/SchemaRegistryContainer.java b/src/test/java/com/microsoft/azure/kusto/kafka/connect/sink/it/containers/SchemaRegistryContainer.java
index eae57d65..40acf7b3 100644
--- a/src/test/java/com/microsoft/azure/kusto/kafka/connect/sink/it/containers/SchemaRegistryContainer.java
+++ b/src/test/java/com/microsoft/azure/kusto/kafka/connect/sink/it/containers/SchemaRegistryContainer.java
@@ -1,5 +1,6 @@
package com.microsoft.azure.kusto.kafka.connect.sink.it.containers;
+import org.jetbrains.annotations.NotNull;
import org.testcontainers.containers.GenericContainer;
import org.testcontainers.containers.KafkaContainer;
import org.testcontainers.containers.Network;
@@ -15,12 +16,13 @@ public SchemaRegistryContainer(String version) {
withExposedPorts(SCHEMA_REGISTRY_PORT);
}
- public SchemaRegistryContainer withKafka(KafkaContainer kafka) {
+ public SchemaRegistryContainer withKafka(@NotNull KafkaContainer kafka) {
return withKafka(kafka.getNetwork(), kafka.getNetworkAliases().get(0) + ":9092");
}
public SchemaRegistryContainer withKafka(Network network, String bootstrapServers) {
- withNetwork(network).withEnv("SCHEMA_REGISTRY_HOST_NAME", "schema-registry").withEnv("SCHEMA_REGISTRY_LISTENERS", "http://0.0.0.0:8081")
+ withNetwork(network).withEnv("SCHEMA_REGISTRY_HOST_NAME", "schema-registry")
+ .withEnv("SCHEMA_REGISTRY_LISTENERS", "http://0.0.0.0:8081")
.withEnv("SCHEMA_REGISTRY_KAFKASTORE_BOOTSTRAP_SERVERS", "PLAINTEXT://" + bootstrapServers);
return self();
}
diff --git a/src/test/resources/avro-complex-data/complex-avro-1.avro b/src/test/resources/avro-complex-data/complex-avro-1.avro
new file mode 100644
index 00000000..e07d3859
--- /dev/null
+++ b/src/test/resources/avro-complex-data/complex-avro-1.avro
@@ -0,0 +1,7242 @@
+Objavro.schema{"type":"record","name":"MixerOfferEvent","namespace":"es.eci.firefly.avro.mixer.event","fields":[{"name":"business_event_type","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"business_object","type":{"type":"record","name":"MixerOffer","namespace":"es.eci.firefly.avro.mixer.model","fields":[{"name":"applied_taxes","type":["null",{"type":"array","items":{"type":"record","name":"AppliedTaxes","namespace":"es.eci.firefly.avro.mixer.canonical.model.offer","fields":[{"name":"type","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"value","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null}],"connect.name":"es.eci.firefly.avro.mixer.canonical.model.offer.AppliedTaxes"}}],"default":null},{"name":"availability_ends","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"availability_starts","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"barra","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"bundle_of","type":["null",{"type":"array","items":{"type":"record","name":"Product","namespace":"es.eci.firefly.avro.mixer.canonical.model.offer","fields":[{"name":"additional_product_descriptions","type":["null",{"type":"array","items":{"type":"record","name":"ProductAdditionalDescription","fields":[{"name":"end_date","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"start_date","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"type","type":{"type":"string","connect.parameters":{"io.confluent.connect.avro.Enum":"es.eci.firefly.avro.mixer.canonical.model.offer.ProductAdditionalDescriptionType","io.confluent.connect.avro.Enum.VARIETY":"VARIETY","io.confluent.connect.avro.Enum.VARIETY_STORE":"VARIETY_STORE","io.confluent.connect.avro.Enum.OTHER":"OTHER","io.confluent.connect.avro.Enum.ARTICLE":"ARTICLE","io.confluent.connect.avro.Enum.TEMPORAL":"TEMPORAL","io.confluent.connect.avro.Enum.TICKET":"TICKET"},"connect.name":"es.eci.firefly.avro.mixer.canonical.model.offer.ProductAdditionalDescriptionType"}},{"name":"value","type":["null",{"type":"array","items":{"type":"record","name":"I18nText","fields":[{"name":"locale","type":{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}},{"name":"value","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null}],"connect.name":"es.eci.firefly.avro.mixer.canonical.model.offer.I18nText"}}],"default":null}],"connect.name":"es.eci.firefly.avro.mixer.canonical.model.offer.ProductAdditionalDescription"}}],"default":null},{"name":"badges","type":["null",{"type":"array","items":{"type":"record","name":"Badge","fields":[{"name":"description","type":["null",{"type":"array","items":"I18nText"}],"default":null},{"name":"end_date","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"external_identifier","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"glossary","type":["null",{"type":"record","name":"Glossary","fields":[{"name":"description","type":["null",{"type":"array","items":"I18nText"}],"default":null},{"name":"external_identifier","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"identifier","type":{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}},{"name":"media_objects","type":["null",{"type":"array","items":{"type":"record","name":"MediaObject","fields":[{"name":"description","type":["null",{"type":"array","items":"I18nText"}],"default":null},{"name":"media_category_order","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"media_category_type","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"media_type","type":{"type":"string","connect.parameters":{"io.confluent.connect.avro.Enum":"es.eci.firefly.avro.mixer.canonical.model.offer.MediaObjectType","io.confluent.connect.avro.Enum.AUDIO":"AUDIO","io.confluent.connect.avro.Enum.IMAGE":"IMAGE","io.confluent.connect.avro.Enum.VIDEO":"VIDEO","io.confluent.connect.avro.Enum.DOCUMENT":"DOCUMENT","io.confluent.connect.avro.Enum.LINK":"LINK"},"connect.name":"es.eci.firefly.avro.mixer.canonical.model.offer.MediaObjectType"}},{"name":"size","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"tags","type":["null",{"type":"array","items":{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}}],"default":null},{"name":"url","type":{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}}],"connect.name":"es.eci.firefly.avro.mixer.canonical.model.offer.MediaObject"}}],"default":null},{"name":"name","type":["null",{"type":"array","items":"I18nText"}],"default":null},{"name":"term","type":["null",{"type":"array","items":"I18nText"}],"default":null}],"connect.name":"es.eci.firefly.avro.mixer.canonical.model.offer.Glossary"}],"default":null},{"name":"icon","type":["null","MediaObject"],"default":null},{"name":"identifier","type":{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}},{"name":"name","type":["null",{"type":"array","items":"I18nText"}],"default":null},{"name":"start_date","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"type","type":{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}},{"name":"value","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null}],"connect.name":"es.eci.firefly.avro.mixer.canonical.model.offer.Badge"}}],"default":null},{"name":"bar_code_variants","type":["null",{"type":"array","items":{"type":"record","name":"BarCodeVariant","fields":[{"name":"date_created_procurement","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"gtin","type":{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}},{"name":"is_main","type":["null","boolean"],"default":null},{"name":"promotional_end_date","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"promotional_start_date","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null}],"connect.name":"es.eci.firefly.avro.mixer.canonical.model.offer.BarCodeVariant"}}],"default":null},{"name":"basic_color","type":["null",{"type":"record","name":"Color","fields":[{"name":"color_code","type":{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}},{"name":"color_description","type":["null",{"type":"array","items":"I18nText"}],"default":null},{"name":"external_identifier","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"glossary","type":["null","Glossary"],"default":null},{"name":"identifier","type":{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}},{"name":"name","type":["null",{"type":"array","items":"I18nText"}],"default":null}],"connect.name":"es.eci.firefly.avro.mixer.canonical.model.offer.Color"}],"default":null},{"name":"brand","type":["null",{"type":"record","name":"Brand","fields":[{"name":"external_id","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"external_identifier","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"identifier","type":{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}},{"name":"is_eci_brand","type":["null","boolean"],"default":null},{"name":"media_objects","type":["null",{"type":"array","items":"MediaObject"}],"default":null},{"name":"name","type":["null",{"type":"array","items":"I18nText"}],"default":null},{"name":"url_links","type":["null",{"type":"array","items":{"type":"record","name":"BrandSiteLink","fields":[{"name":"site","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"url","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null}],"connect.name":"es.eci.firefly.avro.mixer.canonical.model.offer.BrandSiteLink"}}],"default":null}],"connect.name":"es.eci.firefly.avro.mixer.canonical.model.offer.Brand"}],"default":null},{"name":"calculated_badges","type":["null",{"type":"array","items":{"type":"record","name":"CalculatedBadge","fields":[{"name":"center_value","type":["null",{"type":"array","items":{"type":"record","name":"PackingUnitsBadge","fields":[{"name":"center","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"value","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null}],"connect.name":"es.eci.firefly.avro.mixer.canonical.model.offer.PackingUnitsBadge"}}],"default":null},{"name":"type","type":{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}},{"name":"value","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null}],"connect.name":"es.eci.firefly.avro.mixer.canonical.model.offer.CalculatedBadge"}}],"default":null},{"name":"canonical_seo","type":["null",{"type":"array","items":"I18nText"}],"default":null},{"name":"categories","type":["null",{"type":"array","items":{"type":"record","name":"Category","fields":[{"name":"classifications","type":["null",{"type":"array","items":{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}}],"default":null},{"name":"description","type":["null",{"type":"array","items":"I18nText"}],"default":null},{"name":"external_identifier","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"has_more_direct_children","type":["null","boolean"],"default":null},{"name":"identifier","type":{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}},{"name":"metadata","type":["null",{"type":"record","name":"Metadata","fields":[{"name":"is_publish","type":["null","boolean"],"default":null}],"connect.name":"es.eci.firefly.avro.mixer.canonical.model.offer.Metadata"}],"default":null},{"name":"name","type":["null",{"type":"array","items":"I18nText"}],"default":null},{"name":"order_in_parent","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"parent_category","type":["null","Category"],"default":null},{"name":"type","type":["null",{"type":"string","connect.parameters":{"io.confluent.connect.avro.Enum":"es.eci.firefly.avro.mixer.canonical.model.offer.CategoryType","io.confluent.connect.avro.Enum.CD":"CD","io.confluent.connect.avro.Enum.CG":"CG","io.confluent.connect.avro.Enum.I":"I","io.confluent.connect.avro.Enum.L":"L","io.confluent.connect.avro.Enum.LC":"LC","io.confluent.connect.avro.Enum.LE":"LE","io.confluent.connect.avro.Enum.LM":"LM","io.confluent.connect.avro.Enum.LP":"LP","io.confluent.connect.avro.Enum.LS":"LS","io.confluent.connect.avro.Enum.MM":"MM","io.confluent.connect.avro.Enum.MP":"MP","io.confluent.connect.avro.Enum.N":"N","io.confluent.connect.avro.Enum.PP":"PP","io.confluent.connect.avro.Enum.PT":"PT","io.confluent.connect.avro.Enum.R":"R"},"connect.name":"es.eci.firefly.avro.mixer.canonical.model.offer.CategoryType"}],"default":null}],"connect.name":"es.eci.firefly.avro.mixer.canonical.model.offer.Category"}}],"default":null},{"name":"center_packing_units","type":["null",{"type":"array","items":{"type":"record","name":"CenterPackingUnits","fields":[{"name":"center_id","type":{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}},{"name":"units_by_center","type":{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}}],"connect.name":"es.eci.firefly.avro.mixer.canonical.model.offer.CenterPackingUnits"}}],"default":null},{"name":"color","type":["null","Color"],"default":null},{"name":"cross_selling","type":["null",{"type":"array","items":"BarCodeVariant"}],"default":null},{"name":"cross_selling_product_ids","type":["null",{"type":"array","items":{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}}],"default":null},{"name":"customised_request","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"date_created_catalog","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"depth","type":["null",{"type":"record","name":"QuantitativeValue","fields":[{"name":"unit_code","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"unit_text","type":["null",{"type":"array","items":"I18nText"}],"default":null},{"name":"value","type":{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}}],"connect.name":"es.eci.firefly.avro.mixer.canonical.model.offer.QuantitativeValue"}],"default":null},{"name":"description","type":["null",{"type":"array","items":"I18nText"}],"default":null},{"name":"description_seo","type":["null",{"type":"array","items":"I18nText"}],"default":null},{"name":"digital_store","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"edition_type","type":["null",{"type":"record","name":"LookupTable","fields":[{"name":"description","type":{"type":"array","items":"I18nText"}},{"name":"external_identifier","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"glossary","type":["null","Glossary"],"default":null},{"name":"identifier","type":{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}},{"name":"name","type":["null",{"type":"array","items":"I18nText"}],"default":null},{"name":"type_lookup","type":{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}}],"connect.name":"es.eci.firefly.avro.mixer.canonical.model.offer.LookupTable"}],"default":null},{"name":"edition_types","type":["null",{"type":"array","items":"LookupTable"}],"default":null},{"name":"energy_rating","type":["null",{"type":"record","name":"EnergyRating","fields":[{"name":"classification","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"description","type":{"type":"array","items":"I18nText"}},{"name":"external_identifier","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"glossary","type":["null","Glossary"],"default":null},{"name":"identifier","type":{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}},{"name":"name","type":["null",{"type":"array","items":"I18nText"}],"default":null},{"name":"type_lookup","type":{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}}],"connect.name":"es.eci.firefly.avro.mixer.canonical.model.offer.EnergyRating"}],"default":null},{"name":"environment_signal","type":["null",{"type":"record","name":"EnvironmentSignal","fields":[{"name":"description","type":{"type":"array","items":"I18nText"}},{"name":"external_identifier","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"glossary","type":["null","Glossary"],"default":null},{"name":"identifier","type":{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}},{"name":"name","type":["null",{"type":"array","items":"I18nText"}],"default":null},{"name":"type_lookup","type":{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}}],"connect.name":"es.eci.firefly.avro.mixer.canonical.model.offer.EnvironmentSignal"}],"default":null},{"name":"external_identifier","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"gift_card_type","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"gross_volume","type":["null","QuantitativeValue"],"default":null},{"name":"gtin","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"h1_seo","type":["null",{"type":"array","items":"I18nText"}],"default":null},{"name":"has_complete_documentation","type":["null","boolean"],"default":null},{"name":"has_own_pum","type":["null","boolean"],"default":null},{"name":"height","type":["null","QuantitativeValue"],"default":null},{"name":"highlighted","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"identifier","type":{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}},{"name":"image_color","type":["null",{"type":"record","name":"ImageColor","fields":[{"name":"mode","type":{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}},{"name":"name","type":{"type":"array","items":"I18nText"}},{"name":"ratio","type":{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}},{"name":"url","type":{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}}],"connect.name":"es.eci.firefly.avro.mixer.canonical.model.offer.ImageColor"}],"default":null},{"name":"internal_gtin","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"is_best_sellers_excluded","type":["null","boolean"],"default":null},{"name":"is_comparable","type":["null","boolean"],"default":null},{"name":"is_digital","type":["null","boolean"],"default":null},{"name":"is_eci_express","type":["null","boolean"],"default":null},{"name":"is_eci_express_exclusive","type":["null","boolean"],"default":null},{"name":"is_followed","type":["null","boolean"],"default":null},{"name":"is_indexed","type":["null","boolean"],"default":null},{"name":"is_infinite_stock","type":["null","boolean"],"default":null},{"name":"is_luxury","type":["null","boolean"],"default":null},{"name":"is_soldable_without_image","type":["null","boolean"],"default":null},{"name":"is_variant_of","type":["null","Product"],"default":null},{"name":"is_weight_scale","type":["null","boolean"],"default":null},{"name":"keyword_seo","type":["null",{"type":"array","items":"I18nText"}],"default":null},{"name":"legislation","type":["null",{"type":"array","items":{"type":"record","name":"Legislation","fields":[{"name":"applicable_regulation","type":["null",{"type":"array","items":{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}}],"default":null},{"name":"product_type","type":["null",{"type":"array","items":"I18nText"}],"default":null},{"name":"security_information","type":["null",{"type":"array","items":{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}}],"default":null}],"connect.name":"es.eci.firefly.avro.mixer.canonical.model.offer.Legislation"}}],"default":null},{"name":"license","type":["null","LookupTable"],"default":null},{"name":"main_category","type":["null","Category"],"default":null},{"name":"manufacturer","type":["null",{"type":"record","name":"Organization","fields":[{"name":"description","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"external_identifier","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"identifier","type":{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}},{"name":"name","type":["null",{"type":"array","items":"I18nText"}],"default":null},{"name":"postal_address","type":["null",{"type":"record","name":"PostalAddress","fields":[{"name":"street_address","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null}],"connect.name":"es.eci.firefly.avro.mixer.canonical.model.offer.PostalAddress"}],"default":null},{"name":"telephone","type":["null",{"type":"record","name":"Telephone","fields":[{"name":"number","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null}],"connect.name":"es.eci.firefly.avro.mixer.canonical.model.offer.Telephone"}],"default":null}],"connect.name":"es.eci.firefly.avro.mixer.canonical.model.offer.Organization"}],"default":null},{"name":"max_weight","type":["null","QuantitativeValue"],"default":null},{"name":"min_weight","type":["null","QuantitativeValue"],"default":null},{"name":"model_detail","type":["null",{"type":"record","name":"ModelDetail","fields":[{"name":"model_id","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"web_model_description","type":["null",{"type":"array","items":"I18nText"}],"default":null},{"name":"web_model_id","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null}],"connect.name":"es.eci.firefly.avro.mixer.canonical.model.offer.ModelDetail"}],"default":null},{"name":"name","type":["null",{"type":"array","items":"I18nText"}],"default":null},{"name":"offers","type":["null",{"type":"array","items":{"type":"record","name":"Offer","fields":[{"name":"applied_taxes","type":["null",{"type":"array","items":"AppliedTaxes"}],"default":null},{"name":"availability_ends","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"availability_starts","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"barra","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"bundle_of","type":["null",{"type":"array","items":"Product"}],"default":null},{"name":"company","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"date_deleted","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"date_recoded","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"delivery_point","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"delivery_type_badges","type":["null",{"type":"record","name":"DeliveryTypes","fields":[{"name":"is_eci_express","type":["null","boolean"],"default":null},{"name":"is_eci_pickup","type":["null","boolean"],"default":null},{"name":"is_home_delivery","type":["null","boolean"],"default":null},{"name":"is_internet_exclusive","type":["null","boolean"],"default":null},{"name":"is_sts_pickup","type":["null","boolean"],"default":null},{"name":"sts_companies","type":["null",{"type":"array","items":"Organization"}],"default":null}],"connect.name":"es.eci.firefly.avro.mixer.canonical.model.offer.DeliveryTypes"}],"default":null},{"name":"department","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"external_id","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"external_identifier","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"family","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"financing_data","type":["null",{"type":"record","name":"FinancingData","fields":[{"name":"end_date","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"grace_period","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"has_calculator","type":["null","boolean"],"default":null},{"name":"is_no_interest","type":["null","boolean"],"default":null},{"name":"monthly_payments","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"number_payments","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"start_date","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"total_amount","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"type","type":["null","LookupTable"],"default":null}],"connect.name":"es.eci.firefly.avro.mixer.canonical.model.offer.FinancingData"}],"default":null},{"name":"good_types","type":["null",{"type":"array","items":{"type":"record","name":"GoodType","fields":[{"name":"external_identifier","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"identifier","type":{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}},{"name":"name","type":["null",{"type":"array","items":"I18nText"}],"default":null}],"connect.name":"es.eci.firefly.avro.mixer.canonical.model.offer.GoodType"}}],"default":null},{"name":"has_european_conformity_certification","type":["null","boolean"],"default":null},{"name":"identifier","type":{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}},{"name":"is_non_technological","type":["null","boolean"],"default":null},{"name":"is_publish","type":["null","boolean"],"default":null},{"name":"is_return_allowed","type":["null","boolean"],"default":null},{"name":"is_salable","type":["null","boolean"],"default":null},{"name":"is_searchable","type":["null","boolean"],"default":null},{"name":"is_show_offer","type":["null","boolean"],"default":null},{"name":"item_offered","type":["null",{"type":"record","name":"ProductVariant","fields":[{"name":"additional_product_descriptions","type":["null",{"type":"array","items":"ProductAdditionalDescription"}],"default":null},{"name":"badges","type":["null",{"type":"array","items":"Badge"}],"default":null},{"name":"bar_code_variants","type":["null",{"type":"array","items":"BarCodeVariant"}],"default":null},{"name":"basic_color","type":["null","Color"],"default":null},{"name":"calculated_badges","type":["null",{"type":"array","items":"CalculatedBadge"}],"default":null},{"name":"color","type":["null","Color"],"default":null},{"name":"customised_request","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"date_created_catalog","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"depth","type":["null","QuantitativeValue"],"default":null},{"name":"description","type":["null",{"type":"array","items":"I18nText"}],"default":null},{"name":"digital_store","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"edition_type","type":["null","LookupTable"],"default":null},{"name":"edition_types","type":["null",{"type":"array","items":"LookupTable"}],"default":null},{"name":"energy_rating","type":["null","EnergyRating"],"default":null},{"name":"environment_signal","type":["null","EnvironmentSignal"],"default":null},{"name":"external_identifier","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"gross_volume","type":["null","QuantitativeValue"],"default":null},{"name":"gtin","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"has_complete_documentation","type":["null","boolean"],"default":null},{"name":"has_own_pum","type":["null","boolean"],"default":null},{"name":"height","type":["null","QuantitativeValue"],"default":null},{"name":"highlighted","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"identifier","type":{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}},{"name":"image_color","type":["null","ImageColor"],"default":null},{"name":"internal_gtin","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"is_best_sellers_excluded","type":["null","boolean"],"default":null},{"name":"is_comparable","type":["null","boolean"],"default":null},{"name":"is_digital","type":["null","boolean"],"default":null},{"name":"is_eci_express","type":["null","boolean"],"default":null},{"name":"is_eci_express_exclusive","type":["null","boolean"],"default":null},{"name":"is_infinite_stock","type":["null","boolean"],"default":null},{"name":"is_luxury","type":["null","boolean"],"default":null},{"name":"is_soldable_without_image","type":["null","boolean"],"default":null},{"name":"is_variant_of","type":["null","Product"],"default":null},{"name":"is_weight_scale","type":["null","boolean"],"default":null},{"name":"license","type":["null","LookupTable"],"default":null},{"name":"manufacturer","type":["null","Organization"],"default":null},{"name":"max_weight","type":["null","QuantitativeValue"],"default":null},{"name":"min_weight","type":["null","QuantitativeValue"],"default":null},{"name":"model_detail","type":["null","ModelDetail"],"default":null},{"name":"name","type":["null",{"type":"array","items":"I18nText"}],"default":null},{"name":"offers","type":["null",{"type":"array","items":"Offer"}],"default":null},{"name":"pantone","type":["null","LookupTable"],"default":null},{"name":"piece_weight","type":["null","QuantitativeValue"],"default":null},{"name":"presentation_details","type":["null",{"type":"record","name":"PresentationDetails","fields":[{"name":"format","type":["null",{"type":"record","name":"Format","fields":[{"name":"description","type":{"type":"array","items":"I18nText"}},{"name":"external_identifier","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"glossary","type":["null","Glossary"],"default":null},{"name":"identifier","type":{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}},{"name":"name","type":["null",{"type":"array","items":"I18nText"}],"default":null},{"name":"type_lookup","type":{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}}],"connect.name":"es.eci.firefly.avro.mixer.canonical.model.offer.Format"}],"default":null},{"name":"freshness","type":["null",{"type":"record","name":"Freshness","fields":[{"name":"freshness_days","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"is_alive","type":["null","boolean"],"default":null},{"name":"is_consum_day","type":["null","boolean"],"default":null}],"connect.name":"es.eci.firefly.avro.mixer.canonical.model.offer.Freshness"}],"default":null},{"name":"is_panification_product","type":["null","boolean"],"default":null},{"name":"package_unit","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"presentation","type":["null",{"type":"record","name":"Presentation","fields":[{"name":"description","type":{"type":"array","items":"I18nText"}},{"name":"external_identifier","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"glossary","type":["null","Glossary"],"default":null},{"name":"identifier","type":{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}},{"name":"name","type":["null",{"type":"array","items":"I18nText"}],"default":null},{"name":"type_lookup","type":{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}}],"connect.name":"es.eci.firefly.avro.mixer.canonical.model.offer.Presentation"}],"default":null},{"name":"quantity","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null}],"connect.name":"es.eci.firefly.avro.mixer.canonical.model.offer.PresentationDetails"}],"default":null},{"name":"previous_gtins","type":["null",{"type":"array","items":{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}}],"default":null},{"name":"priority_number","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"product_description","type":["null",{"type":"array","items":{"type":"record","name":"I18nProductDescription","fields":[{"name":"locale","type":{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}},{"name":"value","type":["null",{"type":"array","items":{"type":"record","name":"ProductDescription","fields":[{"name":"name","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"value","type":["null",{"type":"array","items":{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}}],"default":null}],"connect.name":"es.eci.firefly.avro.mixer.canonical.model.offer.ProductDescription"}}],"default":null}],"connect.name":"es.eci.firefly.avro.mixer.canonical.model.offer.I18nProductDescription"}}],"default":null},{"name":"purchase_management_stock","type":["null",{"type":"record","name":"PurchaseManagementStock","fields":[{"name":"classifications","type":["null",{"type":"array","items":{"type":"record","name":"Classification","fields":[{"name":"external_identifier","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"identifier","type":{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}},{"name":"name","type":["null",{"type":"array","items":"I18nText"}],"default":null},{"name":"value","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null}],"connect.name":"es.eci.firefly.avro.mixer.canonical.model.offer.Classification"}}],"default":null},{"name":"management_type","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null}],"connect.name":"es.eci.firefly.avro.mixer.canonical.model.offer.PurchaseManagementStock"}],"default":null},{"name":"range","type":["null",{"type":"record","name":"Range","namespace":"es.eci.firefly.avro.mixer.canonical.model.offer.features","fields":[{"name":"external_identifier","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"identifier","type":{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}},{"name":"name","type":["null",{"type":"array","items":"es.eci.firefly.avro.mixer.canonical.model.offer.I18nText"}],"default":null}],"connect.name":"es.eci.firefly.avro.mixer.canonical.model.offer.features.Range"}],"default":null},{"name":"related_services","type":["null",{"type":"array","items":{"type":"record","name":"SignalService","fields":[{"name":"mode","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"name","type":{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}}],"connect.name":"es.eci.firefly.avro.mixer.canonical.model.offer.SignalService"}}],"default":null},{"name":"sale_type","type":["null","LookupTable"],"default":null},{"name":"service_agreement_type","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"sizes","type":["null",{"type":"array","items":{"type":"record","name":"SizeInfo","fields":[{"name":"main_size","type":["null",{"type":"record","name":"Size","fields":[{"name":"external_identifier","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"identifier","type":{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}},{"name":"name","type":["null",{"type":"array","items":"I18nText"}],"default":null},{"name":"size_type","type":["null",{"type":"record","name":"SizeType","fields":[{"name":"external_identifier","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"identifier","type":{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}},{"name":"name","type":["null",{"type":"array","items":"I18nText"}],"default":null}],"connect.name":"es.eci.firefly.avro.mixer.canonical.model.offer.SizeType"}],"default":null}],"connect.name":"es.eci.firefly.avro.mixer.canonical.model.offer.Size"}],"default":null},{"name":"manufacturer_size","type":["null","Size"],"default":null},{"name":"name","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null}],"connect.name":"es.eci.firefly.avro.mixer.canonical.model.offer.SizeInfo"}}],"default":null},{"name":"special_handling_type","type":["null",{"type":"record","name":"SpecialHandlingType","fields":[{"name":"is_alcohol","type":["null","boolean"],"default":null},{"name":"is_preparation_required","type":["null","boolean"],"default":null}],"connect.name":"es.eci.firefly.avro.mixer.canonical.model.offer.SpecialHandlingType"}],"default":null},{"name":"special_merchandise_indicator","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"specific_attribute_groups","type":["null",{"type":"array","items":{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}}],"default":null},{"name":"specific_attributes","type":["null",{"type":"array","items":{"type":"record","name":"SpecificAttributes","fields":[{"name":"attributes","type":["null",{"type":"array","items":{"type":"record","name":"Attribute","fields":[{"name":"cardinality","type":{"type":"string","connect.parameters":{"io.confluent.connect.avro.Enum":"es.eci.firefly.avro.mixer.canonical.model.offer.AttributeCardinality","io.confluent.connect.avro.Enum.ONE":"ONE","io.confluent.connect.avro.Enum.MANY":"MANY"},"connect.name":"es.eci.firefly.avro.mixer.canonical.model.offer.AttributeCardinality"}},{"name":"is_i18n","type":"boolean"},{"name":"name","type":{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}},{"name":"type","type":{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}},{"name":"values","type":["null",{"type":"array","items":{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}}],"default":null}],"connect.name":"es.eci.firefly.avro.mixer.canonical.model.offer.Attribute"}}],"default":null},{"name":"group_name","type":{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}},{"name":"group_type","type":["null",{"type":"string","connect.parameters":{"io.confluent.connect.avro.Enum":"es.eci.firefly.avro.mixer.canonical.model.offer.GroupType","io.confluent.connect.avro.Enum.SPEC_STORE":"SPEC_STORE","io.confluent.connect.avro.Enum.SPEC_SECONDARY_TEMPLATE":"SPEC_SECONDARY_TEMPLATE"},"connect.name":"es.eci.firefly.avro.mixer.canonical.model.offer.GroupType"}],"default":null}],"connect.name":"es.eci.firefly.avro.mixer.canonical.model.offer.SpecificAttributes"}}],"default":null},{"name":"target_market","type":["null",{"type":"record","name":"TargetMarket","fields":[{"name":"age_from","type":["null","QuantitativeValue"],"default":null},{"name":"age_to","type":["null","QuantitativeValue"],"default":null},{"name":"gender_segmentation","type":["null",{"type":"array","items":{"type":"record","name":"GenderSegmentation","fields":[{"name":"description","type":{"type":"array","items":"I18nText"}},{"name":"external_identifier","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"glossary","type":["null","Glossary"],"default":null},{"name":"identifier","type":{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}},{"name":"name","type":["null",{"type":"array","items":"I18nText"}],"default":null},{"name":"type_lookup","type":{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}}],"connect.name":"es.eci.firefly.avro.mixer.canonical.model.offer.GenderSegmentation"}}],"default":null},{"name":"recommended_age","type":["null","QuantitativeValue"],"default":null}],"connect.name":"es.eci.firefly.avro.mixer.canonical.model.offer.TargetMarket"}],"default":null},{"name":"time_restrictions","type":["null",{"type":"record","name":"TimeRestriction","fields":[{"name":"is_alcohol","type":["null","boolean"],"default":null},{"name":"is_bread","type":["null","boolean"],"default":null},{"name":"is_cooled","type":["null","boolean"],"default":null},{"name":"is_frozen","type":["null","boolean"],"default":null},{"name":"is_required_preparation","type":["null","boolean"],"default":null}],"connect.name":"es.eci.firefly.avro.mixer.canonical.model.offer.TimeRestriction"}],"default":null},{"name":"transport_details","type":["null",{"type":"record","name":"TransportDetails","fields":[{"name":"bulky_description","type":["null","LookupTable"],"default":null},{"name":"is_bulky","type":["null","boolean"],"default":null}],"connect.name":"es.eci.firefly.avro.mixer.canonical.model.offer.TransportDetails"}],"default":null},{"name":"unique_code","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"width","type":["null","QuantitativeValue"],"default":null}],"connect.name":"es.eci.firefly.avro.mixer.canonical.model.offer.ProductVariant"}],"default":null},{"name":"launch_date","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"name","type":["null",{"type":"array","items":"I18nText"}],"default":null},{"name":"origin","type":["null",{"type":"record","name":"Origin","fields":[{"name":"identifier","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null}],"connect.name":"es.eci.firefly.avro.mixer.canonical.model.offer.Origin"}],"default":null},{"name":"publish_ends","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"publish_starts","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"refurbishment_level","type":["null","LookupTable"],"default":null},{"name":"sales_reference","type":{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}},{"name":"section","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"seller","type":{"type":"record","name":"Seller","fields":[{"name":"external_identifier","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"identifier","type":{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}},{"name":"name","type":["null",{"type":"array","items":"I18nText"}],"default":null}],"connect.name":"es.eci.firefly.avro.mixer.canonical.model.offer.Seller"}},{"name":"size_code","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"type_reference","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"volume_limit","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null}],"connect.name":"es.eci.firefly.avro.mixer.canonical.model.offer.Offer"}}],"default":null},{"name":"pack_products","type":["null",{"type":"array","items":{"type":"record","name":"Bundle","fields":[{"name":"is_linkable","type":["null","boolean"],"default":null},{"name":"product","type":"Product"},{"name":"quantity","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null}],"connect.name":"es.eci.firefly.avro.mixer.canonical.model.offer.Bundle"}}],"default":null},{"name":"pack_type","type":["null",{"type":"string","connect.parameters":{"io.confluent.connect.avro.Enum":"es.eci.firefly.avro.mixer.canonical.model.offer.PackType","io.confluent.connect.avro.Enum.COLLECTION":"COLLECTION","io.confluent.connect.avro.Enum.AMBIENT":"AMBIENT","io.confluent.connect.avro.Enum.COORDINATED":"COORDINATED","io.confluent.connect.avro.Enum.LOTE":"LOTE","io.confluent.connect.avro.Enum.CONFIGURABLE":"CONFIGURABLE","io.confluent.connect.avro.Enum.FABRICANTE":"FABRICANTE"},"connect.name":"es.eci.firefly.avro.mixer.canonical.model.offer.PackType"}],"default":null},{"name":"pantone","type":["null","LookupTable"],"default":null},{"name":"piece_weight","type":["null","QuantitativeValue"],"default":null},{"name":"presentation_details","type":["null","PresentationDetails"],"default":null},{"name":"previous_gtins","type":["null",{"type":"array","items":{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}}],"default":null},{"name":"priority_number","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"product_description","type":["null",{"type":"array","items":"I18nProductDescription"}],"default":null},{"name":"product_id","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"product_sorting_details","type":["null",{"type":"record","name":"ProductSortingDetails","fields":[{"name":"brand_sorting","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"category_sorting","type":["null",{"type":"array","items":{"type":"record","name":"ProductCategorySorting","fields":[{"name":"category","type":"Category"},{"name":"sort","type":{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}}],"connect.name":"es.eci.firefly.avro.mixer.canonical.model.offer.ProductCategorySorting"}}],"default":null},{"name":"size_sorting","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null}],"connect.name":"es.eci.firefly.avro.mixer.canonical.model.offer.ProductSortingDetails"}],"default":null},{"name":"product_variants","type":["null",{"type":"array","items":"ProductVariant"}],"default":null},{"name":"provider","type":["null",{"type":"record","name":"Provider","fields":[{"name":"brand","type":["null","Brand"],"default":null},{"name":"description","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"external_identifier","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"identifier","type":{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}},{"name":"name","type":["null",{"type":"array","items":"I18nText"}],"default":null},{"name":"postal_address","type":["null","PostalAddress"],"default":null},{"name":"telephone","type":["null","Telephone"],"default":null}],"connect.name":"es.eci.firefly.avro.mixer.canonical.model.offer.Provider"}],"default":null},{"name":"purchase_management_stock","type":["null","PurchaseManagementStock"],"default":null},{"name":"range","type":["null","es.eci.firefly.avro.mixer.canonical.model.offer.features.Range"],"default":null},{"name":"related_cross_sell_product_ids","type":["null",{"type":"array","items":{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}}],"default":null},{"name":"related_services","type":["null",{"type":"array","items":"SignalService"}],"default":null},{"name":"robots_seo","type":["null",{"type":"array","items":"I18nText"}],"default":null},{"name":"sale_type","type":["null","LookupTable"],"default":null},{"name":"service_agreement_type","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"sizes","type":["null",{"type":"array","items":"SizeInfo"}],"default":null},{"name":"special_handling_type","type":["null","SpecialHandlingType"],"default":null},{"name":"special_merchandise_indicator","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"specific_attribute_groups","type":["null",{"type":"array","items":{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}}],"default":null},{"name":"specific_attributes","type":["null",{"type":"array","items":"SpecificAttributes"}],"default":null},{"name":"sub_brand","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"target_market","type":["null","TargetMarket"],"default":null},{"name":"time_restrictions","type":["null","TimeRestriction"],"default":null},{"name":"title_seo","type":["null",{"type":"array","items":"I18nText"}],"default":null},{"name":"transport_details","type":["null","TransportDetails"],"default":null},{"name":"type","type":["null",{"type":"record","name":"ProductType","fields":[{"name":"description","type":{"type":"array","items":"I18nText"}},{"name":"external_identifier","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"identifier","type":{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}},{"name":"name","type":["null",{"type":"array","items":"I18nText"}],"default":null},{"name":"parent_product_type","type":["null","ProductType"],"default":null}],"connect.name":"es.eci.firefly.avro.mixer.canonical.model.offer.ProductType"}],"default":null},{"name":"unique_code","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"varies_by","type":["null",{"type":"array","items":{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}}],"default":null},{"name":"warranty","type":["null",{"type":"record","name":"Warranty","fields":[{"name":"additional_info","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"description","type":["null",{"type":"array","items":"I18nText"}],"default":null},{"name":"duration","type":["null","QuantitativeValue"],"default":null},{"name":"url","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"urls","type":["null",{"type":"array","items":"I18nText"}],"default":null}],"connect.name":"es.eci.firefly.avro.mixer.canonical.model.offer.Warranty"}],"default":null},{"name":"width","type":["null","QuantitativeValue"],"default":null}],"connect.name":"es.eci.firefly.avro.mixer.canonical.model.offer.Product"}}],"default":null},{"name":"company","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"date_deleted","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"date_recoded","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"delivery_point","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"delivery_type_badges","type":["null","es.eci.firefly.avro.mixer.canonical.model.offer.DeliveryTypes"],"default":null},{"name":"department","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"external_id","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"external_identifier","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"family","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"financing_data","type":["null","es.eci.firefly.avro.mixer.canonical.model.offer.FinancingData"],"default":null},{"name":"good_types","type":["null",{"type":"array","items":"es.eci.firefly.avro.mixer.canonical.model.offer.GoodType"}],"default":null},{"name":"has_european_conformity_certification","type":["null","boolean"],"default":null},{"name":"identifier","type":{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}},{"name":"is_non_technological","type":["null","boolean"],"default":null},{"name":"is_publish","type":["null","boolean"],"default":null},{"name":"is_return_allowed","type":["null","boolean"],"default":null},{"name":"is_salable","type":["null","boolean"],"default":null},{"name":"is_searchable","type":["null","boolean"],"default":null},{"name":"is_show_offer","type":["null","boolean"],"default":null},{"name":"item_offered","type":["null","es.eci.firefly.avro.mixer.canonical.model.offer.ProductVariant"],"default":null},{"name":"launch_date","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"mixing_information","type":["null",{"type":"record","name":"MixingInformation","namespace":"es.eci.firefly.avro.mixer.canonical.model.mixer","fields":[{"name":"child_references","type":["null",{"type":"array","items":{"type":"record","name":"SiteOfferReference","namespace":"es.eci.firefly.avro.mixer.canonical.model.site","fields":[{"name":"barra","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"bundle_of","type":["null",{"type":"array","items":{"type":"record","name":"ProductReference","namespace":"es.eci.firefly.avro.mixer.canonical.model.offer","fields":[{"name":"identifier","type":{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}},{"name":"pack_products","type":["null",{"type":"array","items":{"type":"record","name":"BundleReference","fields":[{"name":"product","type":"ProductReference"},{"name":"quantity","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null}],"connect.name":"es.eci.firefly.avro.mixer.canonical.model.offer.BundleReference"}}],"default":null},{"name":"pack_type","type":["null",{"type":"string","connect.parameters":{"io.confluent.connect.avro.Enum":"es.eci.firefly.avro.mixer.canonical.model.offer.PackType","io.confluent.connect.avro.Enum.COLLECTION":"COLLECTION","io.confluent.connect.avro.Enum.AMBIENT":"AMBIENT","io.confluent.connect.avro.Enum.COORDINATED":"COORDINATED","io.confluent.connect.avro.Enum.LOTE":"LOTE","io.confluent.connect.avro.Enum.CONFIGURABLE":"CONFIGURABLE","io.confluent.connect.avro.Enum.FABRICANTE":"FABRICANTE"},"connect.name":"es.eci.firefly.avro.mixer.canonical.model.offer.PackType"}],"default":null},{"name":"product_id","type":{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}},{"name":"product_variants","type":["null",{"type":"array","items":{"type":"record","name":"ProductVariantReference","fields":[{"name":"bar_code_variants","type":["null",{"type":"array","items":{"type":"record","name":"BarCodeVariantReference","fields":[{"name":"gtin","type":{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}},{"name":"is_main","type":["null","boolean"],"default":null}],"connect.name":"es.eci.firefly.avro.mixer.canonical.model.offer.BarCodeVariantReference"}}],"default":null},{"name":"gtin","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"identifier","type":{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}},{"name":"is_variant_of","type":["null","ProductReference"],"default":null},{"name":"offers","type":["null",{"type":"array","items":{"type":"record","name":"OfferReference","fields":[{"name":"barra","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"bundle_of","type":["null",{"type":"array","items":"ProductReference"}],"default":null},{"name":"company","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"department","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"family","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"good_types","type":["null",{"type":"array","items":{"type":"record","name":"GoodType","namespace":"es.eci.firefly.avro.mixer.canonical.model.mixer","fields":[{"name":"identifier","type":{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}},{"name":"name","type":["null",{"type":"array","items":"es.eci.firefly.avro.mixer.canonical.model.offer.I18nText"}],"default":null}],"connect.name":"es.eci.firefly.avro.mixer.canonical.model.mixer.GoodType"}}],"default":null},{"name":"is_salable","type":["null","boolean"],"default":null},{"name":"is_show_offer","type":["null","boolean"],"default":null},{"name":"item_offered","type":["null","ProductVariantReference"],"default":null},{"name":"sales_reference","type":{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}},{"name":"section","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"seller","type":{"type":"record","name":"SellerReference","fields":[{"name":"identifier","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null}],"connect.name":"es.eci.firefly.avro.mixer.canonical.model.offer.SellerReference"}},{"name":"size_code","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null}],"connect.name":"es.eci.firefly.avro.mixer.canonical.model.offer.OfferReference"}}],"default":null},{"name":"previous_gtins","type":["null",{"type":"array","items":{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}}],"default":null},{"name":"unique_code","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null}],"connect.name":"es.eci.firefly.avro.mixer.canonical.model.offer.ProductVariantReference"}}],"default":null}],"connect.name":"es.eci.firefly.avro.mixer.canonical.model.offer.ProductReference"}}],"default":null},{"name":"company","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"department","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"family","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"good_types","type":["null",{"type":"array","items":"es.eci.firefly.avro.mixer.canonical.model.mixer.GoodType"}],"default":null},{"name":"is_salable","type":["null","boolean"],"default":null},{"name":"is_show_offer","type":["null","boolean"],"default":null},{"name":"item_offered","type":["null","es.eci.firefly.avro.mixer.canonical.model.offer.ProductVariantReference"],"default":null},{"name":"sales_reference","type":{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}},{"name":"section","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"seller","type":"es.eci.firefly.avro.mixer.canonical.model.offer.SellerReference"},{"name":"site","type":["null",{"type":"record","name":"SiteInfo","fields":[{"name":"delivery_method","type":["null",{"type":"record","name":"DeliveryMethods","fields":[{"name":"available_centers","type":["null",{"type":"array","items":{"type":"record","name":"Center","fields":[{"name":"code","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null}],"connect.name":"es.eci.firefly.avro.mixer.canonical.model.site.Center"}}],"default":null}],"connect.name":"es.eci.firefly.avro.mixer.canonical.model.site.DeliveryMethods"}],"default":null},{"name":"identifier","type":{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}},{"name":"parent","type":["null",{"type":"record","name":"Site","fields":[{"name":"identifier","type":{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}},{"name":"parent","type":["null","Site"],"default":null}],"connect.name":"es.eci.firefly.avro.mixer.canonical.model.site.Site"}],"default":null},{"name":"price","type":["null",{"type":"record","name":"Price","fields":[{"name":"available_centers","type":["null",{"type":"array","items":"Center"}],"default":null}],"connect.name":"es.eci.firefly.avro.mixer.canonical.model.site.Price"}],"default":null}],"connect.name":"es.eci.firefly.avro.mixer.canonical.model.site.SiteInfo"}],"default":null},{"name":"size_code","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null}],"connect.name":"es.eci.firefly.avro.mixer.canonical.model.site.SiteOfferReference"}}],"default":null},{"name":"old_reference","type":["null","es.eci.firefly.avro.mixer.canonical.model.site.SiteOfferReference"],"default":null},{"name":"parent_reference","type":["null","es.eci.firefly.avro.mixer.canonical.model.site.SiteOfferReference"],"default":null},{"name":"type","type":{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}}],"connect.name":"es.eci.firefly.avro.mixer.canonical.model.mixer.MixingInformation"}],"default":null},{"name":"name","type":["null",{"type":"array","items":"es.eci.firefly.avro.mixer.canonical.model.offer.I18nText"}],"default":null},{"name":"origin","type":["null","es.eci.firefly.avro.mixer.canonical.model.offer.Origin"],"default":null},{"name":"publish_ends","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"publish_starts","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"refurbishment_level","type":["null","es.eci.firefly.avro.mixer.canonical.model.offer.LookupTable"],"default":null},{"name":"sales_reference","type":{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}},{"name":"section","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"seller","type":"es.eci.firefly.avro.mixer.canonical.model.offer.Seller"},{"name":"site","type":"es.eci.firefly.avro.mixer.canonical.model.site.Site"},{"name":"size_code","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"type_reference","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"volume_limit","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null}],"connect.name":"es.eci.firefly.avro.mixer.model.MixerOffer"}},{"name":"date","type":["null","long"],"default":null},{"name":"event_id","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"event_type","type":{"type":"string","connect.parameters":{"io.confluent.connect.avro.Enum":"es.eci.firefly.avro.commons.event.EventType","io.confluent.connect.avro.Enum.CREATE":"CREATE","io.confluent.connect.avro.Enum.UPDATE":"UPDATE","io.confluent.connect.avro.Enum.UPSERT":"UPSERT","io.confluent.connect.avro.Enum.DELETE":"DELETE","io.confluent.connect.avro.Enum.TIMER":"TIMER"},"connect.name":"es.eci.firefly.avro.commons.event.EventType"}},{"name":"is_priority","type":"boolean","default":false},{"name":"metadata","type":["null",{"type":"array","items":{"type":"record","name":"EventMetadata","namespace":"es.eci.firefly.avro.commons.event","fields":[{"name":"date_created","type":{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}},{"name":"origin","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"trace_id","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"user","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null}],"connect.name":"es.eci.firefly.avro.commons.event.EventMetadata"}}],"default":null},{"name":"process","type":["null",{"type":"string","connect.parameters":{"avro.java.string":"String"},"avro.java.string":"String"}],"default":null},{"name":"partition","type":["null","int"],"default":null},{"name":"offset","type":["null","long"],"default":null},{"name":"timestamp","type":["null","long"],"default":null}],"connect.version":19,"connect.name":"es.eci.firefly.avro.mixer.event.MixerOfferEvent"} ,7ǻmf!Loffer(A41279013).publish_ends_extendedIVA02IGIC91
+01497 001 004340964247001 471 COMPRA EN FIRME
+es_ESCOMPRA EN FIRME STERLING
+es_ESSTERLING $001004347101497004 &includeAsWebNovelty
+es_ESNovedad DVD "2021-09-29T22:00Z8434915528847 Multicolor
+es_ESMulticolor Multicolor
+es_ESMulticolor Multicolor
+es_ESMulticolor 352633090
+es_ESMulticolor 598507054
+20205
+IMAGE
+small https://sgfm.elcorteingles.es/SGFM/dctm/NO_RENDER02/201205/22/___20205O1.png
+es_ES6COLOR_MULTICOLOR_SECUNDARIO
+es_ES6Color Multicolor Secundario Multicolor
+es_ESMulticolor 0(2021-12-14T12:54:39Z
+es_ES(PACK 2 BOXER POPELIN 155
+es_ES,Mercancía general ECI 274979090 ,Mercancía general ECI 395 ML
+es_ESml
+0.0008434915528847 $000001519039300004TRADICIONAL
+es_ES6COLOR_MULTICOLOR_SECUNDARIO
+1.000https://sgfm.elcorteingles.es/SGFM/dctm/NO_RENDER02/201205/22/___20205O1.png2401812972260 123284056
+MIRTO marcasUrlLogo
+IMAGEmedium https://sgfm.elcorteingles.es/SGFM/69/35/2/00001269352/00001269352529mM1011.jpg
+es_ES
+Mirto
+es_ESBoxers de tela 999.39091454013
+es_ESBoxers de tela
+2.000
+es_ESRopa Interior 999.39072784013
+es_ESRopa Interior
+2.000
+es_ES Interior y baño 999.39072777013
+es_ES Interior y baño
+4.000
+es_ESModa hombre 999.39072454013
+es_ESModa hombre NNNN 155
+es_ES,Mercancía general ECI 274979090 ,Mercancía general ECI 395 A41279013
+es_ESBoxers de tela 999.39091454013
+es_ESBoxers de tela
+2.000
+es_ESRopa Interior 999.39072784013
+es_ESRopa Interior
+2.000
+es_ES Interior y baño 999.39072777013
+es_ES Interior y baño
+4.000
+es_ESModa hombre 999.39072454013
+es_ESModa hombre NNNN 5143471600362624
+es_ES*CREACIONES MIRTO S.A. pCALLE EMILIO MUÑOZ.,57-28037 MADRID - MADRID (ESPAñA)
+es_ES^Pack de dos boxers de tela de hombre multicolor
+es_ES Pack de dos boxers de tela con abertura y botón, tejido 100% algodón. Cintura elástica con el interior forrado de felpa para una mayor comodidad. Ropa Interior. Por razones de higiene, solo se admiten cambios y devoluciones de artículos precintados de origen y que no hayan sido desprecintados. A41279013 123284056
+MIRTO
+es_ES
+MIRTO 5143471600362624
+es_ES*CREACIONES MIRTO S.A. pCALLE EMILIO MUÑOZ.,57-28037 MADRID - MADRID (ESPAñA) 998.00195
+es_ESModa 998.00193
+es_ESModa 998
+es_ES@Jerarquía de Ficha de Productos fashionSize 302
+es_ES^Pack de dos boxers de tela de hombre multicolor
+es_ES^Pack de dos boxers de tela de hombre multicolor
+es_ES Pack de dos boxers de tela con abertura y botón, tejido 100% algodón. Cintura elástica con el interior forrado de felpa para una mayor comodidad. Ropa Interior. Por razones de higiene, solo se admiten cambios y devoluciones de artículos precintados de origen y que no hayan sido desprecintados. calificacion1 A calificacion2 A calificacion3 RV calificacion4 VV calificacion5 B calificacion6 T 00 N47936909052
+es_ES52 E
+es_ESEspañol 479437090L
+es_ESL 18171618090Fabricante
+Talla pz3c2qg46aax5qpler4twpwa2rdqphm44qlzhmwa2ypaxawji3i4mkoq ONE a2wkf4iqk324hsBOOLEAN
+false ONE a3frrndcro7fwwBOOLEAN
+false ONE a3l5iwa4qvnwkgBOOLEANtrue ONE a4qcksvnig5nwyENTITY{"identifier":"A156","name":[{"value":"A156","locale":"es_ES"}],"external_identifier":"24400027090","description":[{"value":"Despublicado por el agotador","locale":"es_ES"}],"type_lookup":"10110"} ONE a5c22lw7zunsbkBOOLEAN
+false ONE aaiiovqahsixwqBOOLEAN
+false ONE ae4bi37uqkffjcTIMESTAMP02022-09-15T03:12:35.000Z ONE afabpb3cbt6g3oTIMESTAMP02022-09-16T03:12:35.000Z ONE akzd7vzeui6lmuSTRINGD ONE apruxddhqugapwBOOLEAN
+false ONE apuaxkclolc2qcTIMESTAMP02022-05-10T04:17:34.000Z ONE ate3liesiuooycBOOLEAN
+false Fechas_WWW_ISS.SPEC_SECONDARY_TEMPLATEMANY a2hzkuh27pdfr2ENTITY
+ {"identifier":"Lavar máximo 30º","name":[{"value":"Lavar máximo 30º","locale":"es_ES"}],"external_identifier":"344723090","description":[{"value":"Lavar máximo 30º","locale":"es_ES"}],"type_lookup":"1584","glossary":{"term":[{"value":"LavarMaxTreinta","locale":"es_ES"}],"media_objects":[{"media_type":"IMAGE","url":"https://sgfm.elcorteingles.es/SGFM/dctm/NO_RENDER/20100622/___51640O1.png","size":"small"}],"description":[{"value":"Lavar máximo 30º","locale":"es_ES"}],"identifier":"51640","name":[{"value":"LAVARMAXTREINTA","locale":"es_ES"}],"external_identifier":"599743054"}}{"identifier":"No usar lejía","name":[{"value":"No usar lejía","locale":"es_ES"}],"external_identifier":"344724090","description":[{"value":"No usar lejía","locale":"es_ES"}],"type_lookup":"1584","glossary":{"term":[{"value":"NoUsarLejía","locale":"es_ES"}],"media_objects":[{"media_type":"IMAGE","url":"https://sgfm.elcorteingles.es/SGFM/dctm/NO_RENDER/20100622/___51313O1.jpg","size":"small"}],"description":[{"value":"No usar lejía.","locale":"es_ES"}],"identifier":"51313","name":[{"value":"NOUSARLEJIA","locale":"es_ES"}],"external_identifier":"599720054"}}{"identifier":"Planchado max 150º","name":[{"value":"Planchado max 150º","locale":"es_ES"}],"external_identifier":"349576090","description":[{"value":"Planchado max 150º","locale":"es_ES"}],"type_lookup":"1584","glossary":{"term":[{"value":"Planchar máximo a 150º","locale":"es_ES"},{"value":"Engomar no máximo a 150º","locale":"pt_PT"}],"media_objects":[{"media_type":"IMAGE","url":"https://sgfm.elcorteingles.es/SGFM/dctm/NO_RENDER02/201205/09/___20536O1.png","size":"small"}],"description":[{"value":"Planchar máximo a 150º","locale":"es_ES"},{"value":"Engomar no máximo a 150º","locale":"pt_PT"}],"identifier":"20536","name":[{"value":"PLANCHAR_MAXIMO_A_150º","locale":"es_ES"}],"external_identifier":"598551054"}}{"identifier":"Limpieza con percloroetileno","name":[{"value":"Limpieza con percloroetileno","locale":"es_ES"}],"external_identifier":"484888090","description":[{"value":"Limpieza con percloroetileno","locale":"es_ES"}],"type_lookup":"1584"} {"identifier":"No usar secadora","name":[{"value":"No usar secadora","locale":"es_ES"}],"external_identifier":"347671090","description":[{"value":"No usar secadora","locale":"es_ES"}],"type_lookup":"1584","glossary":{"term":[{"value":"No usar secadora","locale":"es_ES"}],"media_objects":[{"media_type":"IMAGE","url":"https://sgfm.elcorteingles.es/SGFM/dctm/NO_RENDER02/201205/10/___35917O1.png","size":"small"}],"description":[{"value":"No utilizar secadora","locale":"es_ES"}],"identifier":"35917","name":[{"value":"NO_USAR_SECADORA","locale":"es_ES"}],"external_identifier":"599260054"}} MANY a4ljbqm5nabztwENTITY{"identifier":"Primavera-Verano","name":[{"value":"Primavera-Verano","locale":"es_ES"}],"external_identifier":"479799090","description":[{"value":"Primavera-Verano","locale":"es_ES"}],"type_lookup":"1577"} ONE a4urgt4zxudifcENTITY{"identifier":"Multiestampado","name":[{"value":"Multiestampado","locale":"es_ES"}],"external_identifier":"26027803090","description":[{"value":"Multiestampado","locale":"es_ES"}],"type_lookup":"2504"} ONE ac5swxsdtmjapuENTITY{"identifier":"Boxers de tela","name":[{"value":"Boxers de tela","locale":"es_ES"}],"external_identifier":"35251408090","description":[{"value":"Boxers de tela","locale":"es_ES"}],"type_lookup":"1617"} MANY adla3f5bsg737wENTITY{"identifier":"Hombre","name":[{"value":"Hombre","locale":"es_ES"}],"external_identifier":"344773090","description":[{"value":"Hombre","locale":"es_ES"}],"type_lookup":"1575"} ONE agr5xh55nzbgcyENTITY{"identifier":"Ropa interior y Calcetines","name":[{"value":"Ropa interior y Calcetines","locale":"es_ES"}],"external_identifier":"2700612090","description":[{"value":"Ropa interior y Calcetines","locale":"es_ES"}],"type_lookup":"1599"} MANY asxdasitnac2owCOMPLEX
+{"adoygdonopx3qq":[{"ascegjzrpcrvti":[{"adbyfb4jfvuv5k":100.0,"aydgn5j2m5edgq":{"identifier":"%","name":[{"value":"%","locale":"es_ES"}],"external_identifier":"344726090","description":[{"value":"%","locale":"en_GB"},{"value":"%","locale":"es_ES"},{"value":"%","locale":"fr_FR"},{"value":"%","locale":"pt_PT"}],"type_lookup":"214"},"a6zd77kj2mm2n4":{"identifier":"Algodón","name":[{"value":"Algodón","locale":"es_ES"}],"external_identifier":"344730090","description":[{"value":"Cotone","locale":"it_IT"},{"value":"Cotton","locale":"en_GB"},{"value":"Algodón","locale":"es_ES"},{"value":"Coton","locale":"fr_FR"},{"value":"Algodão","locale":"pt_PT"}],"type_lookup":"1633"}}]}]} MANY atc4u6vijhkdp6NUMBER2022 Moda_ISS.SPEC_SECONDARY_TEMPLATE
+es_ESNo voluminoso 347190090 No voluminoso
+es_ESNo voluminoso 401 $000001519039300004 (2022-05-10T04:17:34Z (2022-09-16T03:12:35Z(2022-09-15T03:12:35Z $001004347101497004 001001
+es_ES*El Corte Ingles, S.A. 2eciStore_MercanciaGeneraleciStore 004 H6d3af6bf-f7b5-4979-a431-4772650cfdd9DELETE <2023-09-02T17:23:44.030441150Z4dataloader-mongo-kafka-job 62023-09-02T17:23:50.973413Z(mixer-gtin-agr-kfreeH8d510ac8-003f-4aff-a208-087996c86ad6 62023-09-02T17:25:13.418344Z,mixer-gtin-calc-pcsflmHc64b2b7b-4068-477a-b289-5b292709bb87 62023-09-02T17:25:14.608444Z,mixer-gtin-calc-pcsflmH4d2bcafa-feea-46a4-bcda-a194144e0600 62023-09-03T03:12:27.241572Z0mixer-product-calc-kfreeH12f8fbc8-1523-473c-a768-8b56df682377 <2024-01-19T11:18:44.571341461Z$mixer-xsell-mixtriH9031b762-43e6-4c0a-92ad-13458f43afa5 62024-01-19T11:18:44.711178Z4mixer-packing-units-mixtriH0d9547c3-f666-45a4-87ca-29d5f5572990 62024-01-19T11:18:46.211192Z.mixer-offer-badge-mixbiH58b55b2d-b940-4840-8820-b5c36f620802 62024-01-19T11:25:55.895067Z.mixer-offer-badge-mixbiH5958faf0-c5f6-4eb3-9fd3-ed56984f9f84 (7Φ×cLoffer(A40778268).publish_ends_extendedIVA02IGIC91
+00080 001 088639986772001 237 (EXPLOTACIÓN DIRECTA
+es_ES(EXPLOTACIÓN DIRECTA STERLING
+es_ESSTERLING $001088623700080100 &includeAsWebNovelty
+es_ESNovedad DVD "2021-08-09T22:00Z3540513705957 Natural
+es_ESNatural Natural
+es_ESNatural
+Beige
+es_ES
+Beige 344754090
+es_ES
+Beige 599054054
+25619
+IMAGE
+small https://sgfm.elcorteingles.es/SGFM/dctm/NO_RENDER/20100629/___25619O1.png
+es_ESCOLOR_BEIGE
+es_ESColor Beige
+Beige
+es_ES
+Beige 0"2021-08-24T16:14Z
+es_ES*SUJETADOR DE CAPACIDA 158
+es_ES,Mercancía general ECI 274979090 ,Mercancía general ECI 395 ML
+es_ESml
+0.0003540513705957 $000001517200940100TRADICIONAL
+es_ESCOLOR_BEIGE
+1.000https://sgfm.elcorteingles.es/SGFM/dctm/NO_RENDER/20100629/___25619O1.png2401808447598 152002056LOU
+es_ESLou
+es_ESModa mujer 997.44571721011
+es_ESModa mujer 21.000
+es_ES30 997.39154516011
+es_ES30
+4.000
+es_ESjCross - Contenedor Venta Privada NO Acumulable OI2223 997.39128268011
+es_ESLContenedor Venta Privada no acumulable CDNN
+es_ESLencería 997.41964064011
+es_ESLencería
+6.000
+es_ESModa Mujer 997.41875780011
+es_ESModa Mujer 32.000
+es_ES,Cross - Contenedor 8DO 997.41837795011
+es_ESContenedor 8DO CDNN
+es_ES&Sujetadores con aro 999.39072749013
+es_ES&Sujetadores con aro
+8.000
+es_ESSujetadores 999.39072737013
+es_ESSujetadores 12.000
+es_ESLencería 999.39072733013
+es_ESLencería
+6.000
+es_ESModa mujer 999.39072442013
+es_ESModa mujer NNNN 158
+es_ES,Mercancía general ECI 274979090 ,Mercancía general ECI 395 A40778268
+es_ES&Sujetadores con aro 999.39072749013
+es_ES&Sujetadores con aro
+8.000
+es_ESSujetadores 999.39072737013
+es_ESSujetadores 12.000
+es_ESLencería 999.39072733013
+es_ESLencería
+6.000
+es_ESModa mujer 999.39072442013
+es_ESModa mujer NNNN 4872291600050500
+es_ES$LACELIER EUROPE SA CALLE PZ DE EUROPA, 22-24. 3ª PLANTA,..-08902 HOSPITALET DE LLOBREGAT - BARCELONA (ESPAñA)
+es_ES