[HUDI-679] Make io package Spark free (#1460)
* [HUDI-679] Make io package Spark free
This commit is contained in:
@@ -89,7 +89,7 @@ public class TestUpdateSchemaEvolution extends HoodieClientTestHarness {
|
||||
.add(new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3));
|
||||
|
||||
HoodieCreateHandle createHandle =
|
||||
new HoodieCreateHandle(config, "100", table, rowChange1.getPartitionPath(), "f1-0", insertRecords.iterator());
|
||||
new HoodieCreateHandle(config, "100", table, rowChange1.getPartitionPath(), "f1-0", insertRecords.iterator(), supplier);
|
||||
createHandle.write();
|
||||
return createHandle.close();
|
||||
}).collect();
|
||||
@@ -119,7 +119,7 @@ public class TestUpdateSchemaEvolution extends HoodieClientTestHarness {
|
||||
|
||||
try {
|
||||
HoodieMergeHandle mergeHandle = new HoodieMergeHandle(config2, "101", table2,
|
||||
updateRecords.iterator(), record1.getPartitionPath(), fileId);
|
||||
updateRecords.iterator(), record1.getPartitionPath(), fileId, supplier);
|
||||
Configuration conf = new Configuration();
|
||||
AvroReadSupport.setAvroReadSchema(conf, mergeHandle.getWriterSchema());
|
||||
List<GenericRecord> oldRecords = ParquetUtils.readAvroRecords(conf,
|
||||
|
||||
@@ -17,6 +17,7 @@
|
||||
|
||||
package org.apache.hudi.common;
|
||||
|
||||
import org.apache.hudi.client.SparkTaskContextSupplier;
|
||||
import org.apache.hudi.client.TestHoodieClientBase;
|
||||
import org.apache.hudi.common.minicluster.HdfsTestService;
|
||||
import org.apache.hudi.common.model.HoodieTestUtils;
|
||||
@@ -55,6 +56,8 @@ public abstract class HoodieClientTestHarness extends HoodieCommonTestHarness im
|
||||
protected transient HoodieTableMetaClient metaClient;
|
||||
private static AtomicInteger instantGen = new AtomicInteger(1);
|
||||
|
||||
protected final SparkTaskContextSupplier supplier = new SparkTaskContextSupplier();
|
||||
|
||||
public String getNextInstant() {
|
||||
return String.format("%09d", instantGen.getAndIncrement());
|
||||
}
|
||||
|
||||
@@ -19,6 +19,7 @@
|
||||
package org.apache.hudi.common;
|
||||
|
||||
import org.apache.hudi.client.HoodieReadClient;
|
||||
import org.apache.hudi.client.SparkTaskContextSupplier;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.avro.HoodieAvroWriteSupport;
|
||||
import org.apache.hudi.common.bloom.filter.BloomFilter;
|
||||
@@ -230,7 +231,8 @@ public class HoodieClientTestUtils {
|
||||
ParquetWriter.DEFAULT_BLOCK_SIZE, ParquetWriter.DEFAULT_PAGE_SIZE, 120 * 1024 * 1024,
|
||||
HoodieTestUtils.getDefaultHadoopConf(), Double.valueOf(HoodieStorageConfig.DEFAULT_STREAM_COMPRESSION_RATIO));
|
||||
HoodieParquetWriter writer =
|
||||
new HoodieParquetWriter(instantTime, new Path(basePath + "/" + partitionPath + "/" + filename), config, schema);
|
||||
new HoodieParquetWriter(instantTime, new Path(basePath + "/" + partitionPath + "/" + filename), config,
|
||||
schema, new SparkTaskContextSupplier());
|
||||
int seqId = 1;
|
||||
for (HoodieRecord record : records) {
|
||||
GenericRecord avroRecord = (GenericRecord) record.getData().getInsertValue(schema).get();
|
||||
|
||||
@@ -18,6 +18,7 @@
|
||||
|
||||
package org.apache.hudi.io.storage;
|
||||
|
||||
import org.apache.hudi.client.SparkTaskContextSupplier;
|
||||
import org.apache.hudi.client.TestHoodieClientBase;
|
||||
import org.apache.hudi.common.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
@@ -44,15 +45,16 @@ public class TestHoodieStorageWriterFactory extends TestHoodieClientBase {
|
||||
final Path parquetPath = new Path(basePath + "/partition/path/f1_1-0-1_000.parquet");
|
||||
final HoodieWriteConfig cfg = getConfig();
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
|
||||
SparkTaskContextSupplier supplier = new SparkTaskContextSupplier();
|
||||
HoodieStorageWriter<IndexedRecord> parquetWriter = HoodieStorageWriterFactory.getStorageWriter(instantTime,
|
||||
parquetPath, table, cfg, HoodieTestDataGenerator.AVRO_SCHEMA);
|
||||
parquetPath, table, cfg, HoodieTestDataGenerator.AVRO_SCHEMA, supplier);
|
||||
Assert.assertTrue(parquetWriter instanceof HoodieParquetWriter);
|
||||
|
||||
// other file format exception.
|
||||
final Path logPath = new Path(basePath + "/partition/path/f.b51192a8-574b-4a85-b246-bcfec03ac8bf_100.log.2_1-0-1");
|
||||
try {
|
||||
HoodieStorageWriter<IndexedRecord> logWriter = HoodieStorageWriterFactory.getStorageWriter(instantTime, logPath,
|
||||
table, cfg, HoodieTestDataGenerator.AVRO_SCHEMA);
|
||||
table, cfg, HoodieTestDataGenerator.AVRO_SCHEMA, supplier);
|
||||
fail("should fail since log storage writer is not supported yet.");
|
||||
} catch (Exception e) {
|
||||
Assert.assertTrue(e instanceof UnsupportedOperationException);
|
||||
|
||||
@@ -103,7 +103,7 @@ public class TestCopyOnWriteTable extends HoodieClientTestHarness {
|
||||
when(record.getPartitionPath()).thenReturn(partitionPath);
|
||||
String writeToken = FSUtils.makeWriteToken(TaskContext.getPartitionId(), TaskContext.get().stageId(),
|
||||
TaskContext.get().taskAttemptId());
|
||||
HoodieCreateHandle io = new HoodieCreateHandle(config, instantTime, table, partitionPath, fileName);
|
||||
HoodieCreateHandle io = new HoodieCreateHandle(config, instantTime, table, partitionPath, fileName, supplier);
|
||||
return Pair.of(io.makeNewPath(record.getPartitionPath()), writeToken);
|
||||
}).collect().get(0);
|
||||
|
||||
|
||||
Reference in New Issue
Block a user