Split insert() into insert() & bulkInsert() (#69)
- Behaviour change for existing insert() users - Made the current insert() implementation, as something to use for bulkInsert() - Normal inserts now share a lot of code with upsert, which provides benefits like small file handling - Refactored/Cleaned up code in HoodieWriteClient for reuse - Added a unit test, switching few tests to call bulkInsert() and few to call insert()
This commit is contained in:
@@ -25,6 +25,7 @@ import com.uber.hoodie.common.model.HoodieRecordLocation;
|
||||
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
||||
import com.uber.hoodie.common.model.HoodieTableMetadata;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.exception.HoodieInsertException;
|
||||
import com.uber.hoodie.exception.HoodieUpsertException;
|
||||
import com.uber.hoodie.func.LazyInsertIterable;
|
||||
import com.uber.hoodie.io.HoodieUpdateHandle;
|
||||
@@ -376,7 +377,7 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
|
||||
|
||||
@Override
|
||||
public Partitioner getInsertPartitioner(WorkloadProfile profile) {
|
||||
return null;
|
||||
return getUpsertPartitioner(profile);
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -385,7 +386,6 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
|
||||
}
|
||||
|
||||
|
||||
|
||||
public Iterator<List<WriteStatus>> handleUpdate(String fileLoc, Iterator<HoodieRecord<T>> recordItr) throws Exception {
|
||||
// these are updates
|
||||
HoodieUpdateHandle upsertHandle =
|
||||
@@ -449,4 +449,11 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
|
||||
throw new HoodieUpsertException(msg, t);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterator<List<WriteStatus>> handleInsertPartition(Integer partition,
|
||||
Iterator recordItr,
|
||||
Partitioner partitioner) {
|
||||
return handleUpsertPartition(partition, recordItr, partitioner);
|
||||
}
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user