1
0

Explicitly handle lack of append() support during LogWriting

This commit is contained in:
Vinoth Chandar
2018-11-27 16:54:46 -08:00
committed by vinoth chandar
parent d0fde47458
commit fa65db9c4c
4 changed files with 130 additions and 31 deletions

View File

@@ -16,16 +16,15 @@
package com.uber.hoodie.io.storage;
import com.uber.hoodie.common.storage.StorageSchemes;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.exception.HoodieIOException;
import java.io.IOException;
import java.net.URI;
import java.net.URISyntaxException;
import java.util.EnumSet;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import org.apache.hadoop.conf.Configuration;
@@ -60,20 +59,6 @@ import org.apache.hadoop.util.Progressable;
public class HoodieWrapperFileSystem extends FileSystem {
public static final String HOODIE_SCHEME_PREFIX = "hoodie-";
private static final Set<String> SUPPORT_SCHEMES;
static {
SUPPORT_SCHEMES = new HashSet<>();
SUPPORT_SCHEMES.add("file");
SUPPORT_SCHEMES.add("hdfs");
SUPPORT_SCHEMES.add("s3");
SUPPORT_SCHEMES.add("s3a");
// Hoodie currently relies on underlying object store being fully
// consistent so only regional buckets should be used.
SUPPORT_SCHEMES.add("gs");
SUPPORT_SCHEMES.add("viewfs");
}
private ConcurrentMap<String, SizeAwareFSDataOutputStream> openStreams = new
ConcurrentHashMap<>();
@@ -104,7 +89,7 @@ public class HoodieWrapperFileSystem extends FileSystem {
public static String getHoodieScheme(String scheme) {
String newScheme;
if (SUPPORT_SCHEMES.contains(scheme)) {
if (StorageSchemes.isSchemeSupported(scheme)) {
newScheme = HOODIE_SCHEME_PREFIX + scheme;
} else {
throw new IllegalArgumentException(