diff --git a/janusgraph-berkeleyje/src/main/java/org/janusgraph/diskstorage/berkeleyje/BerkeleyJEKeyValueStore.java b/janusgraph-berkeleyje/src/main/java/org/janusgraph/diskstorage/berkeleyje/BerkeleyJEKeyValueStore.java index d6caacdb98..9df3d9b802 100644 --- a/janusgraph-berkeleyje/src/main/java/org/janusgraph/diskstorage/berkeleyje/BerkeleyJEKeyValueStore.java +++ b/janusgraph-berkeleyje/src/main/java/org/janusgraph/diskstorage/berkeleyje/BerkeleyJEKeyValueStore.java @@ -27,14 +27,11 @@ import com.sleepycat.je.OperationStatus; import com.sleepycat.je.Put; import com.sleepycat.je.ReadOptions; -import com.sleepycat.je.ThreadInterruptedException; import com.sleepycat.je.Transaction; import com.sleepycat.je.WriteOptions; -import org.apache.tinkerpop.gremlin.process.traversal.util.TraversalInterruptedException; import org.janusgraph.diskstorage.BackendException; import org.janusgraph.diskstorage.PermanentBackendException; import org.janusgraph.diskstorage.StaticBuffer; -import org.janusgraph.diskstorage.TemporaryBackendException; import org.janusgraph.diskstorage.keycolumnvalue.StoreTransaction; import org.janusgraph.diskstorage.keycolumnvalue.keyvalue.KVQuery; import org.janusgraph.diskstorage.keycolumnvalue.keyvalue.KeySelector; @@ -63,10 +60,10 @@ public class BerkeleyJEKeyValueStore implements OrderedKeyValueStore { public static Function ttlConverter = ttl -> (int) Math.max(1, Duration.of(ttl, ChronoUnit.SECONDS).toHours()); - private volatile Database db; + private final Database db; private final String name; private final BerkeleyJEStoreManager manager; - private volatile boolean isOpen; + private boolean isOpen; public BerkeleyJEKeyValueStore(String n, Database data, BerkeleyJEStoreManager m) { db = data; @@ -78,11 +75,6 @@ public BerkeleyJEKeyValueStore(String n, Database data, BerkeleyJEStoreManager m public DatabaseConfig getConfiguration() throws BackendException { try { return db.getConfig(); - } catch (ThreadInterruptedException e) { - Thread.currentThread().interrupt(); - throw (TraversalInterruptedException) new TraversalInterruptedException().initCause(e); - } catch (EnvironmentFailureException e) { - throw new TemporaryBackendException(e); } catch (DatabaseException e) { throw new PermanentBackendException(e); } @@ -103,24 +95,15 @@ private Cursor openCursor(StoreTransaction txh) throws BackendException { return ((BerkeleyJETx) txh).openCursor(db); } - private static void closeCursor(StoreTransaction txh, Cursor cursor) throws BackendException { + private static void closeCursor(StoreTransaction txh, Cursor cursor) { Preconditions.checkArgument(txh!=null); ((BerkeleyJETx) txh).closeCursor(cursor); } - public void reopen(final Database db) { - this.db = db; - } - @Override public synchronized void close() throws BackendException { try { if(isOpen) db.close(); - } catch (ThreadInterruptedException e) { - Thread.currentThread().interrupt(); - throw (TraversalInterruptedException) new TraversalInterruptedException().initCause(e); - } catch (EnvironmentFailureException e) { - throw new TemporaryBackendException(e); } catch (DatabaseException e) { throw new PermanentBackendException(e); } @@ -144,11 +127,6 @@ public StaticBuffer get(StaticBuffer key, StoreTransaction txh) throws BackendEx } else { return null; } - } catch (ThreadInterruptedException e) { - Thread.currentThread().interrupt(); - throw (TraversalInterruptedException) new TraversalInterruptedException().initCause(e); - } catch (EnvironmentFailureException e) { - throw new TemporaryBackendException(e); } catch (DatabaseException e) { throw new PermanentBackendException(e); } @@ -183,11 +161,7 @@ public RecordIterator getSlice(KVQuery query, StoreTransaction tx @Override public boolean hasNext() { if (current == null) { - try { - current = getNextEntry(); - } catch (BackendException e) { - throw new RuntimeException(e); - } + current = getNextEntry(); } return current != null; } @@ -202,26 +176,16 @@ public KeyValueEntry next() { return next; } - private KeyValueEntry getNextEntry() throws BackendException { + private KeyValueEntry getNextEntry() { if (status != null && status != OperationStatus.SUCCESS) { return null; } while (!selector.reachedLimit()) { - try { - if (status == null) { - status = cursor.get(foundKey, foundData, Get.SEARCH_GTE, getReadOptions(txh)) == null ? OperationStatus.NOTFOUND : OperationStatus.SUCCESS; - } else { - status = cursor.get(foundKey, foundData, Get.NEXT, getReadOptions(txh)) == null ? OperationStatus.NOTFOUND : OperationStatus.SUCCESS; - } - } catch (ThreadInterruptedException e) { - Thread.currentThread().interrupt(); - throw (TraversalInterruptedException) new TraversalInterruptedException().initCause(e); - } catch (EnvironmentFailureException e) { - throw new TemporaryBackendException(e); - } catch (DatabaseException e) { - throw new PermanentBackendException(e); + if (status == null) { + status = cursor.get(foundKey, foundData, Get.SEARCH_GTE, getReadOptions(txh)) == null ? OperationStatus.NOTFOUND : OperationStatus.SUCCESS; + } else { + status = cursor.get(foundKey, foundData, Get.NEXT, getReadOptions(txh)) == null ? OperationStatus.NOTFOUND : OperationStatus.SUCCESS; } - if (status != OperationStatus.SUCCESS) { break; } @@ -241,11 +205,7 @@ private KeyValueEntry getNextEntry() throws BackendException { @Override public void close() { - try { - closeCursor(txh, cursor); - } catch (BackendException e) { - throw new RuntimeException(e); - } + closeCursor(txh, cursor); } @Override @@ -277,22 +237,13 @@ public void insert(StaticBuffer key, StaticBuffer value, StoreTransaction txh, b int convertedTtl = ttlConverter.apply(ttl); writeOptions.setTTL(convertedTtl, TimeUnit.HOURS); } - try { - if (allowOverwrite) { - OperationResult result = db.put(tx, key.as(ENTRY_FACTORY), value.as(ENTRY_FACTORY), Put.OVERWRITE, writeOptions); - EnvironmentFailureException.assertState(result != null); - status = OperationStatus.SUCCESS; - } else { - OperationResult result = db.put(tx, key.as(ENTRY_FACTORY), value.as(ENTRY_FACTORY), Put.NO_OVERWRITE, writeOptions); - status = result == null ? OperationStatus.KEYEXIST : OperationStatus.SUCCESS; - } - } catch (ThreadInterruptedException e) { - Thread.currentThread().interrupt(); - throw (TraversalInterruptedException) new TraversalInterruptedException().initCause(e); - } catch (EnvironmentFailureException e) { - throw new TemporaryBackendException(e); - } catch (DatabaseException e) { - throw new PermanentBackendException(e); + if (allowOverwrite) { + OperationResult result = db.put(tx, key.as(ENTRY_FACTORY), value.as(ENTRY_FACTORY), Put.OVERWRITE, writeOptions); + EnvironmentFailureException.assertState(result != null); + status = OperationStatus.SUCCESS; + } else { + OperationResult result = db.put(tx, key.as(ENTRY_FACTORY), value.as(ENTRY_FACTORY), Put.NO_OVERWRITE, writeOptions); + status = result == null ? OperationStatus.KEYEXIST : OperationStatus.SUCCESS; } if (status != OperationStatus.SUCCESS) { @@ -310,11 +261,6 @@ public void delete(StaticBuffer key, StoreTransaction txh) throws BackendExcepti if (status != OperationStatus.SUCCESS && status != OperationStatus.NOTFOUND) { throw new PermanentBackendException("Could not remove: " + status); } - } catch (ThreadInterruptedException e) { - Thread.currentThread().interrupt(); - throw (TraversalInterruptedException) new TraversalInterruptedException().initCause(e); - } catch (EnvironmentFailureException e) { - throw new TemporaryBackendException(e); } catch (DatabaseException e) { throw new PermanentBackendException(e); } diff --git a/janusgraph-berkeleyje/src/main/java/org/janusgraph/diskstorage/berkeleyje/BerkeleyJEStoreManager.java b/janusgraph-berkeleyje/src/main/java/org/janusgraph/diskstorage/berkeleyje/BerkeleyJEStoreManager.java index 55d405adff..632d37914e 100644 --- a/janusgraph-berkeleyje/src/main/java/org/janusgraph/diskstorage/berkeleyje/BerkeleyJEStoreManager.java +++ b/janusgraph-berkeleyje/src/main/java/org/janusgraph/diskstorage/berkeleyje/BerkeleyJEStoreManager.java @@ -22,17 +22,13 @@ import com.sleepycat.je.DatabaseException; import com.sleepycat.je.Environment; import com.sleepycat.je.EnvironmentConfig; -import com.sleepycat.je.EnvironmentFailureException; import com.sleepycat.je.LockMode; -import com.sleepycat.je.ThreadInterruptedException; import com.sleepycat.je.Transaction; import com.sleepycat.je.TransactionConfig; -import org.apache.tinkerpop.gremlin.process.traversal.util.TraversalInterruptedException; import org.janusgraph.diskstorage.BackendException; import org.janusgraph.diskstorage.BaseTransactionConfig; import org.janusgraph.diskstorage.PermanentBackendException; import org.janusgraph.diskstorage.StaticBuffer; -import org.janusgraph.diskstorage.TemporaryBackendException; import org.janusgraph.diskstorage.common.LocalStoreManager; import org.janusgraph.diskstorage.configuration.ConfigNamespace; import org.janusgraph.diskstorage.configuration.ConfigOption; @@ -52,10 +48,9 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; import static org.janusgraph.diskstorage.configuration.ConfigOption.disallowEmpty; @@ -93,16 +88,19 @@ public class BerkeleyJEStoreManager extends LocalStoreManager implements Ordered ConfigOption.Type.MASKABLE, String.class, IsolationLevel.REPEATABLE_READ.toString(), disallowEmpty(String.class)); - private final ConcurrentMap stores; + private final Map stores; - protected volatile Environment environment; + protected Environment environment; protected final StoreFeatures features; public BerkeleyJEStoreManager(Configuration configuration) throws BackendException { super(configuration); - stores = new ConcurrentHashMap<>(); + stores = new HashMap<>(); - initialize(); + int cachePercentage = configuration.get(JVM_CACHE); + boolean sharedCache = configuration.get(SHARED_CACHE); + CacheMode cacheMode = ConfigOption.getEnumValue(configuration.get(CACHE_MODE), CacheMode.class); + initialize(cachePercentage, sharedCache, cacheMode); features = new StandardStoreFeatures.Builder() .orderedScan(true) @@ -113,24 +111,14 @@ public BerkeleyJEStoreManager(Configuration configuration) throws BackendExcepti .scanTxConfig(GraphDatabaseConfiguration.buildGraphConfiguration() .set(ISOLATION_LEVEL, IsolationLevel.READ_UNCOMMITTED.toString()) ) - .supportsInterruption(true) + .supportsInterruption(false) .cellTTL(true) .optimisticLocking(false) .build(); } - private synchronized void initialize() throws BackendException { + private void initialize(int cachePercent, final boolean sharedCache, final CacheMode cacheMode) throws BackendException { try { - if (environment != null && environment.isValid()) { - return; - } - - close(true); - - int cachePercent = storageConfig.get(JVM_CACHE); - boolean sharedCache = storageConfig.get(SHARED_CACHE); - CacheMode cacheMode = ConfigOption.getEnumValue(storageConfig.get(CACHE_MODE), CacheMode.class); - EnvironmentConfig envConfig = new EnvironmentConfig(); envConfig.setAllowCreate(true); envConfig.setTransactional(transactional); @@ -143,28 +131,15 @@ private synchronized void initialize() throws BackendException { envConfig.setConfigParam(EnvironmentConfig.ENV_RUN_CLEANER, "false"); } - // Open the environment + //Open the environment environment = new Environment(directory, envConfig); - // Reopen any existing DB connections - for (String storeName : stores.keySet()) { - openDatabase(storeName, true); - } } catch (DatabaseException e) { throw new PermanentBackendException("Error during BerkeleyJE initialization: ", e); } } - private synchronized void reInitialize(DatabaseException exception) throws BackendException { - initialize(); - - if (exception instanceof ThreadInterruptedException) { - Thread.currentThread().interrupt(); - throw (TraversalInterruptedException) new TraversalInterruptedException().initCause(exception); - } - } - @Override public StoreFeatures getFeatures() { return features; @@ -175,7 +150,8 @@ public List getLocalKeyPartition() throws BackendException { throw new UnsupportedOperationException(); } - private BerkeleyJETx beginTransaction(final BaseTransactionConfig txCfg, boolean retryEnvironmentFailure) throws BackendException { + @Override + public BerkeleyJETx beginTransaction(final BaseTransactionConfig txCfg) throws BackendException { try { Transaction tx = null; @@ -206,27 +182,15 @@ private BerkeleyJETx beginTransaction(final BaseTransactionConfig txCfg, boolean } return btx; - } catch (EnvironmentFailureException e) { - reInitialize(e); - - if (retryEnvironmentFailure) { - return beginTransaction(txCfg, false); - } - - throw new TemporaryBackendException("Could not start BerkeleyJE transaction", e); } catch (DatabaseException e) { throw new PermanentBackendException("Could not start BerkeleyJE transaction", e); } } @Override - public BerkeleyJETx beginTransaction(final BaseTransactionConfig txCfg) throws BackendException { - return beginTransaction(txCfg, true); - } - - private BerkeleyJEKeyValueStore openDatabase(String name, boolean force, boolean retryEnvironmentFailure) throws BackendException { + public BerkeleyJEKeyValueStore openDatabase(String name) throws BackendException { Preconditions.checkNotNull(name); - if (stores.containsKey(name) && !force) { + if (stores.containsKey(name)) { return stores.get(name); } try { @@ -245,34 +209,13 @@ private BerkeleyJEKeyValueStore openDatabase(String name, boolean force, boolean log.debug("Opened database {}", name); BerkeleyJEKeyValueStore store = new BerkeleyJEKeyValueStore(name, db, this); - if (stores.containsKey(name)) { - stores.get(name).reopen(db); - } else { - stores.put(name, store); - } + stores.put(name, store); return store; - } catch (EnvironmentFailureException e) { - reInitialize(e); - - if (retryEnvironmentFailure) { - return openDatabase(name, force, false); - } - - throw new TemporaryBackendException("Could not open BerkeleyJE data store", e); } catch (DatabaseException e) { throw new PermanentBackendException("Could not open BerkeleyJE data store", e); } } - private BerkeleyJEKeyValueStore openDatabase(String name, boolean force) throws BackendException { - return openDatabase(name, force, true); - } - - @Override - public BerkeleyJEKeyValueStore openDatabase(String name) throws BackendException { - return openDatabase(name, false, true); - } - @Override public void mutateMany(Map mutations, StoreTransaction txh) throws BackendException { for (Map.Entry mutation : mutations.entrySet()) { @@ -309,16 +252,18 @@ void removeDatabase(BerkeleyJEKeyValueStore db) { log.debug("Removed database {}", name); } - public void close(boolean force) throws BackendException { + + @Override + public void close() throws BackendException { if (environment != null) { - if (!force && !stores.isEmpty()) + if (!stores.isEmpty()) throw new IllegalStateException("Cannot shutdown manager since some databases are still open"); try { // TODO this looks like a race condition //Wait just a little bit before closing so that independent transaction threads can clean up. Thread.sleep(30); } catch (InterruptedException e) { - Thread.currentThread().interrupt(); + //Ignore } try { environment.close(); @@ -329,11 +274,6 @@ public void close(boolean force) throws BackendException { } - @Override - public void close() throws BackendException { - close(false); - } - private static final Transaction NULL_TRANSACTION = null; @Override diff --git a/janusgraph-berkeleyje/src/main/java/org/janusgraph/diskstorage/berkeleyje/BerkeleyJETx.java b/janusgraph-berkeleyje/src/main/java/org/janusgraph/diskstorage/berkeleyje/BerkeleyJETx.java index 0da80e3863..43387f6e70 100644 --- a/janusgraph-berkeleyje/src/main/java/org/janusgraph/diskstorage/berkeleyje/BerkeleyJETx.java +++ b/janusgraph-berkeleyje/src/main/java/org/janusgraph/diskstorage/berkeleyje/BerkeleyJETx.java @@ -19,15 +19,11 @@ import com.sleepycat.je.Cursor; import com.sleepycat.je.Database; import com.sleepycat.je.DatabaseException; -import com.sleepycat.je.EnvironmentFailureException; import com.sleepycat.je.LockMode; -import com.sleepycat.je.ThreadInterruptedException; import com.sleepycat.je.Transaction; -import org.apache.tinkerpop.gremlin.process.traversal.util.TraversalInterruptedException; import org.janusgraph.diskstorage.BackendException; import org.janusgraph.diskstorage.BaseTransactionConfig; import org.janusgraph.diskstorage.PermanentBackendException; -import org.janusgraph.diskstorage.TemporaryBackendException; import org.janusgraph.diskstorage.common.AbstractStoreTransaction; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -64,36 +60,16 @@ Cursor openCursor(Database db) throws BackendException { if (!isOpen) { throw new PermanentBackendException("Transaction already closed"); } - - try { - Cursor cursor = db.openCursor(tx, null); - openCursors.add(cursor); - return cursor; - } catch (ThreadInterruptedException e) { - Thread.currentThread().interrupt(); - throw (TraversalInterruptedException) new TraversalInterruptedException().initCause(e); - } catch (EnvironmentFailureException e) { - throw new TemporaryBackendException(e); - } catch (DatabaseException e) { - throw new PermanentBackendException(e); - } + Cursor cursor = db.openCursor(tx, null); + openCursors.add(cursor); + return cursor; } } - void closeCursor(Cursor cursor) throws BackendException { + void closeCursor(Cursor cursor) { synchronized (openCursors) { - try { - cursor.close(); - } catch (ThreadInterruptedException e) { - Thread.currentThread().interrupt(); - throw (TraversalInterruptedException) new TraversalInterruptedException().initCause(e); - } catch (EnvironmentFailureException e) { - throw new TemporaryBackendException(e); - } catch (DatabaseException e) { - throw new PermanentBackendException(e); - } finally { - openCursors.remove(cursor); - } + cursor.close(); + openCursors.remove(cursor); } } @@ -122,15 +98,6 @@ public synchronized void rollback() throws BackendException { closeOpenCursors(); tx.abort(); tx = null; - } catch (ThreadInterruptedException e) { - Thread.currentThread().interrupt(); - } catch (EnvironmentFailureException e) { - // Ignore to avoid issues when backend is invalid - } catch (IllegalStateException e) { - // Ignore to avoid issues when backend was closed - if (!e.getMessage().equals("Database was closed.") && !e.getMessage().equals("Environment is closed.")) { - throw e; - } } catch (DatabaseException e) { throw new PermanentBackendException(e); } @@ -147,11 +114,6 @@ public synchronized void commit() throws BackendException { closeOpenCursors(); tx.commit(); tx = null; - } catch (ThreadInterruptedException e) { - Thread.currentThread().interrupt(); - throw (TraversalInterruptedException) new TraversalInterruptedException().initCause(e); - } catch (EnvironmentFailureException e) { - throw new TemporaryBackendException(e); } catch (DatabaseException e) { throw new PermanentBackendException(e); } diff --git a/janusgraph-berkeleyje/src/test/java/org/janusgraph/BerkeleyInterruptionTest.java b/janusgraph-berkeleyje/src/test/java/org/janusgraph/BerkeleyInterruptionTest.java deleted file mode 100644 index f2157fdc87..0000000000 --- a/janusgraph-berkeleyje/src/test/java/org/janusgraph/BerkeleyInterruptionTest.java +++ /dev/null @@ -1,57 +0,0 @@ -// Copyright 2022 JanusGraph Authors -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. -package org.janusgraph; - -import org.apache.tinkerpop.gremlin.process.traversal.dsl.graph.GraphTraversalSource; -import org.apache.tinkerpop.gremlin.process.traversal.util.TraversalInterruptedException; -import org.apache.tinkerpop.gremlin.structure.Transaction; -import org.janusgraph.core.JanusGraph; -import org.janusgraph.core.JanusGraphException; -import org.janusgraph.core.JanusGraphFactory; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.io.TempDir; - -import java.io.File; - -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertThrows; - -public class BerkeleyInterruptionTest { - - @Test - public void interruptedEnvironmentShouldBeRestarted(@TempDir File dir) { - try (JanusGraph graph = JanusGraphFactory.open("berkeleyje:" + dir.getAbsolutePath())) { - assertThrows(JanusGraphException.class, () -> { - Transaction tx = graph.tx(); - GraphTraversalSource gtx = tx.begin(); - - gtx.addV().iterate(); - - Thread.currentThread().interrupt(); - tx.commit(); - }); - - // Retry until BerkeleyJE DB environment is reopened - while (true) { - try { - graph.traversal().addV().iterate(); - break; - } catch (TraversalInterruptedException ignored) { - } - } - - assertEquals(1, graph.traversal().V().count().next()); - } - } -} diff --git a/janusgraph-berkeleyje/src/test/java/org/janusgraph/blueprints/process/BerkeleyJanusGraphComputerTest.java b/janusgraph-berkeleyje/src/test/java/org/janusgraph/blueprints/process/BerkeleyJanusGraphComputerTest.java index 9845ca484f..5e406cda19 100644 --- a/janusgraph-berkeleyje/src/test/java/org/janusgraph/blueprints/process/BerkeleyJanusGraphComputerTest.java +++ b/janusgraph-berkeleyje/src/test/java/org/janusgraph/blueprints/process/BerkeleyJanusGraphComputerTest.java @@ -15,7 +15,6 @@ package org.janusgraph.blueprints.process; import org.apache.tinkerpop.gremlin.GraphProviderClass; -import org.apache.tinkerpop.gremlin.process.ProcessComputerSuite; import org.janusgraph.blueprints.BerkeleyGraphComputerProvider; import org.janusgraph.core.JanusGraph; import org.junit.runner.RunWith; @@ -23,7 +22,7 @@ /** * @author Matthias Broecheler (me@matthiasb.com) */ -@RunWith(ProcessComputerSuite.class) +@RunWith(BerkeleyProcessComputerSuite.class) @GraphProviderClass(provider = BerkeleyGraphComputerProvider.class, graph = JanusGraph.class) public class BerkeleyJanusGraphComputerTest { } diff --git a/janusgraph-berkeleyje/src/test/java/org/janusgraph/blueprints/process/BerkeleyJanusGraphProcessTest.java b/janusgraph-berkeleyje/src/test/java/org/janusgraph/blueprints/process/BerkeleyJanusGraphProcessTest.java index 0c80a3079a..46a05ea15b 100644 --- a/janusgraph-berkeleyje/src/test/java/org/janusgraph/blueprints/process/BerkeleyJanusGraphProcessTest.java +++ b/janusgraph-berkeleyje/src/test/java/org/janusgraph/blueprints/process/BerkeleyJanusGraphProcessTest.java @@ -15,7 +15,6 @@ package org.janusgraph.blueprints.process; import org.apache.tinkerpop.gremlin.GraphProviderClass; -import org.apache.tinkerpop.gremlin.process.ProcessStandardSuite; import org.janusgraph.blueprints.BerkeleyGraphProvider; import org.janusgraph.core.JanusGraph; import org.junit.runner.RunWith; @@ -23,7 +22,7 @@ /** * @author Matthias Broecheler (me@matthiasb.com) */ -@RunWith(ProcessStandardSuite.class) +@RunWith(BerkeleyProcessStandardSuite.class) @GraphProviderClass(provider = BerkeleyGraphProvider.class, graph = JanusGraph.class) public class BerkeleyJanusGraphProcessTest { } diff --git a/janusgraph-berkeleyje/src/test/java/org/janusgraph/blueprints/process/BerkeleyMultiQueryJanusGraphProcessTest.java b/janusgraph-berkeleyje/src/test/java/org/janusgraph/blueprints/process/BerkeleyMultiQueryJanusGraphProcessTest.java index 81f00e9294..b9b8507abe 100644 --- a/janusgraph-berkeleyje/src/test/java/org/janusgraph/blueprints/process/BerkeleyMultiQueryJanusGraphProcessTest.java +++ b/janusgraph-berkeleyje/src/test/java/org/janusgraph/blueprints/process/BerkeleyMultiQueryJanusGraphProcessTest.java @@ -15,7 +15,6 @@ package org.janusgraph.blueprints.process; import org.apache.tinkerpop.gremlin.GraphProviderClass; -import org.apache.tinkerpop.gremlin.process.ProcessStandardSuite; import org.janusgraph.blueprints.BerkeleyMultiQueryGraphProvider; import org.janusgraph.core.JanusGraph; import org.junit.runner.RunWith; @@ -23,7 +22,7 @@ /** * @author Ted Wilmes (twilmes@gmail.com) */ -@RunWith(ProcessStandardSuite.class) +@RunWith(BerkeleyProcessStandardSuite.class) @GraphProviderClass(provider = BerkeleyMultiQueryGraphProvider.class, graph = JanusGraph.class) public class BerkeleyMultiQueryJanusGraphProcessTest { } diff --git a/janusgraph-berkeleyje/src/test/java/org/janusgraph/blueprints/process/BerkeleyProcessComputerSuite.java b/janusgraph-berkeleyje/src/test/java/org/janusgraph/blueprints/process/BerkeleyProcessComputerSuite.java new file mode 100644 index 0000000000..632481024a --- /dev/null +++ b/janusgraph-berkeleyje/src/test/java/org/janusgraph/blueprints/process/BerkeleyProcessComputerSuite.java @@ -0,0 +1,44 @@ +// Copyright 2019 JanusGraph Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package org.janusgraph.blueprints.process; + +import org.apache.commons.lang3.ArrayUtils; +import org.apache.tinkerpop.gremlin.process.ProcessComputerSuite; +import org.apache.tinkerpop.gremlin.process.traversal.TraversalInterruptionComputerTest; +import org.junit.runners.model.InitializationError; +import org.junit.runners.model.RunnerBuilder; + +import java.lang.reflect.Field; + +/** + * Custom TinkerPop {@link ProcessComputerSuite} that excludes {@link TraversalInterruptionComputerTest} for compatibility with + * BerkeleyDB JE, which does not support thread interrupts. + */ +public class BerkeleyProcessComputerSuite extends ProcessComputerSuite { + + public BerkeleyProcessComputerSuite(final Class classToTest, final RunnerBuilder builder) throws InitializationError { + super(classToTest, builder, getTestList()); + } + + private static Class[] getTestList() throws InitializationError { + try { + final Field field = ProcessComputerSuite.class.getDeclaredField("allTests"); + field.setAccessible(true); + return (Class[]) ArrayUtils.removeElement((Class[]) field.get(null), TraversalInterruptionComputerTest.class); + } catch (ReflectiveOperationException e) { + throw new InitializationError("Unable to create test list"); + } + } +} diff --git a/janusgraph-berkeleyje/src/test/java/org/janusgraph/blueprints/process/BerkeleyProcessStandardSuite.java b/janusgraph-berkeleyje/src/test/java/org/janusgraph/blueprints/process/BerkeleyProcessStandardSuite.java new file mode 100644 index 0000000000..554887de5e --- /dev/null +++ b/janusgraph-berkeleyje/src/test/java/org/janusgraph/blueprints/process/BerkeleyProcessStandardSuite.java @@ -0,0 +1,44 @@ +// Copyright 2017 JanusGraph Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package org.janusgraph.blueprints.process; + +import org.apache.commons.lang3.ArrayUtils; +import org.apache.tinkerpop.gremlin.process.ProcessStandardSuite; +import org.apache.tinkerpop.gremlin.process.traversal.TraversalInterruptionTest; +import org.junit.runners.model.InitializationError; +import org.junit.runners.model.RunnerBuilder; + +import java.lang.reflect.Field; + +/** + * Custom TinkerPop {@link ProcessStandardSuite} that excludes {@link TraversalInterruptionTest} for compatibility with + * BerkeleyDB JE, which does not support thread interrupts. + */ +public class BerkeleyProcessStandardSuite extends ProcessStandardSuite { + + public BerkeleyProcessStandardSuite(final Class classToTest, final RunnerBuilder builder) throws InitializationError { + super(classToTest, builder, getTestList()); + } + + private static Class[] getTestList() throws InitializationError { + try { + final Field field = ProcessStandardSuite.class.getDeclaredField("allTests"); + field.setAccessible(true); + return (Class[]) ArrayUtils.removeElement((Class[]) field.get(null), TraversalInterruptionTest.class); + } catch (ReflectiveOperationException e) { + throw new InitializationError("Unable to create test list"); + } + } +}