This is an automated email from the ASF dual-hosted git repository.
szetszwo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ozone.git
The following commit(s) were added to refs/heads/master by this push:
new 037014de5f HDDS-9749. Infinite loop in
ReconUtils.nextClosestPowerIndexOfTwo(). (#5654)
037014de5f is described below
commit 037014de5f4a177406be5acf892091bca45fd39f
Author: Tsz-Wo Nicholas Sze <[email protected]>
AuthorDate: Wed Nov 22 21:36:15 2023 -0800
HDDS-9749. Infinite loop in ReconUtils.nextClosestPowerIndexOfTwo(). (#5654)
---
.../org/apache/hadoop/ozone/recon/ReconUtils.java | 17 ++++---
.../ozone/recon/tasks/ContainerSizeCountTask.java | 58 +++++++++++-----------
.../apache/hadoop/ozone/recon/TestReconUtils.java | 45 +++++++++++++++++
.../recon/tasks/TestContainerSizeCountTask.java | 6 +++
4 files changed, 91 insertions(+), 35 deletions(-)
diff --git
a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/ReconUtils.java
b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/ReconUtils.java
index 0a83088a0a..0d0c57fbe3 100644
---
a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/ReconUtils.java
+++
b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/ReconUtils.java
@@ -30,6 +30,7 @@ import java.nio.file.Path;
import java.nio.file.Paths;
import java.sql.Timestamp;
+import com.google.common.base.Preconditions;
import com.google.inject.Singleton;
import org.apache.hadoop.hdds.HddsConfigKeys;
import org.apache.hadoop.hdds.HddsUtils;
@@ -314,6 +315,8 @@ public class ReconUtils {
public static int getFileSizeBinIndex(long fileSize) {
+ Preconditions.checkArgument(fileSize >= 0,
+ "fileSize = %s < 0", fileSize);
// if the file size is larger than our track scope,
// we map it to the last bin
if (fileSize >= ReconConstants.MAX_FILE_SIZE_UPPER_BOUND) {
@@ -326,6 +329,8 @@ public class ReconUtils {
}
public static int getContainerSizeBinIndex(long containerSize) {
+ Preconditions.checkArgument(containerSize >= 0,
+ "containerSize = %s < 0", containerSize);
// if the container size is larger than our track scope,
// we map it to the last bin
if (containerSize >= ReconConstants.MAX_CONTAINER_SIZE_UPPER_BOUND) {
@@ -337,13 +342,11 @@ public class ReconUtils {
return index < 29 ? 0 : index - 29;
}
- private static int nextClosestPowerIndexOfTwo(long dataSize) {
- int index = 0;
- while (dataSize != 0) {
- dataSize >>= 1;
- index += 1;
- }
- return index;
+ static int nextClosestPowerIndexOfTwo(long n) {
+ return n > 0 ? 64 - Long.numberOfLeadingZeros(n - 1)
+ : n == 0 ? 0
+ : n == Long.MIN_VALUE ? -63
+ : -nextClosestPowerIndexOfTwo(-n);
}
public SCMNodeDetails getReconNodeDetails(OzoneConfiguration conf) {
diff --git
a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerSizeCountTask.java
b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerSizeCountTask.java
index b65df23996..fb387861f0 100644
---
a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerSizeCountTask.java
+++
b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerSizeCountTask.java
@@ -118,6 +118,16 @@ public class ContainerSizeCountTask extends ReconScmTask {
}
}
+ private void process(ContainerInfo container,
+ Map<ContainerSizeCountKey, Long> map) {
+ final ContainerID id = container.containerID();
+ final long currentSize = container.getUsedBytes();
+ final Long previousSize = processedContainers.put(id, currentSize);
+ if (previousSize != null) {
+ decrementContainerSizeCount(previousSize, map);
+ }
+ incrementContainerSizeCount(currentSize, map);
+ }
/**
* The process() function is responsible for updating the counts of
@@ -144,30 +154,21 @@ public class ContainerSizeCountTask extends ReconScmTask {
public void process(List<ContainerInfo> containers) {
lock.writeLock().lock();
try {
- HashMap<ContainerSizeCountKey, Long> containerSizeCountMap =
- new HashMap<>();
- HashMap<ContainerID, Long> deletedContainers = new HashMap<>();
- deletedContainers.putAll(processedContainers);
+ final Map<ContainerSizeCountKey, Long> containerSizeCountMap
+ = new HashMap<>();
+ final Map<ContainerID, Long> deletedContainers
+ = new HashMap<>(processedContainers);
// Loop to handle container create and size-update operations
for (ContainerInfo container : containers) {
// The containers present in the cache hence it is not yet deleted
deletedContainers.remove(container.containerID());
// For New Container being created
- if (!processedContainers.containsKey(container.containerID())) {
- incrementContainerSizeCount(container.getUsedBytes(),
- containerSizeCountMap);
- processedContainers.put(container.containerID(),
- container.getUsedBytes());
- } else if (processedContainers.get(container.containerID()) !=
- container.getUsedBytes()) { // If the Container Size is Updated
- decrementContainerSizeCount(
- processedContainers.get(container.containerID()),
- containerSizeCountMap);
- incrementContainerSizeCount(container.getUsedBytes(),
- containerSizeCountMap);
- processedContainers.put(container.containerID(),
- container.getUsedBytes());
+ try {
+ process(container, containerSizeCountMap);
+ } catch (Exception e) {
+ // FIXME: it is a bug if there is an exception.
+ LOG.error("FIXME: Failed to process " + container, e);
}
}
@@ -281,12 +282,9 @@ public class ContainerSizeCountTask extends ReconScmTask {
*
* @param containerSize to calculate the upperSizeBound
*/
- private void incrementContainerSizeCount(long containerSize,
+ private static void incrementContainerSizeCount(long containerSize,
Map<ContainerSizeCountKey, Long> containerSizeCountMap) {
- ContainerSizeCountKey key = getContainerSizeCountKey(containerSize);
- Long count = containerSizeCountMap.containsKey(key) ?
- containerSizeCountMap.get(key) + 1L : 1L;
- containerSizeCountMap.put(key, count);
+ updateContainerSizeCount(containerSize, 1, containerSizeCountMap);
}
/**
@@ -305,12 +303,16 @@ public class ContainerSizeCountTask extends ReconScmTask {
*
* @param containerSize to calculate the upperSizeBound
*/
- private void decrementContainerSizeCount(long containerSize,
+ private static void decrementContainerSizeCount(long containerSize,
Map<ContainerSizeCountKey, Long> containerSizeCountMap) {
+ updateContainerSizeCount(containerSize, -1, containerSizeCountMap);
+ }
+
+ private static void updateContainerSizeCount(long containerSize, int delta,
+ Map<ContainerSizeCountKey, Long> containerSizeCountMap) {
ContainerSizeCountKey key = getContainerSizeCountKey(containerSize);
- Long count = containerSizeCountMap.containsKey(key) ?
- containerSizeCountMap.get(key) - 1L : -1L;
- containerSizeCountMap.put(key, count);
+ containerSizeCountMap.compute(key,
+ (k, previous) -> previous != null ? previous + delta : delta);
}
/**
@@ -325,7 +327,7 @@ public class ContainerSizeCountTask extends ReconScmTask {
*
* @param containerSize to calculate the upperSizeBound
*/
- private ContainerSizeCountKey getContainerSizeCountKey(
+ private static ContainerSizeCountKey getContainerSizeCountKey(
long containerSize) {
return new ContainerSizeCountKey(
ReconUtils.getContainerSizeUpperBound(containerSize));
diff --git
a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/TestReconUtils.java
b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/TestReconUtils.java
index 95facc314b..b138b934fb 100644
---
a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/TestReconUtils.java
+++
b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/TestReconUtils.java
@@ -38,6 +38,7 @@ import java.net.HttpURLConnection;
import java.nio.charset.Charset;
import java.nio.file.Paths;
import java.net.URL;
+import java.util.Random;
import org.apache.commons.io.FileUtils;
import org.apache.commons.io.IOUtils;
@@ -196,4 +197,48 @@ public class TestReconUtils {
File latestValidFile = reconUtils.getLastKnownDB(newDir, "valid");
assertTrue(latestValidFile.getName().equals("valid_2"));
}
+
+ @Test
+ public void testNextClosestPowerIndexOfTwo() {
+ assertNextClosestPowerIndexOfTwo(0);
+ assertNextClosestPowerIndexOfTwo(1);
+ assertNextClosestPowerIndexOfTwo(-1);
+ assertNextClosestPowerIndexOfTwo(Long.MAX_VALUE);
+ assertNextClosestPowerIndexOfTwo(Long.MIN_VALUE);
+
+ for (long n = 1; n != 0; n <<= 1) {
+ assertNextClosestPowerIndexOfTwo(n);
+ assertNextClosestPowerIndexOfTwo(n + 1);
+ assertNextClosestPowerIndexOfTwo(n - 1);
+ }
+
+ final Random random = new Random();
+ for (int i = 0; i < 10; i++) {
+ assertNextClosestPowerIndexOfTwo(random.nextLong());
+ }
+ }
+
+ static void assertNextClosestPowerIndexOfTwo(long n) {
+ final int expected = oldNextClosestPowerIndexOfTwoFixed(n);
+ final int computed = ReconUtils.nextClosestPowerIndexOfTwo(n);
+ Assert.assertEquals("n=" + n, expected, computed);
+ }
+
+ private static int oldNextClosestPowerIndexOfTwoFixed(long n) {
+ return n == 0 ? 0
+ : n == Long.MIN_VALUE ? -63
+ : n == Long.highestOneBit(n) ? 63 - Long.numberOfLeadingZeros(n)
+ : n > 0 ? oldNextClosestPowerIndexOfTwo(n)
+ : -oldNextClosestPowerIndexOfTwoFixed(-n);
+ }
+
+ /** The old buggy method works only for n >= 0 with n not a power of 2. */
+ private static int oldNextClosestPowerIndexOfTwo(long dataSize) {
+ int index = 0;
+ while (dataSize != 0) {
+ dataSize >>= 1;
+ index += 1;
+ }
+ return index;
+ }
}
diff --git
a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestContainerSizeCountTask.java
b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestContainerSizeCountTask.java
index 09fb3f0af3..9b5dc3bc9b 100644
---
a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestContainerSizeCountTask.java
+++
b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestContainerSizeCountTask.java
@@ -79,6 +79,11 @@ public class TestContainerSizeCountTask extends
AbstractReconSqlDBTest {
@Test
public void testProcess() {
+ // mock a container with invalid used bytes
+ final ContainerInfo omContainerInfo0 = mock(ContainerInfo.class);
+ given(omContainerInfo0.containerID()).willReturn(new ContainerID(0));
+ given(omContainerInfo0.getUsedBytes()).willReturn(-1L);
+
// Write 2 keys
ContainerInfo omContainerInfo1 = mock(ContainerInfo.class);
given(omContainerInfo1.containerID()).willReturn(new ContainerID(1));
@@ -90,6 +95,7 @@ public class TestContainerSizeCountTask extends
AbstractReconSqlDBTest {
// mock getContainers method to return a list of containers
List<ContainerInfo> containers = new ArrayList<>();
+ containers.add(omContainerInfo0);
containers.add(omContainerInfo1);
containers.add(omContainerInfo2);
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]