You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ozone.apache.org by "swamirishi (via GitHub)" <gi...@apache.org> on 2023/02/24 16:47:31 UTC

[GitHub] [ozone] swamirishi opened a new pull request, #4315: HDDS-8028: JNI for RocksDB SST Dump tool

swamirishi opened a new pull request, #4315:
URL: https://github.com/apache/ozone/pull/4315

   ## What changes were proposed in this pull request?
   
   JNI for RocksDB SST Dump tool for reading tombstone entries from SST file for smart computation of SnapDiff.
   
   ## What is the link to the Apache JIRA
   https://issues.apache.org/jira/browse/HDDS-8028
   ## How was this patch tested?
   Manual Testing.


-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] swamirishi commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "swamirishi (via GitHub)" <gi...@apache.org>.
swamirishi commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1125512913


##########
hadoop-ozone/common/pom.xml:
##########
@@ -107,6 +107,10 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd">
       <groupId>org.junit.jupiter</groupId>
       <artifactId>junit-jupiter-params</artifactId>
     </dependency>
+    <dependency>
+      <groupId>org.apache.ozone</groupId>
+      <artifactId>hdds-server-framework</artifactId>
+    </dependency>

Review Comment:
   done



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1125273757


##########
hadoop-hdds/rocks-native/src/main/java/org/apache/hadoop/hdds/utils/NativeLibraryLoader.java:
##########
@@ -0,0 +1,138 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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;
+
+
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Files;
+import java.nio.file.StandardCopyOption;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Class to load Native Libraries.
+ */
+public class NativeLibraryLoader {
+  private static final String OS = System.getProperty("os.name").toLowerCase();
+  private Map<String, Boolean> librariesLoaded;
+  private static volatile NativeLibraryLoader instance;
+
+  public NativeLibraryLoader(final Map<String, Boolean> librariesLoaded) {
+    this.librariesLoaded = librariesLoaded;
+  }
+
+  private static synchronized void initNewInstance() {
+    if (instance == null) {
+      instance = new NativeLibraryLoader(new ConcurrentHashMap<>());
+    }
+  }
+
+  public static NativeLibraryLoader getInstance() {
+    if (instance == null) {
+      initNewInstance();
+    }
+    return instance;
+  }
+
+  public static String getJniLibraryFileName(String libraryName) {
+    return appendLibOsSuffix("lib" + libraryName);
+  }
+
+  public static boolean isMac() {
+    return OS.contains("mac");
+  }
+
+  public static boolean isWindows() {
+    return OS.contains("win");

Review Comment:
   ```suggestion
       return OS.startsWith("win");
   ```



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1125282860


##########
hadoop-hdds/rocks-native/src/main/java/org/apache/hadoop/hdds/utils/db/managed/PipeInputStream.java:
##########
@@ -0,0 +1,81 @@
+package org.apache.hadoop.hdds.utils.db.managed;
+
+import org.apache.hadoop.hdds.utils.NativeLibraryLoader;
+import org.apache.hadoop.hdds.utils.NativeLibraryNotLoadedException;
+import java.io.InputStream;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.apache.hadoop.hdds.utils.NativeConstants.ROCKS_TOOLS_NATIVE_LIBRARY_NAME;
+
+/**
+ * JNI for reading data from pipe.
+ */
+public class PipeInputStream extends InputStream {
+
+  static {
+    NativeLibraryLoader.getInstance()
+            .loadLibrary(ROCKS_TOOLS_NATIVE_LIBRARY_NAME);
+  }
+  private byte[] byteBuffer;
+  private long nativeHandle;
+  private int numberOfBytesLeftToRead;
+  private int index = 0;
+  private int capacity;
+
+  private AtomicBoolean cleanup;
+
+  PipeInputStream(int capacity) throws NativeLibraryNotLoadedException {
+    if (!NativeLibraryLoader.getInstance()
+            .isLibraryLoaded(ROCKS_TOOLS_NATIVE_LIBRARY_NAME)) {
+      throw new NativeLibraryNotLoadedException(
+              ROCKS_TOOLS_NATIVE_LIBRARY_NAME);
+    }
+
+    this.byteBuffer = new byte[capacity];
+    this.numberOfBytesLeftToRead = 0;
+    this.capacity = capacity;
+    this.nativeHandle = newPipe();
+    this.cleanup = new AtomicBoolean(false);
+  }
+
+  long getNativeHandle() {
+    return nativeHandle;
+  }
+
+  @Override
+  public int read() {
+    if (numberOfBytesLeftToRead < 0) {
+      this.close();
+      return -1;
+    }
+    if (numberOfBytesLeftToRead == 0) {
+      numberOfBytesLeftToRead = readInternal(byteBuffer, capacity,
+              nativeHandle);
+      index = 0;
+      return read();
+    }
+    numberOfBytesLeftToRead--;
+    int ret = byteBuffer[index] & 0xFF;
+    index += 1;
+    return ret;

Review Comment:
   Can we just return a `byte` here?



##########
hadoop-hdds/rocks-native/src/main/java/org/apache/hadoop/hdds/utils/db/managed/PipeInputStream.java:
##########
@@ -0,0 +1,81 @@
+package org.apache.hadoop.hdds.utils.db.managed;
+
+import org.apache.hadoop.hdds.utils.NativeLibraryLoader;
+import org.apache.hadoop.hdds.utils.NativeLibraryNotLoadedException;
+import java.io.InputStream;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.apache.hadoop.hdds.utils.NativeConstants.ROCKS_TOOLS_NATIVE_LIBRARY_NAME;
+
+/**
+ * JNI for reading data from pipe.
+ */
+public class PipeInputStream extends InputStream {
+
+  static {
+    NativeLibraryLoader.getInstance()
+            .loadLibrary(ROCKS_TOOLS_NATIVE_LIBRARY_NAME);
+  }
+  private byte[] byteBuffer;
+  private long nativeHandle;
+  private int numberOfBytesLeftToRead;
+  private int index = 0;
+  private int capacity;
+
+  private AtomicBoolean cleanup;
+
+  PipeInputStream(int capacity) throws NativeLibraryNotLoadedException {
+    if (!NativeLibraryLoader.getInstance()
+            .isLibraryLoaded(ROCKS_TOOLS_NATIVE_LIBRARY_NAME)) {
+      throw new NativeLibraryNotLoadedException(
+              ROCKS_TOOLS_NATIVE_LIBRARY_NAME);
+    }
+
+    this.byteBuffer = new byte[capacity];
+    this.numberOfBytesLeftToRead = 0;
+    this.capacity = capacity;
+    this.nativeHandle = newPipe();
+    this.cleanup = new AtomicBoolean(false);
+  }
+
+  long getNativeHandle() {
+    return nativeHandle;
+  }
+
+  @Override
+  public int read() {
+    if (numberOfBytesLeftToRead < 0) {
+      this.close();
+      return -1;
+    }
+    if (numberOfBytesLeftToRead == 0) {
+      numberOfBytesLeftToRead = readInternal(byteBuffer, capacity,
+              nativeHandle);
+      index = 0;
+      return read();
+    }
+    numberOfBytesLeftToRead--;
+    int ret = byteBuffer[index] & 0xFF;
+    index += 1;
+    return ret;

Review Comment:
   Can we just return a `byte` here? Or there are some other reasons



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] swamirishi commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "swamirishi (via GitHub)" <gi...@apache.org>.
swamirishi commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1129986880


##########
hadoop-hdds/rocks-native/src/main/java/org/apache/hadoop/hdds/utils/db/managed/package-info.java:
##########
@@ -0,0 +1,24 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.
+ */
+
+/**
+ * Annotation processors used at compile time by the Ozone project to validate
+ * internal annotations and related code as needed, if needed.
+ */
+
+package org.apache.hadoop.hdds.utils.db.managed;

Review Comment:
   what is the change here?



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1129965201


##########
hadoop-hdds/rocks-native/src/main/native/Pipe.h:
##########
@@ -0,0 +1,55 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.
+ */
+
+#ifndef ROCKS_NATIVE_PIPE_H
+#define ROCKS_NATIVE_PIPE_H
+
+#include <stdio.h>
+
+class Pipe {
+    public:
+        static const int READ_FILE_DESCRIPTOR_IDX;
+        static const int WRITE_FILE_DESCRIPTOR_IDX;
+        Pipe();
+        ~Pipe();
+        void close();
+        int getReadFd() {
+            return getPipeFileDescriptorIndex(READ_FILE_DESCRIPTOR_IDX);
+        }
+
+        int getWriteFd() {
+            return getPipeFileDescriptorIndex(WRITE_FILE_DESCRIPTOR_IDX);
+        }
+
+        int getPipeFileDescriptorIndex(int idx) {
+            return p[idx];
+        }
+
+        bool isOpen() {
+            return open;
+        }
+
+
+    private:
+        int p[2];
+        FILE* wr;
+        bool open;
+
+};
+
+#endif //ROCKS_NATIVE_PIPE_H

Review Comment:
   ```suggestion
   #endif //ROCKS_NATIVE_PIPE_H
   
   ```



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1129956985


##########
hadoop-hdds/rocks-native/src/main/java/org/apache/hadoop/hdds/utils/db/managed/PipeInputStream.java:
##########
@@ -0,0 +1,81 @@
+package org.apache.hadoop.hdds.utils.db.managed;
+
+import org.apache.hadoop.hdds.utils.NativeLibraryLoader;
+import org.apache.hadoop.hdds.utils.NativeLibraryNotLoadedException;
+import java.io.InputStream;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.apache.hadoop.hdds.utils.NativeConstants.ROCKS_TOOLS_NATIVE_LIBRARY_NAME;
+
+/**
+ * JNI for reading data from pipe.
+ */
+public class PipeInputStream extends InputStream {
+
+  static {
+    NativeLibraryLoader.getInstance()
+            .loadLibrary(ROCKS_TOOLS_NATIVE_LIBRARY_NAME);
+  }
+  private byte[] byteBuffer;
+  private long nativeHandle;
+  private int numberOfBytesLeftToRead;
+  private int index = 0;
+  private int capacity;
+
+  private AtomicBoolean cleanup;
+
+  PipeInputStream(int capacity) throws NativeLibraryNotLoadedException {
+    if (!NativeLibraryLoader.getInstance()
+            .isLibraryLoaded(ROCKS_TOOLS_NATIVE_LIBRARY_NAME)) {
+      throw new NativeLibraryNotLoadedException(
+              ROCKS_TOOLS_NATIVE_LIBRARY_NAME);
+    }
+
+    this.byteBuffer = new byte[capacity];
+    this.numberOfBytesLeftToRead = 0;
+    this.capacity = capacity;
+    this.nativeHandle = newPipe();
+    this.cleanup = new AtomicBoolean(false);
+  }
+
+  long getNativeHandle() {
+    return nativeHandle;
+  }
+
+  @Override
+  public int read() {
+    if (numberOfBytesLeftToRead < 0) {
+      this.close();
+      return -1;
+    }
+    if (numberOfBytesLeftToRead == 0) {
+      numberOfBytesLeftToRead = readInternal(byteBuffer, capacity,
+              nativeHandle);
+      index = 0;
+      return read();
+    }
+    numberOfBytesLeftToRead--;
+    int ret = byteBuffer[index] & 0xFF;
+    index += 1;
+    return ret;

Review Comment:
   Got it. I wonder if there a way to read from it more efficiently while maintaining the correctness? `int` is at least 4 bytes but we are only utilizing 1 byte.



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] swamirishi commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "swamirishi (via GitHub)" <gi...@apache.org>.
swamirishi commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1130006436


##########
hadoop-hdds/rocks-native/src/main/java/org/apache/hadoop/hdds/utils/db/managed/PipeInputStream.java:
##########
@@ -0,0 +1,99 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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 org.apache.hadoop.hdds.utils.NativeLibraryLoader;
+import org.apache.hadoop.hdds.utils.NativeLibraryNotLoadedException;
+import java.io.InputStream;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.apache.hadoop.hdds.utils.NativeConstants.ROCKS_TOOLS_NATIVE_LIBRARY_NAME;
+
+/**
+ * JNI for reading data from pipe.
+ */
+public class PipeInputStream extends InputStream {
+
+  static {
+    NativeLibraryLoader.getInstance()
+            .loadLibrary(ROCKS_TOOLS_NATIVE_LIBRARY_NAME);
+  }
+  private byte[] byteBuffer;
+  private long nativeHandle;
+  private int numberOfBytesLeftToRead;
+  private int index = 0;
+  private int capacity;
+
+  private AtomicBoolean cleanup;
+
+  PipeInputStream(int capacity) throws NativeLibraryNotLoadedException {
+    if (!NativeLibraryLoader.getInstance()
+            .isLibraryLoaded(ROCKS_TOOLS_NATIVE_LIBRARY_NAME)) {
+      throw new NativeLibraryNotLoadedException(
+              ROCKS_TOOLS_NATIVE_LIBRARY_NAME);
+    }
+
+    this.byteBuffer = new byte[capacity];
+    this.numberOfBytesLeftToRead = 0;
+    this.capacity = capacity;
+    this.nativeHandle = newPipe();
+    this.cleanup = new AtomicBoolean(false);
+  }
+
+  long getNativeHandle() {
+    return nativeHandle;
+  }
+
+  @Override
+  public int read() {
+    if (numberOfBytesLeftToRead < 0) {
+      this.close();
+      return -1;
+    }
+    if (numberOfBytesLeftToRead == 0) {
+      numberOfBytesLeftToRead = readInternal(byteBuffer, capacity,
+              nativeHandle);
+      index = 0;
+      return read();
+    }
+    numberOfBytesLeftToRead--;
+    int ret = byteBuffer[index] & 0xFF;
+    index += 1;
+    return ret;
+  }
+
+  private native long newPipe();
+
+  private native int readInternal(byte[] buff, int numberOfBytes,
+                                  long pipeHandle);

Review Comment:
   done



##########
hadoop-hdds/rocks-native/src/main/java/org/apache/hadoop/hdds/utils/db/managed/package-info.java:
##########
@@ -0,0 +1,24 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.
+ */
+
+/**
+ * Annotation processors used at compile time by the Ozone project to validate
+ * internal annotations and related code as needed, if needed.
+ */
+
+package org.apache.hadoop.hdds.utils.db.managed;

Review Comment:
   done



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] adoroszlai commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "adoroszlai (via GitHub)" <gi...@apache.org>.
adoroszlai commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1132822442


##########
hadoop-ozone/common/pom.xml:
##########
@@ -107,6 +107,10 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd">
       <groupId>org.junit.jupiter</groupId>
       <artifactId>junit-jupiter-params</artifactId>
     </dependency>
+    <dependency>
+      <groupId>org.apache.ozone</groupId>
+      <artifactId>hdds-server-framework</artifactId>
+    </dependency>

Review Comment:
   It's still there:
   
   https://github.com/apache/ozone/blob/dbc31173a2982ea8d96a66c64aab9d21f4b4104b/hadoop-ozone/common/pom.xml#L110-L113



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1131830542


##########
hadoop-hdds/rocks-native/src/main/native/SSTDumpTool.cpp:
##########
@@ -0,0 +1,49 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.
+ */
+
+#include "org_apache_hadoop_hdds_utils_db_managed_ManagedSSTDumpTool.h"
+#include "rocksdb/options.h"
+#include "rocksdb/sst_dump_tool.h"
+#include <string>
+#include "cplusplus_to_java_convert.h"
+#include "Pipe.h"
+#include <iostream>
+
+jint Java_org_apache_hadoop_hdds_utils_db_managed_ManagedSSTDumpTool_runInternal(JNIEnv *env, jobject obj,
+ jobjectArray argsArray, jlong optionsHandle, jlong pipeHandle) {
+    ROCKSDB_NAMESPACE::SSTDumpTool dumpTool;
+    ROCKSDB_NAMESPACE::Options *options = reinterpret_cast<ROCKSDB_NAMESPACE::Options *>(optionsHandle);
+    Pipe *pipe = reinterpret_cast<Pipe *>(pipeHandle);
+    int length = env->GetArrayLength(argsArray);
+    char *args[length + 1];
+    args[0] = strdup("./sst_dump");
+    for (int i = 0; i < length; i++) {
+        jstring str_val = (jstring)env->GetObjectArrayElement(argsArray, (jsize)i);
+        char *utf_str = (char *)env->GetStringUTFChars(str_val, JNI_FALSE);
+        args[i + 1] = strdup(utf_str);
+        env->ReleaseStringUTFChars(str_val, utf_str);
+    }
+    FILE *wr = fdopen(pipe->getWriteFd(), "w");
+    int ret = dumpTool.Run(length + 1, args, *options, wr);
+    for (int i = 0; i < length + 1; i++) {
+        free(args[i]);
+    }
+    fclose(wr);
+    pipe->close();

Review Comment:
   Could `PipeInputStream` Java object be garbage collected by JVM before C++ code here finished using it? Looks like it could be?
   
   In other words, is there any guarantee in KVM that objects passed through JNI won't be GC'ed?



##########
hadoop-hdds/rocks-native/src/main/native/SSTDumpTool.cpp:
##########
@@ -0,0 +1,49 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.
+ */
+
+#include "org_apache_hadoop_hdds_utils_db_managed_ManagedSSTDumpTool.h"
+#include "rocksdb/options.h"
+#include "rocksdb/sst_dump_tool.h"
+#include <string>
+#include "cplusplus_to_java_convert.h"
+#include "Pipe.h"
+#include <iostream>
+
+jint Java_org_apache_hadoop_hdds_utils_db_managed_ManagedSSTDumpTool_runInternal(JNIEnv *env, jobject obj,
+ jobjectArray argsArray, jlong optionsHandle, jlong pipeHandle) {
+    ROCKSDB_NAMESPACE::SSTDumpTool dumpTool;
+    ROCKSDB_NAMESPACE::Options *options = reinterpret_cast<ROCKSDB_NAMESPACE::Options *>(optionsHandle);
+    Pipe *pipe = reinterpret_cast<Pipe *>(pipeHandle);
+    int length = env->GetArrayLength(argsArray);
+    char *args[length + 1];
+    args[0] = strdup("./sst_dump");
+    for (int i = 0; i < length; i++) {
+        jstring str_val = (jstring)env->GetObjectArrayElement(argsArray, (jsize)i);
+        char *utf_str = (char *)env->GetStringUTFChars(str_val, JNI_FALSE);
+        args[i + 1] = strdup(utf_str);
+        env->ReleaseStringUTFChars(str_val, utf_str);
+    }
+    FILE *wr = fdopen(pipe->getWriteFd(), "w");
+    int ret = dumpTool.Run(length + 1, args, *options, wr);
+    for (int i = 0; i < length + 1; i++) {
+        free(args[i]);
+    }
+    fclose(wr);
+    pipe->close();

Review Comment:
   Could `PipeInputStream` Java object be garbage collected by JVM before C++ code here finished using it? Looks like it could be?
   
   In other words, is there any guarantee in JVM that objects passed through JNI won't be GC'ed?



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1131828526


##########
hadoop-hdds/rocks-native/src/main/native/SSTDumpTool.cpp:
##########
@@ -0,0 +1,49 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.
+ */
+
+#include "org_apache_hadoop_hdds_utils_db_managed_ManagedSSTDumpTool.h"
+#include "rocksdb/options.h"
+#include "rocksdb/sst_dump_tool.h"
+#include <string>
+#include "cplusplus_to_java_convert.h"
+#include "Pipe.h"
+#include <iostream>
+
+jint Java_org_apache_hadoop_hdds_utils_db_managed_ManagedSSTDumpTool_runInternal(JNIEnv *env, jobject obj,
+ jobjectArray argsArray, jlong optionsHandle, jlong pipeHandle) {

Review Comment:
   nit
   ```suggestion
     jobjectArray argsArray, jlong optionsHandle, jlong pipeHandle) {
   ```



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] swamirishi commented on pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "swamirishi (via GitHub)" <gi...@apache.org>.
swamirishi commented on PR #4315:
URL: https://github.com/apache/ozone/pull/4315#issuecomment-1452466399

   > n let me know if there are any special build parameters you have in mind for building this like the native library compile for hadoop?
   There are no special build parameters. Currently the module is built by default for all runs.


-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1123910804


##########
hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/MessageQueue.java:
##########
@@ -0,0 +1,188 @@
+/**
+ * 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;
+
+import org.apache.commons.io.input.buffer.CircularByteBuffer;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.concurrent.Callable;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * Message Queue to Pipe output from one output stream to another inputstream.
+ */
+public class MessageQueue {
+  private CircularByteBuffer byteBuffer;
+  private AtomicBoolean isRunning;
+
+  private MessageOutputStream messageOutputStream;
+  private MessageInputStream messageInputStream;
+
+  public MessageQueue(int bufferSize, long pollIntervalMillis) {
+    this.pollIntervalMillis = pollIntervalMillis;
+    init(bufferSize);
+  }
+
+  private void init(int bufferSize) {
+    this.byteBuffer = new CircularByteBuffer(bufferSize);
+    this.isRunning = new AtomicBoolean(false);
+    this.messageInputStream = new MessageInputStream(this);
+    this.messageOutputStream = new MessageOutputStream(this);
+  }
+
+  public void start() {
+    this.isRunning.set(true);
+  }
+
+  public void stop() {
+    this.isRunning.set(false);
+  }
+
+  public MessageOutputStream getMessageOutputStream() {
+    return messageOutputStream;
+  }
+
+  public MessageInputStream getMessageInputStream() {
+    return messageInputStream;
+  }
+
+  private long pollIntervalMillis;
+
+  public boolean isRunning() {
+    return isRunning.get();
+  }
+
+  private long getPollIntervalMillis() {
+    return pollIntervalMillis;
+  }
+
+  private boolean hasSpace(int requiredLength) {
+    return this.byteBuffer.hasSpace(requiredLength);
+  }
+
+  private boolean hasBytes() {
+    return this.byteBuffer.hasBytes();
+  }
+
+  private int getCurrentNumberOfBytes() {
+    return this.byteBuffer.getCurrentNumberOfBytes();
+  }
+
+  private void add(byte[] b, int off, int len) {
+    this.byteBuffer.add(b, off, len);
+  }
+
+  private int read(byte[] b, int off, int len) {
+    this.byteBuffer.read(b, off, len);
+    return len;
+  }
+
+  private static  <T> T callWithLock(Lock lock, Callable<T> callable)
+          throws Exception {
+    lock.lock();
+    try {
+      return callable.call();
+    } finally {
+      lock.unlock();
+    }
+  }
+  private static final class MessageOutputStream extends OutputStream {
+
+    private MessageQueue messageQueue;
+    private Lock writeLock;
+
+    private MessageOutputStream(MessageQueue messageQueue) {
+      this.messageQueue = messageQueue;
+      this.writeLock = new ReentrantLock();
+
+    }
+
+    private void waitForBytes(int requiredLength) throws InterruptedException {
+      while (!this.messageQueue.hasSpace(requiredLength)) {
+        Thread.sleep(this.messageQueue.getPollIntervalMillis());
+      }
+    }
+
+
+
+    @Override
+    public void write(int b) throws IOException {
+      this.write(new byte[]{(byte) b});
+    }
+
+    @Override
+    public void write(byte[] b, int off, int len) throws IOException {
+      try {
+        if (!this.messageQueue.isRunning()) {
+          throw new IOException("Message Queue is Closed");
+        }
+        waitForBytes(len);
+        callWithLock(this.writeLock, () -> {
+          waitForBytes(len);
+          this.messageQueue.add(b, off, len);
+          return true;
+        });
+      } catch (Exception e) {
+        throw new IOException(e);
+      }
+    }
+  }
+  private static final class MessageInputStream extends InputStream {

Review Comment:
   and javadoc?
   
   ```suggestion
   
     private static final class MessageInputStream extends InputStream {
   ```



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] swamirishi commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "swamirishi (via GitHub)" <gi...@apache.org>.
swamirishi commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1125377698


##########
hadoop-hdds/rocks-native/src/main/java/org/apache/hadoop/hdds/utils/NativeLibraryLoader.java:
##########
@@ -0,0 +1,138 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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;
+
+
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Files;
+import java.nio.file.StandardCopyOption;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Class to load Native Libraries.
+ */
+public class NativeLibraryLoader {
+  private static final String OS = System.getProperty("os.name").toLowerCase();
+  private Map<String, Boolean> librariesLoaded;
+  private static volatile NativeLibraryLoader instance;
+
+  public NativeLibraryLoader(final Map<String, Boolean> librariesLoaded) {
+    this.librariesLoaded = librariesLoaded;
+  }
+
+  private static synchronized void initNewInstance() {
+    if (instance == null) {
+      instance = new NativeLibraryLoader(new ConcurrentHashMap<>());
+    }
+  }
+
+  public static NativeLibraryLoader getInstance() {
+    if (instance == null) {
+      initNewInstance();
+    }
+    return instance;
+  }
+
+  public static String getJniLibraryFileName(String libraryName) {
+    return appendLibOsSuffix("lib" + libraryName);
+  }
+
+  public static boolean isMac() {

Review Comment:
   This is to get only the suffix. The lib file created would only have the extension .dll,.dylib or .so



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1125274559


##########
hadoop-hdds/rocks-native/src/main/java/org/apache/hadoop/hdds/utils/NativeLibraryLoader.java:
##########
@@ -0,0 +1,138 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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;
+
+
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Files;
+import java.nio.file.StandardCopyOption;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Class to load Native Libraries.
+ */
+public class NativeLibraryLoader {
+  private static final String OS = System.getProperty("os.name").toLowerCase();
+  private Map<String, Boolean> librariesLoaded;
+  private static volatile NativeLibraryLoader instance;
+
+  public NativeLibraryLoader(final Map<String, Boolean> librariesLoaded) {
+    this.librariesLoaded = librariesLoaded;
+  }
+
+  private static synchronized void initNewInstance() {
+    if (instance == null) {
+      instance = new NativeLibraryLoader(new ConcurrentHashMap<>());
+    }
+  }
+
+  public static NativeLibraryLoader getInstance() {
+    if (instance == null) {
+      initNewInstance();
+    }
+    return instance;
+  }
+
+  public static String getJniLibraryFileName(String libraryName) {
+    return appendLibOsSuffix("lib" + libraryName);
+  }
+
+  public static boolean isMac() {

Review Comment:
   What about `isLinux()`?
   
   https://github.com/apache/hadoop/blob/f7b1bb4dccc83eb26e661241ebf9f767f52b291b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java#L181-L197



##########
hadoop-hdds/rocks-native/src/main/java/org/apache/hadoop/hdds/utils/NativeLibraryLoader.java:
##########
@@ -0,0 +1,138 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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;
+
+
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Files;
+import java.nio.file.StandardCopyOption;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Class to load Native Libraries.
+ */
+public class NativeLibraryLoader {
+  private static final String OS = System.getProperty("os.name").toLowerCase();
+  private Map<String, Boolean> librariesLoaded;
+  private static volatile NativeLibraryLoader instance;
+
+  public NativeLibraryLoader(final Map<String, Boolean> librariesLoaded) {
+    this.librariesLoaded = librariesLoaded;
+  }
+
+  private static synchronized void initNewInstance() {
+    if (instance == null) {
+      instance = new NativeLibraryLoader(new ConcurrentHashMap<>());
+    }
+  }
+
+  public static NativeLibraryLoader getInstance() {
+    if (instance == null) {
+      initNewInstance();
+    }
+    return instance;
+  }
+
+  public static String getJniLibraryFileName(String libraryName) {
+    return appendLibOsSuffix("lib" + libraryName);
+  }
+
+  public static boolean isMac() {

Review Comment:
   What about Linux?
   
   https://github.com/apache/hadoop/blob/f7b1bb4dccc83eb26e661241ebf9f767f52b291b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java#L181-L197



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1125266942


##########
hadoop-hdds/rocks-native/src/main/native/SSTDumpTool.cpp:
##########
@@ -0,0 +1,49 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.
+ */
+
+#include "org_apache_hadoop_hdds_utils_db_managed_ManagedSSTDumpTool.h"
+#include "rocksdb/options.h"
+#include "rocksdb/sst_dump_tool.h"
+#include "string"
+#include "cplusplus_to_java_convert.h"
+#include "Pipe.h"
+#include "iostream"
+
+jint Java_org_apache_hadoop_hdds_utils_db_managed_ManagedSSTDumpTool_runInternal(JNIEnv *env, jobject obj, jobjectArray argsArray,
+jlong optionsHandle, jlong pipeHandle) {
+    ROCKSDB_NAMESPACE::SSTDumpTool dumpTool;
+    ROCKSDB_NAMESPACE::Options* options = reinterpret_cast<ROCKSDB_NAMESPACE::Options*>(optionsHandle);
+    Pipe* pipe = reinterpret_cast<Pipe*>(pipeHandle);
+    int length = env->GetArrayLength(argsArray);
+    char* args[length + 1];
+    args[0] = strdup("./sst_dump");
+    for(int i = 0; i < length; i++) {
+        jstring str_val = (jstring)env->GetObjectArrayElement(argsArray, (jsize)i);
+        char* utf_str = (char*)env->GetStringUTFChars(str_val, JNI_FALSE);
+        args[i+1] = strdup(utf_str);
+        env->ReleaseStringUTFChars(str_val, utf_str);
+    }
+    FILE* wr = fdopen(pipe->getWriteFd(), "w");

Review Comment:
   ```suggestion
       FILE *wr = fdopen(pipe->getWriteFd(), "w");
   ```



##########
hadoop-hdds/rocks-native/src/main/native/SSTDumpTool.cpp:
##########
@@ -0,0 +1,49 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.
+ */
+
+#include "org_apache_hadoop_hdds_utils_db_managed_ManagedSSTDumpTool.h"
+#include "rocksdb/options.h"
+#include "rocksdb/sst_dump_tool.h"
+#include "string"
+#include "cplusplus_to_java_convert.h"
+#include "Pipe.h"
+#include "iostream"
+
+jint Java_org_apache_hadoop_hdds_utils_db_managed_ManagedSSTDumpTool_runInternal(JNIEnv *env, jobject obj, jobjectArray argsArray,
+jlong optionsHandle, jlong pipeHandle) {
+    ROCKSDB_NAMESPACE::SSTDumpTool dumpTool;
+    ROCKSDB_NAMESPACE::Options* options = reinterpret_cast<ROCKSDB_NAMESPACE::Options*>(optionsHandle);
+    Pipe* pipe = reinterpret_cast<Pipe*>(pipeHandle);
+    int length = env->GetArrayLength(argsArray);
+    char* args[length + 1];
+    args[0] = strdup("./sst_dump");
+    for(int i = 0; i < length; i++) {
+        jstring str_val = (jstring)env->GetObjectArrayElement(argsArray, (jsize)i);
+        char* utf_str = (char*)env->GetStringUTFChars(str_val, JNI_FALSE);
+        args[i+1] = strdup(utf_str);
+        env->ReleaseStringUTFChars(str_val, utf_str);
+    }
+    FILE* wr = fdopen(pipe->getWriteFd(), "w");
+    int ret = dumpTool.Run(length + 1, args, *options, wr);
+    for (int i = 0; i<length+1; i++) {

Review Comment:
   ```suggestion
       for (int i = 0; i < length + 1; i++) {
   ```



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1125278927


##########
hadoop-hdds/rocks-native/src/main/java/org/apache/hadoop/hdds/utils/db/managed/ManagedSSTDumpTool.java:
##########
@@ -0,0 +1,100 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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 org.apache.hadoop.hdds.utils.NativeLibraryLoader;
+import org.apache.hadoop.hdds.utils.NativeLibraryNotLoadedException;
+
+import java.util.Map;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+
+import static org.apache.hadoop.hdds.utils.NativeConstants.ROCKS_TOOLS_NATIVE_LIBRARY_NAME;
+
+/**
+ * JNI for RocksDB SSTDumpTool. Pipes the output to an output stream
+ */
+public class ManagedSSTDumpTool {
+
+  static {
+    NativeLibraryLoader.getInstance()
+            .loadLibrary(ROCKS_TOOLS_NATIVE_LIBRARY_NAME);
+  }
+  private int bufferCapacity;
+  private ExecutorService executorService;
+
+  public ManagedSSTDumpTool(ExecutorService executorService,
+                            int bufferCapacity)
+          throws NativeLibraryNotLoadedException {
+    if (!NativeLibraryLoader.getInstance()
+            .isLibraryLoaded(ROCKS_TOOLS_NATIVE_LIBRARY_NAME)) {
+      throw new NativeLibraryNotLoadedException(
+              ROCKS_TOOLS_NATIVE_LIBRARY_NAME);
+    }
+    this.bufferCapacity = bufferCapacity;
+    this.executorService = executorService;
+  }
+
+  public SSTDumpToolTask run(String[] args, ManagedOptions options)
+          throws NativeLibraryNotLoadedException {
+    PipeInputStream pipeInputStream = new PipeInputStream(bufferCapacity);
+    return new SSTDumpToolTask(this.executorService.submit(() ->
+            this.runInternal(args, options.getNativeHandle(),
+            pipeInputStream.getNativeHandle())), pipeInputStream);
+  }
+
+  public SSTDumpToolTask run(Map<String, String> args, ManagedOptions options)
+          throws NativeLibraryNotLoadedException {
+    return this.run(args.entrySet().stream().map(e -> "--"
+            + (e.getValue() == null || e.getValue().isEmpty() ? e.getKey() :
+            e.getKey() + "=" + e.getValue())).toArray(String[]::new), options);
+  }
+
+  private native int runInternal(String[] args, long optionsHandle,
+                                  long pipeHandle);

Review Comment:
   ```suggestion
                                    long pipeHandle);
   ```



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1125270950


##########
hadoop-hdds/rocks-native/src/CMakeLists.txt:
##########
@@ -0,0 +1,78 @@
+#
+# 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
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# 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.
+#
+
+#
+# CMake configuration.
+#
+
+cmake_minimum_required(VERSION 3.1 FATAL_ERROR)

Review Comment:
   Also:
   
   > The FATAL_ERROR option is accepted but ignored by CMake 2.6 and higher.
   
   https://cmake.org/cmake/help/latest/command/cmake_minimum_required.html



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] swamirishi commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "swamirishi (via GitHub)" <gi...@apache.org>.
swamirishi commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1128826596


##########
hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/managed/ManagedSSTDumpIterator.java:
##########
@@ -0,0 +1,203 @@
+/**
+ * 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.collect.Lists;
+import org.apache.hadoop.hdds.utils.HddsServerUtil;
+import org.eclipse.jetty.io.RuntimeIOException;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * Iterator to Parse output of RocksDBSSTDumpTool.
+ */
+public class ManagedSSTDumpIterator implements

Review Comment:
   Moved it to rocks-native module



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] swamirishi commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "swamirishi (via GitHub)" <gi...@apache.org>.
swamirishi commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1129914589


##########
hadoop-hdds/rocks-native/src/main/java/org/apache/hadoop/hdds/utils/db/managed/ManagedSSTDumpIterator.java:
##########
@@ -0,0 +1,239 @@
+/**
+ * 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;

Review Comment:
   done



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1129975828


##########
hadoop-hdds/rocks-native/src/main/native/cplusplus_to_java_convert.h:
##########
@@ -0,0 +1,50 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.
+ */
+

Review Comment:
   I believe we still need to keep the original license header from RocksDB.
   
   ```suggestion
   
   // Copyright (c) 2011-present, Facebook, Inc.  All rights reserved.
   //  This source code is licensed under both the GPLv2 (found in the
   //  COPYING file in the root directory) and Apache 2.0 License
   //  (found in the LICENSE.Apache file in the root directory).
   
   ```



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1129985440


##########
hadoop-hdds/rocks-native/src/main/java/org/apache/hadoop/hdds/utils/db/managed/ManagedSSTDumpTool.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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 org.apache.hadoop.hdds.utils.NativeLibraryLoader;
+import org.apache.hadoop.hdds.utils.NativeLibraryNotLoadedException;
+
+import java.util.Map;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+
+import static org.apache.hadoop.hdds.utils.NativeConstants.ROCKS_TOOLS_NATIVE_LIBRARY_NAME;
+
+/**
+ * JNI for RocksDB SSTDumpTool. Pipes the output to an output stream
+ */
+public class ManagedSSTDumpTool {
+
+  static {
+    NativeLibraryLoader.getInstance()
+            .loadLibrary(ROCKS_TOOLS_NATIVE_LIBRARY_NAME);
+  }
+  private int bufferCapacity;
+  private ExecutorService executorService;
+
+  public ManagedSSTDumpTool(ExecutorService executorService,
+                            int bufferCapacity)
+          throws NativeLibraryNotLoadedException {
+    if (!NativeLibraryLoader.getInstance()
+            .isLibraryLoaded(ROCKS_TOOLS_NATIVE_LIBRARY_NAME)) {
+      throw new NativeLibraryNotLoadedException(
+              ROCKS_TOOLS_NATIVE_LIBRARY_NAME);
+    }
+    this.bufferCapacity = bufferCapacity;
+    this.executorService = executorService;
+  }
+
+  public SSTDumpToolTask run(String[] args, ManagedOptions options)
+          throws NativeLibraryNotLoadedException {
+    PipeInputStream pipeInputStream = new PipeInputStream(bufferCapacity);
+    return new SSTDumpToolTask(this.executorService.submit(() ->
+            this.runInternal(args, options.getNativeHandle(),
+            pipeInputStream.getNativeHandle())), pipeInputStream);

Review Comment:
   ```suggestion
       try (PipeInputStream pipeInputStream = new PipeInputStream(
           bufferCapacity)) {
         return new SSTDumpToolTask(this.executorService.submit(() ->
             this.runInternal(args, options.getNativeHandle(),
                 pipeInputStream.getNativeHandle())), pipeInputStream);
       }
   ```
   
   Related to https://github.com/apache/ozone/pull/4315/files#r1129983604



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] swamirishi commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "swamirishi (via GitHub)" <gi...@apache.org>.
swamirishi commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1132818293


##########
hadoop-hdds/rocks-native/pom.xml:
##########
@@ -0,0 +1,405 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed 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
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+  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. See accompanying LICENSE file.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <parent>
+        <artifactId>hdds</artifactId>
+        <groupId>org.apache.ozone</groupId>
+        <version>1.4.0-SNAPSHOT</version>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+    <name>Apache Ozone HDDS RocksDB Tools</name>
+    <artifactId>hdds-rocks-native</artifactId>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.ozone</groupId>
+            <artifactId>hdds-managed-rocksdb</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.eclipse.jetty</groupId>
+            <artifactId>jetty-io</artifactId>
+        </dependency>
+    </dependencies>
+    <properties>
+        <maven.compiler.source>8</maven.compiler.source>
+        <maven.compiler.target>8</maven.compiler.target>
+    </properties>
+
+    <profiles>
+        <profile>
+            <id>rocks_tools_native</id>
+            <activation>
+                <property>
+                    <name>rocks_tools_native</name>
+                </property>
+            </activation>
+            <properties>
+                <cmake.standards>23</cmake.standards>
+                <sstDump.include>true</sstDump.include>
+            </properties>
+            <build>
+                <plugins>
+                    <plugin>
+                        <groupId>com.googlecode.maven-download-plugin</groupId>
+                        <artifactId>download-maven-plugin</artifactId>
+                        <executions>
+                            <execution>
+                                <id>rocksdb source download</id>
+                                <phase>generate-sources</phase>
+                                <goals>
+                                    <goal>wget</goal>
+                                </goals>
+                                <configuration>
+                                    <url>https://github.com/facebook/rocksdb/archive/refs/tags/v${rocksdb.version}.tar.gz</url>
+                                    <outputFileName>rocksdb-v${rocksdb.version}.tar.gz</outputFileName>
+                                    <outputDirectory>${project.build.directory}/rocksdb</outputDirectory>
+                                </configuration>
+                            </execution>
+                            <execution>
+                                <id>zlib source download</id>
+                                <phase>generate-sources</phase>
+                                <goals>
+                                    <goal>wget</goal>
+                                </goals>
+                                <configuration>
+                                    <url>https://zlib.net/zlib-${zlib.version}.tar.gz</url>
+                                    <outputFileName>zlib-${zlib.version}.tar.gz</outputFileName>
+                                    <outputDirectory>${project.build.directory}/zlib</outputDirectory>
+                                </configuration>
+                            </execution>
+                            <execution>
+                                <id>bzip2 source download</id>
+                                <phase>generate-sources</phase>
+                                <goals>
+                                    <goal>wget</goal>
+                                </goals>
+                                <configuration>
+                                    <url>https://sourceware.org/pub/bzip2/bzip2-${bzip2.version}.tar.gz</url>
+                                    <outputFileName>bzip2-v${bzip2.version}.tar.gz</outputFileName>
+                                    <outputDirectory>${project.build.directory}/bzip2</outputDirectory>
+                                </configuration>
+                            </execution>
+                            <execution>
+                                <id>lz4 source download</id>
+                                <phase>generate-sources</phase>
+                                <goals>
+                                    <goal>wget</goal>
+                                </goals>
+                                <configuration>
+                                    <url>https://github.com/lz4/lz4/archive/refs/tags/v${lz4.version}.tar.gz</url>
+                                    <outputFileName>lz4-v${lz4.version}.tar.gz</outputFileName>
+                                    <outputDirectory>${project.build.directory}/lz4</outputDirectory>
+                                </configuration>
+                            </execution>
+                            <execution>
+                                <id>snappy source download</id>
+                                <phase>generate-sources</phase>
+                                <goals>
+                                    <goal>wget</goal>
+                                </goals>
+                                <configuration>
+                                    <url>https://github.com/google/snappy/archive/refs/tags/${snappy.version}.tar.gz</url>
+                                    <outputFileName>snappy-v${snappy.version}.tar.gz</outputFileName>
+                                    <outputDirectory>${project.build.directory}/snappy</outputDirectory>
+                                </configuration>
+                            </execution>
+                            <execution>
+                                <id>zstd source download</id>
+                                <phase>generate-sources</phase>
+                                <goals>
+                                    <goal>wget</goal>
+                                </goals>
+                                <configuration>
+                                    <url>https://github.com/facebook/zstd/archive/refs/tags/v${zstd.version}.tar.gz</url>
+                                    <outputFileName>zstd-v${zstd.version}.tar.gz</outputFileName>
+                                    <outputDirectory>${project.build.directory}/zstd</outputDirectory>
+                                </configuration>
+                            </execution>
+                        </executions>
+                    </plugin>
+                    <plugin>
+                        <groupId>org.apache.maven.plugins</groupId>
+                        <artifactId>maven-patch-plugin</artifactId>
+                        <version>1.1.1</version>
+                        <configuration>
+                            <patchFile>${basedir}/src/main/patches/rocks-native.patch</patchFile>
+                            <strip>1</strip>
+                            <targetDirectory>${project.build.directory}/rocksdb/rocksdb-${rocksdb.version}</targetDirectory>
+                        </configuration>
+                        <executions>
+                            <execution>
+                                <id>patch</id>
+                                <phase>process-sources</phase>
+                                <goals>
+                                    <goal>apply</goal>
+                                </goals>
+                            </execution>
+                        </executions>
+                    </plugin>
+                    <plugin>
+                        <artifactId>maven-antrun-plugin</artifactId>
+                        <executions>
+                            <execution>
+                                <id>unzip-artifact</id>
+                                <phase>generate-sources</phase>
+                                <configuration>
+                                    <tasks>
+                                        <untar src="${project.build.directory}/rocksdb/rocksdb-v${rocksdb.version}.tar.gz" compression="gzip" dest="${project.build.directory}/rocksdb/" />
+                                        <untar src="${project.build.directory}/zlib/zlib-${zlib.version}.tar.gz" compression="gzip" dest="${project.build.directory}/zlib/" />
+                                        <untar src="${project.build.directory}/bzip2/bzip2-v${bzip2.version}.tar.gz" compression="gzip" dest="${project.build.directory}/bzip2/" />
+                                        <untar src="${project.build.directory}/lz4/lz4-v${lz4.version}.tar.gz" compression="gzip" dest="${project.build.directory}/lz4/" />
+                                        <untar src="${project.build.directory}/snappy/snappy-v${snappy.version}.tar.gz" compression="gzip" dest="${project.build.directory}/snappy/" />
+                                        <untar src="${project.build.directory}/zstd/zstd-v${zstd.version}.tar.gz" compression="gzip" dest="${project.build.directory}/zstd/" />
+                                    </tasks>
+                                </configuration>
+                                <goals>
+                                    <goal>run</goal>
+                                </goals>
+                            </execution>
+                            <execution>
+                                <id>build-zlib</id>
+                                <phase>process-sources</phase>
+                                <configuration>
+                                    <tasks>
+                                        <chmod file="${project.build.directory}/zlib/zlib-${zlib.version}/configure" perm="775" />
+                                        <exec executable="./configure" dir="${project.build.directory}/zlib/zlib-${zlib.version}" failonerror="true">
+                                            <arg line="--static"/>
+                                            <env key="CFLAGS" value="-fPIC"/>
+                                        </exec>
+                                        <exec executable="make" dir="${project.build.directory}/zlib/zlib-${zlib.version}" failonerror="true"/>
+                                    </tasks>
+                                </configuration>
+                                <goals>
+                                    <goal>run</goal>
+                                </goals>
+                            </execution>
+                            <execution>
+                                <id>build-bzip2</id>
+                                <phase>process-sources</phase>
+                                <configuration>
+                                    <tasks>
+                                        <exec executable="make" dir="${project.build.directory}/bzip2/bzip2-${bzip2.version}" failonerror="true">
+                                            <arg line="CFLAGS='-fPIC'"/>
+                                        </exec>
+                                    </tasks>
+                                </configuration>
+                                <goals>
+                                    <goal>run</goal>
+                                </goals>
+                            </execution>
+                            <execution>
+                                <id>build-lz4</id>
+                                <phase>process-sources</phase>
+                                <configuration>
+                                    <tasks>
+                                        <exec executable="make" dir="${project.build.directory}/lz4/lz4-${lz4.version}" failonerror="true">
+                                            <arg line="CFLAGS='-fPIC'"/>
+                                        </exec>
+                                    </tasks>
+                                </configuration>
+                                <goals>
+                                    <goal>run</goal>
+                                </goals>
+                            </execution>
+                            <execution>
+                                <id>build-zstd</id>
+                                <phase>process-sources</phase>
+                                <configuration>
+                                    <tasks>
+                                        <exec executable="make" dir="${project.build.directory}/zstd/zstd-${zstd.version}" failonerror="true">
+                                            <arg line="CFLAGS='-fPIC'"/>
+                                        </exec>
+                                    </tasks>
+                                </configuration>
+                                <goals>
+                                    <goal>run</goal>
+                                </goals>
+                            </execution>
+                            <execution>
+                                <id>build-snappy</id>
+                                <phase>process-sources</phase>
+                                <configuration>
+                                    <tasks>
+                                        <mkdir dir="${project.build.directory}/snappy/lib"/>
+                                        <exec executable="cmake" failonerror="true" dir="${project.build.directory}/snappy/lib">
+                                            <arg line="${project.build.directory}/snappy/snappy-${snappy.version}"/>
+                                            <env key="CFLAGS" value="-fPIC"/>
+                                            <env key="CXXFLAGS" value="-fPIC"/>
+                                        </exec>
+                                        <exec executable="make" dir="${project.build.directory}/snappy/lib" failonerror="true"/>
+                                    </tasks>
+                                </configuration>
+                                <goals>
+                                    <goal>run</goal>
+                                </goals>
+                            </execution>
+                            <execution>
+                                <id>build-rocksjava</id>
+                                <phase>process-resources</phase>
+                                <configuration>
+                                    <tasks>
+                                        <exec executable="chmod" failonerror="true">
+                                            <arg line="-R"/>
+                                            <arg line="775"/>
+                                            <arg line="${project.build.directory}/rocksdb/rocksdb-${rocksdb.version}"/>
+                                        </exec>
+                                        <exec executable="make" dir="${project.build.directory}/rocksdb/rocksdb-${rocksdb.version}" failonerror="true">
+                                            <arg line="EXTRA_CXXFLAGS='-fPIC -I${project.build.directory}/snappy/lib -I${project.build.directory}/snappy/snappy-${snappy.version} -I${project.build.directory}/lz4/lz4-${lz4.version}/lib -I${project.build.directory}/zstd/zstd-${zstd.version}/lib -I${project.build.directory}/zstd/zstd-${zstd.version}/lib/dictBuilder -I${project.build.directory}/bzip2/bzip2-${bzip2.version} -I${project.build.directory}/zlib/zlib-${zlib.version}'"/>
+                                            <arg line="EXTRA_LDFLAGS='-L${project.build.directory}/snappy/lib -L${project.build.directory}/lz4/lz4-${lz4.version}/lib -L${project.build.directory}/zstd/zstd-${zstd.version}/lib -L${project.build.directory}/bzip2/bzip2-${bzip2.version} -L${project.build.directory}/zlib/zlib-${zlib.version}'"/>
+                                            <arg line="-j4"/>

Review Comment:
   done



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] swamirishi commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "swamirishi (via GitHub)" <gi...@apache.org>.
swamirishi commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1132817346


##########
hadoop-hdds/rocks-native/src/main/native/SSTDumpTool.cpp:
##########
@@ -0,0 +1,49 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.
+ */
+
+#include "org_apache_hadoop_hdds_utils_db_managed_ManagedSSTDumpTool.h"
+#include "rocksdb/options.h"
+#include "rocksdb/sst_dump_tool.h"
+#include <string>
+#include "cplusplus_to_java_convert.h"
+#include "Pipe.h"
+#include <iostream>
+
+jint Java_org_apache_hadoop_hdds_utils_db_managed_ManagedSSTDumpTool_runInternal(JNIEnv *env, jobject obj,
+ jobjectArray argsArray, jlong optionsHandle, jlong pipeHandle) {
+    ROCKSDB_NAMESPACE::SSTDumpTool dumpTool;
+    ROCKSDB_NAMESPACE::Options *options = reinterpret_cast<ROCKSDB_NAMESPACE::Options *>(optionsHandle);
+    Pipe *pipe = reinterpret_cast<Pipe *>(pipeHandle);
+    int length = env->GetArrayLength(argsArray);
+    char *args[length + 1];
+    args[0] = strdup("./sst_dump");
+    for (int i = 0; i < length; i++) {
+        jstring str_val = (jstring)env->GetObjectArrayElement(argsArray, (jsize)i);
+        char *utf_str = (char *)env->GetStringUTFChars(str_val, JNI_FALSE);
+        args[i + 1] = strdup(utf_str);
+        env->ReleaseStringUTFChars(str_val, utf_str);
+    }
+    FILE *wr = fdopen(pipe->getWriteFd(), "w");
+    int ret = dumpTool.Run(length + 1, args, *options, wr);
+    for (int i = 0; i < length + 1; i++) {
+        free(args[i]);
+    }

Review Comment:
   SSTDumptool does it we dont have to check it here.



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1123886443


##########
hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/HddsServerUtil.java:
##########
@@ -78,23 +83,19 @@
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_HEARTBEAT_LOG_WARN_DEFAULT;
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_HEARTBEAT_LOG_WARN_INTERVAL_COUNT;
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL;
-import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_HEARTBEAT_RPC_TIMEOUT;
-import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_HEARTBEAT_RPC_TIMEOUT_DEFAULT;
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_HEARTBEAT_RPC_RETRY_COUNT;
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_HEARTBEAT_RPC_RETRY_COUNT_DEFAULT;
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_HEARTBEAT_RPC_RETRY_INTERVAL;
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_HEARTBEAT_RPC_RETRY_INTERVAL_DEFAULT;
-import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_STALENODE_INTERVAL;
-import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_STALENODE_INTERVAL_DEFAULT;
+import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_HEARTBEAT_RPC_TIMEOUT;
+import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_HEARTBEAT_RPC_TIMEOUT_DEFAULT;
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_INFO_WAIT_DURATION;
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_INFO_WAIT_DURATION_DEFAULT;
+import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_STALENODE_INTERVAL;

Review Comment:
   @prashantpogde It's probably IntelliJ.
   
   @swamirishi You can turn off IntelliJ's "Optimize imports on the fly": 
   
   https://www.jetbrains.com/help/idea/creating-and-optimizing-imports.html#184c396b



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1123908176


##########
hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/MessageQueue.java:
##########
@@ -0,0 +1,188 @@
+/**
+ * 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;
+
+import org.apache.commons.io.input.buffer.CircularByteBuffer;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.concurrent.Callable;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * Message Queue to Pipe output from one output stream to another inputstream.
+ */
+public class MessageQueue {
+  private CircularByteBuffer byteBuffer;
+  private AtomicBoolean isRunning;
+
+  private MessageOutputStream messageOutputStream;
+  private MessageInputStream messageInputStream;
+
+  public MessageQueue(int bufferSize, long pollIntervalMillis) {
+    this.pollIntervalMillis = pollIntervalMillis;
+    init(bufferSize);
+  }
+
+  private void init(int bufferSize) {
+    this.byteBuffer = new CircularByteBuffer(bufferSize);
+    this.isRunning = new AtomicBoolean(false);
+    this.messageInputStream = new MessageInputStream(this);
+    this.messageOutputStream = new MessageOutputStream(this);
+  }
+
+  public void start() {
+    this.isRunning.set(true);
+  }
+
+  public void stop() {
+    this.isRunning.set(false);
+  }
+
+  public MessageOutputStream getMessageOutputStream() {
+    return messageOutputStream;
+  }
+
+  public MessageInputStream getMessageInputStream() {
+    return messageInputStream;
+  }
+
+  private long pollIntervalMillis;
+
+  public boolean isRunning() {
+    return isRunning.get();
+  }
+
+  private long getPollIntervalMillis() {
+    return pollIntervalMillis;
+  }
+
+  private boolean hasSpace(int requiredLength) {
+    return this.byteBuffer.hasSpace(requiredLength);
+  }
+
+  private boolean hasBytes() {
+    return this.byteBuffer.hasBytes();
+  }
+
+  private int getCurrentNumberOfBytes() {
+    return this.byteBuffer.getCurrentNumberOfBytes();
+  }
+
+  private void add(byte[] b, int off, int len) {
+    this.byteBuffer.add(b, off, len);
+  }
+
+  private int read(byte[] b, int off, int len) {
+    this.byteBuffer.read(b, off, len);
+    return len;
+  }
+
+  private static  <T> T callWithLock(Lock lock, Callable<T> callable)
+          throws Exception {
+    lock.lock();
+    try {
+      return callable.call();
+    } finally {
+      lock.unlock();
+    }
+  }
+  private static final class MessageOutputStream extends OutputStream {
+
+    private MessageQueue messageQueue;
+    private Lock writeLock;
+
+    private MessageOutputStream(MessageQueue messageQueue) {
+      this.messageQueue = messageQueue;
+      this.writeLock = new ReentrantLock();
+

Review Comment:
   ```suggestion
   ```



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1129959980


##########
hadoop-hdds/rocks-native/src/main/native/Pipe.cpp:
##########
@@ -0,0 +1,37 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.
+ */
+
+#include "Pipe.h"
+#include <unistd.h>
+
+const int Pipe::READ_FILE_DESCRIPTOR_IDX = 0;
+const int Pipe::WRITE_FILE_DESCRIPTOR_IDX = 1;
+
+Pipe::Pipe() {
+    pipe(p);
+    open = true;
+}
+
+Pipe::~Pipe() {
+    ::close(p[0]);
+    ::close(p[1]);

Review Comment:
   Use `READ_FILE_DESCRIPTOR_IDX` and `WRITE_FILE_DESCRIPTOR_IDX` defined right above?



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1129959039


##########
hadoop-hdds/rocks-native/src/main/java/org/apache/hadoop/hdds/utils/package-info.java:
##########
@@ -0,0 +1,24 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.
+ */
+
+/**
+ * Annotation processors used at compile time by the Ozone project to validate
+ * internal annotations and related code as needed, if needed.
+ */
+
+package org.apache.hadoop.hdds.utils;

Review Comment:
   HDDS-8077
   
   ```suggestion
   package org.apache.hadoop.hdds.utils;
   
   ```



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1129970979


##########
hadoop-hdds/rocks-native/src/main/native/Pipe.cpp:
##########
@@ -0,0 +1,37 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.
+ */
+
+#include "Pipe.h"
+#include <unistd.h>
+
+const int Pipe::READ_FILE_DESCRIPTOR_IDX = 0;
+const int Pipe::WRITE_FILE_DESCRIPTOR_IDX = 1;
+
+Pipe::Pipe() {
+    pipe(p);
+    open = true;
+}
+
+Pipe::~Pipe() {
+    ::close(p[0]);
+    ::close(p[1]);

Review Comment:
   ~~`unistd.h` defined `STDIN_FILENO`, `STDOUT_FILENO` and `STDERR_FILENO`~~



##########
hadoop-hdds/rocks-native/src/main/native/Pipe.cpp:
##########
@@ -0,0 +1,37 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.
+ */
+
+#include "Pipe.h"
+#include <unistd.h>
+
+const int Pipe::READ_FILE_DESCRIPTOR_IDX = 0;
+const int Pipe::WRITE_FILE_DESCRIPTOR_IDX = 1;
+
+Pipe::Pipe() {
+    pipe(p);
+    open = true;
+}
+
+Pipe::~Pipe() {
+    ::close(p[0]);
+    ::close(p[1]);

Review Comment:
   Use `READ_FILE_DESCRIPTOR_IDX` and `WRITE_FILE_DESCRIPTOR_IDX` defined right above?



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1129974543


##########
hadoop-hdds/rocks-native/src/main/native/Pipe.h:
##########
@@ -0,0 +1,55 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.
+ */
+
+#ifndef ROCKS_NATIVE_PIPE_H
+#define ROCKS_NATIVE_PIPE_H
+
+#include <stdio.h>
+
+class Pipe {
+    public:
+        static const int READ_FILE_DESCRIPTOR_IDX;
+        static const int WRITE_FILE_DESCRIPTOR_IDX;
+        Pipe();
+        ~Pipe();
+        void close();
+        int getReadFd() {
+            return getPipeFileDescriptorIndex(READ_FILE_DESCRIPTOR_IDX);
+        }
+
+        int getWriteFd() {
+            return getPipeFileDescriptorIndex(WRITE_FILE_DESCRIPTOR_IDX);
+        }
+
+        int getPipeFileDescriptorIndex(int idx) {
+            return p[idx];
+        }
+
+        bool isOpen() {
+            return open;
+        }
+
+
+    private:
+        int p[2];
+        FILE* wr;
+        bool open;
+
+};
+
+#endif //ROCKS_NATIVE_PIPE_H

Review Comment:
   Looks like code on your branch actually has new line but Github PR diff doesn't show it. Disregard if this is the case.



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] swamirishi commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "swamirishi (via GitHub)" <gi...@apache.org>.
swamirishi commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1129983847


##########
hadoop-hdds/rocks-native/src/main/native/Pipe.h:
##########
@@ -0,0 +1,55 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.
+ */
+
+#ifndef ROCKS_NATIVE_PIPE_H
+#define ROCKS_NATIVE_PIPE_H
+
+#include <stdio.h>
+
+class Pipe {
+    public:
+        static const int READ_FILE_DESCRIPTOR_IDX;
+        static const int WRITE_FILE_DESCRIPTOR_IDX;
+        Pipe();
+        ~Pipe();
+        void close();
+        int getReadFd() {
+            return getPipeFileDescriptorIndex(READ_FILE_DESCRIPTOR_IDX);
+        }
+
+        int getWriteFd() {
+            return getPipeFileDescriptorIndex(WRITE_FILE_DESCRIPTOR_IDX);
+        }
+
+        int getPipeFileDescriptorIndex(int idx) {
+            return p[idx];
+        }
+
+        bool isOpen() {
+            return open;
+        }
+
+
+    private:
+        int p[2];
+        FILE* wr;
+        bool open;
+
+};
+
+#endif //ROCKS_NATIVE_PIPE_H

Review Comment:
   yeah it has



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] swamirishi commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "swamirishi (via GitHub)" <gi...@apache.org>.
swamirishi commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1130004472


##########
hadoop-hdds/rocks-native/src/main/java/org/apache/hadoop/hdds/utils/db/managed/ManagedSSTDumpIterator.java:
##########
@@ -0,0 +1,238 @@
+/**
+ * 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 org.apache.hadoop.hdds.utils.NativeLibraryNotLoadedException;
+import org.eclipse.jetty.io.RuntimeIOException;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
+import java.util.Iterator;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * Iterator to Parse output of RocksDBSSTDumpTool.
+ */
+public class ManagedSSTDumpIterator implements
+        Iterator<ManagedSSTDumpIterator.KeyValue>, AutoCloseable {
+  private static final String SST_DUMP_TOOL_CLASS =
+          "org.apache.hadoop.hdds.utils.db.managed.ManagedSSTDumpTool";
+  private static final String PATTERN_REGEX =
+          "'([^=>]+)' seq:([0-9]+), type:([0-9]+) =>";
+
+  public static final int PATTERN_KEY_GROUP_NUMBER = 1;
+  public static final int PATTERN_SEQ_GROUP_NUMBER = 2;
+  public static final int PATTERN_TYPE_GROUP_NUMBER = 3;
+  private static final Pattern PATTERN_MATCHER =
+          Pattern.compile(PATTERN_REGEX);
+  private BufferedReader processOutput;
+  private StringBuilder stdoutString;
+
+  private Matcher currentMatcher;
+  private int prevMatchEndIndex;
+  private KeyValue currentKey;
+  private char[] charBuffer;
+  private KeyValue nextKey;
+
+  private long pollIntervalMillis;
+  private ManagedSSTDumpTool.SSTDumpToolTask sstDumpToolTask;
+  private Lock lock;
+  private AtomicBoolean open;
+
+
+  public ManagedSSTDumpIterator(ManagedSSTDumpTool sstDumpTool,
+                                String sstFilePath,
+                                ManagedOptions options,
+                                long pollIntervalMillis) throws IOException,
+          NativeLibraryNotLoadedException {
+    File sstFile = new File(sstFilePath);
+    if (!sstFile.exists() || !sstFile.isFile()) {
+      throw new IOException(String.format("Invalid SST File Path : %s",
+              sstFile.getAbsolutePath()));
+    }
+    this.pollIntervalMillis = pollIntervalMillis;
+    this.lock = new ReentrantLock();
+    init(sstDumpTool, sstFile, options);
+  }
+
+  private void init(ManagedSSTDumpTool sstDumpTool, File sstFile,
+                    ManagedOptions options)
+          throws NativeLibraryNotLoadedException {
+    String[] args = {"--file=" + sstFile.getAbsolutePath(),
+                     "--command=scan"};
+    this.sstDumpToolTask = sstDumpTool.run(args, options);
+    processOutput = new BufferedReader(new InputStreamReader(
+            sstDumpToolTask.getPipedOutput(), StandardCharsets.UTF_8));
+    stdoutString = new StringBuilder();
+    currentMatcher = PATTERN_MATCHER.matcher(stdoutString);
+    charBuffer = new char[8192];
+    open = new AtomicBoolean(true);
+    next();
+  }
+
+  /**
+   * Throws Runtime exception in the case iterator is closed or
+   * the native Dumptool exited with non zero exit value.
+   */
+  private void checkSanityOfProcess() {
+    if (!this.open.get()) {
+      throw new RuntimeException("Iterator has been closed");
+    }
+    if (sstDumpToolTask.getFuture().isDone()
+            && sstDumpToolTask.exitValue() != 0) {
+      throw new RuntimeException("Process Terminated with non zero " +
+              String.format("exit value %d", sstDumpToolTask.exitValue()));
+    }
+  }
+
+  /**
+   *
+   * @return
+   * Throws Runtime Exception in case of SST File read failure
+   */
+
+  @Override
+  public boolean hasNext() {
+    checkSanityOfProcess();
+    return nextKey != null;
+  }
+
+  /**
+   *
+   * @return next Key
+   * Throws Runtime Exception incase of failure.
+   */
+  @Override
+  public KeyValue next() {
+    lock.lock();
+    try {
+      checkSanityOfProcess();
+      currentKey = nextKey;
+      nextKey = null;
+      while (!currentMatcher.find()) {
+        try {
+          if (prevMatchEndIndex != 0) {
+            stdoutString = new StringBuilder(stdoutString.substring(
+                    prevMatchEndIndex, stdoutString.length()));
+            prevMatchEndIndex = 0;
+            currentMatcher = PATTERN_MATCHER.matcher(stdoutString);
+          }
+          Thread.sleep(pollIntervalMillis);
+          int numberOfCharsRead = processOutput.read(charBuffer);
+          if (numberOfCharsRead < 0) {
+            if (currentKey != null) {
+              currentKey.setValue(stdoutString.toString());
+            }
+            return currentKey;
+          }
+          stdoutString.append(charBuffer, 0, numberOfCharsRead);
+          currentMatcher.reset();
+        } catch (IOException | InterruptedException e) {
+          throw new RuntimeIOException(e);

Review Comment:
   Cannot throw exception since this implements Iterator



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] swamirishi commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "swamirishi (via GitHub)" <gi...@apache.org>.
swamirishi commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1132833164


##########
hadoop-ozone/common/pom.xml:
##########
@@ -107,6 +107,10 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd">
       <groupId>org.junit.jupiter</groupId>
       <artifactId>junit-jupiter-params</artifactId>
     </dependency>
+    <dependency>
+      <groupId>org.apache.ozone</groupId>
+      <artifactId>hdds-server-framework</artifactId>
+    </dependency>

Review Comment:
   removed 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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on PR #4315:
URL: https://github.com/apache/ozone/pull/4315#issuecomment-1464381647

   Verified that build time is unchanged with normal dev workflow.
   
   Compilation passed locally in 1m59s on db3300d6caf9ca468214e081152f1ce1ce1bc7a9 with skip tests/javadoc/shade:
   
   ```
   $ mvn clean install -DskipTests -e -Dmaven.javadoc.skip=true -DskipShade
   ...
   [INFO] BUILD SUCCESS
   [INFO] ------------------------------------------------------------------------
   [INFO] Total time:  01:59 min
   [INFO] Finished at: 2023-03-10T12:25:36-08:00
   ```
   
   Current master branch b1695e30f51cd069a98753e8bf67fb41f9216080 with same maven args:
   ```
   $ mvn clean install -DskipTests -e -Dmaven.javadoc.skip=true -DskipShade
   [INFO] BUILD SUCCESS
   [INFO] ------------------------------------------------------------------------
   [INFO] Total time:  02:02 min
   [INFO] Finished at: 2023-03-10T12:28:38-08:00
   ```


-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1125253991


##########
hadoop-hdds/rocks-native/src/main/java/org/apache/hadoop/hdds/utils/db/managed/ManagedSSTDumpIterator.java:
##########
@@ -0,0 +1,238 @@
+/**
+ * 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 org.apache.hadoop.hdds.utils.NativeLibraryNotLoadedException;
+import org.eclipse.jetty.io.RuntimeIOException;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
+import java.util.Iterator;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * Iterator to Parse output of RocksDBSSTDumpTool.
+ */
+public class ManagedSSTDumpIterator implements
+        Iterator<ManagedSSTDumpIterator.KeyValue>, AutoCloseable {
+  private static final String SST_DUMP_TOOL_CLASS =
+          "org.apache.hadoop.hdds.utils.db.managed.ManagedSSTDumpTool";
+  private static final String PATTERN_REGEX =
+          "'([^=>]+)' seq:([0-9]+), type:([0-9]+) =>";
+
+  public static final int PATTERN_KEY_GROUP_NUMBER = 1;
+  public static final int PATTERN_SEQ_GROUP_NUMBER = 2;
+  public static final int PATTERN_TYPE_GROUP_NUMBER = 3;
+  private static final Pattern PATTERN_MATCHER =
+          Pattern.compile(PATTERN_REGEX);
+  private BufferedReader processOutput;
+  private StringBuilder stdoutString;
+
+  private Matcher currentMatcher;
+  private int prevMatchEndIndex;
+  private KeyValue currentKey;
+  private char[] charBuffer;
+  private KeyValue nextKey;
+
+  private long pollIntervalMillis;
+  private ManagedSSTDumpTool.SSTDumpToolTask sstDumpToolTask;
+  private Lock lock;
+  private AtomicBoolean open;
+
+
+  public ManagedSSTDumpIterator(ManagedSSTDumpTool sstDumpTool,
+                                String sstFilePath,
+                                ManagedOptions options,
+                                long pollIntervalMillis) throws IOException,
+          NativeLibraryNotLoadedException {
+    File sstFile = new File(sstFilePath);
+    if (!sstFile.exists() || !sstFile.isFile()) {
+      throw new IOException(String.format("Invalid SST File Path : %s",
+              sstFile.getAbsolutePath()));
+    }
+    this.pollIntervalMillis = pollIntervalMillis;
+    this.lock = new ReentrantLock();
+    init(sstDumpTool, sstFile, options);
+  }
+
+  private void init(ManagedSSTDumpTool sstDumpTool, File sstFile,
+                    ManagedOptions options)
+          throws NativeLibraryNotLoadedException {
+    String[] args = {"--file=" + sstFile.getAbsolutePath(),
+                     "--command=scan"};
+    this.sstDumpToolTask = sstDumpTool.run(args, options);
+    processOutput = new BufferedReader(new InputStreamReader(
+            sstDumpToolTask.getPipedOutput(), StandardCharsets.UTF_8));
+    stdoutString = new StringBuilder();
+    currentMatcher = PATTERN_MATCHER.matcher(stdoutString);
+    charBuffer = new char[8192];
+    open = new AtomicBoolean(true);
+    next();
+  }
+
+  /**
+   * Throws Runtime exception in the case iterator is closed or
+   * the native Dumptool exited with non zero exit value.
+   */
+  private void checkSanityOfProcess() {
+    if (!this.open.get()) {
+      throw new RuntimeException("Iterator has been closed");
+    }
+    if (sstDumpToolTask.getFuture().isDone()
+            && sstDumpToolTask.exitValue() != 0) {
+      throw new RuntimeException("Process Terminated with non zero " +
+              String.format("exit value %d", sstDumpToolTask.exitValue()));
+    }
+  }
+
+  /**
+   *
+   * @return
+   * Throws Runtime Exception in case of SST File read failure
+   */
+
+  @Override
+  public boolean hasNext() {
+    checkSanityOfProcess();
+    return nextKey != null;
+  }
+
+  /**
+   *
+   * @return next Key
+   * Throws Runtime Exception incase of failure.
+   */
+  @Override
+  public KeyValue next() {

Review Comment:
   javadoc



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1125261059


##########
hadoop-hdds/rocks-native/src/main/native/SSTDumpTool.cpp:
##########
@@ -0,0 +1,49 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.
+ */
+
+#include "org_apache_hadoop_hdds_utils_db_managed_ManagedSSTDumpTool.h"
+#include "rocksdb/options.h"
+#include "rocksdb/sst_dump_tool.h"
+#include "string"

Review Comment:
   nit: STL
   
   ```suggestion
   #include <string>
   ```



##########
hadoop-hdds/rocks-native/src/main/native/SSTDumpTool.cpp:
##########
@@ -0,0 +1,49 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.
+ */
+
+#include "org_apache_hadoop_hdds_utils_db_managed_ManagedSSTDumpTool.h"
+#include "rocksdb/options.h"
+#include "rocksdb/sst_dump_tool.h"
+#include "string"
+#include "cplusplus_to_java_convert.h"
+#include "Pipe.h"
+#include "iostream"

Review Comment:
   nit: STL
   
   ```suggestion
   #include <iostream>
   ```



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1125264489


##########
hadoop-hdds/rocks-native/src/main/native/SSTDumpTool.cpp:
##########
@@ -0,0 +1,49 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.
+ */
+
+#include "org_apache_hadoop_hdds_utils_db_managed_ManagedSSTDumpTool.h"
+#include "rocksdb/options.h"
+#include "rocksdb/sst_dump_tool.h"
+#include "string"
+#include "cplusplus_to_java_convert.h"
+#include "Pipe.h"
+#include "iostream"
+
+jint Java_org_apache_hadoop_hdds_utils_db_managed_ManagedSSTDumpTool_runInternal(JNIEnv *env, jobject obj, jobjectArray argsArray,
+jlong optionsHandle, jlong pipeHandle) {
+    ROCKSDB_NAMESPACE::SSTDumpTool dumpTool;
+    ROCKSDB_NAMESPACE::Options* options = reinterpret_cast<ROCKSDB_NAMESPACE::Options*>(optionsHandle);
+    Pipe* pipe = reinterpret_cast<Pipe*>(pipeHandle);

Review Comment:
   Conventionally we do this:
   
   ```suggestion
       Pipe *pipe = reinterpret_cast<Pipe *>(pipeHandle);
   ```
   
   ref: https://stackoverflow.com/a/8947986



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] swamirishi commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "swamirishi (via GitHub)" <gi...@apache.org>.
swamirishi commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1128825303


##########
.github/workflows/ci.yml:
##########
@@ -61,7 +61,7 @@ jobs:
     needs:
       - build-info
     runs-on: ubuntu-20.04
-    timeout-minutes: 30
+    timeout-minutes: 60

Review Comment:
   reverted done



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1128962661


##########
hadoop-hdds/rocks-native/pom.xml:
##########
@@ -0,0 +1,405 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed 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
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+  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. See accompanying LICENSE file.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <parent>
+        <artifactId>hdds</artifactId>
+        <groupId>org.apache.ozone</groupId>
+        <version>1.4.0-SNAPSHOT</version>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+    <name>Apache Ozone HDDS Rocks</name>

Review Comment:
   Maybe this name is better? Or call it `RocksDB Native Tools`
   ```suggestion
       <name>Apache Ozone HDDS RocksDB Tools</name>
   ```



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] swamirishi commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "swamirishi (via GitHub)" <gi...@apache.org>.
swamirishi commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1132780915


##########
hadoop-hdds/rocks-native/src/main/native/SSTDumpTool.cpp:
##########
@@ -0,0 +1,49 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.
+ */
+
+#include "org_apache_hadoop_hdds_utils_db_managed_ManagedSSTDumpTool.h"
+#include "rocksdb/options.h"
+#include "rocksdb/sst_dump_tool.h"
+#include <string>
+#include "cplusplus_to_java_convert.h"
+#include "Pipe.h"
+#include <iostream>
+
+jint Java_org_apache_hadoop_hdds_utils_db_managed_ManagedSSTDumpTool_runInternal(JNIEnv *env, jobject obj,
+ jobjectArray argsArray, jlong optionsHandle, jlong pipeHandle) {
+    ROCKSDB_NAMESPACE::SSTDumpTool dumpTool;
+    ROCKSDB_NAMESPACE::Options *options = reinterpret_cast<ROCKSDB_NAMESPACE::Options *>(optionsHandle);
+    Pipe *pipe = reinterpret_cast<Pipe *>(pipeHandle);
+    int length = env->GetArrayLength(argsArray);
+    char *args[length + 1];
+    args[0] = strdup("./sst_dump");
+    for (int i = 0; i < length; i++) {
+        jstring str_val = (jstring)env->GetObjectArrayElement(argsArray, (jsize)i);
+        char *utf_str = (char *)env->GetStringUTFChars(str_val, JNI_FALSE);
+        args[i + 1] = strdup(utf_str);
+        env->ReleaseStringUTFChars(str_val, utf_str);
+    }
+    FILE *wr = fdopen(pipe->getWriteFd(), "w");
+    int ret = dumpTool.Run(length + 1, args, *options, wr);
+    for (int i = 0; i < length + 1; i++) {
+        free(args[i]);
+    }
+    fclose(wr);
+    pipe->close();

Review Comment:
   The finalize method has been overriden for this.
   



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] swamirishi commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "swamirishi (via GitHub)" <gi...@apache.org>.
swamirishi commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1125512879


##########
hadoop-hdds/rocks-native/pom.xml:
##########
@@ -0,0 +1,233 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed 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
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+  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. See accompanying LICENSE file.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <parent>
+        <artifactId>hdds</artifactId>
+        <groupId>org.apache.ozone</groupId>
+        <version>1.4.0-SNAPSHOT</version>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+    <name>Apache Ozone HDDS Rocks</name>
+    <artifactId>hdds-rocks-native</artifactId>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.ozone</groupId>
+            <artifactId>hdds-server-framework</artifactId>
+        </dependency>
+    </dependencies>
+    <properties>
+        <maven.compiler.source>8</maven.compiler.source>
+        <maven.compiler.target>8</maven.compiler.target>
+        <cmake.standards>23</cmake.standards>
+        <sstDump.include>true</sstDump.include>
+    </properties>
+
+
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>com.googlecode.maven-download-plugin</groupId>
+                <artifactId>download-maven-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <id>rocksdb source download</id>
+                        <phase>generate-resources</phase>
+                        <goals>
+                            <goal>wget</goal>
+                        </goals>
+                        <configuration>
+                            <url>https://github.com/facebook/rocksdb/archive/refs/tags/v${rocksdb.version}.tar.gz</url>
+                            <outputFileName>rocksdb-v${rocksdb.version}.tar.gz</outputFileName>
+                            <outputDirectory>${project.build.directory}/rocksdb</outputDirectory>
+                        </configuration>
+                    </execution>
+                    <execution>
+                        <id>zlib source download</id>
+                        <phase>generate-resources</phase>
+                        <goals>
+                            <goal>wget</goal>
+                        </goals>
+                        <configuration>
+                            <url>https://zlib.net/zlib-${zlib.version}.tar.gz</url>
+                            <outputFileName>zlib-${zlib.version}.tar.gz</outputFileName>
+                            <outputDirectory>${project.build.directory}/zlib</outputDirectory>
+                        </configuration>
+                    </execution>
+                    <execution>
+                        <id>bzip2 source download</id>
+                        <phase>generate-resources</phase>
+                        <goals>
+                            <goal>wget</goal>
+                        </goals>
+                        <configuration>
+                            <url>https://sourceware.org/pub/bzip2/bzip2-${bzip2.version}.tar.gz</url>
+                            <outputFileName>bzip2-v${bzip2.version}.tar.gz</outputFileName>
+                            <outputDirectory>${project.build.directory}/bzip2</outputDirectory>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+            <plugin>
+                <artifactId>maven-antrun-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <id>unzip-artifact</id>
+                        <phase>generate-resources</phase>
+                        <configuration>
+                            <tasks>
+                                <untar src="${project.build.directory}/rocksdb/rocksdb-v${rocksdb.version}.tar.gz" compression="gzip" dest="${project.build.directory}/rocksdb/" />
+                                <untar src="${project.build.directory}/zlib/zlib-${zlib.version}.tar.gz" compression="gzip" dest="${project.build.directory}/zlib/" />
+                                <untar src="${project.build.directory}/bzip2/bzip2-v${bzip2.version}.tar.gz" compression="gzip" dest="${project.build.directory}/bzip2/" />
+                            </tasks>
+                        </configuration>
+                        <goals>
+                            <goal>run</goal>
+                        </goals>
+                    </execution>
+                    <execution>
+                        <id>build-zlib</id>
+                        <phase>generate-resources</phase>
+                        <configuration>
+                            <tasks>
+                                <chmod file="${project.build.directory}/zlib/zlib-${zlib.version}/configure" perm="775" />
+                                <exec executable="${shell-executable}" dir="${project.build.directory}/zlib/zlib-${zlib.version}" failonerror="true">
+                                    <arg line="./configure"/>
+                                </exec>
+                                <exec executable="make" dir="${project.build.directory}/zlib/zlib-${zlib.version}" failonerror="true"/>
+                            </tasks>
+                        </configuration>
+                        <goals>
+                            <goal>run</goal>
+                        </goals>
+                    </execution>
+                    <execution>
+                        <id>build-bzip2</id>
+                        <phase>generate-resources</phase>
+                        <configuration>
+                            <tasks>
+                                <exec executable="make" dir="${project.build.directory}/bzip2/bzip2-${bzip2.version}" failonerror="true"/>
+                            </tasks>
+                        </configuration>
+                        <goals>
+                            <goal>run</goal>
+                        </goals>
+                    </execution>
+                    <execution>
+                        <id>build-rocksjava</id>
+                        <phase>generate-resources</phase>
+                        <configuration>
+                            <tasks>
+                                <exec executable="chmod" failonerror="true">
+                                    <arg line="-R"/>
+                                    <arg line="775"/>
+                                    <arg line="${project.build.directory}/rocksdb/rocksdb-${rocksdb.version}"/>
+                                </exec>
+                                <exec executable="make" dir="${project.build.directory}/rocksdb/rocksdb-${rocksdb.version}" failonerror="true">
+                                    <arg line="-j16"/>
+                                    <arg line="tools"/>
+                                </exec>>
+                            </tasks>
+                        </configuration>
+                        <goals>
+                            <goal>run</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
+            <plugin>
+                <groupId>org.codehaus.mojo</groupId>
+                <artifactId>native-maven-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <phase>compile</phase>
+                        <goals>
+                            <goal>javah</goal>
+                        </goals>
+                        <configuration>
+                            <javahPath>${env.JAVA_HOME}/bin/javah</javahPath>
+                            <javahClassNames>
+                                <javahClassName>org.apache.hadoop.hdds.utils.db.managed.ManagedSSTDumpTool</javahClassName>
+                            </javahClassNames>
+                            <javahOutputDirectory>${project.build.directory}/native/javah</javahOutputDirectory>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+            <plugin>
+                <groupId>org.apache.hadoop</groupId>
+                <artifactId>hadoop-maven-plugins</artifactId>
+                <executions>
+                    <execution>
+                        <id>cmake-compile</id>
+                        <phase>compile</phase>
+                        <goals><goal>cmake-compile</goal></goals>
+                        <configuration>
+                            <source>${basedir}/src</source>
+                            <vars>
+                                <GENERATED_JAVAH>${project.build.directory}/native/javah</GENERATED_JAVAH>
+                                <NATIVE_DIR>${basedir}/src/main/native</NATIVE_DIR>
+                                <SST_DUMP_INCLUDE>${sstDump.include}</SST_DUMP_INCLUDE>
+                                <CMAKE_STANDARDS>${cmake.standards}</CMAKE_STANDARDS>
+                                <ROCKSDB_HEADERS>${project.build.directory}/rocksdb/rocksdb-${rocksdb.version}/include</ROCKSDB_HEADERS>
+                                <ROCKSDB_LIB>${project.build.directory}/rocksdb/rocksdb-${rocksdb.version}</ROCKSDB_LIB>
+                                <ZLIB_LIB>${project.build.directory}/zlib/zlib-${zlib.version}</ZLIB_LIB>
+                                <BZIP2_LIB>${project.build.directory}/bzip2/bzip2-${bzip2.version}</BZIP2_LIB>
+                            </vars>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+            <plugin>
+            <artifactId>maven-antrun-plugin</artifactId>

Review Comment:
   done



##########
pom.xml:
##########
@@ -1533,6 +1538,11 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs
         <artifactId>curator-client</artifactId>
         <version>${curator.version}</version>
       </dependency>
+      <dependency>
+        <groupId>com.googlecode.maven-download-plugin</groupId>
+        <artifactId>download-maven-plugin</artifactId>
+        <version>${download-maven-plugin.version}</version>
+      </dependency>

Review Comment:
   done



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1125257742


##########
hadoop-hdds/rocks-native/src/main/native/Pipe.h:
##########
@@ -0,0 +1,37 @@
+//
+// Created by Swaminathan Balachandran on 3/2/23.
+//
+
+#ifndef UNTITLED_PIPE_H
+#define UNTITLED_PIPE_H

Review Comment:
   ```suggestion
   #ifndef ROCKS_NATIVE_PIPE_H
   #define ROCKS_NATIVE_PIPE_H
   ```



##########
hadoop-hdds/rocks-native/src/main/native/Pipe.h:
##########
@@ -0,0 +1,37 @@
+//
+// Created by Swaminathan Balachandran on 3/2/23.
+//
+
+#ifndef UNTITLED_PIPE_H
+#define UNTITLED_PIPE_H
+
+
+#include <stdio.h>
+
+class Pipe {
+    public:
+        Pipe();
+        ~Pipe();
+        void close();
+        int getReadFd() {
+            return p[0];
+        }
+
+        int getWriteFd() {
+            return p[1];
+        }
+
+        bool isOpen() {
+            return open;
+        }
+
+
+    private:
+        int p[2];
+        FILE* wr;
+        bool open;
+
+};
+
+
+#endif //UNTITLED_PIPE_H

Review Comment:
   ```suggestion
   #endif //ROCKS_NATIVE_PIPE_H
   ```



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] swamirishi closed pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "swamirishi (via GitHub)" <gi...@apache.org>.
swamirishi closed pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool
URL: https://github.com/apache/ozone/pull/4315


-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1123926827


##########
hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/managed/ManagedSSTDumpIterator.java:
##########
@@ -0,0 +1,203 @@
+/**
+ * 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.collect.Lists;
+import org.apache.hadoop.hdds.utils.HddsServerUtil;
+import org.eclipse.jetty.io.RuntimeIOException;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * Iterator to Parse output of RocksDBSSTDumpTool.
+ */
+public class ManagedSSTDumpIterator implements
+        Iterator<ManagedSSTDumpIterator.KeyValue>, AutoCloseable {
+  private Process process;
+  private static final String SST_DUMP_TOOL_CLASS =
+          "org.apache.hadoop.hdds.utils.db.managed.ManagedSSTDumpTool";
+  private static final String PATTERN_REGEX =
+          "'([^=>]+)' seq:([0-9]+), type:([0-9]+) =>";
+
+  public static final int PATTERN_KEY_GROUP_NUMBER = 1;
+  public static final int PATTERN_SEQ_GROUP_NUMBER = 2;
+  public static final int PATTERN_TYPE_GROUP_NUMBER = 3;
+  private static final Pattern PATTERN_MATCHER =
+          Pattern.compile(PATTERN_REGEX);
+  private BufferedReader processOutput;
+  private StringBuilder stdoutString;
+
+  private Matcher currentMatcher;
+  private int prevMatchEndIndex;
+  private KeyValue currentKey;
+  private char[] charBuffer;
+  private KeyValue nextKey;
+
+  private long pollIntervalMillis;
+  private Lock lock;
+
+
+  public ManagedSSTDumpIterator(String sstDumptoolJarPath,
+                                String sstFilePath,
+                                long pollIntervalMillis) throws IOException {
+    File sstFile = new File(sstFilePath);
+    if (!sstFile.exists() || !sstFile.isFile()) {
+      throw new IOException(String.format("Invalid SST File Path : %s",
+              sstFile.getAbsolutePath()));
+    }
+    this.pollIntervalMillis = pollIntervalMillis;
+    this.lock = new ReentrantLock();
+    init(sstFile, sstDumptoolJarPath);
+  }
+
+  private void init(File sstFile, String sstDumptoolJarPath)
+          throws IOException {
+    List<String> args = Lists.newArrayList(
+            "--file=" + sstFile.getAbsolutePath(),
+            "--command=scan");
+    process = HddsServerUtil.getJavaProcess(Collections.emptyList(),
+            sstDumptoolJarPath, SST_DUMP_TOOL_CLASS, args).start();
+    processOutput = new BufferedReader(new InputStreamReader(
+            process.getInputStream(), StandardCharsets.UTF_8));
+    stdoutString = new StringBuilder();
+    currentMatcher = PATTERN_MATCHER.matcher(stdoutString);
+    charBuffer = new char[8192];
+    next();
+  }
+
+  private void checkSanityOfProcess() {
+    if (!process.isAlive() && process.exitValue() != 0) {
+      throw new RuntimeException("Process Terminated with non zero " +
+              String.format("exit value %d", process.exitValue()));
+    }
+  }
+
+  @Override
+  public boolean hasNext() {
+    checkSanityOfProcess();
+    return nextKey != null;
+  }
+
+  @Override
+  public KeyValue next() throws RuntimeIOException {

Review Comment:
   Unchcked exception like `RuntimeIOException` does not need to be declared.
   
   ```suggestion
     public KeyValue next() {
   ```



##########
hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/managed/ManagedSSTDumpIterator.java:
##########
@@ -0,0 +1,203 @@
+/**
+ * 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.collect.Lists;
+import org.apache.hadoop.hdds.utils.HddsServerUtil;
+import org.eclipse.jetty.io.RuntimeIOException;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * Iterator to Parse output of RocksDBSSTDumpTool.
+ */
+public class ManagedSSTDumpIterator implements
+        Iterator<ManagedSSTDumpIterator.KeyValue>, AutoCloseable {
+  private Process process;
+  private static final String SST_DUMP_TOOL_CLASS =
+          "org.apache.hadoop.hdds.utils.db.managed.ManagedSSTDumpTool";
+  private static final String PATTERN_REGEX =
+          "'([^=>]+)' seq:([0-9]+), type:([0-9]+) =>";
+
+  public static final int PATTERN_KEY_GROUP_NUMBER = 1;
+  public static final int PATTERN_SEQ_GROUP_NUMBER = 2;
+  public static final int PATTERN_TYPE_GROUP_NUMBER = 3;
+  private static final Pattern PATTERN_MATCHER =
+          Pattern.compile(PATTERN_REGEX);
+  private BufferedReader processOutput;
+  private StringBuilder stdoutString;
+
+  private Matcher currentMatcher;
+  private int prevMatchEndIndex;
+  private KeyValue currentKey;
+  private char[] charBuffer;
+  private KeyValue nextKey;
+
+  private long pollIntervalMillis;
+  private Lock lock;
+
+
+  public ManagedSSTDumpIterator(String sstDumptoolJarPath,
+                                String sstFilePath,
+                                long pollIntervalMillis) throws IOException {
+    File sstFile = new File(sstFilePath);
+    if (!sstFile.exists() || !sstFile.isFile()) {
+      throw new IOException(String.format("Invalid SST File Path : %s",
+              sstFile.getAbsolutePath()));
+    }
+    this.pollIntervalMillis = pollIntervalMillis;
+    this.lock = new ReentrantLock();
+    init(sstFile, sstDumptoolJarPath);
+  }
+
+  private void init(File sstFile, String sstDumptoolJarPath)
+          throws IOException {
+    List<String> args = Lists.newArrayList(
+            "--file=" + sstFile.getAbsolutePath(),
+            "--command=scan");
+    process = HddsServerUtil.getJavaProcess(Collections.emptyList(),
+            sstDumptoolJarPath, SST_DUMP_TOOL_CLASS, args).start();
+    processOutput = new BufferedReader(new InputStreamReader(
+            process.getInputStream(), StandardCharsets.UTF_8));
+    stdoutString = new StringBuilder();
+    currentMatcher = PATTERN_MATCHER.matcher(stdoutString);
+    charBuffer = new char[8192];
+    next();
+  }
+
+  private void checkSanityOfProcess() {
+    if (!process.isAlive() && process.exitValue() != 0) {
+      throw new RuntimeException("Process Terminated with non zero " +
+              String.format("exit value %d", process.exitValue()));
+    }
+  }
+
+  @Override
+  public boolean hasNext() {
+    checkSanityOfProcess();
+    return nextKey != null;
+  }
+
+  @Override
+  public KeyValue next() throws RuntimeIOException {

Review Comment:
   Unchecked exception like `RuntimeIOException` does not need to be declared.
   
   ```suggestion
     public KeyValue next() {
   ```



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1125276838


##########
hadoop-hdds/rocks-native/src/main/java/org/apache/hadoop/hdds/utils/NativeLibraryLoader.java:
##########
@@ -0,0 +1,138 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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;
+
+
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Files;
+import java.nio.file.StandardCopyOption;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Class to load Native Libraries.
+ */
+public class NativeLibraryLoader {
+  private static final String OS = System.getProperty("os.name").toLowerCase();
+  private Map<String, Boolean> librariesLoaded;
+  private static volatile NativeLibraryLoader instance;
+
+  public NativeLibraryLoader(final Map<String, Boolean> librariesLoaded) {
+    this.librariesLoaded = librariesLoaded;
+  }
+
+  private static synchronized void initNewInstance() {
+    if (instance == null) {
+      instance = new NativeLibraryLoader(new ConcurrentHashMap<>());
+    }
+  }
+
+  public static NativeLibraryLoader getInstance() {
+    if (instance == null) {
+      initNewInstance();
+    }
+    return instance;
+  }
+
+  public static String getJniLibraryFileName(String libraryName) {
+    return appendLibOsSuffix("lib" + libraryName);
+  }
+
+  public static boolean isMac() {
+    return OS.contains("mac");
+  }
+
+  public static boolean isWindows() {
+    return OS.contains("win");
+  }
+
+  private static String getLibOsSuffix() {
+    if (isMac()) {
+      return ".dylib";
+    } else if (isWindows()) {
+      return ".dll";
+    }
+    return ".so";
+  }
+  private static String appendLibOsSuffix(String libraryFileName) {

Review Comment:
   ```suggestion
   
     private static String appendLibOsSuffix(String libraryFileName) {
   ```



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1125283487


##########
hadoop-hdds/rocks-native/src/main/java/org/apache/hadoop/hdds/utils/db/managed/PipeInputStream.java:
##########
@@ -0,0 +1,81 @@
+package org.apache.hadoop.hdds.utils.db.managed;
+
+import org.apache.hadoop.hdds.utils.NativeLibraryLoader;
+import org.apache.hadoop.hdds.utils.NativeLibraryNotLoadedException;
+import java.io.InputStream;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.apache.hadoop.hdds.utils.NativeConstants.ROCKS_TOOLS_NATIVE_LIBRARY_NAME;
+
+/**
+ * JNI for reading data from pipe.
+ */
+public class PipeInputStream extends InputStream {
+
+  static {
+    NativeLibraryLoader.getInstance()
+            .loadLibrary(ROCKS_TOOLS_NATIVE_LIBRARY_NAME);
+  }
+  private byte[] byteBuffer;
+  private long nativeHandle;
+  private int numberOfBytesLeftToRead;
+  private int index = 0;
+  private int capacity;
+
+  private AtomicBoolean cleanup;
+
+  PipeInputStream(int capacity) throws NativeLibraryNotLoadedException {
+    if (!NativeLibraryLoader.getInstance()
+            .isLibraryLoaded(ROCKS_TOOLS_NATIVE_LIBRARY_NAME)) {
+      throw new NativeLibraryNotLoadedException(
+              ROCKS_TOOLS_NATIVE_LIBRARY_NAME);
+    }
+
+    this.byteBuffer = new byte[capacity];
+    this.numberOfBytesLeftToRead = 0;
+    this.capacity = capacity;
+    this.nativeHandle = newPipe();
+    this.cleanup = new AtomicBoolean(false);
+  }
+
+  long getNativeHandle() {
+    return nativeHandle;
+  }
+
+  @Override
+  public int read() {
+    if (numberOfBytesLeftToRead < 0) {
+      this.close();
+      return -1;
+    }
+    if (numberOfBytesLeftToRead == 0) {
+      numberOfBytesLeftToRead = readInternal(byteBuffer, capacity,
+              nativeHandle);
+      index = 0;
+      return read();
+    }
+    numberOfBytesLeftToRead--;
+    int ret = byteBuffer[index] & 0xFF;
+    index += 1;
+    return ret;
+  }
+
+  private native long newPipe();
+
+  private native int readInternal(byte[] buff, int numberOfBytes,
+                                  long pipeHandle);
+  private native void closeInternal(long pipeHandle);
+
+  @Override
+  public void close() {
+    if (this.cleanup.compareAndSet(false, true)) {
+      closeInternal(this.nativeHandle);
+    }
+  }
+
+  @Override
+  protected void finalize() throws Throwable {
+    close();
+    super.finalize();
+  }
+}

Review Comment:
   ```suggestion
   }
   
   ```



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1125278443


##########
pom.xml:
##########
@@ -292,6 +293,12 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs
     <aspectj-plugin.version>1.14.0</aspectj-plugin.version>
     <restrict-imports.enforcer.version>2.0.0</restrict-imports.enforcer.version>
     <awaitility.version>4.2.0</awaitility.version>
+    <bzip2.version>1.0.8</bzip2.version>
+    <zlib.version>1.2.13</zlib.version>
+    <lz4.version>1.9.3</lz4.version>
+    <snappy.version>1.1.8</snappy.version>
+    <zstd.version>1.4.9</zstd.version>
+

Review Comment:
   ```suggestion
   ```



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1129936935


##########
hadoop-hdds/rocks-native/pom.xml:
##########
@@ -0,0 +1,405 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed 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
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+  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. See accompanying LICENSE file.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <parent>
+        <artifactId>hdds</artifactId>
+        <groupId>org.apache.ozone</groupId>
+        <version>1.4.0-SNAPSHOT</version>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+    <name>Apache Ozone HDDS RocksDB Tools</name>
+    <artifactId>hdds-rocks-native</artifactId>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.ozone</groupId>
+            <artifactId>hdds-managed-rocksdb</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.eclipse.jetty</groupId>
+            <artifactId>jetty-io</artifactId>
+        </dependency>
+    </dependencies>
+    <properties>
+        <maven.compiler.source>8</maven.compiler.source>
+        <maven.compiler.target>8</maven.compiler.target>
+    </properties>
+
+    <profiles>
+        <profile>
+            <id>rocks_tools_native</id>
+            <activation>
+                <property>
+                    <name>rocks_tools_native</name>
+                </property>
+            </activation>
+            <properties>
+                <cmake.standards>23</cmake.standards>
+                <sstDump.include>true</sstDump.include>
+            </properties>
+            <build>
+                <plugins>
+                    <plugin>
+                        <groupId>com.googlecode.maven-download-plugin</groupId>
+                        <artifactId>download-maven-plugin</artifactId>
+                        <executions>
+                            <execution>
+                                <id>rocksdb source download</id>
+                                <phase>generate-sources</phase>
+                                <goals>
+                                    <goal>wget</goal>
+                                </goals>
+                                <configuration>
+                                    <url>https://github.com/facebook/rocksdb/archive/refs/tags/v${rocksdb.version}.tar.gz</url>
+                                    <outputFileName>rocksdb-v${rocksdb.version}.tar.gz</outputFileName>
+                                    <outputDirectory>${project.build.directory}/rocksdb</outputDirectory>
+                                </configuration>
+                            </execution>
+                            <execution>
+                                <id>zlib source download</id>
+                                <phase>generate-sources</phase>
+                                <goals>
+                                    <goal>wget</goal>
+                                </goals>
+                                <configuration>
+                                    <url>https://zlib.net/zlib-${zlib.version}.tar.gz</url>
+                                    <outputFileName>zlib-${zlib.version}.tar.gz</outputFileName>
+                                    <outputDirectory>${project.build.directory}/zlib</outputDirectory>
+                                </configuration>
+                            </execution>
+                            <execution>
+                                <id>bzip2 source download</id>
+                                <phase>generate-sources</phase>
+                                <goals>
+                                    <goal>wget</goal>
+                                </goals>
+                                <configuration>
+                                    <url>https://sourceware.org/pub/bzip2/bzip2-${bzip2.version}.tar.gz</url>
+                                    <outputFileName>bzip2-v${bzip2.version}.tar.gz</outputFileName>
+                                    <outputDirectory>${project.build.directory}/bzip2</outputDirectory>
+                                </configuration>
+                            </execution>
+                            <execution>
+                                <id>lz4 source download</id>
+                                <phase>generate-sources</phase>
+                                <goals>
+                                    <goal>wget</goal>
+                                </goals>
+                                <configuration>
+                                    <url>https://github.com/lz4/lz4/archive/refs/tags/v${lz4.version}.tar.gz</url>
+                                    <outputFileName>lz4-v${lz4.version}.tar.gz</outputFileName>
+                                    <outputDirectory>${project.build.directory}/lz4</outputDirectory>
+                                </configuration>
+                            </execution>
+                            <execution>
+                                <id>snappy source download</id>
+                                <phase>generate-sources</phase>
+                                <goals>
+                                    <goal>wget</goal>
+                                </goals>
+                                <configuration>
+                                    <url>https://github.com/google/snappy/archive/refs/tags/${snappy.version}.tar.gz</url>
+                                    <outputFileName>snappy-v${snappy.version}.tar.gz</outputFileName>
+                                    <outputDirectory>${project.build.directory}/snappy</outputDirectory>
+                                </configuration>
+                            </execution>
+                            <execution>
+                                <id>zstd source download</id>
+                                <phase>generate-sources</phase>
+                                <goals>
+                                    <goal>wget</goal>
+                                </goals>
+                                <configuration>
+                                    <url>https://github.com/facebook/zstd/archive/refs/tags/v${zstd.version}.tar.gz</url>
+                                    <outputFileName>zstd-v${zstd.version}.tar.gz</outputFileName>
+                                    <outputDirectory>${project.build.directory}/zstd</outputDirectory>
+                                </configuration>
+                            </execution>
+                        </executions>
+                    </plugin>
+                    <plugin>
+                        <groupId>org.apache.maven.plugins</groupId>
+                        <artifactId>maven-patch-plugin</artifactId>
+                        <version>1.1.1</version>
+                        <configuration>
+                            <patchFile>${basedir}/src/main/patches/rocks-native.patch</patchFile>
+                            <strip>1</strip>
+                            <targetDirectory>${project.build.directory}/rocksdb/rocksdb-${rocksdb.version}</targetDirectory>
+                        </configuration>
+                        <executions>
+                            <execution>
+                                <id>patch</id>
+                                <phase>process-sources</phase>
+                                <goals>
+                                    <goal>apply</goal>
+                                </goals>
+                            </execution>
+                        </executions>
+                    </plugin>
+                    <plugin>
+                        <artifactId>maven-antrun-plugin</artifactId>
+                        <executions>
+                            <execution>
+                                <id>unzip-artifact</id>
+                                <phase>generate-sources</phase>
+                                <configuration>
+                                    <tasks>
+                                        <untar src="${project.build.directory}/rocksdb/rocksdb-v${rocksdb.version}.tar.gz" compression="gzip" dest="${project.build.directory}/rocksdb/" />
+                                        <untar src="${project.build.directory}/zlib/zlib-${zlib.version}.tar.gz" compression="gzip" dest="${project.build.directory}/zlib/" />
+                                        <untar src="${project.build.directory}/bzip2/bzip2-v${bzip2.version}.tar.gz" compression="gzip" dest="${project.build.directory}/bzip2/" />
+                                        <untar src="${project.build.directory}/lz4/lz4-v${lz4.version}.tar.gz" compression="gzip" dest="${project.build.directory}/lz4/" />
+                                        <untar src="${project.build.directory}/snappy/snappy-v${snappy.version}.tar.gz" compression="gzip" dest="${project.build.directory}/snappy/" />
+                                        <untar src="${project.build.directory}/zstd/zstd-v${zstd.version}.tar.gz" compression="gzip" dest="${project.build.directory}/zstd/" />
+                                    </tasks>
+                                </configuration>
+                                <goals>
+                                    <goal>run</goal>
+                                </goals>
+                            </execution>
+                            <execution>
+                                <id>build-zlib</id>
+                                <phase>process-sources</phase>
+                                <configuration>
+                                    <tasks>
+                                        <chmod file="${project.build.directory}/zlib/zlib-${zlib.version}/configure" perm="775" />
+                                        <exec executable="./configure" dir="${project.build.directory}/zlib/zlib-${zlib.version}" failonerror="true">
+                                            <arg line="--static"/>
+                                            <env key="CFLAGS" value="-fPIC"/>
+                                        </exec>
+                                        <exec executable="make" dir="${project.build.directory}/zlib/zlib-${zlib.version}" failonerror="true"/>
+                                    </tasks>
+                                </configuration>
+                                <goals>
+                                    <goal>run</goal>
+                                </goals>
+                            </execution>
+                            <execution>
+                                <id>build-bzip2</id>
+                                <phase>process-sources</phase>
+                                <configuration>
+                                    <tasks>
+                                        <exec executable="make" dir="${project.build.directory}/bzip2/bzip2-${bzip2.version}" failonerror="true">
+                                            <arg line="CFLAGS='-fPIC'"/>
+                                        </exec>
+                                    </tasks>
+                                </configuration>
+                                <goals>
+                                    <goal>run</goal>
+                                </goals>
+                            </execution>
+                            <execution>
+                                <id>build-lz4</id>
+                                <phase>process-sources</phase>
+                                <configuration>
+                                    <tasks>
+                                        <exec executable="make" dir="${project.build.directory}/lz4/lz4-${lz4.version}" failonerror="true">
+                                            <arg line="CFLAGS='-fPIC'"/>
+                                        </exec>
+                                    </tasks>
+                                </configuration>
+                                <goals>
+                                    <goal>run</goal>
+                                </goals>
+                            </execution>
+                            <execution>
+                                <id>build-zstd</id>
+                                <phase>process-sources</phase>
+                                <configuration>
+                                    <tasks>
+                                        <exec executable="make" dir="${project.build.directory}/zstd/zstd-${zstd.version}" failonerror="true">
+                                            <arg line="CFLAGS='-fPIC'"/>
+                                        </exec>
+                                    </tasks>
+                                </configuration>
+                                <goals>
+                                    <goal>run</goal>
+                                </goals>
+                            </execution>
+                            <execution>
+                                <id>build-snappy</id>
+                                <phase>process-sources</phase>
+                                <configuration>
+                                    <tasks>
+                                        <mkdir dir="${project.build.directory}/snappy/lib"/>
+                                        <exec executable="cmake" failonerror="true" dir="${project.build.directory}/snappy/lib">
+                                            <arg line="${project.build.directory}/snappy/snappy-${snappy.version}"/>
+                                            <env key="CFLAGS" value="-fPIC"/>
+                                            <env key="CXXFLAGS" value="-fPIC"/>
+                                        </exec>
+                                        <exec executable="make" dir="${project.build.directory}/snappy/lib" failonerror="true"/>
+                                    </tasks>
+                                </configuration>
+                                <goals>
+                                    <goal>run</goal>
+                                </goals>
+                            </execution>
+                            <execution>
+                                <id>build-rocksjava</id>
+                                <phase>process-resources</phase>
+                                <configuration>
+                                    <tasks>
+                                        <exec executable="chmod" failonerror="true">
+                                            <arg line="-R"/>
+                                            <arg line="775"/>
+                                            <arg line="${project.build.directory}/rocksdb/rocksdb-${rocksdb.version}"/>
+                                        </exec>
+                                        <exec executable="make" dir="${project.build.directory}/rocksdb/rocksdb-${rocksdb.version}" failonerror="true">
+                                            <arg line="EXTRA_CXXFLAGS='-fPIC -I${project.build.directory}/snappy/lib -I${project.build.directory}/snappy/snappy-${snappy.version} -I${project.build.directory}/lz4/lz4-${lz4.version}/lib -I${project.build.directory}/zstd/zstd-${zstd.version}/lib -I${project.build.directory}/zstd/zstd-${zstd.version}/lib/dictBuilder -I${project.build.directory}/bzip2/bzip2-${bzip2.version} -I${project.build.directory}/zlib/zlib-${zlib.version}'"/>
+                                            <arg line="EXTRA_LDFLAGS='-L${project.build.directory}/snappy/lib -L${project.build.directory}/lz4/lz4-${lz4.version}/lib -L${project.build.directory}/zstd/zstd-${zstd.version}/lib -L${project.build.directory}/bzip2/bzip2-${bzip2.version} -L${project.build.directory}/zlib/zlib-${zlib.version}'"/>
+                                            <arg line="-j4"/>

Review Comment:
   Is this the fastest the Github worker can run at with [7GB RAM and 2-core](https://docs.github.com/en/actions/using-github-hosted-runners/about-github-hosted-runners#supported-runners-and-hardware-resources) before OOM?
   Does `-j5` or `-j6` yield any build time reduction?



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1129932678


##########
hadoop-hdds/rocks-native/pom.xml:
##########
@@ -0,0 +1,405 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed 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
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+  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. See accompanying LICENSE file.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <parent>
+        <artifactId>hdds</artifactId>
+        <groupId>org.apache.ozone</groupId>
+        <version>1.4.0-SNAPSHOT</version>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+    <name>Apache Ozone HDDS RocksDB Tools</name>
+    <artifactId>hdds-rocks-native</artifactId>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.ozone</groupId>
+            <artifactId>hdds-managed-rocksdb</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.eclipse.jetty</groupId>
+            <artifactId>jetty-io</artifactId>
+        </dependency>
+    </dependencies>
+    <properties>
+        <maven.compiler.source>8</maven.compiler.source>
+        <maven.compiler.target>8</maven.compiler.target>
+    </properties>
+
+    <profiles>
+        <profile>
+            <id>rocks_tools_native</id>
+            <activation>
+                <property>
+                    <name>rocks_tools_native</name>
+                </property>
+            </activation>
+            <properties>
+                <cmake.standards>23</cmake.standards>
+                <sstDump.include>true</sstDump.include>
+            </properties>
+            <build>
+                <plugins>
+                    <plugin>
+                        <groupId>com.googlecode.maven-download-plugin</groupId>
+                        <artifactId>download-maven-plugin</artifactId>
+                        <executions>
+                            <execution>
+                                <id>rocksdb source download</id>
+                                <phase>generate-sources</phase>
+                                <goals>
+                                    <goal>wget</goal>

Review Comment:
   Would the build fail (as expected) when `wget` binary doesn't exist on the host system (e.g. macOS doesn't have `wget` by default) ?



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1129960521


##########
hadoop-hdds/rocks-native/src/main/java/org/apache/hadoop/hdds/utils/db/managed/PipeInputStream.java:
##########
@@ -0,0 +1,99 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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 org.apache.hadoop.hdds.utils.NativeLibraryLoader;
+import org.apache.hadoop.hdds.utils.NativeLibraryNotLoadedException;
+import java.io.InputStream;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.apache.hadoop.hdds.utils.NativeConstants.ROCKS_TOOLS_NATIVE_LIBRARY_NAME;
+
+/**
+ * JNI for reading data from pipe.
+ */
+public class PipeInputStream extends InputStream {
+
+  static {
+    NativeLibraryLoader.getInstance()
+            .loadLibrary(ROCKS_TOOLS_NATIVE_LIBRARY_NAME);
+  }
+  private byte[] byteBuffer;
+  private long nativeHandle;
+  private int numberOfBytesLeftToRead;
+  private int index = 0;
+  private int capacity;
+
+  private AtomicBoolean cleanup;
+
+  PipeInputStream(int capacity) throws NativeLibraryNotLoadedException {
+    if (!NativeLibraryLoader.getInstance()
+            .isLibraryLoaded(ROCKS_TOOLS_NATIVE_LIBRARY_NAME)) {
+      throw new NativeLibraryNotLoadedException(
+              ROCKS_TOOLS_NATIVE_LIBRARY_NAME);
+    }
+
+    this.byteBuffer = new byte[capacity];
+    this.numberOfBytesLeftToRead = 0;
+    this.capacity = capacity;
+    this.nativeHandle = newPipe();
+    this.cleanup = new AtomicBoolean(false);
+  }
+
+  long getNativeHandle() {
+    return nativeHandle;
+  }
+
+  @Override
+  public int read() {
+    if (numberOfBytesLeftToRead < 0) {
+      this.close();
+      return -1;
+    }
+    if (numberOfBytesLeftToRead == 0) {
+      numberOfBytesLeftToRead = readInternal(byteBuffer, capacity,
+              nativeHandle);
+      index = 0;
+      return read();
+    }
+    numberOfBytesLeftToRead--;
+    int ret = byteBuffer[index] & 0xFF;
+    index += 1;
+    return ret;
+  }
+
+  private native long newPipe();
+
+  private native int readInternal(byte[] buff, int numberOfBytes,
+                                  long pipeHandle);

Review Comment:
   nit
   ```suggestion
                                     long pipeHandle);
   
   ```



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] swamirishi commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "swamirishi (via GitHub)" <gi...@apache.org>.
swamirishi commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1129995208


##########
hadoop-hdds/rocks-native/src/main/java/org/apache/hadoop/hdds/utils/db/managed/ManagedSSTDumpTool.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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 org.apache.hadoop.hdds.utils.NativeLibraryLoader;
+import org.apache.hadoop.hdds.utils.NativeLibraryNotLoadedException;
+
+import java.util.Map;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+
+import static org.apache.hadoop.hdds.utils.NativeConstants.ROCKS_TOOLS_NATIVE_LIBRARY_NAME;
+
+/**
+ * JNI for RocksDB SSTDumpTool. Pipes the output to an output stream
+ */
+public class ManagedSSTDumpTool {
+
+  static {
+    NativeLibraryLoader.getInstance()
+            .loadLibrary(ROCKS_TOOLS_NATIVE_LIBRARY_NAME);
+  }
+  private int bufferCapacity;
+  private ExecutorService executorService;
+
+  public ManagedSSTDumpTool(ExecutorService executorService,
+                            int bufferCapacity)
+          throws NativeLibraryNotLoadedException {
+    if (!NativeLibraryLoader.getInstance()
+            .isLibraryLoaded(ROCKS_TOOLS_NATIVE_LIBRARY_NAME)) {
+      throw new NativeLibraryNotLoadedException(
+              ROCKS_TOOLS_NATIVE_LIBRARY_NAME);
+    }
+    this.bufferCapacity = bufferCapacity;
+    this.executorService = executorService;
+  }
+
+  public SSTDumpToolTask run(String[] args, ManagedOptions options)
+          throws NativeLibraryNotLoadedException {
+    PipeInputStream pipeInputStream = new PipeInputStream(bufferCapacity);
+    return new SSTDumpToolTask(this.executorService.submit(() ->
+            this.runInternal(args, options.getNativeHandle(),
+            pipeInputStream.getNativeHandle())), pipeInputStream);

Review Comment:
   This would close pipeInputstream instantly right after submitting to the threadpool



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] swamirishi commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "swamirishi (via GitHub)" <gi...@apache.org>.
swamirishi commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1130006861


##########
hadoop-hdds/rocks-native/src/main/native/Pipe.cpp:
##########
@@ -0,0 +1,37 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.
+ */
+
+#include "Pipe.h"
+#include <unistd.h>
+
+const int Pipe::READ_FILE_DESCRIPTOR_IDX = 0;
+const int Pipe::WRITE_FILE_DESCRIPTOR_IDX = 1;
+
+Pipe::Pipe() {
+    pipe(p);
+    open = true;
+}
+
+Pipe::~Pipe() {
+    ::close(p[0]);
+    ::close(p[1]);
+}
+
+void Pipe::close() {
+    open = false;
+}

Review Comment:
   it is already there.



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1131831571


##########
hadoop-hdds/rocks-native/src/main/java/org/apache/hadoop/hdds/utils/db/managed/ManagedSSTDumpIterator.java:
##########
@@ -0,0 +1,221 @@
+/**
+ * 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 org.apache.hadoop.hdds.utils.NativeLibraryNotLoadedException;
+import org.eclipse.jetty.io.RuntimeIOException;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
+import java.util.Iterator;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * Iterator to Parse output of RocksDBSSTDumpTool.
+ */
+public class ManagedSSTDumpIterator implements
+        Iterator<ManagedSSTDumpIterator.KeyValue>, AutoCloseable {
+  private static final String SST_DUMP_TOOL_CLASS =
+          "org.apache.hadoop.hdds.utils.db.managed.ManagedSSTDumpTool";
+  private static final String PATTERN_REGEX =
+          "'([^=>]+)' seq:([0-9]+), type:([0-9]+) => ";
+
+  public static final int PATTERN_KEY_GROUP_NUMBER = 1;
+  public static final int PATTERN_SEQ_GROUP_NUMBER = 2;
+  public static final int PATTERN_TYPE_GROUP_NUMBER = 3;
+  private static final Pattern PATTERN_MATCHER =
+          Pattern.compile(PATTERN_REGEX);
+  private BufferedReader processOutput;
+  private StringBuilder stdoutString;
+
+  private Matcher currentMatcher;
+  private int prevMatchEndIndex;
+  private KeyValue currentKey;
+  private char[] charBuffer;
+  private KeyValue nextKey;
+
+  private ManagedSSTDumpTool.SSTDumpToolTask sstDumpToolTask;
+  private AtomicBoolean open;
+
+
+  public ManagedSSTDumpIterator(ManagedSSTDumpTool sstDumpTool,
+                                String sstFilePath,
+                                ManagedOptions options) throws IOException,
+          NativeLibraryNotLoadedException {
+    File sstFile = new File(sstFilePath);
+    if (!sstFile.exists() || !sstFile.isFile()) {
+      throw new IOException(String.format("Invalid SST File Path : %s",
+              sstFile.getAbsolutePath()));
+    }
+    init(sstDumpTool, sstFile, options);
+  }
+
+  private void init(ManagedSSTDumpTool sstDumpTool, File sstFile,
+                    ManagedOptions options)
+          throws NativeLibraryNotLoadedException {
+    String[] args = {"--file=" + sstFile.getAbsolutePath(),
+                     "--command=scan"};
+    this.sstDumpToolTask = sstDumpTool.run(args, options);
+    processOutput = new BufferedReader(new InputStreamReader(
+            sstDumpToolTask.getPipedOutput(), StandardCharsets.UTF_8));
+    stdoutString = new StringBuilder();
+    currentMatcher = PATTERN_MATCHER.matcher(stdoutString);
+    charBuffer = new char[8192];
+    open = new AtomicBoolean(true);
+    next();
+  }
+
+  /**
+   * Throws Runtime exception in the case iterator is closed or
+   * the native Dumptool exited with non zero exit value.
+   */
+  private void checkSanityOfProcess() {
+    if (!this.open.get()) {
+      throw new RuntimeException("Iterator has been closed");
+    }
+    if (sstDumpToolTask.getFuture().isDone()
+            && sstDumpToolTask.exitValue() != 0) {
+      throw new RuntimeException("Process Terminated with non zero " +
+              String.format("exit value %d", sstDumpToolTask.exitValue()));
+    }
+  }
+
+  /**
+   * Checks the status of the process & sees if there is another record.
+   * @return True if next exists & false otherwise
+   * Throws Runtime Exception in case of SST File read failure
+   */
+
+  @Override
+  public boolean hasNext() {
+    checkSanityOfProcess();
+    return nextKey != null;
+  }
+
+  /**
+   * Returns the next record from SSTDumpTool.
+   * @return next Key
+   * Throws Runtime Exception incase of failure.
+   */
+  @Override
+  public KeyValue next() {
+    checkSanityOfProcess();
+    currentKey = nextKey;
+    nextKey = null;
+    while (!currentMatcher.find()) {
+      try {
+        if (prevMatchEndIndex != 0) {
+          stdoutString = new StringBuilder(stdoutString.substring(
+                  prevMatchEndIndex, stdoutString.length()));
+          prevMatchEndIndex = 0;
+          currentMatcher = PATTERN_MATCHER.matcher(stdoutString);
+        }
+        int numberOfCharsRead = processOutput.read(charBuffer);
+        if (numberOfCharsRead < 0) {
+          if (currentKey != null) {
+            currentKey.setValue(stdoutString.substring(0,
+                    Math.max(stdoutString.length() - 1, 0)));
+          }
+          return currentKey;
+        }
+        stdoutString.append(charBuffer, 0, numberOfCharsRead);
+        currentMatcher.reset();
+      } catch (IOException e) {
+        throw new RuntimeIOException(e);
+      }
+    }
+    if (currentKey != null) {
+      currentKey.setValue(stdoutString.substring(prevMatchEndIndex,
+              currentMatcher.start() - 1));
+    }
+    prevMatchEndIndex = currentMatcher.end();
+    nextKey =  new KeyValue(

Review Comment:
   nit
   ```suggestion
       nextKey = new KeyValue(
   ```



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1125275742


##########
hadoop-hdds/rocks-native/src/main/java/org/apache/hadoop/hdds/utils/NativeLibraryLoader.java:
##########
@@ -0,0 +1,138 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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;
+
+
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Files;
+import java.nio.file.StandardCopyOption;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Class to load Native Libraries.
+ */
+public class NativeLibraryLoader {
+  private static final String OS = System.getProperty("os.name").toLowerCase();
+  private Map<String, Boolean> librariesLoaded;
+  private static volatile NativeLibraryLoader instance;
+
+  public NativeLibraryLoader(final Map<String, Boolean> librariesLoaded) {
+    this.librariesLoaded = librariesLoaded;
+  }
+
+  private static synchronized void initNewInstance() {
+    if (instance == null) {
+      instance = new NativeLibraryLoader(new ConcurrentHashMap<>());
+    }
+  }
+
+  public static NativeLibraryLoader getInstance() {
+    if (instance == null) {
+      initNewInstance();
+    }
+    return instance;
+  }
+
+  public static String getJniLibraryFileName(String libraryName) {
+    return appendLibOsSuffix("lib" + libraryName);
+  }
+
+  public static boolean isMac() {
+    return OS.contains("mac");
+  }
+
+  public static boolean isWindows() {
+    return OS.contains("win");
+  }
+
+  private static String getLibOsSuffix() {
+    if (isMac()) {
+      return ".dylib";
+    } else if (isWindows()) {
+      return ".dll";
+    }
+    return ".so";

Review Comment:
   Better have explicit check for Linux rather than making it the default.
   
   Reject if OS name is unknown.



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1123871987


##########
.github/workflows/ci.yml:
##########
@@ -61,7 +61,7 @@ jobs:
     needs:
       - build-info
     runs-on: ubuntu-20.04
-    timeout-minutes: 30
+    timeout-minutes: 60

Review Comment:
   For testing CI this is fine. But let's revert it back once the compilation is hidden behind `-Pnative`.



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1123916890


##########
hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/managed/ManagedSSTDumpIterator.java:
##########
@@ -0,0 +1,203 @@
+/**
+ * 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.collect.Lists;
+import org.apache.hadoop.hdds.utils.HddsServerUtil;
+import org.eclipse.jetty.io.RuntimeIOException;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * Iterator to Parse output of RocksDBSSTDumpTool.
+ */
+public class ManagedSSTDumpIterator implements
+        Iterator<ManagedSSTDumpIterator.KeyValue>, AutoCloseable {
+  private Process process;
+  private static final String SST_DUMP_TOOL_CLASS =
+          "org.apache.hadoop.hdds.utils.db.managed.ManagedSSTDumpTool";
+  private static final String PATTERN_REGEX =
+          "'([^=>]+)' seq:([0-9]+), type:([0-9]+) =>";
+
+  public static final int PATTERN_KEY_GROUP_NUMBER = 1;
+  public static final int PATTERN_SEQ_GROUP_NUMBER = 2;
+  public static final int PATTERN_TYPE_GROUP_NUMBER = 3;
+  private static final Pattern PATTERN_MATCHER =
+          Pattern.compile(PATTERN_REGEX);
+  private BufferedReader processOutput;
+  private StringBuilder stdoutString;
+
+  private Matcher currentMatcher;
+  private int prevMatchEndIndex;
+  private KeyValue currentKey;
+  private char[] charBuffer;
+  private KeyValue nextKey;
+
+  private long pollIntervalMillis;
+  private Lock lock;
+
+
+  public ManagedSSTDumpIterator(String sstDumptoolJarPath,
+                                String sstFilePath,
+                                long pollIntervalMillis) throws IOException {
+    File sstFile = new File(sstFilePath);
+    if (!sstFile.exists() || !sstFile.isFile()) {
+      throw new IOException(String.format("Invalid SST File Path : %s",
+              sstFile.getAbsolutePath()));
+    }
+    this.pollIntervalMillis = pollIntervalMillis;
+    this.lock = new ReentrantLock();
+    init(sstFile, sstDumptoolJarPath);
+  }
+
+  private void init(File sstFile, String sstDumptoolJarPath)
+          throws IOException {
+    List<String> args = Lists.newArrayList(
+            "--file=" + sstFile.getAbsolutePath(),
+            "--command=scan");
+    process = HddsServerUtil.getJavaProcess(Collections.emptyList(),
+            sstDumptoolJarPath, SST_DUMP_TOOL_CLASS, args).start();

Review Comment:
   Regarding error handling: What happens here if the java process failed to start for some reason (OOM, file not found, etc.)?
   
   Ideally in this case we should log an error in OM and move on. But SnapDiff won't be able to use read tombstones. Make sure such errors are elegantly handled.



##########
hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/managed/ManagedSSTDumpIterator.java:
##########
@@ -0,0 +1,203 @@
+/**
+ * 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.collect.Lists;
+import org.apache.hadoop.hdds.utils.HddsServerUtil;
+import org.eclipse.jetty.io.RuntimeIOException;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * Iterator to Parse output of RocksDBSSTDumpTool.
+ */
+public class ManagedSSTDumpIterator implements
+        Iterator<ManagedSSTDumpIterator.KeyValue>, AutoCloseable {
+  private Process process;
+  private static final String SST_DUMP_TOOL_CLASS =
+          "org.apache.hadoop.hdds.utils.db.managed.ManagedSSTDumpTool";
+  private static final String PATTERN_REGEX =
+          "'([^=>]+)' seq:([0-9]+), type:([0-9]+) =>";
+
+  public static final int PATTERN_KEY_GROUP_NUMBER = 1;
+  public static final int PATTERN_SEQ_GROUP_NUMBER = 2;
+  public static final int PATTERN_TYPE_GROUP_NUMBER = 3;
+  private static final Pattern PATTERN_MATCHER =
+          Pattern.compile(PATTERN_REGEX);
+  private BufferedReader processOutput;
+  private StringBuilder stdoutString;
+
+  private Matcher currentMatcher;
+  private int prevMatchEndIndex;
+  private KeyValue currentKey;
+  private char[] charBuffer;
+  private KeyValue nextKey;
+
+  private long pollIntervalMillis;
+  private Lock lock;
+
+
+  public ManagedSSTDumpIterator(String sstDumptoolJarPath,
+                                String sstFilePath,
+                                long pollIntervalMillis) throws IOException {
+    File sstFile = new File(sstFilePath);
+    if (!sstFile.exists() || !sstFile.isFile()) {
+      throw new IOException(String.format("Invalid SST File Path : %s",
+              sstFile.getAbsolutePath()));
+    }
+    this.pollIntervalMillis = pollIntervalMillis;
+    this.lock = new ReentrantLock();
+    init(sstFile, sstDumptoolJarPath);
+  }
+
+  private void init(File sstFile, String sstDumptoolJarPath)
+          throws IOException {
+    List<String> args = Lists.newArrayList(
+            "--file=" + sstFile.getAbsolutePath(),
+            "--command=scan");
+    process = HddsServerUtil.getJavaProcess(Collections.emptyList(),
+            sstDumptoolJarPath, SST_DUMP_TOOL_CLASS, args).start();

Review Comment:
   Regarding error handling: What happens here if the java process failed to start for some reason (OOM, file not found, etc.)?
   
   Ideally in this case we should log an error in OM and move on. But SnapDiff won't be able to use read tombstones. Make sure such errors are gracefully handled.



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1123914705


##########
hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/managed/ManagedSSTDumpIterator.java:
##########
@@ -0,0 +1,203 @@
+/**
+ * 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.collect.Lists;
+import org.apache.hadoop.hdds.utils.HddsServerUtil;
+import org.eclipse.jetty.io.RuntimeIOException;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * Iterator to Parse output of RocksDBSSTDumpTool.
+ */
+public class ManagedSSTDumpIterator implements
+        Iterator<ManagedSSTDumpIterator.KeyValue>, AutoCloseable {
+  private Process process;
+  private static final String SST_DUMP_TOOL_CLASS =
+          "org.apache.hadoop.hdds.utils.db.managed.ManagedSSTDumpTool";
+  private static final String PATTERN_REGEX =
+          "'([^=>]+)' seq:([0-9]+), type:([0-9]+) =>";
+
+  public static final int PATTERN_KEY_GROUP_NUMBER = 1;
+  public static final int PATTERN_SEQ_GROUP_NUMBER = 2;
+  public static final int PATTERN_TYPE_GROUP_NUMBER = 3;
+  private static final Pattern PATTERN_MATCHER =
+          Pattern.compile(PATTERN_REGEX);
+  private BufferedReader processOutput;
+  private StringBuilder stdoutString;
+
+  private Matcher currentMatcher;
+  private int prevMatchEndIndex;
+  private KeyValue currentKey;
+  private char[] charBuffer;
+  private KeyValue nextKey;
+
+  private long pollIntervalMillis;
+  private Lock lock;
+
+
+  public ManagedSSTDumpIterator(String sstDumptoolJarPath,
+                                String sstFilePath,
+                                long pollIntervalMillis) throws IOException {
+    File sstFile = new File(sstFilePath);
+    if (!sstFile.exists() || !sstFile.isFile()) {
+      throw new IOException(String.format("Invalid SST File Path : %s",
+              sstFile.getAbsolutePath()));
+    }

Review Comment:
   ```suggestion
       if (!sstFile.exists()) {
         throw new IOException(String.format("SST file doesn't exist: %s",
             sstFile.getAbsolutePath()));
       }
       if (!sstFile.isFile()) {
         throw new IOException(String.format("Not a file: %s",
             sstFile.getAbsolutePath()));
       }
   ```



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1123912025


##########
hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/managed/ManagedSSTDumpIterator.java:
##########
@@ -0,0 +1,203 @@
+/**
+ * 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.collect.Lists;
+import org.apache.hadoop.hdds.utils.HddsServerUtil;
+import org.eclipse.jetty.io.RuntimeIOException;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * Iterator to Parse output of RocksDBSSTDumpTool.
+ */
+public class ManagedSSTDumpIterator implements
+        Iterator<ManagedSSTDumpIterator.KeyValue>, AutoCloseable {
+  private Process process;
+  private static final String SST_DUMP_TOOL_CLASS =
+          "org.apache.hadoop.hdds.utils.db.managed.ManagedSSTDumpTool";
+  private static final String PATTERN_REGEX =
+          "'([^=>]+)' seq:([0-9]+), type:([0-9]+) =>";
+
+  public static final int PATTERN_KEY_GROUP_NUMBER = 1;
+  public static final int PATTERN_SEQ_GROUP_NUMBER = 2;
+  public static final int PATTERN_TYPE_GROUP_NUMBER = 3;
+  private static final Pattern PATTERN_MATCHER =
+          Pattern.compile(PATTERN_REGEX);
+  private BufferedReader processOutput;
+  private StringBuilder stdoutString;
+
+  private Matcher currentMatcher;
+  private int prevMatchEndIndex;
+  private KeyValue currentKey;
+  private char[] charBuffer;
+  private KeyValue nextKey;
+
+  private long pollIntervalMillis;
+  private Lock lock;
+
+

Review Comment:
   nit
   ```suggestion
   ```



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] swamirishi commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "swamirishi (via GitHub)" <gi...@apache.org>.
swamirishi commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1132153666


##########
hadoop-hdds/rocks-native/src/main/java/org/apache/hadoop/hdds/utils/db/managed/ManagedSSTDumpIterator.java:
##########
@@ -0,0 +1,221 @@
+/**
+ * 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 org.apache.hadoop.hdds.utils.NativeLibraryNotLoadedException;
+import org.eclipse.jetty.io.RuntimeIOException;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
+import java.util.Iterator;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * Iterator to Parse output of RocksDBSSTDumpTool.
+ */
+public class ManagedSSTDumpIterator implements
+        Iterator<ManagedSSTDumpIterator.KeyValue>, AutoCloseable {
+  private static final String SST_DUMP_TOOL_CLASS =
+          "org.apache.hadoop.hdds.utils.db.managed.ManagedSSTDumpTool";
+  private static final String PATTERN_REGEX =
+          "'([^=>]+)' seq:([0-9]+), type:([0-9]+) => ";
+
+  public static final int PATTERN_KEY_GROUP_NUMBER = 1;
+  public static final int PATTERN_SEQ_GROUP_NUMBER = 2;
+  public static final int PATTERN_TYPE_GROUP_NUMBER = 3;
+  private static final Pattern PATTERN_MATCHER =
+          Pattern.compile(PATTERN_REGEX);
+  private BufferedReader processOutput;
+  private StringBuilder stdoutString;
+
+  private Matcher currentMatcher;
+  private int prevMatchEndIndex;
+  private KeyValue currentKey;
+  private char[] charBuffer;
+  private KeyValue nextKey;
+
+  private ManagedSSTDumpTool.SSTDumpToolTask sstDumpToolTask;
+  private AtomicBoolean open;
+
+
+  public ManagedSSTDumpIterator(ManagedSSTDumpTool sstDumpTool,
+                                String sstFilePath,
+                                ManagedOptions options) throws IOException,
+          NativeLibraryNotLoadedException {
+    File sstFile = new File(sstFilePath);
+    if (!sstFile.exists() || !sstFile.isFile()) {
+      throw new IOException(String.format("Invalid SST File Path : %s",
+              sstFile.getAbsolutePath()));
+    }
+    init(sstDumpTool, sstFile, options);
+  }
+
+  private void init(ManagedSSTDumpTool sstDumpTool, File sstFile,
+                    ManagedOptions options)
+          throws NativeLibraryNotLoadedException {
+    String[] args = {"--file=" + sstFile.getAbsolutePath(),
+                     "--command=scan"};
+    this.sstDumpToolTask = sstDumpTool.run(args, options);
+    processOutput = new BufferedReader(new InputStreamReader(
+            sstDumpToolTask.getPipedOutput(), StandardCharsets.UTF_8));
+    stdoutString = new StringBuilder();
+    currentMatcher = PATTERN_MATCHER.matcher(stdoutString);
+    charBuffer = new char[8192];
+    open = new AtomicBoolean(true);
+    next();
+  }
+
+  /**
+   * Throws Runtime exception in the case iterator is closed or
+   * the native Dumptool exited with non zero exit value.
+   */
+  private void checkSanityOfProcess() {
+    if (!this.open.get()) {
+      throw new RuntimeException("Iterator has been closed");
+    }
+    if (sstDumpToolTask.getFuture().isDone()
+            && sstDumpToolTask.exitValue() != 0) {
+      throw new RuntimeException("Process Terminated with non zero " +
+              String.format("exit value %d", sstDumpToolTask.exitValue()));
+    }
+  }
+
+  /**
+   * Checks the status of the process & sees if there is another record.
+   * @return True if next exists & false otherwise
+   * Throws Runtime Exception in case of SST File read failure
+   */
+
+  @Override
+  public boolean hasNext() {
+    checkSanityOfProcess();
+    return nextKey != null;
+  }
+
+  /**
+   * Returns the next record from SSTDumpTool.
+   * @return next Key
+   * Throws Runtime Exception incase of failure.
+   */
+  @Override
+  public KeyValue next() {
+    checkSanityOfProcess();
+    currentKey = nextKey;
+    nextKey = null;
+    while (!currentMatcher.find()) {
+      try {
+        if (prevMatchEndIndex != 0) {
+          stdoutString = new StringBuilder(stdoutString.substring(
+                  prevMatchEndIndex, stdoutString.length()));
+          prevMatchEndIndex = 0;
+          currentMatcher = PATTERN_MATCHER.matcher(stdoutString);
+        }
+        int numberOfCharsRead = processOutput.read(charBuffer);
+        if (numberOfCharsRead < 0) {
+          if (currentKey != null) {
+            currentKey.setValue(stdoutString.substring(0,
+                    Math.max(stdoutString.length() - 1, 0)));
+          }
+          return currentKey;
+        }
+        stdoutString.append(charBuffer, 0, numberOfCharsRead);
+        currentMatcher.reset();
+      } catch (IOException e) {
+        throw new RuntimeIOException(e);
+      }
+    }
+    if (currentKey != null) {
+      currentKey.setValue(stdoutString.substring(prevMatchEndIndex,
+              currentMatcher.start() - 1));
+    }
+    prevMatchEndIndex = currentMatcher.end();
+    nextKey =  new KeyValue(

Review Comment:
   done



##########
hadoop-hdds/rocks-native/src/main/native/SSTDumpTool.cpp:
##########
@@ -0,0 +1,49 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.
+ */
+
+#include "org_apache_hadoop_hdds_utils_db_managed_ManagedSSTDumpTool.h"
+#include "rocksdb/options.h"
+#include "rocksdb/sst_dump_tool.h"
+#include <string>
+#include "cplusplus_to_java_convert.h"
+#include "Pipe.h"
+#include <iostream>
+
+jint Java_org_apache_hadoop_hdds_utils_db_managed_ManagedSSTDumpTool_runInternal(JNIEnv *env, jobject obj,
+ jobjectArray argsArray, jlong optionsHandle, jlong pipeHandle) {

Review Comment:
   done



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1131827704


##########
hadoop-hdds/rocks-native/src/main/native/PipeInputStream.cpp:
##########
@@ -0,0 +1,48 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.
+ */
+
+#include <unistd.h>
+#include <iostream>
+#include "Pipe.h"
+#include "cplusplus_to_java_convert.h"
+#include "org_apache_hadoop_hdds_utils_db_managed_PipeInputStream.h"
+
+
+jlong Java_org_apache_hadoop_hdds_utils_db_managed_PipeInputStream_newPipe(JNIEnv *, jobject) {
+    Pipe *pipe = new Pipe();
+    return GET_CPLUSPLUS_POINTER(pipe);
+}
+
+jint Java_org_apache_hadoop_hdds_utils_db_managed_PipeInputStream_readInternal(JNIEnv *env, jobject object, jbyteArray jbyteArray, jint capacity, jlong nativeHandle) {
+    int cap_int = capacity;
+    Pipe *pipe = reinterpret_cast<Pipe *>(nativeHandle);
+    jbyte *b = (env)->GetByteArrayElements(jbyteArray, JNI_FALSE);
+    cap_int = read(pipe->getReadFd(), b, cap_int);
+    if (cap_int == 0) {
+        if (!pipe->isOpen()) {
+            cap_int = -1;
+        }
+    }
+    (env)->ReleaseByteArrayElements(jbyteArray, b, 0);

Review Comment:
   nit: the bracket is unnecessary?
   
   ```suggestion
       env->ReleaseByteArrayElements(jbyteArray, b, 0);
   ```



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] swamirishi commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "swamirishi (via GitHub)" <gi...@apache.org>.
swamirishi commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1132800889


##########
hadoop-hdds/rocks-native/pom.xml:
##########
@@ -0,0 +1,405 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed 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
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+  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. See accompanying LICENSE file.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <parent>
+        <artifactId>hdds</artifactId>
+        <groupId>org.apache.ozone</groupId>
+        <version>1.4.0-SNAPSHOT</version>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+    <name>Apache Ozone HDDS RocksDB Tools</name>
+    <artifactId>hdds-rocks-native</artifactId>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.ozone</groupId>
+            <artifactId>hdds-managed-rocksdb</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.eclipse.jetty</groupId>
+            <artifactId>jetty-io</artifactId>
+        </dependency>
+    </dependencies>
+    <properties>
+        <maven.compiler.source>8</maven.compiler.source>
+        <maven.compiler.target>8</maven.compiler.target>
+    </properties>
+
+    <profiles>
+        <profile>
+            <id>rocks_tools_native</id>
+            <activation>
+                <property>
+                    <name>rocks_tools_native</name>
+                </property>
+            </activation>
+            <properties>
+                <cmake.standards>23</cmake.standards>
+                <sstDump.include>true</sstDump.include>
+            </properties>
+            <build>
+                <plugins>
+                    <plugin>
+                        <groupId>com.googlecode.maven-download-plugin</groupId>
+                        <artifactId>download-maven-plugin</artifactId>
+                        <executions>
+                            <execution>
+                                <id>rocksdb source download</id>
+                                <phase>generate-sources</phase>
+                                <goals>
+                                    <goal>wget</goal>

Review Comment:
   yeah if wget is no there it would fail.



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] swamirishi commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "swamirishi (via GitHub)" <gi...@apache.org>.
swamirishi commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1132814281


##########
hadoop-hdds/rocks-native/src/main/native/cplusplus_to_java_convert.h:
##########
@@ -0,0 +1,50 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.
+ */
+

Review Comment:
   done



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] swamirishi commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "swamirishi (via GitHub)" <gi...@apache.org>.
swamirishi commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1132781493


##########
hadoop-hdds/rocks-native/src/main/native/Pipe.cpp:
##########
@@ -0,0 +1,37 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.
+ */
+
+#include "Pipe.h"
+#include <unistd.h>
+
+const int Pipe::READ_FILE_DESCRIPTOR_IDX = 0;
+const int Pipe::WRITE_FILE_DESCRIPTOR_IDX = 1;
+
+Pipe::Pipe() {
+    pipe(p);
+    open = true;
+}
+
+Pipe::~Pipe() {
+    ::close(p[0]);
+    ::close(p[1]);

Review Comment:
   done



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] swamirishi commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "swamirishi (via GitHub)" <gi...@apache.org>.
swamirishi commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1132819325


##########
hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/managed/ManagedSSTDumpIterator.java:
##########
@@ -0,0 +1,203 @@
+/**
+ * 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.collect.Lists;
+import org.apache.hadoop.hdds.utils.HddsServerUtil;
+import org.eclipse.jetty.io.RuntimeIOException;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * Iterator to Parse output of RocksDBSSTDumpTool.
+ */
+public class ManagedSSTDumpIterator implements
+        Iterator<ManagedSSTDumpIterator.KeyValue>, AutoCloseable {
+  private Process process;
+  private static final String SST_DUMP_TOOL_CLASS =
+          "org.apache.hadoop.hdds.utils.db.managed.ManagedSSTDumpTool";
+  private static final String PATTERN_REGEX =
+          "'([^=>]+)' seq:([0-9]+), type:([0-9]+) =>";
+
+  public static final int PATTERN_KEY_GROUP_NUMBER = 1;
+  public static final int PATTERN_SEQ_GROUP_NUMBER = 2;
+  public static final int PATTERN_TYPE_GROUP_NUMBER = 3;
+  private static final Pattern PATTERN_MATCHER =
+          Pattern.compile(PATTERN_REGEX);
+  private BufferedReader processOutput;
+  private StringBuilder stdoutString;
+
+  private Matcher currentMatcher;
+  private int prevMatchEndIndex;
+  private KeyValue currentKey;
+  private char[] charBuffer;
+  private KeyValue nextKey;
+
+  private long pollIntervalMillis;
+  private Lock lock;
+
+
+  public ManagedSSTDumpIterator(String sstDumptoolJarPath,
+                                String sstFilePath,
+                                long pollIntervalMillis) throws IOException {
+    File sstFile = new File(sstFilePath);
+    if (!sstFile.exists() || !sstFile.isFile()) {
+      throw new IOException(String.format("Invalid SST File Path : %s",
+              sstFile.getAbsolutePath()));
+    }

Review Comment:
   done



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1129936935


##########
hadoop-hdds/rocks-native/pom.xml:
##########
@@ -0,0 +1,405 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed 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
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+  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. See accompanying LICENSE file.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <parent>
+        <artifactId>hdds</artifactId>
+        <groupId>org.apache.ozone</groupId>
+        <version>1.4.0-SNAPSHOT</version>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+    <name>Apache Ozone HDDS RocksDB Tools</name>
+    <artifactId>hdds-rocks-native</artifactId>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.ozone</groupId>
+            <artifactId>hdds-managed-rocksdb</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.eclipse.jetty</groupId>
+            <artifactId>jetty-io</artifactId>
+        </dependency>
+    </dependencies>
+    <properties>
+        <maven.compiler.source>8</maven.compiler.source>
+        <maven.compiler.target>8</maven.compiler.target>
+    </properties>
+
+    <profiles>
+        <profile>
+            <id>rocks_tools_native</id>
+            <activation>
+                <property>
+                    <name>rocks_tools_native</name>
+                </property>
+            </activation>
+            <properties>
+                <cmake.standards>23</cmake.standards>
+                <sstDump.include>true</sstDump.include>
+            </properties>
+            <build>
+                <plugins>
+                    <plugin>
+                        <groupId>com.googlecode.maven-download-plugin</groupId>
+                        <artifactId>download-maven-plugin</artifactId>
+                        <executions>
+                            <execution>
+                                <id>rocksdb source download</id>
+                                <phase>generate-sources</phase>
+                                <goals>
+                                    <goal>wget</goal>
+                                </goals>
+                                <configuration>
+                                    <url>https://github.com/facebook/rocksdb/archive/refs/tags/v${rocksdb.version}.tar.gz</url>
+                                    <outputFileName>rocksdb-v${rocksdb.version}.tar.gz</outputFileName>
+                                    <outputDirectory>${project.build.directory}/rocksdb</outputDirectory>
+                                </configuration>
+                            </execution>
+                            <execution>
+                                <id>zlib source download</id>
+                                <phase>generate-sources</phase>
+                                <goals>
+                                    <goal>wget</goal>
+                                </goals>
+                                <configuration>
+                                    <url>https://zlib.net/zlib-${zlib.version}.tar.gz</url>
+                                    <outputFileName>zlib-${zlib.version}.tar.gz</outputFileName>
+                                    <outputDirectory>${project.build.directory}/zlib</outputDirectory>
+                                </configuration>
+                            </execution>
+                            <execution>
+                                <id>bzip2 source download</id>
+                                <phase>generate-sources</phase>
+                                <goals>
+                                    <goal>wget</goal>
+                                </goals>
+                                <configuration>
+                                    <url>https://sourceware.org/pub/bzip2/bzip2-${bzip2.version}.tar.gz</url>
+                                    <outputFileName>bzip2-v${bzip2.version}.tar.gz</outputFileName>
+                                    <outputDirectory>${project.build.directory}/bzip2</outputDirectory>
+                                </configuration>
+                            </execution>
+                            <execution>
+                                <id>lz4 source download</id>
+                                <phase>generate-sources</phase>
+                                <goals>
+                                    <goal>wget</goal>
+                                </goals>
+                                <configuration>
+                                    <url>https://github.com/lz4/lz4/archive/refs/tags/v${lz4.version}.tar.gz</url>
+                                    <outputFileName>lz4-v${lz4.version}.tar.gz</outputFileName>
+                                    <outputDirectory>${project.build.directory}/lz4</outputDirectory>
+                                </configuration>
+                            </execution>
+                            <execution>
+                                <id>snappy source download</id>
+                                <phase>generate-sources</phase>
+                                <goals>
+                                    <goal>wget</goal>
+                                </goals>
+                                <configuration>
+                                    <url>https://github.com/google/snappy/archive/refs/tags/${snappy.version}.tar.gz</url>
+                                    <outputFileName>snappy-v${snappy.version}.tar.gz</outputFileName>
+                                    <outputDirectory>${project.build.directory}/snappy</outputDirectory>
+                                </configuration>
+                            </execution>
+                            <execution>
+                                <id>zstd source download</id>
+                                <phase>generate-sources</phase>
+                                <goals>
+                                    <goal>wget</goal>
+                                </goals>
+                                <configuration>
+                                    <url>https://github.com/facebook/zstd/archive/refs/tags/v${zstd.version}.tar.gz</url>
+                                    <outputFileName>zstd-v${zstd.version}.tar.gz</outputFileName>
+                                    <outputDirectory>${project.build.directory}/zstd</outputDirectory>
+                                </configuration>
+                            </execution>
+                        </executions>
+                    </plugin>
+                    <plugin>
+                        <groupId>org.apache.maven.plugins</groupId>
+                        <artifactId>maven-patch-plugin</artifactId>
+                        <version>1.1.1</version>
+                        <configuration>
+                            <patchFile>${basedir}/src/main/patches/rocks-native.patch</patchFile>
+                            <strip>1</strip>
+                            <targetDirectory>${project.build.directory}/rocksdb/rocksdb-${rocksdb.version}</targetDirectory>
+                        </configuration>
+                        <executions>
+                            <execution>
+                                <id>patch</id>
+                                <phase>process-sources</phase>
+                                <goals>
+                                    <goal>apply</goal>
+                                </goals>
+                            </execution>
+                        </executions>
+                    </plugin>
+                    <plugin>
+                        <artifactId>maven-antrun-plugin</artifactId>
+                        <executions>
+                            <execution>
+                                <id>unzip-artifact</id>
+                                <phase>generate-sources</phase>
+                                <configuration>
+                                    <tasks>
+                                        <untar src="${project.build.directory}/rocksdb/rocksdb-v${rocksdb.version}.tar.gz" compression="gzip" dest="${project.build.directory}/rocksdb/" />
+                                        <untar src="${project.build.directory}/zlib/zlib-${zlib.version}.tar.gz" compression="gzip" dest="${project.build.directory}/zlib/" />
+                                        <untar src="${project.build.directory}/bzip2/bzip2-v${bzip2.version}.tar.gz" compression="gzip" dest="${project.build.directory}/bzip2/" />
+                                        <untar src="${project.build.directory}/lz4/lz4-v${lz4.version}.tar.gz" compression="gzip" dest="${project.build.directory}/lz4/" />
+                                        <untar src="${project.build.directory}/snappy/snappy-v${snappy.version}.tar.gz" compression="gzip" dest="${project.build.directory}/snappy/" />
+                                        <untar src="${project.build.directory}/zstd/zstd-v${zstd.version}.tar.gz" compression="gzip" dest="${project.build.directory}/zstd/" />
+                                    </tasks>
+                                </configuration>
+                                <goals>
+                                    <goal>run</goal>
+                                </goals>
+                            </execution>
+                            <execution>
+                                <id>build-zlib</id>
+                                <phase>process-sources</phase>
+                                <configuration>
+                                    <tasks>
+                                        <chmod file="${project.build.directory}/zlib/zlib-${zlib.version}/configure" perm="775" />
+                                        <exec executable="./configure" dir="${project.build.directory}/zlib/zlib-${zlib.version}" failonerror="true">
+                                            <arg line="--static"/>
+                                            <env key="CFLAGS" value="-fPIC"/>
+                                        </exec>
+                                        <exec executable="make" dir="${project.build.directory}/zlib/zlib-${zlib.version}" failonerror="true"/>
+                                    </tasks>
+                                </configuration>
+                                <goals>
+                                    <goal>run</goal>
+                                </goals>
+                            </execution>
+                            <execution>
+                                <id>build-bzip2</id>
+                                <phase>process-sources</phase>
+                                <configuration>
+                                    <tasks>
+                                        <exec executable="make" dir="${project.build.directory}/bzip2/bzip2-${bzip2.version}" failonerror="true">
+                                            <arg line="CFLAGS='-fPIC'"/>
+                                        </exec>
+                                    </tasks>
+                                </configuration>
+                                <goals>
+                                    <goal>run</goal>
+                                </goals>
+                            </execution>
+                            <execution>
+                                <id>build-lz4</id>
+                                <phase>process-sources</phase>
+                                <configuration>
+                                    <tasks>
+                                        <exec executable="make" dir="${project.build.directory}/lz4/lz4-${lz4.version}" failonerror="true">
+                                            <arg line="CFLAGS='-fPIC'"/>
+                                        </exec>
+                                    </tasks>
+                                </configuration>
+                                <goals>
+                                    <goal>run</goal>
+                                </goals>
+                            </execution>
+                            <execution>
+                                <id>build-zstd</id>
+                                <phase>process-sources</phase>
+                                <configuration>
+                                    <tasks>
+                                        <exec executable="make" dir="${project.build.directory}/zstd/zstd-${zstd.version}" failonerror="true">
+                                            <arg line="CFLAGS='-fPIC'"/>
+                                        </exec>
+                                    </tasks>
+                                </configuration>
+                                <goals>
+                                    <goal>run</goal>
+                                </goals>
+                            </execution>
+                            <execution>
+                                <id>build-snappy</id>
+                                <phase>process-sources</phase>
+                                <configuration>
+                                    <tasks>
+                                        <mkdir dir="${project.build.directory}/snappy/lib"/>
+                                        <exec executable="cmake" failonerror="true" dir="${project.build.directory}/snappy/lib">
+                                            <arg line="${project.build.directory}/snappy/snappy-${snappy.version}"/>
+                                            <env key="CFLAGS" value="-fPIC"/>
+                                            <env key="CXXFLAGS" value="-fPIC"/>
+                                        </exec>
+                                        <exec executable="make" dir="${project.build.directory}/snappy/lib" failonerror="true"/>
+                                    </tasks>
+                                </configuration>
+                                <goals>
+                                    <goal>run</goal>
+                                </goals>
+                            </execution>
+                            <execution>
+                                <id>build-rocksjava</id>
+                                <phase>process-resources</phase>
+                                <configuration>
+                                    <tasks>
+                                        <exec executable="chmod" failonerror="true">
+                                            <arg line="-R"/>
+                                            <arg line="775"/>
+                                            <arg line="${project.build.directory}/rocksdb/rocksdb-${rocksdb.version}"/>
+                                        </exec>
+                                        <exec executable="make" dir="${project.build.directory}/rocksdb/rocksdb-${rocksdb.version}" failonerror="true">
+                                            <arg line="EXTRA_CXXFLAGS='-fPIC -I${project.build.directory}/snappy/lib -I${project.build.directory}/snappy/snappy-${snappy.version} -I${project.build.directory}/lz4/lz4-${lz4.version}/lib -I${project.build.directory}/zstd/zstd-${zstd.version}/lib -I${project.build.directory}/zstd/zstd-${zstd.version}/lib/dictBuilder -I${project.build.directory}/bzip2/bzip2-${bzip2.version} -I${project.build.directory}/zlib/zlib-${zlib.version}'"/>
+                                            <arg line="EXTRA_LDFLAGS='-L${project.build.directory}/snappy/lib -L${project.build.directory}/lz4/lz4-${lz4.version}/lib -L${project.build.directory}/zstd/zstd-${zstd.version}/lib -L${project.build.directory}/bzip2/bzip2-${bzip2.version} -L${project.build.directory}/zlib/zlib-${zlib.version}'"/>
+                                            <arg line="-j4"/>

Review Comment:
   Is this the maximum the Github worker can run at with 7GB RAM? Does `-j6` fail the build?



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1129945436


##########
hadoop-hdds/rocks-native/src/main/java/org/apache/hadoop/hdds/utils/NativeLibraryLoader.java:
##########
@@ -0,0 +1,138 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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;
+
+
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Files;
+import java.nio.file.StandardCopyOption;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Class to load Native Libraries.
+ */
+public class NativeLibraryLoader {
+  private static final String OS = System.getProperty("os.name").toLowerCase();
+  private Map<String, Boolean> librariesLoaded;
+  private static volatile NativeLibraryLoader instance;
+
+  public NativeLibraryLoader(final Map<String, Boolean> librariesLoaded) {
+    this.librariesLoaded = librariesLoaded;
+  }
+
+  private static synchronized void initNewInstance() {
+    if (instance == null) {
+      instance = new NativeLibraryLoader(new ConcurrentHashMap<>());
+    }
+  }
+
+  public static NativeLibraryLoader getInstance() {
+    if (instance == null) {
+      initNewInstance();
+    }
+    return instance;
+  }
+
+  public static String getJniLibraryFileName(String libraryName) {
+    return appendLibOsSuffix("lib" + libraryName);
+  }
+
+  public static boolean isMac() {
+    return OS.contains("mac");
+  }
+
+  public static boolean isWindows() {
+    return OS.contains("win");
+  }
+
+  private static String getLibOsSuffix() {
+    if (isMac()) {
+      return ".dylib";
+    } else if (isWindows()) {
+      return ".dll";
+    }
+    return ".so";
+  }
+  private static String appendLibOsSuffix(String libraryFileName) {
+    return libraryFileName + getLibOsSuffix();
+  }
+
+  public boolean isLibraryLoaded(final String libraryName) {
+    return librariesLoaded.getOrDefault(libraryName, false);
+  }
+
+  public synchronized boolean loadLibrary(final String libraryName) {
+    if (isLibraryLoaded(libraryName)) {
+      return true;
+    }
+    boolean loaded = false;
+    try {
+      System.loadLibrary(libraryName);
+      loaded = true;
+    } catch (final UnsatisfiedLinkError ule) {
+
+    }
+    if (!loaded) {
+      try {
+        Optional<File> file = copyResourceFromJarToTemp(libraryName);
+        if (file.isPresent()) {
+          System.load(file.get().getAbsolutePath());
+          loaded = true;
+        }
+
+      } catch (IOException e) {
+
+      }
+
+    }
+    this.librariesLoaded.put(libraryName, loaded);
+    return isLibraryLoaded(libraryName);
+  }
+  private Optional<File> copyResourceFromJarToTemp(final String libraryName)
+          throws IOException {
+    final String libraryFileName = getJniLibraryFileName(libraryName);
+    InputStream is = null;
+    try {
+      is = getClass().getClassLoader().getResourceAsStream(libraryFileName);
+      if (is == null) {
+        return Optional.empty();
+      }
+
+      // create a temporary file to copy the library to
+      final File temp = File.createTempFile(libraryName, getLibOsSuffix());
+      if (!temp.exists()) {
+        return Optional.empty();
+      } else {
+        temp.deleteOnExit();
+      }
+
+      Files.copy(is, temp.toPath(), StandardCopyOption.REPLACE_EXISTING);
+      return Optional.ofNullable(temp);
+
+    } finally {
+      if (is != null) {
+        is.close();
+      }
+    }
+  }
+}

Review Comment:
   Have we added this newline?



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1129951764


##########
hadoop-hdds/rocks-native/src/main/java/org/apache/hadoop/hdds/utils/db/managed/ManagedSSTDumpTool.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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 org.apache.hadoop.hdds.utils.NativeLibraryLoader;
+import org.apache.hadoop.hdds.utils.NativeLibraryNotLoadedException;
+
+import java.util.Map;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+
+import static org.apache.hadoop.hdds.utils.NativeConstants.ROCKS_TOOLS_NATIVE_LIBRARY_NAME;
+
+/**
+ * JNI for RocksDB SSTDumpTool. Pipes the output to an output stream
+ */
+public class ManagedSSTDumpTool {
+
+  static {
+    NativeLibraryLoader.getInstance()
+            .loadLibrary(ROCKS_TOOLS_NATIVE_LIBRARY_NAME);
+  }
+  private int bufferCapacity;
+  private ExecutorService executorService;
+
+  public ManagedSSTDumpTool(ExecutorService executorService,
+                            int bufferCapacity)
+          throws NativeLibraryNotLoadedException {
+    if (!NativeLibraryLoader.getInstance()
+            .isLibraryLoaded(ROCKS_TOOLS_NATIVE_LIBRARY_NAME)) {
+      throw new NativeLibraryNotLoadedException(
+              ROCKS_TOOLS_NATIVE_LIBRARY_NAME);
+    }
+    this.bufferCapacity = bufferCapacity;
+    this.executorService = executorService;
+  }
+
+  public SSTDumpToolTask run(String[] args, ManagedOptions options)
+          throws NativeLibraryNotLoadedException {
+    PipeInputStream pipeInputStream = new PipeInputStream(bufferCapacity);
+    return new SSTDumpToolTask(this.executorService.submit(() ->
+            this.runInternal(args, options.getNativeHandle(),
+            pipeInputStream.getNativeHandle())), pipeInputStream);
+  }
+
+  public SSTDumpToolTask run(Map<String, String> args, ManagedOptions options)
+          throws NativeLibraryNotLoadedException {
+    return this.run(args.entrySet().stream().map(e -> "--"

Review Comment:
   What is the `--` prefix for?



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1130001787


##########
hadoop-hdds/rocks-native/src/main/java/org/apache/hadoop/hdds/utils/db/managed/ManagedSSTDumpTool.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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 org.apache.hadoop.hdds.utils.NativeLibraryLoader;
+import org.apache.hadoop.hdds.utils.NativeLibraryNotLoadedException;
+
+import java.util.Map;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+
+import static org.apache.hadoop.hdds.utils.NativeConstants.ROCKS_TOOLS_NATIVE_LIBRARY_NAME;
+
+/**
+ * JNI for RocksDB SSTDumpTool. Pipes the output to an output stream
+ */
+public class ManagedSSTDumpTool {
+
+  static {
+    NativeLibraryLoader.getInstance()
+            .loadLibrary(ROCKS_TOOLS_NATIVE_LIBRARY_NAME);
+  }
+  private int bufferCapacity;
+  private ExecutorService executorService;
+
+  public ManagedSSTDumpTool(ExecutorService executorService,
+                            int bufferCapacity)
+          throws NativeLibraryNotLoadedException {
+    if (!NativeLibraryLoader.getInstance()
+            .isLibraryLoaded(ROCKS_TOOLS_NATIVE_LIBRARY_NAME)) {
+      throw new NativeLibraryNotLoadedException(
+              ROCKS_TOOLS_NATIVE_LIBRARY_NAME);
+    }
+    this.bufferCapacity = bufferCapacity;
+    this.executorService = executorService;
+  }
+
+  public SSTDumpToolTask run(String[] args, ManagedOptions options)
+          throws NativeLibraryNotLoadedException {
+    PipeInputStream pipeInputStream = new PipeInputStream(bufferCapacity);
+    return new SSTDumpToolTask(this.executorService.submit(() ->
+            this.runInternal(args, options.getNativeHandle(),
+            pipeInputStream.getNativeHandle())), pipeInputStream);

Review Comment:
   oh, I see. Let's add a comment here stating that it is intentional `PipeInputStream` would be closed in C++ code then.



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1125262835


##########
hadoop-hdds/rocks-native/src/main/native/SSTDumpTool.cpp:
##########
@@ -0,0 +1,49 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.
+ */
+
+#include "org_apache_hadoop_hdds_utils_db_managed_ManagedSSTDumpTool.h"
+#include "rocksdb/options.h"
+#include "rocksdb/sst_dump_tool.h"
+#include "string"
+#include "cplusplus_to_java_convert.h"
+#include "Pipe.h"
+#include "iostream"
+
+jint Java_org_apache_hadoop_hdds_utils_db_managed_ManagedSSTDumpTool_runInternal(JNIEnv *env, jobject obj, jobjectArray argsArray,
+jlong optionsHandle, jlong pipeHandle) {
+    ROCKSDB_NAMESPACE::SSTDumpTool dumpTool;
+    ROCKSDB_NAMESPACE::Options* options = reinterpret_cast<ROCKSDB_NAMESPACE::Options*>(optionsHandle);
+    Pipe* pipe = reinterpret_cast<Pipe*>(pipeHandle);
+    int length = env->GetArrayLength(argsArray);
+    char* args[length + 1];
+    args[0] = strdup("./sst_dump");
+    for(int i = 0; i < length; i++) {
+        jstring str_val = (jstring)env->GetObjectArrayElement(argsArray, (jsize)i);
+        char* utf_str = (char*)env->GetStringUTFChars(str_val, JNI_FALSE);

Review Comment:
   nit
   ```suggestion
           char *utf_str = (char *)env->GetStringUTFChars(str_val, JNI_FALSE);
   ```



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1125257513


##########
hadoop-hdds/rocks-native/src/main/native/Pipe.h:
##########
@@ -0,0 +1,37 @@
+//
+// Created by Swaminathan Balachandran on 3/2/23.
+//

Review Comment:
   Need Apache License header



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1125277803


##########
hadoop-hdds/rocks-native/src/main/java/org/apache/hadoop/hdds/utils/NativeLibraryLoader.java:
##########
@@ -0,0 +1,138 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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;
+
+
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Files;
+import java.nio.file.StandardCopyOption;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Class to load Native Libraries.
+ */
+public class NativeLibraryLoader {
+  private static final String OS = System.getProperty("os.name").toLowerCase();
+  private Map<String, Boolean> librariesLoaded;
+  private static volatile NativeLibraryLoader instance;
+
+  public NativeLibraryLoader(final Map<String, Boolean> librariesLoaded) {
+    this.librariesLoaded = librariesLoaded;
+  }
+
+  private static synchronized void initNewInstance() {
+    if (instance == null) {
+      instance = new NativeLibraryLoader(new ConcurrentHashMap<>());
+    }
+  }
+
+  public static NativeLibraryLoader getInstance() {
+    if (instance == null) {
+      initNewInstance();
+    }
+    return instance;
+  }
+
+  public static String getJniLibraryFileName(String libraryName) {
+    return appendLibOsSuffix("lib" + libraryName);
+  }
+
+  public static boolean isMac() {
+    return OS.contains("mac");
+  }
+
+  public static boolean isWindows() {
+    return OS.contains("win");
+  }
+
+  private static String getLibOsSuffix() {
+    if (isMac()) {
+      return ".dylib";
+    } else if (isWindows()) {
+      return ".dll";
+    }
+    return ".so";
+  }
+  private static String appendLibOsSuffix(String libraryFileName) {
+    return libraryFileName + getLibOsSuffix();
+  }
+
+  public boolean isLibraryLoaded(final String libraryName) {
+    return librariesLoaded.getOrDefault(libraryName, false);
+  }
+
+  public synchronized boolean loadLibrary(final String libraryName) {
+    if (isLibraryLoaded(libraryName)) {
+      return true;
+    }
+    boolean loaded = false;
+    try {
+      System.loadLibrary(libraryName);
+      loaded = true;
+    } catch (final UnsatisfiedLinkError ule) {
+
+    }
+    if (!loaded) {
+      try {
+        Optional<File> file = copyResourceFromJarToTemp(libraryName);
+        if (file.isPresent()) {
+          System.load(file.get().getAbsolutePath());
+          loaded = true;
+        }
+
+      } catch (IOException e) {
+
+      }
+
+    }
+    this.librariesLoaded.put(libraryName, loaded);
+    return isLibraryLoaded(libraryName);
+  }
+  private Optional<File> copyResourceFromJarToTemp(final String libraryName)

Review Comment:
   ```suggestion
   
     private Optional<File> copyResourceFromJarToTemp(final String libraryName)
   ```



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] kerneltime commented on pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "kerneltime (via GitHub)" <gi...@apache.org>.
kerneltime commented on PR #4315:
URL: https://github.com/apache/ozone/pull/4315#issuecomment-1446729088

   Open questions
   
   - [ ] How can we avoid building the JNI as part of each build? Can we separate out the JNI build as an independent artifact that gets pulled in as a JNI dependency?
   - [ ] Can we build the JNI and update the docker images used via CI/CD to avoid the hit per build


-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1123895618


##########
hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/managed/ManagedRocksObject.java:
##########
@@ -0,0 +1,35 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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 org.rocksdb.RocksObject;
+/**
+ * Managed RocksObject.
+ */
+public abstract class ManagedRocksObject extends RocksObject {

Review Comment:
   This file needs to be moved under `common/` since @sadanand48 's change #4273 is just merged.



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1123924806


##########
hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/managed/ManagedSSTDumpIterator.java:
##########
@@ -0,0 +1,203 @@
+/**
+ * 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.collect.Lists;
+import org.apache.hadoop.hdds.utils.HddsServerUtil;
+import org.eclipse.jetty.io.RuntimeIOException;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * Iterator to Parse output of RocksDBSSTDumpTool.
+ */
+public class ManagedSSTDumpIterator implements
+        Iterator<ManagedSSTDumpIterator.KeyValue>, AutoCloseable {
+  private Process process;
+  private static final String SST_DUMP_TOOL_CLASS =
+          "org.apache.hadoop.hdds.utils.db.managed.ManagedSSTDumpTool";
+  private static final String PATTERN_REGEX =
+          "'([^=>]+)' seq:([0-9]+), type:([0-9]+) =>";
+
+  public static final int PATTERN_KEY_GROUP_NUMBER = 1;
+  public static final int PATTERN_SEQ_GROUP_NUMBER = 2;
+  public static final int PATTERN_TYPE_GROUP_NUMBER = 3;
+  private static final Pattern PATTERN_MATCHER =
+          Pattern.compile(PATTERN_REGEX);
+  private BufferedReader processOutput;
+  private StringBuilder stdoutString;
+
+  private Matcher currentMatcher;
+  private int prevMatchEndIndex;
+  private KeyValue currentKey;
+  private char[] charBuffer;
+  private KeyValue nextKey;
+
+  private long pollIntervalMillis;
+  private Lock lock;
+
+
+  public ManagedSSTDumpIterator(String sstDumptoolJarPath,
+                                String sstFilePath,
+                                long pollIntervalMillis) throws IOException {
+    File sstFile = new File(sstFilePath);
+    if (!sstFile.exists() || !sstFile.isFile()) {
+      throw new IOException(String.format("Invalid SST File Path : %s",
+              sstFile.getAbsolutePath()));
+    }
+    this.pollIntervalMillis = pollIntervalMillis;
+    this.lock = new ReentrantLock();
+    init(sstFile, sstDumptoolJarPath);
+  }
+
+  private void init(File sstFile, String sstDumptoolJarPath)
+          throws IOException {
+    List<String> args = Lists.newArrayList(
+            "--file=" + sstFile.getAbsolutePath(),
+            "--command=scan");
+    process = HddsServerUtil.getJavaProcess(Collections.emptyList(),
+            sstDumptoolJarPath, SST_DUMP_TOOL_CLASS, args).start();
+    processOutput = new BufferedReader(new InputStreamReader(
+            process.getInputStream(), StandardCharsets.UTF_8));
+    stdoutString = new StringBuilder();
+    currentMatcher = PATTERN_MATCHER.matcher(stdoutString);
+    charBuffer = new char[8192];
+    next();
+  }
+
+  private void checkSanityOfProcess() {
+    if (!process.isAlive() && process.exitValue() != 0) {
+      throw new RuntimeException("Process Terminated with non zero " +
+              String.format("exit value %d", process.exitValue()));
+    }
+  }
+
+  @Override
+  public boolean hasNext() {
+    checkSanityOfProcess();
+    return nextKey != null;
+  }
+
+  @Override
+  public KeyValue next() throws RuntimeIOException {

Review Comment:
   We could really use some comments here explaining what each chunk of code is doing.



##########
hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/managed/ManagedSSTDumpIterator.java:
##########
@@ -0,0 +1,203 @@
+/**
+ * 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.collect.Lists;
+import org.apache.hadoop.hdds.utils.HddsServerUtil;
+import org.eclipse.jetty.io.RuntimeIOException;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * Iterator to Parse output of RocksDBSSTDumpTool.
+ */
+public class ManagedSSTDumpIterator implements
+        Iterator<ManagedSSTDumpIterator.KeyValue>, AutoCloseable {
+  private Process process;
+  private static final String SST_DUMP_TOOL_CLASS =
+          "org.apache.hadoop.hdds.utils.db.managed.ManagedSSTDumpTool";
+  private static final String PATTERN_REGEX =
+          "'([^=>]+)' seq:([0-9]+), type:([0-9]+) =>";
+
+  public static final int PATTERN_KEY_GROUP_NUMBER = 1;
+  public static final int PATTERN_SEQ_GROUP_NUMBER = 2;
+  public static final int PATTERN_TYPE_GROUP_NUMBER = 3;
+  private static final Pattern PATTERN_MATCHER =
+          Pattern.compile(PATTERN_REGEX);
+  private BufferedReader processOutput;
+  private StringBuilder stdoutString;
+
+  private Matcher currentMatcher;
+  private int prevMatchEndIndex;
+  private KeyValue currentKey;
+  private char[] charBuffer;
+  private KeyValue nextKey;
+
+  private long pollIntervalMillis;
+  private Lock lock;
+
+
+  public ManagedSSTDumpIterator(String sstDumptoolJarPath,
+                                String sstFilePath,
+                                long pollIntervalMillis) throws IOException {
+    File sstFile = new File(sstFilePath);
+    if (!sstFile.exists() || !sstFile.isFile()) {
+      throw new IOException(String.format("Invalid SST File Path : %s",
+              sstFile.getAbsolutePath()));
+    }
+    this.pollIntervalMillis = pollIntervalMillis;
+    this.lock = new ReentrantLock();
+    init(sstFile, sstDumptoolJarPath);
+  }
+
+  private void init(File sstFile, String sstDumptoolJarPath)
+          throws IOException {
+    List<String> args = Lists.newArrayList(
+            "--file=" + sstFile.getAbsolutePath(),
+            "--command=scan");
+    process = HddsServerUtil.getJavaProcess(Collections.emptyList(),
+            sstDumptoolJarPath, SST_DUMP_TOOL_CLASS, args).start();
+    processOutput = new BufferedReader(new InputStreamReader(
+            process.getInputStream(), StandardCharsets.UTF_8));
+    stdoutString = new StringBuilder();
+    currentMatcher = PATTERN_MATCHER.matcher(stdoutString);
+    charBuffer = new char[8192];
+    next();
+  }
+
+  private void checkSanityOfProcess() {
+    if (!process.isAlive() && process.exitValue() != 0) {
+      throw new RuntimeException("Process Terminated with non zero " +
+              String.format("exit value %d", process.exitValue()));
+    }
+  }
+
+  @Override
+  public boolean hasNext() {
+    checkSanityOfProcess();
+    return nextKey != null;
+  }
+
+  @Override
+  public KeyValue next() throws RuntimeIOException {

Review Comment:
   We could really use some comments in this method here explaining what each chunk of code is doing.



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1123913570


##########
hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/managed/ManagedSSTDumpIterator.java:
##########
@@ -0,0 +1,203 @@
+/**
+ * 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.collect.Lists;
+import org.apache.hadoop.hdds.utils.HddsServerUtil;
+import org.eclipse.jetty.io.RuntimeIOException;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * Iterator to Parse output of RocksDBSSTDumpTool.
+ */
+public class ManagedSSTDumpIterator implements
+        Iterator<ManagedSSTDumpIterator.KeyValue>, AutoCloseable {
+  private Process process;
+  private static final String SST_DUMP_TOOL_CLASS =
+          "org.apache.hadoop.hdds.utils.db.managed.ManagedSSTDumpTool";
+  private static final String PATTERN_REGEX =
+          "'([^=>]+)' seq:([0-9]+), type:([0-9]+) =>";
+
+  public static final int PATTERN_KEY_GROUP_NUMBER = 1;
+  public static final int PATTERN_SEQ_GROUP_NUMBER = 2;
+  public static final int PATTERN_TYPE_GROUP_NUMBER = 3;
+  private static final Pattern PATTERN_MATCHER =
+          Pattern.compile(PATTERN_REGEX);
+  private BufferedReader processOutput;
+  private StringBuilder stdoutString;
+
+  private Matcher currentMatcher;
+  private int prevMatchEndIndex;
+  private KeyValue currentKey;
+  private char[] charBuffer;
+  private KeyValue nextKey;
+
+  private long pollIntervalMillis;
+  private Lock lock;
+
+
+  public ManagedSSTDumpIterator(String sstDumptoolJarPath,
+                                String sstFilePath,
+                                long pollIntervalMillis) throws IOException {
+    File sstFile = new File(sstFilePath);
+    if (!sstFile.exists() || !sstFile.isFile()) {
+      throw new IOException(String.format("Invalid SST File Path : %s",
+              sstFile.getAbsolutePath()));
+    }
+    this.pollIntervalMillis = pollIntervalMillis;
+    this.lock = new ReentrantLock();
+    init(sstFile, sstDumptoolJarPath);
+  }
+
+  private void init(File sstFile, String sstDumptoolJarPath)
+          throws IOException {
+    List<String> args = Lists.newArrayList(
+            "--file=" + sstFile.getAbsolutePath(),
+            "--command=scan");
+    process = HddsServerUtil.getJavaProcess(Collections.emptyList(),
+            sstDumptoolJarPath, SST_DUMP_TOOL_CLASS, args).start();
+    processOutput = new BufferedReader(new InputStreamReader(
+            process.getInputStream(), StandardCharsets.UTF_8));
+    stdoutString = new StringBuilder();
+    currentMatcher = PATTERN_MATCHER.matcher(stdoutString);
+    charBuffer = new char[8192];
+    next();
+  }
+
+  private void checkSanityOfProcess() {
+    if (!process.isAlive() && process.exitValue() != 0) {
+      throw new RuntimeException("Process Terminated with non zero " +
+              String.format("exit value %d", process.exitValue()));
+    }
+  }
+
+  @Override
+  public boolean hasNext() {
+    checkSanityOfProcess();
+    return nextKey != null;
+  }
+
+  @Override
+  public KeyValue next() throws RuntimeIOException {
+    checkSanityOfProcess();
+    try {
+      lock.lock();
+      currentKey = nextKey;
+      nextKey = null;
+      while (!currentMatcher.find()) {
+        try {
+          if (prevMatchEndIndex != 0) {
+            stdoutString = new StringBuilder(stdoutString.substring(
+                    prevMatchEndIndex, stdoutString.length()));
+            prevMatchEndIndex = 0;
+            currentMatcher = PATTERN_MATCHER.matcher(stdoutString);
+          }
+          Thread.sleep(pollIntervalMillis);

Review Comment:
   Is this strictly necessary? Is there a better way to achieve the same purpose?
   
   Generally we do want to avoid `sleep` in the prod code.



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1123910449


##########
hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/MessageQueue.java:
##########
@@ -0,0 +1,188 @@
+/**
+ * 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;
+
+import org.apache.commons.io.input.buffer.CircularByteBuffer;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.concurrent.Callable;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * Message Queue to Pipe output from one output stream to another inputstream.
+ */
+public class MessageQueue {
+  private CircularByteBuffer byteBuffer;
+  private AtomicBoolean isRunning;
+
+  private MessageOutputStream messageOutputStream;
+  private MessageInputStream messageInputStream;
+
+  public MessageQueue(int bufferSize, long pollIntervalMillis) {
+    this.pollIntervalMillis = pollIntervalMillis;
+    init(bufferSize);
+  }
+
+  private void init(int bufferSize) {
+    this.byteBuffer = new CircularByteBuffer(bufferSize);
+    this.isRunning = new AtomicBoolean(false);
+    this.messageInputStream = new MessageInputStream(this);
+    this.messageOutputStream = new MessageOutputStream(this);
+  }
+
+  public void start() {
+    this.isRunning.set(true);
+  }
+
+  public void stop() {
+    this.isRunning.set(false);
+  }
+
+  public MessageOutputStream getMessageOutputStream() {
+    return messageOutputStream;
+  }
+
+  public MessageInputStream getMessageInputStream() {
+    return messageInputStream;
+  }
+
+  private long pollIntervalMillis;
+
+  public boolean isRunning() {
+    return isRunning.get();
+  }
+
+  private long getPollIntervalMillis() {
+    return pollIntervalMillis;
+  }
+
+  private boolean hasSpace(int requiredLength) {
+    return this.byteBuffer.hasSpace(requiredLength);
+  }
+
+  private boolean hasBytes() {
+    return this.byteBuffer.hasBytes();
+  }
+
+  private int getCurrentNumberOfBytes() {
+    return this.byteBuffer.getCurrentNumberOfBytes();
+  }
+
+  private void add(byte[] b, int off, int len) {
+    this.byteBuffer.add(b, off, len);
+  }
+
+  private int read(byte[] b, int off, int len) {
+    this.byteBuffer.read(b, off, len);
+    return len;
+  }
+
+  private static  <T> T callWithLock(Lock lock, Callable<T> callable)
+          throws Exception {
+    lock.lock();
+    try {
+      return callable.call();
+    } finally {
+      lock.unlock();
+    }
+  }
+  private static final class MessageOutputStream extends OutputStream {
+
+    private MessageQueue messageQueue;
+    private Lock writeLock;
+
+    private MessageOutputStream(MessageQueue messageQueue) {
+      this.messageQueue = messageQueue;
+      this.writeLock = new ReentrantLock();
+
+    }
+
+    private void waitForBytes(int requiredLength) throws InterruptedException {
+      while (!this.messageQueue.hasSpace(requiredLength)) {
+        Thread.sleep(this.messageQueue.getPollIntervalMillis());
+      }
+    }
+
+
+

Review Comment:
   ```suggestion
   
   ```



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] swamirishi commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "swamirishi (via GitHub)" <gi...@apache.org>.
swamirishi commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1128826061


##########
hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/HddsServerUtil.java:
##########
@@ -583,4 +584,29 @@ public static IOException toIOException(String msg, RocksDBException e) {
   public static void addSuppressedLoggingExceptions(RPC.Server server) {
     server.addSuppressedLoggingExceptions(ServerNotLeaderException.class);
   }
+
+  /**
+   * Build javaProcessBuilder.
+   * @param jvmArgs
+   * @param classpath
+   * @param className
+   * @param args
+   * @return ProcessBuilder
+   */
+  public static ProcessBuilder getJavaProcess(List<String> jvmArgs,
+                                           String classpath,
+                                           String className,
+                                           List<String> args) {
+    String javaHome = System.getProperty("java.home");
+    String javaBin = String.format("%1$s%2$sbin%2$sjava",
+            javaHome, File.separator);

Review Comment:
   Removed function not needed



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] swamirishi commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "swamirishi (via GitHub)" <gi...@apache.org>.
swamirishi commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1132780391


##########
hadoop-hdds/rocks-native/src/main/native/SSTDumpTool.cpp:
##########
@@ -0,0 +1,49 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.
+ */
+
+#include "org_apache_hadoop_hdds_utils_db_managed_ManagedSSTDumpTool.h"
+#include "rocksdb/options.h"
+#include "rocksdb/sst_dump_tool.h"
+#include <string>
+#include "cplusplus_to_java_convert.h"
+#include "Pipe.h"
+#include <iostream>
+
+jint Java_org_apache_hadoop_hdds_utils_db_managed_ManagedSSTDumpTool_runInternal(JNIEnv *env, jobject obj,
+ jobjectArray argsArray, jlong optionsHandle, jlong pipeHandle) {
+    ROCKSDB_NAMESPACE::SSTDumpTool dumpTool;
+    ROCKSDB_NAMESPACE::Options *options = reinterpret_cast<ROCKSDB_NAMESPACE::Options *>(optionsHandle);
+    Pipe *pipe = reinterpret_cast<Pipe *>(pipeHandle);
+    int length = env->GetArrayLength(argsArray);
+    char *args[length + 1];
+    args[0] = strdup("./sst_dump");
+    for (int i = 0; i < length; i++) {
+        jstring str_val = (jstring)env->GetObjectArrayElement(argsArray, (jsize)i);
+        char *utf_str = (char *)env->GetStringUTFChars(str_val, JNI_FALSE);
+        args[i + 1] = strdup(utf_str);
+        env->ReleaseStringUTFChars(str_val, utf_str);
+    }
+    FILE *wr = fdopen(pipe->getWriteFd(), "w");
+    int ret = dumpTool.Run(length + 1, args, *options, wr);
+    for (int i = 0; i < length + 1; i++) {
+        free(args[i]);
+    }
+    fclose(wr);
+    pipe->close();

Review Comment:
   If GC is triggered on PipeInputStream then we close the pipe, it basically means that we are not using the inputstream right & thus wouldn't be needing the pipe.



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1129947698


##########
hadoop-hdds/rocks-native/src/main/java/org/apache/hadoop/hdds/utils/db/managed/ManagedSSTDumpIterator.java:
##########
@@ -0,0 +1,239 @@
+/**
+ * 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 org.apache.hadoop.hdds.utils.NativeLibraryNotLoadedException;
+import org.eclipse.jetty.io.RuntimeIOException;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
+import java.util.Iterator;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * Iterator to Parse output of RocksDBSSTDumpTool.
+ */
+public class ManagedSSTDumpIterator implements
+        Iterator<ManagedSSTDumpIterator.KeyValue>, AutoCloseable {
+  private static final String SST_DUMP_TOOL_CLASS =
+          "org.apache.hadoop.hdds.utils.db.managed.ManagedSSTDumpTool";
+  private static final String PATTERN_REGEX =
+          "'([^=>]+)' seq:([0-9]+), type:([0-9]+) => ";
+
+  public static final int PATTERN_KEY_GROUP_NUMBER = 1;
+  public static final int PATTERN_SEQ_GROUP_NUMBER = 2;
+  public static final int PATTERN_TYPE_GROUP_NUMBER = 3;
+  private static final Pattern PATTERN_MATCHER =
+          Pattern.compile(PATTERN_REGEX);
+  private BufferedReader processOutput;
+  private StringBuilder stdoutString;
+
+  private Matcher currentMatcher;
+  private int prevMatchEndIndex;
+  private KeyValue currentKey;
+  private char[] charBuffer;
+  private KeyValue nextKey;
+
+  private long pollIntervalMillis;
+  private ManagedSSTDumpTool.SSTDumpToolTask sstDumpToolTask;
+  private Lock lock;
+  private AtomicBoolean open;
+
+
+  public ManagedSSTDumpIterator(ManagedSSTDumpTool sstDumpTool,
+                                String sstFilePath,
+                                ManagedOptions options,
+                                long pollIntervalMillis) throws IOException,
+          NativeLibraryNotLoadedException {
+    File sstFile = new File(sstFilePath);
+    if (!sstFile.exists() || !sstFile.isFile()) {
+      throw new IOException(String.format("Invalid SST File Path : %s",
+              sstFile.getAbsolutePath()));
+    }
+    this.pollIntervalMillis = pollIntervalMillis;
+    this.lock = new ReentrantLock();
+    init(sstDumpTool, sstFile, options);
+  }
+
+  private void init(ManagedSSTDumpTool sstDumpTool, File sstFile,
+                    ManagedOptions options)
+          throws NativeLibraryNotLoadedException {
+    String[] args = {"--file=" + sstFile.getAbsolutePath(),
+                     "--command=scan"};
+    this.sstDumpToolTask = sstDumpTool.run(args, options);
+    processOutput = new BufferedReader(new InputStreamReader(
+            sstDumpToolTask.getPipedOutput(), StandardCharsets.UTF_8));
+    stdoutString = new StringBuilder();
+    currentMatcher = PATTERN_MATCHER.matcher(stdoutString);
+    charBuffer = new char[8192];
+    open = new AtomicBoolean(true);
+    next();
+  }
+
+  /**
+   * Throws Runtime exception in the case iterator is closed or
+   * the native Dumptool exited with non zero exit value.
+   */
+  private void checkSanityOfProcess() {
+    if (!this.open.get()) {
+      throw new RuntimeException("Iterator has been closed");
+    }
+    if (sstDumpToolTask.getFuture().isDone()
+            && sstDumpToolTask.exitValue() != 0) {
+      throw new RuntimeException("Process Terminated with non zero " +
+              String.format("exit value %d", sstDumpToolTask.exitValue()));
+    }
+  }
+
+  /**
+   *
+   * @return
+   * Throws Runtime Exception in case of SST File read failure
+   */
+
+  @Override
+  public boolean hasNext() {
+    checkSanityOfProcess();
+    return nextKey != null;
+  }
+
+  /**
+   *
+   * @return next Key
+   * Throws Runtime Exception incase of failure.
+   */
+  @Override
+  public KeyValue next() {
+    lock.lock();
+    try {
+      checkSanityOfProcess();
+      currentKey = nextKey;
+      nextKey = null;
+      while (!currentMatcher.find()) {
+        try {
+          if (prevMatchEndIndex != 0) {
+            stdoutString = new StringBuilder(stdoutString.substring(
+                    prevMatchEndIndex, stdoutString.length()));
+            prevMatchEndIndex = 0;
+            currentMatcher = PATTERN_MATCHER.matcher(stdoutString);
+          }
+          Thread.sleep(pollIntervalMillis);

Review Comment:
   Same question as https://github.com/apache/ozone/pull/4315#discussion_r1123913570 unanswered



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] swamirishi commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "swamirishi (via GitHub)" <gi...@apache.org>.
swamirishi commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1130005603


##########
hadoop-hdds/rocks-native/src/main/java/org/apache/hadoop/hdds/utils/db/managed/ManagedSSTDumpIterator.java:
##########
@@ -0,0 +1,238 @@
+/**
+ * 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 org.apache.hadoop.hdds.utils.NativeLibraryNotLoadedException;
+import org.eclipse.jetty.io.RuntimeIOException;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
+import java.util.Iterator;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * Iterator to Parse output of RocksDBSSTDumpTool.
+ */
+public class ManagedSSTDumpIterator implements
+        Iterator<ManagedSSTDumpIterator.KeyValue>, AutoCloseable {
+  private static final String SST_DUMP_TOOL_CLASS =
+          "org.apache.hadoop.hdds.utils.db.managed.ManagedSSTDumpTool";
+  private static final String PATTERN_REGEX =
+          "'([^=>]+)' seq:([0-9]+), type:([0-9]+) =>";
+
+  public static final int PATTERN_KEY_GROUP_NUMBER = 1;
+  public static final int PATTERN_SEQ_GROUP_NUMBER = 2;
+  public static final int PATTERN_TYPE_GROUP_NUMBER = 3;
+  private static final Pattern PATTERN_MATCHER =
+          Pattern.compile(PATTERN_REGEX);
+  private BufferedReader processOutput;
+  private StringBuilder stdoutString;
+
+  private Matcher currentMatcher;
+  private int prevMatchEndIndex;
+  private KeyValue currentKey;
+  private char[] charBuffer;
+  private KeyValue nextKey;
+
+  private long pollIntervalMillis;
+  private ManagedSSTDumpTool.SSTDumpToolTask sstDumpToolTask;
+  private Lock lock;
+  private AtomicBoolean open;
+
+
+  public ManagedSSTDumpIterator(ManagedSSTDumpTool sstDumpTool,
+                                String sstFilePath,
+                                ManagedOptions options,
+                                long pollIntervalMillis) throws IOException,
+          NativeLibraryNotLoadedException {
+    File sstFile = new File(sstFilePath);
+    if (!sstFile.exists() || !sstFile.isFile()) {
+      throw new IOException(String.format("Invalid SST File Path : %s",
+              sstFile.getAbsolutePath()));
+    }
+    this.pollIntervalMillis = pollIntervalMillis;
+    this.lock = new ReentrantLock();
+    init(sstDumpTool, sstFile, options);
+  }
+
+  private void init(ManagedSSTDumpTool sstDumpTool, File sstFile,
+                    ManagedOptions options)
+          throws NativeLibraryNotLoadedException {
+    String[] args = {"--file=" + sstFile.getAbsolutePath(),
+                     "--command=scan"};
+    this.sstDumpToolTask = sstDumpTool.run(args, options);
+    processOutput = new BufferedReader(new InputStreamReader(
+            sstDumpToolTask.getPipedOutput(), StandardCharsets.UTF_8));
+    stdoutString = new StringBuilder();
+    currentMatcher = PATTERN_MATCHER.matcher(stdoutString);
+    charBuffer = new char[8192];
+    open = new AtomicBoolean(true);
+    next();
+  }
+
+  /**
+   * Throws Runtime exception in the case iterator is closed or
+   * the native Dumptool exited with non zero exit value.
+   */
+  private void checkSanityOfProcess() {
+    if (!this.open.get()) {
+      throw new RuntimeException("Iterator has been closed");
+    }
+    if (sstDumpToolTask.getFuture().isDone()
+            && sstDumpToolTask.exitValue() != 0) {
+      throw new RuntimeException("Process Terminated with non zero " +
+              String.format("exit value %d", sstDumpToolTask.exitValue()));
+    }
+  }
+
+  /**
+   *
+   * @return
+   * Throws Runtime Exception in case of SST File read failure
+   */
+
+  @Override
+  public boolean hasNext() {
+    checkSanityOfProcess();
+    return nextKey != null;
+  }
+
+  /**
+   *
+   * @return next Key
+   * Throws Runtime Exception incase of failure.
+   */
+  @Override
+  public KeyValue next() {
+    lock.lock();
+    try {
+      checkSanityOfProcess();
+      currentKey = nextKey;
+      nextKey = null;
+      while (!currentMatcher.find()) {
+        try {
+          if (prevMatchEndIndex != 0) {
+            stdoutString = new StringBuilder(stdoutString.substring(
+                    prevMatchEndIndex, stdoutString.length()));
+            prevMatchEndIndex = 0;
+            currentMatcher = PATTERN_MATCHER.matcher(stdoutString);
+          }
+          Thread.sleep(pollIntervalMillis);
+          int numberOfCharsRead = processOutput.read(charBuffer);
+          if (numberOfCharsRead < 0) {
+            if (currentKey != null) {
+              currentKey.setValue(stdoutString.toString());
+            }
+            return currentKey;
+          }
+          stdoutString.append(charBuffer, 0, numberOfCharsRead);
+          currentMatcher.reset();
+        } catch (IOException | InterruptedException e) {
+          throw new RuntimeIOException(e);
+        }
+      }
+      if (currentKey != null) {
+        currentKey.setValue(stdoutString.substring(prevMatchEndIndex,
+                currentMatcher.start()));
+      }
+      prevMatchEndIndex = currentMatcher.end();
+      nextKey =  new KeyValue(
+              currentMatcher.group(PATTERN_KEY_GROUP_NUMBER),
+              currentMatcher.group(PATTERN_SEQ_GROUP_NUMBER),
+              currentMatcher.group(PATTERN_TYPE_GROUP_NUMBER));
+      return currentKey;
+    } finally {
+      lock.unlock();
+    }
+  }
+
+  @Override
+  public synchronized void close() throws Exception {
+    lock.lock();
+    try {
+      if (this.sstDumpToolTask != null) {
+        if (!this.sstDumpToolTask.getFuture().isDone()) {
+          this.sstDumpToolTask.getFuture().cancel(true);
+        }
+        this.processOutput.close();
+      }
+      open.compareAndSet(true, false);
+    } finally {
+      lock.unlock();
+    }
+  }
+
+  @Override
+  protected void finalize() throws Throwable {
+    this.close();
+  }
+
+  /**
+   * Class containing Parsed KeyValue Record from Sst Dumptool output.
+   */
+  public static final class KeyValue {

Review Comment:
   yes, since this would be used in the snapdiff code.



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] swamirishi commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "swamirishi (via GitHub)" <gi...@apache.org>.
swamirishi commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1130005942


##########
hadoop-hdds/rocks-native/src/main/java/org/apache/hadoop/hdds/utils/db/managed/ManagedSSTDumpIterator.java:
##########
@@ -0,0 +1,238 @@
+/**
+ * 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 org.apache.hadoop.hdds.utils.NativeLibraryNotLoadedException;
+import org.eclipse.jetty.io.RuntimeIOException;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
+import java.util.Iterator;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * Iterator to Parse output of RocksDBSSTDumpTool.
+ */
+public class ManagedSSTDumpIterator implements
+        Iterator<ManagedSSTDumpIterator.KeyValue>, AutoCloseable {
+  private static final String SST_DUMP_TOOL_CLASS =
+          "org.apache.hadoop.hdds.utils.db.managed.ManagedSSTDumpTool";
+  private static final String PATTERN_REGEX =
+          "'([^=>]+)' seq:([0-9]+), type:([0-9]+) =>";
+
+  public static final int PATTERN_KEY_GROUP_NUMBER = 1;
+  public static final int PATTERN_SEQ_GROUP_NUMBER = 2;
+  public static final int PATTERN_TYPE_GROUP_NUMBER = 3;
+  private static final Pattern PATTERN_MATCHER =
+          Pattern.compile(PATTERN_REGEX);
+  private BufferedReader processOutput;
+  private StringBuilder stdoutString;
+
+  private Matcher currentMatcher;
+  private int prevMatchEndIndex;
+  private KeyValue currentKey;
+  private char[] charBuffer;
+  private KeyValue nextKey;
+
+  private long pollIntervalMillis;
+  private ManagedSSTDumpTool.SSTDumpToolTask sstDumpToolTask;
+  private Lock lock;
+  private AtomicBoolean open;
+
+
+  public ManagedSSTDumpIterator(ManagedSSTDumpTool sstDumpTool,
+                                String sstFilePath,
+                                ManagedOptions options,
+                                long pollIntervalMillis) throws IOException,
+          NativeLibraryNotLoadedException {
+    File sstFile = new File(sstFilePath);
+    if (!sstFile.exists() || !sstFile.isFile()) {
+      throw new IOException(String.format("Invalid SST File Path : %s",
+              sstFile.getAbsolutePath()));
+    }
+    this.pollIntervalMillis = pollIntervalMillis;
+    this.lock = new ReentrantLock();
+    init(sstDumpTool, sstFile, options);
+  }
+
+  private void init(ManagedSSTDumpTool sstDumpTool, File sstFile,
+                    ManagedOptions options)
+          throws NativeLibraryNotLoadedException {
+    String[] args = {"--file=" + sstFile.getAbsolutePath(),
+                     "--command=scan"};
+    this.sstDumpToolTask = sstDumpTool.run(args, options);
+    processOutput = new BufferedReader(new InputStreamReader(
+            sstDumpToolTask.getPipedOutput(), StandardCharsets.UTF_8));
+    stdoutString = new StringBuilder();
+    currentMatcher = PATTERN_MATCHER.matcher(stdoutString);
+    charBuffer = new char[8192];
+    open = new AtomicBoolean(true);
+    next();
+  }
+
+  /**
+   * Throws Runtime exception in the case iterator is closed or
+   * the native Dumptool exited with non zero exit value.
+   */
+  private void checkSanityOfProcess() {
+    if (!this.open.get()) {
+      throw new RuntimeException("Iterator has been closed");
+    }
+    if (sstDumpToolTask.getFuture().isDone()
+            && sstDumpToolTask.exitValue() != 0) {
+      throw new RuntimeException("Process Terminated with non zero " +
+              String.format("exit value %d", sstDumpToolTask.exitValue()));
+    }
+  }
+
+  /**
+   *
+   * @return
+   * Throws Runtime Exception in case of SST File read failure
+   */
+
+  @Override
+  public boolean hasNext() {
+    checkSanityOfProcess();
+    return nextKey != null;
+  }
+
+  /**
+   *
+   * @return next Key
+   * Throws Runtime Exception incase of failure.
+   */
+  @Override
+  public KeyValue next() {
+    lock.lock();
+    try {
+      checkSanityOfProcess();
+      currentKey = nextKey;
+      nextKey = null;
+      while (!currentMatcher.find()) {
+        try {
+          if (prevMatchEndIndex != 0) {
+            stdoutString = new StringBuilder(stdoutString.substring(
+                    prevMatchEndIndex, stdoutString.length()));
+            prevMatchEndIndex = 0;
+            currentMatcher = PATTERN_MATCHER.matcher(stdoutString);
+          }
+          Thread.sleep(pollIntervalMillis);
+          int numberOfCharsRead = processOutput.read(charBuffer);
+          if (numberOfCharsRead < 0) {
+            if (currentKey != null) {
+              currentKey.setValue(stdoutString.toString());
+            }
+            return currentKey;
+          }
+          stdoutString.append(charBuffer, 0, numberOfCharsRead);
+          currentMatcher.reset();
+        } catch (IOException | InterruptedException e) {
+          throw new RuntimeIOException(e);
+        }
+      }
+      if (currentKey != null) {
+        currentKey.setValue(stdoutString.substring(prevMatchEndIndex,
+                currentMatcher.start()));
+      }
+      prevMatchEndIndex = currentMatcher.end();
+      nextKey =  new KeyValue(
+              currentMatcher.group(PATTERN_KEY_GROUP_NUMBER),
+              currentMatcher.group(PATTERN_SEQ_GROUP_NUMBER),
+              currentMatcher.group(PATTERN_TYPE_GROUP_NUMBER));
+      return currentKey;
+    } finally {
+      lock.unlock();
+    }
+  }
+
+  @Override
+  public synchronized void close() throws Exception {
+    lock.lock();
+    try {
+      if (this.sstDumpToolTask != null) {
+        if (!this.sstDumpToolTask.getFuture().isDone()) {
+          this.sstDumpToolTask.getFuture().cancel(true);
+        }
+        this.processOutput.close();
+      }
+      open.compareAndSet(true, false);
+    } finally {
+      lock.unlock();
+    }
+  }
+
+  @Override
+  protected void finalize() throws Throwable {
+    this.close();
+  }
+
+  /**
+   * Class containing Parsed KeyValue Record from Sst Dumptool output.
+   */
+  public static final class KeyValue {

Review Comment:
   Should I move this to another class file?



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on PR #4315:
URL: https://github.com/apache/ozone/pull/4315#issuecomment-1463065504

   lgtm apart from a few minor comments inline.


-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1125254386


##########
hadoop-hdds/rocks-native/src/main/java/org/apache/hadoop/hdds/utils/db/managed/ManagedSSTDumpIterator.java:
##########
@@ -0,0 +1,238 @@
+/**
+ * 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 org.apache.hadoop.hdds.utils.NativeLibraryNotLoadedException;
+import org.eclipse.jetty.io.RuntimeIOException;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
+import java.util.Iterator;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * Iterator to Parse output of RocksDBSSTDumpTool.
+ */
+public class ManagedSSTDumpIterator implements
+        Iterator<ManagedSSTDumpIterator.KeyValue>, AutoCloseable {
+  private static final String SST_DUMP_TOOL_CLASS =
+          "org.apache.hadoop.hdds.utils.db.managed.ManagedSSTDumpTool";
+  private static final String PATTERN_REGEX =
+          "'([^=>]+)' seq:([0-9]+), type:([0-9]+) =>";
+
+  public static final int PATTERN_KEY_GROUP_NUMBER = 1;
+  public static final int PATTERN_SEQ_GROUP_NUMBER = 2;
+  public static final int PATTERN_TYPE_GROUP_NUMBER = 3;
+  private static final Pattern PATTERN_MATCHER =
+          Pattern.compile(PATTERN_REGEX);
+  private BufferedReader processOutput;
+  private StringBuilder stdoutString;
+
+  private Matcher currentMatcher;
+  private int prevMatchEndIndex;
+  private KeyValue currentKey;
+  private char[] charBuffer;
+  private KeyValue nextKey;
+
+  private long pollIntervalMillis;
+  private ManagedSSTDumpTool.SSTDumpToolTask sstDumpToolTask;
+  private Lock lock;
+  private AtomicBoolean open;
+
+
+  public ManagedSSTDumpIterator(ManagedSSTDumpTool sstDumpTool,
+                                String sstFilePath,
+                                ManagedOptions options,
+                                long pollIntervalMillis) throws IOException,
+          NativeLibraryNotLoadedException {
+    File sstFile = new File(sstFilePath);
+    if (!sstFile.exists() || !sstFile.isFile()) {
+      throw new IOException(String.format("Invalid SST File Path : %s",
+              sstFile.getAbsolutePath()));
+    }
+    this.pollIntervalMillis = pollIntervalMillis;
+    this.lock = new ReentrantLock();
+    init(sstDumpTool, sstFile, options);
+  }
+
+  private void init(ManagedSSTDumpTool sstDumpTool, File sstFile,
+                    ManagedOptions options)
+          throws NativeLibraryNotLoadedException {
+    String[] args = {"--file=" + sstFile.getAbsolutePath(),
+                     "--command=scan"};
+    this.sstDumpToolTask = sstDumpTool.run(args, options);
+    processOutput = new BufferedReader(new InputStreamReader(
+            sstDumpToolTask.getPipedOutput(), StandardCharsets.UTF_8));
+    stdoutString = new StringBuilder();
+    currentMatcher = PATTERN_MATCHER.matcher(stdoutString);
+    charBuffer = new char[8192];
+    open = new AtomicBoolean(true);
+    next();
+  }
+
+  /**
+   * Throws Runtime exception in the case iterator is closed or
+   * the native Dumptool exited with non zero exit value.
+   */
+  private void checkSanityOfProcess() {
+    if (!this.open.get()) {
+      throw new RuntimeException("Iterator has been closed");
+    }
+    if (sstDumpToolTask.getFuture().isDone()
+            && sstDumpToolTask.exitValue() != 0) {
+      throw new RuntimeException("Process Terminated with non zero " +
+              String.format("exit value %d", sstDumpToolTask.exitValue()));
+    }
+  }
+
+  /**
+   *
+   * @return
+   * Throws Runtime Exception in case of SST File read failure
+   */
+
+  @Override
+  public boolean hasNext() {
+    checkSanityOfProcess();
+    return nextKey != null;
+  }
+
+  /**
+   *
+   * @return next Key
+   * Throws Runtime Exception incase of failure.
+   */
+  @Override
+  public KeyValue next() {
+    lock.lock();
+    try {
+      checkSanityOfProcess();
+      currentKey = nextKey;
+      nextKey = null;
+      while (!currentMatcher.find()) {
+        try {
+          if (prevMatchEndIndex != 0) {
+            stdoutString = new StringBuilder(stdoutString.substring(
+                    prevMatchEndIndex, stdoutString.length()));
+            prevMatchEndIndex = 0;
+            currentMatcher = PATTERN_MATCHER.matcher(stdoutString);
+          }
+          Thread.sleep(pollIntervalMillis);
+          int numberOfCharsRead = processOutput.read(charBuffer);
+          if (numberOfCharsRead < 0) {
+            if (currentKey != null) {
+              currentKey.setValue(stdoutString.toString());
+            }
+            return currentKey;
+          }
+          stdoutString.append(charBuffer, 0, numberOfCharsRead);
+          currentMatcher.reset();
+        } catch (IOException | InterruptedException e) {
+          throw new RuntimeIOException(e);

Review Comment:
   Consider if just throwing a checked exception is better



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1125273410


##########
hadoop-hdds/rocks-native/src/main/java/org/apache/hadoop/hdds/utils/NativeLibraryLoader.java:
##########
@@ -0,0 +1,138 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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;
+
+
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Files;
+import java.nio.file.StandardCopyOption;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Class to load Native Libraries.
+ */
+public class NativeLibraryLoader {
+  private static final String OS = System.getProperty("os.name").toLowerCase();
+  private Map<String, Boolean> librariesLoaded;
+  private static volatile NativeLibraryLoader instance;
+
+  public NativeLibraryLoader(final Map<String, Boolean> librariesLoaded) {
+    this.librariesLoaded = librariesLoaded;
+  }
+
+  private static synchronized void initNewInstance() {
+    if (instance == null) {
+      instance = new NativeLibraryLoader(new ConcurrentHashMap<>());
+    }
+  }
+
+  public static NativeLibraryLoader getInstance() {
+    if (instance == null) {
+      initNewInstance();
+    }
+    return instance;
+  }
+
+  public static String getJniLibraryFileName(String libraryName) {
+    return appendLibOsSuffix("lib" + libraryName);
+  }
+
+  public static boolean isMac() {
+    return OS.contains("mac");

Review Comment:
   ```suggestion
       return OS. startsWith("mac");
   ```



##########
hadoop-hdds/rocks-native/src/main/java/org/apache/hadoop/hdds/utils/NativeLibraryLoader.java:
##########
@@ -0,0 +1,138 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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;
+
+
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Files;
+import java.nio.file.StandardCopyOption;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Class to load Native Libraries.
+ */
+public class NativeLibraryLoader {
+  private static final String OS = System.getProperty("os.name").toLowerCase();
+  private Map<String, Boolean> librariesLoaded;
+  private static volatile NativeLibraryLoader instance;
+
+  public NativeLibraryLoader(final Map<String, Boolean> librariesLoaded) {
+    this.librariesLoaded = librariesLoaded;
+  }
+
+  private static synchronized void initNewInstance() {
+    if (instance == null) {
+      instance = new NativeLibraryLoader(new ConcurrentHashMap<>());
+    }
+  }
+
+  public static NativeLibraryLoader getInstance() {
+    if (instance == null) {
+      initNewInstance();
+    }
+    return instance;
+  }
+
+  public static String getJniLibraryFileName(String libraryName) {
+    return appendLibOsSuffix("lib" + libraryName);
+  }
+
+  public static boolean isMac() {
+    return OS.contains("mac");

Review Comment:
   ```suggestion
       return OS.startsWith("mac");
   ```



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1125267281


##########
hadoop-hdds/rocks-native/src/main/native/SSTDumpTool.cpp:
##########
@@ -0,0 +1,49 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.
+ */
+
+#include "org_apache_hadoop_hdds_utils_db_managed_ManagedSSTDumpTool.h"
+#include "rocksdb/options.h"
+#include "rocksdb/sst_dump_tool.h"
+#include "string"
+#include "cplusplus_to_java_convert.h"
+#include "Pipe.h"
+#include "iostream"
+
+jint Java_org_apache_hadoop_hdds_utils_db_managed_ManagedSSTDumpTool_runInternal(JNIEnv *env, jobject obj, jobjectArray argsArray,
+jlong optionsHandle, jlong pipeHandle) {
+    ROCKSDB_NAMESPACE::SSTDumpTool dumpTool;
+    ROCKSDB_NAMESPACE::Options* options = reinterpret_cast<ROCKSDB_NAMESPACE::Options*>(optionsHandle);

Review Comment:
   ```suggestion
       ROCKSDB_NAMESPACE::Options *options = reinterpret_cast<ROCKSDB_NAMESPACE::Options*>(optionsHandle);
   ```



##########
hadoop-hdds/rocks-native/src/main/native/SSTDumpTool.cpp:
##########
@@ -0,0 +1,49 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.
+ */
+
+#include "org_apache_hadoop_hdds_utils_db_managed_ManagedSSTDumpTool.h"
+#include "rocksdb/options.h"
+#include "rocksdb/sst_dump_tool.h"
+#include "string"
+#include "cplusplus_to_java_convert.h"
+#include "Pipe.h"
+#include "iostream"
+
+jint Java_org_apache_hadoop_hdds_utils_db_managed_ManagedSSTDumpTool_runInternal(JNIEnv *env, jobject obj, jobjectArray argsArray,
+jlong optionsHandle, jlong pipeHandle) {
+    ROCKSDB_NAMESPACE::SSTDumpTool dumpTool;
+    ROCKSDB_NAMESPACE::Options* options = reinterpret_cast<ROCKSDB_NAMESPACE::Options*>(optionsHandle);

Review Comment:
   ```suggestion
       ROCKSDB_NAMESPACE::Options *options = reinterpret_cast<ROCKSDB_NAMESPACE::Options *>(optionsHandle);
   ```



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1125267740


##########
hadoop-hdds/rocks-native/src/main/native/SSTDumpTool.cpp:
##########
@@ -0,0 +1,49 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.
+ */
+
+#include "org_apache_hadoop_hdds_utils_db_managed_ManagedSSTDumpTool.h"
+#include "rocksdb/options.h"
+#include "rocksdb/sst_dump_tool.h"
+#include "string"
+#include "cplusplus_to_java_convert.h"
+#include "Pipe.h"
+#include "iostream"
+
+jint Java_org_apache_hadoop_hdds_utils_db_managed_ManagedSSTDumpTool_runInternal(JNIEnv *env, jobject obj, jobjectArray argsArray,
+jlong optionsHandle, jlong pipeHandle) {
+    ROCKSDB_NAMESPACE::SSTDumpTool dumpTool;
+    ROCKSDB_NAMESPACE::Options* options = reinterpret_cast<ROCKSDB_NAMESPACE::Options*>(optionsHandle);
+    Pipe* pipe = reinterpret_cast<Pipe*>(pipeHandle);
+    int length = env->GetArrayLength(argsArray);
+    char* args[length + 1];
+    args[0] = strdup("./sst_dump");
+    for(int i = 0; i < length; i++) {
+        jstring str_val = (jstring)env->GetObjectArrayElement(argsArray, (jsize)i);
+        char* utf_str = (char*)env->GetStringUTFChars(str_val, JNI_FALSE);
+        args[i+1] = strdup(utf_str);
+        env->ReleaseStringUTFChars(str_val, utf_str);
+    }
+    FILE* wr = fdopen(pipe->getWriteFd(), "w");
+    int ret = dumpTool.Run(length + 1, args, *options, wr);
+    for (int i = 0; i<length+1; i++) {
+        free(args[i]);
+    }
+    fclose(wr);
+    pipe->close();
+    return ret;
+}

Review Comment:
   new line
   ```suggestion
   }
   
   ```



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1125268295


##########
hadoop-hdds/rocks-native/src/main/native/cplusplus_to_java_convert.h:
##########
@@ -0,0 +1,37 @@
+// Copyright (c) 2011-present, Facebook, Inc.  All rights reserved.

Review Comment:
   Probably need to add a another Apache 2.0 license header before this original one?



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1125275742


##########
hadoop-hdds/rocks-native/src/main/java/org/apache/hadoop/hdds/utils/NativeLibraryLoader.java:
##########
@@ -0,0 +1,138 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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;
+
+
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Files;
+import java.nio.file.StandardCopyOption;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Class to load Native Libraries.
+ */
+public class NativeLibraryLoader {
+  private static final String OS = System.getProperty("os.name").toLowerCase();
+  private Map<String, Boolean> librariesLoaded;
+  private static volatile NativeLibraryLoader instance;
+
+  public NativeLibraryLoader(final Map<String, Boolean> librariesLoaded) {
+    this.librariesLoaded = librariesLoaded;
+  }
+
+  private static synchronized void initNewInstance() {
+    if (instance == null) {
+      instance = new NativeLibraryLoader(new ConcurrentHashMap<>());
+    }
+  }
+
+  public static NativeLibraryLoader getInstance() {
+    if (instance == null) {
+      initNewInstance();
+    }
+    return instance;
+  }
+
+  public static String getJniLibraryFileName(String libraryName) {
+    return appendLibOsSuffix("lib" + libraryName);
+  }
+
+  public static boolean isMac() {
+    return OS.contains("mac");
+  }
+
+  public static boolean isWindows() {
+    return OS.contains("win");
+  }
+
+  private static String getLibOsSuffix() {
+    if (isMac()) {
+      return ".dylib";
+    } else if (isWindows()) {
+      return ".dll";
+    }
+    return ".so";

Review Comment:
   We'd better have explicit check for Linux rather than making it the default.
   
   Reject if OS name is unknown.



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1124931038


##########
hadoop-hdds/rocks-native/src/main/java/org/apache/hadoop/hdds/utils/NativeLibraryNotLoadedException.java:
##########
@@ -0,0 +1,30 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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;
+
+/**
+  Exception when native library not loaded.
+ */
+public class NativeLibraryNotLoadedException extends Exception {
+  public NativeLibraryNotLoadedException(String libraryName) {
+    super(String.format("Unable to load library %s from both " +
+            "java.library.path & resource file %s from jar.", libraryName,
+            NativeLibraryLoader.getJniLibraryFileName(libraryName)));
+  }
+}

Review Comment:
   nit: new line at the end of the file
   ```suggestion
   }
   
   ```



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] swamirishi commented on pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "swamirishi (via GitHub)" <gi...@apache.org>.
swamirishi commented on PR #4315:
URL: https://github.com/apache/ozone/pull/4315#issuecomment-1454835571

   > Thanks @swamirishi . If would be a great help if you can let me know if there are any special build parameters you have in mind for building this like the native library compile for hadoop? ie. `$ mvn package -Pdist,native -DskipTests -Dtar`
   > 
   > Also, what would be a good way that I can manually test the jni build?
   
   You can activate rocks-native-build by passing -Drocks_tools_native while maven build. 
   **mvn clean install -DskipTests -Drocks_tools_native -pl :hdds-rocks-native**


-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1123894034


##########
hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/HddsServerUtil.java:
##########
@@ -583,4 +584,29 @@ public static IOException toIOException(String msg, RocksDBException e) {
   public static void addSuppressedLoggingExceptions(RPC.Server server) {
     server.addSuppressedLoggingExceptions(ServerNotLeaderException.class);
   }
+
+  /**
+   * Build javaProcessBuilder.
+   * @param jvmArgs
+   * @param classpath
+   * @param className
+   * @param args
+   * @return ProcessBuilder
+   */
+  public static ProcessBuilder getJavaProcess(List<String> jvmArgs,
+                                           String classpath,
+                                           String className,
+                                           List<String> args) {
+    String javaHome = System.getProperty("java.home");
+    String javaBin = String.format("%1$s%2$sbin%2$sjava",
+            javaHome, File.separator);

Review Comment:
   fancy stuff.
   
   Add a comment saying we are trying to guess the `java` executable path here?



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1129982025


##########
hadoop-hdds/rocks-native/src/main/native/SSTDumpTool.cpp:
##########
@@ -0,0 +1,49 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.
+ */
+
+#include "org_apache_hadoop_hdds_utils_db_managed_ManagedSSTDumpTool.h"
+#include "rocksdb/options.h"
+#include "rocksdb/sst_dump_tool.h"
+#include <string>
+#include "cplusplus_to_java_convert.h"
+#include "Pipe.h"
+#include <iostream>
+
+jint Java_org_apache_hadoop_hdds_utils_db_managed_ManagedSSTDumpTool_runInternal(JNIEnv *env, jobject obj,
+ jobjectArray argsArray, jlong optionsHandle, jlong pipeHandle) {
+    ROCKSDB_NAMESPACE::SSTDumpTool dumpTool;
+    ROCKSDB_NAMESPACE::Options *options = reinterpret_cast<ROCKSDB_NAMESPACE::Options *>(optionsHandle);
+    Pipe *pipe = reinterpret_cast<Pipe *>(pipeHandle);
+    int length = env->GetArrayLength(argsArray);
+    char *args[length + 1];
+    args[0] = strdup("./sst_dump");
+    for (int i = 0; i < length; i++) {
+        jstring str_val = (jstring)env->GetObjectArrayElement(argsArray, (jsize)i);
+        char *utf_str = (char *)env->GetStringUTFChars(str_val, JNI_FALSE);
+        args[i + 1] = strdup(utf_str);
+        env->ReleaseStringUTFChars(str_val, utf_str);
+    }
+    FILE *wr = fdopen(pipe->getWriteFd(), "w");
+    int ret = dumpTool.Run(length + 1, args, *options, wr);
+    for (int i = 0; i < length + 1; i++) {
+        free(args[i]);
+    }
+    fclose(wr);
+    pipe->close();

Review Comment:
   ~~Need to close `options` as well.~~ e.g.
   
   https://github.com/facebook/rocksdb/blob/dec144f172165e9ce007aa12fbeaacf748cea5c1/java/src/test/java/org/rocksdb/SstFileReaderTest.java#L112
   
   Nope. It is passed in from Java. JVM GC should handle that instead.



##########
hadoop-hdds/rocks-native/src/main/native/SSTDumpTool.cpp:
##########
@@ -0,0 +1,49 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.
+ */
+
+#include "org_apache_hadoop_hdds_utils_db_managed_ManagedSSTDumpTool.h"
+#include "rocksdb/options.h"
+#include "rocksdb/sst_dump_tool.h"
+#include <string>
+#include "cplusplus_to_java_convert.h"
+#include "Pipe.h"
+#include <iostream>
+
+jint Java_org_apache_hadoop_hdds_utils_db_managed_ManagedSSTDumpTool_runInternal(JNIEnv *env, jobject obj,
+ jobjectArray argsArray, jlong optionsHandle, jlong pipeHandle) {
+    ROCKSDB_NAMESPACE::SSTDumpTool dumpTool;
+    ROCKSDB_NAMESPACE::Options *options = reinterpret_cast<ROCKSDB_NAMESPACE::Options *>(optionsHandle);
+    Pipe *pipe = reinterpret_cast<Pipe *>(pipeHandle);
+    int length = env->GetArrayLength(argsArray);
+    char *args[length + 1];
+    args[0] = strdup("./sst_dump");
+    for (int i = 0; i < length; i++) {
+        jstring str_val = (jstring)env->GetObjectArrayElement(argsArray, (jsize)i);
+        char *utf_str = (char *)env->GetStringUTFChars(str_val, JNI_FALSE);
+        args[i + 1] = strdup(utf_str);
+        env->ReleaseStringUTFChars(str_val, utf_str);
+    }
+    FILE *wr = fdopen(pipe->getWriteFd(), "w");
+    int ret = dumpTool.Run(length + 1, args, *options, wr);
+    for (int i = 0; i < length + 1; i++) {
+        free(args[i]);
+    }
+    fclose(wr);
+    pipe->close();

Review Comment:
   ~~Need to close `options` as well.~~ e.g.
   
   https://github.com/facebook/rocksdb/blob/dec144f172165e9ce007aa12fbeaacf748cea5c1/java/src/test/java/org/rocksdb/SstFileReaderTest.java#L112
   
   nvm. It is passed in from Java. JVM would handle that instead.



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] swamirishi commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "swamirishi (via GitHub)" <gi...@apache.org>.
swamirishi commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1129957585


##########
hadoop-hdds/rocks-native/src/main/java/org/apache/hadoop/hdds/utils/db/managed/ManagedSSTDumpIterator.java:
##########
@@ -0,0 +1,239 @@
+/**
+ * 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 org.apache.hadoop.hdds.utils.NativeLibraryNotLoadedException;
+import org.eclipse.jetty.io.RuntimeIOException;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
+import java.util.Iterator;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * Iterator to Parse output of RocksDBSSTDumpTool.
+ */
+public class ManagedSSTDumpIterator implements
+        Iterator<ManagedSSTDumpIterator.KeyValue>, AutoCloseable {
+  private static final String SST_DUMP_TOOL_CLASS =
+          "org.apache.hadoop.hdds.utils.db.managed.ManagedSSTDumpTool";
+  private static final String PATTERN_REGEX =
+          "'([^=>]+)' seq:([0-9]+), type:([0-9]+) => ";
+
+  public static final int PATTERN_KEY_GROUP_NUMBER = 1;
+  public static final int PATTERN_SEQ_GROUP_NUMBER = 2;
+  public static final int PATTERN_TYPE_GROUP_NUMBER = 3;
+  private static final Pattern PATTERN_MATCHER =
+          Pattern.compile(PATTERN_REGEX);
+  private BufferedReader processOutput;
+  private StringBuilder stdoutString;
+
+  private Matcher currentMatcher;
+  private int prevMatchEndIndex;
+  private KeyValue currentKey;
+  private char[] charBuffer;
+  private KeyValue nextKey;
+
+  private long pollIntervalMillis;
+  private ManagedSSTDumpTool.SSTDumpToolTask sstDumpToolTask;
+  private Lock lock;
+  private AtomicBoolean open;
+
+
+  public ManagedSSTDumpIterator(ManagedSSTDumpTool sstDumpTool,
+                                String sstFilePath,
+                                ManagedOptions options,
+                                long pollIntervalMillis) throws IOException,
+          NativeLibraryNotLoadedException {
+    File sstFile = new File(sstFilePath);
+    if (!sstFile.exists() || !sstFile.isFile()) {
+      throw new IOException(String.format("Invalid SST File Path : %s",
+              sstFile.getAbsolutePath()));
+    }
+    this.pollIntervalMillis = pollIntervalMillis;
+    this.lock = new ReentrantLock();
+    init(sstDumpTool, sstFile, options);
+  }
+
+  private void init(ManagedSSTDumpTool sstDumpTool, File sstFile,
+                    ManagedOptions options)
+          throws NativeLibraryNotLoadedException {
+    String[] args = {"--file=" + sstFile.getAbsolutePath(),
+                     "--command=scan"};
+    this.sstDumpToolTask = sstDumpTool.run(args, options);
+    processOutput = new BufferedReader(new InputStreamReader(
+            sstDumpToolTask.getPipedOutput(), StandardCharsets.UTF_8));
+    stdoutString = new StringBuilder();
+    currentMatcher = PATTERN_MATCHER.matcher(stdoutString);
+    charBuffer = new char[8192];
+    open = new AtomicBoolean(true);
+    next();
+  }
+
+  /**
+   * Throws Runtime exception in the case iterator is closed or
+   * the native Dumptool exited with non zero exit value.
+   */
+  private void checkSanityOfProcess() {
+    if (!this.open.get()) {
+      throw new RuntimeException("Iterator has been closed");
+    }
+    if (sstDumpToolTask.getFuture().isDone()
+            && sstDumpToolTask.exitValue() != 0) {
+      throw new RuntimeException("Process Terminated with non zero " +
+              String.format("exit value %d", sstDumpToolTask.exitValue()));
+    }
+  }
+
+  /**
+   *
+   * @return
+   * Throws Runtime Exception in case of SST File read failure
+   */
+
+  @Override
+  public boolean hasNext() {
+    checkSanityOfProcess();
+    return nextKey != null;
+  }
+
+  /**
+   *
+   * @return next Key
+   * Throws Runtime Exception incase of failure.
+   */
+  @Override
+  public KeyValue next() {
+    lock.lock();
+    try {
+      checkSanityOfProcess();
+      currentKey = nextKey;
+      nextKey = null;
+      while (!currentMatcher.find()) {
+        try {
+          if (prevMatchEndIndex != 0) {
+            stdoutString = new StringBuilder(stdoutString.substring(
+                    prevMatchEndIndex, stdoutString.length()));
+            prevMatchEndIndex = 0;
+            currentMatcher = PATTERN_MATCHER.matcher(stdoutString);
+          }
+          Thread.sleep(pollIntervalMillis);

Review Comment:
   We need to wait for the sst dump tool to flush the output to pipe. It is better to sleep till then.



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1129983604


##########
hadoop-hdds/rocks-native/src/main/native/SSTDumpTool.cpp:
##########
@@ -0,0 +1,49 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.
+ */
+
+#include "org_apache_hadoop_hdds_utils_db_managed_ManagedSSTDumpTool.h"
+#include "rocksdb/options.h"
+#include "rocksdb/sst_dump_tool.h"
+#include <string>
+#include "cplusplus_to_java_convert.h"
+#include "Pipe.h"
+#include <iostream>
+
+jint Java_org_apache_hadoop_hdds_utils_db_managed_ManagedSSTDumpTool_runInternal(JNIEnv *env, jobject obj,
+ jobjectArray argsArray, jlong optionsHandle, jlong pipeHandle) {
+    ROCKSDB_NAMESPACE::SSTDumpTool dumpTool;
+    ROCKSDB_NAMESPACE::Options *options = reinterpret_cast<ROCKSDB_NAMESPACE::Options *>(optionsHandle);
+    Pipe *pipe = reinterpret_cast<Pipe *>(pipeHandle);
+    int length = env->GetArrayLength(argsArray);
+    char *args[length + 1];
+    args[0] = strdup("./sst_dump");
+    for (int i = 0; i < length; i++) {
+        jstring str_val = (jstring)env->GetObjectArrayElement(argsArray, (jsize)i);
+        char *utf_str = (char *)env->GetStringUTFChars(str_val, JNI_FALSE);
+        args[i + 1] = strdup(utf_str);
+        env->ReleaseStringUTFChars(str_val, utf_str);
+    }
+    FILE *wr = fdopen(pipe->getWriteFd(), "w");
+    int ret = dumpTool.Run(length + 1, args, *options, wr);
+    for (int i = 0; i < length + 1; i++) {
+        free(args[i]);
+    }
+    fclose(wr);
+    pipe->close();

Review Comment:
   Since `pipe` is also passed in from the Java side. Should Java code be taking care of `pipe` as well? (i.e. don't close `pipe` in C++ code here). Do it in Java instead for consistency: https://github.com/apache/ozone/pull/4315/files#r1129985440



##########
hadoop-hdds/rocks-native/src/main/native/SSTDumpTool.cpp:
##########
@@ -0,0 +1,49 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.
+ */
+
+#include "org_apache_hadoop_hdds_utils_db_managed_ManagedSSTDumpTool.h"
+#include "rocksdb/options.h"
+#include "rocksdb/sst_dump_tool.h"
+#include <string>
+#include "cplusplus_to_java_convert.h"
+#include "Pipe.h"
+#include <iostream>
+
+jint Java_org_apache_hadoop_hdds_utils_db_managed_ManagedSSTDumpTool_runInternal(JNIEnv *env, jobject obj,
+ jobjectArray argsArray, jlong optionsHandle, jlong pipeHandle) {
+    ROCKSDB_NAMESPACE::SSTDumpTool dumpTool;
+    ROCKSDB_NAMESPACE::Options *options = reinterpret_cast<ROCKSDB_NAMESPACE::Options *>(optionsHandle);
+    Pipe *pipe = reinterpret_cast<Pipe *>(pipeHandle);
+    int length = env->GetArrayLength(argsArray);
+    char *args[length + 1];
+    args[0] = strdup("./sst_dump");
+    for (int i = 0; i < length; i++) {
+        jstring str_val = (jstring)env->GetObjectArrayElement(argsArray, (jsize)i);
+        char *utf_str = (char *)env->GetStringUTFChars(str_val, JNI_FALSE);
+        args[i + 1] = strdup(utf_str);
+        env->ReleaseStringUTFChars(str_val, utf_str);
+    }
+    FILE *wr = fdopen(pipe->getWriteFd(), "w");
+    int ret = dumpTool.Run(length + 1, args, *options, wr);
+    for (int i = 0; i < length + 1; i++) {
+        free(args[i]);
+    }
+    fclose(wr);
+    pipe->close();

Review Comment:
   Since `pipe` is also passed in from the Java side. Should Java code be taking care of `pipe` as well? (i.e. don't close `pipe` in C++ code here).
   
   Do it in Java instead for consistency: https://github.com/apache/ozone/pull/4315/files#r1129985440



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1131661168


##########
hadoop-hdds/rocks-native/src/main/java/org/apache/hadoop/hdds/utils/db/managed/ManagedSSTDumpIterator.java:
##########
@@ -0,0 +1,239 @@
+/**
+ * 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 org.apache.hadoop.hdds.utils.NativeLibraryNotLoadedException;
+import org.eclipse.jetty.io.RuntimeIOException;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
+import java.util.Iterator;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * Iterator to Parse output of RocksDBSSTDumpTool.
+ */
+public class ManagedSSTDumpIterator implements
+        Iterator<ManagedSSTDumpIterator.KeyValue>, AutoCloseable {
+  private static final String SST_DUMP_TOOL_CLASS =
+          "org.apache.hadoop.hdds.utils.db.managed.ManagedSSTDumpTool";
+  private static final String PATTERN_REGEX =
+          "'([^=>]+)' seq:([0-9]+), type:([0-9]+) => ";
+
+  public static final int PATTERN_KEY_GROUP_NUMBER = 1;
+  public static final int PATTERN_SEQ_GROUP_NUMBER = 2;
+  public static final int PATTERN_TYPE_GROUP_NUMBER = 3;
+  private static final Pattern PATTERN_MATCHER =
+          Pattern.compile(PATTERN_REGEX);
+  private BufferedReader processOutput;
+  private StringBuilder stdoutString;
+
+  private Matcher currentMatcher;
+  private int prevMatchEndIndex;
+  private KeyValue currentKey;
+  private char[] charBuffer;
+  private KeyValue nextKey;
+
+  private long pollIntervalMillis;
+  private ManagedSSTDumpTool.SSTDumpToolTask sstDumpToolTask;
+  private Lock lock;
+  private AtomicBoolean open;
+
+
+  public ManagedSSTDumpIterator(ManagedSSTDumpTool sstDumpTool,
+                                String sstFilePath,
+                                ManagedOptions options,
+                                long pollIntervalMillis) throws IOException,
+          NativeLibraryNotLoadedException {
+    File sstFile = new File(sstFilePath);
+    if (!sstFile.exists() || !sstFile.isFile()) {
+      throw new IOException(String.format("Invalid SST File Path : %s",
+              sstFile.getAbsolutePath()));
+    }
+    this.pollIntervalMillis = pollIntervalMillis;
+    this.lock = new ReentrantLock();
+    init(sstDumpTool, sstFile, options);
+  }
+
+  private void init(ManagedSSTDumpTool sstDumpTool, File sstFile,
+                    ManagedOptions options)
+          throws NativeLibraryNotLoadedException {
+    String[] args = {"--file=" + sstFile.getAbsolutePath(),
+                     "--command=scan"};
+    this.sstDumpToolTask = sstDumpTool.run(args, options);
+    processOutput = new BufferedReader(new InputStreamReader(
+            sstDumpToolTask.getPipedOutput(), StandardCharsets.UTF_8));
+    stdoutString = new StringBuilder();
+    currentMatcher = PATTERN_MATCHER.matcher(stdoutString);
+    charBuffer = new char[8192];
+    open = new AtomicBoolean(true);
+    next();
+  }
+
+  /**
+   * Throws Runtime exception in the case iterator is closed or
+   * the native Dumptool exited with non zero exit value.
+   */
+  private void checkSanityOfProcess() {
+    if (!this.open.get()) {
+      throw new RuntimeException("Iterator has been closed");
+    }
+    if (sstDumpToolTask.getFuture().isDone()
+            && sstDumpToolTask.exitValue() != 0) {
+      throw new RuntimeException("Process Terminated with non zero " +
+              String.format("exit value %d", sstDumpToolTask.exitValue()));
+    }
+  }
+
+  /**
+   *
+   * @return
+   * Throws Runtime Exception in case of SST File read failure
+   */
+
+  @Override
+  public boolean hasNext() {
+    checkSanityOfProcess();
+    return nextKey != null;
+  }
+
+  /**
+   *
+   * @return next Key
+   * Throws Runtime Exception incase of failure.
+   */
+  @Override
+  public KeyValue next() {
+    lock.lock();
+    try {
+      checkSanityOfProcess();
+      currentKey = nextKey;
+      nextKey = null;
+      while (!currentMatcher.find()) {
+        try {
+          if (prevMatchEndIndex != 0) {
+            stdoutString = new StringBuilder(stdoutString.substring(
+                    prevMatchEndIndex, stdoutString.length()));
+            prevMatchEndIndex = 0;
+            currentMatcher = PATTERN_MATCHER.matcher(stdoutString);
+          }
+          Thread.sleep(pollIntervalMillis);

Review Comment:
   `processOutput.read()` appears to be a blocking call. Thus this `sleep()` might not be really necessary.



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1128465663


##########
hadoop-hdds/rocks-native/src/main/java/org/apache/hadoop/hdds/utils/db/managed/ManagedSSTDumpIterator.java:
##########
@@ -0,0 +1,239 @@
+/**
+ * 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;

Review Comment:
   fyi in HDDS-8079, we have moved Managed DB classes (again) under a new path. e.g.
   
   ```
   hadoop-hdds/managed-rocksdb/src/main/java/org/apache/hadoop/hdds/utils/db/managed/ManagedDBOptions.java
   ```
   
   It could be an option to move this class there later.



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] swamirishi commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "swamirishi (via GitHub)" <gi...@apache.org>.
swamirishi commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1128826992


##########
hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/MessageQueue.java:
##########
@@ -0,0 +1,188 @@
+/**
+ * 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;
+
+import org.apache.commons.io.input.buffer.CircularByteBuffer;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.concurrent.Callable;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * Message Queue to Pipe output from one output stream to another inputstream.
+ */
+public class MessageQueue {
+  private CircularByteBuffer byteBuffer;
+  private AtomicBoolean isRunning;
+
+  private MessageOutputStream messageOutputStream;
+  private MessageInputStream messageInputStream;
+
+  public MessageQueue(int bufferSize, long pollIntervalMillis) {
+    this.pollIntervalMillis = pollIntervalMillis;
+    init(bufferSize);
+  }
+
+  private void init(int bufferSize) {
+    this.byteBuffer = new CircularByteBuffer(bufferSize);
+    this.isRunning = new AtomicBoolean(false);
+    this.messageInputStream = new MessageInputStream(this);
+    this.messageOutputStream = new MessageOutputStream(this);
+  }
+
+  public void start() {
+    this.isRunning.set(true);
+  }
+
+  public void stop() {
+    this.isRunning.set(false);
+  }
+
+  public MessageOutputStream getMessageOutputStream() {
+    return messageOutputStream;
+  }
+
+  public MessageInputStream getMessageInputStream() {
+    return messageInputStream;
+  }
+
+  private long pollIntervalMillis;
+
+  public boolean isRunning() {
+    return isRunning.get();
+  }
+
+  private long getPollIntervalMillis() {
+    return pollIntervalMillis;
+  }
+
+  private boolean hasSpace(int requiredLength) {
+    return this.byteBuffer.hasSpace(requiredLength);
+  }
+
+  private boolean hasBytes() {
+    return this.byteBuffer.hasBytes();
+  }
+
+  private int getCurrentNumberOfBytes() {
+    return this.byteBuffer.getCurrentNumberOfBytes();
+  }
+
+  private void add(byte[] b, int off, int len) {
+    this.byteBuffer.add(b, off, len);
+  }
+
+  private int read(byte[] b, int off, int len) {
+    this.byteBuffer.read(b, off, len);
+    return len;
+  }
+
+  private static  <T> T callWithLock(Lock lock, Callable<T> callable)
+          throws Exception {
+    lock.lock();
+    try {
+      return callable.call();
+    } finally {
+      lock.unlock();
+    }
+  }
+  private static final class MessageOutputStream extends OutputStream {

Review Comment:
   removed class 



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] swamirishi commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "swamirishi (via GitHub)" <gi...@apache.org>.
swamirishi commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1128825811


##########
hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/HddsServerUtil.java:
##########
@@ -583,4 +584,29 @@ public static IOException toIOException(String msg, RocksDBException e) {
   public static void addSuppressedLoggingExceptions(RPC.Server server) {
     server.addSuppressedLoggingExceptions(ServerNotLeaderException.class);
   }
+
+  /**
+   * Build javaProcessBuilder.
+   * @param jvmArgs
+   * @param classpath
+   * @param className
+   * @param args
+   * @return ProcessBuilder
+   */
+  public static ProcessBuilder getJavaProcess(List<String> jvmArgs,
+                                           String classpath,
+                                           String className,
+                                           List<String> args) {

Review Comment:
   removed the function all together



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] swamirishi commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "swamirishi (via GitHub)" <gi...@apache.org>.
swamirishi commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1132153976


##########
hadoop-hdds/rocks-native/src/main/native/PipeInputStream.cpp:
##########
@@ -0,0 +1,48 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.
+ */
+
+#include <unistd.h>
+#include <iostream>
+#include "Pipe.h"
+#include "cplusplus_to_java_convert.h"
+#include "org_apache_hadoop_hdds_utils_db_managed_PipeInputStream.h"
+
+
+jlong Java_org_apache_hadoop_hdds_utils_db_managed_PipeInputStream_newPipe(JNIEnv *, jobject) {
+    Pipe *pipe = new Pipe();
+    return GET_CPLUSPLUS_POINTER(pipe);
+}
+
+jint Java_org_apache_hadoop_hdds_utils_db_managed_PipeInputStream_readInternal(JNIEnv *env, jobject object, jbyteArray jbyteArray, jint capacity, jlong nativeHandle) {
+    int cap_int = capacity;
+    Pipe *pipe = reinterpret_cast<Pipe *>(nativeHandle);
+    jbyte *b = (env)->GetByteArrayElements(jbyteArray, JNI_FALSE);
+    cap_int = read(pipe->getReadFd(), b, cap_int);
+    if (cap_int == 0) {
+        if (!pipe->isOpen()) {
+            cap_int = -1;
+        }
+    }
+    (env)->ReleaseByteArrayElements(jbyteArray, b, 0);

Review Comment:
   done



##########
hadoop-hdds/rocks-native/src/main/java/org/apache/hadoop/hdds/utils/db/managed/ManagedSSTDumpIterator.java:
##########
@@ -0,0 +1,239 @@
+/**
+ * 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 org.apache.hadoop.hdds.utils.NativeLibraryNotLoadedException;
+import org.eclipse.jetty.io.RuntimeIOException;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
+import java.util.Iterator;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * Iterator to Parse output of RocksDBSSTDumpTool.
+ */
+public class ManagedSSTDumpIterator implements
+        Iterator<ManagedSSTDumpIterator.KeyValue>, AutoCloseable {
+  private static final String SST_DUMP_TOOL_CLASS =
+          "org.apache.hadoop.hdds.utils.db.managed.ManagedSSTDumpTool";
+  private static final String PATTERN_REGEX =
+          "'([^=>]+)' seq:([0-9]+), type:([0-9]+) => ";
+
+  public static final int PATTERN_KEY_GROUP_NUMBER = 1;
+  public static final int PATTERN_SEQ_GROUP_NUMBER = 2;
+  public static final int PATTERN_TYPE_GROUP_NUMBER = 3;
+  private static final Pattern PATTERN_MATCHER =
+          Pattern.compile(PATTERN_REGEX);
+  private BufferedReader processOutput;
+  private StringBuilder stdoutString;
+
+  private Matcher currentMatcher;
+  private int prevMatchEndIndex;
+  private KeyValue currentKey;
+  private char[] charBuffer;
+  private KeyValue nextKey;
+
+  private long pollIntervalMillis;
+  private ManagedSSTDumpTool.SSTDumpToolTask sstDumpToolTask;
+  private Lock lock;
+  private AtomicBoolean open;
+
+
+  public ManagedSSTDumpIterator(ManagedSSTDumpTool sstDumpTool,
+                                String sstFilePath,
+                                ManagedOptions options,
+                                long pollIntervalMillis) throws IOException,
+          NativeLibraryNotLoadedException {
+    File sstFile = new File(sstFilePath);
+    if (!sstFile.exists() || !sstFile.isFile()) {
+      throw new IOException(String.format("Invalid SST File Path : %s",
+              sstFile.getAbsolutePath()));
+    }
+    this.pollIntervalMillis = pollIntervalMillis;
+    this.lock = new ReentrantLock();
+    init(sstDumpTool, sstFile, options);
+  }
+
+  private void init(ManagedSSTDumpTool sstDumpTool, File sstFile,
+                    ManagedOptions options)
+          throws NativeLibraryNotLoadedException {
+    String[] args = {"--file=" + sstFile.getAbsolutePath(),
+                     "--command=scan"};
+    this.sstDumpToolTask = sstDumpTool.run(args, options);
+    processOutput = new BufferedReader(new InputStreamReader(
+            sstDumpToolTask.getPipedOutput(), StandardCharsets.UTF_8));
+    stdoutString = new StringBuilder();
+    currentMatcher = PATTERN_MATCHER.matcher(stdoutString);
+    charBuffer = new char[8192];
+    open = new AtomicBoolean(true);
+    next();
+  }
+
+  /**
+   * Throws Runtime exception in the case iterator is closed or
+   * the native Dumptool exited with non zero exit value.
+   */
+  private void checkSanityOfProcess() {
+    if (!this.open.get()) {
+      throw new RuntimeException("Iterator has been closed");
+    }
+    if (sstDumpToolTask.getFuture().isDone()
+            && sstDumpToolTask.exitValue() != 0) {
+      throw new RuntimeException("Process Terminated with non zero " +
+              String.format("exit value %d", sstDumpToolTask.exitValue()));
+    }
+  }
+
+  /**
+   * Checks the status of the process & sees if there is another record.
+   * @return True if next exists & false otherwise
+   * Throws Runtime Exception in case of SST File read failure
+   */
+
+  @Override
+  public boolean hasNext() {
+    checkSanityOfProcess();
+    return nextKey != null;
+  }
+
+  /**
+   * Returns the next record from SSTDumpTool.
+   * @return next Key
+   * Throws Runtime Exception incase of failure.
+   */
+  @Override
+  public KeyValue next() {
+    lock.lock();
+    try {
+      checkSanityOfProcess();
+      currentKey = nextKey;
+      nextKey = null;
+      while (!currentMatcher.find()) {
+        try {
+          if (prevMatchEndIndex != 0) {
+            stdoutString = new StringBuilder(stdoutString.substring(
+                    prevMatchEndIndex, stdoutString.length()));
+            prevMatchEndIndex = 0;
+            currentMatcher = PATTERN_MATCHER.matcher(stdoutString);
+          }
+          Thread.sleep(pollIntervalMillis);
+          int numberOfCharsRead = processOutput.read(charBuffer);
+          if (numberOfCharsRead < 0) {
+            if (currentKey != null) {
+              currentKey.setValue(stdoutString.substring(0,
+                      Math.max(stdoutString.length() - 1, 0)));
+            }
+            return currentKey;
+          }
+          stdoutString.append(charBuffer, 0, numberOfCharsRead);
+          currentMatcher.reset();
+        } catch (IOException | InterruptedException e) {
+          throw new RuntimeIOException(e);
+        }
+      }
+      if (currentKey != null) {
+        currentKey.setValue(stdoutString.substring(prevMatchEndIndex,
+                currentMatcher.start() - 1));
+      }
+      prevMatchEndIndex = currentMatcher.end();
+      nextKey =  new KeyValue(

Review Comment:
   done



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1132792151


##########
hadoop-hdds/rocks-native/pom.xml:
##########
@@ -0,0 +1,405 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed 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
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+  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. See accompanying LICENSE file.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <parent>
+        <artifactId>hdds</artifactId>
+        <groupId>org.apache.ozone</groupId>
+        <version>1.4.0-SNAPSHOT</version>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+    <name>Apache Ozone HDDS RocksDB Tools</name>
+    <artifactId>hdds-rocks-native</artifactId>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.ozone</groupId>
+            <artifactId>hdds-managed-rocksdb</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.eclipse.jetty</groupId>
+            <artifactId>jetty-io</artifactId>
+        </dependency>
+    </dependencies>
+    <properties>
+        <maven.compiler.source>8</maven.compiler.source>
+        <maven.compiler.target>8</maven.compiler.target>
+    </properties>
+
+    <profiles>
+        <profile>
+            <id>rocks_tools_native</id>
+            <activation>
+                <property>
+                    <name>rocks_tools_native</name>
+                </property>
+            </activation>
+            <properties>
+                <cmake.standards>23</cmake.standards>
+                <sstDump.include>true</sstDump.include>
+            </properties>
+            <build>
+                <plugins>
+                    <plugin>
+                        <groupId>com.googlecode.maven-download-plugin</groupId>
+                        <artifactId>download-maven-plugin</artifactId>
+                        <executions>
+                            <execution>
+                                <id>rocksdb source download</id>
+                                <phase>generate-sources</phase>
+                                <goals>
+                                    <goal>wget</goal>
+                                </goals>
+                                <configuration>
+                                    <url>https://github.com/facebook/rocksdb/archive/refs/tags/v${rocksdb.version}.tar.gz</url>
+                                    <outputFileName>rocksdb-v${rocksdb.version}.tar.gz</outputFileName>
+                                    <outputDirectory>${project.build.directory}/rocksdb</outputDirectory>
+                                </configuration>
+                            </execution>
+                            <execution>
+                                <id>zlib source download</id>
+                                <phase>generate-sources</phase>
+                                <goals>
+                                    <goal>wget</goal>
+                                </goals>
+                                <configuration>
+                                    <url>https://zlib.net/zlib-${zlib.version}.tar.gz</url>
+                                    <outputFileName>zlib-${zlib.version}.tar.gz</outputFileName>
+                                    <outputDirectory>${project.build.directory}/zlib</outputDirectory>
+                                </configuration>
+                            </execution>
+                            <execution>
+                                <id>bzip2 source download</id>
+                                <phase>generate-sources</phase>
+                                <goals>
+                                    <goal>wget</goal>
+                                </goals>
+                                <configuration>
+                                    <url>https://sourceware.org/pub/bzip2/bzip2-${bzip2.version}.tar.gz</url>
+                                    <outputFileName>bzip2-v${bzip2.version}.tar.gz</outputFileName>
+                                    <outputDirectory>${project.build.directory}/bzip2</outputDirectory>
+                                </configuration>
+                            </execution>
+                            <execution>
+                                <id>lz4 source download</id>
+                                <phase>generate-sources</phase>
+                                <goals>
+                                    <goal>wget</goal>
+                                </goals>
+                                <configuration>
+                                    <url>https://github.com/lz4/lz4/archive/refs/tags/v${lz4.version}.tar.gz</url>
+                                    <outputFileName>lz4-v${lz4.version}.tar.gz</outputFileName>
+                                    <outputDirectory>${project.build.directory}/lz4</outputDirectory>
+                                </configuration>
+                            </execution>
+                            <execution>
+                                <id>snappy source download</id>
+                                <phase>generate-sources</phase>
+                                <goals>
+                                    <goal>wget</goal>
+                                </goals>
+                                <configuration>
+                                    <url>https://github.com/google/snappy/archive/refs/tags/${snappy.version}.tar.gz</url>
+                                    <outputFileName>snappy-v${snappy.version}.tar.gz</outputFileName>
+                                    <outputDirectory>${project.build.directory}/snappy</outputDirectory>
+                                </configuration>
+                            </execution>
+                            <execution>
+                                <id>zstd source download</id>
+                                <phase>generate-sources</phase>
+                                <goals>
+                                    <goal>wget</goal>
+                                </goals>
+                                <configuration>
+                                    <url>https://github.com/facebook/zstd/archive/refs/tags/v${zstd.version}.tar.gz</url>
+                                    <outputFileName>zstd-v${zstd.version}.tar.gz</outputFileName>
+                                    <outputDirectory>${project.build.directory}/zstd</outputDirectory>
+                                </configuration>
+                            </execution>
+                        </executions>
+                    </plugin>
+                    <plugin>
+                        <groupId>org.apache.maven.plugins</groupId>
+                        <artifactId>maven-patch-plugin</artifactId>
+                        <version>1.1.1</version>
+                        <configuration>
+                            <patchFile>${basedir}/src/main/patches/rocks-native.patch</patchFile>
+                            <strip>1</strip>
+                            <targetDirectory>${project.build.directory}/rocksdb/rocksdb-${rocksdb.version}</targetDirectory>
+                        </configuration>
+                        <executions>
+                            <execution>
+                                <id>patch</id>
+                                <phase>process-sources</phase>
+                                <goals>
+                                    <goal>apply</goal>
+                                </goals>
+                            </execution>
+                        </executions>
+                    </plugin>
+                    <plugin>
+                        <artifactId>maven-antrun-plugin</artifactId>
+                        <executions>
+                            <execution>
+                                <id>unzip-artifact</id>
+                                <phase>generate-sources</phase>
+                                <configuration>
+                                    <tasks>
+                                        <untar src="${project.build.directory}/rocksdb/rocksdb-v${rocksdb.version}.tar.gz" compression="gzip" dest="${project.build.directory}/rocksdb/" />
+                                        <untar src="${project.build.directory}/zlib/zlib-${zlib.version}.tar.gz" compression="gzip" dest="${project.build.directory}/zlib/" />
+                                        <untar src="${project.build.directory}/bzip2/bzip2-v${bzip2.version}.tar.gz" compression="gzip" dest="${project.build.directory}/bzip2/" />
+                                        <untar src="${project.build.directory}/lz4/lz4-v${lz4.version}.tar.gz" compression="gzip" dest="${project.build.directory}/lz4/" />
+                                        <untar src="${project.build.directory}/snappy/snappy-v${snappy.version}.tar.gz" compression="gzip" dest="${project.build.directory}/snappy/" />
+                                        <untar src="${project.build.directory}/zstd/zstd-v${zstd.version}.tar.gz" compression="gzip" dest="${project.build.directory}/zstd/" />
+                                    </tasks>
+                                </configuration>
+                                <goals>
+                                    <goal>run</goal>
+                                </goals>
+                            </execution>
+                            <execution>
+                                <id>build-zlib</id>
+                                <phase>process-sources</phase>
+                                <configuration>
+                                    <tasks>
+                                        <chmod file="${project.build.directory}/zlib/zlib-${zlib.version}/configure" perm="775" />
+                                        <exec executable="./configure" dir="${project.build.directory}/zlib/zlib-${zlib.version}" failonerror="true">
+                                            <arg line="--static"/>
+                                            <env key="CFLAGS" value="-fPIC"/>
+                                        </exec>
+                                        <exec executable="make" dir="${project.build.directory}/zlib/zlib-${zlib.version}" failonerror="true"/>
+                                    </tasks>
+                                </configuration>
+                                <goals>
+                                    <goal>run</goal>
+                                </goals>
+                            </execution>
+                            <execution>
+                                <id>build-bzip2</id>
+                                <phase>process-sources</phase>
+                                <configuration>
+                                    <tasks>
+                                        <exec executable="make" dir="${project.build.directory}/bzip2/bzip2-${bzip2.version}" failonerror="true">
+                                            <arg line="CFLAGS='-fPIC'"/>
+                                        </exec>
+                                    </tasks>
+                                </configuration>
+                                <goals>
+                                    <goal>run</goal>
+                                </goals>
+                            </execution>
+                            <execution>
+                                <id>build-lz4</id>
+                                <phase>process-sources</phase>
+                                <configuration>
+                                    <tasks>
+                                        <exec executable="make" dir="${project.build.directory}/lz4/lz4-${lz4.version}" failonerror="true">
+                                            <arg line="CFLAGS='-fPIC'"/>
+                                        </exec>
+                                    </tasks>
+                                </configuration>
+                                <goals>
+                                    <goal>run</goal>
+                                </goals>
+                            </execution>
+                            <execution>
+                                <id>build-zstd</id>
+                                <phase>process-sources</phase>
+                                <configuration>
+                                    <tasks>
+                                        <exec executable="make" dir="${project.build.directory}/zstd/zstd-${zstd.version}" failonerror="true">
+                                            <arg line="CFLAGS='-fPIC'"/>
+                                        </exec>
+                                    </tasks>
+                                </configuration>
+                                <goals>
+                                    <goal>run</goal>
+                                </goals>
+                            </execution>
+                            <execution>
+                                <id>build-snappy</id>
+                                <phase>process-sources</phase>
+                                <configuration>
+                                    <tasks>
+                                        <mkdir dir="${project.build.directory}/snappy/lib"/>
+                                        <exec executable="cmake" failonerror="true" dir="${project.build.directory}/snappy/lib">
+                                            <arg line="${project.build.directory}/snappy/snappy-${snappy.version}"/>
+                                            <env key="CFLAGS" value="-fPIC"/>
+                                            <env key="CXXFLAGS" value="-fPIC"/>
+                                        </exec>
+                                        <exec executable="make" dir="${project.build.directory}/snappy/lib" failonerror="true"/>
+                                    </tasks>
+                                </configuration>
+                                <goals>
+                                    <goal>run</goal>
+                                </goals>
+                            </execution>
+                            <execution>
+                                <id>build-rocksjava</id>
+                                <phase>process-resources</phase>
+                                <configuration>
+                                    <tasks>
+                                        <exec executable="chmod" failonerror="true">
+                                            <arg line="-R"/>
+                                            <arg line="775"/>
+                                            <arg line="${project.build.directory}/rocksdb/rocksdb-${rocksdb.version}"/>
+                                        </exec>
+                                        <exec executable="make" dir="${project.build.directory}/rocksdb/rocksdb-${rocksdb.version}" failonerror="true">
+                                            <arg line="EXTRA_CXXFLAGS='-fPIC -I${project.build.directory}/snappy/lib -I${project.build.directory}/snappy/snappy-${snappy.version} -I${project.build.directory}/lz4/lz4-${lz4.version}/lib -I${project.build.directory}/zstd/zstd-${zstd.version}/lib -I${project.build.directory}/zstd/zstd-${zstd.version}/lib/dictBuilder -I${project.build.directory}/bzip2/bzip2-${bzip2.version} -I${project.build.directory}/zlib/zlib-${zlib.version}'"/>
+                                            <arg line="EXTRA_LDFLAGS='-L${project.build.directory}/snappy/lib -L${project.build.directory}/lz4/lz4-${lz4.version}/lib -L${project.build.directory}/zstd/zstd-${zstd.version}/lib -L${project.build.directory}/bzip2/bzip2-${bzip2.version} -L${project.build.directory}/zlib/zlib-${zlib.version}'"/>
+                                            <arg line="-j4"/>

Review Comment:
   TODO: Use `nproc` or equivalent



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1125253772


##########
hadoop-hdds/rocks-native/src/main/java/org/apache/hadoop/hdds/utils/db/managed/ManagedSSTDumpIterator.java:
##########
@@ -0,0 +1,238 @@
+/**
+ * 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 org.apache.hadoop.hdds.utils.NativeLibraryNotLoadedException;
+import org.eclipse.jetty.io.RuntimeIOException;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
+import java.util.Iterator;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * Iterator to Parse output of RocksDBSSTDumpTool.
+ */
+public class ManagedSSTDumpIterator implements
+        Iterator<ManagedSSTDumpIterator.KeyValue>, AutoCloseable {
+  private static final String SST_DUMP_TOOL_CLASS =
+          "org.apache.hadoop.hdds.utils.db.managed.ManagedSSTDumpTool";
+  private static final String PATTERN_REGEX =
+          "'([^=>]+)' seq:([0-9]+), type:([0-9]+) =>";
+
+  public static final int PATTERN_KEY_GROUP_NUMBER = 1;
+  public static final int PATTERN_SEQ_GROUP_NUMBER = 2;
+  public static final int PATTERN_TYPE_GROUP_NUMBER = 3;
+  private static final Pattern PATTERN_MATCHER =
+          Pattern.compile(PATTERN_REGEX);
+  private BufferedReader processOutput;
+  private StringBuilder stdoutString;
+
+  private Matcher currentMatcher;
+  private int prevMatchEndIndex;
+  private KeyValue currentKey;
+  private char[] charBuffer;
+  private KeyValue nextKey;
+
+  private long pollIntervalMillis;
+  private ManagedSSTDumpTool.SSTDumpToolTask sstDumpToolTask;
+  private Lock lock;
+  private AtomicBoolean open;
+
+
+  public ManagedSSTDumpIterator(ManagedSSTDumpTool sstDumpTool,
+                                String sstFilePath,
+                                ManagedOptions options,
+                                long pollIntervalMillis) throws IOException,
+          NativeLibraryNotLoadedException {
+    File sstFile = new File(sstFilePath);
+    if (!sstFile.exists() || !sstFile.isFile()) {
+      throw new IOException(String.format("Invalid SST File Path : %s",
+              sstFile.getAbsolutePath()));
+    }
+    this.pollIntervalMillis = pollIntervalMillis;
+    this.lock = new ReentrantLock();
+    init(sstDumpTool, sstFile, options);
+  }
+
+  private void init(ManagedSSTDumpTool sstDumpTool, File sstFile,
+                    ManagedOptions options)
+          throws NativeLibraryNotLoadedException {
+    String[] args = {"--file=" + sstFile.getAbsolutePath(),
+                     "--command=scan"};
+    this.sstDumpToolTask = sstDumpTool.run(args, options);
+    processOutput = new BufferedReader(new InputStreamReader(
+            sstDumpToolTask.getPipedOutput(), StandardCharsets.UTF_8));
+    stdoutString = new StringBuilder();
+    currentMatcher = PATTERN_MATCHER.matcher(stdoutString);
+    charBuffer = new char[8192];
+    open = new AtomicBoolean(true);
+    next();
+  }
+
+  /**
+   * Throws Runtime exception in the case iterator is closed or
+   * the native Dumptool exited with non zero exit value.
+   */
+  private void checkSanityOfProcess() {
+    if (!this.open.get()) {
+      throw new RuntimeException("Iterator has been closed");
+    }
+    if (sstDumpToolTask.getFuture().isDone()
+            && sstDumpToolTask.exitValue() != 0) {
+      throw new RuntimeException("Process Terminated with non zero " +
+              String.format("exit value %d", sstDumpToolTask.exitValue()));
+    }
+  }
+
+  /**
+   *
+   * @return
+   * Throws Runtime Exception in case of SST File read failure
+   */
+
+  @Override
+  public boolean hasNext() {

Review Comment:
   incomplete javadoc?



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1125265191


##########
hadoop-hdds/rocks-native/src/main/native/SSTDumpTool.cpp:
##########
@@ -0,0 +1,49 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.
+ */
+
+#include "org_apache_hadoop_hdds_utils_db_managed_ManagedSSTDumpTool.h"
+#include "rocksdb/options.h"
+#include "rocksdb/sst_dump_tool.h"
+#include "string"
+#include "cplusplus_to_java_convert.h"
+#include "Pipe.h"
+#include "iostream"
+
+jint Java_org_apache_hadoop_hdds_utils_db_managed_ManagedSSTDumpTool_runInternal(JNIEnv *env, jobject obj, jobjectArray argsArray,
+jlong optionsHandle, jlong pipeHandle) {
+    ROCKSDB_NAMESPACE::SSTDumpTool dumpTool;
+    ROCKSDB_NAMESPACE::Options* options = reinterpret_cast<ROCKSDB_NAMESPACE::Options*>(optionsHandle);
+    Pipe* pipe = reinterpret_cast<Pipe*>(pipeHandle);
+    int length = env->GetArrayLength(argsArray);
+    char* args[length + 1];

Review Comment:
   ```suggestion
       char *args[length + 1];
   ```



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1125258870


##########
hadoop-hdds/rocks-native/src/main/native/PipeInputStream.cpp:
##########
@@ -0,0 +1,34 @@
+//
+// Created by Swaminathan Balachandran on 3/2/23.
+//

Review Comment:
   License header



##########
hadoop-hdds/rocks-native/src/main/native/PipeInputStream.cpp:
##########
@@ -0,0 +1,34 @@
+//
+// Created by Swaminathan Balachandran on 3/2/23.
+//
+#include <unistd.h>
+#include <iostream>
+//#include <stdio.h>

Review Comment:
   deadcode



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] swamirishi commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "swamirishi (via GitHub)" <gi...@apache.org>.
swamirishi commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1132815925


##########
hadoop-hdds/rocks-native/src/main/native/SSTDumpTool.cpp:
##########
@@ -0,0 +1,49 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.
+ */
+
+#include "org_apache_hadoop_hdds_utils_db_managed_ManagedSSTDumpTool.h"
+#include "rocksdb/options.h"
+#include "rocksdb/sst_dump_tool.h"
+#include <string>
+#include "cplusplus_to_java_convert.h"
+#include "Pipe.h"
+#include <iostream>
+
+jint Java_org_apache_hadoop_hdds_utils_db_managed_ManagedSSTDumpTool_runInternal(JNIEnv *env, jobject obj,
+ jobjectArray argsArray, jlong optionsHandle, jlong pipeHandle) {
+    ROCKSDB_NAMESPACE::SSTDumpTool dumpTool;
+    ROCKSDB_NAMESPACE::Options *options = reinterpret_cast<ROCKSDB_NAMESPACE::Options *>(optionsHandle);
+    Pipe *pipe = reinterpret_cast<Pipe *>(pipeHandle);
+    int length = env->GetArrayLength(argsArray);
+    char *args[length + 1];
+    args[0] = strdup("./sst_dump");
+    for (int i = 0; i < length; i++) {
+        jstring str_val = (jstring)env->GetObjectArrayElement(argsArray, (jsize)i);
+        char *utf_str = (char *)env->GetStringUTFChars(str_val, JNI_FALSE);
+        args[i + 1] = strdup(utf_str);
+        env->ReleaseStringUTFChars(str_val, utf_str);
+    }
+    FILE *wr = fdopen(pipe->getWriteFd(), "w");
+    int ret = dumpTool.Run(length + 1, args, *options, wr);
+    for (int i = 0; i < length + 1; i++) {
+        free(args[i]);
+    }
+    fclose(wr);
+    pipe->close();

Review Comment:
   no we shouldn;t close options here since option is passed to sst dump tool.



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] swamirishi commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "swamirishi (via GitHub)" <gi...@apache.org>.
swamirishi commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1132815039


##########
hadoop-hdds/rocks-native/src/main/native/PipeInputStream.cpp:
##########
@@ -0,0 +1,48 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.
+ */
+
+#include <unistd.h>
+#include <iostream>
+#include "Pipe.h"
+#include "cplusplus_to_java_convert.h"
+#include "org_apache_hadoop_hdds_utils_db_managed_PipeInputStream.h"
+
+
+jlong Java_org_apache_hadoop_hdds_utils_db_managed_PipeInputStream_newPipe(JNIEnv *, jobject) {
+    Pipe* pipe = new Pipe();
+    return GET_CPLUSPLUS_POINTER(pipe);
+}
+
+jint Java_org_apache_hadoop_hdds_utils_db_managed_PipeInputStream_readInternal(JNIEnv *env, jobject object, jbyteArray jbyteArray, jint capacity, jlong nativeHandle) {
+    int cap_int = capacity;
+    Pipe* pipe = reinterpret_cast<Pipe*>(nativeHandle);
+    jbyte* b = (env)->GetByteArrayElements(jbyteArray, JNI_FALSE);

Review Comment:
   done



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on PR #4315:
URL: https://github.com/apache/ozone/pull/4315#issuecomment-1464560676

   Thanks @swamirishi for the patch. It is a huge effort to put this all together.
   
   Thanks @adoroszlai @prashantpogde for the code review.
   
   > if there are any special build parameters you have in mind for building this like the native library compile for hadoop? ie. `$ mvn package -Pdist,native -DskipTests -Dtar`
   
   Hi @neils-dev , the extra maven parameter has been included in the PR description now, namely `-Drocks_tools_native`.
   
   
   @kerneltime w.r.t. to the build time concern, as tested [above](https://github.com/apache/ozone/pull/4315#issuecomment-1464381647), by default it will not impact the build time. Adding `-Drocks_tools_native` to trigger the RocksDB tool compilation adds another ~1.5 min to the total build time (M1 Pro MBP), which is not required for most other dev work.


-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl merged pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl merged PR #4315:
URL: https://github.com/apache/ozone/pull/4315


-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1129971325


##########
hadoop-hdds/rocks-native/src/main/native/Pipe.h:
##########
@@ -0,0 +1,55 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.
+ */
+
+#ifndef ROCKS_NATIVE_PIPE_H
+#define ROCKS_NATIVE_PIPE_H
+
+#include <stdio.h>
+
+class Pipe {
+    public:
+        static const int READ_FILE_DESCRIPTOR_IDX;
+        static const int WRITE_FILE_DESCRIPTOR_IDX;
+        Pipe();
+        ~Pipe();
+        void close();
+        int getReadFd() {
+            return getPipeFileDescriptorIndex(READ_FILE_DESCRIPTOR_IDX);
+        }
+
+        int getWriteFd() {
+            return getPipeFileDescriptorIndex(WRITE_FILE_DESCRIPTOR_IDX);

Review Comment:
   Try `STDIN_FILENO` and `STDOUT_FILENO`



##########
hadoop-hdds/rocks-native/src/main/native/Pipe.cpp:
##########
@@ -0,0 +1,37 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.
+ */
+
+#include "Pipe.h"
+#include <unistd.h>
+
+const int Pipe::READ_FILE_DESCRIPTOR_IDX = 0;
+const int Pipe::WRITE_FILE_DESCRIPTOR_IDX = 1;
+
+Pipe::Pipe() {
+    pipe(p);
+    open = true;
+}
+
+Pipe::~Pipe() {
+    ::close(p[0]);
+    ::close(p[1]);

Review Comment:
   Actually, `unistd.h` should have `STDIN_FILENO`, `STDOUT_FILENO` and `STDERR_FILENO` already. Use those instead.



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1129959980


##########
hadoop-hdds/rocks-native/src/main/native/Pipe.cpp:
##########
@@ -0,0 +1,37 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.
+ */
+
+#include "Pipe.h"
+#include <unistd.h>
+
+const int Pipe::READ_FILE_DESCRIPTOR_IDX = 0;
+const int Pipe::WRITE_FILE_DESCRIPTOR_IDX = 1;
+
+Pipe::Pipe() {
+    pipe(p);
+    open = true;
+}
+
+Pipe::~Pipe() {
+    ::close(p[0]);
+    ::close(p[1]);

Review Comment:
   ~~Use `READ_FILE_DESCRIPTOR_IDX` and `WRITE_FILE_DESCRIPTOR_IDX` defined right above?~~



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] swamirishi commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "swamirishi (via GitHub)" <gi...@apache.org>.
swamirishi commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1129992689


##########
hadoop-hdds/rocks-native/src/main/native/SSTDumpTool.cpp:
##########
@@ -0,0 +1,49 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.
+ */
+
+#include "org_apache_hadoop_hdds_utils_db_managed_ManagedSSTDumpTool.h"
+#include "rocksdb/options.h"
+#include "rocksdb/sst_dump_tool.h"
+#include <string>
+#include "cplusplus_to_java_convert.h"
+#include "Pipe.h"
+#include <iostream>
+
+jint Java_org_apache_hadoop_hdds_utils_db_managed_ManagedSSTDumpTool_runInternal(JNIEnv *env, jobject obj,
+ jobjectArray argsArray, jlong optionsHandle, jlong pipeHandle) {
+    ROCKSDB_NAMESPACE::SSTDumpTool dumpTool;
+    ROCKSDB_NAMESPACE::Options *options = reinterpret_cast<ROCKSDB_NAMESPACE::Options *>(optionsHandle);
+    Pipe *pipe = reinterpret_cast<Pipe *>(pipeHandle);
+    int length = env->GetArrayLength(argsArray);
+    char *args[length + 1];
+    args[0] = strdup("./sst_dump");
+    for (int i = 0; i < length; i++) {
+        jstring str_val = (jstring)env->GetObjectArrayElement(argsArray, (jsize)i);
+        char *utf_str = (char *)env->GetStringUTFChars(str_val, JNI_FALSE);
+        args[i + 1] = strdup(utf_str);
+        env->ReleaseStringUTFChars(str_val, utf_str);
+    }
+    FILE *wr = fdopen(pipe->getWriteFd(), "w");
+    int ret = dumpTool.Run(length + 1, args, *options, wr);
+    for (int i = 0; i < length + 1; i++) {
+        free(args[i]);
+    }
+    fclose(wr);
+    pipe->close();

Review Comment:
   we shouldn't close options here as we are passing the already created options to sst dump tool constructor. I did this deliberately.



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1131610528


##########
hadoop-hdds/rocks-native/src/main/native/cplusplus_to_java_convert.h:
##########
@@ -0,0 +1,50 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.
+ */
+

Review Comment:
   ok. this looks like a pick-one situation here. might be fine.



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] swamirishi commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "swamirishi (via GitHub)" <gi...@apache.org>.
swamirishi commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1130009833


##########
hadoop-hdds/rocks-native/src/main/native/cplusplus_to_java_convert.h:
##########
@@ -0,0 +1,50 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.
+ */
+

Review Comment:
   RocksDB has apache2.0 license. So would that be sufficient?



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] swamirishi commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "swamirishi (via GitHub)" <gi...@apache.org>.
swamirishi commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1128827254


##########
hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/MessageQueue.java:
##########
@@ -0,0 +1,188 @@
+/**
+ * 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;
+
+import org.apache.commons.io.input.buffer.CircularByteBuffer;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.concurrent.Callable;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * Message Queue to Pipe output from one output stream to another inputstream.
+ */
+public class MessageQueue {
+  private CircularByteBuffer byteBuffer;
+  private AtomicBoolean isRunning;
+
+  private MessageOutputStream messageOutputStream;
+  private MessageInputStream messageInputStream;
+
+  public MessageQueue(int bufferSize, long pollIntervalMillis) {
+    this.pollIntervalMillis = pollIntervalMillis;
+    init(bufferSize);
+  }
+
+  private void init(int bufferSize) {
+    this.byteBuffer = new CircularByteBuffer(bufferSize);
+    this.isRunning = new AtomicBoolean(false);
+    this.messageInputStream = new MessageInputStream(this);
+    this.messageOutputStream = new MessageOutputStream(this);
+  }
+
+  public void start() {
+    this.isRunning.set(true);
+  }
+
+  public void stop() {
+    this.isRunning.set(false);
+  }
+
+  public MessageOutputStream getMessageOutputStream() {
+    return messageOutputStream;
+  }
+
+  public MessageInputStream getMessageInputStream() {
+    return messageInputStream;
+  }
+
+  private long pollIntervalMillis;
+
+  public boolean isRunning() {
+    return isRunning.get();
+  }
+
+  private long getPollIntervalMillis() {
+    return pollIntervalMillis;
+  }
+
+  private boolean hasSpace(int requiredLength) {
+    return this.byteBuffer.hasSpace(requiredLength);
+  }
+
+  private boolean hasBytes() {
+    return this.byteBuffer.hasBytes();
+  }
+
+  private int getCurrentNumberOfBytes() {
+    return this.byteBuffer.getCurrentNumberOfBytes();
+  }
+
+  private void add(byte[] b, int off, int len) {
+    this.byteBuffer.add(b, off, len);
+  }
+
+  private int read(byte[] b, int off, int len) {
+    this.byteBuffer.read(b, off, len);
+    return len;
+  }
+
+  private static  <T> T callWithLock(Lock lock, Callable<T> callable)
+          throws Exception {
+    lock.lock();
+    try {
+      return callable.call();
+    } finally {
+      lock.unlock();
+    }
+  }
+  private static final class MessageOutputStream extends OutputStream {
+
+    private MessageQueue messageQueue;
+    private Lock writeLock;
+
+    private MessageOutputStream(MessageQueue messageQueue) {
+      this.messageQueue = messageQueue;
+      this.writeLock = new ReentrantLock();
+
+    }
+
+    private void waitForBytes(int requiredLength) throws InterruptedException {
+      while (!this.messageQueue.hasSpace(requiredLength)) {
+        Thread.sleep(this.messageQueue.getPollIntervalMillis());
+      }
+    }
+
+
+
+    @Override

Review Comment:
   removed class not required



##########
hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/MessageQueue.java:
##########
@@ -0,0 +1,188 @@
+/**
+ * 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;
+
+import org.apache.commons.io.input.buffer.CircularByteBuffer;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.concurrent.Callable;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * Message Queue to Pipe output from one output stream to another inputstream.
+ */
+public class MessageQueue {
+  private CircularByteBuffer byteBuffer;
+  private AtomicBoolean isRunning;
+
+  private MessageOutputStream messageOutputStream;
+  private MessageInputStream messageInputStream;
+
+  public MessageQueue(int bufferSize, long pollIntervalMillis) {
+    this.pollIntervalMillis = pollIntervalMillis;
+    init(bufferSize);
+  }
+
+  private void init(int bufferSize) {
+    this.byteBuffer = new CircularByteBuffer(bufferSize);
+    this.isRunning = new AtomicBoolean(false);
+    this.messageInputStream = new MessageInputStream(this);
+    this.messageOutputStream = new MessageOutputStream(this);
+  }
+
+  public void start() {
+    this.isRunning.set(true);
+  }
+
+  public void stop() {
+    this.isRunning.set(false);
+  }
+
+  public MessageOutputStream getMessageOutputStream() {
+    return messageOutputStream;
+  }
+
+  public MessageInputStream getMessageInputStream() {
+    return messageInputStream;
+  }
+
+  private long pollIntervalMillis;
+
+  public boolean isRunning() {
+    return isRunning.get();
+  }
+
+  private long getPollIntervalMillis() {
+    return pollIntervalMillis;
+  }
+
+  private boolean hasSpace(int requiredLength) {
+    return this.byteBuffer.hasSpace(requiredLength);
+  }
+
+  private boolean hasBytes() {
+    return this.byteBuffer.hasBytes();
+  }
+
+  private int getCurrentNumberOfBytes() {
+    return this.byteBuffer.getCurrentNumberOfBytes();
+  }
+
+  private void add(byte[] b, int off, int len) {
+    this.byteBuffer.add(b, off, len);
+  }
+
+  private int read(byte[] b, int off, int len) {
+    this.byteBuffer.read(b, off, len);
+    return len;
+  }
+
+  private static  <T> T callWithLock(Lock lock, Callable<T> callable)
+          throws Exception {
+    lock.lock();
+    try {
+      return callable.call();
+    } finally {
+      lock.unlock();
+    }
+  }
+  private static final class MessageOutputStream extends OutputStream {
+
+    private MessageQueue messageQueue;
+    private Lock writeLock;
+
+    private MessageOutputStream(MessageQueue messageQueue) {
+      this.messageQueue = messageQueue;
+      this.writeLock = new ReentrantLock();
+
+    }
+
+    private void waitForBytes(int requiredLength) throws InterruptedException {
+      while (!this.messageQueue.hasSpace(requiredLength)) {
+        Thread.sleep(this.messageQueue.getPollIntervalMillis());
+      }
+    }
+
+
+
+    @Override
+    public void write(int b) throws IOException {
+      this.write(new byte[]{(byte) b});
+    }
+
+    @Override
+    public void write(byte[] b, int off, int len) throws IOException {
+      try {
+        if (!this.messageQueue.isRunning()) {
+          throw new IOException("Message Queue is Closed");
+        }
+        waitForBytes(len);
+        callWithLock(this.writeLock, () -> {
+          waitForBytes(len);
+          this.messageQueue.add(b, off, len);
+          return true;
+        });
+      } catch (Exception e) {
+        throw new IOException(e);
+      }
+    }
+  }
+  private static final class MessageInputStream extends InputStream {

Review Comment:
   Removed class not required.



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] swamirishi commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "swamirishi (via GitHub)" <gi...@apache.org>.
swamirishi commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1128829477


##########
hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/managed/ManagedRocksObject.java:
##########
@@ -0,0 +1,35 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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 org.rocksdb.RocksObject;
+/**
+ * Managed RocksObject.
+ */
+public abstract class ManagedRocksObject extends RocksObject {

Review Comment:
   removed



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1131713860


##########
hadoop-hdds/rocks-native/src/main/java/org/apache/hadoop/hdds/utils/db/managed/ManagedSSTDumpIterator.java:
##########
@@ -0,0 +1,239 @@
+/**
+ * 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 org.apache.hadoop.hdds.utils.NativeLibraryNotLoadedException;
+import org.eclipse.jetty.io.RuntimeIOException;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
+import java.util.Iterator;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * Iterator to Parse output of RocksDBSSTDumpTool.
+ */
+public class ManagedSSTDumpIterator implements
+        Iterator<ManagedSSTDumpIterator.KeyValue>, AutoCloseable {
+  private static final String SST_DUMP_TOOL_CLASS =
+          "org.apache.hadoop.hdds.utils.db.managed.ManagedSSTDumpTool";
+  private static final String PATTERN_REGEX =
+          "'([^=>]+)' seq:([0-9]+), type:([0-9]+) => ";
+
+  public static final int PATTERN_KEY_GROUP_NUMBER = 1;
+  public static final int PATTERN_SEQ_GROUP_NUMBER = 2;
+  public static final int PATTERN_TYPE_GROUP_NUMBER = 3;
+  private static final Pattern PATTERN_MATCHER =
+          Pattern.compile(PATTERN_REGEX);
+  private BufferedReader processOutput;
+  private StringBuilder stdoutString;
+
+  private Matcher currentMatcher;
+  private int prevMatchEndIndex;
+  private KeyValue currentKey;
+  private char[] charBuffer;
+  private KeyValue nextKey;
+
+  private long pollIntervalMillis;
+  private ManagedSSTDumpTool.SSTDumpToolTask sstDumpToolTask;
+  private Lock lock;
+  private AtomicBoolean open;
+
+
+  public ManagedSSTDumpIterator(ManagedSSTDumpTool sstDumpTool,
+                                String sstFilePath,
+                                ManagedOptions options,
+                                long pollIntervalMillis) throws IOException,
+          NativeLibraryNotLoadedException {
+    File sstFile = new File(sstFilePath);
+    if (!sstFile.exists() || !sstFile.isFile()) {
+      throw new IOException(String.format("Invalid SST File Path : %s",
+              sstFile.getAbsolutePath()));
+    }
+    this.pollIntervalMillis = pollIntervalMillis;
+    this.lock = new ReentrantLock();
+    init(sstDumpTool, sstFile, options);
+  }
+
+  private void init(ManagedSSTDumpTool sstDumpTool, File sstFile,
+                    ManagedOptions options)
+          throws NativeLibraryNotLoadedException {
+    String[] args = {"--file=" + sstFile.getAbsolutePath(),
+                     "--command=scan"};
+    this.sstDumpToolTask = sstDumpTool.run(args, options);
+    processOutput = new BufferedReader(new InputStreamReader(
+            sstDumpToolTask.getPipedOutput(), StandardCharsets.UTF_8));
+    stdoutString = new StringBuilder();
+    currentMatcher = PATTERN_MATCHER.matcher(stdoutString);
+    charBuffer = new char[8192];
+    open = new AtomicBoolean(true);
+    next();
+  }
+
+  /**
+   * Throws Runtime exception in the case iterator is closed or
+   * the native Dumptool exited with non zero exit value.
+   */
+  private void checkSanityOfProcess() {
+    if (!this.open.get()) {
+      throw new RuntimeException("Iterator has been closed");
+    }
+    if (sstDumpToolTask.getFuture().isDone()
+            && sstDumpToolTask.exitValue() != 0) {
+      throw new RuntimeException("Process Terminated with non zero " +
+              String.format("exit value %d", sstDumpToolTask.exitValue()));
+    }
+  }
+
+  /**
+   * Checks the status of the process & sees if there is another record.
+   * @return True if next exists & false otherwise
+   * Throws Runtime Exception in case of SST File read failure
+   */
+
+  @Override
+  public boolean hasNext() {
+    checkSanityOfProcess();
+    return nextKey != null;
+  }
+
+  /**
+   * Returns the next record from SSTDumpTool.
+   * @return next Key
+   * Throws Runtime Exception incase of failure.
+   */
+  @Override
+  public KeyValue next() {
+    lock.lock();
+    try {
+      checkSanityOfProcess();
+      currentKey = nextKey;
+      nextKey = null;
+      while (!currentMatcher.find()) {
+        try {
+          if (prevMatchEndIndex != 0) {
+            stdoutString = new StringBuilder(stdoutString.substring(
+                    prevMatchEndIndex, stdoutString.length()));
+            prevMatchEndIndex = 0;
+            currentMatcher = PATTERN_MATCHER.matcher(stdoutString);
+          }
+          Thread.sleep(pollIntervalMillis);
+          int numberOfCharsRead = processOutput.read(charBuffer);
+          if (numberOfCharsRead < 0) {
+            if (currentKey != null) {
+              currentKey.setValue(stdoutString.substring(0,
+                      Math.max(stdoutString.length() - 1, 0)));
+            }
+            return currentKey;
+          }
+          stdoutString.append(charBuffer, 0, numberOfCharsRead);
+          currentMatcher.reset();
+        } catch (IOException | InterruptedException e) {
+          throw new RuntimeIOException(e);
+        }
+      }
+      if (currentKey != null) {
+        currentKey.setValue(stdoutString.substring(prevMatchEndIndex,
+                currentMatcher.start() - 1));
+      }
+      prevMatchEndIndex = currentMatcher.end();
+      nextKey =  new KeyValue(

Review Comment:
   nit
   ```suggestion
         nextKey = new KeyValue(
   ```



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1123910142


##########
hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/MessageQueue.java:
##########
@@ -0,0 +1,188 @@
+/**
+ * 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;
+
+import org.apache.commons.io.input.buffer.CircularByteBuffer;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.concurrent.Callable;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * Message Queue to Pipe output from one output stream to another inputstream.
+ */
+public class MessageQueue {
+  private CircularByteBuffer byteBuffer;
+  private AtomicBoolean isRunning;
+
+  private MessageOutputStream messageOutputStream;
+  private MessageInputStream messageInputStream;
+
+  public MessageQueue(int bufferSize, long pollIntervalMillis) {
+    this.pollIntervalMillis = pollIntervalMillis;
+    init(bufferSize);
+  }
+
+  private void init(int bufferSize) {
+    this.byteBuffer = new CircularByteBuffer(bufferSize);
+    this.isRunning = new AtomicBoolean(false);
+    this.messageInputStream = new MessageInputStream(this);
+    this.messageOutputStream = new MessageOutputStream(this);
+  }
+
+  public void start() {
+    this.isRunning.set(true);
+  }
+
+  public void stop() {
+    this.isRunning.set(false);
+  }
+
+  public MessageOutputStream getMessageOutputStream() {
+    return messageOutputStream;
+  }
+
+  public MessageInputStream getMessageInputStream() {
+    return messageInputStream;
+  }
+
+  private long pollIntervalMillis;
+
+  public boolean isRunning() {
+    return isRunning.get();
+  }
+
+  private long getPollIntervalMillis() {
+    return pollIntervalMillis;
+  }
+
+  private boolean hasSpace(int requiredLength) {
+    return this.byteBuffer.hasSpace(requiredLength);
+  }
+
+  private boolean hasBytes() {
+    return this.byteBuffer.hasBytes();
+  }
+
+  private int getCurrentNumberOfBytes() {
+    return this.byteBuffer.getCurrentNumberOfBytes();
+  }
+
+  private void add(byte[] b, int off, int len) {
+    this.byteBuffer.add(b, off, len);
+  }
+
+  private int read(byte[] b, int off, int len) {
+    this.byteBuffer.read(b, off, len);
+    return len;
+  }
+
+  private static  <T> T callWithLock(Lock lock, Callable<T> callable)
+          throws Exception {
+    lock.lock();
+    try {
+      return callable.call();
+    } finally {
+      lock.unlock();
+    }
+  }
+  private static final class MessageOutputStream extends OutputStream {

Review Comment:
   Also, add a javadoc for the class
   
   ```suggestion
   
     private static final class MessageOutputStream extends OutputStream {
   ```



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1125271697


##########
hadoop-hdds/rocks-native/src/CMakeLists.txt:
##########
@@ -0,0 +1,78 @@
+#
+# 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
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# 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.
+#
+
+#
+# CMake configuration.
+#
+
+cmake_minimum_required(VERSION 3.1 FATAL_ERROR)
+set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -fPIC")
+set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -fPIC")
+project(ozone_native)
+set(CMAKE_BUILD_TYPE Release)
+find_package(JNI REQUIRED)
+include_directories(${JNI_INCLUDE_DIRS})
+set(CMAKE_CXX_STANDARD ${CMAKE_STANDARDS})
+
+set(linked_libraries "")
+if(NOT GENERATED_JAVAH)
+    message(FATAL_ERROR "You must set the CMake variable GENERATED_JAVAH")
+endif()
+include_directories(${GENERATED_JAVAH})
+if(${SST_DUMP_INCLUDE})
+    include_directories(${ROCKSDB_HEADERS})
+    set(SOURCE_FILES ${NATIVE_DIR}/SSTDumpTool.cpp ${NATIVE_DIR}/PipeInputStream.cpp ${NATIVE_DIR}/Pipe.h ${NATIVE_DIR}/Pipe.cpp ${NATIVE_DIR}/cplusplus_to_java_convert.h)
+    ADD_LIBRARY(rocksdb STATIC IMPORTED)
+    set_target_properties(
+            rocksdb
+            PROPERTIES
+            IMPORTED_LOCATION ${ROCKSDB_LIB}/librocksdb_debug.a)
+    ADD_LIBRARY(rocks_tools STATIC IMPORTED)
+    set_target_properties(
+            rocks_tools
+            PROPERTIES
+            IMPORTED_LOCATION ${ROCKSDB_LIB}/librocksdb_tools_debug.a)
+    ADD_LIBRARY(bz2 STATIC IMPORTED)
+    set_target_properties(
+            bz2
+            PROPERTIES
+            IMPORTED_LOCATION ${BZIP2_LIB}/libbz2.a)
+    ADD_LIBRARY(zlib STATIC IMPORTED)
+    set_target_properties(
+            zlib
+            PROPERTIES
+            IMPORTED_LOCATION ${ZLIB_LIB}/libz.a)
+    ADD_LIBRARY(lz4 STATIC IMPORTED)
+    set_target_properties(
+            lz4
+            PROPERTIES
+            IMPORTED_LOCATION ${LZ4_LIB}/liblz4.a)
+    ADD_LIBRARY(snappy STATIC IMPORTED)
+    set_target_properties(
+            snappy
+            PROPERTIES
+            IMPORTED_LOCATION ${SNAPPY_LIB}/libsnappy.a)
+    ADD_LIBRARY(zstd STATIC IMPORTED)
+        set_target_properties(
+                zstd
+                PROPERTIES
+                IMPORTED_LOCATION ${ZSTD_LIB}/libzstd.a)
+    set(linked_libraries ${linked_libraries} bz2 zlib rocks_tools rocksdb lz4 snappy zstd)
+endif()
+add_library(ozone_rocksdb_tools SHARED ${SOURCE_FILES})
+target_link_libraries(ozone_rocksdb_tools ${linked_libraries})

Review Comment:
   newline
   
   ```suggestion
   target_link_libraries(ozone_rocksdb_tools ${linked_libraries})
   
   ```



##########
hadoop-hdds/rocks-native/src/main/java/org/apache/hadoop/hdds/utils/NativeLibraryLoader.java:
##########
@@ -0,0 +1,138 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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;
+

Review Comment:
   ```suggestion
   ```



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] prashantpogde commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "prashantpogde (via GitHub)" <gi...@apache.org>.
prashantpogde commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1123777887


##########
hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/HddsServerUtil.java:
##########
@@ -78,23 +83,19 @@
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_HEARTBEAT_LOG_WARN_DEFAULT;
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_HEARTBEAT_LOG_WARN_INTERVAL_COUNT;
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL;
-import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_HEARTBEAT_RPC_TIMEOUT;
-import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_HEARTBEAT_RPC_TIMEOUT_DEFAULT;
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_HEARTBEAT_RPC_RETRY_COUNT;
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_HEARTBEAT_RPC_RETRY_COUNT_DEFAULT;
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_HEARTBEAT_RPC_RETRY_INTERVAL;
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_HEARTBEAT_RPC_RETRY_INTERVAL_DEFAULT;
-import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_STALENODE_INTERVAL;
-import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_STALENODE_INTERVAL_DEFAULT;
+import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_HEARTBEAT_RPC_TIMEOUT;
+import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_HEARTBEAT_RPC_TIMEOUT_DEFAULT;
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_INFO_WAIT_DURATION;
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_INFO_WAIT_DURATION_DEFAULT;
+import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_STALENODE_INTERVAL;

Review Comment:
   Why is the order of imports changed ?



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1129946181


##########
hadoop-hdds/rocks-native/src/main/java/org/apache/hadoop/hdds/utils/db/managed/ManagedSSTDumpIterator.java:
##########
@@ -0,0 +1,238 @@
+/**
+ * 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 org.apache.hadoop.hdds.utils.NativeLibraryNotLoadedException;
+import org.eclipse.jetty.io.RuntimeIOException;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
+import java.util.Iterator;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * Iterator to Parse output of RocksDBSSTDumpTool.
+ */
+public class ManagedSSTDumpIterator implements
+        Iterator<ManagedSSTDumpIterator.KeyValue>, AutoCloseable {
+  private static final String SST_DUMP_TOOL_CLASS =
+          "org.apache.hadoop.hdds.utils.db.managed.ManagedSSTDumpTool";
+  private static final String PATTERN_REGEX =
+          "'([^=>]+)' seq:([0-9]+), type:([0-9]+) =>";
+
+  public static final int PATTERN_KEY_GROUP_NUMBER = 1;
+  public static final int PATTERN_SEQ_GROUP_NUMBER = 2;
+  public static final int PATTERN_TYPE_GROUP_NUMBER = 3;
+  private static final Pattern PATTERN_MATCHER =
+          Pattern.compile(PATTERN_REGEX);
+  private BufferedReader processOutput;
+  private StringBuilder stdoutString;
+
+  private Matcher currentMatcher;
+  private int prevMatchEndIndex;
+  private KeyValue currentKey;
+  private char[] charBuffer;
+  private KeyValue nextKey;
+
+  private long pollIntervalMillis;
+  private ManagedSSTDumpTool.SSTDumpToolTask sstDumpToolTask;
+  private Lock lock;
+  private AtomicBoolean open;
+
+
+  public ManagedSSTDumpIterator(ManagedSSTDumpTool sstDumpTool,
+                                String sstFilePath,
+                                ManagedOptions options,
+                                long pollIntervalMillis) throws IOException,
+          NativeLibraryNotLoadedException {
+    File sstFile = new File(sstFilePath);
+    if (!sstFile.exists() || !sstFile.isFile()) {
+      throw new IOException(String.format("Invalid SST File Path : %s",
+              sstFile.getAbsolutePath()));
+    }
+    this.pollIntervalMillis = pollIntervalMillis;
+    this.lock = new ReentrantLock();
+    init(sstDumpTool, sstFile, options);
+  }
+
+  private void init(ManagedSSTDumpTool sstDumpTool, File sstFile,
+                    ManagedOptions options)
+          throws NativeLibraryNotLoadedException {
+    String[] args = {"--file=" + sstFile.getAbsolutePath(),
+                     "--command=scan"};
+    this.sstDumpToolTask = sstDumpTool.run(args, options);
+    processOutput = new BufferedReader(new InputStreamReader(
+            sstDumpToolTask.getPipedOutput(), StandardCharsets.UTF_8));
+    stdoutString = new StringBuilder();
+    currentMatcher = PATTERN_MATCHER.matcher(stdoutString);
+    charBuffer = new char[8192];
+    open = new AtomicBoolean(true);
+    next();
+  }
+
+  /**
+   * Throws Runtime exception in the case iterator is closed or
+   * the native Dumptool exited with non zero exit value.
+   */
+  private void checkSanityOfProcess() {
+    if (!this.open.get()) {
+      throw new RuntimeException("Iterator has been closed");
+    }
+    if (sstDumpToolTask.getFuture().isDone()
+            && sstDumpToolTask.exitValue() != 0) {
+      throw new RuntimeException("Process Terminated with non zero " +
+              String.format("exit value %d", sstDumpToolTask.exitValue()));
+    }
+  }
+
+  /**
+   *
+   * @return
+   * Throws Runtime Exception in case of SST File read failure
+   */
+
+  @Override
+  public boolean hasNext() {

Review Comment:
   Need to fix javadoc



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1129972898


##########
hadoop-hdds/rocks-native/src/main/native/Pipe.h:
##########
@@ -0,0 +1,55 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.
+ */
+
+#ifndef ROCKS_NATIVE_PIPE_H
+#define ROCKS_NATIVE_PIPE_H
+
+#include <stdio.h>
+
+class Pipe {
+    public:
+        static const int READ_FILE_DESCRIPTOR_IDX;
+        static const int WRITE_FILE_DESCRIPTOR_IDX;
+        Pipe();
+        ~Pipe();
+        void close();
+        int getReadFd() {
+            return getPipeFileDescriptorIndex(READ_FILE_DESCRIPTOR_IDX);
+        }
+
+        int getWriteFd() {
+            return getPipeFileDescriptorIndex(WRITE_FILE_DESCRIPTOR_IDX);

Review Comment:
   Relevant to https://github.com/apache/ozone/pull/4315/files#r1129959980



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] swamirishi commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "swamirishi (via GitHub)" <gi...@apache.org>.
swamirishi commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1129987832


##########
hadoop-hdds/rocks-native/src/main/java/org/apache/hadoop/hdds/utils/package-info.java:
##########
@@ -0,0 +1,24 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.
+ */
+
+/**
+ * Annotation processors used at compile time by the Ozone project to validate
+ * internal annotations and related code as needed, if needed.
+ */
+
+package org.apache.hadoop.hdds.utils;

Review Comment:
   there is a new line at the end. The PR doesn't seem to show 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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] swamirishi commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "swamirishi (via GitHub)" <gi...@apache.org>.
swamirishi commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1129914454


##########
hadoop-hdds/rocks-native/pom.xml:
##########
@@ -0,0 +1,405 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed 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
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+  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. See accompanying LICENSE file.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <parent>
+        <artifactId>hdds</artifactId>
+        <groupId>org.apache.ozone</groupId>
+        <version>1.4.0-SNAPSHOT</version>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+    <name>Apache Ozone HDDS Rocks</name>

Review Comment:
   done



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1129952617


##########
hadoop-hdds/rocks-native/src/main/java/org/apache/hadoop/hdds/utils/db/managed/ManagedSSTDumpTool.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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 org.apache.hadoop.hdds.utils.NativeLibraryLoader;
+import org.apache.hadoop.hdds.utils.NativeLibraryNotLoadedException;
+
+import java.util.Map;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+
+import static org.apache.hadoop.hdds.utils.NativeConstants.ROCKS_TOOLS_NATIVE_LIBRARY_NAME;
+
+/**
+ * JNI for RocksDB SSTDumpTool. Pipes the output to an output stream
+ */
+public class ManagedSSTDumpTool {
+
+  static {
+    NativeLibraryLoader.getInstance()
+            .loadLibrary(ROCKS_TOOLS_NATIVE_LIBRARY_NAME);
+  }
+  private int bufferCapacity;
+  private ExecutorService executorService;
+
+  public ManagedSSTDumpTool(ExecutorService executorService,
+                            int bufferCapacity)
+          throws NativeLibraryNotLoadedException {
+    if (!NativeLibraryLoader.getInstance()
+            .isLibraryLoaded(ROCKS_TOOLS_NATIVE_LIBRARY_NAME)) {
+      throw new NativeLibraryNotLoadedException(
+              ROCKS_TOOLS_NATIVE_LIBRARY_NAME);
+    }
+    this.bufferCapacity = bufferCapacity;
+    this.executorService = executorService;
+  }
+
+  public SSTDumpToolTask run(String[] args, ManagedOptions options)
+          throws NativeLibraryNotLoadedException {
+    PipeInputStream pipeInputStream = new PipeInputStream(bufferCapacity);
+    return new SSTDumpToolTask(this.executorService.submit(() ->
+            this.runInternal(args, options.getNativeHandle(),
+            pipeInputStream.getNativeHandle())), pipeInputStream);
+  }
+
+  public SSTDumpToolTask run(Map<String, String> args, ManagedOptions options)
+          throws NativeLibraryNotLoadedException {
+    return this.run(args.entrySet().stream().map(e -> "--"
+            + (e.getValue() == null || e.getValue().isEmpty() ? e.getKey() :
+            e.getKey() + "=" + e.getValue())).toArray(String[]::new), options);
+  }
+
+  private native int runInternal(String[] args, long optionsHandle,
+                                 long pipeHandle);
+
+  /**
+   * Class holding piped output of SST Dumptool & future of command.
+   */
+  static class SSTDumpToolTask {
+    private Future<Integer> future;
+    private PipeInputStream pipedOutput;
+
+    SSTDumpToolTask(Future<Integer> future, PipeInputStream pipedOutput) {
+      this.future = future;
+      this.pipedOutput = pipedOutput;
+    }
+
+    public Future<Integer> getFuture() {
+      return future;
+    }
+
+    public PipeInputStream getPipedOutput() {
+      return pipedOutput;
+    }
+
+    public int exitValue() {
+      if (this.future.isDone()) {
+        try {
+          return future.get();
+        } catch (InterruptedException | ExecutionException e) {
+          return 1;
+        }
+      }
+      return 0;
+    }
+  }
+}

Review Comment:
   new line required as of HDDS-8077
   
   ```suggestion
   }
   
   ```



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] swamirishi commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "swamirishi (via GitHub)" <gi...@apache.org>.
swamirishi commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1129999491


##########
hadoop-hdds/rocks-native/src/main/native/SSTDumpTool.cpp:
##########
@@ -0,0 +1,49 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.
+ */
+
+#include "org_apache_hadoop_hdds_utils_db_managed_ManagedSSTDumpTool.h"
+#include "rocksdb/options.h"
+#include "rocksdb/sst_dump_tool.h"
+#include <string>
+#include "cplusplus_to_java_convert.h"
+#include "Pipe.h"
+#include <iostream>
+
+jint Java_org_apache_hadoop_hdds_utils_db_managed_ManagedSSTDumpTool_runInternal(JNIEnv *env, jobject obj,
+ jobjectArray argsArray, jlong optionsHandle, jlong pipeHandle) {
+    ROCKSDB_NAMESPACE::SSTDumpTool dumpTool;
+    ROCKSDB_NAMESPACE::Options *options = reinterpret_cast<ROCKSDB_NAMESPACE::Options *>(optionsHandle);
+    Pipe *pipe = reinterpret_cast<Pipe *>(pipeHandle);
+    int length = env->GetArrayLength(argsArray);
+    char *args[length + 1];
+    args[0] = strdup("./sst_dump");
+    for (int i = 0; i < length; i++) {
+        jstring str_val = (jstring)env->GetObjectArrayElement(argsArray, (jsize)i);
+        char *utf_str = (char *)env->GetStringUTFChars(str_val, JNI_FALSE);
+        args[i + 1] = strdup(utf_str);
+        env->ReleaseStringUTFChars(str_val, utf_str);
+    }
+    FILE *wr = fdopen(pipe->getWriteFd(), "w");
+    int ret = dumpTool.Run(length + 1, args, *options, wr);
+    for (int i = 0; i < length + 1; i++) {
+        free(args[i]);
+    }

Review Comment:
   SST dump tool already does this check. Do we need to have another check here? Doesn't just forwarding the request make sense here?



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1129995703


##########
hadoop-hdds/rocks-native/src/main/native/SSTDumpTool.cpp:
##########
@@ -0,0 +1,49 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.
+ */
+
+#include "org_apache_hadoop_hdds_utils_db_managed_ManagedSSTDumpTool.h"
+#include "rocksdb/options.h"
+#include "rocksdb/sst_dump_tool.h"
+#include <string>
+#include "cplusplus_to_java_convert.h"
+#include "Pipe.h"
+#include <iostream>
+
+jint Java_org_apache_hadoop_hdds_utils_db_managed_ManagedSSTDumpTool_runInternal(JNIEnv *env, jobject obj,
+ jobjectArray argsArray, jlong optionsHandle, jlong pipeHandle) {
+    ROCKSDB_NAMESPACE::SSTDumpTool dumpTool;
+    ROCKSDB_NAMESPACE::Options *options = reinterpret_cast<ROCKSDB_NAMESPACE::Options *>(optionsHandle);
+    Pipe *pipe = reinterpret_cast<Pipe *>(pipeHandle);
+    int length = env->GetArrayLength(argsArray);
+    char *args[length + 1];
+    args[0] = strdup("./sst_dump");
+    for (int i = 0; i < length; i++) {
+        jstring str_val = (jstring)env->GetObjectArrayElement(argsArray, (jsize)i);
+        char *utf_str = (char *)env->GetStringUTFChars(str_val, JNI_FALSE);
+        args[i + 1] = strdup(utf_str);
+        env->ReleaseStringUTFChars(str_val, utf_str);
+    }
+    FILE *wr = fdopen(pipe->getWriteFd(), "w");
+    int ret = dumpTool.Run(length + 1, args, *options, wr);
+    for (int i = 0; i < length + 1; i++) {
+        free(args[i]);
+    }

Review Comment:
   array boundry looks fine, as long as `length >= 0`.
   
   Can add a check to return `-1` when `length < 1`.



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1131611751


##########
hadoop-hdds/rocks-native/src/main/native/cplusplus_to_java_convert.h:
##########
@@ -0,0 +1,50 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.
+ */
+

Review Comment:
   Ref: https://issues.apache.org/jira/browse/LEGAL-303?focusedCommentId=16089073&page=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#comment-16089073



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1131833693


##########
hadoop-hdds/rocks-native/src/main/java/org/apache/hadoop/hdds/utils/db/managed/ManagedSSTDumpIterator.java:
##########
@@ -0,0 +1,238 @@
+/**
+ * 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 org.apache.hadoop.hdds.utils.NativeLibraryNotLoadedException;
+import org.eclipse.jetty.io.RuntimeIOException;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
+import java.util.Iterator;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * Iterator to Parse output of RocksDBSSTDumpTool.
+ */
+public class ManagedSSTDumpIterator implements
+        Iterator<ManagedSSTDumpIterator.KeyValue>, AutoCloseable {
+  private static final String SST_DUMP_TOOL_CLASS =
+          "org.apache.hadoop.hdds.utils.db.managed.ManagedSSTDumpTool";
+  private static final String PATTERN_REGEX =
+          "'([^=>]+)' seq:([0-9]+), type:([0-9]+) =>";
+
+  public static final int PATTERN_KEY_GROUP_NUMBER = 1;
+  public static final int PATTERN_SEQ_GROUP_NUMBER = 2;
+  public static final int PATTERN_TYPE_GROUP_NUMBER = 3;
+  private static final Pattern PATTERN_MATCHER =
+          Pattern.compile(PATTERN_REGEX);
+  private BufferedReader processOutput;
+  private StringBuilder stdoutString;
+
+  private Matcher currentMatcher;
+  private int prevMatchEndIndex;
+  private KeyValue currentKey;
+  private char[] charBuffer;
+  private KeyValue nextKey;
+
+  private long pollIntervalMillis;
+  private ManagedSSTDumpTool.SSTDumpToolTask sstDumpToolTask;
+  private Lock lock;
+  private AtomicBoolean open;
+
+
+  public ManagedSSTDumpIterator(ManagedSSTDumpTool sstDumpTool,
+                                String sstFilePath,
+                                ManagedOptions options,
+                                long pollIntervalMillis) throws IOException,
+          NativeLibraryNotLoadedException {
+    File sstFile = new File(sstFilePath);
+    if (!sstFile.exists() || !sstFile.isFile()) {
+      throw new IOException(String.format("Invalid SST File Path : %s",
+              sstFile.getAbsolutePath()));
+    }
+    this.pollIntervalMillis = pollIntervalMillis;
+    this.lock = new ReentrantLock();
+    init(sstDumpTool, sstFile, options);
+  }
+
+  private void init(ManagedSSTDumpTool sstDumpTool, File sstFile,
+                    ManagedOptions options)
+          throws NativeLibraryNotLoadedException {
+    String[] args = {"--file=" + sstFile.getAbsolutePath(),
+                     "--command=scan"};
+    this.sstDumpToolTask = sstDumpTool.run(args, options);
+    processOutput = new BufferedReader(new InputStreamReader(
+            sstDumpToolTask.getPipedOutput(), StandardCharsets.UTF_8));
+    stdoutString = new StringBuilder();
+    currentMatcher = PATTERN_MATCHER.matcher(stdoutString);
+    charBuffer = new char[8192];
+    open = new AtomicBoolean(true);
+    next();
+  }
+
+  /**
+   * Throws Runtime exception in the case iterator is closed or
+   * the native Dumptool exited with non zero exit value.
+   */
+  private void checkSanityOfProcess() {
+    if (!this.open.get()) {
+      throw new RuntimeException("Iterator has been closed");
+    }
+    if (sstDumpToolTask.getFuture().isDone()
+            && sstDumpToolTask.exitValue() != 0) {
+      throw new RuntimeException("Process Terminated with non zero " +
+              String.format("exit value %d", sstDumpToolTask.exitValue()));
+    }
+  }
+
+  /**
+   *
+   * @return
+   * Throws Runtime Exception in case of SST File read failure
+   */
+
+  @Override
+  public boolean hasNext() {
+    checkSanityOfProcess();
+    return nextKey != null;
+  }
+
+  /**
+   *
+   * @return next Key
+   * Throws Runtime Exception incase of failure.
+   */
+  @Override
+  public KeyValue next() {
+    lock.lock();
+    try {
+      checkSanityOfProcess();
+      currentKey = nextKey;
+      nextKey = null;
+      while (!currentMatcher.find()) {
+        try {
+          if (prevMatchEndIndex != 0) {
+            stdoutString = new StringBuilder(stdoutString.substring(
+                    prevMatchEndIndex, stdoutString.length()));
+            prevMatchEndIndex = 0;
+            currentMatcher = PATTERN_MATCHER.matcher(stdoutString);
+          }
+          Thread.sleep(pollIntervalMillis);
+          int numberOfCharsRead = processOutput.read(charBuffer);
+          if (numberOfCharsRead < 0) {
+            if (currentKey != null) {
+              currentKey.setValue(stdoutString.toString());
+            }
+            return currentKey;
+          }
+          stdoutString.append(charBuffer, 0, numberOfCharsRead);
+          currentMatcher.reset();
+        } catch (IOException | InterruptedException e) {
+          throw new RuntimeIOException(e);
+        }
+      }
+      if (currentKey != null) {
+        currentKey.setValue(stdoutString.substring(prevMatchEndIndex,
+                currentMatcher.start()));
+      }
+      prevMatchEndIndex = currentMatcher.end();
+      nextKey =  new KeyValue(
+              currentMatcher.group(PATTERN_KEY_GROUP_NUMBER),
+              currentMatcher.group(PATTERN_SEQ_GROUP_NUMBER),
+              currentMatcher.group(PATTERN_TYPE_GROUP_NUMBER));
+      return currentKey;
+    } finally {
+      lock.unlock();
+    }
+  }
+
+  @Override
+  public synchronized void close() throws Exception {
+    lock.lock();
+    try {
+      if (this.sstDumpToolTask != null) {
+        if (!this.sstDumpToolTask.getFuture().isDone()) {
+          this.sstDumpToolTask.getFuture().cancel(true);
+        }
+        this.processOutput.close();
+      }
+      open.compareAndSet(true, false);
+    } finally {
+      lock.unlock();
+    }
+  }
+
+  @Override
+  protected void finalize() throws Throwable {
+    this.close();
+  }
+
+  /**
+   * Class containing Parsed KeyValue Record from Sst Dumptool output.
+   */
+  public static final class KeyValue {

Review Comment:
   Yup you can move it to another class with a not-so-generic name like `SstDumpKVEntry`.



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1131830542


##########
hadoop-hdds/rocks-native/src/main/native/SSTDumpTool.cpp:
##########
@@ -0,0 +1,49 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.
+ */
+
+#include "org_apache_hadoop_hdds_utils_db_managed_ManagedSSTDumpTool.h"
+#include "rocksdb/options.h"
+#include "rocksdb/sst_dump_tool.h"
+#include <string>
+#include "cplusplus_to_java_convert.h"
+#include "Pipe.h"
+#include <iostream>
+
+jint Java_org_apache_hadoop_hdds_utils_db_managed_ManagedSSTDumpTool_runInternal(JNIEnv *env, jobject obj,
+ jobjectArray argsArray, jlong optionsHandle, jlong pipeHandle) {
+    ROCKSDB_NAMESPACE::SSTDumpTool dumpTool;
+    ROCKSDB_NAMESPACE::Options *options = reinterpret_cast<ROCKSDB_NAMESPACE::Options *>(optionsHandle);
+    Pipe *pipe = reinterpret_cast<Pipe *>(pipeHandle);
+    int length = env->GetArrayLength(argsArray);
+    char *args[length + 1];
+    args[0] = strdup("./sst_dump");
+    for (int i = 0; i < length; i++) {
+        jstring str_val = (jstring)env->GetObjectArrayElement(argsArray, (jsize)i);
+        char *utf_str = (char *)env->GetStringUTFChars(str_val, JNI_FALSE);
+        args[i + 1] = strdup(utf_str);
+        env->ReleaseStringUTFChars(str_val, utf_str);
+    }
+    FILE *wr = fdopen(pipe->getWriteFd(), "w");
+    int ret = dumpTool.Run(length + 1, args, *options, wr);
+    for (int i = 0; i < length + 1; i++) {
+        free(args[i]);
+    }
+    fclose(wr);
+    pipe->close();

Review Comment:
   Could `PipeInputStream` Java object could be garbage collected by JVM before C++ code here finished using it? Looks like it could be?
   
   In other words, is there any guarantee in KVM that objects passed through JNI won't be GC'ed?



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1125255305


##########
hadoop-hdds/rocks-native/src/main/java/org/apache/hadoop/hdds/utils/db/managed/ManagedSSTDumpIterator.java:
##########
@@ -0,0 +1,238 @@
+/**
+ * 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 org.apache.hadoop.hdds.utils.NativeLibraryNotLoadedException;
+import org.eclipse.jetty.io.RuntimeIOException;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
+import java.util.Iterator;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * Iterator to Parse output of RocksDBSSTDumpTool.
+ */
+public class ManagedSSTDumpIterator implements
+        Iterator<ManagedSSTDumpIterator.KeyValue>, AutoCloseable {
+  private static final String SST_DUMP_TOOL_CLASS =
+          "org.apache.hadoop.hdds.utils.db.managed.ManagedSSTDumpTool";
+  private static final String PATTERN_REGEX =
+          "'([^=>]+)' seq:([0-9]+), type:([0-9]+) =>";
+
+  public static final int PATTERN_KEY_GROUP_NUMBER = 1;
+  public static final int PATTERN_SEQ_GROUP_NUMBER = 2;
+  public static final int PATTERN_TYPE_GROUP_NUMBER = 3;
+  private static final Pattern PATTERN_MATCHER =
+          Pattern.compile(PATTERN_REGEX);
+  private BufferedReader processOutput;
+  private StringBuilder stdoutString;
+
+  private Matcher currentMatcher;
+  private int prevMatchEndIndex;
+  private KeyValue currentKey;
+  private char[] charBuffer;
+  private KeyValue nextKey;
+
+  private long pollIntervalMillis;
+  private ManagedSSTDumpTool.SSTDumpToolTask sstDumpToolTask;
+  private Lock lock;
+  private AtomicBoolean open;
+
+
+  public ManagedSSTDumpIterator(ManagedSSTDumpTool sstDumpTool,
+                                String sstFilePath,
+                                ManagedOptions options,
+                                long pollIntervalMillis) throws IOException,
+          NativeLibraryNotLoadedException {
+    File sstFile = new File(sstFilePath);
+    if (!sstFile.exists() || !sstFile.isFile()) {
+      throw new IOException(String.format("Invalid SST File Path : %s",
+              sstFile.getAbsolutePath()));
+    }
+    this.pollIntervalMillis = pollIntervalMillis;
+    this.lock = new ReentrantLock();
+    init(sstDumpTool, sstFile, options);
+  }
+
+  private void init(ManagedSSTDumpTool sstDumpTool, File sstFile,
+                    ManagedOptions options)
+          throws NativeLibraryNotLoadedException {
+    String[] args = {"--file=" + sstFile.getAbsolutePath(),
+                     "--command=scan"};
+    this.sstDumpToolTask = sstDumpTool.run(args, options);
+    processOutput = new BufferedReader(new InputStreamReader(
+            sstDumpToolTask.getPipedOutput(), StandardCharsets.UTF_8));
+    stdoutString = new StringBuilder();
+    currentMatcher = PATTERN_MATCHER.matcher(stdoutString);
+    charBuffer = new char[8192];
+    open = new AtomicBoolean(true);
+    next();
+  }
+
+  /**
+   * Throws Runtime exception in the case iterator is closed or
+   * the native Dumptool exited with non zero exit value.
+   */
+  private void checkSanityOfProcess() {
+    if (!this.open.get()) {
+      throw new RuntimeException("Iterator has been closed");
+    }
+    if (sstDumpToolTask.getFuture().isDone()
+            && sstDumpToolTask.exitValue() != 0) {
+      throw new RuntimeException("Process Terminated with non zero " +
+              String.format("exit value %d", sstDumpToolTask.exitValue()));
+    }
+  }
+
+  /**
+   *
+   * @return
+   * Throws Runtime Exception in case of SST File read failure
+   */
+
+  @Override
+  public boolean hasNext() {
+    checkSanityOfProcess();
+    return nextKey != null;
+  }
+
+  /**
+   *
+   * @return next Key
+   * Throws Runtime Exception incase of failure.
+   */
+  @Override
+  public KeyValue next() {
+    lock.lock();
+    try {
+      checkSanityOfProcess();
+      currentKey = nextKey;
+      nextKey = null;
+      while (!currentMatcher.find()) {
+        try {
+          if (prevMatchEndIndex != 0) {
+            stdoutString = new StringBuilder(stdoutString.substring(
+                    prevMatchEndIndex, stdoutString.length()));
+            prevMatchEndIndex = 0;
+            currentMatcher = PATTERN_MATCHER.matcher(stdoutString);
+          }
+          Thread.sleep(pollIntervalMillis);
+          int numberOfCharsRead = processOutput.read(charBuffer);
+          if (numberOfCharsRead < 0) {
+            if (currentKey != null) {
+              currentKey.setValue(stdoutString.toString());
+            }
+            return currentKey;
+          }
+          stdoutString.append(charBuffer, 0, numberOfCharsRead);
+          currentMatcher.reset();
+        } catch (IOException | InterruptedException e) {
+          throw new RuntimeIOException(e);
+        }
+      }
+      if (currentKey != null) {
+        currentKey.setValue(stdoutString.substring(prevMatchEndIndex,
+                currentMatcher.start()));
+      }
+      prevMatchEndIndex = currentMatcher.end();
+      nextKey =  new KeyValue(
+              currentMatcher.group(PATTERN_KEY_GROUP_NUMBER),
+              currentMatcher.group(PATTERN_SEQ_GROUP_NUMBER),
+              currentMatcher.group(PATTERN_TYPE_GROUP_NUMBER));
+      return currentKey;
+    } finally {
+      lock.unlock();
+    }
+  }
+
+  @Override
+  public synchronized void close() throws Exception {
+    lock.lock();
+    try {
+      if (this.sstDumpToolTask != null) {
+        if (!this.sstDumpToolTask.getFuture().isDone()) {
+          this.sstDumpToolTask.getFuture().cancel(true);
+        }
+        this.processOutput.close();
+      }
+      open.compareAndSet(true, false);
+    } finally {
+      lock.unlock();
+    }
+  }
+
+  @Override
+  protected void finalize() throws Throwable {
+    this.close();
+  }
+
+  /**
+   * Class containing Parsed KeyValue Record from Sst Dumptool output.
+   */
+  public static final class KeyValue {

Review Comment:
   Does this need to be `public`?



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1125258482


##########
hadoop-hdds/rocks-native/src/main/native/Pipe.h:
##########
@@ -0,0 +1,37 @@
+//
+// Created by Swaminathan Balachandran on 3/2/23.
+//
+
+#ifndef UNTITLED_PIPE_H
+#define UNTITLED_PIPE_H
+
+
+#include <stdio.h>
+
+class Pipe {
+    public:
+        Pipe();
+        ~Pipe();
+        void close();
+        int getReadFd() {
+            return p[0];
+        }
+
+        int getWriteFd() {
+            return p[1];

Review Comment:
   Define index 0,1,2 rather than throwing those magic numbers around.



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] swamirishi commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "swamirishi (via GitHub)" <gi...@apache.org>.
swamirishi commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1132815278


##########
hadoop-hdds/rocks-native/src/main/native/PipeInputStream.cpp:
##########
@@ -0,0 +1,48 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.
+ */
+
+#include <unistd.h>
+#include <iostream>
+#include "Pipe.h"
+#include "cplusplus_to_java_convert.h"
+#include "org_apache_hadoop_hdds_utils_db_managed_PipeInputStream.h"
+
+
+jlong Java_org_apache_hadoop_hdds_utils_db_managed_PipeInputStream_newPipe(JNIEnv *, jobject) {
+    Pipe* pipe = new Pipe();
+    return GET_CPLUSPLUS_POINTER(pipe);
+}
+
+jint Java_org_apache_hadoop_hdds_utils_db_managed_PipeInputStream_readInternal(JNIEnv *env, jobject object, jbyteArray jbyteArray, jint capacity, jlong nativeHandle) {
+    int cap_int = capacity;
+    Pipe* pipe = reinterpret_cast<Pipe*>(nativeHandle);
+    jbyte* b = (env)->GetByteArrayElements(jbyteArray, JNI_FALSE);
+    cap_int = read(pipe->getReadFd(), b, cap_int);
+    if (cap_int == 0) {
+        if (!pipe->isOpen()) {
+            cap_int = -1;
+        }
+    }
+    (env)->ReleaseByteArrayElements(jbyteArray, b, 0);
+    return cap_int;
+}
+
+void Java_org_apache_hadoop_hdds_utils_db_managed_PipeInputStream_closeInternal(JNIEnv *env, jobject object, jlong nativeHandle) {
+    delete reinterpret_cast<Pipe*>(nativeHandle);

Review Comment:
   done



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] adoroszlai commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "adoroszlai (via GitHub)" <gi...@apache.org>.
adoroszlai commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1117338245


##########
hadoop-ozone/common/pom.xml:
##########
@@ -107,6 +107,10 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd">
       <groupId>org.junit.jupiter</groupId>
       <artifactId>junit-jupiter-params</artifactId>
     </dependency>
+    <dependency>
+      <groupId>org.apache.ozone</groupId>
+      <artifactId>hdds-server-framework</artifactId>
+    </dependency>

Review Comment:
   `ozone-common` is shared by Ozone client and server code, thus it should not depend on `hdds-server-framework` (server-side stuff).



##########
pom.xml:
##########
@@ -1533,6 +1538,11 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs
         <artifactId>curator-client</artifactId>
         <version>${curator.version}</version>
       </dependency>
+      <dependency>
+        <groupId>com.googlecode.maven-download-plugin</groupId>
+        <artifactId>download-maven-plugin</artifactId>
+        <version>${download-maven-plugin.version}</version>
+      </dependency>

Review Comment:
   I think this belongs to `pluginManagement`, not `dependencyManagement`.
   
   ```
   'build.plugins.plugin.version' for com.googlecode.maven-download-plugin:download-maven-plugin is missing. @ line 43, column 21
   ```



##########
hadoop-hdds/rocks-native/pom.xml:
##########
@@ -0,0 +1,233 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed 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
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+  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. See accompanying LICENSE file.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <parent>
+        <artifactId>hdds</artifactId>
+        <groupId>org.apache.ozone</groupId>
+        <version>1.4.0-SNAPSHOT</version>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+    <name>Apache Ozone HDDS Rocks</name>
+    <artifactId>hdds-rocks-native</artifactId>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.ozone</groupId>
+            <artifactId>hdds-server-framework</artifactId>
+        </dependency>
+    </dependencies>
+    <properties>
+        <maven.compiler.source>8</maven.compiler.source>
+        <maven.compiler.target>8</maven.compiler.target>
+        <cmake.standards>23</cmake.standards>
+        <sstDump.include>true</sstDump.include>
+    </properties>
+
+
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>com.googlecode.maven-download-plugin</groupId>
+                <artifactId>download-maven-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <id>rocksdb source download</id>
+                        <phase>generate-resources</phase>
+                        <goals>
+                            <goal>wget</goal>
+                        </goals>
+                        <configuration>
+                            <url>https://github.com/facebook/rocksdb/archive/refs/tags/v${rocksdb.version}.tar.gz</url>
+                            <outputFileName>rocksdb-v${rocksdb.version}.tar.gz</outputFileName>
+                            <outputDirectory>${project.build.directory}/rocksdb</outputDirectory>
+                        </configuration>
+                    </execution>
+                    <execution>
+                        <id>zlib source download</id>
+                        <phase>generate-resources</phase>
+                        <goals>
+                            <goal>wget</goal>
+                        </goals>
+                        <configuration>
+                            <url>https://zlib.net/zlib-${zlib.version}.tar.gz</url>
+                            <outputFileName>zlib-${zlib.version}.tar.gz</outputFileName>
+                            <outputDirectory>${project.build.directory}/zlib</outputDirectory>
+                        </configuration>
+                    </execution>
+                    <execution>
+                        <id>bzip2 source download</id>
+                        <phase>generate-resources</phase>
+                        <goals>
+                            <goal>wget</goal>
+                        </goals>
+                        <configuration>
+                            <url>https://sourceware.org/pub/bzip2/bzip2-${bzip2.version}.tar.gz</url>
+                            <outputFileName>bzip2-v${bzip2.version}.tar.gz</outputFileName>
+                            <outputDirectory>${project.build.directory}/bzip2</outputDirectory>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+            <plugin>
+                <artifactId>maven-antrun-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <id>unzip-artifact</id>
+                        <phase>generate-resources</phase>
+                        <configuration>
+                            <tasks>
+                                <untar src="${project.build.directory}/rocksdb/rocksdb-v${rocksdb.version}.tar.gz" compression="gzip" dest="${project.build.directory}/rocksdb/" />
+                                <untar src="${project.build.directory}/zlib/zlib-${zlib.version}.tar.gz" compression="gzip" dest="${project.build.directory}/zlib/" />
+                                <untar src="${project.build.directory}/bzip2/bzip2-v${bzip2.version}.tar.gz" compression="gzip" dest="${project.build.directory}/bzip2/" />
+                            </tasks>
+                        </configuration>
+                        <goals>
+                            <goal>run</goal>
+                        </goals>
+                    </execution>
+                    <execution>
+                        <id>build-zlib</id>
+                        <phase>generate-resources</phase>
+                        <configuration>
+                            <tasks>
+                                <chmod file="${project.build.directory}/zlib/zlib-${zlib.version}/configure" perm="775" />
+                                <exec executable="${shell-executable}" dir="${project.build.directory}/zlib/zlib-${zlib.version}" failonerror="true">
+                                    <arg line="./configure"/>
+                                </exec>
+                                <exec executable="make" dir="${project.build.directory}/zlib/zlib-${zlib.version}" failonerror="true"/>
+                            </tasks>
+                        </configuration>
+                        <goals>
+                            <goal>run</goal>
+                        </goals>
+                    </execution>
+                    <execution>
+                        <id>build-bzip2</id>
+                        <phase>generate-resources</phase>
+                        <configuration>
+                            <tasks>
+                                <exec executable="make" dir="${project.build.directory}/bzip2/bzip2-${bzip2.version}" failonerror="true"/>
+                            </tasks>
+                        </configuration>
+                        <goals>
+                            <goal>run</goal>
+                        </goals>
+                    </execution>
+                    <execution>
+                        <id>build-rocksjava</id>
+                        <phase>generate-resources</phase>
+                        <configuration>
+                            <tasks>
+                                <exec executable="chmod" failonerror="true">
+                                    <arg line="-R"/>
+                                    <arg line="775"/>
+                                    <arg line="${project.build.directory}/rocksdb/rocksdb-${rocksdb.version}"/>
+                                </exec>
+                                <exec executable="make" dir="${project.build.directory}/rocksdb/rocksdb-${rocksdb.version}" failonerror="true">
+                                    <arg line="-j16"/>
+                                    <arg line="tools"/>
+                                </exec>>
+                            </tasks>
+                        </configuration>
+                        <goals>
+                            <goal>run</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
+            <plugin>
+                <groupId>org.codehaus.mojo</groupId>
+                <artifactId>native-maven-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <phase>compile</phase>
+                        <goals>
+                            <goal>javah</goal>
+                        </goals>
+                        <configuration>
+                            <javahPath>${env.JAVA_HOME}/bin/javah</javahPath>
+                            <javahClassNames>
+                                <javahClassName>org.apache.hadoop.hdds.utils.db.managed.ManagedSSTDumpTool</javahClassName>
+                            </javahClassNames>
+                            <javahOutputDirectory>${project.build.directory}/native/javah</javahOutputDirectory>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+            <plugin>
+                <groupId>org.apache.hadoop</groupId>
+                <artifactId>hadoop-maven-plugins</artifactId>
+                <executions>
+                    <execution>
+                        <id>cmake-compile</id>
+                        <phase>compile</phase>
+                        <goals><goal>cmake-compile</goal></goals>
+                        <configuration>
+                            <source>${basedir}/src</source>
+                            <vars>
+                                <GENERATED_JAVAH>${project.build.directory}/native/javah</GENERATED_JAVAH>
+                                <NATIVE_DIR>${basedir}/src/main/native</NATIVE_DIR>
+                                <SST_DUMP_INCLUDE>${sstDump.include}</SST_DUMP_INCLUDE>
+                                <CMAKE_STANDARDS>${cmake.standards}</CMAKE_STANDARDS>
+                                <ROCKSDB_HEADERS>${project.build.directory}/rocksdb/rocksdb-${rocksdb.version}/include</ROCKSDB_HEADERS>
+                                <ROCKSDB_LIB>${project.build.directory}/rocksdb/rocksdb-${rocksdb.version}</ROCKSDB_LIB>
+                                <ZLIB_LIB>${project.build.directory}/zlib/zlib-${zlib.version}</ZLIB_LIB>
+                                <BZIP2_LIB>${project.build.directory}/bzip2/bzip2-${bzip2.version}</BZIP2_LIB>
+                            </vars>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+            <plugin>
+            <artifactId>maven-antrun-plugin</artifactId>

Review Comment:
   ```
   'build.plugins.plugin.(groupId:artifactId)' must be unique but found duplicate declaration of plugin org.apache.maven.plugins:maven-antrun-plugin @ line 195, column 21
   ```



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] neils-dev commented on pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "neils-dev (via GitHub)" <gi...@apache.org>.
neils-dev commented on PR #4315:
URL: https://github.com/apache/ozone/pull/4315#issuecomment-1452433956

   Thanks @swamirishi .  If would be a great help if you can let me know if there are any special build parameters you have in mind for building this like the native library compile for hadoop?
   ie. `$ mvn package -Pdist,native -DskipTests -Dtar`
   
   Also, what would be a good way that I can manually test the jni build?
   


-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1123896824


##########
hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/managed/ManagedSSTDumpIterator.java:
##########
@@ -0,0 +1,203 @@
+/**
+ * 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.collect.Lists;
+import org.apache.hadoop.hdds.utils.HddsServerUtil;
+import org.eclipse.jetty.io.RuntimeIOException;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * Iterator to Parse output of RocksDBSSTDumpTool.
+ */
+public class ManagedSSTDumpIterator implements

Review Comment:
   Move this under `hadoop-hdds/common/` as well



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1123895618


##########
hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/managed/ManagedRocksObject.java:
##########
@@ -0,0 +1,35 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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 org.rocksdb.RocksObject;
+/**
+ * Managed RocksObject.
+ */
+public abstract class ManagedRocksObject extends RocksObject {

Review Comment:
   This file needs to be moved under `hadoop-hdds/common/` since @sadanand48 's change #4273 is just merged.



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1123910804


##########
hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/MessageQueue.java:
##########
@@ -0,0 +1,188 @@
+/**
+ * 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;
+
+import org.apache.commons.io.input.buffer.CircularByteBuffer;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.concurrent.Callable;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * Message Queue to Pipe output from one output stream to another inputstream.
+ */
+public class MessageQueue {
+  private CircularByteBuffer byteBuffer;
+  private AtomicBoolean isRunning;
+
+  private MessageOutputStream messageOutputStream;
+  private MessageInputStream messageInputStream;
+
+  public MessageQueue(int bufferSize, long pollIntervalMillis) {
+    this.pollIntervalMillis = pollIntervalMillis;
+    init(bufferSize);
+  }
+
+  private void init(int bufferSize) {
+    this.byteBuffer = new CircularByteBuffer(bufferSize);
+    this.isRunning = new AtomicBoolean(false);
+    this.messageInputStream = new MessageInputStream(this);
+    this.messageOutputStream = new MessageOutputStream(this);
+  }
+
+  public void start() {
+    this.isRunning.set(true);
+  }
+
+  public void stop() {
+    this.isRunning.set(false);
+  }
+
+  public MessageOutputStream getMessageOutputStream() {
+    return messageOutputStream;
+  }
+
+  public MessageInputStream getMessageInputStream() {
+    return messageInputStream;
+  }
+
+  private long pollIntervalMillis;
+
+  public boolean isRunning() {
+    return isRunning.get();
+  }
+
+  private long getPollIntervalMillis() {
+    return pollIntervalMillis;
+  }
+
+  private boolean hasSpace(int requiredLength) {
+    return this.byteBuffer.hasSpace(requiredLength);
+  }
+
+  private boolean hasBytes() {
+    return this.byteBuffer.hasBytes();
+  }
+
+  private int getCurrentNumberOfBytes() {
+    return this.byteBuffer.getCurrentNumberOfBytes();
+  }
+
+  private void add(byte[] b, int off, int len) {
+    this.byteBuffer.add(b, off, len);
+  }
+
+  private int read(byte[] b, int off, int len) {
+    this.byteBuffer.read(b, off, len);
+    return len;
+  }
+
+  private static  <T> T callWithLock(Lock lock, Callable<T> callable)
+          throws Exception {
+    lock.lock();
+    try {
+      return callable.call();
+    } finally {
+      lock.unlock();
+    }
+  }
+  private static final class MessageOutputStream extends OutputStream {
+
+    private MessageQueue messageQueue;
+    private Lock writeLock;
+
+    private MessageOutputStream(MessageQueue messageQueue) {
+      this.messageQueue = messageQueue;
+      this.writeLock = new ReentrantLock();
+
+    }
+
+    private void waitForBytes(int requiredLength) throws InterruptedException {
+      while (!this.messageQueue.hasSpace(requiredLength)) {
+        Thread.sleep(this.messageQueue.getPollIntervalMillis());
+      }
+    }
+
+
+
+    @Override
+    public void write(int b) throws IOException {
+      this.write(new byte[]{(byte) b});
+    }
+
+    @Override
+    public void write(byte[] b, int off, int len) throws IOException {
+      try {
+        if (!this.messageQueue.isRunning()) {
+          throw new IOException("Message Queue is Closed");
+        }
+        waitForBytes(len);
+        callWithLock(this.writeLock, () -> {
+          waitForBytes(len);
+          this.messageQueue.add(b, off, len);
+          return true;
+        });
+      } catch (Exception e) {
+        throw new IOException(e);
+      }
+    }
+  }
+  private static final class MessageInputStream extends InputStream {

Review Comment:
   ```suggestion
   
     private static final class MessageInputStream extends InputStream {
   ```



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1123910594


##########
hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/MessageQueue.java:
##########
@@ -0,0 +1,188 @@
+/**
+ * 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;
+
+import org.apache.commons.io.input.buffer.CircularByteBuffer;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.concurrent.Callable;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * Message Queue to Pipe output from one output stream to another inputstream.
+ */
+public class MessageQueue {
+  private CircularByteBuffer byteBuffer;
+  private AtomicBoolean isRunning;
+
+  private MessageOutputStream messageOutputStream;
+  private MessageInputStream messageInputStream;
+
+  public MessageQueue(int bufferSize, long pollIntervalMillis) {
+    this.pollIntervalMillis = pollIntervalMillis;
+    init(bufferSize);
+  }
+
+  private void init(int bufferSize) {
+    this.byteBuffer = new CircularByteBuffer(bufferSize);
+    this.isRunning = new AtomicBoolean(false);
+    this.messageInputStream = new MessageInputStream(this);
+    this.messageOutputStream = new MessageOutputStream(this);
+  }
+
+  public void start() {
+    this.isRunning.set(true);
+  }
+
+  public void stop() {
+    this.isRunning.set(false);
+  }
+
+  public MessageOutputStream getMessageOutputStream() {
+    return messageOutputStream;
+  }
+
+  public MessageInputStream getMessageInputStream() {
+    return messageInputStream;
+  }
+
+  private long pollIntervalMillis;
+
+  public boolean isRunning() {
+    return isRunning.get();
+  }
+
+  private long getPollIntervalMillis() {
+    return pollIntervalMillis;
+  }
+
+  private boolean hasSpace(int requiredLength) {
+    return this.byteBuffer.hasSpace(requiredLength);
+  }
+
+  private boolean hasBytes() {
+    return this.byteBuffer.hasBytes();
+  }
+
+  private int getCurrentNumberOfBytes() {
+    return this.byteBuffer.getCurrentNumberOfBytes();
+  }
+
+  private void add(byte[] b, int off, int len) {
+    this.byteBuffer.add(b, off, len);
+  }
+
+  private int read(byte[] b, int off, int len) {
+    this.byteBuffer.read(b, off, len);
+    return len;
+  }
+
+  private static  <T> T callWithLock(Lock lock, Callable<T> callable)
+          throws Exception {
+    lock.lock();
+    try {
+      return callable.call();
+    } finally {
+      lock.unlock();
+    }
+  }
+  private static final class MessageOutputStream extends OutputStream {
+
+    private MessageQueue messageQueue;
+    private Lock writeLock;
+
+    private MessageOutputStream(MessageQueue messageQueue) {
+      this.messageQueue = messageQueue;
+      this.writeLock = new ReentrantLock();
+
+    }
+
+    private void waitForBytes(int requiredLength) throws InterruptedException {
+      while (!this.messageQueue.hasSpace(requiredLength)) {
+        Thread.sleep(this.messageQueue.getPollIntervalMillis());
+      }
+    }
+
+
+
+    @Override

Review Comment:
   ```suggestion
       }
   
       @Override
   ```



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1123909177


##########
hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/MessageQueue.java:
##########
@@ -0,0 +1,188 @@
+/**
+ * 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;
+
+import org.apache.commons.io.input.buffer.CircularByteBuffer;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.concurrent.Callable;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * Message Queue to Pipe output from one output stream to another inputstream.
+ */
+public class MessageQueue {
+  private CircularByteBuffer byteBuffer;
+  private AtomicBoolean isRunning;
+
+  private MessageOutputStream messageOutputStream;
+  private MessageInputStream messageInputStream;
+
+  public MessageQueue(int bufferSize, long pollIntervalMillis) {
+    this.pollIntervalMillis = pollIntervalMillis;
+    init(bufferSize);
+  }
+
+  private void init(int bufferSize) {
+    this.byteBuffer = new CircularByteBuffer(bufferSize);
+    this.isRunning = new AtomicBoolean(false);
+    this.messageInputStream = new MessageInputStream(this);
+    this.messageOutputStream = new MessageOutputStream(this);
+  }
+
+  public void start() {
+    this.isRunning.set(true);
+  }
+
+  public void stop() {
+    this.isRunning.set(false);
+  }
+
+  public MessageOutputStream getMessageOutputStream() {
+    return messageOutputStream;
+  }
+
+  public MessageInputStream getMessageInputStream() {
+    return messageInputStream;
+  }
+
+  private long pollIntervalMillis;

Review Comment:
   Move this to the beginning of the class.



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1123911675


##########
hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/managed/ManagedSSTDumpIterator.java:
##########
@@ -0,0 +1,203 @@
+/**
+ * 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.collect.Lists;
+import org.apache.hadoop.hdds.utils.HddsServerUtil;
+import org.eclipse.jetty.io.RuntimeIOException;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * Iterator to Parse output of RocksDBSSTDumpTool.
+ */
+public class ManagedSSTDumpIterator implements
+        Iterator<ManagedSSTDumpIterator.KeyValue>, AutoCloseable {
+  private Process process;
+  private static final String SST_DUMP_TOOL_CLASS =
+          "org.apache.hadoop.hdds.utils.db.managed.ManagedSSTDumpTool";
+  private static final String PATTERN_REGEX =
+          "'([^=>]+)' seq:([0-9]+), type:([0-9]+) =>";

Review Comment:
   nice



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1123913570


##########
hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/managed/ManagedSSTDumpIterator.java:
##########
@@ -0,0 +1,203 @@
+/**
+ * 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.collect.Lists;
+import org.apache.hadoop.hdds.utils.HddsServerUtil;
+import org.eclipse.jetty.io.RuntimeIOException;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * Iterator to Parse output of RocksDBSSTDumpTool.
+ */
+public class ManagedSSTDumpIterator implements
+        Iterator<ManagedSSTDumpIterator.KeyValue>, AutoCloseable {
+  private Process process;
+  private static final String SST_DUMP_TOOL_CLASS =
+          "org.apache.hadoop.hdds.utils.db.managed.ManagedSSTDumpTool";
+  private static final String PATTERN_REGEX =
+          "'([^=>]+)' seq:([0-9]+), type:([0-9]+) =>";
+
+  public static final int PATTERN_KEY_GROUP_NUMBER = 1;
+  public static final int PATTERN_SEQ_GROUP_NUMBER = 2;
+  public static final int PATTERN_TYPE_GROUP_NUMBER = 3;
+  private static final Pattern PATTERN_MATCHER =
+          Pattern.compile(PATTERN_REGEX);
+  private BufferedReader processOutput;
+  private StringBuilder stdoutString;
+
+  private Matcher currentMatcher;
+  private int prevMatchEndIndex;
+  private KeyValue currentKey;
+  private char[] charBuffer;
+  private KeyValue nextKey;
+
+  private long pollIntervalMillis;
+  private Lock lock;
+
+
+  public ManagedSSTDumpIterator(String sstDumptoolJarPath,
+                                String sstFilePath,
+                                long pollIntervalMillis) throws IOException {
+    File sstFile = new File(sstFilePath);
+    if (!sstFile.exists() || !sstFile.isFile()) {
+      throw new IOException(String.format("Invalid SST File Path : %s",
+              sstFile.getAbsolutePath()));
+    }
+    this.pollIntervalMillis = pollIntervalMillis;
+    this.lock = new ReentrantLock();
+    init(sstFile, sstDumptoolJarPath);
+  }
+
+  private void init(File sstFile, String sstDumptoolJarPath)
+          throws IOException {
+    List<String> args = Lists.newArrayList(
+            "--file=" + sstFile.getAbsolutePath(),
+            "--command=scan");
+    process = HddsServerUtil.getJavaProcess(Collections.emptyList(),
+            sstDumptoolJarPath, SST_DUMP_TOOL_CLASS, args).start();
+    processOutput = new BufferedReader(new InputStreamReader(
+            process.getInputStream(), StandardCharsets.UTF_8));
+    stdoutString = new StringBuilder();
+    currentMatcher = PATTERN_MATCHER.matcher(stdoutString);
+    charBuffer = new char[8192];
+    next();
+  }
+
+  private void checkSanityOfProcess() {
+    if (!process.isAlive() && process.exitValue() != 0) {
+      throw new RuntimeException("Process Terminated with non zero " +
+              String.format("exit value %d", process.exitValue()));
+    }
+  }
+
+  @Override
+  public boolean hasNext() {
+    checkSanityOfProcess();
+    return nextKey != null;
+  }
+
+  @Override
+  public KeyValue next() throws RuntimeIOException {
+    checkSanityOfProcess();
+    try {
+      lock.lock();
+      currentKey = nextKey;
+      nextKey = null;
+      while (!currentMatcher.find()) {
+        try {
+          if (prevMatchEndIndex != 0) {
+            stdoutString = new StringBuilder(stdoutString.substring(
+                    prevMatchEndIndex, stdoutString.length()));
+            prevMatchEndIndex = 0;
+            currentMatcher = PATTERN_MATCHER.matcher(stdoutString);
+          }
+          Thread.sleep(pollIntervalMillis);

Review Comment:
   Is this strictly necessary? Is there a better way to achieve the same goal?
   
   Generally we do want to avoid `sleep` in the prod code.



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1123916890


##########
hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/managed/ManagedSSTDumpIterator.java:
##########
@@ -0,0 +1,203 @@
+/**
+ * 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.collect.Lists;
+import org.apache.hadoop.hdds.utils.HddsServerUtil;
+import org.eclipse.jetty.io.RuntimeIOException;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * Iterator to Parse output of RocksDBSSTDumpTool.
+ */
+public class ManagedSSTDumpIterator implements
+        Iterator<ManagedSSTDumpIterator.KeyValue>, AutoCloseable {
+  private Process process;
+  private static final String SST_DUMP_TOOL_CLASS =
+          "org.apache.hadoop.hdds.utils.db.managed.ManagedSSTDumpTool";
+  private static final String PATTERN_REGEX =
+          "'([^=>]+)' seq:([0-9]+), type:([0-9]+) =>";
+
+  public static final int PATTERN_KEY_GROUP_NUMBER = 1;
+  public static final int PATTERN_SEQ_GROUP_NUMBER = 2;
+  public static final int PATTERN_TYPE_GROUP_NUMBER = 3;
+  private static final Pattern PATTERN_MATCHER =
+          Pattern.compile(PATTERN_REGEX);
+  private BufferedReader processOutput;
+  private StringBuilder stdoutString;
+
+  private Matcher currentMatcher;
+  private int prevMatchEndIndex;
+  private KeyValue currentKey;
+  private char[] charBuffer;
+  private KeyValue nextKey;
+
+  private long pollIntervalMillis;
+  private Lock lock;
+
+
+  public ManagedSSTDumpIterator(String sstDumptoolJarPath,
+                                String sstFilePath,
+                                long pollIntervalMillis) throws IOException {
+    File sstFile = new File(sstFilePath);
+    if (!sstFile.exists() || !sstFile.isFile()) {
+      throw new IOException(String.format("Invalid SST File Path : %s",
+              sstFile.getAbsolutePath()));
+    }
+    this.pollIntervalMillis = pollIntervalMillis;
+    this.lock = new ReentrantLock();
+    init(sstFile, sstDumptoolJarPath);
+  }
+
+  private void init(File sstFile, String sstDumptoolJarPath)
+          throws IOException {
+    List<String> args = Lists.newArrayList(
+            "--file=" + sstFile.getAbsolutePath(),
+            "--command=scan");
+    process = HddsServerUtil.getJavaProcess(Collections.emptyList(),
+            sstDumptoolJarPath, SST_DUMP_TOOL_CLASS, args).start();

Review Comment:
   Regarding error handling: What happens here if the java process failed to start for some reason? And what happens if the process crashed after running for a while? (OOM, file not found, etc.)
   
   Ideally in this case we should log an error in OM and move on. But SnapDiff won't be able to use read tombstones. Make sure such errors are gracefully handled.



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1123913072


##########
hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/managed/ManagedSSTDumpIterator.java:
##########
@@ -0,0 +1,203 @@
+/**
+ * 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.collect.Lists;
+import org.apache.hadoop.hdds.utils.HddsServerUtil;
+import org.eclipse.jetty.io.RuntimeIOException;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * Iterator to Parse output of RocksDBSSTDumpTool.
+ */
+public class ManagedSSTDumpIterator implements
+        Iterator<ManagedSSTDumpIterator.KeyValue>, AutoCloseable {
+  private Process process;
+  private static final String SST_DUMP_TOOL_CLASS =
+          "org.apache.hadoop.hdds.utils.db.managed.ManagedSSTDumpTool";
+  private static final String PATTERN_REGEX =
+          "'([^=>]+)' seq:([0-9]+), type:([0-9]+) =>";
+
+  public static final int PATTERN_KEY_GROUP_NUMBER = 1;
+  public static final int PATTERN_SEQ_GROUP_NUMBER = 2;
+  public static final int PATTERN_TYPE_GROUP_NUMBER = 3;
+  private static final Pattern PATTERN_MATCHER =
+          Pattern.compile(PATTERN_REGEX);
+  private BufferedReader processOutput;
+  private StringBuilder stdoutString;
+
+  private Matcher currentMatcher;
+  private int prevMatchEndIndex;
+  private KeyValue currentKey;
+  private char[] charBuffer;
+  private KeyValue nextKey;
+
+  private long pollIntervalMillis;
+  private Lock lock;

Review Comment:
   Lock var name could be more specific.
   
   ```suggestion
     private Lock iterLock;
   ```



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1125274015


##########
hadoop-hdds/rocks-native/src/main/java/org/apache/hadoop/hdds/utils/NativeLibraryLoader.java:
##########
@@ -0,0 +1,138 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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;
+
+
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Files;
+import java.nio.file.StandardCopyOption;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Class to load Native Libraries.
+ */
+public class NativeLibraryLoader {
+  private static final String OS = System.getProperty("os.name").toLowerCase();
+  private Map<String, Boolean> librariesLoaded;
+  private static volatile NativeLibraryLoader instance;
+
+  public NativeLibraryLoader(final Map<String, Boolean> librariesLoaded) {
+    this.librariesLoaded = librariesLoaded;
+  }
+
+  private static synchronized void initNewInstance() {
+    if (instance == null) {
+      instance = new NativeLibraryLoader(new ConcurrentHashMap<>());
+    }
+  }
+
+  public static NativeLibraryLoader getInstance() {
+    if (instance == null) {
+      initNewInstance();
+    }
+    return instance;
+  }
+
+  public static String getJniLibraryFileName(String libraryName) {
+    return appendLibOsSuffix("lib" + libraryName);
+  }
+
+  public static boolean isMac() {
+    return OS.contains("mac");
+  }
+
+  public static boolean isWindows() {
+    return OS.contains("win");
+  }
+
+  private static String getLibOsSuffix() {
+    if (isMac()) {
+      return ".dylib";
+    } else if (isWindows()) {
+      return ".dll";
+    }
+    return ".so";
+  }
+  private static String appendLibOsSuffix(String libraryFileName) {
+    return libraryFileName + getLibOsSuffix();
+  }
+
+  public boolean isLibraryLoaded(final String libraryName) {
+    return librariesLoaded.getOrDefault(libraryName, false);
+  }
+
+  public synchronized boolean loadLibrary(final String libraryName) {
+    if (isLibraryLoaded(libraryName)) {
+      return true;
+    }
+    boolean loaded = false;
+    try {
+      System.loadLibrary(libraryName);
+      loaded = true;
+    } catch (final UnsatisfiedLinkError ule) {
+
+    }
+    if (!loaded) {
+      try {
+        Optional<File> file = copyResourceFromJarToTemp(libraryName);
+        if (file.isPresent()) {
+          System.load(file.get().getAbsolutePath());
+          loaded = true;
+        }
+
+      } catch (IOException e) {
+
+      }
+
+    }
+    this.librariesLoaded.put(libraryName, loaded);
+    return isLibraryLoaded(libraryName);
+  }
+  private Optional<File> copyResourceFromJarToTemp(final String libraryName)
+          throws IOException {
+    final String libraryFileName = getJniLibraryFileName(libraryName);
+    InputStream is = null;
+    try {
+      is = getClass().getClassLoader().getResourceAsStream(libraryFileName);
+      if (is == null) {
+        return Optional.empty();
+      }
+
+      // create a temporary file to copy the library to
+      final File temp = File.createTempFile(libraryName, getLibOsSuffix());
+      if (!temp.exists()) {
+        return Optional.empty();
+      } else {
+        temp.deleteOnExit();
+      }
+
+      Files.copy(is, temp.toPath(), StandardCopyOption.REPLACE_EXISTING);
+      return Optional.ofNullable(temp);
+
+    } finally {
+      if (is != null) {
+        is.close();
+      }
+    }
+  }
+}

Review Comment:
   ```suggestion
   }
   
   ```



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1125265549


##########
hadoop-hdds/rocks-native/src/main/native/SSTDumpTool.cpp:
##########
@@ -0,0 +1,49 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.
+ */
+
+#include "org_apache_hadoop_hdds_utils_db_managed_ManagedSSTDumpTool.h"
+#include "rocksdb/options.h"
+#include "rocksdb/sst_dump_tool.h"
+#include "string"
+#include "cplusplus_to_java_convert.h"
+#include "Pipe.h"
+#include "iostream"
+
+jint Java_org_apache_hadoop_hdds_utils_db_managed_ManagedSSTDumpTool_runInternal(JNIEnv *env, jobject obj, jobjectArray argsArray,
+jlong optionsHandle, jlong pipeHandle) {
+    ROCKSDB_NAMESPACE::SSTDumpTool dumpTool;
+    ROCKSDB_NAMESPACE::Options* options = reinterpret_cast<ROCKSDB_NAMESPACE::Options*>(optionsHandle);
+    Pipe* pipe = reinterpret_cast<Pipe*>(pipeHandle);
+    int length = env->GetArrayLength(argsArray);
+    char* args[length + 1];
+    args[0] = strdup("./sst_dump");
+    for(int i = 0; i < length; i++) {
+        jstring str_val = (jstring)env->GetObjectArrayElement(argsArray, (jsize)i);
+        char* utf_str = (char*)env->GetStringUTFChars(str_val, JNI_FALSE);
+        args[i+1] = strdup(utf_str);

Review Comment:
   ```suggestion
           args[i + 1] = strdup(utf_str);
   ```



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1125266266


##########
hadoop-hdds/rocks-native/src/main/native/SSTDumpTool.cpp:
##########
@@ -0,0 +1,49 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.
+ */
+
+#include "org_apache_hadoop_hdds_utils_db_managed_ManagedSSTDumpTool.h"
+#include "rocksdb/options.h"
+#include "rocksdb/sst_dump_tool.h"
+#include "string"
+#include "cplusplus_to_java_convert.h"
+#include "Pipe.h"
+#include "iostream"
+
+jint Java_org_apache_hadoop_hdds_utils_db_managed_ManagedSSTDumpTool_runInternal(JNIEnv *env, jobject obj, jobjectArray argsArray,
+jlong optionsHandle, jlong pipeHandle) {
+    ROCKSDB_NAMESPACE::SSTDumpTool dumpTool;
+    ROCKSDB_NAMESPACE::Options* options = reinterpret_cast<ROCKSDB_NAMESPACE::Options*>(optionsHandle);
+    Pipe* pipe = reinterpret_cast<Pipe*>(pipeHandle);
+    int length = env->GetArrayLength(argsArray);
+    char* args[length + 1];
+    args[0] = strdup("./sst_dump");
+    for(int i = 0; i < length; i++) {

Review Comment:
   ```suggestion
       for (int i = 0; i < length; i++) {
   ```



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1125270374


##########
hadoop-hdds/rocks-native/src/CMakeLists.txt:
##########
@@ -0,0 +1,78 @@
+#
+# 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
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# 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.
+#
+
+#
+# CMake configuration.
+#
+
+cmake_minimum_required(VERSION 3.1 FATAL_ERROR)

Review Comment:
   Does 2.8 work?
   
   ```suggestion
   cmake_minimum_required(VERSION 2.8)
   ```
   
   Since Ozone had another CMakeLists that uses the older version, we better be consistent: https://github.com/apache/ozone/blob/ac8aee7f8521ff3b70c8cc293432cee5d774896d/tools/fault-injection-service/CMakeLists.txt#L20



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1125257190


##########
hadoop-hdds/rocks-native/src/main/native/Pipe.cpp:
##########
@@ -0,0 +1,20 @@
+//
+// Created by Swaminathan Balachandran on 3/2/23.
+//

Review Comment:
   Missing Apache License header. e.g. https://github.com/apache/hadoop/blob/a7b4e8f03e55e2b5862c1a7328a15bd08f4e0279/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/hdfs.c#L1-L17



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1125287041


##########
hadoop-hdds/rocks-native/pom.xml:
##########
@@ -0,0 +1,391 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed 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
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+  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. See accompanying LICENSE file.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <parent>
+        <artifactId>hdds</artifactId>
+        <groupId>org.apache.ozone</groupId>
+        <version>1.4.0-SNAPSHOT</version>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+    <name>Apache Ozone HDDS Rocks</name>
+    <artifactId>hdds-rocks-native</artifactId>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.ozone</groupId>
+            <artifactId>hdds-server-framework</artifactId>
+        </dependency>
+    </dependencies>
+    <properties>
+        <maven.compiler.source>8</maven.compiler.source>
+        <maven.compiler.target>8</maven.compiler.target>
+        <cmake.standards>23</cmake.standards>
+        <sstDump.include>true</sstDump.include>
+    </properties>
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>com.googlecode.maven-download-plugin</groupId>
+                <artifactId>download-maven-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <id>rocksdb source download</id>
+                        <phase>generate-sources</phase>
+                        <goals>
+                            <goal>wget</goal>
+                        </goals>
+                        <configuration>
+                            <url>https://github.com/facebook/rocksdb/archive/refs/tags/v${rocksdb.version}.tar.gz</url>
+                            <outputFileName>rocksdb-v${rocksdb.version}.tar.gz</outputFileName>
+                            <outputDirectory>${project.build.directory}/rocksdb</outputDirectory>
+                        </configuration>
+                    </execution>
+                    <execution>
+                        <id>zlib source download</id>
+                        <phase>generate-sources</phase>
+                        <goals>
+                            <goal>wget</goal>
+                        </goals>
+                        <configuration>
+                            <url>https://zlib.net/zlib-${zlib.version}.tar.gz</url>
+                            <outputFileName>zlib-${zlib.version}.tar.gz</outputFileName>
+                            <outputDirectory>${project.build.directory}/zlib</outputDirectory>
+                        </configuration>
+                    </execution>
+                    <execution>
+                        <id>bzip2 source download</id>
+                        <phase>generate-sources</phase>
+                        <goals>
+                            <goal>wget</goal>
+                        </goals>
+                        <configuration>
+                            <url>https://sourceware.org/pub/bzip2/bzip2-${bzip2.version}.tar.gz</url>
+                            <outputFileName>bzip2-v${bzip2.version}.tar.gz</outputFileName>
+                            <outputDirectory>${project.build.directory}/bzip2</outputDirectory>
+                        </configuration>
+                    </execution>
+                    <execution>
+                        <id>lz4 source download</id>
+                        <phase>generate-sources</phase>
+                        <goals>
+                            <goal>wget</goal>
+                        </goals>
+                        <configuration>
+                            <url>https://github.com/lz4/lz4/archive/refs/tags/v${lz4.version}.tar.gz</url>
+                            <outputFileName>lz4-v${lz4.version}.tar.gz</outputFileName>
+                            <outputDirectory>${project.build.directory}/lz4</outputDirectory>
+                        </configuration>
+                    </execution>
+                    <execution>
+                        <id>snappy source download</id>
+                        <phase>generate-sources</phase>
+                        <goals>
+                            <goal>wget</goal>
+                        </goals>
+                        <configuration>
+                            <url>https://github.com/google/snappy/archive/refs/tags/${snappy.version}.tar.gz</url>
+                            <outputFileName>snappy-v${snappy.version}.tar.gz</outputFileName>
+                            <outputDirectory>${project.build.directory}/snappy</outputDirectory>
+                        </configuration>
+                    </execution>
+                    <execution>
+                        <id>zstd source download</id>
+                        <phase>generate-sources</phase>
+                        <goals>
+                            <goal>wget</goal>
+                        </goals>
+                        <configuration>
+                            <url>https://github.com/facebook/zstd/archive/refs/tags/v${zstd.version}.tar.gz</url>
+                            <outputFileName>zstd-v${zstd.version}.tar.gz</outputFileName>
+                            <outputDirectory>${project.build.directory}/zstd</outputDirectory>
+                        </configuration>
+                    </execution>
+
+                </executions>
+            </plugin>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-patch-plugin</artifactId>
+                <version>1.1.1</version>
+                <configuration>
+                    <patchFile>${basedir}/src/main/patches/rocks-native.patch</patchFile>
+                    <strip>1</strip>
+                    <targetDirectory>${project.build.directory}/rocksdb/rocksdb-${rocksdb.version}</targetDirectory>
+                </configuration>
+                <executions>
+                    <execution>
+                        <id>patch</id>
+                        <phase>process-sources</phase>
+                        <goals>
+                            <goal>apply</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
+            <plugin>
+            <artifactId>maven-antrun-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <id>unzip-artifact</id>
+                        <phase>generate-sources</phase>
+                        <configuration>
+                            <tasks>
+                                <untar src="${project.build.directory}/rocksdb/rocksdb-v${rocksdb.version}.tar.gz" compression="gzip" dest="${project.build.directory}/rocksdb/" />
+                                <untar src="${project.build.directory}/zlib/zlib-${zlib.version}.tar.gz" compression="gzip" dest="${project.build.directory}/zlib/" />
+                                <untar src="${project.build.directory}/bzip2/bzip2-v${bzip2.version}.tar.gz" compression="gzip" dest="${project.build.directory}/bzip2/" />
+                                <untar src="${project.build.directory}/lz4/lz4-v${lz4.version}.tar.gz" compression="gzip" dest="${project.build.directory}/lz4/" />
+                                <untar src="${project.build.directory}/snappy/snappy-v${snappy.version}.tar.gz" compression="gzip" dest="${project.build.directory}/snappy/" />
+                                <untar src="${project.build.directory}/zstd/zstd-v${zstd.version}.tar.gz" compression="gzip" dest="${project.build.directory}/zstd/" />
+                            </tasks>
+                        </configuration>
+                        <goals>
+                            <goal>run</goal>
+                        </goals>
+                    </execution>
+                    <execution>
+                        <id>build-zlib</id>
+                        <phase>process-sources</phase>
+                        <configuration>
+                            <tasks>
+                                <chmod file="${project.build.directory}/zlib/zlib-${zlib.version}/configure" perm="775" />
+                                <exec executable="./configure" dir="${project.build.directory}/zlib/zlib-${zlib.version}" failonerror="true">
+                                    <arg line="--static"/>
+                                    <env key="CFLAGS" value="-fPIC"/>
+                                </exec>
+                                <exec executable="make" dir="${project.build.directory}/zlib/zlib-${zlib.version}" failonerror="true"/>
+                            </tasks>
+                        </configuration>
+                        <goals>
+                            <goal>run</goal>
+                        </goals>
+                    </execution>
+                    <execution>
+                        <id>build-bzip2</id>
+                        <phase>process-sources</phase>
+                        <configuration>
+                            <tasks>
+                                <exec executable="make" dir="${project.build.directory}/bzip2/bzip2-${bzip2.version}" failonerror="true">
+                                    <arg line="CFLAGS='-fPIC'"/>
+                                </exec>
+                            </tasks>
+                        </configuration>
+                        <goals>
+                            <goal>run</goal>
+                        </goals>
+                    </execution>
+                    <execution>
+                        <id>build-lz4</id>
+                        <phase>process-sources</phase>
+                        <configuration>
+                            <tasks>
+                                <exec executable="make" dir="${project.build.directory}/lz4/lz4-${lz4.version}" failonerror="true">
+                                    <arg line="CFLAGS='-fPIC'"/>
+                                </exec>
+                            </tasks>
+                        </configuration>
+                        <goals>
+                            <goal>run</goal>
+                        </goals>
+                    </execution>
+                    <execution>
+                        <id>build-zstd</id>
+                        <phase>process-sources</phase>
+                        <configuration>
+                            <tasks>
+                                <exec executable="make" dir="${project.build.directory}/zstd/zstd-${zstd.version}" failonerror="true">
+                                    <arg line="CFLAGS='-fPIC'"/>
+                                </exec>
+                            </tasks>
+                        </configuration>
+                        <goals>
+                            <goal>run</goal>
+                        </goals>
+                    </execution>
+                    <execution>
+                        <id>build-snappy</id>
+                        <phase>process-sources</phase>
+                        <configuration>
+                            <tasks>
+                                <mkdir dir="${project.build.directory}/snappy/lib"/>
+                                <exec executable="cmake" failonerror="true" dir="${project.build.directory}/snappy/lib">
+                                    <arg line="${project.build.directory}/snappy/snappy-${snappy.version}"/>
+                                    <env key="CFLAGS" value="-fPIC"/>
+                                    <env key="CXXFLAGS" value="-fPIC"/>
+                                </exec>
+                                <exec executable="make" dir="${project.build.directory}/snappy/lib" failonerror="true"/>
+                            </tasks>
+                        </configuration>
+                        <goals>
+                            <goal>run</goal>
+                        </goals>
+                    </execution>
+                    <execution>
+                        <id>build-rocksjava</id>
+                        <phase>process-resources</phase>
+                        <configuration>
+                            <tasks>
+                                <exec executable="chmod" failonerror="true">
+                                    <arg line="-R"/>
+                                    <arg line="775"/>
+                                    <arg line="${project.build.directory}/rocksdb/rocksdb-${rocksdb.version}"/>
+                                </exec>
+                                <exec executable="make" dir="${project.build.directory}/rocksdb/rocksdb-${rocksdb.version}" failonerror="true">
+                                    <arg line="EXTRA_CXXFLAGS='-fPIC -I${project.build.directory}/snappy/lib -I${project.build.directory}/snappy/snappy-${snappy.version} -I${project.build.directory}/lz4/lz4-${lz4.version}/lib -I${project.build.directory}/zstd/zstd-${zstd.version}/lib -I${project.build.directory}/zstd/zstd-${zstd.version}/lib/dictBuilder -I${project.build.directory}/bzip2/bzip2-${bzip2.version} -I${project.build.directory}/zlib/zlib-${zlib.version}'"/>
+                                    <arg line="EXTRA_LDFLAGS='-L${project.build.directory}/snappy/lib -L${project.build.directory}/lz4/lz4-${lz4.version}/lib -L${project.build.directory}/zstd/zstd-${zstd.version}/lib -L${project.build.directory}/bzip2/bzip2-${bzip2.version} -L${project.build.directory}/zlib/zlib-${zlib.version}'"/>
+                                    <arg line="-j10"/>
+                                    <arg line="tools"/>
+                                </exec>
+                            </tasks>
+                        </configuration>
+                        <goals>
+                            <goal>run</goal>
+                        </goals>
+                    </execution>
+                    <execution>
+                        <id>build-rocks-tools</id>
+                        <phase>process-classes</phase>
+                        <configuration>
+                            <tasks>
+                                <mkdir dir="${project.build.directory}/native/rocksdb"/>
+                                <exec executable="cmake" failonerror="true" dir="${project.build.directory}/native/rocksdb">
+                                    <env key="CFLAGS" value="-fPIC"/>
+                                    <env key="CXXFLAGS" value="-fPIC"/>
+                                    <arg line="${basedir}/src"/>
+                                    <arg line="-DGENERATED_JAVAH=${project.build.directory}/native/javah"/>
+                                    <arg line="-DNATIVE_DIR=${basedir}/src/main/native"/>
+                                    <arg line="-DSST_DUMP_INCLUDE=${sstDump.include}"/>
+                                    <arg line="-DCMAKE_STANDARDS=${cmake.standards}"/>
+                                    <arg line="-DROCKSDB_HEADERS=${project.build.directory}/rocksdb/rocksdb-${rocksdb.version}/include"/>
+                                    <arg line="-DROCKSDB_LIB=${project.build.directory}/rocksdb/rocksdb-${rocksdb.version}"/>
+                                    <arg line="-DZLIB_LIB=${project.build.directory}/zlib/zlib-${zlib.version}"/>
+                                    <arg line="-DBZIP2_LIB=${project.build.directory}/bzip2/bzip2-${bzip2.version}"/>
+                                    <arg line="-DLZ4_LIB=${project.build.directory}/lz4/lz4-${lz4.version}/lib"/>
+                                    <arg line="-DSNAPPY_LIB=${project.build.directory}/snappy/lib"/>
+                                    <arg line="-DZSTD_LIB=${project.build.directory}/zstd/zstd-${zstd.version}/lib"/>
+                                </exec>
+                                <exec executable="make" dir="${project.build.directory}/native/rocksdb" failonerror="true"/>
+                            </tasks>
+                        </configuration>
+                        <goals>
+                            <goal>run</goal>
+                        </goals>
+                    </execution>
+                    <execution>
+                        <id>copy-lib-file</id>
+                        <phase>prepare-package</phase>
+                        <configuration>
+                            <tasks>
+                                <copy toDir="${project.build.outputDirectory}">
+                                    <fileset dir="${project.build.directory}/native/rocksdb" includes="**/lib*.*" />
+                                </copy>
+                            </tasks>
+                        </configuration>
+                        <goals>
+                            <goal>run</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-jar-plugin</artifactId>
+                <version>${maven-jar-plugin.version}</version>
+                <configuration>
+                    <includes>
+                        <include>**/*.class</include>
+                        <include>**/lib*.dylib</include>
+                        <include>**/lib*.so</include>
+                        <include>**/lib*.jnilib</include>
+                        <include>**/lib*.dll</include>
+                    </includes>
+                </configuration>
+            </plugin>
+        </plugins>
+    </build>
+
+    <profiles>
+        <profile>
+            <id>java-8</id>
+            <activation>
+                <jdk>1.8</jdk>
+            </activation>
+            <build>
+                <plugins>
+                    <plugin>
+                        <groupId>org.codehaus.mojo</groupId>
+                        <artifactId>native-maven-plugin</artifactId>
+                        <executions>
+                            <execution>
+                                <phase>compile</phase>
+                                <goals>
+                                    <goal>javah</goal>
+                                </goals>
+                                <configuration>
+                                    <javahPath>${env.JAVA_HOME}/bin/javah</javahPath>
+                                    <javahClassNames>
+                                        <javahClassName>org.apache.hadoop.hdds.utils.db.managed.ManagedSSTDumpTool</javahClassName>
+                                        <javahClassName>org.apache.hadoop.hdds.utils.db.managed.PipeInputStream</javahClassName>
+                                    </javahClassNames>
+                                    <javahOutputDirectory>${project.build.directory}/native/javah</javahOutputDirectory>
+                                </configuration>
+                            </execution>
+                        </executions>
+                    </plugin>
+                </plugins>
+            </build>
+        </profile>
+        <profile>
+            <id>java-11</id>
+            <activation>
+                <property>
+                    <name>javac.version</name>
+                    <value>11</value>
+                </property>
+            </activation>
+            <build>
+                <plugins>
+                    <plugin>
+                        <groupId>org.codehaus.mojo</groupId>
+                        <artifactId>exec-maven-plugin</artifactId>
+                        <executions>
+                            <execution>
+                                <id>javach</id>
+                                <goals>
+                                    <goal>exec</goal>
+                                </goals>
+                                <phase>compile</phase>
+                                <configuration>
+                                    <executable>${env.JAVA_HOME}/bin/javac</executable>
+                                    <arguments>
+                                        <argument>-classpath</argument>
+                                        <argument>${project.build.outputDirectory}</argument>
+                                        <argument>-h</argument>
+                                        <argument>${project.build.directory}/native/javah</argument>
+                                        <argument>${project.basedir}/src/main/java/org/apache/hadoop/hdds/utils/db/managed/ManagedSSTDumpTool.java</argument>
+                                        <argument>${project.basedir}/src/main/java/org/apache/hadoop/hdds/utils/db/managed/PipeInputStream.java</argument>
+                                    </arguments>
+                                </configuration>
+                            </execution>
+                        </executions>
+                    </plugin>
+                </plugins>
+            </build>
+        </profile>
+    </profiles>
+
+
+</project>

Review Comment:
   ```suggestion
   </project>
   
   ```



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1125285353


##########
hadoop-hdds/rocks-native/pom.xml:
##########
@@ -0,0 +1,391 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed 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
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+  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. See accompanying LICENSE file.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <parent>
+        <artifactId>hdds</artifactId>
+        <groupId>org.apache.ozone</groupId>
+        <version>1.4.0-SNAPSHOT</version>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+    <name>Apache Ozone HDDS Rocks</name>
+    <artifactId>hdds-rocks-native</artifactId>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.ozone</groupId>
+            <artifactId>hdds-server-framework</artifactId>
+        </dependency>
+    </dependencies>
+    <properties>
+        <maven.compiler.source>8</maven.compiler.source>
+        <maven.compiler.target>8</maven.compiler.target>
+        <cmake.standards>23</cmake.standards>
+        <sstDump.include>true</sstDump.include>
+    </properties>
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>com.googlecode.maven-download-plugin</groupId>
+                <artifactId>download-maven-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <id>rocksdb source download</id>
+                        <phase>generate-sources</phase>
+                        <goals>
+                            <goal>wget</goal>
+                        </goals>
+                        <configuration>
+                            <url>https://github.com/facebook/rocksdb/archive/refs/tags/v${rocksdb.version}.tar.gz</url>
+                            <outputFileName>rocksdb-v${rocksdb.version}.tar.gz</outputFileName>
+                            <outputDirectory>${project.build.directory}/rocksdb</outputDirectory>
+                        </configuration>
+                    </execution>
+                    <execution>
+                        <id>zlib source download</id>
+                        <phase>generate-sources</phase>
+                        <goals>
+                            <goal>wget</goal>
+                        </goals>
+                        <configuration>
+                            <url>https://zlib.net/zlib-${zlib.version}.tar.gz</url>
+                            <outputFileName>zlib-${zlib.version}.tar.gz</outputFileName>
+                            <outputDirectory>${project.build.directory}/zlib</outputDirectory>
+                        </configuration>
+                    </execution>
+                    <execution>
+                        <id>bzip2 source download</id>
+                        <phase>generate-sources</phase>
+                        <goals>
+                            <goal>wget</goal>
+                        </goals>
+                        <configuration>
+                            <url>https://sourceware.org/pub/bzip2/bzip2-${bzip2.version}.tar.gz</url>
+                            <outputFileName>bzip2-v${bzip2.version}.tar.gz</outputFileName>
+                            <outputDirectory>${project.build.directory}/bzip2</outputDirectory>
+                        </configuration>
+                    </execution>
+                    <execution>
+                        <id>lz4 source download</id>
+                        <phase>generate-sources</phase>
+                        <goals>
+                            <goal>wget</goal>
+                        </goals>
+                        <configuration>
+                            <url>https://github.com/lz4/lz4/archive/refs/tags/v${lz4.version}.tar.gz</url>
+                            <outputFileName>lz4-v${lz4.version}.tar.gz</outputFileName>
+                            <outputDirectory>${project.build.directory}/lz4</outputDirectory>
+                        </configuration>
+                    </execution>
+                    <execution>
+                        <id>snappy source download</id>
+                        <phase>generate-sources</phase>
+                        <goals>
+                            <goal>wget</goal>
+                        </goals>
+                        <configuration>
+                            <url>https://github.com/google/snappy/archive/refs/tags/${snappy.version}.tar.gz</url>
+                            <outputFileName>snappy-v${snappy.version}.tar.gz</outputFileName>
+                            <outputDirectory>${project.build.directory}/snappy</outputDirectory>
+                        </configuration>
+                    </execution>
+                    <execution>
+                        <id>zstd source download</id>
+                        <phase>generate-sources</phase>
+                        <goals>
+                            <goal>wget</goal>
+                        </goals>
+                        <configuration>
+                            <url>https://github.com/facebook/zstd/archive/refs/tags/v${zstd.version}.tar.gz</url>
+                            <outputFileName>zstd-v${zstd.version}.tar.gz</outputFileName>
+                            <outputDirectory>${project.build.directory}/zstd</outputDirectory>
+                        </configuration>
+                    </execution>
+
+                </executions>
+            </plugin>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-patch-plugin</artifactId>
+                <version>1.1.1</version>
+                <configuration>
+                    <patchFile>${basedir}/src/main/patches/rocks-native.patch</patchFile>

Review Comment:
   nice. but where is this file?



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1125278927


##########
hadoop-hdds/rocks-native/src/main/java/org/apache/hadoop/hdds/utils/db/managed/ManagedSSTDumpTool.java:
##########
@@ -0,0 +1,100 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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 org.apache.hadoop.hdds.utils.NativeLibraryLoader;
+import org.apache.hadoop.hdds.utils.NativeLibraryNotLoadedException;
+
+import java.util.Map;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+
+import static org.apache.hadoop.hdds.utils.NativeConstants.ROCKS_TOOLS_NATIVE_LIBRARY_NAME;
+
+/**
+ * JNI for RocksDB SSTDumpTool. Pipes the output to an output stream
+ */
+public class ManagedSSTDumpTool {
+
+  static {
+    NativeLibraryLoader.getInstance()
+            .loadLibrary(ROCKS_TOOLS_NATIVE_LIBRARY_NAME);
+  }
+  private int bufferCapacity;
+  private ExecutorService executorService;
+
+  public ManagedSSTDumpTool(ExecutorService executorService,
+                            int bufferCapacity)
+          throws NativeLibraryNotLoadedException {
+    if (!NativeLibraryLoader.getInstance()
+            .isLibraryLoaded(ROCKS_TOOLS_NATIVE_LIBRARY_NAME)) {
+      throw new NativeLibraryNotLoadedException(
+              ROCKS_TOOLS_NATIVE_LIBRARY_NAME);
+    }
+    this.bufferCapacity = bufferCapacity;
+    this.executorService = executorService;
+  }
+
+  public SSTDumpToolTask run(String[] args, ManagedOptions options)
+          throws NativeLibraryNotLoadedException {
+    PipeInputStream pipeInputStream = new PipeInputStream(bufferCapacity);
+    return new SSTDumpToolTask(this.executorService.submit(() ->
+            this.runInternal(args, options.getNativeHandle(),
+            pipeInputStream.getNativeHandle())), pipeInputStream);
+  }
+
+  public SSTDumpToolTask run(Map<String, String> args, ManagedOptions options)
+          throws NativeLibraryNotLoadedException {
+    return this.run(args.entrySet().stream().map(e -> "--"
+            + (e.getValue() == null || e.getValue().isEmpty() ? e.getKey() :
+            e.getKey() + "=" + e.getValue())).toArray(String[]::new), options);
+  }
+
+  private native int runInternal(String[] args, long optionsHandle,
+                                  long pipeHandle);

Review Comment:
   ```suggestion
                                    long pipeHandle);
   
   ```



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] swamirishi commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "swamirishi (via GitHub)" <gi...@apache.org>.
swamirishi commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1125382282


##########
hadoop-hdds/rocks-native/src/main/java/org/apache/hadoop/hdds/utils/db/managed/PipeInputStream.java:
##########
@@ -0,0 +1,81 @@
+package org.apache.hadoop.hdds.utils.db.managed;
+
+import org.apache.hadoop.hdds.utils.NativeLibraryLoader;
+import org.apache.hadoop.hdds.utils.NativeLibraryNotLoadedException;
+import java.io.InputStream;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.apache.hadoop.hdds.utils.NativeConstants.ROCKS_TOOLS_NATIVE_LIBRARY_NAME;
+
+/**
+ * JNI for reading data from pipe.
+ */
+public class PipeInputStream extends InputStream {
+
+  static {
+    NativeLibraryLoader.getInstance()
+            .loadLibrary(ROCKS_TOOLS_NATIVE_LIBRARY_NAME);
+  }
+  private byte[] byteBuffer;
+  private long nativeHandle;
+  private int numberOfBytesLeftToRead;
+  private int index = 0;
+  private int capacity;
+
+  private AtomicBoolean cleanup;
+
+  PipeInputStream(int capacity) throws NativeLibraryNotLoadedException {
+    if (!NativeLibraryLoader.getInstance()
+            .isLibraryLoaded(ROCKS_TOOLS_NATIVE_LIBRARY_NAME)) {
+      throw new NativeLibraryNotLoadedException(
+              ROCKS_TOOLS_NATIVE_LIBRARY_NAME);
+    }
+
+    this.byteBuffer = new byte[capacity];
+    this.numberOfBytesLeftToRead = 0;
+    this.capacity = capacity;
+    this.nativeHandle = newPipe();
+    this.cleanup = new AtomicBoolean(false);
+  }
+
+  long getNativeHandle() {
+    return nativeHandle;
+  }
+
+  @Override
+  public int read() {
+    if (numberOfBytesLeftToRead < 0) {
+      this.close();
+      return -1;
+    }
+    if (numberOfBytesLeftToRead == 0) {
+      numberOfBytesLeftToRead = readInternal(byteBuffer, capacity,
+              nativeHandle);
+      index = 0;
+      return read();
+    }
+    numberOfBytesLeftToRead--;
+    int ret = byteBuffer[index] & 0xFF;
+    index += 1;
+    return ret;

Review Comment:
   this is overriding the InputStream interface



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] swamirishi commented on pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "swamirishi (via GitHub)" <gi...@apache.org>.
swamirishi commented on PR #4315:
URL: https://github.com/apache/ozone/pull/4315#issuecomment-1454833100

   @smengcl Please review the c++ code closely for possible memory leaks, this is my first time writing c++ code. I have done my due diligence on this but want a second eye on this.


-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] prashantpogde commented on pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "prashantpogde (via GitHub)" <gi...@apache.org>.
prashantpogde commented on PR #4315:
URL: https://github.com/apache/ozone/pull/4315#issuecomment-1452573002

   > Open questions
   > 
   > * [ ]  How can we avoid building the JNI as part of each build? Can we separate out the JNI build as an independent artifact that gets pulled in as a JNI dependency?
   > * [ ]  Can we build the JNI and update the docker images used via CI/CD to avoid the hit per build
   
   We are planning to have a seprate profile to enable efficient snapdiff build. The default build will just include regular snapdiff path.


-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1123911276


##########
hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/MessageQueue.java:
##########
@@ -0,0 +1,188 @@
+/**
+ * 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;
+
+import org.apache.commons.io.input.buffer.CircularByteBuffer;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.concurrent.Callable;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * Message Queue to Pipe output from one output stream to another inputstream.
+ */
+public class MessageQueue {
+  private CircularByteBuffer byteBuffer;
+  private AtomicBoolean isRunning;
+
+  private MessageOutputStream messageOutputStream;
+  private MessageInputStream messageInputStream;
+
+  public MessageQueue(int bufferSize, long pollIntervalMillis) {
+    this.pollIntervalMillis = pollIntervalMillis;
+    init(bufferSize);
+  }
+
+  private void init(int bufferSize) {
+    this.byteBuffer = new CircularByteBuffer(bufferSize);
+    this.isRunning = new AtomicBoolean(false);
+    this.messageInputStream = new MessageInputStream(this);
+    this.messageOutputStream = new MessageOutputStream(this);
+  }
+
+  public void start() {
+    this.isRunning.set(true);
+  }
+
+  public void stop() {
+    this.isRunning.set(false);
+  }
+
+  public MessageOutputStream getMessageOutputStream() {
+    return messageOutputStream;
+  }
+
+  public MessageInputStream getMessageInputStream() {
+    return messageInputStream;
+  }
+
+  private long pollIntervalMillis;
+
+  public boolean isRunning() {
+    return isRunning.get();
+  }
+
+  private long getPollIntervalMillis() {
+    return pollIntervalMillis;
+  }
+
+  private boolean hasSpace(int requiredLength) {
+    return this.byteBuffer.hasSpace(requiredLength);
+  }
+
+  private boolean hasBytes() {
+    return this.byteBuffer.hasBytes();
+  }
+
+  private int getCurrentNumberOfBytes() {
+    return this.byteBuffer.getCurrentNumberOfBytes();
+  }
+
+  private void add(byte[] b, int off, int len) {
+    this.byteBuffer.add(b, off, len);
+  }
+
+  private int read(byte[] b, int off, int len) {
+    this.byteBuffer.read(b, off, len);
+    return len;
+  }
+
+  private static  <T> T callWithLock(Lock lock, Callable<T> callable)
+          throws Exception {
+    lock.lock();
+    try {
+      return callable.call();
+    } finally {
+      lock.unlock();
+    }
+  }
+  private static final class MessageOutputStream extends OutputStream {
+
+    private MessageQueue messageQueue;
+    private Lock writeLock;
+
+    private MessageOutputStream(MessageQueue messageQueue) {
+      this.messageQueue = messageQueue;
+      this.writeLock = new ReentrantLock();
+
+    }
+
+    private void waitForBytes(int requiredLength) throws InterruptedException {
+      while (!this.messageQueue.hasSpace(requiredLength)) {
+        Thread.sleep(this.messageQueue.getPollIntervalMillis());
+      }
+    }
+
+
+
+    @Override
+    public void write(int b) throws IOException {
+      this.write(new byte[]{(byte) b});
+    }
+
+    @Override
+    public void write(byte[] b, int off, int len) throws IOException {
+      try {
+        if (!this.messageQueue.isRunning()) {
+          throw new IOException("Message Queue is Closed");
+        }
+        waitForBytes(len);
+        callWithLock(this.writeLock, () -> {
+          waitForBytes(len);
+          this.messageQueue.add(b, off, len);
+          return true;
+        });
+      } catch (Exception e) {
+        throw new IOException(e);
+      }
+    }
+  }

Review Comment:
   Add some comments.



##########
hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/MessageQueue.java:
##########
@@ -0,0 +1,188 @@
+/**
+ * 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;
+
+import org.apache.commons.io.input.buffer.CircularByteBuffer;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.concurrent.Callable;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * Message Queue to Pipe output from one output stream to another inputstream.
+ */
+public class MessageQueue {
+  private CircularByteBuffer byteBuffer;
+  private AtomicBoolean isRunning;
+
+  private MessageOutputStream messageOutputStream;
+  private MessageInputStream messageInputStream;
+
+  public MessageQueue(int bufferSize, long pollIntervalMillis) {
+    this.pollIntervalMillis = pollIntervalMillis;
+    init(bufferSize);
+  }
+
+  private void init(int bufferSize) {
+    this.byteBuffer = new CircularByteBuffer(bufferSize);
+    this.isRunning = new AtomicBoolean(false);
+    this.messageInputStream = new MessageInputStream(this);
+    this.messageOutputStream = new MessageOutputStream(this);
+  }
+
+  public void start() {
+    this.isRunning.set(true);
+  }
+
+  public void stop() {
+    this.isRunning.set(false);
+  }
+
+  public MessageOutputStream getMessageOutputStream() {
+    return messageOutputStream;
+  }
+
+  public MessageInputStream getMessageInputStream() {
+    return messageInputStream;
+  }
+
+  private long pollIntervalMillis;
+
+  public boolean isRunning() {
+    return isRunning.get();
+  }
+
+  private long getPollIntervalMillis() {
+    return pollIntervalMillis;
+  }
+
+  private boolean hasSpace(int requiredLength) {
+    return this.byteBuffer.hasSpace(requiredLength);
+  }
+
+  private boolean hasBytes() {
+    return this.byteBuffer.hasBytes();
+  }
+
+  private int getCurrentNumberOfBytes() {
+    return this.byteBuffer.getCurrentNumberOfBytes();
+  }
+
+  private void add(byte[] b, int off, int len) {
+    this.byteBuffer.add(b, off, len);
+  }
+
+  private int read(byte[] b, int off, int len) {
+    this.byteBuffer.read(b, off, len);
+    return len;
+  }
+
+  private static  <T> T callWithLock(Lock lock, Callable<T> callable)
+          throws Exception {
+    lock.lock();
+    try {
+      return callable.call();
+    } finally {
+      lock.unlock();
+    }
+  }
+  private static final class MessageOutputStream extends OutputStream {
+
+    private MessageQueue messageQueue;
+    private Lock writeLock;
+
+    private MessageOutputStream(MessageQueue messageQueue) {
+      this.messageQueue = messageQueue;
+      this.writeLock = new ReentrantLock();
+
+    }
+
+    private void waitForBytes(int requiredLength) throws InterruptedException {
+      while (!this.messageQueue.hasSpace(requiredLength)) {
+        Thread.sleep(this.messageQueue.getPollIntervalMillis());
+      }
+    }
+
+
+
+    @Override
+    public void write(int b) throws IOException {
+      this.write(new byte[]{(byte) b});
+    }
+
+    @Override
+    public void write(byte[] b, int off, int len) throws IOException {
+      try {
+        if (!this.messageQueue.isRunning()) {
+          throw new IOException("Message Queue is Closed");
+        }
+        waitForBytes(len);
+        callWithLock(this.writeLock, () -> {
+          waitForBytes(len);
+          this.messageQueue.add(b, off, len);
+          return true;
+        });
+      } catch (Exception e) {
+        throw new IOException(e);
+      }
+    }
+  }
+  private static final class MessageInputStream extends InputStream {
+
+    private MessageQueue messageQueue;
+    private Lock readLock;
+
+    private MessageInputStream(MessageQueue messageQueue) {
+      this.messageQueue = messageQueue;
+      this.readLock = new ReentrantLock();
+    }
+
+    private void waitForBytes() throws InterruptedException {
+      while (!this.messageQueue.hasBytes() && this.messageQueue.isRunning()) {
+        Thread.sleep(messageQueue.getPollIntervalMillis());
+      }
+    }
+
+    @Override
+    public int read() throws IOException {
+      byte[] readByte = new byte[1];
+      int numberOfBytesRead = this.read(readByte);
+      return numberOfBytesRead == -1 ? -1 : (readByte[0] & 0xff);
+    }
+
+    @Override
+    public int read(byte[] b, int off, int len) throws IOException {
+      try {
+        return MessageQueue.callWithLock(this.readLock, () -> {
+          waitForBytes();
+          if (!this.messageQueue.isRunning()) {
+            return -1;
+          }
+          return this.messageQueue.read(b, off, Math.min(len,
+                  this.messageQueue.getCurrentNumberOfBytes()));
+        });
+      } catch (Exception e) {
+        throw new IOException(e);
+      }
+    }
+  }

Review Comment:
   Add some comments.



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1123886741


##########
hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/HddsServerUtil.java:
##########
@@ -583,4 +584,29 @@ public static IOException toIOException(String msg, RocksDBException e) {
   public static void addSuppressedLoggingExceptions(RPC.Server server) {
     server.addSuppressedLoggingExceptions(ServerNotLeaderException.class);
   }
+
+  /**
+   * Build javaProcessBuilder.
+   * @param jvmArgs
+   * @param classpath
+   * @param className
+   * @param args
+   * @return ProcessBuilder
+   */
+  public static ProcessBuilder getJavaProcess(List<String> jvmArgs,
+                                           String classpath,
+                                           String className,
+                                           List<String> args) {

Review Comment:
   Parameter alignment is off.



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] swamirishi commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "swamirishi (via GitHub)" <gi...@apache.org>.
swamirishi commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1128826781


##########
hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/MessageQueue.java:
##########
@@ -0,0 +1,188 @@
+/**
+ * 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;
+
+import org.apache.commons.io.input.buffer.CircularByteBuffer;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.concurrent.Callable;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * Message Queue to Pipe output from one output stream to another inputstream.
+ */
+public class MessageQueue {
+  private CircularByteBuffer byteBuffer;
+  private AtomicBoolean isRunning;
+
+  private MessageOutputStream messageOutputStream;
+  private MessageInputStream messageInputStream;
+
+  public MessageQueue(int bufferSize, long pollIntervalMillis) {
+    this.pollIntervalMillis = pollIntervalMillis;
+    init(bufferSize);
+  }
+
+  private void init(int bufferSize) {
+    this.byteBuffer = new CircularByteBuffer(bufferSize);
+    this.isRunning = new AtomicBoolean(false);
+    this.messageInputStream = new MessageInputStream(this);
+    this.messageOutputStream = new MessageOutputStream(this);
+  }
+
+  public void start() {
+    this.isRunning.set(true);
+  }
+
+  public void stop() {
+    this.isRunning.set(false);
+  }
+
+  public MessageOutputStream getMessageOutputStream() {
+    return messageOutputStream;
+  }
+
+  public MessageInputStream getMessageInputStream() {
+    return messageInputStream;
+  }
+
+  private long pollIntervalMillis;
+
+  public boolean isRunning() {
+    return isRunning.get();
+  }
+
+  private long getPollIntervalMillis() {
+    return pollIntervalMillis;
+  }
+
+  private boolean hasSpace(int requiredLength) {
+    return this.byteBuffer.hasSpace(requiredLength);
+  }
+
+  private boolean hasBytes() {
+    return this.byteBuffer.hasBytes();
+  }
+
+  private int getCurrentNumberOfBytes() {
+    return this.byteBuffer.getCurrentNumberOfBytes();
+  }
+
+  private void add(byte[] b, int off, int len) {
+    this.byteBuffer.add(b, off, len);
+  }
+
+  private int read(byte[] b, int off, int len) {
+    this.byteBuffer.read(b, off, len);
+    return len;
+  }
+
+  private static  <T> T callWithLock(Lock lock, Callable<T> callable)
+          throws Exception {
+    lock.lock();
+    try {
+      return callable.call();
+    } finally {
+      lock.unlock();
+    }
+  }
+  private static final class MessageOutputStream extends OutputStream {
+
+    private MessageQueue messageQueue;
+    private Lock writeLock;
+
+    private MessageOutputStream(MessageQueue messageQueue) {
+      this.messageQueue = messageQueue;
+      this.writeLock = new ReentrantLock();
+

Review Comment:
   removed



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] swamirishi commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "swamirishi (via GitHub)" <gi...@apache.org>.
swamirishi commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1132781324


##########
hadoop-hdds/rocks-native/src/main/native/PipeInputStream.cpp:
##########
@@ -0,0 +1,48 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.
+ */
+
+#include <unistd.h>
+#include <iostream>
+#include "Pipe.h"
+#include "cplusplus_to_java_convert.h"
+#include "org_apache_hadoop_hdds_utils_db_managed_PipeInputStream.h"
+
+
+jlong Java_org_apache_hadoop_hdds_utils_db_managed_PipeInputStream_newPipe(JNIEnv *, jobject) {
+    Pipe* pipe = new Pipe();
+    return GET_CPLUSPLUS_POINTER(pipe);
+}
+
+jint Java_org_apache_hadoop_hdds_utils_db_managed_PipeInputStream_readInternal(JNIEnv *env, jobject object, jbyteArray jbyteArray, jint capacity, jlong nativeHandle) {
+    int cap_int = capacity;
+    Pipe* pipe = reinterpret_cast<Pipe*>(nativeHandle);
+    jbyte* b = (env)->GetByteArrayElements(jbyteArray, JNI_FALSE);
+    cap_int = read(pipe->getReadFd(), b, cap_int);
+    if (cap_int == 0) {
+        if (!pipe->isOpen()) {
+            cap_int = -1;
+        }
+    }
+    (env)->ReleaseByteArrayElements(jbyteArray, b, 0);
+    return cap_int;
+}
+
+void Java_org_apache_hadoop_hdds_utils_db_managed_PipeInputStream_closeInternal(JNIEnv *env, jobject object, jlong nativeHandle) {
+    delete reinterpret_cast<Pipe*>(nativeHandle);

Review Comment:
   done



##########
hadoop-hdds/rocks-native/src/main/native/PipeInputStream.cpp:
##########
@@ -0,0 +1,48 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.
+ */
+
+#include <unistd.h>
+#include <iostream>
+#include "Pipe.h"
+#include "cplusplus_to_java_convert.h"
+#include "org_apache_hadoop_hdds_utils_db_managed_PipeInputStream.h"
+
+
+jlong Java_org_apache_hadoop_hdds_utils_db_managed_PipeInputStream_newPipe(JNIEnv *, jobject) {
+    Pipe* pipe = new Pipe();
+    return GET_CPLUSPLUS_POINTER(pipe);
+}
+
+jint Java_org_apache_hadoop_hdds_utils_db_managed_PipeInputStream_readInternal(JNIEnv *env, jobject object, jbyteArray jbyteArray, jint capacity, jlong nativeHandle) {
+    int cap_int = capacity;
+    Pipe* pipe = reinterpret_cast<Pipe*>(nativeHandle);
+    jbyte* b = (env)->GetByteArrayElements(jbyteArray, JNI_FALSE);

Review Comment:
   done



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] swamirishi commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "swamirishi (via GitHub)" <gi...@apache.org>.
swamirishi commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1132816788


##########
hadoop-hdds/rocks-native/src/main/java/org/apache/hadoop/hdds/utils/db/managed/ManagedSSTDumpTool.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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 org.apache.hadoop.hdds.utils.NativeLibraryLoader;
+import org.apache.hadoop.hdds.utils.NativeLibraryNotLoadedException;
+
+import java.util.Map;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+
+import static org.apache.hadoop.hdds.utils.NativeConstants.ROCKS_TOOLS_NATIVE_LIBRARY_NAME;
+
+/**
+ * JNI for RocksDB SSTDumpTool. Pipes the output to an output stream
+ */
+public class ManagedSSTDumpTool {
+
+  static {
+    NativeLibraryLoader.getInstance()
+            .loadLibrary(ROCKS_TOOLS_NATIVE_LIBRARY_NAME);
+  }
+  private int bufferCapacity;
+  private ExecutorService executorService;
+
+  public ManagedSSTDumpTool(ExecutorService executorService,
+                            int bufferCapacity)
+          throws NativeLibraryNotLoadedException {
+    if (!NativeLibraryLoader.getInstance()
+            .isLibraryLoaded(ROCKS_TOOLS_NATIVE_LIBRARY_NAME)) {
+      throw new NativeLibraryNotLoadedException(
+              ROCKS_TOOLS_NATIVE_LIBRARY_NAME);
+    }
+    this.bufferCapacity = bufferCapacity;
+    this.executorService = executorService;
+  }
+
+  public SSTDumpToolTask run(String[] args, ManagedOptions options)
+          throws NativeLibraryNotLoadedException {
+    PipeInputStream pipeInputStream = new PipeInputStream(bufferCapacity);
+    return new SSTDumpToolTask(this.executorService.submit(() ->
+            this.runInternal(args, options.getNativeHandle(),
+            pipeInputStream.getNativeHandle())), pipeInputStream);

Review Comment:
   We are submitting this to a threadpool, so we shouldn't surround it here.



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] swamirishi commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "swamirishi (via GitHub)" <gi...@apache.org>.
swamirishi commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1132787211


##########
hadoop-hdds/rocks-native/src/main/java/org/apache/hadoop/hdds/utils/db/managed/ManagedSSTDumpIterator.java:
##########
@@ -0,0 +1,238 @@
+/**
+ * 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 org.apache.hadoop.hdds.utils.NativeLibraryNotLoadedException;
+import org.eclipse.jetty.io.RuntimeIOException;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
+import java.util.Iterator;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * Iterator to Parse output of RocksDBSSTDumpTool.
+ */
+public class ManagedSSTDumpIterator implements
+        Iterator<ManagedSSTDumpIterator.KeyValue>, AutoCloseable {
+  private static final String SST_DUMP_TOOL_CLASS =
+          "org.apache.hadoop.hdds.utils.db.managed.ManagedSSTDumpTool";
+  private static final String PATTERN_REGEX =
+          "'([^=>]+)' seq:([0-9]+), type:([0-9]+) =>";
+
+  public static final int PATTERN_KEY_GROUP_NUMBER = 1;
+  public static final int PATTERN_SEQ_GROUP_NUMBER = 2;
+  public static final int PATTERN_TYPE_GROUP_NUMBER = 3;
+  private static final Pattern PATTERN_MATCHER =
+          Pattern.compile(PATTERN_REGEX);
+  private BufferedReader processOutput;
+  private StringBuilder stdoutString;
+
+  private Matcher currentMatcher;
+  private int prevMatchEndIndex;
+  private KeyValue currentKey;
+  private char[] charBuffer;
+  private KeyValue nextKey;
+
+  private long pollIntervalMillis;
+  private ManagedSSTDumpTool.SSTDumpToolTask sstDumpToolTask;
+  private Lock lock;
+  private AtomicBoolean open;
+
+
+  public ManagedSSTDumpIterator(ManagedSSTDumpTool sstDumpTool,
+                                String sstFilePath,
+                                ManagedOptions options,
+                                long pollIntervalMillis) throws IOException,
+          NativeLibraryNotLoadedException {
+    File sstFile = new File(sstFilePath);
+    if (!sstFile.exists() || !sstFile.isFile()) {
+      throw new IOException(String.format("Invalid SST File Path : %s",
+              sstFile.getAbsolutePath()));
+    }
+    this.pollIntervalMillis = pollIntervalMillis;
+    this.lock = new ReentrantLock();
+    init(sstDumpTool, sstFile, options);
+  }
+
+  private void init(ManagedSSTDumpTool sstDumpTool, File sstFile,
+                    ManagedOptions options)
+          throws NativeLibraryNotLoadedException {
+    String[] args = {"--file=" + sstFile.getAbsolutePath(),
+                     "--command=scan"};
+    this.sstDumpToolTask = sstDumpTool.run(args, options);
+    processOutput = new BufferedReader(new InputStreamReader(
+            sstDumpToolTask.getPipedOutput(), StandardCharsets.UTF_8));
+    stdoutString = new StringBuilder();
+    currentMatcher = PATTERN_MATCHER.matcher(stdoutString);
+    charBuffer = new char[8192];
+    open = new AtomicBoolean(true);
+    next();
+  }
+
+  /**
+   * Throws Runtime exception in the case iterator is closed or
+   * the native Dumptool exited with non zero exit value.
+   */
+  private void checkSanityOfProcess() {
+    if (!this.open.get()) {
+      throw new RuntimeException("Iterator has been closed");
+    }
+    if (sstDumpToolTask.getFuture().isDone()
+            && sstDumpToolTask.exitValue() != 0) {
+      throw new RuntimeException("Process Terminated with non zero " +
+              String.format("exit value %d", sstDumpToolTask.exitValue()));
+    }
+  }
+
+  /**
+   *
+   * @return
+   * Throws Runtime Exception in case of SST File read failure
+   */
+
+  @Override
+  public boolean hasNext() {
+    checkSanityOfProcess();
+    return nextKey != null;
+  }
+
+  /**
+   *
+   * @return next Key
+   * Throws Runtime Exception incase of failure.
+   */
+  @Override
+  public KeyValue next() {
+    lock.lock();
+    try {
+      checkSanityOfProcess();
+      currentKey = nextKey;
+      nextKey = null;
+      while (!currentMatcher.find()) {
+        try {
+          if (prevMatchEndIndex != 0) {
+            stdoutString = new StringBuilder(stdoutString.substring(
+                    prevMatchEndIndex, stdoutString.length()));
+            prevMatchEndIndex = 0;
+            currentMatcher = PATTERN_MATCHER.matcher(stdoutString);
+          }
+          Thread.sleep(pollIntervalMillis);
+          int numberOfCharsRead = processOutput.read(charBuffer);
+          if (numberOfCharsRead < 0) {
+            if (currentKey != null) {
+              currentKey.setValue(stdoutString.toString());
+            }
+            return currentKey;
+          }
+          stdoutString.append(charBuffer, 0, numberOfCharsRead);
+          currentMatcher.reset();
+        } catch (IOException | InterruptedException e) {
+          throw new RuntimeIOException(e);
+        }
+      }
+      if (currentKey != null) {
+        currentKey.setValue(stdoutString.substring(prevMatchEndIndex,
+                currentMatcher.start()));
+      }
+      prevMatchEndIndex = currentMatcher.end();
+      nextKey =  new KeyValue(
+              currentMatcher.group(PATTERN_KEY_GROUP_NUMBER),
+              currentMatcher.group(PATTERN_SEQ_GROUP_NUMBER),
+              currentMatcher.group(PATTERN_TYPE_GROUP_NUMBER));
+      return currentKey;
+    } finally {
+      lock.unlock();
+    }
+  }
+
+  @Override
+  public synchronized void close() throws Exception {
+    lock.lock();
+    try {
+      if (this.sstDumpToolTask != null) {
+        if (!this.sstDumpToolTask.getFuture().isDone()) {
+          this.sstDumpToolTask.getFuture().cancel(true);
+        }
+        this.processOutput.close();
+      }
+      open.compareAndSet(true, false);
+    } finally {
+      lock.unlock();
+    }
+  }
+
+  @Override
+  protected void finalize() throws Throwable {
+    this.close();
+  }
+
+  /**
+   * Class containing Parsed KeyValue Record from Sst Dumptool output.
+   */
+  public static final class KeyValue {

Review Comment:
   Yeah will move this in the snapdiff PR



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1129936935


##########
hadoop-hdds/rocks-native/pom.xml:
##########
@@ -0,0 +1,405 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed 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
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+  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. See accompanying LICENSE file.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <parent>
+        <artifactId>hdds</artifactId>
+        <groupId>org.apache.ozone</groupId>
+        <version>1.4.0-SNAPSHOT</version>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+    <name>Apache Ozone HDDS RocksDB Tools</name>
+    <artifactId>hdds-rocks-native</artifactId>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.ozone</groupId>
+            <artifactId>hdds-managed-rocksdb</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.eclipse.jetty</groupId>
+            <artifactId>jetty-io</artifactId>
+        </dependency>
+    </dependencies>
+    <properties>
+        <maven.compiler.source>8</maven.compiler.source>
+        <maven.compiler.target>8</maven.compiler.target>
+    </properties>
+
+    <profiles>
+        <profile>
+            <id>rocks_tools_native</id>
+            <activation>
+                <property>
+                    <name>rocks_tools_native</name>
+                </property>
+            </activation>
+            <properties>
+                <cmake.standards>23</cmake.standards>
+                <sstDump.include>true</sstDump.include>
+            </properties>
+            <build>
+                <plugins>
+                    <plugin>
+                        <groupId>com.googlecode.maven-download-plugin</groupId>
+                        <artifactId>download-maven-plugin</artifactId>
+                        <executions>
+                            <execution>
+                                <id>rocksdb source download</id>
+                                <phase>generate-sources</phase>
+                                <goals>
+                                    <goal>wget</goal>
+                                </goals>
+                                <configuration>
+                                    <url>https://github.com/facebook/rocksdb/archive/refs/tags/v${rocksdb.version}.tar.gz</url>
+                                    <outputFileName>rocksdb-v${rocksdb.version}.tar.gz</outputFileName>
+                                    <outputDirectory>${project.build.directory}/rocksdb</outputDirectory>
+                                </configuration>
+                            </execution>
+                            <execution>
+                                <id>zlib source download</id>
+                                <phase>generate-sources</phase>
+                                <goals>
+                                    <goal>wget</goal>
+                                </goals>
+                                <configuration>
+                                    <url>https://zlib.net/zlib-${zlib.version}.tar.gz</url>
+                                    <outputFileName>zlib-${zlib.version}.tar.gz</outputFileName>
+                                    <outputDirectory>${project.build.directory}/zlib</outputDirectory>
+                                </configuration>
+                            </execution>
+                            <execution>
+                                <id>bzip2 source download</id>
+                                <phase>generate-sources</phase>
+                                <goals>
+                                    <goal>wget</goal>
+                                </goals>
+                                <configuration>
+                                    <url>https://sourceware.org/pub/bzip2/bzip2-${bzip2.version}.tar.gz</url>
+                                    <outputFileName>bzip2-v${bzip2.version}.tar.gz</outputFileName>
+                                    <outputDirectory>${project.build.directory}/bzip2</outputDirectory>
+                                </configuration>
+                            </execution>
+                            <execution>
+                                <id>lz4 source download</id>
+                                <phase>generate-sources</phase>
+                                <goals>
+                                    <goal>wget</goal>
+                                </goals>
+                                <configuration>
+                                    <url>https://github.com/lz4/lz4/archive/refs/tags/v${lz4.version}.tar.gz</url>
+                                    <outputFileName>lz4-v${lz4.version}.tar.gz</outputFileName>
+                                    <outputDirectory>${project.build.directory}/lz4</outputDirectory>
+                                </configuration>
+                            </execution>
+                            <execution>
+                                <id>snappy source download</id>
+                                <phase>generate-sources</phase>
+                                <goals>
+                                    <goal>wget</goal>
+                                </goals>
+                                <configuration>
+                                    <url>https://github.com/google/snappy/archive/refs/tags/${snappy.version}.tar.gz</url>
+                                    <outputFileName>snappy-v${snappy.version}.tar.gz</outputFileName>
+                                    <outputDirectory>${project.build.directory}/snappy</outputDirectory>
+                                </configuration>
+                            </execution>
+                            <execution>
+                                <id>zstd source download</id>
+                                <phase>generate-sources</phase>
+                                <goals>
+                                    <goal>wget</goal>
+                                </goals>
+                                <configuration>
+                                    <url>https://github.com/facebook/zstd/archive/refs/tags/v${zstd.version}.tar.gz</url>
+                                    <outputFileName>zstd-v${zstd.version}.tar.gz</outputFileName>
+                                    <outputDirectory>${project.build.directory}/zstd</outputDirectory>
+                                </configuration>
+                            </execution>
+                        </executions>
+                    </plugin>
+                    <plugin>
+                        <groupId>org.apache.maven.plugins</groupId>
+                        <artifactId>maven-patch-plugin</artifactId>
+                        <version>1.1.1</version>
+                        <configuration>
+                            <patchFile>${basedir}/src/main/patches/rocks-native.patch</patchFile>
+                            <strip>1</strip>
+                            <targetDirectory>${project.build.directory}/rocksdb/rocksdb-${rocksdb.version}</targetDirectory>
+                        </configuration>
+                        <executions>
+                            <execution>
+                                <id>patch</id>
+                                <phase>process-sources</phase>
+                                <goals>
+                                    <goal>apply</goal>
+                                </goals>
+                            </execution>
+                        </executions>
+                    </plugin>
+                    <plugin>
+                        <artifactId>maven-antrun-plugin</artifactId>
+                        <executions>
+                            <execution>
+                                <id>unzip-artifact</id>
+                                <phase>generate-sources</phase>
+                                <configuration>
+                                    <tasks>
+                                        <untar src="${project.build.directory}/rocksdb/rocksdb-v${rocksdb.version}.tar.gz" compression="gzip" dest="${project.build.directory}/rocksdb/" />
+                                        <untar src="${project.build.directory}/zlib/zlib-${zlib.version}.tar.gz" compression="gzip" dest="${project.build.directory}/zlib/" />
+                                        <untar src="${project.build.directory}/bzip2/bzip2-v${bzip2.version}.tar.gz" compression="gzip" dest="${project.build.directory}/bzip2/" />
+                                        <untar src="${project.build.directory}/lz4/lz4-v${lz4.version}.tar.gz" compression="gzip" dest="${project.build.directory}/lz4/" />
+                                        <untar src="${project.build.directory}/snappy/snappy-v${snappy.version}.tar.gz" compression="gzip" dest="${project.build.directory}/snappy/" />
+                                        <untar src="${project.build.directory}/zstd/zstd-v${zstd.version}.tar.gz" compression="gzip" dest="${project.build.directory}/zstd/" />
+                                    </tasks>
+                                </configuration>
+                                <goals>
+                                    <goal>run</goal>
+                                </goals>
+                            </execution>
+                            <execution>
+                                <id>build-zlib</id>
+                                <phase>process-sources</phase>
+                                <configuration>
+                                    <tasks>
+                                        <chmod file="${project.build.directory}/zlib/zlib-${zlib.version}/configure" perm="775" />
+                                        <exec executable="./configure" dir="${project.build.directory}/zlib/zlib-${zlib.version}" failonerror="true">
+                                            <arg line="--static"/>
+                                            <env key="CFLAGS" value="-fPIC"/>
+                                        </exec>
+                                        <exec executable="make" dir="${project.build.directory}/zlib/zlib-${zlib.version}" failonerror="true"/>
+                                    </tasks>
+                                </configuration>
+                                <goals>
+                                    <goal>run</goal>
+                                </goals>
+                            </execution>
+                            <execution>
+                                <id>build-bzip2</id>
+                                <phase>process-sources</phase>
+                                <configuration>
+                                    <tasks>
+                                        <exec executable="make" dir="${project.build.directory}/bzip2/bzip2-${bzip2.version}" failonerror="true">
+                                            <arg line="CFLAGS='-fPIC'"/>
+                                        </exec>
+                                    </tasks>
+                                </configuration>
+                                <goals>
+                                    <goal>run</goal>
+                                </goals>
+                            </execution>
+                            <execution>
+                                <id>build-lz4</id>
+                                <phase>process-sources</phase>
+                                <configuration>
+                                    <tasks>
+                                        <exec executable="make" dir="${project.build.directory}/lz4/lz4-${lz4.version}" failonerror="true">
+                                            <arg line="CFLAGS='-fPIC'"/>
+                                        </exec>
+                                    </tasks>
+                                </configuration>
+                                <goals>
+                                    <goal>run</goal>
+                                </goals>
+                            </execution>
+                            <execution>
+                                <id>build-zstd</id>
+                                <phase>process-sources</phase>
+                                <configuration>
+                                    <tasks>
+                                        <exec executable="make" dir="${project.build.directory}/zstd/zstd-${zstd.version}" failonerror="true">
+                                            <arg line="CFLAGS='-fPIC'"/>
+                                        </exec>
+                                    </tasks>
+                                </configuration>
+                                <goals>
+                                    <goal>run</goal>
+                                </goals>
+                            </execution>
+                            <execution>
+                                <id>build-snappy</id>
+                                <phase>process-sources</phase>
+                                <configuration>
+                                    <tasks>
+                                        <mkdir dir="${project.build.directory}/snappy/lib"/>
+                                        <exec executable="cmake" failonerror="true" dir="${project.build.directory}/snappy/lib">
+                                            <arg line="${project.build.directory}/snappy/snappy-${snappy.version}"/>
+                                            <env key="CFLAGS" value="-fPIC"/>
+                                            <env key="CXXFLAGS" value="-fPIC"/>
+                                        </exec>
+                                        <exec executable="make" dir="${project.build.directory}/snappy/lib" failonerror="true"/>
+                                    </tasks>
+                                </configuration>
+                                <goals>
+                                    <goal>run</goal>
+                                </goals>
+                            </execution>
+                            <execution>
+                                <id>build-rocksjava</id>
+                                <phase>process-resources</phase>
+                                <configuration>
+                                    <tasks>
+                                        <exec executable="chmod" failonerror="true">
+                                            <arg line="-R"/>
+                                            <arg line="775"/>
+                                            <arg line="${project.build.directory}/rocksdb/rocksdb-${rocksdb.version}"/>
+                                        </exec>
+                                        <exec executable="make" dir="${project.build.directory}/rocksdb/rocksdb-${rocksdb.version}" failonerror="true">
+                                            <arg line="EXTRA_CXXFLAGS='-fPIC -I${project.build.directory}/snappy/lib -I${project.build.directory}/snappy/snappy-${snappy.version} -I${project.build.directory}/lz4/lz4-${lz4.version}/lib -I${project.build.directory}/zstd/zstd-${zstd.version}/lib -I${project.build.directory}/zstd/zstd-${zstd.version}/lib/dictBuilder -I${project.build.directory}/bzip2/bzip2-${bzip2.version} -I${project.build.directory}/zlib/zlib-${zlib.version}'"/>
+                                            <arg line="EXTRA_LDFLAGS='-L${project.build.directory}/snappy/lib -L${project.build.directory}/lz4/lz4-${lz4.version}/lib -L${project.build.directory}/zstd/zstd-${zstd.version}/lib -L${project.build.directory}/bzip2/bzip2-${bzip2.version} -L${project.build.directory}/zlib/zlib-${zlib.version}'"/>
+                                            <arg line="-j4"/>

Review Comment:
   Is this the fastest the Github worker can run at with 7GB RAM and 2-core?
   Does `-j5` or `-j6` yield any build time reduction?



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1129978217


##########
hadoop-hdds/rocks-native/src/main/native/PipeInputStream.cpp:
##########
@@ -0,0 +1,48 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.
+ */
+
+#include <unistd.h>
+#include <iostream>
+#include "Pipe.h"
+#include "cplusplus_to_java_convert.h"
+#include "org_apache_hadoop_hdds_utils_db_managed_PipeInputStream.h"
+
+
+jlong Java_org_apache_hadoop_hdds_utils_db_managed_PipeInputStream_newPipe(JNIEnv *, jobject) {
+    Pipe* pipe = new Pipe();
+    return GET_CPLUSPLUS_POINTER(pipe);
+}
+
+jint Java_org_apache_hadoop_hdds_utils_db_managed_PipeInputStream_readInternal(JNIEnv *env, jobject object, jbyteArray jbyteArray, jint capacity, jlong nativeHandle) {
+    int cap_int = capacity;
+    Pipe* pipe = reinterpret_cast<Pipe*>(nativeHandle);
+    jbyte* b = (env)->GetByteArrayElements(jbyteArray, JNI_FALSE);

Review Comment:
   nit
   ```suggestion
       Pipe *pipe = reinterpret_cast<Pipe *>(nativeHandle);
       jbyte *b = (env)->GetByteArrayElements(jbyteArray, JNI_FALSE);
   ```



##########
hadoop-hdds/rocks-native/src/main/native/PipeInputStream.cpp:
##########
@@ -0,0 +1,48 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.
+ */
+
+#include <unistd.h>
+#include <iostream>
+#include "Pipe.h"
+#include "cplusplus_to_java_convert.h"
+#include "org_apache_hadoop_hdds_utils_db_managed_PipeInputStream.h"
+
+
+jlong Java_org_apache_hadoop_hdds_utils_db_managed_PipeInputStream_newPipe(JNIEnv *, jobject) {
+    Pipe* pipe = new Pipe();
+    return GET_CPLUSPLUS_POINTER(pipe);
+}
+
+jint Java_org_apache_hadoop_hdds_utils_db_managed_PipeInputStream_readInternal(JNIEnv *env, jobject object, jbyteArray jbyteArray, jint capacity, jlong nativeHandle) {
+    int cap_int = capacity;
+    Pipe* pipe = reinterpret_cast<Pipe*>(nativeHandle);
+    jbyte* b = (env)->GetByteArrayElements(jbyteArray, JNI_FALSE);
+    cap_int = read(pipe->getReadFd(), b, cap_int);
+    if (cap_int == 0) {
+        if (!pipe->isOpen()) {
+            cap_int = -1;
+        }
+    }
+    (env)->ReleaseByteArrayElements(jbyteArray, b, 0);
+    return cap_int;
+}
+
+void Java_org_apache_hadoop_hdds_utils_db_managed_PipeInputStream_closeInternal(JNIEnv *env, jobject object, jlong nativeHandle) {
+    delete reinterpret_cast<Pipe*>(nativeHandle);

Review Comment:
   ```suggestion
       delete reinterpret_cast<Pipe *>(nativeHandle);
   ```



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1129959408


##########
hadoop-hdds/rocks-native/src/main/native/Pipe.cpp:
##########
@@ -0,0 +1,37 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.
+ */
+
+#include "Pipe.h"
+#include <unistd.h>
+
+const int Pipe::READ_FILE_DESCRIPTOR_IDX = 0;
+const int Pipe::WRITE_FILE_DESCRIPTOR_IDX = 1;
+
+Pipe::Pipe() {
+    pipe(p);
+    open = true;
+}
+
+Pipe::~Pipe() {
+    ::close(p[0]);
+    ::close(p[1]);
+}
+
+void Pipe::close() {
+    open = false;
+}

Review Comment:
   new line at EOF
   
   ```suggestion
   }
   
   ```



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] swamirishi commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "swamirishi (via GitHub)" <gi...@apache.org>.
swamirishi commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1130003367


##########
hadoop-hdds/rocks-native/src/main/java/org/apache/hadoop/hdds/utils/db/managed/ManagedSSTDumpIterator.java:
##########
@@ -0,0 +1,238 @@
+/**
+ * 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 org.apache.hadoop.hdds.utils.NativeLibraryNotLoadedException;
+import org.eclipse.jetty.io.RuntimeIOException;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
+import java.util.Iterator;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * Iterator to Parse output of RocksDBSSTDumpTool.
+ */
+public class ManagedSSTDumpIterator implements
+        Iterator<ManagedSSTDumpIterator.KeyValue>, AutoCloseable {
+  private static final String SST_DUMP_TOOL_CLASS =
+          "org.apache.hadoop.hdds.utils.db.managed.ManagedSSTDumpTool";
+  private static final String PATTERN_REGEX =
+          "'([^=>]+)' seq:([0-9]+), type:([0-9]+) =>";
+
+  public static final int PATTERN_KEY_GROUP_NUMBER = 1;
+  public static final int PATTERN_SEQ_GROUP_NUMBER = 2;
+  public static final int PATTERN_TYPE_GROUP_NUMBER = 3;
+  private static final Pattern PATTERN_MATCHER =
+          Pattern.compile(PATTERN_REGEX);
+  private BufferedReader processOutput;
+  private StringBuilder stdoutString;
+
+  private Matcher currentMatcher;
+  private int prevMatchEndIndex;
+  private KeyValue currentKey;
+  private char[] charBuffer;
+  private KeyValue nextKey;
+
+  private long pollIntervalMillis;
+  private ManagedSSTDumpTool.SSTDumpToolTask sstDumpToolTask;
+  private Lock lock;
+  private AtomicBoolean open;
+
+
+  public ManagedSSTDumpIterator(ManagedSSTDumpTool sstDumpTool,
+                                String sstFilePath,
+                                ManagedOptions options,
+                                long pollIntervalMillis) throws IOException,
+          NativeLibraryNotLoadedException {
+    File sstFile = new File(sstFilePath);
+    if (!sstFile.exists() || !sstFile.isFile()) {
+      throw new IOException(String.format("Invalid SST File Path : %s",
+              sstFile.getAbsolutePath()));
+    }
+    this.pollIntervalMillis = pollIntervalMillis;
+    this.lock = new ReentrantLock();
+    init(sstDumpTool, sstFile, options);
+  }
+
+  private void init(ManagedSSTDumpTool sstDumpTool, File sstFile,
+                    ManagedOptions options)
+          throws NativeLibraryNotLoadedException {
+    String[] args = {"--file=" + sstFile.getAbsolutePath(),
+                     "--command=scan"};
+    this.sstDumpToolTask = sstDumpTool.run(args, options);
+    processOutput = new BufferedReader(new InputStreamReader(
+            sstDumpToolTask.getPipedOutput(), StandardCharsets.UTF_8));
+    stdoutString = new StringBuilder();
+    currentMatcher = PATTERN_MATCHER.matcher(stdoutString);
+    charBuffer = new char[8192];
+    open = new AtomicBoolean(true);
+    next();
+  }
+
+  /**
+   * Throws Runtime exception in the case iterator is closed or
+   * the native Dumptool exited with non zero exit value.
+   */
+  private void checkSanityOfProcess() {
+    if (!this.open.get()) {
+      throw new RuntimeException("Iterator has been closed");
+    }
+    if (sstDumpToolTask.getFuture().isDone()
+            && sstDumpToolTask.exitValue() != 0) {
+      throw new RuntimeException("Process Terminated with non zero " +
+              String.format("exit value %d", sstDumpToolTask.exitValue()));
+    }
+  }
+
+  /**
+   *
+   * @return
+   * Throws Runtime Exception in case of SST File read failure
+   */
+
+  @Override
+  public boolean hasNext() {

Review Comment:
   done



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] swamirishi commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "swamirishi (via GitHub)" <gi...@apache.org>.
swamirishi commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1129955441


##########
hadoop-hdds/rocks-native/src/main/java/org/apache/hadoop/hdds/utils/db/managed/ManagedSSTDumpTool.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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 org.apache.hadoop.hdds.utils.NativeLibraryLoader;
+import org.apache.hadoop.hdds.utils.NativeLibraryNotLoadedException;
+
+import java.util.Map;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+
+import static org.apache.hadoop.hdds.utils.NativeConstants.ROCKS_TOOLS_NATIVE_LIBRARY_NAME;
+
+/**
+ * JNI for RocksDB SSTDumpTool. Pipes the output to an output stream
+ */
+public class ManagedSSTDumpTool {
+
+  static {
+    NativeLibraryLoader.getInstance()
+            .loadLibrary(ROCKS_TOOLS_NATIVE_LIBRARY_NAME);
+  }
+  private int bufferCapacity;
+  private ExecutorService executorService;
+
+  public ManagedSSTDumpTool(ExecutorService executorService,
+                            int bufferCapacity)
+          throws NativeLibraryNotLoadedException {
+    if (!NativeLibraryLoader.getInstance()
+            .isLibraryLoaded(ROCKS_TOOLS_NATIVE_LIBRARY_NAME)) {
+      throw new NativeLibraryNotLoadedException(
+              ROCKS_TOOLS_NATIVE_LIBRARY_NAME);
+    }
+    this.bufferCapacity = bufferCapacity;
+    this.executorService = executorService;
+  }
+
+  public SSTDumpToolTask run(String[] args, ManagedOptions options)
+          throws NativeLibraryNotLoadedException {
+    PipeInputStream pipeInputStream = new PipeInputStream(bufferCapacity);
+    return new SSTDumpToolTask(this.executorService.submit(() ->
+            this.runInternal(args, options.getNativeHandle(),
+            pipeInputStream.getNativeHandle())), pipeInputStream);
+  }
+
+  public SSTDumpToolTask run(Map<String, String> args, ManagedOptions options)
+          throws NativeLibraryNotLoadedException {
+    return this.run(args.entrySet().stream().map(e -> "--"

Review Comment:
   this is to pass the arg to the sst dump tool



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1129957455


##########
hadoop-hdds/rocks-native/src/main/java/org/apache/hadoop/hdds/utils/db/managed/PipeInputStream.java:
##########
@@ -0,0 +1,81 @@
+package org.apache.hadoop.hdds.utils.db.managed;
+
+import org.apache.hadoop.hdds.utils.NativeLibraryLoader;
+import org.apache.hadoop.hdds.utils.NativeLibraryNotLoadedException;
+import java.io.InputStream;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.apache.hadoop.hdds.utils.NativeConstants.ROCKS_TOOLS_NATIVE_LIBRARY_NAME;
+
+/**
+ * JNI for reading data from pipe.
+ */
+public class PipeInputStream extends InputStream {
+
+  static {
+    NativeLibraryLoader.getInstance()
+            .loadLibrary(ROCKS_TOOLS_NATIVE_LIBRARY_NAME);
+  }
+  private byte[] byteBuffer;
+  private long nativeHandle;
+  private int numberOfBytesLeftToRead;
+  private int index = 0;
+  private int capacity;
+
+  private AtomicBoolean cleanup;
+
+  PipeInputStream(int capacity) throws NativeLibraryNotLoadedException {
+    if (!NativeLibraryLoader.getInstance()
+            .isLibraryLoaded(ROCKS_TOOLS_NATIVE_LIBRARY_NAME)) {
+      throw new NativeLibraryNotLoadedException(
+              ROCKS_TOOLS_NATIVE_LIBRARY_NAME);
+    }
+
+    this.byteBuffer = new byte[capacity];
+    this.numberOfBytesLeftToRead = 0;
+    this.capacity = capacity;
+    this.nativeHandle = newPipe();
+    this.cleanup = new AtomicBoolean(false);
+  }
+
+  long getNativeHandle() {
+    return nativeHandle;
+  }
+
+  @Override
+  public int read() {
+    if (numberOfBytesLeftToRead < 0) {
+      this.close();
+      return -1;
+    }
+    if (numberOfBytesLeftToRead == 0) {
+      numberOfBytesLeftToRead = readInternal(byteBuffer, capacity,
+              nativeHandle);
+      index = 0;
+      return read();
+    }
+    numberOfBytesLeftToRead--;
+    int ret = byteBuffer[index] & 0xFF;
+    index += 1;
+    return ret;
+  }
+
+  private native long newPipe();
+
+  private native int readInternal(byte[] buff, int numberOfBytes,
+                                  long pipeHandle);
+  private native void closeInternal(long pipeHandle);
+
+  @Override
+  public void close() {
+    if (this.cleanup.compareAndSet(false, true)) {
+      closeInternal(this.nativeHandle);
+    }
+  }
+
+  @Override
+  protected void finalize() throws Throwable {
+    close();
+    super.finalize();
+  }
+}

Review Comment:
   See HDDS-8077



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1129958781


##########
hadoop-hdds/rocks-native/src/main/java/org/apache/hadoop/hdds/utils/db/managed/package-info.java:
##########
@@ -0,0 +1,24 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.
+ */
+
+/**
+ * Annotation processors used at compile time by the Ozone project to validate
+ * internal annotations and related code as needed, if needed.
+ */
+
+package org.apache.hadoop.hdds.utils.db.managed;

Review Comment:
   HDDS-8077
   
   ```suggestion
   package org.apache.hadoop.hdds.utils.db.managed;
   
   ```



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1129982025


##########
hadoop-hdds/rocks-native/src/main/native/SSTDumpTool.cpp:
##########
@@ -0,0 +1,49 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.
+ */
+
+#include "org_apache_hadoop_hdds_utils_db_managed_ManagedSSTDumpTool.h"
+#include "rocksdb/options.h"
+#include "rocksdb/sst_dump_tool.h"
+#include <string>
+#include "cplusplus_to_java_convert.h"
+#include "Pipe.h"
+#include <iostream>
+
+jint Java_org_apache_hadoop_hdds_utils_db_managed_ManagedSSTDumpTool_runInternal(JNIEnv *env, jobject obj,
+ jobjectArray argsArray, jlong optionsHandle, jlong pipeHandle) {
+    ROCKSDB_NAMESPACE::SSTDumpTool dumpTool;
+    ROCKSDB_NAMESPACE::Options *options = reinterpret_cast<ROCKSDB_NAMESPACE::Options *>(optionsHandle);
+    Pipe *pipe = reinterpret_cast<Pipe *>(pipeHandle);
+    int length = env->GetArrayLength(argsArray);
+    char *args[length + 1];
+    args[0] = strdup("./sst_dump");
+    for (int i = 0; i < length; i++) {
+        jstring str_val = (jstring)env->GetObjectArrayElement(argsArray, (jsize)i);
+        char *utf_str = (char *)env->GetStringUTFChars(str_val, JNI_FALSE);
+        args[i + 1] = strdup(utf_str);
+        env->ReleaseStringUTFChars(str_val, utf_str);
+    }
+    FILE *wr = fdopen(pipe->getWriteFd(), "w");
+    int ret = dumpTool.Run(length + 1, args, *options, wr);
+    for (int i = 0; i < length + 1; i++) {
+        free(args[i]);
+    }
+    fclose(wr);
+    pipe->close();

Review Comment:
   Need to close `options` as well. e.g.
   
   https://github.com/facebook/rocksdb/blob/dec144f172165e9ce007aa12fbeaacf748cea5c1/java/src/test/java/org/rocksdb/SstFileReaderTest.java#L112



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #4315: HDDS-8028. JNI for RocksDB SST Dump tool

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4315:
URL: https://github.com/apache/ozone/pull/4315#discussion_r1129983604


##########
hadoop-hdds/rocks-native/src/main/native/SSTDumpTool.cpp:
##########
@@ -0,0 +1,49 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.
+ */
+
+#include "org_apache_hadoop_hdds_utils_db_managed_ManagedSSTDumpTool.h"
+#include "rocksdb/options.h"
+#include "rocksdb/sst_dump_tool.h"
+#include <string>
+#include "cplusplus_to_java_convert.h"
+#include "Pipe.h"
+#include <iostream>
+
+jint Java_org_apache_hadoop_hdds_utils_db_managed_ManagedSSTDumpTool_runInternal(JNIEnv *env, jobject obj,
+ jobjectArray argsArray, jlong optionsHandle, jlong pipeHandle) {
+    ROCKSDB_NAMESPACE::SSTDumpTool dumpTool;
+    ROCKSDB_NAMESPACE::Options *options = reinterpret_cast<ROCKSDB_NAMESPACE::Options *>(optionsHandle);
+    Pipe *pipe = reinterpret_cast<Pipe *>(pipeHandle);
+    int length = env->GetArrayLength(argsArray);
+    char *args[length + 1];
+    args[0] = strdup("./sst_dump");
+    for (int i = 0; i < length; i++) {
+        jstring str_val = (jstring)env->GetObjectArrayElement(argsArray, (jsize)i);
+        char *utf_str = (char *)env->GetStringUTFChars(str_val, JNI_FALSE);
+        args[i + 1] = strdup(utf_str);
+        env->ReleaseStringUTFChars(str_val, utf_str);
+    }
+    FILE *wr = fdopen(pipe->getWriteFd(), "w");
+    int ret = dumpTool.Run(length + 1, args, *options, wr);
+    for (int i = 0; i < length + 1; i++) {
+        free(args[i]);
+    }
+    fclose(wr);
+    pipe->close();

Review Comment:
   Since `pipe` is also passed in from the Java side. Should Java code be taking care of `pipe` as well? (i.e. don't close `pipe` in C++ code here)



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org