hemantk-12 commented on code in PR #4584:
URL: https://github.com/apache/ozone/pull/4584#discussion_r1192611717
##########
hadoop-hdds/rocks-native/pom.xml:
##########
@@ -33,6 +33,25 @@
<groupId>org.eclipse.jetty</groupId>
<artifactId>jetty-io</artifactId>
</dependency>
+ <dependency>
+ <groupId>com.github.spotbugs</groupId>
+ <artifactId>spotbugs-annotations</artifactId>
+ <scope>compile</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.junit.jupiter</groupId>
+ <artifactId>junit-jupiter-api</artifactId>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.junit.jupiter</groupId>
+ <artifactId>junit-jupiter-params</artifactId>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.mockito</groupId>
+ <artifactId>mockito-all</artifactId>
Review Comment:
I don't think you need this one. You can just use `mockito-junit-jupiter`.
Also limit the scope to `test`.
##########
hadoop-hdds/rocks-native/src/test/java/org/apache/hadoop/hdds/utils/db/managed/TestManagedSSTDumpIterator.java:
##########
@@ -0,0 +1,243 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdds.utils.db.managed;
+
+import com.google.common.primitives.Bytes;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.utils.NativeLibraryNotLoadedException;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Named;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
+import org.junit.jupiter.params.provider.MethodSource;
+import org.mockito.Matchers;
+import org.mockito.Mockito;
+
+import java.io.ByteArrayInputStream;
+import java.io.File;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.Map;
+import java.util.Optional;
+import java.util.TreeMap;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import java.util.stream.Stream;
+
+/**
+ * Test for ManagedSSTDumpIterator.
+ */
+public class TestManagedSSTDumpIterator {
+
+ private void testSSTDumpIteratorWithKeys(
+ TreeMap<Pair<String, Integer>, String> keys) throws Exception {
+ File file = File.createTempFile("tmp_sst_file", ".sst");
+ file.deleteOnExit();
+ try (ManagedEnvOptions envOptions = new ManagedEnvOptions();
+ ManagedOptions managedOptions = new ManagedOptions();
+ ManagedSstFileWriter sstFileWriter = new ManagedSstFileWriter(
+ envOptions, managedOptions)) {
+ sstFileWriter.open(file.getAbsolutePath());
+ for (Map.Entry<Pair<String, Integer>, String> entry : keys.entrySet()) {
+ if (entry.getKey().getValue() == 0) {
+ sstFileWriter.delete(entry.getKey().getKey()
+ .getBytes(StandardCharsets.UTF_8));
+ } else {
+ sstFileWriter.put(entry.getKey().getKey()
+ .getBytes(StandardCharsets.UTF_8),
+ entry.getValue().getBytes(StandardCharsets.UTF_8));
+ }
+ }
+ sstFileWriter.finish();
+ sstFileWriter.close();
Review Comment:
No need to close it explicitly since you are using `try-with-resources`.
##########
hadoop-hdds/rocks-native/src/test/java/org/apache/hadoop/hdds/utils/db/managed/TestManagedSSTDumpIterator.java:
##########
@@ -0,0 +1,243 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdds.utils.db.managed;
+
+import com.google.common.primitives.Bytes;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.utils.NativeLibraryNotLoadedException;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Named;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
+import org.junit.jupiter.params.provider.MethodSource;
+import org.mockito.Matchers;
+import org.mockito.Mockito;
+
+import java.io.ByteArrayInputStream;
+import java.io.File;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.Map;
+import java.util.Optional;
+import java.util.TreeMap;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import java.util.stream.Stream;
+
+/**
+ * Test for ManagedSSTDumpIterator.
+ */
+public class TestManagedSSTDumpIterator {
+
+ private void testSSTDumpIteratorWithKeys(
+ TreeMap<Pair<String, Integer>, String> keys) throws Exception {
+ File file = File.createTempFile("tmp_sst_file", ".sst");
+ file.deleteOnExit();
+ try (ManagedEnvOptions envOptions = new ManagedEnvOptions();
+ ManagedOptions managedOptions = new ManagedOptions();
+ ManagedSstFileWriter sstFileWriter = new ManagedSstFileWriter(
+ envOptions, managedOptions)) {
+ sstFileWriter.open(file.getAbsolutePath());
+ for (Map.Entry<Pair<String, Integer>, String> entry : keys.entrySet()) {
+ if (entry.getKey().getValue() == 0) {
+ sstFileWriter.delete(entry.getKey().getKey()
+ .getBytes(StandardCharsets.UTF_8));
+ } else {
+ sstFileWriter.put(entry.getKey().getKey()
+ .getBytes(StandardCharsets.UTF_8),
+ entry.getValue().getBytes(StandardCharsets.UTF_8));
+ }
+ }
+ sstFileWriter.finish();
+ sstFileWriter.close();
+ ExecutorService executorService =
+ new ThreadPoolExecutor(1, 1, 0, TimeUnit.SECONDS,
+ new ArrayBlockingQueue<>(1),
+ new ThreadPoolExecutor.CallerRunsPolicy());
+ ManagedSSTDumpTool tool = new ManagedSSTDumpTool(executorService, 8192);
+ try (ManagedOptions options = new ManagedOptions();
+ ManagedSSTDumpIterator<ManagedSSTDumpIterator.KeyValue> iterator =
+ new ManagedSSTDumpIterator<ManagedSSTDumpIterator.KeyValue>(tool,
+ file.getAbsolutePath(), options) {
+
+ @Override
+ protected KeyValue getTransformedValue(Optional<KeyValue> value) {
+ return value.orElse(null);
+ }
+ }) {
+ while (iterator.hasNext()) {
+ ManagedSSTDumpIterator.KeyValue r = iterator.next();
+ Pair<String, Integer> recordKey = Pair.of(new String(r.getKey(),
+ StandardCharsets.UTF_8), r.getType());
+ Assertions.assertTrue(keys.containsKey(recordKey));
+ Assertions.assertEquals(
+ Optional.ofNullable(keys.get(recordKey)).orElse(""),
+ new String(r.getValue(), StandardCharsets.UTF_8));
+ keys.remove(recordKey);
+ }
+ Assertions.assertEquals(0, keys.size());
+ }
+ executorService.shutdown();
+ }
+ }
+
+ private static Stream<? extends Arguments> keyValueFormatArgs() {
+ return Stream.of(
+ Arguments.of(
+ Named.of("Key starting with a single quote",
+ "'key%1$d=>"),
+ Named.of("Value starting with a number ending with a" +
+ " single quote", "%1$dvalue'")
+ ),
+ Arguments.of(
+ Named.of("Key ending with a number", "key%1$d"),
+ Named.of("Value starting & ending with a number", "%1$dvalue%1$d")
+ ),
+ Arguments.of(
+ Named.of("Key starting with a single quote & ending" +
+ " with a number", "'key%1$d"),
+ Named.of("Value starting & ending with a number " +
+ "& elosed within quotes", "%1$d'value%1$d'")),
+ Arguments.of(
+ Named.of("Key starting with a single quote & ending" +
+ " with a number", "'key%1$d"),
+ Named.of("Value starting & ending with a number " +
+ "& elosed within quotes", "%1$d'value%1$d'")
+ ),
+ Arguments.of(
+ Named.of("Key ending with a number", "key%1$d"),
+ Named.of("Value starting & ending with a number " +
+ "& containing null character & new line character",
+ "%1$dvalue\n\0%1$d")
+ ),
+ Arguments.of(
+ Named.of("Key ending with a number & containing" +
+ " a null character", "key\0%1$d"),
+ Named.of("Value starting & ending with a number " +
+ "& elosed within quotes", "%1$dvalue\r%1$d")
+ )
+ );
+ }
+
+ private static byte[] getBytes(Integer val) {
Review Comment:
For int and long, you can either one of the below:
* Option 1: Use
[IntegerCodec](https://github.com/apache/ozone/blob/75f5a28f376516e7252e750e86a10c2f9d874d5a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/IntegerCodec.java#L29)
and
[LongCodec](https://github.com/apache/ozone/blob/75f5a28f376516e7252e750e86a10c2f9d874d5a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/LongCodec.java#L29)
respectively
* Option 2: Use `ByteBuffer.allocate(4).putInt().array()` and
`ByteBuffer.allocate(8).putLong()` respectively.
##########
hadoop-hdds/rocks-native/src/main/java/org/apache/hadoop/hdds/utils/db/managed/ManagedSSTDumpIterator.java:
##########
@@ -180,33 +182,32 @@ public boolean hasNext() {
public T next() {
checkSanityOfProcess();
currentKey = nextKey;
- nextKey = null;
- boolean keyFound = false;
- while (!keyFound) {
- try {
- Optional<Integer> keyLength = getNextNumberInStream();
- if (!keyLength.isPresent()) {
- return getTransformedValue(currentKey);
- }
- String keyStr = readNextNumberOfCharsFromStream(keyLength.get());
- Matcher matcher = PATTERN_MATCHER.matcher(keyStr);
- if (keyStr.length() == keyLength.get() && matcher.find()) {
- Optional<Integer> valueLength = getNextNumberInStream();
- if (valueLength.isPresent()) {
- String valueStr = readNextNumberOfCharsFromStream(
- valueLength.get());
- if (valueStr.length() == valueLength.get()) {
- keyFound = true;
- nextKey = new KeyValue(matcher.group(PATTERN_KEY_GROUP_NUMBER),
- matcher.group(PATTERN_SEQ_GROUP_NUMBER),
- matcher.group(PATTERN_TYPE_GROUP_NUMBER),
- valueStr);
- }
- }
- }
- } catch (IOException e) {
- throw new RuntimeIOException(e);
+ nextKey = Optional.empty();
+ try {
+ Optional<byte[]> key = getNextByteArray();
+ if (!key.isPresent()) {
+ return getTransformedValue(currentKey);
}
+ UnsignedLong sequenceNumber = getNextUnsignedLong()
Review Comment:
I don't think you want to throw an exception in case `getNextUnsignedLong`
returns `empty()`. If that's the case, then `getNextUnsignedLong` should throw
exception instead of returning `Optional.empty()`.
##########
hadoop-hdds/rocks-native/src/main/java/org/apache/hadoop/hdds/utils/db/managed/ManagedSSTDumpIterator.java:
##########
@@ -244,40 +245,46 @@ protected void finalize() throws Throwable {
* Class containing Parsed KeyValue Record from Sst Dumptool output.
*/
public static final class KeyValue {
- private String key;
- private Integer sequence;
- private Integer type;
- private String value;
+ private final byte[] key;
+ private final UnsignedLong sequence;
+ private final Integer type;
+ private final byte[] value;
- private KeyValue(String key, String sequence, String type,
- String value) {
+ private KeyValue(byte[] key, UnsignedLong sequence, Integer type,
+ byte[] value) {
this.key = key;
- this.sequence = Integer.valueOf(sequence);
- this.type = Integer.valueOf(type);
+ this.sequence = sequence;
+ this.type = type;
this.value = value;
}
- public String getKey() {
+ @SuppressFBWarnings("EI_EXPOSE_REP")
+ public byte[] getKey() {
return key;
}
- public Integer getSequence() {
+ public UnsignedLong getSequence() {
return sequence;
}
public Integer getType() {
return type;
}
- public String getValue() {
+ @SuppressFBWarnings("EI_EXPOSE_REP")
+ public byte[] getValue() {
return value;
}
@Override
public String toString() {
- return "KeyValue{" + "key='" + key + '\'' + ", sequence=" + sequence +
- ", type=" + type + ", value='" + value + '\'' + '}';
+ return "KeyValue{" +
+ "key=" + StringUtils.bytes2String(key) +
Review Comment:
Can `key` and `value` be null? If so, `StringUtils.bytes2String()` will
throw NPE if I'm not wrong.
##########
hadoop-hdds/rocks-native/src/test/java/org/apache/hadoop/hdds/utils/db/managed/TestManagedSSTDumpIterator.java:
##########
@@ -0,0 +1,243 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdds.utils.db.managed;
+
+import com.google.common.primitives.Bytes;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.utils.NativeLibraryNotLoadedException;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Named;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
+import org.junit.jupiter.params.provider.MethodSource;
+import org.mockito.Matchers;
+import org.mockito.Mockito;
+
+import java.io.ByteArrayInputStream;
+import java.io.File;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.Map;
+import java.util.Optional;
+import java.util.TreeMap;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import java.util.stream.Stream;
+
+/**
+ * Test for ManagedSSTDumpIterator.
+ */
+public class TestManagedSSTDumpIterator {
+
+ private void testSSTDumpIteratorWithKeys(
+ TreeMap<Pair<String, Integer>, String> keys) throws Exception {
+ File file = File.createTempFile("tmp_sst_file", ".sst");
+ file.deleteOnExit();
+ try (ManagedEnvOptions envOptions = new ManagedEnvOptions();
+ ManagedOptions managedOptions = new ManagedOptions();
+ ManagedSstFileWriter sstFileWriter = new ManagedSstFileWriter(
+ envOptions, managedOptions)) {
+ sstFileWriter.open(file.getAbsolutePath());
+ for (Map.Entry<Pair<String, Integer>, String> entry : keys.entrySet()) {
+ if (entry.getKey().getValue() == 0) {
+ sstFileWriter.delete(entry.getKey().getKey()
+ .getBytes(StandardCharsets.UTF_8));
+ } else {
+ sstFileWriter.put(entry.getKey().getKey()
+ .getBytes(StandardCharsets.UTF_8),
+ entry.getValue().getBytes(StandardCharsets.UTF_8));
+ }
+ }
+ sstFileWriter.finish();
+ sstFileWriter.close();
+ ExecutorService executorService =
+ new ThreadPoolExecutor(1, 1, 0, TimeUnit.SECONDS,
+ new ArrayBlockingQueue<>(1),
+ new ThreadPoolExecutor.CallerRunsPolicy());
+ ManagedSSTDumpTool tool = new ManagedSSTDumpTool(executorService, 8192);
+ try (ManagedOptions options = new ManagedOptions();
+ ManagedSSTDumpIterator<ManagedSSTDumpIterator.KeyValue> iterator =
+ new ManagedSSTDumpIterator<ManagedSSTDumpIterator.KeyValue>(tool,
+ file.getAbsolutePath(), options) {
+
+ @Override
+ protected KeyValue getTransformedValue(Optional<KeyValue> value) {
+ return value.orElse(null);
+ }
+ }) {
+ while (iterator.hasNext()) {
+ ManagedSSTDumpIterator.KeyValue r = iterator.next();
+ Pair<String, Integer> recordKey = Pair.of(new String(r.getKey(),
+ StandardCharsets.UTF_8), r.getType());
+ Assertions.assertTrue(keys.containsKey(recordKey));
+ Assertions.assertEquals(
+ Optional.ofNullable(keys.get(recordKey)).orElse(""),
+ new String(r.getValue(), StandardCharsets.UTF_8));
+ keys.remove(recordKey);
+ }
+ Assertions.assertEquals(0, keys.size());
+ }
+ executorService.shutdown();
+ }
+ }
+
+ private static Stream<? extends Arguments> keyValueFormatArgs() {
+ return Stream.of(
+ Arguments.of(
+ Named.of("Key starting with a single quote",
+ "'key%1$d=>"),
+ Named.of("Value starting with a number ending with a" +
+ " single quote", "%1$dvalue'")
+ ),
+ Arguments.of(
+ Named.of("Key ending with a number", "key%1$d"),
+ Named.of("Value starting & ending with a number", "%1$dvalue%1$d")
+ ),
+ Arguments.of(
+ Named.of("Key starting with a single quote & ending" +
+ " with a number", "'key%1$d"),
+ Named.of("Value starting & ending with a number " +
+ "& elosed within quotes", "%1$d'value%1$d'")),
+ Arguments.of(
+ Named.of("Key starting with a single quote & ending" +
+ " with a number", "'key%1$d"),
+ Named.of("Value starting & ending with a number " +
+ "& elosed within quotes", "%1$d'value%1$d'")
+ ),
+ Arguments.of(
+ Named.of("Key ending with a number", "key%1$d"),
+ Named.of("Value starting & ending with a number " +
+ "& containing null character & new line character",
+ "%1$dvalue\n\0%1$d")
+ ),
+ Arguments.of(
+ Named.of("Key ending with a number & containing" +
+ " a null character", "key\0%1$d"),
+ Named.of("Value starting & ending with a number " +
+ "& elosed within quotes", "%1$dvalue\r%1$d")
+ )
+ );
+ }
+
+ private static byte[] getBytes(Integer val) {
+ byte[] b = new byte[4];
+ for (int i = 3; i >= 0; i--) {
+ b[i] = val.byteValue();
+ val = val >> 8;
+ }
+ return b;
+ }
+
+ private static byte[] getBytes(Long val) {
+ byte[] b = new byte[8];
+ for (int i = 7; i >= 0; i--) {
+ b[i] = (byte)(val & 0xff);
+ val = val >> 8;
+ }
+ return b;
+ }
+
+ private static byte[] getBytes(String val) {
+ byte[] b = new byte[val.length()];
+ for (int i = 0; i < val.length(); i++) {
+ b[i] = (byte) val.charAt(i);
+ }
+ return b;
+ }
+
+ private static Stream<? extends Arguments> invalidPipeInputStreamBytes() {
+ return Stream.of(
+ Arguments.of(Named.of("Invalid 3 byte integer",
+ new byte[] {0, 0, 0})),
+ Arguments.of(Named.of("Invalid 2 byte integer",
+ new byte[] {0, 0})),
+ Arguments.of(Named.of("Invalid 1 byte integer",
+ new byte[] {0, 0})),
+ Arguments.of(Named.of("Invalid key name length",
+ Bytes.concat(getBytes(4), getBytes("key")))),
+ Arguments.of(Named.of("Invalid Unsigned Long length",
+ Bytes.concat(getBytes(4), getBytes("key1"),
+ new byte[]{0, 0}))),
+ Arguments.of(Named.of("Invalid Sequence number",
+ Bytes.concat(getBytes(4), getBytes("key1")))),
+ Arguments.of(Named.of("Invalid Type",
+ Bytes.concat(getBytes(4), getBytes("key1"),
+ getBytes(4L)))),
+ Arguments.of(Named.of("Invalid Value",
+ Bytes.concat(getBytes(4), getBytes("key"),
+ getBytes(4L), getBytes(0)))),
+ Arguments.of(Named.of("Invalid Value length",
+ Bytes.concat(getBytes(4), getBytes("key"),
+ getBytes(4L), getBytes(1), getBytes(6),
+ getBytes("val"))))
+ );
+ }
+
+
Review Comment:
nit: Remove this extra lines.
##########
hadoop-hdds/rocks-native/src/main/java/org/apache/hadoop/hdds/utils/db/managed/ManagedSSTDumpIterator.java:
##########
@@ -180,33 +182,32 @@ public boolean hasNext() {
public T next() {
checkSanityOfProcess();
currentKey = nextKey;
- nextKey = null;
- boolean keyFound = false;
- while (!keyFound) {
- try {
- Optional<Integer> keyLength = getNextNumberInStream();
- if (!keyLength.isPresent()) {
- return getTransformedValue(currentKey);
- }
- String keyStr = readNextNumberOfCharsFromStream(keyLength.get());
- Matcher matcher = PATTERN_MATCHER.matcher(keyStr);
- if (keyStr.length() == keyLength.get() && matcher.find()) {
- Optional<Integer> valueLength = getNextNumberInStream();
- if (valueLength.isPresent()) {
- String valueStr = readNextNumberOfCharsFromStream(
- valueLength.get());
- if (valueStr.length() == valueLength.get()) {
- keyFound = true;
- nextKey = new KeyValue(matcher.group(PATTERN_KEY_GROUP_NUMBER),
- matcher.group(PATTERN_SEQ_GROUP_NUMBER),
- matcher.group(PATTERN_TYPE_GROUP_NUMBER),
- valueStr);
- }
- }
- }
- } catch (IOException e) {
- throw new RuntimeIOException(e);
+ nextKey = Optional.empty();
+ try {
+ Optional<byte[]> key = getNextByteArray();
+ if (!key.isPresent()) {
+ return getTransformedValue(currentKey);
}
+ UnsignedLong sequenceNumber = getNextUnsignedLong()
+ .orElseThrow(() -> new IllegalStateException(
+ String.format("Error while trying to read sequence number" +
+ " for key %s", StringUtils.bytes2String(key.get()))));
+
+ Integer type = getNextNumberInStream()
+ .orElseThrow(() -> new IllegalStateException(
+ String.format("Error while trying to read sequence number for " +
+ "key %s with sequence number %s",
+ StringUtils.bytes2String(key.get()),
+ sequenceNumber.toString())));
+ byte[] val = getNextByteArray().orElseThrow(() ->
+ new IllegalStateException(
+ String.format("Error while trying to read sequence number for " +
+ "key %s with sequence number %s of type %d",
+ StringUtils.bytes2String(key.get()),
+ sequenceNumber.toString(), type)));
+ nextKey = Optional.of(new KeyValue(key.get(), sequenceNumber, type,
val));
+ } catch (IOException e) {
+ throw new RuntimeIOException(e);
Review Comment:
nit: add a TODO (with snapshot tag) to throw custom exception.
##########
hadoop-hdds/rocks-native/src/test/java/org/apache/hadoop/hdds/utils/db/managed/TestManagedSSTDumpIterator.java:
##########
@@ -0,0 +1,243 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdds.utils.db.managed;
+
+import com.google.common.primitives.Bytes;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.utils.NativeLibraryNotLoadedException;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Named;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
+import org.junit.jupiter.params.provider.MethodSource;
+import org.mockito.Matchers;
+import org.mockito.Mockito;
+
+import java.io.ByteArrayInputStream;
+import java.io.File;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.Map;
+import java.util.Optional;
+import java.util.TreeMap;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import java.util.stream.Stream;
+
+/**
+ * Test for ManagedSSTDumpIterator.
+ */
+public class TestManagedSSTDumpIterator {
+
+ private void testSSTDumpIteratorWithKeys(
+ TreeMap<Pair<String, Integer>, String> keys) throws Exception {
+ File file = File.createTempFile("tmp_sst_file", ".sst");
+ file.deleteOnExit();
+ try (ManagedEnvOptions envOptions = new ManagedEnvOptions();
+ ManagedOptions managedOptions = new ManagedOptions();
+ ManagedSstFileWriter sstFileWriter = new ManagedSstFileWriter(
+ envOptions, managedOptions)) {
+ sstFileWriter.open(file.getAbsolutePath());
+ for (Map.Entry<Pair<String, Integer>, String> entry : keys.entrySet()) {
+ if (entry.getKey().getValue() == 0) {
+ sstFileWriter.delete(entry.getKey().getKey()
+ .getBytes(StandardCharsets.UTF_8));
+ } else {
+ sstFileWriter.put(entry.getKey().getKey()
+ .getBytes(StandardCharsets.UTF_8),
+ entry.getValue().getBytes(StandardCharsets.UTF_8));
+ }
+ }
+ sstFileWriter.finish();
+ sstFileWriter.close();
+ ExecutorService executorService =
+ new ThreadPoolExecutor(1, 1, 0, TimeUnit.SECONDS,
+ new ArrayBlockingQueue<>(1),
+ new ThreadPoolExecutor.CallerRunsPolicy());
+ ManagedSSTDumpTool tool = new ManagedSSTDumpTool(executorService, 8192);
+ try (ManagedOptions options = new ManagedOptions();
+ ManagedSSTDumpIterator<ManagedSSTDumpIterator.KeyValue> iterator =
+ new ManagedSSTDumpIterator<ManagedSSTDumpIterator.KeyValue>(tool,
+ file.getAbsolutePath(), options) {
+
+ @Override
+ protected KeyValue getTransformedValue(Optional<KeyValue> value) {
+ return value.orElse(null);
+ }
+ }) {
+ while (iterator.hasNext()) {
+ ManagedSSTDumpIterator.KeyValue r = iterator.next();
+ Pair<String, Integer> recordKey = Pair.of(new String(r.getKey(),
+ StandardCharsets.UTF_8), r.getType());
+ Assertions.assertTrue(keys.containsKey(recordKey));
+ Assertions.assertEquals(
+ Optional.ofNullable(keys.get(recordKey)).orElse(""),
+ new String(r.getValue(), StandardCharsets.UTF_8));
+ keys.remove(recordKey);
+ }
+ Assertions.assertEquals(0, keys.size());
+ }
+ executorService.shutdown();
+ }
+ }
+
+ private static Stream<? extends Arguments> keyValueFormatArgs() {
+ return Stream.of(
+ Arguments.of(
+ Named.of("Key starting with a single quote",
+ "'key%1$d=>"),
+ Named.of("Value starting with a number ending with a" +
+ " single quote", "%1$dvalue'")
+ ),
+ Arguments.of(
+ Named.of("Key ending with a number", "key%1$d"),
+ Named.of("Value starting & ending with a number", "%1$dvalue%1$d")
+ ),
+ Arguments.of(
+ Named.of("Key starting with a single quote & ending" +
+ " with a number", "'key%1$d"),
+ Named.of("Value starting & ending with a number " +
+ "& elosed within quotes", "%1$d'value%1$d'")),
+ Arguments.of(
+ Named.of("Key starting with a single quote & ending" +
+ " with a number", "'key%1$d"),
+ Named.of("Value starting & ending with a number " +
+ "& elosed within quotes", "%1$d'value%1$d'")
+ ),
+ Arguments.of(
+ Named.of("Key ending with a number", "key%1$d"),
+ Named.of("Value starting & ending with a number " +
+ "& containing null character & new line character",
+ "%1$dvalue\n\0%1$d")
+ ),
+ Arguments.of(
+ Named.of("Key ending with a number & containing" +
+ " a null character", "key\0%1$d"),
+ Named.of("Value starting & ending with a number " +
+ "& elosed within quotes", "%1$dvalue\r%1$d")
+ )
+ );
+ }
+
+ private static byte[] getBytes(Integer val) {
+ byte[] b = new byte[4];
+ for (int i = 3; i >= 0; i--) {
+ b[i] = val.byteValue();
+ val = val >> 8;
+ }
+ return b;
+ }
+
+ private static byte[] getBytes(Long val) {
+ byte[] b = new byte[8];
+ for (int i = 7; i >= 0; i--) {
+ b[i] = (byte)(val & 0xff);
+ val = val >> 8;
+ }
+ return b;
+ }
+
+ private static byte[] getBytes(String val) {
Review Comment:
Use `StringUtils.string2Bytes()` instead of creating new one.
##########
hadoop-hdds/rocks-native/src/test/java/org/apache/hadoop/hdds/utils/db/managed/TestManagedSSTDumpIterator.java:
##########
@@ -0,0 +1,243 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdds.utils.db.managed;
+
+import com.google.common.primitives.Bytes;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.utils.NativeLibraryNotLoadedException;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Named;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
+import org.junit.jupiter.params.provider.MethodSource;
+import org.mockito.Matchers;
+import org.mockito.Mockito;
+
+import java.io.ByteArrayInputStream;
+import java.io.File;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.Map;
+import java.util.Optional;
+import java.util.TreeMap;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import java.util.stream.Stream;
+
+/**
+ * Test for ManagedSSTDumpIterator.
+ */
+public class TestManagedSSTDumpIterator {
+
+ private void testSSTDumpIteratorWithKeys(
+ TreeMap<Pair<String, Integer>, String> keys) throws Exception {
+ File file = File.createTempFile("tmp_sst_file", ".sst");
+ file.deleteOnExit();
+ try (ManagedEnvOptions envOptions = new ManagedEnvOptions();
+ ManagedOptions managedOptions = new ManagedOptions();
+ ManagedSstFileWriter sstFileWriter = new ManagedSstFileWriter(
+ envOptions, managedOptions)) {
+ sstFileWriter.open(file.getAbsolutePath());
+ for (Map.Entry<Pair<String, Integer>, String> entry : keys.entrySet()) {
+ if (entry.getKey().getValue() == 0) {
+ sstFileWriter.delete(entry.getKey().getKey()
+ .getBytes(StandardCharsets.UTF_8));
+ } else {
+ sstFileWriter.put(entry.getKey().getKey()
+ .getBytes(StandardCharsets.UTF_8),
+ entry.getValue().getBytes(StandardCharsets.UTF_8));
+ }
+ }
+ sstFileWriter.finish();
+ sstFileWriter.close();
+ ExecutorService executorService =
+ new ThreadPoolExecutor(1, 1, 0, TimeUnit.SECONDS,
+ new ArrayBlockingQueue<>(1),
+ new ThreadPoolExecutor.CallerRunsPolicy());
+ ManagedSSTDumpTool tool = new ManagedSSTDumpTool(executorService, 8192);
+ try (ManagedOptions options = new ManagedOptions();
Review Comment:
Please avoid nested try. Please change it to this way.
```
try {
// First try
try {
// Second try
}
```
Or extract out the code to separate functions.
##########
hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdb/util/ManagedSstFileReader.java:
##########
@@ -140,8 +141,9 @@ protected ClosableIterator<String>
getKeyIteratorForFile(String file)
return new ManagedSSTDumpIterator<String>(sstDumpTool, file,
options) {
@Override
- protected String getTransformedValue(KeyValue value) {
- return value.getKey();
+ protected String getTransformedValue(Optional<KeyValue> value) {
+ return value.map(v -> new String(v.getKey(), UTF_8))
Review Comment:
nit:
```suggestion
return value.map(v ->
StringUtils.bytes2String(v.getKey())).orElse(null);
```
##########
hadoop-hdds/rocks-native/src/test/java/org/apache/hadoop/hdds/utils/db/managed/TestManagedSSTDumpIterator.java:
##########
@@ -0,0 +1,243 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdds.utils.db.managed;
+
+import com.google.common.primitives.Bytes;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.utils.NativeLibraryNotLoadedException;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Named;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
+import org.junit.jupiter.params.provider.MethodSource;
+import org.mockito.Matchers;
+import org.mockito.Mockito;
+
+import java.io.ByteArrayInputStream;
+import java.io.File;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.Map;
+import java.util.Optional;
+import java.util.TreeMap;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import java.util.stream.Stream;
+
+/**
+ * Test for ManagedSSTDumpIterator.
+ */
+public class TestManagedSSTDumpIterator {
+
+ private void testSSTDumpIteratorWithKeys(
+ TreeMap<Pair<String, Integer>, String> keys) throws Exception {
+ File file = File.createTempFile("tmp_sst_file", ".sst");
+ file.deleteOnExit();
+ try (ManagedEnvOptions envOptions = new ManagedEnvOptions();
+ ManagedOptions managedOptions = new ManagedOptions();
+ ManagedSstFileWriter sstFileWriter = new ManagedSstFileWriter(
+ envOptions, managedOptions)) {
+ sstFileWriter.open(file.getAbsolutePath());
+ for (Map.Entry<Pair<String, Integer>, String> entry : keys.entrySet()) {
+ if (entry.getKey().getValue() == 0) {
+ sstFileWriter.delete(entry.getKey().getKey()
+ .getBytes(StandardCharsets.UTF_8));
+ } else {
+ sstFileWriter.put(entry.getKey().getKey()
+ .getBytes(StandardCharsets.UTF_8),
+ entry.getValue().getBytes(StandardCharsets.UTF_8));
+ }
+ }
+ sstFileWriter.finish();
+ sstFileWriter.close();
+ ExecutorService executorService =
+ new ThreadPoolExecutor(1, 1, 0, TimeUnit.SECONDS,
+ new ArrayBlockingQueue<>(1),
+ new ThreadPoolExecutor.CallerRunsPolicy());
+ ManagedSSTDumpTool tool = new ManagedSSTDumpTool(executorService, 8192);
+ try (ManagedOptions options = new ManagedOptions();
+ ManagedSSTDumpIterator<ManagedSSTDumpIterator.KeyValue> iterator =
+ new ManagedSSTDumpIterator<ManagedSSTDumpIterator.KeyValue>(tool,
+ file.getAbsolutePath(), options) {
+
+ @Override
+ protected KeyValue getTransformedValue(Optional<KeyValue> value) {
+ return value.orElse(null);
+ }
+ }) {
+ while (iterator.hasNext()) {
+ ManagedSSTDumpIterator.KeyValue r = iterator.next();
+ Pair<String, Integer> recordKey = Pair.of(new String(r.getKey(),
+ StandardCharsets.UTF_8), r.getType());
+ Assertions.assertTrue(keys.containsKey(recordKey));
+ Assertions.assertEquals(
+ Optional.ofNullable(keys.get(recordKey)).orElse(""),
+ new String(r.getValue(), StandardCharsets.UTF_8));
+ keys.remove(recordKey);
+ }
+ Assertions.assertEquals(0, keys.size());
+ }
+ executorService.shutdown();
+ }
+ }
+
+ private static Stream<? extends Arguments> keyValueFormatArgs() {
+ return Stream.of(
+ Arguments.of(
+ Named.of("Key starting with a single quote",
+ "'key%1$d=>"),
+ Named.of("Value starting with a number ending with a" +
+ " single quote", "%1$dvalue'")
+ ),
+ Arguments.of(
+ Named.of("Key ending with a number", "key%1$d"),
+ Named.of("Value starting & ending with a number", "%1$dvalue%1$d")
+ ),
+ Arguments.of(
+ Named.of("Key starting with a single quote & ending" +
+ " with a number", "'key%1$d"),
+ Named.of("Value starting & ending with a number " +
+ "& elosed within quotes", "%1$d'value%1$d'")),
+ Arguments.of(
+ Named.of("Key starting with a single quote & ending" +
+ " with a number", "'key%1$d"),
+ Named.of("Value starting & ending with a number " +
+ "& elosed within quotes", "%1$d'value%1$d'")
+ ),
+ Arguments.of(
+ Named.of("Key ending with a number", "key%1$d"),
+ Named.of("Value starting & ending with a number " +
+ "& containing null character & new line character",
+ "%1$dvalue\n\0%1$d")
+ ),
+ Arguments.of(
+ Named.of("Key ending with a number & containing" +
+ " a null character", "key\0%1$d"),
+ Named.of("Value starting & ending with a number " +
+ "& elosed within quotes", "%1$dvalue\r%1$d")
+ )
+ );
+ }
+
+ private static byte[] getBytes(Integer val) {
+ byte[] b = new byte[4];
+ for (int i = 3; i >= 0; i--) {
+ b[i] = val.byteValue();
+ val = val >> 8;
+ }
+ return b;
+ }
+
+ private static byte[] getBytes(Long val) {
+ byte[] b = new byte[8];
+ for (int i = 7; i >= 0; i--) {
+ b[i] = (byte)(val & 0xff);
+ val = val >> 8;
+ }
+ return b;
+ }
+
+ private static byte[] getBytes(String val) {
+ byte[] b = new byte[val.length()];
+ for (int i = 0; i < val.length(); i++) {
+ b[i] = (byte) val.charAt(i);
+ }
+ return b;
+ }
+
+ private static Stream<? extends Arguments> invalidPipeInputStreamBytes() {
+ return Stream.of(
+ Arguments.of(Named.of("Invalid 3 byte integer",
+ new byte[] {0, 0, 0})),
+ Arguments.of(Named.of("Invalid 2 byte integer",
+ new byte[] {0, 0})),
+ Arguments.of(Named.of("Invalid 1 byte integer",
+ new byte[] {0, 0})),
+ Arguments.of(Named.of("Invalid key name length",
+ Bytes.concat(getBytes(4), getBytes("key")))),
+ Arguments.of(Named.of("Invalid Unsigned Long length",
+ Bytes.concat(getBytes(4), getBytes("key1"),
+ new byte[]{0, 0}))),
+ Arguments.of(Named.of("Invalid Sequence number",
+ Bytes.concat(getBytes(4), getBytes("key1")))),
+ Arguments.of(Named.of("Invalid Type",
+ Bytes.concat(getBytes(4), getBytes("key1"),
+ getBytes(4L)))),
+ Arguments.of(Named.of("Invalid Value",
+ Bytes.concat(getBytes(4), getBytes("key"),
+ getBytes(4L), getBytes(0)))),
+ Arguments.of(Named.of("Invalid Value length",
+ Bytes.concat(getBytes(4), getBytes("key"),
+ getBytes(4L), getBytes(1), getBytes(6),
+ getBytes("val"))))
+ );
+ }
+
+
+
+ @Native("Managed Rocks Tools")
+ @ParameterizedTest
+ @MethodSource("keyValueFormatArgs")
+ public void testSSTDumpIteratorWithKeyFormat(String keyFormat,
+ String valueFormat) throws Exception {
+ TreeMap<Pair<String, Integer>, String> keys =
+ IntStream.range(0, 100).boxed().collect(
+ Collectors.toMap(
+ i -> Pair.of(String.format(keyFormat, i), i % 2),
+ i -> i % 2 == 0 ? "" : String.format(valueFormat, i),
+ (v1, v2) -> v2,
+ TreeMap::new));
+ testSSTDumpIteratorWithKeys(keys);
Review Comment:
It is not a good idea to delegate one
test:`testSSTDumpIteratorWithKeyFormat` on another:
`testSSTDumpIteratorWithKeys`.
If you want, can divide the big test into parts like:
```
public void test() {
- Given
// Test setup and mocking. Can be another method, if needed.
- When
// method call writing test for. Can be another method, if needed.
- Then
// Assertions and verifications. Can be another method, if needed.
}
```
##########
hadoop-hdds/rocks-native/src/main/java/org/apache/hadoop/hdds/utils/db/managed/ManagedSSTDumpIterator.java:
##########
@@ -86,51 +79,47 @@ public ManagedSSTDumpIterator(ManagedSSTDumpTool
sstDumpTool,
* @return Optional of the integer empty if no integer exists
*/
private Optional<Integer> getNextNumberInStream() throws IOException {
- StringBuilder value = new StringBuilder();
- int val;
- while ((val = processOutput.read()) != -1) {
- if (val >= '0' && val <= '9') {
- value.append((char) val);
- } else if (value.length() > 0) {
- break;
- }
+ int n = processOutput.read(intBuffer, 0, 4);
+ if (n == 4) {
Review Comment:
Thanks
##########
hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdb/util/ManagedSstFileReader.java:
##########
@@ -216,7 +215,8 @@ protected abstract ClosableIterator<T>
getKeyIteratorForFile(String file)
public boolean hasNext() {
try {
do {
- if (currentFileIterator.hasNext()) {
+ if (!Objects.isNull(currentFileIterator) &&
Review Comment:
nit: no need to use double negation.
```suggestion
if (Objects.nonNull(currentFileIterator) &&
```
##########
hadoop-hdds/rocks-native/src/main/java/org/apache/hadoop/hdds/utils/db/managed/PipeInputStream.java:
##########
@@ -65,11 +65,13 @@ public int read() {
this.close();
return -1;
}
- if (numberOfBytesLeftToRead == 0) {
+ while (numberOfBytesLeftToRead == 0) {
numberOfBytesLeftToRead = readInternal(byteBuffer, capacity,
nativeHandle);
index = 0;
- return read();
+ if (numberOfBytesLeftToRead != 0) {
Review Comment:
Does it mean it can return 0 or -1? If so, we should handle both cases. If
not, then it is OK to leave as it it.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]