Implement Merge on Read Storage (#76)
1. Create HoodieTable abstraction for commits and fileSystemView 2. HoodieMergeOnReadTable created 3. View is now always obtained from the table and the correct view based on the table type is returned
This commit is contained in:
committed by
Prasanna Rajaperumal
parent
11d2fd3428
commit
eb46e7c72b
@@ -20,6 +20,7 @@ import com.esotericsoftware.kryo.Kryo;
|
||||
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;
|
||||
@@ -32,12 +33,16 @@ 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;
|
||||
|
||||
@@ -58,6 +63,7 @@ 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;
|
||||
@@ -192,15 +198,28 @@ public class HoodieTestUtils {
|
||||
r.getRecordKey(),
|
||||
r.getPartitionPath(),
|
||||
"");
|
||||
return val;
|
||||
return (IndexedRecord) val;
|
||||
} catch (IOException e) {
|
||||
return null;
|
||||
}
|
||||
}).collect(Collectors.toList()));
|
||||
}).collect(Collectors.toList()).iterator());
|
||||
log.close();
|
||||
} catch (Exception e) {
|
||||
fail(e.toString());
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
public static FileStatus[] listAllDataFilesInPath(FileSystem fs, String basePath)
|
||||
throws IOException {
|
||||
RemoteIterator<LocatedFileStatus> itr = fs.listFiles(new Path(basePath), true);
|
||||
List<FileStatus> returns = Lists.newArrayList();
|
||||
while(itr.hasNext()) {
|
||||
LocatedFileStatus status = itr.next();
|
||||
if(status.getPath().getName().contains(".parquet")) {
|
||||
returns.add(status);
|
||||
}
|
||||
}
|
||||
return returns.toArray(new FileStatus[returns.size()]);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -85,7 +85,7 @@ public class AvroLogAppenderTest {
|
||||
.withBaseCommitTime("100")
|
||||
.withSchema(SchemaTestUtil.getSimpleSchema()).withFs(fs).build();
|
||||
RollingAvroLogAppender logAppender = new RollingAvroLogAppender(logConfig);
|
||||
logAppender.append(SchemaTestUtil.generateTestRecords(0, 100));
|
||||
logAppender.append(SchemaTestUtil.generateTestRecords(0, 100).iterator());
|
||||
long size1 = logAppender.getCurrentSize();
|
||||
assertTrue("", size1 > 0);
|
||||
assertEquals("", size1, fs.getFileStatus(logConfig.getLogFile().getPath()).getLen());
|
||||
@@ -93,7 +93,7 @@ public class AvroLogAppenderTest {
|
||||
|
||||
// Close and Open again and append 100 more records
|
||||
logAppender = new RollingAvroLogAppender(logConfig);
|
||||
logAppender.append(SchemaTestUtil.generateTestRecords(100, 100));
|
||||
logAppender.append(SchemaTestUtil.generateTestRecords(100, 100).iterator());
|
||||
long size2 = logAppender.getCurrentSize();
|
||||
assertTrue("", size2 > size1);
|
||||
assertEquals("", size2, fs.getFileStatus(logConfig.getLogFile().getPath()).getLen());
|
||||
@@ -101,7 +101,7 @@ public class AvroLogAppenderTest {
|
||||
|
||||
// Close and Open again and append 100 more records
|
||||
logAppender = new RollingAvroLogAppender(logConfig);
|
||||
logAppender.append(SchemaTestUtil.generateTestRecords(200, 100));
|
||||
logAppender.append(SchemaTestUtil.generateTestRecords(200, 100).iterator());
|
||||
long size3 = logAppender.getCurrentSize();
|
||||
assertTrue("", size3 > size2);
|
||||
assertEquals("", size3, fs.getFileStatus(logConfig.getLogFile().getPath()).getLen());
|
||||
@@ -123,13 +123,13 @@ public class AvroLogAppenderTest {
|
||||
.withBaseCommitTime("100")
|
||||
.withSchema(SchemaTestUtil.getSimpleSchema()).withFs(fs).build();
|
||||
RollingAvroLogAppender logAppender = new RollingAvroLogAppender(logConfig);
|
||||
logAppender.append(SchemaTestUtil.generateTestRecords(0, 100));
|
||||
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));
|
||||
logAppender.append(SchemaTestUtil.generateTestRecords(100, 100).iterator());
|
||||
assertEquals("", logAppender.getCurrentSize(),
|
||||
fs.getFileStatus(logConfig.getLogFile().getPath()).getLen());
|
||||
logAppender.close();
|
||||
@@ -144,7 +144,7 @@ public class AvroLogAppenderTest {
|
||||
.withBaseCommitTime("100")
|
||||
.withSchema(SchemaTestUtil.getSimpleSchema()).withFs(fs).build();
|
||||
RollingAvroLogAppender logAppender = new RollingAvroLogAppender(logConfig);
|
||||
logAppender.append(SchemaTestUtil.generateTestRecords(0, 100));
|
||||
logAppender.append(SchemaTestUtil.generateTestRecords(0, 100).iterator());
|
||||
logAppender.close();
|
||||
|
||||
// Append some arbit byte[] to thee end of the log (mimics a partially written commit)
|
||||
@@ -157,7 +157,7 @@ public class AvroLogAppenderTest {
|
||||
outputStream.close();
|
||||
|
||||
logAppender = new RollingAvroLogAppender(logConfig);
|
||||
logAppender.append(SchemaTestUtil.generateTestRecords(100, 100));
|
||||
logAppender.append(SchemaTestUtil.generateTestRecords(100, 100).iterator());
|
||||
logAppender.close();
|
||||
}
|
||||
|
||||
@@ -175,7 +175,7 @@ public class AvroLogAppenderTest {
|
||||
long size1 = logAppender.getCurrentSize();
|
||||
|
||||
List<IndexedRecord> inputRecords = SchemaTestUtil.generateTestRecords(0, 100);
|
||||
logAppender.append(inputRecords);
|
||||
logAppender.append(inputRecords.iterator());
|
||||
logAppender.close();
|
||||
|
||||
AvroLogReader logReader =
|
||||
@@ -195,21 +195,21 @@ public class AvroLogAppenderTest {
|
||||
.withBaseCommitTime("100")
|
||||
.withSchema(SchemaTestUtil.getSimpleSchema()).withFs(fs).build();
|
||||
RollingAvroLogAppender logAppender = new RollingAvroLogAppender(logConfig);
|
||||
logAppender.append(SchemaTestUtil.generateTestRecords(0, 100));
|
||||
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);
|
||||
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);
|
||||
logAppender.append(lastBatchInput.iterator());
|
||||
long size3 = logAppender.getCurrentSize();
|
||||
logAppender.close();
|
||||
|
||||
@@ -242,7 +242,7 @@ public class AvroLogAppenderTest {
|
||||
.withSchema(SchemaTestUtil.getSimpleSchema()).withFs(fs).build();
|
||||
RollingAvroLogAppender logAppender = new RollingAvroLogAppender(logConfig);
|
||||
long size1 = logAppender.getCurrentSize();
|
||||
logAppender.append(SchemaTestUtil.generateTestRecords(0, 100));
|
||||
logAppender.append(SchemaTestUtil.generateTestRecords(0, 100).iterator());
|
||||
logAppender.close();
|
||||
|
||||
// Append some arbit byte[] to thee end of the log (mimics a partially written commit)
|
||||
@@ -256,7 +256,7 @@ public class AvroLogAppenderTest {
|
||||
|
||||
logAppender = new RollingAvroLogAppender(logConfig);
|
||||
long size2 = logAppender.getCurrentSize();
|
||||
logAppender.append(SchemaTestUtil.generateTestRecords(100, 100));
|
||||
logAppender.append(SchemaTestUtil.generateTestRecords(100, 100).iterator());
|
||||
logAppender.close();
|
||||
|
||||
AvroLogReader logReader =
|
||||
@@ -285,7 +285,7 @@ public class AvroLogAppenderTest {
|
||||
RollingAvroLogAppender logAppender = new RollingAvroLogAppender(logConfig);
|
||||
long size1 = logAppender.getCurrentSize();
|
||||
List<IndexedRecord> input1 = SchemaTestUtil.generateTestRecords(0, 100);
|
||||
logAppender.append(input1);
|
||||
logAppender.append(input1.iterator());
|
||||
logAppender.close();
|
||||
|
||||
// Need to rebuild config to set the latest version as path
|
||||
@@ -296,7 +296,7 @@ public class AvroLogAppenderTest {
|
||||
logAppender = new RollingAvroLogAppender(logConfig);
|
||||
long size2 = logAppender.getCurrentSize();
|
||||
List<IndexedRecord> input2 = SchemaTestUtil.generateTestRecords(100, 100);
|
||||
logAppender.append(input2);
|
||||
logAppender.append(input2.iterator());
|
||||
logAppender.close();
|
||||
|
||||
logConfig = HoodieLogAppendConfig.newBuilder().onPartitionPath(partitionPath)
|
||||
|
||||
@@ -54,12 +54,14 @@ public class ReadOptimizedTableViewTest {
|
||||
folder.create();
|
||||
this.basePath = folder.getRoot().getAbsolutePath();
|
||||
metaClient = HoodieTestUtils.init(basePath);
|
||||
fsView = new ReadOptimizedTableView(HoodieTestUtils.fs, metaClient);
|
||||
fsView = new HoodieTableFileSystemView(metaClient,
|
||||
metaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants());
|
||||
}
|
||||
|
||||
private void refreshFsView() {
|
||||
metaClient = new HoodieTableMetaClient(HoodieTestUtils.fs, basePath, true);
|
||||
fsView = new ReadOptimizedTableView(HoodieTestUtils.fs, metaClient);
|
||||
fsView = new HoodieTableFileSystemView(metaClient,
|
||||
metaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
||||
Reference in New Issue
Block a user