Skip to content

Commit c74366c

Browse files
authored
HBASE-26049 Remove DfsBuilderUtility (#3444)
Signed-off-by: Duo Zhang <zhangduo@apache.org>
1 parent f0324a7 commit c74366c

File tree

4 files changed

+28
-77
lines changed

4 files changed

+28
-77
lines changed

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

Lines changed: 0 additions & 74 deletions
Original file line numberDiff line numberDiff line change
@@ -20,8 +20,6 @@
2020

2121
import java.io.FileNotFoundException;
2222
import java.io.IOException;
23-
import java.lang.reflect.InvocationTargetException;
24-
import java.lang.reflect.Method;
2523
import java.net.URI;
2624
import java.net.URISyntaxException;
2725
import java.util.List;
@@ -30,7 +28,6 @@
3028
import java.util.concurrent.ConcurrentHashMap;
3129
import org.apache.hadoop.conf.Configuration;
3230
import org.apache.hadoop.fs.FSDataOutputStream;
33-
import org.apache.hadoop.fs.FSDataOutputStreamBuilder;
3431
import org.apache.hadoop.fs.FileStatus;
3532
import org.apache.hadoop.fs.FileSystem;
3633
import org.apache.hadoop.fs.LocatedFileStatus;
@@ -766,77 +763,6 @@ public static void checkShortCircuitReadBufferSize(final Configuration conf) {
766763
conf.setIfUnset(dfsKey, Integer.toString(hbaseSize));
767764
}
768765

769-
private static final class DfsBuilderUtility {
770-
private static final Class<?> BUILDER;
771-
private static final Method REPLICATE;
772-
773-
static {
774-
String builderName = "org.apache.hadoop.hdfs.DistributedFileSystem$HdfsDataOutputStreamBuilder";
775-
Class<?> builderClass = null;
776-
try {
777-
builderClass = Class.forName(builderName);
778-
} catch (ClassNotFoundException e) {
779-
LOG.debug("{} not available, will not set replicate when creating output stream", builderName);
780-
}
781-
Method replicateMethod = null;
782-
if (builderClass != null) {
783-
try {
784-
replicateMethod = builderClass.getMethod("replicate");
785-
LOG.debug("Using builder API via reflection for DFS file creation.");
786-
} catch (NoSuchMethodException e) {
787-
LOG.debug("Could not find replicate method on builder; will not set replicate when" +
788-
" creating output stream", e);
789-
}
790-
}
791-
BUILDER = builderClass;
792-
REPLICATE = replicateMethod;
793-
}
794-
795-
/**
796-
* Attempt to use builder API via reflection to call the replicate method on the given builder.
797-
*/
798-
static void replicate(FSDataOutputStreamBuilder<?, ?> builder) {
799-
if (BUILDER != null && REPLICATE != null && BUILDER.isAssignableFrom(builder.getClass())) {
800-
try {
801-
REPLICATE.invoke(builder);
802-
} catch (IllegalAccessException | InvocationTargetException e) {
803-
// Should have caught this failure during initialization, so log full trace here
804-
LOG.warn("Couldn't use reflection with builder API", e);
805-
}
806-
}
807-
}
808-
}
809-
810-
/**
811-
* Attempt to use builder API via reflection to create a file with the given parameters and
812-
* replication enabled.
813-
* <p/>
814-
* Will not attempt to enable replication when passed an HFileSystem.
815-
*/
816-
public static FSDataOutputStream createForWal(FileSystem fs, Path path, boolean overwrite)
817-
throws IOException {
818-
FSDataOutputStreamBuilder<?, ?> builder = fs.createFile(path).overwrite(overwrite);
819-
DfsBuilderUtility.replicate(builder);
820-
return builder.build();
821-
}
822-
823-
/**
824-
* Attempt to use builder API via reflection to create a file with the given parameters and
825-
* replication enabled.
826-
* <p/>
827-
* Will not attempt to enable replication when passed an HFileSystem.
828-
*/
829-
public static FSDataOutputStream createForWal(FileSystem fs, Path path, boolean overwrite,
830-
int bufferSize, short replication, long blockSize, boolean isRecursive) throws IOException {
831-
FSDataOutputStreamBuilder<?, ?> builder = fs.createFile(path).overwrite(overwrite)
832-
.bufferSize(bufferSize).replication(replication).blockSize(blockSize);
833-
if (isRecursive) {
834-
builder.recursive();
835-
}
836-
DfsBuilderUtility.replicate(builder);
837-
return builder.build();
838-
}
839-
840766
/**
841767
* Helper exception for those cases where the place where we need to check a stream capability
842768
* is not where we have the needed context to explain the impact and mitigation for a lack.

hbase-procedure/pom.xml

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -144,6 +144,10 @@
144144
<artifactId>log4j-slf4j-impl</artifactId>
145145
<scope>test</scope>
146146
</dependency>
147+
<dependency>
148+
<artifactId>hadoop-hdfs-client</artifactId>
149+
<groupId>org.apache.hadoop</groupId>
150+
</dependency>
147151
</dependencies>
148152

149153
<profiles>

hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/WALProcedureStore.java

Lines changed: 9 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -35,6 +35,7 @@
3535
import java.util.concurrent.locks.ReentrantLock;
3636
import org.apache.hadoop.conf.Configuration;
3737
import org.apache.hadoop.fs.FSDataOutputStream;
38+
import org.apache.hadoop.fs.FSDataOutputStreamBuilder;
3839
import org.apache.hadoop.fs.FSError;
3940
import org.apache.hadoop.fs.FileAlreadyExistsException;
4041
import org.apache.hadoop.fs.FileStatus;
@@ -55,6 +56,7 @@
5556
import org.apache.hadoop.hbase.util.CommonFSUtils;
5657
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
5758
import org.apache.hadoop.hbase.util.Threads;
59+
import org.apache.hadoop.hdfs.DistributedFileSystem;
5860
import org.apache.hadoop.ipc.RemoteException;
5961
import org.apache.yetus.audience.InterfaceAudience;
6062
import org.slf4j.Logger;
@@ -1070,7 +1072,13 @@ boolean rollWriter(long logId) throws IOException {
10701072
long startPos = -1;
10711073
newLogFile = getLogFilePath(logId);
10721074
try {
1073-
newStream = CommonFSUtils.createForWal(fs, newLogFile, false);
1075+
FSDataOutputStreamBuilder<?, ?> builder = fs.createFile(newLogFile).overwrite(false);
1076+
if (builder instanceof DistributedFileSystem.HdfsDataOutputStreamBuilder) {
1077+
newStream = ((DistributedFileSystem.HdfsDataOutputStreamBuilder) builder)
1078+
.replicate().build();
1079+
} else {
1080+
newStream = builder.build();
1081+
}
10741082
} catch (FileAlreadyExistsException e) {
10751083
LOG.error("Log file with id={} already exists", logId, e);
10761084
return false;

hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogWriter.java

Lines changed: 15 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -22,6 +22,7 @@
2222
import java.util.concurrent.atomic.AtomicLong;
2323

2424
import org.apache.hadoop.fs.FSDataOutputStream;
25+
import org.apache.hadoop.fs.FSDataOutputStreamBuilder;
2526
import org.apache.hadoop.fs.FileSystem;
2627
import org.apache.hadoop.fs.Path;
2728
import org.apache.hadoop.fs.StreamCapabilities;
@@ -31,6 +32,7 @@
3132
import org.apache.hadoop.hbase.util.CommonFSUtils.StreamLacksCapabilityException;
3233
import org.apache.hadoop.hbase.wal.FSHLogProvider;
3334
import org.apache.hadoop.hbase.wal.WAL.Entry;
35+
import org.apache.hadoop.hdfs.DistributedFileSystem;
3436
import org.apache.yetus.audience.InterfaceAudience;
3537
import org.slf4j.Logger;
3638
import org.slf4j.LoggerFactory;
@@ -105,8 +107,19 @@ public FSDataOutputStream getStream() {
105107
@Override
106108
protected void initOutput(FileSystem fs, Path path, boolean overwritable, int bufferSize,
107109
short replication, long blockSize) throws IOException, StreamLacksCapabilityException {
108-
this.output = CommonFSUtils.createForWal(fs, path, overwritable, bufferSize, replication,
109-
blockSize, false);
110+
FSDataOutputStreamBuilder<?, ?> builder = fs
111+
.createFile(path)
112+
.overwrite(overwritable)
113+
.bufferSize(bufferSize)
114+
.replication(replication)
115+
.blockSize(blockSize);
116+
if (builder instanceof DistributedFileSystem.HdfsDataOutputStreamBuilder) {
117+
this.output = ((DistributedFileSystem.HdfsDataOutputStreamBuilder) builder)
118+
.replicate().build();
119+
} else {
120+
this.output = builder.build();
121+
}
122+
110123
if (fs.getConf().getBoolean(CommonFSUtils.UNSAFE_STREAM_CAPABILITY_ENFORCE, true)) {
111124
if (!output.hasCapability(StreamCapabilities.HFLUSH)) {
112125
throw new StreamLacksCapabilityException(StreamCapabilities.HFLUSH);

0 commit comments

Comments
 (0)