Skip to content

Commit bcde90c

Browse files
authored
Merge branch 'apache:master' into HDDS-11463
2 parents 1ef930b + b968353 commit bcde90c

File tree

453 files changed

+15238
-3062
lines changed

Some content is hidden

Large Commits have some content hidden by default. Use the searchbox below for content that may be hidden.

453 files changed

+15238
-3062
lines changed

CONTRIBUTING.md

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -90,6 +90,10 @@ When creating a new jira for any kind of new feature, improvement or bug, please
9090
* If it is follow up of another issue, please link the previous jira to it so that context is preserve.
9191
3. **Jira examples:** [HDDS-9272](https://issues.apache.org/jira/browse/HDDS-9272), [HDDS-9322](https://issues.apache.org/jira/browse/HDDS-9322), [HDDS-9291](https://issues.apache.org/jira/browse/HDDS-9291), [HDDS-8940](https://issues.apache.org/jira/browse/HDDS-8940), [HDDS-9282](https://issues.apache.org/jira/browse/HDDS-9282)
9292

93+
## New feature development
94+
95+
For large feature development changes, we use a process called "Ozone Enhancement Proposals" (OEP). This process is designed to ensure that major changes to Ozone are well-designed and have community consensus. If you are planning to propose a significant change, please read the [Ozone Enhancement Proposals](https://ozone.apache.org/docs/edge/design/ozone-enhancement-proposals.html) documentation and create a design document before you start coding. Please note that we only accept design documents in Markdown format; PDF or Google Docs are no longer accepted.
96+
9397
## Contribute your modifications
9498

9599
We use GitHub pull requests for contributing changes to the repository. The main workflow is as follows:

README.md

Lines changed: 13 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1,3 +1,16 @@
1+
<div align="center">
2+
<a href="https://ozone.apache.org">
3+
<img src="https://www.apache.org/logos/res/ozone/default.png" alt="Apache Ozone Logo" />
4+
</a>
5+
</div>
6+
7+
[![License](https://img.shields.io/:license-Apache%202-blue.svg)](https://www.apache.org/licenses/LICENSE-2.0.txt)
8+
[![Docker Pulls](https://img.shields.io/docker/pulls/apache/ozone.svg)](https://hub.docker.com/r/apache/ozone)
9+
[![Docker Stars](https://img.shields.io/docker/stars/apache/ozone.svg)](https://hub.docker.com/r/apache/ozone)
10+
[![Contributors](https://img.shields.io/github/contributors/apache/ozone)](https://github.com/apache/ozone/graphs/contributors)
11+
[![Commit Activity](https://img.shields.io/github/commit-activity/m/apache/ozone)](https://github.com/apache/ozone/commits/master)
12+
[![OSSRank](https://shields.io/endpoint?url=https://ossrank.com/shield/3018)](https://ossrank.com/p/3018-apache-ozone)
13+
114
Apache Ozone
215
===
316

Lines changed: 28 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,28 @@
1+
# Licensed to the Apache Software Foundation (ASF) under one or more
2+
# contributor license agreements. See the NOTICE file distributed with
3+
# this work for additional information regarding copyright ownership.
4+
# The ASF licenses this file to You under the Apache License, Version 2.0
5+
# (the "License"); you may not use this file except in compliance with
6+
# the License. You may obtain a copy of the License at
7+
#
8+
# http://www.apache.org/licenses/LICENSE-2.0
9+
#
10+
# Unless required by applicable law or agreed to in writing, software
11+
# distributed under the License is distributed on an "AS IS" BASIS,
12+
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13+
# See the License for the specific language governing permissions and
14+
# limitations under the License.
15+
16+
#
17+
# This script instruments ozone manager to fail a CreateBucket request for a specific name
18+
#
19+
20+
RULE Crash OM with CreateBucket
21+
CLASS org.apache.hadoop.ozone.om.request.bucket.OMBucketCreateRequest
22+
METHOD validateAndUpdateCache
23+
AT ENTRY
24+
IF TRUE
25+
DO
26+
traceln("--> crashing CreateBucket request");
27+
THROW new RuntimeException("Byteman crashes OM");
28+
ENDRULE

hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsConfigKeys.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -255,7 +255,7 @@ public final class HddsConfigKeys {
255255

256256
public static final String HDDS_SECRET_KEY_EXPIRY_DURATION =
257257
"hdds.secret.key.expiry.duration";
258-
public static final String HDDS_SECRET_KEY_EXPIRY_DURATION_DEFAULT = "7d";
258+
public static final String HDDS_SECRET_KEY_EXPIRY_DURATION_DEFAULT = "9d";
259259

260260
public static final String HDDS_SECRET_KEY_ROTATE_DURATION =
261261
"hdds.secret.key.rotate.duration";

hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfig.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -104,7 +104,7 @@ public class ScmConfig extends ReconfigurableConfig {
104104

105105
@Config(key = "hdds.scm.block.deletion.per-interval.max",
106106
type = ConfigType.INT,
107-
defaultValue = "100000",
107+
defaultValue = "500000",
108108
reconfigurable = true,
109109
tags = { ConfigTag.SCM, ConfigTag.DELETION},
110110
description =

hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java

Lines changed: 7 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -456,6 +456,11 @@ public final class ScmConfigKeys {
456456
"ozone.scm.block.deletion.max.retry";
457457
public static final int OZONE_SCM_BLOCK_DELETION_MAX_RETRY_DEFAULT = 4096;
458458

459+
public static final String OZONE_SCM_BLOCK_DELETION_PER_DN_DISTRIBUTION_FACTOR =
460+
"ozone.scm.block.deletion.per.dn.distribution.factor";
461+
462+
public static final int OZONE_SCM_BLOCK_DELETION_PER_DN_DISTRIBUTION_FACTOR_DEFAULT = 8;
463+
459464
public static final String OZONE_SCM_SEQUENCE_ID_BATCH_SIZE =
460465
"ozone.scm.sequence.id.batch.size";
461466
public static final int OZONE_SCM_SEQUENCE_ID_BATCH_SIZE_DEFAULT = 1000;
@@ -521,7 +526,7 @@ public final class ScmConfigKeys {
521526

522527
public static final String OZONE_SCM_HA_RAFT_SEGMENT_SIZE =
523528
"ozone.scm.ha.ratis.segment.size";
524-
public static final String OZONE_SCM_HA_RAFT_SEGMENT_SIZE_DEFAULT = "4MB";
529+
public static final String OZONE_SCM_HA_RAFT_SEGMENT_SIZE_DEFAULT = "64MB";
525530

526531
public static final String OZONE_SCM_HA_RAFT_SEGMENT_PRE_ALLOCATED_SIZE =
527532
"ozone.scm.ha.ratis.segment.preallocated.size";
@@ -617,7 +622,7 @@ public final class ScmConfigKeys {
617622
public static final String OZONE_SCM_HA_DBTRANSACTIONBUFFER_FLUSH_INTERVAL =
618623
"ozone.scm.ha.dbtransactionbuffer.flush.interval";
619624
public static final long
620-
OZONE_SCM_HA_DBTRANSACTIONBUFFER_FLUSH_INTERVAL_DEFAULT = 600 * 1000L;
625+
OZONE_SCM_HA_DBTRANSACTIONBUFFER_FLUSH_INTERVAL_DEFAULT = 60 * 1000L;
621626

622627
public static final String NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY =
623628
"net.topology.node.switch.mapping.impl";

hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/upgrade/HDDSLayoutFeature.java

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -41,7 +41,8 @@ public enum HDDSLayoutFeature implements LayoutFeature {
4141
HADOOP_PRC_PORTS_IN_DATANODEDETAILS(7, "Adding Hadoop RPC ports " +
4242
"to DatanodeDetails."),
4343
HBASE_SUPPORT(8, "Datanode RocksDB Schema Version 3 has an extra table " +
44-
"for the last chunk of blocks to support HBase.)");
44+
"for the last chunk of blocks to support HBase.)"),
45+
WITNESSED_CONTAINER_DB_PROTO_VALUE(9, "ContainerID table schema to use value type as proto");
4546

4647
////////////////////////////// //////////////////////////////
4748

hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/utils/BackgroundService.java

Lines changed: 43 additions & 26 deletions
Original file line numberDiff line numberDiff line change
@@ -77,11 +77,11 @@ protected CompletableFuture<Void> getFuture() {
7777
}
7878

7979
@VisibleForTesting
80-
public ExecutorService getExecutorService() {
80+
public synchronized ExecutorService getExecutorService() {
8181
return this.exec;
8282
}
8383

84-
public void setPoolSize(int size) {
84+
public synchronized void setPoolSize(int size) {
8585
if (size <= 0) {
8686
throw new IllegalArgumentException("Pool size must be positive.");
8787
}
@@ -108,6 +108,7 @@ public void runPeriodicalTaskNow() throws Exception {
108108
while (!tasks.isEmpty()) {
109109
tasks.poll().call();
110110
}
111+
execTaskCompletion();
111112
}
112113

113114
// start service
@@ -125,15 +126,30 @@ protected synchronized void setInterval(long newInterval, TimeUnit newUnit) {
125126
this.unit = newUnit;
126127
}
127128

129+
protected synchronized long getIntervalMillis() {
130+
return this.unit.toMillis(interval);
131+
}
132+
128133
public abstract BackgroundTaskQueue getTasks();
129134

135+
protected void execTaskCompletion() { }
136+
130137
/**
131138
* Run one or more background tasks concurrently.
132139
* Wait until all tasks to return the result.
133140
*/
134141
public class PeriodicalTask implements Runnable {
135142
@Override
136-
public synchronized void run() {
143+
public void run() {
144+
// wait for previous set of tasks to complete
145+
try {
146+
future.join();
147+
} catch (RuntimeException e) {
148+
LOG.error("Background service execution failed.", e);
149+
} finally {
150+
execTaskCompletion();
151+
}
152+
137153
if (LOG.isDebugEnabled()) {
138154
LOG.debug("Running background service : {}", serviceName);
139155
}
@@ -146,35 +162,36 @@ public synchronized void run() {
146162
if (LOG.isDebugEnabled()) {
147163
LOG.debug("Number of background tasks to execute : {}", tasks.size());
148164
}
149-
150-
while (!tasks.isEmpty()) {
151-
BackgroundTask task = tasks.poll();
152-
future = future.thenCombine(CompletableFuture.runAsync(() -> {
153-
long startTime = System.nanoTime();
154-
try {
155-
BackgroundTaskResult result = task.call();
156-
if (LOG.isDebugEnabled()) {
157-
LOG.debug("task execution result size {}", result.getSize());
158-
}
159-
} catch (Throwable e) {
160-
LOG.error("Background task execution failed", e);
161-
if (e instanceof Error) {
162-
throw (Error) e;
163-
}
164-
} finally {
165-
long endTime = System.nanoTime();
166-
if (endTime - startTime > serviceTimeoutInNanos) {
167-
LOG.warn("{} Background task execution took {}ns > {}ns(timeout)",
168-
serviceName, endTime - startTime, serviceTimeoutInNanos);
165+
synchronized (BackgroundService.this) {
166+
while (!tasks.isEmpty()) {
167+
BackgroundTask task = tasks.poll();
168+
future = future.thenCombine(CompletableFuture.runAsync(() -> {
169+
long startTime = System.nanoTime();
170+
try {
171+
BackgroundTaskResult result = task.call();
172+
if (LOG.isDebugEnabled()) {
173+
LOG.debug("task execution result size {}", result.getSize());
174+
}
175+
} catch (Throwable e) {
176+
LOG.error("Background task execution failed", e);
177+
if (e instanceof Error) {
178+
throw (Error) e;
179+
}
180+
} finally {
181+
long endTime = System.nanoTime();
182+
if (endTime - startTime > serviceTimeoutInNanos) {
183+
LOG.warn("{} Background task execution took {}ns > {}ns(timeout)",
184+
serviceName, endTime - startTime, serviceTimeoutInNanos);
185+
}
169186
}
170-
}
171-
}, exec), (Void1, Void) -> null);
187+
}, exec).exceptionally(e -> null), (Void1, Void) -> null);
188+
}
172189
}
173190
}
174191
}
175192

176193
// shutdown and make sure all threads are properly released.
177-
public void shutdown() {
194+
public synchronized void shutdown() {
178195
LOG.info("Shutting down service {}", this.serviceName);
179196
exec.shutdown();
180197
try {

0 commit comments

Comments
 (0)