1) Separated rollback as a table operation 2) Implement rollback for MOR
This commit is contained in:
committed by
vinoth chandar
parent
e1fe3ab937
commit
c7d63a7622
@@ -18,16 +18,12 @@
|
||||
|
||||
package com.uber.hoodie.hadoop.realtime;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
import com.uber.hoodie.common.model.HoodieAvroPayload;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
import com.uber.hoodie.common.table.log.HoodieCompactedLogRecordScanner;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.exception.HoodieException;
|
||||
import com.uber.hoodie.exception.HoodieIOException;
|
||||
import java.util.Set;
|
||||
import java.util.TreeMap;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericArray;
|
||||
import org.apache.avro.generic.GenericFixed;
|
||||
@@ -49,6 +45,9 @@ import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.io.Writable;
|
||||
import org.apache.hadoop.mapred.JobConf;
|
||||
import org.apache.hadoop.mapred.RecordReader;
|
||||
import parquet.avro.AvroSchemaConverter;
|
||||
import parquet.hadoop.ParquetFileReader;
|
||||
import parquet.schema.MessageType;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
@@ -56,10 +55,9 @@ import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import parquet.avro.AvroSchemaConverter;
|
||||
import parquet.hadoop.ParquetFileReader;
|
||||
import parquet.schema.MessageType;
|
||||
import java.util.Set;
|
||||
import java.util.TreeMap;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* Record Reader implementation to merge fresh avro data with base parquet data, to support real time
|
||||
@@ -127,7 +125,7 @@ public class HoodieRealtimeRecordReader implements RecordReader<Void, ArrayWrita
|
||||
|
||||
HoodieCompactedLogRecordScanner compactedLogRecordScanner =
|
||||
new HoodieCompactedLogRecordScanner(FSUtils.getFs(), split.getDeltaFilePaths(),
|
||||
readerSchema);
|
||||
readerSchema, split.getMaxCommitTime());
|
||||
|
||||
// NOTE: HoodieCompactedLogRecordScanner will not return records for an in-flight commit
|
||||
// but can return records for completed commits > the commit we are trying to read (if using readCommit() API)
|
||||
|
||||
@@ -19,11 +19,13 @@
|
||||
package com.uber.hoodie.hadoop.realtime;
|
||||
|
||||
|
||||
import com.google.common.collect.Maps;
|
||||
import com.uber.hoodie.common.model.HoodieLogFile;
|
||||
import com.uber.hoodie.common.model.HoodieTableType;
|
||||
import com.uber.hoodie.common.model.HoodieTestUtils;
|
||||
import com.uber.hoodie.common.table.log.HoodieLogFormat;
|
||||
import com.uber.hoodie.common.table.log.block.HoodieAvroDataBlock;
|
||||
import com.uber.hoodie.common.table.log.block.HoodieLogBlock;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.common.util.HoodieAvroUtils;
|
||||
import com.uber.hoodie.common.util.SchemaTestUtil;
|
||||
@@ -55,6 +57,7 @@ import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.junit.Assert.assertTrue;
|
||||
@@ -81,7 +84,9 @@ public class HoodieRealtimeRecordReaderTest {
|
||||
records.add(SchemaTestUtil.generateAvroRecordFromJson(schema, i, newCommit, "fileid0"));
|
||||
}
|
||||
Schema writeSchema = records.get(0).getSchema();
|
||||
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, writeSchema);
|
||||
Map<HoodieLogBlock.LogMetadataType, String> metadata = Maps.newHashMap();
|
||||
metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, newCommit);
|
||||
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, writeSchema, metadata);
|
||||
writer = writer.appendBlock(dataBlock);
|
||||
long size = writer.getCurrentSize();
|
||||
return writer;
|
||||
|
||||
Reference in New Issue
Block a user