From f3b7e0bb9c141058fdbcf202a4b8a47a25237613 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Mon, 3 Oct 2016 12:09:18 -0700 Subject: [PATCH 01/18] SHS-NG M1: Add KVStore abstraction, LevelDB implementation. The interface is described in KVIndex.java (see javadoc). Specifics of the LevelDB implementation are discussed in the javadocs of both LevelDB.java and LevelDBTypeInfo.java. Included also are a few small benchmarks just to get some idea of latency. Because they're too slow for regular unit test runs, they're disabled by default. --- common/kvstore/pom.xml | 90 ++++ .../org/apache/spark/kvstore/KVIndex.java | 69 +++ .../org/apache/spark/kvstore/KVStore.java | 137 ++++++ .../apache/spark/kvstore/KVStoreIterator.java | 47 +++ .../spark/kvstore/KVStoreSerializer.java | 70 +++ .../org/apache/spark/kvstore/KVStoreView.java | 91 ++++ .../org/apache/spark/kvstore/LevelDB.java | 239 +++++++++++ .../apache/spark/kvstore/LevelDBIterator.java | 249 +++++++++++ .../apache/spark/kvstore/LevelDBTypeInfo.java | 375 ++++++++++++++++ .../UnsupportedStoreVersionException.java | 27 ++ .../org/apache/spark/kvstore/CustomType1.java | 60 +++ .../spark/kvstore/LevelDBBenchmark.java | 323 ++++++++++++++ .../spark/kvstore/LevelDBIteratorSuite.java | 399 ++++++++++++++++++ .../apache/spark/kvstore/LevelDBSuite.java | 281 ++++++++++++ .../spark/kvstore/LevelDBTypeInfoSuite.java | 177 ++++++++ .../src/test/resources/log4j.properties | 27 ++ pom.xml | 11 + project/SparkBuild.scala | 6 +- 18 files changed, 2675 insertions(+), 3 deletions(-) create mode 100644 common/kvstore/pom.xml create mode 100644 common/kvstore/src/main/java/org/apache/spark/kvstore/KVIndex.java create mode 100644 common/kvstore/src/main/java/org/apache/spark/kvstore/KVStore.java create mode 100644 common/kvstore/src/main/java/org/apache/spark/kvstore/KVStoreIterator.java create mode 100644 common/kvstore/src/main/java/org/apache/spark/kvstore/KVStoreSerializer.java create mode 100644 common/kvstore/src/main/java/org/apache/spark/kvstore/KVStoreView.java create mode 100644 common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDB.java create mode 100644 common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBIterator.java create mode 100644 common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBTypeInfo.java create mode 100644 common/kvstore/src/main/java/org/apache/spark/kvstore/UnsupportedStoreVersionException.java create mode 100644 common/kvstore/src/test/java/org/apache/spark/kvstore/CustomType1.java create mode 100644 common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBBenchmark.java create mode 100644 common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBIteratorSuite.java create mode 100644 common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBSuite.java create mode 100644 common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBTypeInfoSuite.java create mode 100644 common/kvstore/src/test/resources/log4j.properties diff --git a/common/kvstore/pom.xml b/common/kvstore/pom.xml new file mode 100644 index 0000000000000..ab296c5b5fb9b --- /dev/null +++ b/common/kvstore/pom.xml @@ -0,0 +1,90 @@ + + + + + 4.0.0 + + org.apache.spark + spark-parent_2.11 + 2.3.0-SNAPSHOT + ../../pom.xml + + + spark-kvstore_2.11 + jar + Spark Project Local DB + http://spark.apache.org/ + + kvstore + + + + + com.google.guava + guava + + + org.fusesource.leveldbjni + leveldbjni-all + + + com.fasterxml.jackson.core + jackson-core + + + com.fasterxml.jackson.core + jackson-databind + + + com.fasterxml.jackson.core + jackson-annotations + + + + commons-io + commons-io + test + + + log4j + log4j + test + + + org.slf4j + slf4j-api + test + + + org.slf4j + slf4j-log4j12 + test + + + io.dropwizard.metrics + metrics-core + test + + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + diff --git a/common/kvstore/src/main/java/org/apache/spark/kvstore/KVIndex.java b/common/kvstore/src/main/java/org/apache/spark/kvstore/KVIndex.java new file mode 100644 index 0000000000000..3c61e7706079a --- /dev/null +++ b/common/kvstore/src/main/java/org/apache/spark/kvstore/KVIndex.java @@ -0,0 +1,69 @@ +/* + * 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.kvstore; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +/** + * Tags a field to be indexed when storing an object. + * + *

+ * Types are required to have a natural index that uniquely identifies instances in the store. + * The default value of the annotation identifies the natural index for the type. + *

+ * + *

+ * Indexes allow for more efficient sorting of data read from the store. By annotating a field or + * "getter" method with this annotation, an index will be created that will provide sorting based on + * the string value of that field. + *

+ * + *

+ * Note that creating indices means more space will be needed, and maintenance operations like + * updating or deleting a value will become more expensive. + *

+ * + *

+ * Indices are restricted to String, and integral types (byte, short, int, long, boolean). + *

