Fix HUDI-27 : Support num_cores > 1 for writing through spark
- Users using spark.executor.cores > 1 used to fail due to "FileSystem closed" - This is due to HoodieWrapperFileSystem closing the wrapped filesytem obj - FileSystem.getInternal caching code races threads and closes the extra fs instance(s) - Bumped up num cores in tests to 8, speeds up tests by 3-4 mins
This commit is contained in:
committed by
vinoth chandar
parent
f1410bfdcd
commit
5847f0c934
@@ -34,13 +34,18 @@ import com.uber.hoodie.config.HoodieStorageConfig;
|
||||
import com.uber.hoodie.exception.HoodieException;
|
||||
import com.uber.hoodie.io.storage.HoodieParquetConfig;
|
||||
import com.uber.hoodie.io.storage.HoodieParquetWriter;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.io.RandomAccessFile;
|
||||
import java.util.*;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.UUID;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
@@ -108,7 +113,7 @@ public class HoodieClientTestUtils {
|
||||
public static SparkConf getSparkConfForTest(String appName) {
|
||||
SparkConf sparkConf = new SparkConf().setAppName(appName)
|
||||
.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
|
||||
.setMaster("local[1]");
|
||||
.setMaster("local[8]");
|
||||
return HoodieReadClient.addHoodieSupport(sparkConf);
|
||||
}
|
||||
|
||||
|
||||
Reference in New Issue
Block a user