Skip to content

Commit 1e5a6aa

Browse files
authored
Merge branch 'apache:master' into HDDS-11420
2 parents ef78593 + 7a452ca commit 1e5a6aa

File tree

30 files changed

+2145
-1385
lines changed

30 files changed

+2145
-1385
lines changed

hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/OzoneClientConfig.java

Lines changed: 65 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -247,28 +247,49 @@ public enum ChecksumCombineMode {
247247
tags = ConfigTag.CLIENT)
248248
private String fsDefaultBucketLayout = "FILE_SYSTEM_OPTIMIZED";
249249

250+
// ozone.client.hbase.enhancements.allowed
251+
@Config(key = "hbase.enhancements.allowed",
252+
defaultValue = "false",
253+
description = "When set to false, client-side HBase enhancement-related Ozone (experimental) features " +
254+
"are disabled (not allowed to be enabled) regardless of whether those configs are set.\n" +
255+
"\n" +
256+
"Here is the list of configs and values overridden when this config is set to false:\n" +
257+
"1. ozone.fs.hsync.enabled = false\n" +
258+
"2. ozone.client.incremental.chunk.list = false\n" +
259+
"3. ozone.client.stream.putblock.piggybacking = false\n" +
260+
"4. ozone.client.key.write.concurrency = 1\n" +
261+
"\n" +
262+
"A warning message will be printed if any of the above configs are overridden by this.",
263+
tags = ConfigTag.CLIENT)
264+
private boolean hbaseEnhancementsAllowed = false;
265+
266+
// ozone.client.incremental.chunk.list
250267
@Config(key = "incremental.chunk.list",
251-
defaultValue = "true",
268+
defaultValue = "false",
252269
type = ConfigType.BOOLEAN,
253270
description = "Client PutBlock request can choose incremental chunk " +
254271
"list rather than full chunk list to optimize performance. " +
255-
"Critical to HBase. EC does not support this feature.",
272+
"Critical to HBase. EC does not support this feature. " +
273+
"Can be enabled only when ozone.client.hbase.enhancements.allowed = true",
256274
tags = ConfigTag.CLIENT)
257-
private boolean incrementalChunkList = true;
275+
private boolean incrementalChunkList = false;
258276

277+
// ozone.client.stream.putblock.piggybacking
259278
@Config(key = "stream.putblock.piggybacking",
260-
defaultValue = "true",
279+
defaultValue = "false",
261280
type = ConfigType.BOOLEAN,
262-
description = "Allow PutBlock to be piggybacked in WriteChunk " +
263-
"requests if the chunk is small.",
281+
description = "Allow PutBlock to be piggybacked in WriteChunk requests if the chunk is small. " +
282+
"Can be enabled only when ozone.client.hbase.enhancements.allowed = true",
264283
tags = ConfigTag.CLIENT)
265-
private boolean enablePutblockPiggybacking = true;
284+
private boolean enablePutblockPiggybacking = false;
266285

286+
// ozone.client.key.write.concurrency
267287
@Config(key = "key.write.concurrency",
268288
defaultValue = "1",
269289
description = "Maximum concurrent writes allowed on each key. " +
270290
"Defaults to 1 which matches the behavior before HDDS-9844. " +
271-
"For unlimited write concurrency, set this to -1 or any negative integer value.",
291+
"For unlimited write concurrency, set this to -1 or any negative integer value. " +
292+
"Any value other than 1 is effective only when ozone.client.hbase.enhancements.allowed = true",
272293
tags = ConfigTag.CLIENT)
273294
private int maxConcurrentWritePerKey = 1;
274295

@@ -298,6 +319,34 @@ public void validate() {
298319
OzoneConfigKeys.OZONE_CLIENT_BYTES_PER_CHECKSUM_MIN_SIZE;
299320
}
300321

