You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2022/08/26 05:44:48 UTC

[GitHub] [spark] Ngone51 commented on a diff in pull request #37610: [SPARK-38888][BUILD][CORE][YARN][DOCS] Add `RocksDB` support for shuffle state store

Ngone51 commented on code in PR #37610:
URL: https://github.com/apache/spark/pull/37610#discussion_r955586985


##########
common/network-common/src/main/java/org/apache/spark/network/shuffledb/RocksDBIterator.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.spark.network.shuffledb;
+
+import java.io.IOException;
+import java.util.AbstractMap;
+import java.util.Map;
+import java.util.NoSuchElementException;
+
+import com.google.common.base.Throwables;
+import org.rocksdb.RocksIterator;
+
+/**
+ * RocksDB implementation of `DBIterator`.
+ */
+public class RocksDBIterator implements DBIterator {
+
+    private final RocksIterator it;
+
+    private boolean checkedNext;
+
+    private boolean closed;
+
+    private Map.Entry<byte[], byte[]> next;
+
+    public RocksDBIterator(RocksIterator it) {
+        this.it = it;
+    }
+
+    @Override
+    public boolean hasNext() {
+      if (!checkedNext && !closed) {
+        next = loadNext();
+        checkedNext = true;
+      }
+      if (!closed && next == null) {
+       try {

Review Comment:
   2 indents?



##########
common/network-common/src/main/java/org/apache/spark/network/shuffledb/RocksDBIterator.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.spark.network.shuffledb;
+
+import java.io.IOException;
+import java.util.AbstractMap;
+import java.util.Map;
+import java.util.NoSuchElementException;
+
+import com.google.common.base.Throwables;
+import org.rocksdb.RocksIterator;
+
+/**
+ * RocksDB implementation of `DBIterator`.
+ */
+public class RocksDBIterator implements DBIterator {
+
+    private final RocksIterator it;
+
+    private boolean checkedNext;
+
+    private boolean closed;
+
+    private Map.Entry<byte[], byte[]> next;
+
+    public RocksDBIterator(RocksIterator it) {
+        this.it = it;
+    }
+
+    @Override
+    public boolean hasNext() {
+      if (!checkedNext && !closed) {
+        next = loadNext();
+        checkedNext = true;
+      }
+      if (!closed && next == null) {
+       try {
+         close();
+       } catch (IOException ioe) {
+           throw Throwables.propagate(ioe);
+       }
+      }
+      return next != null;
+    }
+
+    @Override
+    public Map.Entry<byte[], byte[]> next() {
+      if (!hasNext()) {
+        throw new NoSuchElementException();
+      }
+      checkedNext = false;
+      Map.Entry<byte[], byte[]> ret = next;
+      next = null;
+      return ret;
+    }
+
+    @Override
+    public void close() throws IOException {
+      if (!closed) {
+       it.close();

Review Comment:
   2 indents



##########
docs/spark-standalone.md:
##########
@@ -328,6 +328,17 @@ SPARK_WORKER_OPTS supports the following system properties:
   </td>
   <td>3.0.0</td>
 </tr>
+<tr>
+  <td><code>spark.shuffle.service.db.backend</code></td>
+  <td>LEVELDB</td>
+  <td>
+    When <code>spark.shuffle.service.db.enabled</code> is true, user can use this to specify the kind of disk-based 
+    store used in shuffle state store. This supports `LEVELDB` and `ROCKSDB` now and `LEVELDB` as default value. 
+    This only affects standalone mode (yarn always has this behavior enabled). 

Review Comment:
   Do you mean yarn always enable db or always use LEVELDB as its db? Could you make it more clear?



##########
common/network-common/src/main/java/org/apache/spark/network/util/RocksDBProvider.java:
##########
@@ -0,0 +1,170 @@
+/*
+ * 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.spark.network.util;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Objects;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.annotations.VisibleForTesting;
+import org.rocksdb.*;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.shuffledb.StoreVersion;
+
+/**
+ * RocksDB utility class available in the network package.
+ */
+public class RocksDBProvider {
+
+    static {
+      org.rocksdb.RocksDB.loadLibrary();
+    }
+
+    private static final Logger logger = LoggerFactory.getLogger(RocksDBProvider.class);
+
+    public static RocksDB initRockDB(File dbFile, StoreVersion version, ObjectMapper mapper) throws
+        IOException {
+      RocksDB tmpDb = null;
+      if (dbFile != null) {
+        BloomFilter fullFilter =
+          new BloomFilter(10.0D /* BloomFilter.DEFAULT_BITS_PER_KEY */, false);
+        BlockBasedTableConfig tableFormatConfig = new BlockBasedTableConfig()
+          .setFilterPolicy(fullFilter)
+          .setEnableIndexCompression(false)
+          .setIndexBlockRestartInterval(8)
+          .setFormatVersion(5);
+
+        Options dbOptions = new Options();
+        RocksDBLogger rocksDBLogger = new RocksDBLogger(dbOptions);
+
+        dbOptions.setCreateIfMissing(false);
+        dbOptions.setBottommostCompressionType(CompressionType.ZSTD_COMPRESSION);
+        dbOptions.setCompressionType(CompressionType.LZ4_COMPRESSION);
+        dbOptions.setTableFormatConfig(tableFormatConfig);
+        dbOptions.setLogger(rocksDBLogger);
+
+        try {
+          tmpDb = RocksDB.open(dbOptions, dbFile.toString());
+        } catch (RocksDBException e) {
+          if (e.getStatus().getCode() == Status.Code.NotFound) {
+            logger.info("Creating state database at " + dbFile);
+            dbOptions.setCreateIfMissing(true);
+            try {
+              tmpDb = RocksDB.open(dbOptions, dbFile.toString());
+            } catch (RocksDBException dbExc) {
+              throw new IOException("Unable to create state store", dbExc);
+            }
+          } else {
+            // the RocksDB file seems to be corrupt somehow.  Let's just blow it away and create
+            // a new one, so we can keep processing new apps
+            logger.error("error opening rocksdb file {}. Creating new file, will not be able to " +
+              "recover state for existing applications", dbFile, e);
+            if (dbFile.isDirectory()) {
+              for (File f : Objects.requireNonNull(dbFile.listFiles())) {
+                if (!f.delete()) {
+                  logger.warn("error deleting {}", f.getPath());
+                }
+              }
+            }
+            if (!dbFile.delete()) {
+              logger.warn("error deleting {}", dbFile.getPath());
+            }
+            dbOptions.setCreateIfMissing(true);

Review Comment:
   This will not take effect if `dbFile.delete()` failed?



##########
docs/configuration.md:
##########
@@ -1102,6 +1102,27 @@ Apart from these, the following properties are also available, and may be useful
   </td>
   <td>3.0.0</td>
 </tr>
+<tr>
+  <td><code>spark.shuffle.service.db.enabled</code></td>
+  <td>true</td>
+  <td>
+    To determine whether external shuffle uses db to store shuffle state.

Review Comment:
   ```suggestion
       To determine whether external shuffle service uses db to store shuffle state.
   ```



##########
docs/configuration.md:
##########
@@ -1102,6 +1102,27 @@ Apart from these, the following properties are also available, and may be useful
   </td>
   <td>3.0.0</td>
 </tr>
+<tr>
+  <td><code>spark.shuffle.service.db.enabled</code></td>
+  <td>true</td>
+  <td>
+    To determine whether external shuffle uses db to store shuffle state.
+    In standalone and Yarn modes, for more detail, see
+    <a href="spark-standalone.html#Cluster Launch Scripts">this description</a>.
+  </td>
+  <td>3.0.0</td>
+</tr>
+<tr>
+  <td><code>spark.shuffle.service.db.backend</code></td>
+  <td>LEVELDB</td>
+  <td>
+    To specify a disk-based store used in shuffle service local db when 
+    <code>spark.shuffle.service.db.enabled</code> is true, LEVELDB or ROCKSDB.
+    In standalone and Yarn modes, for more detail, see
+    <a href="spark-standalone.html#Cluster Launch Scripts">this description</a>.

Review Comment:
   ditto



##########
common/network-common/src/main/java/org/apache/spark/network/shuffledb/RocksDBIterator.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.spark.network.shuffledb;
+
+import java.io.IOException;
+import java.util.AbstractMap;
+import java.util.Map;
+import java.util.NoSuchElementException;
+
+import com.google.common.base.Throwables;
+import org.rocksdb.RocksIterator;
+
+/**
+ * RocksDB implementation of `DBIterator`.
+ */
+public class RocksDBIterator implements DBIterator {
+
+    private final RocksIterator it;
+
+    private boolean checkedNext;
+
+    private boolean closed;
+
+    private Map.Entry<byte[], byte[]> next;
+
+    public RocksDBIterator(RocksIterator it) {
+        this.it = it;
+    }
+
+    @Override
+    public boolean hasNext() {
+      if (!checkedNext && !closed) {
+        next = loadNext();
+        checkedNext = true;
+      }
+      if (!closed && next == null) {
+       try {
+         close();
+       } catch (IOException ioe) {
+           throw Throwables.propagate(ioe);

Review Comment:
   2 indents?



##########
common/network-common/src/main/java/org/apache/spark/network/util/RocksDBProvider.java:
##########
@@ -0,0 +1,170 @@
+/*
+ * 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.spark.network.util;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Objects;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.annotations.VisibleForTesting;
+import org.rocksdb.*;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.shuffledb.StoreVersion;
+
+/**
+ * RocksDB utility class available in the network package.
+ */
+public class RocksDBProvider {
+
+    static {
+      org.rocksdb.RocksDB.loadLibrary();
+    }
+
+    private static final Logger logger = LoggerFactory.getLogger(RocksDBProvider.class);
+
+    public static RocksDB initRockDB(File dbFile, StoreVersion version, ObjectMapper mapper) throws
+        IOException {
+      RocksDB tmpDb = null;
+      if (dbFile != null) {
+        BloomFilter fullFilter =
+          new BloomFilter(10.0D /* BloomFilter.DEFAULT_BITS_PER_KEY */, false);
+        BlockBasedTableConfig tableFormatConfig = new BlockBasedTableConfig()
+          .setFilterPolicy(fullFilter)
+          .setEnableIndexCompression(false)
+          .setIndexBlockRestartInterval(8)
+          .setFormatVersion(5);
+
+        Options dbOptions = new Options();
+        RocksDBLogger rocksDBLogger = new RocksDBLogger(dbOptions);
+
+        dbOptions.setCreateIfMissing(false);
+        dbOptions.setBottommostCompressionType(CompressionType.ZSTD_COMPRESSION);
+        dbOptions.setCompressionType(CompressionType.LZ4_COMPRESSION);
+        dbOptions.setTableFormatConfig(tableFormatConfig);
+        dbOptions.setLogger(rocksDBLogger);
+
+        try {
+          tmpDb = RocksDB.open(dbOptions, dbFile.toString());
+        } catch (RocksDBException e) {
+          if (e.getStatus().getCode() == Status.Code.NotFound) {
+            logger.info("Creating state database at " + dbFile);
+            dbOptions.setCreateIfMissing(true);
+            try {
+              tmpDb = RocksDB.open(dbOptions, dbFile.toString());
+            } catch (RocksDBException dbExc) {
+              throw new IOException("Unable to create state store", dbExc);
+            }
+          } else {
+            // the RocksDB file seems to be corrupt somehow.  Let's just blow it away and create
+            // a new one, so we can keep processing new apps
+            logger.error("error opening rocksdb file {}. Creating new file, will not be able to " +
+              "recover state for existing applications", dbFile, e);
+            if (dbFile.isDirectory()) {
+              for (File f : Objects.requireNonNull(dbFile.listFiles())) {

Review Comment:
   Is it possible to have sub-dir here?



##########
docs/spark-standalone.md:
##########
@@ -328,6 +328,17 @@ SPARK_WORKER_OPTS supports the following system properties:
   </td>
   <td>3.0.0</td>
 </tr>
+<tr>
+  <td><code>spark.shuffle.service.db.backend</code></td>
+  <td>LEVELDB</td>
+  <td>
+    When <code>spark.shuffle.service.db.enabled</code> is true, user can use this to specify the kind of disk-based 
+    store used in shuffle state store. This supports `LEVELDB` and `ROCKSDB` now and `LEVELDB` as default value. 
+    This only affects standalone mode (yarn always has this behavior enabled). 
+    The original data store in `LevelDB/RocksDB` will not be automatically convert to another kind of storage now.

Review Comment:
   Do you mean the data that is stored in LevelDB is not available after changing to RocksDB for example?



##########
common/network-common/src/main/java/org/apache/spark/network/shuffledb/RocksDBIterator.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.spark.network.shuffledb;
+
+import java.io.IOException;
+import java.util.AbstractMap;
+import java.util.Map;
+import java.util.NoSuchElementException;
+
+import com.google.common.base.Throwables;
+import org.rocksdb.RocksIterator;
+
+/**
+ * RocksDB implementation of `DBIterator`.
+ */
+public class RocksDBIterator implements DBIterator {
+
+    private final RocksIterator it;
+
+    private boolean checkedNext;
+
+    private boolean closed;
+
+    private Map.Entry<byte[], byte[]> next;
+
+    public RocksDBIterator(RocksIterator it) {
+        this.it = it;
+    }
+
+    @Override
+    public boolean hasNext() {
+      if (!checkedNext && !closed) {
+        next = loadNext();
+        checkedNext = true;
+      }
+      if (!closed && next == null) {
+       try {
+         close();
+       } catch (IOException ioe) {
+           throw Throwables.propagate(ioe);
+       }
+      }
+      return next != null;
+    }
+
+    @Override
+    public Map.Entry<byte[], byte[]> next() {
+      if (!hasNext()) {
+        throw new NoSuchElementException();
+      }
+      checkedNext = false;
+      Map.Entry<byte[], byte[]> ret = next;
+      next = null;
+      return ret;
+    }
+
+    @Override
+    public void close() throws IOException {
+      if (!closed) {
+       it.close();
+       closed = true;
+       next = null;
+      }
+    }
+
+    @Override
+    public void seek(byte[] key) {
+      it.seek(key);
+    }
+
+    @Override
+    public void remove() {

Review Comment:
   Wondering why we have a `remove()` (and even no parameter) API for the iterator? What's the expected behavior with a possible implementation?



##########
common/network-common/src/main/java/org/apache/spark/network/util/RocksDBProvider.java:
##########
@@ -0,0 +1,170 @@
+/*
+ * 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.spark.network.util;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Objects;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.annotations.VisibleForTesting;
+import org.rocksdb.*;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.shuffledb.StoreVersion;
+
+/**
+ * RocksDB utility class available in the network package.
+ */
+public class RocksDBProvider {
+
+    static {
+      org.rocksdb.RocksDB.loadLibrary();
+    }
+
+    private static final Logger logger = LoggerFactory.getLogger(RocksDBProvider.class);
+
+    public static RocksDB initRockDB(File dbFile, StoreVersion version, ObjectMapper mapper) throws
+        IOException {
+      RocksDB tmpDb = null;
+      if (dbFile != null) {
+        BloomFilter fullFilter =
+          new BloomFilter(10.0D /* BloomFilter.DEFAULT_BITS_PER_KEY */, false);
+        BlockBasedTableConfig tableFormatConfig = new BlockBasedTableConfig()
+          .setFilterPolicy(fullFilter)
+          .setEnableIndexCompression(false)
+          .setIndexBlockRestartInterval(8)
+          .setFormatVersion(5);
+
+        Options dbOptions = new Options();
+        RocksDBLogger rocksDBLogger = new RocksDBLogger(dbOptions);
+
+        dbOptions.setCreateIfMissing(false);
+        dbOptions.setBottommostCompressionType(CompressionType.ZSTD_COMPRESSION);
+        dbOptions.setCompressionType(CompressionType.LZ4_COMPRESSION);
+        dbOptions.setTableFormatConfig(tableFormatConfig);
+        dbOptions.setLogger(rocksDBLogger);
+
+        try {
+          tmpDb = RocksDB.open(dbOptions, dbFile.toString());
+        } catch (RocksDBException e) {
+          if (e.getStatus().getCode() == Status.Code.NotFound) {
+            logger.info("Creating state database at " + dbFile);
+            dbOptions.setCreateIfMissing(true);
+            try {
+              tmpDb = RocksDB.open(dbOptions, dbFile.toString());
+            } catch (RocksDBException dbExc) {
+              throw new IOException("Unable to create state store", dbExc);
+            }
+          } else {
+            // the RocksDB file seems to be corrupt somehow.  Let's just blow it away and create
+            // a new one, so we can keep processing new apps
+            logger.error("error opening rocksdb file {}. Creating new file, will not be able to " +
+              "recover state for existing applications", dbFile, e);
+            if (dbFile.isDirectory()) {
+              for (File f : Objects.requireNonNull(dbFile.listFiles())) {
+                if (!f.delete()) {
+                  logger.warn("error deleting {}", f.getPath());
+                }
+              }
+            }
+            if (!dbFile.delete()) {
+              logger.warn("error deleting {}", dbFile.getPath());
+            }
+            dbOptions.setCreateIfMissing(true);
+            try {
+              tmpDb = RocksDB.open(dbOptions, dbFile.toString());
+            } catch (RocksDBException dbExc) {
+              throw new IOException("Unable to create state store", dbExc);
+            }
+          }
+        }
+        try {
+          // if there is a version mismatch, we throw an exception, which means the service
+          // is unusable
+          checkVersion(tmpDb, version, mapper);
+        } catch (RocksDBException e) {
+          throw new IOException(e.getMessage(), e);
+        }
+      }
+      return tmpDb;
+    }
+
+    @VisibleForTesting
+    static RocksDB initRocksDB(File file) throws IOException {
+      BloomFilter fullFilter =
+        new BloomFilter(10.0D /* BloomFilter.DEFAULT_BITS_PER_KEY */, false);
+      BlockBasedTableConfig tableFormatConfig = new BlockBasedTableConfig()
+        .setFilterPolicy(fullFilter)
+        .setEnableIndexCompression(false)
+        .setIndexBlockRestartInterval(8)
+        .setFormatVersion(5);
+
+      Options dbOptions = new Options();
+      dbOptions.setCreateIfMissing(false);
+      dbOptions.setBottommostCompressionType(CompressionType.ZSTD_COMPRESSION);
+      dbOptions.setCompressionType(CompressionType.LZ4_COMPRESSION);
+      dbOptions.setTableFormatConfig(tableFormatConfig);
+      try {
+        return RocksDB.open(dbOptions, file.toString());

Review Comment:
   Why don't we have `setCreateIfMissing` resetting as we do in ` initRockDB(File dbFile, StoreVersion version, ObjectMapper mapper)`? Is the call on this method supposed to be the first time RocksDB creation?



##########
docs/configuration.md:
##########
@@ -1102,6 +1102,27 @@ Apart from these, the following properties are also available, and may be useful
   </td>
   <td>3.0.0</td>
 </tr>
+<tr>
+  <td><code>spark.shuffle.service.db.enabled</code></td>
+  <td>true</td>
+  <td>
+    To determine whether external shuffle uses db to store shuffle state.
+    In standalone and Yarn modes, for more detail, see
+    <a href="spark-standalone.html#Cluster Launch Scripts">this description</a>.

Review Comment:
   Should also link the detailed page for Yarn?



##########
common/network-common/src/main/java/org/apache/spark/network/shuffledb/RocksDBIterator.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.spark.network.shuffledb;
+
+import java.io.IOException;
+import java.util.AbstractMap;
+import java.util.Map;
+import java.util.NoSuchElementException;
+
+import com.google.common.base.Throwables;
+import org.rocksdb.RocksIterator;
+
+/**
+ * RocksDB implementation of `DBIterator`.
+ */
+public class RocksDBIterator implements DBIterator {
+
+    private final RocksIterator it;
+
+    private boolean checkedNext;
+
+    private boolean closed;
+
+    private Map.Entry<byte[], byte[]> next;
+
+    public RocksDBIterator(RocksIterator it) {
+        this.it = it;

Review Comment:
   2 indents



-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org