|
1 | | -/** |
| 1 | +/* |
2 | 2 | * |
3 | 3 | * Licensed to the Apache Software Foundation (ASF) under one |
4 | 4 | * or more contributor license agreements. See the NOTICE file |
|
28 | 28 | import java.util.Locale; |
29 | 29 | import java.util.Map; |
30 | 30 | import java.util.concurrent.ConcurrentHashMap; |
31 | | -import org.apache.hadoop.HadoopIllegalArgumentException; |
32 | 31 | import org.apache.hadoop.conf.Configuration; |
33 | 32 | import org.apache.hadoop.fs.FSDataOutputStream; |
34 | 33 | import org.apache.hadoop.fs.FSDataOutputStreamBuilder; |
35 | 34 | import org.apache.hadoop.fs.FileStatus; |
36 | 35 | import org.apache.hadoop.fs.FileSystem; |
| 36 | +import org.apache.hadoop.fs.LocalFileSystem; |
37 | 37 | import org.apache.hadoop.fs.LocatedFileStatus; |
38 | 38 | import org.apache.hadoop.fs.Path; |
39 | 39 | import org.apache.hadoop.fs.PathFilter; |
40 | 40 | import org.apache.hadoop.fs.RemoteIterator; |
41 | 41 | import org.apache.hadoop.fs.permission.FsPermission; |
42 | 42 | import org.apache.hadoop.hbase.HConstants; |
43 | 43 | import org.apache.hadoop.hbase.TableName; |
44 | | -import org.apache.hadoop.ipc.RemoteException; |
45 | 44 | import org.apache.yetus.audience.InterfaceAudience; |
46 | 45 | import org.slf4j.Logger; |
47 | 46 | import org.slf4j.LoggerFactory; |
48 | | - |
49 | 47 | import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; |
50 | 48 | import org.apache.hbase.thirdparty.com.google.common.collect.Lists; |
51 | 49 |
|
@@ -350,11 +348,19 @@ public static void setWALRootDir(final Configuration c, final Path root) { |
350 | 348 | public static FileSystem getWALFileSystem(final Configuration c) throws IOException { |
351 | 349 | Path p = getWALRootDir(c); |
352 | 350 | 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 |
354 | 352 | String enforceStreamCapability = c.get(UNSAFE_STREAM_CAPABILITY_ENFORCE); |
355 | 353 | if (enforceStreamCapability != null) { |
356 | 354 | fs.getConf().set(UNSAFE_STREAM_CAPABILITY_ENFORCE, enforceStreamCapability); |
357 | 355 | } |
| 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 | + } |
358 | 364 | return fs; |
359 | 365 | } |
360 | 366 |
|
|
0 commit comments