From 10b22ddea39a98073cba570ed80184afa19efc10 Mon Sep 17 00:00:00 2001 From: Konstantine Karantasis Date: Thu, 20 Apr 2017 14:25:39 -0700 Subject: [PATCH 01/58] CC-491: Consolidate and simplify unit tests of HDFS connector --- .../connect/hdfs/avro/DataWriterAvroTest.java | 619 ++++++++---------- 1 file changed, 277 insertions(+), 342 deletions(-) diff --git a/src/test/java/io/confluent/connect/hdfs/avro/DataWriterAvroTest.java b/src/test/java/io/confluent/connect/hdfs/avro/DataWriterAvroTest.java index c22f2f2b3..1e3d26b53 100644 --- a/src/test/java/io/confluent/connect/hdfs/avro/DataWriterAvroTest.java +++ b/src/test/java/io/confluent/connect/hdfs/avro/DataWriterAvroTest.java @@ -35,15 +35,20 @@ import io.confluent.connect.hdfs.storage.StorageFactory; import io.confluent.connect.hdfs.wal.WAL; +import java.io.FileNotFoundException; +import java.io.IOException; import java.util.ArrayList; import java.util.Collection; +import java.util.Collections; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; +import static org.hamcrest.CoreMatchers.is; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -52,47 +57,23 @@ public class DataWriterAvroTest extends TestWithMiniDFSCluster { private static final String extension = ".avro"; private static final String ZERO_PAD_FMT = "%010d"; private SchemaFileReader schemaFileReader = new AvroFileReader(avroData); - + Partitioner partitioner; + @Test public void testWriteRecord() throws Exception { DataWriter hdfsWriter = new DataWriter(connectorConfig, context, avroData); - Partitioner partitioner = hdfsWriter.getPartitioner(); + partitioner = hdfsWriter.getPartitioner(); hdfsWriter.recover(TOPIC_PARTITION); - String key = "key"; - Schema schema = createSchema(); - Struct record = createRecord(schema); + List sinkRecords = createRecords(7); - Collection sinkRecords = new ArrayList<>(); - for (long offset = 0; offset < 7; offset++) { - SinkRecord sinkRecord = - new SinkRecord(TOPIC, PARTITION, Schema.STRING_SCHEMA, key, schema, record, offset); - - sinkRecords.add(sinkRecord); - } hdfsWriter.write(sinkRecords); hdfsWriter.close(assignment); hdfsWriter.stop(); - String encodedPartition = "partition=" + String.valueOf(PARTITION); - String directory = partitioner.generatePartitionedPath(TOPIC, encodedPartition); - // Last file (offset 6) doesn't satisfy size requirement and gets discarded on close - long[] validOffsets = {-1, 2, 5}; - for (int i = 1; i < validOffsets.length; i++) { - long startOffset = validOffsets[i - 1] + 1; - long endOffset = validOffsets[i]; - Path path = - new Path(FileUtils - .committedFileName(url, topicsDir, directory, TOPIC_PARTITION, startOffset, - endOffset, extension, ZERO_PAD_FMT)); - Collection records = schemaFileReader.readData(conf, path); - long size = endOffset - startOffset + 1; - assertEquals(size, records.size()); - for (Object avroRecord : records) { - assertEquals(avroData.fromConnectData(schema, record), avroRecord); - } - } + long[] validOffsets = {0, 3, 6}; + verify(sinkRecords, validOffsets); } @Test @@ -103,99 +84,58 @@ public void testRecovery() throws Exception { Class storageClass = (Class) Class.forName(connectorConfig.getString(HdfsSinkConnectorConfig.STORAGE_CLASS_CONFIG)); Storage storage = StorageFactory.createStorage(storageClass, conf, url); + DataWriter hdfsWriter = new DataWriter(connectorConfig, context, avroData); + partitioner = hdfsWriter.getPartitioner(); WAL wal = storage.wal(logsDir, TOPIC_PARTITION); wal.append(WAL.beginMarker, ""); - Set committedFiles = new HashSet<>(); - - String directory = TOPIC + "/" + "partition=" + String.valueOf(PARTITION); for (int i = 0; i < 5; ++i) { long startOffset = i * 10; long endOffset = (i + 1) * 10 - 1; - String tempfile = FileUtils.tempFileName(url, topicsDir, directory, extension); + String tempfile = FileUtils.tempFileName(url, topicsDir, getDirectory(), extension); fs.createNewFile(new Path(tempfile)); - String committedFile = FileUtils.committedFileName(url, topicsDir, directory, TOPIC_PARTITION, startOffset, + String committedFile = FileUtils.committedFileName(url, topicsDir, getDirectory(), TOPIC_PARTITION, startOffset, endOffset, extension, ZERO_PAD_FMT); - committedFiles.add(committedFile); wal.append(tempfile, committedFile); } wal.append(WAL.endMarker, ""); wal.close(); - DataWriter hdfsWriter = new DataWriter(connectorConfig, context, avroData); hdfsWriter.recover(TOPIC_PARTITION); Map offsets = context.offsets(); assertTrue(offsets.containsKey(TOPIC_PARTITION)); assertEquals(50L, (long) offsets.get(TOPIC_PARTITION)); - String key = "key"; - Schema schema = createSchema(); - Struct record = createRecord(schema); - - // Need enough records to trigger file rotation - ArrayList sinkRecords = new ArrayList<>(); - for (int i = 0; i < 3; i++) - sinkRecords.add( - new SinkRecord(TOPIC, PARTITION, Schema.STRING_SCHEMA, key, schema, record, 50 + i)); + List sinkRecords = createRecords(3, 50); hdfsWriter.write(sinkRecords); hdfsWriter.close(assignment); hdfsWriter.stop(); - committedFiles.add(FileUtils.committedFileName(url, topicsDir, directory, TOPIC_PARTITION, - 50, 52, extension, ZERO_PAD_FMT)); - FileStatus[] statuses = fs.listStatus(new Path(FileUtils.directoryName(url, topicsDir, directory)), - new TopicPartitionCommittedFileFilter(TOPIC_PARTITION)); - assertEquals(committedFiles.size(), statuses.length); - for (FileStatus status : statuses) { - assertTrue(committedFiles.contains(status.getPath().toString())); - } + long[] validOffsets = {0, 10, 20, 30, 40, 50, 53}; + verifyFileListing(validOffsets, Collections.singleton(new TopicPartition(TOPIC, PARTITION))); } @Test public void testWriteRecordMultiplePartitions() throws Exception { DataWriter hdfsWriter = new DataWriter(connectorConfig, context, avroData); + partitioner = hdfsWriter.getPartitioner(); for (TopicPartition tp: assignment) { hdfsWriter.recover(tp); } - String key = "key"; - Schema schema = createSchema(); - Struct record = createRecord(schema); - Collection sinkRecords = new ArrayList<>(); - for (TopicPartition tp: assignment) { - for (long offset = 0; offset < 7; offset++) { - SinkRecord sinkRecord = - new SinkRecord(tp.topic(), tp.partition(), Schema.STRING_SCHEMA, key, schema, record, offset); - sinkRecords.add(sinkRecord); - } - } + List sinkRecords = createRecords(7, 0, assignment); + hdfsWriter.write(sinkRecords); hdfsWriter.close(assignment); hdfsWriter.stop(); // Last file (offset 6) doesn't satisfy size requirement and gets discarded on close - long[] validOffsets = {-1, 2, 5}; - - for (TopicPartition tp : assignment) { - String directory = tp.topic() + "/" + "partition=" + String.valueOf(tp.partition()); - for (int j = 1; j < validOffsets.length; ++j) { - long startOffset = validOffsets[j - 1] + 1; - long endOffset = validOffsets[j]; - Path path = new Path( - FileUtils.committedFileName(url, topicsDir, directory, tp, startOffset, endOffset, - extension, ZERO_PAD_FMT)); - Collection records = schemaFileReader.readData(conf, path); - long size = endOffset - startOffset + 1; - assertEquals(records.size(), size); - for (Object avroRecord : records) { - assertEquals(avroRecord, avroData.fromConnectData(schema, record)); - } - } - } + long[] validOffsets = {0, 3, 6}; + verify(sinkRecords, validOffsets, assignment); } @Test @@ -229,65 +169,33 @@ public void testGetPreviousOffsets() throws Exception { } @Test - public void testWriteRecordNonZeroInitailOffset() throws Exception { + public void testWriteRecordNonZeroInitialOffset() throws Exception { DataWriter hdfsWriter = new DataWriter(connectorConfig, context, avroData); - Partitioner partitioner = hdfsWriter.getPartitioner(); + partitioner = hdfsWriter.getPartitioner(); hdfsWriter.recover(TOPIC_PARTITION); - String key = "key"; - Schema schema = createSchema(); - Struct record = createRecord(schema); - - Collection sinkRecords = new ArrayList<>(); - for (long offset = 3; offset < 10; offset++) { - SinkRecord sinkRecord = - new SinkRecord(TOPIC, PARTITION, Schema.STRING_SCHEMA, key, schema, record, offset); - sinkRecords.add(sinkRecord); - } + List sinkRecords = createRecords(7, 3); hdfsWriter.write(sinkRecords); hdfsWriter.close(assignment); hdfsWriter.stop(); - String directory = partitioner.generatePartitionedPath(TOPIC, "partition=" + String.valueOf(PARTITION)); - // Last file (offset 9) doesn't satisfy size requirement and gets discarded on close - long[] validOffsets = {2, 5, 8}; - for (int i = 1; i < validOffsets.length; i++) { - long startOffset = validOffsets[i - 1] + 1; - long endOffset = validOffsets[i]; - Path path = new Path(FileUtils.committedFileName(url, topicsDir, directory, - TOPIC_PARTITION, startOffset, endOffset, - extension, ZERO_PAD_FMT)); - Collection records = schemaFileReader.readData(conf, path); - long size = endOffset - startOffset + 1; - assertEquals(size, records.size()); - for (Object avroRecord : records) { - assertEquals(avroData.fromConnectData(schema, record), avroRecord); - } - } + long[] validOffsets = {3, 6, 9}; + verify(sinkRecords, validOffsets); } @Test public void testRebalance() throws Exception { DataWriter hdfsWriter = new DataWriter(connectorConfig, context, avroData); + partitioner = hdfsWriter.getPartitioner(); // Initial assignment is {TP1, TP2} for (TopicPartition tp: assignment) { hdfsWriter.recover(tp); } - String key = "key"; - Schema schema = createSchema(); - Struct record = createRecord(schema); - Collection sinkRecords = new ArrayList<>(); - for (TopicPartition tp: assignment) { - for (long offset = 0; offset < 7; offset++) { - SinkRecord sinkRecord = - new SinkRecord(tp.topic(), tp.partition(), Schema.STRING_SCHEMA, key, schema, record, offset); - sinkRecords.add(sinkRecord); - } - } + List sinkRecords = createRecords(7, 0, assignment); hdfsWriter.write(sinkRecords); Set oldAssignment = new HashSet<>(assignment); @@ -305,72 +213,26 @@ public void testRebalance() throws Exception { assertNotNull(hdfsWriter.getBucketWriter(TOPIC_PARTITION3)); // Last file (offset 6) doesn't satisfy size requirement and gets discarded on close - long[] validOffsetsTopicPartition2 = {-1, 2, 5}; - String directory2 = TOPIC + "/" + "partition=" + String.valueOf(PARTITION2); - for (int j = 1; j < validOffsetsTopicPartition2.length; ++j) { - long startOffset = validOffsetsTopicPartition2[j - 1] + 1; - long endOffset = validOffsetsTopicPartition2[j]; - Path path = new Path(FileUtils.committedFileName(url, topicsDir, directory2, - TOPIC_PARTITION2, startOffset, endOffset, - extension, ZERO_PAD_FMT)); - Collection records = schemaFileReader.readData(conf, path); - long size = endOffset - startOffset + 1; - assertEquals(records.size(), size); - for (Object avroRecord : records) { - assertEquals(avroData.fromConnectData(schema, record), avroRecord); - } - } + long[] validOffsetsTopicPartition2 = {0, 3, 6}; + verify(sinkRecords, validOffsetsTopicPartition2, Collections.singleton(TOPIC_PARTITION2), true); - sinkRecords.clear(); - for (TopicPartition tp: assignment) { - // Message offsets start at 6 because we discarded the in-progress temp file on rebalance - for (long offset = 6; offset < 10; offset++) { - SinkRecord sinkRecord = - new SinkRecord(tp.topic(), tp.partition(), Schema.STRING_SCHEMA, key, schema, record, offset); - sinkRecords.add(sinkRecord); - } - } + // Message offsets start at 6 because we discarded the in-progress temp file on re-balance + sinkRecords = createRecords(3, 6, assignment); hdfsWriter.write(sinkRecords); hdfsWriter.close(newAssignment); hdfsWriter.stop(); // Last file (offset 9) doesn't satisfy size requirement and gets discarded on close - long[] validOffsetsTopicPartition1 = {5, 8}; - String directory1 = TOPIC + "/" + "partition=" + String.valueOf(PARTITION); - for (int j = 1; j < validOffsetsTopicPartition1.length; ++j) { - long startOffset = validOffsetsTopicPartition1[j - 1] + 1; - long endOffset = validOffsetsTopicPartition1[j]; - Path path = new Path(FileUtils.committedFileName(url, topicsDir, directory1 , - TOPIC_PARTITION, startOffset, endOffset, - extension, ZERO_PAD_FMT)); - Collection records = schemaFileReader.readData(conf, path); - long size = endOffset - startOffset + 1; - assertEquals(records.size(), size); - for (Object avroRecord : records) { - assertEquals(avroData.fromConnectData(schema, record), avroRecord); - } - } + long[] validOffsetsTopicPartition1 = {6, 9}; + verify(sinkRecords, validOffsetsTopicPartition1, Collections.singleton(TOPIC_PARTITION), true); + + long[] validOffsetsTopicPartition3 = {6, 9}; + verify(sinkRecords, validOffsetsTopicPartition3, Collections.singleton(TOPIC_PARTITION3), true); - long[] validOffsetsTopicPartition3 = {5, 8}; - String directory3 = TOPIC + "/" + "partition=" + String.valueOf(PARTITION3); - for (int j = 1; j < validOffsetsTopicPartition3.length; ++j) { - long startOffset = validOffsetsTopicPartition3[j - 1] + 1; - long endOffset = validOffsetsTopicPartition3[j]; - Path path = new Path(FileUtils.committedFileName(url, topicsDir, directory3, - TOPIC_PARTITION3, startOffset, endOffset, - extension, ZERO_PAD_FMT)); - Collection records = schemaFileReader.readData(conf, path); - long size = endOffset - startOffset + 1; - assertEquals(records.size(), size); - for (Object avroRecord : records) { - assertEquals(avroData.fromConnectData(schema, record), avroRecord); - } - } assignment = oldAssignment; } - @Test public void testProjectBackWard() throws Exception { Map props = createProps(); @@ -379,53 +241,16 @@ public void testProjectBackWard() throws Exception { HdfsSinkConnectorConfig connectorConfig = new HdfsSinkConnectorConfig(props); DataWriter hdfsWriter = new DataWriter(connectorConfig, context, avroData); + partitioner = hdfsWriter.getPartitioner(); hdfsWriter.recover(TOPIC_PARTITION); - String key = "key"; - Schema schema = createSchema(); - Struct record = createRecord(schema); - - Schema newSchema = createNewSchema(); - Struct newRecord = createNewRecord(newSchema); - - Collection sinkRecords = new ArrayList<>(); - sinkRecords.add(new SinkRecord(TOPIC, PARTITION, Schema.STRING_SCHEMA, key, newSchema, newRecord, 0L)); - sinkRecords.add(new SinkRecord(TOPIC, PARTITION, Schema.STRING_SCHEMA, key, schema, record, 1L)); + List sinkRecords = createRecordsWithAlteringSchemas(7, 0); hdfsWriter.write(sinkRecords); hdfsWriter.close(assignment); hdfsWriter.stop(); - - String DIRECTORY = TOPIC + "/" + "partition=" + String.valueOf(PARTITION); - Path path = new Path(FileUtils.committedFileName(url, topicsDir, DIRECTORY, TOPIC_PARTITION, - 0L, 1L, extension, ZERO_PAD_FMT)); - ArrayList records = (ArrayList) schemaFileReader.readData(conf, path); - assertEquals(2, records.size()); - - assertEquals(avroData.fromConnectData(newSchema, newRecord), records.get(0)); - - Object projected = SchemaProjector.project(schema, record, newSchema); - assertEquals(avroData.fromConnectData(newSchema, projected), records.get(1)); - - hdfsWriter = new DataWriter(connectorConfig, context, avroData); - hdfsWriter.recover(TOPIC_PARTITION); - - sinkRecords.clear(); - sinkRecords.add(new SinkRecord(TOPIC, PARTITION, Schema.STRING_SCHEMA, key, schema, record, 2L)); - sinkRecords.add(new SinkRecord(TOPIC, PARTITION, Schema.STRING_SCHEMA, key, newSchema, - newRecord, 3L)); - - hdfsWriter.write(sinkRecords); - hdfsWriter.close(assignment); - hdfsWriter.stop(); - - path = new Path(FileUtils.committedFileName(url, topicsDir, DIRECTORY, TOPIC_PARTITION, 2L, - 3L, extension, ZERO_PAD_FMT)); - records = (ArrayList) schemaFileReader.readData(conf, path); - assertEquals(2, records.size()); - - assertEquals(avroData.fromConnectData(newSchema, projected), records.get(0)); - assertEquals(avroData.fromConnectData(newSchema, newRecord), records.get(1)); + long[] validOffsets = {0, 1, 3, 5, 7}; + verify(sinkRecords, validOffsets); } @Test @@ -435,58 +260,17 @@ public void testProjectNone() throws Exception { HdfsSinkConnectorConfig connectorConfig = new HdfsSinkConnectorConfig(props); DataWriter hdfsWriter = new DataWriter(connectorConfig, context, avroData); + partitioner = hdfsWriter.getPartitioner(); hdfsWriter.recover(TOPIC_PARTITION); - String key = "key"; - Schema schema = createSchema(); - Struct record = createRecord(schema); - - Schema newSchema = createNewSchema(); - Struct newRecord = createNewRecord(newSchema); - - Collection sinkRecords = new ArrayList<>(); - sinkRecords.add(new SinkRecord(TOPIC, PARTITION, Schema.STRING_SCHEMA, key, newSchema, newRecord, 0L)); - sinkRecords.add(new SinkRecord(TOPIC, PARTITION, Schema.STRING_SCHEMA, key, schema, record, 1L)); - // Include one more to get to forced file rotation - sinkRecords.add(new SinkRecord(TOPIC, PARTITION, Schema.STRING_SCHEMA, key, schema, record, 2L)); - - hdfsWriter.write(sinkRecords); - hdfsWriter.close(assignment); - hdfsWriter.stop(); - - String DIRECTORY = TOPIC + "/" + "partition=" + String.valueOf(PARTITION); - Path path = new Path(FileUtils.committedFileName(url, topicsDir, DIRECTORY, TOPIC_PARTITION, - 0L, 0L, extension, ZERO_PAD_FMT)); - ArrayList records = (ArrayList) schemaFileReader.readData(conf, path); - assertEquals(1, records.size()); - - - assertEquals(avroData.fromConnectData(newSchema, newRecord), records.get(0)); - - path = new Path(FileUtils.committedFileName(url, topicsDir, DIRECTORY, TOPIC_PARTITION, 1L, - 2L, extension, ZERO_PAD_FMT)); - records = (ArrayList) schemaFileReader.readData(conf, path); - assertEquals(avroData.fromConnectData(schema, record), records.get(0)); - assertEquals(avroData.fromConnectData(schema, record), records.get(1)); - - hdfsWriter = new DataWriter(connectorConfig, context, avroData); - hdfsWriter.recover(TOPIC_PARTITION); - - sinkRecords.clear(); - sinkRecords.add(new SinkRecord(TOPIC, PARTITION, Schema.STRING_SCHEMA, key, newSchema, newRecord, 3L)); - sinkRecords.add(new SinkRecord(TOPIC, PARTITION, Schema.STRING_SCHEMA, key, newSchema, newRecord, 4L)); + List sinkRecords = createRecordsWithAlteringSchemas(7, 0); hdfsWriter.write(sinkRecords); hdfsWriter.close(assignment); hdfsWriter.stop(); - path = new Path(FileUtils.committedFileName(url, topicsDir, DIRECTORY, TOPIC_PARTITION, 3L, - 4L, extension, ZERO_PAD_FMT)); - records = (ArrayList) schemaFileReader.readData(conf, path); - assertEquals(2, records.size()); - - assertEquals(avroData.fromConnectData(newSchema, newRecord), records.get(0)); - assertEquals(avroData.fromConnectData(newSchema, newRecord), records.get(1)); + long[] validOffsets = {0, 1, 2, 3, 4, 5, 6}; + verify(sinkRecords, validOffsets); } @Test @@ -497,99 +281,44 @@ public void testProjectForward() throws Exception { HdfsSinkConnectorConfig connectorConfig = new HdfsSinkConnectorConfig(props); DataWriter hdfsWriter = new DataWriter(connectorConfig, context, avroData); + partitioner = hdfsWriter.getPartitioner(); hdfsWriter.recover(TOPIC_PARTITION); - String key = "key"; - Schema schema = createSchema(); - Struct record = createRecord(schema); - - Schema newSchema = createNewSchema(); - Struct newRecord = createNewRecord(newSchema); - - Collection sinkRecords = new ArrayList<>(); - sinkRecords.add(new SinkRecord(TOPIC, PARTITION, Schema.STRING_SCHEMA, key, newSchema, newRecord, 0L)); - sinkRecords.add(new SinkRecord(TOPIC, PARTITION, Schema.STRING_SCHEMA, key, schema, record, 1L)); - // Include one more to get to forced file rotation - sinkRecords.add(new SinkRecord(TOPIC, PARTITION, Schema.STRING_SCHEMA, key, schema, record, 2L)); - - hdfsWriter.write(sinkRecords); - hdfsWriter.close(assignment); - hdfsWriter.stop(); - - String DIRECTORY = TOPIC + "/" + "partition=" + String.valueOf(PARTITION); - Path path = new Path(FileUtils.committedFileName(url, topicsDir, DIRECTORY, TOPIC_PARTITION, - 0L, 0L, extension, ZERO_PAD_FMT)); - ArrayList records = (ArrayList) schemaFileReader.readData(conf, path); - assertEquals(1, records.size()); - - assertEquals(avroData.fromConnectData(newSchema, newRecord), records.get(0)); - - path = new Path(FileUtils.committedFileName(url, topicsDir, DIRECTORY, TOPIC_PARTITION, 1L, - 2L, extension, ZERO_PAD_FMT)); - records = (ArrayList) schemaFileReader.readData(conf, path); - assertEquals(avroData.fromConnectData(schema, record), records.get(0)); - assertEquals(avroData.fromConnectData(schema, record), records.get(1)); - - hdfsWriter = new DataWriter(connectorConfig, context, avroData); - hdfsWriter.recover(TOPIC_PARTITION); - - sinkRecords.clear(); - sinkRecords.add(new SinkRecord(TOPIC, PARTITION, Schema.STRING_SCHEMA, key, newSchema, newRecord, 3L)); - sinkRecords.add(new SinkRecord(TOPIC, PARTITION, Schema.STRING_SCHEMA, key, newSchema, newRecord, 4L)); + // By excluding the first element we get a list starting with record having the new schema. + List sinkRecords = createRecordsWithAlteringSchemas(8, 0).subList(1, 8); hdfsWriter.write(sinkRecords); hdfsWriter.close(assignment); hdfsWriter.stop(); - path = new Path(FileUtils.committedFileName(url, topicsDir, DIRECTORY, TOPIC_PARTITION, 3L, - 4L, extension, ZERO_PAD_FMT)); - records = (ArrayList) schemaFileReader.readData(conf, path); - assertEquals(2, records.size()); - - assertEquals(avroData.fromConnectData(schema, record), records.get(0)); - assertEquals(avroData.fromConnectData(schema, record), records.get(1)); + long[] validOffsets = {1, 2, 4, 6, 8}; + verify(sinkRecords, validOffsets); } @Test public void testProjectNoVersion() throws Exception { - Schema schemaNoVersion = SchemaBuilder.struct().name("record") - .field("boolean", Schema.BOOLEAN_SCHEMA) - .field("int", Schema.INT32_SCHEMA) - .field("long", Schema.INT64_SCHEMA) - .field("float", Schema.FLOAT32_SCHEMA) - .field("double", Schema.FLOAT64_SCHEMA) - .build(); - - Struct recordNoVersion = new Struct(schemaNoVersion); - recordNoVersion.put("boolean", true) - .put("int", 12) - .put("long", 12L) - .put("float", 12.2f) - .put("double", 12.2); - Map props = createProps(); props.put(HdfsSinkConnectorConfig.SCHEMA_COMPATIBILITY_CONFIG, "BACKWARD"); HdfsSinkConnectorConfig connectorConfig = new HdfsSinkConnectorConfig(props); DataWriter hdfsWriter = new DataWriter(connectorConfig, context, avroData); + partitioner = hdfsWriter.getPartitioner(); hdfsWriter.recover(TOPIC_PARTITION); - String key = "key"; - Schema schema = createSchema(); - Struct record = createRecord(schema); - - Collection sinkRecords = new ArrayList<>(); - sinkRecords.add(new SinkRecord(TOPIC, PARTITION, Schema.STRING_SCHEMA, key, schemaNoVersion, recordNoVersion, 0L)); - sinkRecords.add(new SinkRecord(TOPIC, PARTITION, Schema.STRING_SCHEMA, key, schema, record, 1L)); + List sinkRecords = createRecordsNoVersion(1, 0); + sinkRecords.addAll(createRecordsWithAlteringSchemas(7, 0)); try { hdfsWriter.write(sinkRecords); fail("Version is required for Backward compatibility."); } catch (RuntimeException e) { // expected + } finally { + hdfsWriter.close(assignment); + hdfsWriter.stop(); + long[] validOffsets = {}; + verify(Collections.emptyList(), validOffsets); } - hdfsWriter.close(assignment); - hdfsWriter.stop(); } @Test @@ -600,7 +329,7 @@ public void testFlushPartialFile() throws Exception { String FLUSH_SIZE_CONFIG = "10"; // send 1.5 * FLUSH_SIZE_CONFIG records - long NUMBER_OF_RECORD = Long.valueOf(FLUSH_SIZE_CONFIG) + Long.valueOf(FLUSH_SIZE_CONFIG) / 2; + int NUMBER_OF_RECORDS = Integer.valueOf(FLUSH_SIZE_CONFIG) + Integer.valueOf(FLUSH_SIZE_CONFIG) / 2; Map props = createProps(); props.put(HdfsSinkConnectorConfig.FLUSH_SIZE_CONFIG, FLUSH_SIZE_CONFIG); @@ -610,17 +339,10 @@ public void testFlushPartialFile() throws Exception { assignment.add(TOPIC_PARTITION); DataWriter hdfsWriter = new DataWriter(connectorConfig, context, avroData); + partitioner = hdfsWriter.getPartitioner(); hdfsWriter.recover(TOPIC_PARTITION); - String key = "key"; - Schema schema = createSchema(); - Struct record = createRecord(schema); - - Collection sinkRecords = new ArrayList<>(); - for (long offset = 0; offset < NUMBER_OF_RECORD; offset++) { - SinkRecord sinkRecord = new SinkRecord(TOPIC, PARTITION, Schema.STRING_SCHEMA, key, schema, record, offset); - sinkRecords.add(sinkRecord); - } + List sinkRecords = createRecords(NUMBER_OF_RECORDS); hdfsWriter.write(sinkRecords); // wait for rotation to happen @@ -634,11 +356,224 @@ public void testFlushPartialFile() throws Exception { Map committedOffsets = hdfsWriter.getCommittedOffsets(); assertTrue(committedOffsets.containsKey(TOPIC_PARTITION)); long previousOffset = committedOffsets.get(TOPIC_PARTITION); - assertEquals(NUMBER_OF_RECORD, previousOffset); + assertEquals(NUMBER_OF_RECORDS, previousOffset); hdfsWriter.close(assignment); hdfsWriter.stop(); } + /** + * Return a list of new records starting at zero offset. + * + * @param size the number of records to return. + * @return the list of records. + */ + protected List createRecords(int size) { + return createRecords(size, 0); + } + + /** + * Return a list of new records starting at the given offset. + * + * @param size the number of records to return. + * @param startOffset the starting offset. + * @return the list of records. + */ + protected List createRecords(int size, long startOffset) { + return createRecords(size, startOffset, Collections.singleton(new TopicPartition(TOPIC, PARTITION))); + } + + /** + * Return a list of new records for a set of partitions, starting at the given offset in each partition. + * + * @param size the number of records to return. + * @param startOffset the starting offset. + * @param partitions the set of partitions to create records for. + * @return the list of records. + */ + protected List createRecords(int size, long startOffset, Set partitions) { + String key = "key"; + Schema schema = createSchema(); + Struct record = createRecord(schema); + + List sinkRecords = new ArrayList<>(); + for (TopicPartition tp : partitions) { + for (long offset = startOffset; offset < startOffset + size; ++offset) { + sinkRecords.add(new SinkRecord(TOPIC, tp.partition(), Schema.STRING_SCHEMA, key, schema, record, offset)); + } + } + return sinkRecords; + } + + protected List createRecordsNoVersion(int size, long startOffset) { + String key = "key"; + Schema schemaNoVersion = SchemaBuilder.struct().name("record") + .field("boolean", Schema.BOOLEAN_SCHEMA) + .field("int", Schema.INT32_SCHEMA) + .field("long", Schema.INT64_SCHEMA) + .field("float", Schema.FLOAT32_SCHEMA) + .field("double", Schema.FLOAT64_SCHEMA) + .build(); + + Struct recordNoVersion = new Struct(schemaNoVersion); + recordNoVersion.put("boolean", true) + .put("int", 12) + .put("long", 12L) + .put("float", 12.2f) + .put("double", 12.2); + + List sinkRecords = new ArrayList<>(); + for (long offset = startOffset; offset < startOffset + size; ++offset) { + sinkRecords.add(new SinkRecord(TOPIC, PARTITION, Schema.STRING_SCHEMA, key, schemaNoVersion, + recordNoVersion, offset)); + } + return sinkRecords; + } + + protected List createRecordsWithAlteringSchemas(int size, long startOffset) { + String key = "key"; + Schema schema = createSchema(); + Struct record = createRecord(schema); + Schema newSchema = createNewSchema(); + Struct newRecord = createNewRecord(newSchema); + + int limit = (size / 2) * 2; + boolean remainder = size % 2 > 0; + List sinkRecords = new ArrayList<>(); + for (long offset = startOffset; offset < startOffset + limit; ++offset) { + sinkRecords.add(new SinkRecord(TOPIC, PARTITION, Schema.STRING_SCHEMA, key, schema, record, offset)); + sinkRecords.add(new SinkRecord(TOPIC, PARTITION, Schema.STRING_SCHEMA, key, newSchema, newRecord, ++offset)); + } + if (remainder) { + sinkRecords.add(new SinkRecord(TOPIC, PARTITION, Schema.STRING_SCHEMA, key, schema, record, + startOffset + size - 1)); + } + return sinkRecords; + } + + protected List createRecordsInterleaved(int size, long startOffset, Set partitions) { + String key = "key"; + Schema schema = createSchema(); + Struct record = createRecord(schema); + + List sinkRecords = new ArrayList<>(); + for (long offset = startOffset, total = 0; total < size; ++offset) { + for (TopicPartition tp : partitions) { + sinkRecords.add(new SinkRecord(TOPIC, tp.partition(), Schema.STRING_SCHEMA, key, schema, record, offset)); + if (++total >= size) { + break; + } + } + } + return sinkRecords; + } + + protected String getDirectory() { + return getDirectory(TOPIC, PARTITION); + } + + protected String getDirectory(String topic, int partition) { + String encodedPartition = "partition=" + String.valueOf(partition); + return partitioner.generatePartitionedPath(topic, encodedPartition); + } + + /** + * Verify files and records are uploaded appropriately. + * @param sinkRecords a flat list of the records that need to appear in potentially several files in S3. + * @param validOffsets an array containing the offsets that map to uploaded files for a topic-partition. + * Offsets appear in ascending order, the difference between two consecutive offsets + * equals the expected size of the file, and last offset is exclusive. + */ + protected void verify(List sinkRecords, long[] validOffsets) throws IOException { + verify(sinkRecords, validOffsets, Collections.singleton(new TopicPartition(TOPIC, PARTITION)), false); + } + + protected void verify(List sinkRecords, long[] validOffsets, Set partitions) + throws IOException { + verify(sinkRecords, validOffsets, partitions, false); + } + + /** + * Verify files and records are uploaded appropriately. + * @param sinkRecords a flat list of the records that need to appear in potentially several files in S3. + * @param validOffsets an array containing the offsets that map to uploaded files for a topic-partition. + * Offsets appear in ascending order, the difference between two consecutive offsets + * equals the expected size of the file, and last offset is exclusive. + * @param partitions the set of partitions to verify records for. + */ + protected void verify(List sinkRecords, long[] validOffsets, Set partitions, + boolean skipFileListing) throws IOException { + if (!skipFileListing) { + verifyFileListing(validOffsets, partitions); + } + + for (TopicPartition tp : partitions) { + for (int i = 1, j = 0; i < validOffsets.length; ++i) { + long startOffset = validOffsets[i - 1]; + long endOffset = validOffsets[i] - 1; + + String filename = FileUtils.committedFileName(url, topicsDir, getDirectory(tp.topic(), tp.partition()), tp, + startOffset, endOffset, extension, ZERO_PAD_FMT); + Path path = new Path(filename); + Collection records = schemaFileReader.readData(conf, path); + + long size = endOffset - startOffset + 1; + assertEquals(size, records.size()); + verifyContents(sinkRecords, j, records); + j += size; + } + } + } + + protected List getExpectedFiles(long[] validOffsets, TopicPartition tp) { + List expectedFiles = new ArrayList<>(); + for (int i = 1; i < validOffsets.length; ++i) { + long startOffset = validOffsets[i - 1]; + long endOffset = validOffsets[i] - 1; + expectedFiles.add(FileUtils.committedFileName(url, topicsDir, getDirectory(tp.topic(), tp.partition()), tp, + startOffset, endOffset, extension, ZERO_PAD_FMT)); + } + return expectedFiles; + } + + protected void verifyFileListing(long[] validOffsets, Set partitions) throws IOException { + for (TopicPartition tp : partitions) { + verifyFileListing(getExpectedFiles(validOffsets, tp), tp); + } + } + + protected void verifyFileListing(List expectedFiles, TopicPartition tp) throws IOException { + FileStatus[] statuses = {}; + try { + statuses = fs.listStatus( + new Path(FileUtils.directoryName(url, topicsDir, getDirectory(tp.topic(), tp.partition()))), + new TopicPartitionCommittedFileFilter(tp)); + } catch (FileNotFoundException e) { + // the directory does not exist. + } + + List actualFiles = new ArrayList<>(); + for (FileStatus status : statuses) { + actualFiles.add(status.getPath().toString()); + } + + Collections.sort(actualFiles); + Collections.sort(expectedFiles); + assertThat(actualFiles, is(expectedFiles)); + } + + protected void verifyContents(List expectedRecords, int startIndex, Collection records) { + Schema expectedSchema = null; + for (Object avroRecord : records) { + if (expectedSchema == null) { + expectedSchema = expectedRecords.get(startIndex).valueSchema(); + } + Object expectedValue = SchemaProjector.project(expectedRecords.get(startIndex).valueSchema(), + expectedRecords.get(startIndex++).value(), + expectedSchema); + assertEquals(avroData.fromConnectData(expectedSchema, expectedValue), avroRecord); + } + } + } From e0a014c60c06dc40b1bf00c832983d9f0d504115 Mon Sep 17 00:00:00 2001 From: Konstantine Karantasis Date: Thu, 20 Apr 2017 15:46:44 -0700 Subject: [PATCH 02/58] Move verification methods and members to TestWithMiniDFSCluster --- .../connect/hdfs/TestWithMiniDFSCluster.java | 241 ++++++++++++++++ .../connect/hdfs/avro/DataWriterAvroTest.java | 257 ++---------------- .../hdfs/avro/TopicPartitionWriterTest.java | 17 +- .../hdfs/parquet/DataWriterParquetTest.java | 53 +--- 4 files changed, 277 insertions(+), 291 deletions(-) diff --git a/src/test/java/io/confluent/connect/hdfs/TestWithMiniDFSCluster.java b/src/test/java/io/confluent/connect/hdfs/TestWithMiniDFSCluster.java index 6e29a4f1e..efc92ea09 100644 --- a/src/test/java/io/confluent/connect/hdfs/TestWithMiniDFSCluster.java +++ b/src/test/java/io/confluent/connect/hdfs/TestWithMiniDFSCluster.java @@ -17,18 +17,44 @@ package io.confluent.connect.hdfs; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.SchemaBuilder; +import org.apache.kafka.connect.data.SchemaProjector; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.sink.SinkRecord; import org.junit.After; import org.junit.Before; +import java.io.FileNotFoundException; import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; import java.util.Map; +import java.util.Set; + +import io.confluent.connect.hdfs.filter.TopicPartitionCommittedFileFilter; +import io.confluent.connect.hdfs.partitioner.Partitioner; + +import static org.hamcrest.CoreMatchers.is; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThat; public class TestWithMiniDFSCluster extends HdfsSinkConnectorTestBase { protected MiniDFSCluster cluster; protected FileSystem fs; + protected SchemaFileReader schemaFileReader; + protected Partitioner partitioner; + protected String extension; + // The default based on default configuration of 10 + protected String zeroPadFormat = "%010d"; @Before public void setUp() throws Exception { @@ -69,4 +95,219 @@ protected Map createProps() { props.put(HdfsSinkConnectorConfig.HDFS_URL_CONFIG, url); return props; } + + + /** + * Return a list of new records starting at zero offset. + * + * @param size the number of records to return. + * @return the list of records. + */ + protected List createRecords(int size) { + return createRecords(size, 0); + } + + /** + * Return a list of new records starting at the given offset. + * + * @param size the number of records to return. + * @param startOffset the starting offset. + * @return the list of records. + */ + protected List createRecords(int size, long startOffset) { + return createRecords(size, startOffset, Collections.singleton(new TopicPartition(TOPIC, PARTITION))); + } + + /** + * Return a list of new records for a set of partitions, starting at the given offset in each partition. + * + * @param size the number of records to return. + * @param startOffset the starting offset. + * @param partitions the set of partitions to create records for. + * @return the list of records. + */ + protected List createRecords(int size, long startOffset, Set partitions) { + String key = "key"; + Schema schema = createSchema(); + Struct record = createRecord(schema); + + List sinkRecords = new ArrayList<>(); + for (TopicPartition tp : partitions) { + for (long offset = startOffset; offset < startOffset + size; ++offset) { + sinkRecords.add(new SinkRecord(TOPIC, tp.partition(), Schema.STRING_SCHEMA, key, schema, record, offset)); + } + } + return sinkRecords; + } + + protected List createRecordsNoVersion(int size, long startOffset) { + String key = "key"; + Schema schemaNoVersion = SchemaBuilder.struct().name("record") + .field("boolean", Schema.BOOLEAN_SCHEMA) + .field("int", Schema.INT32_SCHEMA) + .field("long", Schema.INT64_SCHEMA) + .field("float", Schema.FLOAT32_SCHEMA) + .field("double", Schema.FLOAT64_SCHEMA) + .build(); + + Struct recordNoVersion = new Struct(schemaNoVersion); + recordNoVersion.put("boolean", true) + .put("int", 12) + .put("long", 12L) + .put("float", 12.2f) + .put("double", 12.2); + + List sinkRecords = new ArrayList<>(); + for (long offset = startOffset; offset < startOffset + size; ++offset) { + sinkRecords.add(new SinkRecord(TOPIC, PARTITION, Schema.STRING_SCHEMA, key, schemaNoVersion, + recordNoVersion, offset)); + } + return sinkRecords; + } + + protected List createRecordsWithAlteringSchemas(int size, long startOffset) { + String key = "key"; + Schema schema = createSchema(); + Struct record = createRecord(schema); + Schema newSchema = createNewSchema(); + Struct newRecord = createNewRecord(newSchema); + + int limit = (size / 2) * 2; + boolean remainder = size % 2 > 0; + List sinkRecords = new ArrayList<>(); + for (long offset = startOffset; offset < startOffset + limit; ++offset) { + sinkRecords.add(new SinkRecord(TOPIC, PARTITION, Schema.STRING_SCHEMA, key, schema, record, offset)); + sinkRecords.add(new SinkRecord(TOPIC, PARTITION, Schema.STRING_SCHEMA, key, newSchema, newRecord, ++offset)); + } + if (remainder) { + sinkRecords.add(new SinkRecord(TOPIC, PARTITION, Schema.STRING_SCHEMA, key, schema, record, + startOffset + size - 1)); + } + return sinkRecords; + } + + protected List createRecordsInterleaved(int size, long startOffset, Set partitions) { + String key = "key"; + Schema schema = createSchema(); + Struct record = createRecord(schema); + + List sinkRecords = new ArrayList<>(); + for (long offset = startOffset, total = 0; total < size; ++offset) { + for (TopicPartition tp : partitions) { + sinkRecords.add(new SinkRecord(TOPIC, tp.partition(), Schema.STRING_SCHEMA, key, schema, record, offset)); + if (++total >= size) { + break; + } + } + } + return sinkRecords; + } + + protected String getDirectory() { + return getDirectory(TOPIC, PARTITION); + } + + protected String getDirectory(String topic, int partition) { + String encodedPartition = "partition=" + String.valueOf(partition); + return partitioner.generatePartitionedPath(topic, encodedPartition); + } + + /** + * Verify files and records are uploaded appropriately. + * @param sinkRecords a flat list of the records that need to appear in potentially several files in S3. + * @param validOffsets an array containing the offsets that map to uploaded files for a topic-partition. + * Offsets appear in ascending order, the difference between two consecutive offsets + * equals the expected size of the file, and last offset is exclusive. + */ + protected void verify(List sinkRecords, long[] validOffsets) throws IOException { + verify(sinkRecords, validOffsets, Collections.singleton(new TopicPartition(TOPIC, PARTITION)), false); + } + + protected void verify(List sinkRecords, long[] validOffsets, Set partitions) + throws IOException { + verify(sinkRecords, validOffsets, partitions, false); + } + + /** + * Verify files and records are uploaded appropriately. + * @param sinkRecords a flat list of the records that need to appear in potentially several files in S3. + * @param validOffsets an array containing the offsets that map to uploaded files for a topic-partition. + * Offsets appear in ascending order, the difference between two consecutive offsets + * equals the expected size of the file, and last offset is exclusive. + * @param partitions the set of partitions to verify records for. + */ + protected void verify(List sinkRecords, long[] validOffsets, Set partitions, + boolean skipFileListing) throws IOException { + if (!skipFileListing) { + verifyFileListing(validOffsets, partitions); + } + + for (TopicPartition tp : partitions) { + for (int i = 1, j = 0; i < validOffsets.length; ++i) { + long startOffset = validOffsets[i - 1]; + long endOffset = validOffsets[i] - 1; + + String filename = FileUtils.committedFileName(url, topicsDir, getDirectory(tp.topic(), tp.partition()), tp, + startOffset, endOffset, extension, zeroPadFormat); + Path path = new Path(filename); + Collection records = schemaFileReader.readData(conf, path); + + long size = endOffset - startOffset + 1; + assertEquals(size, records.size()); + verifyContents(sinkRecords, j, records); + j += size; + } + } + } + + protected List getExpectedFiles(long[] validOffsets, TopicPartition tp) { + List expectedFiles = new ArrayList<>(); + for (int i = 1; i < validOffsets.length; ++i) { + long startOffset = validOffsets[i - 1]; + long endOffset = validOffsets[i] - 1; + expectedFiles.add(FileUtils.committedFileName(url, topicsDir, getDirectory(tp.topic(), tp.partition()), tp, + startOffset, endOffset, extension, zeroPadFormat)); + } + return expectedFiles; + } + + protected void verifyFileListing(long[] validOffsets, Set partitions) throws IOException { + for (TopicPartition tp : partitions) { + verifyFileListing(getExpectedFiles(validOffsets, tp), tp); + } + } + + protected void verifyFileListing(List expectedFiles, TopicPartition tp) throws IOException { + FileStatus[] statuses = {}; + try { + statuses = fs.listStatus( + new Path(FileUtils.directoryName(url, topicsDir, getDirectory(tp.topic(), tp.partition()))), + new TopicPartitionCommittedFileFilter(tp)); + } catch (FileNotFoundException e) { + // the directory does not exist. + } + + List actualFiles = new ArrayList<>(); + for (FileStatus status : statuses) { + actualFiles.add(status.getPath().toString()); + } + + Collections.sort(actualFiles); + Collections.sort(expectedFiles); + assertThat(actualFiles, is(expectedFiles)); + } + + protected void verifyContents(List expectedRecords, int startIndex, Collection records) { + Schema expectedSchema = null; + for (Object avroRecord : records) { + if (expectedSchema == null) { + expectedSchema = expectedRecords.get(startIndex).valueSchema(); + } + Object expectedValue = SchemaProjector.project(expectedRecords.get(startIndex).valueSchema(), + expectedRecords.get(startIndex++).value(), + expectedSchema); + assertEquals(avroData.fromConnectData(expectedSchema, expectedValue), avroRecord); + } + } + } diff --git a/src/test/java/io/confluent/connect/hdfs/avro/DataWriterAvroTest.java b/src/test/java/io/confluent/connect/hdfs/avro/DataWriterAvroTest.java index 1e3d26b53..48e73a2f7 100644 --- a/src/test/java/io/confluent/connect/hdfs/avro/DataWriterAvroTest.java +++ b/src/test/java/io/confluent/connect/hdfs/avro/DataWriterAvroTest.java @@ -14,50 +14,40 @@ package io.confluent.connect.hdfs.avro; -import io.confluent.connect.hdfs.DataWriter; -import io.confluent.connect.hdfs.FileUtils; -import io.confluent.connect.hdfs.HdfsSinkConnectorConfig; -import io.confluent.connect.hdfs.SchemaFileReader; -import io.confluent.connect.hdfs.TestWithMiniDFSCluster; -import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.connect.data.Schema; -import org.apache.kafka.connect.data.SchemaBuilder; -import org.apache.kafka.connect.data.SchemaProjector; -import org.apache.kafka.connect.data.Struct; import org.apache.kafka.connect.sink.SinkRecord; +import org.junit.Before; import org.junit.Test; -import io.confluent.connect.hdfs.filter.TopicPartitionCommittedFileFilter; -import io.confluent.connect.hdfs.partitioner.Partitioner; -import io.confluent.connect.hdfs.storage.Storage; -import io.confluent.connect.hdfs.storage.StorageFactory; -import io.confluent.connect.hdfs.wal.WAL; - -import java.io.FileNotFoundException; -import java.io.IOException; import java.util.ArrayList; -import java.util.Collection; import java.util.Collections; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; -import static org.hamcrest.CoreMatchers.is; +import io.confluent.connect.hdfs.DataWriter; +import io.confluent.connect.hdfs.FileUtils; +import io.confluent.connect.hdfs.HdfsSinkConnectorConfig; +import io.confluent.connect.hdfs.TestWithMiniDFSCluster; +import io.confluent.connect.hdfs.storage.Storage; +import io.confluent.connect.hdfs.storage.StorageFactory; +import io.confluent.connect.hdfs.wal.WAL; + import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; public class DataWriterAvroTest extends TestWithMiniDFSCluster { - private static final String extension = ".avro"; - private static final String ZERO_PAD_FMT = "%010d"; - private SchemaFileReader schemaFileReader = new AvroFileReader(avroData); - Partitioner partitioner; + @Before + public void setUp() throws Exception { + super.setUp(); + schemaFileReader = new AvroFileReader(avroData); + extension = ".avro"; + } @Test public void testWriteRecord() throws Exception { @@ -97,7 +87,7 @@ public void testRecovery() throws Exception { String tempfile = FileUtils.tempFileName(url, topicsDir, getDirectory(), extension); fs.createNewFile(new Path(tempfile)); String committedFile = FileUtils.committedFileName(url, topicsDir, getDirectory(), TOPIC_PARTITION, startOffset, - endOffset, extension, ZERO_PAD_FMT); + endOffset, extension, zeroPadFormat); wal.append(tempfile, committedFile); } wal.append(WAL.endMarker, ""); @@ -146,7 +136,7 @@ public void testGetPreviousOffsets() throws Exception { for (int i = 0; i < startOffsets.length; ++i) { Path path = new Path(FileUtils.committedFileName(url, topicsDir, directory, TOPIC_PARTITION, startOffsets[i], - endOffsets[i], extension, ZERO_PAD_FMT)); + endOffsets[i], extension, zeroPadFormat)); fs.createNewFile(path); } Path path = new Path(FileUtils.tempFileName(url, topicsDir, directory, extension)); @@ -362,218 +352,5 @@ public void testFlushPartialFile() throws Exception { hdfsWriter.stop(); } - /** - * Return a list of new records starting at zero offset. - * - * @param size the number of records to return. - * @return the list of records. - */ - protected List createRecords(int size) { - return createRecords(size, 0); - } - - /** - * Return a list of new records starting at the given offset. - * - * @param size the number of records to return. - * @param startOffset the starting offset. - * @return the list of records. - */ - protected List createRecords(int size, long startOffset) { - return createRecords(size, startOffset, Collections.singleton(new TopicPartition(TOPIC, PARTITION))); - } - - /** - * Return a list of new records for a set of partitions, starting at the given offset in each partition. - * - * @param size the number of records to return. - * @param startOffset the starting offset. - * @param partitions the set of partitions to create records for. - * @return the list of records. - */ - protected List createRecords(int size, long startOffset, Set partitions) { - String key = "key"; - Schema schema = createSchema(); - Struct record = createRecord(schema); - - List sinkRecords = new ArrayList<>(); - for (TopicPartition tp : partitions) { - for (long offset = startOffset; offset < startOffset + size; ++offset) { - sinkRecords.add(new SinkRecord(TOPIC, tp.partition(), Schema.STRING_SCHEMA, key, schema, record, offset)); - } - } - return sinkRecords; - } - - protected List createRecordsNoVersion(int size, long startOffset) { - String key = "key"; - Schema schemaNoVersion = SchemaBuilder.struct().name("record") - .field("boolean", Schema.BOOLEAN_SCHEMA) - .field("int", Schema.INT32_SCHEMA) - .field("long", Schema.INT64_SCHEMA) - .field("float", Schema.FLOAT32_SCHEMA) - .field("double", Schema.FLOAT64_SCHEMA) - .build(); - - Struct recordNoVersion = new Struct(schemaNoVersion); - recordNoVersion.put("boolean", true) - .put("int", 12) - .put("long", 12L) - .put("float", 12.2f) - .put("double", 12.2); - - List sinkRecords = new ArrayList<>(); - for (long offset = startOffset; offset < startOffset + size; ++offset) { - sinkRecords.add(new SinkRecord(TOPIC, PARTITION, Schema.STRING_SCHEMA, key, schemaNoVersion, - recordNoVersion, offset)); - } - return sinkRecords; - } - - protected List createRecordsWithAlteringSchemas(int size, long startOffset) { - String key = "key"; - Schema schema = createSchema(); - Struct record = createRecord(schema); - Schema newSchema = createNewSchema(); - Struct newRecord = createNewRecord(newSchema); - - int limit = (size / 2) * 2; - boolean remainder = size % 2 > 0; - List sinkRecords = new ArrayList<>(); - for (long offset = startOffset; offset < startOffset + limit; ++offset) { - sinkRecords.add(new SinkRecord(TOPIC, PARTITION, Schema.STRING_SCHEMA, key, schema, record, offset)); - sinkRecords.add(new SinkRecord(TOPIC, PARTITION, Schema.STRING_SCHEMA, key, newSchema, newRecord, ++offset)); - } - if (remainder) { - sinkRecords.add(new SinkRecord(TOPIC, PARTITION, Schema.STRING_SCHEMA, key, schema, record, - startOffset + size - 1)); - } - return sinkRecords; - } - - protected List createRecordsInterleaved(int size, long startOffset, Set partitions) { - String key = "key"; - Schema schema = createSchema(); - Struct record = createRecord(schema); - - List sinkRecords = new ArrayList<>(); - for (long offset = startOffset, total = 0; total < size; ++offset) { - for (TopicPartition tp : partitions) { - sinkRecords.add(new SinkRecord(TOPIC, tp.partition(), Schema.STRING_SCHEMA, key, schema, record, offset)); - if (++total >= size) { - break; - } - } - } - return sinkRecords; - } - - protected String getDirectory() { - return getDirectory(TOPIC, PARTITION); - } - - protected String getDirectory(String topic, int partition) { - String encodedPartition = "partition=" + String.valueOf(partition); - return partitioner.generatePartitionedPath(topic, encodedPartition); - } - - /** - * Verify files and records are uploaded appropriately. - * @param sinkRecords a flat list of the records that need to appear in potentially several files in S3. - * @param validOffsets an array containing the offsets that map to uploaded files for a topic-partition. - * Offsets appear in ascending order, the difference between two consecutive offsets - * equals the expected size of the file, and last offset is exclusive. - */ - protected void verify(List sinkRecords, long[] validOffsets) throws IOException { - verify(sinkRecords, validOffsets, Collections.singleton(new TopicPartition(TOPIC, PARTITION)), false); - } - - protected void verify(List sinkRecords, long[] validOffsets, Set partitions) - throws IOException { - verify(sinkRecords, validOffsets, partitions, false); - } - - /** - * Verify files and records are uploaded appropriately. - * @param sinkRecords a flat list of the records that need to appear in potentially several files in S3. - * @param validOffsets an array containing the offsets that map to uploaded files for a topic-partition. - * Offsets appear in ascending order, the difference between two consecutive offsets - * equals the expected size of the file, and last offset is exclusive. - * @param partitions the set of partitions to verify records for. - */ - protected void verify(List sinkRecords, long[] validOffsets, Set partitions, - boolean skipFileListing) throws IOException { - if (!skipFileListing) { - verifyFileListing(validOffsets, partitions); - } - - for (TopicPartition tp : partitions) { - for (int i = 1, j = 0; i < validOffsets.length; ++i) { - long startOffset = validOffsets[i - 1]; - long endOffset = validOffsets[i] - 1; - - String filename = FileUtils.committedFileName(url, topicsDir, getDirectory(tp.topic(), tp.partition()), tp, - startOffset, endOffset, extension, ZERO_PAD_FMT); - Path path = new Path(filename); - Collection records = schemaFileReader.readData(conf, path); - - long size = endOffset - startOffset + 1; - assertEquals(size, records.size()); - verifyContents(sinkRecords, j, records); - j += size; - } - } - } - - protected List getExpectedFiles(long[] validOffsets, TopicPartition tp) { - List expectedFiles = new ArrayList<>(); - for (int i = 1; i < validOffsets.length; ++i) { - long startOffset = validOffsets[i - 1]; - long endOffset = validOffsets[i] - 1; - expectedFiles.add(FileUtils.committedFileName(url, topicsDir, getDirectory(tp.topic(), tp.partition()), tp, - startOffset, endOffset, extension, ZERO_PAD_FMT)); - } - return expectedFiles; - } - - protected void verifyFileListing(long[] validOffsets, Set partitions) throws IOException { - for (TopicPartition tp : partitions) { - verifyFileListing(getExpectedFiles(validOffsets, tp), tp); - } - } - - protected void verifyFileListing(List expectedFiles, TopicPartition tp) throws IOException { - FileStatus[] statuses = {}; - try { - statuses = fs.listStatus( - new Path(FileUtils.directoryName(url, topicsDir, getDirectory(tp.topic(), tp.partition()))), - new TopicPartitionCommittedFileFilter(tp)); - } catch (FileNotFoundException e) { - // the directory does not exist. - } - - List actualFiles = new ArrayList<>(); - for (FileStatus status : statuses) { - actualFiles.add(status.getPath().toString()); - } - - Collections.sort(actualFiles); - Collections.sort(expectedFiles); - assertThat(actualFiles, is(expectedFiles)); - } - - protected void verifyContents(List expectedRecords, int startIndex, Collection records) { - Schema expectedSchema = null; - for (Object avroRecord : records) { - if (expectedSchema == null) { - expectedSchema = expectedRecords.get(startIndex).valueSchema(); - } - Object expectedValue = SchemaProjector.project(expectedRecords.get(startIndex).valueSchema(), - expectedRecords.get(startIndex++).value(), - expectedSchema); - assertEquals(avroData.fromConnectData(expectedSchema, expectedValue), avroRecord); - } - } - } diff --git a/src/test/java/io/confluent/connect/hdfs/avro/TopicPartitionWriterTest.java b/src/test/java/io/confluent/connect/hdfs/avro/TopicPartitionWriterTest.java index 21b26f167..3b7ba954f 100644 --- a/src/test/java/io/confluent/connect/hdfs/avro/TopicPartitionWriterTest.java +++ b/src/test/java/io/confluent/connect/hdfs/avro/TopicPartitionWriterTest.java @@ -52,13 +52,8 @@ import static org.junit.Assert.assertTrue; public class TopicPartitionWriterTest extends TestWithMiniDFSCluster { - // The default based on default configuration of 10 - private static final String ZERO_PAD_FMT = "%010d"; - private RecordWriterProvider writerProvider; - private SchemaFileReader schemaFileReader; private Storage storage; - private static String extension; @Before public void setUp() throws Exception { @@ -142,9 +137,9 @@ public void testWriteRecordFieldPartitioner() throws Exception { String directory3 = partitioner.generatePartitionedPath(TOPIC, partitionField + "=" + String.valueOf(18)); Set expectedFiles = new HashSet<>(); - expectedFiles.add(new Path(FileUtils.committedFileName(url, topicsDir, directory1, TOPIC_PARTITION, 0, 2, extension, ZERO_PAD_FMT))); - expectedFiles.add(new Path(FileUtils.committedFileName(url, topicsDir, directory2, TOPIC_PARTITION, 3, 5, extension, ZERO_PAD_FMT))); - expectedFiles.add(new Path(FileUtils.committedFileName(url, topicsDir, directory3, TOPIC_PARTITION, 6, 8, extension, ZERO_PAD_FMT))); + expectedFiles.add(new Path(FileUtils.committedFileName(url, topicsDir, directory1, TOPIC_PARTITION, 0, 2, extension, zeroPadFormat))); + expectedFiles.add(new Path(FileUtils.committedFileName(url, topicsDir, directory2, TOPIC_PARTITION, 3, 5, extension, zeroPadFormat))); + expectedFiles.add(new Path(FileUtils.committedFileName(url, topicsDir, directory3, TOPIC_PARTITION, 6, 8, extension, zeroPadFormat))); verify(expectedFiles, records, schema); } @@ -183,9 +178,9 @@ public void testWriteRecordTimeBasedPartition() throws Exception { String directory = partitioner.generatePartitionedPath(TOPIC, encodedPartition); Set expectedFiles = new HashSet<>(); - expectedFiles.add(new Path(FileUtils.committedFileName(url, topicsDir, directory, TOPIC_PARTITION, 0, 2, extension, ZERO_PAD_FMT))); - expectedFiles.add(new Path(FileUtils.committedFileName(url, topicsDir, directory, TOPIC_PARTITION, 3, 5, extension, ZERO_PAD_FMT))); - expectedFiles.add(new Path(FileUtils.committedFileName(url, topicsDir, directory, TOPIC_PARTITION, 6, 8, extension, ZERO_PAD_FMT))); + expectedFiles.add(new Path(FileUtils.committedFileName(url, topicsDir, directory, TOPIC_PARTITION, 0, 2, extension, zeroPadFormat))); + expectedFiles.add(new Path(FileUtils.committedFileName(url, topicsDir, directory, TOPIC_PARTITION, 3, 5, extension, zeroPadFormat))); + expectedFiles.add(new Path(FileUtils.committedFileName(url, topicsDir, directory, TOPIC_PARTITION, 6, 8, extension, zeroPadFormat))); verify(expectedFiles, records, schema); } diff --git a/src/test/java/io/confluent/connect/hdfs/parquet/DataWriterParquetTest.java b/src/test/java/io/confluent/connect/hdfs/parquet/DataWriterParquetTest.java index 0a0c39977..6c71336dc 100644 --- a/src/test/java/io/confluent/connect/hdfs/parquet/DataWriterParquetTest.java +++ b/src/test/java/io/confluent/connect/hdfs/parquet/DataWriterParquetTest.java @@ -15,29 +15,26 @@ package io.confluent.connect.hdfs.parquet; -import org.apache.hadoop.fs.Path; -import org.apache.kafka.connect.data.Schema; -import org.apache.kafka.connect.data.Struct; import org.apache.kafka.connect.sink.SinkRecord; +import org.junit.Before; import org.junit.Test; -import java.util.ArrayList; -import java.util.Collection; +import java.util.List; import java.util.Map; import io.confluent.connect.hdfs.DataWriter; -import io.confluent.connect.hdfs.FileUtils; import io.confluent.connect.hdfs.HdfsSinkConnectorConfig; -import io.confluent.connect.hdfs.SchemaFileReader; import io.confluent.connect.hdfs.TestWithMiniDFSCluster; import io.confluent.connect.hdfs.partitioner.Partitioner; -import static org.junit.Assert.assertEquals; - public class DataWriterParquetTest extends TestWithMiniDFSCluster { - private static final String ZERO_PAD_FMT = "%010d"; - private static final String extension = ".parquet"; - private final SchemaFileReader schemaFileReader = new ParquetFileReader(avroData); + + @Before + public void setUp() throws Exception { + super.setUp(); + schemaFileReader = new ParquetFileReader(avroData); + extension = ".parquet"; + } @Override protected Map createProps() { @@ -49,41 +46,17 @@ protected Map createProps() { @Test public void testWriteRecord() throws Exception { DataWriter hdfsWriter = new DataWriter(connectorConfig, context, avroData); - Partitioner partitioner = hdfsWriter.getPartitioner(); + partitioner = hdfsWriter.getPartitioner(); hdfsWriter.recover(TOPIC_PARTITION); - String key = "key"; - Schema schema = createSchema(); - Struct record = createRecord(schema); + List sinkRecords = createRecords(7); - Collection sinkRecords = new ArrayList<>(); - for (long offset = 0; offset < 7; offset++) { - SinkRecord sinkRecord = - new SinkRecord(TOPIC, PARTITION, Schema.STRING_SCHEMA, key, schema, record, offset); - - sinkRecords.add(sinkRecord); - } hdfsWriter.write(sinkRecords); hdfsWriter.close(assignment); hdfsWriter.stop(); - String encodedPartition = "partition=" + String.valueOf(PARTITION); - String directory = partitioner.generatePartitionedPath(TOPIC, encodedPartition); - // Last file (offset 6) doesn't satisfy size requirement and gets discarded on close - long[] validOffsets = {-1, 2, 5}; - for (int i = 1; i < validOffsets.length; i++) { - long startOffset = validOffsets[i - 1] + 1; - long endOffset = validOffsets[i]; - Path path = new Path( - FileUtils.committedFileName(url, topicsDir, directory, TOPIC_PARTITION, startOffset, - endOffset, extension, ZERO_PAD_FMT)); - Collection records = schemaFileReader.readData(conf, path); - long size = endOffset - startOffset + 1; - assertEquals(size, records.size()); - for (Object avroRecord : records) { - assertEquals(avroData.fromConnectData(schema, record), avroRecord); - } - } + long[] validOffsets = {0, 3, 6}; + verify(sinkRecords, validOffsets); } } From 56e2642f342513bf0a19fae14292f9a12cc8a9dd Mon Sep 17 00:00:00 2001 From: Konstantine Karantasis Date: Fri, 21 Apr 2017 10:39:47 -0700 Subject: [PATCH 03/58] Consolidate records creation methods in TopicPartitionWriterTest too. --- .../hdfs/HdfsSinkConnectorTestBase.java | 40 ++++++-- .../connect/hdfs/TestWithMiniDFSCluster.java | 44 +++++++-- .../connect/hdfs/avro/DataWriterAvroTest.java | 24 ++--- .../hdfs/avro/TopicPartitionWriterTest.java | 95 ++++++------------- .../hdfs/parquet/DataWriterParquetTest.java | 2 +- 5 files changed, 111 insertions(+), 94 deletions(-) diff --git a/src/test/java/io/confluent/connect/hdfs/HdfsSinkConnectorTestBase.java b/src/test/java/io/confluent/connect/hdfs/HdfsSinkConnectorTestBase.java index 039fc1d72..b1e6158fc 100644 --- a/src/test/java/io/confluent/connect/hdfs/HdfsSinkConnectorTestBase.java +++ b/src/test/java/io/confluent/connect/hdfs/HdfsSinkConnectorTestBase.java @@ -25,8 +25,10 @@ import org.junit.After; import org.junit.Before; +import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; +import java.util.List; import java.util.Map; import java.util.Set; @@ -72,13 +74,18 @@ protected Schema createSchema() { .build(); } - protected Struct createRecord(Schema schema) { + // Create a batch of records with incremental numeric field values. Total number of records is given by 'size'. + protected Struct createRecord(Schema schema, int ibase, float fbase) { return new Struct(schema) - .put("boolean", true) - .put("int", 12) - .put("long", 12L) - .put("float", 12.2f) - .put("double", 12.2); + .put("boolean", true) + .put("int", ibase) + .put("long", (long) ibase) + .put("float", fbase) + .put("double", (double) fbase); + } + + protected Struct createRecord(Schema schema) { + return createRecord(schema, 12, 12.2f); } protected Schema createNewSchema() { @@ -102,6 +109,27 @@ protected Struct createNewRecord(Schema newSchema) { .put("string", "def"); } + // Create a batch of records with incremental numeric field values. Total number of records is given by 'size'. + protected List createRecordBatch(Schema schema, int size) { + ArrayList records = new ArrayList<>(size); + int ibase = 16; + float fbase = 12.2f; + + for (int i = 0; i < size; ++i) { + records.add(createRecord(schema, ibase + i, fbase + i)); + } + return records; + } + + // Create a list of records by repeating the same record batch. Total number of records: 'batchesNum' x 'batchSize' + protected List createRecordBatches(Schema schema, int batchSize, int batchesNum) { + ArrayList records = new ArrayList<>(); + for (int i = 0; i < batchesNum; ++i) { + records.addAll(createRecordBatch(schema, batchSize)); + } + return records; + } + @Before public void setUp() throws Exception { conf = new Configuration(); diff --git a/src/test/java/io/confluent/connect/hdfs/TestWithMiniDFSCluster.java b/src/test/java/io/confluent/connect/hdfs/TestWithMiniDFSCluster.java index efc92ea09..abc658eb1 100644 --- a/src/test/java/io/confluent/connect/hdfs/TestWithMiniDFSCluster.java +++ b/src/test/java/io/confluent/connect/hdfs/TestWithMiniDFSCluster.java @@ -96,15 +96,14 @@ protected Map createProps() { return props; } - /** * Return a list of new records starting at zero offset. * * @param size the number of records to return. * @return the list of records. */ - protected List createRecords(int size) { - return createRecords(size, 0); + protected List createSinkRecords(int size) { + return createSinkRecords(size, 0); } /** @@ -114,8 +113,8 @@ protected List createRecords(int size) { * @param startOffset the starting offset. * @return the list of records. */ - protected List createRecords(int size, long startOffset) { - return createRecords(size, startOffset, Collections.singleton(new TopicPartition(TOPIC, PARTITION))); + protected List createSinkRecords(int size, long startOffset) { + return createSinkRecords(size, startOffset, Collections.singleton(new TopicPartition(TOPIC, PARTITION))); } /** @@ -126,7 +125,8 @@ protected List createRecords(int size, long startOffset) { * @param partitions the set of partitions to create records for. * @return the list of records. */ - protected List createRecords(int size, long startOffset, Set partitions) { + protected List createSinkRecords(int size, long startOffset, Set partitions) { + /* String key = "key"; Schema schema = createSchema(); Struct record = createRecord(schema); @@ -138,9 +138,35 @@ protected List createRecords(int size, long startOffset, Set same = new ArrayList<>(); + for (int i = 0; i < size; ++i) { + same.add(record); + } + return createSinkRecords(same, schema, startOffset, partitions); + } + + protected List createSinkRecords(List records, Schema schema) { + return createSinkRecords(records, schema, 0, Collections.singleton(new TopicPartition(TOPIC, PARTITION))); + } + + protected List createSinkRecords(List records, Schema schema, long startOffset, + Set partitions) { + String key = "key"; + List sinkRecords = new ArrayList<>(); + for (TopicPartition tp : partitions) { + long offset = startOffset; + for (Struct record : records) { + sinkRecords.add(new SinkRecord(TOPIC, tp.partition(), Schema.STRING_SCHEMA, key, schema, record, offset++)); + } + } + return sinkRecords; } - protected List createRecordsNoVersion(int size, long startOffset) { + protected List createSinkRecordsNoVersion(int size, long startOffset) { String key = "key"; Schema schemaNoVersion = SchemaBuilder.struct().name("record") .field("boolean", Schema.BOOLEAN_SCHEMA) @@ -165,7 +191,7 @@ protected List createRecordsNoVersion(int size, long startOffset) { return sinkRecords; } - protected List createRecordsWithAlteringSchemas(int size, long startOffset) { + protected List createSinkRecordsWithAlteringSchemas(int size, long startOffset) { String key = "key"; Schema schema = createSchema(); Struct record = createRecord(schema); @@ -186,7 +212,7 @@ protected List createRecordsWithAlteringSchemas(int size, long start return sinkRecords; } - protected List createRecordsInterleaved(int size, long startOffset, Set partitions) { + protected List createSinkRecordsInterleaved(int size, long startOffset, Set partitions) { String key = "key"; Schema schema = createSchema(); Struct record = createRecord(schema); diff --git a/src/test/java/io/confluent/connect/hdfs/avro/DataWriterAvroTest.java b/src/test/java/io/confluent/connect/hdfs/avro/DataWriterAvroTest.java index 48e73a2f7..d0f461525 100644 --- a/src/test/java/io/confluent/connect/hdfs/avro/DataWriterAvroTest.java +++ b/src/test/java/io/confluent/connect/hdfs/avro/DataWriterAvroTest.java @@ -55,7 +55,7 @@ public void testWriteRecord() throws Exception { partitioner = hdfsWriter.getPartitioner(); hdfsWriter.recover(TOPIC_PARTITION); - List sinkRecords = createRecords(7); + List sinkRecords = createSinkRecords(7); hdfsWriter.write(sinkRecords); hdfsWriter.close(assignment); @@ -98,7 +98,7 @@ public void testRecovery() throws Exception { assertTrue(offsets.containsKey(TOPIC_PARTITION)); assertEquals(50L, (long) offsets.get(TOPIC_PARTITION)); - List sinkRecords = createRecords(3, 50); + List sinkRecords = createSinkRecords(3, 50); hdfsWriter.write(sinkRecords); hdfsWriter.close(assignment); @@ -117,7 +117,7 @@ public void testWriteRecordMultiplePartitions() throws Exception { hdfsWriter.recover(tp); } - List sinkRecords = createRecords(7, 0, assignment); + List sinkRecords = createSinkRecords(7, 0, assignment); hdfsWriter.write(sinkRecords); hdfsWriter.close(assignment); @@ -164,7 +164,7 @@ public void testWriteRecordNonZeroInitialOffset() throws Exception { partitioner = hdfsWriter.getPartitioner(); hdfsWriter.recover(TOPIC_PARTITION); - List sinkRecords = createRecords(7, 3); + List sinkRecords = createSinkRecords(7, 3); hdfsWriter.write(sinkRecords); hdfsWriter.close(assignment); @@ -185,7 +185,7 @@ public void testRebalance() throws Exception { hdfsWriter.recover(tp); } - List sinkRecords = createRecords(7, 0, assignment); + List sinkRecords = createSinkRecords(7, 0, assignment); hdfsWriter.write(sinkRecords); Set oldAssignment = new HashSet<>(assignment); @@ -207,7 +207,7 @@ public void testRebalance() throws Exception { verify(sinkRecords, validOffsetsTopicPartition2, Collections.singleton(TOPIC_PARTITION2), true); // Message offsets start at 6 because we discarded the in-progress temp file on re-balance - sinkRecords = createRecords(3, 6, assignment); + sinkRecords = createSinkRecords(3, 6, assignment); hdfsWriter.write(sinkRecords); hdfsWriter.close(newAssignment); @@ -234,7 +234,7 @@ public void testProjectBackWard() throws Exception { partitioner = hdfsWriter.getPartitioner(); hdfsWriter.recover(TOPIC_PARTITION); - List sinkRecords = createRecordsWithAlteringSchemas(7, 0); + List sinkRecords = createSinkRecordsWithAlteringSchemas(7, 0); hdfsWriter.write(sinkRecords); hdfsWriter.close(assignment); @@ -253,7 +253,7 @@ public void testProjectNone() throws Exception { partitioner = hdfsWriter.getPartitioner(); hdfsWriter.recover(TOPIC_PARTITION); - List sinkRecords = createRecordsWithAlteringSchemas(7, 0); + List sinkRecords = createSinkRecordsWithAlteringSchemas(7, 0); hdfsWriter.write(sinkRecords); hdfsWriter.close(assignment); @@ -275,7 +275,7 @@ public void testProjectForward() throws Exception { hdfsWriter.recover(TOPIC_PARTITION); // By excluding the first element we get a list starting with record having the new schema. - List sinkRecords = createRecordsWithAlteringSchemas(8, 0).subList(1, 8); + List sinkRecords = createSinkRecordsWithAlteringSchemas(8, 0).subList(1, 8); hdfsWriter.write(sinkRecords); hdfsWriter.close(assignment); @@ -295,8 +295,8 @@ public void testProjectNoVersion() throws Exception { partitioner = hdfsWriter.getPartitioner(); hdfsWriter.recover(TOPIC_PARTITION); - List sinkRecords = createRecordsNoVersion(1, 0); - sinkRecords.addAll(createRecordsWithAlteringSchemas(7, 0)); + List sinkRecords = createSinkRecordsNoVersion(1, 0); + sinkRecords.addAll(createSinkRecordsWithAlteringSchemas(7, 0)); try { hdfsWriter.write(sinkRecords); @@ -332,7 +332,7 @@ public void testFlushPartialFile() throws Exception { partitioner = hdfsWriter.getPartitioner(); hdfsWriter.recover(TOPIC_PARTITION); - List sinkRecords = createRecords(NUMBER_OF_RECORDS); + List sinkRecords = createSinkRecords(NUMBER_OF_RECORDS); hdfsWriter.write(sinkRecords); // wait for rotation to happen diff --git a/src/test/java/io/confluent/connect/hdfs/avro/TopicPartitionWriterTest.java b/src/test/java/io/confluent/connect/hdfs/avro/TopicPartitionWriterTest.java index 3b7ba954f..8cf84ea51 100644 --- a/src/test/java/io/confluent/connect/hdfs/avro/TopicPartitionWriterTest.java +++ b/src/test/java/io/confluent/connect/hdfs/avro/TopicPartitionWriterTest.java @@ -28,6 +28,7 @@ import java.util.Collections; import java.util.HashMap; import java.util.HashSet; +import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.TimeUnit; @@ -36,7 +37,6 @@ import io.confluent.connect.hdfs.Format; import io.confluent.connect.hdfs.HdfsSinkConnectorConfig; import io.confluent.connect.hdfs.RecordWriterProvider; -import io.confluent.connect.hdfs.SchemaFileReader; import io.confluent.connect.hdfs.TestWithMiniDFSCluster; import io.confluent.connect.hdfs.TopicPartitionWriter; import io.confluent.connect.hdfs.filter.CommittedFileFilter; @@ -81,11 +81,11 @@ public void testWriteRecordDefaultWithPadding() throws Exception { TopicPartitionWriter topicPartitionWriter = new TopicPartitionWriter( TOPIC_PARTITION, storage, writerProvider, partitioner, connectorConfig, context, avroData); - String key = "key"; Schema schema = createSchema(); - Struct[] records = createRecords(schema); - - Collection sinkRecords = createSinkRecords(records, key, schema); + List records = createRecordBatches(schema, 3, 3); + // Add a single records at the end of the batches sequence. Total records: 10 + records.add(createRecord(schema)); + List sinkRecords = createSinkRecords(records, schema); for (SinkRecord record : sinkRecords) { topicPartitionWriter.buffer(record); @@ -102,7 +102,8 @@ public void testWriteRecordDefaultWithPadding() throws Exception { "/" + TOPIC + "+" + PARTITION + "+03+05" + extension)); expectedFiles.add(new Path(url + "/" + topicsDir + "/" + TOPIC + "/partition=" + PARTITION + "/" + TOPIC + "+" + PARTITION + "+06+08" + extension)); - verify(expectedFiles, records, schema); + int expectedBatchSize = 3; + verify(expectedFiles, expectedBatchSize, records, schema); } @@ -117,11 +118,17 @@ public void testWriteRecordFieldPartitioner() throws Exception { TopicPartitionWriter topicPartitionWriter = new TopicPartitionWriter( TOPIC_PARTITION, storage, writerProvider, partitioner, connectorConfig, context, avroData); - String key = "key"; Schema schema = createSchema(); - Struct[] records = createRecords(schema); + List records = new ArrayList<>(); + for (int i = 16; i < 19; ++i) { + for (int j = 0; j < 3; ++j) { + records.add(createRecord(schema, i, 12.2f)); - Collection sinkRecords = createSinkRecords(records, key, schema); + } + } + // Add a single records at the end of the batches sequence. Total records: 10 + records.add(createRecord(schema)); + List sinkRecords = createSinkRecords(records, schema); for (SinkRecord record : sinkRecords) { topicPartitionWriter.buffer(record); @@ -131,7 +138,6 @@ public void testWriteRecordFieldPartitioner() throws Exception { topicPartitionWriter.write(); topicPartitionWriter.close(); - String directory1 = partitioner.generatePartitionedPath(TOPIC, partitionField + "=" + String.valueOf(16)); String directory2 = partitioner.generatePartitionedPath(TOPIC, partitionField + "=" + String.valueOf(17)); String directory3 = partitioner.generatePartitionedPath(TOPIC, partitionField + "=" + String.valueOf(18)); @@ -141,7 +147,8 @@ public void testWriteRecordFieldPartitioner() throws Exception { expectedFiles.add(new Path(FileUtils.committedFileName(url, topicsDir, directory2, TOPIC_PARTITION, 3, 5, extension, zeroPadFormat))); expectedFiles.add(new Path(FileUtils.committedFileName(url, topicsDir, directory3, TOPIC_PARTITION, 6, 8, extension, zeroPadFormat))); - verify(expectedFiles, records, schema); + int expectedBatchSize = 3; + verify(expectedFiles, expectedBatchSize, records, schema); } @Test @@ -153,11 +160,11 @@ public void testWriteRecordTimeBasedPartition() throws Exception { TopicPartitionWriter topicPartitionWriter = new TopicPartitionWriter( TOPIC_PARTITION, storage, writerProvider, partitioner, connectorConfig, context, avroData); - String key = "key"; Schema schema = createSchema(); - Struct[] records = createRecords(schema); - - Collection sinkRecords = createSinkRecords(records, key, schema); + List records = createRecordBatches(schema, 3, 3); + // Add a single records at the end of the batches sequence. Total records: 10 + records.add(createRecord(schema)); + List sinkRecords = createSinkRecords(records, schema); for (SinkRecord record : sinkRecords) { topicPartitionWriter.buffer(record); @@ -167,7 +174,6 @@ public void testWriteRecordTimeBasedPartition() throws Exception { topicPartitionWriter.write(); topicPartitionWriter.close(); - long partitionDurationMs = (Long) config.get(HdfsSinkConnectorConfig.PARTITION_DURATION_MS_CONFIG); String pathFormat = (String) config.get(HdfsSinkConnectorConfig.PATH_FORMAT_CONFIG); String timeZoneString = (String) config.get(HdfsSinkConnectorConfig.TIMEZONE_CONFIG); @@ -182,7 +188,8 @@ public void testWriteRecordTimeBasedPartition() throws Exception { expectedFiles.add(new Path(FileUtils.committedFileName(url, topicsDir, directory, TOPIC_PARTITION, 3, 5, extension, zeroPadFormat))); expectedFiles.add(new Path(FileUtils.committedFileName(url, topicsDir, directory, TOPIC_PARTITION, 6, 8, extension, zeroPadFormat))); - verify(expectedFiles, records, schema); + int expectedBatchSize = 3; + verify(expectedFiles, expectedBatchSize, records, schema); } private Map createConfig() { @@ -209,51 +216,7 @@ private void createLogsDir(String url, String logsDir) throws IOException { } } - private Struct[] createRecords(Schema schema) { - Struct record1 = new Struct(schema) - .put("boolean", true) - .put("int", 16) - .put("long", 12L) - .put("float", 12.2f) - .put("double", 12.2); - - Struct record2 = new Struct(schema) - .put("boolean", true) - .put("int", 17) - .put("long", 12L) - .put("float", 12.2f) - .put("double", 12.2); - - Struct record3 = new Struct(schema) - .put("boolean", true) - .put("int", 18) - .put("long", 12L) - .put("float", 12.2f) - .put("double", 12.2); - - ArrayList records = new ArrayList<>(); - records.add(record1); - records.add(record2); - records.add(record3); - return records.toArray(new Struct[records.size()]); - } - - - private ArrayList createSinkRecords(Struct[] records, String key, Schema schema) { - ArrayList sinkRecords = new ArrayList<>(); - long offset = 0; - for (Struct record : records) { - for (long count = 0; count < 3; count++) { - SinkRecord sinkRecord = new SinkRecord(TOPIC, PARTITION, Schema.STRING_SCHEMA, key, schema, record, - offset + count); - sinkRecords.add(sinkRecord); - } - offset = offset + 3; - } - return sinkRecords; - } - - private void verify(Set expectedFiles, Struct[] records, Schema schema) throws IOException { + private void verify(Set expectedFiles, int expectedSize, List records, Schema schema) throws IOException { Path path = new Path(FileUtils.topicDirectory(url, topicsDir, TOPIC)); FileStatus[] statuses = FileUtils.traverse(storage, path, new CommittedFileFilter()); assertEquals(expectedFiles.size(), statuses.length); @@ -262,11 +225,11 @@ private void verify(Set expectedFiles, Struct[] records, Schema schema) th Path filePath = status.getPath(); assertTrue(expectedFiles.contains(status.getPath())); Collection avroRecords = schemaFileReader.readData(conf, filePath); - assertEquals(3, avroRecords.size()); - for (Object avroRecord: avroRecords) { - assertEquals(avroData.fromConnectData(schema, records[index]), avroRecord); + assertEquals(expectedSize, avroRecords.size()); + for (Object avroRecord : avroRecords) { + assertEquals(avroData.fromConnectData(schema, records.get(index++)), avroRecord); } - index++; } } + } diff --git a/src/test/java/io/confluent/connect/hdfs/parquet/DataWriterParquetTest.java b/src/test/java/io/confluent/connect/hdfs/parquet/DataWriterParquetTest.java index 6c71336dc..ac4ae9930 100644 --- a/src/test/java/io/confluent/connect/hdfs/parquet/DataWriterParquetTest.java +++ b/src/test/java/io/confluent/connect/hdfs/parquet/DataWriterParquetTest.java @@ -49,7 +49,7 @@ public void testWriteRecord() throws Exception { partitioner = hdfsWriter.getPartitioner(); hdfsWriter.recover(TOPIC_PARTITION); - List sinkRecords = createRecords(7); + List sinkRecords = createSinkRecords(7); hdfsWriter.write(sinkRecords); hdfsWriter.close(assignment); From bfedc9651572d8c03e708477e83565e7d631db79 Mon Sep 17 00:00:00 2001 From: Konstantine Karantasis Date: Fri, 21 Apr 2017 22:23:57 -0700 Subject: [PATCH 04/58] Fix hive tests. --- .../connect/hdfs/avro/AvroHiveUtilTest.java | 41 +++-- .../parquet/HiveIntegrationParquetTest.java | 148 ++++++------------ .../hdfs/parquet/ParquetHiveUtilTest.java | 31 ++-- 3 files changed, 85 insertions(+), 135 deletions(-) diff --git a/src/test/java/io/confluent/connect/hdfs/avro/AvroHiveUtilTest.java b/src/test/java/io/confluent/connect/hdfs/avro/AvroHiveUtilTest.java index 52d553b4a..74322e188 100644 --- a/src/test/java/io/confluent/connect/hdfs/avro/AvroHiveUtilTest.java +++ b/src/test/java/io/confluent/connect/hdfs/avro/AvroHiveUtilTest.java @@ -26,7 +26,6 @@ import org.junit.Test; import java.util.ArrayList; -import java.util.Collection; import java.util.List; import io.confluent.connect.avro.AvroData; @@ -58,14 +57,17 @@ public void testCreateTable() throws Exception { String location = "partition=" + String.valueOf(PARTITION); hiveMetaStore.addPartition(hiveDatabase, TOPIC, location); + Struct expectedRecord = createRecord(schema); + List expectedResult = new ArrayList<>(); List expectedColumnNames = new ArrayList<>(); - for (Field field: schema.fields()) { + for (Field field : schema.fields()) { expectedColumnNames.add(field.name()); + expectedResult.add(String.valueOf(expectedRecord.get(field.name()))); } Table table = hiveMetaStore.getTable(hiveDatabase, TOPIC); List actualColumnNames = new ArrayList<>(); - for (FieldSchema column: table.getSd().getCols()) { + for (FieldSchema column : table.getSd().getCols()) { actualColumnNames.add(column.getName()); } @@ -74,15 +76,15 @@ public void testCreateTable() throws Exception { assertEquals(1, partitionCols.size()); assertEquals("partition", partitionCols.get(0).getName()); - String[] expectedResult = {"true", "12", "12", "12.2", "12.2", "12"}; String result = HiveTestUtils.runHive(hiveExec, "SELECT * FROM " + TOPIC); String[] rows = result.split("\n"); // Only 6 of the 7 records should have been delivered due to flush_size = 3 assertEquals(6, rows.length); - for (String row: rows) { + for (String row : rows) { String[] parts = HiveTestUtils.parseOutput(row); - for (int j = 0; j < expectedResult.length; ++j) { - assertEquals(expectedResult[j], parts[j]); + int j = 0; + for (String expectedValue : expectedResult) { + assertEquals(expectedValue, parts[j++]); } } } @@ -97,14 +99,17 @@ public void testAlterSchema() throws Exception { String location = "partition=" + String.valueOf(PARTITION); hiveMetaStore.addPartition(hiveDatabase, TOPIC, location); + Struct expectedRecord = createRecord(schema); + List expectedResult = new ArrayList<>(); List expectedColumnNames = new ArrayList<>(); - for (Field field: schema.fields()) { + for (Field field : schema.fields()) { expectedColumnNames.add(field.name()); + expectedResult.add(String.valueOf(expectedRecord.get(field.name()))); } Table table = hiveMetaStore.getTable(hiveDatabase, TOPIC); List actualColumnNames = new ArrayList<>(); - for (FieldSchema column: table.getSd().getCols()) { + for (FieldSchema column : table.getSd().getCols()) { actualColumnNames.add(column.getName()); } @@ -114,15 +119,15 @@ public void testAlterSchema() throws Exception { hive.alterSchema(hiveDatabase, TOPIC, newSchema); - String[] expectedResult = {"true", "12", "12", "12.2", "12.2", "abc", "12"}; String result = HiveTestUtils.runHive(hiveExec, "SELECT * from " + TOPIC); String[] rows = result.split("\n"); // Only 6 of the 7 records should have been delivered due to flush_size = 3 assertEquals(6, rows.length); - for (String row: rows) { + for (String row : rows) { String[] parts = HiveTestUtils.parseOutput(row); - for (int j = 0; j < expectedResult.length; ++j) { - assertEquals(expectedResult[j], parts[j]); + int j = 0; + for (String expectedValue : expectedResult) { + assertEquals(expectedValue, parts[j++]); } } } @@ -131,16 +136,8 @@ private void prepareData(String topic, int partition) throws Exception { TopicPartition tp = new TopicPartition(topic, partition); DataWriter hdfsWriter = createWriter(context, avroData); hdfsWriter.recover(tp); - String key = "key"; - Schema schema = createSchema(); - Struct record = createRecord(schema); - Collection sinkRecords = new ArrayList<>(); - for (long offset = 0; offset < 7; offset++) { - SinkRecord sinkRecord = - new SinkRecord(topic, partition, Schema.STRING_SCHEMA, key, schema, record, offset); - sinkRecords.add(sinkRecord); - } + List sinkRecords = createSinkRecords(7); hdfsWriter.write(sinkRecords); hdfsWriter.close(assignment); diff --git a/src/test/java/io/confluent/connect/hdfs/parquet/HiveIntegrationParquetTest.java b/src/test/java/io/confluent/connect/hdfs/parquet/HiveIntegrationParquetTest.java index 6bf4a3290..5a7601e52 100644 --- a/src/test/java/io/confluent/connect/hdfs/parquet/HiveIntegrationParquetTest.java +++ b/src/test/java/io/confluent/connect/hdfs/parquet/HiveIntegrationParquetTest.java @@ -25,7 +25,7 @@ import org.junit.Test; import java.util.ArrayList; -import java.util.Collection; +import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.concurrent.TimeUnit; @@ -58,16 +58,7 @@ public void testSyncWithHiveParquet() throws Exception { DataWriter hdfsWriter = new DataWriter(connectorConfig, context, avroData); hdfsWriter.recover(TOPIC_PARTITION); - String key = "key"; - Schema schema = createSchema(); - Struct record = createRecord(schema); - - Collection sinkRecords = new ArrayList<>(); - for (long offset = 0; offset < 7; offset++) { - SinkRecord sinkRecord = - new SinkRecord(TOPIC, PARTITION, Schema.STRING_SCHEMA, key, schema, record, offset); - sinkRecords.add(sinkRecord); - } + List sinkRecords = createSinkRecords(7); hdfsWriter.write(sinkRecords); hdfsWriter.close(assignment); @@ -80,14 +71,18 @@ public void testSyncWithHiveParquet() throws Exception { hdfsWriter = new DataWriter(config, context, avroData); hdfsWriter.syncWithHive(); + Schema schema = createSchema(); + Struct expectedRecord = createRecord(schema); + List expectedResult = new ArrayList<>(); List expectedColumnNames = new ArrayList<>(); - for (Field field: schema.fields()) { + for (Field field : schema.fields()) { expectedColumnNames.add(field.name()); + expectedResult.add(String.valueOf(expectedRecord.get(field.name()))); } Table table = hiveMetaStore.getTable(hiveDatabase, TOPIC); List actualColumnNames = new ArrayList<>(); - for (FieldSchema column: table.getSd().getCols()) { + for (FieldSchema column : table.getSd().getCols()) { actualColumnNames.add(column.getName()); } assertEquals(expectedColumnNames, actualColumnNames); @@ -113,29 +108,21 @@ public void testHiveIntegrationParquet() throws Exception { DataWriter hdfsWriter = new DataWriter(config, context, avroData); hdfsWriter.recover(TOPIC_PARTITION); - String key = "key"; - Schema schema = createSchema(); - Struct record = createRecord(schema); - - Collection sinkRecords = new ArrayList<>(); - for (long offset = 0; offset < 7; offset++) { - SinkRecord sinkRecord = - new SinkRecord(TOPIC, PARTITION, Schema.STRING_SCHEMA, key, schema, record, offset); + List sinkRecords = createSinkRecords(7); - sinkRecords.add(sinkRecord); - } hdfsWriter.write(sinkRecords); hdfsWriter.close(assignment); hdfsWriter.stop(); + Schema schema = createSchema(); Table table = hiveMetaStore.getTable(hiveDatabase, TOPIC); List expectedColumnNames = new ArrayList<>(); - for (Field field: schema.fields()) { + for (Field field : schema.fields()) { expectedColumnNames.add(field.name()); } List actualColumnNames = new ArrayList<>(); - for (FieldSchema column: table.getSd().getCols()) { + for (FieldSchema column : table.getSd().getCols()) { actualColumnNames.add(column.getName()); } assertEquals(expectedColumnNames, actualColumnNames); @@ -159,20 +146,9 @@ public void testHiveIntegrationFieldPartitionerParquet() throws Exception { HdfsSinkConnectorConfig config = new HdfsSinkConnectorConfig(props); DataWriter hdfsWriter = new DataWriter(config, context, avroData); - String key = "key"; Schema schema = createSchema(); - - Struct[] records = createRecords(schema); - ArrayList sinkRecords = new ArrayList<>(); - long offset = 0; - for (Struct record : records) { - for (long count = 0; count < 3; count++) { - SinkRecord sinkRecord = new SinkRecord(TOPIC, PARTITION, Schema.STRING_SCHEMA, key, schema, record, - offset + count); - sinkRecords.add(sinkRecord); - } - offset = offset + 3; - } + List records = createRecordBatches(schema, 3, 3); + List sinkRecords = createSinkRecords(records, schema); hdfsWriter.write(sinkRecords); hdfsWriter.close(assignment); @@ -181,17 +157,16 @@ public void testHiveIntegrationFieldPartitionerParquet() throws Exception { Table table = hiveMetaStore.getTable(hiveDatabase, TOPIC); List expectedColumnNames = new ArrayList<>(); - for (Field field: schema.fields()) { + for (Field field : schema.fields()) { expectedColumnNames.add(field.name()); } List actualColumnNames = new ArrayList<>(); - for (FieldSchema column: table.getSd().getCols()) { + for (FieldSchema column : table.getSd().getCols()) { actualColumnNames.add(column.getName()); } assertEquals(expectedColumnNames, actualColumnNames); - String partitionFieldName = config.getString(HdfsSinkConnectorConfig.PARTITION_FIELD_NAME_CONFIG); String directory1 = TOPIC + "/" + partitionFieldName + "=" + String.valueOf(16); String directory2 = TOPIC + "/" + partitionFieldName + "=" + String.valueOf(17); @@ -206,20 +181,25 @@ public void testHiveIntegrationFieldPartitionerParquet() throws Exception { assertEquals(expectedPartitions, partitions); - ArrayList expectedResult = new ArrayList<>(); - for (int i = 16; i <= 18; ++i) { - String[] part = {"true", String.valueOf(i), "12", "12.2", "12.2"}; + List> expectedResults = new ArrayList<>(); + for (int i = 0; i < 3; ++i) { for (int j = 0; j < 3; ++j) { - expectedResult.add(part); + List result = new ArrayList<>(); + for (Field field : schema.fields()) { + result.add(String.valueOf(records.get(i).get(field.name()))); + } + expectedResults.add(result); } } + String result = HiveTestUtils.runHive(hiveExec, "SELECT * FROM " + TOPIC); String[] rows = result.split("\n"); assertEquals(9, rows.length); for (int i = 0; i < rows.length; ++i) { String[] parts = HiveTestUtils.parseOutput(rows[i]); - for (int j = 0; j < expectedResult.get(i).length; ++j) { - assertEquals(expectedResult.get(i)[j], parts[j]); + int j = 0; + for (String expectedValue : expectedResults.get(i)) { + assertEquals(expectedValue, parts[j++]); } } } @@ -235,20 +215,9 @@ public void testHiveIntegrationTimeBasedPartitionerParquet() throws Exception { HdfsSinkConnectorConfig config = new HdfsSinkConnectorConfig(props); DataWriter hdfsWriter = new DataWriter(config, context, avroData); - String key = "key"; Schema schema = createSchema(); - - Struct[] records = createRecords(schema); - ArrayList sinkRecords = new ArrayList<>(); - long offset = 0; - for (Struct record : records) { - for (long count = 0; count < 3; count++) { - SinkRecord sinkRecord = new SinkRecord(TOPIC, PARTITION, Schema.STRING_SCHEMA, key, schema, record, - offset + count); - sinkRecords.add(sinkRecord); - } - offset = offset + 3; - } + List records = createRecordBatches(schema, 3, 3); + List sinkRecords = createSinkRecords(records, schema); hdfsWriter.write(sinkRecords); hdfsWriter.close(assignment); @@ -257,12 +226,12 @@ public void testHiveIntegrationTimeBasedPartitionerParquet() throws Exception { Table table = hiveMetaStore.getTable(hiveDatabase, TOPIC); List expectedColumnNames = new ArrayList<>(); - for (Field field: schema.fields()) { + for (Field field : schema.fields()) { expectedColumnNames.add(field.name()); } List actualColumnNames = new ArrayList<>(); - for (FieldSchema column: table.getSd().getCols()) { + for (FieldSchema column : table.getSd().getCols()) { actualColumnNames.add(column.getName()); } assertEquals(expectedColumnNames, actualColumnNames); @@ -281,17 +250,23 @@ public void testHiveIntegrationTimeBasedPartitionerParquet() throws Exception { ArrayList partitionFields = new ArrayList<>(); String[] groups = encodedPartition.split("/"); - for (String group: groups) { + for (String group : groups) { String field = group.split("=")[1]; partitionFields.add(field); } - ArrayList expectedResult = new ArrayList<>(); - for (int i = 16; i <= 18; ++i) { - String[] part = {"true", String.valueOf(i), "12", "12.2", "12.2", - partitionFields.get(0), partitionFields.get(1), partitionFields.get(2)}; - for (int j = 0; j < 3; ++j) { - expectedResult.add(part); + List> expectedResults = new ArrayList<>(); + for (int j = 0; j < 3; ++j) { + for (int i = 0; i < 3; ++i) { + List result = Arrays.asList("true", + String.valueOf(16 + i), + String.valueOf((long) (16 + i)), + String.valueOf(12.2f + i), + String.valueOf((double) (12.2f + i)), + partitionFields.get(0), + partitionFields.get(1), + partitionFields.get(2)); + expectedResults.add(result); } } @@ -300,38 +275,11 @@ public void testHiveIntegrationTimeBasedPartitionerParquet() throws Exception { assertEquals(9, rows.length); for (int i = 0; i < rows.length; ++i) { String[] parts = HiveTestUtils.parseOutput(rows[i]); - for (int j = 0; j < expectedResult.get(i).length; ++j) { - assertEquals(expectedResult.get(i)[j], parts[j]); + int j = 0; + for (String expectedValue : expectedResults.get(i)) { + System.err.println("Exp: " + expectedValue + " Actual: " + parts[j]); + assertEquals(expectedValue, parts[j++]); } } } - - private Struct[] createRecords(Schema schema) { - Struct record1 = new Struct(schema) - .put("boolean", true) - .put("int", 16) - .put("long", 12L) - .put("float", 12.2f) - .put("double", 12.2); - - Struct record2 = new Struct(schema) - .put("boolean", true) - .put("int", 17) - .put("long", 12L) - .put("float", 12.2f) - .put("double", 12.2); - - Struct record3 = new Struct(schema) - .put("boolean", true) - .put("int", 18) - .put("long", 12L) - .put("float", 12.2f) - .put("double", 12.2); - - ArrayList records = new ArrayList<>(); - records.add(record1); - records.add(record2); - records.add(record3); - return records.toArray(new Struct[records.size()]); - } } diff --git a/src/test/java/io/confluent/connect/hdfs/parquet/ParquetHiveUtilTest.java b/src/test/java/io/confluent/connect/hdfs/parquet/ParquetHiveUtilTest.java index 1df3455cf..2467cf916 100644 --- a/src/test/java/io/confluent/connect/hdfs/parquet/ParquetHiveUtilTest.java +++ b/src/test/java/io/confluent/connect/hdfs/parquet/ParquetHiveUtilTest.java @@ -69,9 +69,12 @@ public void testCreateTable() throws Exception { String location = "partition=" + String.valueOf(PARTITION); hiveMetaStore.addPartition(hiveDatabase, TOPIC, location); + Struct expectedRecord = createRecord(schema); + List expectedResult = new ArrayList<>(); List expectedColumnNames = new ArrayList<>(); - for (Field field: schema.fields()) { + for (Field field : schema.fields()) { expectedColumnNames.add(field.name()); + expectedResult.add(String.valueOf(expectedRecord.get(field.name()))); } Table table = hiveMetaStore.getTable(hiveDatabase, TOPIC); @@ -85,15 +88,15 @@ public void testCreateTable() throws Exception { assertEquals(1, partitionCols.size()); assertEquals("partition", partitionCols.get(0).getName()); - String[] expectedResult = {"true", "12", "12", "12.2", "12.2", "12"}; - String result = HiveTestUtils.runHive(hiveExec, "SELECT * FROM " + TOPIC); + String result = HiveTestUtils.runHive(hiveExec, "SELECT * from " + TOPIC); String[] rows = result.split("\n"); // Only 6 of the 7 records should have been delivered due to flush_size = 3 assertEquals(6, rows.length); - for (String row: rows) { + for (String row : rows) { String[] parts = HiveTestUtils.parseOutput(row); - for (int j = 0; j < expectedResult.length; ++j) { - assertEquals(expectedResult[j], parts[j]); + int j = 0; + for (String expectedValue : expectedResult) { + assertEquals(expectedValue, parts[j++]); } } } @@ -108,9 +111,13 @@ public void testAlterSchema() throws Exception { String location = "partition=" + String.valueOf(PARTITION); hiveMetaStore.addPartition(hiveDatabase, TOPIC, location); + Schema newSchema = createNewSchema(); + Struct expectedRecord = createRecord(newSchema); + List expectedResult = new ArrayList<>(); List expectedColumnNames = new ArrayList<>(); - for (Field field: schema.fields()) { + for (Field field : schema.fields()) { expectedColumnNames.add(field.name()); + expectedResult.add(String.valueOf(expectedRecord.get(field.name()))); } Table table = hiveMetaStore.getTable(hiveDatabase, TOPIC); @@ -121,19 +128,17 @@ public void testAlterSchema() throws Exception { assertEquals(expectedColumnNames, actualColumnNames); - Schema newSchema = createNewSchema(); - hive.alterSchema(hiveDatabase, TOPIC, newSchema); - String[] expectedResult = {"true", "12", "12", "12.2", "12.2", "NULL", "12"}; String result = HiveTestUtils.runHive(hiveExec, "SELECT * from " + TOPIC); String[] rows = result.split("\n"); // Only 6 of the 7 records should have been delivered due to flush_size = 3 assertEquals(6, rows.length); - for (String row: rows) { + for (String row : rows) { String[] parts = HiveTestUtils.parseOutput(row); - for (int j = 0; j < expectedResult.length; ++j) { - assertEquals(expectedResult[j], parts[j]); + int j = 0; + for (String expectedValue : expectedResult) { + assertEquals(expectedValue, parts[j++]); } } } From 104bab7d2da9a5233400d0040ff74a68e63b3836 Mon Sep 17 00:00:00 2001 From: Konstantine Karantasis Date: Fri, 21 Apr 2017 22:34:43 -0700 Subject: [PATCH 05/58] Add tests with interleaved (non consecutive) records between partitions. --- .../connect/hdfs/avro/DataWriterAvroTest.java | 34 +++++++++++++++++++ 1 file changed, 34 insertions(+) diff --git a/src/test/java/io/confluent/connect/hdfs/avro/DataWriterAvroTest.java b/src/test/java/io/confluent/connect/hdfs/avro/DataWriterAvroTest.java index d0f461525..c7352c682 100644 --- a/src/test/java/io/confluent/connect/hdfs/avro/DataWriterAvroTest.java +++ b/src/test/java/io/confluent/connect/hdfs/avro/DataWriterAvroTest.java @@ -128,6 +128,40 @@ public void testWriteRecordMultiplePartitions() throws Exception { verify(sinkRecords, validOffsets, assignment); } + @Test + public void testWriteInterleavedRecordsInMultiplePartitions() throws Exception { + DataWriter hdfsWriter = new DataWriter(connectorConfig, context, avroData); + partitioner = hdfsWriter.getPartitioner(); + + for (TopicPartition tp: assignment) { + hdfsWriter.recover(tp); + } + + List sinkRecords = createSinkRecordsInterleaved(7 * assignment.size(), 0, assignment); + + hdfsWriter.write(sinkRecords); + hdfsWriter.close(assignment); + hdfsWriter.stop(); + + long[] validOffsets = {0, 3, 6}; + verify(sinkRecords, validOffsets, assignment); + } + + @Test + public void testWriteInterleavedRecordsInMultiplePartitionsNonZeroInitialOffset() throws Exception { + DataWriter hdfsWriter = new DataWriter(connectorConfig, context, avroData); + partitioner = hdfsWriter.getPartitioner(); + + List sinkRecords = createSinkRecordsInterleaved(7 * assignment.size(), 9, assignment); + + hdfsWriter.write(sinkRecords); + hdfsWriter.close(assignment); + hdfsWriter.stop(); + + long[] validOffsets = {9, 12, 15}; + verify(sinkRecords, validOffsets, assignment); + } + @Test public void testGetPreviousOffsets() throws Exception { String directory = TOPIC + "/" + "partition=" + String.valueOf(PARTITION); From b7a878d5fccf14eaff4231b0faf8f3d5546c190d Mon Sep 17 00:00:00 2001 From: Konstantine Karantasis Date: Fri, 21 Apr 2017 22:45:04 -0700 Subject: [PATCH 06/58] Code style fixes. --- .../hdfs/HdfsSinkConnectorTestBase.java | 11 +++++----- .../connect/hdfs/TestWithMiniDFSCluster.java | 20 +++++++++++-------- 2 files changed, 17 insertions(+), 14 deletions(-) diff --git a/src/test/java/io/confluent/connect/hdfs/HdfsSinkConnectorTestBase.java b/src/test/java/io/confluent/connect/hdfs/HdfsSinkConnectorTestBase.java index b1e6158fc..4145ef825 100644 --- a/src/test/java/io/confluent/connect/hdfs/HdfsSinkConnectorTestBase.java +++ b/src/test/java/io/confluent/connect/hdfs/HdfsSinkConnectorTestBase.java @@ -56,7 +56,6 @@ public class HdfsSinkConnectorTestBase { protected static final TopicPartition TOPIC_WITH_DOTS_PARTITION = new TopicPartition(TOPIC_WITH_DOTS, PARTITION); protected static Set assignment; - protected Map createProps() { Map props = new HashMap<>(); props.put(HdfsSinkConnectorConfig.HDFS_URL_CONFIG, url); @@ -77,11 +76,11 @@ protected Schema createSchema() { // Create a batch of records with incremental numeric field values. Total number of records is given by 'size'. protected Struct createRecord(Schema schema, int ibase, float fbase) { return new Struct(schema) - .put("boolean", true) - .put("int", ibase) - .put("long", (long) ibase) - .put("float", fbase) - .put("double", (double) fbase); + .put("boolean", true) + .put("int", ibase) + .put("long", (long) ibase) + .put("float", fbase) + .put("double", (double) fbase); } protected Struct createRecord(Schema schema) { diff --git a/src/test/java/io/confluent/connect/hdfs/TestWithMiniDFSCluster.java b/src/test/java/io/confluent/connect/hdfs/TestWithMiniDFSCluster.java index abc658eb1..df97afb6d 100644 --- a/src/test/java/io/confluent/connect/hdfs/TestWithMiniDFSCluster.java +++ b/src/test/java/io/confluent/connect/hdfs/TestWithMiniDFSCluster.java @@ -240,10 +240,12 @@ protected String getDirectory(String topic, int partition) { /** * Verify files and records are uploaded appropriately. - * @param sinkRecords a flat list of the records that need to appear in potentially several files in S3. - * @param validOffsets an array containing the offsets that map to uploaded files for a topic-partition. - * Offsets appear in ascending order, the difference between two consecutive offsets - * equals the expected size of the file, and last offset is exclusive. + * + * @param sinkRecords a flat list of the records that need to appear in potentially several files + * in HDFS. + * @param validOffsets an array containing the offsets that map to uploaded files for a + * topic-partition. Offsets appear in ascending order, the difference between two consecutive + * offsets equals the expected size of the file, and last offset is exclusive. */ protected void verify(List sinkRecords, long[] validOffsets) throws IOException { verify(sinkRecords, validOffsets, Collections.singleton(new TopicPartition(TOPIC, PARTITION)), false); @@ -256,10 +258,12 @@ protected void verify(List sinkRecords, long[] validOffsets, Set sinkRecords, long[] validOffsets, Set partitions, From 73a5dc7c27edd6fb2cd51fae6748b238c05bf40d Mon Sep 17 00:00:00 2001 From: Konstantine Karantasis Date: Mon, 28 Nov 2016 10:01:44 -0800 Subject: [PATCH 07/58] Increase maximum perm size to accommodate failing tests. --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 5af177573..c8a425514 100644 --- a/pom.xml +++ b/pom.xml @@ -223,7 +223,7 @@ maven-surefire-plugin 2.18.1 - -Djava.awt.headless=true + -Djava.awt.headless=true -XX:MaxPermSize=512m pertest From 29bce6ae1c8ac23a8a5aa8fcaf9b1fd35425f146 Mon Sep 17 00:00:00 2001 From: Konstantine Karantasis Date: Wed, 14 Dec 2016 15:35:18 -0800 Subject: [PATCH 08/58] Upgrade surefire parameters for test forking. --- pom.xml | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index c8a425514..31a74bc7e 100644 --- a/pom.xml +++ b/pom.xml @@ -224,7 +224,8 @@ 2.18.1 -Djava.awt.headless=true -XX:MaxPermSize=512m - pertest + false + 1 From 8630f5bb67f30c3573f6a0608d88946cf1db02e0 Mon Sep 17 00:00:00 2001 From: Konstantine Karantasis Date: Tue, 25 Apr 2017 17:30:14 -0700 Subject: [PATCH 09/58] Clean commented out code and fix typo. --- .../connect/hdfs/HdfsSinkConnectorTestBase.java | 4 ++-- .../connect/hdfs/TestWithMiniDFSCluster.java | 16 +--------------- .../connect/hdfs/avro/DataWriterAvroTest.java | 8 ++++---- 3 files changed, 7 insertions(+), 21 deletions(-) diff --git a/src/test/java/io/confluent/connect/hdfs/HdfsSinkConnectorTestBase.java b/src/test/java/io/confluent/connect/hdfs/HdfsSinkConnectorTestBase.java index 4145ef825..835a0b83d 100644 --- a/src/test/java/io/confluent/connect/hdfs/HdfsSinkConnectorTestBase.java +++ b/src/test/java/io/confluent/connect/hdfs/HdfsSinkConnectorTestBase.java @@ -73,7 +73,6 @@ protected Schema createSchema() { .build(); } - // Create a batch of records with incremental numeric field values. Total number of records is given by 'size'. protected Struct createRecord(Schema schema, int ibase, float fbase) { return new Struct(schema) .put("boolean", true) @@ -108,7 +107,8 @@ protected Struct createNewRecord(Schema newSchema) { .put("string", "def"); } - // Create a batch of records with incremental numeric field values. Total number of records is given by 'size'. + // Create a batch of records with incremental numeric field values. Total number of records is + // given by 'size'. protected List createRecordBatch(Schema schema, int size) { ArrayList records = new ArrayList<>(size); int ibase = 16; diff --git a/src/test/java/io/confluent/connect/hdfs/TestWithMiniDFSCluster.java b/src/test/java/io/confluent/connect/hdfs/TestWithMiniDFSCluster.java index df97afb6d..ffa26fb1a 100644 --- a/src/test/java/io/confluent/connect/hdfs/TestWithMiniDFSCluster.java +++ b/src/test/java/io/confluent/connect/hdfs/TestWithMiniDFSCluster.java @@ -126,20 +126,6 @@ protected List createSinkRecords(int size, long startOffset) { * @return the list of records. */ protected List createSinkRecords(int size, long startOffset, Set partitions) { - /* - String key = "key"; - Schema schema = createSchema(); - Struct record = createRecord(schema); - - List sinkRecords = new ArrayList<>(); - for (TopicPartition tp : partitions) { - for (long offset = startOffset; offset < startOffset + size; ++offset) { - sinkRecords.add(new SinkRecord(TOPIC, tp.partition(), Schema.STRING_SCHEMA, key, schema, record, offset)); - } - } - return sinkRecords; - */ - Schema schema = createSchema(); Struct record = createRecord(schema); List same = new ArrayList<>(); @@ -191,7 +177,7 @@ protected List createSinkRecordsNoVersion(int size, long startOffset return sinkRecords; } - protected List createSinkRecordsWithAlteringSchemas(int size, long startOffset) { + protected List createSinkRecordsWithAlternatingSchemas(int size, long startOffset) { String key = "key"; Schema schema = createSchema(); Struct record = createRecord(schema); diff --git a/src/test/java/io/confluent/connect/hdfs/avro/DataWriterAvroTest.java b/src/test/java/io/confluent/connect/hdfs/avro/DataWriterAvroTest.java index c7352c682..ca80a7a8f 100644 --- a/src/test/java/io/confluent/connect/hdfs/avro/DataWriterAvroTest.java +++ b/src/test/java/io/confluent/connect/hdfs/avro/DataWriterAvroTest.java @@ -268,7 +268,7 @@ public void testProjectBackWard() throws Exception { partitioner = hdfsWriter.getPartitioner(); hdfsWriter.recover(TOPIC_PARTITION); - List sinkRecords = createSinkRecordsWithAlteringSchemas(7, 0); + List sinkRecords = createSinkRecordsWithAlternatingSchemas(7, 0); hdfsWriter.write(sinkRecords); hdfsWriter.close(assignment); @@ -287,7 +287,7 @@ public void testProjectNone() throws Exception { partitioner = hdfsWriter.getPartitioner(); hdfsWriter.recover(TOPIC_PARTITION); - List sinkRecords = createSinkRecordsWithAlteringSchemas(7, 0); + List sinkRecords = createSinkRecordsWithAlternatingSchemas(7, 0); hdfsWriter.write(sinkRecords); hdfsWriter.close(assignment); @@ -309,7 +309,7 @@ public void testProjectForward() throws Exception { hdfsWriter.recover(TOPIC_PARTITION); // By excluding the first element we get a list starting with record having the new schema. - List sinkRecords = createSinkRecordsWithAlteringSchemas(8, 0).subList(1, 8); + List sinkRecords = createSinkRecordsWithAlternatingSchemas(8, 0).subList(1, 8); hdfsWriter.write(sinkRecords); hdfsWriter.close(assignment); @@ -330,7 +330,7 @@ public void testProjectNoVersion() throws Exception { hdfsWriter.recover(TOPIC_PARTITION); List sinkRecords = createSinkRecordsNoVersion(1, 0); - sinkRecords.addAll(createSinkRecordsWithAlteringSchemas(7, 0)); + sinkRecords.addAll(createSinkRecordsWithAlternatingSchemas(7, 0)); try { hdfsWriter.write(sinkRecords); From 41fd370cce5dc3edbc3ab92be7302598227a05ac Mon Sep 17 00:00:00 2001 From: Konstantine Karantasis Date: Tue, 22 Nov 2016 11:50:54 -0600 Subject: [PATCH 10/58] CC-397: Refactoring reusable components. This commit contains mainly the refactoring of interfaces. --- pom.xml | 5 + .../io/confluent/connect/hdfs/Format.java | 7 +- .../connect/hdfs/HdfsSinkConnectorConfig.java | 8 +- .../confluent/connect/hdfs/RecordWriter.java | 5 +- .../connect/hdfs/RecordWriterProvider.java | 8 +- .../connect/hdfs/SchemaFileReader.java | 5 +- .../hdfs/errors/HiveMetaStoreException.java | 3 +- .../connect/hdfs/hive/HiveMetaStore.java | 342 +----------------- .../confluent/connect/hdfs/hive/HiveUtil.java | 21 +- .../connect/hdfs/partitioner/Partitioner.java | 7 +- .../connect/hdfs/storage/Storage.java | 24 +- .../io/confluent/connect/hdfs/wal/WAL.java | 15 +- 12 files changed, 90 insertions(+), 360 deletions(-) diff --git a/pom.xml b/pom.xml index 31a74bc7e..89fc3b97a 100644 --- a/pom.xml +++ b/pom.xml @@ -83,6 +83,11 @@ kafka-connect-avro-converter ${confluent.version} + + io.confluent + kafka-connect-storage-common + 3.2.0-SNAPSHOT + org.apache.hadoop hadoop-client diff --git a/src/main/java/io/confluent/connect/hdfs/Format.java b/src/main/java/io/confluent/connect/hdfs/Format.java index eb4c6029e..f1ae43978 100644 --- a/src/main/java/io/confluent/connect/hdfs/Format.java +++ b/src/main/java/io/confluent/connect/hdfs/Format.java @@ -18,8 +18,13 @@ import io.confluent.connect.hdfs.hive.HiveMetaStore; import io.confluent.connect.hdfs.hive.HiveUtil; -public interface Format { +@Deprecated +public interface Format + extends io.confluent.connect.storage.Format { + @Override RecordWriterProvider getRecordWriterProvider(); + @Override SchemaFileReader getSchemaFileReader(AvroData avroData); + @Override HiveUtil getHiveUtil(HdfsSinkConnectorConfig config, AvroData avroData, HiveMetaStore hiveMetaStore); } diff --git a/src/main/java/io/confluent/connect/hdfs/HdfsSinkConnectorConfig.java b/src/main/java/io/confluent/connect/hdfs/HdfsSinkConnectorConfig.java index fac3e01cb..5683c63e5 100644 --- a/src/main/java/io/confluent/connect/hdfs/HdfsSinkConnectorConfig.java +++ b/src/main/java/io/confluent/connect/hdfs/HdfsSinkConnectorConfig.java @@ -14,7 +14,6 @@ package io.confluent.connect.hdfs; -import org.apache.kafka.common.config.AbstractConfig; import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.common.config.ConfigDef.Importance; import org.apache.kafka.common.config.ConfigDef.Type; @@ -32,8 +31,9 @@ import io.confluent.connect.hdfs.partitioner.HourlyPartitioner; import io.confluent.connect.hdfs.partitioner.Partitioner; import io.confluent.connect.hdfs.partitioner.TimeBasedPartitioner; +import io.confluent.connect.storage.StorageSinkConnectorConfig; -public class HdfsSinkConnectorConfig extends AbstractConfig { +public class HdfsSinkConnectorConfig extends StorageSinkConnectorConfig { // HDFS Group public static final String HDFS_URL_CONFIG = "hdfs.url"; @@ -259,8 +259,6 @@ public class HdfsSinkConnectorConfig extends AbstractConfig { private static final ConfigDef.Recommender partitionerClassDependentsRecommender = new PartitionerClassDependentsRecommender(); private static final ConfigDef.Recommender schemaCompatibilityRecommender = new SchemaCompatibilityRecommender(); - private static ConfigDef config = new ConfigDef(); - static { // Define HDFS configuration group @@ -405,7 +403,7 @@ public static ConfigDef getConfig() { } public HdfsSinkConnectorConfig(Map props) { - super(config, props); + super(props); } public static void main(String[] args) { diff --git a/src/main/java/io/confluent/connect/hdfs/RecordWriter.java b/src/main/java/io/confluent/connect/hdfs/RecordWriter.java index 2249e7093..9497df34d 100644 --- a/src/main/java/io/confluent/connect/hdfs/RecordWriter.java +++ b/src/main/java/io/confluent/connect/hdfs/RecordWriter.java @@ -18,7 +18,10 @@ import java.io.IOException; -public interface RecordWriter { +@Deprecated +public interface RecordWriter extends io.confluent.connect.storage.RecordWriter { + @Override void write(V value) throws IOException; + @Override void close() throws IOException; } diff --git a/src/main/java/io/confluent/connect/hdfs/RecordWriterProvider.java b/src/main/java/io/confluent/connect/hdfs/RecordWriterProvider.java index 6507cab1a..0ce22ebe4 100644 --- a/src/main/java/io/confluent/connect/hdfs/RecordWriterProvider.java +++ b/src/main/java/io/confluent/connect/hdfs/RecordWriterProvider.java @@ -21,7 +21,11 @@ import io.confluent.connect.avro.AvroData; -public interface RecordWriterProvider { +@Deprecated +public interface RecordWriterProvider extends io.confluent.connect.storage.RecordWriterProvider { + @Override String getExtension(); - RecordWriter getRecordWriter(Configuration conf, String fileName, SinkRecord record, AvroData avroData) throws IOException; + @Override + RecordWriter getRecordWriter(Configuration conf, String fileName, SinkRecord record, AvroData avroData) + throws IOException; } diff --git a/src/main/java/io/confluent/connect/hdfs/SchemaFileReader.java b/src/main/java/io/confluent/connect/hdfs/SchemaFileReader.java index 11b3a1ab9..9d5481820 100644 --- a/src/main/java/io/confluent/connect/hdfs/SchemaFileReader.java +++ b/src/main/java/io/confluent/connect/hdfs/SchemaFileReader.java @@ -21,7 +21,10 @@ import java.io.IOException; import java.util.Collection; -public interface SchemaFileReader { +@Deprecated +public interface SchemaFileReader extends io.confluent.connect.storage.SchemaFileReader { + @Override Schema getSchema(Configuration conf, Path path) throws IOException; + @Override Collection readData(Configuration conf, Path path) throws IOException; } diff --git a/src/main/java/io/confluent/connect/hdfs/errors/HiveMetaStoreException.java b/src/main/java/io/confluent/connect/hdfs/errors/HiveMetaStoreException.java index 56ffb62c9..165810e38 100644 --- a/src/main/java/io/confluent/connect/hdfs/errors/HiveMetaStoreException.java +++ b/src/main/java/io/confluent/connect/hdfs/errors/HiveMetaStoreException.java @@ -14,7 +14,8 @@ package io.confluent.connect.hdfs.errors; -public class HiveMetaStoreException extends RuntimeException{ +@Deprecated +public class HiveMetaStoreException extends io.confluent.connect.storage.errors.HiveMetaStoreException { public HiveMetaStoreException(String s) { super(s); diff --git a/src/main/java/io/confluent/connect/hdfs/hive/HiveMetaStore.java b/src/main/java/io/confluent/connect/hdfs/hive/HiveMetaStore.java index dca869a23..fc2f5316a 100644 --- a/src/main/java/io/confluent/connect/hdfs/hive/HiveMetaStore.java +++ b/src/main/java/io/confluent/connect/hdfs/hive/HiveMetaStore.java @@ -1,12 +1,12 @@ /** * Copyright 2015 Confluent Inc. - * + *

* Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. * You may obtain a copy of the License at - * + *

* http://www.apache.org/licenses/LICENSE-2.0 - * + *

* Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. @@ -16,343 +16,15 @@ package io.confluent.connect.hdfs.hive; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.metastore.IMetaStoreClient; -import org.apache.hadoop.hive.metastore.api.AlreadyExistsException; -import org.apache.hadoop.hive.metastore.api.Database; -import org.apache.hadoop.hive.metastore.api.InvalidObjectException; -import org.apache.hadoop.hive.metastore.api.InvalidOperationException; -import org.apache.hadoop.hive.metastore.api.MetaException; -import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; -import org.apache.hadoop.hive.metastore.api.Partition; -import org.apache.hadoop.hive.metastore.api.UnknownDBException; -import org.apache.hadoop.hive.ql.metadata.Table; -import org.apache.hive.hcatalog.common.HCatUtil; -import org.apache.thrift.TException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.File; -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; - import io.confluent.connect.hdfs.HdfsSinkConnectorConfig; import io.confluent.connect.hdfs.errors.HiveMetaStoreException; +import org.apache.hadoop.conf.Configuration; -public class HiveMetaStore { - - private static final Logger log = LoggerFactory.getLogger(HiveMetaStore.class); - private final IMetaStoreClient client; +@Deprecated +public class HiveMetaStore extends io.confluent.connect.storage.hive.HiveMetaStore { public HiveMetaStore(Configuration conf, HdfsSinkConnectorConfig connectorConfig) throws HiveMetaStoreException { - HiveConf hiveConf = new HiveConf(conf, HiveConf.class); - String hiveConfDir = connectorConfig.getString(HdfsSinkConnectorConfig.HIVE_CONF_DIR_CONFIG); - String hiveMetaStoreURIs = connectorConfig.getString(HdfsSinkConnectorConfig.HIVE_METASTORE_URIS_CONFIG); - if (hiveMetaStoreURIs.isEmpty()) { - log.warn("hive.metastore.uris empty, an embedded Hive metastore will be " - + "created in the directory the connector is started. " - + "You need to start Hive in that specific directory to query the data."); - } - if (!hiveConfDir.equals("")) { - String hiveSitePath = hiveConfDir + "/hive-site.xml"; - File hiveSite = new File(hiveSitePath); - if (!hiveSite.exists()) { - log.warn("hive-site.xml does not exist in provided Hive configuration directory {}.", hiveConf); - } - hiveConf.addResource(new Path(hiveSitePath)); - } - hiveConf.set("hive.metastore.uris", hiveMetaStoreURIs); - try { - client = HCatUtil.getHiveMetastoreClient(hiveConf); - } catch (IOException | MetaException e) { - throw new HiveMetaStoreException(e); - } - } - - private interface ClientAction { - R call() throws TException; - } - - private R doAction(ClientAction action) throws TException { - // No need to implement retries here. We use RetryingMetaStoreClient - // which creates a proxy for a IMetaStoreClient implementation and - // retries calls to it on failure. The retrying client is conscious - // of the socket timeout and does not call reconnect on an open connection. - // Since HiveMetaStoreClient's reconnect method does not check the status - // of the connection, blind retries may cause a huge spike in the number - // of connections to the Hive MetaStore. - return action.call(); - } - - public void addPartition(final String database, final String tableName, final String path) throws HiveMetaStoreException { - ClientAction addPartition = new ClientAction() { - @Override - public Void call() throws TException { - // purposely don't check if the partition already exists because - // getPartition(db, table, path) will throw an exception to indicate the - // partition doesn't exist also. this way, it's only one call. - client.appendPartition(database, tableNameConverter(tableName), path); - return null; - } - }; - - try { - doAction(addPartition); - } catch (AlreadyExistsException e) { - // this is okay - } catch (InvalidObjectException e) { - throw new HiveMetaStoreException("Invalid partition for " + database + "." + tableNameConverter(tableName) + ": " + path, e); - } catch (MetaException e) { - throw new HiveMetaStoreException("Hive MetaStore exception", e); - } catch (TException e) { - throw new HiveMetaStoreException("Exception communicating with the Hive MetaStore", e); - } - } - - public void dropPartition(final String database, final String tableName, final String path) throws HiveMetaStoreException { - ClientAction dropPartition = new ClientAction() { - @Override - public Void call() throws TException { - client.dropPartition(database, tableNameConverter(tableName), path, false); - return null; - } - }; - - try { - doAction(dropPartition); - } catch (NoSuchObjectException e) { - // this is okay - } catch (InvalidObjectException e) { - throw new HiveMetaStoreException("Invalid partition for " + database + "." + tableNameConverter(tableName) + ": " + path, e); - } catch (MetaException e) { - throw new HiveMetaStoreException("Hive MetaStore exception", e); - } catch (TException e) { - throw new HiveMetaStoreException("Exception communicating with the Hive MetaStore", e); - } - } - - - public void createDatabase(final String database) throws HiveMetaStoreException { - ClientAction create = new ClientAction() { - @Override - public Void call() throws TException { - client.createDatabase(new Database(database, "Database created by Kafka Connect", null, null)); - return null; - } - }; - - try { - doAction(create); - } catch (AlreadyExistsException e) { - log.warn("Hive database already exists: {}", database); - } catch (InvalidObjectException e) { - throw new HiveMetaStoreException("Invalid database: " + database, e); - } catch (MetaException e) { - throw new HiveMetaStoreException("Hive MetaStore exception", e); - } catch (TException e) { - throw new HiveMetaStoreException("Exception communicating with the Hive MetaStore", e); - } - } - - - public void dropDatabase(final String name, final boolean deleteData) throws HiveMetaStoreException { - ClientAction drop = new ClientAction() { - @Override - public Void call() throws TException { - client.dropDatabase(name, deleteData, true); - return null; - } - }; - - try { - doAction(drop); - } catch (NoSuchObjectException e) { - // this is okey - } catch (MetaException e) { - throw new HiveMetaStoreException("Hive MetaStore exception", e); - } catch (TException e) { - throw new HiveMetaStoreException("Exception communicating with the Hive MetaStore", e); - } - } - - public void createTable(final Table table) throws HiveMetaStoreException { - ClientAction create = new ClientAction() { - @Override - public Void call() throws TException { - client.createTable(table.getTTable()); - return null; - } - }; - - createDatabase(table.getDbName()); - - try { - doAction(create); - } catch (NoSuchObjectException e) { - throw new HiveMetaStoreException("Hive table not found: " + table.getDbName() + "." + tableNameConverter(table.getTableName())); - } catch (AlreadyExistsException e) { - // this is okey - log.warn("Hive table already exists: {}.{}", table.getDbName(), table.getTableName()); - } catch (InvalidObjectException e) { - throw new HiveMetaStoreException("Invalid table", e); - } catch (MetaException e) { - throw new HiveMetaStoreException("Hive MetaStore exception", e); - } catch (TException e) { - throw new HiveMetaStoreException("Exception communicating with the Hive MetaStore", e); - } - } - - public void alterTable(final Table table) throws HiveMetaStoreException { - ClientAction alter = new ClientAction() { - @Override - public Void call() throws TException { - client.alter_table(table.getDbName(), tableNameConverter(table.getTableName()), table.getTTable()); - return null; - } - }; - - try { - doAction(alter); - } catch (NoSuchObjectException e) { - throw new HiveMetaStoreException("Hive table not found: " + table.getDbName() + "." + table.getTableName()); - } catch (InvalidObjectException e) { - throw new HiveMetaStoreException("Invalid table", e); - } catch (InvalidOperationException e) { - throw new HiveMetaStoreException("Invalid table change", e); - } catch (MetaException e) { - throw new HiveMetaStoreException("Hive MetaStore exception", e); - } catch (TException e) { - throw new HiveMetaStoreException("Exception communicating with the Hive MetaStore", e); - } - } - - public void dropTable(final String database, final String tableName) { - ClientAction drop = new ClientAction() { - @Override - public Void call() throws TException { - client.dropTable(database, tableNameConverter(tableName), false, true); - return null; - } - }; - - try { - doAction(drop); - } catch (NoSuchObjectException e) { - // this is okay - } catch (MetaException e) { - throw new HiveMetaStoreException("Hive MetaStore exception", e); - } catch (TException e) { - throw new HiveMetaStoreException("Exception communicating with the Hive MetaStore", e); - } - } - - public boolean tableExists(final String database, final String tableName) throws HiveMetaStoreException { - ClientAction exists = new ClientAction() { - @Override - public Boolean call() throws TException { - return client.tableExists(database, tableNameConverter(tableName)); - } - }; - try { - return doAction(exists); - } catch (UnknownDBException e) { - return false; - } catch (MetaException e) { - throw new HiveMetaStoreException("Hive MetaStore exception", e); - } catch (TException e) { - throw new HiveMetaStoreException("Exception communicating with the Hive MetaStore", e); - } - } - - public Table getTable(final String database, final String tableName) throws HiveMetaStoreException { - ClientAction getTable = new ClientAction
() { - @Override - public Table call() throws TException { - return new Table(client.getTable(database, tableNameConverter(tableName))); - } - }; - - Table table; - try { - table = doAction(getTable); - } catch (NoSuchObjectException e) { - throw new HiveMetaStoreException("Hive table not found: " + database + "." + tableNameConverter(tableName)); - } catch (MetaException e) { - throw new HiveMetaStoreException("Hive table lookup exception", e); - } catch (TException e) { - throw new HiveMetaStoreException("Exception communicating with the Hive MetaStore", e); - } - - if (table == null) { - throw new HiveMetaStoreException("Could not find info for table: " + tableNameConverter(tableName)); - } - return table; - } - - public List listPartitions(final String database, final String tableName, final short max) throws HiveMetaStoreException { - ClientAction> listPartitions = new ClientAction>() { - @Override - public List call() throws TException { - List partitions = client.listPartitions(database, tableNameConverter(tableName), max); - List paths = new ArrayList<>(); - for (Partition partition : partitions) { - paths.add(partition.getSd().getLocation()); - } - return paths; - } - }; - - try { - return doAction(listPartitions); - } catch (NoSuchObjectException e) { - return new ArrayList<>(); - } catch (MetaException e) { - throw new HiveMetaStoreException("Hive MetaStore exception", e); - } catch (TException e) { - throw new HiveMetaStoreException("Exception communicating with the Hive MetaStore", e); - } - } - - public List getAllTables(final String database) throws HiveMetaStoreException { - ClientAction> getAllTables = new ClientAction>() { - @Override - public List call() throws TException { - return client.getAllTables(database); - } - }; - - try { - return doAction(getAllTables); - } catch (NoSuchObjectException e) { - return new ArrayList<>(); - } catch (MetaException e) { - throw new HiveMetaStoreException("Hive MetaStore exception", e); - } catch (TException e) { - throw new HiveMetaStoreException("Exception communicating with the Hive MetaStore", e); - } - } - - public List getAllDatabases() throws HiveMetaStoreException { - ClientAction> create = - new ClientAction>() { - @Override - public List call() throws TException { - return client.getAllDatabases(); - } - }; - - try { - return doAction(create); - } catch (NoSuchObjectException e) { - return new ArrayList<>(); - } catch (MetaException e) { - throw new HiveMetaStoreException("Hive MetaStore exception", e); - } catch (TException e) { - throw new HiveMetaStoreException("Exception communicating with the Hive MetaStore", e); - } + super(conf, connectorConfig); } public String tableNameConverter(String table){ diff --git a/src/main/java/io/confluent/connect/hdfs/hive/HiveUtil.java b/src/main/java/io/confluent/connect/hdfs/hive/HiveUtil.java index 7de26e6d1..021f8d83e 100644 --- a/src/main/java/io/confluent/connect/hdfs/hive/HiveUtil.java +++ b/src/main/java/io/confluent/connect/hdfs/hive/HiveUtil.java @@ -21,23 +21,22 @@ import io.confluent.connect.hdfs.HdfsSinkConnectorConfig; import io.confluent.connect.hdfs.partitioner.Partitioner; -public abstract class HiveUtil { - - protected final String url; - protected final String topicsDir; - protected final AvroData avroData; - protected final HiveMetaStore hiveMetaStore; - +@Deprecated +public abstract class HiveUtil extends io.confluent.connect.storage.hive.HiveUtil { public HiveUtil(HdfsSinkConnectorConfig connectorConfig, AvroData avroData, HiveMetaStore hiveMetaStore) { - this.url = connectorConfig.getString(HdfsSinkConnectorConfig.HDFS_URL_CONFIG); - this.topicsDir = connectorConfig.getString(HdfsSinkConnectorConfig.TOPICS_DIR_CONFIG); - this.avroData = avroData; - this.hiveMetaStore = hiveMetaStore; + super(connectorConfig, avroData, hiveMetaStore); + } + + @Override + public void createTable(String database, String tableName, + Schema schema, io.confluent.connect.storage.partitioner.Partitioner partitioner) { + createTable(database, tableName, schema, (Partitioner) partitioner); } public abstract void createTable(String database, String tableName, Schema schema, Partitioner partitioner); + @Override public abstract void alterSchema(String database, String tableName, Schema schema); public Table newTable(String database, String table){ diff --git a/src/main/java/io/confluent/connect/hdfs/partitioner/Partitioner.java b/src/main/java/io/confluent/connect/hdfs/partitioner/Partitioner.java index 1effdd102..91150a9a8 100644 --- a/src/main/java/io/confluent/connect/hdfs/partitioner/Partitioner.java +++ b/src/main/java/io/confluent/connect/hdfs/partitioner/Partitioner.java @@ -24,9 +24,14 @@ * Partition incoming records, and generates directories and file names in which to store the * incoming records. */ -public interface Partitioner { +@Deprecated +public interface Partitioner extends io.confluent.connect.storage.partitioner.Partitioner { + @Override void configure(Map config); + @Override String encodePartition(SinkRecord sinkRecord); + @Override String generatePartitionedPath(String topic, String encodedPartition); + @Override List partitionFields(); } diff --git a/src/main/java/io/confluent/connect/hdfs/storage/Storage.java b/src/main/java/io/confluent/connect/hdfs/storage/Storage.java index e1f6a1f1b..fdbe719f7 100644 --- a/src/main/java/io/confluent/connect/hdfs/storage/Storage.java +++ b/src/main/java/io/confluent/connect/hdfs/storage/Storage.java @@ -25,16 +25,38 @@ import io.confluent.connect.hdfs.wal.WAL; -public interface Storage { +@Deprecated +public interface Storage extends io.confluent.connect.storage.Storage { + @Override boolean exists(String filename) throws IOException; + + @Override boolean mkdirs(String filename) throws IOException; + + @Override void append(String filename, Object object) throws IOException; + + @Override void delete(String filename) throws IOException; + + @Override void commit(String tempFile, String committedFile) throws IOException; + + @Override void close() throws IOException; + + @Override WAL wal(String topicsDir, TopicPartition topicPart); + + @Override FileStatus[] listStatus(String path, PathFilter filter) throws IOException; + + @Override FileStatus[] listStatus(String path) throws IOException; + + @Override String url(); + + @Override Configuration conf(); } diff --git a/src/main/java/io/confluent/connect/hdfs/wal/WAL.java b/src/main/java/io/confluent/connect/hdfs/wal/WAL.java index f14bf1276..0e8094151 100644 --- a/src/main/java/io/confluent/connect/hdfs/wal/WAL.java +++ b/src/main/java/io/confluent/connect/hdfs/wal/WAL.java @@ -18,13 +18,26 @@ import org.apache.kafka.connect.errors.ConnectException; -public interface WAL { +@Deprecated +public interface WAL extends io.confluent.connect.storage.wal.WAL { String beginMarker = "BEGIN"; String endMarker = "END"; + + @Override void acquireLease() throws ConnectException; + + @Override void append(String tempFile, String committedFile) throws ConnectException; + + @Override void apply() throws ConnectException; + + @Override void truncate() throws ConnectException; + + @Override void close() throws ConnectException; + + @Override String getLogFile(); } From b9ba62b50ab0e2d782be157cae416e01eee0c859 Mon Sep 17 00:00:00 2001 From: Konstantine Karantasis Date: Tue, 22 Nov 2016 15:47:09 -0600 Subject: [PATCH 11/58] Refactor config. --- .../io/confluent/connect/hdfs/DataWriter.java | 1 + .../connect/hdfs/HdfsSinkConnectorConfig.java | 297 +----------------- 2 files changed, 6 insertions(+), 292 deletions(-) diff --git a/src/main/java/io/confluent/connect/hdfs/DataWriter.java b/src/main/java/io/confluent/connect/hdfs/DataWriter.java index 62f00c12e..73c0e15c3 100644 --- a/src/main/java/io/confluent/connect/hdfs/DataWriter.java +++ b/src/main/java/io/confluent/connect/hdfs/DataWriter.java @@ -372,6 +372,7 @@ private void createDir(String dir) throws IOException { @SuppressWarnings("unchecked") private Format getFormat() throws ClassNotFoundException, IllegalAccessException, InstantiationException{ + String name = connectorConfig.getString(HdfsSinkConnectorConfig.FORMAT_CLASS_CONFIG); return ((Class) Class.forName(connectorConfig.getString(HdfsSinkConnectorConfig.FORMAT_CLASS_CONFIG))).newInstance(); } diff --git a/src/main/java/io/confluent/connect/hdfs/HdfsSinkConnectorConfig.java b/src/main/java/io/confluent/connect/hdfs/HdfsSinkConnectorConfig.java index 5683c63e5..1d99469de 100644 --- a/src/main/java/io/confluent/connect/hdfs/HdfsSinkConnectorConfig.java +++ b/src/main/java/io/confluent/connect/hdfs/HdfsSinkConnectorConfig.java @@ -18,30 +18,17 @@ import org.apache.kafka.common.config.ConfigDef.Importance; import org.apache.kafka.common.config.ConfigDef.Type; import org.apache.kafka.common.config.ConfigDef.Width; -import org.apache.kafka.common.config.ConfigException; import java.util.Arrays; import java.util.LinkedList; import java.util.List; import java.util.Map; -import io.confluent.connect.hdfs.partitioner.DailyPartitioner; -import io.confluent.connect.hdfs.partitioner.DefaultPartitioner; -import io.confluent.connect.hdfs.partitioner.FieldPartitioner; -import io.confluent.connect.hdfs.partitioner.HourlyPartitioner; -import io.confluent.connect.hdfs.partitioner.Partitioner; -import io.confluent.connect.hdfs.partitioner.TimeBasedPartitioner; import io.confluent.connect.storage.StorageSinkConnectorConfig; public class HdfsSinkConnectorConfig extends StorageSinkConnectorConfig { // HDFS Group - public static final String HDFS_URL_CONFIG = "hdfs.url"; - private static final String HDFS_URL_DOC = - "The HDFS connection URL. This configuration has the format of hdfs:://hostname:port and " - + "specifies the HDFS to export data to."; - private static final String HDFS_URL_DISPLAY = "HDFS URL"; - public static final String HADOOP_CONF_DIR_CONFIG = "hadoop.conf.dir"; private static final String HADOOP_CONF_DIR_DOC = "The Hadoop configuration directory."; @@ -54,54 +41,6 @@ public class HdfsSinkConnectorConfig extends StorageSinkConnectorConfig { public static final String HADOOP_HOME_DEFAULT = ""; private static final String HADOOP_HOME_DISPLAY = "Hadoop home directory"; - public static final String TOPICS_DIR_CONFIG = "topics.dir"; - private static final String TOPICS_DIR_DOC = - "Top level HDFS directory to store the data ingested from Kafka."; - public static final String TOPICS_DIR_DEFAULT = "topics"; - private static final String TOPICS_DIR_DISPLAY = "Topics directory"; - - public static final String LOGS_DIR_CONFIG = "logs.dir"; - private static final String LOGS_DIR_DOC = - "Top level HDFS directory to store the write ahead logs."; - public static final String LOGS_DIR_DEFAULT = "logs"; - private static final String LOGS_DIR_DISPLAY = "Logs directory"; - - public static final String FORMAT_CLASS_CONFIG = "format.class"; - private static final String FORMAT_CLASS_DOC = - "The format class to use when writing data to HDFS. "; - public static final String FORMAT_CLASS_DEFAULT = "io.confluent.connect.hdfs.avro.AvroFormat"; - private static final String FORMAT_CLASS_DISPLAY = "Format class"; - - // Hive group - public static final String HIVE_INTEGRATION_CONFIG = "hive.integration"; - private static final String HIVE_INTEGRATION_DOC = - "Configuration indicating whether to integrate with Hive when running the connector."; - public static final boolean HIVE_INTEGRATION_DEFAULT = false; - private static final String HIVE_INTEGRATION_DISPLAY = "Hive Integration"; - - public static final String HIVE_METASTORE_URIS_CONFIG = "hive.metastore.uris"; - private static final String HIVE_METASTORE_URIS_DOC = - "The Hive metastore URIs, can be IP address or fully-qualified domain name " - + "and port of the metastore host."; - public static final String HIVE_METASTORE_URIS_DEFAULT = ""; - private static final String HIVE_METASTORE_URIS_DISPLAY = "Hive Metastore URIs"; - - public static final String HIVE_CONF_DIR_CONFIG = "hive.conf.dir"; - private static final String HIVE_CONF_DIR_DOC = "Hive configuration directory"; - public static final String HIVE_CONF_DIR_DEFAULT = ""; - private static final String HIVE_CONF_DIR_DISPLAY = "Hive configuration directory."; - - public static final String HIVE_HOME_CONFIG = "hive.home"; - private static final String HIVE_HOME_DOC = "Hive home directory."; - public static final String HIVE_HOME_DEFAULT = ""; - private static final String HIVE_HOME_DISPLAY = "Hive home directory"; - - public static final String HIVE_DATABASE_CONFIG = "hive.database"; - private static final String HIVE_DATABASE_DOC = - "The database to use when the connector creates tables in Hive."; - private static final String HIVE_DATABASE_DEFAULT = "default"; - private static final String HIVE_DATABASE_DISPLAY = "Hive database"; - // Security group public static final String HDFS_AUTHENTICATION_KERBEROS_CONFIG = "hdfs.authentication.kerberos"; private static final String HDFS_AUTHENTICATION_KERBEROS_DOC = @@ -133,150 +72,15 @@ public class HdfsSinkConnectorConfig extends StorageSinkConnectorConfig { public static final long KERBEROS_TICKET_RENEW_PERIOD_MS_DEFAULT = 60000 * 60; private static final String KERBEROS_TICKET_RENEW_PERIOD_MS_DISPLAY = "Kerberos Ticket Renew Period (ms)"; - // Connector group - public static final String FLUSH_SIZE_CONFIG = "flush.size"; - private static final String FLUSH_SIZE_DOC = - "Number of records written to HDFS before invoking file commits."; - private static final String FLUSH_SIZE_DISPLAY = "Flush Size"; - - public static final String ROTATE_INTERVAL_MS_CONFIG = "rotate.interval.ms"; - private static final String ROTATE_INTERVAL_MS_DOC = - "The time interval in milliseconds to invoke file commits. This configuration ensures that " - + "file commits are invoked every configured interval. This configuration is useful when data " - + "ingestion rate is low and the connector didn't write enough messages to commit files." - + "The default value -1 means that this feature is disabled."; - private static final long ROTATE_INTERVAL_MS_DEFAULT = -1L; - private static final String ROTATE_INTERVAL_MS_DISPLAY = "Rotate Interval (ms)"; - - public static final String ROTATE_SCHEDULE_INTERVAL_MS_CONFIG = "rotate.schedule.interval.ms"; - private static final String ROTATE_SCHEDULE_INTERVAL_MS_DOC = - "The time interval in milliseconds to periodically invoke file commits. This configuration ensures that " - + "file commits are invoked every configured interval. Time of commit will be adjusted to 00:00 of selected timezone. " - + "Commit will be performed at scheduled time regardless previous commit time or number of messages. " - + "This configuration is useful when you have to commit your data based on current server time, like at the beginning of every hour. " - + "The default value -1 means that this feature is disabled."; - private static final long ROTATE_SCHEDULE_INTERVAL_MS_DEFAULT = -1L; - private static final String ROTATE_SCHEDULE_INTERVAL_MS_DISPLAY = "Rotate Schedule Interval (ms)"; - - public static final String RETRY_BACKOFF_CONFIG = "retry.backoff.ms"; - private static final String RETRY_BACKOFF_DOC = - "The retry backoff in milliseconds. This config is used to " - + "notify Kafka connect to retry delivering a message batch or performing recovery in case " - + "of transient exceptions."; - public static final long RETRY_BACKOFF_DEFAULT = 5000L; - private static final String RETRY_BACKOFF_DISPLAY = "Retry Backoff (ms)"; - - public static final String SHUTDOWN_TIMEOUT_CONFIG = "shutdown.timeout.ms"; - private static final String SHUTDOWN_TIMEOUT_DOC = - "Clean shutdown timeout. This makes sure that asynchronous Hive metastore updates are " - + "completed during connector shutdown."; - private static final long SHUTDOWN_TIMEOUT_DEFAULT = 3000L; - private static final String SHUTDOWN_TIMEOUT_DISPLAY = "Shutdown Timeout (ms)"; - - public static final String PARTITIONER_CLASS_CONFIG = "partitioner.class"; - private static final String PARTITIONER_CLASS_DOC = - "The partitioner to use when writing data to HDFS. You can use ``DefaultPartitioner``, " - + "which preserves the Kafka partitions; ``FieldPartitioner``, which partitions the data to " - + "different directories according to the value of the partitioning field specified " - + "in ``partition.field.name``; ``TimebasedPartitioner``, which partitions data " - + "according to the time ingested to HDFS."; - public static final String PARTITIONER_CLASS_DEFAULT = - "io.confluent.connect.hdfs.partitioner.DefaultPartitioner"; - private static final String PARTITIONER_CLASS_DISPLAY = "Partitioner Class"; - - public static final String PARTITION_FIELD_NAME_CONFIG = "partition.field.name"; - private static final String PARTITION_FIELD_NAME_DOC = - "The name of the partitioning field when FieldPartitioner is used."; - public static final String PARTITION_FIELD_NAME_DEFAULT = ""; - public static final String PARTITION_FIELD_NAME_DISPLAY = "Partition Field Name"; - - public static final String PARTITION_DURATION_MS_CONFIG = "partition.duration.ms"; - private static final String PARTITION_DURATION_MS_DOC = - "The duration of a partition milliseconds used by ``TimeBasedPartitioner``. " - + "The default value -1 means that we are not using ``TimebasedPartitioner``."; - public static final long PARTITION_DURATION_MS_DEFAULT = -1L; - private static final String PARTITION_DURATION_MS_DISPLAY = "Partition Duration (ms)"; - - public static final String PATH_FORMAT_CONFIG = "path.format"; - private static final String PATH_FORMAT_DOC = - "This configuration is used to set the format of the data directories when partitioning with " - + "``TimeBasedPartitioner``. The format set in this configuration converts the Unix timestamp " - + "to proper directories strings. For example, if you set " - + "``path.format='year'=YYYY/'month'=MM/'day'=dd/'hour'=HH/``, the data directories will have" - + " the format ``/year=2015/month=12/day=07/hour=15``."; - public static final String PATH_FORMAT_DEFAULT = ""; - private static final String PATH_FORMAT_DISPLAY = "Path Format"; - - public static final String LOCALE_CONFIG = "locale"; - private static final String LOCALE_DOC = - "The locale to use when partitioning with ``TimeBasedPartitioner``."; - public static final String LOCALE_DEFAULT = ""; - private static final String LOCALE_DISPLAY = "Locale"; - - public static final String TIMEZONE_CONFIG = "timezone"; - private static final String TIMEZONE_DOC = - "The timezone to use when partitioning with ``TimeBasedPartitioner``."; - public static final String TIMEZONE_DEFAULT = ""; - private static final String TIMEZONE_DISPLAY = "Timezone"; - - public static final String FILENAME_OFFSET_ZERO_PAD_WIDTH_CONFIG = "filename.offset.zero.pad.width"; - private static final String FILENAME_OFFSET_ZERO_PAD_WIDTH_DOC = - "Width to zero pad offsets in HDFS filenames to if the offsets is too short in order to " - + "provide fixed width filenames that can be ordered by simple lexicographic sorting."; - public static final int FILENAME_OFFSET_ZERO_PAD_WIDTH_DEFAULT = 10; - private static final String FILENAME_OFFSET_ZERO_PAD_WIDTH_DISPLAY = "Filename Offset Zero Pad Width"; - - // Schema group - public static final String SCHEMA_COMPATIBILITY_CONFIG = "schema.compatibility"; - private static final String SCHEMA_COMPATIBILITY_DOC = - "The schema compatibility rule to use when the connector is observing schema changes. The " - + "supported configurations are NONE, BACKWARD, FORWARD and FULL."; - private static final String SCHEMA_COMPATIBILITY_DEFAULT = "NONE"; - private static final String SCHEMA_COMPATIBILITY_DISPLAY = "Schema Compatibility"; - - public static final String SCHEMA_CACHE_SIZE_CONFIG = "schema.cache.size"; - private static final String SCHEMA_CACHE_SIZE_DOC = - "The size of the schema cache used in the Avro converter."; - public static final int SCHEMA_CACHE_SIZE_DEFAULT = 1000; - private static final String SCHEMA_CACHE_SIZE_DISPLAY = "Schema Cache Size"; - - // Internal group - public static final String STORAGE_CLASS_CONFIG = "storage.class"; - private static final String STORAGE_CLASS_DOC = - "The underlying storage layer. The default is HDFS."; - public static final String STORAGE_CLASS_DEFAULT = "io.confluent.connect.hdfs.storage.HdfsStorage"; - private static final String STORAGE_CLASS_DISPLAY = "Storage Class"; - public static final String HDFS_GROUP = "HDFS"; - public static final String HIVE_GROUP = "Hive"; public static final String SECURITY_GROUP = "Security"; - public static final String SCHEMA_GROUP = "Schema"; - public static final String CONNECTOR_GROUP = "Connector"; - public static final String INTERNAL_GROUP = "Internal"; - private static final ConfigDef.Recommender hiveIntegrationDependentsRecommender = new BooleanParentRecommender(HIVE_INTEGRATION_CONFIG); private static final ConfigDef.Recommender hdfsAuthenticationKerberosDependentsRecommender = new BooleanParentRecommender(HDFS_AUTHENTICATION_KERBEROS_CONFIG); - private static final ConfigDef.Recommender partitionerClassDependentsRecommender = new PartitionerClassDependentsRecommender(); - private static final ConfigDef.Recommender schemaCompatibilityRecommender = new SchemaCompatibilityRecommender(); static { - // Define HDFS configuration group - config.define(HDFS_URL_CONFIG, Type.STRING, Importance.HIGH, HDFS_URL_DOC, HDFS_GROUP, 1, Width.MEDIUM, HDFS_URL_DISPLAY) - .define(HADOOP_CONF_DIR_CONFIG, Type.STRING, HADOOP_CONF_DIR_DEFAULT, Importance.HIGH, HADOOP_CONF_DIR_DOC, HDFS_GROUP, 2, Width.MEDIUM, HADOOP_CONF_DIR_DISPLAY) - .define(HADOOP_HOME_CONFIG, Type.STRING, HADOOP_HOME_DEFAULT, Importance.HIGH, HADOOP_HOME_DOC, HDFS_GROUP, 3, Width.SHORT, HADOOP_HOME_DISPLAY) - .define(TOPICS_DIR_CONFIG, Type.STRING, TOPICS_DIR_DEFAULT, Importance.HIGH, TOPICS_DIR_DOC, HDFS_GROUP, 4, Width.SHORT, TOPICS_DIR_DISPLAY) - .define(LOGS_DIR_CONFIG, Type.STRING, LOGS_DIR_DEFAULT, Importance.HIGH, LOGS_DIR_DOC, HDFS_GROUP, 5, Width.SHORT, LOGS_DIR_DISPLAY) - .define(FORMAT_CLASS_CONFIG, Type.STRING, FORMAT_CLASS_DEFAULT, Importance.HIGH, FORMAT_CLASS_DOC, HDFS_GROUP, 6, Width.SHORT, FORMAT_CLASS_DISPLAY); - - // Define Hive configuration group - config.define(HIVE_INTEGRATION_CONFIG, Type.BOOLEAN, HIVE_INTEGRATION_DEFAULT, Importance.HIGH, HIVE_INTEGRATION_DOC, HIVE_GROUP, 1, Width.SHORT, HIVE_INTEGRATION_DISPLAY, - Arrays.asList(HIVE_METASTORE_URIS_CONFIG, HIVE_CONF_DIR_CONFIG, HIVE_HOME_CONFIG, HIVE_DATABASE_CONFIG, SCHEMA_COMPATIBILITY_CONFIG)) - .define(HIVE_METASTORE_URIS_CONFIG, Type.STRING, HIVE_METASTORE_URIS_DEFAULT, Importance.HIGH, HIVE_METASTORE_URIS_DOC, HIVE_GROUP, 2, Width.MEDIUM, - HIVE_METASTORE_URIS_DISPLAY, hiveIntegrationDependentsRecommender) - .define(HIVE_CONF_DIR_CONFIG, Type.STRING, HIVE_CONF_DIR_DEFAULT, Importance.HIGH, HIVE_CONF_DIR_DOC, HIVE_GROUP, 3, Width.MEDIUM, HIVE_CONF_DIR_DISPLAY, hiveIntegrationDependentsRecommender) - .define(HIVE_HOME_CONFIG, Type.STRING, HIVE_HOME_DEFAULT, Importance.HIGH, HIVE_HOME_DOC, HIVE_GROUP, 4, Width.MEDIUM, HIVE_HOME_DISPLAY, hiveIntegrationDependentsRecommender) - .define(HIVE_DATABASE_CONFIG, Type.STRING, HIVE_DATABASE_DEFAULT, Importance.HIGH, HIVE_DATABASE_DOC, HIVE_GROUP, 5, Width.SHORT, HIVE_DATABASE_DISPLAY, hiveIntegrationDependentsRecommender); + config.define(HADOOP_CONF_DIR_CONFIG, Type.STRING, HADOOP_CONF_DIR_DEFAULT, Importance.HIGH, HADOOP_CONF_DIR_DOC, HDFS_GROUP, 2, Width.MEDIUM, HADOOP_CONF_DIR_DISPLAY) + .define(HADOOP_HOME_CONFIG, Type.STRING, HADOOP_HOME_DEFAULT, Importance.HIGH, HADOOP_HOME_DOC, HDFS_GROUP, 3, Width.SHORT, HADOOP_HOME_DISPLAY); // Define Security configuration group config.define(HDFS_AUTHENTICATION_KERBEROS_CONFIG, Type.BOOLEAN, HDFS_AUTHENTICATION_KERBEROS_DEFAULT, Importance.HIGH, HDFS_AUTHENTICATION_KERBEROS_DOC, @@ -290,65 +94,16 @@ public class HdfsSinkConnectorConfig extends StorageSinkConnectorConfig { SECURITY_GROUP, 4, Width.MEDIUM, HDFS_NAMENODE_PRINCIPAL_DISPLAY, hdfsAuthenticationKerberosDependentsRecommender) .define(KERBEROS_TICKET_RENEW_PERIOD_MS_CONFIG, Type.LONG, KERBEROS_TICKET_RENEW_PERIOD_MS_DEFAULT, Importance.LOW, KERBEROS_TICKET_RENEW_PERIOD_MS_DOC, SECURITY_GROUP, 5, Width.SHORT, KERBEROS_TICKET_RENEW_PERIOD_MS_DISPLAY, hdfsAuthenticationKerberosDependentsRecommender); - - // Define Schema configuration group - config.define(SCHEMA_COMPATIBILITY_CONFIG, Type.STRING, SCHEMA_COMPATIBILITY_DEFAULT, Importance.HIGH, SCHEMA_COMPATIBILITY_DOC, SCHEMA_GROUP, 1, Width.SHORT, - SCHEMA_COMPATIBILITY_DISPLAY, schemaCompatibilityRecommender) - .define(SCHEMA_CACHE_SIZE_CONFIG, Type.INT, SCHEMA_CACHE_SIZE_DEFAULT, Importance.LOW, SCHEMA_CACHE_SIZE_DOC, SCHEMA_GROUP, 2, Width.SHORT, SCHEMA_CACHE_SIZE_DISPLAY); - - // Define Connector configuration group - config.define(FLUSH_SIZE_CONFIG, Type.INT, Importance.HIGH, FLUSH_SIZE_DOC, CONNECTOR_GROUP, 1, Width.SHORT, FLUSH_SIZE_DISPLAY) - .define(ROTATE_INTERVAL_MS_CONFIG, Type.LONG, ROTATE_INTERVAL_MS_DEFAULT, Importance.HIGH, ROTATE_INTERVAL_MS_DOC, CONNECTOR_GROUP, 2, Width.SHORT, ROTATE_INTERVAL_MS_DISPLAY) - .define(ROTATE_SCHEDULE_INTERVAL_MS_CONFIG, Type.LONG, ROTATE_SCHEDULE_INTERVAL_MS_DEFAULT, Importance.MEDIUM, ROTATE_SCHEDULE_INTERVAL_MS_DOC, CONNECTOR_GROUP, 3, Width.SHORT, ROTATE_SCHEDULE_INTERVAL_MS_DISPLAY) - .define(RETRY_BACKOFF_CONFIG, Type.LONG, RETRY_BACKOFF_DEFAULT, Importance.LOW, RETRY_BACKOFF_DOC, CONNECTOR_GROUP, 4, Width.SHORT, RETRY_BACKOFF_DISPLAY) - .define(SHUTDOWN_TIMEOUT_CONFIG, Type.LONG, SHUTDOWN_TIMEOUT_DEFAULT, Importance.MEDIUM, SHUTDOWN_TIMEOUT_DOC, CONNECTOR_GROUP, 5, Width.SHORT, SHUTDOWN_TIMEOUT_DISPLAY) - .define(PARTITIONER_CLASS_CONFIG, Type.STRING, PARTITIONER_CLASS_DEFAULT, Importance.HIGH, PARTITIONER_CLASS_DOC, CONNECTOR_GROUP, 6, Width.LONG, PARTITIONER_CLASS_DISPLAY, - Arrays.asList(PARTITION_FIELD_NAME_CONFIG, PARTITION_DURATION_MS_CONFIG, PATH_FORMAT_CONFIG, LOCALE_CONFIG, TIMEZONE_CONFIG)) - .define(PARTITION_FIELD_NAME_CONFIG, Type.STRING, PARTITION_FIELD_NAME_DEFAULT, Importance.MEDIUM, PARTITION_FIELD_NAME_DOC, CONNECTOR_GROUP, 7, Width.MEDIUM, - PARTITION_FIELD_NAME_DISPLAY, partitionerClassDependentsRecommender) - .define(PARTITION_DURATION_MS_CONFIG, Type.LONG, PARTITION_DURATION_MS_DEFAULT, Importance.MEDIUM, PARTITION_DURATION_MS_DOC, CONNECTOR_GROUP, 8, Width.SHORT, - PARTITION_DURATION_MS_DISPLAY, partitionerClassDependentsRecommender) - .define(PATH_FORMAT_CONFIG, Type.STRING, PATH_FORMAT_DEFAULT, Importance.MEDIUM, PATH_FORMAT_DOC, CONNECTOR_GROUP, 9, Width.LONG, PATH_FORMAT_DISPLAY, - partitionerClassDependentsRecommender) - .define(LOCALE_CONFIG, Type.STRING, LOCALE_DEFAULT, Importance.MEDIUM, LOCALE_DOC, CONNECTOR_GROUP, 10, Width.MEDIUM, LOCALE_DISPLAY, partitionerClassDependentsRecommender) - .define(TIMEZONE_CONFIG, Type.STRING, TIMEZONE_DEFAULT, Importance.MEDIUM, TIMEZONE_DOC, CONNECTOR_GROUP, 11, Width.MEDIUM, TIMEZONE_DISPLAY, partitionerClassDependentsRecommender) - .define(FILENAME_OFFSET_ZERO_PAD_WIDTH_CONFIG, Type.INT, FILENAME_OFFSET_ZERO_PAD_WIDTH_DEFAULT, ConfigDef.Range.atLeast(0), Importance.LOW, FILENAME_OFFSET_ZERO_PAD_WIDTH_DOC, - CONNECTOR_GROUP, 12, Width.SHORT, FILENAME_OFFSET_ZERO_PAD_WIDTH_DISPLAY); - - // Define Internal configuration group - config.define(STORAGE_CLASS_CONFIG, Type.STRING, STORAGE_CLASS_DEFAULT, Importance.LOW, STORAGE_CLASS_DOC, INTERNAL_GROUP, 1, Width.MEDIUM, STORAGE_CLASS_DISPLAY); } - private static class SchemaCompatibilityRecommender extends BooleanParentRecommender { - - public SchemaCompatibilityRecommender() { - super(HIVE_INTEGRATION_CONFIG); - } - - @Override - public List validValues(String name, Map connectorConfigs) { - boolean hiveIntegration = (Boolean) connectorConfigs.get(parentConfigName); - if (hiveIntegration) { - return Arrays.asList("BACKWARD", "FORWARD", "FULL"); - } else { - return Arrays.asList("NONE", "BACKWARD", "FORWARD", "FULL"); - } - } - - @Override - public boolean visible(String name, Map connectorConfigs) { - return true; - } - } - private static class BooleanParentRecommender implements ConfigDef.Recommender { - + protected String parentConfigName; - + public BooleanParentRecommender(String parentConfigName) { this.parentConfigName = parentConfigName; } - + @Override public List validValues(String name, Map connectorConfigs) { return new LinkedList<>(); @@ -360,44 +115,6 @@ public boolean visible(String name, Map connectorConfigs) { } } - private static class PartitionerClassDependentsRecommender implements ConfigDef.Recommender { - - @Override - public List validValues(String name, Map props) { - return new LinkedList<>(); - } - - @Override - public boolean visible(String name, Map connectorConfigs) { - String partitionerName = (String) connectorConfigs.get(PARTITIONER_CLASS_CONFIG); - try { - @SuppressWarnings("unchecked") - Class partitioner = (Class) Class.forName(partitionerName); - if (classNameEquals(partitionerName, DefaultPartitioner.class)) { - return false; - } else if (FieldPartitioner.class.isAssignableFrom(partitioner)) { - // subclass of FieldPartitioner - return name.equals(PARTITION_FIELD_NAME_CONFIG); - } else if (TimeBasedPartitioner.class.isAssignableFrom(partitioner)) { - // subclass of TimeBasedPartitioner - if (classNameEquals(partitionerName, DailyPartitioner.class) || classNameEquals(partitionerName, HourlyPartitioner.class)) { - return name.equals(LOCALE_CONFIG) || name.equals(TIMEZONE_CONFIG); - } else { - return name.equals(PARTITION_DURATION_MS_CONFIG) || name.equals(PATH_FORMAT_CONFIG) || name.equals(LOCALE_CONFIG) || name.equals(TIMEZONE_CONFIG); - } - } else { - throw new ConfigException("Not a valid partitioner class: " + partitionerName); - } - } catch (ClassNotFoundException e) { - throw new ConfigException("Partitioner class not found: " + partitionerName); - } - } - } - - private static boolean classNameEquals(String className, Class clazz) { - return className.equals(clazz.getSimpleName()) || className.equals(clazz.getCanonicalName()); - } - public static ConfigDef getConfig() { return config; } @@ -405,8 +122,4 @@ public static ConfigDef getConfig() { public HdfsSinkConnectorConfig(Map props) { super(props); } - - public static void main(String[] args) { - System.out.println(config.toEnrichedRst()); - } } From a8ba06c17e6a4c164bfec818101a4498275f6584 Mon Sep 17 00:00:00 2001 From: Konstantine Karantasis Date: Mon, 28 Nov 2016 10:05:05 -0800 Subject: [PATCH 12/58] Refactor partitioners --- .../hdfs/errors/PartitionException.java | 3 +- .../hdfs/partitioner/DailyPartitioner.java | 38 +----- .../hdfs/partitioner/DefaultPartitioner.java | 35 +----- .../hdfs/partitioner/FieldPartitioner.java | 69 +---------- .../hdfs/partitioner/HourlyPartitioner.java | 39 +----- .../partitioner/TimeBasedPartitioner.java | 115 +----------------- 6 files changed, 16 insertions(+), 283 deletions(-) diff --git a/src/main/java/io/confluent/connect/hdfs/errors/PartitionException.java b/src/main/java/io/confluent/connect/hdfs/errors/PartitionException.java index 074881c19..e7142ed2f 100644 --- a/src/main/java/io/confluent/connect/hdfs/errors/PartitionException.java +++ b/src/main/java/io/confluent/connect/hdfs/errors/PartitionException.java @@ -14,7 +14,8 @@ package io.confluent.connect.hdfs.errors; -public class PartitionException extends RuntimeException { +@Deprecated +public class PartitionException extends io.confluent.connect.storage.errors.PartitionException { public PartitionException(String s) { super(s); diff --git a/src/main/java/io/confluent/connect/hdfs/partitioner/DailyPartitioner.java b/src/main/java/io/confluent/connect/hdfs/partitioner/DailyPartitioner.java index ece620b74..30e11fa31 100644 --- a/src/main/java/io/confluent/connect/hdfs/partitioner/DailyPartitioner.java +++ b/src/main/java/io/confluent/connect/hdfs/partitioner/DailyPartitioner.java @@ -14,40 +14,6 @@ package io.confluent.connect.hdfs.partitioner; -import org.apache.kafka.common.config.ConfigException; -import org.joda.time.DateTimeZone; - -import java.util.Locale; -import java.util.Map; -import java.util.concurrent.TimeUnit; - -import io.confluent.connect.hdfs.HdfsSinkConnectorConfig; - -public class DailyPartitioner extends TimeBasedPartitioner { - - private static long partitionDurationMs = TimeUnit.HOURS.toMillis(24); - private static String pathFormat = "'year'=YYYY/'month'=MM/'day'=dd/"; - - @Override - public void configure(Map config) { - String localeString = (String) config.get(HdfsSinkConnectorConfig.LOCALE_CONFIG); - if (localeString.equals("")) { - throw new ConfigException(HdfsSinkConnectorConfig.LOCALE_CONFIG, - localeString, "Locale cannot be empty."); - } - String timeZoneString = (String) config.get(HdfsSinkConnectorConfig.TIMEZONE_CONFIG); - if (timeZoneString.equals("")) { - throw new ConfigException(HdfsSinkConnectorConfig.TIMEZONE_CONFIG, - timeZoneString, "Timezone cannot be empty."); - } - String hiveIntString = (String) config.get(HdfsSinkConnectorConfig.HIVE_INTEGRATION_CONFIG); - boolean hiveIntegration = hiveIntString != null && hiveIntString.toLowerCase().equals("true"); - Locale locale = new Locale(localeString); - DateTimeZone timeZone = DateTimeZone.forID(timeZoneString); - init(partitionDurationMs, pathFormat, locale, timeZone, hiveIntegration); - } - - public String getPathFormat() { - return pathFormat; - } +@Deprecated +public class DailyPartitioner extends io.confluent.connect.storage.partitioner.DailyPartitioner implements Partitioner { } diff --git a/src/main/java/io/confluent/connect/hdfs/partitioner/DefaultPartitioner.java b/src/main/java/io/confluent/connect/hdfs/partitioner/DefaultPartitioner.java index b217085e5..ba4ce098e 100644 --- a/src/main/java/io/confluent/connect/hdfs/partitioner/DefaultPartitioner.java +++ b/src/main/java/io/confluent/connect/hdfs/partitioner/DefaultPartitioner.java @@ -14,36 +14,7 @@ package io.confluent.connect.hdfs.partitioner; -import org.apache.hadoop.hive.metastore.api.FieldSchema; -import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; -import org.apache.kafka.connect.sink.SinkRecord; - -import java.util.ArrayList; -import java.util.List; -import java.util.Map; - -public class DefaultPartitioner implements Partitioner { - - private static final String partitionField = "partition"; - private final List partitionFields = new ArrayList<>();; - - @Override - public void configure(Map config) { - partitionFields.add(new FieldSchema(partitionField, TypeInfoFactory.stringTypeInfo.toString(), "")); - } - - @Override - public String encodePartition(SinkRecord sinkRecord) { - return partitionField + "=" + String.valueOf(sinkRecord.kafkaPartition()); - } - - @Override - public String generatePartitionedPath(String topic, String encodedPartition) { - return topic + "/" + encodedPartition; - } - - @Override - public List partitionFields() { - return partitionFields; - } +@Deprecated +public class DefaultPartitioner extends io.confluent.connect.storage.partitioner.DefaultPartitioner + implements Partitioner{ } diff --git a/src/main/java/io/confluent/connect/hdfs/partitioner/FieldPartitioner.java b/src/main/java/io/confluent/connect/hdfs/partitioner/FieldPartitioner.java index 97a5f720d..2ede53ae0 100644 --- a/src/main/java/io/confluent/connect/hdfs/partitioner/FieldPartitioner.java +++ b/src/main/java/io/confluent/connect/hdfs/partitioner/FieldPartitioner.java @@ -14,70 +14,7 @@ package io.confluent.connect.hdfs.partitioner; -import org.apache.hadoop.hive.metastore.api.FieldSchema; -import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; -import org.apache.kafka.connect.data.Schema; -import org.apache.kafka.connect.data.Schema.Type; -import org.apache.kafka.connect.data.Struct; -import org.apache.kafka.connect.sink.SinkRecord; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.ArrayList; -import java.util.List; -import java.util.Map; - -import io.confluent.connect.hdfs.HdfsSinkConnectorConfig; -import io.confluent.connect.hdfs.errors.PartitionException; - -public class FieldPartitioner implements Partitioner { - private static final Logger log = LoggerFactory.getLogger(FieldPartitioner.class); - private static String fieldName; - private List partitionFields = new ArrayList<>(); - - @Override - public void configure(Map config) { - fieldName = (String) config.get(HdfsSinkConnectorConfig.PARTITION_FIELD_NAME_CONFIG); - partitionFields.add(new FieldSchema(fieldName, TypeInfoFactory.stringTypeInfo.toString(), "")); - } - - @Override - public String encodePartition(SinkRecord sinkRecord) { - Object value = sinkRecord.value(); - Schema valueSchema = sinkRecord.valueSchema(); - if (value instanceof Struct) { - Struct struct = (Struct) value; - Object partitionKey = struct.get(fieldName); - Type type = valueSchema.field(fieldName).schema().type(); - switch (type) { - case INT8: - case INT16: - case INT32: - case INT64: - Number record = (Number) partitionKey; - return fieldName + "=" + record.toString(); - case STRING: - return fieldName + "=" + (String) partitionKey; - case BOOLEAN: - boolean booleanRecord = (boolean) partitionKey; - return fieldName + "=" + Boolean.toString(booleanRecord); - default: - log.error("Type {} is not supported as a partition key.", type.getName()); - throw new PartitionException("Error encoding partition."); - } - } else { - log.error("Value is not Struct type."); - throw new PartitionException("Error encoding partition."); - } - } - - @Override - public String generatePartitionedPath(String topic, String encodedPartition) { - return topic + "/" + encodedPartition; - } - - @Override - public List partitionFields() { - return partitionFields; - } +@Deprecated +public class FieldPartitioner extends io.confluent.connect.storage.partitioner.FieldPartitioner + implements Partitioner { } diff --git a/src/main/java/io/confluent/connect/hdfs/partitioner/HourlyPartitioner.java b/src/main/java/io/confluent/connect/hdfs/partitioner/HourlyPartitioner.java index 777da994c..cf03ef6e8 100644 --- a/src/main/java/io/confluent/connect/hdfs/partitioner/HourlyPartitioner.java +++ b/src/main/java/io/confluent/connect/hdfs/partitioner/HourlyPartitioner.java @@ -14,40 +14,7 @@ package io.confluent.connect.hdfs.partitioner; -import org.apache.kafka.common.config.ConfigException; -import org.joda.time.DateTimeZone; - -import java.util.Locale; -import java.util.Map; -import java.util.concurrent.TimeUnit; - -import io.confluent.connect.hdfs.HdfsSinkConnectorConfig; - -public class HourlyPartitioner extends TimeBasedPartitioner { - - private static long partitionDurationMs = TimeUnit.HOURS.toMillis(1); - private static String pathFormat = "'year'=YYYY/'month'=MM/'day'=dd/'hour'=HH/"; - - @Override - public void configure(Map config) { - String localeString = (String) config.get(HdfsSinkConnectorConfig.LOCALE_CONFIG); - if (localeString.equals("")) { - throw new ConfigException(HdfsSinkConnectorConfig.LOCALE_CONFIG, - localeString, "Locale cannot be empty."); - } - String timeZoneString = (String) config.get(HdfsSinkConnectorConfig.TIMEZONE_CONFIG); - if (timeZoneString.equals("")) { - throw new ConfigException(HdfsSinkConnectorConfig.TIMEZONE_CONFIG, - timeZoneString, "Timezone cannot be empty."); - } - String hiveIntString = (String) config.get(HdfsSinkConnectorConfig.HIVE_INTEGRATION_CONFIG); - boolean hiveIntegration = hiveIntString != null && hiveIntString.toLowerCase().equals("true"); - Locale locale = new Locale(localeString); - DateTimeZone timeZone = DateTimeZone.forID(timeZoneString); - init(partitionDurationMs, pathFormat, locale, timeZone, hiveIntegration); - } - - public String getPathFormat() { - return pathFormat; - } +@Deprecated +public class HourlyPartitioner extends io.confluent.connect.storage.partitioner.HourlyPartitioner + implements Partitioner { } diff --git a/src/main/java/io/confluent/connect/hdfs/partitioner/TimeBasedPartitioner.java b/src/main/java/io/confluent/connect/hdfs/partitioner/TimeBasedPartitioner.java index 359b10820..c8989e97a 100644 --- a/src/main/java/io/confluent/connect/hdfs/partitioner/TimeBasedPartitioner.java +++ b/src/main/java/io/confluent/connect/hdfs/partitioner/TimeBasedPartitioner.java @@ -14,116 +14,7 @@ package io.confluent.connect.hdfs.partitioner; -import org.apache.hadoop.hive.metastore.api.FieldSchema; -import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; -import org.apache.kafka.common.config.ConfigException; -import org.apache.kafka.connect.sink.SinkRecord; -import org.joda.time.DateTime; -import org.joda.time.DateTimeZone; -import org.joda.time.format.DateTimeFormat; -import org.joda.time.format.DateTimeFormatter; - -import java.util.ArrayList; -import java.util.List; -import java.util.Locale; -import java.util.Map; -import java.util.regex.Matcher; -import java.util.regex.Pattern; - -import io.confluent.connect.hdfs.HdfsSinkConnectorConfig; - -public class TimeBasedPartitioner implements Partitioner { - - // Duration of a partition in milliseconds. - private long partitionDurationMs; - private DateTimeFormatter formatter; - protected List partitionFields = new ArrayList<>(); - private static String patternString = "'year'=Y{1,5}/('month'=M{1,5}/)?('day'=d{1,3}/)?('hour'=H{1,3}/)?('minute'=m{1,3}/)?"; - private static Pattern pattern = Pattern.compile(patternString); - - protected void init(long partitionDurationMs, String pathFormat, Locale locale, - DateTimeZone timeZone, boolean hiveIntegration) { - this.partitionDurationMs = partitionDurationMs; - this.formatter = getDateTimeFormatter(pathFormat, timeZone).withLocale(locale); - addToPartitionFields(pathFormat, hiveIntegration); - } - - private static DateTimeFormatter getDateTimeFormatter(String str, DateTimeZone timeZone) { - return DateTimeFormat.forPattern(str).withZone(timeZone); - } - - public static long getPartition(long timeGranularityMs, long timestamp, DateTimeZone timeZone) { - long adjustedTimeStamp = timeZone.convertUTCToLocal(timestamp); - long partitionedTime = (adjustedTimeStamp / timeGranularityMs) * timeGranularityMs; - return timeZone.convertLocalToUTC(partitionedTime, false); - } - - @Override - public void configure(Map config) { - long partitionDurationMs = (long) config.get(HdfsSinkConnectorConfig.PARTITION_DURATION_MS_CONFIG); - if (partitionDurationMs < 0) { - throw new ConfigException(HdfsSinkConnectorConfig.PARTITION_DURATION_MS_CONFIG, - partitionDurationMs, "Partition duration needs to be a positive."); - } - - String pathFormat = (String) config.get(HdfsSinkConnectorConfig.PATH_FORMAT_CONFIG); - if (pathFormat.equals("")) { - throw new ConfigException(HdfsSinkConnectorConfig.PATH_FORMAT_CONFIG, - pathFormat, "Path format cannot be empty."); - } - - String localeString = (String) config.get(HdfsSinkConnectorConfig.LOCALE_CONFIG); - if (localeString.equals("")) { - throw new ConfigException(HdfsSinkConnectorConfig.LOCALE_CONFIG, - localeString, "Locale cannot be empty."); - } - String timeZoneString = (String) config.get(HdfsSinkConnectorConfig.TIMEZONE_CONFIG); - if (timeZoneString.equals("")) { - throw new ConfigException(HdfsSinkConnectorConfig.TIMEZONE_CONFIG, - timeZoneString, "Timezone cannot be empty."); - } - - String hiveIntString = (String) config.get(HdfsSinkConnectorConfig.HIVE_INTEGRATION_CONFIG); - boolean hiveIntegration = hiveIntString != null && hiveIntString.toLowerCase().equals("true"); - - Locale locale = new Locale(localeString); - DateTimeZone timeZone = DateTimeZone.forID(timeZoneString); - init(partitionDurationMs, pathFormat, locale, timeZone, hiveIntegration); - } - - @Override - public String encodePartition(SinkRecord sinkRecord) { - long timestamp = System.currentTimeMillis(); - DateTime bucket = new DateTime(getPartition(partitionDurationMs, timestamp, formatter.getZone())); - return bucket.toString(formatter); - } - - - @Override - public String generatePartitionedPath(String topic, String encodedPartition) { - return topic + "/" + encodedPartition; - } - - @Override - public List partitionFields() { - return partitionFields; - } - - private boolean verifyDateTimeFormat(String pathFormat) { - Matcher m = pattern.matcher(pathFormat); - return m.matches(); - } - - private void addToPartitionFields(String pathFormat, boolean hiveIntegration) { - if (hiveIntegration && !verifyDateTimeFormat(pathFormat)) { - throw new ConfigException(HdfsSinkConnectorConfig.PATH_FORMAT_CONFIG, pathFormat, - "Path format doesn't meet the requirements for Hive integration, " - + "which require prefixing each DateTime component with its name."); - } - for (String field: pathFormat.split("/")) { - String[] parts = field.split("="); - FieldSchema fieldSchema = new FieldSchema(parts[0].replace("'", ""), TypeInfoFactory.stringTypeInfo.toString(), ""); - partitionFields.add(fieldSchema); - } - } +@Deprecated +public class TimeBasedPartitioner extends io.confluent.connect.storage.partitioner.TimeBasedPartitioner + implements Partitioner { } From e62c94403587f61649f96d1f6e44bd97673eabdb Mon Sep 17 00:00:00 2001 From: Konstantine Karantasis Date: Mon, 6 Mar 2017 13:27:28 -0800 Subject: [PATCH 13/58] Extend pom to add submodules from storage-common. --- pom.xml | 42 ++++++++++++------------------------ src/assembly/development.xml | 1 - src/assembly/package.xml | 1 - 3 files changed, 14 insertions(+), 30 deletions(-) diff --git a/pom.xml b/pom.xml index 89fc3b97a..487bf4ed4 100644 --- a/pom.xml +++ b/pom.xml @@ -86,42 +86,27 @@ io.confluent kafka-connect-storage-common - 3.2.0-SNAPSHOT - - - org.apache.hadoop - hadoop-client - ${hadoop.version} - - - org.apache.hive - hive-cli - ${hive.version} - - - org.apache.hive - hive-common - ${hive.version} + ${confluent.version} - org.apache.avro - avro-mapred - ${avro.version} + io.confluent + kafka-connect-storage-core + ${confluent.version} - org.apache.parquet - parquet-column - ${parquet.version} + io.confluent + kafka-connect-storage-format + ${confluent.version} - org.apache.parquet - parquet-avro - ${parquet.version} + io.confluent + kafka-connect-storage-partitioner + ${confluent.version} - commons-io - commons-io - ${commons-io.version} + io.confluent + kafka-connect-storage-wal + ${confluent.version} junit @@ -212,6 +197,7 @@ src/assembly/development.xml src/assembly/package.xml + false diff --git a/src/assembly/development.xml b/src/assembly/development.xml index 498a914a2..96cf78082 100644 --- a/src/assembly/development.xml +++ b/src/assembly/development.xml @@ -19,7 +19,6 @@ shipped with Confluent platform and other dependencies such as Hadoop and Avro. This allows correctly setup of CLASSPATH in kafka-run-class.sh when running kafka connect hdfs connector. --> - org.apache.kafka:connect-api org.mortbay.jetty:* com.sun.jersey:* org.eclipse.jetty.aggregate:jetty-all diff --git a/src/assembly/package.xml b/src/assembly/package.xml index 3d9dae049..23024a3cb 100644 --- a/src/assembly/package.xml +++ b/src/assembly/package.xml @@ -38,7 +38,6 @@ shipped with Confluent platform and other dependencies such as Hadoop and Avro. This allows correctly setup of CLASSPATH in kafka-run-class.sh when running kafka connect hdfs connector. --> - org.apache.kafka:connect-api org.mortbay.jetty:* com.sun.jersey:* org.eclipse.jetty.aggregate:jetty-all From da57a4d561220caa7f2c8464fc6048c1a9fc96a2 Mon Sep 17 00:00:00 2001 From: Konstantine Karantasis Date: Sat, 22 Apr 2017 14:13:14 -0700 Subject: [PATCH 14/58] Refactor avro and parquet formats plus required utils --- .../io/confluent/connect/hdfs/FileUtils.java | 4 - .../io/confluent/connect/hdfs/Format.java | 13 ++- .../confluent/connect/hdfs/RecordWriter.java | 9 +-- .../connect/hdfs/RecordWriterProvider.java | 9 +-- .../connect/hdfs/SchemaFileReader.java | 6 +- .../connect/hdfs/TopicPartitionWriter.java | 24 +++--- .../connect/hdfs/avro/AvroFileReader.java | 41 ++++++---- .../connect/hdfs/avro/AvroFormat.java | 19 ++++- .../connect/hdfs/avro/AvroHiveFactory.java | 36 +++++++++ .../connect/hdfs/avro/AvroHiveUtil.java | 24 +++--- .../hdfs/avro/AvroRecordWriterProvider.java | 62 +++++++++------ .../hdfs/hive/HiveSchemaConverter.java | 79 +------------------ .../hdfs/parquet/ParquetFileReader.java | 41 ++++++---- .../connect/hdfs/parquet/ParquetFormat.java | 21 ++++- .../hdfs/parquet/ParquetHiveFactory.java | 35 ++++++++ .../connect/hdfs/parquet/ParquetHiveUtil.java | 25 +++--- .../parquet/ParquetRecordWriterProvider.java | 51 +++++++----- .../hdfs/HdfsSinkTaskTestWithSecureHDFS.java | 1 + .../connect/hdfs/utils/MemoryFormat.java | 18 +++++ .../hdfs/utils/MemoryRecordWriter.java | 9 ++- .../utils/MemoryRecordWriterProvider.java | 8 +- 21 files changed, 305 insertions(+), 230 deletions(-) create mode 100644 src/main/java/io/confluent/connect/hdfs/avro/AvroHiveFactory.java create mode 100644 src/main/java/io/confluent/connect/hdfs/parquet/ParquetHiveFactory.java diff --git a/src/main/java/io/confluent/connect/hdfs/FileUtils.java b/src/main/java/io/confluent/connect/hdfs/FileUtils.java index e5ae7a790..7a99adcd9 100644 --- a/src/main/java/io/confluent/connect/hdfs/FileUtils.java +++ b/src/main/java/io/confluent/connect/hdfs/FileUtils.java @@ -52,10 +52,6 @@ public static String fileName(String url, String topicsDir, TopicPartition topic return url + "/" + topicsDir + "/" + topic + "/" + partition + "/" + name; } - public static String hiveDirectoryName(String url, String topicsDir, String topic) { - return url + "/" + topicsDir + "/" + topic + "/"; - } - public static String fileName(String url, String topicsDir, String directory, String name) { return url + "/" + topicsDir + "/" + directory + "/" + name; } diff --git a/src/main/java/io/confluent/connect/hdfs/Format.java b/src/main/java/io/confluent/connect/hdfs/Format.java index f1ae43978..30862aa04 100644 --- a/src/main/java/io/confluent/connect/hdfs/Format.java +++ b/src/main/java/io/confluent/connect/hdfs/Format.java @@ -14,17 +14,26 @@ package io.confluent.connect.hdfs; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.kafka.common.config.AbstractConfig; + import io.confluent.connect.avro.AvroData; import io.confluent.connect.hdfs.hive.HiveMetaStore; import io.confluent.connect.hdfs.hive.HiveUtil; +import io.confluent.connect.storage.hive.HiveFactory; @Deprecated public interface Format - extends io.confluent.connect.storage.Format { + extends io.confluent.connect.storage.format.Format { @Override RecordWriterProvider getRecordWriterProvider(); + @Override SchemaFileReader getSchemaFileReader(AvroData avroData); - @Override + HiveUtil getHiveUtil(HdfsSinkConnectorConfig config, AvroData avroData, HiveMetaStore hiveMetaStore); + + @Override + HiveFactory getHiveFactory(); } diff --git a/src/main/java/io/confluent/connect/hdfs/RecordWriter.java b/src/main/java/io/confluent/connect/hdfs/RecordWriter.java index 9497df34d..4dfa0dad0 100644 --- a/src/main/java/io/confluent/connect/hdfs/RecordWriter.java +++ b/src/main/java/io/confluent/connect/hdfs/RecordWriter.java @@ -16,12 +16,11 @@ package io.confluent.connect.hdfs; -import java.io.IOException; - @Deprecated -public interface RecordWriter extends io.confluent.connect.storage.RecordWriter { +public interface RecordWriter extends io.confluent.connect.storage.format.RecordWriter { @Override - void write(V value) throws IOException; + void write(V value); + @Override - void close() throws IOException; + void close(); } diff --git a/src/main/java/io/confluent/connect/hdfs/RecordWriterProvider.java b/src/main/java/io/confluent/connect/hdfs/RecordWriterProvider.java index 0ce22ebe4..91613e0cd 100644 --- a/src/main/java/io/confluent/connect/hdfs/RecordWriterProvider.java +++ b/src/main/java/io/confluent/connect/hdfs/RecordWriterProvider.java @@ -17,15 +17,14 @@ import org.apache.hadoop.conf.Configuration; import org.apache.kafka.connect.sink.SinkRecord; -import java.io.IOException; - import io.confluent.connect.avro.AvroData; @Deprecated -public interface RecordWriterProvider extends io.confluent.connect.storage.RecordWriterProvider { +public interface RecordWriterProvider + extends io.confluent.connect.storage.format.RecordWriterProvider { @Override String getExtension(); + @Override - RecordWriter getRecordWriter(Configuration conf, String fileName, SinkRecord record, AvroData avroData) - throws IOException; + RecordWriter getRecordWriter(Configuration conf, String fileName, SinkRecord record, AvroData avroData); } diff --git a/src/main/java/io/confluent/connect/hdfs/SchemaFileReader.java b/src/main/java/io/confluent/connect/hdfs/SchemaFileReader.java index 9d5481820..88c261520 100644 --- a/src/main/java/io/confluent/connect/hdfs/SchemaFileReader.java +++ b/src/main/java/io/confluent/connect/hdfs/SchemaFileReader.java @@ -22,9 +22,9 @@ import java.util.Collection; @Deprecated -public interface SchemaFileReader extends io.confluent.connect.storage.SchemaFileReader { +public interface SchemaFileReader extends io.confluent.connect.storage.format.SchemaFileReader { @Override - Schema getSchema(Configuration conf, Path path) throws IOException; + Schema getSchema(Configuration conf, Path path); @Override - Collection readData(Configuration conf, Path path) throws IOException; + Collection readData(Configuration conf, Path path); } diff --git a/src/main/java/io/confluent/connect/hdfs/TopicPartitionWriter.java b/src/main/java/io/confluent/connect/hdfs/TopicPartitionWriter.java index 630d8e30d..160e0a8eb 100644 --- a/src/main/java/io/confluent/connect/hdfs/TopicPartitionWriter.java +++ b/src/main/java/io/confluent/connect/hdfs/TopicPartitionWriter.java @@ -450,21 +450,17 @@ private void resume() { private RecordWriter getWriter(SinkRecord record, String encodedPartition) throws ConnectException { - try { - if (writers.containsKey(encodedPartition)) { - return writers.get(encodedPartition); - } - String tempFile = getTempFile(encodedPartition); - RecordWriter writer = writerProvider.getRecordWriter(conf, tempFile, record, avroData); - writers.put(encodedPartition, writer); - if (hiveIntegration && !hivePartitions.contains(encodedPartition)) { - addHivePartition(encodedPartition); - hivePartitions.add(encodedPartition); - } - return writer; - } catch (IOException e) { - throw new ConnectException(e); + if (writers.containsKey(encodedPartition)) { + return writers.get(encodedPartition); + } + String tempFile = getTempFile(encodedPartition); + RecordWriter writer = writerProvider.getRecordWriter(conf, tempFile, record, avroData); + writers.put(encodedPartition, writer); + if (hiveIntegration && !hivePartitions.contains(encodedPartition)) { + addHivePartition(encodedPartition); + hivePartitions.add(encodedPartition); } + return writer; } private String getTempFile(String encodedPartition) { diff --git a/src/main/java/io/confluent/connect/hdfs/avro/AvroFileReader.java b/src/main/java/io/confluent/connect/hdfs/avro/AvroFileReader.java index 9773b1d7f..dc7f545dc 100644 --- a/src/main/java/io/confluent/connect/hdfs/avro/AvroFileReader.java +++ b/src/main/java/io/confluent/connect/hdfs/avro/AvroFileReader.java @@ -23,6 +23,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.errors.ConnectException; import java.io.IOException; import java.util.ArrayList; @@ -31,8 +32,8 @@ import io.confluent.connect.avro.AvroData; import io.confluent.connect.hdfs.SchemaFileReader; -public class AvroFileReader implements SchemaFileReader { - +public class AvroFileReader implements SchemaFileReader, + io.confluent.connect.storage.format.SchemaFileReader { private AvroData avroData; public AvroFileReader(AvroData avroData) { @@ -40,25 +41,33 @@ public AvroFileReader(AvroData avroData) { } @Override - public Schema getSchema(Configuration conf, Path path) throws IOException { - SeekableInput input = new FsInput(path, conf); - DatumReader reader = new GenericDatumReader<>(); - FileReader fileReader = DataFileReader.openReader(input, reader); - org.apache.avro.Schema schema = fileReader.getSchema(); - fileReader.close(); - return avroData.toConnectSchema(schema); + public Schema getSchema(Configuration conf, Path path) { + try { + SeekableInput input = new FsInput(path, conf); + DatumReader reader = new GenericDatumReader<>(); + FileReader fileReader = DataFileReader.openReader(input, reader); + org.apache.avro.Schema schema = fileReader.getSchema(); + fileReader.close(); + return avroData.toConnectSchema(schema); + } catch (IOException e) { + throw new ConnectException(e); + } } @Override - public Collection readData(Configuration conf, Path path) throws IOException { + public Collection readData(Configuration conf, Path path) { ArrayList collection = new ArrayList<>(); - SeekableInput input = new FsInput(path, conf); - DatumReader reader = new GenericDatumReader<>(); - FileReader fileReader = DataFileReader.openReader(input, reader); - for (Object object: fileReader) { - collection.add(object); + try { + SeekableInput input = new FsInput(path, conf); + DatumReader reader = new GenericDatumReader<>(); + FileReader fileReader = DataFileReader.openReader(input, reader); + for (Object object : fileReader) { + collection.add(object); + } + fileReader.close(); + } catch (IOException e) { + throw new ConnectException(e); } - fileReader.close(); return collection; } } diff --git a/src/main/java/io/confluent/connect/hdfs/avro/AvroFormat.java b/src/main/java/io/confluent/connect/hdfs/avro/AvroFormat.java index b5a047b8b..2c55e2a9f 100644 --- a/src/main/java/io/confluent/connect/hdfs/avro/AvroFormat.java +++ b/src/main/java/io/confluent/connect/hdfs/avro/AvroFormat.java @@ -10,10 +10,13 @@ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express * or implied. See the License for the specific language governing permissions and limitations under * the License. - **/ + */ package io.confluent.connect.hdfs.avro; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; + import io.confluent.connect.avro.AvroData; import io.confluent.connect.hdfs.Format; import io.confluent.connect.hdfs.HdfsSinkConnectorConfig; @@ -21,18 +24,28 @@ import io.confluent.connect.hdfs.SchemaFileReader; import io.confluent.connect.hdfs.hive.HiveMetaStore; import io.confluent.connect.hdfs.hive.HiveUtil; +import io.confluent.connect.storage.hive.HiveFactory; -public class AvroFormat implements Format { +public class AvroFormat implements Format, io.confluent.connect.storage.format.Format { + @Override public RecordWriterProvider getRecordWriterProvider() { return new AvroRecordWriterProvider(); } + @Override public SchemaFileReader getSchemaFileReader(AvroData avroData) { return new AvroFileReader(avroData); } + @Deprecated + @Override public HiveUtil getHiveUtil(HdfsSinkConnectorConfig config, AvroData avroData, HiveMetaStore hiveMetaStore) { - return new AvroHiveUtil(config, avroData, hiveMetaStore); + return (HiveUtil) getHiveFactory().createHiveUtil(config, avroData, hiveMetaStore); + } + + @Override + public HiveFactory getHiveFactory() { + return new AvroHiveFactory(); } } diff --git a/src/main/java/io/confluent/connect/hdfs/avro/AvroHiveFactory.java b/src/main/java/io/confluent/connect/hdfs/avro/AvroHiveFactory.java new file mode 100644 index 000000000..3c30ae8c3 --- /dev/null +++ b/src/main/java/io/confluent/connect/hdfs/avro/AvroHiveFactory.java @@ -0,0 +1,36 @@ +/* + * Copyright 2016 Confluent Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file exceptin compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.confluent.connect.hdfs.avro; + +import io.confluent.connect.avro.AvroData; +import io.confluent.connect.hdfs.HdfsSinkConnectorConfig; +import io.confluent.connect.hdfs.hive.HiveMetaStore; +import io.confluent.connect.hdfs.hive.HiveUtil; +import io.confluent.connect.storage.hive.HiveFactory; + +public class AvroHiveFactory implements HiveFactory { + @Override + public io.confluent.connect.storage.hive.HiveUtil createHiveUtil(HdfsSinkConnectorConfig config, AvroData avroData, + io.confluent.connect.storage.hive.HiveMetaStore hiveMetaStore) { + return createHiveUtil(config, avroData, (HiveMetaStore) hiveMetaStore); + } + + @Deprecated + public HiveUtil createHiveUtil(HdfsSinkConnectorConfig config, AvroData avroData, HiveMetaStore hiveMetaStore) { + return new AvroHiveUtil(config, avroData, hiveMetaStore); + } +} diff --git a/src/main/java/io/confluent/connect/hdfs/avro/AvroHiveUtil.java b/src/main/java/io/confluent/connect/hdfs/avro/AvroHiveUtil.java index 85d9de8fb..d34d8eaed 100644 --- a/src/main/java/io/confluent/connect/hdfs/avro/AvroHiveUtil.java +++ b/src/main/java/io/confluent/connect/hdfs/avro/AvroHiveUtil.java @@ -10,7 +10,7 @@ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express * or implied. See the License for the specific language governing permissions and limitations under * the License. - **/ + */ package io.confluent.connect.hdfs.avro; @@ -24,22 +24,20 @@ import java.util.List; import io.confluent.connect.avro.AvroData; -import io.confluent.connect.hdfs.FileUtils; -import io.confluent.connect.hdfs.errors.HiveMetaStoreException; +import io.confluent.connect.hdfs.HdfsSinkConnectorConfig; import io.confluent.connect.hdfs.hive.HiveMetaStore; -import io.confluent.connect.hdfs.hive.HiveSchemaConverter; import io.confluent.connect.hdfs.hive.HiveUtil; import io.confluent.connect.hdfs.partitioner.Partitioner; -import io.confluent.connect.hdfs.HdfsSinkConnectorConfig; +import io.confluent.connect.storage.errors.HiveMetaStoreException; +import io.confluent.connect.storage.hive.HiveSchemaConverter; public class AvroHiveUtil extends HiveUtil { - private static final String avroSerde = "org.apache.hadoop.hive.serde2.avro.AvroSerDe"; - private static final String avroInputFormat = "org.apache.hadoop.hive.ql.io.avro.AvroContainerInputFormat"; - private static final String avroOutputFormat = "org.apache.hadoop.hive.ql.io.avro.AvroContainerOutputFormat"; + private static final String AVRO_SERDE = "org.apache.hadoop.hive.serde2.avro.AvroSerDe"; + private static final String AVRO_INPUT_FORMAT = "org.apache.hadoop.hive.ql.io.avro.AvroContainerInputFormat"; + private static final String AVRO_OUTPUT_FORMAT = "org.apache.hadoop.hive.ql.io.avro.AvroContainerOutputFormat"; private static final String AVRO_SCHEMA_LITERAL = "avro.schema.literal"; - public AvroHiveUtil(HdfsSinkConnectorConfig connectorConfig, AvroData avroData, HiveMetaStore hiveMetaStore) { super(connectorConfig, avroData, hiveMetaStore); } @@ -63,12 +61,12 @@ private Table constructAvroTable(String database, String tableName, Schema schem Table table = newTable(database, tableName); table.setTableType(TableType.EXTERNAL_TABLE); table.getParameters().put("EXTERNAL", "TRUE"); - String tablePath = FileUtils.hiveDirectoryName(url, topicsDir, tableName); + String tablePath = hiveDirectoryName(url, topicsDir, tableName); table.setDataLocation(new Path(tablePath)); - table.setSerializationLib(avroSerde); + table.setSerializationLib(AVRO_SERDE); try { - table.setInputFormatClass(avroInputFormat); - table.setOutputFormatClass(avroOutputFormat); + table.setInputFormatClass(AVRO_INPUT_FORMAT); + table.setOutputFormatClass(AVRO_OUTPUT_FORMAT); } catch (HiveException e) { throw new HiveMetaStoreException("Cannot find input/output format:", e); } diff --git a/src/main/java/io/confluent/connect/hdfs/avro/AvroRecordWriterProvider.java b/src/main/java/io/confluent/connect/hdfs/avro/AvroRecordWriterProvider.java index bbf89b3e0..634e65ba3 100644 --- a/src/main/java/io/confluent/connect/hdfs/avro/AvroRecordWriterProvider.java +++ b/src/main/java/io/confluent/connect/hdfs/avro/AvroRecordWriterProvider.java @@ -1,18 +1,16 @@ /** * Copyright 2015 Confluent Inc. * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - **/ + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ package io.confluent.connect.hdfs.avro; @@ -24,6 +22,7 @@ import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.Path; import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.sink.SinkRecord; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -34,8 +33,8 @@ import io.confluent.connect.hdfs.RecordWriter; import io.confluent.connect.hdfs.RecordWriterProvider; -public class AvroRecordWriterProvider implements RecordWriterProvider { - +public class AvroRecordWriterProvider implements RecordWriterProvider, + io.confluent.connect.storage.format.RecordWriterProvider { private static final Logger log = LoggerFactory.getLogger(AvroRecordWriterProvider.class); private final static String EXTENSION = ".avro"; @@ -46,33 +45,44 @@ public String getExtension() { @Override public RecordWriter getRecordWriter(Configuration conf, final String fileName, - SinkRecord record, final AvroData avroData) - throws IOException { + SinkRecord record, final AvroData avroData) { DatumWriter datumWriter = new GenericDatumWriter<>(); final DataFileWriter writer = new DataFileWriter<>(datumWriter); Path path = new Path(fileName); final Schema schema = record.valueSchema(); - final FSDataOutputStream out = path.getFileSystem(conf).create(path); - org.apache.avro.Schema avroSchema = avroData.fromConnectSchema(schema); - writer.create(avroSchema, out); + try { + final FSDataOutputStream out = path.getFileSystem(conf).create(path); + org.apache.avro.Schema avroSchema = avroData.fromConnectSchema(schema); + writer.create(avroSchema, out); + } catch (IOException e) { + throw new ConnectException(e); + } - return new RecordWriter(){ + return new RecordWriter() { @Override - public void write(SinkRecord record) throws IOException { + public void write(SinkRecord record) { log.trace("Sink record: {}", record.toString()); 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); + try { + // 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); + } catch (IOException e) { + throw new ConnectException(e); + } } @Override - public void close() throws IOException { - writer.close(); + public void close() { + try { + writer.close(); + } catch (IOException e) { + throw new ConnectException(e); + } } }; } diff --git a/src/main/java/io/confluent/connect/hdfs/hive/HiveSchemaConverter.java b/src/main/java/io/confluent/connect/hdfs/hive/HiveSchemaConverter.java index 3366dee48..369b0655d 100644 --- a/src/main/java/io/confluent/connect/hdfs/hive/HiveSchemaConverter.java +++ b/src/main/java/io/confluent/connect/hdfs/hive/HiveSchemaConverter.java @@ -14,81 +14,6 @@ package io.confluent.connect.hdfs.hive; -import org.apache.hadoop.hive.metastore.api.FieldSchema; -import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; -import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; -import org.apache.kafka.connect.data.Field; -import org.apache.kafka.connect.data.Schema; -import org.apache.kafka.connect.data.Schema.Type; - -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -public class HiveSchemaConverter { - - private static final Map TYPE_TO_TYPEINFO; - - static { - TYPE_TO_TYPEINFO = new HashMap<>(); - TYPE_TO_TYPEINFO.put(Type.BOOLEAN, TypeInfoFactory.booleanTypeInfo); - TYPE_TO_TYPEINFO.put(Type.INT8, TypeInfoFactory.byteTypeInfo); - TYPE_TO_TYPEINFO.put(Type.INT16, TypeInfoFactory.shortTypeInfo); - TYPE_TO_TYPEINFO.put(Type.INT32, TypeInfoFactory.intTypeInfo); - TYPE_TO_TYPEINFO.put(Type.INT64, TypeInfoFactory.longTypeInfo); - TYPE_TO_TYPEINFO.put(Type.FLOAT32, TypeInfoFactory.floatTypeInfo); - TYPE_TO_TYPEINFO.put(Type.FLOAT64, TypeInfoFactory.doubleTypeInfo); - TYPE_TO_TYPEINFO.put(Type.BYTES, TypeInfoFactory.binaryTypeInfo); - TYPE_TO_TYPEINFO.put(Type.STRING, TypeInfoFactory.stringTypeInfo); - } - - public static List convertSchema(Schema schema) { - List columns = new ArrayList<>(); - if (Schema.Type.STRUCT.equals(schema.type())) { - for (Field field: schema.fields()) { - columns.add(new FieldSchema( - field.name(), convert(field.schema()).getTypeName(), field.schema().doc())); - } - } - return columns; - } - - public static TypeInfo convert(Schema schema) { - // TODO: throw an error on recursive types - switch (schema.type()) { - case STRUCT: - return convertStruct(schema); - case ARRAY: - return convertArray(schema); - case MAP: - return convertMap(schema); - default: - return convertPrimitive(schema); - } - } - - public static TypeInfo convertStruct(Schema schema) { - final List fields = schema.fields(); - final List names = new ArrayList<>(fields.size()); - final List types = new ArrayList<>(fields.size()); - for (Field field : fields) { - names.add(field.name()); - types.add(convert(field.schema())); - } - return TypeInfoFactory.getStructTypeInfo(names, types); - } - - public static TypeInfo convertArray(Schema schema) { - return TypeInfoFactory.getListTypeInfo(convert(schema.valueSchema())); - } - - public static TypeInfo convertMap(Schema schema) { - return TypeInfoFactory.getMapTypeInfo( - convert(schema.keySchema()), convert(schema.valueSchema())); - } - - public static TypeInfo convertPrimitive(Schema schema) { - return TYPE_TO_TYPEINFO.get(schema.type()); - } +@Deprecated +public class HiveSchemaConverter extends io.confluent.connect.storage.hive.HiveSchemaConverter { } diff --git a/src/main/java/io/confluent/connect/hdfs/parquet/ParquetFileReader.java b/src/main/java/io/confluent/connect/hdfs/parquet/ParquetFileReader.java index d08e30171..91e8f6468 100644 --- a/src/main/java/io/confluent/connect/hdfs/parquet/ParquetFileReader.java +++ b/src/main/java/io/confluent/connect/hdfs/parquet/ParquetFileReader.java @@ -18,6 +18,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.errors.ConnectException; import org.apache.parquet.avro.AvroReadSupport; import org.apache.parquet.hadoop.ParquetReader; @@ -28,8 +29,8 @@ import io.confluent.connect.avro.AvroData; import io.confluent.connect.hdfs.SchemaFileReader; -public class ParquetFileReader implements SchemaFileReader { - +public class ParquetFileReader implements SchemaFileReader, + io.confluent.connect.storage.format.SchemaFileReader { private AvroData avroData; public ParquetFileReader(AvroData avroData) { @@ -37,30 +38,38 @@ public ParquetFileReader(AvroData avroData) { } @Override - public Schema getSchema(Configuration conf, Path path) throws IOException { + public Schema getSchema(Configuration conf, Path path) { AvroReadSupport readSupport = new AvroReadSupport<>(); ParquetReader.Builder builder = ParquetReader.builder(readSupport, path); - ParquetReader parquetReader = builder.withConf(conf).build(); - GenericRecord record; - Schema schema = null; - while ((record = parquetReader.read()) != null) { - schema = avroData.toConnectSchema(record.getSchema()); + try { + ParquetReader parquetReader = builder.withConf(conf).build(); + GenericRecord record; + Schema schema = null; + while ((record = parquetReader.read()) != null) { + schema = avroData.toConnectSchema(record.getSchema()); + } + parquetReader.close(); + return schema; + } catch (IOException e) { + throw new ConnectException(e); } - parquetReader.close(); - return schema; } @Override - public Collection readData(Configuration conf, Path path) throws IOException { + public Collection readData(Configuration conf, Path path) { Collection result = new ArrayList<>(); AvroReadSupport readSupport = new AvroReadSupport<>(); ParquetReader.Builder builder = ParquetReader.builder(readSupport, path); - ParquetReader parquetReader = builder.withConf(conf).build(); - GenericRecord record; - while ((record = parquetReader.read()) != null) { - result.add(record); + try { + ParquetReader parquetReader = builder.withConf(conf).build(); + GenericRecord record; + while ((record = parquetReader.read()) != null) { + result.add(record); + } + parquetReader.close(); + } catch (IOException e) { + throw new ConnectException(e); } - parquetReader.close(); return result; } } diff --git a/src/main/java/io/confluent/connect/hdfs/parquet/ParquetFormat.java b/src/main/java/io/confluent/connect/hdfs/parquet/ParquetFormat.java index 2d930e5aa..e7334e53f 100644 --- a/src/main/java/io/confluent/connect/hdfs/parquet/ParquetFormat.java +++ b/src/main/java/io/confluent/connect/hdfs/parquet/ParquetFormat.java @@ -10,10 +10,13 @@ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express * or implied. See the License for the specific language governing permissions and limitations under * the License. - **/ + */ package io.confluent.connect.hdfs.parquet; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; + import io.confluent.connect.avro.AvroData; import io.confluent.connect.hdfs.Format; import io.confluent.connect.hdfs.HdfsSinkConnectorConfig; @@ -21,17 +24,29 @@ import io.confluent.connect.hdfs.SchemaFileReader; import io.confluent.connect.hdfs.hive.HiveMetaStore; import io.confluent.connect.hdfs.hive.HiveUtil; +import io.confluent.connect.storage.hive.HiveFactory; + +public class ParquetFormat implements Format, + io.confluent.connect.storage.format.Format { -public class ParquetFormat implements Format { + @Override public RecordWriterProvider getRecordWriterProvider() { return new ParquetRecordWriterProvider(); } + @Override public SchemaFileReader getSchemaFileReader(AvroData avroData) { return new ParquetFileReader(avroData); } + @Deprecated + @Override public HiveUtil getHiveUtil(HdfsSinkConnectorConfig config, AvroData avroData, HiveMetaStore hiveMetaStore) { - return new ParquetHiveUtil(config, avroData, hiveMetaStore); + return (HiveUtil) getHiveFactory().createHiveUtil(config, avroData, hiveMetaStore); + } + + @Override + public HiveFactory getHiveFactory() { + return new ParquetHiveFactory(); } } diff --git a/src/main/java/io/confluent/connect/hdfs/parquet/ParquetHiveFactory.java b/src/main/java/io/confluent/connect/hdfs/parquet/ParquetHiveFactory.java new file mode 100644 index 000000000..a88789dde --- /dev/null +++ b/src/main/java/io/confluent/connect/hdfs/parquet/ParquetHiveFactory.java @@ -0,0 +1,35 @@ +/* + * Copyright 2016 Confluent Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file exceptin compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.confluent.connect.hdfs.parquet; + +import io.confluent.connect.avro.AvroData; +import io.confluent.connect.hdfs.HdfsSinkConnectorConfig; +import io.confluent.connect.storage.hive.HiveFactory; +import io.confluent.connect.hdfs.hive.HiveMetaStore; +import io.confluent.connect.storage.hive.HiveUtil; + +public class ParquetHiveFactory implements HiveFactory { + @Override + public HiveUtil createHiveUtil(HdfsSinkConnectorConfig config, AvroData avroData, + io.confluent.connect.storage.hive.HiveMetaStore hiveMetaStore) { + return createHiveUtil(config, avroData, (HiveMetaStore) hiveMetaStore); + } + + public HiveUtil createHiveUtil(HdfsSinkConnectorConfig config, AvroData avroData, HiveMetaStore hiveMetaStore) { + return new ParquetHiveUtil(config, avroData, hiveMetaStore); + } +} diff --git a/src/main/java/io/confluent/connect/hdfs/parquet/ParquetHiveUtil.java b/src/main/java/io/confluent/connect/hdfs/parquet/ParquetHiveUtil.java index 9fa6aaa53..0dfa3af9e 100644 --- a/src/main/java/io/confluent/connect/hdfs/parquet/ParquetHiveUtil.java +++ b/src/main/java/io/confluent/connect/hdfs/parquet/ParquetHiveUtil.java @@ -1,18 +1,16 @@ /** * Copyright 2015 Confluent Inc. * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - **/ + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ package io.confluent.connect.hdfs.parquet; @@ -26,13 +24,12 @@ import java.util.List; import io.confluent.connect.avro.AvroData; -import io.confluent.connect.hdfs.FileUtils; +import io.confluent.connect.hdfs.HdfsSinkConnectorConfig; import io.confluent.connect.hdfs.hive.HiveMetaStore; -import io.confluent.connect.hdfs.hive.HiveSchemaConverter; import io.confluent.connect.hdfs.hive.HiveUtil; import io.confluent.connect.hdfs.partitioner.Partitioner; -import io.confluent.connect.hdfs.HdfsSinkConnectorConfig; -import io.confluent.connect.hdfs.errors.HiveMetaStoreException; +import io.confluent.connect.storage.errors.HiveMetaStoreException; +import io.confluent.connect.storage.hive.HiveSchemaConverter; public class ParquetHiveUtil extends HiveUtil { @@ -58,7 +55,7 @@ private Table constructParquetTable(String database, String tableName, Schema sc Table table = newTable(database, tableName); table.setTableType(TableType.EXTERNAL_TABLE); table.getParameters().put("EXTERNAL", "TRUE"); - String tablePath = FileUtils.hiveDirectoryName(url, topicsDir, tableName); + String tablePath = hiveDirectoryName(url, topicsDir, tableName); table.setDataLocation(new Path(tablePath)); table.setSerializationLib(getHiveParquetSerde()); try { diff --git a/src/main/java/io/confluent/connect/hdfs/parquet/ParquetRecordWriterProvider.java b/src/main/java/io/confluent/connect/hdfs/parquet/ParquetRecordWriterProvider.java index eb4d1c396..73a31601f 100644 --- a/src/main/java/io/confluent/connect/hdfs/parquet/ParquetRecordWriterProvider.java +++ b/src/main/java/io/confluent/connect/hdfs/parquet/ParquetRecordWriterProvider.java @@ -10,13 +10,15 @@ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express * or implied. See the License for the specific language governing permissions and limitations under * the License. - **/ + */ + package io.confluent.connect.hdfs.parquet; import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; +import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.sink.SinkRecord; import org.apache.parquet.avro.AvroParquetWriter; import org.apache.parquet.hadoop.ParquetWriter; @@ -28,8 +30,8 @@ import io.confluent.connect.hdfs.RecordWriterProvider; import io.confluent.connect.hdfs.RecordWriter; -public class ParquetRecordWriterProvider implements RecordWriterProvider { - +public class ParquetRecordWriterProvider implements RecordWriterProvider, + io.confluent.connect.storage.format.RecordWriterProvider { private final static String EXTENSION = ".parquet"; @Override @@ -38,9 +40,8 @@ public String getExtension() { } @Override - public RecordWriter getRecordWriter( - Configuration conf, final String fileName, SinkRecord record, final AvroData avroData) - throws IOException { + public RecordWriter getRecordWriter(Configuration conf, final String fileName, SinkRecord record, + final AvroData avroData) { final Schema avroSchema = avroData.fromConnectSchema(record.valueSchema()); CompressionCodecName compressionCodecName = CompressionCodecName.SNAPPY; @@ -48,20 +49,32 @@ public RecordWriter getRecordWriter( int pageSize = 64 * 1024; Path path = new Path(fileName); - final ParquetWriter writer = - new AvroParquetWriter<>(path, avroSchema, compressionCodecName, blockSize, pageSize, true, conf); + try { + final ParquetWriter writer = + new AvroParquetWriter<>(path, avroSchema, compressionCodecName, blockSize, pageSize, true, conf); - return new RecordWriter() { - @Override - public void write(SinkRecord record) throws IOException { - Object value = avroData.fromConnectData(record.valueSchema(), record.value()); - writer.write((GenericRecord) value); - } + return new RecordWriter() { + @Override + public void write(SinkRecord record) { + Object value = avroData.fromConnectData(record.valueSchema(), record.value()); + try { + writer.write((GenericRecord) value); + } catch (IOException e) { + throw new ConnectException(e); + } + } - @Override - public void close() throws IOException { - writer.close(); - } - }; + @Override + public void close() { + try { + writer.close(); + } catch (IOException e) { + throw new ConnectException(e); + } + } + }; + } catch (IOException e) { + throw new ConnectException(e); + } } } diff --git a/src/test/java/io/confluent/connect/hdfs/HdfsSinkTaskTestWithSecureHDFS.java b/src/test/java/io/confluent/connect/hdfs/HdfsSinkTaskTestWithSecureHDFS.java index dbc6ab679..640535550 100644 --- a/src/test/java/io/confluent/connect/hdfs/HdfsSinkTaskTestWithSecureHDFS.java +++ b/src/test/java/io/confluent/connect/hdfs/HdfsSinkTaskTestWithSecureHDFS.java @@ -14,6 +14,7 @@ package io.confluent.connect.hdfs; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.connect.data.Schema; diff --git a/src/test/java/io/confluent/connect/hdfs/utils/MemoryFormat.java b/src/test/java/io/confluent/connect/hdfs/utils/MemoryFormat.java index bf077ab99..0c978946b 100644 --- a/src/test/java/io/confluent/connect/hdfs/utils/MemoryFormat.java +++ b/src/test/java/io/confluent/connect/hdfs/utils/MemoryFormat.java @@ -1,5 +1,7 @@ package io.confluent.connect.hdfs.utils; +import org.apache.kafka.common.config.AbstractConfig; + import io.confluent.connect.avro.AvroData; import io.confluent.connect.hdfs.Format; import io.confluent.connect.hdfs.HdfsSinkConnectorConfig; @@ -7,18 +9,34 @@ import io.confluent.connect.hdfs.SchemaFileReader; import io.confluent.connect.hdfs.hive.HiveMetaStore; import io.confluent.connect.hdfs.hive.HiveUtil; +import io.confluent.connect.storage.hive.HiveFactory; public class MemoryFormat implements Format { + @Override public RecordWriterProvider getRecordWriterProvider() { return new MemoryRecordWriterProvider(); } + @Override public SchemaFileReader getSchemaFileReader(AvroData avroData) { return null; } + @Override public HiveUtil getHiveUtil(HdfsSinkConnectorConfig config, AvroData avroData, HiveMetaStore hiveMetaStore) { return null; } + + @Override + public HiveFactory getHiveFactory() { + return new HiveFactory() { + @Override + public io.confluent.connect.storage.hive.HiveUtil createHiveUtil(HdfsSinkConnectorConfig abstractConfig, + AvroData avroData, + io.confluent.connect.storage.hive.HiveMetaStore hiveMetaStore) { + return null; + } + }; + } } diff --git a/src/test/java/io/confluent/connect/hdfs/utils/MemoryRecordWriter.java b/src/test/java/io/confluent/connect/hdfs/utils/MemoryRecordWriter.java index 7ebbc6cec..83b941348 100644 --- a/src/test/java/io/confluent/connect/hdfs/utils/MemoryRecordWriter.java +++ b/src/test/java/io/confluent/connect/hdfs/utils/MemoryRecordWriter.java @@ -14,6 +14,7 @@ package io.confluent.connect.hdfs.utils; +import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.sink.SinkRecord; import java.io.IOException; @@ -38,20 +39,20 @@ public MemoryRecordWriter(String filename) { } @Override - public void write(SinkRecord record) throws IOException { + public void write(SinkRecord record) { if (failure == Failure.writeFailure) { failure = Failure.noFailure; - throw new IOException("write failed."); + throw new ConnectException("write failed."); } data.get(filename).add(record); } @Override - public void close() throws IOException { + public void close() { if (failure == Failure.closeFailure) { failure = Failure.noFailure; - throw new IOException("close failed."); + throw new ConnectException("close failed."); } } diff --git a/src/test/java/io/confluent/connect/hdfs/utils/MemoryRecordWriterProvider.java b/src/test/java/io/confluent/connect/hdfs/utils/MemoryRecordWriterProvider.java index d173eb526..2e49cf682 100644 --- a/src/test/java/io/confluent/connect/hdfs/utils/MemoryRecordWriterProvider.java +++ b/src/test/java/io/confluent/connect/hdfs/utils/MemoryRecordWriterProvider.java @@ -19,7 +19,6 @@ import org.apache.hadoop.conf.Configuration; import org.apache.kafka.connect.sink.SinkRecord; -import java.io.IOException; import java.util.LinkedList; import java.util.List; import java.util.Map; @@ -36,10 +35,8 @@ public String getExtension() { } @Override - public RecordWriter getRecordWriter( - Configuration conf, final String fileName, SinkRecord record, final AvroData avroData) - throws IOException { - + public RecordWriter getRecordWriter(Configuration conf, final String fileName, SinkRecord record, + final AvroData avroData) { final Map> data = Data.getData(); if (!data.containsKey(fileName)) { @@ -49,5 +46,4 @@ public RecordWriter getRecordWriter( return new MemoryRecordWriter(fileName); } - } From 93275a85a8c5933aed15981357f6a961f549f6db Mon Sep 17 00:00:00 2001 From: Konstantine Karantasis Date: Fri, 16 Dec 2016 16:44:59 -0800 Subject: [PATCH 15/58] Refactor base test class. --- .../hdfs/HdfsSinkConnectorTestBase.java | 131 ++---------------- .../hdfs/avro/TopicPartitionWriterTest.java | 2 +- 2 files changed, 9 insertions(+), 124 deletions(-) diff --git a/src/test/java/io/confluent/connect/hdfs/HdfsSinkConnectorTestBase.java b/src/test/java/io/confluent/connect/hdfs/HdfsSinkConnectorTestBase.java index 835a0b83d..105cf469c 100644 --- a/src/test/java/io/confluent/connect/hdfs/HdfsSinkConnectorTestBase.java +++ b/src/test/java/io/confluent/connect/hdfs/HdfsSinkConnectorTestBase.java @@ -19,43 +19,29 @@ import org.apache.hadoop.conf.Configuration; import org.apache.kafka.common.TopicPartition; 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.SinkTaskContext; import org.junit.After; -import org.junit.Before; import java.util.ArrayList; import java.util.HashMap; -import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Set; import io.confluent.connect.avro.AvroData; +import io.confluent.connect.storage.StorageSinkTestBase; -public class HdfsSinkConnectorTestBase { +public class HdfsSinkConnectorTestBase extends StorageSinkTestBase { protected Configuration conf; - protected String url; - protected Map connectorProps; protected HdfsSinkConnectorConfig connectorConfig; protected String topicsDir; protected String logsDir; protected AvroData avroData; - protected MockSinkTaskContext context; - protected static final String TOPIC = "topic"; protected static final String TOPIC_WITH_DOTS = "topic.with.dots"; - protected static final int PARTITION = 12; - protected static final int PARTITION2 = 13; - protected static final int PARTITION3 = 14; - protected static final TopicPartition TOPIC_PARTITION = new TopicPartition(TOPIC, PARTITION); - protected static final TopicPartition TOPIC_PARTITION2 = new TopicPartition(TOPIC, PARTITION2); - protected static final TopicPartition TOPIC_PARTITION3 = new TopicPartition(TOPIC, PARTITION3); protected static final TopicPartition TOPIC_WITH_DOTS_PARTITION = new TopicPartition(TOPIC_WITH_DOTS, PARTITION); - protected static Set assignment; + @Override protected Map createProps() { Map props = new HashMap<>(); props.put(HdfsSinkConnectorConfig.HDFS_URL_CONFIG, url); @@ -63,16 +49,6 @@ protected Map createProps() { return props; } - protected Schema createSchema() { - return SchemaBuilder.struct().name("record").version(1) - .field("boolean", Schema.BOOLEAN_SCHEMA) - .field("int", Schema.INT32_SCHEMA) - .field("long", Schema.INT64_SCHEMA) - .field("float", Schema.FLOAT32_SCHEMA) - .field("double", Schema.FLOAT64_SCHEMA) - .build(); - } - protected Struct createRecord(Schema schema, int ibase, float fbase) { return new Struct(schema) .put("boolean", true) @@ -82,31 +58,6 @@ protected Struct createRecord(Schema schema, int ibase, float fbase) { .put("double", (double) fbase); } - protected Struct createRecord(Schema schema) { - return createRecord(schema, 12, 12.2f); - } - - protected Schema createNewSchema() { - return SchemaBuilder.struct().name("record").version(2) - .field("boolean", Schema.BOOLEAN_SCHEMA) - .field("int", Schema.INT32_SCHEMA) - .field("long", Schema.INT64_SCHEMA) - .field("float", Schema.FLOAT32_SCHEMA) - .field("double", Schema.FLOAT64_SCHEMA) - .field("string", SchemaBuilder.string().defaultValue("abc").build()) - .build(); - } - - protected Struct createNewRecord(Schema newSchema) { - return new Struct(newSchema) - .put("boolean", true) - .put("int", 12) - .put("long", 12L) - .put("float", 12.2f) - .put("double", 12.2) - .put("string", "def"); - } - // Create a batch of records with incremental numeric field values. Total number of records is // given by 'size'. protected List createRecordBatch(Schema schema, int size) { @@ -129,93 +80,27 @@ protected List createRecordBatches(Schema schema, int batchSize, int bat return records; } - @Before + //@Before + @Override public void setUp() throws Exception { conf = new Configuration(); url = "memory://"; - connectorProps = createProps(); + super.setUp(); // Configure immediately in setup for common case of just using this default. Subclasses can // re-call this safely. configureConnector(); - assignment = new HashSet<>(); - assignment.add(TOPIC_PARTITION); - assignment.add(TOPIC_PARTITION2); - context = new MockSinkTaskContext(); } @After public void tearDown() throws Exception { - if (assignment != null) { - assignment.clear(); - } + super.tearDown(); } protected void configureConnector() { - connectorConfig = new HdfsSinkConnectorConfig(connectorProps); + connectorConfig = new HdfsSinkConnectorConfig(properties); topicsDir = connectorConfig.getString(HdfsSinkConnectorConfig.TOPICS_DIR_CONFIG); logsDir = connectorConfig.getString(HdfsSinkConnectorConfig.LOGS_DIR_CONFIG); int schemaCacheSize = connectorConfig.getInt(HdfsSinkConnectorConfig.SCHEMA_CACHE_SIZE_CONFIG); avroData = new AvroData(schemaCacheSize); } - - protected static class MockSinkTaskContext implements SinkTaskContext { - - private Map offsets; - private long timeoutMs; - - public MockSinkTaskContext() { - this.offsets = new HashMap<>(); - this.timeoutMs = -1L; - } - - @Override - public void offset(Map offsets) { - this.offsets.putAll(offsets); - } - - @Override - public void offset(TopicPartition tp, long offset) { - offsets.put(tp, offset); - } - - /** - * Get offsets that the SinkTask has submitted to be reset. Used by the Copycat framework. - * @return the map of offsets - */ - public Map offsets() { - return offsets; - } - - @Override - public void timeout(long timeoutMs) { - this.timeoutMs = timeoutMs; - } - - /** - * Get the timeout in milliseconds set by SinkTasks. Used by the Copycat framework. - * @return the backoff timeout in milliseconds. - */ - public long timeout() { - return timeoutMs; - } - - /** - * Get the timeout in milliseconds set by SinkTasks. Used by the Copycat framework. - * @return the backoff timeout in milliseconds. - */ - - @Override - public Set assignment() { - return assignment; - } - - @Override - public void pause(TopicPartition... partitions) {} - - @Override - public void resume(TopicPartition... partitions) {} - - @Override - public void requestCommit() {} - } } diff --git a/src/test/java/io/confluent/connect/hdfs/avro/TopicPartitionWriterTest.java b/src/test/java/io/confluent/connect/hdfs/avro/TopicPartitionWriterTest.java index 8cf84ea51..658458a97 100644 --- a/src/test/java/io/confluent/connect/hdfs/avro/TopicPartitionWriterTest.java +++ b/src/test/java/io/confluent/connect/hdfs/avro/TopicPartitionWriterTest.java @@ -76,7 +76,7 @@ public void setUp() throws Exception { public void testWriteRecordDefaultWithPadding() throws Exception { Partitioner partitioner = new DefaultPartitioner(); partitioner.configure(Collections.emptyMap()); - connectorProps.put(HdfsSinkConnectorConfig.FILENAME_OFFSET_ZERO_PAD_WIDTH_CONFIG, "2"); + properties.put(HdfsSinkConnectorConfig.FILENAME_OFFSET_ZERO_PAD_WIDTH_CONFIG, "2"); configureConnector(); TopicPartitionWriter topicPartitionWriter = new TopicPartitionWriter( TOPIC_PARTITION, storage, writerProvider, partitioner, connectorConfig, context, avroData); From c4143e9a6c5ca88b5dd1efd9d5dc7c1310fdd80b Mon Sep 17 00:00:00 2001 From: Konstantine Karantasis Date: Sat, 17 Dec 2016 01:56:53 -0800 Subject: [PATCH 16/58] Apply changes to Storage and WAL interfaces. --- .../io/confluent/connect/hdfs/DataWriter.java | 4 +- .../io/confluent/connect/hdfs/FileUtils.java | 11 ++--- .../connect/hdfs/TopicPartitionWriter.java | 2 +- .../connect/hdfs/storage/HdfsStorage.java | 19 ++++----- .../connect/hdfs/storage/Storage.java | 40 +------------------ .../connect/hdfs/storage/StorageFactory.java | 1 + .../hdfs/HdfsSinkConnectorTestBase.java | 1 + .../connect/hdfs/HdfsSinkTaskTest.java | 9 +++-- .../hdfs/avro/TopicPartitionWriterTest.java | 5 ++- .../connect/hdfs/utils/MemoryStorage.java | 8 ++-- .../confluent/connect/hdfs/wal/WALTest.java | 6 ++- 11 files changed, 39 insertions(+), 67 deletions(-) diff --git a/src/main/java/io/confluent/connect/hdfs/DataWriter.java b/src/main/java/io/confluent/connect/hdfs/DataWriter.java index 73c0e15c3..8a849a4f9 100644 --- a/src/main/java/io/confluent/connect/hdfs/DataWriter.java +++ b/src/main/java/io/confluent/connect/hdfs/DataWriter.java @@ -164,8 +164,8 @@ public void run() { @SuppressWarnings("unchecked") Class storageClass = (Class) Class - .forName(connectorConfig.getString(HdfsSinkConnectorConfig.STORAGE_CLASS_CONFIG)); - storage = StorageFactory.createStorage(storageClass, conf, url); + .forName(connectorConfig.getString(HdfsSinkConnectorConfig.STORAGE_CLASS_CONFIG)); + storage = io.confluent.connect.storage.StorageFactory.createStorage(storageClass, Configuration.class, conf, url); createDir(topicsDir); createDir(topicsDir + HdfsSinkConnectorConstants.TEMPFILE_DIRECTORY); diff --git a/src/main/java/io/confluent/connect/hdfs/FileUtils.java b/src/main/java/io/confluent/connect/hdfs/FileUtils.java index 7a99adcd9..764376a1a 100644 --- a/src/main/java/io/confluent/connect/hdfs/FileUtils.java +++ b/src/main/java/io/confluent/connect/hdfs/FileUtils.java @@ -26,6 +26,7 @@ import java.io.IOException; import java.util.ArrayList; +import java.util.List; import java.util.UUID; import java.util.regex.Matcher; @@ -95,7 +96,7 @@ private static ArrayList traverseImpl(Storage storage, Path path, Pa return new ArrayList<>(); } ArrayList result = new ArrayList<>(); - FileStatus[] statuses = storage.listStatus(path.toString()); + List statuses = storage.listStatus(path.toString()); for (FileStatus status : statuses) { if (status.isDirectory()) { result.addAll(traverseImpl(storage, status.getPath(), filter)); @@ -121,7 +122,7 @@ public static FileStatus fileStatusWithMaxOffset(Storage storage, Path path, } long maxOffset = -1L; FileStatus fileStatusWithMaxOffset = null; - FileStatus[] statuses = storage.listStatus(path.toString()); + List statuses = storage.listStatus(path.toString()); for (FileStatus status : statuses) { if (status.isDirectory()) { FileStatus fileStatus = fileStatusWithMaxOffset(storage, status.getPath(), filter); @@ -158,12 +159,12 @@ public static long extractOffset(String filename) { private static ArrayList getDirectoriesImpl(Storage storage, Path path) throws IOException { - FileStatus[] statuses = storage.listStatus(path.toString()); + List statuses = storage.listStatus(path.toString()); ArrayList result = new ArrayList<>(); for (FileStatus status : statuses) { if (status.isDirectory()) { int count = 0; - FileStatus[] fileStatuses = storage.listStatus(status.getPath().toString()); + List fileStatuses = storage.listStatus(status.getPath().toString()); for (FileStatus fileStatus : fileStatuses) { if (fileStatus.isDirectory()) { result.addAll(getDirectoriesImpl(storage, fileStatus.getPath())); @@ -171,7 +172,7 @@ private static ArrayList getDirectoriesImpl(Storage storage, Path pa count++; } } - if (count == fileStatuses.length) { + if (count == fileStatuses.size()) { result.add(status); } } diff --git a/src/main/java/io/confluent/connect/hdfs/TopicPartitionWriter.java b/src/main/java/io/confluent/connect/hdfs/TopicPartitionWriter.java index 160e0a8eb..a0ea7aa9f 100644 --- a/src/main/java/io/confluent/connect/hdfs/TopicPartitionWriter.java +++ b/src/main/java/io/confluent/connect/hdfs/TopicPartitionWriter.java @@ -53,7 +53,7 @@ import io.confluent.connect.hdfs.schema.Compatibility; import io.confluent.connect.hdfs.schema.SchemaUtils; import io.confluent.connect.hdfs.storage.Storage; -import io.confluent.connect.hdfs.wal.WAL; +import io.confluent.connect.storage.wal.WAL; public class TopicPartitionWriter { private static final Logger log = LoggerFactory.getLogger(TopicPartitionWriter.class); diff --git a/src/main/java/io/confluent/connect/hdfs/storage/HdfsStorage.java b/src/main/java/io/confluent/connect/hdfs/storage/HdfsStorage.java index 5069fbf7e..8efeb1e3b 100644 --- a/src/main/java/io/confluent/connect/hdfs/storage/HdfsStorage.java +++ b/src/main/java/io/confluent/connect/hdfs/storage/HdfsStorage.java @@ -25,11 +25,14 @@ import java.io.IOException; import java.net.URI; +import java.util.Arrays; +import java.util.List; import io.confluent.connect.hdfs.wal.FSWAL; -import io.confluent.connect.hdfs.wal.WAL; +import io.confluent.connect.storage.wal.WAL; -public class HdfsStorage implements Storage { +public class HdfsStorage implements io.confluent.connect.storage.Storage, PathFilter, Configuration>, + Storage { private final FileSystem fs; private final Configuration conf; @@ -42,19 +45,17 @@ public HdfsStorage(Configuration conf, String url) throws IOException { } @Override - public FileStatus[] listStatus(String path, PathFilter filter) throws IOException { - return fs.listStatus(new Path(path), filter); + public List listStatus(String path, PathFilter filter) throws IOException { + return Arrays.asList(fs.listStatus(new Path(path), filter)); } @Override - public FileStatus[] listStatus(String path) throws IOException { - return fs.listStatus(new Path(path)); + public List listStatus(String path) throws IOException { + return Arrays.asList(fs.listStatus(new Path(path))); } @Override - public void append(String filename, Object object) throws IOException { - - } + public void append(String filename, Object object) throws IOException {} @Override public boolean mkdirs(String filename) throws IOException { diff --git a/src/main/java/io/confluent/connect/hdfs/storage/Storage.java b/src/main/java/io/confluent/connect/hdfs/storage/Storage.java index fdbe719f7..366090c77 100644 --- a/src/main/java/io/confluent/connect/hdfs/storage/Storage.java +++ b/src/main/java/io/confluent/connect/hdfs/storage/Storage.java @@ -19,44 +19,8 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.PathFilter; -import org.apache.kafka.common.TopicPartition; -import java.io.IOException; - -import io.confluent.connect.hdfs.wal.WAL; +import java.util.List; @Deprecated -public interface Storage extends io.confluent.connect.storage.Storage { - @Override - boolean exists(String filename) throws IOException; - - @Override - boolean mkdirs(String filename) throws IOException; - - @Override - void append(String filename, Object object) throws IOException; - - @Override - void delete(String filename) throws IOException; - - @Override - void commit(String tempFile, String committedFile) throws IOException; - - @Override - void close() throws IOException; - - @Override - WAL wal(String topicsDir, TopicPartition topicPart); - - @Override - FileStatus[] listStatus(String path, PathFilter filter) throws IOException; - - @Override - FileStatus[] listStatus(String path) throws IOException; - - @Override - String url(); - - @Override - Configuration conf(); -} +public interface Storage extends io.confluent.connect.storage.Storage, PathFilter, Configuration> {} diff --git a/src/main/java/io/confluent/connect/hdfs/storage/StorageFactory.java b/src/main/java/io/confluent/connect/hdfs/storage/StorageFactory.java index 479442346..248e4a096 100644 --- a/src/main/java/io/confluent/connect/hdfs/storage/StorageFactory.java +++ b/src/main/java/io/confluent/connect/hdfs/storage/StorageFactory.java @@ -21,6 +21,7 @@ import java.lang.reflect.Constructor; import java.lang.reflect.InvocationTargetException; +@Deprecated public class StorageFactory { public static Storage createStorage(Class storageClass, Configuration conf, String url) { try { diff --git a/src/test/java/io/confluent/connect/hdfs/HdfsSinkConnectorTestBase.java b/src/test/java/io/confluent/connect/hdfs/HdfsSinkConnectorTestBase.java index 105cf469c..516826f31 100644 --- a/src/test/java/io/confluent/connect/hdfs/HdfsSinkConnectorTestBase.java +++ b/src/test/java/io/confluent/connect/hdfs/HdfsSinkConnectorTestBase.java @@ -92,6 +92,7 @@ public void setUp() throws Exception { } @After + @Override public void tearDown() throws Exception { super.tearDown(); } diff --git a/src/test/java/io/confluent/connect/hdfs/HdfsSinkTaskTest.java b/src/test/java/io/confluent/connect/hdfs/HdfsSinkTaskTest.java index bbb781d74..a098ce91f 100644 --- a/src/test/java/io/confluent/connect/hdfs/HdfsSinkTaskTest.java +++ b/src/test/java/io/confluent/connect/hdfs/HdfsSinkTaskTest.java @@ -14,7 +14,7 @@ package io.confluent.connect.hdfs; -import io.confluent.kafka.serializers.NonRecordContainer; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.connect.data.Schema; @@ -32,8 +32,9 @@ import io.confluent.connect.avro.AvroData; import io.confluent.connect.hdfs.avro.AvroFileReader; import io.confluent.connect.hdfs.storage.Storage; -import io.confluent.connect.hdfs.storage.StorageFactory; -import io.confluent.connect.hdfs.wal.WAL; +import io.confluent.connect.storage.StorageFactory; +import io.confluent.connect.storage.wal.WAL; +import io.confluent.kafka.serializers.NonRecordContainer; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; @@ -243,7 +244,7 @@ private void createWALs(Map> tempfiles, @SuppressWarnings("unchecked") Class storageClass = (Class) Class.forName(connectorConfig.getString(HdfsSinkConnectorConfig.STORAGE_CLASS_CONFIG)); - Storage storage = StorageFactory.createStorage(storageClass, conf, url); + Storage storage = StorageFactory.createStorage(storageClass, Configuration.class, conf, url); for (TopicPartition tp: tempfiles.keySet()) { WAL wal = storage.wal(logsDir, tp); diff --git a/src/test/java/io/confluent/connect/hdfs/avro/TopicPartitionWriterTest.java b/src/test/java/io/confluent/connect/hdfs/avro/TopicPartitionWriterTest.java index 658458a97..0bd773202 100644 --- a/src/test/java/io/confluent/connect/hdfs/avro/TopicPartitionWriterTest.java +++ b/src/test/java/io/confluent/connect/hdfs/avro/TopicPartitionWriterTest.java @@ -14,6 +14,7 @@ package io.confluent.connect.hdfs.avro; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; import org.apache.kafka.connect.data.Schema; @@ -46,7 +47,7 @@ import io.confluent.connect.hdfs.partitioner.TimeBasedPartitioner; import io.confluent.connect.hdfs.partitioner.TimeUtils; import io.confluent.connect.hdfs.storage.Storage; -import io.confluent.connect.hdfs.storage.StorageFactory; +import io.confluent.connect.storage.StorageFactory; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; @@ -67,7 +68,7 @@ public void setUp() throws Exception { @SuppressWarnings("unchecked") Class storageClass = (Class) Class .forName(connectorConfig.getString(HdfsSinkConnectorConfig.STORAGE_CLASS_CONFIG)); - storage = StorageFactory.createStorage(storageClass, conf, url); + storage = StorageFactory.createStorage(storageClass, Configuration.class, conf, url); createTopicDir(url, topicsDir, TOPIC); createLogsDir(url, logsDir); } diff --git a/src/test/java/io/confluent/connect/hdfs/utils/MemoryStorage.java b/src/test/java/io/confluent/connect/hdfs/utils/MemoryStorage.java index 4b892d0da..713a5fd9c 100644 --- a/src/test/java/io/confluent/connect/hdfs/utils/MemoryStorage.java +++ b/src/test/java/io/confluent/connect/hdfs/utils/MemoryStorage.java @@ -55,7 +55,7 @@ public MemoryStorage(Configuration conf, String url) { } @Override - public FileStatus[] listStatus(String path) throws IOException { + public List listStatus(String path) throws IOException { List result = new ArrayList<>(); for (String key: data.keySet()) { if (key.startsWith(path)) { @@ -63,11 +63,11 @@ public FileStatus[] listStatus(String path) throws IOException { result.add(status); } } - return result.toArray(new FileStatus[result.size()]); + return result; } @Override - public FileStatus[] listStatus(String path, PathFilter filter) throws IOException { + public List listStatus(String path, PathFilter filter) throws IOException { if (failure == Failure.listStatusFailure) { failure = Failure.noFailure; throw new IOException("listStatus failed."); @@ -79,7 +79,7 @@ public FileStatus[] listStatus(String path, PathFilter filter) throws IOExceptio result.add(status); } } - return result.toArray(new FileStatus[result.size()]); + return result; } @Override diff --git a/src/test/java/io/confluent/connect/hdfs/wal/WALTest.java b/src/test/java/io/confluent/connect/hdfs/wal/WALTest.java index 114843a88..6364aa3fa 100644 --- a/src/test/java/io/confluent/connect/hdfs/wal/WALTest.java +++ b/src/test/java/io/confluent/connect/hdfs/wal/WALTest.java @@ -14,15 +14,17 @@ package io.confluent.connect.hdfs.wal; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.kafka.connect.errors.ConnectException; import org.junit.Test; -import io.confluent.connect.hdfs.storage.StorageFactory; +import io.confluent.connect.storage.StorageFactory; import io.confluent.connect.hdfs.FileUtils; import io.confluent.connect.hdfs.HdfsSinkConnectorConfig; import io.confluent.connect.hdfs.TestWithMiniDFSCluster; import io.confluent.connect.hdfs.storage.Storage; +import io.confluent.connect.storage.wal.WAL; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; @@ -40,7 +42,7 @@ public void testWALMultiClient() throws Exception { @SuppressWarnings("unchecked") Class storageClass = (Class) Class.forName(connectorConfig.getString(HdfsSinkConnectorConfig.STORAGE_CLASS_CONFIG)); - Storage storage = StorageFactory.createStorage(storageClass, conf, url); + Storage storage = StorageFactory.createStorage(storageClass, Configuration.class, conf, url); final WAL wal1 = storage.wal(topicsDir, TOPIC_PARTITION); final WAL wal2 = storage.wal(topicsDir, TOPIC_PARTITION); From b967381a08da0ea3a77801847ab4864944e8701e Mon Sep 17 00:00:00 2001 From: Konstantine Karantasis Date: Sat, 17 Dec 2016 20:15:22 -0800 Subject: [PATCH 17/58] Refactor schema package to common and wrap IOException in HdfsStorage. --- .../io/confluent/connect/hdfs/DataWriter.java | 13 ++-- .../connect/hdfs/HdfsSinkConnectorConfig.java | 6 +- .../confluent/connect/hdfs/HdfsSinkTask.java | 7 +- .../connect/hdfs/TopicPartitionWriter.java | 14 ++-- .../connect/hdfs/schema/Compatibility.java | 1 + .../connect/hdfs/schema/SchemaUtils.java | 1 + .../connect/hdfs/storage/HdfsStorage.java | 68 +++++++++++++------ .../io/confluent/connect/hdfs/wal/FSWAL.java | 16 ++--- .../connect/hdfs/utils/MemoryStorage.java | 32 ++++----- .../connect/hdfs/utils/MemoryWAL.java | 37 +++------- 10 files changed, 99 insertions(+), 96 deletions(-) diff --git a/src/main/java/io/confluent/connect/hdfs/DataWriter.java b/src/main/java/io/confluent/connect/hdfs/DataWriter.java index 8a849a4f9..75168a39e 100644 --- a/src/main/java/io/confluent/connect/hdfs/DataWriter.java +++ b/src/main/java/io/confluent/connect/hdfs/DataWriter.java @@ -52,7 +52,7 @@ import io.confluent.connect.hdfs.hive.HiveUtil; import io.confluent.connect.hdfs.partitioner.Partitioner; import io.confluent.connect.hdfs.storage.Storage; -import io.confluent.connect.hdfs.storage.StorageFactory; +import io.confluent.connect.storage.StorageFactory; public class DataWriter { private static final Logger log = LoggerFactory.getLogger(DataWriter.class); @@ -165,7 +165,7 @@ public void run() { @SuppressWarnings("unchecked") Class storageClass = (Class) Class .forName(connectorConfig.getString(HdfsSinkConnectorConfig.STORAGE_CLASS_CONFIG)); - storage = io.confluent.connect.storage.StorageFactory.createStorage(storageClass, Configuration.class, conf, url); + storage = StorageFactory.createStorage(storageClass, Configuration.class, conf, url); createDir(topicsDir); createDir(topicsDir + HdfsSinkConnectorConstants.TEMPFILE_DIRECTORY); @@ -322,11 +322,8 @@ public void stop() { } } - try { - storage.close(); - } catch (IOException e) { - throw new ConnectException(e); - } + storage.close(); + if (ticketRenewThread != null) { synchronized (this) { isRunning = false; @@ -363,7 +360,7 @@ public Map getTempFileNames(TopicPartition tp) { return topicPartitionWriter.getTempFiles(); } - private void createDir(String dir) throws IOException { + private void createDir(String dir) { String path = url + "/" + dir; if (!storage.exists(path)) { storage.mkdirs(path); diff --git a/src/main/java/io/confluent/connect/hdfs/HdfsSinkConnectorConfig.java b/src/main/java/io/confluent/connect/hdfs/HdfsSinkConnectorConfig.java index 1d99469de..8b079bc7d 100644 --- a/src/main/java/io/confluent/connect/hdfs/HdfsSinkConnectorConfig.java +++ b/src/main/java/io/confluent/connect/hdfs/HdfsSinkConnectorConfig.java @@ -79,11 +79,11 @@ public class HdfsSinkConnectorConfig extends StorageSinkConnectorConfig { static { // Define HDFS configuration group - config.define(HADOOP_CONF_DIR_CONFIG, Type.STRING, HADOOP_CONF_DIR_DEFAULT, Importance.HIGH, HADOOP_CONF_DIR_DOC, HDFS_GROUP, 2, Width.MEDIUM, HADOOP_CONF_DIR_DISPLAY) + CONFIG_DEF.define(HADOOP_CONF_DIR_CONFIG, Type.STRING, HADOOP_CONF_DIR_DEFAULT, Importance.HIGH, HADOOP_CONF_DIR_DOC, HDFS_GROUP, 2, Width.MEDIUM, HADOOP_CONF_DIR_DISPLAY) .define(HADOOP_HOME_CONFIG, Type.STRING, HADOOP_HOME_DEFAULT, Importance.HIGH, HADOOP_HOME_DOC, HDFS_GROUP, 3, Width.SHORT, HADOOP_HOME_DISPLAY); // Define Security configuration group - config.define(HDFS_AUTHENTICATION_KERBEROS_CONFIG, Type.BOOLEAN, HDFS_AUTHENTICATION_KERBEROS_DEFAULT, Importance.HIGH, HDFS_AUTHENTICATION_KERBEROS_DOC, + CONFIG_DEF.define(HDFS_AUTHENTICATION_KERBEROS_CONFIG, Type.BOOLEAN, HDFS_AUTHENTICATION_KERBEROS_DEFAULT, Importance.HIGH, HDFS_AUTHENTICATION_KERBEROS_DOC, SECURITY_GROUP, 1, Width.SHORT, HDFS_AUTHENTICATION_KERBEROS_DISPLAY, Arrays.asList(CONNECT_HDFS_PRINCIPAL_CONFIG, CONNECT_HDFS_KEYTAB_CONFIG, HDFS_NAMENODE_PRINCIPAL_CONFIG, KERBEROS_TICKET_RENEW_PERIOD_MS_CONFIG)) .define(CONNECT_HDFS_PRINCIPAL_CONFIG, Type.STRING, CONNECT_HDFS_PRINCIPAL_DEFAULT, Importance.HIGH, CONNECT_HDFS_PRINCIPAL_DOC, @@ -116,7 +116,7 @@ public boolean visible(String name, Map connectorConfigs) { } public static ConfigDef getConfig() { - return config; + return CONFIG_DEF; } public HdfsSinkConnectorConfig(Map props) { diff --git a/src/main/java/io/confluent/connect/hdfs/HdfsSinkTask.java b/src/main/java/io/confluent/connect/hdfs/HdfsSinkTask.java index 66af31182..3a38e1dd3 100644 --- a/src/main/java/io/confluent/connect/hdfs/HdfsSinkTask.java +++ b/src/main/java/io/confluent/connect/hdfs/HdfsSinkTask.java @@ -29,8 +29,7 @@ import java.util.Set; import io.confluent.connect.avro.AvroData; -import io.confluent.connect.hdfs.schema.Compatibility; -import io.confluent.connect.hdfs.schema.SchemaUtils; +import io.confluent.connect.storage.schema.StorageSchemaCompatibility; public class HdfsSinkTask extends SinkTask { @@ -54,9 +53,9 @@ public void start(Map props) { HdfsSinkConnectorConfig connectorConfig = new HdfsSinkConnectorConfig(props); boolean hiveIntegration = connectorConfig.getBoolean(HdfsSinkConnectorConfig.HIVE_INTEGRATION_CONFIG); if (hiveIntegration) { - Compatibility compatibility = SchemaUtils.getCompatibility( + StorageSchemaCompatibility compatibility = StorageSchemaCompatibility.getCompatibility( connectorConfig.getString(HdfsSinkConnectorConfig.SCHEMA_COMPATIBILITY_CONFIG)); - if (compatibility == Compatibility.NONE) { + if (compatibility == StorageSchemaCompatibility.NONE) { throw new ConfigException("Hive Integration requires schema compatibility to be BACKWARD, FORWARD or FULL"); } } diff --git a/src/main/java/io/confluent/connect/hdfs/TopicPartitionWriter.java b/src/main/java/io/confluent/connect/hdfs/TopicPartitionWriter.java index a0ea7aa9f..22e95f159 100644 --- a/src/main/java/io/confluent/connect/hdfs/TopicPartitionWriter.java +++ b/src/main/java/io/confluent/connect/hdfs/TopicPartitionWriter.java @@ -18,7 +18,6 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.config.ConfigException; import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.errors.IllegalWorkerStateException; @@ -50,9 +49,8 @@ import io.confluent.connect.hdfs.hive.HiveMetaStore; import io.confluent.connect.hdfs.hive.HiveUtil; import io.confluent.connect.hdfs.partitioner.Partitioner; -import io.confluent.connect.hdfs.schema.Compatibility; -import io.confluent.connect.hdfs.schema.SchemaUtils; import io.confluent.connect.hdfs.storage.Storage; +import io.confluent.connect.storage.schema.StorageSchemaCompatibility; import io.confluent.connect.storage.wal.WAL; public class TopicPartitionWriter { @@ -85,7 +83,7 @@ public class TopicPartitionWriter { private Map offsets; private long timeoutMs; private long failureTime; - private Compatibility compatibility; + private StorageSchemaCompatibility compatibility; private Schema currentSchema; private HdfsSinkConnectorConfig connectorConfig; private String extension; @@ -141,7 +139,7 @@ public TopicPartitionWriter( rotateIntervalMs = connectorConfig.getLong(HdfsSinkConnectorConfig.ROTATE_INTERVAL_MS_CONFIG); rotateScheduleIntervalMs = connectorConfig.getLong(HdfsSinkConnectorConfig.ROTATE_SCHEDULE_INTERVAL_MS_CONFIG); timeoutMs = connectorConfig.getLong(HdfsSinkConnectorConfig.RETRY_BACKOFF_CONFIG); - compatibility = SchemaUtils.getCompatibility( + compatibility = StorageSchemaCompatibility.getCompatibility( connectorConfig.getString(HdfsSinkConnectorConfig.SCHEMA_COMPATIBILITY_CONFIG)); String logsDir = connectorConfig.getString(HdfsSinkConnectorConfig.LOGS_DIR_CONFIG); @@ -267,7 +265,7 @@ public void write() { nextState(); case WRITE_PARTITION_PAUSED: if (currentSchema == null) { - if (compatibility != Compatibility.NONE && offset != -1) { + if (compatibility != StorageSchemaCompatibility.NONE && offset != -1) { String topicDir = FileUtils.topicDirectory(url, topicsDir, tp.topic()); CommittedFileFilter filter = new TopicPartitionCommittedFileFilter(tp); FileStatus fileStatusWithMaxOffset = FileUtils.fileStatusWithMaxOffset(storage, new Path(topicDir), filter); @@ -278,7 +276,7 @@ public void write() { } SinkRecord record = buffer.peek(); Schema valueSchema = record.valueSchema(); - if (SchemaUtils.shouldChangeSchema(valueSchema, currentSchema, compatibility)) { + if (compatibility.shouldChangeSchema(valueSchema, currentSchema)) { currentSchema = valueSchema; if (hiveIntegration) { createHiveTable(); @@ -290,7 +288,7 @@ public void write() { break; } } else { - SinkRecord projectedRecord = SchemaUtils.project(record, currentSchema, compatibility); + SinkRecord projectedRecord = compatibility.project(record, currentSchema); writeRecord(projectedRecord); buffer.poll(); if (shouldRotate(now)) { diff --git a/src/main/java/io/confluent/connect/hdfs/schema/Compatibility.java b/src/main/java/io/confluent/connect/hdfs/schema/Compatibility.java index 27333bf5c..62d0cc893 100644 --- a/src/main/java/io/confluent/connect/hdfs/schema/Compatibility.java +++ b/src/main/java/io/confluent/connect/hdfs/schema/Compatibility.java @@ -14,6 +14,7 @@ package io.confluent.connect.hdfs.schema; +@Deprecated public enum Compatibility { NONE, BACKWARD, diff --git a/src/main/java/io/confluent/connect/hdfs/schema/SchemaUtils.java b/src/main/java/io/confluent/connect/hdfs/schema/SchemaUtils.java index c7e695ffd..21c586c75 100644 --- a/src/main/java/io/confluent/connect/hdfs/schema/SchemaUtils.java +++ b/src/main/java/io/confluent/connect/hdfs/schema/SchemaUtils.java @@ -19,6 +19,7 @@ import org.apache.kafka.connect.errors.SchemaProjectorException; import org.apache.kafka.connect.sink.SinkRecord; +@Deprecated public class SchemaUtils { public static Compatibility getCompatibility(String compatibilityString) { diff --git a/src/main/java/io/confluent/connect/hdfs/storage/HdfsStorage.java b/src/main/java/io/confluent/connect/hdfs/storage/HdfsStorage.java index 8efeb1e3b..328544e92 100644 --- a/src/main/java/io/confluent/connect/hdfs/storage/HdfsStorage.java +++ b/src/main/java/io/confluent/connect/hdfs/storage/HdfsStorage.java @@ -22,6 +22,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.PathFilter; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.connect.errors.ConnectException; import java.io.IOException; import java.net.URI; @@ -45,43 +46,66 @@ public HdfsStorage(Configuration conf, String url) throws IOException { } @Override - public List listStatus(String path, PathFilter filter) throws IOException { - return Arrays.asList(fs.listStatus(new Path(path), filter)); + public List listStatus(String path, PathFilter filter) { + try { + return Arrays.asList(fs.listStatus(new Path(path), filter)); + } catch (IOException e) { + throw new ConnectException(e); + } } @Override - public List listStatus(String path) throws IOException { - return Arrays.asList(fs.listStatus(new Path(path))); + public List listStatus(String path) { + try { + return Arrays.asList(fs.listStatus(new Path(path))); + } catch (IOException e) { + throw new ConnectException(e); + } } @Override - public void append(String filename, Object object) throws IOException {} + public void append(String filename, Object object) {} @Override - public boolean mkdirs(String filename) throws IOException { - return fs.mkdirs(new Path(filename)); + public boolean mkdirs(String filename) { + try { + return fs.mkdirs(new Path(filename)); + } catch (IOException e) { + throw new ConnectException(e); + } } @Override - public boolean exists(String filename) throws IOException { - return fs.exists(new Path(filename)); + public boolean exists(String filename) { + try { + return fs.exists(new Path(filename)); + } catch (IOException e) { + throw new ConnectException(e); + } } @Override - public void commit(String tempFile, String committedFile) throws IOException { + public void commit(String tempFile, String committedFile) { renameFile(tempFile, committedFile); } - @Override - public void delete(String filename) throws IOException { - fs.delete(new Path(filename), true); + public void delete(String filename) { + try { + fs.delete(new Path(filename), true); + } catch (IOException e) { + throw new ConnectException(e); + } } @Override - public void close() throws IOException { + public void close() { if (fs != null) { - fs.close(); + try { + fs.close(); + } catch (IOException e) { + throw new ConnectException(e); + } } } @@ -100,14 +124,18 @@ public String url() { return url; } - private void renameFile(String sourcePath, String targetPath) throws IOException { + private void renameFile(String sourcePath, String targetPath) { if (sourcePath.equals(targetPath)) { return; } - final Path srcPath = new Path(sourcePath); - final Path dstPath = new Path(targetPath); - if (fs.exists(srcPath)) { - fs.rename(srcPath, dstPath); + try { + final Path srcPath = new Path(sourcePath); + final Path dstPath = new Path(targetPath); + if (fs.exists(srcPath)) { + fs.rename(srcPath, dstPath); + } + } catch (IOException e) { + throw new ConnectException(e); } } } diff --git a/src/main/java/io/confluent/connect/hdfs/wal/FSWAL.java b/src/main/java/io/confluent/connect/hdfs/wal/FSWAL.java index c4ce18c65..a218e6b1f 100644 --- a/src/main/java/io/confluent/connect/hdfs/wal/FSWAL.java +++ b/src/main/java/io/confluent/connect/hdfs/wal/FSWAL.java @@ -134,16 +134,12 @@ public void apply() throws ConnectException { @Override public void truncate() throws ConnectException { - try { - String oldLogFile = logFile + ".1"; - storage.delete(oldLogFile); - storage.commit(logFile, oldLogFile); - // Clean out references to the current WAL file. - // Open a new one on the next lease acquisition. - close(); - } catch (IOException e) { - throw new ConnectException(e); - } + String oldLogFile = logFile + ".1"; + storage.delete(oldLogFile); + storage.commit(logFile, oldLogFile); + // Clean out references to the current WAL file. + // Open a new one on the next lease acquisition. + close(); } @Override diff --git a/src/test/java/io/confluent/connect/hdfs/utils/MemoryStorage.java b/src/test/java/io/confluent/connect/hdfs/utils/MemoryStorage.java index 713a5fd9c..1fa2cc337 100644 --- a/src/test/java/io/confluent/connect/hdfs/utils/MemoryStorage.java +++ b/src/test/java/io/confluent/connect/hdfs/utils/MemoryStorage.java @@ -21,8 +21,8 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.PathFilter; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.connect.errors.ConnectException; -import java.io.IOException; import java.util.ArrayList; import java.util.LinkedList; import java.util.List; @@ -55,7 +55,7 @@ public MemoryStorage(Configuration conf, String url) { } @Override - public List listStatus(String path) throws IOException { + public List listStatus(String path) { List result = new ArrayList<>(); for (String key: data.keySet()) { if (key.startsWith(path)) { @@ -67,10 +67,10 @@ public List listStatus(String path) throws IOException { } @Override - public List listStatus(String path, PathFilter filter) throws IOException { + public List listStatus(String path, PathFilter filter) { if (failure == Failure.listStatusFailure) { failure = Failure.noFailure; - throw new IOException("listStatus failed."); + throw new ConnectException("listStatus failed."); } List result = new ArrayList<>(); for (String key: data.keySet()) { @@ -83,10 +83,10 @@ public List listStatus(String path, PathFilter filter) throws IOExce } @Override - public void append(String filename, Object object) throws IOException { + public void append(String filename, Object object) { if (failure == Failure.appendFailure) { failure = Failure.noFailure; - throw new IOException("append failed."); + throw new ConnectException("append failed."); } if (!data.containsKey(filename)) { data.put(filename, new LinkedList<>()); @@ -95,28 +95,28 @@ public void append(String filename, Object object) throws IOException { } @Override - public boolean mkdirs(String filename) throws IOException { + public boolean mkdirs(String filename) { if (failure == Failure.mkdirsFailure) { failure = Failure.noFailure; - throw new IOException("mkdirs failed."); + throw new ConnectException("mkdirs failed."); } return true; } @Override - public boolean exists(String filename) throws IOException { + public boolean exists(String filename) { if (failure == Failure.existsFailure) { failure = Failure.noFailure; - throw new IOException("exists failed."); + throw new ConnectException("exists failed."); } return data.containsKey(filename); } @Override - public void delete(String filename) throws IOException { + public void delete(String filename) { if (failure == Failure.deleteFailure) { failure = Failure.noFailure; - throw new IOException("delete failed."); + throw new ConnectException("delete failed."); } if (data.containsKey(filename)) { data.get(filename).clear(); @@ -125,10 +125,10 @@ public void delete(String filename) throws IOException { } @Override - public void commit(String tempFile, String committedFile) throws IOException { + public void commit(String tempFile, String committedFile) { if (failure == Failure.commitFailure) { failure = Failure.noFailure; - throw new IOException("commit failed."); + throw new ConnectException("commit failed."); } if (!data.containsKey(committedFile)) { List entryList = data.get(tempFile); @@ -138,10 +138,10 @@ public void commit(String tempFile, String committedFile) throws IOException { } @Override - public void close() throws IOException { + public void close() { if (failure == Failure.closeFailure) { failure = Failure.noFailure; - throw new IOException("close failed."); + throw new ConnectException("close failed."); } data.clear(); } diff --git a/src/test/java/io/confluent/connect/hdfs/utils/MemoryWAL.java b/src/test/java/io/confluent/connect/hdfs/utils/MemoryWAL.java index d522208cf..12d85879c 100644 --- a/src/test/java/io/confluent/connect/hdfs/utils/MemoryWAL.java +++ b/src/test/java/io/confluent/connect/hdfs/utils/MemoryWAL.java @@ -19,7 +19,6 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.connect.errors.ConnectException; -import java.io.IOException; import java.util.List; import java.util.Map; @@ -48,46 +47,30 @@ public void acquireLease() throws ConnectException { @Override public void append(String tempFile, String committedFile) throws ConnectException { - try { - LogEntry entry = new LogEntry(tempFile, committedFile); - storage.append(logFile, entry); - } catch (IOException e) { - throw new ConnectException(e); - } + LogEntry entry = new LogEntry(tempFile, committedFile); + storage.append(logFile, entry); } @Override public void apply() throws ConnectException { - try { - if (data.containsKey(logFile)) { - List entryList = data.get(logFile); - for (Object entry : entryList) { - LogEntry logEntry = (LogEntry) entry; - storage.commit(logEntry.key(), logEntry.value()); - } + if (data.containsKey(logFile)) { + List entryList = data.get(logFile); + for (Object entry : entryList) { + LogEntry logEntry = (LogEntry) entry; + storage.commit(logEntry.key(), logEntry.value()); } - } catch (IOException e) { - throw new ConnectException(e); } } @Override public void truncate() throws ConnectException { - try { - storage.commit(logFile, logFile + ".1"); - storage.delete(logFile); - } catch (IOException e) { - throw new ConnectException(e); - } + storage.commit(logFile, logFile + ".1"); + storage.delete(logFile); } @Override public void close() throws ConnectException { - try { - storage.close(); - } catch (IOException e) { - throw new ConnectException(e); - } + storage.close(); } @Override From 7daa02f52bd4b9845e4387dd56efa918ce527490 Mon Sep 17 00:00:00 2001 From: Konstantine Karantasis Date: Sat, 17 Dec 2016 23:56:36 -0800 Subject: [PATCH 18/58] Move DateTimeUtils to storage common. --- .../confluent/connect/hdfs/DateTimeUtils.java | 22 ++----------------- 1 file changed, 2 insertions(+), 20 deletions(-) diff --git a/src/main/java/io/confluent/connect/hdfs/DateTimeUtils.java b/src/main/java/io/confluent/connect/hdfs/DateTimeUtils.java index 9e338270f..42f32e61e 100644 --- a/src/main/java/io/confluent/connect/hdfs/DateTimeUtils.java +++ b/src/main/java/io/confluent/connect/hdfs/DateTimeUtils.java @@ -1,22 +1,4 @@ package io.confluent.connect.hdfs; -import org.joda.time.DateTimeZone; - -public class DateTimeUtils { - private static final long DAY_IN_MS = 24 * 60 * 60 * 1000; - - /** - * Calculates next period of periodMs after currentTimeMs starting from midnight in given timeZone. - * If the next period is in next day then 12am of next day will be returned - * @param currentTimeMs time to calculate at - * @param periodMs period in ms - * @param timeZone timezone to get midnight time - * @return timestamp in ms - */ - public static long getNextTimeAdjustedByDay(long currentTimeMs, long periodMs, DateTimeZone timeZone) { - long startOfDay = timeZone.convertLocalToUTC(timeZone.convertUTCToLocal(currentTimeMs) / DAY_IN_MS * DAY_IN_MS, true); - long nextPeriodOffset = ((currentTimeMs - startOfDay) / periodMs + 1) * periodMs; - long offset = Math.min(nextPeriodOffset, DAY_IN_MS); - return startOfDay + offset; - } -} +@Deprecated +public class DateTimeUtils extends io.confluent.connect.storage.util.DateTimeUtils {} From 07dae4b1b2e6da7a48ae5ed88c0f007d6e8a0108 Mon Sep 17 00:00:00 2001 From: Konstantine Karantasis Date: Mon, 2 Jan 2017 20:12:23 -0800 Subject: [PATCH 19/58] Fix typo in Kerberos authentication (Kerboros -> Kerberos) --- docs/hdfs_connector.rst | 2 +- src/main/java/io/confluent/connect/hdfs/DataWriter.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/hdfs_connector.rst b/docs/hdfs_connector.rst index aaf5c86ae..592b20e7f 100644 --- a/docs/hdfs_connector.rst +++ b/docs/hdfs_connector.rst @@ -212,7 +212,7 @@ To work with secure HDFS and Hive metastore, you need to specify ``hdfs.authenti connect.hdfs.keytab=path to the connector keytab hdfs.namenode.principal=namenode principal -You need to create the Kafka connect principals and keytab files via Kerboros and distribute the +You need to create the Kafka connect principals and keytab files via Kerberos and distribute the keytab file to all hosts that running the connector and ensures that only the connector user has read access to the keytab file. diff --git a/src/main/java/io/confluent/connect/hdfs/DataWriter.java b/src/main/java/io/confluent/connect/hdfs/DataWriter.java index 75168a39e..37e76abe0 100644 --- a/src/main/java/io/confluent/connect/hdfs/DataWriter.java +++ b/src/main/java/io/confluent/connect/hdfs/DataWriter.java @@ -105,7 +105,7 @@ public DataWriter(HdfsSinkConnectorConfig connectorConfig, SinkTaskContext conte if (principalConfig == null || keytab == null) { throw new ConfigException( - "Hadoop is using Kerboros for authentication, you need to provide both a connect principal and " + "Hadoop is using Kerberos for authentication, you need to provide both a connect principal and " + "the path to the keytab of the principal."); } From 6231552c38c8cd5212bac2b0cd7447b66ece7d96 Mon Sep 17 00:00:00 2001 From: Konstantine Karantasis Date: Tue, 3 Jan 2017 11:29:13 -0800 Subject: [PATCH 20/58] Cleanup unused methods, imports, exceptions (especially IOException) --- .../io/confluent/connect/hdfs/DataWriter.java | 1 - .../io/confluent/connect/hdfs/FileUtils.java | 9 ++-- .../confluent/connect/hdfs/HdfsSinkTask.java | 2 +- .../connect/hdfs/SchemaFileReader.java | 1 - .../connect/hdfs/TopicPartitionWriter.java | 48 ++++++++----------- .../connect/hdfs/avro/AvroFileReader.java | 5 +- .../hdfs/avro/AvroRecordWriterProvider.java | 8 ++-- .../TopicPartitionCommittedFileFilter.java | 5 +- .../hdfs/parquet/ParquetFileReader.java | 5 +- .../parquet/ParquetRecordWriterProvider.java | 5 +- .../io/confluent/connect/hdfs/wal/FSWAL.java | 9 ++-- .../hdfs/HdfsSinkTaskTestWithSecureHDFS.java | 1 - .../confluent/connect/hdfs/hive/HiveExec.java | 2 +- .../connect/hdfs/hive/HiveTestBase.java | 2 +- .../hdfs/parquet/ParquetHiveUtilTest.java | 2 +- .../hdfs/utils/MemoryRecordWriter.java | 1 - 16 files changed, 46 insertions(+), 60 deletions(-) diff --git a/src/main/java/io/confluent/connect/hdfs/DataWriter.java b/src/main/java/io/confluent/connect/hdfs/DataWriter.java index 37e76abe0..41f84e0b9 100644 --- a/src/main/java/io/confluent/connect/hdfs/DataWriter.java +++ b/src/main/java/io/confluent/connect/hdfs/DataWriter.java @@ -369,7 +369,6 @@ private void createDir(String dir) { @SuppressWarnings("unchecked") private Format getFormat() throws ClassNotFoundException, IllegalAccessException, InstantiationException{ - String name = connectorConfig.getString(HdfsSinkConnectorConfig.FORMAT_CLASS_CONFIG); return ((Class) Class.forName(connectorConfig.getString(HdfsSinkConnectorConfig.FORMAT_CLASS_CONFIG))).newInstance(); } diff --git a/src/main/java/io/confluent/connect/hdfs/FileUtils.java b/src/main/java/io/confluent/connect/hdfs/FileUtils.java index 764376a1a..60753baa8 100644 --- a/src/main/java/io/confluent/connect/hdfs/FileUtils.java +++ b/src/main/java/io/confluent/connect/hdfs/FileUtils.java @@ -90,8 +90,7 @@ public static String topicDirectory(String url, String topicsDir, String topic) return url + "/" + topicsDir + "/" + topic; } - private static ArrayList traverseImpl(Storage storage, Path path, PathFilter filter) - throws IOException { + private static ArrayList traverseImpl(Storage storage, Path path, PathFilter filter) { if (!storage.exists(path.toString())) { return new ArrayList<>(); } @@ -115,8 +114,7 @@ public static FileStatus[] traverse(Storage storage, Path path, PathFilter filte return result.toArray(new FileStatus[result.size()]); } - public static FileStatus fileStatusWithMaxOffset(Storage storage, Path path, - CommittedFileFilter filter) throws IOException { + public static FileStatus fileStatusWithMaxOffset(Storage storage, Path path, CommittedFileFilter filter) { if (!storage.exists(path.toString())) { return null; } @@ -157,8 +155,7 @@ public static long extractOffset(String filename) { return Long.parseLong(m.group(HdfsSinkConnectorConstants.PATTERN_END_OFFSET_GROUP)); } - private static ArrayList getDirectoriesImpl(Storage storage, Path path) - throws IOException { + private static ArrayList getDirectoriesImpl(Storage storage, Path path) { List statuses = storage.listStatus(path.toString()); ArrayList result = new ArrayList<>(); for (FileStatus status : statuses) { diff --git a/src/main/java/io/confluent/connect/hdfs/HdfsSinkTask.java b/src/main/java/io/confluent/connect/hdfs/HdfsSinkTask.java index 3a38e1dd3..c86e04aa7 100644 --- a/src/main/java/io/confluent/connect/hdfs/HdfsSinkTask.java +++ b/src/main/java/io/confluent/connect/hdfs/HdfsSinkTask.java @@ -48,7 +48,7 @@ public String version() { @Override public void start(Map props) { - Set assignment = context.assignment();; + Set assignment = context.assignment(); try { HdfsSinkConnectorConfig connectorConfig = new HdfsSinkConnectorConfig(props); boolean hiveIntegration = connectorConfig.getBoolean(HdfsSinkConnectorConfig.HIVE_INTEGRATION_CONFIG); diff --git a/src/main/java/io/confluent/connect/hdfs/SchemaFileReader.java b/src/main/java/io/confluent/connect/hdfs/SchemaFileReader.java index 88c261520..4f7eb2cbc 100644 --- a/src/main/java/io/confluent/connect/hdfs/SchemaFileReader.java +++ b/src/main/java/io/confluent/connect/hdfs/SchemaFileReader.java @@ -18,7 +18,6 @@ import org.apache.hadoop.fs.Path; import org.apache.kafka.connect.data.Schema; -import java.io.IOException; import java.util.Collection; @Deprecated diff --git a/src/main/java/io/confluent/connect/hdfs/TopicPartitionWriter.java b/src/main/java/io/confluent/connect/hdfs/TopicPartitionWriter.java index 22e95f159..fb58ce148 100644 --- a/src/main/java/io/confluent/connect/hdfs/TopicPartitionWriter.java +++ b/src/main/java/io/confluent/connect/hdfs/TopicPartitionWriter.java @@ -20,6 +20,7 @@ import org.apache.kafka.common.TopicPartition; 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.errors.IllegalWorkerStateException; import org.apache.kafka.connect.errors.SchemaProjectorException; import org.apache.kafka.connect.sink.SinkRecord; @@ -29,7 +30,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; @@ -85,7 +85,6 @@ public class TopicPartitionWriter { private long failureTime; private StorageSchemaCompatibility compatibility; private Schema currentSchema; - private HdfsSinkConnectorConfig connectorConfig; private String extension; private final String zeroPadOffsetFormat; private DateTimeZone timeZone; @@ -124,7 +123,6 @@ public TopicPartitionWriter( ExecutorService executorService, Queue> hiveUpdateFutures) { this.tp = tp; - this.connectorConfig = connectorConfig; this.context = context; this.avroData = avroData; this.storage = storage; @@ -318,7 +316,7 @@ public void write() { } } catch (SchemaProjectorException | IllegalWorkerStateException | HiveMetaStoreException e ) { throw new RuntimeException(e); - } catch (IOException | ConnectException e) { + } catch (ConnectException e) { log.error("Exception on topic partition {}: ", tp, e); failureTime = System.currentTimeMillis(); setRetryTimeout(timeoutMs); @@ -335,7 +333,7 @@ public void write() { closeTempFile(); appendToWAL(); commitFile(); - } catch (IOException e) { + } catch (DataException e) { log.error("Exception on topic partition {}: ", tp, e); failureTime = System.currentTimeMillis(); setRetryTimeout(timeoutMs); @@ -358,7 +356,7 @@ public void close() throws ConnectException { closeTempFile(encodedPartition); deleteTempFile(encodedPartition); } - } catch (IOException e) { + } catch (DataException e) { log.error("Error discarding temp file {} for {} {} when closing TopicPartitionWriter:", tempFiles.get(encodedPartition), tp, encodedPartition, e); } @@ -401,10 +399,6 @@ public Map getTempFiles() { return tempFiles; } - public String getExtension() { - return writerProvider.getExtension(); - } - private String getDirectory(String encodedPartition) { return partitioner.generatePartitionedPath(tp.topic(), encodedPartition); } @@ -426,15 +420,11 @@ private boolean shouldRotate(long now) { } private void readOffset() throws ConnectException { - try { - String path = FileUtils.topicDirectory(url, topicsDir, tp.topic()); - CommittedFileFilter filter = new TopicPartitionCommittedFileFilter(tp); - FileStatus fileStatusWithMaxOffset = FileUtils.fileStatusWithMaxOffset(storage, new Path(path), filter); - if (fileStatusWithMaxOffset != null) { - offset = FileUtils.extractOffset(fileStatusWithMaxOffset.getPath().getName()) + 1; - } - } catch (IOException e) { - throw new ConnectException(e); + String path = FileUtils.topicDirectory(url, topicsDir, tp.topic()); + CommittedFileFilter filter = new TopicPartitionCommittedFileFilter(tp); + FileStatus fileStatusWithMaxOffset = FileUtils.fileStatusWithMaxOffset(storage, new Path(path), filter); + if (fileStatusWithMaxOffset != null) { + offset = FileUtils.extractOffset(fileStatusWithMaxOffset.getPath().getName()) + 1; } } @@ -501,7 +491,7 @@ private void resetOffsets() throws ConnectException { } } - private void writeRecord(SinkRecord record) throws IOException { + private void writeRecord(SinkRecord record) { long expectedOffset = offset + recordCounter; if (offset == -1) { offset = record.kafkaOffset(); @@ -537,7 +527,7 @@ private void writeRecord(SinkRecord record) throws IOException { recordCounter++; } - private void closeTempFile(String encodedPartition) throws IOException { + private void closeTempFile(String encodedPartition) { if (writers.containsKey(encodedPartition)) { RecordWriter writer = writers.get(encodedPartition); writer.close(); @@ -545,13 +535,13 @@ private void closeTempFile(String encodedPartition) throws IOException { } } - private void closeTempFile() throws IOException { + private void closeTempFile() { for (String encodedPartition: tempFiles.keySet()) { closeTempFile(encodedPartition); } } - private void appendToWAL(String encodedPartition) throws IOException { + private void appendToWAL(String encodedPartition) { String tempFile = tempFiles.get(encodedPartition); if (appended.contains(tempFile)) { return; @@ -569,7 +559,7 @@ private void appendToWAL(String encodedPartition) throws IOException { appended.add(tempFile); } - private void appendToWAL() throws IOException { + private void appendToWAL() { beginAppend(); for (String encodedPartition: tempFiles.keySet()) { appendToWAL(encodedPartition); @@ -577,26 +567,26 @@ private void appendToWAL() throws IOException { endAppend(); } - private void beginAppend() throws IOException { + private void beginAppend() { if (!appended.contains(WAL.beginMarker)) { wal.append(WAL.beginMarker, ""); } } - private void endAppend() throws IOException { + private void endAppend() { if (!appended.contains(WAL.endMarker)) { wal.append(WAL.endMarker, ""); } } - private void commitFile() throws IOException { + private void commitFile() { appended.clear(); for (String encodedPartition: tempFiles.keySet()) { commitFile(encodedPartition); } } - private void commitFile(String encodedPartiton) throws IOException { + private void commitFile(String encodedPartiton) { if (!startOffsets.containsKey(encodedPartiton)) { return; } @@ -619,7 +609,7 @@ private void commitFile(String encodedPartiton) throws IOException { log.info("Committed {} for {}", committedFile, tp); } - private void deleteTempFile(String encodedPartiton) throws IOException { + private void deleteTempFile(String encodedPartiton) { storage.delete(tempFiles.get(encodedPartiton)); } diff --git a/src/main/java/io/confluent/connect/hdfs/avro/AvroFileReader.java b/src/main/java/io/confluent/connect/hdfs/avro/AvroFileReader.java index dc7f545dc..761102b99 100644 --- a/src/main/java/io/confluent/connect/hdfs/avro/AvroFileReader.java +++ b/src/main/java/io/confluent/connect/hdfs/avro/AvroFileReader.java @@ -24,6 +24,7 @@ import org.apache.hadoop.fs.Path; import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.errors.ConnectException; +import org.apache.kafka.connect.errors.DataException; import java.io.IOException; import java.util.ArrayList; @@ -50,7 +51,7 @@ public Schema getSchema(Configuration conf, Path path) { fileReader.close(); return avroData.toConnectSchema(schema); } catch (IOException e) { - throw new ConnectException(e); + throw new DataException(e); } } @@ -66,7 +67,7 @@ public Collection readData(Configuration conf, Path path) { } fileReader.close(); } catch (IOException e) { - throw new ConnectException(e); + throw new DataException(e); } return collection; } diff --git a/src/main/java/io/confluent/connect/hdfs/avro/AvroRecordWriterProvider.java b/src/main/java/io/confluent/connect/hdfs/avro/AvroRecordWriterProvider.java index 634e65ba3..98eac3b97 100644 --- a/src/main/java/io/confluent/connect/hdfs/avro/AvroRecordWriterProvider.java +++ b/src/main/java/io/confluent/connect/hdfs/avro/AvroRecordWriterProvider.java @@ -22,7 +22,7 @@ import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.Path; 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; @@ -56,7 +56,7 @@ public RecordWriter getRecordWriter(Configuration conf, final String org.apache.avro.Schema avroSchema = avroData.fromConnectSchema(schema); writer.create(avroSchema, out); } catch (IOException e) { - throw new ConnectException(e); + throw new DataException(e); } return new RecordWriter() { @@ -72,7 +72,7 @@ public void write(SinkRecord record) { else writer.append(value); } catch (IOException e) { - throw new ConnectException(e); + throw new DataException(e); } } @@ -81,7 +81,7 @@ public void close() { try { writer.close(); } catch (IOException e) { - throw new ConnectException(e); + throw new DataException(e); } } }; diff --git a/src/main/java/io/confluent/connect/hdfs/filter/TopicPartitionCommittedFileFilter.java b/src/main/java/io/confluent/connect/hdfs/filter/TopicPartitionCommittedFileFilter.java index efa4aa449..3feffbadb 100644 --- a/src/main/java/io/confluent/connect/hdfs/filter/TopicPartitionCommittedFileFilter.java +++ b/src/main/java/io/confluent/connect/hdfs/filter/TopicPartitionCommittedFileFilter.java @@ -17,11 +17,10 @@ import org.apache.hadoop.fs.Path; import org.apache.kafka.common.TopicPartition; -import io.confluent.connect.hdfs.HdfsSinkConnector; -import io.confluent.connect.hdfs.HdfsSinkConnectorConstants; - import java.util.regex.Matcher; +import io.confluent.connect.hdfs.HdfsSinkConnectorConstants; + public class TopicPartitionCommittedFileFilter extends CommittedFileFilter { private TopicPartition tp; diff --git a/src/main/java/io/confluent/connect/hdfs/parquet/ParquetFileReader.java b/src/main/java/io/confluent/connect/hdfs/parquet/ParquetFileReader.java index 91e8f6468..6f80f9177 100644 --- a/src/main/java/io/confluent/connect/hdfs/parquet/ParquetFileReader.java +++ b/src/main/java/io/confluent/connect/hdfs/parquet/ParquetFileReader.java @@ -19,6 +19,7 @@ import org.apache.hadoop.fs.Path; import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.errors.ConnectException; +import org.apache.kafka.connect.errors.DataException; import org.apache.parquet.avro.AvroReadSupport; import org.apache.parquet.hadoop.ParquetReader; @@ -51,7 +52,7 @@ public Schema getSchema(Configuration conf, Path path) { parquetReader.close(); return schema; } catch (IOException e) { - throw new ConnectException(e); + throw new DataException(e); } } @@ -68,7 +69,7 @@ public Collection readData(Configuration conf, Path path) { } parquetReader.close(); } catch (IOException e) { - throw new ConnectException(e); + throw new DataException(e); } return result; } diff --git a/src/main/java/io/confluent/connect/hdfs/parquet/ParquetRecordWriterProvider.java b/src/main/java/io/confluent/connect/hdfs/parquet/ParquetRecordWriterProvider.java index 73a31601f..c8e561274 100644 --- a/src/main/java/io/confluent/connect/hdfs/parquet/ParquetRecordWriterProvider.java +++ b/src/main/java/io/confluent/connect/hdfs/parquet/ParquetRecordWriterProvider.java @@ -19,6 +19,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.kafka.connect.errors.ConnectException; +import org.apache.kafka.connect.errors.DataException; import org.apache.kafka.connect.sink.SinkRecord; import org.apache.parquet.avro.AvroParquetWriter; import org.apache.parquet.hadoop.ParquetWriter; @@ -60,7 +61,7 @@ public void write(SinkRecord record) { try { writer.write((GenericRecord) value); } catch (IOException e) { - throw new ConnectException(e); + throw new DataException(e); } } @@ -69,7 +70,7 @@ public void close() { try { writer.close(); } catch (IOException e) { - throw new ConnectException(e); + throw new DataException(e); } } }; diff --git a/src/main/java/io/confluent/connect/hdfs/wal/FSWAL.java b/src/main/java/io/confluent/connect/hdfs/wal/FSWAL.java index a218e6b1f..60cc70a3a 100644 --- a/src/main/java/io/confluent/connect/hdfs/wal/FSWAL.java +++ b/src/main/java/io/confluent/connect/hdfs/wal/FSWAL.java @@ -19,6 +19,7 @@ import org.apache.hadoop.ipc.RemoteException; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.connect.errors.ConnectException; +import org.apache.kafka.connect.errors.DataException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -60,7 +61,7 @@ public void append(String tempFile, String committedFile) throws ConnectExceptio writer.append(key, value); writer.hsync(); } catch (IOException e) { - throw new ConnectException(e); + throw new DataException(e); } } @@ -88,7 +89,7 @@ public void acquireLease() throws ConnectException { throw new ConnectException(e); } } catch (IOException e) { - throw new ConnectException("Error creating writer for log file " + logFile, e); + throw new DataException("Error creating writer for log file " + logFile, e); } } if (sleepIntervalMs >= MAX_SLEEP_INTERVAL_MS) { @@ -128,7 +129,7 @@ public void apply() throws ConnectException { } } } catch (IOException e) { - throw new ConnectException(e); + throw new DataException(e); } } @@ -154,7 +155,7 @@ public void close() throws ConnectException { reader = null; } } catch (IOException e) { - throw new ConnectException("Error closing " + logFile, e); + throw new DataException("Error closing " + logFile, e); } } diff --git a/src/test/java/io/confluent/connect/hdfs/HdfsSinkTaskTestWithSecureHDFS.java b/src/test/java/io/confluent/connect/hdfs/HdfsSinkTaskTestWithSecureHDFS.java index 640535550..dbc6ab679 100644 --- a/src/test/java/io/confluent/connect/hdfs/HdfsSinkTaskTestWithSecureHDFS.java +++ b/src/test/java/io/confluent/connect/hdfs/HdfsSinkTaskTestWithSecureHDFS.java @@ -14,7 +14,6 @@ package io.confluent.connect.hdfs; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.connect.data.Schema; diff --git a/src/test/java/io/confluent/connect/hdfs/hive/HiveExec.java b/src/test/java/io/confluent/connect/hdfs/hive/HiveExec.java index 7539bed44..6e955dc4c 100644 --- a/src/test/java/io/confluent/connect/hdfs/hive/HiveExec.java +++ b/src/test/java/io/confluent/connect/hdfs/hive/HiveExec.java @@ -70,7 +70,7 @@ public void executeQuery(String query) throws IOException { } - private String[] getHiveArgs(String... args) throws IOException { + private String[] getHiveArgs(String... args) { List newArgs = new LinkedList<>(); newArgs.addAll(Arrays.asList(args)); if (hiveConf.getBoolean(HIVE_SASL_ENABLED, false)) { diff --git a/src/test/java/io/confluent/connect/hdfs/hive/HiveTestBase.java b/src/test/java/io/confluent/connect/hdfs/hive/HiveTestBase.java index 3b8704f32..3dee2dd76 100644 --- a/src/test/java/io/confluent/connect/hdfs/hive/HiveTestBase.java +++ b/src/test/java/io/confluent/connect/hdfs/hive/HiveTestBase.java @@ -50,7 +50,7 @@ protected Map createProps() { return props; } - private void cleanHive() throws Exception { + private void cleanHive() { // ensures all tables are removed for (String database : hiveMetaStore.getAllDatabases()) { for (String table : hiveMetaStore.getAllTables(database)) { diff --git a/src/test/java/io/confluent/connect/hdfs/parquet/ParquetHiveUtilTest.java b/src/test/java/io/confluent/connect/hdfs/parquet/ParquetHiveUtilTest.java index 2467cf916..7f11763d1 100644 --- a/src/test/java/io/confluent/connect/hdfs/parquet/ParquetHiveUtilTest.java +++ b/src/test/java/io/confluent/connect/hdfs/parquet/ParquetHiveUtilTest.java @@ -143,7 +143,7 @@ public void testAlterSchema() throws Exception { } } - private void prepareData(String topic, int partition) throws Exception { + private void prepareData(String topic, int partition) { TopicPartition tp = new TopicPartition(topic, partition); DataWriter hdfsWriter = createWriter(context, avroData); hdfsWriter.recover(tp); diff --git a/src/test/java/io/confluent/connect/hdfs/utils/MemoryRecordWriter.java b/src/test/java/io/confluent/connect/hdfs/utils/MemoryRecordWriter.java index 83b941348..603f1b60d 100644 --- a/src/test/java/io/confluent/connect/hdfs/utils/MemoryRecordWriter.java +++ b/src/test/java/io/confluent/connect/hdfs/utils/MemoryRecordWriter.java @@ -17,7 +17,6 @@ import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.sink.SinkRecord; -import java.io.IOException; import java.util.List; import java.util.Map; From 2b3569387c85308415f22a4ecbcc082850861859 Mon Sep 17 00:00:00 2001 From: Konstantine Karantasis Date: Tue, 3 Jan 2017 19:17:07 -0800 Subject: [PATCH 21/58] Fix typo for variable in TopicPartitionWriter --- .../connect/hdfs/TopicPartitionWriter.java | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/src/main/java/io/confluent/connect/hdfs/TopicPartitionWriter.java b/src/main/java/io/confluent/connect/hdfs/TopicPartitionWriter.java index fb58ce148..de1fe9213 100644 --- a/src/main/java/io/confluent/connect/hdfs/TopicPartitionWriter.java +++ b/src/main/java/io/confluent/connect/hdfs/TopicPartitionWriter.java @@ -586,14 +586,14 @@ private void commitFile() { } } - private void commitFile(String encodedPartiton) { - if (!startOffsets.containsKey(encodedPartiton)) { + private void commitFile(String encodedPartition) { + if (!startOffsets.containsKey(encodedPartition)) { return; } - long startOffset = startOffsets.get(encodedPartiton); - long endOffset = offsets.get(encodedPartiton); - String tempFile = tempFiles.get(encodedPartiton); - String directory = getDirectory(encodedPartiton); + long startOffset = startOffsets.get(encodedPartition); + long endOffset = offsets.get(encodedPartition); + String tempFile = tempFiles.get(encodedPartition); + String directory = getDirectory(encodedPartition); String committedFile = FileUtils.committedFileName(url, topicsDir, directory, tp, startOffset, endOffset, extension, zeroPadOffsetFormat); @@ -603,14 +603,14 @@ private void commitFile(String encodedPartiton) { storage.mkdirs(directoryName); } storage.commit(tempFile, committedFile); - startOffsets.remove(encodedPartiton); + startOffsets.remove(encodedPartition); offset = offset + recordCounter; recordCounter = 0; log.info("Committed {} for {}", committedFile, tp); } - private void deleteTempFile(String encodedPartiton) { - storage.delete(tempFiles.get(encodedPartiton)); + private void deleteTempFile(String encodedPartition) { + storage.delete(tempFiles.get(encodedPartition)); } private void setRetryTimeout(long timeoutMs) { From 2ef2a2991d1148440bbea4755624f7ebaf75d807 Mon Sep 17 00:00:00 2001 From: Konstantine Karantasis Date: Thu, 5 Jan 2017 17:39:56 -0800 Subject: [PATCH 22/58] Remove unused argument in DataWriter.close. --- .../io/confluent/connect/hdfs/DataWriter.java | 2 +- .../confluent/connect/hdfs/HdfsSinkTask.java | 4 +- .../connect/hdfs/FailureRecoveryTest.java | 6 +- .../connect/hdfs/avro/AvroHiveUtilTest.java | 2 +- .../connect/hdfs/avro/DataWriterAvroTest.java | 75 +++++++++---------- .../hdfs/avro/HiveIntegrationAvroTest.java | 10 +-- .../hdfs/parquet/DataWriterParquetTest.java | 2 +- .../parquet/HiveIntegrationParquetTest.java | 10 +-- .../hdfs/parquet/ParquetHiveUtilTest.java | 2 +- 9 files changed, 56 insertions(+), 57 deletions(-) diff --git a/src/main/java/io/confluent/connect/hdfs/DataWriter.java b/src/main/java/io/confluent/connect/hdfs/DataWriter.java index 41f84e0b9..3fd0f2d6f 100644 --- a/src/main/java/io/confluent/connect/hdfs/DataWriter.java +++ b/src/main/java/io/confluent/connect/hdfs/DataWriter.java @@ -282,7 +282,7 @@ public void open(Collection partitions) { } } - public void close(Collection partitions) { + public void close() { // Close any writers we have. We may get assigned the same partitions and end up duplicating // some effort since we'll have to reprocess those messages. It may be possible to hold on to // the TopicPartitionWriter and continue to use the temp file, but this can get significantly diff --git a/src/main/java/io/confluent/connect/hdfs/HdfsSinkTask.java b/src/main/java/io/confluent/connect/hdfs/HdfsSinkTask.java index c86e04aa7..3ca769959 100644 --- a/src/main/java/io/confluent/connect/hdfs/HdfsSinkTask.java +++ b/src/main/java/io/confluent/connect/hdfs/HdfsSinkTask.java @@ -83,7 +83,7 @@ public void start(Map props) { log.info("Couldn't start HdfsSinkConnector:", e); log.info("Shutting down HdfsSinkConnector."); if (hdfsWriter != null) { - hdfsWriter.close(assignment); + hdfsWriter.close(); hdfsWriter.stop(); } } @@ -117,7 +117,7 @@ public void open(Collection partitions) { @Override public void close(Collection partitions) { - hdfsWriter.close(partitions); + hdfsWriter.close(); } private void recover(Set assignment) { diff --git a/src/test/java/io/confluent/connect/hdfs/FailureRecoveryTest.java b/src/test/java/io/confluent/connect/hdfs/FailureRecoveryTest.java index 6e3ce5e78..5dff2c4d3 100644 --- a/src/test/java/io/confluent/connect/hdfs/FailureRecoveryTest.java +++ b/src/test/java/io/confluent/connect/hdfs/FailureRecoveryTest.java @@ -87,7 +87,7 @@ public void testCommitFailure() throws Exception { content = data.get(logFile); assertEquals(6, content.size()); - hdfsWriter.close(assignment); + hdfsWriter.close(); hdfsWriter.stop(); } @@ -163,7 +163,7 @@ public void testWriterFailureMultiPartitions() throws Exception { } hdfsWriter.write(new ArrayList()); - hdfsWriter.close(assignment); + hdfsWriter.close(); hdfsWriter.stop(); } @@ -226,7 +226,7 @@ public void testWriterFailure() throws Exception { } hdfsWriter.write(new ArrayList()); - hdfsWriter.close(assignment); + hdfsWriter.close(); hdfsWriter.stop(); } } diff --git a/src/test/java/io/confluent/connect/hdfs/avro/AvroHiveUtilTest.java b/src/test/java/io/confluent/connect/hdfs/avro/AvroHiveUtilTest.java index 74322e188..d0ecaf321 100644 --- a/src/test/java/io/confluent/connect/hdfs/avro/AvroHiveUtilTest.java +++ b/src/test/java/io/confluent/connect/hdfs/avro/AvroHiveUtilTest.java @@ -140,7 +140,7 @@ private void prepareData(String topic, int partition) throws Exception { List sinkRecords = createSinkRecords(7); hdfsWriter.write(sinkRecords); - hdfsWriter.close(assignment); + hdfsWriter.close(); hdfsWriter.stop(); } diff --git a/src/test/java/io/confluent/connect/hdfs/avro/DataWriterAvroTest.java b/src/test/java/io/confluent/connect/hdfs/avro/DataWriterAvroTest.java index ca80a7a8f..ba5885f41 100644 --- a/src/test/java/io/confluent/connect/hdfs/avro/DataWriterAvroTest.java +++ b/src/test/java/io/confluent/connect/hdfs/avro/DataWriterAvroTest.java @@ -58,7 +58,7 @@ public void testWriteRecord() throws Exception { List sinkRecords = createSinkRecords(7); hdfsWriter.write(sinkRecords); - hdfsWriter.close(assignment); + hdfsWriter.close(); hdfsWriter.stop(); // Last file (offset 6) doesn't satisfy size requirement and gets discarded on close @@ -101,7 +101,7 @@ public void testRecovery() throws Exception { List sinkRecords = createSinkRecords(3, 50); hdfsWriter.write(sinkRecords); - hdfsWriter.close(assignment); + hdfsWriter.close(); hdfsWriter.stop(); long[] validOffsets = {0, 10, 20, 30, 40, 50, 53}; @@ -113,19 +113,19 @@ public void testWriteRecordMultiplePartitions() throws Exception { DataWriter hdfsWriter = new DataWriter(connectorConfig, context, avroData); partitioner = hdfsWriter.getPartitioner(); - for (TopicPartition tp: assignment) { + for (TopicPartition tp : context.assignment()) { hdfsWriter.recover(tp); } - List sinkRecords = createSinkRecords(7, 0, assignment); + List sinkRecords = createSinkRecords(7, 0, context.assignment()); hdfsWriter.write(sinkRecords); - hdfsWriter.close(assignment); + hdfsWriter.close(); hdfsWriter.stop(); // Last file (offset 6) doesn't satisfy size requirement and gets discarded on close long[] validOffsets = {0, 3, 6}; - verify(sinkRecords, validOffsets, assignment); + verify(sinkRecords, validOffsets, context.assignment()); } @Test @@ -133,18 +133,19 @@ public void testWriteInterleavedRecordsInMultiplePartitions() throws Exception { DataWriter hdfsWriter = new DataWriter(connectorConfig, context, avroData); partitioner = hdfsWriter.getPartitioner(); - for (TopicPartition tp: assignment) { + for (TopicPartition tp : context.assignment()) { hdfsWriter.recover(tp); } - List sinkRecords = createSinkRecordsInterleaved(7 * assignment.size(), 0, assignment); + List sinkRecords = createSinkRecordsInterleaved(7 * context.assignment().size(), 0, + context.assignment()); hdfsWriter.write(sinkRecords); - hdfsWriter.close(assignment); + hdfsWriter.close(); hdfsWriter.stop(); long[] validOffsets = {0, 3, 6}; - verify(sinkRecords, validOffsets, assignment); + verify(sinkRecords, validOffsets, context.assignment()); } @Test @@ -152,14 +153,15 @@ public void testWriteInterleavedRecordsInMultiplePartitionsNonZeroInitialOffset( DataWriter hdfsWriter = new DataWriter(connectorConfig, context, avroData); partitioner = hdfsWriter.getPartitioner(); - List sinkRecords = createSinkRecordsInterleaved(7 * assignment.size(), 9, assignment); + List sinkRecords = createSinkRecordsInterleaved(7 * context.assignment().size(), 9, + context.assignment()); hdfsWriter.write(sinkRecords); - hdfsWriter.close(assignment); + hdfsWriter.close(); hdfsWriter.stop(); long[] validOffsets = {9, 12, 15}; - verify(sinkRecords, validOffsets, assignment); + verify(sinkRecords, validOffsets, context.assignment()); } @Test @@ -188,7 +190,7 @@ public void testGetPreviousOffsets() throws Exception { long previousOffset = committedOffsets.get(TOPIC_PARTITION); assertEquals(previousOffset, 6L); - hdfsWriter.close(assignment); + hdfsWriter.close(); hdfsWriter.stop(); } @@ -201,7 +203,7 @@ public void testWriteRecordNonZeroInitialOffset() throws Exception { List sinkRecords = createSinkRecords(7, 3); hdfsWriter.write(sinkRecords); - hdfsWriter.close(assignment); + hdfsWriter.close(); hdfsWriter.stop(); // Last file (offset 9) doesn't satisfy size requirement and gets discarded on close @@ -214,23 +216,23 @@ public void testRebalance() throws Exception { DataWriter hdfsWriter = new DataWriter(connectorConfig, context, avroData); partitioner = hdfsWriter.getPartitioner(); - // Initial assignment is {TP1, TP2} - for (TopicPartition tp: assignment) { + Set originalAssignment = new HashSet<>(context.assignment()); + // Starts with TOPIC_PARTITION and TOPIC_PARTITION2 + for (TopicPartition tp : originalAssignment) { hdfsWriter.recover(tp); } - List sinkRecords = createSinkRecords(7, 0, assignment); - hdfsWriter.write(sinkRecords); - - Set oldAssignment = new HashSet<>(assignment); + Set nextAssignment = new HashSet<>(); + nextAssignment.add(TOPIC_PARTITION); + nextAssignment.add(TOPIC_PARTITION3); - Set newAssignment = new HashSet<>(); - newAssignment.add(TOPIC_PARTITION); - newAssignment.add(TOPIC_PARTITION3); + List sinkRecords = createSinkRecords(7, 0, originalAssignment); - hdfsWriter.close(assignment); - assignment = newAssignment; - hdfsWriter.open(newAssignment); + hdfsWriter.write(sinkRecords); + hdfsWriter.close(); + // Set the new assignment to the context + context.setAssignment(nextAssignment); + hdfsWriter.open(nextAssignment); assertEquals(null, hdfsWriter.getBucketWriter(TOPIC_PARTITION2)); assertNotNull(hdfsWriter.getBucketWriter(TOPIC_PARTITION)); @@ -241,10 +243,10 @@ public void testRebalance() throws Exception { verify(sinkRecords, validOffsetsTopicPartition2, Collections.singleton(TOPIC_PARTITION2), true); // Message offsets start at 6 because we discarded the in-progress temp file on re-balance - sinkRecords = createSinkRecords(3, 6, assignment); + sinkRecords = createSinkRecords(3, 6, context.assignment()); hdfsWriter.write(sinkRecords); - hdfsWriter.close(newAssignment); + hdfsWriter.close(); hdfsWriter.stop(); // Last file (offset 9) doesn't satisfy size requirement and gets discarded on close @@ -253,8 +255,6 @@ public void testRebalance() throws Exception { long[] validOffsetsTopicPartition3 = {6, 9}; verify(sinkRecords, validOffsetsTopicPartition3, Collections.singleton(TOPIC_PARTITION3), true); - - assignment = oldAssignment; } @Test @@ -271,7 +271,7 @@ public void testProjectBackWard() throws Exception { List sinkRecords = createSinkRecordsWithAlternatingSchemas(7, 0); hdfsWriter.write(sinkRecords); - hdfsWriter.close(assignment); + hdfsWriter.close(); hdfsWriter.stop(); long[] validOffsets = {0, 1, 3, 5, 7}; verify(sinkRecords, validOffsets); @@ -290,7 +290,7 @@ public void testProjectNone() throws Exception { List sinkRecords = createSinkRecordsWithAlternatingSchemas(7, 0); hdfsWriter.write(sinkRecords); - hdfsWriter.close(assignment); + hdfsWriter.close(); hdfsWriter.stop(); long[] validOffsets = {0, 1, 2, 3, 4, 5, 6}; @@ -312,7 +312,7 @@ public void testProjectForward() throws Exception { List sinkRecords = createSinkRecordsWithAlternatingSchemas(8, 0).subList(1, 8); hdfsWriter.write(sinkRecords); - hdfsWriter.close(assignment); + hdfsWriter.close(); hdfsWriter.stop(); long[] validOffsets = {1, 2, 4, 6, 8}; @@ -338,7 +338,7 @@ public void testProjectNoVersion() throws Exception { } catch (RuntimeException e) { // expected } finally { - hdfsWriter.close(assignment); + hdfsWriter.close(); hdfsWriter.stop(); long[] validOffsets = {}; verify(Collections.emptyList(), validOffsets); @@ -359,8 +359,7 @@ public void testFlushPartialFile() throws Exception { props.put(HdfsSinkConnectorConfig.FLUSH_SIZE_CONFIG, FLUSH_SIZE_CONFIG); props.put(HdfsSinkConnectorConfig.ROTATE_INTERVAL_MS_CONFIG, ROTATE_INTERVAL_MS_CONFIG); HdfsSinkConnectorConfig connectorConfig = new HdfsSinkConnectorConfig(props); - assignment = new HashSet<>(); - assignment.add(TOPIC_PARTITION); + context.assignment().add(TOPIC_PARTITION); DataWriter hdfsWriter = new DataWriter(connectorConfig, context, avroData); partitioner = hdfsWriter.getPartitioner(); @@ -382,7 +381,7 @@ public void testFlushPartialFile() throws Exception { long previousOffset = committedOffsets.get(TOPIC_PARTITION); assertEquals(NUMBER_OF_RECORDS, previousOffset); - hdfsWriter.close(assignment); + hdfsWriter.close(); hdfsWriter.stop(); } diff --git a/src/test/java/io/confluent/connect/hdfs/avro/HiveIntegrationAvroTest.java b/src/test/java/io/confluent/connect/hdfs/avro/HiveIntegrationAvroTest.java index bd6a9832b..988835144 100644 --- a/src/test/java/io/confluent/connect/hdfs/avro/HiveIntegrationAvroTest.java +++ b/src/test/java/io/confluent/connect/hdfs/avro/HiveIntegrationAvroTest.java @@ -67,7 +67,7 @@ public void testSyncWithHiveAvro() throws Exception { } hdfsWriter.write(sinkRecords); - hdfsWriter.close(assignment); + hdfsWriter.close(); hdfsWriter.stop(); Map props = createProps(); @@ -97,7 +97,7 @@ public void testSyncWithHiveAvro() throws Exception { assertEquals(expectedPartitions, partitions); - hdfsWriter.close(assignment); + hdfsWriter.close(); hdfsWriter.stop(); } @@ -123,7 +123,7 @@ public void testHiveIntegrationAvro() throws Exception { } hdfsWriter.write(sinkRecords); - hdfsWriter.close(assignment); + hdfsWriter.close(); hdfsWriter.stop(); Table table = hiveMetaStore.getTable(hiveDatabase, TOPIC); @@ -221,7 +221,7 @@ public void testHiveIntegrationFieldPartitionerAvro() throws Exception { } hdfsWriter.write(sinkRecords); - hdfsWriter.close(assignment); + hdfsWriter.close(); hdfsWriter.stop(); Table table = hiveMetaStore.getTable(hiveDatabase, TOPIC); @@ -297,7 +297,7 @@ public void testHiveIntegrationTimeBasedPartitionerAvro() throws Exception { } hdfsWriter.write(sinkRecords); - hdfsWriter.close(assignment); + hdfsWriter.close(); hdfsWriter.stop(); Table table = hiveMetaStore.getTable(hiveDatabase, TOPIC); diff --git a/src/test/java/io/confluent/connect/hdfs/parquet/DataWriterParquetTest.java b/src/test/java/io/confluent/connect/hdfs/parquet/DataWriterParquetTest.java index ac4ae9930..5ed7df97b 100644 --- a/src/test/java/io/confluent/connect/hdfs/parquet/DataWriterParquetTest.java +++ b/src/test/java/io/confluent/connect/hdfs/parquet/DataWriterParquetTest.java @@ -52,7 +52,7 @@ public void testWriteRecord() throws Exception { List sinkRecords = createSinkRecords(7); hdfsWriter.write(sinkRecords); - hdfsWriter.close(assignment); + hdfsWriter.close(); hdfsWriter.stop(); // Last file (offset 6) doesn't satisfy size requirement and gets discarded on close diff --git a/src/test/java/io/confluent/connect/hdfs/parquet/HiveIntegrationParquetTest.java b/src/test/java/io/confluent/connect/hdfs/parquet/HiveIntegrationParquetTest.java index 5a7601e52..77a0fd650 100644 --- a/src/test/java/io/confluent/connect/hdfs/parquet/HiveIntegrationParquetTest.java +++ b/src/test/java/io/confluent/connect/hdfs/parquet/HiveIntegrationParquetTest.java @@ -61,7 +61,7 @@ public void testSyncWithHiveParquet() throws Exception { List sinkRecords = createSinkRecords(7); hdfsWriter.write(sinkRecords); - hdfsWriter.close(assignment); + hdfsWriter.close(); hdfsWriter.stop(); props = createProps(); @@ -95,7 +95,7 @@ public void testSyncWithHiveParquet() throws Exception { assertEquals(expectedPartitions, partitions); - hdfsWriter.close(assignment); + hdfsWriter.close(); hdfsWriter.stop(); } @@ -111,7 +111,7 @@ public void testHiveIntegrationParquet() throws Exception { List sinkRecords = createSinkRecords(7); hdfsWriter.write(sinkRecords); - hdfsWriter.close(assignment); + hdfsWriter.close(); hdfsWriter.stop(); Schema schema = createSchema(); @@ -151,7 +151,7 @@ public void testHiveIntegrationFieldPartitionerParquet() throws Exception { List sinkRecords = createSinkRecords(records, schema); hdfsWriter.write(sinkRecords); - hdfsWriter.close(assignment); + hdfsWriter.close(); hdfsWriter.stop(); Table table = hiveMetaStore.getTable(hiveDatabase, TOPIC); @@ -220,7 +220,7 @@ public void testHiveIntegrationTimeBasedPartitionerParquet() throws Exception { List sinkRecords = createSinkRecords(records, schema); hdfsWriter.write(sinkRecords); - hdfsWriter.close(assignment); + hdfsWriter.close(); hdfsWriter.stop(); Table table = hiveMetaStore.getTable(hiveDatabase, TOPIC); diff --git a/src/test/java/io/confluent/connect/hdfs/parquet/ParquetHiveUtilTest.java b/src/test/java/io/confluent/connect/hdfs/parquet/ParquetHiveUtilTest.java index 7f11763d1..98a047102 100644 --- a/src/test/java/io/confluent/connect/hdfs/parquet/ParquetHiveUtilTest.java +++ b/src/test/java/io/confluent/connect/hdfs/parquet/ParquetHiveUtilTest.java @@ -158,7 +158,7 @@ private void prepareData(String topic, int partition) { sinkRecords.add(sinkRecord); } hdfsWriter.write(sinkRecords); - hdfsWriter.close(assignment); + hdfsWriter.close(); hdfsWriter.stop(); } From 900134ded70a18f6e8a2f9d551aa55e2257b5e62 Mon Sep 17 00:00:00 2001 From: Konstantine Karantasis Date: Sun, 8 Jan 2017 11:34:47 -0800 Subject: [PATCH 23/58] Add open and create methods in storage. --- .../connect/hdfs/storage/HdfsStorage.java | 22 ++++++++++++++++++- .../connect/hdfs/storage/Storage.java | 2 +- .../connect/hdfs/utils/MemoryStorage.java | 12 ++++++++++ 3 files changed, 34 insertions(+), 2 deletions(-) diff --git a/src/main/java/io/confluent/connect/hdfs/storage/HdfsStorage.java b/src/main/java/io/confluent/connect/hdfs/storage/HdfsStorage.java index 328544e92..33a9df521 100644 --- a/src/main/java/io/confluent/connect/hdfs/storage/HdfsStorage.java +++ b/src/main/java/io/confluent/connect/hdfs/storage/HdfsStorage.java @@ -16,6 +16,8 @@ package io.confluent.connect.hdfs.storage; +import org.apache.avro.file.SeekableInput; +import org.apache.avro.mapred.FsInput; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; @@ -25,6 +27,7 @@ import org.apache.kafka.connect.errors.ConnectException; import java.io.IOException; +import java.io.OutputStream; import java.net.URI; import java.util.Arrays; import java.util.List; @@ -32,7 +35,7 @@ import io.confluent.connect.hdfs.wal.FSWAL; import io.confluent.connect.storage.wal.WAL; -public class HdfsStorage implements io.confluent.connect.storage.Storage, PathFilter, Configuration>, +public class HdfsStorage implements io.confluent.connect.storage.Storage>, Storage { private final FileSystem fs; @@ -138,4 +141,21 @@ private void renameFile(String sourcePath, String targetPath) { throw new ConnectException(e); } } + + public SeekableInput open(String filename, Configuration conf) { + try { + return new FsInput(new Path(filename), conf); + } catch (IOException e) { + throw new ConnectException(e); + } + } + + public OutputStream create(String filename, Configuration conf, boolean overwrite) { + try { + Path path = new Path(filename); + return path.getFileSystem(conf).create(path); + } catch (IOException e) { + throw new ConnectException(e); + } + } } diff --git a/src/main/java/io/confluent/connect/hdfs/storage/Storage.java b/src/main/java/io/confluent/connect/hdfs/storage/Storage.java index 366090c77..9c5507de1 100644 --- a/src/main/java/io/confluent/connect/hdfs/storage/Storage.java +++ b/src/main/java/io/confluent/connect/hdfs/storage/Storage.java @@ -23,4 +23,4 @@ import java.util.List; @Deprecated -public interface Storage extends io.confluent.connect.storage.Storage, PathFilter, Configuration> {} +public interface Storage extends io.confluent.connect.storage.Storage> {} diff --git a/src/test/java/io/confluent/connect/hdfs/utils/MemoryStorage.java b/src/test/java/io/confluent/connect/hdfs/utils/MemoryStorage.java index 1fa2cc337..9839fa451 100644 --- a/src/test/java/io/confluent/connect/hdfs/utils/MemoryStorage.java +++ b/src/test/java/io/confluent/connect/hdfs/utils/MemoryStorage.java @@ -16,6 +16,7 @@ package io.confluent.connect.hdfs.utils; +import org.apache.avro.file.SeekableInput; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; @@ -23,6 +24,7 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.connect.errors.ConnectException; +import java.io.OutputStream; import java.util.ArrayList; import java.util.LinkedList; import java.util.List; @@ -164,4 +166,14 @@ public String url() { public void setFailure(Failure failure) { this.failure = failure; } + + @Override + public SeekableInput open(String filename, Configuration conf) { + return null; + } + + @Override + public OutputStream create(String filename, Configuration conf, boolean overwrite) { + return null; + } } From 757c4f19f17c8e9b4f87c1be0fdf050cd64ddae3 Mon Sep 17 00:00:00 2001 From: Konstantine Karantasis Date: Mon, 6 Mar 2017 13:29:04 -0800 Subject: [PATCH 24/58] Consolidate if-else-branch in TopicPartitionWriter. --- .../java/io/confluent/connect/hdfs/TopicPartitionWriter.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/src/main/java/io/confluent/connect/hdfs/TopicPartitionWriter.java b/src/main/java/io/confluent/connect/hdfs/TopicPartitionWriter.java index de1fe9213..5a6b3e8f0 100644 --- a/src/main/java/io/confluent/connect/hdfs/TopicPartitionWriter.java +++ b/src/main/java/io/confluent/connect/hdfs/TopicPartitionWriter.java @@ -520,10 +520,8 @@ private void writeRecord(SinkRecord record) { if (!startOffsets.containsKey(encodedPartition)) { startOffsets.put(encodedPartition, record.kafkaOffset()); - offsets.put(encodedPartition, record.kafkaOffset()); - } else { - offsets.put(encodedPartition, record.kafkaOffset()); } + offsets.put(encodedPartition, record.kafkaOffset()); recordCounter++; } From bd9ecd71950788be500aa1099c2768695a5a7bae Mon Sep 17 00:00:00 2001 From: Konstantine Karantasis Date: Mon, 6 Mar 2017 13:29:59 -0800 Subject: [PATCH 25/58] Add individual configs. Need to overwrite storage.class default. --- .../connect/hdfs/HdfsSinkConnectorConfig.java | 246 ++++++++++++++++-- .../connect/hdfs/avro/AvroFileReader.java | 1 - .../confluent/connect/hdfs/hive/HiveUtil.java | 14 +- 3 files changed, 236 insertions(+), 25 deletions(-) diff --git a/src/main/java/io/confluent/connect/hdfs/HdfsSinkConnectorConfig.java b/src/main/java/io/confluent/connect/hdfs/HdfsSinkConnectorConfig.java index 8b079bc7d..b0334d4c8 100644 --- a/src/main/java/io/confluent/connect/hdfs/HdfsSinkConnectorConfig.java +++ b/src/main/java/io/confluent/connect/hdfs/HdfsSinkConnectorConfig.java @@ -14,21 +14,39 @@ package io.confluent.connect.hdfs; +import org.apache.kafka.common.config.AbstractConfig; import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.common.config.ConfigDef.Importance; import org.apache.kafka.common.config.ConfigDef.Type; import org.apache.kafka.common.config.ConfigDef.Width; import java.util.Arrays; +import java.util.HashMap; +import java.util.HashSet; import java.util.LinkedList; import java.util.List; import java.util.Map; +import java.util.Set; import io.confluent.connect.storage.StorageSinkConnectorConfig; +import io.confluent.connect.storage.common.ComposableConfig; +import io.confluent.connect.storage.common.StorageCommonConfig; +import io.confluent.connect.storage.hive.HiveConfig; +import io.confluent.connect.storage.partitioner.PartitionerConfig; public class HdfsSinkConnectorConfig extends StorageSinkConnectorConfig { // HDFS Group + + // This config is deprecated and will be removed in future releases. Use store.url instead. + public static final String HDFS_URL_CONFIG = "hdfs.url"; + public static final String HDFS_URL_DOC = + "The HDFS connection URL. This configuration has the format of hdfs:://hostname:port and " + + "specifies the HDFS to export data to. This property is deprecated and will be removed in future releases. " + + "Use ``store.url`` instead."; + public static final String HDFS_URL_DEFAULT = null; + public static final String HDFS_URL_DISPLAY = "HDFS URL"; + public static final String HADOOP_CONF_DIR_CONFIG = "hadoop.conf.dir"; private static final String HADOOP_CONF_DIR_DOC = "The Hadoop configuration directory."; @@ -41,6 +59,12 @@ public class HdfsSinkConnectorConfig extends StorageSinkConnectorConfig { public static final String HADOOP_HOME_DEFAULT = ""; private static final String HADOOP_HOME_DISPLAY = "Hadoop home directory"; + public static final String LOGS_DIR_CONFIG = "logs.dir"; + public static final String LOGS_DIR_DOC = + "Top level directory to store the write ahead logs."; + public static final String LOGS_DIR_DEFAULT = "logs"; + public static final String LOGS_DIR_DISPLAY = "Logs directory"; + // Security group public static final String HDFS_AUTHENTICATION_KERBEROS_CONFIG = "hdfs.authentication.kerberos"; private static final String HDFS_AUTHENTICATION_KERBEROS_DOC = @@ -72,28 +96,197 @@ public class HdfsSinkConnectorConfig extends StorageSinkConnectorConfig { public static final long KERBEROS_TICKET_RENEW_PERIOD_MS_DEFAULT = 60000 * 60; private static final String KERBEROS_TICKET_RENEW_PERIOD_MS_DISPLAY = "Kerberos Ticket Renew Period (ms)"; - public static final String HDFS_GROUP = "HDFS"; - public static final String SECURITY_GROUP = "Security"; + // Internal group + // Need to just set the default + public static final String STORAGE_CLASS_DEFAULT = "io.confluent.connect.hdfs.storage.HdfsStorage"; private static final ConfigDef.Recommender hdfsAuthenticationKerberosDependentsRecommender = new BooleanParentRecommender(HDFS_AUTHENTICATION_KERBEROS_CONFIG); + private final String name; + + private final StorageCommonConfig commonConfig; + private final HiveConfig hiveConfig; + private final PartitionerConfig partitionerConfig; + + private final Map propertyToConfig = new HashMap<>(); + private final Set allConfigs = new HashSet<>(); + static { // Define HDFS configuration group - CONFIG_DEF.define(HADOOP_CONF_DIR_CONFIG, Type.STRING, HADOOP_CONF_DIR_DEFAULT, Importance.HIGH, HADOOP_CONF_DIR_DOC, HDFS_GROUP, 2, Width.MEDIUM, HADOOP_CONF_DIR_DISPLAY) - .define(HADOOP_HOME_CONFIG, Type.STRING, HADOOP_HOME_DEFAULT, Importance.HIGH, HADOOP_HOME_DOC, HDFS_GROUP, 3, Width.SHORT, HADOOP_HOME_DISPLAY); - - // Define Security configuration group - CONFIG_DEF.define(HDFS_AUTHENTICATION_KERBEROS_CONFIG, Type.BOOLEAN, HDFS_AUTHENTICATION_KERBEROS_DEFAULT, Importance.HIGH, HDFS_AUTHENTICATION_KERBEROS_DOC, - SECURITY_GROUP, 1, Width.SHORT, HDFS_AUTHENTICATION_KERBEROS_DISPLAY, - Arrays.asList(CONNECT_HDFS_PRINCIPAL_CONFIG, CONNECT_HDFS_KEYTAB_CONFIG, HDFS_NAMENODE_PRINCIPAL_CONFIG, KERBEROS_TICKET_RENEW_PERIOD_MS_CONFIG)) - .define(CONNECT_HDFS_PRINCIPAL_CONFIG, Type.STRING, CONNECT_HDFS_PRINCIPAL_DEFAULT, Importance.HIGH, CONNECT_HDFS_PRINCIPAL_DOC, - SECURITY_GROUP, 2, Width.MEDIUM, CONNECT_HDFS_PRINCIPAL_DISPLAY, hdfsAuthenticationKerberosDependentsRecommender) - .define(CONNECT_HDFS_KEYTAB_CONFIG, Type.STRING, CONNECT_HDFS_KEYTAB_DEFAULT, Importance.HIGH, CONNECT_HDFS_KEYTAB_DOC, - SECURITY_GROUP, 3, Width.MEDIUM, CONNECT_HDFS_KEYTAB_DISPLAY, hdfsAuthenticationKerberosDependentsRecommender) - .define(HDFS_NAMENODE_PRINCIPAL_CONFIG, Type.STRING, HDFS_NAMENODE_PRINCIPAL_DEFAULT, Importance.HIGH, HDFS_NAMENODE_PRINCIPAL_DOC, - SECURITY_GROUP, 4, Width.MEDIUM, HDFS_NAMENODE_PRINCIPAL_DISPLAY, hdfsAuthenticationKerberosDependentsRecommender) - .define(KERBEROS_TICKET_RENEW_PERIOD_MS_CONFIG, Type.LONG, KERBEROS_TICKET_RENEW_PERIOD_MS_DEFAULT, Importance.LOW, KERBEROS_TICKET_RENEW_PERIOD_MS_DOC, - SECURITY_GROUP, 5, Width.SHORT, KERBEROS_TICKET_RENEW_PERIOD_MS_DISPLAY, hdfsAuthenticationKerberosDependentsRecommender); + { + final String group = "HDFS"; + int orderInGroup = 0; + + // HDFS_URL_CONFIG property is retained for backwards compatibility with HDFS connector and + // will be removed in future versions. + CONFIG_DEF.define( + HDFS_URL_CONFIG, + Type.STRING, + HDFS_URL_DEFAULT, + Importance.HIGH, + HDFS_URL_DOC, + group, + ++orderInGroup, + Width.MEDIUM, + HDFS_URL_DISPLAY + ); + + CONFIG_DEF.define( + HADOOP_CONF_DIR_CONFIG, + Type.STRING, + HADOOP_CONF_DIR_DEFAULT, + Importance.HIGH, + HADOOP_CONF_DIR_DOC, + group, + ++orderInGroup, + Width.MEDIUM, + HADOOP_CONF_DIR_DISPLAY + ); + + CONFIG_DEF.define( + HADOOP_HOME_CONFIG, + Type.STRING, + HADOOP_HOME_DEFAULT, + Importance.HIGH, + HADOOP_HOME_DOC, + group, + ++orderInGroup, + Width.SHORT, + HADOOP_HOME_DISPLAY + ); + + CONFIG_DEF.define( + LOGS_DIR_CONFIG, + Type.STRING, + LOGS_DIR_DEFAULT, + Importance.HIGH, + LOGS_DIR_DOC, + group, + ++orderInGroup, + Width.SHORT, + LOGS_DIR_DISPLAY + ); + } + + { + final String group = "Security"; + int orderInGroup = 0; + // Define Security configuration group + CONFIG_DEF.define( + HDFS_AUTHENTICATION_KERBEROS_CONFIG, + Type.BOOLEAN, + HDFS_AUTHENTICATION_KERBEROS_DEFAULT, + Importance.HIGH, + HDFS_AUTHENTICATION_KERBEROS_DOC, + group, + ++orderInGroup, + Width.SHORT, + HDFS_AUTHENTICATION_KERBEROS_DISPLAY, + Arrays.asList( + CONNECT_HDFS_PRINCIPAL_CONFIG, + CONNECT_HDFS_KEYTAB_CONFIG, + HDFS_NAMENODE_PRINCIPAL_CONFIG, + KERBEROS_TICKET_RENEW_PERIOD_MS_CONFIG + ) + ); + + CONFIG_DEF.define( + CONNECT_HDFS_PRINCIPAL_CONFIG, + Type.STRING, + CONNECT_HDFS_PRINCIPAL_DEFAULT, + Importance.HIGH, + CONNECT_HDFS_PRINCIPAL_DOC, + group, + ++orderInGroup, + Width.MEDIUM, + CONNECT_HDFS_PRINCIPAL_DISPLAY, + hdfsAuthenticationKerberosDependentsRecommender + ); + + CONFIG_DEF.define( + CONNECT_HDFS_KEYTAB_CONFIG, + Type.STRING, + CONNECT_HDFS_KEYTAB_DEFAULT, + Importance.HIGH, + CONNECT_HDFS_KEYTAB_DOC, + group, + ++orderInGroup, + Width.MEDIUM, + CONNECT_HDFS_KEYTAB_DISPLAY, + hdfsAuthenticationKerberosDependentsRecommender + ); + + CONFIG_DEF.define( + HDFS_NAMENODE_PRINCIPAL_CONFIG, + Type.STRING, + HDFS_NAMENODE_PRINCIPAL_DEFAULT, + Importance.HIGH, + HDFS_NAMENODE_PRINCIPAL_DOC, + group, + ++orderInGroup, + Width.MEDIUM, + HDFS_NAMENODE_PRINCIPAL_DISPLAY, + hdfsAuthenticationKerberosDependentsRecommender + ); + + CONFIG_DEF.define( + KERBEROS_TICKET_RENEW_PERIOD_MS_CONFIG, + Type.LONG, + KERBEROS_TICKET_RENEW_PERIOD_MS_DEFAULT, + Importance.LOW, + KERBEROS_TICKET_RENEW_PERIOD_MS_DOC, + group, + ++orderInGroup, + Width.SHORT, + KERBEROS_TICKET_RENEW_PERIOD_MS_DISPLAY, + hdfsAuthenticationKerberosDependentsRecommender + ); + } + + } + + public HdfsSinkConnectorConfig(Map props) { + this(CONFIG_DEF, props); + } + + protected HdfsSinkConnectorConfig(ConfigDef configDef, Map props) { + super(configDef, props); + commonConfig = new StorageCommonConfig(originalsStrings()); + hiveConfig = new HiveConfig(originalsStrings()); + partitionerConfig = new PartitionerConfig(originalsStrings()); + this.name = parseName(originalsStrings()); + addToGlobal(hiveConfig); + addToGlobal(partitionerConfig); + addToGlobal(commonConfig); + addToGlobal(this); + } + + private void addToGlobal(AbstractConfig config) { + allConfigs.add(config); + addConfig(config.values(), (ComposableConfig) config); + } + + private void addConfig(Map parsedProps, ComposableConfig config) { + for (String key : parsedProps.keySet()) { + propertyToConfig.put(key, config); + } + } + + protected static String parseName(Map props) { + String nameProp = props.get("name"); + return nameProp != null ? nameProp : "S3-sink"; + } + + public String getName() { + return name; + } + + public Map plainValues() { + Map map = new HashMap<>(); + for (AbstractConfig config : allConfigs) { + map.putAll(config.values()); + } + return map; } private static class BooleanParentRecommender implements ConfigDef.Recommender { @@ -116,10 +309,21 @@ public boolean visible(String name, Map connectorConfigs) { } public static ConfigDef getConfig() { - return CONFIG_DEF; - } + Map everything = new HashMap<>(CONFIG_DEF.configKeys()); + everything.putAll(StorageCommonConfig.getConfig().configKeys()); + everything.putAll(PartitionerConfig.getConfig().configKeys()); - public HdfsSinkConnectorConfig(Map props) { - super(props); + Set blacklist = new HashSet<>(); + blacklist.add(StorageSinkConnectorConfig.ROTATE_INTERVAL_MS_CONFIG); + blacklist.add(StorageSinkConnectorConfig.ROTATE_SCHEDULE_INTERVAL_MS_CONFIG); + blacklist.add(StorageSinkConnectorConfig.SHUTDOWN_TIMEOUT_CONFIG); + + ConfigDef visible = new ConfigDef(); + for (ConfigDef.ConfigKey key : everything.values()) { + if(!blacklist.contains(key.name)) { + visible.define(key); + } + } + return visible; } } diff --git a/src/main/java/io/confluent/connect/hdfs/avro/AvroFileReader.java b/src/main/java/io/confluent/connect/hdfs/avro/AvroFileReader.java index 761102b99..bff91162f 100644 --- a/src/main/java/io/confluent/connect/hdfs/avro/AvroFileReader.java +++ b/src/main/java/io/confluent/connect/hdfs/avro/AvroFileReader.java @@ -23,7 +23,6 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.kafka.connect.data.Schema; -import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.errors.DataException; import java.io.IOException; diff --git a/src/main/java/io/confluent/connect/hdfs/hive/HiveUtil.java b/src/main/java/io/confluent/connect/hdfs/hive/HiveUtil.java index 021f8d83e..948d46489 100644 --- a/src/main/java/io/confluent/connect/hdfs/hive/HiveUtil.java +++ b/src/main/java/io/confluent/connect/hdfs/hive/HiveUtil.java @@ -17,15 +17,23 @@ import org.apache.hadoop.hive.ql.metadata.Table; import org.apache.kafka.connect.data.Schema; -import io.confluent.connect.avro.AvroData; import io.confluent.connect.hdfs.HdfsSinkConnectorConfig; import io.confluent.connect.hdfs.partitioner.Partitioner; +import io.confluent.connect.storage.common.StorageCommonConfig; @Deprecated public abstract class HiveUtil extends io.confluent.connect.storage.hive.HiveUtil { - public HiveUtil(HdfsSinkConnectorConfig connectorConfig, AvroData avroData, HiveMetaStore hiveMetaStore) { - super(connectorConfig, avroData, hiveMetaStore); + public HiveUtil(HdfsSinkConnectorConfig connectorConfig, HiveMetaStore hiveMetaStore) { + super(connectorConfig, hiveMetaStore); + String urlKey; + + urlKey = connectorConfig.getString(StorageCommonConfig.STORE_URL_CONFIG); + if (urlKey == null || urlKey.equals(StorageCommonConfig.STORE_URL_DEFAULT)) { + urlKey = connectorConfig.getString(HdfsSinkConnectorConfig.HDFS_URL_CONFIG); + } + + this.url = urlKey; } @Override From e8cec3a3b4205427d2f0a406658705bb2637a40f Mon Sep 17 00:00:00 2001 From: Konstantine Karantasis Date: Sat, 22 Apr 2017 17:44:26 -0700 Subject: [PATCH 26/58] Fix storage package. --- .../connect/hdfs/storage/HdfsStorage.java | 21 ++++++++++--------- .../connect/hdfs/storage/Storage.java | 4 ++-- 2 files changed, 13 insertions(+), 12 deletions(-) diff --git a/src/main/java/io/confluent/connect/hdfs/storage/HdfsStorage.java b/src/main/java/io/confluent/connect/hdfs/storage/HdfsStorage.java index 33a9df521..08dbdaa61 100644 --- a/src/main/java/io/confluent/connect/hdfs/storage/HdfsStorage.java +++ b/src/main/java/io/confluent/connect/hdfs/storage/HdfsStorage.java @@ -33,10 +33,10 @@ import java.util.List; import io.confluent.connect.hdfs.wal.FSWAL; -import io.confluent.connect.storage.wal.WAL; +import io.confluent.connect.hdfs.wal.WAL; -public class HdfsStorage implements io.confluent.connect.storage.Storage>, - Storage { +public class HdfsStorage + implements io.confluent.connect.storage.Storage>, Storage { private final FileSystem fs; private final Configuration conf; @@ -48,8 +48,7 @@ public HdfsStorage(Configuration conf, String url) throws IOException { this.url = url; } - @Override - public List listStatus(String path, PathFilter filter) { + public List list(String path, PathFilter filter) { try { return Arrays.asList(fs.listStatus(new Path(path), filter)); } catch (IOException e) { @@ -58,7 +57,7 @@ public List listStatus(String path, PathFilter filter) { } @Override - public List listStatus(String path) { + public List list(String path) { try { return Arrays.asList(fs.listStatus(new Path(path))); } catch (IOException e) { @@ -67,10 +66,12 @@ public List listStatus(String path) { } @Override - public void append(String filename, Object object) {} + public OutputStream append(String filename) { + throw new UnsupportedOperationException(); + } @Override - public boolean mkdirs(String filename) { + public boolean create(String filename) { try { return fs.mkdirs(new Path(filename)); } catch (IOException e) { @@ -87,7 +88,6 @@ public boolean exists(String filename) { } } - @Override public void commit(String tempFile, String committedFile) { renameFile(tempFile, committedFile); } @@ -112,7 +112,6 @@ public void close() { } } - @Override public WAL wal(String topicsDir, TopicPartition topicPart) { return new FSWAL(topicsDir, topicPart, this); } @@ -142,6 +141,7 @@ private void renameFile(String sourcePath, String targetPath) { } } + @Override public SeekableInput open(String filename, Configuration conf) { try { return new FsInput(new Path(filename), conf); @@ -150,6 +150,7 @@ public SeekableInput open(String filename, Configuration conf) { } } + @Override public OutputStream create(String filename, Configuration conf, boolean overwrite) { try { Path path = new Path(filename); diff --git a/src/main/java/io/confluent/connect/hdfs/storage/Storage.java b/src/main/java/io/confluent/connect/hdfs/storage/Storage.java index 9c5507de1..f753ee8ad 100644 --- a/src/main/java/io/confluent/connect/hdfs/storage/Storage.java +++ b/src/main/java/io/confluent/connect/hdfs/storage/Storage.java @@ -18,9 +18,9 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.PathFilter; import java.util.List; @Deprecated -public interface Storage extends io.confluent.connect.storage.Storage> {} +public interface Storage + extends io.confluent.connect.storage.Storage> {} From d1c954100294f807342ea8a15dfae0613bf0febe Mon Sep 17 00:00:00 2001 From: Konstantine Karantasis Date: Sat, 22 Apr 2017 17:45:04 -0700 Subject: [PATCH 27/58] Fix tests after changes in configs and storage package. --- .../connect/hdfs/HdfsSinkConnectorTestBase.java | 3 ++- .../confluent/connect/hdfs/HdfsSinkTaskTest.java | 1 - .../connect/hdfs/avro/DataWriterAvroTest.java | 15 ++++++--------- 3 files changed, 8 insertions(+), 11 deletions(-) diff --git a/src/test/java/io/confluent/connect/hdfs/HdfsSinkConnectorTestBase.java b/src/test/java/io/confluent/connect/hdfs/HdfsSinkConnectorTestBase.java index 516826f31..35b721095 100644 --- a/src/test/java/io/confluent/connect/hdfs/HdfsSinkConnectorTestBase.java +++ b/src/test/java/io/confluent/connect/hdfs/HdfsSinkConnectorTestBase.java @@ -29,6 +29,7 @@ import io.confluent.connect.avro.AvroData; import io.confluent.connect.storage.StorageSinkTestBase; +import io.confluent.connect.storage.common.StorageCommonConfig; public class HdfsSinkConnectorTestBase extends StorageSinkTestBase { @@ -99,7 +100,7 @@ public void tearDown() throws Exception { protected void configureConnector() { connectorConfig = new HdfsSinkConnectorConfig(properties); - topicsDir = connectorConfig.getString(HdfsSinkConnectorConfig.TOPICS_DIR_CONFIG); + topicsDir = connectorConfig.getString(StorageCommonConfig.TOPICS_DIR_CONFIG); logsDir = connectorConfig.getString(HdfsSinkConnectorConfig.LOGS_DIR_CONFIG); int schemaCacheSize = connectorConfig.getInt(HdfsSinkConnectorConfig.SCHEMA_CACHE_SIZE_CONFIG); avroData = new AvroData(schemaCacheSize); diff --git a/src/test/java/io/confluent/connect/hdfs/HdfsSinkTaskTest.java b/src/test/java/io/confluent/connect/hdfs/HdfsSinkTaskTest.java index a098ce91f..eba45e47a 100644 --- a/src/test/java/io/confluent/connect/hdfs/HdfsSinkTaskTest.java +++ b/src/test/java/io/confluent/connect/hdfs/HdfsSinkTaskTest.java @@ -34,7 +34,6 @@ import io.confluent.connect.hdfs.storage.Storage; import io.confluent.connect.storage.StorageFactory; import io.confluent.connect.storage.wal.WAL; -import io.confluent.kafka.serializers.NonRecordContainer; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; diff --git a/src/test/java/io/confluent/connect/hdfs/avro/DataWriterAvroTest.java b/src/test/java/io/confluent/connect/hdfs/avro/DataWriterAvroTest.java index ba5885f41..7a419e934 100644 --- a/src/test/java/io/confluent/connect/hdfs/avro/DataWriterAvroTest.java +++ b/src/test/java/io/confluent/connect/hdfs/avro/DataWriterAvroTest.java @@ -31,9 +31,9 @@ import io.confluent.connect.hdfs.FileUtils; import io.confluent.connect.hdfs.HdfsSinkConnectorConfig; import io.confluent.connect.hdfs.TestWithMiniDFSCluster; -import io.confluent.connect.hdfs.storage.Storage; -import io.confluent.connect.hdfs.storage.StorageFactory; +import io.confluent.connect.hdfs.storage.HdfsStorage; import io.confluent.connect.hdfs.wal.WAL; +import io.confluent.connect.storage.hive.HiveConfig; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; @@ -70,10 +70,7 @@ public void testWriteRecord() throws Exception { public void testRecovery() throws Exception { fs.delete(new Path(FileUtils.directoryName(url, topicsDir, TOPIC_PARTITION)), true); - @SuppressWarnings("unchecked") - Class storageClass = (Class) - Class.forName(connectorConfig.getString(HdfsSinkConnectorConfig.STORAGE_CLASS_CONFIG)); - Storage storage = StorageFactory.createStorage(storageClass, conf, url); + HdfsStorage storage = new HdfsStorage(conf, url); DataWriter hdfsWriter = new DataWriter(connectorConfig, context, avroData); partitioner = hdfsWriter.getPartitioner(); @@ -261,7 +258,7 @@ public void testRebalance() throws Exception { public void testProjectBackWard() throws Exception { Map props = createProps(); props.put(HdfsSinkConnectorConfig.FLUSH_SIZE_CONFIG, "2"); - props.put(HdfsSinkConnectorConfig.SCHEMA_COMPATIBILITY_CONFIG, "BACKWARD"); + props.put(HiveConfig.SCHEMA_COMPATIBILITY_CONFIG, "BACKWARD"); HdfsSinkConnectorConfig connectorConfig = new HdfsSinkConnectorConfig(props); DataWriter hdfsWriter = new DataWriter(connectorConfig, context, avroData); @@ -301,7 +298,7 @@ public void testProjectNone() throws Exception { public void testProjectForward() throws Exception { Map props = createProps(); props.put(HdfsSinkConnectorConfig.FLUSH_SIZE_CONFIG, "2"); - props.put(HdfsSinkConnectorConfig.SCHEMA_COMPATIBILITY_CONFIG, "FORWARD"); + props.put(HiveConfig.SCHEMA_COMPATIBILITY_CONFIG, "FORWARD"); HdfsSinkConnectorConfig connectorConfig = new HdfsSinkConnectorConfig(props); DataWriter hdfsWriter = new DataWriter(connectorConfig, context, avroData); @@ -322,7 +319,7 @@ public void testProjectForward() throws Exception { @Test public void testProjectNoVersion() throws Exception { Map props = createProps(); - props.put(HdfsSinkConnectorConfig.SCHEMA_COMPATIBILITY_CONFIG, "BACKWARD"); + props.put(HiveConfig.SCHEMA_COMPATIBILITY_CONFIG, "BACKWARD"); HdfsSinkConnectorConfig connectorConfig = new HdfsSinkConnectorConfig(props); DataWriter hdfsWriter = new DataWriter(connectorConfig, context, avroData); From 4122de8ec6591976705b9b7657de512809c65f30 Mon Sep 17 00:00:00 2001 From: Konstantine Karantasis Date: Sat, 22 Apr 2017 18:05:51 -0700 Subject: [PATCH 28/58] Fix missing generic in partitioners. --- .../confluent/connect/hdfs/hive/HiveUtil.java | 20 ++++++++++++++----- .../hdfs/partitioner/DailyPartitioner.java | 6 +++++- .../hdfs/partitioner/DefaultPartitioner.java | 7 +++++-- .../hdfs/partitioner/FieldPartitioner.java | 5 ++++- .../hdfs/partitioner/HourlyPartitioner.java | 5 ++++- .../connect/hdfs/partitioner/Partitioner.java | 3 ++- .../partitioner/TimeBasedPartitioner.java | 5 ++++- 7 files changed, 39 insertions(+), 12 deletions(-) diff --git a/src/main/java/io/confluent/connect/hdfs/hive/HiveUtil.java b/src/main/java/io/confluent/connect/hdfs/hive/HiveUtil.java index 948d46489..a56b18c08 100644 --- a/src/main/java/io/confluent/connect/hdfs/hive/HiveUtil.java +++ b/src/main/java/io/confluent/connect/hdfs/hive/HiveUtil.java @@ -14,6 +14,7 @@ package io.confluent.connect.hdfs.hive; +import org.apache.hadoop.hive.metastore.api.FieldSchema; import org.apache.hadoop.hive.ql.metadata.Table; import org.apache.kafka.connect.data.Schema; @@ -37,17 +38,26 @@ public HiveUtil(HdfsSinkConnectorConfig connectorConfig, HiveMetaStore hiveMetaS } @Override - public void createTable(String database, String tableName, - Schema schema, io.confluent.connect.storage.partitioner.Partitioner partitioner) { + public void createTable( + String database, + String tableName, + Schema schema, + io.confluent.connect.storage.partitioner.Partitioner partitioner + ) { createTable(database, tableName, schema, (Partitioner) partitioner); } - public abstract void createTable(String database, String tableName, Schema schema, Partitioner partitioner); + public abstract void createTable( + String database, + String tableName, + Schema schema, + Partitioner partitioner + ); @Override public abstract void alterSchema(String database, String tableName, Schema schema); - - public Table newTable(String database, String table){ + + public Table newTable(String database, String table) { return new Table(database, hiveMetaStore.tableNameConverter(table)); } } diff --git a/src/main/java/io/confluent/connect/hdfs/partitioner/DailyPartitioner.java b/src/main/java/io/confluent/connect/hdfs/partitioner/DailyPartitioner.java index 30e11fa31..191d1a738 100644 --- a/src/main/java/io/confluent/connect/hdfs/partitioner/DailyPartitioner.java +++ b/src/main/java/io/confluent/connect/hdfs/partitioner/DailyPartitioner.java @@ -14,6 +14,10 @@ package io.confluent.connect.hdfs.partitioner; +import org.apache.hadoop.hive.metastore.api.FieldSchema; + @Deprecated -public class DailyPartitioner extends io.confluent.connect.storage.partitioner.DailyPartitioner implements Partitioner { +public class DailyPartitioner + extends io.confluent.connect.storage.partitioner.DailyPartitioner + implements Partitioner { } diff --git a/src/main/java/io/confluent/connect/hdfs/partitioner/DefaultPartitioner.java b/src/main/java/io/confluent/connect/hdfs/partitioner/DefaultPartitioner.java index ba4ce098e..29569de73 100644 --- a/src/main/java/io/confluent/connect/hdfs/partitioner/DefaultPartitioner.java +++ b/src/main/java/io/confluent/connect/hdfs/partitioner/DefaultPartitioner.java @@ -14,7 +14,10 @@ package io.confluent.connect.hdfs.partitioner; +import org.apache.hadoop.hive.metastore.api.FieldSchema; + @Deprecated -public class DefaultPartitioner extends io.confluent.connect.storage.partitioner.DefaultPartitioner - implements Partitioner{ +public class DefaultPartitioner + extends io.confluent.connect.storage.partitioner.DefaultPartitioner + implements Partitioner { } diff --git a/src/main/java/io/confluent/connect/hdfs/partitioner/FieldPartitioner.java b/src/main/java/io/confluent/connect/hdfs/partitioner/FieldPartitioner.java index 2ede53ae0..cd5d0d0d0 100644 --- a/src/main/java/io/confluent/connect/hdfs/partitioner/FieldPartitioner.java +++ b/src/main/java/io/confluent/connect/hdfs/partitioner/FieldPartitioner.java @@ -14,7 +14,10 @@ package io.confluent.connect.hdfs.partitioner; +import org.apache.hadoop.hive.metastore.api.FieldSchema; + @Deprecated -public class FieldPartitioner extends io.confluent.connect.storage.partitioner.FieldPartitioner +public class FieldPartitioner + extends io.confluent.connect.storage.partitioner.FieldPartitioner implements Partitioner { } diff --git a/src/main/java/io/confluent/connect/hdfs/partitioner/HourlyPartitioner.java b/src/main/java/io/confluent/connect/hdfs/partitioner/HourlyPartitioner.java index cf03ef6e8..d7de2ea15 100644 --- a/src/main/java/io/confluent/connect/hdfs/partitioner/HourlyPartitioner.java +++ b/src/main/java/io/confluent/connect/hdfs/partitioner/HourlyPartitioner.java @@ -14,7 +14,10 @@ package io.confluent.connect.hdfs.partitioner; +import org.apache.hadoop.hive.metastore.api.FieldSchema; + @Deprecated -public class HourlyPartitioner extends io.confluent.connect.storage.partitioner.HourlyPartitioner +public class HourlyPartitioner + extends io.confluent.connect.storage.partitioner.HourlyPartitioner implements Partitioner { } diff --git a/src/main/java/io/confluent/connect/hdfs/partitioner/Partitioner.java b/src/main/java/io/confluent/connect/hdfs/partitioner/Partitioner.java index 91150a9a8..221324cc8 100644 --- a/src/main/java/io/confluent/connect/hdfs/partitioner/Partitioner.java +++ b/src/main/java/io/confluent/connect/hdfs/partitioner/Partitioner.java @@ -25,7 +25,8 @@ * incoming records. */ @Deprecated -public interface Partitioner extends io.confluent.connect.storage.partitioner.Partitioner { +public interface Partitioner + extends io.confluent.connect.storage.partitioner.Partitioner { @Override void configure(Map config); @Override diff --git a/src/main/java/io/confluent/connect/hdfs/partitioner/TimeBasedPartitioner.java b/src/main/java/io/confluent/connect/hdfs/partitioner/TimeBasedPartitioner.java index c8989e97a..d96440270 100644 --- a/src/main/java/io/confluent/connect/hdfs/partitioner/TimeBasedPartitioner.java +++ b/src/main/java/io/confluent/connect/hdfs/partitioner/TimeBasedPartitioner.java @@ -14,7 +14,10 @@ package io.confluent.connect.hdfs.partitioner; +import org.apache.hadoop.hive.metastore.api.FieldSchema; + @Deprecated -public class TimeBasedPartitioner extends io.confluent.connect.storage.partitioner.TimeBasedPartitioner +public class TimeBasedPartitioner + extends io.confluent.connect.storage.partitioner.TimeBasedPartitioner implements Partitioner { } From 5a278581436dc9fdc4e8252f4c4f29f12100d29d Mon Sep 17 00:00:00 2001 From: Konstantine Karantasis Date: Sat, 22 Apr 2017 22:16:08 -0700 Subject: [PATCH 29/58] Embed hadoop config in connector's config. --- .../confluent/connect/hdfs/HdfsSinkConnectorConfig.java | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/src/main/java/io/confluent/connect/hdfs/HdfsSinkConnectorConfig.java b/src/main/java/io/confluent/connect/hdfs/HdfsSinkConnectorConfig.java index b0334d4c8..d4dcba13e 100644 --- a/src/main/java/io/confluent/connect/hdfs/HdfsSinkConnectorConfig.java +++ b/src/main/java/io/confluent/connect/hdfs/HdfsSinkConnectorConfig.java @@ -14,6 +14,7 @@ package io.confluent.connect.hdfs; +import org.apache.hadoop.conf.Configuration; import org.apache.kafka.common.config.AbstractConfig; import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.common.config.ConfigDef.Importance; @@ -37,7 +38,6 @@ public class HdfsSinkConnectorConfig extends StorageSinkConnectorConfig { // HDFS Group - // This config is deprecated and will be removed in future releases. Use store.url instead. public static final String HDFS_URL_CONFIG = "hdfs.url"; public static final String HDFS_URL_DOC = @@ -96,13 +96,13 @@ public class HdfsSinkConnectorConfig extends StorageSinkConnectorConfig { public static final long KERBEROS_TICKET_RENEW_PERIOD_MS_DEFAULT = 60000 * 60; private static final String KERBEROS_TICKET_RENEW_PERIOD_MS_DISPLAY = "Kerberos Ticket Renew Period (ms)"; - // Internal group // Need to just set the default public static final String STORAGE_CLASS_DEFAULT = "io.confluent.connect.hdfs.storage.HdfsStorage"; private static final ConfigDef.Recommender hdfsAuthenticationKerberosDependentsRecommender = new BooleanParentRecommender(HDFS_AUTHENTICATION_KERBEROS_CONFIG); private final String name; + private final Configuration hadoopConfig; private final StorageCommonConfig commonConfig; private final HiveConfig hiveConfig; @@ -255,6 +255,7 @@ protected HdfsSinkConnectorConfig(ConfigDef configDef, Map props hiveConfig = new HiveConfig(originalsStrings()); partitionerConfig = new PartitionerConfig(originalsStrings()); this.name = parseName(originalsStrings()); + this.hadoopConfig = new Configuration(); addToGlobal(hiveConfig); addToGlobal(partitionerConfig); addToGlobal(commonConfig); @@ -281,6 +282,10 @@ public String getName() { return name; } + public Configuration getHadoopConfiguration() { + return hadoopConfig; + } + public Map plainValues() { Map map = new HashMap<>(); for (AbstractConfig config : allConfigs) { From 24310e302602deb222114bddc17ad5bc22f78bc9 Mon Sep 17 00:00:00 2001 From: Konstantine Karantasis Date: Sat, 22 Apr 2017 22:20:12 -0700 Subject: [PATCH 30/58] Fix wal package. --- .../java/io/confluent/connect/hdfs/wal/FSWAL.java | 12 ++++++------ .../io/confluent/connect/hdfs/wal/WALFile.java | 15 ++++++++------- 2 files changed, 14 insertions(+), 13 deletions(-) diff --git a/src/main/java/io/confluent/connect/hdfs/wal/FSWAL.java b/src/main/java/io/confluent/connect/hdfs/wal/FSWAL.java index 60cc70a3a..bc3cb52a4 100644 --- a/src/main/java/io/confluent/connect/hdfs/wal/FSWAL.java +++ b/src/main/java/io/confluent/connect/hdfs/wal/FSWAL.java @@ -14,7 +14,6 @@ package io.confluent.connect.hdfs.wal; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.ipc.RemoteException; import org.apache.kafka.common.TopicPartition; @@ -28,7 +27,8 @@ import java.util.Map; import io.confluent.connect.hdfs.FileUtils; -import io.confluent.connect.hdfs.storage.Storage; +import io.confluent.connect.hdfs.HdfsSinkConnectorConfig; +import io.confluent.connect.hdfs.storage.HdfsStorage; import io.confluent.connect.hdfs.wal.WALFile.Reader; import io.confluent.connect.hdfs.wal.WALFile.Writer; @@ -41,10 +41,10 @@ public class FSWAL implements WAL { private WALFile.Writer writer = null; private WALFile.Reader reader = null; private String logFile = null; - private Configuration conf = null; - private Storage storage = null; + private HdfsSinkConnectorConfig conf = null; + private HdfsStorage storage = null; - public FSWAL(String logsDir, TopicPartition topicPart, Storage storage) + public FSWAL(String logsDir, TopicPartition topicPart, HdfsStorage storage) throws ConnectException { this.storage = storage; this.conf = storage.conf(); @@ -105,7 +105,7 @@ public void apply() throws ConnectException { } acquireLease(); if (reader == null) { - reader = new WALFile.Reader(conf, Reader.file(new Path(logFile))); + reader = new WALFile.Reader(conf.getHadoopConfiguration(), Reader.file(new Path(logFile))); } Map entries = new HashMap<>(); WALEntry key = new WALEntry(); diff --git a/src/main/java/io/confluent/connect/hdfs/wal/WALFile.java b/src/main/java/io/confluent/connect/hdfs/wal/WALFile.java index 2e95cd73d..bc7fbb467 100644 --- a/src/main/java/io/confluent/connect/hdfs/wal/WALFile.java +++ b/src/main/java/io/confluent/connect/hdfs/wal/WALFile.java @@ -44,6 +44,8 @@ import java.security.MessageDigest; import java.util.Arrays; +import io.confluent.connect.hdfs.HdfsSinkConnectorConfig; + public class WALFile { private static final Log log = LogFactory.getLog(WALFile.class); @@ -63,14 +65,12 @@ public class WALFile { private WALFile() {} - public static Writer createWriter(Configuration conf, Writer.Option... opts) throws IOException { + public static Writer createWriter(HdfsSinkConnectorConfig conf, Writer.Option... opts) throws IOException { return new Writer(conf, opts); } public static class Writer implements Closeable, Syncable { - - private Configuration conf; private FSDataOutputStream out; private DataOutputBuffer buffer = new DataOutputBuffer(); boolean ownOutputStream = true; @@ -170,7 +170,8 @@ static class AppendIfExistsOption extends Options.BooleanOption implements Optio } - Writer(Configuration conf, Option... opts) throws IOException { + Writer(HdfsSinkConnectorConfig connectorConfig, Option... opts) throws IOException { + Configuration conf = connectorConfig.getHadoopConfiguration(); BlockSizeOption blockSizeOption = Options.getOption(BlockSizeOption.class, opts); BufferSizeOption bufferSizeOption = @@ -227,12 +228,12 @@ static class AppendIfExistsOption extends Options.BooleanOption implements Optio out = streamOption.getValue(); } - init(conf, out, ownStream); + init(connectorConfig, out, ownStream); } - void init(Configuration conf, FSDataOutputStream out, boolean ownStream) + void init(HdfsSinkConnectorConfig connectorConfig, FSDataOutputStream out, boolean ownStream) throws IOException { - this.conf = conf; + Configuration conf = connectorConfig.getHadoopConfiguration(); this.out = out; this.ownOutputStream = ownStream; SerializationFactory serializationFactory = new SerializationFactory(conf); From e0bb839663ecff915ef0c4bf02b4688c5fb7f579 Mon Sep 17 00:00:00 2001 From: Konstantine Karantasis Date: Sat, 22 Apr 2017 22:20:45 -0700 Subject: [PATCH 31/58] Fix storage package. --- .../connect/hdfs/storage/HdfsStorage.java | 25 +++++++++++-------- .../connect/hdfs/storage/Storage.java | 5 ++-- 2 files changed, 18 insertions(+), 12 deletions(-) diff --git a/src/main/java/io/confluent/connect/hdfs/storage/HdfsStorage.java b/src/main/java/io/confluent/connect/hdfs/storage/HdfsStorage.java index 08dbdaa61..bd7dd30da 100644 --- a/src/main/java/io/confluent/connect/hdfs/storage/HdfsStorage.java +++ b/src/main/java/io/confluent/connect/hdfs/storage/HdfsStorage.java @@ -18,7 +18,6 @@ import org.apache.avro.file.SeekableInput; import org.apache.avro.mapred.FsInput; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -32,20 +31,22 @@ import java.util.Arrays; import java.util.List; +import io.confluent.connect.hdfs.HdfsSinkConnectorConfig; import io.confluent.connect.hdfs.wal.FSWAL; import io.confluent.connect.hdfs.wal.WAL; public class HdfsStorage - implements io.confluent.connect.storage.Storage>, Storage { + implements io.confluent.connect.storage.Storage>, + Storage { private final FileSystem fs; - private final Configuration conf; + private final HdfsSinkConnectorConfig conf; private final String url; - public HdfsStorage(Configuration conf, String url) throws IOException { - fs = FileSystem.newInstance(URI.create(url), conf); + public HdfsStorage(HdfsSinkConnectorConfig conf, String url) throws IOException { this.conf = conf; this.url = url; + fs = FileSystem.newInstance(URI.create(url), conf.getHadoopConfiguration()); } public List list(String path, PathFilter filter) { @@ -117,7 +118,7 @@ public WAL wal(String topicsDir, TopicPartition topicPart) { } @Override - public Configuration conf() { + public HdfsSinkConnectorConfig conf() { return conf; } @@ -142,19 +143,23 @@ private void renameFile(String sourcePath, String targetPath) { } @Override - public SeekableInput open(String filename, Configuration conf) { + public SeekableInput open(String filename, HdfsSinkConnectorConfig conf) { try { - return new FsInput(new Path(filename), conf); + return new FsInput(new Path(filename), conf.getHadoopConfiguration()); } catch (IOException e) { throw new ConnectException(e); } } + public OutputStream create(String filename, boolean overwrite) { + return create(filename, this.conf, overwrite); + } + @Override - public OutputStream create(String filename, Configuration conf, boolean overwrite) { + public OutputStream create(String filename, HdfsSinkConnectorConfig conf, boolean overwrite) { try { Path path = new Path(filename); - return path.getFileSystem(conf).create(path); + return path.getFileSystem(conf.getHadoopConfiguration()).create(path); } catch (IOException e) { throw new ConnectException(e); } diff --git a/src/main/java/io/confluent/connect/hdfs/storage/Storage.java b/src/main/java/io/confluent/connect/hdfs/storage/Storage.java index f753ee8ad..598d5caa4 100644 --- a/src/main/java/io/confluent/connect/hdfs/storage/Storage.java +++ b/src/main/java/io/confluent/connect/hdfs/storage/Storage.java @@ -16,11 +16,12 @@ package io.confluent.connect.hdfs.storage; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import java.util.List; +import io.confluent.connect.hdfs.HdfsSinkConnectorConfig; + @Deprecated public interface Storage - extends io.confluent.connect.storage.Storage> {} + extends io.confluent.connect.storage.Storage> {} From c2f347d0640d31dcd2902246cb9770cc3bbc88df Mon Sep 17 00:00:00 2001 From: Konstantine Karantasis Date: Sat, 22 Apr 2017 22:22:21 -0700 Subject: [PATCH 32/58] Fix formats and their interfaces. --- .../io/confluent/connect/hdfs/Format.java | 17 +--- .../confluent/connect/hdfs/RecordWriter.java | 7 +- .../connect/hdfs/RecordWriterProvider.java | 15 +-- .../connect/hdfs/SchemaFileReader.java | 10 +- .../connect/hdfs/avro/AvroFileReader.java | 31 ++++-- .../connect/hdfs/avro/AvroFormat.java | 27 ++++-- .../connect/hdfs/avro/AvroHiveFactory.java | 22 +++-- .../connect/hdfs/avro/AvroHiveUtil.java | 10 +- .../hdfs/avro/AvroRecordWriterProvider.java | 82 ++++++++++------ .../hdfs/parquet/ParquetFileReader.java | 32 +++++- .../connect/hdfs/parquet/ParquetFormat.java | 23 +++-- .../hdfs/parquet/ParquetHiveFactory.java | 17 ++-- .../connect/hdfs/parquet/ParquetHiveUtil.java | 21 +++- .../parquet/ParquetRecordWriterProvider.java | 97 +++++++++++++------ 14 files changed, 274 insertions(+), 137 deletions(-) diff --git a/src/main/java/io/confluent/connect/hdfs/Format.java b/src/main/java/io/confluent/connect/hdfs/Format.java index 30862aa04..6650f22ac 100644 --- a/src/main/java/io/confluent/connect/hdfs/Format.java +++ b/src/main/java/io/confluent/connect/hdfs/Format.java @@ -14,26 +14,13 @@ package io.confluent.connect.hdfs; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; -import org.apache.kafka.common.config.AbstractConfig; -import io.confluent.connect.avro.AvroData; import io.confluent.connect.hdfs.hive.HiveMetaStore; import io.confluent.connect.hdfs.hive.HiveUtil; -import io.confluent.connect.storage.hive.HiveFactory; @Deprecated public interface Format - extends io.confluent.connect.storage.format.Format { - @Override - RecordWriterProvider getRecordWriterProvider(); - - @Override - SchemaFileReader getSchemaFileReader(AvroData avroData); - - HiveUtil getHiveUtil(HdfsSinkConnectorConfig config, AvroData avroData, HiveMetaStore hiveMetaStore); - - @Override - HiveFactory getHiveFactory(); + extends io.confluent.connect.storage.format.Format { + HiveUtil getHiveUtil(HdfsSinkConnectorConfig config, HiveMetaStore hiveMetaStore); } diff --git a/src/main/java/io/confluent/connect/hdfs/RecordWriter.java b/src/main/java/io/confluent/connect/hdfs/RecordWriter.java index 4dfa0dad0..f620c37eb 100644 --- a/src/main/java/io/confluent/connect/hdfs/RecordWriter.java +++ b/src/main/java/io/confluent/connect/hdfs/RecordWriter.java @@ -17,10 +17,5 @@ package io.confluent.connect.hdfs; @Deprecated -public interface RecordWriter extends io.confluent.connect.storage.format.RecordWriter { - @Override - void write(V value); - - @Override - void close(); +public interface RecordWriter extends io.confluent.connect.storage.format.RecordWriter { } diff --git a/src/main/java/io/confluent/connect/hdfs/RecordWriterProvider.java b/src/main/java/io/confluent/connect/hdfs/RecordWriterProvider.java index 91613e0cd..218e4cd86 100644 --- a/src/main/java/io/confluent/connect/hdfs/RecordWriterProvider.java +++ b/src/main/java/io/confluent/connect/hdfs/RecordWriterProvider.java @@ -14,17 +14,18 @@ package io.confluent.connect.hdfs; -import org.apache.hadoop.conf.Configuration; import org.apache.kafka.connect.sink.SinkRecord; import io.confluent.connect.avro.AvroData; @Deprecated public interface RecordWriterProvider - extends io.confluent.connect.storage.format.RecordWriterProvider { - @Override - String getExtension(); - - @Override - RecordWriter getRecordWriter(Configuration conf, String fileName, SinkRecord record, AvroData avroData); + extends io.confluent.connect.storage.format.RecordWriterProvider + { + RecordWriter getRecordWriter( + HdfsSinkConnectorConfig conf, + String fileName, + SinkRecord record, + final AvroData avroData + ); } diff --git a/src/main/java/io/confluent/connect/hdfs/SchemaFileReader.java b/src/main/java/io/confluent/connect/hdfs/SchemaFileReader.java index 4f7eb2cbc..7cdea26db 100644 --- a/src/main/java/io/confluent/connect/hdfs/SchemaFileReader.java +++ b/src/main/java/io/confluent/connect/hdfs/SchemaFileReader.java @@ -14,16 +14,12 @@ package io.confluent.connect.hdfs; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; -import org.apache.kafka.connect.data.Schema; import java.util.Collection; @Deprecated -public interface SchemaFileReader extends io.confluent.connect.storage.format.SchemaFileReader { - @Override - Schema getSchema(Configuration conf, Path path); - @Override - Collection readData(Configuration conf, Path path); +public interface SchemaFileReader + extends io.confluent.connect.storage.format.SchemaFileReader { + Collection readData(HdfsSinkConnectorConfig conf, Path path); } diff --git a/src/main/java/io/confluent/connect/hdfs/avro/AvroFileReader.java b/src/main/java/io/confluent/connect/hdfs/avro/AvroFileReader.java index bff91162f..56ffddaf4 100644 --- a/src/main/java/io/confluent/connect/hdfs/avro/AvroFileReader.java +++ b/src/main/java/io/confluent/connect/hdfs/avro/AvroFileReader.java @@ -28,12 +28,14 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Collection; +import java.util.Iterator; import io.confluent.connect.avro.AvroData; +import io.confluent.connect.hdfs.HdfsSinkConnectorConfig; import io.confluent.connect.hdfs.SchemaFileReader; public class AvroFileReader implements SchemaFileReader, - io.confluent.connect.storage.format.SchemaFileReader { + io.confluent.connect.storage.format.SchemaFileReader { private AvroData avroData; public AvroFileReader(AvroData avroData) { @@ -41,9 +43,9 @@ public AvroFileReader(AvroData avroData) { } @Override - public Schema getSchema(Configuration conf, Path path) { + public Schema getSchema(HdfsSinkConnectorConfig conf, Path path) { try { - SeekableInput input = new FsInput(path, conf); + SeekableInput input = new FsInput(path, conf.getHadoopConfiguration()); DatumReader reader = new GenericDatumReader<>(); FileReader fileReader = DataFileReader.openReader(input, reader); org.apache.avro.Schema schema = fileReader.getSchema(); @@ -54,11 +56,10 @@ public Schema getSchema(Configuration conf, Path path) { } } - @Override - public Collection readData(Configuration conf, Path path) { + public Collection readData(HdfsSinkConnectorConfig conf, Path path) { ArrayList collection = new ArrayList<>(); try { - SeekableInput input = new FsInput(path, conf); + SeekableInput input = new FsInput(path, conf.getHadoopConfiguration()); DatumReader reader = new GenericDatumReader<>(); FileReader fileReader = DataFileReader.openReader(input, reader); for (Object object : fileReader) { @@ -70,4 +71,22 @@ public Collection readData(Configuration conf, Path path) { } return collection; } + + public boolean hasNext() { + throw new UnsupportedOperationException(); + } + + public Object next() { + throw new UnsupportedOperationException(); + } + + public void remove() { + throw new UnsupportedOperationException(); + } + + public Iterator iterator() { + throw new UnsupportedOperationException(); + } + + public void close() {} } diff --git a/src/main/java/io/confluent/connect/hdfs/avro/AvroFormat.java b/src/main/java/io/confluent/connect/hdfs/avro/AvroFormat.java index 2c55e2a9f..04779ac7c 100644 --- a/src/main/java/io/confluent/connect/hdfs/avro/AvroFormat.java +++ b/src/main/java/io/confluent/connect/hdfs/avro/AvroFormat.java @@ -14,7 +14,6 @@ package io.confluent.connect.hdfs.avro; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import io.confluent.connect.avro.AvroData; @@ -24,28 +23,40 @@ import io.confluent.connect.hdfs.SchemaFileReader; import io.confluent.connect.hdfs.hive.HiveMetaStore; import io.confluent.connect.hdfs.hive.HiveUtil; +import io.confluent.connect.hdfs.storage.HdfsStorage; import io.confluent.connect.storage.hive.HiveFactory; -public class AvroFormat implements Format, io.confluent.connect.storage.format.Format { +public class AvroFormat implements Format, + io.confluent.connect.storage.format.Format { + private final AvroData avroData; + + public AvroFormat(HdfsStorage storage) { + this.avroData = new AvroData( + storage.conf().getInt(HdfsSinkConnectorConfig.SCHEMA_CACHE_SIZE_CONFIG) + ); + } @Override public RecordWriterProvider getRecordWriterProvider() { - return new AvroRecordWriterProvider(); + return new AvroRecordWriterProvider(avroData); } @Override - public SchemaFileReader getSchemaFileReader(AvroData avroData) { + public SchemaFileReader getSchemaFileReader() { return new AvroFileReader(avroData); } @Deprecated @Override - public HiveUtil getHiveUtil(HdfsSinkConnectorConfig config, AvroData avroData, HiveMetaStore hiveMetaStore) { - return (HiveUtil) getHiveFactory().createHiveUtil(config, avroData, hiveMetaStore); + public HiveUtil getHiveUtil( + HdfsSinkConnectorConfig config, + HiveMetaStore hiveMetaStore + ) { + return (HiveUtil) getHiveFactory().createHiveUtil(config, hiveMetaStore); } @Override - public HiveFactory getHiveFactory() { - return new AvroHiveFactory(); + public HiveFactory getHiveFactory() { + return new AvroHiveFactory(avroData); } } diff --git a/src/main/java/io/confluent/connect/hdfs/avro/AvroHiveFactory.java b/src/main/java/io/confluent/connect/hdfs/avro/AvroHiveFactory.java index 3c30ae8c3..7b24d8306 100644 --- a/src/main/java/io/confluent/connect/hdfs/avro/AvroHiveFactory.java +++ b/src/main/java/io/confluent/connect/hdfs/avro/AvroHiveFactory.java @@ -16,21 +16,31 @@ package io.confluent.connect.hdfs.avro; +import org.apache.kafka.common.config.AbstractConfig; + import io.confluent.connect.avro.AvroData; import io.confluent.connect.hdfs.HdfsSinkConnectorConfig; import io.confluent.connect.hdfs.hive.HiveMetaStore; import io.confluent.connect.hdfs.hive.HiveUtil; import io.confluent.connect.storage.hive.HiveFactory; -public class AvroHiveFactory implements HiveFactory { +public class AvroHiveFactory implements HiveFactory { + private final AvroData avroData; + + public AvroHiveFactory(AvroData avroData) { + this.avroData = avroData; + } + @Override - public io.confluent.connect.storage.hive.HiveUtil createHiveUtil(HdfsSinkConnectorConfig config, AvroData avroData, - io.confluent.connect.storage.hive.HiveMetaStore hiveMetaStore) { - return createHiveUtil(config, avroData, (HiveMetaStore) hiveMetaStore); + public io.confluent.connect.storage.hive.HiveUtil createHiveUtil( + AbstractConfig conf, + io.confluent.connect.storage.hive.HiveMetaStore hiveMetaStore + ) { + return createHiveUtil((HdfsSinkConnectorConfig) conf, (HiveMetaStore) hiveMetaStore); } @Deprecated - public HiveUtil createHiveUtil(HdfsSinkConnectorConfig config, AvroData avroData, HiveMetaStore hiveMetaStore) { - return new AvroHiveUtil(config, avroData, hiveMetaStore); + public HiveUtil createHiveUtil(HdfsSinkConnectorConfig conf, HiveMetaStore hiveMetaStore) { + return new AvroHiveUtil(conf, avroData, hiveMetaStore); } } diff --git a/src/main/java/io/confluent/connect/hdfs/avro/AvroHiveUtil.java b/src/main/java/io/confluent/connect/hdfs/avro/AvroHiveUtil.java index d34d8eaed..b11662457 100644 --- a/src/main/java/io/confluent/connect/hdfs/avro/AvroHiveUtil.java +++ b/src/main/java/io/confluent/connect/hdfs/avro/AvroHiveUtil.java @@ -28,6 +28,7 @@ import io.confluent.connect.hdfs.hive.HiveMetaStore; import io.confluent.connect.hdfs.hive.HiveUtil; import io.confluent.connect.hdfs.partitioner.Partitioner; +import io.confluent.connect.storage.common.StorageCommonConfig; import io.confluent.connect.storage.errors.HiveMetaStoreException; import io.confluent.connect.storage.hive.HiveSchemaConverter; @@ -37,9 +38,14 @@ public class AvroHiveUtil extends HiveUtil { private static final String AVRO_INPUT_FORMAT = "org.apache.hadoop.hive.ql.io.avro.AvroContainerInputFormat"; private static final String AVRO_OUTPUT_FORMAT = "org.apache.hadoop.hive.ql.io.avro.AvroContainerOutputFormat"; private static final String AVRO_SCHEMA_LITERAL = "avro.schema.literal"; + private final AvroData avroData; + private final String topicsDir; - public AvroHiveUtil(HdfsSinkConnectorConfig connectorConfig, AvroData avroData, HiveMetaStore hiveMetaStore) { - super(connectorConfig, avroData, hiveMetaStore); + public AvroHiveUtil(HdfsSinkConnectorConfig conf, AvroData avroData, HiveMetaStore + hiveMetaStore) { + super(conf, hiveMetaStore); + this.avroData = avroData; + this.topicsDir = conf.getString(StorageCommonConfig.TOPICS_DIR_CONFIG); } @Override diff --git a/src/main/java/io/confluent/connect/hdfs/avro/AvroRecordWriterProvider.java b/src/main/java/io/confluent/connect/hdfs/avro/AvroRecordWriterProvider.java index 98eac3b97..9e20509c4 100644 --- a/src/main/java/io/confluent/connect/hdfs/avro/AvroRecordWriterProvider.java +++ b/src/main/java/io/confluent/connect/hdfs/avro/AvroRecordWriterProvider.java @@ -14,14 +14,12 @@ package io.confluent.connect.hdfs.avro; -import io.confluent.kafka.serializers.NonRecordContainer; import org.apache.avro.file.DataFileWriter; import org.apache.avro.generic.GenericDatumWriter; -import org.apache.avro.io.DatumWriter; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.Path; 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; @@ -30,13 +28,20 @@ import java.io.IOException; import io.confluent.connect.avro.AvroData; +import io.confluent.connect.hdfs.HdfsSinkConnectorConfig; import io.confluent.connect.hdfs.RecordWriter; import io.confluent.connect.hdfs.RecordWriterProvider; +import io.confluent.kafka.serializers.NonRecordContainer; public class AvroRecordWriterProvider implements RecordWriterProvider, - io.confluent.connect.storage.format.RecordWriterProvider { + io.confluent.connect.storage.format.RecordWriterProvider { private static final Logger log = LoggerFactory.getLogger(AvroRecordWriterProvider.class); private final static String EXTENSION = ".avro"; + private final AvroData avroData; + + AvroRecordWriterProvider(AvroData avroData) { + this.avroData = avroData; + } @Override public String getExtension() { @@ -44,38 +49,57 @@ public String getExtension() { } @Override - public RecordWriter getRecordWriter(Configuration conf, final String fileName, - SinkRecord record, final AvroData avroData) { - DatumWriter datumWriter = new GenericDatumWriter<>(); - final DataFileWriter writer = new DataFileWriter<>(datumWriter); - Path path = new Path(fileName); + public RecordWriter getRecordWriter( + HdfsSinkConnectorConfig conf, + final String fileName, + SinkRecord record, + final AvroData avroData + ) { + return getRecordWriter(conf, fileName); + } - final Schema schema = record.valueSchema(); - try { - final FSDataOutputStream out = path.getFileSystem(conf).create(path); - org.apache.avro.Schema avroSchema = avroData.fromConnectSchema(schema); - writer.create(avroSchema, out); - } catch (IOException e) { - throw new DataException(e); - } + @Override + public RecordWriter getRecordWriter( + final HdfsSinkConnectorConfig conf, + final String filename + ) { + return new RecordWriter() { + final DataFileWriter writer = new DataFileWriter<>(new GenericDatumWriter<>()); + final Path path = new Path(filename); + Schema schema = null; - return new RecordWriter() { @Override public void write(SinkRecord record) { - log.trace("Sink record: {}", record.toString()); - Object value = avroData.fromConnectData(schema, record.value()); - try { - // 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); - } catch (IOException e) { - throw new DataException(e); + if (schema == null) { + schema = record.valueSchema(); + try { + log.info("Opening record writer for: {}", filename); + final FSDataOutputStream out = path.getFileSystem(conf.getHadoopConfiguration()) + .create(path); + org.apache.avro.Schema avroSchema = avroData.fromConnectSchema(schema); + writer.create(avroSchema, out); + } catch (IOException e) { + throw new ConnectException(e); + } + + log.trace("Sink record: {}", record.toString()); + Object value = avroData.fromConnectData(schema, record.value()); + try { + // 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); + } catch (IOException e) { + throw new DataException(e); + } } } + @Override + public void commit() {} + @Override public void close() { try { diff --git a/src/main/java/io/confluent/connect/hdfs/parquet/ParquetFileReader.java b/src/main/java/io/confluent/connect/hdfs/parquet/ParquetFileReader.java index 6f80f9177..b0c69ecc2 100644 --- a/src/main/java/io/confluent/connect/hdfs/parquet/ParquetFileReader.java +++ b/src/main/java/io/confluent/connect/hdfs/parquet/ParquetFileReader.java @@ -26,12 +26,14 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Collection; +import java.util.Iterator; import io.confluent.connect.avro.AvroData; +import io.confluent.connect.hdfs.HdfsSinkConnectorConfig; import io.confluent.connect.hdfs.SchemaFileReader; public class ParquetFileReader implements SchemaFileReader, - io.confluent.connect.storage.format.SchemaFileReader { + io.confluent.connect.storage.format.SchemaFileReader { private AvroData avroData; public ParquetFileReader(AvroData avroData) { @@ -39,11 +41,12 @@ public ParquetFileReader(AvroData avroData) { } @Override - public Schema getSchema(Configuration conf, Path path) { + public Schema getSchema(HdfsSinkConnectorConfig conf, Path path) { AvroReadSupport readSupport = new AvroReadSupport<>(); ParquetReader.Builder builder = ParquetReader.builder(readSupport, path); try { - ParquetReader parquetReader = builder.withConf(conf).build(); + ParquetReader parquetReader = builder.withConf(conf.getHadoopConfiguration()) + .build(); GenericRecord record; Schema schema = null; while ((record = parquetReader.read()) != null) { @@ -57,12 +60,13 @@ public Schema getSchema(Configuration conf, Path path) { } @Override - public Collection readData(Configuration conf, Path path) { + public Collection readData(HdfsSinkConnectorConfig conf, Path path) { Collection result = new ArrayList<>(); AvroReadSupport readSupport = new AvroReadSupport<>(); ParquetReader.Builder builder = ParquetReader.builder(readSupport, path); try { - ParquetReader parquetReader = builder.withConf(conf).build(); + ParquetReader parquetReader = builder.withConf(conf.getHadoopConfiguration()) + .build(); GenericRecord record; while ((record = parquetReader.read()) != null) { result.add(record); @@ -73,4 +77,22 @@ public Collection readData(Configuration conf, Path path) { } return result; } + + public boolean hasNext() { + throw new UnsupportedOperationException(); + } + + public Object next() { + throw new UnsupportedOperationException(); + } + + public void remove() { + throw new UnsupportedOperationException(); + } + + public Iterator iterator() { + throw new UnsupportedOperationException(); + } + + public void close() {} } diff --git a/src/main/java/io/confluent/connect/hdfs/parquet/ParquetFormat.java b/src/main/java/io/confluent/connect/hdfs/parquet/ParquetFormat.java index e7334e53f..9247567b3 100644 --- a/src/main/java/io/confluent/connect/hdfs/parquet/ParquetFormat.java +++ b/src/main/java/io/confluent/connect/hdfs/parquet/ParquetFormat.java @@ -24,29 +24,40 @@ import io.confluent.connect.hdfs.SchemaFileReader; import io.confluent.connect.hdfs.hive.HiveMetaStore; import io.confluent.connect.hdfs.hive.HiveUtil; +import io.confluent.connect.hdfs.storage.HdfsStorage; import io.confluent.connect.storage.hive.HiveFactory; public class ParquetFormat implements Format, - io.confluent.connect.storage.format.Format { + io.confluent.connect.storage.format.Format { + private final AvroData avroData; + + public ParquetFormat(HdfsStorage storage) { + this.avroData = new AvroData( + storage.conf().getInt(HdfsSinkConnectorConfig.SCHEMA_CACHE_SIZE_CONFIG) + ); + } @Override public RecordWriterProvider getRecordWriterProvider() { - return new ParquetRecordWriterProvider(); + return new ParquetRecordWriterProvider(avroData); } @Override - public SchemaFileReader getSchemaFileReader(AvroData avroData) { + public SchemaFileReader getSchemaFileReader() { return new ParquetFileReader(avroData); } @Deprecated @Override - public HiveUtil getHiveUtil(HdfsSinkConnectorConfig config, AvroData avroData, HiveMetaStore hiveMetaStore) { - return (HiveUtil) getHiveFactory().createHiveUtil(config, avroData, hiveMetaStore); + public HiveUtil getHiveUtil( + HdfsSinkConnectorConfig config, + HiveMetaStore hiveMetaStore + ) { + return (HiveUtil) getHiveFactory().createHiveUtil(config, hiveMetaStore); } @Override - public HiveFactory getHiveFactory() { + public HiveFactory getHiveFactory() { return new ParquetHiveFactory(); } } diff --git a/src/main/java/io/confluent/connect/hdfs/parquet/ParquetHiveFactory.java b/src/main/java/io/confluent/connect/hdfs/parquet/ParquetHiveFactory.java index a88789dde..6f9a01385 100644 --- a/src/main/java/io/confluent/connect/hdfs/parquet/ParquetHiveFactory.java +++ b/src/main/java/io/confluent/connect/hdfs/parquet/ParquetHiveFactory.java @@ -16,20 +16,25 @@ package io.confluent.connect.hdfs.parquet; +import org.apache.kafka.common.config.AbstractConfig; + import io.confluent.connect.avro.AvroData; import io.confluent.connect.hdfs.HdfsSinkConnectorConfig; import io.confluent.connect.storage.hive.HiveFactory; import io.confluent.connect.hdfs.hive.HiveMetaStore; import io.confluent.connect.storage.hive.HiveUtil; -public class ParquetHiveFactory implements HiveFactory { +public class ParquetHiveFactory implements HiveFactory { @Override - public HiveUtil createHiveUtil(HdfsSinkConnectorConfig config, AvroData avroData, - io.confluent.connect.storage.hive.HiveMetaStore hiveMetaStore) { - return createHiveUtil(config, avroData, (HiveMetaStore) hiveMetaStore); + public HiveUtil createHiveUtil( + AbstractConfig config, + io.confluent.connect.storage.hive.HiveMetaStore hiveMetaStore + ) { + return createHiveUtil((HdfsSinkConnectorConfig) config, (HiveMetaStore) hiveMetaStore); } - public HiveUtil createHiveUtil(HdfsSinkConnectorConfig config, AvroData avroData, HiveMetaStore hiveMetaStore) { - return new ParquetHiveUtil(config, avroData, hiveMetaStore); + @Deprecated + public HiveUtil createHiveUtil(HdfsSinkConnectorConfig config, HiveMetaStore hiveMetaStore) { + return new ParquetHiveUtil(config, hiveMetaStore); } } diff --git a/src/main/java/io/confluent/connect/hdfs/parquet/ParquetHiveUtil.java b/src/main/java/io/confluent/connect/hdfs/parquet/ParquetHiveUtil.java index 0dfa3af9e..43f76b6df 100644 --- a/src/main/java/io/confluent/connect/hdfs/parquet/ParquetHiveUtil.java +++ b/src/main/java/io/confluent/connect/hdfs/parquet/ParquetHiveUtil.java @@ -28,17 +28,25 @@ import io.confluent.connect.hdfs.hive.HiveMetaStore; import io.confluent.connect.hdfs.hive.HiveUtil; import io.confluent.connect.hdfs.partitioner.Partitioner; +import io.confluent.connect.storage.common.StorageCommonConfig; import io.confluent.connect.storage.errors.HiveMetaStoreException; import io.confluent.connect.storage.hive.HiveSchemaConverter; public class ParquetHiveUtil extends HiveUtil { + private final String topicsDir; - public ParquetHiveUtil(HdfsSinkConnectorConfig connectorConfig, AvroData avroData, HiveMetaStore hiveMetaStore) { - super(connectorConfig, avroData, hiveMetaStore); + public ParquetHiveUtil(HdfsSinkConnectorConfig conf, HiveMetaStore hiveMetaStore) { + super(conf, hiveMetaStore); + this.topicsDir = conf.getString(StorageCommonConfig.TOPICS_DIR_CONFIG); } @Override - public void createTable(String database, String tableName, Schema schema, Partitioner partitioner) throws HiveMetaStoreException { + public void createTable( + String database, + String tableName, + Schema schema, + Partitioner partitioner + ) throws HiveMetaStoreException { Table table = constructParquetTable(database, tableName, schema, partitioner); hiveMetaStore.createTable(table); } @@ -51,7 +59,12 @@ public void alterSchema(String database, String tableName, Schema schema) { hiveMetaStore.alterTable(table); } - private Table constructParquetTable(String database, String tableName, Schema schema, Partitioner partitioner) throws HiveMetaStoreException { + private Table constructParquetTable( + String database, + String tableName, + Schema schema, + Partitioner partitioner + ) throws HiveMetaStoreException { Table table = newTable(database, tableName); table.setTableType(TableType.EXTERNAL_TABLE); table.getParameters().put("EXTERNAL", "TRUE"); diff --git a/src/main/java/io/confluent/connect/hdfs/parquet/ParquetRecordWriterProvider.java b/src/main/java/io/confluent/connect/hdfs/parquet/ParquetRecordWriterProvider.java index c8e561274..2a639234c 100644 --- a/src/main/java/io/confluent/connect/hdfs/parquet/ParquetRecordWriterProvider.java +++ b/src/main/java/io/confluent/connect/hdfs/parquet/ParquetRecordWriterProvider.java @@ -14,26 +14,33 @@ package io.confluent.connect.hdfs.parquet; -import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; +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.apache.parquet.avro.AvroParquetWriter; import org.apache.parquet.hadoop.ParquetWriter; import org.apache.parquet.hadoop.metadata.CompressionCodecName; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.io.IOException; import io.confluent.connect.avro.AvroData; -import io.confluent.connect.hdfs.RecordWriterProvider; +import io.confluent.connect.hdfs.HdfsSinkConnectorConfig; import io.confluent.connect.hdfs.RecordWriter; +import io.confluent.connect.hdfs.RecordWriterProvider; public class ParquetRecordWriterProvider implements RecordWriterProvider, - io.confluent.connect.storage.format.RecordWriterProvider { + io.confluent.connect.storage.format.RecordWriterProvider { + private static final Logger log = LoggerFactory.getLogger(ParquetRecordWriterProvider.class); private final static String EXTENSION = ".parquet"; + private final AvroData avroData; + + ParquetRecordWriterProvider(AvroData avroData) { + this.avroData = avroData; + } @Override public String getExtension() { @@ -41,41 +48,71 @@ public String getExtension() { } @Override - public RecordWriter getRecordWriter(Configuration conf, final String fileName, SinkRecord record, - final AvroData avroData) { - final Schema avroSchema = avroData.fromConnectSchema(record.valueSchema()); - CompressionCodecName compressionCodecName = CompressionCodecName.SNAPPY; - - int blockSize = 256 * 1024 * 1024; - int pageSize = 64 * 1024; + public RecordWriter getRecordWriter( + HdfsSinkConnectorConfig conf, + final String filename, + SinkRecord record, + final AvroData avroData + ) { + return getRecordWriter(conf, filename); + } - Path path = new Path(fileName); - try { - final ParquetWriter writer = - new AvroParquetWriter<>(path, avroSchema, compressionCodecName, blockSize, pageSize, true, conf); + @Override + public RecordWriter getRecordWriter( + final HdfsSinkConnectorConfig conf, + final String filename + ) { + return new RecordWriter() { + final CompressionCodecName compressionCodecName = CompressionCodecName.SNAPPY; + final int blockSize = 256 * 1024 * 1024; + final int pageSize = 64 * 1024; + final Path path = new Path(filename); + Schema schema = null; + ParquetWriter writer = null; - return new RecordWriter() { - @Override - public void write(SinkRecord record) { - Object value = avroData.fromConnectData(record.valueSchema(), record.value()); + @Override + public void write(SinkRecord record) { + if (schema == null) { + schema = record.valueSchema(); try { - writer.write((GenericRecord) value); + log.info("Opening record writer for: {}", filename); + org.apache.avro.Schema avroSchema = avroData.fromConnectSchema(schema); + writer = new AvroParquetWriter<>( + path, + avroSchema, + compressionCodecName, + blockSize, + pageSize, + true, + conf.getHadoopConfiguration() + ); } catch (IOException e) { - throw new DataException(e); + throw new ConnectException(e); } } - @Override - public void close() { + log.trace("Sink record: {}", record.toString()); + Object value = avroData.fromConnectData(record.valueSchema(), record.value()); + try { + writer.write((GenericRecord) value); + } catch (IOException e) { + throw new ConnectException(e); + } + } + + @Override + public void commit() {} + + @Override + public void close() { + if (writer != null) { try { writer.close(); - } catch (IOException e) { - throw new DataException(e); + } catch(IOException e){ + throw new ConnectException(e); } } - }; - } catch (IOException e) { - throw new ConnectException(e); - } + } + }; } } From e93be62da8e0d5be7cbcefe5e98e6c7a5f039cfc Mon Sep 17 00:00:00 2001 From: Konstantine Karantasis Date: Sun, 23 Apr 2017 07:36:04 -0700 Subject: [PATCH 33/58] Fix file utils (storage package related) --- .../io/confluent/connect/hdfs/FileUtils.java | 37 +++++++++++++------ 1 file changed, 26 insertions(+), 11 deletions(-) diff --git a/src/main/java/io/confluent/connect/hdfs/FileUtils.java b/src/main/java/io/confluent/connect/hdfs/FileUtils.java index 60753baa8..8706cbb92 100644 --- a/src/main/java/io/confluent/connect/hdfs/FileUtils.java +++ b/src/main/java/io/confluent/connect/hdfs/FileUtils.java @@ -46,8 +46,12 @@ public static String directoryName(String url, String topicsDir, TopicPartition return url + "/" + topicsDir + "/" + topic + "/" + partition; } - public static String fileName(String url, String topicsDir, TopicPartition topicPart, - String name) { + public static String fileName( + String url, + String topicsDir, + TopicPartition topicPart, + String name + ) { String topic = topicPart.topic(); int partition = topicPart.partition(); return url + "/" + topicsDir + "/" + topic + "/" + partition + "/" + name; @@ -61,16 +65,27 @@ public static String directoryName(String url, String topicsDir, String director return url + "/" + topicsDir + "/" + directory; } - public static String tempFileName(String url, String topicsDir, String directory, - String extension) { + public static String tempFileName( + String url, + String topicsDir, + String directory, + String extension + ) { UUID id = UUID.randomUUID(); String name = id.toString() + "_" + "tmp" + extension; return fileName(url, topicsDir, directory, name); } - public static String committedFileName(String url, String topicsDir, String directory, - TopicPartition topicPart, long startOffset, long endOffset, - String extension, String zeroPadFormat) { + public static String committedFileName( + String url, + String topicsDir, + String directory, + TopicPartition topicPart, + long startOffset, + long endOffset, + String extension, + String zeroPadFormat + ) { String topic = topicPart.topic(); int partition = topicPart.partition(); StringBuilder sb = new StringBuilder(); @@ -95,7 +110,7 @@ private static ArrayList traverseImpl(Storage storage, Path path, Pa return new ArrayList<>(); } ArrayList result = new ArrayList<>(); - List statuses = storage.listStatus(path.toString()); + List statuses = storage.list(path.toString()); for (FileStatus status : statuses) { if (status.isDirectory()) { result.addAll(traverseImpl(storage, status.getPath(), filter)); @@ -120,7 +135,7 @@ public static FileStatus fileStatusWithMaxOffset(Storage storage, Path path, Com } long maxOffset = -1L; FileStatus fileStatusWithMaxOffset = null; - List statuses = storage.listStatus(path.toString()); + List statuses = storage.list(path.toString()); for (FileStatus status : statuses) { if (status.isDirectory()) { FileStatus fileStatus = fileStatusWithMaxOffset(storage, status.getPath(), filter); @@ -156,12 +171,12 @@ public static long extractOffset(String filename) { } private static ArrayList getDirectoriesImpl(Storage storage, Path path) { - List statuses = storage.listStatus(path.toString()); + List statuses = storage.list(path.toString()); ArrayList result = new ArrayList<>(); for (FileStatus status : statuses) { if (status.isDirectory()) { int count = 0; - List fileStatuses = storage.listStatus(status.getPath().toString()); + List fileStatuses = storage.list(status.getPath().toString()); for (FileStatus fileStatus : fileStatuses) { if (fileStatus.isDirectory()) { result.addAll(getDirectoriesImpl(storage, fileStatus.getPath())); From ba83a4ab13163d44587f82c19d8a42fe0dd64b0c Mon Sep 17 00:00:00 2001 From: Konstantine Karantasis Date: Sun, 23 Apr 2017 07:36:58 -0700 Subject: [PATCH 34/58] Fix hive. --- .../java/io/confluent/connect/hdfs/hive/HiveMetaStore.java | 3 --- src/main/java/io/confluent/connect/hdfs/hive/HiveUtil.java | 6 ------ 2 files changed, 9 deletions(-) diff --git a/src/main/java/io/confluent/connect/hdfs/hive/HiveMetaStore.java b/src/main/java/io/confluent/connect/hdfs/hive/HiveMetaStore.java index fc2f5316a..7eb17b6f1 100644 --- a/src/main/java/io/confluent/connect/hdfs/hive/HiveMetaStore.java +++ b/src/main/java/io/confluent/connect/hdfs/hive/HiveMetaStore.java @@ -27,7 +27,4 @@ public HiveMetaStore(Configuration conf, HdfsSinkConnectorConfig connectorConfig super(conf, connectorConfig); } - public String tableNameConverter(String table){ - return table == null ? table : table.replaceAll("\\.", "_"); - } } diff --git a/src/main/java/io/confluent/connect/hdfs/hive/HiveUtil.java b/src/main/java/io/confluent/connect/hdfs/hive/HiveUtil.java index a56b18c08..49a4aa055 100644 --- a/src/main/java/io/confluent/connect/hdfs/hive/HiveUtil.java +++ b/src/main/java/io/confluent/connect/hdfs/hive/HiveUtil.java @@ -54,10 +54,4 @@ public abstract void createTable( Partitioner partitioner ); - @Override - public abstract void alterSchema(String database, String tableName, Schema schema); - - public Table newTable(String database, String table) { - return new Table(database, hiveMetaStore.tableNameConverter(table)); - } } From 594955c7aa2e1741436e261606e650bee788608e Mon Sep 17 00:00:00 2001 From: Konstantine Karantasis Date: Sun, 23 Apr 2017 07:37:26 -0700 Subject: [PATCH 35/58] Fix main task classes. --- .../io/confluent/connect/hdfs/DataWriter.java | 88 +++++++++++++------ .../confluent/connect/hdfs/HdfsSinkTask.java | 10 ++- .../connect/hdfs/TopicPartitionWriter.java | 58 +++++++----- .../connect/hdfs/storage/HdfsStorage.java | 7 ++ 4 files changed, 114 insertions(+), 49 deletions(-) diff --git a/src/main/java/io/confluent/connect/hdfs/DataWriter.java b/src/main/java/io/confluent/connect/hdfs/DataWriter.java index 3fd0f2d6f..9157a22e7 100644 --- a/src/main/java/io/confluent/connect/hdfs/DataWriter.java +++ b/src/main/java/io/confluent/connect/hdfs/DataWriter.java @@ -51,16 +51,18 @@ import io.confluent.connect.hdfs.hive.HiveMetaStore; import io.confluent.connect.hdfs.hive.HiveUtil; import io.confluent.connect.hdfs.partitioner.Partitioner; +import io.confluent.connect.hdfs.storage.HdfsStorage; import io.confluent.connect.hdfs.storage.Storage; -import io.confluent.connect.storage.StorageFactory; +import io.confluent.connect.storage.common.StorageCommonConfig; +import io.confluent.connect.storage.hive.HiveConfig; +import io.confluent.connect.storage.partitioner.PartitionerConfig; public class DataWriter { private static final Logger log = LoggerFactory.getLogger(DataWriter.class); private Map topicPartitionWriters; private String url; - private Storage storage; - private Configuration conf; + private HdfsStorage storage; private String topicsDir; private Format format; private Set assignment; @@ -91,7 +93,7 @@ public DataWriter(HdfsSinkConnectorConfig connectorConfig, SinkTaskContext conte String hadoopConfDir = connectorConfig.getString(HdfsSinkConnectorConfig.HADOOP_CONF_DIR_CONFIG); log.info("Hadoop configuration directory {}", hadoopConfDir); - conf = new Configuration(); + Configuration conf = connectorConfig.getHadoopConfiguration(); if (!hadoopConfDir.equals("")) { conf.addResource(new Path(hadoopConfDir + "/core-site.xml")); conf.addResource(new Path(hadoopConfDir + "/hdfs-site.xml")); @@ -159,13 +161,18 @@ public void run() { } url = connectorConfig.getString(HdfsSinkConnectorConfig.HDFS_URL_CONFIG); - topicsDir = connectorConfig.getString(HdfsSinkConnectorConfig.TOPICS_DIR_CONFIG); + topicsDir = connectorConfig.getString(StorageCommonConfig.TOPICS_DIR_CONFIG); String logsDir = connectorConfig.getString(HdfsSinkConnectorConfig.LOGS_DIR_CONFIG); @SuppressWarnings("unchecked") - Class storageClass = (Class) Class - .forName(connectorConfig.getString(HdfsSinkConnectorConfig.STORAGE_CLASS_CONFIG)); - storage = StorageFactory.createStorage(storageClass, Configuration.class, conf, url); + Class storageClass = (Class) connectorConfig + .getClass(StorageCommonConfig.STORAGE_CLASS_CONFIG); + storage = io.confluent.connect.storage.StorageFactory.createStorage( + storageClass, + HdfsSinkConnectorConfig.class, + connectorConfig, + url + ); createDir(topicsDir); createDir(topicsDir + HdfsSinkConnectorConstants.TEMPFILE_DIRECTORY); @@ -173,18 +180,18 @@ public void run() { format = getFormat(); writerProvider = format.getRecordWriterProvider(); - schemaFileReader = format.getSchemaFileReader(avroData); + schemaFileReader = format.getSchemaFileReader(); partitioner = createPartitioner(connectorConfig); assignment = new HashSet<>(context.assignment()); offsets = new HashMap<>(); - hiveIntegration = connectorConfig.getBoolean(HdfsSinkConnectorConfig.HIVE_INTEGRATION_CONFIG); + hiveIntegration = connectorConfig.getBoolean(HiveConfig.HIVE_INTEGRATION_CONFIG); if (hiveIntegration) { - hiveDatabase = connectorConfig.getString(HdfsSinkConnectorConfig.HIVE_DATABASE_CONFIG); + hiveDatabase = connectorConfig.getString(HiveConfig.HIVE_DATABASE_CONFIG); hiveMetaStore = new HiveMetaStore(conf, connectorConfig); - hive = format.getHiveUtil(connectorConfig, avroData, hiveMetaStore); + hive = format.getHiveUtil(connectorConfig, hiveMetaStore); executorService = Executors.newSingleThreadExecutor(); hiveUpdateFutures = new LinkedList<>(); } @@ -192,8 +199,19 @@ public void run() { topicPartitionWriters = new HashMap<>(); for (TopicPartition tp: assignment) { TopicPartitionWriter topicPartitionWriter = new TopicPartitionWriter( - tp, storage, writerProvider, partitioner, connectorConfig, context, avroData, hiveMetaStore, hive, schemaFileReader, executorService, - hiveUpdateFutures); + tp, + storage, + writerProvider, + partitioner, + connectorConfig, + context, + avroData, + hiveMetaStore, + hive, + schemaFileReader, + executorService, + hiveUpdateFutures + ); topicPartitionWriters.put(tp, topicPartitionWriter); } } catch (ClassNotFoundException | IllegalAccessException | InstantiationException e) { @@ -251,7 +269,7 @@ public void syncWithHive() throws ConnectException { CommittedFileFilter filter = new TopicCommittedFileFilter(topic); FileStatus fileStatusWithMaxOffset = FileUtils.fileStatusWithMaxOffset(storage, new Path(topicDir), filter); if (fileStatusWithMaxOffset != null) { - Schema latestSchema = schemaFileReader.getSchema(conf, fileStatusWithMaxOffset.getPath()); + Schema latestSchema = schemaFileReader.getSchema(connectorConfig, fileStatusWithMaxOffset.getPath()); hive.createTable(hiveDatabase, topic, latestSchema, partitioner); List partitions = hiveMetaStore.listPartitions(hiveDatabase, topic, (short) -1); FileStatus[] statuses = FileUtils.getDirectories(storage, new Path(topicDir)); @@ -273,8 +291,19 @@ public void open(Collection partitions) { assignment = new HashSet<>(partitions); for (TopicPartition tp: assignment) { TopicPartitionWriter topicPartitionWriter = new TopicPartitionWriter( - tp, storage, writerProvider, partitioner, connectorConfig, context, avroData, - hiveMetaStore, hive, schemaFileReader, executorService, hiveUpdateFutures); + tp, + storage, + writerProvider, + partitioner, + connectorConfig, + context, + avroData, + hiveMetaStore, + hive, + schemaFileReader, + executorService, + hiveUpdateFutures + ); topicPartitionWriters.put(tp, topicPartitionWriter); // We need to immediately start recovery to ensure we pause consumption of messages for the // assigned topics while we try to recover offsets and rewind. @@ -351,7 +380,7 @@ public Storage getStorage() { return storage; } - public Map> getWriters(TopicPartition tp) { + public Map getWriters(TopicPartition tp) { return topicPartitionWriters.get(tp).getWriters(); } @@ -363,7 +392,7 @@ public Map getTempFileNames(TopicPartition tp) { private void createDir(String dir) { String path = url + "/" + dir; if (!storage.exists(path)) { - storage.mkdirs(path); + storage.create(path); } } @@ -388,7 +417,7 @@ private Partitioner createPartitioner(HdfsSinkConnectorConfig config) @SuppressWarnings("unchecked") Class partitionerClasss = (Class) - Class.forName(config.getString(HdfsSinkConnectorConfig.PARTITIONER_CLASS_CONFIG)); + Class.forName(config.getString(PartitionerConfig.PARTITIONER_CLASS_CONFIG)); Map map = copyConfig(config); Partitioner partitioner = partitionerClasss.newInstance(); @@ -398,11 +427,20 @@ private Partitioner createPartitioner(HdfsSinkConnectorConfig config) private Map copyConfig(HdfsSinkConnectorConfig config) { Map map = new HashMap<>(); - map.put(HdfsSinkConnectorConfig.PARTITION_FIELD_NAME_CONFIG, config.getString(HdfsSinkConnectorConfig.PARTITION_FIELD_NAME_CONFIG)); - map.put(HdfsSinkConnectorConfig.PARTITION_DURATION_MS_CONFIG, config.getLong(HdfsSinkConnectorConfig.PARTITION_DURATION_MS_CONFIG)); - map.put(HdfsSinkConnectorConfig.PATH_FORMAT_CONFIG, config.getString(HdfsSinkConnectorConfig.PATH_FORMAT_CONFIG)); - map.put(HdfsSinkConnectorConfig.LOCALE_CONFIG, config.getString(HdfsSinkConnectorConfig.LOCALE_CONFIG)); - map.put(HdfsSinkConnectorConfig.TIMEZONE_CONFIG, config.getString(HdfsSinkConnectorConfig.TIMEZONE_CONFIG)); + map.put( + PartitionerConfig.PARTITION_FIELD_NAME_CONFIG, + config.getString(PartitionerConfig.PARTITION_FIELD_NAME_CONFIG) + ); + map.put( + PartitionerConfig.PARTITION_DURATION_MS_CONFIG, + config.getLong(PartitionerConfig.PARTITION_DURATION_MS_CONFIG) + ); + map.put( + PartitionerConfig.PATH_FORMAT_CONFIG, + config.getString(PartitionerConfig.PATH_FORMAT_CONFIG) + ); + map.put(PartitionerConfig.LOCALE_CONFIG, config.getString(PartitionerConfig.LOCALE_CONFIG)); + map.put(PartitionerConfig.TIMEZONE_CONFIG, config.getString(PartitionerConfig.TIMEZONE_CONFIG)); return map; } } diff --git a/src/main/java/io/confluent/connect/hdfs/HdfsSinkTask.java b/src/main/java/io/confluent/connect/hdfs/HdfsSinkTask.java index 3ca769959..9c3e1c170 100644 --- a/src/main/java/io/confluent/connect/hdfs/HdfsSinkTask.java +++ b/src/main/java/io/confluent/connect/hdfs/HdfsSinkTask.java @@ -29,6 +29,8 @@ import java.util.Set; import io.confluent.connect.avro.AvroData; +import io.confluent.connect.storage.hive.HiveConfig; +import io.confluent.connect.storage.partitioner.PartitionerConfig; import io.confluent.connect.storage.schema.StorageSchemaCompatibility; public class HdfsSinkTask extends SinkTask { @@ -51,10 +53,10 @@ public void start(Map props) { Set assignment = context.assignment(); try { HdfsSinkConnectorConfig connectorConfig = new HdfsSinkConnectorConfig(props); - boolean hiveIntegration = connectorConfig.getBoolean(HdfsSinkConnectorConfig.HIVE_INTEGRATION_CONFIG); + boolean hiveIntegration = connectorConfig.getBoolean(HiveConfig.HIVE_INTEGRATION_CONFIG); if (hiveIntegration) { StorageSchemaCompatibility compatibility = StorageSchemaCompatibility.getCompatibility( - connectorConfig.getString(HdfsSinkConnectorConfig.SCHEMA_COMPATIBILITY_CONFIG)); + connectorConfig.getString(HiveConfig.SCHEMA_COMPATIBILITY_CONFIG)); if (compatibility == StorageSchemaCompatibility.NONE) { throw new ConfigException("Hive Integration requires schema compatibility to be BACKWARD, FORWARD or FULL"); } @@ -62,9 +64,9 @@ public void start(Map props) { //check that timezone it setup correctly in case of scheduled rotation if(connectorConfig.getLong(HdfsSinkConnectorConfig.ROTATE_SCHEDULE_INTERVAL_MS_CONFIG) > 0) { - String timeZoneString = connectorConfig.getString(HdfsSinkConnectorConfig.TIMEZONE_CONFIG); + String timeZoneString = connectorConfig.getString(PartitionerConfig.TIMEZONE_CONFIG); if (timeZoneString.equals("")) { - throw new ConfigException(HdfsSinkConnectorConfig.TIMEZONE_CONFIG, + throw new ConfigException(PartitionerConfig.TIMEZONE_CONFIG, timeZoneString, "Timezone cannot be empty when using scheduled file rotation."); } DateTimeZone.forID(timeZoneString); diff --git a/src/main/java/io/confluent/connect/hdfs/TopicPartitionWriter.java b/src/main/java/io/confluent/connect/hdfs/TopicPartitionWriter.java index 5a6b3e8f0..215f11a32 100644 --- a/src/main/java/io/confluent/connect/hdfs/TopicPartitionWriter.java +++ b/src/main/java/io/confluent/connect/hdfs/TopicPartitionWriter.java @@ -49,7 +49,11 @@ import io.confluent.connect.hdfs.hive.HiveMetaStore; import io.confluent.connect.hdfs.hive.HiveUtil; import io.confluent.connect.hdfs.partitioner.Partitioner; +import io.confluent.connect.hdfs.storage.HdfsStorage; import io.confluent.connect.hdfs.storage.Storage; +import io.confluent.connect.storage.common.StorageCommonConfig; +import io.confluent.connect.storage.hive.HiveConfig; +import io.confluent.connect.storage.partitioner.PartitionerConfig; import io.confluent.connect.storage.schema.StorageSchemaCompatibility; import io.confluent.connect.storage.wal.WAL; @@ -57,7 +61,7 @@ public class TopicPartitionWriter { private static final Logger log = LoggerFactory.getLogger(TopicPartitionWriter.class); private WAL wal; private Map tempFiles; - private Map> writers; + private Map writers; private TopicPartition tp; private Partitioner partitioner; private String url; @@ -65,7 +69,7 @@ public class TopicPartitionWriter { private State state; private Queue buffer; private boolean recovered; - private Storage storage; + private HdfsStorage storage; private SinkTaskContext context; private int recordCounter; private int flushSize; @@ -75,6 +79,7 @@ public class TopicPartitionWriter { private long nextScheduledRotate; private RecordWriterProvider writerProvider; private Configuration conf; + private HdfsSinkConnectorConfig connectorConfig; private AvroData avroData; private Set appended; private long offset; @@ -100,7 +105,7 @@ public class TopicPartitionWriter { public TopicPartitionWriter( TopicPartition tp, - Storage storage, + HdfsStorage storage, RecordWriterProvider writerProvider, Partitioner partitioner, HdfsSinkConnectorConfig connectorConfig, @@ -111,7 +116,7 @@ public TopicPartitionWriter( public TopicPartitionWriter( TopicPartition tp, - Storage storage, + HdfsStorage storage, RecordWriterProvider writerProvider, Partitioner partitioner, HdfsSinkConnectorConfig connectorConfig, @@ -129,16 +134,17 @@ public TopicPartitionWriter( this.writerProvider = writerProvider; this.partitioner = partitioner; this.url = storage.url(); - this.conf = storage.conf(); + this.connectorConfig = storage.conf(); + this.conf = storage.conf().getHadoopConfiguration(); this.schemaFileReader = schemaFileReader; - topicsDir = connectorConfig.getString(HdfsSinkConnectorConfig.TOPICS_DIR_CONFIG); + topicsDir = connectorConfig.getString(StorageCommonConfig.TOPICS_DIR_CONFIG); flushSize = connectorConfig.getInt(HdfsSinkConnectorConfig.FLUSH_SIZE_CONFIG); rotateIntervalMs = connectorConfig.getLong(HdfsSinkConnectorConfig.ROTATE_INTERVAL_MS_CONFIG); rotateScheduleIntervalMs = connectorConfig.getLong(HdfsSinkConnectorConfig.ROTATE_SCHEDULE_INTERVAL_MS_CONFIG); timeoutMs = connectorConfig.getLong(HdfsSinkConnectorConfig.RETRY_BACKOFF_CONFIG); compatibility = StorageSchemaCompatibility.getCompatibility( - connectorConfig.getString(HdfsSinkConnectorConfig.SCHEMA_COMPATIBILITY_CONFIG)); + connectorConfig.getString(HiveConfig.SCHEMA_COMPATIBILITY_CONFIG)); String logsDir = connectorConfig.getString(HdfsSinkConnectorConfig.LOGS_DIR_CONFIG); wal = storage.wal(logsDir, tp); @@ -159,9 +165,9 @@ public TopicPartitionWriter( connectorConfig.getInt(HdfsSinkConnectorConfig.FILENAME_OFFSET_ZERO_PAD_WIDTH_CONFIG) + "d"; - hiveIntegration = connectorConfig.getBoolean(HdfsSinkConnectorConfig.HIVE_INTEGRATION_CONFIG); + hiveIntegration = connectorConfig.getBoolean(HiveConfig.HIVE_INTEGRATION_CONFIG); if (hiveIntegration) { - hiveDatabase = connectorConfig.getString(HdfsSinkConnectorConfig.HIVE_DATABASE_CONFIG); + hiveDatabase = connectorConfig.getString(HiveConfig.HIVE_DATABASE_CONFIG); this.hiveMetaStore = hiveMetaStore; this.hive = hive; this.executorService = executorService; @@ -170,7 +176,7 @@ public TopicPartitionWriter( } if(rotateScheduleIntervalMs > 0) { - timeZone = DateTimeZone.forID(connectorConfig.getString(HdfsSinkConnectorConfig.TIMEZONE_CONFIG)); + timeZone = DateTimeZone.forID(connectorConfig.getString(PartitionerConfig.TIMEZONE_CONFIG)); } // Initialize rotation timers @@ -266,15 +272,22 @@ public void write() { if (compatibility != StorageSchemaCompatibility.NONE && offset != -1) { String topicDir = FileUtils.topicDirectory(url, topicsDir, tp.topic()); CommittedFileFilter filter = new TopicPartitionCommittedFileFilter(tp); - FileStatus fileStatusWithMaxOffset = FileUtils.fileStatusWithMaxOffset(storage, new Path(topicDir), filter); + FileStatus fileStatusWithMaxOffset = FileUtils.fileStatusWithMaxOffset( + storage, + new Path(topicDir), + filter + ); if (fileStatusWithMaxOffset != null) { - currentSchema = schemaFileReader.getSchema(conf, fileStatusWithMaxOffset.getPath()); + currentSchema = schemaFileReader.getSchema( + connectorConfig, + fileStatusWithMaxOffset.getPath() + ); } } } SinkRecord record = buffer.peek(); Schema valueSchema = record.valueSchema(); - if (compatibility.shouldChangeSchema(valueSchema, currentSchema)) { + if (compatibility.shouldChangeSchema(record, null, currentSchema)) { currentSchema = valueSchema; if (hiveIntegration) { createHiveTable(); @@ -286,7 +299,7 @@ public void write() { break; } } else { - SinkRecord projectedRecord = compatibility.project(record, currentSchema); + SinkRecord projectedRecord = compatibility.project(record, null, currentSchema); writeRecord(projectedRecord); buffer.poll(); if (shouldRotate(now)) { @@ -391,7 +404,7 @@ public long offset() { return offset; } - public Map> getWriters() { + public Map getWriters() { return writers; } @@ -436,13 +449,18 @@ private void resume() { context.resume(tp); } - private RecordWriter getWriter(SinkRecord record, String encodedPartition) + private RecordWriter getWriter(SinkRecord record, String encodedPartition) throws ConnectException { if (writers.containsKey(encodedPartition)) { return writers.get(encodedPartition); } String tempFile = getTempFile(encodedPartition); - RecordWriter writer = writerProvider.getRecordWriter(conf, tempFile, record, avroData); + RecordWriter writer = writerProvider.getRecordWriter( + connectorConfig, + tempFile, + record, + avroData + ); writers.put(encodedPartition, writer); if (hiveIntegration && !hivePartitions.contains(encodedPartition)) { addHivePartition(encodedPartition); @@ -515,7 +533,7 @@ private void writeRecord(SinkRecord record) { } String encodedPartition = partitioner.encodePartition(record); - RecordWriter writer = getWriter(record, encodedPartition); + RecordWriter writer = getWriter(record, encodedPartition); writer.write(record); if (!startOffsets.containsKey(encodedPartition)) { @@ -527,7 +545,7 @@ private void writeRecord(SinkRecord record) { private void closeTempFile(String encodedPartition) { if (writers.containsKey(encodedPartition)) { - RecordWriter writer = writers.get(encodedPartition); + RecordWriter writer = writers.get(encodedPartition); writer.close(); writers.remove(encodedPartition); } @@ -598,7 +616,7 @@ private void commitFile(String encodedPartition) { String directoryName = FileUtils.directoryName(url, topicsDir, directory); if (!storage.exists(directoryName)) { - storage.mkdirs(directoryName); + storage.create(directoryName); } storage.commit(tempFile, committedFile); startOffsets.remove(encodedPartition); diff --git a/src/main/java/io/confluent/connect/hdfs/storage/HdfsStorage.java b/src/main/java/io/confluent/connect/hdfs/storage/HdfsStorage.java index bd7dd30da..b0c6e6cd5 100644 --- a/src/main/java/io/confluent/connect/hdfs/storage/HdfsStorage.java +++ b/src/main/java/io/confluent/connect/hdfs/storage/HdfsStorage.java @@ -43,6 +43,13 @@ public class HdfsStorage private final HdfsSinkConnectorConfig conf; private final String url; + // Visible for testing. + protected HdfsStorage(HdfsSinkConnectorConfig conf, String url, FileSystem fs) { + this.conf = conf; + this.url = url; + this.fs = fs; + } + public HdfsStorage(HdfsSinkConnectorConfig conf, String url) throws IOException { this.conf = conf; this.url = url; From 4197d0134332a1412bb9703d916b3e69fa551e51 Mon Sep 17 00:00:00 2001 From: Konstantine Karantasis Date: Mon, 24 Apr 2017 19:37:52 -0700 Subject: [PATCH 36/58] Fix formats and their interfaces 3. --- src/main/java/io/confluent/connect/hdfs/Format.java | 5 +++++ .../io/confluent/connect/hdfs/RecordWriterProvider.java | 9 +++++++-- .../java/io/confluent/connect/hdfs/avro/AvroFormat.java | 6 ++++++ .../io/confluent/connect/hdfs/parquet/ParquetFormat.java | 7 +++++++ 4 files changed, 25 insertions(+), 2 deletions(-) diff --git a/src/main/java/io/confluent/connect/hdfs/Format.java b/src/main/java/io/confluent/connect/hdfs/Format.java index 6650f22ac..5be03f93d 100644 --- a/src/main/java/io/confluent/connect/hdfs/Format.java +++ b/src/main/java/io/confluent/connect/hdfs/Format.java @@ -16,11 +16,16 @@ import org.apache.hadoop.fs.Path; +import io.confluent.connect.avro.AvroData; import io.confluent.connect.hdfs.hive.HiveMetaStore; import io.confluent.connect.hdfs.hive.HiveUtil; @Deprecated public interface Format extends io.confluent.connect.storage.format.Format { + RecordWriterProvider getRecordWriterProvider(); + SchemaFileReader getSchemaFileReader(AvroData avroData); + @Override + SchemaFileReader getSchemaFileReader(); HiveUtil getHiveUtil(HdfsSinkConnectorConfig config, HiveMetaStore hiveMetaStore); } diff --git a/src/main/java/io/confluent/connect/hdfs/RecordWriterProvider.java b/src/main/java/io/confluent/connect/hdfs/RecordWriterProvider.java index 218e4cd86..5ff6b014b 100644 --- a/src/main/java/io/confluent/connect/hdfs/RecordWriterProvider.java +++ b/src/main/java/io/confluent/connect/hdfs/RecordWriterProvider.java @@ -20,12 +20,17 @@ @Deprecated public interface RecordWriterProvider - extends io.confluent.connect.storage.format.RecordWriterProvider - { + extends io.confluent.connect.storage.format.RecordWriterProvider { RecordWriter getRecordWriter( HdfsSinkConnectorConfig conf, String fileName, SinkRecord record, final AvroData avroData ); + + @Override + RecordWriter getRecordWriter( + HdfsSinkConnectorConfig conf, + String fileName + ); } diff --git a/src/main/java/io/confluent/connect/hdfs/avro/AvroFormat.java b/src/main/java/io/confluent/connect/hdfs/avro/AvroFormat.java index 04779ac7c..2fe03a34a 100644 --- a/src/main/java/io/confluent/connect/hdfs/avro/AvroFormat.java +++ b/src/main/java/io/confluent/connect/hdfs/avro/AvroFormat.java @@ -46,6 +46,12 @@ public SchemaFileReader getSchemaFileReader() { return new AvroFileReader(avroData); } + @Override + public SchemaFileReader getSchemaFileReader(AvroData avroData) { + // Argument is ignored. + return getSchemaFileReader(); + } + @Deprecated @Override public HiveUtil getHiveUtil( diff --git a/src/main/java/io/confluent/connect/hdfs/parquet/ParquetFormat.java b/src/main/java/io/confluent/connect/hdfs/parquet/ParquetFormat.java index 9247567b3..a4c2365cd 100644 --- a/src/main/java/io/confluent/connect/hdfs/parquet/ParquetFormat.java +++ b/src/main/java/io/confluent/connect/hdfs/parquet/ParquetFormat.java @@ -47,6 +47,13 @@ public SchemaFileReader getSchemaFileReader() { return new ParquetFileReader(avroData); } + @Deprecated + @Override + public SchemaFileReader getSchemaFileReader(AvroData avroData) { + // Argument is ignored. + return getSchemaFileReader(); + } + @Deprecated @Override public HiveUtil getHiveUtil( From 9809b1a17147a0a9befa1e9218f9900c89d2e1c7 Mon Sep 17 00:00:00 2001 From: Konstantine Karantasis Date: Mon, 24 Apr 2017 19:42:14 -0700 Subject: [PATCH 37/58] Test fixes in wal package. --- .../confluent/connect/hdfs/wal/FSWALTest.java | 2 +- .../connect/hdfs/wal/WALFileTest.java | 13 ++++--- .../confluent/connect/hdfs/wal/WALTest.java | 36 ++++++++++++------- 3 files changed, 33 insertions(+), 18 deletions(-) diff --git a/src/test/java/io/confluent/connect/hdfs/wal/FSWALTest.java b/src/test/java/io/confluent/connect/hdfs/wal/FSWALTest.java index edcca1737..be98c63f9 100644 --- a/src/test/java/io/confluent/connect/hdfs/wal/FSWALTest.java +++ b/src/test/java/io/confluent/connect/hdfs/wal/FSWALTest.java @@ -29,7 +29,7 @@ public class FSWALTest extends TestWithMiniDFSCluster { @Test public void testTruncate() throws Exception { - Storage storage = new HdfsStorage(conf, url); + HdfsStorage storage = new HdfsStorage(connectorConfig, url); TopicPartition tp = new TopicPartition("mytopic", 123); FSWAL wal = new FSWAL("/logs", tp, storage); wal.append("a", "b"); diff --git a/src/test/java/io/confluent/connect/hdfs/wal/WALFileTest.java b/src/test/java/io/confluent/connect/hdfs/wal/WALFileTest.java index 4bb6bc629..fc59fdaa4 100644 --- a/src/test/java/io/confluent/connect/hdfs/wal/WALFileTest.java +++ b/src/test/java/io/confluent/connect/hdfs/wal/WALFileTest.java @@ -26,6 +26,7 @@ import io.confluent.connect.hdfs.FileUtils; import io.confluent.connect.hdfs.HdfsSinkConnectorConfig; import io.confluent.connect.hdfs.TestWithMiniDFSCluster; +import io.confluent.connect.storage.common.StorageCommonConfig; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; @@ -33,18 +34,18 @@ public class WALFileTest extends TestWithMiniDFSCluster { @Test - public void testeAppend() throws Exception { + public void testAppend() throws Exception { Map props = createProps(); HdfsSinkConnectorConfig connectorConfig = new HdfsSinkConnectorConfig(props); - String topicsDir = connectorConfig.getString(HdfsSinkConnectorConfig.TOPICS_DIR_CONFIG); + String topicsDir = connectorConfig.getString(StorageCommonConfig.TOPICS_DIR_CONFIG); String topic = "topic"; int partition = 0; TopicPartition topicPart = new TopicPartition(topic, partition); Path file = new Path(FileUtils.logFileName(url, topicsDir, topicPart)); - WALFile.Writer writer = WALFile.createWriter(conf, WALFile.Writer.file(file)); + WALFile.Writer writer = WALFile.createWriter(connectorConfig, WALFile.Writer.file(file)); WALEntry key1 = new WALEntry("key1"); WALEntry val1 = new WALEntry("val1"); @@ -58,7 +59,11 @@ public void testeAppend() throws Exception { verify2Values(file); - writer = WALFile.createWriter(conf, WALFile.Writer.file(file), WALFile.Writer.appendIfExists(true)); + writer = WALFile.createWriter( + connectorConfig, + WALFile.Writer.file(file), + WALFile.Writer.appendIfExists(true) + ); WALEntry key3 = new WALEntry("key3"); WALEntry val3 = new WALEntry("val3"); diff --git a/src/test/java/io/confluent/connect/hdfs/wal/WALTest.java b/src/test/java/io/confluent/connect/hdfs/wal/WALTest.java index 6364aa3fa..6a61ca800 100644 --- a/src/test/java/io/confluent/connect/hdfs/wal/WALTest.java +++ b/src/test/java/io/confluent/connect/hdfs/wal/WALTest.java @@ -14,16 +14,15 @@ package io.confluent.connect.hdfs.wal; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.kafka.connect.errors.ConnectException; import org.junit.Test; -import io.confluent.connect.storage.StorageFactory; import io.confluent.connect.hdfs.FileUtils; import io.confluent.connect.hdfs.HdfsSinkConnectorConfig; import io.confluent.connect.hdfs.TestWithMiniDFSCluster; -import io.confluent.connect.hdfs.storage.Storage; +import io.confluent.connect.hdfs.storage.HdfsStorage; +import io.confluent.connect.storage.common.StorageCommonConfig; import io.confluent.connect.storage.wal.WAL; import static org.junit.Assert.assertFalse; @@ -31,6 +30,7 @@ public class WALTest extends TestWithMiniDFSCluster { private static final String ZERO_PAD_FMT = "%010d"; + private HdfsStorage storage; private boolean closed; private static final String extension = ".avro"; @@ -40,24 +40,34 @@ public void testWALMultiClient() throws Exception { fs.delete(new Path(FileUtils.directoryName(url, topicsDir, TOPIC_PARTITION)), true); @SuppressWarnings("unchecked") - Class storageClass = (Class) - Class.forName(connectorConfig.getString(HdfsSinkConnectorConfig.STORAGE_CLASS_CONFIG)); - Storage storage = StorageFactory.createStorage(storageClass, Configuration.class, conf, url); - + Class storageClass = (Class) connectorConfig + .getClass(StorageCommonConfig.STORAGE_CLASS_CONFIG); + storage = io.confluent.connect.storage.StorageFactory.createStorage( + storageClass, + HdfsSinkConnectorConfig.class, + connectorConfig, + url + ); final WAL wal1 = storage.wal(topicsDir, TOPIC_PARTITION); final WAL wal2 = storage.wal(topicsDir, TOPIC_PARTITION); String directory = TOPIC + "/" + String.valueOf(PARTITION); final String tempfile = FileUtils.tempFileName(url, topicsDir, directory, extension); - final String commitedFile = FileUtils.committedFileName(url, topicsDir, directory, - TOPIC_PARTITION, 0, 10, extension, - ZERO_PAD_FMT); - + final String committedFile = FileUtils.committedFileName( + url, + topicsDir, + directory, + TOPIC_PARTITION, + 0, + 10, + extension, + ZERO_PAD_FMT + ); fs.createNewFile(new Path(tempfile)); wal1.acquireLease(); wal1.append(WAL.beginMarker, ""); - wal1.append(tempfile, commitedFile); + wal1.append(tempfile, committedFile); wal1.append(WAL.endMarker, ""); Thread thread = new Thread(new Runnable() { @@ -80,7 +90,7 @@ public void run() { wal2.apply(); wal2.close(); - assertTrue(fs.exists(new Path(commitedFile))); + assertTrue(fs.exists(new Path(committedFile))); assertFalse(fs.exists(new Path(tempfile))); storage.close(); } From 06f40b0d603884dd8fddd981d6f7bee93d9b2b4a Mon Sep 17 00:00:00 2001 From: Konstantine Karantasis Date: Mon, 24 Apr 2017 19:54:14 -0700 Subject: [PATCH 38/58] Test fixes in utils/memory storage. --- .../connect/hdfs/utils/MemoryFormat.java | 18 +++++++++----- .../hdfs/utils/MemoryRecordWriter.java | 5 +++- .../utils/MemoryRecordWriterProvider.java | 23 ++++++++++++++---- .../connect/hdfs/utils/MemoryStorage.java | 24 +++++++++---------- .../connect/hdfs/utils/MemoryWAL.java | 4 ++-- 5 files changed, 48 insertions(+), 26 deletions(-) diff --git a/src/test/java/io/confluent/connect/hdfs/utils/MemoryFormat.java b/src/test/java/io/confluent/connect/hdfs/utils/MemoryFormat.java index 0c978946b..58fcefd84 100644 --- a/src/test/java/io/confluent/connect/hdfs/utils/MemoryFormat.java +++ b/src/test/java/io/confluent/connect/hdfs/utils/MemoryFormat.java @@ -24,17 +24,23 @@ public SchemaFileReader getSchemaFileReader(AvroData avroData) { } @Override - public HiveUtil getHiveUtil(HdfsSinkConnectorConfig config, AvroData avroData, HiveMetaStore hiveMetaStore) { + public SchemaFileReader getSchemaFileReader() { return null; } @Override - public HiveFactory getHiveFactory() { - return new HiveFactory() { + public HiveUtil getHiveUtil(HdfsSinkConnectorConfig config, HiveMetaStore hiveMetaStore) { + return null; + } + + @Override + public HiveFactory getHiveFactory() { + return new HiveFactory() { @Override - public io.confluent.connect.storage.hive.HiveUtil createHiveUtil(HdfsSinkConnectorConfig abstractConfig, - AvroData avroData, - io.confluent.connect.storage.hive.HiveMetaStore hiveMetaStore) { + public io.confluent.connect.storage.hive.HiveUtil createHiveUtil( + AbstractConfig abstractConfig, + io.confluent.connect.storage.hive.HiveMetaStore hiveMetaStore + ) { return null; } }; diff --git a/src/test/java/io/confluent/connect/hdfs/utils/MemoryRecordWriter.java b/src/test/java/io/confluent/connect/hdfs/utils/MemoryRecordWriter.java index 603f1b60d..94723204d 100644 --- a/src/test/java/io/confluent/connect/hdfs/utils/MemoryRecordWriter.java +++ b/src/test/java/io/confluent/connect/hdfs/utils/MemoryRecordWriter.java @@ -22,7 +22,7 @@ import io.confluent.connect.hdfs.RecordWriter; -public class MemoryRecordWriter implements RecordWriter { +public class MemoryRecordWriter implements RecordWriter { private String filename; private static final Map> data = Data.getData(); private Failure failure = Failure.noFailure; @@ -47,6 +47,9 @@ public void write(SinkRecord record) { } + @Override + public void commit() {} + @Override public void close() { if (failure == Failure.closeFailure) { diff --git a/src/test/java/io/confluent/connect/hdfs/utils/MemoryRecordWriterProvider.java b/src/test/java/io/confluent/connect/hdfs/utils/MemoryRecordWriterProvider.java index 2e49cf682..2f77b3734 100644 --- a/src/test/java/io/confluent/connect/hdfs/utils/MemoryRecordWriterProvider.java +++ b/src/test/java/io/confluent/connect/hdfs/utils/MemoryRecordWriterProvider.java @@ -24,6 +24,7 @@ import java.util.Map; import io.confluent.connect.avro.AvroData; +import io.confluent.connect.hdfs.HdfsSinkConnectorConfig; import io.confluent.connect.hdfs.RecordWriter; import io.confluent.connect.hdfs.RecordWriterProvider; @@ -35,15 +36,27 @@ public String getExtension() { } @Override - public RecordWriter getRecordWriter(Configuration conf, final String fileName, SinkRecord record, - final AvroData avroData) { + public RecordWriter getRecordWriter( + HdfsSinkConnectorConfig conf, + final String filename, + SinkRecord record, + final AvroData avroData + ) { + return getRecordWriter(conf, filename); + } + + @Override + public RecordWriter getRecordWriter( + HdfsSinkConnectorConfig conf, + final String filename + ) { final Map> data = Data.getData(); - if (!data.containsKey(fileName)) { - data.put(fileName, new LinkedList<>()); + if (!data.containsKey(filename)) { + data.put(filename, new LinkedList<>()); } - return new MemoryRecordWriter(fileName); + return new MemoryRecordWriter(filename); } } diff --git a/src/test/java/io/confluent/connect/hdfs/utils/MemoryStorage.java b/src/test/java/io/confluent/connect/hdfs/utils/MemoryStorage.java index 9839fa451..7c459c313 100644 --- a/src/test/java/io/confluent/connect/hdfs/utils/MemoryStorage.java +++ b/src/test/java/io/confluent/connect/hdfs/utils/MemoryStorage.java @@ -30,13 +30,14 @@ import java.util.List; import java.util.Map; -import io.confluent.connect.hdfs.storage.Storage; +import io.confluent.connect.hdfs.HdfsSinkConnectorConfig; +import io.confluent.connect.hdfs.storage.HdfsStorage; import io.confluent.connect.hdfs.wal.WAL; -public class MemoryStorage implements Storage { +public class MemoryStorage extends HdfsStorage { private static final Map> data = Data.getData(); - private Configuration conf; + private HdfsSinkConnectorConfig conf; private String url; private Failure failure = Failure.noFailure; @@ -51,13 +52,14 @@ public enum Failure { closeFailure } - public MemoryStorage(Configuration conf, String url) { + public MemoryStorage(HdfsSinkConnectorConfig conf, String url) { + super(conf, url, null); this.conf = conf; this.url = url; } @Override - public List listStatus(String path) { + public List list(String path) { List result = new ArrayList<>(); for (String key: data.keySet()) { if (key.startsWith(path)) { @@ -68,8 +70,7 @@ public List listStatus(String path) { return result; } - @Override - public List listStatus(String path, PathFilter filter) { + public List list(String path, PathFilter filter) { if (failure == Failure.listStatusFailure) { failure = Failure.noFailure; throw new ConnectException("listStatus failed."); @@ -84,7 +85,6 @@ public List listStatus(String path, PathFilter filter) { return result; } - @Override public void append(String filename, Object object) { if (failure == Failure.appendFailure) { failure = Failure.noFailure; @@ -97,7 +97,7 @@ public void append(String filename, Object object) { } @Override - public boolean mkdirs(String filename) { + public boolean create(String filename) { if (failure == Failure.mkdirsFailure) { failure = Failure.noFailure; throw new ConnectException("mkdirs failed."); @@ -154,7 +154,7 @@ public WAL wal(String topicsDir, TopicPartition topicPart) { } @Override - public Configuration conf() { + public HdfsSinkConnectorConfig conf() { return conf; } @@ -168,12 +168,12 @@ public void setFailure(Failure failure) { } @Override - public SeekableInput open(String filename, Configuration conf) { + public SeekableInput open(String filename, HdfsSinkConnectorConfig conf) { return null; } @Override - public OutputStream create(String filename, Configuration conf, boolean overwrite) { + public OutputStream create(String filename, HdfsSinkConnectorConfig conf, boolean overwrite) { return null; } } diff --git a/src/test/java/io/confluent/connect/hdfs/utils/MemoryWAL.java b/src/test/java/io/confluent/connect/hdfs/utils/MemoryWAL.java index 12d85879c..e16add5dc 100644 --- a/src/test/java/io/confluent/connect/hdfs/utils/MemoryWAL.java +++ b/src/test/java/io/confluent/connect/hdfs/utils/MemoryWAL.java @@ -29,10 +29,10 @@ public class MemoryWAL implements WAL { private String logFile; - private Storage storage; + private MemoryStorage storage; private static Map> data = Data.getData(); - public MemoryWAL(String topicsDir, TopicPartition topicPart, Storage storage) + public MemoryWAL(String topicsDir, TopicPartition topicPart, MemoryStorage storage) throws ConnectException { this.storage = storage; String url = storage.url(); From e6cd4ee90f285638b710f84606553b25afc165fc Mon Sep 17 00:00:00 2001 From: Konstantine Karantasis Date: Tue, 25 Apr 2017 17:22:27 -0700 Subject: [PATCH 39/58] Fix in main config. --- .../connect/hdfs/HdfsSinkConnectorConfig.java | 17 ++++++++++++++++- 1 file changed, 16 insertions(+), 1 deletion(-) diff --git a/src/main/java/io/confluent/connect/hdfs/HdfsSinkConnectorConfig.java b/src/main/java/io/confluent/connect/hdfs/HdfsSinkConnectorConfig.java index d4dcba13e..402c148e8 100644 --- a/src/main/java/io/confluent/connect/hdfs/HdfsSinkConnectorConfig.java +++ b/src/main/java/io/confluent/connect/hdfs/HdfsSinkConnectorConfig.java @@ -20,6 +20,7 @@ import org.apache.kafka.common.config.ConfigDef.Importance; import org.apache.kafka.common.config.ConfigDef.Type; import org.apache.kafka.common.config.ConfigDef.Width; +import org.apache.kafka.common.config.ConfigException; import java.util.Arrays; import java.util.HashMap; @@ -102,7 +103,7 @@ public class HdfsSinkConnectorConfig extends StorageSinkConnectorConfig { private static final ConfigDef.Recommender hdfsAuthenticationKerberosDependentsRecommender = new BooleanParentRecommender(HDFS_AUTHENTICATION_KERBEROS_CONFIG); private final String name; - private final Configuration hadoopConfig; + private Configuration hadoopConfig; private final StorageCommonConfig commonConfig; private final HiveConfig hiveConfig; @@ -282,10 +283,24 @@ public String getName() { return name; } + @Override + public Object get(String key) { + ComposableConfig config = propertyToConfig.get(key); + if (config == null) { + throw new ConfigException(String.format("Unknown configuration '%s'", key)); + } + return config == this ? super.get(key) : config.get(key); + } + public Configuration getHadoopConfiguration() { return hadoopConfig; } + // Visible for testing. + public void setHadoopConfiguration(Configuration conf) { + hadoopConfig = conf; + } + public Map plainValues() { Map map = new HashMap<>(); for (AbstractConfig config : allConfigs) { From cbdbffa8456640158822419035a1c76fb94fabe5 Mon Sep 17 00:00:00 2001 From: Konstantine Karantasis Date: Mon, 24 Apr 2017 21:59:13 -0700 Subject: [PATCH 40/58] Test fixes in main test classes. --- .../connect/hdfs/FailureRecoveryTest.java | 22 ++++---- .../hdfs/HdfsSinkConnectorTestBase.java | 53 +++++++++++++------ .../connect/hdfs/HdfsSinkTaskTest.java | 34 +++++++----- .../hdfs/HdfsSinkTaskTestWithSecureHDFS.java | 10 ++-- .../connect/hdfs/TestWithMiniDFSCluster.java | 32 +++++------ 5 files changed, 90 insertions(+), 61 deletions(-) diff --git a/src/test/java/io/confluent/connect/hdfs/FailureRecoveryTest.java b/src/test/java/io/confluent/connect/hdfs/FailureRecoveryTest.java index 5dff2c4d3..25772ef24 100644 --- a/src/test/java/io/confluent/connect/hdfs/FailureRecoveryTest.java +++ b/src/test/java/io/confluent/connect/hdfs/FailureRecoveryTest.java @@ -29,6 +29,7 @@ import io.confluent.connect.hdfs.utils.MemoryFormat; import io.confluent.connect.hdfs.utils.MemoryRecordWriter; import io.confluent.connect.hdfs.utils.MemoryStorage; +import io.confluent.connect.storage.common.StorageCommonConfig; import static org.junit.Assert.assertEquals; @@ -36,23 +37,21 @@ public class FailureRecoveryTest extends HdfsSinkConnectorTestBase { private static final String ZERO_PAD_FMT = "%010d"; private static final String extension = ""; - @Before - public void setUp() throws Exception { - super.setUp(); - } - @Override protected Map createProps() { Map props = super.createProps(); - props.put(HdfsSinkConnectorConfig.STORAGE_CLASS_CONFIG, MemoryStorage.class.getName()); + props.put(StorageCommonConfig.STORAGE_CLASS_CONFIG, MemoryStorage.class.getName()); props.put(HdfsSinkConnectorConfig.FORMAT_CLASS_CONFIG, MemoryFormat.class.getName()); return props; } + public void setUp() throws Exception { + super.setUp(); + } + @Test public void testCommitFailure() throws Exception { - Map props = createProps(); - HdfsSinkConnectorConfig connectorConfig = new HdfsSinkConnectorConfig(props); + setUp(); String key = "key"; Schema schema = createSchema(); @@ -93,8 +92,7 @@ public void testCommitFailure() throws Exception { @Test public void testWriterFailureMultiPartitions() throws Exception { - Map props = createProps(); - HdfsSinkConnectorConfig connectorConfig = new HdfsSinkConnectorConfig(props); + setUp(); String key = "key"; Schema schema = createSchema(); @@ -121,7 +119,7 @@ public void testWriterFailureMultiPartitions() throws Exception { } String encodedPartition = "partition=" + String.valueOf(PARTITION); - Map> writers = hdfsWriter.getWriters(TOPIC_PARTITION); + Map writers = hdfsWriter.getWriters(TOPIC_PARTITION); MemoryRecordWriter writer = (MemoryRecordWriter) writers.get(encodedPartition); writer.setFailure(MemoryRecordWriter.Failure.writeFailure); hdfsWriter.write(sinkRecords); @@ -190,7 +188,7 @@ public void testWriterFailure() throws Exception { } String encodedPartition = "partition=" + String.valueOf(PARTITION); - Map> writers = hdfsWriter.getWriters(TOPIC_PARTITION); + Map writers = hdfsWriter.getWriters(TOPIC_PARTITION); MemoryRecordWriter writer = (MemoryRecordWriter) writers.get(encodedPartition); writer.setFailure(MemoryRecordWriter.Failure.writeFailure); diff --git a/src/test/java/io/confluent/connect/hdfs/HdfsSinkConnectorTestBase.java b/src/test/java/io/confluent/connect/hdfs/HdfsSinkConnectorTestBase.java index 35b721095..19dd8f6d0 100644 --- a/src/test/java/io/confluent/connect/hdfs/HdfsSinkConnectorTestBase.java +++ b/src/test/java/io/confluent/connect/hdfs/HdfsSinkConnectorTestBase.java @@ -26,15 +26,20 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.concurrent.TimeUnit; import io.confluent.connect.avro.AvroData; +import io.confluent.connect.hdfs.avro.AvroFormat; import io.confluent.connect.storage.StorageSinkTestBase; import io.confluent.connect.storage.common.StorageCommonConfig; +import io.confluent.connect.storage.hive.schema.DefaultSchemaGenerator; +import io.confluent.connect.storage.partitioner.PartitionerConfig; public class HdfsSinkConnectorTestBase extends StorageSinkTestBase { - protected Configuration conf; protected HdfsSinkConnectorConfig connectorConfig; + protected Map parsedConfig; + protected Configuration conf; protected String topicsDir; protected String logsDir; protected AvroData avroData; @@ -44,9 +49,32 @@ public class HdfsSinkConnectorTestBase extends StorageSinkTestBase { @Override protected Map createProps() { - Map props = new HashMap<>(); + url = "memory://"; + Map props = super.createProps(); props.put(HdfsSinkConnectorConfig.HDFS_URL_CONFIG, url); props.put(HdfsSinkConnectorConfig.FLUSH_SIZE_CONFIG, "3"); + props.put( + StorageCommonConfig.STORAGE_CLASS_CONFIG, + "io.confluent.connect.s3.storage.S3Storage" + ); + props.put(HdfsSinkConnectorConfig.FORMAT_CLASS_CONFIG, AvroFormat.class.getName()); + props.put( + PartitionerConfig.PARTITIONER_CLASS_CONFIG, + PartitionerConfig.PARTITIONER_CLASS_DEFAULT.getName() + ); + props.put( + PartitionerConfig.SCHEMA_GENERATOR_CLASS_CONFIG, + DefaultSchemaGenerator.class.getName() + ); + props.put(PartitionerConfig.PARTITION_FIELD_NAME_CONFIG, "int"); + props.put( + PartitionerConfig.PARTITION_DURATION_MS_CONFIG, + String.valueOf(TimeUnit.HOURS.toMillis(1)) + ); + props.put(PartitionerConfig.PATH_FORMAT_CONFIG, "'year'=YYYY/'month'=MM/'day'=dd/'hour'=HH/"); + props.put(PartitionerConfig.LOCALE_CONFIG, "en"); + props.put(PartitionerConfig.TIMEZONE_CONFIG, "America/Los_Angeles"); + return props; } @@ -84,12 +112,15 @@ protected List createRecordBatches(Schema schema, int batchSize, int bat //@Before @Override public void setUp() throws Exception { - conf = new Configuration(); - url = "memory://"; super.setUp(); - // Configure immediately in setup for common case of just using this default. Subclasses can - // re-call this safely. - configureConnector(); + connectorConfig = new HdfsSinkConnectorConfig(properties); + parsedConfig = new HashMap<>(connectorConfig.plainValues()); + conf = connectorConfig.getHadoopConfiguration(); + topicsDir = connectorConfig.getString(StorageCommonConfig.TOPICS_DIR_CONFIG); + logsDir = connectorConfig.getString(HdfsSinkConnectorConfig.LOGS_DIR_CONFIG); + avroData = new AvroData( + connectorConfig.getInt(HdfsSinkConnectorConfig.SCHEMA_CACHE_SIZE_CONFIG) + ); } @After @@ -97,12 +128,4 @@ public void setUp() throws Exception { public void tearDown() throws Exception { super.tearDown(); } - - protected void configureConnector() { - connectorConfig = new HdfsSinkConnectorConfig(properties); - topicsDir = connectorConfig.getString(StorageCommonConfig.TOPICS_DIR_CONFIG); - logsDir = connectorConfig.getString(HdfsSinkConnectorConfig.LOGS_DIR_CONFIG); - int schemaCacheSize = connectorConfig.getInt(HdfsSinkConnectorConfig.SCHEMA_CACHE_SIZE_CONFIG); - avroData = new AvroData(schemaCacheSize); - } } diff --git a/src/test/java/io/confluent/connect/hdfs/HdfsSinkTaskTest.java b/src/test/java/io/confluent/connect/hdfs/HdfsSinkTaskTest.java index eba45e47a..c5744fee4 100644 --- a/src/test/java/io/confluent/connect/hdfs/HdfsSinkTaskTest.java +++ b/src/test/java/io/confluent/connect/hdfs/HdfsSinkTaskTest.java @@ -14,7 +14,6 @@ package io.confluent.connect.hdfs; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.connect.data.Schema; @@ -31,8 +30,9 @@ import io.confluent.connect.avro.AvroData; import io.confluent.connect.hdfs.avro.AvroFileReader; -import io.confluent.connect.hdfs.storage.Storage; +import io.confluent.connect.hdfs.storage.HdfsStorage; import io.confluent.connect.storage.StorageFactory; +import io.confluent.connect.storage.common.StorageCommonConfig; import io.confluent.connect.storage.wal.WAL; import static org.junit.Assert.assertEquals; @@ -119,13 +119,13 @@ public void testSinkTaskStartWithRecovery() throws Exception { fs.createNewFile(new Path(file)); } } - createWALs(tempfiles, committedFiles); - Map props = createProps(); + setUp(); + createWALs(tempfiles, committedFiles); HdfsSinkTask task = new HdfsSinkTask(); task.initialize(context); - task.start(props); + task.start(properties); Map offsets = context.offsets(); assertEquals(2, offsets.size()); @@ -146,7 +146,7 @@ public void testSinkTaskPut() throws Exception { Schema schema = createSchema(); Struct record = createRecord(schema); Collection sinkRecords = new ArrayList<>(); - for (TopicPartition tp: assignment) { + for (TopicPartition tp : context.assignment()) { for (long offset = 0; offset < 7; offset++) { SinkRecord sinkRecord = new SinkRecord(tp.topic(), tp.partition(), Schema.STRING_SCHEMA, key, schema, record, offset); @@ -162,7 +162,7 @@ public void testSinkTaskPut() throws Exception { // Last file (offset 6) doesn't satisfy size requirement and gets discarded on close long[] validOffsets = {-1, 2, 5}; - for (TopicPartition tp : assignment) { + for (TopicPartition tp : context.assignment()) { String directory = tp.topic() + "/" + "partition=" + String.valueOf(tp.partition()); for (int j = 1; j < validOffsets.length; ++j) { long startOffset = validOffsets[j - 1] + 1; @@ -170,7 +170,7 @@ public void testSinkTaskPut() throws Exception { Path path = new Path(FileUtils.committedFileName(url, topicsDir, directory, tp, startOffset, endOffset, extension, ZERO_PAD_FMT)); - Collection records = schemaFileReader.readData(conf, path); + Collection records = schemaFileReader.readData(connectorConfig, path); long size = endOffset - startOffset + 1; assertEquals(records.size(), size); for (Object avroRecord : records) { @@ -189,7 +189,7 @@ public void testSinkTaskPutPrimitive() throws Exception { final Schema schema = Schema.INT32_SCHEMA; final int record = 12; Collection sinkRecords = new ArrayList<>(); - for (TopicPartition tp: assignment) { + for (TopicPartition tp : context.assignment()) { for (long offset = 0; offset < 7; offset++) { SinkRecord sinkRecord = new SinkRecord(tp.topic(), tp.partition(), Schema.STRING_SCHEMA, key, schema, record, offset); @@ -205,7 +205,7 @@ final int record = 12; // Last file (offset 6) doesn't satisfy size requirement and gets discarded on close long[] validOffsets = {-1, 2, 5}; - for (TopicPartition tp : assignment) { + for (TopicPartition tp : context.assignment()) { String directory = tp.topic() + "/" + "partition=" + String.valueOf(tp.partition()); for (int j = 1; j < validOffsets.length; ++j) { long startOffset = validOffsets[j - 1] + 1; @@ -213,7 +213,7 @@ final int record = 12; Path path = new Path(FileUtils.committedFileName(url, topicsDir, directory, tp, startOffset, endOffset, extension, ZERO_PAD_FMT)); - Collection records = schemaFileReader.readData(conf, path); + Collection records = schemaFileReader.readData(connectorConfig, path); long size = endOffset - startOffset + 1; assertEquals(records.size(), size); for (Object avroRecord : records) { @@ -241,9 +241,15 @@ private void createCommittedFiles() throws IOException { private void createWALs(Map> tempfiles, Map> committedFiles) throws Exception { @SuppressWarnings("unchecked") - Class storageClass = (Class) - Class.forName(connectorConfig.getString(HdfsSinkConnectorConfig.STORAGE_CLASS_CONFIG)); - Storage storage = StorageFactory.createStorage(storageClass, Configuration.class, conf, url); + Class storageClass = (Class) Class.forName( + connectorConfig.getString(StorageCommonConfig.STORAGE_CLASS_CONFIG) + ); + HdfsStorage storage = StorageFactory.createStorage( + storageClass, + HdfsSinkConnectorConfig.class, + connectorConfig, + url + ); for (TopicPartition tp: tempfiles.keySet()) { WAL wal = storage.wal(logsDir, tp); diff --git a/src/test/java/io/confluent/connect/hdfs/HdfsSinkTaskTestWithSecureHDFS.java b/src/test/java/io/confluent/connect/hdfs/HdfsSinkTaskTestWithSecureHDFS.java index dbc6ab679..83e2eddcc 100644 --- a/src/test/java/io/confluent/connect/hdfs/HdfsSinkTaskTestWithSecureHDFS.java +++ b/src/test/java/io/confluent/connect/hdfs/HdfsSinkTaskTestWithSecureHDFS.java @@ -38,14 +38,14 @@ public class HdfsSinkTaskTestWithSecureHDFS extends TestWithSecureMiniDFSCluster @Test public void testSinkTaskPut() throws Exception { - Map props = createProps(); + setUp(); HdfsSinkTask task = new HdfsSinkTask(); String key = "key"; Schema schema = createSchema(); Struct record = createRecord(schema); Collection sinkRecords = new ArrayList<>(); - for (TopicPartition tp : assignment) { + for (TopicPartition tp : context.assignment()) { for (long offset = 0; offset < 7; offset++) { SinkRecord sinkRecord = new SinkRecord(tp.topic(), tp.partition(), Schema.STRING_SCHEMA, key, schema, record, @@ -54,7 +54,7 @@ public void testSinkTaskPut() throws Exception { } } task.initialize(context); - task.start(props); + task.start(properties); task.put(sinkRecords); task.stop(); @@ -62,7 +62,7 @@ public void testSinkTaskPut() throws Exception { // Last file (offset 6) doesn't satisfy size requirement and gets discarded on close long[] validOffsets = {-1, 2, 5}; - for (TopicPartition tp : assignment) { + for (TopicPartition tp : context.assignment()) { String directory = tp.topic() + "/" + "partition=" + String.valueOf(tp.partition()); for (int j = 1; j < validOffsets.length; ++j) { long startOffset = validOffsets[j - 1] + 1; @@ -70,7 +70,7 @@ public void testSinkTaskPut() throws Exception { Path path = new Path(FileUtils.committedFileName(url, topicsDir, directory, tp, startOffset, endOffset, extension, ZERO_PAD_FMT)); - Collection records = schemaFileReader.readData(conf, path); + Collection records = schemaFileReader.readData(connectorConfig, path); long size = endOffset - startOffset + 1; assertEquals(records.size(), size); for (Object avroRecord : records) { diff --git a/src/test/java/io/confluent/connect/hdfs/TestWithMiniDFSCluster.java b/src/test/java/io/confluent/connect/hdfs/TestWithMiniDFSCluster.java index ffa26fb1a..9cf76f1a4 100644 --- a/src/test/java/io/confluent/connect/hdfs/TestWithMiniDFSCluster.java +++ b/src/test/java/io/confluent/connect/hdfs/TestWithMiniDFSCluster.java @@ -28,19 +28,20 @@ import org.apache.kafka.connect.data.Struct; import org.apache.kafka.connect.sink.SinkRecord; import org.junit.After; -import org.junit.Before; import java.io.FileNotFoundException; import java.io.IOException; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Set; import io.confluent.connect.hdfs.filter.TopicPartitionCommittedFileFilter; import io.confluent.connect.hdfs.partitioner.Partitioner; +import io.confluent.connect.storage.common.StorageCommonConfig; import static org.hamcrest.CoreMatchers.is; import static org.junit.Assert.assertEquals; @@ -55,17 +56,25 @@ public class TestWithMiniDFSCluster extends HdfsSinkConnectorTestBase { protected String extension; // The default based on default configuration of 10 protected String zeroPadFormat = "%010d"; + private Map localProps = new HashMap<>(); - @Before + @Override + protected Map createProps() { + Map props = super.createProps(); + url = "hdfs://" + cluster.getNameNode().getClientNamenodeAddress(); + // Override configs using url here + localProps.put(HdfsSinkConnectorConfig.HDFS_URL_CONFIG, url); + localProps.put(StorageCommonConfig.STORE_URL_CONFIG, url); + props.putAll(localProps); + return props; + } + + //@Before public void setUp() throws Exception { - super.setUp(); - conf = new Configuration(); cluster = createDFSCluster(conf); cluster.waitActive(); - url = "hdfs://" + cluster.getNameNode().getClientNamenodeAddress(); fs = cluster.getFileSystem(); - Map props = createProps(); - connectorConfig = new HdfsSinkConnectorConfig(props); + super.setUp(); } @After @@ -89,13 +98,6 @@ private MiniDFSCluster createDFSCluster(Configuration conf) throws IOException { return cluster; } - @Override - protected Map createProps() { - Map props = super.createProps(); - props.put(HdfsSinkConnectorConfig.HDFS_URL_CONFIG, url); - return props; - } - /** * Return a list of new records starting at zero offset. * @@ -266,7 +268,7 @@ protected void verify(List sinkRecords, long[] validOffsets, Set records = schemaFileReader.readData(conf, path); + Collection records = schemaFileReader.readData(connectorConfig, path); long size = endOffset - startOffset + 1; assertEquals(size, records.size()); From 7acd95b8c5d4842bdd3aec5c5201ef40406842d7 Mon Sep 17 00:00:00 2001 From: Konstantine Karantasis Date: Mon, 24 Apr 2017 21:59:38 -0700 Subject: [PATCH 41/58] Test fixes in hive. --- .../confluent/connect/hdfs/hive/HiveExec.java | 3 ++- .../connect/hdfs/hive/HiveTestBase.java | 21 +++++++++---------- 2 files changed, 12 insertions(+), 12 deletions(-) diff --git a/src/test/java/io/confluent/connect/hdfs/hive/HiveExec.java b/src/test/java/io/confluent/connect/hdfs/hive/HiveExec.java index 6e955dc4c..0b7640608 100644 --- a/src/test/java/io/confluent/connect/hdfs/hive/HiveExec.java +++ b/src/test/java/io/confluent/connect/hdfs/hive/HiveExec.java @@ -30,6 +30,7 @@ import java.util.List; import io.confluent.connect.hdfs.HdfsSinkConnectorConfig; +import io.confluent.connect.storage.hive.HiveConfig; public class HiveExec { @@ -44,7 +45,7 @@ public class HiveExec { */ public HiveExec(HdfsSinkConnectorConfig config) { hiveConf = new HiveConf(); - String hiveConfDir = config.getString(HdfsSinkConnectorConfig.HIVE_CONF_DIR_CONFIG); + String hiveConfDir = config.getString(HiveConfig.HIVE_CONF_DIR_CONFIG); hiveConf.addResource(new Path(hiveConfDir, "hive-site.xml")); SessionState.start(new CliSessionState(hiveConf)); cliDriver = new CliDriver(); diff --git a/src/test/java/io/confluent/connect/hdfs/hive/HiveTestBase.java b/src/test/java/io/confluent/connect/hdfs/hive/HiveTestBase.java index 3dee2dd76..fc2b63f21 100644 --- a/src/test/java/io/confluent/connect/hdfs/hive/HiveTestBase.java +++ b/src/test/java/io/confluent/connect/hdfs/hive/HiveTestBase.java @@ -15,12 +15,11 @@ package io.confluent.connect.hdfs.hive; import org.junit.After; -import org.junit.Before; import java.util.Map; -import io.confluent.connect.hdfs.HdfsSinkConnectorConfig; import io.confluent.connect.hdfs.TestWithMiniDFSCluster; +import io.confluent.connect.storage.hive.HiveConfig; public class HiveTestBase extends TestWithMiniDFSCluster { @@ -28,10 +27,17 @@ public class HiveTestBase extends TestWithMiniDFSCluster { protected HiveMetaStore hiveMetaStore; protected HiveExec hiveExec; - @Before + @Override + protected Map createProps() { + Map props = super.createProps(); + props.put(HiveConfig.HIVE_CONF_DIR_CONFIG, "hive_conf"); + return props; + } + + //@Before should be omitted in order to be able to add properties per test. public void setUp() throws Exception { super.setUp(); - hiveDatabase = connectorConfig.getString(HdfsSinkConnectorConfig.HIVE_DATABASE_CONFIG); + hiveDatabase = connectorConfig.getString(HiveConfig.HIVE_DATABASE_CONFIG); hiveMetaStore = new HiveMetaStore(conf, connectorConfig); hiveExec = new HiveExec(connectorConfig); cleanHive(); @@ -43,13 +49,6 @@ public void tearDown() throws Exception { super.tearDown(); } - @Override - protected Map createProps() { - Map props = super.createProps(); - props.put(HdfsSinkConnectorConfig.HIVE_CONF_DIR_CONFIG, "hive_conf"); - return props; - } - private void cleanHive() { // ensures all tables are removed for (String database : hiveMetaStore.getAllDatabases()) { From 69326dd508fe5a1fa90fc66352685d5aafe582e1 Mon Sep 17 00:00:00 2001 From: Konstantine Karantasis Date: Mon, 24 Apr 2017 22:00:19 -0700 Subject: [PATCH 42/58] Test fixes in partitioner classes. --- .../hdfs/partitioner/DailyPartitionerTest.java | 18 ++++++------------ .../partitioner/HourlyPartitionerTest.java | 17 ++++++----------- .../partitioner/TimeBasedPartitionerTest.java | 11 +++++++---- 3 files changed, 19 insertions(+), 27 deletions(-) diff --git a/src/test/java/io/confluent/connect/hdfs/partitioner/DailyPartitionerTest.java b/src/test/java/io/confluent/connect/hdfs/partitioner/DailyPartitionerTest.java index 416de6d10..f47cb6081 100644 --- a/src/test/java/io/confluent/connect/hdfs/partitioner/DailyPartitionerTest.java +++ b/src/test/java/io/confluent/connect/hdfs/partitioner/DailyPartitionerTest.java @@ -23,32 +23,26 @@ import java.util.concurrent.TimeUnit; import io.confluent.connect.hdfs.HdfsSinkConnectorConfig; +import io.confluent.connect.hdfs.TestWithMiniDFSCluster; +import io.confluent.connect.storage.partitioner.PartitionerConfig; import static org.junit.Assert.assertEquals; -public class DailyPartitionerTest { - +public class DailyPartitionerTest extends TestWithMiniDFSCluster { private static final long partitionDurationMs = TimeUnit.HOURS.toMillis(24); @Test public void testDailyPartitioner() throws Exception { - Map config = createConfig(); - + setUp(); DailyPartitioner partitioner = new DailyPartitioner(); - partitioner.configure(config); + partitioner.configure(parsedConfig); String pathFormat = partitioner.getPathFormat(); - String timeZoneString = (String) config.get(HdfsSinkConnectorConfig.TIMEZONE_CONFIG); + String timeZoneString = (String) parsedConfig.get(PartitionerConfig.TIMEZONE_CONFIG); long timestamp = new DateTime(2014, 2, 1, 3, 0, 0, 0, DateTimeZone.forID(timeZoneString)).getMillis(); String encodedPartition = TimeUtils.encodeTimestamp(partitionDurationMs, pathFormat, timeZoneString, timestamp); String path = partitioner.generatePartitionedPath("topic", encodedPartition); assertEquals("topic/year=2014/month=02/day=01/", path); } - private Map createConfig() { - Map config = new HashMap<>(); - config.put(HdfsSinkConnectorConfig.LOCALE_CONFIG, "en"); - config.put(HdfsSinkConnectorConfig.TIMEZONE_CONFIG, "America/Los_Angeles"); - return config; - } } diff --git a/src/test/java/io/confluent/connect/hdfs/partitioner/HourlyPartitionerTest.java b/src/test/java/io/confluent/connect/hdfs/partitioner/HourlyPartitionerTest.java index bfadc1707..1606bddc9 100644 --- a/src/test/java/io/confluent/connect/hdfs/partitioner/HourlyPartitionerTest.java +++ b/src/test/java/io/confluent/connect/hdfs/partitioner/HourlyPartitionerTest.java @@ -23,22 +23,23 @@ import java.util.concurrent.TimeUnit; import io.confluent.connect.hdfs.HdfsSinkConnectorConfig; +import io.confluent.connect.hdfs.TestWithMiniDFSCluster; +import io.confluent.connect.storage.partitioner.PartitionerConfig; import static org.junit.Assert.assertEquals; -public class HourlyPartitionerTest { +public class HourlyPartitionerTest extends TestWithMiniDFSCluster { private static final long partitionDurationMs = TimeUnit.HOURS.toMillis(1); @Test public void testHourlyPartitioner() throws Exception { - Map config = createConfig(); - + setUp(); HourlyPartitioner partitioner = new HourlyPartitioner(); - partitioner.configure(config); + partitioner.configure(parsedConfig); String pathFormat = partitioner.getPathFormat(); - String timeZoneString = (String) config.get(HdfsSinkConnectorConfig.TIMEZONE_CONFIG); + String timeZoneString = (String) parsedConfig.get(PartitionerConfig.TIMEZONE_CONFIG); long timestamp = new DateTime(2015, 2, 1, 3, 0, 0, 0, DateTimeZone.forID(timeZoneString)).getMillis(); String encodedPartition = TimeUtils.encodeTimestamp(partitionDurationMs, pathFormat, timeZoneString, timestamp); @@ -46,10 +47,4 @@ public void testHourlyPartitioner() throws Exception { assertEquals("topic/year=2015/month=02/day=01/hour=03/", path); } - private Map createConfig() { - Map config = new HashMap<>(); - config.put(HdfsSinkConnectorConfig.LOCALE_CONFIG, "en"); - config.put(HdfsSinkConnectorConfig.TIMEZONE_CONFIG, "America/Los_Angeles"); - return config; - } } diff --git a/src/test/java/io/confluent/connect/hdfs/partitioner/TimeBasedPartitionerTest.java b/src/test/java/io/confluent/connect/hdfs/partitioner/TimeBasedPartitionerTest.java index 4e4f3fbdb..491b7311c 100644 --- a/src/test/java/io/confluent/connect/hdfs/partitioner/TimeBasedPartitionerTest.java +++ b/src/test/java/io/confluent/connect/hdfs/partitioner/TimeBasedPartitionerTest.java @@ -24,16 +24,19 @@ import java.util.Map; import java.util.concurrent.TimeUnit; +import io.confluent.connect.hdfs.HdfsSinkConnectorTestBase; + import static org.junit.Assert.assertEquals; -public class TimeBasedPartitionerTest { +public class TimeBasedPartitionerTest extends HdfsSinkConnectorTestBase { private static final String timeZoneString = "America/Los_Angeles"; private static final DateTimeZone DATE_TIME_ZONE = DateTimeZone.forID(timeZoneString); - private BiHourlyPartitioner partitioner = new BiHourlyPartitioner(); @Test public void testGeneratePartitionedPath() throws Exception { - partitioner.configure(null); + setUp(); + BiHourlyPartitioner partitioner = new BiHourlyPartitioner(); + partitioner.configure(parsedConfig); String pathFormat = partitioner.getPathFormat(); long partitionDurationMs = TimeUnit.HOURS.toMillis(2); long timestamp = new DateTime(2015, 1, 1, 3, 0, 0, 0, DateTimeZone.forID(timeZoneString)).getMillis(); @@ -60,7 +63,7 @@ private static class BiHourlyPartitioner extends TimeBasedPartitioner { @Override public void configure(Map config) { - init(partitionDurationMs, pathFormat, Locale.FRENCH, DATE_TIME_ZONE, true); + init(partitionDurationMs, pathFormat, Locale.FRENCH, DATE_TIME_ZONE, config); } public String getPathFormat() { From 3202f521c89efd72907f44f620740ed07eaba27a Mon Sep 17 00:00:00 2001 From: Konstantine Karantasis Date: Mon, 24 Apr 2017 22:00:37 -0700 Subject: [PATCH 43/58] Test fixes in formats. --- .../connect/hdfs/avro/AvroHiveUtilTest.java | 15 ++- .../connect/hdfs/avro/DataWriterAvroTest.java | 2 +- .../hdfs/avro/HiveIntegrationAvroTest.java | 71 +++++++----- .../hdfs/avro/TopicPartitionWriterTest.java | 105 ++++++++++++------ .../parquet/HiveIntegrationParquetTest.java | 55 ++++----- .../hdfs/parquet/ParquetHiveUtilTest.java | 14 +-- 6 files changed, 159 insertions(+), 103 deletions(-) diff --git a/src/test/java/io/confluent/connect/hdfs/avro/AvroHiveUtilTest.java b/src/test/java/io/confluent/connect/hdfs/avro/AvroHiveUtilTest.java index d0ecaf321..42f70ddc0 100644 --- a/src/test/java/io/confluent/connect/hdfs/avro/AvroHiveUtilTest.java +++ b/src/test/java/io/confluent/connect/hdfs/avro/AvroHiveUtilTest.java @@ -26,7 +26,9 @@ import org.junit.Test; import java.util.ArrayList; +import java.util.HashMap; import java.util.List; +import java.util.Map; import io.confluent.connect.avro.AvroData; import io.confluent.connect.hdfs.DataWriter; @@ -38,10 +40,17 @@ import static org.junit.Assert.assertEquals; public class AvroHiveUtilTest extends HiveTestBase { - private HiveUtil hive; + private Map localProps = new HashMap<>(); + + @Override + protected Map createProps() { + Map props = super.createProps(); + props.putAll(localProps); + return props; + } - @Before + //@Before should be omitted in order to be able to add properties per test. public void setUp() throws Exception { super.setUp(); hive = new AvroHiveUtil(connectorConfig, avroData, hiveMetaStore); @@ -49,6 +58,7 @@ public void setUp() throws Exception { @Test public void testCreateTable() throws Exception { + setUp(); prepareData(TOPIC, PARTITION); Partitioner partitioner = HiveTestUtils.getPartitioner(); @@ -91,6 +101,7 @@ public void testCreateTable() throws Exception { @Test public void testAlterSchema() throws Exception { + setUp(); prepareData(TOPIC, PARTITION); Partitioner partitioner = HiveTestUtils.getPartitioner(); Schema schema = createSchema(); diff --git a/src/test/java/io/confluent/connect/hdfs/avro/DataWriterAvroTest.java b/src/test/java/io/confluent/connect/hdfs/avro/DataWriterAvroTest.java index 7a419e934..4c04f7b39 100644 --- a/src/test/java/io/confluent/connect/hdfs/avro/DataWriterAvroTest.java +++ b/src/test/java/io/confluent/connect/hdfs/avro/DataWriterAvroTest.java @@ -70,7 +70,7 @@ public void testWriteRecord() throws Exception { public void testRecovery() throws Exception { fs.delete(new Path(FileUtils.directoryName(url, topicsDir, TOPIC_PARTITION)), true); - HdfsStorage storage = new HdfsStorage(conf, url); + HdfsStorage storage = new HdfsStorage(connectorConfig, url); DataWriter hdfsWriter = new DataWriter(connectorConfig, context, avroData); partitioner = hdfsWriter.getPartitioner(); diff --git a/src/test/java/io/confluent/connect/hdfs/avro/HiveIntegrationAvroTest.java b/src/test/java/io/confluent/connect/hdfs/avro/HiveIntegrationAvroTest.java index 988835144..8ac238085 100644 --- a/src/test/java/io/confluent/connect/hdfs/avro/HiveIntegrationAvroTest.java +++ b/src/test/java/io/confluent/connect/hdfs/avro/HiveIntegrationAvroTest.java @@ -26,6 +26,7 @@ import java.util.ArrayList; import java.util.Collection; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.concurrent.TimeUnit; @@ -38,20 +39,30 @@ import io.confluent.connect.hdfs.partitioner.DailyPartitioner; import io.confluent.connect.hdfs.partitioner.FieldPartitioner; import io.confluent.connect.hdfs.partitioner.TimeUtils; +import io.confluent.connect.storage.hive.HiveConfig; +import io.confluent.connect.storage.partitioner.PartitionerConfig; import static org.junit.Assert.assertEquals; public class HiveIntegrationAvroTest extends HiveTestBase { + private Map localProps = new HashMap<>(); @Override protected Map createProps() { Map props = super.createProps(); props.put(HdfsSinkConnectorConfig.SHUTDOWN_TIMEOUT_CONFIG, "10000"); + props.putAll(localProps); return props; } + //@Before should be omitted in order to be able to add properties per test. + public void setUp() throws Exception { + super.setUp(); + } + @Test public void testSyncWithHiveAvro() throws Exception { + setUp(); DataWriter hdfsWriter = new DataWriter(connectorConfig, context, avroData); hdfsWriter.recover(TOPIC_PARTITION); @@ -71,7 +82,7 @@ public void testSyncWithHiveAvro() throws Exception { hdfsWriter.stop(); Map props = createProps(); - props.put(HdfsSinkConnectorConfig.HIVE_INTEGRATION_CONFIG, "true"); + props.put(HiveConfig.HIVE_INTEGRATION_CONFIG, "true"); HdfsSinkConnectorConfig config = new HdfsSinkConnectorConfig(props); hdfsWriter = new DataWriter(config, context, avroData); @@ -103,11 +114,9 @@ public void testSyncWithHiveAvro() throws Exception { @Test public void testHiveIntegrationAvro() throws Exception { - Map props = createProps(); - props.put(HdfsSinkConnectorConfig.HIVE_INTEGRATION_CONFIG, "true"); - HdfsSinkConnectorConfig config = new HdfsSinkConnectorConfig(props); - - DataWriter hdfsWriter = new DataWriter(config, context, avroData); + localProps.put(HiveConfig.HIVE_INTEGRATION_CONFIG, "true"); + setUp(); + DataWriter hdfsWriter = new DataWriter(connectorConfig, context, avroData); hdfsWriter.recover(TOPIC_PARTITION); String key = "key"; @@ -149,13 +158,11 @@ public void testHiveIntegrationAvro() throws Exception { @Test public void testHiveIntegrationTopicWithDotsAvro() throws Exception { - assignment.add(TOPIC_WITH_DOTS_PARTITION); + localProps.put(HiveConfig.HIVE_INTEGRATION_CONFIG, "true"); + setUp(); + context.assignment().add(TOPIC_WITH_DOTS_PARTITION); - Map props = createProps(); - props.put(HdfsSinkConnectorConfig.HIVE_INTEGRATION_CONFIG, "true"); - HdfsSinkConnectorConfig config = new HdfsSinkConnectorConfig(props); - - DataWriter hdfsWriter = new DataWriter(config, context, avroData); + DataWriter hdfsWriter = new DataWriter(connectorConfig, context, avroData); hdfsWriter.recover(TOPIC_WITH_DOTS_PARTITION); String key = "key"; @@ -171,7 +178,7 @@ public void testHiveIntegrationTopicWithDotsAvro() throws Exception { } hdfsWriter.write(sinkRecords); - hdfsWriter.close(assignment); + hdfsWriter.close(); hdfsWriter.stop(); Table table = hiveMetaStore.getTable(hiveDatabase, TOPIC_WITH_DOTS); @@ -197,13 +204,12 @@ public void testHiveIntegrationTopicWithDotsAvro() throws Exception { @Test public void testHiveIntegrationFieldPartitionerAvro() throws Exception { - Map props = createProps(); - props.put(HdfsSinkConnectorConfig.HIVE_INTEGRATION_CONFIG, "true"); - props.put(HdfsSinkConnectorConfig.PARTITIONER_CLASS_CONFIG, FieldPartitioner.class.getName()); - props.put(HdfsSinkConnectorConfig.PARTITION_FIELD_NAME_CONFIG, "int"); + localProps.put(HiveConfig.HIVE_INTEGRATION_CONFIG, "true"); + localProps.put(PartitionerConfig.PARTITIONER_CLASS_CONFIG, FieldPartitioner.class.getName()); + localProps.put(PartitionerConfig.PARTITION_FIELD_NAME_CONFIG, "int"); + setUp(); - HdfsSinkConnectorConfig config = new HdfsSinkConnectorConfig(props); - DataWriter hdfsWriter = new DataWriter(config, context, avroData); + DataWriter hdfsWriter = new DataWriter(connectorConfig, context, avroData); String key = "key"; Schema schema = createSchema(); @@ -238,7 +244,9 @@ public void testHiveIntegrationFieldPartitionerAvro() throws Exception { assertEquals(expectedColumnNames, actualColumnNames); - String partitionFieldName = config.getString(HdfsSinkConnectorConfig.PARTITION_FIELD_NAME_CONFIG); + String partitionFieldName = connectorConfig.getString( + PartitionerConfig.PARTITION_FIELD_NAME_CONFIG + ); String directory1 = TOPIC + "/" + partitionFieldName + "=" + String.valueOf(16); String directory2 = TOPIC + "/" + partitionFieldName + "=" + String.valueOf(17); String directory3 = TOPIC + "/" + partitionFieldName + "=" + String.valueOf(18); @@ -272,14 +280,10 @@ public void testHiveIntegrationFieldPartitionerAvro() throws Exception { @Test public void testHiveIntegrationTimeBasedPartitionerAvro() throws Exception { - Map props = createProps(); - props.put(HdfsSinkConnectorConfig.HIVE_INTEGRATION_CONFIG, "true"); - props.put(HdfsSinkConnectorConfig.PARTITIONER_CLASS_CONFIG, DailyPartitioner.class.getName()); - props.put(HdfsSinkConnectorConfig.TIMEZONE_CONFIG, "America/Los_Angeles"); - props.put(HdfsSinkConnectorConfig.LOCALE_CONFIG, "en"); - - HdfsSinkConnectorConfig config = new HdfsSinkConnectorConfig(props); - DataWriter hdfsWriter = new DataWriter(config, context, avroData); + localProps.put(HiveConfig.HIVE_INTEGRATION_CONFIG, "true"); + localProps.put(PartitionerConfig.PARTITIONER_CLASS_CONFIG, DailyPartitioner.class.getName()); + setUp(); + DataWriter hdfsWriter = new DataWriter(connectorConfig, context, avroData); String key = "key"; Schema schema = createSchema(); @@ -289,8 +293,15 @@ public void testHiveIntegrationTimeBasedPartitionerAvro() throws Exception { long offset = 0; for (Struct record : records) { for (long count = 0; count < 3; count++) { - SinkRecord sinkRecord = new SinkRecord(TOPIC, PARTITION, Schema.STRING_SCHEMA, key, schema, record, - offset + count); + SinkRecord sinkRecord = new SinkRecord( + TOPIC, + PARTITION, + Schema.STRING_SCHEMA, + key, + schema, + record, + offset + count + ); sinkRecords.add(sinkRecord); } offset = offset + 3; diff --git a/src/test/java/io/confluent/connect/hdfs/avro/TopicPartitionWriterTest.java b/src/test/java/io/confluent/connect/hdfs/avro/TopicPartitionWriterTest.java index 0bd773202..a3e80a99b 100644 --- a/src/test/java/io/confluent/connect/hdfs/avro/TopicPartitionWriterTest.java +++ b/src/test/java/io/confluent/connect/hdfs/avro/TopicPartitionWriterTest.java @@ -14,13 +14,11 @@ package io.confluent.connect.hdfs.avro; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.data.Struct; import org.apache.kafka.connect.sink.SinkRecord; -import org.junit.Before; import org.junit.Test; import java.io.IOException; @@ -32,7 +30,6 @@ import java.util.List; import java.util.Map; import java.util.Set; -import java.util.concurrent.TimeUnit; import io.confluent.connect.hdfs.FileUtils; import io.confluent.connect.hdfs.Format; @@ -46,41 +43,67 @@ import io.confluent.connect.hdfs.partitioner.Partitioner; import io.confluent.connect.hdfs.partitioner.TimeBasedPartitioner; import io.confluent.connect.hdfs.partitioner.TimeUtils; -import io.confluent.connect.hdfs.storage.Storage; +import io.confluent.connect.hdfs.storage.HdfsStorage; import io.confluent.connect.storage.StorageFactory; +import io.confluent.connect.storage.common.StorageCommonConfig; +import io.confluent.connect.storage.hive.schema.TimeBasedSchemaGenerator; +import io.confluent.connect.storage.partitioner.PartitionerConfig; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; public class TopicPartitionWriterTest extends TestWithMiniDFSCluster { private RecordWriterProvider writerProvider; - private Storage storage; + private HdfsStorage storage; + private Map localProps = new HashMap<>(); + + @Override + protected Map createProps() { + Map props = super.createProps(); + props.putAll(localProps); + return props; + } - @Before + //@Before should be omitted in order to be able to add properties per test. public void setUp() throws Exception { super.setUp(); @SuppressWarnings("unchecked") - Format format = ((Class) Class.forName(connectorConfig.getString(HdfsSinkConnectorConfig.FORMAT_CLASS_CONFIG))).newInstance(); + Class storageClass = (Class) + connectorConfig.getClass(StorageCommonConfig.STORAGE_CLASS_CONFIG); + storage = StorageFactory.createStorage( + storageClass, + HdfsSinkConnectorConfig.class, + connectorConfig, + url + ); + @SuppressWarnings("unchecked") + Class formatClass = (Class) connectorConfig.getClass( + HdfsSinkConnectorConfig.FORMAT_CLASS_CONFIG + ); + Format format = formatClass.getConstructor(HdfsStorage.class).newInstance(storage); writerProvider = format.getRecordWriterProvider(); schemaFileReader = format.getSchemaFileReader(avroData); extension = writerProvider.getExtension(); - @SuppressWarnings("unchecked") - Class storageClass = (Class) Class - .forName(connectorConfig.getString(HdfsSinkConnectorConfig.STORAGE_CLASS_CONFIG)); - storage = StorageFactory.createStorage(storageClass, Configuration.class, conf, url); createTopicDir(url, topicsDir, TOPIC); createLogsDir(url, logsDir); } @Test public void testWriteRecordDefaultWithPadding() throws Exception { + localProps.put(HdfsSinkConnectorConfig.FILENAME_OFFSET_ZERO_PAD_WIDTH_CONFIG, "2"); + setUp(); Partitioner partitioner = new DefaultPartitioner(); - partitioner.configure(Collections.emptyMap()); - properties.put(HdfsSinkConnectorConfig.FILENAME_OFFSET_ZERO_PAD_WIDTH_CONFIG, "2"); - configureConnector(); + partitioner.configure(parsedConfig); TopicPartitionWriter topicPartitionWriter = new TopicPartitionWriter( - TOPIC_PARTITION, storage, writerProvider, partitioner, connectorConfig, context, avroData); + TOPIC_PARTITION, + storage, + writerProvider, + partitioner, + connectorConfig, + context, + avroData + ); Schema schema = createSchema(); List records = createRecordBatches(schema, 3, 3); @@ -107,17 +130,25 @@ public void testWriteRecordDefaultWithPadding() throws Exception { verify(expectedFiles, expectedBatchSize, records, schema); } - @Test public void testWriteRecordFieldPartitioner() throws Exception { - Map config = createConfig(); + setUp(); Partitioner partitioner = new FieldPartitioner(); - partitioner.configure(config); + partitioner.configure(parsedConfig); - String partitionField = (String) config.get(HdfsSinkConnectorConfig.PARTITION_FIELD_NAME_CONFIG); + String partitionField = (String) parsedConfig.get( + PartitionerConfig.PARTITION_FIELD_NAME_CONFIG + ); TopicPartitionWriter topicPartitionWriter = new TopicPartitionWriter( - TOPIC_PARTITION, storage, writerProvider, partitioner, connectorConfig, context, avroData); + TOPIC_PARTITION, + storage, + writerProvider, + partitioner, + connectorConfig, + context, + avroData + ); Schema schema = createSchema(); List records = new ArrayList<>(); @@ -154,12 +185,20 @@ public void testWriteRecordFieldPartitioner() throws Exception { @Test public void testWriteRecordTimeBasedPartition() throws Exception { - Map config = createConfig(); + setUp(); Partitioner partitioner = new TimeBasedPartitioner(); - partitioner.configure(config); + parsedConfig.put(PartitionerConfig.SCHEMA_GENERATOR_CLASS_CONFIG, TimeBasedSchemaGenerator.class); + partitioner.configure(parsedConfig); TopicPartitionWriter topicPartitionWriter = new TopicPartitionWriter( - TOPIC_PARTITION, storage, writerProvider, partitioner, connectorConfig, context, avroData); + TOPIC_PARTITION, + storage, + writerProvider, + partitioner, + connectorConfig, + context, + avroData + ); Schema schema = createSchema(); List records = createRecordBatches(schema, 3, 3); @@ -175,9 +214,11 @@ public void testWriteRecordTimeBasedPartition() throws Exception { topicPartitionWriter.write(); topicPartitionWriter.close(); - long partitionDurationMs = (Long) config.get(HdfsSinkConnectorConfig.PARTITION_DURATION_MS_CONFIG); - String pathFormat = (String) config.get(HdfsSinkConnectorConfig.PATH_FORMAT_CONFIG); - String timeZoneString = (String) config.get(HdfsSinkConnectorConfig.TIMEZONE_CONFIG); + long partitionDurationMs = (Long) parsedConfig.get( + PartitionerConfig.PARTITION_DURATION_MS_CONFIG + ); + String pathFormat = (String) parsedConfig.get(PartitionerConfig.PATH_FORMAT_CONFIG); + String timeZoneString = (String) parsedConfig.get(PartitionerConfig.TIMEZONE_CONFIG); long timestamp = System.currentTimeMillis(); String encodedPartition = TimeUtils.encodeTimestamp(partitionDurationMs, pathFormat, timeZoneString, timestamp); @@ -193,16 +234,6 @@ public void testWriteRecordTimeBasedPartition() throws Exception { verify(expectedFiles, expectedBatchSize, records, schema); } - private Map createConfig() { - Map config = new HashMap<>(); - config.put(HdfsSinkConnectorConfig.PARTITION_FIELD_NAME_CONFIG, "int"); - config.put(HdfsSinkConnectorConfig.PARTITION_DURATION_MS_CONFIG, TimeUnit.HOURS.toMillis(1)); - config.put(HdfsSinkConnectorConfig.PATH_FORMAT_CONFIG, "'year'=YYYY/'month'=MM/'day'=dd/'hour'=HH/"); - config.put(HdfsSinkConnectorConfig.LOCALE_CONFIG, "en"); - config.put(HdfsSinkConnectorConfig.TIMEZONE_CONFIG, "America/Los_Angeles"); - return config; - } - private void createTopicDir(String url, String topicsDir, String topic) throws IOException { Path path = new Path(FileUtils.topicDirectory(url, topicsDir, topic)); if (!fs.exists(path)) { @@ -225,7 +256,7 @@ private void verify(Set expectedFiles, int expectedSize, List reco for (FileStatus status : statuses) { Path filePath = status.getPath(); assertTrue(expectedFiles.contains(status.getPath())); - Collection avroRecords = schemaFileReader.readData(conf, filePath); + Collection avroRecords = schemaFileReader.readData(connectorConfig, filePath); assertEquals(expectedSize, avroRecords.size()); for (Object avroRecord : avroRecords) { assertEquals(avroData.fromConnectData(schema, records.get(index++)), avroRecord); diff --git a/src/test/java/io/confluent/connect/hdfs/parquet/HiveIntegrationParquetTest.java b/src/test/java/io/confluent/connect/hdfs/parquet/HiveIntegrationParquetTest.java index 77a0fd650..073ba721e 100644 --- a/src/test/java/io/confluent/connect/hdfs/parquet/HiveIntegrationParquetTest.java +++ b/src/test/java/io/confluent/connect/hdfs/parquet/HiveIntegrationParquetTest.java @@ -26,6 +26,7 @@ import java.util.ArrayList; import java.util.Arrays; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.concurrent.TimeUnit; @@ -38,23 +39,31 @@ import io.confluent.connect.hdfs.partitioner.DailyPartitioner; import io.confluent.connect.hdfs.partitioner.FieldPartitioner; import io.confluent.connect.hdfs.partitioner.TimeUtils; +import io.confluent.connect.storage.hive.HiveConfig; +import io.confluent.connect.storage.partitioner.PartitionerConfig; import static org.junit.Assert.assertEquals; public class HiveIntegrationParquetTest extends HiveTestBase { + private Map localProps = new HashMap<>(); + @Override protected Map createProps() { Map props = super.createProps(); props.put(HdfsSinkConnectorConfig.SHUTDOWN_TIMEOUT_CONFIG, "10000"); props.put(HdfsSinkConnectorConfig.FORMAT_CLASS_CONFIG, ParquetFormat.class.getName()); + props.putAll(localProps); return props; } + //@Before should be omitted in order to be able to add properties per test. + public void setUp() throws Exception { + super.setUp(); + } + @Test public void testSyncWithHiveParquet() throws Exception { - Map props = createProps(); - HdfsSinkConnectorConfig connectorConfig = new HdfsSinkConnectorConfig(props); - + setUp(); DataWriter hdfsWriter = new DataWriter(connectorConfig, context, avroData); hdfsWriter.recover(TOPIC_PARTITION); @@ -64,9 +73,8 @@ public void testSyncWithHiveParquet() throws Exception { hdfsWriter.close(); hdfsWriter.stop(); - props = createProps(); - props.put(HdfsSinkConnectorConfig.HIVE_INTEGRATION_CONFIG, "true"); - HdfsSinkConnectorConfig config = new HdfsSinkConnectorConfig(props); + localProps.put(HiveConfig.HIVE_INTEGRATION_CONFIG, "true"); + HdfsSinkConnectorConfig config = new HdfsSinkConnectorConfig(createProps()); hdfsWriter = new DataWriter(config, context, avroData); hdfsWriter.syncWithHive(); @@ -101,11 +109,10 @@ public void testSyncWithHiveParquet() throws Exception { @Test public void testHiveIntegrationParquet() throws Exception { - Map props = createProps(); - props.put(HdfsSinkConnectorConfig.HIVE_INTEGRATION_CONFIG, "true"); - HdfsSinkConnectorConfig config = new HdfsSinkConnectorConfig(props); + localProps.put(HiveConfig.HIVE_INTEGRATION_CONFIG, "true"); + setUp(); - DataWriter hdfsWriter = new DataWriter(config, context, avroData); + DataWriter hdfsWriter = new DataWriter(connectorConfig, context, avroData); hdfsWriter.recover(TOPIC_PARTITION); List sinkRecords = createSinkRecords(7); @@ -138,13 +145,11 @@ public void testHiveIntegrationParquet() throws Exception { @Test public void testHiveIntegrationFieldPartitionerParquet() throws Exception { - Map props = createProps(); - props.put(HdfsSinkConnectorConfig.HIVE_INTEGRATION_CONFIG, "true"); - props.put(HdfsSinkConnectorConfig.PARTITIONER_CLASS_CONFIG, FieldPartitioner.class.getName()); - props.put(HdfsSinkConnectorConfig.PARTITION_FIELD_NAME_CONFIG, "int"); - - HdfsSinkConnectorConfig config = new HdfsSinkConnectorConfig(props); - DataWriter hdfsWriter = new DataWriter(config, context, avroData); + localProps.put(HiveConfig.HIVE_INTEGRATION_CONFIG, "true"); + localProps.put(PartitionerConfig.PARTITIONER_CLASS_CONFIG, FieldPartitioner.class.getName()); + localProps.put(PartitionerConfig.PARTITION_FIELD_NAME_CONFIG, "int"); + setUp(); + DataWriter hdfsWriter = new DataWriter(connectorConfig, context, avroData); Schema schema = createSchema(); List records = createRecordBatches(schema, 3, 3); @@ -167,7 +172,9 @@ public void testHiveIntegrationFieldPartitionerParquet() throws Exception { } assertEquals(expectedColumnNames, actualColumnNames); - String partitionFieldName = config.getString(HdfsSinkConnectorConfig.PARTITION_FIELD_NAME_CONFIG); + String partitionFieldName = connectorConfig.getString( + PartitionerConfig.PARTITION_FIELD_NAME_CONFIG + ); String directory1 = TOPIC + "/" + partitionFieldName + "=" + String.valueOf(16); String directory2 = TOPIC + "/" + partitionFieldName + "=" + String.valueOf(17); String directory3 = TOPIC + "/" + partitionFieldName + "=" + String.valueOf(18); @@ -206,14 +213,10 @@ public void testHiveIntegrationFieldPartitionerParquet() throws Exception { @Test public void testHiveIntegrationTimeBasedPartitionerParquet() throws Exception { - Map props = createProps(); - props.put(HdfsSinkConnectorConfig.HIVE_INTEGRATION_CONFIG, "true"); - props.put(HdfsSinkConnectorConfig.PARTITIONER_CLASS_CONFIG, DailyPartitioner.class.getName()); - props.put(HdfsSinkConnectorConfig.TIMEZONE_CONFIG, "America/Los_Angeles"); - props.put(HdfsSinkConnectorConfig.LOCALE_CONFIG, "en"); - - HdfsSinkConnectorConfig config = new HdfsSinkConnectorConfig(props); - DataWriter hdfsWriter = new DataWriter(config, context, avroData); + localProps.put(HiveConfig.HIVE_INTEGRATION_CONFIG, "true"); + localProps.put(PartitionerConfig.PARTITIONER_CLASS_CONFIG, DailyPartitioner.class.getName()); + setUp(); + DataWriter hdfsWriter = new DataWriter(connectorConfig, context, avroData); Schema schema = createSchema(); List records = createRecordBatches(schema, 3, 3); diff --git a/src/test/java/io/confluent/connect/hdfs/parquet/ParquetHiveUtilTest.java b/src/test/java/io/confluent/connect/hdfs/parquet/ParquetHiveUtilTest.java index 98a047102..56b7f8a8b 100644 --- a/src/test/java/io/confluent/connect/hdfs/parquet/ParquetHiveUtilTest.java +++ b/src/test/java/io/confluent/connect/hdfs/parquet/ParquetHiveUtilTest.java @@ -27,6 +27,7 @@ import java.util.ArrayList; import java.util.Collection; +import java.util.HashMap; import java.util.List; import java.util.Map; @@ -41,26 +42,26 @@ import static org.junit.Assert.assertEquals; public class ParquetHiveUtilTest extends HiveTestBase { - private HiveUtil hive; + private Map localProps = new HashMap<>(); @Override protected Map createProps() { Map props = super.createProps(); props.put(HdfsSinkConnectorConfig.FORMAT_CLASS_CONFIG, ParquetFormat.class.getName()); + props.putAll(localProps); return props; } - @Before + //@Before should be omitted in order to be able to add properties per test. public void setUp() throws Exception { super.setUp(); - Map props = createProps(); - HdfsSinkConnectorConfig connectorConfig = new HdfsSinkConnectorConfig(props); - hive = new ParquetHiveUtil(connectorConfig, avroData, hiveMetaStore); + hive = new ParquetHiveUtil(connectorConfig, hiveMetaStore); } @Test public void testCreateTable() throws Exception { + setUp(); prepareData(TOPIC, PARTITION); Partitioner partitioner = HiveTestUtils.getPartitioner(); @@ -103,6 +104,7 @@ public void testCreateTable() throws Exception { @Test public void testAlterSchema() throws Exception { + setUp(); prepareData(TOPIC, PARTITION); Partitioner partitioner = HiveTestUtils.getPartitioner(); Schema schema = createSchema(); @@ -163,8 +165,6 @@ private void prepareData(String topic, int partition) { } private DataWriter createWriter(SinkTaskContext context, AvroData avroData) { - Map props = createProps(); - HdfsSinkConnectorConfig connectorConfig = new HdfsSinkConnectorConfig(props); return new DataWriter(connectorConfig, context, avroData); } } From 24f9cb263b357f9b84e440096f93d7a681733cde Mon Sep 17 00:00:00 2001 From: Konstantine Karantasis Date: Tue, 25 Apr 2017 17:23:47 -0700 Subject: [PATCH 44/58] Fixes in main test classes. --- .../io/confluent/connect/hdfs/HdfsSinkConnectorTestBase.java | 4 ++-- .../io/confluent/connect/hdfs/TestWithMiniDFSCluster.java | 3 ++- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/src/test/java/io/confluent/connect/hdfs/HdfsSinkConnectorTestBase.java b/src/test/java/io/confluent/connect/hdfs/HdfsSinkConnectorTestBase.java index 19dd8f6d0..346d54821 100644 --- a/src/test/java/io/confluent/connect/hdfs/HdfsSinkConnectorTestBase.java +++ b/src/test/java/io/confluent/connect/hdfs/HdfsSinkConnectorTestBase.java @@ -49,13 +49,13 @@ public class HdfsSinkConnectorTestBase extends StorageSinkTestBase { @Override protected Map createProps() { - url = "memory://"; Map props = super.createProps(); + url = "memory://"; props.put(HdfsSinkConnectorConfig.HDFS_URL_CONFIG, url); props.put(HdfsSinkConnectorConfig.FLUSH_SIZE_CONFIG, "3"); props.put( StorageCommonConfig.STORAGE_CLASS_CONFIG, - "io.confluent.connect.s3.storage.S3Storage" + "io.confluent.connect.hdfs.storage.HdfsStorage" ); props.put(HdfsSinkConnectorConfig.FORMAT_CLASS_CONFIG, AvroFormat.class.getName()); props.put( diff --git a/src/test/java/io/confluent/connect/hdfs/TestWithMiniDFSCluster.java b/src/test/java/io/confluent/connect/hdfs/TestWithMiniDFSCluster.java index 9cf76f1a4..4e17c8ce7 100644 --- a/src/test/java/io/confluent/connect/hdfs/TestWithMiniDFSCluster.java +++ b/src/test/java/io/confluent/connect/hdfs/TestWithMiniDFSCluster.java @@ -71,7 +71,8 @@ protected Map createProps() { //@Before public void setUp() throws Exception { - cluster = createDFSCluster(conf); + Configuration localConf = new Configuration(); + cluster = createDFSCluster(localConf); cluster.waitActive(); fs = cluster.getFileSystem(); super.setUp(); From 62910454618e8139f31aaa6871ea8e9cc95b93fa Mon Sep 17 00:00:00 2001 From: Konstantine Karantasis Date: Tue, 25 Apr 2017 18:13:39 -0700 Subject: [PATCH 45/58] Fix avro writer try-catch. --- .../hdfs/avro/AvroRecordWriterProvider.java | 24 +++++++++---------- 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/src/main/java/io/confluent/connect/hdfs/avro/AvroRecordWriterProvider.java b/src/main/java/io/confluent/connect/hdfs/avro/AvroRecordWriterProvider.java index 9e20509c4..5c6e97b3e 100644 --- a/src/main/java/io/confluent/connect/hdfs/avro/AvroRecordWriterProvider.java +++ b/src/main/java/io/confluent/connect/hdfs/avro/AvroRecordWriterProvider.java @@ -81,19 +81,19 @@ public void write(SinkRecord record) { } catch (IOException e) { throw new ConnectException(e); } + } - log.trace("Sink record: {}", record.toString()); - Object value = avroData.fromConnectData(schema, record.value()); - try { - // 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); - } catch (IOException e) { - throw new DataException(e); - } + log.trace("Sink record: {}", record.toString()); + Object value = avroData.fromConnectData(schema, record.value()); + try { + // 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); + } catch (IOException e) { + throw new DataException(e); } } From 0beb584e7423fabed3f294e5022293c03135cc67 Mon Sep 17 00:00:00 2001 From: Konstantine Karantasis Date: Tue, 25 Apr 2017 19:35:49 -0700 Subject: [PATCH 46/58] Fix reflection calls. --- .../io/confluent/connect/hdfs/DataWriter.java | 28 +++++++++++++++---- 1 file changed, 23 insertions(+), 5 deletions(-) diff --git a/src/main/java/io/confluent/connect/hdfs/DataWriter.java b/src/main/java/io/confluent/connect/hdfs/DataWriter.java index 9157a22e7..8e965e327 100644 --- a/src/main/java/io/confluent/connect/hdfs/DataWriter.java +++ b/src/main/java/io/confluent/connect/hdfs/DataWriter.java @@ -29,6 +29,7 @@ import org.slf4j.LoggerFactory; import java.io.IOException; +import java.lang.reflect.InvocationTargetException; import java.net.InetAddress; import java.util.Collection; import java.util.HashMap; @@ -55,6 +56,7 @@ import io.confluent.connect.hdfs.storage.Storage; import io.confluent.connect.storage.common.StorageCommonConfig; import io.confluent.connect.storage.hive.HiveConfig; +import io.confluent.connect.hdfs.partitioner.DefaultPartitioner; import io.confluent.connect.storage.partitioner.PartitionerConfig; public class DataWriter { @@ -178,11 +180,11 @@ public void run() { createDir(topicsDir + HdfsSinkConnectorConstants.TEMPFILE_DIRECTORY); createDir(logsDir); - format = getFormat(); + format = newFormat(); writerProvider = format.getRecordWriterProvider(); schemaFileReader = format.getSchemaFileReader(); - partitioner = createPartitioner(connectorConfig); + partitioner = newPartitioner(connectorConfig); assignment = new HashSet<>(context.assignment()); offsets = new HashMap<>(); @@ -214,7 +216,8 @@ public void run() { ); topicPartitionWriters.put(tp, topicPartitionWriter); } - } catch (ClassNotFoundException | IllegalAccessException | InstantiationException e) { + } catch (ClassNotFoundException | IllegalAccessException | InstantiationException | + InvocationTargetException | NoSuchMethodException e) { throw new ConnectException("Reflection exception: ", e); } catch (IOException e) { throw new ConnectException(e); @@ -397,8 +400,23 @@ private void createDir(String dir) { } @SuppressWarnings("unchecked") - private Format getFormat() throws ClassNotFoundException, IllegalAccessException, InstantiationException{ - return ((Class) Class.forName(connectorConfig.getString(HdfsSinkConnectorConfig.FORMAT_CLASS_CONFIG))).newInstance(); + private Format newFormat() throws ClassNotFoundException, IllegalAccessException, + InstantiationException, InvocationTargetException, NoSuchMethodException { + Class formatClass = + (Class) connectorConfig.getClass(HdfsSinkConnectorConfig.FORMAT_CLASS_CONFIG); + return formatClass.getConstructor(HdfsStorage.class).newInstance(storage); + } + + private Partitioner newPartitioner(HdfsSinkConnectorConfig config) + throws ClassNotFoundException, IllegalAccessException, InstantiationException { + + @SuppressWarnings("unchecked") + Class partitionerClass = + (Class) config.getClass(PartitionerConfig.PARTITIONER_CLASS_CONFIG); + + Partitioner partitioner = partitionerClass.newInstance(); + partitioner.configure(new HashMap<>(config.plainValues())); + return partitioner; } private String getPartitionValue(String path) { From 0a2e00de375ca69689f0a061732a9cda0c4fe4a1 Mon Sep 17 00:00:00 2001 From: Konstantine Karantasis Date: Tue, 25 Apr 2017 19:36:03 -0700 Subject: [PATCH 47/58] Use hdfs's default partitioner in config. --- .../io/confluent/connect/hdfs/HdfsSinkConnectorTestBase.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/test/java/io/confluent/connect/hdfs/HdfsSinkConnectorTestBase.java b/src/test/java/io/confluent/connect/hdfs/HdfsSinkConnectorTestBase.java index 346d54821..d995890bf 100644 --- a/src/test/java/io/confluent/connect/hdfs/HdfsSinkConnectorTestBase.java +++ b/src/test/java/io/confluent/connect/hdfs/HdfsSinkConnectorTestBase.java @@ -30,6 +30,7 @@ import io.confluent.connect.avro.AvroData; import io.confluent.connect.hdfs.avro.AvroFormat; +import io.confluent.connect.hdfs.partitioner.DefaultPartitioner; import io.confluent.connect.storage.StorageSinkTestBase; import io.confluent.connect.storage.common.StorageCommonConfig; import io.confluent.connect.storage.hive.schema.DefaultSchemaGenerator; @@ -60,7 +61,7 @@ protected Map createProps() { props.put(HdfsSinkConnectorConfig.FORMAT_CLASS_CONFIG, AvroFormat.class.getName()); props.put( PartitionerConfig.PARTITIONER_CLASS_CONFIG, - PartitionerConfig.PARTITIONER_CLASS_DEFAULT.getName() + DefaultPartitioner.class.getName() ); props.put( PartitionerConfig.SCHEMA_GENERATOR_CLASS_CONFIG, From 463c74620f131c1806452a7c1c5fa376df222a7c Mon Sep 17 00:00:00 2001 From: Konstantine Karantasis Date: Tue, 25 Apr 2017 22:18:57 -0700 Subject: [PATCH 48/58] Patch for new schema compatibility rules. --- .../java/io/confluent/connect/hdfs/TopicPartitionWriter.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/main/java/io/confluent/connect/hdfs/TopicPartitionWriter.java b/src/main/java/io/confluent/connect/hdfs/TopicPartitionWriter.java index 215f11a32..3cd9ea233 100644 --- a/src/main/java/io/confluent/connect/hdfs/TopicPartitionWriter.java +++ b/src/main/java/io/confluent/connect/hdfs/TopicPartitionWriter.java @@ -287,7 +287,8 @@ public void write() { } SinkRecord record = buffer.peek(); Schema valueSchema = record.valueSchema(); - if (compatibility.shouldChangeSchema(record, null, currentSchema)) { + if ((recordCounter <= 0 && currentSchema == null && valueSchema != null) + || compatibility.shouldChangeSchema(record, null, currentSchema)) { currentSchema = valueSchema; if (hiveIntegration) { createHiveTable(); From a80dc544cc9e139018cea08fd8793dfbb6c8a42f Mon Sep 17 00:00:00 2001 From: Konstantine Karantasis Date: Tue, 25 Apr 2017 22:21:18 -0700 Subject: [PATCH 49/58] Final fixes to tests besides hive. --- .../connect/hdfs/FailureRecoveryTest.java | 9 ++----- .../hdfs/HdfsSinkConnectorTestBase.java | 1 + .../connect/hdfs/HdfsSinkTaskTest.java | 25 ++++++++----------- .../connect/hdfs/avro/AvroHiveUtilTest.java | 5 ++-- .../connect/hdfs/hive/HiveTestUtils.java | 6 ++--- .../hdfs/parquet/ParquetHiveUtilTest.java | 4 +-- .../partitioner/DailyPartitionerTest.java | 11 ++++++++ .../partitioner/HourlyPartitionerTest.java | 12 ++++++++- .../partitioner/TimeBasedPartitionerTest.java | 12 +++++++++ .../connect/hdfs/utils/MemoryFormat.java | 8 ++++++ .../confluent/connect/hdfs/wal/FSWALTest.java | 1 + .../connect/hdfs/wal/WALFileTest.java | 4 +-- .../confluent/connect/hdfs/wal/WALTest.java | 1 + 13 files changed, 67 insertions(+), 32 deletions(-) diff --git a/src/test/java/io/confluent/connect/hdfs/FailureRecoveryTest.java b/src/test/java/io/confluent/connect/hdfs/FailureRecoveryTest.java index 25772ef24..8bbd6eafb 100644 --- a/src/test/java/io/confluent/connect/hdfs/FailureRecoveryTest.java +++ b/src/test/java/io/confluent/connect/hdfs/FailureRecoveryTest.java @@ -45,10 +45,6 @@ protected Map createProps() { return props; } - public void setUp() throws Exception { - super.setUp(); - } - @Test public void testCommitFailure() throws Exception { setUp(); @@ -167,9 +163,8 @@ public void testWriterFailureMultiPartitions() throws Exception { @Test public void testWriterFailure() throws Exception { - Map props = createProps(); - - HdfsSinkConnectorConfig connectorConfig = new HdfsSinkConnectorConfig(props); + setUp(); + HdfsSinkConnectorConfig connectorConfig = new HdfsSinkConnectorConfig(properties); String key = "key"; Schema schema = createSchema(); diff --git a/src/test/java/io/confluent/connect/hdfs/HdfsSinkConnectorTestBase.java b/src/test/java/io/confluent/connect/hdfs/HdfsSinkConnectorTestBase.java index d995890bf..329adefb3 100644 --- a/src/test/java/io/confluent/connect/hdfs/HdfsSinkConnectorTestBase.java +++ b/src/test/java/io/confluent/connect/hdfs/HdfsSinkConnectorTestBase.java @@ -53,6 +53,7 @@ protected Map createProps() { Map props = super.createProps(); url = "memory://"; props.put(HdfsSinkConnectorConfig.HDFS_URL_CONFIG, url); + props.put(StorageCommonConfig.STORE_URL_CONFIG, url); props.put(HdfsSinkConnectorConfig.FLUSH_SIZE_CONFIG, "3"); props.put( StorageCommonConfig.STORAGE_CLASS_CONFIG, diff --git a/src/test/java/io/confluent/connect/hdfs/HdfsSinkTaskTest.java b/src/test/java/io/confluent/connect/hdfs/HdfsSinkTaskTest.java index c5744fee4..5dd9d9e1e 100644 --- a/src/test/java/io/confluent/connect/hdfs/HdfsSinkTaskTest.java +++ b/src/test/java/io/confluent/connect/hdfs/HdfsSinkTaskTest.java @@ -48,13 +48,12 @@ public class HdfsSinkTaskTest extends TestWithMiniDFSCluster { @Test public void testSinkTaskStart() throws Exception { + setUp(); createCommittedFiles(); - - Map props = createProps(); HdfsSinkTask task = new HdfsSinkTask(); task.initialize(context); - task.start(props); + task.start(properties); Map offsets = context.offsets(); assertEquals(offsets.size(), 2); @@ -68,11 +67,11 @@ public void testSinkTaskStart() throws Exception { @Test public void testSinkTaskStartNoCommittedFiles() throws Exception { - Map props = createProps(); + setUp(); HdfsSinkTask task = new HdfsSinkTask(); task.initialize(context); - task.start(props); + task.start(properties); // Even without any files in HDFS, we expect an explicit request to start from the beginning of the topic (which // either exists at offset 0, or offset 0 will be out of range and the consumer will reset to the smallest offset). @@ -88,6 +87,7 @@ public void testSinkTaskStartNoCommittedFiles() throws Exception { @Test public void testSinkTaskStartWithRecovery() throws Exception { + setUp(); Map> tempfiles = new HashMap<>(); List list1 = new ArrayList<>(); list1.add(FileUtils.tempFileName(url, topicsDir, DIRECTORY1, extension)); @@ -120,7 +120,6 @@ public void testSinkTaskStartWithRecovery() throws Exception { } } - setUp(); createWALs(tempfiles, committedFiles); HdfsSinkTask task = new HdfsSinkTask(); @@ -139,7 +138,7 @@ public void testSinkTaskStartWithRecovery() throws Exception { @Test public void testSinkTaskPut() throws Exception { - Map props = createProps(); + setUp(); HdfsSinkTask task = new HdfsSinkTask(); String key = "key"; @@ -154,7 +153,7 @@ public void testSinkTaskPut() throws Exception { } } task.initialize(context); - task.start(props); + task.start(properties); task.put(sinkRecords); task.stop(); @@ -182,7 +181,7 @@ public void testSinkTaskPut() throws Exception { @Test public void testSinkTaskPutPrimitive() throws Exception { - Map props = createProps(); + setUp(); HdfsSinkTask task = new HdfsSinkTask(); final String key = "key"; @@ -197,11 +196,10 @@ final int record = 12; } } task.initialize(context); - task.start(props); + task.start(properties); task.put(sinkRecords); task.stop(); - AvroData avroData = task.getAvroData(); // Last file (offset 6) doesn't satisfy size requirement and gets discarded on close long[] validOffsets = {-1, 2, 5}; @@ -241,9 +239,8 @@ private void createCommittedFiles() throws IOException { private void createWALs(Map> tempfiles, Map> committedFiles) throws Exception { @SuppressWarnings("unchecked") - Class storageClass = (Class) Class.forName( - connectorConfig.getString(StorageCommonConfig.STORAGE_CLASS_CONFIG) - ); + Class storageClass = (Class) + connectorConfig.getClass(StorageCommonConfig.STORAGE_CLASS_CONFIG); HdfsStorage storage = StorageFactory.createStorage( storageClass, HdfsSinkConnectorConfig.class, diff --git a/src/test/java/io/confluent/connect/hdfs/avro/AvroHiveUtilTest.java b/src/test/java/io/confluent/connect/hdfs/avro/AvroHiveUtilTest.java index 42f70ddc0..17cd87969 100644 --- a/src/test/java/io/confluent/connect/hdfs/avro/AvroHiveUtilTest.java +++ b/src/test/java/io/confluent/connect/hdfs/avro/AvroHiveUtilTest.java @@ -22,7 +22,6 @@ import org.apache.kafka.connect.data.Struct; import org.apache.kafka.connect.sink.SinkRecord; import org.apache.kafka.connect.sink.SinkTaskContext; -import org.junit.Before; import org.junit.Test; import java.util.ArrayList; @@ -60,7 +59,7 @@ public void setUp() throws Exception { public void testCreateTable() throws Exception { setUp(); prepareData(TOPIC, PARTITION); - Partitioner partitioner = HiveTestUtils.getPartitioner(); + Partitioner partitioner = HiveTestUtils.getPartitioner(parsedConfig); Schema schema = createSchema(); hive.createTable(hiveDatabase, TOPIC, schema, partitioner); @@ -103,7 +102,7 @@ public void testCreateTable() throws Exception { public void testAlterSchema() throws Exception { setUp(); prepareData(TOPIC, PARTITION); - Partitioner partitioner = HiveTestUtils.getPartitioner(); + Partitioner partitioner = HiveTestUtils.getPartitioner(parsedConfig); Schema schema = createSchema(); hive.createTable(hiveDatabase, TOPIC, schema, partitioner); diff --git a/src/test/java/io/confluent/connect/hdfs/hive/HiveTestUtils.java b/src/test/java/io/confluent/connect/hdfs/hive/HiveTestUtils.java index 09266d9fe..c7a4e89e7 100644 --- a/src/test/java/io/confluent/connect/hdfs/hive/HiveTestUtils.java +++ b/src/test/java/io/confluent/connect/hdfs/hive/HiveTestUtils.java @@ -21,16 +21,16 @@ import java.io.ByteArrayOutputStream; import java.io.InputStreamReader; import java.io.PrintStream; -import java.util.HashMap; +import java.util.Map; import io.confluent.connect.hdfs.partitioner.DefaultPartitioner; import io.confluent.connect.hdfs.partitioner.Partitioner; public class HiveTestUtils { - public static Partitioner getPartitioner() { + public static Partitioner getPartitioner(Map parsedConfig) { Partitioner partitioner = new DefaultPartitioner(); - partitioner.configure(new HashMap()); + partitioner.configure(parsedConfig); return partitioner; } diff --git a/src/test/java/io/confluent/connect/hdfs/parquet/ParquetHiveUtilTest.java b/src/test/java/io/confluent/connect/hdfs/parquet/ParquetHiveUtilTest.java index 56b7f8a8b..20e824366 100644 --- a/src/test/java/io/confluent/connect/hdfs/parquet/ParquetHiveUtilTest.java +++ b/src/test/java/io/confluent/connect/hdfs/parquet/ParquetHiveUtilTest.java @@ -63,7 +63,7 @@ public void setUp() throws Exception { public void testCreateTable() throws Exception { setUp(); prepareData(TOPIC, PARTITION); - Partitioner partitioner = HiveTestUtils.getPartitioner(); + Partitioner partitioner = HiveTestUtils.getPartitioner(parsedConfig); Schema schema = createSchema(); hive.createTable(hiveDatabase, TOPIC, schema, partitioner); @@ -106,7 +106,7 @@ public void testCreateTable() throws Exception { public void testAlterSchema() throws Exception { setUp(); prepareData(TOPIC, PARTITION); - Partitioner partitioner = HiveTestUtils.getPartitioner(); + Partitioner partitioner = HiveTestUtils.getPartitioner(parsedConfig); Schema schema = createSchema(); hive.createTable(hiveDatabase, TOPIC, schema, partitioner); diff --git a/src/test/java/io/confluent/connect/hdfs/partitioner/DailyPartitionerTest.java b/src/test/java/io/confluent/connect/hdfs/partitioner/DailyPartitionerTest.java index f47cb6081..69e5bcd0f 100644 --- a/src/test/java/io/confluent/connect/hdfs/partitioner/DailyPartitionerTest.java +++ b/src/test/java/io/confluent/connect/hdfs/partitioner/DailyPartitionerTest.java @@ -24,6 +24,7 @@ import io.confluent.connect.hdfs.HdfsSinkConnectorConfig; import io.confluent.connect.hdfs.TestWithMiniDFSCluster; +import io.confluent.connect.storage.hive.schema.TimeBasedSchemaGenerator; import io.confluent.connect.storage.partitioner.PartitionerConfig; import static org.junit.Assert.assertEquals; @@ -31,6 +32,16 @@ public class DailyPartitionerTest extends TestWithMiniDFSCluster { private static final long partitionDurationMs = TimeUnit.HOURS.toMillis(24); + @Override + protected Map createProps() { + Map props = super.createProps(); + props.put( + PartitionerConfig.SCHEMA_GENERATOR_CLASS_CONFIG, + TimeBasedSchemaGenerator.class.getName() + ); + return props; + } + @Test public void testDailyPartitioner() throws Exception { setUp(); diff --git a/src/test/java/io/confluent/connect/hdfs/partitioner/HourlyPartitionerTest.java b/src/test/java/io/confluent/connect/hdfs/partitioner/HourlyPartitionerTest.java index 1606bddc9..23b38dd6d 100644 --- a/src/test/java/io/confluent/connect/hdfs/partitioner/HourlyPartitionerTest.java +++ b/src/test/java/io/confluent/connect/hdfs/partitioner/HourlyPartitionerTest.java @@ -24,14 +24,24 @@ import io.confluent.connect.hdfs.HdfsSinkConnectorConfig; import io.confluent.connect.hdfs.TestWithMiniDFSCluster; +import io.confluent.connect.storage.hive.schema.TimeBasedSchemaGenerator; import io.confluent.connect.storage.partitioner.PartitionerConfig; import static org.junit.Assert.assertEquals; public class HourlyPartitionerTest extends TestWithMiniDFSCluster { - private static final long partitionDurationMs = TimeUnit.HOURS.toMillis(1); + @Override + protected Map createProps() { + Map props = super.createProps(); + props.put( + PartitionerConfig.SCHEMA_GENERATOR_CLASS_CONFIG, + TimeBasedSchemaGenerator.class.getName() + ); + return props; + } + @Test public void testHourlyPartitioner() throws Exception { setUp(); diff --git a/src/test/java/io/confluent/connect/hdfs/partitioner/TimeBasedPartitionerTest.java b/src/test/java/io/confluent/connect/hdfs/partitioner/TimeBasedPartitionerTest.java index 491b7311c..fdaeaa214 100644 --- a/src/test/java/io/confluent/connect/hdfs/partitioner/TimeBasedPartitionerTest.java +++ b/src/test/java/io/confluent/connect/hdfs/partitioner/TimeBasedPartitionerTest.java @@ -25,6 +25,8 @@ import java.util.concurrent.TimeUnit; import io.confluent.connect.hdfs.HdfsSinkConnectorTestBase; +import io.confluent.connect.storage.hive.schema.TimeBasedSchemaGenerator; +import io.confluent.connect.storage.partitioner.PartitionerConfig; import static org.junit.Assert.assertEquals; @@ -32,6 +34,16 @@ public class TimeBasedPartitionerTest extends HdfsSinkConnectorTestBase { private static final String timeZoneString = "America/Los_Angeles"; private static final DateTimeZone DATE_TIME_ZONE = DateTimeZone.forID(timeZoneString); + @Override + protected Map createProps() { + Map props = super.createProps(); + props.put( + PartitionerConfig.SCHEMA_GENERATOR_CLASS_CONFIG, + TimeBasedSchemaGenerator.class.getName() + ); + return props; + } + @Test public void testGeneratePartitionedPath() throws Exception { setUp(); diff --git a/src/test/java/io/confluent/connect/hdfs/utils/MemoryFormat.java b/src/test/java/io/confluent/connect/hdfs/utils/MemoryFormat.java index 58fcefd84..2e0a07338 100644 --- a/src/test/java/io/confluent/connect/hdfs/utils/MemoryFormat.java +++ b/src/test/java/io/confluent/connect/hdfs/utils/MemoryFormat.java @@ -9,9 +9,17 @@ import io.confluent.connect.hdfs.SchemaFileReader; import io.confluent.connect.hdfs.hive.HiveMetaStore; import io.confluent.connect.hdfs.hive.HiveUtil; +import io.confluent.connect.hdfs.storage.HdfsStorage; import io.confluent.connect.storage.hive.HiveFactory; public class MemoryFormat implements Format { + private final AvroData avroData; + + public MemoryFormat(HdfsStorage storage) { + this.avroData = new AvroData( + storage.conf().getInt(HdfsSinkConnectorConfig.SCHEMA_CACHE_SIZE_CONFIG) + ); + } @Override public RecordWriterProvider getRecordWriterProvider() { diff --git a/src/test/java/io/confluent/connect/hdfs/wal/FSWALTest.java b/src/test/java/io/confluent/connect/hdfs/wal/FSWALTest.java index be98c63f9..6eaebf6a8 100644 --- a/src/test/java/io/confluent/connect/hdfs/wal/FSWALTest.java +++ b/src/test/java/io/confluent/connect/hdfs/wal/FSWALTest.java @@ -29,6 +29,7 @@ public class FSWALTest extends TestWithMiniDFSCluster { @Test public void testTruncate() throws Exception { + setUp(); HdfsStorage storage = new HdfsStorage(connectorConfig, url); TopicPartition tp = new TopicPartition("mytopic", 123); FSWAL wal = new FSWAL("/logs", tp, storage); diff --git a/src/test/java/io/confluent/connect/hdfs/wal/WALFileTest.java b/src/test/java/io/confluent/connect/hdfs/wal/WALFileTest.java index fc59fdaa4..b29f2e3c6 100644 --- a/src/test/java/io/confluent/connect/hdfs/wal/WALFileTest.java +++ b/src/test/java/io/confluent/connect/hdfs/wal/WALFileTest.java @@ -35,8 +35,8 @@ public class WALFileTest extends TestWithMiniDFSCluster { @Test public void testAppend() throws Exception { - Map props = createProps(); - HdfsSinkConnectorConfig connectorConfig = new HdfsSinkConnectorConfig(props); + setUp(); + HdfsSinkConnectorConfig connectorConfig = new HdfsSinkConnectorConfig(properties); String topicsDir = connectorConfig.getString(StorageCommonConfig.TOPICS_DIR_CONFIG); String topic = "topic"; diff --git a/src/test/java/io/confluent/connect/hdfs/wal/WALTest.java b/src/test/java/io/confluent/connect/hdfs/wal/WALTest.java index 6a61ca800..323b73e8d 100644 --- a/src/test/java/io/confluent/connect/hdfs/wal/WALTest.java +++ b/src/test/java/io/confluent/connect/hdfs/wal/WALTest.java @@ -37,6 +37,7 @@ public class WALTest extends TestWithMiniDFSCluster { @Test public void testWALMultiClient() throws Exception { + setUp(); fs.delete(new Path(FileUtils.directoryName(url, topicsDir, TOPIC_PARTITION)), true); @SuppressWarnings("unchecked") From 4acedcbab22eef9c2a135628384bfca8dcc91dcf Mon Sep 17 00:00:00 2001 From: Ewen Cheslack-Postava Date: Wed, 24 May 2017 20:44:21 -0700 Subject: [PATCH 50/58] Fix PartitionerTests that should no longer expect trailing slashes in encoded partitions. --- .../connect/hdfs/partitioner/DailyPartitionerTest.java | 2 +- .../connect/hdfs/partitioner/HourlyPartitionerTest.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/test/java/io/confluent/connect/hdfs/partitioner/DailyPartitionerTest.java b/src/test/java/io/confluent/connect/hdfs/partitioner/DailyPartitionerTest.java index 69e5bcd0f..e736d9834 100644 --- a/src/test/java/io/confluent/connect/hdfs/partitioner/DailyPartitionerTest.java +++ b/src/test/java/io/confluent/connect/hdfs/partitioner/DailyPartitionerTest.java @@ -53,7 +53,7 @@ public void testDailyPartitioner() throws Exception { long timestamp = new DateTime(2014, 2, 1, 3, 0, 0, 0, DateTimeZone.forID(timeZoneString)).getMillis(); String encodedPartition = TimeUtils.encodeTimestamp(partitionDurationMs, pathFormat, timeZoneString, timestamp); String path = partitioner.generatePartitionedPath("topic", encodedPartition); - assertEquals("topic/year=2014/month=02/day=01/", path); + assertEquals("topic/year=2014/month=02/day=01", path); } } diff --git a/src/test/java/io/confluent/connect/hdfs/partitioner/HourlyPartitionerTest.java b/src/test/java/io/confluent/connect/hdfs/partitioner/HourlyPartitionerTest.java index 23b38dd6d..94f90b9e9 100644 --- a/src/test/java/io/confluent/connect/hdfs/partitioner/HourlyPartitionerTest.java +++ b/src/test/java/io/confluent/connect/hdfs/partitioner/HourlyPartitionerTest.java @@ -54,7 +54,7 @@ public void testHourlyPartitioner() throws Exception { String encodedPartition = TimeUtils.encodeTimestamp(partitionDurationMs, pathFormat, timeZoneString, timestamp); String path = partitioner.generatePartitionedPath("topic", encodedPartition); - assertEquals("topic/year=2015/month=02/day=01/hour=03/", path); + assertEquals("topic/year=2015/month=02/day=01/hour=03", path); } } From a8114de95fc4c7b322cac76ad1a2aa09eed258c2 Mon Sep 17 00:00:00 2001 From: Ewen Cheslack-Postava Date: Thu, 25 May 2017 01:26:33 -0700 Subject: [PATCH 51/58] Fix Hive queries to use table name conversion to handle unsafe topic names --- .../confluent/connect/hdfs/avro/AvroHiveUtilTest.java | 10 ++++++++-- .../connect/hdfs/avro/HiveIntegrationAvroTest.java | 5 ++++- .../hdfs/parquet/HiveIntegrationParquetTest.java | 10 ++++++++-- .../connect/hdfs/parquet/ParquetHiveUtilTest.java | 10 ++++++++-- 4 files changed, 28 insertions(+), 7 deletions(-) diff --git a/src/test/java/io/confluent/connect/hdfs/avro/AvroHiveUtilTest.java b/src/test/java/io/confluent/connect/hdfs/avro/AvroHiveUtilTest.java index 17cd87969..af1b4275c 100644 --- a/src/test/java/io/confluent/connect/hdfs/avro/AvroHiveUtilTest.java +++ b/src/test/java/io/confluent/connect/hdfs/avro/AvroHiveUtilTest.java @@ -85,7 +85,10 @@ public void testCreateTable() throws Exception { assertEquals(1, partitionCols.size()); assertEquals("partition", partitionCols.get(0).getName()); - String result = HiveTestUtils.runHive(hiveExec, "SELECT * FROM " + TOPIC); + String result = HiveTestUtils.runHive( + hiveExec, + "SELECT * FROM " + hiveMetaStore.tableNameConverter(TOPIC) + ); String[] rows = result.split("\n"); // Only 6 of the 7 records should have been delivered due to flush_size = 3 assertEquals(6, rows.length); @@ -129,7 +132,10 @@ public void testAlterSchema() throws Exception { hive.alterSchema(hiveDatabase, TOPIC, newSchema); - String result = HiveTestUtils.runHive(hiveExec, "SELECT * from " + TOPIC); + String result = HiveTestUtils.runHive( + hiveExec, + "SELECT * from " + hiveMetaStore.tableNameConverter(TOPIC) + ); String[] rows = result.split("\n"); // Only 6 of the 7 records should have been delivered due to flush_size = 3 assertEquals(6, rows.length); diff --git a/src/test/java/io/confluent/connect/hdfs/avro/HiveIntegrationAvroTest.java b/src/test/java/io/confluent/connect/hdfs/avro/HiveIntegrationAvroTest.java index 8ac238085..37690074a 100644 --- a/src/test/java/io/confluent/connect/hdfs/avro/HiveIntegrationAvroTest.java +++ b/src/test/java/io/confluent/connect/hdfs/avro/HiveIntegrationAvroTest.java @@ -267,7 +267,10 @@ public void testHiveIntegrationFieldPartitionerAvro() throws Exception { expectedResult.add(part); } } - String result = HiveTestUtils.runHive(hiveExec, "SELECT * FROM " + TOPIC); + String result = HiveTestUtils.runHive( + hiveExec, + "SELECT * FROM " + hiveMetaStore.tableNameConverter(TOPIC) + ); String[] rows = result.split("\n"); assertEquals(9, rows.length); for (int i = 0; i < rows.length; ++i) { diff --git a/src/test/java/io/confluent/connect/hdfs/parquet/HiveIntegrationParquetTest.java b/src/test/java/io/confluent/connect/hdfs/parquet/HiveIntegrationParquetTest.java index 073ba721e..bf234c865 100644 --- a/src/test/java/io/confluent/connect/hdfs/parquet/HiveIntegrationParquetTest.java +++ b/src/test/java/io/confluent/connect/hdfs/parquet/HiveIntegrationParquetTest.java @@ -199,7 +199,10 @@ public void testHiveIntegrationFieldPartitionerParquet() throws Exception { } } - String result = HiveTestUtils.runHive(hiveExec, "SELECT * FROM " + TOPIC); + String result = HiveTestUtils.runHive( + hiveExec, + "SELECT * FROM " + hiveMetaStore.tableNameConverter(TOPIC) + ); String[] rows = result.split("\n"); assertEquals(9, rows.length); for (int i = 0; i < rows.length; ++i) { @@ -273,7 +276,10 @@ public void testHiveIntegrationTimeBasedPartitionerParquet() throws Exception { } } - String result = HiveTestUtils.runHive(hiveExec, "SELECT * FROM " + TOPIC); + String result = HiveTestUtils.runHive( + hiveExec, + "SELECT * FROM " + hiveMetaStore.tableNameConverter(TOPIC) + ); String[] rows = result.split("\n"); assertEquals(9, rows.length); for (int i = 0; i < rows.length; ++i) { diff --git a/src/test/java/io/confluent/connect/hdfs/parquet/ParquetHiveUtilTest.java b/src/test/java/io/confluent/connect/hdfs/parquet/ParquetHiveUtilTest.java index 20e824366..c5cf288c4 100644 --- a/src/test/java/io/confluent/connect/hdfs/parquet/ParquetHiveUtilTest.java +++ b/src/test/java/io/confluent/connect/hdfs/parquet/ParquetHiveUtilTest.java @@ -89,7 +89,10 @@ public void testCreateTable() throws Exception { assertEquals(1, partitionCols.size()); assertEquals("partition", partitionCols.get(0).getName()); - String result = HiveTestUtils.runHive(hiveExec, "SELECT * from " + TOPIC); + String result = HiveTestUtils.runHive( + hiveExec, + "SELECT * from " + hiveMetaStore.tableNameConverter(TOPIC) + ); String[] rows = result.split("\n"); // Only 6 of the 7 records should have been delivered due to flush_size = 3 assertEquals(6, rows.length); @@ -132,7 +135,10 @@ public void testAlterSchema() throws Exception { hive.alterSchema(hiveDatabase, TOPIC, newSchema); - String result = HiveTestUtils.runHive(hiveExec, "SELECT * from " + TOPIC); + String result = HiveTestUtils.runHive( + hiveExec, + "SELECT * from " + hiveMetaStore.tableNameConverter(TOPIC) + ); String[] rows = result.split("\n"); // Only 6 of the 7 records should have been delivered due to flush_size = 3 assertEquals(6, rows.length); From 9a261b6ef79bdb8f738558bc57855c8cc4f3bf3c Mon Sep 17 00:00:00 2001 From: Ewen Cheslack-Postava Date: Mon, 31 Jul 2017 16:54:48 -0700 Subject: [PATCH 52/58] Remove storage-common jars from packaged version --- src/assembly/package.xml | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/assembly/package.xml b/src/assembly/package.xml index 23024a3cb..449606b93 100644 --- a/src/assembly/package.xml +++ b/src/assembly/package.xml @@ -42,6 +42,12 @@ com.sun.jersey:* org.eclipse.jetty.aggregate:jetty-all com.sun.jersey.contribs:jersey-guice + io.confluent:kafka-connect-storage-common + io.confluent:kafka-connect-storage-core + io.confluent:kafka-connect-storage-format + io.confluent:kafka-connect-storage-hive + io.confluent:kafka-connect-storage-partitioner + io.confluent:kafka-connect-storage-wal From d979231ebabd3b6f89e06d57ff56a08bb326e832 Mon Sep 17 00:00:00 2001 From: Ewen Cheslack-Postava Date: Tue, 1 Aug 2017 15:58:59 -0700 Subject: [PATCH 53/58] Provide compatibility with old Format and related classes. --- .../io/confluent/connect/hdfs/DataWriter.java | 104 +++++++++++++--- .../io/confluent/connect/hdfs/Format.java | 8 +- .../connect/hdfs/OldRecordWriterWrapper.java | 45 +++++++ .../confluent/connect/hdfs/RecordWriter.java | 7 +- .../connect/hdfs/RecordWriterProvider.java | 22 ++-- .../connect/hdfs/SchemaFileReader.java | 12 +- .../connect/hdfs/TopicPartitionWriter.java | 86 ++++++++++--- .../connect/hdfs/avro/AvroFileReader.java | 20 +-- .../connect/hdfs/avro/AvroFormat.java | 31 ++--- .../hdfs/avro/AvroRecordWriterProvider.java | 20 +-- .../confluent/connect/hdfs/hive/HiveUtil.java | 1 + .../hdfs/parquet/ParquetFileReader.java | 28 +---- .../connect/hdfs/parquet/ParquetFormat.java | 33 ++--- .../parquet/ParquetRecordWriterProvider.java | 18 +-- .../connect/hdfs/DataFileReader.java | 15 +++ .../connect/hdfs/FailureRecoveryTest.java | 5 +- .../FormatAPIDataWriterCompatibilityTest.java | 75 ++++++++++++ ...TopicPartitionWriterCompatibilityTest.java | 115 ++++++++++++++++++ .../connect/hdfs/HdfsSinkTaskTest.java | 7 +- .../hdfs/HdfsSinkTaskTestWithSecureHDFS.java | 5 +- .../io/confluent/connect/hdfs/OldFormat.java | 86 +++++++++++++ .../connect/hdfs/TestWithMiniDFSCluster.java | 4 +- .../connect/hdfs/avro/AvroDataFileReader.java | 30 +++++ .../connect/hdfs/avro/DataWriterAvroTest.java | 2 +- .../hdfs/avro/TopicPartitionWriterTest.java | 23 ++-- .../hdfs/parquet/DataWriterParquetTest.java | 3 +- .../hdfs/parquet/ParquetDataFileReader.java | 29 +++++ .../connect/hdfs/utils/MemoryFormat.java | 30 ++--- .../hdfs/utils/MemoryRecordWriter.java | 2 +- .../utils/MemoryRecordWriterProvider.java | 21 +--- 30 files changed, 643 insertions(+), 244 deletions(-) create mode 100644 src/main/java/io/confluent/connect/hdfs/OldRecordWriterWrapper.java create mode 100644 src/test/java/io/confluent/connect/hdfs/DataFileReader.java create mode 100644 src/test/java/io/confluent/connect/hdfs/FormatAPIDataWriterCompatibilityTest.java create mode 100644 src/test/java/io/confluent/connect/hdfs/FormatAPITopicPartitionWriterCompatibilityTest.java create mode 100644 src/test/java/io/confluent/connect/hdfs/OldFormat.java create mode 100644 src/test/java/io/confluent/connect/hdfs/avro/AvroDataFileReader.java create mode 100644 src/test/java/io/confluent/connect/hdfs/parquet/ParquetDataFileReader.java diff --git a/src/main/java/io/confluent/connect/hdfs/DataWriter.java b/src/main/java/io/confluent/connect/hdfs/DataWriter.java index 8e965e327..9616afc28 100644 --- a/src/main/java/io/confluent/connect/hdfs/DataWriter.java +++ b/src/main/java/io/confluent/connect/hdfs/DataWriter.java @@ -55,8 +55,8 @@ import io.confluent.connect.hdfs.storage.HdfsStorage; import io.confluent.connect.hdfs.storage.Storage; import io.confluent.connect.storage.common.StorageCommonConfig; +import io.confluent.connect.storage.format.SchemaFileReader; import io.confluent.connect.storage.hive.HiveConfig; -import io.confluent.connect.hdfs.partitioner.DefaultPartitioner; import io.confluent.connect.storage.partitioner.PartitionerConfig; public class DataWriter { @@ -67,10 +67,14 @@ public class DataWriter { private HdfsStorage storage; private String topicsDir; private Format format; + private RecordWriterProvider writerProvider; + private io.confluent.connect.storage.format.RecordWriterProvider + newWriterProvider; + private io.confluent.connect.storage.format.SchemaFileReader + schemaFileReader; + private io.confluent.connect.storage.format.Format newFormat; private Set assignment; private Partitioner partitioner; - private RecordWriterProvider writerProvider; - private SchemaFileReader schemaFileReader; private Map offsets; private HdfsSinkConnectorConfig connectorConfig; private AvroData avroData; @@ -180,9 +184,57 @@ public void run() { createDir(topicsDir + HdfsSinkConnectorConstants.TEMPFILE_DIRECTORY); createDir(logsDir); - format = newFormat(); - writerProvider = format.getRecordWriterProvider(); - schemaFileReader = format.getSchemaFileReader(); + // Try to instantiate as a new-style storage-common type class, then fall back to old-style with + // no parameters + try { + Class formatClass = + (Class) connectorConfig.getClass(HdfsSinkConnectorConfig.FORMAT_CLASS_CONFIG); + newFormat = formatClass.getConstructor(HdfsStorage.class).newInstance(storage); + newWriterProvider = newFormat.getRecordWriterProvider(); + schemaFileReader = newFormat.getSchemaFileReader(); + } catch (NoSuchMethodException e) { + Class formatClass = + (Class) connectorConfig.getClass(HdfsSinkConnectorConfig.FORMAT_CLASS_CONFIG); + format = formatClass.getConstructor().newInstance(); + writerProvider = format.getRecordWriterProvider(); + final io.confluent.connect.hdfs.SchemaFileReader oldReader + = format.getSchemaFileReader(avroData); + schemaFileReader = new SchemaFileReader() { + @Override + public Schema getSchema(HdfsSinkConnectorConfig hdfsSinkConnectorConfig, Path path) { + try { + return oldReader.getSchema(hdfsSinkConnectorConfig.getHadoopConfiguration(), path); + } catch (IOException e) { + throw new ConnectException("Failed to get schema", e); + } + } + + @Override + public Iterator iterator() { + throw new UnsupportedOperationException(); + } + + @Override + public boolean hasNext() { + throw new UnsupportedOperationException(); + } + + @Override + public Object next() { + throw new UnsupportedOperationException(); + } + + @Override + public void remove() { + throw new UnsupportedOperationException(); + } + + @Override + public void close() throws IOException { + + } + }; + } partitioner = newPartitioner(connectorConfig); @@ -193,7 +245,26 @@ public void run() { if (hiveIntegration) { hiveDatabase = connectorConfig.getString(HiveConfig.HIVE_DATABASE_CONFIG); hiveMetaStore = new HiveMetaStore(conf, connectorConfig); - hive = format.getHiveUtil(connectorConfig, hiveMetaStore); + if (format != null) { + hive = format.getHiveUtil(connectorConfig, hiveMetaStore); + } else if (newFormat != null) { + final io.confluent.connect.storage.hive.HiveUtil newHiveUtil + = newFormat.getHiveFactory().createHiveUtil(connectorConfig, hiveMetaStore); + hive = new HiveUtil(connectorConfig, hiveMetaStore) { + @Override + public void createTable(String database, String tableName, Schema schema, + Partitioner partitioner) { + newHiveUtil.createTable(database, tableName, schema, partitioner); + } + + @Override + public void alterSchema(String database, String tableName, Schema schema) { + newHiveUtil.alterSchema(database, tableName, schema); + } + }; + } else { + throw new ConnectException("One of old or new format classes must be provided"); + } executorService = Executors.newSingleThreadExecutor(); hiveUpdateFutures = new LinkedList<>(); } @@ -204,6 +275,7 @@ public void run() { tp, storage, writerProvider, + newWriterProvider, partitioner, connectorConfig, context, @@ -272,7 +344,12 @@ public void syncWithHive() throws ConnectException { CommittedFileFilter filter = new TopicCommittedFileFilter(topic); FileStatus fileStatusWithMaxOffset = FileUtils.fileStatusWithMaxOffset(storage, new Path(topicDir), filter); if (fileStatusWithMaxOffset != null) { - Schema latestSchema = schemaFileReader.getSchema(connectorConfig, fileStatusWithMaxOffset.getPath()); + final Path path = fileStatusWithMaxOffset.getPath(); + final Schema latestSchema; + latestSchema = schemaFileReader.getSchema( + connectorConfig, + path + ); hive.createTable(hiveDatabase, topic, latestSchema, partitioner); List partitions = hiveMetaStore.listPartitions(hiveDatabase, topic, (short) -1); FileStatus[] statuses = FileUtils.getDirectories(storage, new Path(topicDir)); @@ -297,6 +374,7 @@ public void open(Collection partitions) { tp, storage, writerProvider, + newWriterProvider, partitioner, connectorConfig, context, @@ -383,7 +461,7 @@ public Storage getStorage() { return storage; } - public Map getWriters(TopicPartition tp) { + Map getWriters(TopicPartition tp) { return topicPartitionWriters.get(tp).getWriters(); } @@ -399,14 +477,6 @@ private void createDir(String dir) { } } - @SuppressWarnings("unchecked") - private Format newFormat() throws ClassNotFoundException, IllegalAccessException, - InstantiationException, InvocationTargetException, NoSuchMethodException { - Class formatClass = - (Class) connectorConfig.getClass(HdfsSinkConnectorConfig.FORMAT_CLASS_CONFIG); - return formatClass.getConstructor(HdfsStorage.class).newInstance(storage); - } - private Partitioner newPartitioner(HdfsSinkConnectorConfig config) throws ClassNotFoundException, IllegalAccessException, InstantiationException { diff --git a/src/main/java/io/confluent/connect/hdfs/Format.java b/src/main/java/io/confluent/connect/hdfs/Format.java index 5be03f93d..c59f93d2a 100644 --- a/src/main/java/io/confluent/connect/hdfs/Format.java +++ b/src/main/java/io/confluent/connect/hdfs/Format.java @@ -14,18 +14,14 @@ package io.confluent.connect.hdfs; -import org.apache.hadoop.fs.Path; - import io.confluent.connect.avro.AvroData; import io.confluent.connect.hdfs.hive.HiveMetaStore; import io.confluent.connect.hdfs.hive.HiveUtil; +// NOTE: DO NOT add or modify this class as it is maintained for compatibility @Deprecated -public interface Format - extends io.confluent.connect.storage.format.Format { +public interface Format { RecordWriterProvider getRecordWriterProvider(); SchemaFileReader getSchemaFileReader(AvroData avroData); - @Override - SchemaFileReader getSchemaFileReader(); HiveUtil getHiveUtil(HdfsSinkConnectorConfig config, HiveMetaStore hiveMetaStore); } diff --git a/src/main/java/io/confluent/connect/hdfs/OldRecordWriterWrapper.java b/src/main/java/io/confluent/connect/hdfs/OldRecordWriterWrapper.java new file mode 100644 index 000000000..6c675f5e1 --- /dev/null +++ b/src/main/java/io/confluent/connect/hdfs/OldRecordWriterWrapper.java @@ -0,0 +1,45 @@ +package io.confluent.connect.hdfs; + +import org.apache.kafka.connect.errors.ConnectException; +import org.apache.kafka.connect.sink.SinkRecord; + +import java.io.IOException; + +/** + * Wrapper for old-style RecordWriters that implements the new common RecordWriter interface and + * delegates to the old implementation. + */ +public class OldRecordWriterWrapper implements io.confluent.connect.storage.format.RecordWriter { + + // Strictly speaking RecordWriter was generic, but in practice the implementation was always + // using the SinkRecord type despite the type not being specified everywhere. + private final RecordWriter oldWriter; + + public OldRecordWriterWrapper(RecordWriter oldWriter) { + this.oldWriter = oldWriter; + } + + @Override + public void write(SinkRecord sinkRecord) { + try { + oldWriter.write(sinkRecord); + } catch (IOException e) { + throw new ConnectException("Failed to write a record to " + oldWriter, e); + } + } + + @Override + public void commit() { + // Old interface doesn't have commit + } + + + @Override + public void close() { + try { + oldWriter.close(); + } catch (IOException e) { + throw new ConnectException("Failed to close " + oldWriter, e); + } + } +} diff --git a/src/main/java/io/confluent/connect/hdfs/RecordWriter.java b/src/main/java/io/confluent/connect/hdfs/RecordWriter.java index f620c37eb..873358cf6 100644 --- a/src/main/java/io/confluent/connect/hdfs/RecordWriter.java +++ b/src/main/java/io/confluent/connect/hdfs/RecordWriter.java @@ -16,6 +16,11 @@ package io.confluent.connect.hdfs; +import java.io.IOException; + +// NOTE: DO NOT add or modify this class as it is maintained for compatibility @Deprecated -public interface RecordWriter extends io.confluent.connect.storage.format.RecordWriter { +public interface RecordWriter { + void write(V value) throws IOException; + void close() throws IOException; } diff --git a/src/main/java/io/confluent/connect/hdfs/RecordWriterProvider.java b/src/main/java/io/confluent/connect/hdfs/RecordWriterProvider.java index 5ff6b014b..bffd895b9 100644 --- a/src/main/java/io/confluent/connect/hdfs/RecordWriterProvider.java +++ b/src/main/java/io/confluent/connect/hdfs/RecordWriterProvider.java @@ -14,23 +14,21 @@ package io.confluent.connect.hdfs; +import org.apache.hadoop.conf.Configuration; import org.apache.kafka.connect.sink.SinkRecord; +import java.io.IOException; + import io.confluent.connect.avro.AvroData; +// NOTE: DO NOT add or modify this class as it is maintained for compatibility @Deprecated -public interface RecordWriterProvider - extends io.confluent.connect.storage.format.RecordWriterProvider { - RecordWriter getRecordWriter( - HdfsSinkConnectorConfig conf, +public interface RecordWriterProvider { + String getExtension(); + RecordWriter getRecordWriter( + Configuration conf, String fileName, SinkRecord record, - final AvroData avroData - ); - - @Override - RecordWriter getRecordWriter( - HdfsSinkConnectorConfig conf, - String fileName - ); + AvroData avroData + ) throws IOException; } diff --git a/src/main/java/io/confluent/connect/hdfs/SchemaFileReader.java b/src/main/java/io/confluent/connect/hdfs/SchemaFileReader.java index 7cdea26db..c2c3a5a0b 100644 --- a/src/main/java/io/confluent/connect/hdfs/SchemaFileReader.java +++ b/src/main/java/io/confluent/connect/hdfs/SchemaFileReader.java @@ -14,12 +14,18 @@ package io.confluent.connect.hdfs; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; +import org.apache.kafka.connect.data.Schema; +import java.io.IOException; import java.util.Collection; +// NOTE: DO NOT add or modify this class as it is maintained for compatibility @Deprecated -public interface SchemaFileReader - extends io.confluent.connect.storage.format.SchemaFileReader { - Collection readData(HdfsSinkConnectorConfig conf, Path path); +public interface SchemaFileReader { + Schema getSchema(Configuration conf, Path path) throws IOException; + // NOTE: This method is no longer used and was only previously used in tests. It is safe to + // provide a dummy implementation. + Collection readData(Configuration conf, Path path) throws IOException; } diff --git a/src/main/java/io/confluent/connect/hdfs/TopicPartitionWriter.java b/src/main/java/io/confluent/connect/hdfs/TopicPartitionWriter.java index 4b97b5cc7..07da9063d 100644 --- a/src/main/java/io/confluent/connect/hdfs/TopicPartitionWriter.java +++ b/src/main/java/io/confluent/connect/hdfs/TopicPartitionWriter.java @@ -14,7 +14,6 @@ package io.confluent.connect.hdfs; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; import org.apache.kafka.common.TopicPartition; @@ -61,7 +60,7 @@ public class TopicPartitionWriter { private static final Logger log = LoggerFactory.getLogger(TopicPartitionWriter.class); private WAL wal; private Map tempFiles; - private Map writers; + private Map writers; private TopicPartition tp; private Partitioner partitioner; private String url; @@ -77,8 +76,14 @@ public class TopicPartitionWriter { private long lastRotate; private long rotateScheduleIntervalMs; private long nextScheduledRotate; + // This is one case where we cannot simply wrap the old or new RecordWriterProvider with the + // other because they have incompatible requirements for some methods -- one requires the Hadoop + // config + extra parameters, the other requires the ConnectorConfig and doesn't get the other + // extra parameters. Instead, we have to (optionally) store one of each and use whichever one is + // non-null. private RecordWriterProvider writerProvider; - private Configuration conf; + private final io.confluent.connect.storage.format.RecordWriterProvider + newWriterProvider; private HdfsSinkConnectorConfig connectorConfig; private AvroData avroData; private Set appended; @@ -96,7 +101,8 @@ public class TopicPartitionWriter { private final boolean hiveIntegration; private String hiveDatabase; private HiveMetaStore hiveMetaStore; - private SchemaFileReader schemaFileReader; + private io.confluent.connect.storage.format.SchemaFileReader + schemaFileReader; private HiveUtil hive; private ExecutorService executorService; private Queue> hiveUpdateFutures; @@ -106,24 +112,41 @@ public TopicPartitionWriter( TopicPartition tp, HdfsStorage storage, RecordWriterProvider writerProvider, + io.confluent.connect.storage.format.RecordWriterProvider newWriterProvider, Partitioner partitioner, HdfsSinkConnectorConfig connectorConfig, SinkTaskContext context, AvroData avroData) { - this(tp, storage, writerProvider, partitioner, connectorConfig, context, avroData, null, null, null, null, null); + this( + tp, + storage, + writerProvider, + newWriterProvider, + partitioner, + connectorConfig, + context, + avroData, + null, + null, + null, + null, + null + ); } public TopicPartitionWriter( TopicPartition tp, HdfsStorage storage, RecordWriterProvider writerProvider, + io.confluent.connect.storage.format.RecordWriterProvider newWriterProvider, Partitioner partitioner, HdfsSinkConnectorConfig connectorConfig, SinkTaskContext context, AvroData avroData, HiveMetaStore hiveMetaStore, HiveUtil hive, - SchemaFileReader schemaFileReader, + io.confluent.connect.storage.format.SchemaFileReader + schemaFileReader, ExecutorService executorService, Queue> hiveUpdateFutures) { this.tp = tp; @@ -131,10 +154,10 @@ public TopicPartitionWriter( this.avroData = avroData; this.storage = storage; this.writerProvider = writerProvider; + this.newWriterProvider = newWriterProvider; this.partitioner = partitioner; this.url = storage.url(); this.connectorConfig = storage.conf(); - this.conf = storage.conf().getHadoopConfiguration(); this.schemaFileReader = schemaFileReader; topicsDir = connectorConfig.getString(StorageCommonConfig.TOPICS_DIR_CONFIG); @@ -157,7 +180,14 @@ public TopicPartitionWriter( state = State.RECOVERY_STARTED; failureTime = -1L; offset = -1L; - extension = writerProvider.getExtension(); + if (writerProvider != null) { + extension = writerProvider.getExtension(); + } else if (newWriterProvider != null) { + extension = newWriterProvider.getExtension(); + } else { + throw new ConnectException("Invalid state: either old or new RecordWriterProvider must be" + + " provided"); + } zeroPadOffsetFormat = "%0" + connectorConfig.getInt(HdfsSinkConnectorConfig.FILENAME_OFFSET_ZERO_PAD_WIDTH_CONFIG) + @@ -403,7 +433,7 @@ public long offset() { return offset; } - public Map getWriters() { + Map getWriters() { return writers; } @@ -448,18 +478,36 @@ private void resume() { context.resume(tp); } - private RecordWriter getWriter(SinkRecord record, String encodedPartition) - throws ConnectException { + private io.confluent.connect.storage.format.RecordWriter getWriter( + SinkRecord record, + String encodedPartition + ) throws ConnectException { if (writers.containsKey(encodedPartition)) { return writers.get(encodedPartition); } String tempFile = getTempFile(encodedPartition); - RecordWriter writer = writerProvider.getRecordWriter( - connectorConfig, - tempFile, - record, - avroData - ); + + final io.confluent.connect.storage.format.RecordWriter writer; + try { + if (writerProvider != null) { + writer = new OldRecordWriterWrapper( + writerProvider.getRecordWriter( + connectorConfig.getHadoopConfiguration(), + tempFile, + record, + avroData + ) + ); + } else if (newWriterProvider != null) { + writer = newWriterProvider.getRecordWriter(connectorConfig, tempFile); + } else { + throw new ConnectException("Invalid state: either old or new RecordWriterProvider must be" + + " provided"); + } + } catch (IOException e) { + throw new ConnectException("Couldn't create RecordWriter", e); + } + writers.put(encodedPartition, writer); if (hiveIntegration && !hivePartitions.contains(encodedPartition)) { addHivePartition(encodedPartition); @@ -514,7 +562,7 @@ private void writeRecord(SinkRecord record) { } String encodedPartition = partitioner.encodePartition(record); - RecordWriter writer = getWriter(record, encodedPartition); + io.confluent.connect.storage.format.RecordWriter writer = getWriter(record, encodedPartition); writer.write(record); if (!startOffsets.containsKey(encodedPartition)) { @@ -526,7 +574,7 @@ private void writeRecord(SinkRecord record) { private void closeTempFile(String encodedPartition) { if (writers.containsKey(encodedPartition)) { - RecordWriter writer = writers.get(encodedPartition); + io.confluent.connect.storage.format.RecordWriter writer = writers.get(encodedPartition); writer.close(); writers.remove(encodedPartition); } diff --git a/src/main/java/io/confluent/connect/hdfs/avro/AvroFileReader.java b/src/main/java/io/confluent/connect/hdfs/avro/AvroFileReader.java index 56ffddaf4..85954e026 100644 --- a/src/main/java/io/confluent/connect/hdfs/avro/AvroFileReader.java +++ b/src/main/java/io/confluent/connect/hdfs/avro/AvroFileReader.java @@ -34,8 +34,8 @@ import io.confluent.connect.hdfs.HdfsSinkConnectorConfig; import io.confluent.connect.hdfs.SchemaFileReader; -public class AvroFileReader implements SchemaFileReader, - io.confluent.connect.storage.format.SchemaFileReader { +public class AvroFileReader + implements io.confluent.connect.storage.format.SchemaFileReader { private AvroData avroData; public AvroFileReader(AvroData avroData) { @@ -56,22 +56,6 @@ public Schema getSchema(HdfsSinkConnectorConfig conf, Path path) { } } - public Collection readData(HdfsSinkConnectorConfig conf, Path path) { - ArrayList collection = new ArrayList<>(); - try { - SeekableInput input = new FsInput(path, conf.getHadoopConfiguration()); - DatumReader reader = new GenericDatumReader<>(); - FileReader fileReader = DataFileReader.openReader(input, reader); - for (Object object : fileReader) { - collection.add(object); - } - fileReader.close(); - } catch (IOException e) { - throw new DataException(e); - } - return collection; - } - public boolean hasNext() { throw new UnsupportedOperationException(); } diff --git a/src/main/java/io/confluent/connect/hdfs/avro/AvroFormat.java b/src/main/java/io/confluent/connect/hdfs/avro/AvroFormat.java index 2fe03a34a..1be3ecd7a 100644 --- a/src/main/java/io/confluent/connect/hdfs/avro/AvroFormat.java +++ b/src/main/java/io/confluent/connect/hdfs/avro/AvroFormat.java @@ -17,19 +17,17 @@ import org.apache.hadoop.fs.Path; import io.confluent.connect.avro.AvroData; -import io.confluent.connect.hdfs.Format; import io.confluent.connect.hdfs.HdfsSinkConnectorConfig; -import io.confluent.connect.hdfs.RecordWriterProvider; -import io.confluent.connect.hdfs.SchemaFileReader; -import io.confluent.connect.hdfs.hive.HiveMetaStore; -import io.confluent.connect.hdfs.hive.HiveUtil; import io.confluent.connect.hdfs.storage.HdfsStorage; +import io.confluent.connect.storage.format.RecordWriterProvider; +import io.confluent.connect.storage.format.SchemaFileReader; import io.confluent.connect.storage.hive.HiveFactory; -public class AvroFormat implements Format, - io.confluent.connect.storage.format.Format { +public class AvroFormat + implements io.confluent.connect.storage.format.Format { private final AvroData avroData; + // DO NOT change this signature, it is required for instantiation via reflection public AvroFormat(HdfsStorage storage) { this.avroData = new AvroData( storage.conf().getInt(HdfsSinkConnectorConfig.SCHEMA_CACHE_SIZE_CONFIG) @@ -37,30 +35,15 @@ public AvroFormat(HdfsStorage storage) { } @Override - public RecordWriterProvider getRecordWriterProvider() { + public RecordWriterProvider getRecordWriterProvider() { return new AvroRecordWriterProvider(avroData); } @Override - public SchemaFileReader getSchemaFileReader() { + public SchemaFileReader getSchemaFileReader() { return new AvroFileReader(avroData); } - @Override - public SchemaFileReader getSchemaFileReader(AvroData avroData) { - // Argument is ignored. - return getSchemaFileReader(); - } - - @Deprecated - @Override - public HiveUtil getHiveUtil( - HdfsSinkConnectorConfig config, - HiveMetaStore hiveMetaStore - ) { - return (HiveUtil) getHiveFactory().createHiveUtil(config, hiveMetaStore); - } - @Override public HiveFactory getHiveFactory() { return new AvroHiveFactory(avroData); diff --git a/src/main/java/io/confluent/connect/hdfs/avro/AvroRecordWriterProvider.java b/src/main/java/io/confluent/connect/hdfs/avro/AvroRecordWriterProvider.java index 5c6e97b3e..667aeb8f2 100644 --- a/src/main/java/io/confluent/connect/hdfs/avro/AvroRecordWriterProvider.java +++ b/src/main/java/io/confluent/connect/hdfs/avro/AvroRecordWriterProvider.java @@ -29,12 +29,10 @@ import io.confluent.connect.avro.AvroData; import io.confluent.connect.hdfs.HdfsSinkConnectorConfig; -import io.confluent.connect.hdfs.RecordWriter; -import io.confluent.connect.hdfs.RecordWriterProvider; import io.confluent.kafka.serializers.NonRecordContainer; -public class AvroRecordWriterProvider implements RecordWriterProvider, - io.confluent.connect.storage.format.RecordWriterProvider { +public class AvroRecordWriterProvider + implements io.confluent.connect.storage.format.RecordWriterProvider { private static final Logger log = LoggerFactory.getLogger(AvroRecordWriterProvider.class); private final static String EXTENSION = ".avro"; private final AvroData avroData; @@ -49,21 +47,11 @@ public String getExtension() { } @Override - public RecordWriter getRecordWriter( - HdfsSinkConnectorConfig conf, - final String fileName, - SinkRecord record, - final AvroData avroData - ) { - return getRecordWriter(conf, fileName); - } - - @Override - public RecordWriter getRecordWriter( + public io.confluent.connect.storage.format.RecordWriter getRecordWriter( final HdfsSinkConnectorConfig conf, final String filename ) { - return new RecordWriter() { + return new io.confluent.connect.storage.format.RecordWriter() { final DataFileWriter writer = new DataFileWriter<>(new GenericDatumWriter<>()); final Path path = new Path(filename); Schema schema = null; diff --git a/src/main/java/io/confluent/connect/hdfs/hive/HiveUtil.java b/src/main/java/io/confluent/connect/hdfs/hive/HiveUtil.java index 49a4aa055..5d4e7a32c 100644 --- a/src/main/java/io/confluent/connect/hdfs/hive/HiveUtil.java +++ b/src/main/java/io/confluent/connect/hdfs/hive/HiveUtil.java @@ -22,6 +22,7 @@ import io.confluent.connect.hdfs.partitioner.Partitioner; import io.confluent.connect.storage.common.StorageCommonConfig; +// NOTE: DO NOT add or modify this class as it is maintained for compatibility @Deprecated public abstract class HiveUtil extends io.confluent.connect.storage.hive.HiveUtil { diff --git a/src/main/java/io/confluent/connect/hdfs/parquet/ParquetFileReader.java b/src/main/java/io/confluent/connect/hdfs/parquet/ParquetFileReader.java index b0c69ecc2..0e68ad364 100644 --- a/src/main/java/io/confluent/connect/hdfs/parquet/ParquetFileReader.java +++ b/src/main/java/io/confluent/connect/hdfs/parquet/ParquetFileReader.java @@ -15,25 +15,20 @@ package io.confluent.connect.hdfs.parquet; import org.apache.avro.generic.GenericRecord; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.kafka.connect.data.Schema; -import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.errors.DataException; import org.apache.parquet.avro.AvroReadSupport; import org.apache.parquet.hadoop.ParquetReader; import java.io.IOException; -import java.util.ArrayList; -import java.util.Collection; import java.util.Iterator; import io.confluent.connect.avro.AvroData; import io.confluent.connect.hdfs.HdfsSinkConnectorConfig; -import io.confluent.connect.hdfs.SchemaFileReader; -public class ParquetFileReader implements SchemaFileReader, - io.confluent.connect.storage.format.SchemaFileReader { +public class ParquetFileReader + implements io.confluent.connect.storage.format.SchemaFileReader { private AvroData avroData; public ParquetFileReader(AvroData avroData) { @@ -59,25 +54,6 @@ public Schema getSchema(HdfsSinkConnectorConfig conf, Path path) { } } - @Override - public Collection readData(HdfsSinkConnectorConfig conf, Path path) { - Collection result = new ArrayList<>(); - AvroReadSupport readSupport = new AvroReadSupport<>(); - ParquetReader.Builder builder = ParquetReader.builder(readSupport, path); - try { - ParquetReader parquetReader = builder.withConf(conf.getHadoopConfiguration()) - .build(); - GenericRecord record; - while ((record = parquetReader.read()) != null) { - result.add(record); - } - parquetReader.close(); - } catch (IOException e) { - throw new DataException(e); - } - return result; - } - public boolean hasNext() { throw new UnsupportedOperationException(); } diff --git a/src/main/java/io/confluent/connect/hdfs/parquet/ParquetFormat.java b/src/main/java/io/confluent/connect/hdfs/parquet/ParquetFormat.java index a4c2365cd..dd597f320 100644 --- a/src/main/java/io/confluent/connect/hdfs/parquet/ParquetFormat.java +++ b/src/main/java/io/confluent/connect/hdfs/parquet/ParquetFormat.java @@ -14,23 +14,20 @@ package io.confluent.connect.hdfs.parquet; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import io.confluent.connect.avro.AvroData; -import io.confluent.connect.hdfs.Format; import io.confluent.connect.hdfs.HdfsSinkConnectorConfig; -import io.confluent.connect.hdfs.RecordWriterProvider; -import io.confluent.connect.hdfs.SchemaFileReader; -import io.confluent.connect.hdfs.hive.HiveMetaStore; -import io.confluent.connect.hdfs.hive.HiveUtil; import io.confluent.connect.hdfs.storage.HdfsStorage; +import io.confluent.connect.storage.format.RecordWriterProvider; +import io.confluent.connect.storage.format.SchemaFileReader; import io.confluent.connect.storage.hive.HiveFactory; -public class ParquetFormat implements Format, - io.confluent.connect.storage.format.Format { +public class ParquetFormat + implements io.confluent.connect.storage.format.Format { private final AvroData avroData; + // DO NOT change this signature, it is required for instantiation via reflection public ParquetFormat(HdfsStorage storage) { this.avroData = new AvroData( storage.conf().getInt(HdfsSinkConnectorConfig.SCHEMA_CACHE_SIZE_CONFIG) @@ -38,31 +35,15 @@ public ParquetFormat(HdfsStorage storage) { } @Override - public RecordWriterProvider getRecordWriterProvider() { + public RecordWriterProvider getRecordWriterProvider() { return new ParquetRecordWriterProvider(avroData); } @Override - public SchemaFileReader getSchemaFileReader() { + public SchemaFileReader getSchemaFileReader() { return new ParquetFileReader(avroData); } - @Deprecated - @Override - public SchemaFileReader getSchemaFileReader(AvroData avroData) { - // Argument is ignored. - return getSchemaFileReader(); - } - - @Deprecated - @Override - public HiveUtil getHiveUtil( - HdfsSinkConnectorConfig config, - HiveMetaStore hiveMetaStore - ) { - return (HiveUtil) getHiveFactory().createHiveUtil(config, hiveMetaStore); - } - @Override public HiveFactory getHiveFactory() { return new ParquetHiveFactory(); diff --git a/src/main/java/io/confluent/connect/hdfs/parquet/ParquetRecordWriterProvider.java b/src/main/java/io/confluent/connect/hdfs/parquet/ParquetRecordWriterProvider.java index 2a639234c..77e2741cb 100644 --- a/src/main/java/io/confluent/connect/hdfs/parquet/ParquetRecordWriterProvider.java +++ b/src/main/java/io/confluent/connect/hdfs/parquet/ParquetRecordWriterProvider.java @@ -32,8 +32,8 @@ import io.confluent.connect.hdfs.RecordWriter; import io.confluent.connect.hdfs.RecordWriterProvider; -public class ParquetRecordWriterProvider implements RecordWriterProvider, - io.confluent.connect.storage.format.RecordWriterProvider { +public class ParquetRecordWriterProvider + implements io.confluent.connect.storage.format.RecordWriterProvider { private static final Logger log = LoggerFactory.getLogger(ParquetRecordWriterProvider.class); private final static String EXTENSION = ".parquet"; private final AvroData avroData; @@ -48,21 +48,11 @@ public String getExtension() { } @Override - public RecordWriter getRecordWriter( - HdfsSinkConnectorConfig conf, - final String filename, - SinkRecord record, - final AvroData avroData - ) { - return getRecordWriter(conf, filename); - } - - @Override - public RecordWriter getRecordWriter( + public io.confluent.connect.storage.format.RecordWriter getRecordWriter( final HdfsSinkConnectorConfig conf, final String filename ) { - return new RecordWriter() { + return new io.confluent.connect.storage.format.RecordWriter() { final CompressionCodecName compressionCodecName = CompressionCodecName.SNAPPY; final int blockSize = 256 * 1024 * 1024; final int pageSize = 64 * 1024; diff --git a/src/test/java/io/confluent/connect/hdfs/DataFileReader.java b/src/test/java/io/confluent/connect/hdfs/DataFileReader.java new file mode 100644 index 000000000..0c9724c92 --- /dev/null +++ b/src/test/java/io/confluent/connect/hdfs/DataFileReader.java @@ -0,0 +1,15 @@ +package io.confluent.connect.hdfs; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; + +import java.io.IOException; +import java.util.Collection; + +/** + * Interface that corresponds to SchemaFileReader but reads data objects. Only used to validate + * output during tests. + */ +public interface DataFileReader { + Collection readData(Configuration conf, Path path) throws IOException; +} diff --git a/src/test/java/io/confluent/connect/hdfs/FailureRecoveryTest.java b/src/test/java/io/confluent/connect/hdfs/FailureRecoveryTest.java index 8bbd6eafb..03018eb18 100644 --- a/src/test/java/io/confluent/connect/hdfs/FailureRecoveryTest.java +++ b/src/test/java/io/confluent/connect/hdfs/FailureRecoveryTest.java @@ -30,6 +30,7 @@ import io.confluent.connect.hdfs.utils.MemoryRecordWriter; import io.confluent.connect.hdfs.utils.MemoryStorage; import io.confluent.connect.storage.common.StorageCommonConfig; +import io.confluent.connect.storage.format.*; import static org.junit.Assert.assertEquals; @@ -115,7 +116,7 @@ public void testWriterFailureMultiPartitions() throws Exception { } String encodedPartition = "partition=" + String.valueOf(PARTITION); - Map writers = hdfsWriter.getWriters(TOPIC_PARTITION); + Map writers = hdfsWriter.getWriters(TOPIC_PARTITION); MemoryRecordWriter writer = (MemoryRecordWriter) writers.get(encodedPartition); writer.setFailure(MemoryRecordWriter.Failure.writeFailure); hdfsWriter.write(sinkRecords); @@ -183,7 +184,7 @@ public void testWriterFailure() throws Exception { } String encodedPartition = "partition=" + String.valueOf(PARTITION); - Map writers = hdfsWriter.getWriters(TOPIC_PARTITION); + Map writers = hdfsWriter.getWriters(TOPIC_PARTITION); MemoryRecordWriter writer = (MemoryRecordWriter) writers.get(encodedPartition); writer.setFailure(MemoryRecordWriter.Failure.writeFailure); diff --git a/src/test/java/io/confluent/connect/hdfs/FormatAPIDataWriterCompatibilityTest.java b/src/test/java/io/confluent/connect/hdfs/FormatAPIDataWriterCompatibilityTest.java new file mode 100644 index 000000000..2bcd0d4d5 --- /dev/null +++ b/src/test/java/io/confluent/connect/hdfs/FormatAPIDataWriterCompatibilityTest.java @@ -0,0 +1,75 @@ +package io.confluent.connect.hdfs; + +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.sink.SinkRecord; +import org.junit.Before; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Map; + +import io.confluent.connect.hdfs.avro.AvroDataFileReader; +import io.confluent.connect.hdfs.hive.HiveTestBase; +import io.confluent.connect.storage.hive.HiveConfig; + +/** + * Test to ensure we can still instantiate & use the old-style HDFS-only interfaces instead of + * those from storage-common and use them with DataWriter + */ +public class FormatAPIDataWriterCompatibilityTest extends HiveTestBase { + + @Before + public void setUp() throws Exception { + super.setUp(); + dataFileReader = new AvroDataFileReader(); + extension = ".avro"; + } + + + @Test + public void dataWriterNewFormatAPICompatibilityTest() { + DataWriter hdfsWriter = new DataWriter(connectorConfig, context, avroData); + + hdfsWriter.recover(TOPIC_PARTITION); + + String key = "key"; + Schema schema = createSchema(); + Struct record = createRecord(schema); + + Collection sinkRecords = new ArrayList<>(); + for (long offset = 0; offset < 7; offset++) { + SinkRecord sinkRecord = + new SinkRecord(TOPIC, PARTITION, Schema.STRING_SCHEMA, key, schema, record, offset); + sinkRecords.add(sinkRecord); + } + + hdfsWriter.write(sinkRecords); + hdfsWriter.close(); + hdfsWriter.stop(); + + Map props = createProps(); + props.put(HiveConfig.HIVE_INTEGRATION_CONFIG, "true"); + HdfsSinkConnectorConfig config = new HdfsSinkConnectorConfig(props); + + hdfsWriter = new DataWriter(config, context, avroData); + hdfsWriter.syncWithHive(); + + // Since we're not using a real format, we won't validate the output. However, this should at + // least exercise the code paths for the old Format class + + hdfsWriter.close(); + hdfsWriter.stop(); + } + + @Override + protected Map createProps() { + Map props = super.createProps(); + props.put(HdfsSinkConnectorConfig.FORMAT_CLASS_CONFIG, OldFormat.class.getName()); + // Enable Hive integration to make sure we exercise the paths that get HiveUtils + props.put(HiveConfig.HIVE_INTEGRATION_CONFIG, "true"); + return props; + } + +} diff --git a/src/test/java/io/confluent/connect/hdfs/FormatAPITopicPartitionWriterCompatibilityTest.java b/src/test/java/io/confluent/connect/hdfs/FormatAPITopicPartitionWriterCompatibilityTest.java new file mode 100644 index 000000000..78b9795c4 --- /dev/null +++ b/src/test/java/io/confluent/connect/hdfs/FormatAPITopicPartitionWriterCompatibilityTest.java @@ -0,0 +1,115 @@ +package io.confluent.connect.hdfs; + +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.sink.SinkRecord; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; +import java.util.Collection; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import io.confluent.connect.hdfs.avro.AvroDataFileReader; +import io.confluent.connect.hdfs.filter.CommittedFileFilter; +import io.confluent.connect.hdfs.partitioner.DefaultPartitioner; +import io.confluent.connect.hdfs.partitioner.Partitioner; +import io.confluent.connect.hdfs.storage.HdfsStorage; +import io.confluent.connect.storage.StorageFactory; +import io.confluent.connect.storage.common.StorageCommonConfig; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +/** + * Test to ensure we can still instantiate & use the old-style HDFS-only interfaces instead of + * those from storage-common and use them with TopicPartitionWriter + */ +public class FormatAPITopicPartitionWriterCompatibilityTest extends TestWithMiniDFSCluster { + private RecordWriterProvider writerProvider = null; + private io.confluent.connect.storage.format.RecordWriterProvider + newWriterProvider; + private HdfsStorage storage; + + @Override + protected Map createProps() { + return super.createProps(); + } + + @Before + public void setUp() throws Exception { + super.setUp(); + + @SuppressWarnings("unchecked") + Class storageClass = (Class) + connectorConfig.getClass(StorageCommonConfig.STORAGE_CLASS_CONFIG); + storage = StorageFactory.createStorage( + storageClass, + HdfsSinkConnectorConfig.class, + connectorConfig, + url + ); + + Format format = new OldFormat(); + writerProvider = format.getRecordWriterProvider(); + newWriterProvider = null; + dataFileReader = new AvroDataFileReader(); + extension = writerProvider.getExtension(); + createTopicDir(url, topicsDir, TOPIC); + createLogsDir(url, logsDir); + } + + @Test + public void testWriteRecordDefaultWithPadding() throws Exception { + Partitioner partitioner = new DefaultPartitioner(); + partitioner.configure(parsedConfig); + TopicPartitionWriter topicPartitionWriter = new TopicPartitionWriter( + TOPIC_PARTITION, + storage, + writerProvider, + newWriterProvider, + partitioner, + connectorConfig, + context, + avroData + ); + + Schema schema = createSchema(); + List records = createRecordBatches(schema, 3, 3); + // Add a single records at the end of the batches sequence. Total records: 10 + records.add(createRecord(schema)); + List sinkRecords = createSinkRecords(records, schema); + + for (SinkRecord record : sinkRecords) { + topicPartitionWriter.buffer(record); + } + + topicPartitionWriter.recover(); + topicPartitionWriter.write(); + topicPartitionWriter.close(); + + // No verification since the format is a dummy format. We're really just trying to exercise + // the old APIs and any paths that should hit them (and not NPE due to the variables for + // new-style formats being null) + } + + private void createTopicDir(String url, String topicsDir, String topic) throws IOException { + Path path = new Path(FileUtils.topicDirectory(url, topicsDir, topic)); + if (!fs.exists(path)) { + fs.mkdirs(path); + } + } + + private void createLogsDir(String url, String logsDir) throws IOException { + Path path = new Path(url + "/" + logsDir); + if (!fs.exists(path)) { + fs.mkdirs(path); + } + } + +} diff --git a/src/test/java/io/confluent/connect/hdfs/HdfsSinkTaskTest.java b/src/test/java/io/confluent/connect/hdfs/HdfsSinkTaskTest.java index 5dd9d9e1e..32ba8a035 100644 --- a/src/test/java/io/confluent/connect/hdfs/HdfsSinkTaskTest.java +++ b/src/test/java/io/confluent/connect/hdfs/HdfsSinkTaskTest.java @@ -29,6 +29,7 @@ import java.util.Map; import io.confluent.connect.avro.AvroData; +import io.confluent.connect.hdfs.avro.AvroDataFileReader; import io.confluent.connect.hdfs.avro.AvroFileReader; import io.confluent.connect.hdfs.storage.HdfsStorage; import io.confluent.connect.storage.StorageFactory; @@ -44,7 +45,7 @@ public class HdfsSinkTaskTest extends TestWithMiniDFSCluster { private static final String DIRECTORY2 = TOPIC + "/" + "partition=" + String.valueOf(PARTITION2); private static final String extension = ".avro"; private static final String ZERO_PAD_FMT = "%010d"; - private final SchemaFileReader schemaFileReader = new AvroFileReader(avroData); + private final DataFileReader schemaFileReader = new AvroDataFileReader(); @Test public void testSinkTaskStart() throws Exception { @@ -169,7 +170,7 @@ public void testSinkTaskPut() throws Exception { Path path = new Path(FileUtils.committedFileName(url, topicsDir, directory, tp, startOffset, endOffset, extension, ZERO_PAD_FMT)); - Collection records = schemaFileReader.readData(connectorConfig, path); + Collection records = schemaFileReader.readData(connectorConfig.getHadoopConfiguration(), path); long size = endOffset - startOffset + 1; assertEquals(records.size(), size); for (Object avroRecord : records) { @@ -211,7 +212,7 @@ final int record = 12; Path path = new Path(FileUtils.committedFileName(url, topicsDir, directory, tp, startOffset, endOffset, extension, ZERO_PAD_FMT)); - Collection records = schemaFileReader.readData(connectorConfig, path); + Collection records = schemaFileReader.readData(connectorConfig.getHadoopConfiguration(), path); long size = endOffset - startOffset + 1; assertEquals(records.size(), size); for (Object avroRecord : records) { diff --git a/src/test/java/io/confluent/connect/hdfs/HdfsSinkTaskTestWithSecureHDFS.java b/src/test/java/io/confluent/connect/hdfs/HdfsSinkTaskTestWithSecureHDFS.java index 83e2eddcc..7d18e88e2 100644 --- a/src/test/java/io/confluent/connect/hdfs/HdfsSinkTaskTestWithSecureHDFS.java +++ b/src/test/java/io/confluent/connect/hdfs/HdfsSinkTaskTestWithSecureHDFS.java @@ -26,6 +26,7 @@ import java.util.Map; import io.confluent.connect.avro.AvroData; +import io.confluent.connect.hdfs.avro.AvroDataFileReader; import io.confluent.connect.hdfs.avro.AvroFileReader; import static org.junit.Assert.assertEquals; @@ -34,7 +35,7 @@ public class HdfsSinkTaskTestWithSecureHDFS extends TestWithSecureMiniDFSCluster private static final String extension = ".avro"; private static final String ZERO_PAD_FMT = "%010d"; - private final SchemaFileReader schemaFileReader = new AvroFileReader(avroData); + private final DataFileReader schemaFileReader = new AvroDataFileReader(); @Test public void testSinkTaskPut() throws Exception { @@ -70,7 +71,7 @@ public void testSinkTaskPut() throws Exception { Path path = new Path(FileUtils.committedFileName(url, topicsDir, directory, tp, startOffset, endOffset, extension, ZERO_PAD_FMT)); - Collection records = schemaFileReader.readData(connectorConfig, path); + Collection records = schemaFileReader.readData(connectorConfig.getHadoopConfiguration(), path); long size = endOffset - startOffset + 1; assertEquals(records.size(), size); for (Object avroRecord : records) { diff --git a/src/test/java/io/confluent/connect/hdfs/OldFormat.java b/src/test/java/io/confluent/connect/hdfs/OldFormat.java new file mode 100644 index 000000000..962e5d44a --- /dev/null +++ b/src/test/java/io/confluent/connect/hdfs/OldFormat.java @@ -0,0 +1,86 @@ +package io.confluent.connect.hdfs; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.sink.SinkRecord; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Collection; + +import io.confluent.connect.avro.AvroData; +import io.confluent.connect.hdfs.hive.HiveMetaStore; +import io.confluent.connect.hdfs.hive.HiveUtil; +import io.confluent.connect.hdfs.partitioner.Partitioner; + +// Be careful if you need to edit anything in here. The Format, RecordWriterProvider, +// RecordWriter, SchemaFileReader, and HiveUtil classes need to remain compatible. +public class OldFormat implements Format { + + // DO NOT add any other constructors. No-arg constructor must be accessible + public OldFormat() { + + } + + @Override + public RecordWriterProvider getRecordWriterProvider() { + return new RecordWriterProvider() { + @Override + public String getExtension() { + return ".fake"; + } + + @Override + public RecordWriter getRecordWriter( + Configuration conf, + String fileName, + SinkRecord record, + AvroData avroData + ) throws IOException { + return new RecordWriter() { + @Override + public void write(SinkRecord value) throws IOException { + // Intentionally empty + } + + @Override + public void close() throws IOException { + // Intentionally empty + } + }; + } + }; + } + + @Override + public SchemaFileReader getSchemaFileReader(AvroData avroData) { + return new SchemaFileReader() { + @Override + public Schema getSchema(Configuration conf, Path path) throws IOException { + return Schema.INT32_SCHEMA; + } + + @Override + public Collection readData(Configuration conf, Path path) throws IOException { + return Arrays.asList((Object) 1, 2, 3); + } + }; + } + + @Override + public HiveUtil getHiveUtil(HdfsSinkConnectorConfig config, HiveMetaStore hiveMetaStore) { + return new HiveUtil(config, hiveMetaStore) { + @Override + public void createTable(String database, String tableName, Schema schema, + Partitioner partitioner) { + // Intentionally empty + } + + @Override + public void alterSchema(String s, String s1, Schema schema) { + // Intentionally empty + } + }; + } +} diff --git a/src/test/java/io/confluent/connect/hdfs/TestWithMiniDFSCluster.java b/src/test/java/io/confluent/connect/hdfs/TestWithMiniDFSCluster.java index 4e17c8ce7..d8996d34b 100644 --- a/src/test/java/io/confluent/connect/hdfs/TestWithMiniDFSCluster.java +++ b/src/test/java/io/confluent/connect/hdfs/TestWithMiniDFSCluster.java @@ -51,7 +51,7 @@ public class TestWithMiniDFSCluster extends HdfsSinkConnectorTestBase { protected MiniDFSCluster cluster; protected FileSystem fs; - protected SchemaFileReader schemaFileReader; + protected DataFileReader dataFileReader; protected Partitioner partitioner; protected String extension; // The default based on default configuration of 10 @@ -269,7 +269,7 @@ protected void verify(List sinkRecords, long[] validOffsets, Set records = schemaFileReader.readData(connectorConfig, path); + Collection records = dataFileReader.readData(connectorConfig.getHadoopConfiguration(), path); long size = endOffset - startOffset + 1; assertEquals(size, records.size()); diff --git a/src/test/java/io/confluent/connect/hdfs/avro/AvroDataFileReader.java b/src/test/java/io/confluent/connect/hdfs/avro/AvroDataFileReader.java new file mode 100644 index 000000000..0af444098 --- /dev/null +++ b/src/test/java/io/confluent/connect/hdfs/avro/AvroDataFileReader.java @@ -0,0 +1,30 @@ +package io.confluent.connect.hdfs.avro; + +import org.apache.avro.file.FileReader; +import org.apache.avro.file.SeekableInput; +import org.apache.avro.generic.GenericDatumReader; +import org.apache.avro.io.DatumReader; +import org.apache.avro.mapred.FsInput; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; + +import io.confluent.connect.hdfs.DataFileReader; + +public class AvroDataFileReader implements DataFileReader { + @Override + public Collection readData(Configuration conf, Path path) throws IOException { + ArrayList collection = new ArrayList<>(); + SeekableInput input = new FsInput(path, conf); + DatumReader reader = new GenericDatumReader<>(); + FileReader fileReader = org.apache.avro.file.DataFileReader.openReader(input, reader); + for (Object object: fileReader) { + collection.add(object); + } + fileReader.close(); + return collection; + } +} diff --git a/src/test/java/io/confluent/connect/hdfs/avro/DataWriterAvroTest.java b/src/test/java/io/confluent/connect/hdfs/avro/DataWriterAvroTest.java index 4c04f7b39..ceaf4b37c 100644 --- a/src/test/java/io/confluent/connect/hdfs/avro/DataWriterAvroTest.java +++ b/src/test/java/io/confluent/connect/hdfs/avro/DataWriterAvroTest.java @@ -45,7 +45,7 @@ public class DataWriterAvroTest extends TestWithMiniDFSCluster { @Before public void setUp() throws Exception { super.setUp(); - schemaFileReader = new AvroFileReader(avroData); + dataFileReader = new AvroDataFileReader(); extension = ".avro"; } diff --git a/src/test/java/io/confluent/connect/hdfs/avro/TopicPartitionWriterTest.java b/src/test/java/io/confluent/connect/hdfs/avro/TopicPartitionWriterTest.java index a3e80a99b..6b8d76fe1 100644 --- a/src/test/java/io/confluent/connect/hdfs/avro/TopicPartitionWriterTest.java +++ b/src/test/java/io/confluent/connect/hdfs/avro/TopicPartitionWriterTest.java @@ -24,7 +24,6 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Collection; -import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -53,7 +52,9 @@ import static org.junit.Assert.assertTrue; public class TopicPartitionWriterTest extends TestWithMiniDFSCluster { - private RecordWriterProvider writerProvider; + private RecordWriterProvider writerProvider = null; + private io.confluent.connect.storage.format.RecordWriterProvider + newWriterProvider; private HdfsStorage storage; private Map localProps = new HashMap<>(); @@ -78,13 +79,16 @@ public void setUp() throws Exception { url ); @SuppressWarnings("unchecked") - Class formatClass = (Class) connectorConfig.getClass( + Class formatClass + = (Class) connectorConfig.getClass( HdfsSinkConnectorConfig.FORMAT_CLASS_CONFIG ); - Format format = formatClass.getConstructor(HdfsStorage.class).newInstance(storage); - writerProvider = format.getRecordWriterProvider(); - schemaFileReader = format.getSchemaFileReader(avroData); - extension = writerProvider.getExtension(); + io.confluent.connect.storage.format.Format format + = formatClass.getConstructor(HdfsStorage.class).newInstance(storage); + writerProvider = null; + newWriterProvider = format.getRecordWriterProvider(); + dataFileReader = new AvroDataFileReader(); + extension = newWriterProvider.getExtension(); createTopicDir(url, topicsDir, TOPIC); createLogsDir(url, logsDir); } @@ -99,6 +103,7 @@ public void testWriteRecordDefaultWithPadding() throws Exception { TOPIC_PARTITION, storage, writerProvider, + newWriterProvider, partitioner, connectorConfig, context, @@ -144,6 +149,7 @@ public void testWriteRecordFieldPartitioner() throws Exception { TOPIC_PARTITION, storage, writerProvider, + newWriterProvider, partitioner, connectorConfig, context, @@ -194,6 +200,7 @@ public void testWriteRecordTimeBasedPartition() throws Exception { TOPIC_PARTITION, storage, writerProvider, + newWriterProvider, partitioner, connectorConfig, context, @@ -256,7 +263,7 @@ private void verify(Set expectedFiles, int expectedSize, List reco for (FileStatus status : statuses) { Path filePath = status.getPath(); assertTrue(expectedFiles.contains(status.getPath())); - Collection avroRecords = schemaFileReader.readData(connectorConfig, filePath); + Collection avroRecords = dataFileReader.readData(connectorConfig.getHadoopConfiguration(), filePath); assertEquals(expectedSize, avroRecords.size()); for (Object avroRecord : avroRecords) { assertEquals(avroData.fromConnectData(schema, records.get(index++)), avroRecord); diff --git a/src/test/java/io/confluent/connect/hdfs/parquet/DataWriterParquetTest.java b/src/test/java/io/confluent/connect/hdfs/parquet/DataWriterParquetTest.java index 5ed7df97b..33bef5d44 100644 --- a/src/test/java/io/confluent/connect/hdfs/parquet/DataWriterParquetTest.java +++ b/src/test/java/io/confluent/connect/hdfs/parquet/DataWriterParquetTest.java @@ -25,14 +25,13 @@ import io.confluent.connect.hdfs.DataWriter; import io.confluent.connect.hdfs.HdfsSinkConnectorConfig; import io.confluent.connect.hdfs.TestWithMiniDFSCluster; -import io.confluent.connect.hdfs.partitioner.Partitioner; public class DataWriterParquetTest extends TestWithMiniDFSCluster { @Before public void setUp() throws Exception { super.setUp(); - schemaFileReader = new ParquetFileReader(avroData); + dataFileReader = new ParquetDataFileReader(); extension = ".parquet"; } diff --git a/src/test/java/io/confluent/connect/hdfs/parquet/ParquetDataFileReader.java b/src/test/java/io/confluent/connect/hdfs/parquet/ParquetDataFileReader.java new file mode 100644 index 000000000..32f71e7d7 --- /dev/null +++ b/src/test/java/io/confluent/connect/hdfs/parquet/ParquetDataFileReader.java @@ -0,0 +1,29 @@ +package io.confluent.connect.hdfs.parquet; + +import org.apache.avro.generic.GenericRecord; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.parquet.avro.AvroReadSupport; +import org.apache.parquet.hadoop.ParquetReader; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; + +import io.confluent.connect.hdfs.DataFileReader; + +public class ParquetDataFileReader implements DataFileReader { + @Override + public Collection readData(Configuration conf, Path path) throws IOException { + Collection result = new ArrayList<>(); + AvroReadSupport readSupport = new AvroReadSupport<>(); + ParquetReader.Builder builder = ParquetReader.builder(readSupport, path); + ParquetReader parquetReader = builder.withConf(conf).build(); + GenericRecord record; + while ((record = parquetReader.read()) != null) { + result.add(record); + } + parquetReader.close(); + return result; + } +} diff --git a/src/test/java/io/confluent/connect/hdfs/utils/MemoryFormat.java b/src/test/java/io/confluent/connect/hdfs/utils/MemoryFormat.java index 2e0a07338..a283f46e5 100644 --- a/src/test/java/io/confluent/connect/hdfs/utils/MemoryFormat.java +++ b/src/test/java/io/confluent/connect/hdfs/utils/MemoryFormat.java @@ -1,43 +1,27 @@ package io.confluent.connect.hdfs.utils; +import org.apache.hadoop.fs.Path; import org.apache.kafka.common.config.AbstractConfig; -import io.confluent.connect.avro.AvroData; -import io.confluent.connect.hdfs.Format; import io.confluent.connect.hdfs.HdfsSinkConnectorConfig; -import io.confluent.connect.hdfs.RecordWriterProvider; -import io.confluent.connect.hdfs.SchemaFileReader; -import io.confluent.connect.hdfs.hive.HiveMetaStore; -import io.confluent.connect.hdfs.hive.HiveUtil; import io.confluent.connect.hdfs.storage.HdfsStorage; import io.confluent.connect.storage.hive.HiveFactory; -public class MemoryFormat implements Format { - private final AvroData avroData; +public class MemoryFormat + implements io.confluent.connect.storage.format.Format { + // DO NOT change this signature, it is required for instantiation via reflection public MemoryFormat(HdfsStorage storage) { - this.avroData = new AvroData( - storage.conf().getInt(HdfsSinkConnectorConfig.SCHEMA_CACHE_SIZE_CONFIG) - ); } @Override - public RecordWriterProvider getRecordWriterProvider() { + public io.confluent.connect.storage.format.RecordWriterProvider getRecordWriterProvider() { return new MemoryRecordWriterProvider(); } @Override - public SchemaFileReader getSchemaFileReader(AvroData avroData) { - return null; - } - - @Override - public SchemaFileReader getSchemaFileReader() { - return null; - } - - @Override - public HiveUtil getHiveUtil(HdfsSinkConnectorConfig config, HiveMetaStore hiveMetaStore) { + public io.confluent.connect.storage.format.SchemaFileReader + getSchemaFileReader() { return null; } diff --git a/src/test/java/io/confluent/connect/hdfs/utils/MemoryRecordWriter.java b/src/test/java/io/confluent/connect/hdfs/utils/MemoryRecordWriter.java index 94723204d..13fe5ef3b 100644 --- a/src/test/java/io/confluent/connect/hdfs/utils/MemoryRecordWriter.java +++ b/src/test/java/io/confluent/connect/hdfs/utils/MemoryRecordWriter.java @@ -22,7 +22,7 @@ import io.confluent.connect.hdfs.RecordWriter; -public class MemoryRecordWriter implements RecordWriter { +public class MemoryRecordWriter implements io.confluent.connect.storage.format.RecordWriter { private String filename; private static final Map> data = Data.getData(); private Failure failure = Failure.noFailure; diff --git a/src/test/java/io/confluent/connect/hdfs/utils/MemoryRecordWriterProvider.java b/src/test/java/io/confluent/connect/hdfs/utils/MemoryRecordWriterProvider.java index 2f77b3734..a4a5f0b9f 100644 --- a/src/test/java/io/confluent/connect/hdfs/utils/MemoryRecordWriterProvider.java +++ b/src/test/java/io/confluent/connect/hdfs/utils/MemoryRecordWriterProvider.java @@ -16,19 +16,14 @@ package io.confluent.connect.hdfs.utils; -import org.apache.hadoop.conf.Configuration; -import org.apache.kafka.connect.sink.SinkRecord; - import java.util.LinkedList; import java.util.List; import java.util.Map; -import io.confluent.connect.avro.AvroData; import io.confluent.connect.hdfs.HdfsSinkConnectorConfig; -import io.confluent.connect.hdfs.RecordWriter; -import io.confluent.connect.hdfs.RecordWriterProvider; -public class MemoryRecordWriterProvider implements RecordWriterProvider { +public class MemoryRecordWriterProvider + implements io.confluent.connect.storage.format.RecordWriterProvider { @Override public String getExtension() { @@ -36,17 +31,7 @@ public String getExtension() { } @Override - public RecordWriter getRecordWriter( - HdfsSinkConnectorConfig conf, - final String filename, - SinkRecord record, - final AvroData avroData - ) { - return getRecordWriter(conf, filename); - } - - @Override - public RecordWriter getRecordWriter( + public io.confluent.connect.storage.format.RecordWriter getRecordWriter( HdfsSinkConnectorConfig conf, final String filename ) { From 0e99eab37d3913b9ea97ba6f442a1b6611d98a2f Mon Sep 17 00:00:00 2001 From: Ewen Cheslack-Postava Date: Tue, 1 Aug 2017 16:07:23 -0700 Subject: [PATCH 54/58] Fix license reformatting --- .../io/confluent/connect/hdfs/HdfsSinkConnectorConfig.java | 5 ----- .../java/io/confluent/connect/hdfs/hive/HiveMetaStore.java | 6 +++--- 2 files changed, 3 insertions(+), 8 deletions(-) diff --git a/src/main/java/io/confluent/connect/hdfs/HdfsSinkConnectorConfig.java b/src/main/java/io/confluent/connect/hdfs/HdfsSinkConnectorConfig.java index 402c148e8..9c0405100 100644 --- a/src/main/java/io/confluent/connect/hdfs/HdfsSinkConnectorConfig.java +++ b/src/main/java/io/confluent/connect/hdfs/HdfsSinkConnectorConfig.java @@ -296,11 +296,6 @@ public Configuration getHadoopConfiguration() { return hadoopConfig; } - // Visible for testing. - public void setHadoopConfiguration(Configuration conf) { - hadoopConfig = conf; - } - public Map plainValues() { Map map = new HashMap<>(); for (AbstractConfig config : allConfigs) { diff --git a/src/main/java/io/confluent/connect/hdfs/hive/HiveMetaStore.java b/src/main/java/io/confluent/connect/hdfs/hive/HiveMetaStore.java index 7eb17b6f1..fd1bbb987 100644 --- a/src/main/java/io/confluent/connect/hdfs/hive/HiveMetaStore.java +++ b/src/main/java/io/confluent/connect/hdfs/hive/HiveMetaStore.java @@ -1,12 +1,12 @@ /** * Copyright 2015 Confluent Inc. - *

+ * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. * You may obtain a copy of the License at - *

+ * * http://www.apache.org/licenses/LICENSE-2.0 - *

+ * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. From dc3f6377dc1060456d4ce404901aca9ea6e87d37 Mon Sep 17 00:00:00 2001 From: Ewen Cheslack-Postava Date: Tue, 1 Aug 2017 16:12:09 -0700 Subject: [PATCH 55/58] Remove HiveSchemaConverter since the one from storage-common is now used. --- .../hdfs/hive/HiveSchemaConverter.java | 19 ---- .../connect/hdfs/parquet/ParquetHiveUtil.java | 1 - .../hdfs/hive/HiveSchemaConverterTest.java | 102 ------------------ 3 files changed, 122 deletions(-) delete mode 100644 src/main/java/io/confluent/connect/hdfs/hive/HiveSchemaConverter.java delete mode 100644 src/test/java/io/confluent/connect/hdfs/hive/HiveSchemaConverterTest.java diff --git a/src/main/java/io/confluent/connect/hdfs/hive/HiveSchemaConverter.java b/src/main/java/io/confluent/connect/hdfs/hive/HiveSchemaConverter.java deleted file mode 100644 index 369b0655d..000000000 --- a/src/main/java/io/confluent/connect/hdfs/hive/HiveSchemaConverter.java +++ /dev/null @@ -1,19 +0,0 @@ -/** - * Copyright 2015 Confluent Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except - * in compliance with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software distributed under the License - * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express - * or implied. See the License for the specific language governing permissions and limitations under - * the License. - **/ - -package io.confluent.connect.hdfs.hive; - -@Deprecated -public class HiveSchemaConverter extends io.confluent.connect.storage.hive.HiveSchemaConverter { -} diff --git a/src/main/java/io/confluent/connect/hdfs/parquet/ParquetHiveUtil.java b/src/main/java/io/confluent/connect/hdfs/parquet/ParquetHiveUtil.java index 43f76b6df..fbf74b23d 100644 --- a/src/main/java/io/confluent/connect/hdfs/parquet/ParquetHiveUtil.java +++ b/src/main/java/io/confluent/connect/hdfs/parquet/ParquetHiveUtil.java @@ -23,7 +23,6 @@ import java.util.List; -import io.confluent.connect.avro.AvroData; import io.confluent.connect.hdfs.HdfsSinkConnectorConfig; import io.confluent.connect.hdfs.hive.HiveMetaStore; import io.confluent.connect.hdfs.hive.HiveUtil; diff --git a/src/test/java/io/confluent/connect/hdfs/hive/HiveSchemaConverterTest.java b/src/test/java/io/confluent/connect/hdfs/hive/HiveSchemaConverterTest.java deleted file mode 100644 index c4dfe7525..000000000 --- a/src/test/java/io/confluent/connect/hdfs/hive/HiveSchemaConverterTest.java +++ /dev/null @@ -1,102 +0,0 @@ -/** - * Copyright 2015 Confluent Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except - * in compliance with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software distributed under the License - * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express - * or implied. See the License for the specific language governing permissions and limitations under - * the License. - **/ - -package io.confluent.connect.hdfs.hive; - -import org.apache.hadoop.hive.metastore.api.FieldSchema; -import org.apache.hadoop.hive.serde2.typeinfo.StructTypeInfo; -import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; -import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; -import org.apache.kafka.connect.data.Schema; -import org.apache.kafka.connect.data.SchemaBuilder; -import org.junit.Test; - -import java.util.ArrayList; -import java.util.List; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - -public class HiveSchemaConverterTest { - - private static final Schema SIMPLE_STRUCT = SchemaBuilder.struct().name("SimpleStruct") - .field("id", Schema.INT32_SCHEMA) - .field("name", Schema.STRING_SCHEMA) - .build(); - - private static final Schema COMPLEX_STRUCT = SchemaBuilder.struct().name("ComplexStruct") - .field("groupName", Schema.STRING_SCHEMA) - .field("simpleStructs", SchemaBuilder.array(SIMPLE_STRUCT).build()) - .build(); - - - @Test - public void testConvertSimpleStruct() { - TypeInfo type = HiveSchemaConverter.convert(SIMPLE_STRUCT); - assertTrue(type instanceof StructTypeInfo); - - List expectedFieldNames = new ArrayList<>(); - expectedFieldNames.add("id"); - expectedFieldNames.add("name"); - - assertEquals(expectedFieldNames, ((StructTypeInfo) type).getAllStructFieldNames()); - } - - @Test - public void testConvertComplexStruct() { - List fields = HiveSchemaConverter.convertSchema(COMPLEX_STRUCT); - List expectedFieldNames = new ArrayList<>(); - expectedFieldNames.add("groupName"); - expectedFieldNames.add("simpleStructs"); - - List actualFieldNames = new ArrayList<>(); - for (FieldSchema fieldSchema: fields) { - actualFieldNames.add(fieldSchema.getName()); - } - assertEquals(expectedFieldNames, actualFieldNames); - - List expectedTypes = new ArrayList<>(); - List typeInfos = new ArrayList<>(); - typeInfos.add(TypeInfoFactory.intTypeInfo); - typeInfos.add(TypeInfoFactory.stringTypeInfo); - - expectedTypes.add(TypeInfoFactory.stringTypeInfo.toString()); - - List expectedInnerFieldNames = new ArrayList<>(); - expectedInnerFieldNames.add("id"); - expectedInnerFieldNames.add("name"); - TypeInfo structType = TypeInfoFactory.getStructTypeInfo(expectedInnerFieldNames, typeInfos); - - expectedTypes.add(TypeInfoFactory.getListTypeInfo(structType).toString()); - - List actualTypes = new ArrayList<>(); - for (FieldSchema fieldSchema: fields) { - actualTypes.add(fieldSchema.getType()); - } - assertEquals(expectedTypes, actualTypes); - } - - @Test - public void testConvertArray() { - TypeInfo type = HiveSchemaConverter.convert(SchemaBuilder.array(Schema.FLOAT32_SCHEMA)); - assertEquals(TypeInfoFactory.getListTypeInfo(TypeInfoFactory.floatTypeInfo), type); - } - - @Test - public void testConvertMap() { - TypeInfo type = HiveSchemaConverter.convert(SchemaBuilder.map(Schema.STRING_SCHEMA, Schema.FLOAT64_SCHEMA)); - assertEquals(TypeInfoFactory.getMapTypeInfo(TypeInfoFactory.stringTypeInfo, TypeInfoFactory.doubleTypeInfo), type); - - } -} From ce353ac7ad549322b995f5b95a0d1ac637160776 Mon Sep 17 00:00:00 2001 From: Ewen Cheslack-Postava Date: Tue, 1 Aug 2017 16:27:28 -0700 Subject: [PATCH 56/58] Minor cleanup in FailureRecoveryTest --- .../io/confluent/connect/hdfs/FailureRecoveryTest.java | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/test/java/io/confluent/connect/hdfs/FailureRecoveryTest.java b/src/test/java/io/confluent/connect/hdfs/FailureRecoveryTest.java index 03018eb18..81ca75270 100644 --- a/src/test/java/io/confluent/connect/hdfs/FailureRecoveryTest.java +++ b/src/test/java/io/confluent/connect/hdfs/FailureRecoveryTest.java @@ -38,6 +38,11 @@ public class FailureRecoveryTest extends HdfsSinkConnectorTestBase { private static final String ZERO_PAD_FMT = "%010d"; private static final String extension = ""; + @Before + public void setUp() throws Exception { + super.setUp(); + } + @Override protected Map createProps() { Map props = super.createProps(); @@ -48,8 +53,6 @@ protected Map createProps() { @Test public void testCommitFailure() throws Exception { - setUp(); - String key = "key"; Schema schema = createSchema(); Struct record = createRecord(schema); @@ -89,8 +92,6 @@ public void testCommitFailure() throws Exception { @Test public void testWriterFailureMultiPartitions() throws Exception { - setUp(); - String key = "key"; Schema schema = createSchema(); Struct record = createRecord(schema); @@ -164,7 +165,6 @@ public void testWriterFailureMultiPartitions() throws Exception { @Test public void testWriterFailure() throws Exception { - setUp(); HdfsSinkConnectorConfig connectorConfig = new HdfsSinkConnectorConfig(properties); String key = "key"; From 51f460bd2486ece5126a85a8914f112a2d4b5ca9 Mon Sep 17 00:00:00 2001 From: Ewen Cheslack-Postava Date: Wed, 2 Aug 2017 20:16:38 -0700 Subject: [PATCH 57/58] Fix TimestampPartitioner Hive tests. --- .../connect/hdfs/TopicPartitionWriter.java | 18 +++++++++++++++--- .../hdfs/HdfsSinkConnectorTestBase.java | 4 ---- .../hdfs/avro/HiveIntegrationAvroTest.java | 5 ++++- .../hdfs/avro/TopicPartitionWriterTest.java | 1 - .../hdfs/partitioner/DailyPartitionerTest.java | 10 ---------- .../partitioner/HourlyPartitionerTest.java | 10 ---------- .../partitioner/TimeBasedPartitionerTest.java | 10 ---------- 7 files changed, 19 insertions(+), 39 deletions(-) diff --git a/src/main/java/io/confluent/connect/hdfs/TopicPartitionWriter.java b/src/main/java/io/confluent/connect/hdfs/TopicPartitionWriter.java index 07da9063d..145c5b2e6 100644 --- a/src/main/java/io/confluent/connect/hdfs/TopicPartitionWriter.java +++ b/src/main/java/io/confluent/connect/hdfs/TopicPartitionWriter.java @@ -666,7 +666,11 @@ private void createHiveTable() { Future future = executorService.submit(new Callable() { @Override public Void call() throws HiveMetaStoreException { - hive.createTable(hiveDatabase, tp.topic(), currentSchema, partitioner); + try { + hive.createTable(hiveDatabase, tp.topic(), currentSchema, partitioner); + } catch (Throwable e) { + log.error("Creating Hive table threw unexpected error", e); + } return null; } }); @@ -677,7 +681,11 @@ private void alterHiveSchema() { Future future = executorService.submit(new Callable() { @Override public Void call() throws HiveMetaStoreException { - hive.alterSchema(hiveDatabase, tp.topic(), currentSchema); + try { + hive.alterSchema(hiveDatabase, tp.topic(), currentSchema); + } catch (Throwable e) { + log.error("Altering Hive schema threw unexpected error", e); + } return null; } }); @@ -688,7 +696,11 @@ private void addHivePartition(final String location) { Future future = executorService.submit(new Callable() { @Override public Void call() throws Exception { - hiveMetaStore.addPartition(hiveDatabase, tp.topic(), location); + try { + hiveMetaStore.addPartition(hiveDatabase, tp.topic(), location); + } catch (Throwable e) { + log.error("Adding Hive partition threw unexpected error", e); + } return null; } }); diff --git a/src/test/java/io/confluent/connect/hdfs/HdfsSinkConnectorTestBase.java b/src/test/java/io/confluent/connect/hdfs/HdfsSinkConnectorTestBase.java index 329adefb3..142712935 100644 --- a/src/test/java/io/confluent/connect/hdfs/HdfsSinkConnectorTestBase.java +++ b/src/test/java/io/confluent/connect/hdfs/HdfsSinkConnectorTestBase.java @@ -64,10 +64,6 @@ protected Map createProps() { PartitionerConfig.PARTITIONER_CLASS_CONFIG, DefaultPartitioner.class.getName() ); - props.put( - PartitionerConfig.SCHEMA_GENERATOR_CLASS_CONFIG, - DefaultSchemaGenerator.class.getName() - ); props.put(PartitionerConfig.PARTITION_FIELD_NAME_CONFIG, "int"); props.put( PartitionerConfig.PARTITION_DURATION_MS_CONFIG, diff --git a/src/test/java/io/confluent/connect/hdfs/avro/HiveIntegrationAvroTest.java b/src/test/java/io/confluent/connect/hdfs/avro/HiveIntegrationAvroTest.java index 37690074a..2ba50294f 100644 --- a/src/test/java/io/confluent/connect/hdfs/avro/HiveIntegrationAvroTest.java +++ b/src/test/java/io/confluent/connect/hdfs/avro/HiveIntegrationAvroTest.java @@ -353,7 +353,10 @@ public void testHiveIntegrationTimeBasedPartitionerAvro() throws Exception { } } - String result = HiveTestUtils.runHive(hiveExec, "SELECT * FROM " + TOPIC); + String result = HiveTestUtils.runHive( + hiveExec, + "SELECT * FROM " + hiveMetaStore.tableNameConverter(TOPIC) + ); String[] rows = result.split("\n"); assertEquals(9, rows.length); for (int i = 0; i < rows.length; ++i) { diff --git a/src/test/java/io/confluent/connect/hdfs/avro/TopicPartitionWriterTest.java b/src/test/java/io/confluent/connect/hdfs/avro/TopicPartitionWriterTest.java index 6b8d76fe1..88d58f6b7 100644 --- a/src/test/java/io/confluent/connect/hdfs/avro/TopicPartitionWriterTest.java +++ b/src/test/java/io/confluent/connect/hdfs/avro/TopicPartitionWriterTest.java @@ -193,7 +193,6 @@ public void testWriteRecordFieldPartitioner() throws Exception { public void testWriteRecordTimeBasedPartition() throws Exception { setUp(); Partitioner partitioner = new TimeBasedPartitioner(); - parsedConfig.put(PartitionerConfig.SCHEMA_GENERATOR_CLASS_CONFIG, TimeBasedSchemaGenerator.class); partitioner.configure(parsedConfig); TopicPartitionWriter topicPartitionWriter = new TopicPartitionWriter( diff --git a/src/test/java/io/confluent/connect/hdfs/partitioner/DailyPartitionerTest.java b/src/test/java/io/confluent/connect/hdfs/partitioner/DailyPartitionerTest.java index e736d9834..c80e31d88 100644 --- a/src/test/java/io/confluent/connect/hdfs/partitioner/DailyPartitionerTest.java +++ b/src/test/java/io/confluent/connect/hdfs/partitioner/DailyPartitionerTest.java @@ -32,16 +32,6 @@ public class DailyPartitionerTest extends TestWithMiniDFSCluster { private static final long partitionDurationMs = TimeUnit.HOURS.toMillis(24); - @Override - protected Map createProps() { - Map props = super.createProps(); - props.put( - PartitionerConfig.SCHEMA_GENERATOR_CLASS_CONFIG, - TimeBasedSchemaGenerator.class.getName() - ); - return props; - } - @Test public void testDailyPartitioner() throws Exception { setUp(); diff --git a/src/test/java/io/confluent/connect/hdfs/partitioner/HourlyPartitionerTest.java b/src/test/java/io/confluent/connect/hdfs/partitioner/HourlyPartitionerTest.java index 94f90b9e9..e541b17f0 100644 --- a/src/test/java/io/confluent/connect/hdfs/partitioner/HourlyPartitionerTest.java +++ b/src/test/java/io/confluent/connect/hdfs/partitioner/HourlyPartitionerTest.java @@ -32,16 +32,6 @@ public class HourlyPartitionerTest extends TestWithMiniDFSCluster { private static final long partitionDurationMs = TimeUnit.HOURS.toMillis(1); - @Override - protected Map createProps() { - Map props = super.createProps(); - props.put( - PartitionerConfig.SCHEMA_GENERATOR_CLASS_CONFIG, - TimeBasedSchemaGenerator.class.getName() - ); - return props; - } - @Test public void testHourlyPartitioner() throws Exception { setUp(); diff --git a/src/test/java/io/confluent/connect/hdfs/partitioner/TimeBasedPartitionerTest.java b/src/test/java/io/confluent/connect/hdfs/partitioner/TimeBasedPartitionerTest.java index fdaeaa214..3f765a131 100644 --- a/src/test/java/io/confluent/connect/hdfs/partitioner/TimeBasedPartitionerTest.java +++ b/src/test/java/io/confluent/connect/hdfs/partitioner/TimeBasedPartitionerTest.java @@ -34,16 +34,6 @@ public class TimeBasedPartitionerTest extends HdfsSinkConnectorTestBase { private static final String timeZoneString = "America/Los_Angeles"; private static final DateTimeZone DATE_TIME_ZONE = DateTimeZone.forID(timeZoneString); - @Override - protected Map createProps() { - Map props = super.createProps(); - props.put( - PartitionerConfig.SCHEMA_GENERATOR_CLASS_CONFIG, - TimeBasedSchemaGenerator.class.getName() - ); - return props; - } - @Test public void testGeneratePartitionedPath() throws Exception { setUp(); From cff5bde4a7b6e70377e2d91acb05cd593a4edb52 Mon Sep 17 00:00:00 2001 From: Konstantine Karantasis Date: Tue, 8 Aug 2017 17:00:47 -0700 Subject: [PATCH 58/58] Remove redundant overrides from WAL interface. --- .../io/confluent/connect/hdfs/wal/WAL.java | 22 ------------------- 1 file changed, 22 deletions(-) diff --git a/src/main/java/io/confluent/connect/hdfs/wal/WAL.java b/src/main/java/io/confluent/connect/hdfs/wal/WAL.java index 0e8094151..6845a18af 100644 --- a/src/main/java/io/confluent/connect/hdfs/wal/WAL.java +++ b/src/main/java/io/confluent/connect/hdfs/wal/WAL.java @@ -16,28 +16,6 @@ package io.confluent.connect.hdfs.wal; -import org.apache.kafka.connect.errors.ConnectException; - @Deprecated public interface WAL extends io.confluent.connect.storage.wal.WAL { - String beginMarker = "BEGIN"; - String endMarker = "END"; - - @Override - void acquireLease() throws ConnectException; - - @Override - void append(String tempFile, String committedFile) throws ConnectException; - - @Override - void apply() throws ConnectException; - - @Override - void truncate() throws ConnectException; - - @Override - void close() throws ConnectException; - - @Override - String getLogFile(); }