Skip to content

Commit 10cbb3f

Browse files
committed
HBASE-24086 Disable output stream capability enforcement when running on LocalFileSystem
Signed-off-by: stack <[email protected]> Signed-off-by: Bharath Vissapragada <[email protected]> Signed-off-by: Duo Zhang <[email protected]>
1 parent 2a95fff commit 10cbb3f

File tree

1 file changed

+11
-5
lines changed

1 file changed

+11
-5
lines changed

hbase-common/src/main/java/org/apache/hadoop/hbase/util/CommonFSUtils.java

Lines changed: 11 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -1,4 +1,4 @@
1-
/**
1+
/*
22
*
33
* Licensed to the Apache Software Foundation (ASF) under one
44
* or more contributor license agreements. See the NOTICE file
@@ -28,24 +28,22 @@
2828
import java.util.Locale;
2929
import java.util.Map;
3030
import java.util.concurrent.ConcurrentHashMap;
31-
import org.apache.hadoop.HadoopIllegalArgumentException;
3231
import org.apache.hadoop.conf.Configuration;
3332
import org.apache.hadoop.fs.FSDataOutputStream;
3433
import org.apache.hadoop.fs.FSDataOutputStreamBuilder;
3534
import org.apache.hadoop.fs.FileStatus;
3635
import org.apache.hadoop.fs.FileSystem;
36+
import org.apache.hadoop.fs.LocalFileSystem;
3737
import org.apache.hadoop.fs.LocatedFileStatus;
3838
import org.apache.hadoop.fs.Path;
3939
import org.apache.hadoop.fs.PathFilter;
4040
import org.apache.hadoop.fs.RemoteIterator;
4141
import org.apache.hadoop.fs.permission.FsPermission;
4242
import org.apache.hadoop.hbase.HConstants;
4343
import org.apache.hadoop.hbase.TableName;
44-
import org.apache.hadoop.ipc.RemoteException;
4544
import org.apache.yetus.audience.InterfaceAudience;
4645
import org.slf4j.Logger;
4746
import org.slf4j.LoggerFactory;
48-
4947
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
5048
import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
5149

@@ -350,11 +348,19 @@ public static void setWALRootDir(final Configuration c, final Path root) {
350348
public static FileSystem getWALFileSystem(final Configuration c) throws IOException {
351349
Path p = getWALRootDir(c);
352350
FileSystem fs = p.getFileSystem(c);
353-
// hadoop-core does fs caching, so need to propogate this if set
351+
// hadoop-core does fs caching, so need to propagate this if set
354352
String enforceStreamCapability = c.get(UNSAFE_STREAM_CAPABILITY_ENFORCE);
355353
if (enforceStreamCapability != null) {
356354
fs.getConf().set(UNSAFE_STREAM_CAPABILITY_ENFORCE, enforceStreamCapability);
357355
}
356+
if (fs instanceof LocalFileSystem) {
357+
// running on LocalFileSystem, which does not support the required capabilities `HSYNC`
358+
// and `HFLUSH`. disable enforcement.
359+
final boolean value = false;
360+
LOG.warn("Cannot enforce durability guarantees while running on {}. Setting {}={} for"
361+
+ " this FileSystem.", fs.getUri(), UNSAFE_STREAM_CAPABILITY_ENFORCE, value);
362+
fs.getConf().setBoolean(UNSAFE_STREAM_CAPABILITY_ENFORCE, value);
363+
}
358364
return fs;
359365
}
360366

0 commit comments

Comments
 (0)