1
0

Implement HoodieLogFormat replacing Avro as the default log format

This commit is contained in:
Prasanna Rajaperumal
2017-05-17 12:52:36 -07:00
committed by vinoth chandar
parent 3c984447da
commit 240c91241b
30 changed files with 1790 additions and 1260 deletions

View File

@@ -21,37 +21,30 @@ import com.esotericsoftware.kryo.io.Input;
import com.esotericsoftware.kryo.io.Output;
import com.esotericsoftware.kryo.serializers.JavaSerializer;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.uber.hoodie.common.table.HoodieTableConfig;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.log.HoodieLogAppendConfig;
import com.uber.hoodie.common.table.log.HoodieLogFile;
import com.uber.hoodie.common.table.log.avro.AvroLogAppender;
import com.uber.hoodie.common.table.log.avro.RollingAvroLogAppender;
import com.uber.hoodie.common.table.log.HoodieLogFormat;
import com.uber.hoodie.common.table.log.HoodieLogFormat.Writer;
import com.uber.hoodie.common.table.log.block.HoodieAvroDataBlock;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.HoodieAvroUtils;
import com.uber.hoodie.common.util.SchemaTestUtil;
import com.uber.hoodie.exception.HoodieException;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
import org.apache.commons.lang3.tuple.Pair;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.LocatedFileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.RemoteIterator;
import org.apache.jute.Index;
import org.junit.rules.TemporaryFolder;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.File;
import java.io.IOException;
import java.io.ObjectInputStream;
import java.io.ObjectOutputStream;
import java.io.Serializable;
import java.text.SimpleDateFormat;
import java.util.Date;
@@ -60,10 +53,8 @@ import java.util.List;
import java.util.Map;
import java.util.Properties;
import java.util.UUID;
import java.util.function.Function;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import java.util.stream.StreamSupport;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.fail;
@@ -181,17 +172,16 @@ public class HoodieTestUtils {
HoodieRecordLocation location = s.getKey();
String partitionPath = s.getValue().get(0).getPartitionPath();
HoodieLogAppendConfig logConfig = null;
Writer logWriter;
try {
logConfig = HoodieLogAppendConfig.newBuilder()
.onPartitionPath(new Path(basePath, partitionPath))
.withLogFileExtension(HoodieLogFile.DELTA_EXTENSION)
logWriter = HoodieLogFormat.newWriterBuilder()
.onParentPath(new Path(basePath, partitionPath))
.withFileExtension(HoodieLogFile.DELTA_EXTENSION)
.withFileId(location.getFileId())
.withBaseCommitTime(location.getCommitTime())
.withSchema(schema).withFs(fs).build();
.overBaseCommit(location.getCommitTime())
.withFs(fs).build();
AvroLogAppender log = new AvroLogAppender(logConfig);
log.append(s.getValue().stream().map(r -> {
logWriter.appendBlock(new HoodieAvroDataBlock(s.getValue().stream().map(r -> {
try {
GenericRecord val = (GenericRecord) r.getData().getInsertValue(schema).get();
HoodieAvroUtils.addHoodieKeyToRecord(val,
@@ -202,8 +192,8 @@ public class HoodieTestUtils {
} catch (IOException e) {
return null;
}
}).collect(Collectors.toList()).iterator());
log.close();
}).collect(Collectors.toList()), schema));
logWriter.close();
} catch (Exception e) {
fail(e.toString());
}

View File

@@ -0,0 +1,580 @@
/*
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
*
* 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 com.uber.hoodie.common.table.log;
import com.uber.hoodie.common.minicluster.MiniClusterUtil;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.table.log.HoodieLogFormat.Reader;
import com.uber.hoodie.common.table.log.HoodieLogFormat.Writer;
import com.uber.hoodie.common.table.log.block.HoodieAvroDataBlock;
import com.uber.hoodie.common.table.log.block.HoodieCommandBlock;
import com.uber.hoodie.common.table.log.block.HoodieCommandBlock.HoodieCommandBlockTypeEnum;
import com.uber.hoodie.common.table.log.block.HoodieDeleteBlock;
import com.uber.hoodie.common.table.log.block.HoodieLogBlock;
import com.uber.hoodie.common.table.log.block.HoodieLogBlock.HoodieLogBlockType;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.HoodieAvroUtils;
import com.uber.hoodie.common.util.SchemaTestUtil;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashSet;
import java.util.Set;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
import java.io.IOException;
import java.net.URISyntaxException;
import java.util.List;
import java.util.stream.Collectors;
import static com.uber.hoodie.common.util.SchemaTestUtil.getSimpleSchema;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
@SuppressWarnings("Duplicates")
public class HoodieLogFormatTest {
private FileSystem fs;
private Path partitionPath;
@BeforeClass
public static void setUpClass() throws IOException, InterruptedException {
// Append is not supported in LocalFileSystem. HDFS needs to be setup.
MiniClusterUtil.setUp();
}
@AfterClass
public static void tearDownClass() {
MiniClusterUtil.shutdown();
}
@Before
public void setUp() throws IOException, InterruptedException {
this.fs = MiniClusterUtil.fileSystem;
TemporaryFolder folder = new TemporaryFolder();
folder.create();
assertTrue(fs.mkdirs(new Path(folder.getRoot().getPath())));
this.partitionPath = new Path(folder.getRoot().getPath());
}
@After
public void tearDown() throws IOException {
fs.delete(partitionPath, true);
}
@Test
public void testEmptyLog() throws IOException, InterruptedException {
Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath)
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
.overBaseCommit("100").withFs(fs).build();
assertEquals("Just created this log, size should be 0", 0, writer.getCurrentSize());
assertTrue("Check all log files should start with a .",
writer.getLogFile().getFileName().startsWith("."));
assertEquals("Version should be 1 for new log created", 1,
writer.getLogFile().getLogVersion());
}
@Test
public void testBasicAppend() throws IOException, InterruptedException, URISyntaxException {
Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath)
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
.overBaseCommit("100").withFs(fs).build();
List<IndexedRecord> records = SchemaTestUtil.generateTestRecords(0, 100);
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records,
getSimpleSchema());
writer = writer.appendBlock(dataBlock);
long size = writer.getCurrentSize();
assertTrue("We just wrote a block - size should be > 0", size > 0);
assertEquals(
"Write should be auto-flushed. The size reported by FileStatus and the writer should match",
size, fs.getFileStatus(writer.getLogFile().getPath()).getLen());
writer.close();
}
@Test
public void testRollover() throws IOException, InterruptedException, URISyntaxException {
Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath)
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
.overBaseCommit("100").withFs(fs).build();
List<IndexedRecord> records = SchemaTestUtil.generateTestRecords(0, 100);
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records,
getSimpleSchema());
// Write out a block
writer = writer.appendBlock(dataBlock);
// Get the size of the block
long size = writer.getCurrentSize();
writer.close();
// Create a writer with the size threshold as the size we just wrote - so this has to roll
writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath)
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
.overBaseCommit("100").withFs(fs).withSizeThreshold(size - 1).build();
records = SchemaTestUtil.generateTestRecords(0, 100);
dataBlock = new HoodieAvroDataBlock(records,
getSimpleSchema());
writer = writer.appendBlock(dataBlock);
assertEquals("This should be a new log file and hence size should be 0", 0,
writer.getCurrentSize());
assertEquals("Version should be rolled to 2", 2, writer.getLogFile().getLogVersion());
writer.close();
}
@Test
public void testMultipleAppend() throws IOException, URISyntaxException, InterruptedException {
Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath)
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
.overBaseCommit("100").withFs(fs).build();
List<IndexedRecord> records = SchemaTestUtil.generateTestRecords(0, 100);
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records,
getSimpleSchema());
writer = writer.appendBlock(dataBlock);
long size1 = writer.getCurrentSize();
writer.close();
writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath)
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
.overBaseCommit("100").withFs(fs).build();
records = SchemaTestUtil.generateTestRecords(0, 100);
dataBlock = new HoodieAvroDataBlock(records,
getSimpleSchema());
writer = writer.appendBlock(dataBlock);
long size2 = writer.getCurrentSize();
assertTrue("We just wrote a new block - size2 should be > size1", size2 > size1);
assertEquals(
"Write should be auto-flushed. The size reported by FileStatus and the writer should match",
size2, fs.getFileStatus(writer.getLogFile().getPath()).getLen());
writer.close();
// Close and Open again and append 100 more records
writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath)
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
.overBaseCommit("100").withFs(fs).build();
records = SchemaTestUtil.generateTestRecords(0, 100);
dataBlock = new HoodieAvroDataBlock(records,
getSimpleSchema());
writer = writer.appendBlock(dataBlock);
long size3 = writer.getCurrentSize();
assertTrue("We just wrote a new block - size3 should be > size2", size3 > size2);
assertEquals(
"Write should be auto-flushed. The size reported by FileStatus and the writer should match",
size3, fs.getFileStatus(writer.getLogFile().getPath()).getLen());
writer.close();
// Cannot get the current size after closing the log
try {
writer.getCurrentSize();
fail("getCurrentSize should fail after the logAppender is closed");
} catch (IllegalStateException e) {
// pass
}
}
@Test
public void testLeaseRecovery() throws IOException, URISyntaxException, InterruptedException {
Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath)
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
.overBaseCommit("100").withFs(fs).build();
List<IndexedRecord> records = SchemaTestUtil.generateTestRecords(0, 100);
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records,
getSimpleSchema());
writer = writer.appendBlock(dataBlock);
long size1 = writer.getCurrentSize();
// do not close this writer - this simulates a data note appending to a log dying without closing the file
// writer.close();
writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath)
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
.overBaseCommit("100").withFs(fs).build();
records = SchemaTestUtil.generateTestRecords(0, 100);
dataBlock = new HoodieAvroDataBlock(records,
getSimpleSchema());
writer = writer.appendBlock(dataBlock);
long size2 = writer.getCurrentSize();
assertTrue("We just wrote a new block - size2 should be > size1", size2 > size1);
assertEquals(
"Write should be auto-flushed. The size reported by FileStatus and the writer should match",
size2, fs.getFileStatus(writer.getLogFile().getPath()).getLen());
writer.close();
}
@SuppressWarnings("unchecked")
@Test
public void testBasicWriteAndScan()
throws IOException, URISyntaxException, InterruptedException {
Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath)
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
.overBaseCommit("100").withFs(fs).build();
List<IndexedRecord> records = SchemaTestUtil.generateTestRecords(0, 100);
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records,
getSimpleSchema());
writer = writer.appendBlock(dataBlock);
writer.close();
Reader reader = HoodieLogFormat.newReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema());
assertTrue("We wrote a block, we should be able to read it", reader.hasNext());
HoodieLogBlock nextBlock = reader.next();
assertEquals("The next block should be a data block", HoodieLogBlockType.AVRO_DATA_BLOCK,
nextBlock.getBlockType());
HoodieAvroDataBlock dataBlockRead = (HoodieAvroDataBlock) nextBlock;
assertEquals("Read records size should be equal to the written records size",
records.size(), dataBlockRead.getRecords().size());
assertEquals("Both records lists should be the same. (ordering guaranteed)", records,
dataBlockRead.getRecords());
}
@SuppressWarnings("unchecked")
@Test
public void testBasicAppendAndRead()
throws IOException, URISyntaxException, InterruptedException {
Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath)
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
.overBaseCommit("100").withFs(fs).build();
List<IndexedRecord> records1 = SchemaTestUtil.generateTestRecords(0, 100);
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1,
getSimpleSchema());
writer = writer.appendBlock(dataBlock);
writer.close();
writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath)
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
.overBaseCommit("100").withFs(fs).build();
List<IndexedRecord> records2 = SchemaTestUtil.generateTestRecords(0, 100);
dataBlock = new HoodieAvroDataBlock(records2,
getSimpleSchema());
writer = writer.appendBlock(dataBlock);
writer.close();
// Close and Open again and append 100 more records
writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath)
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
.overBaseCommit("100").withFs(fs).build();
List<IndexedRecord> records3 = SchemaTestUtil.generateTestRecords(0, 100);
dataBlock = new HoodieAvroDataBlock(records3,
getSimpleSchema());
writer = writer.appendBlock(dataBlock);
writer.close();
Reader reader = HoodieLogFormat.newReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema());
assertTrue("First block should be available", reader.hasNext());
HoodieLogBlock nextBlock = reader.next();
HoodieAvroDataBlock dataBlockRead = (HoodieAvroDataBlock) nextBlock;
assertEquals("Read records size should be equal to the written records size",
records1.size(), dataBlockRead.getRecords().size());
assertEquals("Both records lists should be the same. (ordering guaranteed)", records1,
dataBlockRead.getRecords());
nextBlock = reader.next();
dataBlockRead = (HoodieAvroDataBlock) nextBlock;
assertEquals("Read records size should be equal to the written records size",
records2.size(), dataBlockRead.getRecords().size());
assertEquals("Both records lists should be the same. (ordering guaranteed)", records2,
dataBlockRead.getRecords());
nextBlock = reader.next();
dataBlockRead = (HoodieAvroDataBlock) nextBlock;
assertEquals("Read records size should be equal to the written records size",
records3.size(), dataBlockRead.getRecords().size());
assertEquals("Both records lists should be the same. (ordering guaranteed)", records3,
dataBlockRead.getRecords());
}
@Test
public void testAppendAndReadOnCorruptedLog()
throws IOException, URISyntaxException, InterruptedException {
Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath)
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
.overBaseCommit("100").withFs(fs).build();
List<IndexedRecord> records = SchemaTestUtil.generateTestRecords(0, 100);
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records,
getSimpleSchema());
writer = writer.appendBlock(dataBlock);
writer.close();
// Append some arbit byte[] to thee end of the log (mimics a partially written commit)
fs = FileSystem.get(fs.getConf());
FSDataOutputStream outputStream = fs.append(writer.getLogFile().getPath());
// create a block with
outputStream.write(HoodieLogFormat.MAGIC);
outputStream.writeInt(HoodieLogBlockType.AVRO_DATA_BLOCK.ordinal());
// Write out a length that does not confirm with the content
outputStream.writeInt(100);
outputStream.write("something-random".getBytes());
outputStream.flush();
outputStream.close();
// First round of reads - we should be able to read the first block and then EOF
Reader reader = HoodieLogFormat.newReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema());
assertTrue("First block should be available", reader.hasNext());
reader.next();
assertTrue("We should have corrupted block next", reader.hasNext());
HoodieLogBlock block = reader.next();
assertEquals("The read block should be a corrupt block", HoodieLogBlockType.CORRUPT_BLOCK,
block.getBlockType());
assertEquals("", "something-random", new String(block.getBytes()));
assertFalse("There should be no more block left", reader.hasNext());
// Simulate another failure back to back
outputStream = fs.append(writer.getLogFile().getPath());
// create a block with
outputStream.write(HoodieLogFormat.MAGIC);
outputStream.writeInt(HoodieLogBlockType.AVRO_DATA_BLOCK.ordinal());
// Write out a length that does not confirm with the content
outputStream.writeInt(100);
outputStream.write("something-else-random".getBytes());
outputStream.flush();
outputStream.close();
// Should be able to append a new block
writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath)
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
.overBaseCommit("100").withFs(fs).build();
records = SchemaTestUtil.generateTestRecords(0, 100);
dataBlock = new HoodieAvroDataBlock(records, getSimpleSchema());
writer = writer.appendBlock(dataBlock);
writer.close();
// Second round of reads - we should be able to read the first and last block
reader = HoodieLogFormat.newReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema());
assertTrue("First block should be available", reader.hasNext());
reader.next();
assertTrue("We should get the 1st corrupted block next", reader.hasNext());
reader.next();
assertTrue("We should get the 2nd corrupted block next", reader.hasNext());
block = reader.next();
assertEquals("The read block should be a corrupt block", HoodieLogBlockType.CORRUPT_BLOCK,
block.getBlockType());
assertEquals("", "something-else-random", new String(block.getBytes()));
assertTrue("We should get the last block next", reader.hasNext());
reader.next();
assertFalse("We should have no more blocks left", reader.hasNext());
}
@Test
public void testAvroLogRecordReaderBasic()
throws IOException, URISyntaxException, InterruptedException {
Schema schema = HoodieAvroUtils.addMetadataFields(getSimpleSchema());
// Set a small threshold so that every block is a new version
Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath)
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
.overBaseCommit("100").withFs(fs).withSizeThreshold(500).build();
// Write 1
List<IndexedRecord> records1 = SchemaTestUtil.generateHoodieTestRecords(0, 100);
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, schema);
writer = writer.appendBlock(dataBlock);
// Write 2
List<IndexedRecord> records2 = SchemaTestUtil.generateHoodieTestRecords(0, 100);
dataBlock = new HoodieAvroDataBlock(records2, schema);
writer = writer.appendBlock(dataBlock);
writer.close();
List<String> allLogFiles = FSUtils
.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, "100")
.map(s -> s.getPath().toString())
.collect(Collectors.toList());
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, allLogFiles,
schema);
assertEquals("", 200, scanner.getTotalLogRecords());
Set<String> readKeys = new HashSet<>(200);
scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey()));
assertEquals("Stream collect should return all 200 records", 200, readKeys.size());
records1.addAll(records2);
Set<String> originalKeys = records1.stream()
.map(s -> ((GenericRecord) s).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString())
.collect(
Collectors.toSet());
assertEquals("CompositeAvroLogReader should return 200 records from 2 versions", originalKeys,
readKeys);
}
@Test
public void testAvroLogRecordReaderWithRollbackTombstone()
throws IOException, URISyntaxException, InterruptedException {
Schema schema = HoodieAvroUtils.addMetadataFields(getSimpleSchema());
// Set a small threshold so that every block is a new version
Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath)
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
.overBaseCommit("100").withFs(fs).build();
// Write 1
List<IndexedRecord> records1 = SchemaTestUtil.generateHoodieTestRecords(0, 100);
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, schema);
writer = writer.appendBlock(dataBlock);
// Write 2
List<IndexedRecord> records2 = SchemaTestUtil.generateHoodieTestRecords(0, 100);
dataBlock = new HoodieAvroDataBlock(records2, schema);
writer = writer.appendBlock(dataBlock);
// Rollback the last write
HoodieCommandBlock commandBlock = new HoodieCommandBlock(
HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK);
writer = writer.appendBlock(commandBlock);
// Write 3
List<IndexedRecord> records3 = SchemaTestUtil.generateHoodieTestRecords(0, 100);
dataBlock = new HoodieAvroDataBlock(records3, schema);
writer = writer.appendBlock(dataBlock);
writer.close();
List<String> allLogFiles = FSUtils
.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, "100")
.map(s -> s.getPath().toString())
.collect(Collectors.toList());
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, allLogFiles,
schema);
assertEquals("We still would read 300 records, but only 200 of them are valid", 300,
scanner.getTotalLogRecords());
Set<String> readKeys = new HashSet<>(200);
scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey()));
assertEquals("Stream collect should return all 200 records", 200, readKeys.size());
records1.addAll(records3);
Set<String> originalKeys = records1.stream()
.map(s -> ((GenericRecord) s).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString())
.collect(
Collectors.toSet());
assertEquals("CompositeAvroLogReader should return 200 records from 2 versions", originalKeys,
readKeys);
}
@Test
public void testAvroLogRecordReaderWithRollbackPartialBlock()
throws IOException, URISyntaxException, InterruptedException {
Schema schema = HoodieAvroUtils.addMetadataFields(getSimpleSchema());
// Set a small threshold so that every block is a new version
Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath)
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
.overBaseCommit("100").withFs(fs).build();
// Write 1
List<IndexedRecord> records1 = SchemaTestUtil.generateHoodieTestRecords(0, 100);
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, schema);
writer = writer.appendBlock(dataBlock);
writer.close();
// Append some arbit byte[] to thee end of the log (mimics a partially written commit)
fs = FileSystem.get(fs.getConf());
FSDataOutputStream outputStream = fs.append(writer.getLogFile().getPath());
// create a block with
outputStream.write(HoodieLogFormat.MAGIC);
outputStream.writeInt(HoodieLogBlockType.AVRO_DATA_BLOCK.ordinal());
// Write out a length that does not confirm with the content
outputStream.writeInt(100);
outputStream.write("something-random".getBytes());
outputStream.flush();
outputStream.close();
// Rollback the last write
HoodieCommandBlock commandBlock = new HoodieCommandBlock(
HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK);
writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath)
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
.overBaseCommit("100").withFs(fs).build();
writer = writer.appendBlock(commandBlock);
// Write 3
List<IndexedRecord> records3 = SchemaTestUtil.generateHoodieTestRecords(0, 100);
dataBlock = new HoodieAvroDataBlock(records3, schema);
writer = writer.appendBlock(dataBlock);
writer.close();
List<String> allLogFiles = FSUtils
.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, "100")
.map(s -> s.getPath().toString())
.collect(Collectors.toList());
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, allLogFiles,
schema);
assertEquals("We would read 200 records", 200,
scanner.getTotalLogRecords());
Set<String> readKeys = new HashSet<>(200);
scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey()));
assertEquals("Stream collect should return all 200 records", 200, readKeys.size());
records1.addAll(records3);
Set<String> originalKeys = records1.stream()
.map(s -> ((GenericRecord) s).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString())
.collect(
Collectors.toSet());
assertEquals("CompositeAvroLogReader should return 200 records from 2 versions", originalKeys,
readKeys);
}
@Test
public void testAvroLogRecordReaderWithDelete()
throws IOException, URISyntaxException, InterruptedException {
Schema schema = HoodieAvroUtils.addMetadataFields(getSimpleSchema());
// Set a small threshold so that every block is a new version
Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath)
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
.overBaseCommit("100").withFs(fs).build();
// Write 1
List<IndexedRecord> records1 = SchemaTestUtil.generateHoodieTestRecords(0, 100);
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, schema);
writer = writer.appendBlock(dataBlock);
// Write 2
List<IndexedRecord> records2 = SchemaTestUtil.generateHoodieTestRecords(0, 100);
dataBlock = new HoodieAvroDataBlock(records2, schema);
writer = writer.appendBlock(dataBlock);
records1.addAll(records2);
List<String> originalKeys = records1.stream()
.map(s -> ((GenericRecord) s).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString())
.collect(
Collectors.toList());
// Delete 50 keys
List<String> deletedKeys = originalKeys.subList(0, 50);
HoodieDeleteBlock deleteBlock = new HoodieDeleteBlock(deletedKeys.toArray(new String[50]));
writer = writer.appendBlock(deleteBlock);
List<String> allLogFiles = FSUtils
.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, "100")
.map(s -> s.getPath().toString())
.collect(Collectors.toList());
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, allLogFiles,
schema);
assertEquals("We still would read 200 records", 200,
scanner.getTotalLogRecords());
List<String> readKeys = new ArrayList<>(200);
scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey()));
assertEquals("Stream collect should return all 200 records", 150, readKeys.size());
originalKeys.removeAll(deletedKeys);
Collections.sort(originalKeys);
Collections.sort(readKeys);
assertEquals("CompositeAvroLogReader should return 200 records from 2 versions", originalKeys,
readKeys);
}
}

View File

@@ -1,326 +0,0 @@
/*
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
*
* 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 com.uber.hoodie.common.table.log.avro;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.uber.hoodie.common.minicluster.MiniClusterUtil;
import com.uber.hoodie.common.table.log.HoodieLogAppendConfig;
import com.uber.hoodie.common.table.log.HoodieLogFile;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.SchemaTestUtil;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
import org.apache.commons.collections.IteratorUtils;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
import java.io.IOException;
import java.net.URISyntaxException;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.SortedMap;
import java.util.stream.Collectors;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
public class AvroLogAppenderTest {
private FileSystem fs;
private Path partitionPath;
@BeforeClass
public static void setUpClass() throws IOException, InterruptedException {
// Append is not supported in LocalFileSystem. HDFS needs to be setup.
MiniClusterUtil.setUp();
}
@AfterClass
public static void tearDownClass() {
MiniClusterUtil.shutdown();
}
@Before
public void setUp() throws IOException, InterruptedException {
this.fs = MiniClusterUtil.fileSystem;
TemporaryFolder folder = new TemporaryFolder();
folder.create();
assertTrue(fs.mkdirs(new Path(folder.getRoot().getPath())));
this.partitionPath = new Path(folder.getRoot().getPath());
}
@After
public void tearDown() throws IOException {
fs.delete(partitionPath, true);
}
@Test
public void testBasicAppend() throws IOException, URISyntaxException, InterruptedException {
HoodieLogAppendConfig logConfig =
HoodieLogAppendConfig.newBuilder().onPartitionPath(partitionPath)
.withLogFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
.withBaseCommitTime("100")
.withSchema(SchemaTestUtil.getSimpleSchema()).withFs(fs).build();
RollingAvroLogAppender logAppender = new RollingAvroLogAppender(logConfig);
logAppender.append(SchemaTestUtil.generateTestRecords(0, 100).iterator());
long size1 = logAppender.getCurrentSize();
assertTrue("", size1 > 0);
assertEquals("", size1, fs.getFileStatus(logConfig.getLogFile().getPath()).getLen());
logAppender.close();
// Close and Open again and append 100 more records
logAppender = new RollingAvroLogAppender(logConfig);
logAppender.append(SchemaTestUtil.generateTestRecords(100, 100).iterator());
long size2 = logAppender.getCurrentSize();
assertTrue("", size2 > size1);
assertEquals("", size2, fs.getFileStatus(logConfig.getLogFile().getPath()).getLen());
logAppender.close();
// Close and Open again and append 100 more records
logAppender = new RollingAvroLogAppender(logConfig);
logAppender.append(SchemaTestUtil.generateTestRecords(200, 100).iterator());
long size3 = logAppender.getCurrentSize();
assertTrue("", size3 > size2);
assertEquals("", size3, fs.getFileStatus(logConfig.getLogFile().getPath()).getLen());
logAppender.close();
// Cannot get the current size after closing the log
try {
logAppender.getCurrentSize();
fail("getCurrentSize should fail after the logAppender is closed");
} catch (IllegalStateException e) {
// pass
}
}
@Test
public void testLeaseRecovery() throws IOException, URISyntaxException, InterruptedException {
HoodieLogAppendConfig logConfig =
HoodieLogAppendConfig.newBuilder().onPartitionPath(partitionPath)
.withLogFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
.withBaseCommitTime("100")
.withSchema(SchemaTestUtil.getSimpleSchema()).withFs(fs).build();
RollingAvroLogAppender logAppender = new RollingAvroLogAppender(logConfig);
logAppender.append(SchemaTestUtil.generateTestRecords(0, 100).iterator());
// do not close this log appender
// logAppender.close();
// Try opening again and append 100 more records
logAppender = new RollingAvroLogAppender(logConfig);
logAppender.append(SchemaTestUtil.generateTestRecords(100, 100).iterator());
assertEquals("", logAppender.getCurrentSize(),
fs.getFileStatus(logConfig.getLogFile().getPath()).getLen());
logAppender.close();
}
@Test
public void testAppendOnCorruptedBlock()
throws IOException, URISyntaxException, InterruptedException {
HoodieLogAppendConfig logConfig =
HoodieLogAppendConfig.newBuilder().onPartitionPath(partitionPath)
.withLogFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
.withBaseCommitTime("100")
.withSchema(SchemaTestUtil.getSimpleSchema()).withFs(fs).build();
RollingAvroLogAppender logAppender = new RollingAvroLogAppender(logConfig);
logAppender.append(SchemaTestUtil.generateTestRecords(0, 100).iterator());
logAppender.close();
// Append some arbit byte[] to thee end of the log (mimics a partially written commit)
assertTrue(fs.exists(logConfig.getLogFile().getPath()));
fs = FileSystem.get(fs.getConf());
FSDataOutputStream outputStream =
fs.append(logConfig.getLogFile().getPath(), logConfig.getBufferSize());
outputStream.write("something-random".getBytes());
outputStream.flush();
outputStream.close();
logAppender = new RollingAvroLogAppender(logConfig);
logAppender.append(SchemaTestUtil.generateTestRecords(100, 100).iterator());
logAppender.close();
}
@SuppressWarnings("unchecked")
@Test
public void testBasicWriteAndRead()
throws IOException, URISyntaxException, InterruptedException {
HoodieLogAppendConfig logConfig =
HoodieLogAppendConfig.newBuilder().onPartitionPath(partitionPath)
.withLogFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
.withBaseCommitTime("100")
.withSchema(SchemaTestUtil.getSimpleSchema()).withFs(fs).build();
RollingAvroLogAppender logAppender = new RollingAvroLogAppender(logConfig);
long size1 = logAppender.getCurrentSize();
List<IndexedRecord> inputRecords = SchemaTestUtil.generateTestRecords(0, 100);
logAppender.append(inputRecords.iterator());
logAppender.close();
AvroLogReader logReader =
new AvroLogReader(logConfig.getLogFile(), fs, logConfig.getSchema());
List<GenericRecord> result = IteratorUtils.toList(logReader.readBlock(size1));
assertEquals("Random access should return 100 records", 100, result.size());
assertEquals("both lists should be the same. (ordering guaranteed)", inputRecords, result);
}
@SuppressWarnings("unchecked")
@Test
public void testBasicAppendAndRead()
throws IOException, URISyntaxException, InterruptedException {
HoodieLogAppendConfig logConfig =
HoodieLogAppendConfig.newBuilder().onPartitionPath(partitionPath)
.withLogFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
.withBaseCommitTime("100")
.withSchema(SchemaTestUtil.getSimpleSchema()).withFs(fs).build();
RollingAvroLogAppender logAppender = new RollingAvroLogAppender(logConfig);
logAppender.append(SchemaTestUtil.generateTestRecords(0, 100).iterator());
long size1 = logAppender.getCurrentSize();
logAppender.close();
// Close and Open again and append 100 more records
logAppender = new RollingAvroLogAppender(logConfig);
List<IndexedRecord> secondBatchInput = SchemaTestUtil.generateTestRecords(100, 100);
logAppender.append(secondBatchInput.iterator());
long size2 = logAppender.getCurrentSize();
logAppender.close();
// Close and Open again and append 100 more records
logAppender = new RollingAvroLogAppender(logConfig);
List<IndexedRecord> lastBatchInput = SchemaTestUtil.generateTestRecords(200, 100);
logAppender.append(lastBatchInput.iterator());
long size3 = logAppender.getCurrentSize();
logAppender.close();
AvroLogReader logReader =
new AvroLogReader(logConfig.getLogFile(), fs, logConfig.getSchema());
// Try to grab the middle block here
List<GenericRecord> secondBatch = IteratorUtils.toList(logReader.readBlock(size1));
assertEquals("Stream collect should return 100 records", 100, secondBatch.size());
assertEquals("Collected list should match the input list (ordering guaranteed)",
secondBatchInput, secondBatch);
// Try to grab the middle block here
List<GenericRecord> lastBatch = IteratorUtils.toList(logReader.readBlock(size2));
assertEquals("Stream collect should return 100 records", 100, secondBatch.size());
assertEquals("Collected list should match the input list (ordering guaranteed)",
lastBatchInput, lastBatch);
List<GenericRecord> imaginaryBatch = IteratorUtils.toList(logReader.readBlock(size3));
assertEquals("Stream collect should return 0 records", 0, imaginaryBatch.size());
}
@Test
public void testAppendAndReadOnCorruptedLog()
throws IOException, URISyntaxException, InterruptedException {
HoodieLogAppendConfig logConfig =
HoodieLogAppendConfig.newBuilder().onPartitionPath(partitionPath)
.withLogFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
.withBaseCommitTime("100")
.withSchema(SchemaTestUtil.getSimpleSchema()).withFs(fs).build();
RollingAvroLogAppender logAppender = new RollingAvroLogAppender(logConfig);
long size1 = logAppender.getCurrentSize();
logAppender.append(SchemaTestUtil.generateTestRecords(0, 100).iterator());
logAppender.close();
// Append some arbit byte[] to thee end of the log (mimics a partially written commit)
assertTrue(fs.exists(logConfig.getLogFile().getPath()));
fs = FileSystem.get(fs.getConf());
FSDataOutputStream outputStream =
fs.append(logConfig.getLogFile().getPath(), logConfig.getBufferSize());
outputStream.write("something-random".getBytes());
outputStream.flush();
outputStream.close();
logAppender = new RollingAvroLogAppender(logConfig);
long size2 = logAppender.getCurrentSize();
logAppender.append(SchemaTestUtil.generateTestRecords(100, 100).iterator());
logAppender.close();
AvroLogReader logReader =
new AvroLogReader(logConfig.getLogFile(), fs, logConfig.getSchema());
// Try to grab the middle block here
List<GenericRecord> secondBatch = IteratorUtils.toList(logReader.readBlock(size1));
assertEquals("Stream collect should return 100 records", 100, secondBatch.size());
// Try to grab the last block here
List<GenericRecord> lastBatch = IteratorUtils.toList(logReader.readBlock(size2));
assertEquals("Stream collect should return 100 records", 100, lastBatch.size());
}
@Test
public void testCompositeAvroLogReader()
throws IOException, URISyntaxException, InterruptedException {
// Set a small threshold so that every block is a new version
HoodieLogAppendConfig logConfig =
HoodieLogAppendConfig.newBuilder().onPartitionPath(partitionPath)
.withLogFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
.withBaseCommitTime("100")
.withSchema(SchemaTestUtil.getSimpleSchema()).withSizeThreshold(500).withFs(fs)
.build();
RollingAvroLogAppender logAppender = new RollingAvroLogAppender(logConfig);
long size1 = logAppender.getCurrentSize();
List<IndexedRecord> input1 = SchemaTestUtil.generateTestRecords(0, 100);
logAppender.append(input1.iterator());
logAppender.close();
// Need to rebuild config to set the latest version as path
logConfig = HoodieLogAppendConfig.newBuilder().onPartitionPath(partitionPath)
.withLogFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
.withBaseCommitTime("100")
.withSchema(SchemaTestUtil.getSimpleSchema()).withSizeThreshold(500).withFs(fs).build();
logAppender = new RollingAvroLogAppender(logConfig);
long size2 = logAppender.getCurrentSize();
List<IndexedRecord> input2 = SchemaTestUtil.generateTestRecords(100, 100);
logAppender.append(input2.iterator());
logAppender.close();
logConfig = HoodieLogAppendConfig.newBuilder().onPartitionPath(partitionPath)
.withLogFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
.withBaseCommitTime("100")
.withSchema(SchemaTestUtil.getSimpleSchema()).withSizeThreshold(500).withFs(fs).build();
List<HoodieLogFile> allLogFiles = FSUtils
.getAllLogFiles(fs, partitionPath, logConfig.getLogFile().getFileId(),
HoodieLogFile.DELTA_EXTENSION, logConfig.getLogFile().getBaseCommitTime())
.collect(Collectors.toList());
assertEquals("", 2, allLogFiles.size());
SortedMap<Integer, List<Long>> offsets = Maps.newTreeMap();
offsets.put(1, Lists.newArrayList(size1));
offsets.put(2, Lists.newArrayList(size2));
CompositeAvroLogReader reader =
new CompositeAvroLogReader(partitionPath, logConfig.getLogFile().getFileId(),
logConfig.getLogFile().getBaseCommitTime(), fs, logConfig.getSchema(),
HoodieLogFile.DELTA_EXTENSION);
Iterator<GenericRecord> results = reader.readBlocks(offsets);
List<GenericRecord> totalBatch = IteratorUtils.toList(results);
assertEquals("Stream collect should return all 200 records", 200, totalBatch.size());
input1.addAll(input2);
assertEquals("CompositeAvroLogReader should return 200 records from 2 versions", input1,
totalBatch);
}
}

View File

@@ -16,9 +16,12 @@
package com.uber.hoodie.common.util;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.exception.HoodieIOException;
import java.util.UUID;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericDatumReader;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
import org.apache.avro.io.DecoderFactory;
@@ -59,4 +62,18 @@ public class SchemaTestUtil {
}
}
public static List<IndexedRecord> generateHoodieTestRecords(int from, int limit)
throws IOException, URISyntaxException {
List<IndexedRecord> records = generateTestRecords(from, limit);
Schema hoodieFieldsSchema = HoodieAvroUtils.addMetadataFields(getSimpleSchema());
return records.stream()
.map(s -> HoodieAvroUtils.rewriteRecord((GenericRecord) s, hoodieFieldsSchema))
.map(p -> {
p.put(HoodieRecord.RECORD_KEY_METADATA_FIELD, UUID.randomUUID().toString());
p.put(HoodieRecord.PARTITION_PATH_METADATA_FIELD, "0000/00/00");
return p;
}).collect(
Collectors.toList());
}
}