Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

HDDS-11342. [hsync] Add a config as HBase-related features master switch #7126

Merged
merged 13 commits into from
Sep 5, 2024
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -247,28 +247,49 @@ public enum ChecksumCombineMode {
tags = ConfigTag.CLIENT)
private String fsDefaultBucketLayout = "FILE_SYSTEM_OPTIMIZED";

// ozone.client.hbase.enhancements.allowed
@Config(key = "hbase.enhancements.allowed",
defaultValue = "false",
description = "When set to false, client-side HBase enhancement-related Ozone (experimental) features " +
"are disabled (not allowed to be enabled) regardless of whether those configs are set.\n" +
"\n" +
"Here is the list of configs and values overridden when this config is set to false:\n" +
"1. ozone.fs.hsync.enabled = false\n" +
"2. ozone.client.incremental.chunk.list = false\n" +
"3. ozone.client.stream.putblock.piggybacking = false\n" +
"4. ozone.client.key.write.concurrency = 1\n" +
"\n" +
"A warning message will be printed if any of the above configs are overridden by this.",
tags = ConfigTag.CLIENT)
private boolean hbaseEnhancementsAllowed = false;

// ozone.client.incremental.chunk.list
@Config(key = "incremental.chunk.list",
defaultValue = "true",
defaultValue = "false",
type = ConfigType.BOOLEAN,
description = "Client PutBlock request can choose incremental chunk " +
"list rather than full chunk list to optimize performance. " +
"Critical to HBase. EC does not support this feature.",
"Critical to HBase. EC does not support this feature. " +
"Can be enabled only when ozone.client.hbase.enhancements.allowed = true",
tags = ConfigTag.CLIENT)
private boolean incrementalChunkList = true;
private boolean incrementalChunkList = false;

// ozone.client.stream.putblock.piggybacking
@Config(key = "stream.putblock.piggybacking",
defaultValue = "true",
defaultValue = "false",
type = ConfigType.BOOLEAN,
description = "Allow PutBlock to be piggybacked in WriteChunk " +
"requests if the chunk is small.",
description = "Allow PutBlock to be piggybacked in WriteChunk requests if the chunk is small. " +
"Can be enabled only when ozone.client.hbase.enhancements.allowed = true",
tags = ConfigTag.CLIENT)
private boolean enablePutblockPiggybacking = true;
private boolean enablePutblockPiggybacking = false;

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

Expand Down Expand Up @@ -298,6 +319,34 @@ public void validate() {
OzoneConfigKeys.OZONE_CLIENT_BYTES_PER_CHECKSUM_MIN_SIZE;
}

// Verify client configs related to HBase enhancements
// Enforce check on ozone.client.hbase.enhancements.allowed
if (!hbaseEnhancementsAllowed) {
// ozone.client.hbase.enhancements.allowed = false
if (incrementalChunkList) {
LOG.warn("Ignoring ozone.client.incremental.chunk.list = true " +
"because HBase enhancements are disallowed. " +
"To enable it, set ozone.client.hbase.enhancements.allowed = true.");
incrementalChunkList = false;
LOG.debug("Final ozone.client.incremental.chunk.list = {}", incrementalChunkList);
}
if (enablePutblockPiggybacking) {
LOG.warn("Ignoring ozone.client.stream.putblock.piggybacking = true " +
"because HBase enhancements are disallowed. " +
"To enable it, set ozone.client.hbase.enhancements.allowed = true.");
enablePutblockPiggybacking = false;
LOG.debug("Final ozone.client.stream.putblock.piggybacking = {}", enablePutblockPiggybacking);
}
if (maxConcurrentWritePerKey != 1) {
LOG.warn("Ignoring ozone.client.key.write.concurrency = {} " +
"because HBase enhancements are disallowed. " +
"To enable it, set ozone.client.hbase.enhancements.allowed = true.",
maxConcurrentWritePerKey);
maxConcurrentWritePerKey = 1;
LOG.debug("Final ozone.client.key.write.concurrency = {}", maxConcurrentWritePerKey);
}
// Note: ozone.fs.hsync.enabled is enforced by OzoneFSUtils#canEnableHsync, not here
}
}

