HBASE-24086 Disable output stream capability enforcement when running on LocalFileSystem

Signed-off-by: stack <stack@apache.org>
Signed-off-by: Bharath Vissapragada <bharathv@apache.org>
Signed-off-by: Duo Zhang <zhangduo@apache.org>
This commit is contained in:
Nick Dimiduk 2020-04-01 11:25:39 -07:00 committed by Nick Dimiduk
parent 2a95fffb3b
commit 10cbb3fb73
1 changed files with 11 additions and 5 deletions

View File

@ -1,4 +1,4 @@
/**
/*
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
@ -28,12 +28,12 @@ import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import org.apache.hadoop.HadoopIllegalArgumentException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FSDataOutputStreamBuilder;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.LocalFileSystem;
import org.apache.hadoop.fs.LocatedFileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.PathFilter;
@ -41,11 +41,9 @@ import org.apache.hadoop.fs.RemoteIterator;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
@ -350,11 +348,19 @@ public abstract class CommonFSUtils {
public static FileSystem getWALFileSystem(final Configuration c) throws IOException {
Path p = getWALRootDir(c);
FileSystem fs = p.getFileSystem(c);
// hadoop-core does fs caching, so need to propogate this if set
// hadoop-core does fs caching, so need to propagate this if set
String enforceStreamCapability = c.get(UNSAFE_STREAM_CAPABILITY_ENFORCE);
if (enforceStreamCapability != null) {
fs.getConf().set(UNSAFE_STREAM_CAPABILITY_ENFORCE, enforceStreamCapability);
}
if (fs instanceof LocalFileSystem) {
// running on LocalFileSystem, which does not support the required capabilities `HSYNC`
// and `HFLUSH`. disable enforcement.
final boolean value = false;
LOG.warn("Cannot enforce durability guarantees while running on {}. Setting {}={} for"
+ " this FileSystem.", fs.getUri(), UNSAFE_STREAM_CAPABILITY_ENFORCE, value);
fs.getConf().setBoolean(UNSAFE_STREAM_CAPABILITY_ENFORCE, value);
}
return fs;
}