1
0

Rename IO Handles & introduce stub for BucketedIndex

- UpdateHandle -> MergeHandle, InsertHandle -> CreateHandle
 - Also bunch of code cleanup in different places
This commit is contained in:
Vinoth Chandar
2017-05-14 09:44:44 -07:00
committed by prazanna
parent b0a2a23372
commit 23e7badd8a
15 changed files with 182 additions and 54 deletions

View File

@@ -34,7 +34,20 @@ import com.uber.hoodie.exception.HoodieIOException;
import com.uber.hoodie.exception.HoodieUpsertException;
import com.uber.hoodie.func.LazyInsertIterable;
import com.uber.hoodie.io.HoodieCleanHelper;
import com.uber.hoodie.io.HoodieUpdateHandle;
import com.uber.hoodie.io.HoodieMergeHandle;
import java.util.Optional;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.parquet.avro.AvroParquetReader;
import org.apache.parquet.avro.AvroReadSupport;
import org.apache.parquet.hadoop.ParquetReader;
import org.apache.spark.Partitioner;
import java.io.IOException;
import java.io.Serializable;
import java.util.ArrayList;
@@ -70,7 +83,7 @@ import scala.Option;
* INSERTS - Produce new files, block aligned to desired size (or)
* Merge with the smallest existing file, to expand it
*
* UPDATES - Produce a new version of the file containing the invalidated records
* UPDATES - Produce a new version of the file, just replacing the updated records with new values
*
*/
public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends HoodieTable {
@@ -405,7 +418,7 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
public Iterator<List<WriteStatus>> handleUpdate(String commitTime, String fileLoc, Iterator<HoodieRecord<T>> recordItr)
throws IOException {
// these are updates
HoodieUpdateHandle upsertHandle = getUpdateHandle(commitTime, fileLoc, recordItr);
HoodieMergeHandle upsertHandle = getUpdateHandle(commitTime, fileLoc, recordItr);
if (upsertHandle.getOldFilePath() == null) {
throw new HoodieUpsertException("Error in finding the old file path at commit " +
commitTime +" at fileLoc: " + fileLoc);
@@ -439,8 +452,8 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
return Collections.singletonList(Collections.singletonList(upsertHandle.getWriteStatus())).iterator();
}
protected HoodieUpdateHandle getUpdateHandle(String commitTime, String fileLoc, Iterator<HoodieRecord<T>> recordItr) {
return new HoodieUpdateHandle<>(config, commitTime, this, recordItr, fileLoc);
protected HoodieMergeHandle getUpdateHandle(String commitTime, String fileLoc, Iterator<HoodieRecord<T>> recordItr) {
return new HoodieMergeHandle<>(config, commitTime, this, recordItr, fileLoc);
}
public Iterator<List<WriteStatus>> handleInsert(String commitTime, Iterator<HoodieRecord<T>> recordItr) throws Exception {