1
0

Multi FS Support

- Reviving PR 191, to make FileSystem creation off actual path
 - Streamline all filesystem access to HoodieTableMetaClient
 - Hadoop Conf from Spark Context serialized & passed to executor code too
 - Pick up env vars prefixed with HOODIE_ENV_ into Configuration object
 - Cleanup usage of FSUtils.getFS, piggybacking off HoodieTableMetaClient.getFS
 - Adding s3a to supported schemes & support escaping "." in env vars
 - Tests use HoodieTestUtils.getDefaultHadoopConf
This commit is contained in:
Vinoth Chandar
2017-12-10 23:31:54 -08:00
committed by vinoth chandar
parent 44839b88c6
commit 0cd186c899
78 changed files with 851 additions and 535 deletions

View File

@@ -19,6 +19,7 @@ package com.uber.hoodie.common.minicluster;
import com.google.common.base.Preconditions;
import com.google.common.io.Files;
import com.uber.hoodie.common.model.HoodieTestUtils;
import java.io.File;
import java.io.IOException;
import org.apache.commons.io.FileUtils;
@@ -54,7 +55,7 @@ public class HdfsTestService {
private MiniDFSCluster miniDfsCluster;
public HdfsTestService() {
hadoopConf = new Configuration();
hadoopConf = HoodieTestUtils.getDefaultHadoopConf();
workDir = Files.createTempDir().getAbsolutePath();
}
@@ -67,7 +68,7 @@ public class HdfsTestService {
.checkState(workDir != null, "The work dir must be set before starting cluster.");
if (hadoopConf == null) {
hadoopConf = new Configuration();
hadoopConf = HoodieTestUtils.getDefaultHadoopConf();
}
// If clean, then remove the work dir so we can start fresh.

View File

@@ -58,6 +58,7 @@ import java.util.stream.Stream;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
@@ -69,18 +70,23 @@ import org.junit.rules.TemporaryFolder;
public class HoodieTestUtils {
public static FileSystem fs = FSUtils.getFs();
public static FileSystem fs;
public static final String TEST_EXTENSION = ".test";
public static final String RAW_TRIPS_TEST_NAME = "raw_trips";
public static final int DEFAULT_TASK_PARTITIONID = 1;
public static final String[] DEFAULT_PARTITION_PATHS = {"2016/03/15", "2015/03/16", "2015/03/17"};
private static Random rand = new Random(46474747);
public static void resetFS() {
HoodieTestUtils.fs = FSUtils.getFs();
public static void resetFS(String basePath) {
HoodieTestUtils.fs = FSUtils.getFs(basePath, HoodieTestUtils.getDefaultHadoopConf());
}
public static Configuration getDefaultHadoopConf() {
return new Configuration();
}
public static HoodieTableMetaClient init(String basePath) throws IOException {
fs = FSUtils.getFs(basePath, HoodieTestUtils.getDefaultHadoopConf());
return initTableType(basePath, HoodieTableType.COPY_ON_WRITE);
}
@@ -211,7 +217,7 @@ public class HoodieTestUtils {
Path commitFile =
new Path(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline
.makeCleanerFileName(commitTime));
FileSystem fs = FSUtils.getFs();
FileSystem fs = FSUtils.getFs(basePath, HoodieTestUtils.getDefaultHadoopConf());
FSDataOutputStream os = fs.create(commitFile, true);
try {
HoodieCleanStat cleanStats = new HoodieCleanStat(

View File

@@ -67,7 +67,7 @@ public class HoodieLogFormatTest {
private FileSystem fs;
private Path partitionPath;
private String basePath;
private static String basePath;
@BeforeClass
public static void setUpClass() throws IOException, InterruptedException {
@@ -78,7 +78,7 @@ public class HoodieLogFormatTest {
@AfterClass
public static void tearDownClass() {
MiniClusterUtil.shutdown();
HoodieTestUtils.resetFS();
HoodieTestUtils.resetFS(basePath);
}
@Before
@@ -343,7 +343,7 @@ public class HoodieLogFormatTest {
writer.close();
// Append some arbit byte[] to thee end of the log (mimics a partially written commit)
fs = FileSystem.get(fs.getConf());
fs = FSUtils.getFs(fs.getUri().toString(), fs.getConf());
FSDataOutputStream outputStream = fs.append(writer.getLogFile().getPath());
// create a block with
outputStream.write(HoodieLogFormat.MAGIC);
@@ -533,7 +533,7 @@ public class HoodieLogFormatTest {
writer.close();
// Append some arbit byte[] to thee end of the log (mimics a partially written commit)
fs = FileSystem.get(fs.getConf());
fs = FSUtils.getFs(fs.getUri().toString(), fs.getConf());
FSDataOutputStream outputStream = fs.append(writer.getLogFile().getPath());
// create a block with
outputStream.write(HoodieLogFormat.MAGIC);

View File

@@ -74,7 +74,7 @@ public class HoodieActiveTimelineTest {
HoodieInstant instant5 =
new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "9");
timeline = new HoodieActiveTimeline(HoodieTestUtils.fs, metaClient.getMetaPath());
timeline = new HoodieActiveTimeline(metaClient);
timeline.saveAsComplete(instant1, Optional.empty());
timeline.saveAsComplete(instant2, Optional.empty());
timeline.saveAsComplete(instant3, Optional.empty());
@@ -98,7 +98,7 @@ public class HoodieActiveTimelineTest {
@Test
public void testTimelineOperationsBasic() throws Exception {
timeline = new HoodieActiveTimeline(HoodieTestUtils.fs, metaClient.getMetaPath());
timeline = new HoodieActiveTimeline(metaClient);
assertTrue(timeline.empty());
assertEquals("", 0, timeline.countInstants());
assertEquals("", Optional.empty(), timeline.firstInstant());

View File

@@ -69,7 +69,7 @@ public class HoodieTableFileSystemViewTest {
}
private void refreshFsView(FileStatus[] statuses) {
metaClient = new HoodieTableMetaClient(HoodieTestUtils.fs, basePath, true);
metaClient = new HoodieTableMetaClient(HoodieTestUtils.fs.getConf(), basePath, true);
if (statuses != null) {
fsView = new HoodieTableFileSystemView(metaClient,
metaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants(),

View File

@@ -16,15 +16,24 @@
package com.uber.hoodie.common.util;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import com.uber.hoodie.common.model.HoodieTestUtils;
import java.text.SimpleDateFormat;
import java.util.Date;
import java.util.UUID;
import org.apache.hadoop.conf.Configuration;
import org.junit.Rule;
import org.junit.Test;
import org.junit.contrib.java.lang.system.EnvironmentVariables;
public class TestFSUtils {
@Rule
public final EnvironmentVariables environmentVariables
= new EnvironmentVariables();
@Test
public void testMakeDataFileName() {
String commitTime = new SimpleDateFormat("yyyyMMddHHmmss").format(new Date());
@@ -59,4 +68,15 @@ public class TestFSUtils {
String fullFileName = FSUtils.makeDataFileName(commitTime, taskPartitionId, fileName);
assertTrue(FSUtils.getFileId(fullFileName).equals(fileName));
}
@Test
public void testEnvVarVariablesPickedup() {
environmentVariables.set("HOODIE_ENV_fs_DOT_key1", "value1");
Configuration conf = FSUtils.prepareHadoopConf(HoodieTestUtils.getDefaultHadoopConf());
assertEquals("value1", conf.get("fs.key1"));
conf.set("fs.key1", "value11");
conf.set("fs.key2", "value2");
assertEquals("value11", conf.get("fs.key1"));
assertEquals("value2", conf.get("fs.key2"));
}
}

View File

@@ -22,6 +22,7 @@ import static org.junit.Assert.assertTrue;
import com.uber.hoodie.avro.HoodieAvroWriteSupport;
import com.uber.hoodie.common.BloomFilter;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieTestUtils;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
@@ -79,12 +80,15 @@ public class TestParquetUtils {
// Read and verify
List<String> rowKeysInFile = new ArrayList<>(
ParquetUtils.readRowKeysFromParquet(new Path(filePath)));
ParquetUtils
.readRowKeysFromParquet(HoodieTestUtils.getDefaultHadoopConf(), new Path(filePath)));
Collections.sort(rowKeysInFile);
Collections.sort(rowKeys);
assertEquals("Did not read back the expected list of keys", rowKeys, rowKeysInFile);
BloomFilter filterInFile = ParquetUtils.readBloomFilterFromParquetMetadata(new Path(filePath));
BloomFilter filterInFile = ParquetUtils
.readBloomFilterFromParquetMetadata(HoodieTestUtils.getDefaultHadoopConf(),
new Path(filePath));
for (String rowKey : rowKeys) {
assertTrue("key should be found in bloom filter", filterInFile.mightContain(rowKey));
}