+ */ +@Retention(RetentionPolicy.RUNTIME) +@Target({ElementType.FIELD, ElementType.METHOD}) +public @interface KVIndex { + + public static final String NATURAL_INDEX_NAME = "__main__"; + + /** + * The name of the index to be created for the annotated entity. Must be unique within + * the class. Index names are not allowed to start with an underscore (that's reserved for + * internal use). The default value is the natural index name (which is always a copy index + * regardless of the annotation's values). + */ + String value() default NATURAL_INDEX_NAME; + + /** + * Whether to copy the instance's data to the index, instead of just storing a pointer to the + * data. The default behavior is to just store a reference; that saves disk space but is slower + * to read, since there's a level of indirection. + */ + boolean copy() default false; + +} diff --git a/common/kvstore/src/main/java/org/apache/spark/kvstore/KVStore.java b/common/kvstore/src/main/java/org/apache/spark/kvstore/KVStore.java new file mode 100644 index 0000000000000..31d4e6fefc289 --- /dev/null +++ b/common/kvstore/src/main/java/org/apache/spark/kvstore/KVStore.java @@ -0,0 +1,137 @@ +/* + * 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.kvstore; + +import java.io.Closeable; +import java.util.Iterator; +import java.util.Map; + +/** + * Abstraction for a local key/value store for storing app data. + * + *

+ * Use {@link KVStoreBuilder} to create an instance. There are two main features provided by the + * implementations of this interface: + *

+ * + * + * + *

Automatic Key Management

+ * + *

+ * When using the built-in key management, the implementation will automatically create unique + * keys for each type written to the store. Keys are based on the type name, and always start + * with the "+" prefix character (so that it's easy to use both manual and automatic key + * management APIs without conflicts). + *

+ * + *

+ * Another feature of automatic key management is indexing; by annotating fields or methods of + * objects written to the store with {@link KVIndex}, indices are created to sort the data + * by the values of those properties. This makes it possible to provide sorting without having + * to load all instances of those types from the store. + *

+ * + *

+ * KVStore instances are thread-safe for both reads and writes. + *

+ */ +public interface KVStore extends Closeable { + + /** + * Returns app-specific metadata from the store, or null if it's not currently set. + * + *

+ * The metadata type is application-specific. This is a convenience method so that applications + * don't need to define their own keys for this information. + *

+ */ + T getMetadata(Class klass) throws Exception; + + /** + * Writes the given value in the store metadata key. + */ + void setMetadata(Object value) throws Exception; + + /** + * Returns the value of a specific key, deserialized to the given type. + */ + T get(byte[] key, Class klass) throws Exception; + + /** + * Write a single key directly to the store, atomically. + */ + void put(byte[] key, Object value) throws Exception; + + /** + * Removes a key from the store. + */ + void delete(byte[] key) throws Exception; + + /** + * Returns an iterator that will only list values with keys starting with the given prefix. + */ + KVStoreIterator iterator(byte[] prefix, Class klass) throws Exception; + + /** + * Read a specific instance of an object. + */ + T read(Class klass, Object naturalKey) throws Exception; + + /** + * Writes the given object to the store, including indexed fields. Indices are updated based + * on the annotated fields of the object's class. + * + *

+ * Writes may be slower when the object already exists in the store, since it will involve + * updating existing indices. + *

+ * + * @param value The object to write. + */ + void write(Object value) throws Exception; + + /** + * Removes an object and all data related to it, like index entries, from the store. + * + * @param type The object's type. + * @param naturalKey The object's "natural key", which uniquely identifies it. + */ + void delete(Class type, Object naturalKey) throws Exception; + + /** + * Returns a configurable view for iterating over entities of the given type. + */ + KVStoreView view(Class type) throws Exception; + + /** + * Returns the number of items of the given type currently in the store. + */ + long count(Class type) throws Exception; + +} diff --git a/common/kvstore/src/main/java/org/apache/spark/kvstore/KVStoreIterator.java b/common/kvstore/src/main/java/org/apache/spark/kvstore/KVStoreIterator.java new file mode 100644 index 0000000000000..3efdec9ed32be --- /dev/null +++ b/common/kvstore/src/main/java/org/apache/spark/kvstore/KVStoreIterator.java @@ -0,0 +1,47 @@ +/* + * 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.kvstore; + +import java.util.Iterator; +import java.util.List; + +/** + * An iterator for KVStore. + * + *

+ * Iterators may keep references to resources that need to be closed. It's recommended that users + * explicitly close iterators after they're used. + *

+ */ +public interface KVStoreIterator extends Iterator, AutoCloseable { + + /** + * Retrieve multiple elements from the store. + * + * @param max Maximum number of elements to retrieve. + */ + List next(int max); + + /** + * Skip in the iterator. + * + * @return Whether there are items left after skipping. + */ + boolean skip(long n); + +} diff --git a/common/kvstore/src/main/java/org/apache/spark/kvstore/KVStoreSerializer.java b/common/kvstore/src/main/java/org/apache/spark/kvstore/KVStoreSerializer.java new file mode 100644 index 0000000000000..d9f9e2646cc14 --- /dev/null +++ b/common/kvstore/src/main/java/org/apache/spark/kvstore/KVStoreSerializer.java @@ -0,0 +1,70 @@ +/* + * 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.kvstore; + +import static java.nio.charset.StandardCharsets.UTF_8; + +import com.fasterxml.jackson.databind.ObjectMapper; + +/** + * Serializer used to translate between app-defined types and the LevelDB store. + * + *

+ * The serializer is based on Jackson, so values are written as JSON. It also allows "naked strings" + * and integers to be written as values directly, which will be written as UTF-8 strings. + *

+ */ +public class KVStoreSerializer { + + /** + * Object mapper used to process app-specific types. If an application requires a specific + * configuration of the mapper, it can subclass this serializer and add custom configuration + * to this object. + */ + protected final ObjectMapper mapper; + + public KVStoreSerializer() { + this.mapper = new ObjectMapper(); + } + + public final byte[] serialize(Object o) throws Exception { + if (o instanceof String) { + return ((String) o).getBytes(UTF_8); + } else { + return mapper.writeValueAsBytes(o); + } + } + + @SuppressWarnings("unchecked") + public final T deserialize(byte[] data, Class klass) throws Exception { + if (klass.equals(String.class)) { + return (T) new String(data, UTF_8); + } else { + return mapper.readValue(data, klass); + } + } + + final byte[] serialize(long value) { + return String.valueOf(value).getBytes(UTF_8); + } + + final long deserializeLong(byte[] data) { + return Long.parseLong(new String(data, UTF_8)); + } + +} diff --git a/common/kvstore/src/main/java/org/apache/spark/kvstore/KVStoreView.java b/common/kvstore/src/main/java/org/apache/spark/kvstore/KVStoreView.java new file mode 100644 index 0000000000000..a68c37942dee4 --- /dev/null +++ b/common/kvstore/src/main/java/org/apache/spark/kvstore/KVStoreView.java @@ -0,0 +1,91 @@ +/* + * 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.kvstore; + +import java.util.Iterator; +import java.util.Map; + +import com.google.common.base.Preconditions; + +/** + * A configurable view that allows iterating over values in a {@link KVStore}. + * + *

+ * The different methods can be used to configure the behavior of the iterator. Calling the same + * method multiple times is allowed; the most recent value will be used. + *

+ * + *

+ * The iterators returns by this view are of type {@link KVStoreIterator}; they auto-close + * when used in a for loop that exhausts their contents, but when used manually, they need + * to be closed explicitly unless all elements are read. + *

+ */ +public abstract class KVStoreView implements Iterable { + + final Class type; + + boolean ascending = true; + String index = KVIndex.NATURAL_INDEX_NAME; + Object first = null; + long skip = 0L; + + public KVStoreView(Class type) { + this.type = type; + } + + /** + * Reverses the order of iteration. By default, iterates in ascending order. + */ + public KVStoreView reverse() { + ascending = !ascending; + return this; + } + + /** + * Iterates according to the given index. + */ + public KVStoreView index(String name) { + this.index = Preconditions.checkNotNull(name); + return this; + } + + /** + * Iterates starting at the given value of the chosen index. + */ + public KVStoreView first(Object value) { + this.first = value; + return this; + } + + /** + * Skips a number of elements in the resulting iterator. + */ + public KVStoreView skip(long n) { + this.skip = n; + return this; + } + + /** + * Returns an iterator for the current configuration. + */ + public KVStoreIterator closeableIterator() throws Exception { + return (KVStoreIterator) iterator(); + } + +} diff --git a/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDB.java b/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDB.java new file mode 100644 index 0000000000000..51287c02ebab1 --- /dev/null +++ b/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDB.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 + * + * 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.kvstore; + +import java.io.File; +import java.io.IOException; +import java.util.Iterator; +import java.util.NoSuchElementException; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import static java.nio.charset.StandardCharsets.UTF_8; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import com.google.common.base.Throwables; +import org.fusesource.leveldbjni.JniDBFactory; +import org.iq80.leveldb.DB; +import org.iq80.leveldb.Options; +import org.iq80.leveldb.WriteBatch; +import org.iq80.leveldb.WriteOptions; + +/** + * Implementation of KVStore that uses LevelDB as the underlying data store. + */ +public class LevelDB implements KVStore { + + @VisibleForTesting + static final long STORE_VERSION = 1L; + + @VisibleForTesting + static final byte[] STORE_VERSION_KEY = "__version__".getBytes(UTF_8); + + /** DB key where app metadata is stored. */ + private static final byte[] METADATA_KEY = "__meta__".getBytes(UTF_8); + + final DB db; + final KVStoreSerializer serializer; + + private final ConcurrentMap, LevelDBTypeInfo> types; + private boolean closed; + + public LevelDB(File path) throws IOException { + this(path, new KVStoreSerializer()); + } + + public LevelDB(File path, KVStoreSerializer serializer) throws IOException { + this.serializer = serializer; + this.types = new ConcurrentHashMap<>(); + + Options options = new Options(); + options.createIfMissing(!path.exists()); + this.db = JniDBFactory.factory.open(path, options); + + byte[] versionData = db.get(STORE_VERSION_KEY); + if (versionData != null) { + long version = serializer.deserializeLong(versionData); + if (version != STORE_VERSION) { + throw new UnsupportedStoreVersionException(); + } + } else { + db.put(STORE_VERSION_KEY, serializer.serialize(STORE_VERSION)); + } + } + + @Override + public T getMetadata(Class klass) throws Exception { + try { + return get(METADATA_KEY, klass); + } catch (NoSuchElementException nsee) { + return null; + } + } + + @Override + public void setMetadata(Object value) throws Exception { + if (value != null) { + put(METADATA_KEY, value); + } else { + db.delete(METADATA_KEY); + } + } + + @Override + public T get(byte[] key, Class klass) throws Exception { + byte[] data = db.get(key); + if (data == null) { + throw new NoSuchElementException(new String(key, UTF_8)); + } + return serializer.deserialize(data, klass); + } + + @Override + public void put(byte[] key, Object value) throws Exception { + Preconditions.checkArgument(value != null, "Null values are not allowed."); + db.put(key, serializer.serialize(value)); + } + + @Override + public void delete(byte[] key) throws Exception { + db.delete(key); + } + + @Override + public KVStoreIterator iterator(byte[] prefix, Class klass) throws Exception { + throw new UnsupportedOperationException(); + } + + @Override + public T read(Class klass, Object naturalKey) throws Exception { + Preconditions.checkArgument(naturalKey != null, "Null keys are not allowed."); + byte[] key = getTypeInfo(klass).naturalIndex().start(naturalKey); + return get(key, klass); + } + + @Override + public void write(Object value) throws Exception { + write(value, false); + } + + public void write(Object value, boolean sync) throws Exception { + Preconditions.checkArgument(value != null, "Null values are not allowed."); + LevelDBTypeInfo ti = getTypeInfo(value.getClass()); + + WriteBatch batch = db.createWriteBatch(); + try { + byte[] data = serializer.serialize(value); + synchronized (ti) { + try { + Object existing = get(ti.naturalIndex().entityKey(value), value.getClass()); + removeInstance(ti, batch, existing); + } catch (NoSuchElementException e) { + // Ignore. No previous value. + } + for (LevelDBTypeInfo.Index idx : ti.indices()) { + idx.add(batch, value, data); + } + db.write(batch, new WriteOptions().sync(sync)); + } + } finally { + batch.close(); + } + } + + @Override + public void delete(Class type, Object naturalKey) throws Exception { + delete(type, naturalKey, false); + } + + public void delete(Class type, Object naturalKey, boolean sync) throws Exception { + Preconditions.checkArgument(naturalKey != null, "Null keys are not allowed."); + WriteBatch batch = db.createWriteBatch(); + try { + LevelDBTypeInfo ti = getTypeInfo(type); + byte[] key = ti.naturalIndex().start(naturalKey); + byte[] data = db.get(key); + if (data != null) { + Object existing = serializer.deserialize(data, type); + synchronized (ti) { + removeInstance(ti, batch, existing); + db.write(batch, new WriteOptions().sync(sync)); + } + } + } finally { + batch.close(); + } + } + + @Override + public KVStoreView view(Class type) throws Exception { + return new KVStoreView(type) { + @Override + public Iterator iterator() { + try { + return new LevelDBIterator<>(LevelDB.this, this); + } catch (Exception e) { + throw Throwables.propagate(e); + } + } + }; + } + + @Override + public long count(Class type) throws Exception { + LevelDBTypeInfo.Index idx = getTypeInfo(type).naturalIndex(); + return idx.getCount(idx.end()); + } + + @Override + public synchronized void close() throws IOException { + if (closed) { + return; + } + + try { + db.close(); + closed = true; + } catch (IOException ioe) { + throw ioe; + } catch (Exception e) { + throw new IOException(e.getMessage(), e); + } + } + + /** Returns metadata about indices for the given type. */ + LevelDBTypeInfo getTypeInfo(Class type) throws Exception { + LevelDBTypeInfo idx = types.get(type); + if (idx == null) { + LevelDBTypeInfo tmp = new LevelDBTypeInfo<>(this, type); + idx = types.putIfAbsent(type, tmp); + if (idx == null) { + idx = tmp; + } + } + return idx; + } + + private void removeInstance(LevelDBTypeInfo ti, WriteBatch batch, Object instance) + throws Exception { + for (LevelDBTypeInfo.Index idx : ti.indices()) { + idx.remove(batch, instance); + } + } + +} diff --git a/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBIterator.java b/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBIterator.java new file mode 100644 index 0000000000000..d0b6e25420812 --- /dev/null +++ b/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBIterator.java @@ -0,0 +1,249 @@ +/* + * 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.kvstore; + +import java.io.IOException; +import java.util.Arrays; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.NoSuchElementException; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Throwables; +import org.iq80.leveldb.DBIterator; + +class LevelDBIterator implements KVStoreIterator { + + private final LevelDB db; + private final boolean ascending; + private final DBIterator it; + private final Class type; + private final LevelDBTypeInfo ti; + private final LevelDBTypeInfo.Index index; + private final byte[] indexKeyPrefix; + private final byte[] end; + + private boolean checkedNext; + private T next; + private boolean closed; + + /** + * Creates a simple iterator over db keys. + */ + LevelDBIterator(LevelDB db, byte[] keyPrefix, Class type) throws Exception { + this.db = db; + this.ascending = true; + this.type = type; + this.ti = null; + this.index = null; + this.it = db.db.iterator(); + this.indexKeyPrefix = keyPrefix; + this.end = null; + it.seek(keyPrefix); + } + + /** + * Creates an iterator for indexed types (i.e., those whose keys are managed by the library). + */ + LevelDBIterator(LevelDB db, KVStoreView params) throws Exception { + this.db = db; + this.ascending = params.ascending; + this.it = db.db.iterator(); + this.type = params.type; + this.ti = db.getTypeInfo(type); + this.index = ti.index(params.index); + this.indexKeyPrefix = index.keyPrefix(); + + byte[] firstKey; + if (params.first != null) { + if (ascending) { + firstKey = index.start(params.first); + } else { + firstKey = index.end(params.first); + } + } else if (ascending) { + firstKey = index.keyPrefix(); + } else { + firstKey = index.end(); + } + it.seek(firstKey); + + if (ascending) { + this.end = index.end(); + } else { + this.end = null; + if (it.hasNext()) { + it.next(); + } + } + + if (params.skip > 0) { + skip(params.skip); + } + } + + @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 T next() { + if (!hasNext()) { + throw new NoSuchElementException(); + } + checkedNext = false; + return next; + } + + @Override + public void remove() { + throw new UnsupportedOperationException(); + } + + @Override + public List next(int max) { + List list = new ArrayList<>(max); + while (hasNext() && list.size() < max) { + list.add(next()); + } + return list; + } + + @Override + public boolean skip(long n) { + long skipped = 0; + while (skipped < n) { + next = null; + boolean hasNext = ascending ? it.hasNext() : it.hasPrev(); + if (!hasNext) { + return false; + } + + Map.Entry e = ascending ? it.next() : it.prev(); + if (!isEndMarker(e.getKey())) { + skipped++; + } + } + + return true; + } + + @Override + public void close() throws IOException { + if (!closed) { + it.close(); + closed = true; + } + } + + private T loadNext() { + try { + while (true) { + boolean hasNext = ascending ? it.hasNext() : it.hasPrev(); + if (!hasNext) { + return null; + } + + Map.Entry nextEntry; + try { + // Avoid races if another thread is updating the DB. + nextEntry = ascending ? it.next() : it.prev(); + } catch (NoSuchElementException e) { + return null; + } + byte[] nextKey = nextEntry.getKey(); + + // If the next key is an end marker, then skip it. + if (isEndMarker(nextKey)) { + continue; + } + + // Next key is not part of the index, stop. + if (!startsWith(nextKey, indexKeyPrefix)) { + return null; + } + + // If there's a known end key and it's found, stop. + if (end != null && Arrays.equals(nextKey, end)) { + return null; + } + + // Next element is part of the iteration, return it. + if (index == null || index.isCopy()) { + return db.serializer.deserialize(nextEntry.getValue(), type); + } else { + byte[] key = stitch(ti.naturalIndex().keyPrefix(), nextEntry.getValue()); + return db.get(key, type); + } + } + } catch (Exception e) { + throw Throwables.propagate(e); + } + } + + @VisibleForTesting + static boolean startsWith(byte[] key, byte[] prefix) { + if (key.length < prefix.length) { + return false; + } + + for (int i = 0; i < prefix.length; i++) { + if (key[i] != prefix[i]) { + return false; + } + } + + return true; + } + + private boolean isEndMarker(byte[] key) { + return (key.length > 2 && + key[key.length - 2] == LevelDBTypeInfo.KEY_SEPARATOR && + key[key.length - 1] == (byte) LevelDBTypeInfo.END_MARKER.charAt(0)); + } + + private byte[] stitch(byte[]... comps) { + int len = 0; + for (byte[] comp : comps) { + len += comp.length; + } + + byte[] dest = new byte[len]; + int written = 0; + for (byte[] comp : comps) { + System.arraycopy(comp, 0, dest, written, comp.length); + written += comp.length; + } + + return dest; + } + +} diff --git a/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBTypeInfo.java b/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBTypeInfo.java new file mode 100644 index 0000000000000..c49b18324c00a --- /dev/null +++ b/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBTypeInfo.java @@ -0,0 +1,375 @@ +/* + * 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.kvstore; + +import java.lang.reflect.Field; +import java.lang.reflect.Method; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.util.Collection; +import java.util.HashMap; +import java.util.Map; +import static java.nio.charset.StandardCharsets.UTF_8; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import com.google.common.base.Throwables; +import org.iq80.leveldb.WriteBatch; + +/** + * Holds metadata about app-specific types stored in LevelDB. Serves as a cache for data collected + * via reflection, to make it cheaper to access it multiple times. + */ +class LevelDBTypeInfo { + + static final String ENTRY_PREFIX = "+"; + static final String END_MARKER = "-"; + static final byte KEY_SEPARATOR = 0x0; + + // These constants are used in the Index.toKey() method below when encoding numbers into keys. + // See javadoc for that method for details. + private static final char POSITIVE_FILL = '.'; + private static final char NEGATIVE_FILL = '~'; + private static final char POSITIVE_MARKER = '='; + private static final char NEGATIVE_MARKER = '*'; + + @VisibleForTesting + static final int BYTE_ENCODED_LEN = String.valueOf(Byte.MAX_VALUE).length() + 1; + @VisibleForTesting + static final int INT_ENCODED_LEN = String.valueOf(Integer.MAX_VALUE).length() + 1; + @VisibleForTesting + static final int LONG_ENCODED_LEN = String.valueOf(Long.MAX_VALUE).length() + 1; + @VisibleForTesting + static final int SHORT_ENCODED_LEN = String.valueOf(Short.MAX_VALUE).length() + 1; + + private final LevelDB db; + private final Class type; + private final Map indices; + private final byte[] typePrefix; + + LevelDBTypeInfo(LevelDB db, Class type) throws Exception { + this.db = db; + this.type = type; + this.indices = new HashMap<>(); + + for (Field f : type.getFields()) { + KVIndex idx = f.getAnnotation(KVIndex.class); + if (idx != null) { + register(idx, new FieldAccessor(f)); + } + } + + for (Method m : type.getMethods()) { + KVIndex idx = m.getAnnotation(KVIndex.class); + if (idx != null) { + Preconditions.checkArgument(m.getParameterTypes().length == 0, + "Annotated method %s::%s should not have any parameters.", type.getName(), m.getName()); + register(idx, new MethodAccessor(m)); + } + } + + Preconditions.checkArgument(indices.get(KVIndex.NATURAL_INDEX_NAME) != null, + "No natural index defined for type %s.", type.getName()); + + ByteArrayOutputStream typePrefix = new ByteArrayOutputStream(); + typePrefix.write(utf8(ENTRY_PREFIX)); + + // Change fully-qualified class names to make keys more spread out by placing the + // class name first, and the package name afterwards. + String[] components = type.getName().split("\\."); + typePrefix.write(utf8(components[components.length - 1])); + if (components.length > 1) { + typePrefix.write(utf8("/")); + } + for (int i = 0; i < components.length - 1; i++) { + typePrefix.write(utf8(components[i])); + if (i < components.length - 2) { + typePrefix.write(utf8(".")); + } + } + typePrefix.write(KEY_SEPARATOR); + this.typePrefix = typePrefix.toByteArray(); + } + + private void register(KVIndex idx, Accessor accessor) { + Preconditions.checkArgument(idx.value() != null && !idx.value().isEmpty(), + "No name provided for index in type %s.", type.getName()); + Preconditions.checkArgument( + !idx.value().startsWith("_") || idx.value().equals(KVIndex.NATURAL_INDEX_NAME), + "Index name %s (in type %s) is not allowed.", idx.value(), type.getName()); + Preconditions.checkArgument(indices.get(idx.value()) == null, + "Duplicate index %s for type %s.", idx.value(), type.getName()); + indices.put(idx.value(), new Index(idx.value(), idx.copy(), accessor)); + } + + Class type() { + return type; + } + + byte[] keyPrefix() { + return buildKey(false); + } + + Index naturalIndex() { + return index(KVIndex.NATURAL_INDEX_NAME); + } + + Index index(String name) { + Index i = indices.get(name); + Preconditions.checkArgument(i != null, "Index %s does not exist for type %s.", name, + type.getName()); + return i; + } + + Collection indices() { + return indices.values(); + } + + private byte[] utf8(String s) { + return s.getBytes(UTF_8); + } + + private byte[] buildKey(boolean trim, String... components) { + try { + ByteArrayOutputStream kos = new ByteArrayOutputStream(typePrefix.length * 2); + kos.write(typePrefix); + for (int i = 0; i < components.length; i++) { + kos.write(utf8(components[i])); + if (!trim || i < components.length - 1) { + kos.write(KEY_SEPARATOR); + } + } + return kos.toByteArray(); + } catch (IOException ioe) { + throw Throwables.propagate(ioe); + } + } + + /** + * Models a single index in LevelDB. Keys are stored under the type's prefix, in sequential + * order according to the indexed value. For non-natural indices, the key also contains the + * entity's natural key after the indexed value, so that it's possible for multiple entities + * to have the same indexed value. + * + *

+ * An end marker is used to mark where the index ends, and the boundaries of each indexed value + * within the index, to make descending iteration faster, at the expense of some disk space and + * minor overhead when iterating. A count of the number of indexed entities is kept at the end + * marker, so that it can be cleaned up when all entries are removed from the index. + *

+ */ + class Index { + + private final boolean copy; + private final boolean isNatural; + private final String name; + + @VisibleForTesting + final Accessor accessor; + + private Index(String name, boolean copy, Accessor accessor) { + this.name = name; + this.isNatural = name.equals(KVIndex.NATURAL_INDEX_NAME); + this.copy = isNatural || copy; + this.accessor = accessor; + } + + boolean isCopy() { + return copy; + } + + /** The prefix for all keys that belong to this index. */ + byte[] keyPrefix() { + return buildKey(false, name); + } + + /** The key where to start ascending iteration for entries that match the given value. */ + byte[] start(Object value) { + return buildKey(isNatural, name, toKey(value)); + } + + /** The key for the index's end marker. */ + byte[] end() { + return buildKey(true, name, END_MARKER); + } + + /** The key for the end marker for index entries with the given value. */ + byte[] end(Object value) throws Exception { + return buildKey(true, name, toKey(value), END_MARKER); + } + + /** The key in the index that identifies the given entity. */ + byte[] entityKey(Object entity) throws Exception { + Object indexValue = accessor.get(entity); + Preconditions.checkNotNull(indexValue, "Null index value for %s in type %s.", + name, type.getName()); + if (isNatural) { + return buildKey(true, name, toKey(indexValue)); + } else { + Object naturalKey = naturalIndex().accessor.get(entity); + return buildKey(true, name, toKey(accessor.get(entity)), toKey(naturalKey)); + } + } + + /** + * Add an entry to the index. + * + * @param batch Write batch with other related changes. + * @param entity The entity being added to the index. + * @param data Serialized entity to store (when storing the entity, not a reference). + * @param naturalKey The value's key. + */ + void add(WriteBatch batch, Object entity, byte[] data) throws Exception { + byte[] stored = data; + if (!copy) { + stored = db.serializer.serialize(toKey(naturalIndex().accessor.get(entity))); + } + batch.put(entityKey(entity), stored); + updateCount(batch, end(accessor.get(entity)), 1L); + updateCount(batch, end(), 1L); + } + + /** + * Remove a value from the index. + * + * @param batch Write batch with other related changes. + * @param entity The entity being removed, to identify the index entry to modify. + * @param naturalKey The value's key. + */ + void remove(WriteBatch batch, Object entity) throws Exception { + batch.delete(entityKey(entity)); + updateCount(batch, end(accessor.get(entity)), -1L); + updateCount(batch, end(), -1L); + } + + long getCount(byte[] key) throws Exception { + byte[] data = db.db.get(key); + return data != null ? db.serializer.deserializeLong(data) : 0; + } + + private void updateCount(WriteBatch batch, byte[] key, long delta) throws Exception { + long count = getCount(key) + delta; + if (count > 0) { + batch.put(key, db.serializer.serialize(count)); + } else { + batch.delete(key); + } + } + + /** + * Translates a value to be used as part of the store key. + * + * Integral numbers are encoded as a string in a way that preserves lexicographical + * ordering. The string is always as long as the maximum value for the given type (e.g. + * 11 characters for integers, including the character for the sign). The first character + * represents the sign (with the character for negative coming before the one for positive, + * which means you cannot use '-'...). The rest of the value is padded with a value that is + * "greater than 9" for negative values, so that for example "-123" comes before "-12" (the + * encoded value would look like "*~~~~~~~123"). For positive values, similarly, a value that + * is "lower than 0" (".") is used for padding. The fill characters were chosen for readability + * when looking at the encoded keys. + */ + @VisibleForTesting + String toKey(Object value) { + StringBuilder sb = new StringBuilder(ENTRY_PREFIX); + + if (value instanceof String) { + sb.append(value); + } else if (value instanceof Boolean) { + sb.append(((Boolean) value).toString().toLowerCase()); + } else { + int encodedLen; + + if (value instanceof Integer) { + encodedLen = INT_ENCODED_LEN; + } else if (value instanceof Long) { + encodedLen = LONG_ENCODED_LEN; + } else if (value instanceof Short) { + encodedLen = SHORT_ENCODED_LEN; + } else if (value instanceof Byte) { + encodedLen = BYTE_ENCODED_LEN; + } else { + throw new IllegalArgumentException(String.format("Type %s not allowed as key.", + value.getClass().getName())); + } + + long longValue = ((Number) value).longValue(); + String strVal; + if (longValue == Long.MIN_VALUE) { + // Math.abs() overflows for Long.MIN_VALUE. + strVal = String.valueOf(longValue).substring(1); + } else { + strVal = String.valueOf(Math.abs(longValue)); + } + + sb.append(longValue >= 0 ? POSITIVE_MARKER : NEGATIVE_MARKER); + + char fill = longValue >= 0 ? POSITIVE_FILL : NEGATIVE_FILL; + for (int i = 0; i < encodedLen - strVal.length() - 1; i++) { + sb.append(fill); + } + + sb.append(strVal); + } + + return sb.toString(); + } + + } + + /** + * Abstracts the difference between invoking a Field and a Method. + */ + @VisibleForTesting + interface Accessor { + + Object get(Object instance) throws Exception; + + } + + private class FieldAccessor implements Accessor { + + private final Field field; + + FieldAccessor(Field field) { + this.field = field; + } + + @Override + public Object get(Object instance) throws Exception { + return field.get(instance); + } + + } + + private class MethodAccessor implements Accessor { + + private final Method method; + + MethodAccessor(Method method) { + this.method = method; + } + + @Override + public Object get(Object instance) throws Exception { + return method.invoke(instance); + } + + } + +} diff --git a/common/kvstore/src/main/java/org/apache/spark/kvstore/UnsupportedStoreVersionException.java b/common/kvstore/src/main/java/org/apache/spark/kvstore/UnsupportedStoreVersionException.java new file mode 100644 index 0000000000000..2ed246e4f4c97 --- /dev/null +++ b/common/kvstore/src/main/java/org/apache/spark/kvstore/UnsupportedStoreVersionException.java @@ -0,0 +1,27 @@ +/* + * 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.kvstore; + +import java.io.IOException; + +/** + * Exception thrown when the store implementation is not compatible with the underlying data. + */ +public class UnsupportedStoreVersionException extends IOException { + +} diff --git a/common/kvstore/src/test/java/org/apache/spark/kvstore/CustomType1.java b/common/kvstore/src/test/java/org/apache/spark/kvstore/CustomType1.java new file mode 100644 index 0000000000000..2bea5b560681f --- /dev/null +++ b/common/kvstore/src/test/java/org/apache/spark/kvstore/CustomType1.java @@ -0,0 +1,60 @@ +/* + * 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.kvstore; + +import com.google.common.base.Objects; + +public class CustomType1 { + + @KVIndex + public String key; + + @KVIndex("id") + public String id; + + @KVIndex(value = "name", copy = true) + public String name; + + @KVIndex("int") + public int num; + + @Override + public boolean equals(Object o) { + if (o instanceof CustomType1) { + CustomType1 other = (CustomType1) o; + return id.equals(other.id) && name.equals(other.name); + } + return false; + } + + @Override + public int hashCode() { + return id.hashCode(); + } + + @Override + public String toString() { + return Objects.toStringHelper(this) + .add("key", key) + .add("id", id) + .add("name", name) + .add("num", num) + .toString(); + } + +} diff --git a/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBBenchmark.java b/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBBenchmark.java new file mode 100644 index 0000000000000..aecea26ec82f3 --- /dev/null +++ b/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBBenchmark.java @@ -0,0 +1,323 @@ +/* + * 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.kvstore; + +import java.io.File; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicInteger; + +import com.codahale.metrics.MetricRegistry; +import com.codahale.metrics.Slf4jReporter; +import com.codahale.metrics.Snapshot; +import com.codahale.metrics.Timer; +import org.apache.commons.io.FileUtils; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.Ignore; +import org.junit.Test; +import org.slf4j.LoggerFactory; +import static org.junit.Assert.*; + +/** + * A set of small benchmarks for the LevelDB implementation. + * + * The benchmarks are run over two different types (one with just a natural index, and one + * with a ref index), over a set of 2^20 elements, and the following tests are performed: + * + * - write (then update) elements in sequential natural key order + * - write (then update) elements in random natural key order + * - iterate over natural index, ascending and descending + * - iterate over ref index, ascending and descending + */ +@Ignore +public class LevelDBBenchmark { + + private static final int COUNT = 1024; + private static final AtomicInteger IDGEN = new AtomicInteger(); + private static final MetricRegistry metrics = new MetricRegistry(); + private static final Timer dbCreation = metrics.timer("dbCreation"); + private static final Timer dbClose = metrics.timer("dbClose"); + + private LevelDB db; + private File dbpath; + + @Before + public void setup() throws Exception { + dbpath = File.createTempFile("test.", ".ldb"); + dbpath.delete(); + try(Timer.Context ctx = dbCreation.time()) { + db = new LevelDB(dbpath); + } + } + + @After + public void cleanup() throws Exception { + if (db != null) { + try(Timer.Context ctx = dbClose.time()) { + db.close(); + } + } + if (dbpath != null) { + FileUtils.deleteQuietly(dbpath); + } + } + + @AfterClass + public static void report() { + if (metrics.getTimers().isEmpty()) { + return; + } + + int headingPrefix = 0; + for (Map.Entry e : metrics.getTimers().entrySet()) { + headingPrefix = Math.max(e.getKey().length(), headingPrefix); + } + headingPrefix += 4; + + StringBuilder heading = new StringBuilder(); + for (int i = 0; i < headingPrefix; i++) { + heading.append(" "); + } + heading.append("\tcount"); + heading.append("\tmean"); + heading.append("\tmin"); + heading.append("\tmax"); + heading.append("\t95th"); + System.out.println(heading); + + for (Map.Entry e : metrics.getTimers().entrySet()) { + StringBuilder row = new StringBuilder(); + row.append(e.getKey()); + for (int i = 0; i < headingPrefix - e.getKey().length(); i++) { + row.append(" "); + } + + Snapshot s = e.getValue().getSnapshot(); + row.append("\t").append(e.getValue().getCount()); + row.append("\t").append(toMs(s.getMean())); + row.append("\t").append(toMs(s.getMin())); + row.append("\t").append(toMs(s.getMax())); + row.append("\t").append(toMs(s.get95thPercentile())); + + System.out.println(row); + } + + Slf4jReporter.forRegistry(metrics).outputTo(LoggerFactory.getLogger(LevelDBBenchmark.class)) + .build().report(); + } + + private static String toMs(double nanos) { + return String.format("%.3f", nanos / 1000 / 1000); + } + + @Test + public void sequentialWritesNoIndex() throws Exception { + List entries = createSimpleType(); + writeAll(entries, false, "sequentialWritesNoIndex"); + writeAll(entries, false, "sequentialUpdatesNoIndex"); + deleteNoIndex(entries, false, "sequentialDeleteNoIndex"); + } + + @Test + public void sequentialSyncWritesNoIndex() throws Exception { + List entries = createSimpleType(); + writeAll(entries, true, "sequentialSyncWritesNoIndex"); + writeAll(entries, true, "sequentialSyncUpdatesNoIndex"); + deleteNoIndex(entries, true, "sequentialSyncDeleteNoIndex"); + } + + @Test + public void randomWritesNoIndex() throws Exception { + List entries = createSimpleType(); + + Collections.shuffle(entries); + writeAll(entries, false, "randomWritesNoIndex"); + + Collections.shuffle(entries); + writeAll(entries, false, "randomUpdatesNoIndex"); + + Collections.shuffle(entries); + deleteNoIndex(entries, false, "randomDeletesNoIndex"); + } + + @Test + public void randomSyncWritesNoIndex() throws Exception { + List entries = createSimpleType(); + + Collections.shuffle(entries); + writeAll(entries, true, "randomSyncWritesNoIndex"); + + Collections.shuffle(entries); + writeAll(entries, true, "randomSyncUpdatesNoIndex"); + + Collections.shuffle(entries); + deleteNoIndex(entries, true, "randomSyncDeletesNoIndex"); + } + + @Test + public void sequentialWritesIndexedType() throws Exception { + List entries = createIndexedType(); + writeAll(entries, false, "sequentialWritesIndexed"); + writeAll(entries, false, "sequentialUpdatesIndexed"); + deleteIndexed(entries, false, "sequentialDeleteIndexed"); + } + + @Test + public void sequentialSyncWritesIndexedType() throws Exception { + List entries = createIndexedType(); + writeAll(entries, true, "sequentialSyncWritesIndexed"); + writeAll(entries, true, "sequentialSyncUpdatesIndexed"); + deleteIndexed(entries, true, "sequentialSyncDeleteIndexed"); + } + + @Test + public void randomWritesIndexedTypeAndIteration() throws Exception { + List entries = createIndexedType(); + + Collections.shuffle(entries); + writeAll(entries, false, "randomWritesIndexed"); + + Collections.shuffle(entries); + writeAll(entries, false, "randomUpdatesIndexed"); + + // Run iteration benchmarks here since we've gone through the trouble of writing all + // the data already. + KVStoreView view = db.view(IndexedType.class); + iterate(view, "naturalIndex"); + iterate(view.reverse(), "naturalIndexDescending"); + iterate(view.index("name"), "refIndex"); + iterate(view.index("name").reverse(), "refIndexDescending"); + + Collections.shuffle(entries); + deleteIndexed(entries, false, "randomDeleteIndexed"); + } + + @Test + public void randomSyncWritesIndexedTypeAndIteration() throws Exception { + List entries = createIndexedType(); + + Collections.shuffle(entries); + writeAll(entries, true, "randomSyncWritesIndexed"); + + Collections.shuffle(entries); + deleteIndexed(entries, true, "randomSyncDeleteIndexed"); + } + + private void iterate(KVStoreView view, String name) throws Exception { + Timer create = metrics.timer(name + "CreateIterator"); + Timer iter = metrics.timer(name + "Iteration"); + KVStoreIterator it = null; + { + // Create the iterator several times, just to have multiple data points. + for (int i = 0; i < 1024; i++) { + if (it != null) { + it.close(); + } + try(Timer.Context ctx = create.time()) { + it = view.closeableIterator(); + } + } + } + + for (; it.hasNext(); ) { + try(Timer.Context ctx = iter.time()) { + it.next(); + } + } + } + + private void writeAll(List entries, boolean sync, String timerName) throws Exception { + Timer timer = newTimer(timerName); + for (Object o : entries) { + try(Timer.Context ctx = timer.time()) { + db.write(o, sync); + } + } + } + + private void deleteNoIndex(List entries, boolean sync, String timerName) + throws Exception { + Timer delete = newTimer(timerName); + for (SimpleType i : entries) { + try(Timer.Context ctx = delete.time()) { + db.delete(i.getClass(), i.key, sync); + } + } + } + + private void deleteIndexed(List entries, boolean sync, String timerName) + throws Exception { + Timer delete = newTimer(timerName); + for (IndexedType i : entries) { + try(Timer.Context ctx = delete.time()) { + db.delete(i.getClass(), i.key, sync); + } + } + } + + private List createSimpleType() { + List entries = new ArrayList<>(); + for (int i = 0; i < COUNT; i++) { + SimpleType t = new SimpleType(); + t.key = IDGEN.getAndIncrement(); + t.name = "name" + (t.key % 1024); + entries.add(t); + } + return entries; + } + + private List createIndexedType() { + List entries = new ArrayList<>(); + for (int i = 0; i < COUNT; i++) { + IndexedType t = new IndexedType(); + t.key = IDGEN.getAndIncrement(); + t.name = "name" + (t.key % 1024); + entries.add(t); + } + return entries; + } + + private Timer newTimer(String name) { + assertNull("Timer already exists: " + name, metrics.getTimers().get(name)); + return metrics.timer(name); + } + + public static class SimpleType { + + @KVIndex + public int key; + + public String name; + + } + + public static class IndexedType { + + @KVIndex + public int key; + + @KVIndex("name") + public String name; + + } + +} diff --git a/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBIteratorSuite.java b/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBIteratorSuite.java new file mode 100644 index 0000000000000..b67503b3fcbea --- /dev/null +++ b/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBIteratorSuite.java @@ -0,0 +1,399 @@ +/* + * 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.kvstore; + +import java.io.File; +import java.util.Arrays; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; +import java.util.Random; + +import com.google.common.base.Predicate; +import com.google.common.collect.Iterables; +import com.google.common.collect.Iterators; +import com.google.common.collect.Lists; +import org.apache.commons.io.FileUtils; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; +import static org.junit.Assert.*; + +public class LevelDBIteratorSuite { + + private static final int MIN_ENTRIES = 42; + private static final int MAX_ENTRIES = 1024; + private static final Random RND = new Random(); + + private static List allEntries; + private static List clashingEntries; + private static LevelDB db; + private static File dbpath; + + private abstract class BaseComparator implements Comparator { + /** + * Returns a comparator that falls back to natural order if this comparator's ordering + * returns equality for two elements. Used to mimic how the index sorts things internally. + */ + BaseComparator fallback() { + return new BaseComparator() { + @Override + public int compare(CustomType1 t1, CustomType1 t2) { + int result = BaseComparator.this.compare(t1, t2); + if (result != 0) { + return result; + } + + return t1.key.compareTo(t2.key); + } + }; + } + + /** Reverses the order of this comparator. */ + BaseComparator reverse() { + return new BaseComparator() { + @Override + public int compare(CustomType1 t1, CustomType1 t2) { + return -BaseComparator.this.compare(t1, t2); + } + }; + } + } + + private final BaseComparator NATURAL_ORDER = new BaseComparator() { + @Override + public int compare(CustomType1 t1, CustomType1 t2) { + return t1.key.compareTo(t2.key); + } + }; + + private final BaseComparator REF_INDEX_ORDER = new BaseComparator() { + @Override + public int compare(CustomType1 t1, CustomType1 t2) { + return t1.id.compareTo(t2.id); + } + }; + + private final BaseComparator COPY_INDEX_ORDER = new BaseComparator() { + @Override + public int compare(CustomType1 t1, CustomType1 t2) { + return t1.name.compareTo(t2.name); + } + }; + + private final BaseComparator NUMERIC_INDEX_ORDER = new BaseComparator() { + @Override + public int compare(CustomType1 t1, CustomType1 t2) { + return t1.num - t2.num; + } + }; + + @BeforeClass + public static void setup() throws Exception { + dbpath = File.createTempFile("test.", ".ldb"); + dbpath.delete(); + db = new LevelDB(dbpath); + + int count = RND.nextInt(MAX_ENTRIES) + MIN_ENTRIES; + + // Instead of generating sequential IDs, generate random unique IDs to avoid the insertion + // order matching the natural ordering. Just in case. + boolean[] usedIDs = new boolean[count]; + + allEntries = new ArrayList<>(count); + for (int i = 0; i < count; i++) { + CustomType1 t = new CustomType1(); + + int id; + do { + id = RND.nextInt(count); + } while (usedIDs[id]); + + usedIDs[id] = true; + t.key = "key" + id; + t.id = "id" + i; + t.name = "name" + RND.nextInt(MAX_ENTRIES); + t.num = RND.nextInt(MAX_ENTRIES); + allEntries.add(t); + db.write(t); + } + + // Pick the first generated value, and forcefully create a few entries that will clash + // with the indexed values (id and name), to make sure the index behaves correctly when + // multiple entities are indexed by the same value. + // + // This also serves as a test for the test code itself, to make sure it's sorting indices + // the same way the store is expected to. + CustomType1 first = allEntries.get(0); + clashingEntries = new ArrayList<>(); + for (int i = 0; i < RND.nextInt(MIN_ENTRIES) + 1; i++) { + CustomType1 t = new CustomType1(); + t.key = "n-key" + (count + i); + t.id = first.id; + t.name = first.name; + t.num = first.num; + allEntries.add(t); + clashingEntries.add(t); + db.write(t); + } + + // Create another entry that could cause problems: take the first entry, and make its indexed + // name be an extension of the existing ones, to make sure the implementation sorts these + // correctly even considering the separator character (shorter strings first). + CustomType1 t = new CustomType1(); + t.key = "extended-key-0"; + t.id = first.id; + t.name = first.name + "a"; + t.num = first.num; + allEntries.add(t); + db.write(t); + } + + @AfterClass + public static void cleanup() throws Exception { + allEntries = null; + if (db != null) { + db.close(); + } + if (dbpath != null) { + FileUtils.deleteQuietly(dbpath); + } + } + + @Test + public void naturalIndex() throws Exception { + testIteration(NATURAL_ORDER, view(), null); + } + + @Test + public void refIndex() throws Exception { + testIteration(REF_INDEX_ORDER, view().index("id"), null); + } + + @Test + public void copyIndex() throws Exception { + testIteration(COPY_INDEX_ORDER, view().index("name"), null); + } + + @Test + public void numericIndex() throws Exception { + testIteration(NUMERIC_INDEX_ORDER, view().index("int"), null); + } + + @Test + public void naturalIndexDescending() throws Exception { + testIteration(NATURAL_ORDER, view().reverse(), null); + } + + @Test + public void refIndexDescending() throws Exception { + testIteration(REF_INDEX_ORDER, view().index("id").reverse(), null); + } + + @Test + public void copyIndexDescending() throws Exception { + testIteration(COPY_INDEX_ORDER, view().index("name").reverse(), null); + } + + @Test + public void numericIndexDescending() throws Exception { + testIteration(NUMERIC_INDEX_ORDER, view().index("int").reverse(), null); + } + + @Test + public void naturalIndexWithStart() throws Exception { + CustomType1 first = pickFirst(); + testIteration(NATURAL_ORDER, view().first(first.key), first); + } + + @Test + public void refIndexWithStart() throws Exception { + CustomType1 first = pickFirst(); + testIteration(REF_INDEX_ORDER, view().index("id").first(first.id), first); + } + + @Test + public void copyIndexWithStart() throws Exception { + CustomType1 first = pickFirst(); + testIteration(COPY_INDEX_ORDER, view().index("name").first(first.name), first); + } + + @Test + public void numericIndexWithStart() throws Exception { + CustomType1 first = pickFirst(); + testIteration(NUMERIC_INDEX_ORDER, view().index("int").first(first.num), first); + } + + @Test + public void naturalIndexDescendingWithStart() throws Exception { + CustomType1 first = pickFirst(); + testIteration(NATURAL_ORDER, view().reverse().first(first.key), first); + } + + @Test + public void refIndexDescendingWithStart() throws Exception { + CustomType1 first = pickFirst(); + testIteration(REF_INDEX_ORDER, view().reverse().index("id").first(first.id), first); + } + + @Test + public void copyIndexDescendingWithStart() throws Exception { + CustomType1 first = pickFirst(); + testIteration(COPY_INDEX_ORDER, view().reverse().index("name").first(first.name), + first); + } + + @Test + public void numericIndexDescendingWithStart() throws Exception { + CustomType1 first = pickFirst(); + testIteration(NUMERIC_INDEX_ORDER, view().reverse().index("int").first(first.num), + first); + } + + @Test + public void naturalIndexWithSkip() throws Exception { + testIteration(NATURAL_ORDER, view().skip(RND.nextInt(allEntries.size() / 2)), null); + } + + @Test + public void refIndexWithSkip() throws Exception { + testIteration(REF_INDEX_ORDER, view().index("id").skip(RND.nextInt(allEntries.size() / 2)), + null); + } + + @Test + public void copyIndexWithSkip() throws Exception { + testIteration(COPY_INDEX_ORDER, view().index("name").skip(RND.nextInt(allEntries.size() / 2)), + null); + } + + @Test + public void testRefWithIntNaturalKey() throws Exception { + LevelDBSuite.IntKeyType i = new LevelDBSuite.IntKeyType(); + i.key = 1; + i.id = "1"; + i.values = Arrays.asList("1"); + + db.write(i); + + try(KVStoreIterator it = db.view(i.getClass()).closeableIterator()) { + Object read = it.next(); + assertEquals(i, read); + } + } + + private CustomType1 pickFirst() { + // Picks a first element that has clashes with other elements in the given index. + return clashingEntries.get(RND.nextInt(clashingEntries.size())); + } + + /** + * Compares the two values and falls back to comparing the natural key of CustomType1 + * if they're the same, to mimic the behavior of the indexing code. + */ + private > int compareWithFallback( + T v1, + T v2, + CustomType1 ct1, + CustomType1 ct2) { + int result = v1.compareTo(v2); + if (result != 0) { + return result; + } + + return ct1.key.compareTo(ct2.key); + } + + private void testIteration( + final BaseComparator order, + final KVStoreView params, + final CustomType1 first) throws Exception { + List indexOrder = sortBy(order.fallback()); + if (!params.ascending) { + indexOrder = Lists.reverse(indexOrder); + } + + Iterable expected = indexOrder; + if (first != null) { + final BaseComparator expectedOrder = params.ascending ? order : order.reverse(); + expected = Iterables.filter(expected, new Predicate() { + @Override + public boolean apply(CustomType1 v) { + return expectedOrder.compare(first, v) <= 0; + } + }); + } + + if (params.skip > 0) { + expected = Iterables.skip(expected, (int) params.skip); + } + + List actual = collect(params); + compareLists(expected, actual); + } + + /** Could use assertEquals(), but that creates hard to read errors for large lists. */ + private void compareLists(Iterable expected, List actual) { + Iterator expectedIt = expected.iterator(); + Iterator actualIt = actual.iterator(); + + int count = 0; + while (expectedIt.hasNext()) { + if (!actualIt.hasNext()) { + break; + } + count++; + assertEquals(expectedIt.next(), actualIt.next()); + } + + String message; + Object[] remaining; + int expectedCount = count; + int actualCount = count; + + if (expectedIt.hasNext()) { + remaining = Iterators.toArray(expectedIt, Object.class); + expectedCount += remaining.length; + message = "missing"; + } else { + remaining = Iterators.toArray(actualIt, Object.class); + actualCount += remaining.length; + message = "stray"; + } + + assertEquals(String.format("Found %s elements: %s", message, Arrays.asList(remaining)), + expectedCount, actualCount); + } + + private KVStoreView view() throws Exception { + return db.view(CustomType1.class); + } + + private List collect(KVStoreView view) throws Exception { + return Arrays.asList(Iterables.toArray(view, CustomType1.class)); + } + + private List sortBy(Comparator comp) { + List copy = new ArrayList<>(allEntries); + Collections.sort(copy, comp); + return copy; + } + +} diff --git a/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBSuite.java b/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBSuite.java new file mode 100644 index 0000000000000..2f83345ba8d5a --- /dev/null +++ b/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBSuite.java @@ -0,0 +1,281 @@ +/* + * 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.kvstore; + +import java.io.File; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.NoSuchElementException; +import static java.nio.charset.StandardCharsets.UTF_8; + +import com.google.common.collect.Iterators; +import org.apache.commons.io.FileUtils; +import org.iq80.leveldb.DBIterator; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import static org.junit.Assert.*; + +public class LevelDBSuite { + + private LevelDB db; + private File dbpath; + + @After + public void cleanup() throws Exception { + if (db != null) { + db.close(); + } + if (dbpath != null) { + FileUtils.deleteQuietly(dbpath); + } + } + + @Before + public void setup() throws Exception { + dbpath = File.createTempFile("test.", ".ldb"); + dbpath.delete(); + db = new LevelDB(dbpath); + } + + @Test + public void testReopenAndVersionCheckDb() throws Exception { + db.close(); + db = null; + assertTrue(dbpath.exists()); + + db = new LevelDB(dbpath); + assertEquals(LevelDB.STORE_VERSION, + db.serializer.deserializeLong(db.db.get(LevelDB.STORE_VERSION_KEY))); + db.db.put(LevelDB.STORE_VERSION_KEY, db.serializer.serialize(LevelDB.STORE_VERSION + 1)); + db.close(); + db = null; + + try { + db = new LevelDB(dbpath); + fail("Should have failed version check."); + } catch (UnsupportedStoreVersionException e) { + // Expected. + } + } + + @Test + public void testStringWriteReadDelete() throws Exception { + String string = "testString"; + byte[] key = string.getBytes(UTF_8); + testReadWriteDelete(key, string); + } + + @Test + public void testIntWriteReadDelete() throws Exception { + int value = 42; + byte[] key = "key".getBytes(UTF_8); + testReadWriteDelete(key, value); + } + + @Test + public void testSimpleTypeWriteReadDelete() throws Exception { + byte[] key = "testKey".getBytes(UTF_8); + CustomType1 t = new CustomType1(); + t.id = "id"; + t.name = "name"; + testReadWriteDelete(key, t); + } + + @Test + public void testObjectWriteReadDelete() throws Exception { + CustomType1 t = new CustomType1(); + t.key = "key"; + t.id = "id"; + t.name = "name"; + + try { + db.read(CustomType1.class, t.key); + fail("Expected exception for non-existant object."); + } catch (NoSuchElementException nsee) { + // Expected. + } + + db.write(t); + assertEquals(t, db.read(t.getClass(), t.key)); + assertEquals(1L, db.count(t.getClass())); + + db.delete(t.getClass(), t.key); + try { + db.read(t.getClass(), t.key); + fail("Expected exception for deleted object."); + } catch (NoSuchElementException nsee) { + // Expected. + } + + // Look into the actual DB and make sure that all the keys related to the type have been + // removed. + assertEquals(0, countKeys(t.getClass())); + } + + @Test + public void testMultipleObjectWriteReadDelete() throws Exception { + CustomType1 t1 = new CustomType1(); + t1.key = "key1"; + t1.id = "id"; + t1.name = "name1"; + + CustomType1 t2 = new CustomType1(); + t2.key = "key2"; + t2.id = "id"; + t2.name = "name2"; + + db.write(t1); + db.write(t2); + + assertEquals(t1, db.read(t1.getClass(), t1.key)); + assertEquals(t2, db.read(t2.getClass(), t2.key)); + assertEquals(2L, db.count(t1.getClass())); + + // There should be one "id" index entry with two values. + assertEquals(2, countIndexEntries(t1.getClass(), "id", t1.id)); + + // Delete the first entry; now there should be 3 remaining keys, since one of the "name" + // index entries should have been removed. + db.delete(t1.getClass(), t1.key); + + // Make sure there's a single entry in the "id" index now. + assertEquals(1, countIndexEntries(t2.getClass(), "id", t2.id)); + + // Delete the remaining entry, make sure all data is gone. + db.delete(t2.getClass(), t2.key); + assertEquals(0, countKeys(t2.getClass())); + } + + @Test + public void testMultipleTypesWriteReadDelete() throws Exception { + CustomType1 t1 = new CustomType1(); + t1.key = "1"; + t1.id = "id"; + t1.name = "name1"; + + IntKeyType t2 = new IntKeyType(); + t2.key = 2; + t2.id = "2"; + t2.values = Arrays.asList("value1", "value2"); + + db.write(t1); + db.write(t2); + + assertEquals(t1, db.read(t1.getClass(), t1.key)); + assertEquals(t2, db.read(t2.getClass(), t2.key)); + + // There should be one "id" index with a single entry for each type. + assertEquals(1, countIndexEntries(t1.getClass(), "id", t1.id)); + assertEquals(1, countIndexEntries(t2.getClass(), "id", t2.id)); + + // Delete the first entry; this should not affect the entries for the second type. + db.delete(t1.getClass(), t1.key); + assertEquals(0, countKeys(t1.getClass())); + assertEquals(1, countIndexEntries(t2.getClass(), "id", t2.id)); + + // Delete the remaining entry, make sure all data is gone. + db.delete(t2.getClass(), t2.key); + assertEquals(0, countKeys(t2.getClass())); + } + + @Test + public void testMetadata() throws Exception { + assertNull(db.getMetadata(CustomType1.class)); + + CustomType1 t = new CustomType1(); + t.id = "id"; + t.name = "name"; + + db.setMetadata(t); + assertEquals(t, db.getMetadata(CustomType1.class)); + + db.setMetadata(null); + assertNull(db.getMetadata(CustomType1.class)); + } + + private long countIndexEntries(Class type, String index, Object value) throws Exception { + LevelDBTypeInfo.Index idx = db.getTypeInfo(type).index(index); + return idx.getCount(idx.end()); + } + + private int countKeys(Class type) throws Exception { + byte[] prefix = db.getTypeInfo(type).keyPrefix(); + int count = 0; + + DBIterator it = db.db.iterator(); + it.seek(prefix); + + while (it.hasNext()) { + byte[] key = it.next().getKey(); + if (LevelDBIterator.startsWith(key, prefix)) { + count++; + } + } + + return count; + } + + private void testReadWriteDelete(byte[] key, T value) throws Exception { + try { + db.get(key, value.getClass()); + fail("Expected exception for non-existent key."); + } catch (NoSuchElementException nsee) { + // Expected. + } + + db.put(key, value); + assertEquals(value, db.get(key, value.getClass())); + + db.delete(key); + try { + db.get(key, value.getClass()); + fail("Expected exception for deleted key."); + } catch (NoSuchElementException nsee) { + // Expected. + } + } + + public static class IntKeyType { + + @KVIndex + public int key; + + @KVIndex("id") + public String id; + + public List values; + + @Override + public boolean equals(Object o) { + if (o instanceof IntKeyType) { + IntKeyType other = (IntKeyType) o; + return key == other.key && id.equals(other.id) && values.equals(other.values); + } + return false; + } + + @Override + public int hashCode() { + return id.hashCode(); + } + + } + +} diff --git a/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBTypeInfoSuite.java b/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBTypeInfoSuite.java new file mode 100644 index 0000000000000..2ef69fd33ce19 --- /dev/null +++ b/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBTypeInfoSuite.java @@ -0,0 +1,177 @@ +/* + * 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.kvstore; + +import static java.nio.charset.StandardCharsets.UTF_8; + +import org.junit.Test; +import static org.junit.Assert.*; + +public class LevelDBTypeInfoSuite { + + @Test + public void testIndexAnnotation() throws Exception { + LevelDBTypeInfo ti = newTypeInfo(CustomType1.class); + assertEquals(4, ti.indices().size()); + + CustomType1 t1 = new CustomType1(); + t1.key = "key"; + t1.id = "id"; + t1.name = "name"; + t1.num = 42; + + assertEquals(t1.key, ti.naturalIndex().accessor.get(t1)); + assertEquals(t1.id, ti.index("id").accessor.get(t1)); + assertEquals(t1.name, ti.index("name").accessor.get(t1)); + assertEquals(t1.num, ti.index("int").accessor.get(t1)); + } + + @Test(expected = IllegalArgumentException.class) + public void testNoNaturalIndex() throws Exception { + newTypeInfo(NoNaturalIndex.class); + } + + @Test(expected = IllegalArgumentException.class) + public void testDuplicateIndex() throws Exception { + newTypeInfo(DuplicateIndex.class); + } + + @Test(expected = IllegalArgumentException.class) + public void testEmptyIndexName() throws Exception { + newTypeInfo(EmptyIndexName.class); + } + + @Test(expected = IllegalArgumentException.class) + public void testIllegalIndexName() throws Exception { + newTypeInfo(IllegalIndexName.class); + } + + @Test(expected = IllegalArgumentException.class) + public void testIllegalIndexMethod() throws Exception { + newTypeInfo(IllegalIndexMethod.class); + } + + @Test + public void testKeyClashes() throws Exception { + LevelDBTypeInfo ti = newTypeInfo(CustomType1.class); + + CustomType1 t1 = new CustomType1(); + t1.key = "key1"; + t1.name = "a"; + + CustomType1 t2 = new CustomType1(); + t2.key = "key2"; + t2.name = "aa"; + + CustomType1 t3 = new CustomType1(); + t3.key = "key3"; + t3.name = "aaa"; + + // Make sure entries with conflicting names are sorted correctly. + assertBefore(ti.index("name").entityKey(t1), ti.index("name").entityKey(t2)); + assertBefore(ti.index("name").entityKey(t1), ti.index("name").entityKey(t3)); + assertBefore(ti.index("name").entityKey(t2), ti.index("name").entityKey(t3)); + } + + @Test + public void testNumEncoding() throws Exception { + LevelDBTypeInfo.Index idx = newTypeInfo(CustomType1.class).indices().iterator().next(); + + assertBefore(idx.toKey(1), idx.toKey(2)); + assertBefore(idx.toKey(-1), idx.toKey(2)); + assertBefore(idx.toKey(-11), idx.toKey(2)); + assertBefore(idx.toKey(-11), idx.toKey(-1)); + assertBefore(idx.toKey(1), idx.toKey(11)); + assertBefore(idx.toKey(Integer.MIN_VALUE), idx.toKey(Integer.MAX_VALUE)); + assertEquals(LevelDBTypeInfo.INT_ENCODED_LEN + LevelDBTypeInfo.ENTRY_PREFIX.length(), + idx.toKey(Integer.MIN_VALUE).length()); + + assertBefore(idx.toKey(1L), idx.toKey(2L)); + assertBefore(idx.toKey(-1L), idx.toKey(2L)); + assertBefore(idx.toKey(Long.MIN_VALUE), idx.toKey(Long.MAX_VALUE)); + assertEquals(LevelDBTypeInfo.LONG_ENCODED_LEN + LevelDBTypeInfo.ENTRY_PREFIX.length(), + idx.toKey(Long.MIN_VALUE).length()); + + assertBefore(idx.toKey((short) 1), idx.toKey((short) 2)); + assertBefore(idx.toKey((short) -1), idx.toKey((short) 2)); + assertBefore(idx.toKey(Short.MIN_VALUE), idx.toKey(Short.MAX_VALUE)); + assertEquals(LevelDBTypeInfo.SHORT_ENCODED_LEN + LevelDBTypeInfo.ENTRY_PREFIX.length(), + idx.toKey(Short.MIN_VALUE).length()); + + assertBefore(idx.toKey((byte) 1), idx.toKey((byte) 2)); + assertBefore(idx.toKey((byte) -1), idx.toKey((byte) 2)); + assertBefore(idx.toKey(Byte.MIN_VALUE), idx.toKey(Byte.MAX_VALUE)); + assertEquals(LevelDBTypeInfo.BYTE_ENCODED_LEN + LevelDBTypeInfo.ENTRY_PREFIX.length(), + idx.toKey(Byte.MIN_VALUE).length()); + + assertEquals(LevelDBTypeInfo.ENTRY_PREFIX + "false", idx.toKey(false)); + assertEquals(LevelDBTypeInfo.ENTRY_PREFIX + "true", idx.toKey(true)); + } + + private LevelDBTypeInfo newTypeInfo(Class type) throws Exception { + return new LevelDBTypeInfo<>(null, type); + } + + private void assertBefore(byte[] key1, byte[] key2) { + assertBefore(new String(key1, UTF_8), new String(key2, UTF_8)); + } + + private void assertBefore(String str1, String str2) { + assertTrue(String.format("%s < %s failed", str1, str2), str1.compareTo(str2) < 0); + } + + public static class NoNaturalIndex { + + public String id; + + } + + public static class DuplicateIndex { + + @KVIndex("id") + public String id; + + @KVIndex("id") + public String id2; + + } + + public static class EmptyIndexName { + + @KVIndex("") + public String id; + + } + + public static class IllegalIndexName { + + @KVIndex("__invalid") + public String id; + + } + + public static class IllegalIndexMethod { + + @KVIndex("id") + public String id(boolean illegalParam) { + return null; + } + + } + +} diff --git a/common/kvstore/src/test/resources/log4j.properties b/common/kvstore/src/test/resources/log4j.properties new file mode 100644 index 0000000000000..e8da774f7ca9e --- /dev/null +++ b/common/kvstore/src/test/resources/log4j.properties @@ -0,0 +1,27 @@ +# +# 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. +# + +# Set everything to be logged to the file target/unit-tests.log +log4j.rootCategory=DEBUG, file +log4j.appender.file=org.apache.log4j.FileAppender +log4j.appender.file.append=true +log4j.appender.file.file=target/unit-tests.log +log4j.appender.file.layout=org.apache.log4j.PatternLayout +log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n + +# Silence verbose logs from 3rd-party libraries. +log4j.logger.io.netty=INFO diff --git a/pom.xml b/pom.xml index 0533a8dcf2e0a..6835ea14cd42b 100644 --- a/pom.xml +++ b/pom.xml @@ -83,6 +83,7 @@ common/sketch + common/kvstore common/network-common common/network-shuffle common/unsafe @@ -441,6 +442,11 @@ httpcore ${commons.httpcore.version} + + org.fusesource.leveldbjni + leveldbjni-all + 1.8 + org.seleniumhq.selenium selenium-java @@ -588,6 +594,11 @@ metrics-graphite ${codahale.metrics.version} + + com.fasterxml.jackson.core + jackson-core + ${fasterxml.jackson.version} + com.fasterxml.jackson.core jackson-databind diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index b5362ec1ae452..89b0c7a3ab7b0 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -50,10 +50,10 @@ object BuildCommons { ).map(ProjectRef(buildLocation, _)) val allProjects@Seq( - core, graphx, mllib, mllibLocal, repl, networkCommon, networkShuffle, launcher, unsafe, tags, sketch, _* + core, graphx, mllib, mllibLocal, repl, networkCommon, networkShuffle, launcher, unsafe, tags, sketch, kvstore, _* ) = Seq( "core", "graphx", "mllib", "mllib-local", "repl", "network-common", "network-shuffle", "launcher", "unsafe", - "tags", "sketch" + "tags", "sketch", "kvstore" ).map(ProjectRef(buildLocation, _)) ++ sqlProjects ++ streamingProjects val optionallyEnabledProjects@Seq(mesos, yarn, sparkGangliaLgpl, @@ -310,7 +310,7 @@ object SparkBuild extends PomBuild { val mimaProjects = allProjects.filterNot { x => Seq( spark, hive, hiveThriftServer, catalyst, repl, networkCommon, networkShuffle, networkYarn, - unsafe, tags, sqlKafka010 + unsafe, tags, sqlKafka010, kvstore ).contains(x) } From 52ed2b45c09e7104e4fef5adcf78025f53b7a8e0 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Tue, 1 Nov 2016 11:34:25 -0700 Subject: [PATCH 02/18] SHS-NG M1: Add support for arrays when indexing. This is needed because some UI types have compound keys. --- .../org/apache/spark/kvstore/KVIndex.java | 3 +- .../apache/spark/kvstore/LevelDBTypeInfo.java | 12 ++++++ .../apache/spark/kvstore/LevelDBSuite.java | 37 ++++++++++++++++++- .../spark/kvstore/LevelDBTypeInfoSuite.java | 12 ++++++ 4 files changed, 62 insertions(+), 2 deletions(-) diff --git a/common/kvstore/src/main/java/org/apache/spark/kvstore/KVIndex.java b/common/kvstore/src/main/java/org/apache/spark/kvstore/KVIndex.java index 3c61e7706079a..bf5e4a66e510f 100644 --- a/common/kvstore/src/main/java/org/apache/spark/kvstore/KVIndex.java +++ b/common/kvstore/src/main/java/org/apache/spark/kvstore/KVIndex.java @@ -42,7 +42,8 @@ *

* *

- * Indices are restricted to String, and integral types (byte, short, int, long, boolean). + * Indices are restricted to String, integral types (byte, short, int, long, boolean), and arrays + * of those values. *

*/ @Retention(RetentionPolicy.RUNTIME) diff --git a/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBTypeInfo.java b/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBTypeInfo.java index c49b18324c00a..947d6c4945e37 100644 --- a/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBTypeInfo.java +++ b/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBTypeInfo.java @@ -17,6 +17,7 @@ package org.apache.spark.kvstore; +import java.lang.reflect.Array; import java.lang.reflect.Field; import java.lang.reflect.Method; import java.io.ByteArrayOutputStream; @@ -283,6 +284,8 @@ private void updateCount(WriteBatch batch, byte[] key, long delta) throws Except * encoded value would look like "*~~~~~~~123"). For positive values, similarly, a value that * is "lower than 0" (".") is used for padding. The fill characters were chosen for readability * when looking at the encoded keys. + * + * Arrays are encoded by encoding each element separately, separated by KEY_SEPARATOR. */ @VisibleForTesting String toKey(Object value) { @@ -292,6 +295,15 @@ String toKey(Object value) { sb.append(value); } else if (value instanceof Boolean) { sb.append(((Boolean) value).toString().toLowerCase()); + } else if (value.getClass().isArray()) { + int length = Array.getLength(value); + for (int i = 0; i < length; i++) { + sb.append(toKey(Array.get(value, i))); + sb.append(KEY_SEPARATOR); + } + if (length > 0) { + sb.setLength(sb.length() - 1); + } } else { int encodedLen; diff --git a/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBSuite.java b/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBSuite.java index 2f83345ba8d5a..868baf743e027 100644 --- a/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBSuite.java +++ b/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBSuite.java @@ -175,24 +175,35 @@ public void testMultipleTypesWriteReadDelete() throws Exception { t2.id = "2"; t2.values = Arrays.asList("value1", "value2"); + ArrayKeyIndexType t3 = new ArrayKeyIndexType(); + t3.key = new int[] { 42, 84 }; + t3.id = new String[] { "id1", "id2" }; + db.write(t1); db.write(t2); + db.write(t3); assertEquals(t1, db.read(t1.getClass(), t1.key)); assertEquals(t2, db.read(t2.getClass(), t2.key)); + assertEquals(t3, db.read(t3.getClass(), t3.key)); // There should be one "id" index with a single entry for each type. assertEquals(1, countIndexEntries(t1.getClass(), "id", t1.id)); assertEquals(1, countIndexEntries(t2.getClass(), "id", t2.id)); + assertEquals(1, countIndexEntries(t3.getClass(), "id", t3.id)); // Delete the first entry; this should not affect the entries for the second type. db.delete(t1.getClass(), t1.key); assertEquals(0, countKeys(t1.getClass())); assertEquals(1, countIndexEntries(t2.getClass(), "id", t2.id)); + assertEquals(1, countIndexEntries(t3.getClass(), "id", t3.id)); - // Delete the remaining entry, make sure all data is gone. + // Delete the remaining entries, make sure all data is gone. db.delete(t2.getClass(), t2.key); assertEquals(0, countKeys(t2.getClass())); + + db.delete(t3.getClass(), t3.key); + assertEquals(0, countKeys(t3.getClass())); } @Test @@ -278,4 +289,28 @@ public int hashCode() { } + public static class ArrayKeyIndexType { + + @KVIndex + public int[] key; + + @KVIndex("id") + public String[] id; + + @Override + public boolean equals(Object o) { + if (o instanceof ArrayKeyIndexType) { + ArrayKeyIndexType other = (ArrayKeyIndexType) o; + return Arrays.equals(key, other.key) && Arrays.equals(id, other.id); + } + return false; + } + + @Override + public int hashCode() { + return key.hashCode(); + } + + } + } diff --git a/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBTypeInfoSuite.java b/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBTypeInfoSuite.java index 2ef69fd33ce19..7ee0b24552219 100644 --- a/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBTypeInfoSuite.java +++ b/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBTypeInfoSuite.java @@ -123,6 +123,18 @@ public void testNumEncoding() throws Exception { assertEquals(LevelDBTypeInfo.ENTRY_PREFIX + "true", idx.toKey(true)); } + @Test + public void testArrayIndices() throws Exception { + LevelDBTypeInfo.Index idx = newTypeInfo(CustomType1.class).indices().iterator().next(); + + assertBefore(idx.toKey(new String[] { "str1" }), idx.toKey(new String[] { "str2" })); + assertBefore(idx.toKey(new String[] { "str1", "str2" }), + idx.toKey(new String[] { "str1", "str3" })); + + assertBefore(idx.toKey(new int[] { 1 }), idx.toKey(new int[] { 2 })); + assertBefore(idx.toKey(new int[] { 1, 2 }), idx.toKey(new int[] { 1, 3 })); + } + private LevelDBTypeInfo newTypeInfo(Class type) throws Exception { return new LevelDBTypeInfo<>(null, type); } From 4112afe723f85412035ad3a9c4801b583e74f876 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Thu, 3 Nov 2016 15:18:24 -0700 Subject: [PATCH 03/18] SHS-NG M1: Fix counts in LevelDB when updating entries. Also add unit test. When updating, the code needs to keep track of the aggregated delta to be added to each count stored in the db, instead of reading the count from the db for each update. --- .../org/apache/spark/kvstore/LevelDB.java | 10 +- .../apache/spark/kvstore/LevelDBTypeInfo.java | 22 +--- .../spark/kvstore/LevelDBWriteBatch.java | 113 ++++++++++++++++++ .../apache/spark/kvstore/LevelDBSuite.java | 26 ++++ 4 files changed, 150 insertions(+), 21 deletions(-) create mode 100644 common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBWriteBatch.java diff --git a/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDB.java b/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDB.java index 51287c02ebab1..2b7ea6889aee1 100644 --- a/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDB.java +++ b/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDB.java @@ -136,7 +136,7 @@ public void write(Object value, boolean sync) throws Exception { Preconditions.checkArgument(value != null, "Null values are not allowed."); LevelDBTypeInfo ti = getTypeInfo(value.getClass()); - WriteBatch batch = db.createWriteBatch(); + LevelDBWriteBatch batch = new LevelDBWriteBatch(this); try { byte[] data = serializer.serialize(value); synchronized (ti) { @@ -149,7 +149,7 @@ public void write(Object value, boolean sync) throws Exception { for (LevelDBTypeInfo.Index idx : ti.indices()) { idx.add(batch, value, data); } - db.write(batch, new WriteOptions().sync(sync)); + batch.write(sync); } } finally { batch.close(); @@ -163,7 +163,7 @@ public void delete(Class type, Object naturalKey) throws Exception { public void delete(Class type, Object naturalKey, boolean sync) throws Exception { Preconditions.checkArgument(naturalKey != null, "Null keys are not allowed."); - WriteBatch batch = db.createWriteBatch(); + LevelDBWriteBatch batch = new LevelDBWriteBatch(this); try { LevelDBTypeInfo ti = getTypeInfo(type); byte[] key = ti.naturalIndex().start(naturalKey); @@ -172,7 +172,7 @@ public void delete(Class type, Object naturalKey, boolean sync) throws Except Object existing = serializer.deserialize(data, type); synchronized (ti) { removeInstance(ti, batch, existing); - db.write(batch, new WriteOptions().sync(sync)); + batch.write(sync); } } } finally { @@ -229,7 +229,7 @@ LevelDBTypeInfo getTypeInfo(Class type) throws Exception { return idx; } - private void removeInstance(LevelDBTypeInfo ti, WriteBatch batch, Object instance) + private void removeInstance(LevelDBTypeInfo ti, LevelDBWriteBatch batch, Object instance) throws Exception { for (LevelDBTypeInfo.Index idx : ti.indices()) { idx.remove(batch, instance); diff --git a/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBTypeInfo.java b/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBTypeInfo.java index 947d6c4945e37..184b6611e0e0a 100644 --- a/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBTypeInfo.java +++ b/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBTypeInfo.java @@ -30,7 +30,6 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.base.Throwables; -import org.iq80.leveldb.WriteBatch; /** * Holds metadata about app-specific types stored in LevelDB. Serves as a cache for data collected @@ -235,14 +234,14 @@ byte[] entityKey(Object entity) throws Exception { * @param data Serialized entity to store (when storing the entity, not a reference). * @param naturalKey The value's key. */ - void add(WriteBatch batch, Object entity, byte[] data) throws Exception { + void add(LevelDBWriteBatch batch, Object entity, byte[] data) throws Exception { byte[] stored = data; if (!copy) { stored = db.serializer.serialize(toKey(naturalIndex().accessor.get(entity))); } batch.put(entityKey(entity), stored); - updateCount(batch, end(accessor.get(entity)), 1L); - updateCount(batch, end(), 1L); + batch.updateCount(end(accessor.get(entity)), 1L); + batch.updateCount(end(), 1L); } /** @@ -252,10 +251,10 @@ void add(WriteBatch batch, Object entity, byte[] data) throws Exception { * @param entity The entity being removed, to identify the index entry to modify. * @param naturalKey The value's key. */ - void remove(WriteBatch batch, Object entity) throws Exception { + void remove(LevelDBWriteBatch batch, Object entity) throws Exception { batch.delete(entityKey(entity)); - updateCount(batch, end(accessor.get(entity)), -1L); - updateCount(batch, end(), -1L); + batch.updateCount(end(accessor.get(entity)), -1L); + batch.updateCount(end(), -1L); } long getCount(byte[] key) throws Exception { @@ -263,15 +262,6 @@ long getCount(byte[] key) throws Exception { return data != null ? db.serializer.deserializeLong(data) : 0; } - private void updateCount(WriteBatch batch, byte[] key, long delta) throws Exception { - long count = getCount(key) + delta; - if (count > 0) { - batch.put(key, db.serializer.serialize(count)); - } else { - batch.delete(key); - } - } - /** * Translates a value to be used as part of the store key. * diff --git a/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBWriteBatch.java b/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBWriteBatch.java new file mode 100644 index 0000000000000..a6ca190222931 --- /dev/null +++ b/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBWriteBatch.java @@ -0,0 +1,113 @@ +/* + * 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.kvstore; + +import java.io.IOException; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; + +import org.iq80.leveldb.DB; +import org.iq80.leveldb.WriteBatch; +import org.iq80.leveldb.WriteOptions; + +/** + * A wrapper around the LevelDB library's WriteBatch with some extra functionality for keeping + * track of counts. + */ +class LevelDBWriteBatch { + + private final LevelDB db; + private final Map deltas; + private final WriteBatch batch; + + LevelDBWriteBatch(LevelDB db) { + this.db = db; + this.batch = db.db.createWriteBatch(); + this.deltas = new HashMap<>(); + } + + void updateCount(byte[] key, long delta) { + KeyWrapper kw = new KeyWrapper(key); + Long fullDelta = deltas.get(kw); + if (fullDelta != null) { + fullDelta += delta; + } else { + fullDelta = delta; + } + deltas.put(kw, fullDelta); + } + + void put(byte[] key, byte[] value) { + batch.put(key, value); + } + + void delete(byte[] key) { + batch.delete(key); + } + + void write(boolean sync) { + for (Map.Entry e : deltas.entrySet()) { + long delta = e.getValue(); + if (delta == 0) { + continue; + } + + byte[] key = e.getKey().key; + byte[] data = db.db.get(key); + long count = data != null ? db.serializer.deserializeLong(data) : 0L; + long newCount = count + delta; + + if (newCount > 0) { + batch.put(key, db.serializer.serialize(newCount)); + } else { + batch.delete(key); + } + } + + db.db.write(batch, new WriteOptions().sync(sync)); + } + + void close() throws IOException { + batch.close(); + } + + private static class KeyWrapper { + + private final byte[] key; + + KeyWrapper(byte[] key) { + this.key = key; + } + + @Override + public boolean equals(Object other) { + if (other instanceof KeyWrapper) { + return Arrays.equals(key, ((KeyWrapper) other).key); + } + return false; + } + + @Override + public int hashCode() { + return Arrays.hashCode(key); + } + + } + +} diff --git a/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBSuite.java b/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBSuite.java index 868baf743e027..5dc1c10765274 100644 --- a/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBSuite.java +++ b/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBSuite.java @@ -226,6 +226,32 @@ private long countIndexEntries(Class type, String index, Object value) throws return idx.getCount(idx.end()); } + @Test + public void testUpdate() throws Exception { + CustomType1 t = new CustomType1(); + t.key = "key"; + t.id = "id"; + t.name = "name"; + + db.write(t); + + t.name = "anotherName"; + + db.write(t); + + assertEquals(1, db.count(t.getClass())); + + LevelDBTypeInfo.Index ni = db.getTypeInfo(t.getClass()).index("name"); + assertEquals(1, ni.getCount(ni.end())); + assertEquals(1, ni.getCount(ni.end("anotherName"))); + try { + db.get(ni.end("name"), Integer.class); + fail("Should have gotten an exception."); + } catch (NoSuchElementException nsee) { + // Expected. + } + } + private int countKeys(Class type) throws Exception { byte[] prefix = db.getTypeInfo(type).keyPrefix(); int count = 0; From 718cabd098dd6a534e7952066cd43f89f6875a14 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Fri, 17 Mar 2017 20:17:04 -0700 Subject: [PATCH 04/18] SHS-NG M1: Try to prevent db use after close. This causes JVM crashes in the leveldb library, so try to avoid it; if there are still issues, we'll neeed locking. --- .../org/apache/spark/kvstore/LevelDB.java | 41 ++++++++++++------- .../apache/spark/kvstore/LevelDBIterator.java | 6 +-- .../apache/spark/kvstore/LevelDBTypeInfo.java | 2 +- .../spark/kvstore/LevelDBWriteBatch.java | 6 +-- .../apache/spark/kvstore/LevelDBSuite.java | 6 +-- 5 files changed, 37 insertions(+), 24 deletions(-) diff --git a/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDB.java b/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDB.java index 2b7ea6889aee1..35cdbb6733a39 100644 --- a/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDB.java +++ b/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDB.java @@ -23,6 +23,7 @@ import java.util.NoSuchElementException; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.atomic.AtomicReference; import static java.nio.charset.StandardCharsets.UTF_8; import com.google.common.annotations.VisibleForTesting; @@ -48,11 +49,10 @@ public class LevelDB implements KVStore { /** DB key where app metadata is stored. */ private static final byte[] METADATA_KEY = "__meta__".getBytes(UTF_8); - final DB db; + final AtomicReference _db; final KVStoreSerializer serializer; private final ConcurrentMap, LevelDBTypeInfo> types; - private boolean closed; public LevelDB(File path) throws IOException { this(path, new KVStoreSerializer()); @@ -64,16 +64,16 @@ public LevelDB(File path, KVStoreSerializer serializer) throws IOException { Options options = new Options(); options.createIfMissing(!path.exists()); - this.db = JniDBFactory.factory.open(path, options); + this._db = new AtomicReference<>(JniDBFactory.factory.open(path, options)); - byte[] versionData = db.get(STORE_VERSION_KEY); + byte[] versionData = db().get(STORE_VERSION_KEY); if (versionData != null) { long version = serializer.deserializeLong(versionData); if (version != STORE_VERSION) { throw new UnsupportedStoreVersionException(); } } else { - db.put(STORE_VERSION_KEY, serializer.serialize(STORE_VERSION)); + db().put(STORE_VERSION_KEY, serializer.serialize(STORE_VERSION)); } } @@ -91,13 +91,13 @@ public void setMetadata(Object value) throws Exception { if (value != null) { put(METADATA_KEY, value); } else { - db.delete(METADATA_KEY); + db().delete(METADATA_KEY); } } @Override public T get(byte[] key, Class klass) throws Exception { - byte[] data = db.get(key); + byte[] data = db().get(key); if (data == null) { throw new NoSuchElementException(new String(key, UTF_8)); } @@ -107,12 +107,12 @@ public T get(byte[] key, Class klass) throws Exception { @Override public void put(byte[] key, Object value) throws Exception { Preconditions.checkArgument(value != null, "Null values are not allowed."); - db.put(key, serializer.serialize(value)); + db().put(key, serializer.serialize(value)); } @Override public void delete(byte[] key) throws Exception { - db.delete(key); + db().delete(key); } @Override @@ -167,7 +167,7 @@ public void delete(Class type, Object naturalKey, boolean sync) throws Except try { LevelDBTypeInfo ti = getTypeInfo(type); byte[] key = ti.naturalIndex().start(naturalKey); - byte[] data = db.get(key); + byte[] data = db().get(key); if (data != null) { Object existing = serializer.deserialize(data, type); synchronized (ti) { @@ -201,14 +201,14 @@ public long count(Class type) throws Exception { } @Override - public synchronized void close() throws IOException { - if (closed) { + public void close() throws IOException { + DB _db = this._db.getAndSet(null); + if (_db == null) { return; } try { - db.close(); - closed = true; + _db.close(); } catch (IOException ioe) { throw ioe; } catch (Exception e) { @@ -229,6 +229,19 @@ LevelDBTypeInfo getTypeInfo(Class type) throws Exception { return idx; } + /** + * Try to avoid use-after close since that has the tendency of crashing the JVM. This doesn't + * prevent methods that retrieved the instance from using it after close, but hopefully will + * catch most cases; otherwise, we'll need some kind of locking. + */ + DB db() { + DB _db = this._db.get(); + if (_db == null) { + throw new IllegalStateException("DB is closed."); + } + return _db; + } + private void removeInstance(LevelDBTypeInfo ti, LevelDBWriteBatch batch, Object instance) throws Exception { for (LevelDBTypeInfo.Index idx : ti.indices()) { diff --git a/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBIterator.java b/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBIterator.java index d0b6e25420812..b777ff7bafc02 100644 --- a/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBIterator.java +++ b/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBIterator.java @@ -52,7 +52,7 @@ class LevelDBIterator implements KVStoreIterator { this.type = type; this.ti = null; this.index = null; - this.it = db.db.iterator(); + this.it = db.db().iterator(); this.indexKeyPrefix = keyPrefix; this.end = null; it.seek(keyPrefix); @@ -64,7 +64,7 @@ class LevelDBIterator implements KVStoreIterator { LevelDBIterator(LevelDB db, KVStoreView params) throws Exception { this.db = db; this.ascending = params.ascending; - this.it = db.db.iterator(); + this.it = db.db().iterator(); this.type = params.type; this.ti = db.getTypeInfo(type); this.index = ti.index(params.index); @@ -157,7 +157,7 @@ public boolean skip(long n) { } @Override - public void close() throws IOException { + public synchronized void close() throws IOException { if (!closed) { it.close(); closed = true; diff --git a/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBTypeInfo.java b/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBTypeInfo.java index 184b6611e0e0a..e0f3dc80cfe62 100644 --- a/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBTypeInfo.java +++ b/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBTypeInfo.java @@ -258,7 +258,7 @@ void remove(LevelDBWriteBatch batch, Object entity) throws Exception { } long getCount(byte[] key) throws Exception { - byte[] data = db.db.get(key); + byte[] data = db.db().get(key); return data != null ? db.serializer.deserializeLong(data) : 0; } diff --git a/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBWriteBatch.java b/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBWriteBatch.java index a6ca190222931..f3de251de554f 100644 --- a/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBWriteBatch.java +++ b/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBWriteBatch.java @@ -38,7 +38,7 @@ class LevelDBWriteBatch { LevelDBWriteBatch(LevelDB db) { this.db = db; - this.batch = db.db.createWriteBatch(); + this.batch = db.db().createWriteBatch(); this.deltas = new HashMap<>(); } @@ -69,7 +69,7 @@ void write(boolean sync) { } byte[] key = e.getKey().key; - byte[] data = db.db.get(key); + byte[] data = db.db().get(key); long count = data != null ? db.serializer.deserializeLong(data) : 0L; long newCount = count + delta; @@ -80,7 +80,7 @@ void write(boolean sync) { } } - db.db.write(batch, new WriteOptions().sync(sync)); + db.db().write(batch, new WriteOptions().sync(sync)); } void close() throws IOException { diff --git a/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBSuite.java b/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBSuite.java index 5dc1c10765274..3b43307c9580c 100644 --- a/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBSuite.java +++ b/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBSuite.java @@ -62,8 +62,8 @@ public void testReopenAndVersionCheckDb() throws Exception { db = new LevelDB(dbpath); assertEquals(LevelDB.STORE_VERSION, - db.serializer.deserializeLong(db.db.get(LevelDB.STORE_VERSION_KEY))); - db.db.put(LevelDB.STORE_VERSION_KEY, db.serializer.serialize(LevelDB.STORE_VERSION + 1)); + db.serializer.deserializeLong(db.db().get(LevelDB.STORE_VERSION_KEY))); + db.db().put(LevelDB.STORE_VERSION_KEY, db.serializer.serialize(LevelDB.STORE_VERSION + 1)); db.close(); db = null; @@ -256,7 +256,7 @@ private int countKeys(Class type) throws Exception { byte[] prefix = db.getTypeInfo(type).keyPrefix(); int count = 0; - DBIterator it = db.db.iterator(); + DBIterator it = db.db().iterator(); it.seek(prefix); while (it.hasNext()) { From 45a027fd5e32421b57846236180d6012ee72e69b Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Fri, 24 Mar 2017 13:19:07 -0700 Subject: [PATCH 05/18] SHS-NG M1: Use Java 8 lambdas. Also rename LevelDBIteratorSuite to work around some super weird issue with sbt. --- ...teratorSuite.java => DBIteratorSuite.java} | 72 ++++++------------- .../apache/spark/kvstore/LevelDBSuite.java | 1 - 2 files changed, 20 insertions(+), 53 deletions(-) rename common/kvstore/src/test/java/org/apache/spark/kvstore/{LevelDBIteratorSuite.java => DBIteratorSuite.java} (86%) diff --git a/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBIteratorSuite.java b/common/kvstore/src/test/java/org/apache/spark/kvstore/DBIteratorSuite.java similarity index 86% rename from common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBIteratorSuite.java rename to common/kvstore/src/test/java/org/apache/spark/kvstore/DBIteratorSuite.java index b67503b3fcbea..88c7cc08984bb 100644 --- a/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBIteratorSuite.java +++ b/common/kvstore/src/test/java/org/apache/spark/kvstore/DBIteratorSuite.java @@ -36,7 +36,11 @@ import org.junit.Test; import static org.junit.Assert.*; -public class LevelDBIteratorSuite { +/** + * This class should really be called "LevelDBIteratorSuite" but for some reason I don't know, + * sbt does not run the tests if it has that name. + */ +public class DBIteratorSuite { private static final int MIN_ENTRIES = 42; private static final int MAX_ENTRIES = 1024; @@ -47,63 +51,32 @@ public class LevelDBIteratorSuite { private static LevelDB db; private static File dbpath; - private abstract class BaseComparator implements Comparator { + private interface BaseComparator extends Comparator { /** * Returns a comparator that falls back to natural order if this comparator's ordering * returns equality for two elements. Used to mimic how the index sorts things internally. */ - BaseComparator fallback() { - return new BaseComparator() { - @Override - public int compare(CustomType1 t1, CustomType1 t2) { - int result = BaseComparator.this.compare(t1, t2); - if (result != 0) { - return result; - } - - return t1.key.compareTo(t2.key); + default BaseComparator fallback() { + return (t1, t2) -> { + int result = BaseComparator.this.compare(t1, t2); + if (result != 0) { + return result; } + + return t1.key.compareTo(t2.key); }; } /** Reverses the order of this comparator. */ - BaseComparator reverse() { - return new BaseComparator() { - @Override - public int compare(CustomType1 t1, CustomType1 t2) { - return -BaseComparator.this.compare(t1, t2); - } - }; + default BaseComparator reverse() { + return (t1, t2) -> -BaseComparator.this.compare(t1, t2); } } - private final BaseComparator NATURAL_ORDER = new BaseComparator() { - @Override - public int compare(CustomType1 t1, CustomType1 t2) { - return t1.key.compareTo(t2.key); - } - }; - - private final BaseComparator REF_INDEX_ORDER = new BaseComparator() { - @Override - public int compare(CustomType1 t1, CustomType1 t2) { - return t1.id.compareTo(t2.id); - } - }; - - private final BaseComparator COPY_INDEX_ORDER = new BaseComparator() { - @Override - public int compare(CustomType1 t1, CustomType1 t2) { - return t1.name.compareTo(t2.name); - } - }; - - private final BaseComparator NUMERIC_INDEX_ORDER = new BaseComparator() { - @Override - public int compare(CustomType1 t1, CustomType1 t2) { - return t1.num - t2.num; - } - }; + private final BaseComparator NATURAL_ORDER = (t1, t2) -> t1.key.compareTo(t2.key); + private final BaseComparator REF_INDEX_ORDER = (t1, t2) -> t1.id.compareTo(t2.id); + private final BaseComparator COPY_INDEX_ORDER = (t1, t2) -> t1.name.compareTo(t2.name); + private final BaseComparator NUMERIC_INDEX_ORDER = (t1, t2) -> t1.num - t2.num; @BeforeClass public static void setup() throws Exception { @@ -333,12 +306,7 @@ private void testIteration( Iterable expected = indexOrder; if (first != null) { final BaseComparator expectedOrder = params.ascending ? order : order.reverse(); - expected = Iterables.filter(expected, new Predicate() { - @Override - public boolean apply(CustomType1 v) { - return expectedOrder.compare(first, v) <= 0; - } - }); + expected = Iterables.filter(expected, v -> expectedOrder.compare(first, v) <= 0); } if (params.skip > 0) { diff --git a/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBSuite.java b/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBSuite.java index 3b43307c9580c..1d33ba099f4f8 100644 --- a/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBSuite.java +++ b/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBSuite.java @@ -24,7 +24,6 @@ import java.util.NoSuchElementException; import static java.nio.charset.StandardCharsets.UTF_8; -import com.google.common.collect.Iterators; import org.apache.commons.io.FileUtils; import org.iq80.leveldb.DBIterator; import org.junit.After; From e592bf69b94c3308d194c2cb678be133931b95b5 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Fri, 24 Mar 2017 17:24:08 -0700 Subject: [PATCH 06/18] SHS-NG M1: Compress values stored in LevelDB. LevelDB has built-in support for snappy compression, but it seems to be buggy in the leveldb-jni library; the compression threads don't seem to run by default, and when you enable them, there are weird issues when stopping the DB. So just do compression manually using the JRE libraries; it's probably a little slower but it saves a good chunk of disk space. --- .../spark/kvstore/KVStoreSerializer.java | 20 +++++++++++++++++-- 1 file changed, 18 insertions(+), 2 deletions(-) diff --git a/common/kvstore/src/main/java/org/apache/spark/kvstore/KVStoreSerializer.java b/common/kvstore/src/main/java/org/apache/spark/kvstore/KVStoreSerializer.java index d9f9e2646cc14..b84ec91cf67a0 100644 --- a/common/kvstore/src/main/java/org/apache/spark/kvstore/KVStoreSerializer.java +++ b/common/kvstore/src/main/java/org/apache/spark/kvstore/KVStoreSerializer.java @@ -17,6 +17,10 @@ package org.apache.spark.kvstore; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.util.zip.GZIPInputStream; +import java.util.zip.GZIPOutputStream; import static java.nio.charset.StandardCharsets.UTF_8; import com.fasterxml.jackson.databind.ObjectMapper; @@ -46,7 +50,14 @@ public final byte[] serialize(Object o) throws Exception { if (o instanceof String) { return ((String) o).getBytes(UTF_8); } else { - return mapper.writeValueAsBytes(o); + ByteArrayOutputStream bytes = new ByteArrayOutputStream(); + GZIPOutputStream out = new GZIPOutputStream(bytes); + try { + mapper.writeValue(out, o); + } finally { + out.close(); + } + return bytes.toByteArray(); } } @@ -55,7 +66,12 @@ public final T deserialize(byte[] data, Class klass) throws Exception { if (klass.equals(String.class)) { return (T) new String(data, UTF_8); } else { - return mapper.readValue(data, klass); + GZIPInputStream in = new GZIPInputStream(new ByteArrayInputStream(data)); + try { + return mapper.readValue(in, klass); + } finally { + in.close(); + } } } From 889963f2ffbcb628f9e53e7142fd37931ba09a54 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Fri, 24 Mar 2017 18:24:58 -0700 Subject: [PATCH 07/18] SHS-NG M1: Use type aliases as keys in Level DB. The type name gets repeated a lot in the store, so using it as the prefix for every key causes disk usage to grow unnecessarily. Instead, create a short alias for the type and keep a mapping of aliases to known types in a map in memory; the map is also saved to the database so it can be read later. --- .../org/apache/spark/kvstore/LevelDB.java | 62 ++++++++++++++++--- .../apache/spark/kvstore/LevelDBTypeInfo.java | 18 +----- .../spark/kvstore/LevelDBTypeInfoSuite.java | 2 +- 3 files changed, 56 insertions(+), 26 deletions(-) diff --git a/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDB.java b/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDB.java index 35cdbb6733a39..e423d71a335e9 100644 --- a/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDB.java +++ b/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDB.java @@ -19,7 +19,9 @@ import java.io.File; import java.io.IOException; +import java.util.HashMap; import java.util.Iterator; +import java.util.Map; import java.util.NoSuchElementException; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; @@ -49,16 +51,20 @@ public class LevelDB implements KVStore { /** DB key where app metadata is stored. */ private static final byte[] METADATA_KEY = "__meta__".getBytes(UTF_8); + /** DB key where type aliases are stored. */ + private static final byte[] TYPE_ALIASES_KEY = "__types__".getBytes(UTF_8); + final AtomicReference _db; final KVStoreSerializer serializer; + private final ConcurrentMap typeAliases; private final ConcurrentMap, LevelDBTypeInfo> types; - public LevelDB(File path) throws IOException { + public LevelDB(File path) throws Exception { this(path, new KVStoreSerializer()); } - public LevelDB(File path, KVStoreSerializer serializer) throws IOException { + public LevelDB(File path, KVStoreSerializer serializer) throws Exception { this.serializer = serializer; this.types = new ConcurrentHashMap<>(); @@ -75,6 +81,14 @@ public LevelDB(File path, KVStoreSerializer serializer) throws IOException { } else { db().put(STORE_VERSION_KEY, serializer.serialize(STORE_VERSION)); } + + Map aliases; + try { + aliases = get(TYPE_ALIASES_KEY, TypeAliases.class).aliases; + } catch (NoSuchElementException e) { + aliases = new HashMap<>(); + } + typeAliases = new ConcurrentHashMap<>(aliases); } @Override @@ -218,15 +232,15 @@ public void close() throws IOException { /** Returns metadata about indices for the given type. */ LevelDBTypeInfo getTypeInfo(Class type) throws Exception { - LevelDBTypeInfo idx = types.get(type); - if (idx == null) { - LevelDBTypeInfo tmp = new LevelDBTypeInfo<>(this, type); - idx = types.putIfAbsent(type, tmp); - if (idx == null) { - idx = tmp; + LevelDBTypeInfo ti = types.get(type); + if (ti == null) { + LevelDBTypeInfo tmp = new LevelDBTypeInfo<>(this, type, getTypeAlias(type)); + ti = types.putIfAbsent(type, tmp); + if (ti == null) { + ti = tmp; } } - return idx; + return ti; } /** @@ -249,4 +263,34 @@ private void removeInstance(LevelDBTypeInfo ti, LevelDBWriteBatch batch, Obje } } + private byte[] getTypeAlias(Class klass) throws Exception { + byte[] alias = typeAliases.get(klass.getName()); + if (alias == null) { + synchronized (typeAliases) { + byte[] tmp = String.valueOf(typeAliases.size()).getBytes(UTF_8); + alias = typeAliases.putIfAbsent(klass.getName(), tmp); + if (alias == null) { + alias = tmp; + put(TYPE_ALIASES_KEY, new TypeAliases(typeAliases)); + } + } + } + return alias; + } + + /** Needs to be public for Jackson. */ + public static class TypeAliases { + + public Map aliases; + + TypeAliases(Map aliases) { + this.aliases = aliases; + } + + TypeAliases() { + this(null); + } + + } + } diff --git a/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBTypeInfo.java b/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBTypeInfo.java index e0f3dc80cfe62..b9bb1959f5ae0 100644 --- a/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBTypeInfo.java +++ b/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBTypeInfo.java @@ -62,7 +62,7 @@ class LevelDBTypeInfo { private final Map indices; private final byte[] typePrefix; - LevelDBTypeInfo(LevelDB db, Class type) throws Exception { + LevelDBTypeInfo(LevelDB db, Class type, byte[] alias) throws Exception { this.db = db; this.type = type; this.indices = new HashMap<>(); @@ -88,21 +88,7 @@ class LevelDBTypeInfo { ByteArrayOutputStream typePrefix = new ByteArrayOutputStream(); typePrefix.write(utf8(ENTRY_PREFIX)); - - // Change fully-qualified class names to make keys more spread out by placing the - // class name first, and the package name afterwards. - String[] components = type.getName().split("\\."); - typePrefix.write(utf8(components[components.length - 1])); - if (components.length > 1) { - typePrefix.write(utf8("/")); - } - for (int i = 0; i < components.length - 1; i++) { - typePrefix.write(utf8(components[i])); - if (i < components.length - 2) { - typePrefix.write(utf8(".")); - } - } - typePrefix.write(KEY_SEPARATOR); + typePrefix.write(alias); this.typePrefix = typePrefix.toByteArray(); } diff --git a/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBTypeInfoSuite.java b/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBTypeInfoSuite.java index 7ee0b24552219..4cddab1acac08 100644 --- a/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBTypeInfoSuite.java +++ b/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBTypeInfoSuite.java @@ -136,7 +136,7 @@ public void testArrayIndices() throws Exception { } private LevelDBTypeInfo newTypeInfo(Class type) throws Exception { - return new LevelDBTypeInfo<>(null, type); + return new LevelDBTypeInfo<>(null, type, type.getName().getBytes(UTF_8)); } private void assertBefore(byte[] key1, byte[] key2) { From 84ab160699ef8dad4df1fa4cbba29deec7c92c06 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Mon, 3 Apr 2017 11:35:50 -0700 Subject: [PATCH 08/18] SHS-NG M1: Separate index introspection from storage. The new KVTypeInfo class can help with writing different implementations of KVStore without duplicating logic from LevelDBTypeInfo. --- .../org/apache/spark/kvstore/KVTypeInfo.java | 135 ++++++++++++++++++ .../org/apache/spark/kvstore/LevelDB.java | 18 +-- .../apache/spark/kvstore/LevelDBIterator.java | 4 +- .../apache/spark/kvstore/LevelDBTypeInfo.java | 87 ++--------- .../apache/spark/kvstore/LevelDBSuite.java | 2 +- .../spark/kvstore/LevelDBTypeInfoSuite.java | 22 +-- 6 files changed, 168 insertions(+), 100 deletions(-) create mode 100644 common/kvstore/src/main/java/org/apache/spark/kvstore/KVTypeInfo.java diff --git a/common/kvstore/src/main/java/org/apache/spark/kvstore/KVTypeInfo.java b/common/kvstore/src/main/java/org/apache/spark/kvstore/KVTypeInfo.java new file mode 100644 index 0000000000000..1a0bee958d482 --- /dev/null +++ b/common/kvstore/src/main/java/org/apache/spark/kvstore/KVTypeInfo.java @@ -0,0 +1,135 @@ +/* + * 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.kvstore; + +import java.lang.reflect.Field; +import java.lang.reflect.Method; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.Map; +import java.util.stream.Stream; + +import com.google.common.base.Preconditions; + +/** + * Wrapper around types managed in a KVStore, providing easy access to their indexed fields. + */ +public class KVTypeInfo { + + private final Class type; + private final Collection indices; + private final Map accessors; + + public KVTypeInfo(Class type) throws Exception { + this.type = type; + this.indices = new ArrayList<>(); + this.accessors = new HashMap<>(); + + for (Field f : type.getFields()) { + KVIndex idx = f.getAnnotation(KVIndex.class); + if (idx != null) { + checkIndex(idx); + indices.add(idx); + accessors.put(idx.value(), new FieldAccessor(f)); + } + } + + for (Method m : type.getMethods()) { + KVIndex idx = m.getAnnotation(KVIndex.class); + if (idx != null) { + checkIndex(idx); + Preconditions.checkArgument(m.getParameterTypes().length == 0, + "Annotated method %s::%s should not have any parameters.", type.getName(), m.getName()); + indices.add(idx); + accessors.put(idx.value(), new MethodAccessor(m)); + } + } + + Preconditions.checkArgument(accessors.containsKey(KVIndex.NATURAL_INDEX_NAME), + "No natural index defined for type %s.", type.getName()); + } + + private void checkIndex(KVIndex idx) { + Preconditions.checkArgument(idx.value() != null && !idx.value().isEmpty(), + "No name provided for index in type %s.", type.getName()); + Preconditions.checkArgument( + !idx.value().startsWith("_") || idx.value().equals(KVIndex.NATURAL_INDEX_NAME), + "Index name %s (in type %s) is not allowed.", idx.value(), type.getName()); + Preconditions.checkArgument(!indices.contains(idx.value()), + "Duplicate index %s for type %s.", idx.value(), type.getName()); + } + + public Class getType() { + return type; + } + + public Object getIndexValue(String indexName, Object instance) throws Exception { + return getAccessor(indexName).get(instance); + } + + public Stream indices() { + return indices.stream(); + } + + Accessor getAccessor(String indexName) { + Accessor a = accessors.get(indexName); + Preconditions.checkArgument(a != null, "No index %s.", indexName); + return a; + } + + /** + * Abstracts the difference between invoking a Field and a Method. + */ + interface Accessor { + + Object get(Object instance) throws Exception; + + } + + private class FieldAccessor implements Accessor { + + private final Field field; + + FieldAccessor(Field field) { + this.field = field; + } + + @Override + public Object get(Object instance) throws Exception { + return field.get(instance); + } + + } + + private class MethodAccessor implements Accessor { + + private final Method method; + + MethodAccessor(Method method) { + this.method = method; + } + + @Override + public Object get(Object instance) throws Exception { + return method.invoke(instance); + } + + } + +} diff --git a/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDB.java b/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDB.java index e423d71a335e9..337b9541e2879 100644 --- a/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDB.java +++ b/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDB.java @@ -148,7 +148,7 @@ public void write(Object value) throws Exception { public void write(Object value, boolean sync) throws Exception { Preconditions.checkArgument(value != null, "Null values are not allowed."); - LevelDBTypeInfo ti = getTypeInfo(value.getClass()); + LevelDBTypeInfo ti = getTypeInfo(value.getClass()); LevelDBWriteBatch batch = new LevelDBWriteBatch(this); try { @@ -160,7 +160,7 @@ public void write(Object value, boolean sync) throws Exception { } catch (NoSuchElementException e) { // Ignore. No previous value. } - for (LevelDBTypeInfo.Index idx : ti.indices()) { + for (LevelDBTypeInfo.Index idx : ti.indices()) { idx.add(batch, value, data); } batch.write(sync); @@ -179,7 +179,7 @@ public void delete(Class type, Object naturalKey, boolean sync) throws Except Preconditions.checkArgument(naturalKey != null, "Null keys are not allowed."); LevelDBWriteBatch batch = new LevelDBWriteBatch(this); try { - LevelDBTypeInfo ti = getTypeInfo(type); + LevelDBTypeInfo ti = getTypeInfo(type); byte[] key = ti.naturalIndex().start(naturalKey); byte[] data = db().get(key); if (data != null) { @@ -210,7 +210,7 @@ public Iterator iterator() { @Override public long count(Class type) throws Exception { - LevelDBTypeInfo.Index idx = getTypeInfo(type).naturalIndex(); + LevelDBTypeInfo.Index idx = getTypeInfo(type).naturalIndex(); return idx.getCount(idx.end()); } @@ -231,10 +231,10 @@ public void close() throws IOException { } /** Returns metadata about indices for the given type. */ - LevelDBTypeInfo getTypeInfo(Class type) throws Exception { - LevelDBTypeInfo ti = types.get(type); + LevelDBTypeInfo getTypeInfo(Class type) throws Exception { + LevelDBTypeInfo ti = types.get(type); if (ti == null) { - LevelDBTypeInfo tmp = new LevelDBTypeInfo<>(this, type, getTypeAlias(type)); + LevelDBTypeInfo tmp = new LevelDBTypeInfo(this, type, getTypeAlias(type)); ti = types.putIfAbsent(type, tmp); if (ti == null) { ti = tmp; @@ -256,9 +256,9 @@ DB db() { return _db; } - private void removeInstance(LevelDBTypeInfo ti, LevelDBWriteBatch batch, Object instance) + private void removeInstance(LevelDBTypeInfo ti, LevelDBWriteBatch batch, Object instance) throws Exception { - for (LevelDBTypeInfo.Index idx : ti.indices()) { + for (LevelDBTypeInfo.Index idx : ti.indices()) { idx.remove(batch, instance); } } diff --git a/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBIterator.java b/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBIterator.java index b777ff7bafc02..3b00c171740db 100644 --- a/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBIterator.java +++ b/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBIterator.java @@ -34,8 +34,8 @@ class LevelDBIterator implements KVStoreIterator { private final boolean ascending; private final DBIterator it; private final Class type; - private final LevelDBTypeInfo ti; - private final LevelDBTypeInfo.Index index; + private final LevelDBTypeInfo ti; + private final LevelDBTypeInfo.Index index; private final byte[] indexKeyPrefix; private final byte[] end; diff --git a/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBTypeInfo.java b/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBTypeInfo.java index b9bb1959f5ae0..826e6cf068fd9 100644 --- a/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBTypeInfo.java +++ b/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBTypeInfo.java @@ -35,7 +35,7 @@ * Holds metadata about app-specific types stored in LevelDB. Serves as a cache for data collected * via reflection, to make it cheaper to access it multiple times. */ -class LevelDBTypeInfo { +class LevelDBTypeInfo { static final String ENTRY_PREFIX = "+"; static final String END_MARKER = "-"; @@ -58,33 +58,19 @@ class LevelDBTypeInfo { static final int SHORT_ENCODED_LEN = String.valueOf(Short.MAX_VALUE).length() + 1; private final LevelDB db; - private final Class type; + private final Class type; private final Map indices; private final byte[] typePrefix; - LevelDBTypeInfo(LevelDB db, Class type, byte[] alias) throws Exception { + LevelDBTypeInfo(LevelDB db, Class type, byte[] alias) throws Exception { this.db = db; this.type = type; this.indices = new HashMap<>(); - for (Field f : type.getFields()) { - KVIndex idx = f.getAnnotation(KVIndex.class); - if (idx != null) { - register(idx, new FieldAccessor(f)); - } - } - - for (Method m : type.getMethods()) { - KVIndex idx = m.getAnnotation(KVIndex.class); - if (idx != null) { - Preconditions.checkArgument(m.getParameterTypes().length == 0, - "Annotated method %s::%s should not have any parameters.", type.getName(), m.getName()); - register(idx, new MethodAccessor(m)); - } - } - - Preconditions.checkArgument(indices.get(KVIndex.NATURAL_INDEX_NAME) != null, - "No natural index defined for type %s.", type.getName()); + KVTypeInfo ti = new KVTypeInfo(type); + ti.indices().forEach(idx -> { + indices.put(idx.value(), new Index(idx.value(), idx.copy(), ti.getAccessor(idx.value()))); + }); ByteArrayOutputStream typePrefix = new ByteArrayOutputStream(); typePrefix.write(utf8(ENTRY_PREFIX)); @@ -92,18 +78,7 @@ class LevelDBTypeInfo { this.typePrefix = typePrefix.toByteArray(); } - private void register(KVIndex idx, Accessor accessor) { - Preconditions.checkArgument(idx.value() != null && !idx.value().isEmpty(), - "No name provided for index in type %s.", type.getName()); - Preconditions.checkArgument( - !idx.value().startsWith("_") || idx.value().equals(KVIndex.NATURAL_INDEX_NAME), - "Index name %s (in type %s) is not allowed.", idx.value(), type.getName()); - Preconditions.checkArgument(indices.get(idx.value()) == null, - "Duplicate index %s for type %s.", idx.value(), type.getName()); - indices.put(idx.value(), new Index(idx.value(), idx.copy(), accessor)); - } - - Class type() { + Class type() { return type; } @@ -164,11 +139,9 @@ class Index { private final boolean copy; private final boolean isNatural; private final String name; + private final KVTypeInfo.Accessor accessor; - @VisibleForTesting - final Accessor accessor; - - private Index(String name, boolean copy, Accessor accessor) { + private Index(String name, boolean copy, KVTypeInfo.Accessor accessor) { this.name = name; this.isNatural = name.equals(KVIndex.NATURAL_INDEX_NAME); this.copy = isNatural || copy; @@ -320,44 +293,4 @@ String toKey(Object value) { } - /** - * Abstracts the difference between invoking a Field and a Method. - */ - @VisibleForTesting - interface Accessor { - - Object get(Object instance) throws Exception; - - } - - private class FieldAccessor implements Accessor { - - private final Field field; - - FieldAccessor(Field field) { - this.field = field; - } - - @Override - public Object get(Object instance) throws Exception { - return field.get(instance); - } - - } - - private class MethodAccessor implements Accessor { - - private final Method method; - - MethodAccessor(Method method) { - this.method = method; - } - - @Override - public Object get(Object instance) throws Exception { - return method.invoke(instance); - } - - } - } diff --git a/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBSuite.java b/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBSuite.java index 1d33ba099f4f8..c3baf76589286 100644 --- a/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBSuite.java +++ b/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBSuite.java @@ -221,7 +221,7 @@ public void testMetadata() throws Exception { } private long countIndexEntries(Class type, String index, Object value) throws Exception { - LevelDBTypeInfo.Index idx = db.getTypeInfo(type).index(index); + LevelDBTypeInfo.Index idx = db.getTypeInfo(type).index(index); return idx.getCount(idx.end()); } diff --git a/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBTypeInfoSuite.java b/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBTypeInfoSuite.java index 4cddab1acac08..cf69f32dfb354 100644 --- a/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBTypeInfoSuite.java +++ b/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBTypeInfoSuite.java @@ -26,8 +26,8 @@ public class LevelDBTypeInfoSuite { @Test public void testIndexAnnotation() throws Exception { - LevelDBTypeInfo ti = newTypeInfo(CustomType1.class); - assertEquals(4, ti.indices().size()); + KVTypeInfo ti = new KVTypeInfo(CustomType1.class); + assertEquals(4, ti.indices().count()); CustomType1 t1 = new CustomType1(); t1.key = "key"; @@ -35,10 +35,10 @@ public void testIndexAnnotation() throws Exception { t1.name = "name"; t1.num = 42; - assertEquals(t1.key, ti.naturalIndex().accessor.get(t1)); - assertEquals(t1.id, ti.index("id").accessor.get(t1)); - assertEquals(t1.name, ti.index("name").accessor.get(t1)); - assertEquals(t1.num, ti.index("int").accessor.get(t1)); + assertEquals(t1.key, ti.getIndexValue(KVIndex.NATURAL_INDEX_NAME, t1)); + assertEquals(t1.id, ti.getIndexValue("id", t1)); + assertEquals(t1.name, ti.getIndexValue("name", t1)); + assertEquals(t1.num, ti.getIndexValue("int", t1)); } @Test(expected = IllegalArgumentException.class) @@ -68,7 +68,7 @@ public void testIllegalIndexMethod() throws Exception { @Test public void testKeyClashes() throws Exception { - LevelDBTypeInfo ti = newTypeInfo(CustomType1.class); + LevelDBTypeInfo ti = newTypeInfo(CustomType1.class); CustomType1 t1 = new CustomType1(); t1.key = "key1"; @@ -90,7 +90,7 @@ public void testKeyClashes() throws Exception { @Test public void testNumEncoding() throws Exception { - LevelDBTypeInfo.Index idx = newTypeInfo(CustomType1.class).indices().iterator().next(); + LevelDBTypeInfo.Index idx = newTypeInfo(CustomType1.class).indices().iterator().next(); assertBefore(idx.toKey(1), idx.toKey(2)); assertBefore(idx.toKey(-1), idx.toKey(2)); @@ -125,7 +125,7 @@ public void testNumEncoding() throws Exception { @Test public void testArrayIndices() throws Exception { - LevelDBTypeInfo.Index idx = newTypeInfo(CustomType1.class).indices().iterator().next(); + LevelDBTypeInfo.Index idx = newTypeInfo(CustomType1.class).indices().iterator().next(); assertBefore(idx.toKey(new String[] { "str1" }), idx.toKey(new String[] { "str2" })); assertBefore(idx.toKey(new String[] { "str1", "str2" }), @@ -135,8 +135,8 @@ public void testArrayIndices() throws Exception { assertBefore(idx.toKey(new int[] { 1, 2 }), idx.toKey(new int[] { 1, 3 })); } - private LevelDBTypeInfo newTypeInfo(Class type) throws Exception { - return new LevelDBTypeInfo<>(null, type, type.getName().getBytes(UTF_8)); + private LevelDBTypeInfo newTypeInfo(Class type) throws Exception { + return new LevelDBTypeInfo(null, type, type.getName().getBytes(UTF_8)); } private void assertBefore(byte[] key1, byte[] key2) { From 7b870212e80e70b8c3f3eb4279e3bb9ec0125d2d Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Wed, 26 Apr 2017 11:54:33 -0700 Subject: [PATCH 09/18] SHS-NG M1: Remove unused methods from KVStore. Turns out I ended up not using the raw storage methods in KVStore, so this change removes them to simplify the API and save some code. --- .../org/apache/spark/kvstore/KVStore.java | 44 +++++-------------- .../org/apache/spark/kvstore/LevelDB.java | 16 +------ .../apache/spark/kvstore/LevelDBIterator.java | 18 -------- .../apache/spark/kvstore/LevelDBSuite.java | 43 ------------------ 4 files changed, 14 insertions(+), 107 deletions(-) diff --git a/common/kvstore/src/main/java/org/apache/spark/kvstore/KVStore.java b/common/kvstore/src/main/java/org/apache/spark/kvstore/KVStore.java index 31d4e6fefc289..667fccccd5428 100644 --- a/common/kvstore/src/main/java/org/apache/spark/kvstore/KVStore.java +++ b/common/kvstore/src/main/java/org/apache/spark/kvstore/KVStore.java @@ -25,21 +25,21 @@ * Abstraction for a local key/value store for storing app data. * *

- * Use {@link KVStoreBuilder} to create an instance. There are two main features provided by the - * implementations of this interface: + * There are two main features provided by the implementations of this interface: *

* - *
    - *
  • serialization: this feature is not optional; data will be serialized to and deserialized - * from the underlying data store using a {@link KVStoreSerializer}, which can be customized by - * the application. The serializer is based on Jackson, so it supports all the Jackson annotations - * for controlling the serialization of app-defined types.
  • + *

    Serialization

    * - *
  • key management: by using {@link #read(Class, Object)} and {@link #write(Class, Object)}, - * applications can leave key management to the implementation. For applications that want to - * manage their own keys, the {@link #get(byte[], Class)} and {@link #set(byte[], Object)} methods - * are available.
  • - *
+ *

+ * Data will be serialized to and deserialized from the underlying data store using a + * {@link KVStoreSerializer}, which can be customized by the application. The serializer is + * based on Jackson, so it supports all the Jackson annotations for controlling the serialization + * of app-defined types. + *

+ * + *

+ * Data is also automatically compressed to save disk space. + *

* *

Automatic Key Management

* @@ -78,26 +78,6 @@ public interface KVStore extends Closeable { */ void setMetadata(Object value) throws Exception; - /** - * Returns the value of a specific key, deserialized to the given type. - */ - T get(byte[] key, Class klass) throws Exception; - - /** - * Write a single key directly to the store, atomically. - */ - void put(byte[] key, Object value) throws Exception; - - /** - * Removes a key from the store. - */ - void delete(byte[] key) throws Exception; - - /** - * Returns an iterator that will only list values with keys starting with the given prefix. - */ - KVStoreIterator iterator(byte[] prefix, Class klass) throws Exception; - /** * Read a specific instance of an object. */ diff --git a/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDB.java b/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDB.java index 337b9541e2879..b40c7950d1d11 100644 --- a/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDB.java +++ b/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDB.java @@ -109,8 +109,7 @@ public void setMetadata(Object value) throws Exception { } } - @Override - public T get(byte[] key, Class klass) throws Exception { + T get(byte[] key, Class klass) throws Exception { byte[] data = db().get(key); if (data == null) { throw new NoSuchElementException(new String(key, UTF_8)); @@ -118,22 +117,11 @@ public T get(byte[] key, Class klass) throws Exception { return serializer.deserialize(data, klass); } - @Override - public void put(byte[] key, Object value) throws Exception { + private void put(byte[] key, Object value) throws Exception { Preconditions.checkArgument(value != null, "Null values are not allowed."); db().put(key, serializer.serialize(value)); } - @Override - public void delete(byte[] key) throws Exception { - db().delete(key); - } - - @Override - public KVStoreIterator iterator(byte[] prefix, Class klass) throws Exception { - throw new UnsupportedOperationException(); - } - @Override public T read(Class klass, Object naturalKey) throws Exception { Preconditions.checkArgument(naturalKey != null, "Null keys are not allowed."); diff --git a/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBIterator.java b/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBIterator.java index 3b00c171740db..f65152a9fc36a 100644 --- a/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBIterator.java +++ b/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBIterator.java @@ -43,24 +43,6 @@ class LevelDBIterator implements KVStoreIterator { private T next; private boolean closed; - /** - * Creates a simple iterator over db keys. - */ - LevelDBIterator(LevelDB db, byte[] keyPrefix, Class type) throws Exception { - this.db = db; - this.ascending = true; - this.type = type; - this.ti = null; - this.index = null; - this.it = db.db().iterator(); - this.indexKeyPrefix = keyPrefix; - this.end = null; - it.seek(keyPrefix); - } - - /** - * Creates an iterator for indexed types (i.e., those whose keys are managed by the library). - */ LevelDBIterator(LevelDB db, KVStoreView params) throws Exception { this.db = db; this.ascending = params.ascending; diff --git a/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBSuite.java b/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBSuite.java index c3baf76589286..1f88aae0be2aa 100644 --- a/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBSuite.java +++ b/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBSuite.java @@ -74,29 +74,6 @@ public void testReopenAndVersionCheckDb() throws Exception { } } - @Test - public void testStringWriteReadDelete() throws Exception { - String string = "testString"; - byte[] key = string.getBytes(UTF_8); - testReadWriteDelete(key, string); - } - - @Test - public void testIntWriteReadDelete() throws Exception { - int value = 42; - byte[] key = "key".getBytes(UTF_8); - testReadWriteDelete(key, value); - } - - @Test - public void testSimpleTypeWriteReadDelete() throws Exception { - byte[] key = "testKey".getBytes(UTF_8); - CustomType1 t = new CustomType1(); - t.id = "id"; - t.name = "name"; - testReadWriteDelete(key, t); - } - @Test public void testObjectWriteReadDelete() throws Exception { CustomType1 t = new CustomType1(); @@ -268,26 +245,6 @@ private int countKeys(Class type) throws Exception { return count; } - private void testReadWriteDelete(byte[] key, T value) throws Exception { - try { - db.get(key, value.getClass()); - fail("Expected exception for non-existent key."); - } catch (NoSuchElementException nsee) { - // Expected. - } - - db.put(key, value); - assertEquals(value, db.get(key, value.getClass())); - - db.delete(key); - try { - db.get(key, value.getClass()); - fail("Expected exception for deleted key."); - } catch (NoSuchElementException nsee) { - // Expected. - } - } - public static class IntKeyType { @KVIndex From 5197c218525db2ad849dfe77d83dddf2311bb5ad Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Fri, 5 May 2017 14:36:00 -0700 Subject: [PATCH 10/18] SHS-NG M1: Add "max" and "last" to kvstore iterators. This makes it easier for callers to control the end of iteration, making it easier to write Scala code that automatically closes underlying iterator resources. Before, code had to use Scala's "takeWhile", convert the result to a list, and manually close the iterators; with these two parameters, that can be avoided in a bunch of cases, with iterators auto-closing when the last element is reached. --- .../org/apache/spark/kvstore/KVStoreView.java | 22 ++- .../apache/spark/kvstore/LevelDBIterator.java | 50 ++++++- .../apache/spark/kvstore/DBIteratorSuite.java | 135 +++++++++++++----- 3 files changed, 167 insertions(+), 40 deletions(-) diff --git a/common/kvstore/src/main/java/org/apache/spark/kvstore/KVStoreView.java b/common/kvstore/src/main/java/org/apache/spark/kvstore/KVStoreView.java index a68c37942dee4..65edc0149b438 100644 --- a/common/kvstore/src/main/java/org/apache/spark/kvstore/KVStoreView.java +++ b/common/kvstore/src/main/java/org/apache/spark/kvstore/KVStoreView.java @@ -43,7 +43,9 @@ public abstract class KVStoreView implements Iterable { boolean ascending = true; String index = KVIndex.NATURAL_INDEX_NAME; Object first = null; + Object last = null; long skip = 0L; + long max = Long.MAX_VALUE; public KVStoreView(Class type) { this.type = type; @@ -74,7 +76,25 @@ public KVStoreView first(Object value) { } /** - * Skips a number of elements in the resulting iterator. + * Stops iteration at the given value of the chosen index. + */ + public KVStoreView last(Object value) { + this.last = value; + return this; + } + + /** + * Stops iteration after a number of elements has been retrieved. + */ + public KVStoreView max(long max) { + Preconditions.checkArgument(max > 0L, "max must be positive."); + this.max = max; + return this; + } + + /** + * Skips a number of elements at the start of iteration. Skipped elements are not accounted + * when using {@link #max(long)}. */ public KVStoreView skip(long n) { this.skip = n; diff --git a/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBIterator.java b/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBIterator.java index f65152a9fc36a..73ca8afc9eb28 100644 --- a/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBIterator.java +++ b/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBIterator.java @@ -38,10 +38,12 @@ class LevelDBIterator implements KVStoreIterator { private final LevelDBTypeInfo.Index index; private final byte[] indexKeyPrefix; private final byte[] end; + private final long max; private boolean checkedNext; private T next; private boolean closed; + private long count; LevelDBIterator(LevelDB db, KVStoreView params) throws Exception { this.db = db; @@ -51,6 +53,7 @@ class LevelDBIterator implements KVStoreIterator { this.ti = db.getTypeInfo(type); this.index = ti.index(params.index); this.indexKeyPrefix = index.keyPrefix(); + this.max = params.max; byte[] firstKey; if (params.first != null) { @@ -66,14 +69,27 @@ class LevelDBIterator implements KVStoreIterator { } it.seek(firstKey); + byte[] end = null; if (ascending) { - this.end = index.end(); + end = params.last != null ? index.end(params.last) : index.end(); } else { - this.end = null; + if (params.last != null) { + end = index.start(params.last); + } if (it.hasNext()) { - it.next(); + // When descending, the caller may have set up the start of iteration at a non-existant + // entry that is guaranteed to be after the desired entry. For example, if you have a + // compound key (a, b) where b is a, integer, you may seek to the end of the elements that + // have the same "a" value by specifying Integer.MAX_VALUE for "b", and that value may not + // exist in the database. So need to check here whether the next value actually belongs to + // the set being returned by the iterator before advancing. + byte[] nextKey = it.peekNext().getKey(); + if (compare(nextKey, indexKeyPrefix) <= 0) { + it.next(); + } } } + this.end = end; if (params.skip > 0) { skip(params.skip); @@ -147,6 +163,10 @@ public synchronized void close() throws IOException { } private T loadNext() { + if (count >= max) { + return null; + } + try { while (true) { boolean hasNext = ascending ? it.hasNext() : it.hasPrev(); @@ -173,11 +193,16 @@ private T loadNext() { return null; } - // If there's a known end key and it's found, stop. - if (end != null && Arrays.equals(nextKey, end)) { - return null; + // If there's a known end key and iteration has gone past it, stop. + if (end != null) { + int comp = compare(nextKey, end) * (ascending ? 1 : -1); + if (comp > 0) { + return null; + } } + count++; + // Next element is part of the iteration, return it. if (index == null || index.isCopy()) { return db.serializer.deserialize(nextEntry.getValue(), type); @@ -228,4 +253,17 @@ private byte[] stitch(byte[]... comps) { return dest; } + private int compare(byte[] a, byte[] b) { + int diff = 0; + int minLen = Math.min(a.length, b.length); + for (int i = 0; i < minLen; i++) { + diff += (a[i] - b[i]); + if (diff != 0) { + return diff; + } + } + + return a.length - b.length; + } + } diff --git a/common/kvstore/src/test/java/org/apache/spark/kvstore/DBIteratorSuite.java b/common/kvstore/src/test/java/org/apache/spark/kvstore/DBIteratorSuite.java index 88c7cc08984bb..6c4469e1ed5d0 100644 --- a/common/kvstore/src/test/java/org/apache/spark/kvstore/DBIteratorSuite.java +++ b/common/kvstore/src/test/java/org/apache/spark/kvstore/DBIteratorSuite.java @@ -152,111 +152,170 @@ public static void cleanup() throws Exception { @Test public void naturalIndex() throws Exception { - testIteration(NATURAL_ORDER, view(), null); + testIteration(NATURAL_ORDER, view(), null, null); } @Test public void refIndex() throws Exception { - testIteration(REF_INDEX_ORDER, view().index("id"), null); + testIteration(REF_INDEX_ORDER, view().index("id"), null, null); } @Test public void copyIndex() throws Exception { - testIteration(COPY_INDEX_ORDER, view().index("name"), null); + testIteration(COPY_INDEX_ORDER, view().index("name"), null, null); } @Test public void numericIndex() throws Exception { - testIteration(NUMERIC_INDEX_ORDER, view().index("int"), null); + testIteration(NUMERIC_INDEX_ORDER, view().index("int"), null, null); } @Test public void naturalIndexDescending() throws Exception { - testIteration(NATURAL_ORDER, view().reverse(), null); + testIteration(NATURAL_ORDER, view().reverse(), null, null); } @Test public void refIndexDescending() throws Exception { - testIteration(REF_INDEX_ORDER, view().index("id").reverse(), null); + testIteration(REF_INDEX_ORDER, view().index("id").reverse(), null, null); } @Test public void copyIndexDescending() throws Exception { - testIteration(COPY_INDEX_ORDER, view().index("name").reverse(), null); + testIteration(COPY_INDEX_ORDER, view().index("name").reverse(), null, null); } @Test public void numericIndexDescending() throws Exception { - testIteration(NUMERIC_INDEX_ORDER, view().index("int").reverse(), null); + testIteration(NUMERIC_INDEX_ORDER, view().index("int").reverse(), null, null); } @Test public void naturalIndexWithStart() throws Exception { - CustomType1 first = pickFirst(); - testIteration(NATURAL_ORDER, view().first(first.key), first); + CustomType1 first = pickLimit(); + testIteration(NATURAL_ORDER, view().first(first.key), first, null); } @Test public void refIndexWithStart() throws Exception { - CustomType1 first = pickFirst(); - testIteration(REF_INDEX_ORDER, view().index("id").first(first.id), first); + CustomType1 first = pickLimit(); + testIteration(REF_INDEX_ORDER, view().index("id").first(first.id), first, null); } @Test public void copyIndexWithStart() throws Exception { - CustomType1 first = pickFirst(); - testIteration(COPY_INDEX_ORDER, view().index("name").first(first.name), first); + CustomType1 first = pickLimit(); + testIteration(COPY_INDEX_ORDER, view().index("name").first(first.name), first, null); } @Test public void numericIndexWithStart() throws Exception { - CustomType1 first = pickFirst(); - testIteration(NUMERIC_INDEX_ORDER, view().index("int").first(first.num), first); + CustomType1 first = pickLimit(); + testIteration(NUMERIC_INDEX_ORDER, view().index("int").first(first.num), first, null); } @Test public void naturalIndexDescendingWithStart() throws Exception { - CustomType1 first = pickFirst(); - testIteration(NATURAL_ORDER, view().reverse().first(first.key), first); + CustomType1 first = pickLimit(); + testIteration(NATURAL_ORDER, view().reverse().first(first.key), first, null); } @Test public void refIndexDescendingWithStart() throws Exception { - CustomType1 first = pickFirst(); - testIteration(REF_INDEX_ORDER, view().reverse().index("id").first(first.id), first); + CustomType1 first = pickLimit(); + testIteration(REF_INDEX_ORDER, view().reverse().index("id").first(first.id), first, null); } @Test public void copyIndexDescendingWithStart() throws Exception { - CustomType1 first = pickFirst(); - testIteration(COPY_INDEX_ORDER, view().reverse().index("name").first(first.name), - first); + CustomType1 first = pickLimit(); + testIteration(COPY_INDEX_ORDER, view().reverse().index("name").first(first.name), first, null); } @Test public void numericIndexDescendingWithStart() throws Exception { - CustomType1 first = pickFirst(); - testIteration(NUMERIC_INDEX_ORDER, view().reverse().index("int").first(first.num), - first); + CustomType1 first = pickLimit(); + testIteration(NUMERIC_INDEX_ORDER, view().reverse().index("int").first(first.num), first, null); } @Test public void naturalIndexWithSkip() throws Exception { - testIteration(NATURAL_ORDER, view().skip(RND.nextInt(allEntries.size() / 2)), null); + testIteration(NATURAL_ORDER, view().skip(RND.nextInt(allEntries.size() / 2)), null, null); } @Test public void refIndexWithSkip() throws Exception { testIteration(REF_INDEX_ORDER, view().index("id").skip(RND.nextInt(allEntries.size() / 2)), - null); + null, null); } @Test public void copyIndexWithSkip() throws Exception { testIteration(COPY_INDEX_ORDER, view().index("name").skip(RND.nextInt(allEntries.size() / 2)), - null); + null, null); } + @Test + public void naturalIndexWithMax() throws Exception { + testIteration(NATURAL_ORDER, view().max(RND.nextInt(allEntries.size() / 2)), null, null); + } + + @Test + public void copyIndexWithMax() throws Exception { + testIteration(COPY_INDEX_ORDER, view().index("name").max(RND.nextInt(allEntries.size() / 2)), + null, null); + } + + @Test + public void naturalIndexWithLast() throws Exception { + CustomType1 last = pickLimit(); + testIteration(NATURAL_ORDER, view().last(last.key), null, last); + } + + @Test + public void refIndexWithLast() throws Exception { + CustomType1 last = pickLimit(); + testIteration(REF_INDEX_ORDER, view().index("id").last(last.id), null, last); + } + + @Test + public void copyIndexWithLast() throws Exception { + CustomType1 last = pickLimit(); + testIteration(COPY_INDEX_ORDER, view().index("name").last(last.name), null, last); + } + + @Test + public void numericIndexWithLast() throws Exception { + CustomType1 last = pickLimit(); + testIteration(NUMERIC_INDEX_ORDER, view().index("int").last(last.num), null, last); + } + + @Test + public void naturalIndexDescendingWithLast() throws Exception { + CustomType1 last = pickLimit(); + testIteration(NATURAL_ORDER, view().reverse().last(last.key), null, last); + } + + @Test + public void refIndexDescendingWithLast() throws Exception { + CustomType1 last = pickLimit(); + testIteration(REF_INDEX_ORDER, view().reverse().index("id").last(last.id), null, last); + } + + @Test + public void copyIndexDescendingWithLast() throws Exception { + CustomType1 last = pickLimit(); + testIteration(COPY_INDEX_ORDER, view().reverse().index("name").last(last.name), + null, last); + } + + @Test + public void numericIndexDescendingWithLast() throws Exception { + CustomType1 last = pickLimit(); + testIteration(NUMERIC_INDEX_ORDER, view().reverse().index("int").last(last.num), + null, last); + } + @Test public void testRefWithIntNaturalKey() throws Exception { LevelDBSuite.IntKeyType i = new LevelDBSuite.IntKeyType(); @@ -272,8 +331,8 @@ public void testRefWithIntNaturalKey() throws Exception { } } - private CustomType1 pickFirst() { - // Picks a first element that has clashes with other elements in the given index. + private CustomType1 pickLimit() { + // Picks an element that has clashes with other elements in the given index. return clashingEntries.get(RND.nextInt(clashingEntries.size())); } @@ -297,22 +356,32 @@ private > int compareWithFallback( private void testIteration( final BaseComparator order, final KVStoreView params, - final CustomType1 first) throws Exception { + final CustomType1 first, + final CustomType1 last) throws Exception { List indexOrder = sortBy(order.fallback()); if (!params.ascending) { indexOrder = Lists.reverse(indexOrder); } Iterable expected = indexOrder; + BaseComparator expectedOrder = params.ascending ? order : order.reverse(); + if (first != null) { - final BaseComparator expectedOrder = params.ascending ? order : order.reverse(); expected = Iterables.filter(expected, v -> expectedOrder.compare(first, v) <= 0); } + if (last != null) { + expected = Iterables.filter(expected, v -> expectedOrder.compare(v, last) <= 0); + } + if (params.skip > 0) { expected = Iterables.skip(expected, (int) params.skip); } + if (params.max != Long.MAX_VALUE) { + expected = Iterables.limit(expected, (int) params.max); + } + List actual = collect(params); compareLists(expected, actual); } From 63e0a58b01bd622d6a3f2dc8fbe72c819493c152 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Mon, 8 May 2017 18:07:46 -0700 Subject: [PATCH 11/18] SHS-NG M1: Add missing test tags dependency. --- common/kvstore/pom.xml | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/common/kvstore/pom.xml b/common/kvstore/pom.xml index ab296c5b5fb9b..d00cf2788b964 100644 --- a/common/kvstore/pom.xml +++ b/common/kvstore/pom.xml @@ -81,6 +81,17 @@ metrics-core test
+ + + + org.apache.spark + spark-tags_${scala.binary.version} + test-jar + test + From e565eb738601118a4eef3957fac34ad19bcd6fd8 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Tue, 9 May 2017 16:32:26 -0700 Subject: [PATCH 12/18] SHS-NG M1: Add child (or secondary) indices. This change adds child indices, which allow sorting a result set according to two indexed fields. To use a concrete example, with this change you can efficiently retrieve a list of tasks that belong to a particular stage attempt (the primary index), sorted by the time they were launched (the secondary index). I also added a more verbose description of what the LevelDB structure looks like in LevelDBTypeInfo.java, since that actually helped me visualize what was going on while adding the new code. I also made a bunch of other changes targeted at speeding up this code: - avoid String -> byte[] conversions and other expensive operations, especially when creating keys based on numeric values. - avoid calculating the same key prefix multiple times when writing. - do not store counts in end markers when not needed; the API only require some of those to be stored for efficient retrieval, and LevelDB doesn't require a key to exist before you can "seek" to it. --- .../org/apache/spark/kvstore/KVIndex.java | 12 + .../org/apache/spark/kvstore/KVStore.java | 5 + .../org/apache/spark/kvstore/KVStoreView.java | 15 + .../org/apache/spark/kvstore/KVTypeInfo.java | 35 +- .../org/apache/spark/kvstore/LevelDB.java | 88 ++-- .../apache/spark/kvstore/LevelDBIterator.java | 53 ++- .../apache/spark/kvstore/LevelDBTypeInfo.java | 385 ++++++++++++------ .../spark/kvstore/LevelDBWriteBatch.java | 5 +- .../org/apache/spark/kvstore/CustomType1.java | 3 + .../apache/spark/kvstore/DBIteratorSuite.java | 78 +++- .../apache/spark/kvstore/LevelDBSuite.java | 37 +- .../spark/kvstore/LevelDBTypeInfoSuite.java | 31 +- 12 files changed, 519 insertions(+), 228 deletions(-) diff --git a/common/kvstore/src/main/java/org/apache/spark/kvstore/KVIndex.java b/common/kvstore/src/main/java/org/apache/spark/kvstore/KVIndex.java index bf5e4a66e510f..8b8899023c938 100644 --- a/common/kvstore/src/main/java/org/apache/spark/kvstore/KVIndex.java +++ b/common/kvstore/src/main/java/org/apache/spark/kvstore/KVIndex.java @@ -60,6 +60,18 @@ */ String value() default NATURAL_INDEX_NAME; + /** + * The name of the parent index of this index. By default there is no parent index, so the + * generated data can be retrieved without having to provide a parent value. + * + *

+ * If a parent index is defined, iterating over the data using the index will require providing + * a single value for the parent index. This serves as a rudimentary way to provide relationships + * between entities in the store. + *

+ */ + String parent() default ""; + /** * Whether to copy the instance's data to the index, instead of just storing a pointer to the * data. The default behavior is to just store a reference; that saves disk space but is slower diff --git a/common/kvstore/src/main/java/org/apache/spark/kvstore/KVStore.java b/common/kvstore/src/main/java/org/apache/spark/kvstore/KVStore.java index 667fccccd5428..9ccf3924e6a69 100644 --- a/common/kvstore/src/main/java/org/apache/spark/kvstore/KVStore.java +++ b/common/kvstore/src/main/java/org/apache/spark/kvstore/KVStore.java @@ -114,4 +114,9 @@ public interface KVStore extends Closeable { */ long count(Class type) throws Exception; + /** + * Returns the number of items of the given type which match the given indexed value. + */ + long count(Class type, String index, Object indexedValue) throws Exception; + } diff --git a/common/kvstore/src/main/java/org/apache/spark/kvstore/KVStoreView.java b/common/kvstore/src/main/java/org/apache/spark/kvstore/KVStoreView.java index 65edc0149b438..cc5874e958873 100644 --- a/common/kvstore/src/main/java/org/apache/spark/kvstore/KVStoreView.java +++ b/common/kvstore/src/main/java/org/apache/spark/kvstore/KVStoreView.java @@ -44,6 +44,7 @@ public abstract class KVStoreView implements Iterable { String index = KVIndex.NATURAL_INDEX_NAME; Object first = null; Object last = null; + Object parent = null; long skip = 0L; long max = Long.MAX_VALUE; @@ -67,6 +68,20 @@ public KVStoreView index(String name) { return this; } + /** + * Defines the value of the parent index when iterating over a child index. Only elements that + * match the parent index's value will be included in the iteration. + * + *

+ * Required for iterating over child indices, will generate an error if iterating over a + * parent-less index. + *

+ */ + public KVStoreView parent(Object value) { + this.parent = value; + return this; + } + /** * Iterates starting at the given value of the chosen index. */ diff --git a/common/kvstore/src/main/java/org/apache/spark/kvstore/KVTypeInfo.java b/common/kvstore/src/main/java/org/apache/spark/kvstore/KVTypeInfo.java index 1a0bee958d482..83df72994c4c8 100644 --- a/common/kvstore/src/main/java/org/apache/spark/kvstore/KVTypeInfo.java +++ b/common/kvstore/src/main/java/org/apache/spark/kvstore/KVTypeInfo.java @@ -33,19 +33,19 @@ public class KVTypeInfo { private final Class type; - private final Collection indices; + private final Map indices; private final Map accessors; public KVTypeInfo(Class type) throws Exception { this.type = type; - this.indices = new ArrayList<>(); this.accessors = new HashMap<>(); + this.indices = new HashMap<>(); for (Field f : type.getFields()) { KVIndex idx = f.getAnnotation(KVIndex.class); if (idx != null) { - checkIndex(idx); - indices.add(idx); + checkIndex(idx, indices); + indices.put(idx.value(), idx); accessors.put(idx.value(), new FieldAccessor(f)); } } @@ -53,25 +53,37 @@ public KVTypeInfo(Class type) throws Exception { for (Method m : type.getMethods()) { KVIndex idx = m.getAnnotation(KVIndex.class); if (idx != null) { - checkIndex(idx); + checkIndex(idx, indices); Preconditions.checkArgument(m.getParameterTypes().length == 0, "Annotated method %s::%s should not have any parameters.", type.getName(), m.getName()); - indices.add(idx); + indices.put(idx.value(), idx); accessors.put(idx.value(), new MethodAccessor(m)); } } Preconditions.checkArgument(accessors.containsKey(KVIndex.NATURAL_INDEX_NAME), "No natural index defined for type %s.", type.getName()); + + for (KVIndex idx : indices.values()) { + if (!idx.parent().isEmpty()) { + KVIndex parent = indices.get(idx.parent()); + Preconditions.checkArgument(parent != null, + "Cannot find parent %s of index %s.", idx.parent(), idx.value()); + Preconditions.checkArgument(parent.parent().isEmpty(), + "Parent index %s of index %s cannot be itself a child index.", idx.parent(), idx.value()); + } + } } - private void checkIndex(KVIndex idx) { + private void checkIndex(KVIndex idx, Map indices) { Preconditions.checkArgument(idx.value() != null && !idx.value().isEmpty(), "No name provided for index in type %s.", type.getName()); Preconditions.checkArgument( !idx.value().startsWith("_") || idx.value().equals(KVIndex.NATURAL_INDEX_NAME), "Index name %s (in type %s) is not allowed.", idx.value(), type.getName()); - Preconditions.checkArgument(!indices.contains(idx.value()), + Preconditions.checkArgument(idx.parent().isEmpty() || !idx.parent().equals(idx.value()), + "Index %s cannot be parent of itself.", idx.value()); + Preconditions.checkArgument(!indices.containsKey(idx.value()), "Duplicate index %s for type %s.", idx.value(), type.getName()); } @@ -84,7 +96,7 @@ public Object getIndexValue(String indexName, Object instance) throws Exception } public Stream indices() { - return indices.stream(); + return indices.values().stream(); } Accessor getAccessor(String indexName) { @@ -93,6 +105,11 @@ Accessor getAccessor(String indexName) { return a; } + Accessor getParentAccessor(String indexName) { + KVIndex index = indices.get(indexName); + return index.parent().isEmpty() ? null : getAccessor(index.parent()); + } + /** * Abstracts the difference between invoking a Field and a Method. */ diff --git a/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDB.java b/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDB.java index b40c7950d1d11..3e56e56416606 100644 --- a/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDB.java +++ b/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDB.java @@ -29,6 +29,7 @@ import static java.nio.charset.StandardCharsets.UTF_8; import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Objects; import com.google.common.base.Preconditions; import com.google.common.base.Throwables; import org.fusesource.leveldbjni.JniDBFactory; @@ -125,7 +126,7 @@ private void put(byte[] key, Object value) throws Exception { @Override public T read(Class klass, Object naturalKey) throws Exception { Preconditions.checkArgument(naturalKey != null, "Null keys are not allowed."); - byte[] key = getTypeInfo(klass).naturalIndex().start(naturalKey); + byte[] key = getTypeInfo(klass).naturalIndex().start(null, naturalKey); return get(key, klass); } @@ -138,23 +139,36 @@ public void write(Object value, boolean sync) throws Exception { Preconditions.checkArgument(value != null, "Null values are not allowed."); LevelDBTypeInfo ti = getTypeInfo(value.getClass()); - LevelDBWriteBatch batch = new LevelDBWriteBatch(this); - try { + try (LevelDBWriteBatch batch = new LevelDBWriteBatch(this)) { byte[] data = serializer.serialize(value); synchronized (ti) { + Object existing; try { - Object existing = get(ti.naturalIndex().entityKey(value), value.getClass()); - removeInstance(ti, batch, existing); + existing = get(ti.naturalIndex().entityKey(null, value), value.getClass()); } catch (NoSuchElementException e) { - // Ignore. No previous value. + existing = null; } + + PrefixCache cache = new PrefixCache(); + for (LevelDBTypeInfo.Index idx : ti.indices()) { - idx.add(batch, value, data); + // Try to avoid unnecessary writes by only updating copy indices, or indices whose value + // has changed. + Object indexed = idx.getValue(value); + byte[] prefix = cache.getPrefix(idx, indexed); + if (existing == null) { + idx.add(batch, value, data, prefix); + } else { + if (idx.isCopy() || !Objects.equal(indexed, idx.getValue(existing))) { + byte[] existingPrefix = idx.isChild() ? idx.parent().childPrefix(existing, true) + : null; + idx.remove(batch, existing, existingPrefix); + idx.add(batch, value, data, prefix); + } + } } batch.write(sync); } - } finally { - batch.close(); } } @@ -165,20 +179,23 @@ public void delete(Class type, Object naturalKey) throws Exception { public void delete(Class type, Object naturalKey, boolean sync) throws Exception { Preconditions.checkArgument(naturalKey != null, "Null keys are not allowed."); - LevelDBWriteBatch batch = new LevelDBWriteBatch(this); - try { + try (LevelDBWriteBatch batch = new LevelDBWriteBatch(this)) { LevelDBTypeInfo ti = getTypeInfo(type); - byte[] key = ti.naturalIndex().start(naturalKey); - byte[] data = db().get(key); - if (data != null) { - Object existing = serializer.deserialize(data, type); - synchronized (ti) { - removeInstance(ti, batch, existing); + byte[] key = ti.naturalIndex().start(null, naturalKey); + PrefixCache cache = new PrefixCache(); + + synchronized (ti) { + byte[] data = db().get(key); + if (data != null) { + Object existing = serializer.deserialize(data, type); + for (LevelDBTypeInfo.Index idx : ti.indices()) { + idx.remove(batch, existing, cache.getPrefix(idx, idx.getValue(existing))); + } batch.write(sync); } } - } finally { - batch.close(); + } catch (NoSuchElementException nse) { + // Ignore. } } @@ -199,7 +216,13 @@ public Iterator iterator() { @Override public long count(Class type) throws Exception { LevelDBTypeInfo.Index idx = getTypeInfo(type).naturalIndex(); - return idx.getCount(idx.end()); + return idx.getCount(idx.end(null)); + } + + @Override + public long count(Class type, String index, Object indexedValue) throws Exception { + LevelDBTypeInfo.Index idx = getTypeInfo(type).index(index); + return idx.getCount(idx.end(null, indexedValue)); } @Override @@ -244,13 +267,6 @@ DB db() { return _db; } - private void removeInstance(LevelDBTypeInfo ti, LevelDBWriteBatch batch, Object instance) - throws Exception { - for (LevelDBTypeInfo.Index idx : ti.indices()) { - idx.remove(batch, instance); - } - } - private byte[] getTypeAlias(Class klass) throws Exception { byte[] alias = typeAliases.get(klass.getName()); if (alias == null) { @@ -281,4 +297,22 @@ public static class TypeAliases { } + private static class PrefixCache { + + private final Map prefixes = new HashMap<>(); + + byte[] getPrefix(LevelDBTypeInfo.Index idx, Object entity) throws Exception { + byte[] prefix = null; + if (idx.isChild()) { + prefix = prefixes.get(idx.parent()); + if (prefix == null) { + prefix = idx.parent().childPrefix(entity, true); + prefixes.put(idx.parent(), prefix); + } + } + return prefix; + } + + } + } diff --git a/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBIterator.java b/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBIterator.java index 73ca8afc9eb28..1467b03f76ac8 100644 --- a/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBIterator.java +++ b/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBIterator.java @@ -25,6 +25,7 @@ import java.util.NoSuchElementException; import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; import com.google.common.base.Throwables; import org.iq80.leveldb.DBIterator; @@ -52,29 +53,38 @@ class LevelDBIterator implements KVStoreIterator { this.type = params.type; this.ti = db.getTypeInfo(type); this.index = ti.index(params.index); - this.indexKeyPrefix = index.keyPrefix(); this.max = params.max; + Preconditions.checkArgument(!index.isChild() || params.parent != null, + "Cannot iterate over child index %s without parent value.", params.index); + byte[] parent = index.isChild() ? index.parent().childPrefix(params.parent, false) : null; + + this.indexKeyPrefix = index.keyPrefix(parent); + byte[] firstKey; if (params.first != null) { if (ascending) { - firstKey = index.start(params.first); + firstKey = index.start(parent, params.first); } else { - firstKey = index.end(params.first); + firstKey = index.end(parent, params.first); } } else if (ascending) { - firstKey = index.keyPrefix(); + firstKey = index.keyPrefix(parent); } else { - firstKey = index.end(); + firstKey = index.end(parent); } it.seek(firstKey); byte[] end = null; if (ascending) { - end = params.last != null ? index.end(params.last) : index.end(); + if (params.last != null) { + end = index.end(parent, params.last); + } else { + end = index.end(parent); + } } else { if (params.last != null) { - end = index.start(params.last); + end = index.start(parent, params.last); } if (it.hasNext()) { // When descending, the caller may have set up the start of iteration at a non-existant @@ -182,17 +192,16 @@ private T loadNext() { return null; } byte[] nextKey = nextEntry.getKey(); + // Next key is not part of the index, stop. + if (!startsWith(nextKey, indexKeyPrefix)) { + return null; + } // If the next key is an end marker, then skip it. if (isEndMarker(nextKey)) { continue; } - // Next key is not part of the index, stop. - if (!startsWith(nextKey, indexKeyPrefix)) { - return null; - } - // If there's a known end key and iteration has gone past it, stop. if (end != null) { int comp = compare(nextKey, end) * (ascending ? 1 : -1); @@ -207,7 +216,7 @@ private T loadNext() { if (index == null || index.isCopy()) { return db.serializer.deserialize(nextEntry.getValue(), type); } else { - byte[] key = stitch(ti.naturalIndex().keyPrefix(), nextEntry.getValue()); + byte[] key = ti.buildKey(false, ti.naturalIndex().keyPrefix(null), nextEntry.getValue()); return db.get(key, type); } } @@ -234,23 +243,7 @@ static boolean startsWith(byte[] key, byte[] prefix) { private boolean isEndMarker(byte[] key) { return (key.length > 2 && key[key.length - 2] == LevelDBTypeInfo.KEY_SEPARATOR && - key[key.length - 1] == (byte) LevelDBTypeInfo.END_MARKER.charAt(0)); - } - - private byte[] stitch(byte[]... comps) { - int len = 0; - for (byte[] comp : comps) { - len += comp.length; - } - - byte[] dest = new byte[len]; - int written = 0; - for (byte[] comp : comps) { - System.arraycopy(comp, 0, dest, written, comp.length); - written += comp.length; - } - - return dest; + key[key.length - 1] == LevelDBTypeInfo.END_MARKER[0]); } private int compare(byte[] a, byte[] b) { diff --git a/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBTypeInfo.java b/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBTypeInfo.java index 826e6cf068fd9..99cfa92a96e21 100644 --- a/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBTypeInfo.java +++ b/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBTypeInfo.java @@ -27,35 +27,100 @@ import java.util.Map; import static java.nio.charset.StandardCharsets.UTF_8; -import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.base.Throwables; /** * Holds metadata about app-specific types stored in LevelDB. Serves as a cache for data collected * via reflection, to make it cheaper to access it multiple times. + * + *

+ * The hierarchy of keys stored in LevelDB looks roughly like the following. This hierarchy ensures + * that iteration over indices is easy, and that updating values in the store is not overly + * expensive. Of note, indices choose using more disk space (one value per key) instead of keeping + * lists of pointers, which would be more expensive to update at runtime. + *

+ * + *

+ * Indentation defines when a sub-key lives under a parent key. In LevelDB, this means the full + * key would be the concatenation of everything up to that point in the hierarchy, with each + * component separated by a NULL byte. + *

+ * + *
+ * +TYPE_NAME
+ *   NATURAL_INDEX
+ *     +NATURAL_KEY
+ *     -
+ *   -NATURAL_INDEX
+ *   INDEX_NAME
+ *     +INDEX_VALUE
+ *       +NATURAL_KEY
+ *     -INDEX_VALUE
+ *     .INDEX_VALUE
+ *       CHILD_INDEX_NAME
+ *         +CHILD_INDEX_VALUE
+ *           NATURAL_KEY_OR_DATA
+ *         -
+ *   -INDEX_NAME
+ * 
+ * + *

+ * Entity data (either the entity's natural key or a copy of the data) is stored in all keys + * that end with "+NATURAL_KEY". A count of all objects that match a particular top-level index + * value is kept at the end marker. A count is also kept at the natural index's end marker, + * to make it easy to retrieve the number of all elements of a particular type. + *

+ * + *

+ * To illustrate, given a type "Foo", with a natural index and a second index called "bar", you'd + * have these keys and values in the store for two instances, one with natural key "key1" and the + * other "key2", both with value "yes" for "bar": + *

+ * + *
+ * Foo __main__ +key1   [data for instance 1]
+ * Foo __main__ +key2   [data for instance 2]
+ * Foo __main__ -       [count of all Foo]
+ * Foo bar +yes +key1   [instance 1 key or data, depending on index type]
+ * Foo bar +yes +key2   [instance 2 key or data, depending on index type]
+ * Foo bar +yes -       [count of all Foo with "bar=yes" ]
+ * 
+ * + *

+ * Note that all indexed values are prepended with "+", even if the index itself does not have an + * explicit end marker. This allows for easily skipping to the end of an index by telling LevelDB + * to seek to the "phantom" end marker of the index. + *

+ * + *

+ * Child indices are stored after their parent index. In the example above, let's assume there is + * a child index "child", whose parent is "bar". If both instances have value "no" for this field, + * the data in the store would look something like the following: + *

+ * + *
+ * ...
+ * Foo bar +yes -
+ * Foo bar .yes .child +no +key1   [instance 1 key or data, depending on index type]
+ * Foo bar .yes .child +no +key2   [instance 2 key or data, depending on index type]
+ * ...
+ * 
*/ class LevelDBTypeInfo { - static final String ENTRY_PREFIX = "+"; - static final String END_MARKER = "-"; + static final byte[] END_MARKER = new byte[] { '-' }; + static final byte ENTRY_PREFIX = (byte) '+'; static final byte KEY_SEPARATOR = 0x0; + static byte TRUE = (byte) '1'; + static byte FALSE = (byte) '0'; - // These constants are used in the Index.toKey() method below when encoding numbers into keys. - // See javadoc for that method for details. - private static final char POSITIVE_FILL = '.'; - private static final char NEGATIVE_FILL = '~'; - private static final char POSITIVE_MARKER = '='; - private static final char NEGATIVE_MARKER = '*'; - - @VisibleForTesting - static final int BYTE_ENCODED_LEN = String.valueOf(Byte.MAX_VALUE).length() + 1; - @VisibleForTesting - static final int INT_ENCODED_LEN = String.valueOf(Integer.MAX_VALUE).length() + 1; - @VisibleForTesting - static final int LONG_ENCODED_LEN = String.valueOf(Long.MAX_VALUE).length() + 1; - @VisibleForTesting - static final int SHORT_ENCODED_LEN = String.valueOf(Short.MAX_VALUE).length() + 1; + private static final byte SECONDARY_IDX_PREFIX = (byte) '.'; + private static final byte POSITIVE_MARKER = (byte) '='; + private static final byte NEGATIVE_MARKER = (byte) '*'; + private static final byte[] HEX_BYTES = new byte[] { + '0', '1', '2', '3', '4', '5', '6', '7', '8', '9', 'a', 'b', 'c', 'd', 'e', 'f' + }; private final LevelDB db; private final Class type; @@ -68,14 +133,21 @@ class LevelDBTypeInfo { this.indices = new HashMap<>(); KVTypeInfo ti = new KVTypeInfo(type); + + // First create the parent indices, then the child indices. ti.indices().forEach(idx -> { - indices.put(idx.value(), new Index(idx.value(), idx.copy(), ti.getAccessor(idx.value()))); + if (idx.parent().isEmpty()) { + indices.put(idx.value(), new Index(idx, ti.getAccessor(idx.value()), null)); + } + }); + ti.indices().forEach(idx -> { + if (!idx.parent().isEmpty()) { + indices.put(idx.value(), new Index(idx, ti.getAccessor(idx.value()), + indices.get(idx.parent()))); + } }); - ByteArrayOutputStream typePrefix = new ByteArrayOutputStream(); - typePrefix.write(utf8(ENTRY_PREFIX)); - typePrefix.write(alias); - this.typePrefix = typePrefix.toByteArray(); + this.typePrefix = alias; } Class type() { @@ -83,7 +155,7 @@ Class type() { } byte[] keyPrefix() { - return buildKey(false); + return typePrefix; } Index naturalIndex() { @@ -101,87 +173,174 @@ Collection indices() { return indices.values(); } - private byte[] utf8(String s) { - return s.getBytes(UTF_8); + byte[] buildKey(byte[]... components) { + return buildKey(true, components); } - private byte[] buildKey(boolean trim, String... components) { - try { - ByteArrayOutputStream kos = new ByteArrayOutputStream(typePrefix.length * 2); - kos.write(typePrefix); - for (int i = 0; i < components.length; i++) { - kos.write(utf8(components[i])); - if (!trim || i < components.length - 1) { - kos.write(KEY_SEPARATOR); - } + byte[] buildKey(boolean addTypePrefix, byte[]... components) { + int len = 0; + if (addTypePrefix) { + len += typePrefix.length + 1; + } + for (byte[] comp : components) { + len += comp.length; + } + len += components.length - 1; + + byte[] dest = new byte[len]; + int written = 0; + + if (addTypePrefix) { + System.arraycopy(typePrefix, 0, dest, 0, typePrefix.length); + dest[typePrefix.length] = KEY_SEPARATOR; + written += typePrefix.length + 1; + } + + for (byte[] comp : components) { + System.arraycopy(comp, 0, dest, written, comp.length); + written += comp.length; + if (written < dest.length) { + dest[written] = KEY_SEPARATOR; + written++; } - return kos.toByteArray(); - } catch (IOException ioe) { - throw Throwables.propagate(ioe); } + + return dest; } /** - * Models a single index in LevelDB. Keys are stored under the type's prefix, in sequential - * order according to the indexed value. For non-natural indices, the key also contains the - * entity's natural key after the indexed value, so that it's possible for multiple entities - * to have the same indexed value. - * - *

- * An end marker is used to mark where the index ends, and the boundaries of each indexed value - * within the index, to make descending iteration faster, at the expense of some disk space and - * minor overhead when iterating. A count of the number of indexed entities is kept at the end - * marker, so that it can be cleaned up when all entries are removed from the index. - *

+ * Models a single index in LevelDB. See top-level class's javadoc for a description of how the + * keys are generated. */ class Index { private final boolean copy; private final boolean isNatural; - private final String name; + private final byte[] name; private final KVTypeInfo.Accessor accessor; + private final Index parent; + + private Index(KVIndex self, KVTypeInfo.Accessor accessor, Index parent) { + byte[] name = self.value().getBytes(UTF_8); + if (parent != null) { + byte[] child = new byte[name.length + 1]; + child[0] = SECONDARY_IDX_PREFIX; + System.arraycopy(name, 0, child, 1, name.length); + } - private Index(String name, boolean copy, KVTypeInfo.Accessor accessor) { this.name = name; - this.isNatural = name.equals(KVIndex.NATURAL_INDEX_NAME); - this.copy = isNatural || copy; + this.isNatural = self.value().equals(KVIndex.NATURAL_INDEX_NAME); + this.copy = isNatural || self.copy(); this.accessor = accessor; + this.parent = parent; } boolean isCopy() { return copy; } + boolean isChild() { + return parent != null; + } + + Index parent() { + return parent; + } + + /** + * Creates a key prefix for child indices of this index. This allows the prefix to be + * calculated only once, avoiding redundant work when multiple child indices of the + * same parent index exist. + */ + byte[] childPrefix(Object value, boolean isEntity) throws Exception { + Preconditions.checkState(parent == null, "Not a parent index."); + return buildKey(name, toParentKey(isEntity ? getValue(value) : value)); + } + + Object getValue(Object entity) throws Exception { + return accessor.get(entity); + } + + private void checkParent(byte[] prefix) { + if (prefix != null) { + Preconditions.checkState(parent != null, "Parent prefix provided for parent index."); + } else { + Preconditions.checkState(parent == null, "Parent prefix missing for child index."); + } + } + /** The prefix for all keys that belong to this index. */ - byte[] keyPrefix() { - return buildKey(false, name); + byte[] keyPrefix(byte[] prefix) { + checkParent(prefix); + return (parent != null) ? buildKey(false, prefix, name) : buildKey(name); } /** The key where to start ascending iteration for entries that match the given value. */ - byte[] start(Object value) { - return buildKey(isNatural, name, toKey(value)); + byte[] start(byte[] prefix, Object value) { + checkParent(prefix); + return (parent != null) ? buildKey(false, prefix, name, toKey(value)) + : buildKey(name, toKey(value)); } /** The key for the index's end marker. */ - byte[] end() { - return buildKey(true, name, END_MARKER); + byte[] end(byte[] prefix) { + checkParent(prefix); + return (parent != null) ? buildKey(false, prefix, name, END_MARKER) + : buildKey(name, END_MARKER); } /** The key for the end marker for index entries with the given value. */ - byte[] end(Object value) throws Exception { - return buildKey(true, name, toKey(value), END_MARKER); + byte[] end(byte[] prefix, Object value) throws Exception { + checkParent(prefix); + return (parent != null) ? buildKey(false, prefix, name, toKey(value), END_MARKER) + : buildKey(name, toKey(value), END_MARKER); } /** The key in the index that identifies the given entity. */ - byte[] entityKey(Object entity) throws Exception { - Object indexValue = accessor.get(entity); + byte[] entityKey(byte[] prefix, Object entity) throws Exception { + Object indexValue = getValue(entity); Preconditions.checkNotNull(indexValue, "Null index value for %s in type %s.", name, type.getName()); - if (isNatural) { - return buildKey(true, name, toKey(indexValue)); + byte[] entityKey = start(prefix, indexValue); + if (!isNatural) { + entityKey = buildKey(false, entityKey, toKey(naturalIndex().getValue(entity))); + } + return entityKey; + } + + private void addOrRemove( + LevelDBWriteBatch batch, + Object entity, + byte[] data, + byte[] prefix) throws Exception { + Object indexValue = getValue(entity); + Preconditions.checkNotNull(indexValue, "Null index value for %s in type %s.", + name, type.getName()); + + byte[] entityKey = start(prefix, indexValue); + if (!isNatural) { + entityKey = buildKey(false, entityKey, toKey(naturalIndex().getValue(entity))); + } + + long delta; + if (data != null) { + byte[] stored = data; + if (!copy) { + stored = toKey(naturalIndex().getValue(entity)); + } + batch.put(entityKey, stored); + delta = 1L; } else { - Object naturalKey = naturalIndex().accessor.get(entity); - return buildKey(true, name, toKey(accessor.get(entity)), toKey(naturalKey)); + batch.delete(entityKey); + delta = -1L; + } + + // Only update counts for the natural index or top-level indices, since that's exposed through + // the API. + if (isNatural) { + batch.updateCount(end(prefix), delta); + } else if (parent == null) { + batch.updateCount(end(prefix, indexValue), delta); } } @@ -192,15 +351,10 @@ byte[] entityKey(Object entity) throws Exception { * @param entity The entity being added to the index. * @param data Serialized entity to store (when storing the entity, not a reference). * @param naturalKey The value's key. + * @param prefix The parent index prefix, if this is a child index. */ - void add(LevelDBWriteBatch batch, Object entity, byte[] data) throws Exception { - byte[] stored = data; - if (!copy) { - stored = db.serializer.serialize(toKey(naturalIndex().accessor.get(entity))); - } - batch.put(entityKey(entity), stored); - batch.updateCount(end(accessor.get(entity)), 1L); - batch.updateCount(end(), 1L); + void add(LevelDBWriteBatch batch, Object entity, byte[] data, byte[] prefix) throws Exception { + addOrRemove(batch, entity, data, prefix); } /** @@ -209,11 +363,10 @@ void add(LevelDBWriteBatch batch, Object entity, byte[] data) throws Exception { * @param batch Write batch with other related changes. * @param entity The entity being removed, to identify the index entry to modify. * @param naturalKey The value's key. + * @param prefix The parent index prefix, if this is a child index. */ - void remove(LevelDBWriteBatch batch, Object entity) throws Exception { - batch.delete(entityKey(entity)); - batch.updateCount(end(accessor.get(entity)), -1L); - batch.updateCount(end(), -1L); + void remove(LevelDBWriteBatch batch, Object entity, byte[] prefix) throws Exception { + addOrRemove(batch, entity, null, prefix); } long getCount(byte[] key) throws Exception { @@ -221,74 +374,74 @@ long getCount(byte[] key) throws Exception { return data != null ? db.serializer.deserializeLong(data) : 0; } + byte[] toParentKey(Object value) { + return toKey(value, SECONDARY_IDX_PREFIX); + } + + byte[] toKey(Object value) { + return toKey(value, ENTRY_PREFIX); + } + /** * Translates a value to be used as part of the store key. * * Integral numbers are encoded as a string in a way that preserves lexicographical - * ordering. The string is always as long as the maximum value for the given type (e.g. - * 11 characters for integers, including the character for the sign). The first character - * represents the sign (with the character for negative coming before the one for positive, - * which means you cannot use '-'...). The rest of the value is padded with a value that is - * "greater than 9" for negative values, so that for example "-123" comes before "-12" (the - * encoded value would look like "*~~~~~~~123"). For positive values, similarly, a value that - * is "lower than 0" (".") is used for padding. The fill characters were chosen for readability - * when looking at the encoded keys. + * ordering. The string is prepended with a marker telling whether the number is negative + * or positive ("*" for negative and "=" for positive are used since "-" and "+" have the + * opposite of the desired order), and then the number is encoded into a hex string (so + * it occupies twice the number of bytes as the original type). * * Arrays are encoded by encoding each element separately, separated by KEY_SEPARATOR. */ - @VisibleForTesting - String toKey(Object value) { - StringBuilder sb = new StringBuilder(ENTRY_PREFIX); + byte[] toKey(Object value, byte prefix) { + final byte[] result; if (value instanceof String) { - sb.append(value); + byte[] str = ((String) value).getBytes(UTF_8); + result = new byte[str.length + 1]; + result[0] = prefix; + System.arraycopy(str, 0, result, 1, str.length); } else if (value instanceof Boolean) { - sb.append(((Boolean) value).toString().toLowerCase()); + result = new byte[] { prefix, (Boolean) value ? TRUE : FALSE }; } else if (value.getClass().isArray()) { int length = Array.getLength(value); + byte[][] components = new byte[length][]; for (int i = 0; i < length; i++) { - sb.append(toKey(Array.get(value, i))); - sb.append(KEY_SEPARATOR); - } - if (length > 0) { - sb.setLength(sb.length() - 1); + components[i] = toKey(Array.get(value, i)); } + result = buildKey(false, components); } else { - int encodedLen; + int bytes; if (value instanceof Integer) { - encodedLen = INT_ENCODED_LEN; + bytes = Integer.SIZE; } else if (value instanceof Long) { - encodedLen = LONG_ENCODED_LEN; + bytes = Long.SIZE; } else if (value instanceof Short) { - encodedLen = SHORT_ENCODED_LEN; + bytes = Short.SIZE; } else if (value instanceof Byte) { - encodedLen = BYTE_ENCODED_LEN; + bytes = Byte.SIZE; } else { throw new IllegalArgumentException(String.format("Type %s not allowed as key.", value.getClass().getName())); } - long longValue = ((Number) value).longValue(); - String strVal; - if (longValue == Long.MIN_VALUE) { - // Math.abs() overflows for Long.MIN_VALUE. - strVal = String.valueOf(longValue).substring(1); - } else { - strVal = String.valueOf(Math.abs(longValue)); - } + bytes = bytes / Byte.SIZE; - sb.append(longValue >= 0 ? POSITIVE_MARKER : NEGATIVE_MARKER); + byte[] key = new byte[bytes * 2 + 2]; + long longValue = ((Number) value).longValue(); + key[0] = prefix; + key[1] = longValue > 0 ? POSITIVE_MARKER : NEGATIVE_MARKER; - char fill = longValue >= 0 ? POSITIVE_FILL : NEGATIVE_FILL; - for (int i = 0; i < encodedLen - strVal.length() - 1; i++) { - sb.append(fill); + for (int i = 0; i < key.length - 2; i++) { + int masked = (int) ((longValue >>> (4 * i)) & 0xF); + key[key.length - i - 1] = HEX_BYTES[masked]; } - sb.append(strVal); + result = key; } - return sb.toString(); + return result; } } diff --git a/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBWriteBatch.java b/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBWriteBatch.java index f3de251de554f..2fb8c79a73ec5 100644 --- a/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBWriteBatch.java +++ b/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBWriteBatch.java @@ -17,7 +17,6 @@ package org.apache.spark.kvstore; -import java.io.IOException; import java.util.Arrays; import java.util.HashMap; import java.util.Map; @@ -30,7 +29,7 @@ * A wrapper around the LevelDB library's WriteBatch with some extra functionality for keeping * track of counts. */ -class LevelDBWriteBatch { +class LevelDBWriteBatch implements AutoCloseable { private final LevelDB db; private final Map deltas; @@ -83,7 +82,7 @@ void write(boolean sync) { db.db().write(batch, new WriteOptions().sync(sync)); } - void close() throws IOException { + public void close() throws Exception { batch.close(); } diff --git a/common/kvstore/src/test/java/org/apache/spark/kvstore/CustomType1.java b/common/kvstore/src/test/java/org/apache/spark/kvstore/CustomType1.java index 2bea5b560681f..afb72b8689223 100644 --- a/common/kvstore/src/test/java/org/apache/spark/kvstore/CustomType1.java +++ b/common/kvstore/src/test/java/org/apache/spark/kvstore/CustomType1.java @@ -33,6 +33,9 @@ public class CustomType1 { @KVIndex("int") public int num; + @KVIndex(value = "child", parent = "id") + public String child; + @Override public boolean equals(Object o) { if (o instanceof CustomType1) { diff --git a/common/kvstore/src/test/java/org/apache/spark/kvstore/DBIteratorSuite.java b/common/kvstore/src/test/java/org/apache/spark/kvstore/DBIteratorSuite.java index 6c4469e1ed5d0..31af349621fe5 100644 --- a/common/kvstore/src/test/java/org/apache/spark/kvstore/DBIteratorSuite.java +++ b/common/kvstore/src/test/java/org/apache/spark/kvstore/DBIteratorSuite.java @@ -77,6 +77,7 @@ default BaseComparator reverse() { private final BaseComparator REF_INDEX_ORDER = (t1, t2) -> t1.id.compareTo(t2.id); private final BaseComparator COPY_INDEX_ORDER = (t1, t2) -> t1.name.compareTo(t2.name); private final BaseComparator NUMERIC_INDEX_ORDER = (t1, t2) -> t1.num - t2.num; + private final BaseComparator CHILD_INDEX_ORDER = (t1, t2) -> t1.child.compareTo(t2.child); @BeforeClass public static void setup() throws Exception { @@ -104,6 +105,7 @@ public static void setup() throws Exception { t.id = "id" + i; t.name = "name" + RND.nextInt(MAX_ENTRIES); t.num = RND.nextInt(MAX_ENTRIES); + t.child = "child" + (i % MIN_ENTRIES); allEntries.add(t); db.write(t); } @@ -122,6 +124,7 @@ public static void setup() throws Exception { t.id = first.id; t.name = first.name; t.num = first.num; + t.child = first.child; allEntries.add(t); clashingEntries.add(t); db.write(t); @@ -135,6 +138,7 @@ public static void setup() throws Exception { t.id = first.id; t.name = first.name + "a"; t.num = first.num; + t.child = first.child; allEntries.add(t); db.write(t); } @@ -170,6 +174,12 @@ public void numericIndex() throws Exception { testIteration(NUMERIC_INDEX_ORDER, view().index("int"), null, null); } + @Test + public void childIndex() throws Exception { + CustomType1 any = pickLimit(); + testIteration(CHILD_INDEX_ORDER, view().index("child").parent(any.id), null, null); + } + @Test public void naturalIndexDescending() throws Exception { testIteration(NATURAL_ORDER, view().reverse(), null, null); @@ -190,6 +200,12 @@ public void numericIndexDescending() throws Exception { testIteration(NUMERIC_INDEX_ORDER, view().index("int").reverse(), null, null); } + @Test + public void childIndexDescending() throws Exception { + CustomType1 any = pickLimit(); + testIteration(CHILD_INDEX_ORDER, view().index("child").parent(any.id).reverse(), null, null); + } + @Test public void naturalIndexWithStart() throws Exception { CustomType1 first = pickLimit(); @@ -214,6 +230,13 @@ public void numericIndexWithStart() throws Exception { testIteration(NUMERIC_INDEX_ORDER, view().index("int").first(first.num), first, null); } + @Test + public void childIndexWithStart() throws Exception { + CustomType1 any = pickLimit(); + testIteration(CHILD_INDEX_ORDER, view().index("child").parent(any.id).first(any.child), null, + null); + } + @Test public void naturalIndexDescendingWithStart() throws Exception { CustomType1 first = pickLimit(); @@ -238,32 +261,50 @@ public void numericIndexDescendingWithStart() throws Exception { testIteration(NUMERIC_INDEX_ORDER, view().reverse().index("int").first(first.num), first, null); } + @Test + public void childIndexDescendingWithStart() throws Exception { + CustomType1 any = pickLimit(); + testIteration(CHILD_INDEX_ORDER, + view().index("child").parent(any.id).first(any.child).reverse(), null, null); + } + @Test public void naturalIndexWithSkip() throws Exception { - testIteration(NATURAL_ORDER, view().skip(RND.nextInt(allEntries.size() / 2)), null, null); + testIteration(NATURAL_ORDER, view().skip(pickCount()), null, null); } @Test public void refIndexWithSkip() throws Exception { - testIteration(REF_INDEX_ORDER, view().index("id").skip(RND.nextInt(allEntries.size() / 2)), - null, null); + testIteration(REF_INDEX_ORDER, view().index("id").skip(pickCount()), null, null); } @Test public void copyIndexWithSkip() throws Exception { - testIteration(COPY_INDEX_ORDER, view().index("name").skip(RND.nextInt(allEntries.size() / 2)), + testIteration(COPY_INDEX_ORDER, view().index("name").skip(pickCount()), null, null); + } + + @Test + public void childIndexWithSkip() throws Exception { + CustomType1 any = pickLimit(); + testIteration(CHILD_INDEX_ORDER, view().index("child").parent(any.id).skip(pickCount()), null, null); } @Test public void naturalIndexWithMax() throws Exception { - testIteration(NATURAL_ORDER, view().max(RND.nextInt(allEntries.size() / 2)), null, null); + testIteration(NATURAL_ORDER, view().max(pickCount()), null, null); } @Test public void copyIndexWithMax() throws Exception { - testIteration(COPY_INDEX_ORDER, view().index("name").max(RND.nextInt(allEntries.size() / 2)), - null, null); + testIteration(COPY_INDEX_ORDER, view().index("name").max(pickCount()), null, null); + } + + @Test + public void childIndexWithMax() throws Exception { + CustomType1 any = pickLimit(); + testIteration(CHILD_INDEX_ORDER, view().index("child").parent(any.id).max(pickCount()), null, + null); } @Test @@ -290,6 +331,13 @@ public void numericIndexWithLast() throws Exception { testIteration(NUMERIC_INDEX_ORDER, view().index("int").last(last.num), null, last); } + @Test + public void childIndexWithLast() throws Exception { + CustomType1 any = pickLimit(); + testIteration(CHILD_INDEX_ORDER, view().index("child").parent(any.id).last(any.child), null, + null); + } + @Test public void naturalIndexDescendingWithLast() throws Exception { CustomType1 last = pickLimit(); @@ -316,6 +364,13 @@ public void numericIndexDescendingWithLast() throws Exception { null, last); } + @Test + public void childIndexDescendingWithLast() throws Exception { + CustomType1 any = pickLimit(); + testIteration(CHILD_INDEX_ORDER, view().index("child").parent(any.id).last(any.child).reverse(), + null, null); + } + @Test public void testRefWithIntNaturalKey() throws Exception { LevelDBSuite.IntKeyType i = new LevelDBSuite.IntKeyType(); @@ -336,6 +391,11 @@ private CustomType1 pickLimit() { return clashingEntries.get(RND.nextInt(clashingEntries.size())); } + private int pickCount() { + int count = RND.nextInt(allEntries.size() / 2); + return Math.max(count, 1); + } + /** * Compares the two values and falls back to comparing the natural key of CustomType1 * if they're the same, to mimic the behavior of the indexing code. @@ -366,6 +426,10 @@ private void testIteration( Iterable expected = indexOrder; BaseComparator expectedOrder = params.ascending ? order : order.reverse(); + if (params.parent != null) { + expected = Iterables.filter(expected, v -> params.parent.equals(v.id)); + } + if (first != null) { expected = Iterables.filter(expected, v -> expectedOrder.compare(first, v) <= 0); } diff --git a/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBSuite.java b/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBSuite.java index 1f88aae0be2aa..5d40309ab203b 100644 --- a/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBSuite.java +++ b/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBSuite.java @@ -80,6 +80,7 @@ public void testObjectWriteReadDelete() throws Exception { t.key = "key"; t.id = "id"; t.name = "name"; + t.child = "child"; try { db.read(CustomType1.class, t.key); @@ -111,11 +112,13 @@ public void testMultipleObjectWriteReadDelete() throws Exception { t1.key = "key1"; t1.id = "id"; t1.name = "name1"; + t1.child = "child1"; CustomType1 t2 = new CustomType1(); t2.key = "key2"; t2.id = "id"; t2.name = "name2"; + t2.child = "child2"; db.write(t1); db.write(t2); @@ -125,14 +128,14 @@ public void testMultipleObjectWriteReadDelete() throws Exception { assertEquals(2L, db.count(t1.getClass())); // There should be one "id" index entry with two values. - assertEquals(2, countIndexEntries(t1.getClass(), "id", t1.id)); + assertEquals(2, db.count(t1.getClass(), "id", t1.id)); // Delete the first entry; now there should be 3 remaining keys, since one of the "name" // index entries should have been removed. db.delete(t1.getClass(), t1.key); // Make sure there's a single entry in the "id" index now. - assertEquals(1, countIndexEntries(t2.getClass(), "id", t2.id)); + assertEquals(1, db.count(t2.getClass(), "id", t2.id)); // Delete the remaining entry, make sure all data is gone. db.delete(t2.getClass(), t2.key); @@ -145,6 +148,7 @@ public void testMultipleTypesWriteReadDelete() throws Exception { t1.key = "1"; t1.id = "id"; t1.name = "name1"; + t1.child = "child1"; IntKeyType t2 = new IntKeyType(); t2.key = 2; @@ -164,15 +168,15 @@ public void testMultipleTypesWriteReadDelete() throws Exception { assertEquals(t3, db.read(t3.getClass(), t3.key)); // There should be one "id" index with a single entry for each type. - assertEquals(1, countIndexEntries(t1.getClass(), "id", t1.id)); - assertEquals(1, countIndexEntries(t2.getClass(), "id", t2.id)); - assertEquals(1, countIndexEntries(t3.getClass(), "id", t3.id)); + assertEquals(1, db.count(t1.getClass(), "id", t1.id)); + assertEquals(1, db.count(t2.getClass(), "id", t2.id)); + assertEquals(1, db.count(t3.getClass(), "id", t3.id)); // Delete the first entry; this should not affect the entries for the second type. db.delete(t1.getClass(), t1.key); assertEquals(0, countKeys(t1.getClass())); - assertEquals(1, countIndexEntries(t2.getClass(), "id", t2.id)); - assertEquals(1, countIndexEntries(t3.getClass(), "id", t3.id)); + assertEquals(1, db.count(t2.getClass(), "id", t2.id)); + assertEquals(1, db.count(t3.getClass(), "id", t3.id)); // Delete the remaining entries, make sure all data is gone. db.delete(t2.getClass(), t2.key); @@ -189,6 +193,7 @@ public void testMetadata() throws Exception { CustomType1 t = new CustomType1(); t.id = "id"; t.name = "name"; + t.child = "child"; db.setMetadata(t); assertEquals(t, db.getMetadata(CustomType1.class)); @@ -197,17 +202,13 @@ public void testMetadata() throws Exception { assertNull(db.getMetadata(CustomType1.class)); } - private long countIndexEntries(Class type, String index, Object value) throws Exception { - LevelDBTypeInfo.Index idx = db.getTypeInfo(type).index(index); - return idx.getCount(idx.end()); - } - @Test public void testUpdate() throws Exception { CustomType1 t = new CustomType1(); t.key = "key"; t.id = "id"; t.name = "name"; + t.child = "child"; db.write(t); @@ -216,16 +217,8 @@ public void testUpdate() throws Exception { db.write(t); assertEquals(1, db.count(t.getClass())); - - LevelDBTypeInfo.Index ni = db.getTypeInfo(t.getClass()).index("name"); - assertEquals(1, ni.getCount(ni.end())); - assertEquals(1, ni.getCount(ni.end("anotherName"))); - try { - db.get(ni.end("name"), Integer.class); - fail("Should have gotten an exception."); - } catch (NoSuchElementException nsee) { - // Expected. - } + assertEquals(1, db.count(t.getClass(), "name", "anotherName")); + assertEquals(0, db.count(t.getClass(), "name", "name")); } private int countKeys(Class type) throws Exception { diff --git a/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBTypeInfoSuite.java b/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBTypeInfoSuite.java index cf69f32dfb354..18b00d6f6f577 100644 --- a/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBTypeInfoSuite.java +++ b/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBTypeInfoSuite.java @@ -27,18 +27,20 @@ public class LevelDBTypeInfoSuite { @Test public void testIndexAnnotation() throws Exception { KVTypeInfo ti = new KVTypeInfo(CustomType1.class); - assertEquals(4, ti.indices().count()); + assertEquals(5, ti.indices().count()); CustomType1 t1 = new CustomType1(); t1.key = "key"; t1.id = "id"; t1.name = "name"; t1.num = 42; + t1.child = "child"; assertEquals(t1.key, ti.getIndexValue(KVIndex.NATURAL_INDEX_NAME, t1)); assertEquals(t1.id, ti.getIndexValue("id", t1)); assertEquals(t1.name, ti.getIndexValue("name", t1)); assertEquals(t1.num, ti.getIndexValue("int", t1)); + assertEquals(t1.child, ti.getIndexValue("child", t1)); } @Test(expected = IllegalArgumentException.class) @@ -83,44 +85,41 @@ public void testKeyClashes() throws Exception { t3.name = "aaa"; // Make sure entries with conflicting names are sorted correctly. - assertBefore(ti.index("name").entityKey(t1), ti.index("name").entityKey(t2)); - assertBefore(ti.index("name").entityKey(t1), ti.index("name").entityKey(t3)); - assertBefore(ti.index("name").entityKey(t2), ti.index("name").entityKey(t3)); + assertBefore(ti.index("name").entityKey(null, t1), ti.index("name").entityKey(null, t2)); + assertBefore(ti.index("name").entityKey(null, t1), ti.index("name").entityKey(null, t3)); + assertBefore(ti.index("name").entityKey(null, t2), ti.index("name").entityKey(null, t3)); } @Test public void testNumEncoding() throws Exception { LevelDBTypeInfo.Index idx = newTypeInfo(CustomType1.class).indices().iterator().next(); + assertEquals("+=00000001", new String(idx.toKey(1), UTF_8)); + assertEquals("+=00000010", new String(idx.toKey(16), UTF_8)); + assertEquals("+=7fffffff", new String(idx.toKey(Integer.MAX_VALUE), UTF_8)); + assertBefore(idx.toKey(1), idx.toKey(2)); assertBefore(idx.toKey(-1), idx.toKey(2)); assertBefore(idx.toKey(-11), idx.toKey(2)); assertBefore(idx.toKey(-11), idx.toKey(-1)); assertBefore(idx.toKey(1), idx.toKey(11)); assertBefore(idx.toKey(Integer.MIN_VALUE), idx.toKey(Integer.MAX_VALUE)); - assertEquals(LevelDBTypeInfo.INT_ENCODED_LEN + LevelDBTypeInfo.ENTRY_PREFIX.length(), - idx.toKey(Integer.MIN_VALUE).length()); assertBefore(idx.toKey(1L), idx.toKey(2L)); assertBefore(idx.toKey(-1L), idx.toKey(2L)); assertBefore(idx.toKey(Long.MIN_VALUE), idx.toKey(Long.MAX_VALUE)); - assertEquals(LevelDBTypeInfo.LONG_ENCODED_LEN + LevelDBTypeInfo.ENTRY_PREFIX.length(), - idx.toKey(Long.MIN_VALUE).length()); assertBefore(idx.toKey((short) 1), idx.toKey((short) 2)); assertBefore(idx.toKey((short) -1), idx.toKey((short) 2)); assertBefore(idx.toKey(Short.MIN_VALUE), idx.toKey(Short.MAX_VALUE)); - assertEquals(LevelDBTypeInfo.SHORT_ENCODED_LEN + LevelDBTypeInfo.ENTRY_PREFIX.length(), - idx.toKey(Short.MIN_VALUE).length()); assertBefore(idx.toKey((byte) 1), idx.toKey((byte) 2)); assertBefore(idx.toKey((byte) -1), idx.toKey((byte) 2)); assertBefore(idx.toKey(Byte.MIN_VALUE), idx.toKey(Byte.MAX_VALUE)); - assertEquals(LevelDBTypeInfo.BYTE_ENCODED_LEN + LevelDBTypeInfo.ENTRY_PREFIX.length(), - idx.toKey(Byte.MIN_VALUE).length()); - assertEquals(LevelDBTypeInfo.ENTRY_PREFIX + "false", idx.toKey(false)); - assertEquals(LevelDBTypeInfo.ENTRY_PREFIX + "true", idx.toKey(true)); + byte prefix = LevelDBTypeInfo.ENTRY_PREFIX; + assertSame(new byte[] { prefix, LevelDBTypeInfo.FALSE }, idx.toKey(false)); + assertSame(new byte[] { prefix, LevelDBTypeInfo.TRUE }, idx.toKey(true)); } @Test @@ -147,6 +146,10 @@ private void assertBefore(String str1, String str2) { assertTrue(String.format("%s < %s failed", str1, str2), str1.compareTo(str2) < 0); } + private void assertSame(byte[] key1, byte[] key2) { + assertEquals(new String(key1, UTF_8), new String(key2, UTF_8)); + } + public static class NoNaturalIndex { public String id; From 85e893021497386effff3d58b902943fe896e858 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Mon, 15 May 2017 11:40:18 -0700 Subject: [PATCH 13/18] SHS-NG M1: Remove LevelDBWriteBatch. After recent changes, LevelDBWriteBatch becomes sort of redundant; the needed logic can be moved to LevelDBTypeInfo to save some more redundant operations (including reads, writes, can calculation of keys), also saving some code. I also backported some changes from a later set, making DBIteratorSuite an abstract class. For whatever odd reason, sbt / junit-interface have a hard time finding all the test suites in this module, and this particular change seems to help. --- .../org/apache/spark/kvstore/KVTypeInfo.java | 4 +- .../org/apache/spark/kvstore/LevelDB.java | 55 ++++----- .../apache/spark/kvstore/LevelDBIterator.java | 5 +- .../apache/spark/kvstore/LevelDBTypeInfo.java | 105 ++++++++++++---- .../spark/kvstore/LevelDBWriteBatch.java | 112 ------------------ .../apache/spark/kvstore/DBIteratorSuite.java | 60 +++++----- .../spark/kvstore/LevelDBBenchmark.java | 79 +++--------- .../spark/kvstore/LevelDBIteratorSuite.java | 48 ++++++++ 8 files changed, 200 insertions(+), 268 deletions(-) delete mode 100644 common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBWriteBatch.java create mode 100644 common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBIteratorSuite.java diff --git a/common/kvstore/src/main/java/org/apache/spark/kvstore/KVTypeInfo.java b/common/kvstore/src/main/java/org/apache/spark/kvstore/KVTypeInfo.java index 83df72994c4c8..19a559f902815 100644 --- a/common/kvstore/src/main/java/org/apache/spark/kvstore/KVTypeInfo.java +++ b/common/kvstore/src/main/java/org/apache/spark/kvstore/KVTypeInfo.java @@ -61,8 +61,10 @@ public KVTypeInfo(Class type) throws Exception { } } - Preconditions.checkArgument(accessors.containsKey(KVIndex.NATURAL_INDEX_NAME), + Preconditions.checkArgument(indices.containsKey(KVIndex.NATURAL_INDEX_NAME), "No natural index defined for type %s.", type.getName()); + Preconditions.checkArgument(indices.get(KVIndex.NATURAL_INDEX_NAME).parent().isEmpty(), + "Natural index of %s cannot have a parent.", type.getName()); for (KVIndex idx : indices.values()) { if (!idx.parent().isEmpty()) { diff --git a/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDB.java b/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDB.java index 3e56e56416606..ef398eba69840 100644 --- a/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDB.java +++ b/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDB.java @@ -36,7 +36,6 @@ import org.iq80.leveldb.DB; import org.iq80.leveldb.Options; import org.iq80.leveldb.WriteBatch; -import org.iq80.leveldb.WriteOptions; /** * Implementation of KVStore that uses LevelDB as the underlying data store. @@ -132,14 +131,10 @@ public T read(Class klass, Object naturalKey) throws Exception { @Override public void write(Object value) throws Exception { - write(value, false); - } - - public void write(Object value, boolean sync) throws Exception { Preconditions.checkArgument(value != null, "Null values are not allowed."); LevelDBTypeInfo ti = getTypeInfo(value.getClass()); - try (LevelDBWriteBatch batch = new LevelDBWriteBatch(this)) { + try (WriteBatch batch = db().createWriteBatch()) { byte[] data = serializer.serialize(value); synchronized (ti) { Object existing; @@ -149,49 +144,33 @@ public void write(Object value, boolean sync) throws Exception { existing = null; } - PrefixCache cache = new PrefixCache(); - + PrefixCache cache = new PrefixCache(value); + byte[] naturalKey = ti.naturalIndex().toKey(ti.naturalIndex().getValue(value)); for (LevelDBTypeInfo.Index idx : ti.indices()) { - // Try to avoid unnecessary writes by only updating copy indices, or indices whose value - // has changed. - Object indexed = idx.getValue(value); - byte[] prefix = cache.getPrefix(idx, indexed); - if (existing == null) { - idx.add(batch, value, data, prefix); - } else { - if (idx.isCopy() || !Objects.equal(indexed, idx.getValue(existing))) { - byte[] existingPrefix = idx.isChild() ? idx.parent().childPrefix(existing, true) - : null; - idx.remove(batch, existing, existingPrefix); - idx.add(batch, value, data, prefix); - } - } + byte[] prefix = cache.getPrefix(idx); + idx.add(batch, value, existing, data, naturalKey, prefix); } - batch.write(sync); + db().write(batch); } } } @Override public void delete(Class type, Object naturalKey) throws Exception { - delete(type, naturalKey, false); - } - - public void delete(Class type, Object naturalKey, boolean sync) throws Exception { Preconditions.checkArgument(naturalKey != null, "Null keys are not allowed."); - try (LevelDBWriteBatch batch = new LevelDBWriteBatch(this)) { + try (WriteBatch batch = db().createWriteBatch()) { LevelDBTypeInfo ti = getTypeInfo(type); byte[] key = ti.naturalIndex().start(null, naturalKey); - PrefixCache cache = new PrefixCache(); - synchronized (ti) { byte[] data = db().get(key); if (data != null) { Object existing = serializer.deserialize(data, type); + PrefixCache cache = new PrefixCache(existing); + byte[] keyBytes = ti.naturalIndex().toKey(ti.naturalIndex().getValue(existing)); for (LevelDBTypeInfo.Index idx : ti.indices()) { - idx.remove(batch, existing, cache.getPrefix(idx, idx.getValue(existing))); + idx.remove(batch, existing, keyBytes, cache.getPrefix(idx)); } - batch.write(sync); + db().write(batch); } } } catch (NoSuchElementException nse) { @@ -299,14 +278,20 @@ public static class TypeAliases { private static class PrefixCache { - private final Map prefixes = new HashMap<>(); + private final Object entity; + private final Map prefixes; + + PrefixCache(Object entity) { + this.entity = entity; + this.prefixes = new HashMap<>(); + } - byte[] getPrefix(LevelDBTypeInfo.Index idx, Object entity) throws Exception { + byte[] getPrefix(LevelDBTypeInfo.Index idx) throws Exception { byte[] prefix = null; if (idx.isChild()) { prefix = prefixes.get(idx.parent()); if (prefix == null) { - prefix = idx.parent().childPrefix(entity, true); + prefix = idx.parent().childPrefix(idx.parent().getValue(entity)); prefixes.put(idx.parent(), prefix); } } diff --git a/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBIterator.java b/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBIterator.java index 1467b03f76ac8..2143a0e877389 100644 --- a/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBIterator.java +++ b/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBIterator.java @@ -57,7 +57,7 @@ class LevelDBIterator implements KVStoreIterator { Preconditions.checkArgument(!index.isChild() || params.parent != null, "Cannot iterate over child index %s without parent value.", params.index); - byte[] parent = index.isChild() ? index.parent().childPrefix(params.parent, false) : null; + byte[] parent = index.isChild() ? index.parent().childPrefix(params.parent) : null; this.indexKeyPrefix = index.keyPrefix(parent); @@ -191,6 +191,7 @@ private T loadNext() { } catch (NoSuchElementException e) { return null; } + byte[] nextKey = nextEntry.getKey(); // Next key is not part of the index, stop. if (!startsWith(nextKey, indexKeyPrefix)) { @@ -246,7 +247,7 @@ private boolean isEndMarker(byte[] key) { key[key.length - 1] == LevelDBTypeInfo.END_MARKER[0]); } - private int compare(byte[] a, byte[] b) { + static int compare(byte[] a, byte[] b) { int diff = 0; int minLen = Math.min(a.length, b.length); for (int i = 0; i < minLen; i++) { diff --git a/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBTypeInfo.java b/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBTypeInfo.java index 99cfa92a96e21..64feefc0f0b8e 100644 --- a/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBTypeInfo.java +++ b/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBTypeInfo.java @@ -29,6 +29,7 @@ import com.google.common.base.Preconditions; import com.google.common.base.Throwables; +import org.iq80.leveldb.WriteBatch; /** * Holds metadata about app-specific types stored in LevelDB. Serves as a cache for data collected @@ -67,9 +68,9 @@ * *

* Entity data (either the entity's natural key or a copy of the data) is stored in all keys - * that end with "+NATURAL_KEY". A count of all objects that match a particular top-level index - * value is kept at the end marker. A count is also kept at the natural index's end marker, - * to make it easy to retrieve the number of all elements of a particular type. + * that end with "+". A count of all objects that match a particular top-level index + * value is kept at the end marker ("-"). A count is also kept at the natural index's end + * marker, to make it easy to retrieve the number of all elements of a particular type. *

* *

@@ -252,9 +253,9 @@ Index parent() { * calculated only once, avoiding redundant work when multiple child indices of the * same parent index exist. */ - byte[] childPrefix(Object value, boolean isEntity) throws Exception { + byte[] childPrefix(Object value) throws Exception { Preconditions.checkState(parent == null, "Not a parent index."); - return buildKey(name, toParentKey(isEntity ? getValue(value) : value)); + return buildKey(name, toParentKey(value)); } Object getValue(Object entity) throws Exception { @@ -308,10 +309,21 @@ byte[] entityKey(byte[] prefix, Object entity) throws Exception { return entityKey; } + private void updateCount(WriteBatch batch, byte[] key, long delta) throws Exception { + long updated = getCount(key) + delta; + if (updated > 0) { + batch.put(key, db.serializer.serialize(updated)); + } else { + batch.delete(key); + } + } + private void addOrRemove( - LevelDBWriteBatch batch, + WriteBatch batch, Object entity, + Object existing, byte[] data, + byte[] naturalKey, byte[] prefix) throws Exception { Object indexValue = getValue(entity); Preconditions.checkNotNull(indexValue, "Null index value for %s in type %s.", @@ -319,28 +331,58 @@ private void addOrRemove( byte[] entityKey = start(prefix, indexValue); if (!isNatural) { - entityKey = buildKey(false, entityKey, toKey(naturalIndex().getValue(entity))); + entityKey = buildKey(false, entityKey, naturalKey); } - long delta; - if (data != null) { - byte[] stored = data; - if (!copy) { - stored = toKey(naturalIndex().getValue(entity)); + boolean needCountUpdate = (existing == null); + + // Check whether the index key for the existing value matches the new value. If it doesn't, + // then explicitly delete the existing key, otherwise just let the "put()" call overwrite it. + // + // Also check whether we need to update the counts. If the indexed value is changing, we + // need to decrement the count at the old index value, and the new indexed value count needs + // to be incremented. + // + // Natural indices don't need to be checked, because by definition both old and new elements + // will have the same key. + if (existing != null && !isNatural) { + byte[] oldPrefix = null; + Object oldIndexedValue = getValue(existing); + boolean removeExisting = !indexValue.equals(oldIndexedValue); + if (!removeExisting && isChild()) { + oldPrefix = parent().childPrefix(parent().getValue(existing)); + removeExisting = LevelDBIterator.compare(prefix, oldPrefix) != 0; + } + + if (removeExisting) { + if (oldPrefix == null && isChild()) { + oldPrefix = parent().childPrefix(parent().getValue(existing)); + } + + byte[] oldKey = entityKey(oldPrefix, existing); + batch.delete(oldKey); + + // If the indexed value has changed, we need to update the counts at the old and new + // end markers for the indexed value. + if (!isChild()) { + byte[] oldCountKey = end(null, oldIndexedValue); + updateCount(batch, oldCountKey, -1L); + needCountUpdate = true; + } } + } + + if (data != null) { + byte[] stored = copy ? data : naturalKey; batch.put(entityKey, stored); - delta = 1L; } else { batch.delete(entityKey); - delta = -1L; } - // Only update counts for the natural index or top-level indices, since that's exposed through - // the API. - if (isNatural) { - batch.updateCount(end(prefix), delta); - } else if (parent == null) { - batch.updateCount(end(prefix, indexValue), delta); + if (needCountUpdate && !isChild()) { + long delta = data != null ? 1L : -1L; + byte[] countKey = isNatural ? end(prefix) : end(prefix, indexValue); + updateCount(batch, countKey, delta); } } @@ -349,12 +391,19 @@ private void addOrRemove( * * @param batch Write batch with other related changes. * @param entity The entity being added to the index. + * @param existing The entity being replaced in the index, or null. * @param data Serialized entity to store (when storing the entity, not a reference). - * @param naturalKey The value's key. + * @param naturalKey The value's natural key (to avoid re-computing it for every index). * @param prefix The parent index prefix, if this is a child index. */ - void add(LevelDBWriteBatch batch, Object entity, byte[] data, byte[] prefix) throws Exception { - addOrRemove(batch, entity, data, prefix); + void add( + WriteBatch batch, + Object entity, + Object existing, + byte[] data, + byte[] naturalKey, + byte[] prefix) throws Exception { + addOrRemove(batch, entity, existing, data, naturalKey, prefix); } /** @@ -362,11 +411,15 @@ void add(LevelDBWriteBatch batch, Object entity, byte[] data, byte[] prefix) thr * * @param batch Write batch with other related changes. * @param entity The entity being removed, to identify the index entry to modify. - * @param naturalKey The value's key. + * @param naturalKey The value's natural key (to avoid re-computing it for every index). * @param prefix The parent index prefix, if this is a child index. */ - void remove(LevelDBWriteBatch batch, Object entity, byte[] prefix) throws Exception { - addOrRemove(batch, entity, null, prefix); + void remove( + WriteBatch batch, + Object entity, + byte[] naturalKey, + byte[] prefix) throws Exception { + addOrRemove(batch, entity, null, null, naturalKey, prefix); } long getCount(byte[] key) throws Exception { diff --git a/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBWriteBatch.java b/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBWriteBatch.java deleted file mode 100644 index 2fb8c79a73ec5..0000000000000 --- a/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBWriteBatch.java +++ /dev/null @@ -1,112 +0,0 @@ -/* - * 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.kvstore; - -import java.util.Arrays; -import java.util.HashMap; -import java.util.Map; - -import org.iq80.leveldb.DB; -import org.iq80.leveldb.WriteBatch; -import org.iq80.leveldb.WriteOptions; - -/** - * A wrapper around the LevelDB library's WriteBatch with some extra functionality for keeping - * track of counts. - */ -class LevelDBWriteBatch implements AutoCloseable { - - private final LevelDB db; - private final Map deltas; - private final WriteBatch batch; - - LevelDBWriteBatch(LevelDB db) { - this.db = db; - this.batch = db.db().createWriteBatch(); - this.deltas = new HashMap<>(); - } - - void updateCount(byte[] key, long delta) { - KeyWrapper kw = new KeyWrapper(key); - Long fullDelta = deltas.get(kw); - if (fullDelta != null) { - fullDelta += delta; - } else { - fullDelta = delta; - } - deltas.put(kw, fullDelta); - } - - void put(byte[] key, byte[] value) { - batch.put(key, value); - } - - void delete(byte[] key) { - batch.delete(key); - } - - void write(boolean sync) { - for (Map.Entry e : deltas.entrySet()) { - long delta = e.getValue(); - if (delta == 0) { - continue; - } - - byte[] key = e.getKey().key; - byte[] data = db.db().get(key); - long count = data != null ? db.serializer.deserializeLong(data) : 0L; - long newCount = count + delta; - - if (newCount > 0) { - batch.put(key, db.serializer.serialize(newCount)); - } else { - batch.delete(key); - } - } - - db.db().write(batch, new WriteOptions().sync(sync)); - } - - public void close() throws Exception { - batch.close(); - } - - private static class KeyWrapper { - - private final byte[] key; - - KeyWrapper(byte[] key) { - this.key = key; - } - - @Override - public boolean equals(Object other) { - if (other instanceof KeyWrapper) { - return Arrays.equals(key, ((KeyWrapper) other).key); - } - return false; - } - - @Override - public int hashCode() { - return Arrays.hashCode(key); - } - - } - -} diff --git a/common/kvstore/src/test/java/org/apache/spark/kvstore/DBIteratorSuite.java b/common/kvstore/src/test/java/org/apache/spark/kvstore/DBIteratorSuite.java index 31af349621fe5..32489a2174c21 100644 --- a/common/kvstore/src/test/java/org/apache/spark/kvstore/DBIteratorSuite.java +++ b/common/kvstore/src/test/java/org/apache/spark/kvstore/DBIteratorSuite.java @@ -17,7 +17,6 @@ package org.apache.spark.kvstore; -import java.io.File; import java.util.Arrays; import java.util.ArrayList; import java.util.Collections; @@ -32,15 +31,11 @@ import com.google.common.collect.Lists; import org.apache.commons.io.FileUtils; import org.junit.AfterClass; -import org.junit.BeforeClass; +import org.junit.Before; import org.junit.Test; import static org.junit.Assert.*; -/** - * This class should really be called "LevelDBIteratorSuite" but for some reason I don't know, - * sbt does not run the tests if it has that name. - */ -public class DBIteratorSuite { +public abstract class DBIteratorSuite { private static final int MIN_ENTRIES = 42; private static final int MAX_ENTRIES = 1024; @@ -48,10 +43,9 @@ public class DBIteratorSuite { private static List allEntries; private static List clashingEntries; - private static LevelDB db; - private static File dbpath; + private static KVStore db; - private interface BaseComparator extends Comparator { + private static interface BaseComparator extends Comparator { /** * Returns a comparator that falls back to natural order if this comparator's ordering * returns equality for two elements. Used to mimic how the index sorts things internally. @@ -73,17 +67,32 @@ default BaseComparator reverse() { } } - private final BaseComparator NATURAL_ORDER = (t1, t2) -> t1.key.compareTo(t2.key); - private final BaseComparator REF_INDEX_ORDER = (t1, t2) -> t1.id.compareTo(t2.id); - private final BaseComparator COPY_INDEX_ORDER = (t1, t2) -> t1.name.compareTo(t2.name); - private final BaseComparator NUMERIC_INDEX_ORDER = (t1, t2) -> t1.num - t2.num; - private final BaseComparator CHILD_INDEX_ORDER = (t1, t2) -> t1.child.compareTo(t2.child); + private static final BaseComparator NATURAL_ORDER = (t1, t2) -> t1.key.compareTo(t2.key); + private static final BaseComparator REF_INDEX_ORDER = (t1, t2) -> t1.id.compareTo(t2.id); + private static final BaseComparator COPY_INDEX_ORDER = (t1, t2) -> t1.name.compareTo(t2.name); + private static final BaseComparator NUMERIC_INDEX_ORDER = (t1, t2) -> t1.num - t2.num; + private static final BaseComparator CHILD_INDEX_ORDER = (t1, t2) -> t1.child.compareTo(t2.child); + + /** + * Implementations should override this method; it is called only once, before all tests are + * run. Any state can be safely stored in static variables and cleaned up in a @AfterClass + * handler. + */ + protected abstract KVStore createStore() throws Exception; + + @AfterClass + public static void cleanupData() throws Exception { + allEntries = null; + db = null; + } + + @Before + public void setup() throws Exception { + if (db != null) { + return; + } - @BeforeClass - public static void setup() throws Exception { - dbpath = File.createTempFile("test.", ".ldb"); - dbpath.delete(); - db = new LevelDB(dbpath); + db = createStore(); int count = RND.nextInt(MAX_ENTRIES) + MIN_ENTRIES; @@ -143,17 +152,6 @@ public static void setup() throws Exception { db.write(t); } - @AfterClass - public static void cleanup() throws Exception { - allEntries = null; - if (db != null) { - db.close(); - } - if (dbpath != null) { - FileUtils.deleteQuietly(dbpath); - } - } - @Test public void naturalIndex() throws Exception { testIteration(NATURAL_ORDER, view(), null, null); diff --git a/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBBenchmark.java b/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBBenchmark.java index aecea26ec82f3..5e33606b12dd4 100644 --- a/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBBenchmark.java +++ b/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBBenchmark.java @@ -132,17 +132,9 @@ private static String toMs(double nanos) { @Test public void sequentialWritesNoIndex() throws Exception { List entries = createSimpleType(); - writeAll(entries, false, "sequentialWritesNoIndex"); - writeAll(entries, false, "sequentialUpdatesNoIndex"); - deleteNoIndex(entries, false, "sequentialDeleteNoIndex"); - } - - @Test - public void sequentialSyncWritesNoIndex() throws Exception { - List entries = createSimpleType(); - writeAll(entries, true, "sequentialSyncWritesNoIndex"); - writeAll(entries, true, "sequentialSyncUpdatesNoIndex"); - deleteNoIndex(entries, true, "sequentialSyncDeleteNoIndex"); + writeAll(entries, "sequentialWritesNoIndex"); + writeAll(entries, "sequentialUpdatesNoIndex"); + deleteNoIndex(entries, "sequentialDeleteNoIndex"); } @Test @@ -150,43 +142,21 @@ public void randomWritesNoIndex() throws Exception { List entries = createSimpleType(); Collections.shuffle(entries); - writeAll(entries, false, "randomWritesNoIndex"); - - Collections.shuffle(entries); - writeAll(entries, false, "randomUpdatesNoIndex"); - - Collections.shuffle(entries); - deleteNoIndex(entries, false, "randomDeletesNoIndex"); - } - - @Test - public void randomSyncWritesNoIndex() throws Exception { - List entries = createSimpleType(); - - Collections.shuffle(entries); - writeAll(entries, true, "randomSyncWritesNoIndex"); + writeAll(entries, "randomWritesNoIndex"); Collections.shuffle(entries); - writeAll(entries, true, "randomSyncUpdatesNoIndex"); + writeAll(entries, "randomUpdatesNoIndex"); Collections.shuffle(entries); - deleteNoIndex(entries, true, "randomSyncDeletesNoIndex"); + deleteNoIndex(entries, "randomDeletesNoIndex"); } @Test public void sequentialWritesIndexedType() throws Exception { List entries = createIndexedType(); - writeAll(entries, false, "sequentialWritesIndexed"); - writeAll(entries, false, "sequentialUpdatesIndexed"); - deleteIndexed(entries, false, "sequentialDeleteIndexed"); - } - - @Test - public void sequentialSyncWritesIndexedType() throws Exception { - List entries = createIndexedType(); - writeAll(entries, true, "sequentialSyncWritesIndexed"); - writeAll(entries, true, "sequentialSyncUpdatesIndexed"); - deleteIndexed(entries, true, "sequentialSyncDeleteIndexed"); + writeAll(entries, "sequentialWritesIndexed"); + writeAll(entries, "sequentialUpdatesIndexed"); + deleteIndexed(entries, "sequentialDeleteIndexed"); } @Test @@ -194,10 +164,10 @@ public void randomWritesIndexedTypeAndIteration() throws Exception { List entries = createIndexedType(); Collections.shuffle(entries); - writeAll(entries, false, "randomWritesIndexed"); + writeAll(entries, "randomWritesIndexed"); Collections.shuffle(entries); - writeAll(entries, false, "randomUpdatesIndexed"); + writeAll(entries, "randomUpdatesIndexed"); // Run iteration benchmarks here since we've gone through the trouble of writing all // the data already. @@ -208,18 +178,7 @@ public void randomWritesIndexedTypeAndIteration() throws Exception { iterate(view.index("name").reverse(), "refIndexDescending"); Collections.shuffle(entries); - deleteIndexed(entries, false, "randomDeleteIndexed"); - } - - @Test - public void randomSyncWritesIndexedTypeAndIteration() throws Exception { - List entries = createIndexedType(); - - Collections.shuffle(entries); - writeAll(entries, true, "randomSyncWritesIndexed"); - - Collections.shuffle(entries); - deleteIndexed(entries, true, "randomSyncDeleteIndexed"); + deleteIndexed(entries, "randomDeleteIndexed"); } private void iterate(KVStoreView view, String name) throws Exception { @@ -245,31 +204,29 @@ private void iterate(KVStoreView view, String name) throws Exception { } } - private void writeAll(List entries, boolean sync, String timerName) throws Exception { + private void writeAll(List entries, String timerName) throws Exception { Timer timer = newTimer(timerName); for (Object o : entries) { try(Timer.Context ctx = timer.time()) { - db.write(o, sync); + db.write(o); } } } - private void deleteNoIndex(List entries, boolean sync, String timerName) - throws Exception { + private void deleteNoIndex(List entries, String timerName) throws Exception { Timer delete = newTimer(timerName); for (SimpleType i : entries) { try(Timer.Context ctx = delete.time()) { - db.delete(i.getClass(), i.key, sync); + db.delete(i.getClass(), i.key); } } } - private void deleteIndexed(List entries, boolean sync, String timerName) - throws Exception { + private void deleteIndexed(List entries, String timerName) throws Exception { Timer delete = newTimer(timerName); for (IndexedType i : entries) { try(Timer.Context ctx = delete.time()) { - db.delete(i.getClass(), i.key, sync); + db.delete(i.getClass(), i.key); } } } diff --git a/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBIteratorSuite.java b/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBIteratorSuite.java new file mode 100644 index 0000000000000..93409712986ca --- /dev/null +++ b/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBIteratorSuite.java @@ -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. + */ + +package org.apache.spark.kvstore; + +import java.io.File; + +import org.apache.commons.io.FileUtils; +import org.junit.AfterClass; + +public class LevelDBIteratorSuite extends DBIteratorSuite { + + private static File dbpath; + private static LevelDB db; + + @AfterClass + public static void cleanup() throws Exception { + if (db != null) { + db.close(); + } + if (dbpath != null) { + FileUtils.deleteQuietly(dbpath); + } + } + + @Override + protected KVStore createStore() throws Exception { + dbpath = File.createTempFile("test.", ".ldb"); + dbpath.delete(); + db = new LevelDB(dbpath); + return db; + } + +} From c4330652ba8a6d053828ba358deafe0ad87422ee Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Fri, 26 May 2017 11:52:08 -0700 Subject: [PATCH 14/18] SHS-NG M1: small fix in loop condition. --- .../test/java/org/apache/spark/kvstore/DBIteratorSuite.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/common/kvstore/src/test/java/org/apache/spark/kvstore/DBIteratorSuite.java b/common/kvstore/src/test/java/org/apache/spark/kvstore/DBIteratorSuite.java index 32489a2174c21..2b15e6e112345 100644 --- a/common/kvstore/src/test/java/org/apache/spark/kvstore/DBIteratorSuite.java +++ b/common/kvstore/src/test/java/org/apache/spark/kvstore/DBIteratorSuite.java @@ -127,7 +127,9 @@ public void setup() throws Exception { // the same way the store is expected to. CustomType1 first = allEntries.get(0); clashingEntries = new ArrayList<>(); - for (int i = 0; i < RND.nextInt(MIN_ENTRIES) + 1; i++) { + + int clashCount = RND.nextInt(MIN_ENTRIES) + 1; + for (int i = 0; i < clashCount; i++) { CustomType1 t = new CustomType1(); t.key = "n-key" + (count + i); t.id = first.id; From cc6662b54f25cab3274a3536c9bb722371278b88 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Wed, 31 May 2017 13:15:08 -0700 Subject: [PATCH 15/18] SHS-NG M1: feedback. --- .../java/org/apache/spark/kvstore/KVStore.java | 17 ++++++++++++----- .../org/apache/spark/kvstore/KVStoreView.java | 6 +++--- .../org/apache/spark/kvstore/KVTypeInfo.java | 6 ++++-- .../java/org/apache/spark/kvstore/LevelDB.java | 5 +++++ 4 files changed, 24 insertions(+), 10 deletions(-) diff --git a/common/kvstore/src/main/java/org/apache/spark/kvstore/KVStore.java b/common/kvstore/src/main/java/org/apache/spark/kvstore/KVStore.java index 9ccf3924e6a69..3be4b829b4d8d 100644 --- a/common/kvstore/src/main/java/org/apache/spark/kvstore/KVStore.java +++ b/common/kvstore/src/main/java/org/apache/spark/kvstore/KVStore.java @@ -20,6 +20,7 @@ import java.io.Closeable; import java.util.Iterator; import java.util.Map; +import java.util.NoSuchElementException; /** * Abstraction for a local key/value store for storing app data. @@ -31,10 +32,10 @@ *

Serialization

* *

- * Data will be serialized to and deserialized from the underlying data store using a - * {@link KVStoreSerializer}, which can be customized by the application. The serializer is - * based on Jackson, so it supports all the Jackson annotations for controlling the serialization - * of app-defined types. + * If the underlying data store requires serialization, data will be serialized to and deserialized + * using a {@link KVStoreSerializer}, which can be customized by the application. The serializer is + * based on Jackson, so it supports all the Jackson annotations for controlling the serialization of + * app-defined types. *

* *

@@ -80,6 +81,10 @@ public interface KVStore extends Closeable { /** * Read a specific instance of an object. + * + * @param naturalKey The object's "natural key", which uniquely identifies it. Null keys + * are not allowed. + * @throws NoSuchElementException If an element with the given key does not exist. */ T read(Class klass, Object naturalKey) throws Exception; @@ -100,7 +105,9 @@ public interface KVStore extends Closeable { * Removes an object and all data related to it, like index entries, from the store. * * @param type The object's type. - * @param naturalKey The object's "natural key", which uniquely identifies it. + * @param naturalKey The object's "natural key", which uniquely identifies it. Null keys + * are not allowed. + * @throws NoSuchElementException If an element with the given key does not exist. */ void delete(Class type, Object naturalKey) throws Exception; diff --git a/common/kvstore/src/main/java/org/apache/spark/kvstore/KVStoreView.java b/common/kvstore/src/main/java/org/apache/spark/kvstore/KVStoreView.java index cc5874e958873..b761640e6da8b 100644 --- a/common/kvstore/src/main/java/org/apache/spark/kvstore/KVStoreView.java +++ b/common/kvstore/src/main/java/org/apache/spark/kvstore/KVStoreView.java @@ -31,7 +31,7 @@ *

* *

- * The iterators returns by this view are of type {@link KVStoreIterator}; they auto-close + * The iterators returned by this view are of type {@link KVStoreIterator}; they auto-close * when used in a for loop that exhausts their contents, but when used manually, they need * to be closed explicitly unless all elements are read. *

@@ -83,7 +83,7 @@ public KVStoreView parent(Object value) { } /** - * Iterates starting at the given value of the chosen index. + * Iterates starting at the given value of the chosen index (inclusive). */ public KVStoreView first(Object value) { this.first = value; @@ -91,7 +91,7 @@ public KVStoreView first(Object value) { } /** - * Stops iteration at the given value of the chosen index. + * Stops iteration at the given value of the chosen index (inclusive). */ public KVStoreView last(Object value) { this.last = value; diff --git a/common/kvstore/src/main/java/org/apache/spark/kvstore/KVTypeInfo.java b/common/kvstore/src/main/java/org/apache/spark/kvstore/KVTypeInfo.java index 19a559f902815..90f2ff0079b8a 100644 --- a/common/kvstore/src/main/java/org/apache/spark/kvstore/KVTypeInfo.java +++ b/common/kvstore/src/main/java/org/apache/spark/kvstore/KVTypeInfo.java @@ -41,22 +41,24 @@ public KVTypeInfo(Class type) throws Exception { this.accessors = new HashMap<>(); this.indices = new HashMap<>(); - for (Field f : type.getFields()) { + for (Field f : type.getDeclaredFields()) { KVIndex idx = f.getAnnotation(KVIndex.class); if (idx != null) { checkIndex(idx, indices); indices.put(idx.value(), idx); + f.setAccessible(true); accessors.put(idx.value(), new FieldAccessor(f)); } } - for (Method m : type.getMethods()) { + for (Method m : type.getDeclaredMethods()) { KVIndex idx = m.getAnnotation(KVIndex.class); if (idx != null) { checkIndex(idx, indices); Preconditions.checkArgument(m.getParameterTypes().length == 0, "Annotated method %s::%s should not have any parameters.", type.getName(), m.getName()); indices.put(idx.value(), idx); + m.setAccessible(true); accessors.put(idx.value(), new MethodAccessor(m)); } } diff --git a/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDB.java b/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDB.java index ef398eba69840..08b22fd8265d8 100644 --- a/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDB.java +++ b/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDB.java @@ -57,6 +57,11 @@ public class LevelDB implements KVStore { final AtomicReference _db; final KVStoreSerializer serializer; + /** + * Keep a mapping of class names to a shorter, unique ID managed by the store. This serves two + * purposes: make the keys stored on disk shorter, and spread out the keys, since class names + * will often have a long, redundant prefix (think "org.apache.spark."). + */ private final ConcurrentMap typeAliases; private final ConcurrentMap, LevelDBTypeInfo> types; From bdf4bd923adb964fcd7d1cb217376c66a1f46511 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Wed, 31 May 2017 19:03:30 -0700 Subject: [PATCH 16/18] SHS-NG M1: Fix LevelDBIterator.skip(). It was counting things incorrectly, and potentially returning the wrong value; the change also defers deserialization to when next() is called, so that hasNext() is cheaper. --- .../apache/spark/kvstore/LevelDBIterator.java | 37 +++++++++++++------ .../apache/spark/kvstore/LevelDBSuite.java | 21 +++++++++++ 2 files changed, 47 insertions(+), 11 deletions(-) diff --git a/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBIterator.java b/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBIterator.java index 2143a0e877389..a5d0f9f4fb373 100644 --- a/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBIterator.java +++ b/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBIterator.java @@ -42,7 +42,7 @@ class LevelDBIterator implements KVStoreIterator { private final long max; private boolean checkedNext; - private T next; + private byte[] next; private boolean closed; private long count; @@ -128,7 +128,20 @@ public T next() { throw new NoSuchElementException(); } checkedNext = false; - return next; + + try { + T ret; + if (index == null || index.isCopy()) { + ret = db.serializer.deserialize(next, type); + } else { + byte[] key = ti.buildKey(false, ti.naturalIndex().keyPrefix(null), next); + ret = db.get(key, type); + } + next = null; + return ret; + } catch (Exception e) { + throw Throwables.propagate(e); + } } @Override @@ -149,9 +162,16 @@ public List next(int max) { public boolean skip(long n) { long skipped = 0; while (skipped < n) { - next = null; + if (next != null) { + checkedNext = false; + next = null; + skipped++; + continue; + } + boolean hasNext = ascending ? it.hasNext() : it.hasPrev(); if (!hasNext) { + checkedNext = true; return false; } @@ -161,7 +181,7 @@ public boolean skip(long n) { } } - return true; + return hasNext(); } @Override @@ -172,7 +192,7 @@ public synchronized void close() throws IOException { } } - private T loadNext() { + private byte[] loadNext() { if (count >= max) { return null; } @@ -214,12 +234,7 @@ private T loadNext() { count++; // Next element is part of the iteration, return it. - if (index == null || index.isCopy()) { - return db.serializer.deserialize(nextEntry.getValue(), type); - } else { - byte[] key = ti.buildKey(false, ti.naturalIndex().keyPrefix(null), nextEntry.getValue()); - return db.get(key, type); - } + return nextEntry.getValue(); } } catch (Exception e) { throw Throwables.propagate(e); diff --git a/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBSuite.java b/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBSuite.java index 5d40309ab203b..ee1c397c08573 100644 --- a/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBSuite.java +++ b/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBSuite.java @@ -221,6 +221,27 @@ public void testUpdate() throws Exception { assertEquals(0, db.count(t.getClass(), "name", "name")); } + @Test + public void testSkip() throws Exception { + for (int i = 0; i < 10; i++) { + CustomType1 t = new CustomType1(); + t.key = "key" + i; + t.id = "id" + i; + t.name = "name" + i; + t.child = "child" + i; + + db.write(t); + } + + KVStoreIterator it = db.view(CustomType1.class).closeableIterator(); + assertTrue(it.hasNext()); + assertTrue(it.skip(5)); + assertEquals("key5", it.next().key); + assertTrue(it.skip(3)); + assertEquals("key9", it.next().key); + assertFalse(it.hasNext()); + } + private int countKeys(Class type) throws Exception { byte[] prefix = db.getTypeInfo(type).keyPrefix(); int count = 0; From 2e809d3ad4e040d3d3168903e223965da2c65f2a Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Thu, 1 Jun 2017 13:14:35 -0700 Subject: [PATCH 17/18] SHS-NG M1: feedback 2. --- .../apache/spark/kvstore/LevelDBTypeInfo.java | 32 +++++++++++++------ .../apache/spark/kvstore/DBIteratorSuite.java | 20 ++++-------- .../spark/kvstore/LevelDBTypeInfoSuite.java | 15 +++++++++ 3 files changed, 45 insertions(+), 22 deletions(-) diff --git a/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBTypeInfo.java b/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBTypeInfo.java index 64feefc0f0b8e..3ab17dbd03ca7 100644 --- a/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBTypeInfo.java +++ b/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBTypeInfo.java @@ -91,7 +91,8 @@ *

* Note that all indexed values are prepended with "+", even if the index itself does not have an * explicit end marker. This allows for easily skipping to the end of an index by telling LevelDB - * to seek to the "phantom" end marker of the index. + * to seek to the "phantom" end marker of the index. Throughout the code and comments, this part + * of the full LevelDB key is generally referred to as the "index value" of the entity. *

* *

@@ -258,6 +259,11 @@ byte[] childPrefix(Object value) throws Exception { return buildKey(name, toParentKey(value)); } + /** + * Gets the index value for a particular entity (which is the value of the field or method + * tagged with the index annotation). This is used as part of the LevelDB key where the + * entity (or its id) is stored. + */ Object getValue(Object entity) throws Exception { return accessor.get(entity); } @@ -276,7 +282,10 @@ byte[] keyPrefix(byte[] prefix) { return (parent != null) ? buildKey(false, prefix, name) : buildKey(name); } - /** The key where to start ascending iteration for entries that match the given value. */ + /** + * The key where to start ascending iteration for entities whose value for the indexed field + * match the given value. + */ byte[] start(byte[] prefix, Object value) { checkParent(prefix); return (parent != null) ? buildKey(false, prefix, name, toKey(value)) @@ -290,14 +299,14 @@ byte[] end(byte[] prefix) { : buildKey(name, END_MARKER); } - /** The key for the end marker for index entries with the given value. */ + /** The key for the end marker for entries with the given value. */ byte[] end(byte[] prefix, Object value) throws Exception { checkParent(prefix); return (parent != null) ? buildKey(false, prefix, name, toKey(value), END_MARKER) : buildKey(name, toKey(value), END_MARKER); } - /** The key in the index that identifies the given entity. */ + /** The full key in the index that identifies the given entity. */ byte[] entityKey(byte[] prefix, Object entity) throws Exception { Object indexValue = getValue(entity); Preconditions.checkNotNull(indexValue, "Null index value for %s in type %s.", @@ -336,15 +345,20 @@ private void addOrRemove( boolean needCountUpdate = (existing == null); - // Check whether the index key for the existing value matches the new value. If it doesn't, - // then explicitly delete the existing key, otherwise just let the "put()" call overwrite it. + // Check whether there's a need to update the index. The index needs to be updated in two + // cases: + // + // - There is no existing value for the entity, so a new index value will be added. + // - If there is a previously stored value for the entity, and the index value for the + // current index does not match the new value, the old entry needs to be deleted and + // the new one added. + // + // Natural indices don't need to be checked, because by definition both old and new entities + // will have the same key. The put() call is all that's needed in that case. // // Also check whether we need to update the counts. If the indexed value is changing, we // need to decrement the count at the old index value, and the new indexed value count needs // to be incremented. - // - // Natural indices don't need to be checked, because by definition both old and new elements - // will have the same key. if (existing != null && !isNatural) { byte[] oldPrefix = null; Object oldIndexedValue = getValue(existing); diff --git a/common/kvstore/src/test/java/org/apache/spark/kvstore/DBIteratorSuite.java b/common/kvstore/src/test/java/org/apache/spark/kvstore/DBIteratorSuite.java index 2b15e6e112345..097f6618d0d69 100644 --- a/common/kvstore/src/test/java/org/apache/spark/kvstore/DBIteratorSuite.java +++ b/common/kvstore/src/test/java/org/apache/spark/kvstore/DBIteratorSuite.java @@ -96,27 +96,21 @@ public void setup() throws Exception { int count = RND.nextInt(MAX_ENTRIES) + MIN_ENTRIES; - // Instead of generating sequential IDs, generate random unique IDs to avoid the insertion - // order matching the natural ordering. Just in case. - boolean[] usedIDs = new boolean[count]; - allEntries = new ArrayList<>(count); for (int i = 0; i < count; i++) { CustomType1 t = new CustomType1(); - - int id; - do { - id = RND.nextInt(count); - } while (usedIDs[id]); - - usedIDs[id] = true; - t.key = "key" + id; + t.key = "key" + i; t.id = "id" + i; t.name = "name" + RND.nextInt(MAX_ENTRIES); t.num = RND.nextInt(MAX_ENTRIES); t.child = "child" + (i % MIN_ENTRIES); allEntries.add(t); - db.write(t); + } + + // Shuffle the entries to avoid the insertion order matching the natural ordering. Just in case. + Collections.shuffle(allEntries, RND); + for (CustomType1 e : allEntries) { + db.write(e); } // Pick the first generated value, and forcefully create a few entries that will clash diff --git a/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBTypeInfoSuite.java b/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBTypeInfoSuite.java index 18b00d6f6f577..8e6196506c6a8 100644 --- a/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBTypeInfoSuite.java +++ b/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBTypeInfoSuite.java @@ -48,6 +48,11 @@ public void testNoNaturalIndex() throws Exception { newTypeInfo(NoNaturalIndex.class); } + @Test(expected = IllegalArgumentException.class) + public void testNoNaturalIndex2() throws Exception { + newTypeInfo(NoNaturalIndex2.class); + } + @Test(expected = IllegalArgumentException.class) public void testDuplicateIndex() throws Exception { newTypeInfo(DuplicateIndex.class); @@ -156,8 +161,18 @@ public static class NoNaturalIndex { } + public static class NoNaturalIndex2 { + + @KVIndex("id") + public String id; + + } + public static class DuplicateIndex { + @KVIndex + public String key; + @KVIndex("id") public String id; From 53d7d57196317ef406875f73d6e806f3b7956ee7 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Mon, 5 Jun 2017 10:45:27 -0700 Subject: [PATCH 18/18] SHS-NG M1: Log the test's random seed. --- .../org/apache/spark/kvstore/DBIteratorSuite.java | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/common/kvstore/src/test/java/org/apache/spark/kvstore/DBIteratorSuite.java b/common/kvstore/src/test/java/org/apache/spark/kvstore/DBIteratorSuite.java index 097f6618d0d69..8549712213393 100644 --- a/common/kvstore/src/test/java/org/apache/spark/kvstore/DBIteratorSuite.java +++ b/common/kvstore/src/test/java/org/apache/spark/kvstore/DBIteratorSuite.java @@ -32,11 +32,16 @@ import org.apache.commons.io.FileUtils; import org.junit.AfterClass; import org.junit.Before; +import org.junit.BeforeClass; import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import static org.junit.Assert.*; public abstract class DBIteratorSuite { + private static final Logger LOG = LoggerFactory.getLogger(DBIteratorSuite.class); + private static final int MIN_ENTRIES = 42; private static final int MAX_ENTRIES = 1024; private static final Random RND = new Random(); @@ -80,6 +85,13 @@ default BaseComparator reverse() { */ protected abstract KVStore createStore() throws Exception; + @BeforeClass + public static void setupClass() { + long seed = RND.nextLong(); + LOG.info("Random seed: {}", seed); + RND.setSeed(seed); + } + @AfterClass public static void cleanupData() throws Exception { allEntries = null;