322+
// Verify client configs related to HBase enhancements
323+
// Enforce check on ozone.client.hbase.enhancements.allowed
324+
if (!hbaseEnhancementsAllowed) {
325+
// ozone.client.hbase.enhancements.allowed = false
326+
if (incrementalChunkList) {
327+
LOG.warn("Ignoring ozone.client.incremental.chunk.list = true " +
328+
"because HBase enhancements are disallowed. " +
329+
"To enable it, set ozone.client.hbase.enhancements.allowed = true.");
330+
incrementalChunkList = false;
331+
LOG.debug("Final ozone.client.incremental.chunk.list = {}", incrementalChunkList);
332+
}
333+
if (enablePutblockPiggybacking) {
334+
LOG.warn("Ignoring ozone.client.stream.putblock.piggybacking = true " +
335+
"because HBase enhancements are disallowed. " +
336+
"To enable it, set ozone.client.hbase.enhancements.allowed = true.");
337+
enablePutblockPiggybacking = false;
338+
LOG.debug("Final ozone.client.stream.putblock.piggybacking = {}", enablePutblockPiggybacking);
339+
}
340+
if (maxConcurrentWritePerKey != 1) {
341+
LOG.warn("Ignoring ozone.client.key.write.concurrency = {} " +
342+
"because HBase enhancements are disallowed. " +
343+
"To enable it, set ozone.client.hbase.enhancements.allowed = true.",
344+
maxConcurrentWritePerKey);
345+
maxConcurrentWritePerKey = 1;
346+
LOG.debug("Final ozone.client.key.write.concurrency = {}", maxConcurrentWritePerKey);
347+
}
348+
// Note: ozone.fs.hsync.enabled is enforced by OzoneFSUtils#canEnableHsync, not here
349+
}
301350
}
302351

