Introduce RealtimeTableView and Implement HoodieRealtimeTableCompactor (#73)
This commit is contained in:
committed by
Prasanna Rajaperumal
parent
48fbb0f425
commit
11d2fd3428
@@ -20,12 +20,25 @@ 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.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.util.FSUtils;
|
||||
|
||||
import com.uber.hoodie.common.util.HoodieAvroUtils;
|
||||
import com.uber.hoodie.common.util.SchemaTestUtil;
|
||||
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.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.jute.Index;
|
||||
import org.junit.rules.TemporaryFolder;
|
||||
|
||||
import java.io.ByteArrayInputStream;
|
||||
@@ -38,11 +51,16 @@ import java.io.Serializable;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.util.Date;
|
||||
import java.util.Iterator;
|
||||
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 static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
public class HoodieTestUtils {
|
||||
public static FileSystem fs = FSUtils.getFs();
|
||||
@@ -51,8 +69,13 @@ public class HoodieTestUtils {
|
||||
public static final int DEFAULT_TASK_PARTITIONID = 1;
|
||||
|
||||
public static HoodieTableMetaClient init(String basePath) throws IOException {
|
||||
return initTableType(basePath, HoodieTableType.COPY_ON_WRITE);
|
||||
}
|
||||
|
||||
public static HoodieTableMetaClient initTableType(String basePath, HoodieTableType tableType) throws IOException {
|
||||
Properties properties = new Properties();
|
||||
properties.setProperty(HoodieTableConfig.HOODIE_TABLE_NAME_PROP_NAME, RAW_TRIPS_TEST_NAME);
|
||||
properties.setProperty(HoodieTableConfig.HOODIE_TABLE_TYPE_PROP_NAME, tableType.name());
|
||||
return HoodieTableMetaClient.initializePathAsHoodieDataset(fs, basePath, properties);
|
||||
}
|
||||
|
||||
@@ -143,4 +166,41 @@ public class HoodieTestUtils {
|
||||
input.close();
|
||||
return deseralizedObject;
|
||||
}
|
||||
|
||||
public static void writeRecordsToLogFiles(String basePath, Schema schema, List<HoodieRecord> updatedRecords) {
|
||||
Map<HoodieRecordLocation, List<HoodieRecord>> groupedUpdated = updatedRecords.stream()
|
||||
.collect(Collectors.groupingBy(HoodieRecord::getCurrentLocation));
|
||||
|
||||
groupedUpdated.entrySet().forEach(s -> {
|
||||
HoodieRecordLocation location = s.getKey();
|
||||
String partitionPath = s.getValue().get(0).getPartitionPath();
|
||||
|
||||
HoodieLogAppendConfig logConfig = null;
|
||||
try {
|
||||
logConfig = HoodieLogAppendConfig.newBuilder()
|
||||
.onPartitionPath(new Path(basePath, partitionPath))
|
||||
.withLogFileExtension(HoodieLogFile.DELTA_EXTENSION)
|
||||
.withFileId(location.getFileId())
|
||||
.withBaseCommitTime(location.getCommitTime())
|
||||
.withSchema(schema).withFs(fs).build();
|
||||
|
||||
AvroLogAppender log = new AvroLogAppender(logConfig);
|
||||
log.append(s.getValue().stream().map(r -> {
|
||||
try {
|
||||
GenericRecord val = (GenericRecord) r.getData().getInsertValue(schema);
|
||||
HoodieAvroUtils.addHoodieKeyToRecord(val,
|
||||
r.getRecordKey(),
|
||||
r.getPartitionPath(),
|
||||
"");
|
||||
return val;
|
||||
} catch (IOException e) {
|
||||
return null;
|
||||
}
|
||||
}).collect(Collectors.toList()));
|
||||
log.close();
|
||||
} catch (Exception e) {
|
||||
fail(e.toString());
|
||||
}
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
@@ -82,6 +82,7 @@ public class AvroLogAppenderTest {
|
||||
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));
|
||||
@@ -119,6 +120,7 @@ public class AvroLogAppenderTest {
|
||||
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));
|
||||
@@ -139,6 +141,7 @@ public class AvroLogAppenderTest {
|
||||
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));
|
||||
@@ -166,6 +169,7 @@ public class AvroLogAppenderTest {
|
||||
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();
|
||||
@@ -188,6 +192,7 @@ public class AvroLogAppenderTest {
|
||||
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));
|
||||
@@ -233,6 +238,7 @@ public class AvroLogAppenderTest {
|
||||
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();
|
||||
@@ -272,6 +278,7 @@ public class AvroLogAppenderTest {
|
||||
HoodieLogAppendConfig logConfig =
|
||||
HoodieLogAppendConfig.newBuilder().onPartitionPath(partitionPath)
|
||||
.withLogFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
|
||||
.withBaseCommitTime("100")
|
||||
.withSchema(SchemaTestUtil.getSimpleSchema()).withSizeThreshold(500).withFs(fs)
|
||||
.build();
|
||||
|
||||
@@ -284,6 +291,7 @@ public class AvroLogAppenderTest {
|
||||
// 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();
|
||||
@@ -293,18 +301,21 @@ public class AvroLogAppenderTest {
|
||||
|
||||
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).collect(Collectors.toList());
|
||||
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(), fs,
|
||||
logConfig.getSchema(), HoodieLogFile.DELTA_EXTENSION);
|
||||
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());
|
||||
|
||||
Reference in New Issue
Block a user