public long getStreamBufferFlushSize() {
Expand Down Expand Up @@ -486,6 +535,14 @@ public void setDatastreamPipelineMode(boolean datastreamPipelineMode) {
this.datastreamPipelineMode = datastreamPipelineMode;
}

public void setHBaseEnhancementsAllowed(boolean isHBaseEnhancementsEnabled) {
this.hbaseEnhancementsAllowed = isHBaseEnhancementsEnabled;
}

public boolean getHBaseEnhancementsAllowed() {
return this.hbaseEnhancementsAllowed;
}

public void setIncrementalChunkList(boolean enable) {
this.incrementalChunkList = enable;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -198,9 +198,7 @@ public BlockOutputStream(
blkIDBuilder.build()).addMetadata(keyValue);
this.pipeline = pipeline;
// tell DataNode I will send incremental chunk list
// EC does not support incremental chunk list.
this.supportIncrementalChunkList = config.getIncrementalChunkList() &&
this instanceof RatisBlockOutputStream && allDataNodesSupportPiggybacking();
this.supportIncrementalChunkList = canEnableIncrementalChunkList();
LOG.debug("incrementalChunkList is {}", supportIncrementalChunkList);
if (supportIncrementalChunkList) {
this.containerBlockData.addMetadata(INCREMENTAL_CHUNK_LIST_KV);
Expand Down Expand Up @@ -237,11 +235,51 @@ public BlockOutputStream(
config.getBytesPerChecksum());
this.clientMetrics = clientMetrics;
this.streamBufferArgs = streamBufferArgs;
this.allowPutBlockPiggybacking = config.getEnablePutblockPiggybacking() &&
allDataNodesSupportPiggybacking();
this.allowPutBlockPiggybacking = canEnablePutblockPiggybacking();
LOG.debug("PutBlock piggybacking is {}", allowPutBlockPiggybacking);
}

/**
* Helper method to check if incremental chunk list can be enabled.
* Prints debug messages if it cannot be enabled.
*/
private boolean canEnableIncrementalChunkList() {
boolean confEnableIncrementalChunkList = config.getIncrementalChunkList();
if (!confEnableIncrementalChunkList) {
return false;
}

if (!(this instanceof RatisBlockOutputStream)) {
// Note: EC does not support incremental chunk list
LOG.debug("Unable to enable incrementalChunkList because BlockOutputStream is not a RatisBlockOutputStream");
return false;
}
if (!allDataNodesSupportPiggybacking()) {
// Not all datanodes support piggybacking and incremental chunk list.
LOG.debug("Unable to enable incrementalChunkList because not all datanodes support piggybacking");
return false;
}
return confEnableIncrementalChunkList;
}

/**
* Helper method to check if PutBlock piggybacking can be enabled.
* Prints debug message if it cannot be enabled.
*/
private boolean canEnablePutblockPiggybacking() {
boolean confEnablePutblockPiggybacking = config.getEnablePutblockPiggybacking();
if (!confEnablePutblockPiggybacking) {
return false;
}

if (!allDataNodesSupportPiggybacking()) {
// Not all datanodes support piggybacking and incremental chunk list.
LOG.debug("Unable to enable PutBlock piggybacking because not all datanodes support piggybacking");
return false;
}
return confEnablePutblockPiggybacking;
}

private boolean allDataNodesSupportPiggybacking() {
// return true only if all DataNodes in the pipeline are on a version
// that supports PutBlock piggybacking.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,8 @@

import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_CLIENT_BYTES_PER_CHECKSUM_MIN_SIZE;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertTrue;

class TestOzoneClientConfig {

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

assertEquals(OZONE_CLIENT_BYTES_PER_CHECKSUM_MIN_SIZE, subject.getBytesPerChecksum());
}

@Test
void testClientHBaseEnhancementsAllowedTrue() {
// When ozone.client.hbase.enhancements.allowed = true,
// related client configs should be effective as-is.
OzoneConfiguration conf = new OzoneConfiguration();
conf.setBoolean("ozone.client.hbase.enhancements.allowed", true);

// Note: ozone.fs.hsync.enabled is checked by OzoneFSUtils.canEnableHsync(), thus not checked here
conf.setBoolean("ozone.client.incremental.chunk.list", true);
conf.setBoolean("ozone.client.stream.putblock.piggybacking", true);
conf.setInt("ozone.client.key.write.concurrency", -1);

OzoneClientConfig subject = conf.getObject(OzoneClientConfig.class);

assertTrue(subject.getIncrementalChunkList());
assertTrue(subject.getEnablePutblockPiggybacking());
assertEquals(-1, subject.getMaxConcurrentWritePerKey());
}

@Test
void testClientHBaseEnhancementsAllowedFalse() {
// When ozone.client.hbase.enhancements.allowed = false,
// related client configs should be reverted back to default.
OzoneConfiguration conf = new OzoneConfiguration();
conf.setBoolean("ozone.client.hbase.enhancements.allowed", false);

// Note: ozone.fs.hsync.enabled is checked by OzoneFSUtils.canEnableHsync(), thus not checked here
conf.setBoolean("ozone.client.incremental.chunk.list", true);
conf.setBoolean("ozone.client.stream.putblock.piggybacking", true);
conf.setInt("ozone.client.key.write.concurrency", -1);

OzoneClientConfig subject = conf.getObject(OzoneClientConfig.class);

assertFalse(subject.getIncrementalChunkList());
assertFalse(subject.getEnablePutblockPiggybacking());
assertEquals(1, subject.getMaxConcurrentWritePerKey());
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -120,6 +120,14 @@ public final class OzoneConfigKeys {
public static final String OZONE_FS_DATASTREAM_AUTO_THRESHOLD_DEFAULT
= "4MB";

/**
* Flag to allow server-side HBase-related features and enhancements to be enabled.
*/
public static final String OZONE_HBASE_ENHANCEMENTS_ALLOWED
= "ozone.hbase.enhancements.allowed";
public static final boolean OZONE_HBASE_ENHANCEMENTS_ALLOWED_DEFAULT
= false;

/**
* Flag to enable hsync/hflush.
*/
Expand Down
19 changes: 17 additions & 2 deletions hadoop-hdds/common/src/main/resources/ozone-default.xml
Original file line number Diff line number Diff line change
Expand Up @@ -4216,12 +4216,27 @@
</description>
</property>

<property>
<name>ozone.hbase.enhancements.allowed</name>
<value>false</value>
<tag>OZONE, OM</tag>
<description>
When set to false, server-side HBase enhancement-related Ozone (experimental) features
are disabled (not allowed to be enabled) regardless of whether those configs are set.

Here is the list of configs and values overridden when this config is set to false:
1. ozone.fs.hsync.enabled = false

A warning message will be printed if any of the above configs are overridden by this.
</description>
</property>
<property>
<name>ozone.fs.hsync.enabled</name>
<value>false</value>
<tag>OZONE, CLIENT</tag>
<tag>OZONE, CLIENT, OM</tag>
<description>
Enable hsync/hflush. By default they are disabled.
Enable hsync/hflush on the Ozone Manager and/or client side. Disabled by default.
Can be enabled only when ozone.hbase.enhancements.allowed = true
</description>
</property>
<property>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,7 @@ public class KeyOutputStreamSemaphore {
private final Semaphore requestSemaphore;

KeyOutputStreamSemaphore(int maxConcurrentWritePerKey) {
LOG.info("Initializing semaphore with maxConcurrentWritePerKey = {}", maxConcurrentWritePerKey);
LOG.debug("Initializing semaphore with maxConcurrentWritePerKey = {}", maxConcurrentWritePerKey);
if (maxConcurrentWritePerKey > 0) {
requestSemaphore = new Semaphore(maxConcurrentWritePerKey);
} else if (maxConcurrentWritePerKey == 0) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@
import org.apache.hadoop.crypto.key.KeyProvider;
import org.apache.hadoop.fs.FileEncryptionInfo;
import org.apache.hadoop.fs.Syncable;
import org.apache.hadoop.ozone.om.helpers.OzoneFSUtils;
import org.apache.hadoop.util.Time;
import org.apache.hadoop.hdds.client.DefaultReplicationConfig;
import org.apache.hadoop.hdds.client.ECReplicationConfig;
Expand Down Expand Up @@ -2489,9 +2490,7 @@ private OzoneOutputStream createOutputStream(OpenKeySession openKey)
private OzoneOutputStream createOutputStream(OpenKeySession openKey,
KeyOutputStream keyOutputStream)
throws IOException {
boolean enableHsync = conf.getBoolean(
OzoneConfigKeys.OZONE_FS_HSYNC_ENABLED,
OzoneConfigKeys.OZONE_FS_HSYNC_ENABLED_DEFAULT);
boolean enableHsync = OzoneFSUtils.canEnableHsync(conf, true);
keyOutputStream
.addPreallocateBlocks(openKey.getKeyInfo().getLatestVersionLocations(),
openKey.getOpenVersion());
Expand All @@ -2503,9 +2502,7 @@ private OzoneOutputStream createOutputStream(OpenKeySession openKey,

private OzoneOutputStream createSecureOutputStream(OpenKeySession openKey,
OutputStream keyOutputStream, Syncable syncable) throws IOException {
boolean enableHsync = conf.getBoolean(
OzoneConfigKeys.OZONE_FS_HSYNC_ENABLED,
OzoneConfigKeys.OZONE_FS_HSYNC_ENABLED_DEFAULT);
boolean enableHsync = OzoneFSUtils.canEnableHsync(conf, true);
final FileEncryptionInfo feInfo =
openKey.getKeyInfo().getFileEncryptionInfo();
if (feInfo != null) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -71,6 +71,8 @@ private void init(boolean incrementalChunkList) throws IOException {

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

((InMemoryConfiguration) config).setBoolean(
OzoneConfigKeys.OZONE_HBASE_ENHANCEMENTS_ALLOWED, true);
((InMemoryConfiguration) config).setBoolean(
OzoneConfigKeys.OZONE_FS_HSYNC_ENABLED, true);

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,10 +18,15 @@
package org.apache.hadoop.ozone.om.helpers;

import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdds.conf.ConfigurationSource;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.util.Time;

import jakarta.annotation.Nonnull;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.nio.file.Paths;
import java.util.UUID;

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

private OzoneFSUtils() { }

Expand Down Expand Up @@ -292,4 +298,29 @@ public static Path trimPathToDepth(Path path, int maxDepth) {
}
return res;
}

/**
* Helper method to return whether Hsync can be enabled.
* And print warning when the config is ignored.
*/
public static boolean canEnableHsync(ConfigurationSource conf, boolean isClient) {
final String confKey = isClient ?
"ozone.client.hbase.enhancements.allowed" :
OzoneConfigKeys.OZONE_HBASE_ENHANCEMENTS_ALLOWED;

boolean confHBaseEnhancementsAllowed = conf.getBoolean(
confKey, OzoneConfigKeys.OZONE_HBASE_ENHANCEMENTS_ALLOWED_DEFAULT);

boolean confHsyncEnabled = conf.getBoolean(
OzoneConfigKeys.OZONE_FS_HSYNC_ENABLED, OzoneConfigKeys.OZONE_FS_HSYNC_ENABLED_DEFAULT);

if (confHBaseEnhancementsAllowed) {
return confHsyncEnabled;
} else {
LOG.warn("Ignoring {} = {} because HBase enhancements are disallowed. To enable it, set {} = true as well.",
OzoneConfigKeys.OZONE_FS_HSYNC_ENABLED, confHsyncEnabled,
confKey);
return false;
}
}
}
Loading