303352
public long getStreamBufferFlushSize() {
@@ -486,6 +535,14 @@ public void setDatastreamPipelineMode(boolean datastreamPipelineMode) {
486535
this.datastreamPipelineMode = datastreamPipelineMode;
487536
}
488537

538+
public void setHBaseEnhancementsAllowed(boolean isHBaseEnhancementsEnabled) {
539+
this.hbaseEnhancementsAllowed = isHBaseEnhancementsEnabled;
540+
}
541+
542+
public boolean getHBaseEnhancementsAllowed() {
543+
return this.hbaseEnhancementsAllowed;
544+
}
545+
489546
public void setIncrementalChunkList(boolean enable) {
490547
this.incrementalChunkList = enable;
491548
}

hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockOutputStream.java

Lines changed: 43 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -198,9 +198,7 @@ public BlockOutputStream(
198198
blkIDBuilder.build()).addMetadata(keyValue);
199199
this.pipeline = pipeline;
200200
// tell DataNode I will send incremental chunk list
201-
// EC does not support incremental chunk list.
202-
this.supportIncrementalChunkList = config.getIncrementalChunkList() &&
203-
this instanceof RatisBlockOutputStream && allDataNodesSupportPiggybacking();
201+
this.supportIncrementalChunkList = canEnableIncrementalChunkList();
204202
LOG.debug("incrementalChunkList is {}", supportIncrementalChunkList);
205203
if (supportIncrementalChunkList) {
206204
this.containerBlockData.addMetadata(INCREMENTAL_CHUNK_LIST_KV);
@@ -237,11 +235,51 @@ public BlockOutputStream(
237235
config.getBytesPerChecksum());
238236
this.clientMetrics = clientMetrics;
239237
this.streamBufferArgs = streamBufferArgs;
240-
this.allowPutBlockPiggybacking = config.getEnablePutblockPiggybacking() &&
241-
allDataNodesSupportPiggybacking();
238+
this.allowPutBlockPiggybacking = canEnablePutblockPiggybacking();
242239
LOG.debug("PutBlock piggybacking is {}", allowPutBlockPiggybacking);
243240
}
244241

242+
/**
243+
* Helper method to check if incremental chunk list can be enabled.
244+
* Prints debug messages if it cannot be enabled.
245+
*/
246+
private boolean canEnableIncrementalChunkList() {
247+
boolean confEnableIncrementalChunkList = config.getIncrementalChunkList();
248+
if (!confEnableIncrementalChunkList) {
249+
return false;
250+
}
251+
252+
if (!(this instanceof RatisBlockOutputStream)) {
253+
// Note: EC does not support incremental chunk list
254+
LOG.debug("Unable to enable incrementalChunkList because BlockOutputStream is not a RatisBlockOutputStream");
255+
return false;
256+
}
257+
if (!allDataNodesSupportPiggybacking()) {
258+
// Not all datanodes support piggybacking and incremental chunk list.
259+
LOG.debug("Unable to enable incrementalChunkList because not all datanodes support piggybacking");
260+
return false;
261+
}
262+
return confEnableIncrementalChunkList;
263+
}
264+
265+
/**
266+
* Helper method to check if PutBlock piggybacking can be enabled.
267+
* Prints debug message if it cannot be enabled.
268+
*/
269+
private boolean canEnablePutblockPiggybacking() {
270+
boolean confEnablePutblockPiggybacking = config.getEnablePutblockPiggybacking();
271+
if (!confEnablePutblockPiggybacking) {
272+
return false;
273+
}
274+
275+
if (!allDataNodesSupportPiggybacking()) {
276+
// Not all datanodes support piggybacking and incremental chunk list.
277+
LOG.debug("Unable to enable PutBlock piggybacking because not all datanodes support piggybacking");
278+
return false;
279+
}
280+
return confEnablePutblockPiggybacking;
281+
}
282+
245283
private boolean allDataNodesSupportPiggybacking() {
246284
// return true only if all DataNodes in the pipeline are on a version
247285
// that supports PutBlock piggybacking.

hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/TestOzoneClientConfig.java

Lines changed: 40 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -22,6 +22,8 @@
2222

2323
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_CLIENT_BYTES_PER_CHECKSUM_MIN_SIZE;
2424
import static org.junit.jupiter.api.Assertions.assertEquals;
25+
import static org.junit.jupiter.api.Assertions.assertFalse;
26+
import static org.junit.jupiter.api.Assertions.assertTrue;
2527

2628
class TestOzoneClientConfig {
2729

@@ -36,4 +38,42 @@ void missingSizeSuffix() {
3638

3739
assertEquals(OZONE_CLIENT_BYTES_PER_CHECKSUM_MIN_SIZE, subject.getBytesPerChecksum());
3840
}
41+
42+
@Test
43+
void testClientHBaseEnhancementsAllowedTrue() {
44+
// When ozone.client.hbase.enhancements.allowed = true,
45+
// related client configs should be effective as-is.
46+
OzoneConfiguration conf = new OzoneConfiguration();
47+
conf.setBoolean("ozone.client.hbase.enhancements.allowed", true);
48+
49+
// Note: ozone.fs.hsync.enabled is checked by OzoneFSUtils.canEnableHsync(), thus not checked here
50+
conf.setBoolean("ozone.client.incremental.chunk.list", true);
51+
conf.setBoolean("ozone.client.stream.putblock.piggybacking", true);
52+
conf.setInt("ozone.client.key.write.concurrency", -1);
53+
54+
OzoneClientConfig subject = conf.getObject(OzoneClientConfig.class);
55+
56+
assertTrue(subject.getIncrementalChunkList());
57+
assertTrue(subject.getEnablePutblockPiggybacking());
58+
assertEquals(-1, subject.getMaxConcurrentWritePerKey());
59+
}
60+
61+
@Test
62+
void testClientHBaseEnhancementsAllowedFalse() {
63+
// When ozone.client.hbase.enhancements.allowed = false,
64+
// related client configs should be reverted back to default.
65+
OzoneConfiguration conf = new OzoneConfiguration();
66+
conf.setBoolean("ozone.client.hbase.enhancements.allowed", false);
67+
68+
// Note: ozone.fs.hsync.enabled is checked by OzoneFSUtils.canEnableHsync(), thus not checked here
69+
conf.setBoolean("ozone.client.incremental.chunk.list", true);
70+
conf.setBoolean("ozone.client.stream.putblock.piggybacking", true);
71+
conf.setInt("ozone.client.key.write.concurrency", -1);
72+
73+
OzoneClientConfig subject = conf.getObject(OzoneClientConfig.class);
74+
75+
assertFalse(subject.getIncrementalChunkList());
76+
assertFalse(subject.getEnablePutblockPiggybacking());
77+
assertEquals(1, subject.getMaxConcurrentWritePerKey());
78+
}
3979
}

hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java

Lines changed: 8 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -120,6 +120,14 @@ public final class OzoneConfigKeys {
120120
public static final String OZONE_FS_DATASTREAM_AUTO_THRESHOLD_DEFAULT
121121
= "4MB";
122122

123+
/**
124+
* Flag to allow server-side HBase-related features and enhancements to be enabled.
125+
*/
126+
public static final String OZONE_HBASE_ENHANCEMENTS_ALLOWED
127+
= "ozone.hbase.enhancements.allowed";
128+
public static final boolean OZONE_HBASE_ENHANCEMENTS_ALLOWED_DEFAULT
129+
= false;
130+
123131
/**
124132
* Flag to enable hsync/hflush.
125133
*/

hadoop-hdds/common/src/main/resources/ozone-default.xml

Lines changed: 17 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -4216,12 +4216,27 @@
42164216
</description>
42174217
</property>
42184218

4219+
<property>
4220+
<name>ozone.hbase.enhancements.allowed</name>
4221+
<value>false</value>
4222+
<tag>OZONE, OM</tag>
4223+
<description>
4224+
When set to false, server-side HBase enhancement-related Ozone (experimental) features
4225+
are disabled (not allowed to be enabled) regardless of whether those configs are set.
4226+
4227+
Here is the list of configs and values overridden when this config is set to false:
4228+
1. ozone.fs.hsync.enabled = false
4229+
4230+
A warning message will be printed if any of the above configs are overridden by this.
4231+
</description>
4232+
</property>
42194233
<property>
42204234
<name>ozone.fs.hsync.enabled</name>
42214235
<value>false</value>
4222-
<tag>OZONE, CLIENT</tag>
4236+
<tag>OZONE, CLIENT, OM</tag>
42234237
<description>
4224-
Enable hsync/hflush. By default they are disabled.
4238+
Enable hsync/hflush on the Ozone Manager and/or client side. Disabled by default.
4239+
Can be enabled only when ozone.hbase.enhancements.allowed = true
42254240
</description>
42264241
</property>
42274242
<property>

hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/KeyOutputStreamSemaphore.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -32,7 +32,7 @@ public class KeyOutputStreamSemaphore {
3232
private final Semaphore requestSemaphore;
3333

3434
KeyOutputStreamSemaphore(int maxConcurrentWritePerKey) {
35-
LOG.info("Initializing semaphore with maxConcurrentWritePerKey = {}", maxConcurrentWritePerKey);
35+
LOG.debug("Initializing semaphore with maxConcurrentWritePerKey = {}", maxConcurrentWritePerKey);
3636
if (maxConcurrentWritePerKey > 0) {
3737
requestSemaphore = new Semaphore(maxConcurrentWritePerKey);
3838
} else if (maxConcurrentWritePerKey == 0) {

hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java

Lines changed: 3 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -34,6 +34,7 @@
3434
import org.apache.hadoop.crypto.key.KeyProvider;
3535
import org.apache.hadoop.fs.FileEncryptionInfo;
3636
import org.apache.hadoop.fs.Syncable;
37+
import org.apache.hadoop.ozone.om.helpers.OzoneFSUtils;
3738
import org.apache.hadoop.util.Time;
3839
import org.apache.hadoop.hdds.client.DefaultReplicationConfig;
3940
import org.apache.hadoop.hdds.client.ECReplicationConfig;
@@ -2489,9 +2490,7 @@ private OzoneOutputStream createOutputStream(OpenKeySession openKey)
24892490
private OzoneOutputStream createOutputStream(OpenKeySession openKey,
24902491
KeyOutputStream keyOutputStream)
24912492
throws IOException {
2492-
boolean enableHsync = conf.getBoolean(
2493-
OzoneConfigKeys.OZONE_FS_HSYNC_ENABLED,
2494-
OzoneConfigKeys.OZONE_FS_HSYNC_ENABLED_DEFAULT);
2493+
boolean enableHsync = OzoneFSUtils.canEnableHsync(conf, true);
24952494
keyOutputStream
24962495
.addPreallocateBlocks(openKey.getKeyInfo().getLatestVersionLocations(),
24972496
openKey.getOpenVersion());
@@ -2503,9 +2502,7 @@ private OzoneOutputStream createOutputStream(OpenKeySession openKey,
25032502

25042503
private OzoneOutputStream createSecureOutputStream(OpenKeySession openKey,
25052504
OutputStream keyOutputStream, Syncable syncable) throws IOException {
2506-
boolean enableHsync = conf.getBoolean(
2507-
OzoneConfigKeys.OZONE_FS_HSYNC_ENABLED,
2508-
OzoneConfigKeys.OZONE_FS_HSYNC_ENABLED_DEFAULT);
2505+
boolean enableHsync = OzoneFSUtils.canEnableHsync(conf, true);
25092506
final FileEncryptionInfo feInfo =
25102507
openKey.getKeyInfo().getFileEncryptionInfo();
25112508
if (feInfo != null) {

hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/TestBlockOutputStreamIncrementalPutBlock.java

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -71,6 +71,8 @@ private void init(boolean incrementalChunkList) throws IOException {
7171

7272
((InMemoryConfiguration)config).setFromObject(clientConfig);
7373

74+
((InMemoryConfiguration) config).setBoolean(
75+
OzoneConfigKeys.OZONE_HBASE_ENHANCEMENTS_ALLOWED, true);
7476
((InMemoryConfiguration) config).setBoolean(
7577
OzoneConfigKeys.OZONE_FS_HSYNC_ENABLED, true);
7678

hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OzoneFSUtils.java

Lines changed: 31 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -18,10 +18,15 @@
1818
package org.apache.hadoop.ozone.om.helpers;
1919

2020
import org.apache.hadoop.fs.Path;
21+
import org.apache.hadoop.hdds.conf.ConfigurationSource;
22+
import org.apache.hadoop.ozone.OzoneConfigKeys;
2123
import org.apache.hadoop.util.StringUtils;
2224
import org.apache.hadoop.util.Time;
2325

2426
import jakarta.annotation.Nonnull;
27+
import org.slf4j.Logger;
28+
import org.slf4j.LoggerFactory;
29+
2530
import java.nio.file.Paths;
2631
import java.util.UUID;
2732

@@ -32,6 +37,7 @@
3237
* Utility class for OzoneFileSystem.
3338
*/
3439
public final class OzoneFSUtils {
40+
static final Logger LOG = LoggerFactory.getLogger(OzoneFSUtils.class);
3541

3642
private OzoneFSUtils() { }
3743

@@ -292,4 +298,29 @@ public static Path trimPathToDepth(Path path, int maxDepth) {
292298
}
293299
return res;
294300
}
301+
302+
/**
303+
* Helper method to return whether Hsync can be enabled.
304+
* And print warning when the config is ignored.
305+
*/
306+
public static boolean canEnableHsync(ConfigurationSource conf, boolean isClient) {
307+
final String confKey = isClient ?
308+
"ozone.client.hbase.enhancements.allowed" :
309+
OzoneConfigKeys.OZONE_HBASE_ENHANCEMENTS_ALLOWED;
310+
311+
boolean confHBaseEnhancementsAllowed = conf.getBoolean(
312+
confKey, OzoneConfigKeys.OZONE_HBASE_ENHANCEMENTS_ALLOWED_DEFAULT);
313+
314+
boolean confHsyncEnabled = conf.getBoolean(
315+
OzoneConfigKeys.OZONE_FS_HSYNC_ENABLED, OzoneConfigKeys.OZONE_FS_HSYNC_ENABLED_DEFAULT);
316+
317+
if (confHBaseEnhancementsAllowed) {
318+
return confHsyncEnabled;
319+
} else {
320+
LOG.warn("Ignoring {} = {} because HBase enhancements are disallowed. To enable it, set {} = true as well.",
321+
OzoneConfigKeys.OZONE_FS_HSYNC_ENABLED, confHsyncEnabled,
322+
confKey);
323+
return false;
324+
}
325+
}
295326
}

0 commit comments

Comments
 (0)