From 5d905dee569a6aa08312e33db2440f311fc99396 Mon Sep 17 00:00:00 2001 From: JackyYangPassion Date: Tue, 14 Dec 2021 23:50:13 +0800 Subject: [PATCH 1/3] add HBase pre-split table feature --- .../store/cassandra/CassandraSerializer.java | 5 + .../cassandra/CassandraStoreProvider.java | 5 +- .../baidu/hugegraph/StandardHugeGraph.java | 8 +- .../serializer/AbstractSerializer.java | 9 ++ .../serializer/BinaryBackendEntry.java | 6 +- .../serializer/BinaryScatterSerializer.java | 5 +- .../backend/serializer/BinarySerializer.java | 112 ++++++++++++++---- .../backend/serializer/BytesBuffer.java | 6 +- .../backend/serializer/SerializerFactory.java | 13 +- .../backend/serializer/TableSerializer.java | 5 + .../backend/serializer/TextSerializer.java | 5 + .../store/AbstractBackendStoreProvider.java | 19 +-- .../backend/store/BackendStoreProvider.java | 7 +- .../store/memory/InMemoryDBStoreProvider.java | 5 +- .../store/raft/RaftBackendStoreProvider.java | 13 +- .../hugegraph/backend/store/ram/RamTable.java | 2 + .../static/conf/graphs/hugegraph.properties | 6 + .../backend/store/hbase/HbaseOptions.java | 25 ++++ .../backend/store/hbase/HbaseSerializer.java | 29 ++++- .../backend/store/hbase/HbaseSessions.java | 17 +++ .../backend/store/hbase/HbaseStore.java | 25 ++-- .../store/hbase/HbaseStoreProvider.java | 7 +- .../backend/store/hbase/HbaseTable.java | 15 ++- .../backend/store/hbase/HbaseTables.java | 28 +++-- .../backend/store/mysql/MysqlSerializer.java | 5 + .../store/mysql/MysqlStoreProvider.java | 5 +- .../backend/store/palo/PaloSerializer.java | 5 + .../backend/store/palo/PaloStoreProvider.java | 5 +- .../postgresql/PostgresqlSerializer.java | 5 + .../postgresql/PostgresqlStoreProvider.java | 5 +- .../store/rocksdb/RocksDBStoreProvider.java | 4 +- .../rocksdbsst/RocksDBSstStoreProvider.java | 3 +- .../store/scylladb/ScyllaDBStoreProvider.java | 4 +- .../com/baidu/hugegraph/unit/FakeObjects.java | 43 +++++++ .../BinaryScatterSerializerTest.java | 7 +- .../unit/serializer/BinarySerializerTest.java | 55 ++++++++- .../serializer/SerializerFactoryTest.java | 21 +++- 37 files changed, 436 insertions(+), 108 deletions(-) diff --git a/hugegraph-cassandra/src/main/java/com/baidu/hugegraph/backend/store/cassandra/CassandraSerializer.java b/hugegraph-cassandra/src/main/java/com/baidu/hugegraph/backend/store/cassandra/CassandraSerializer.java index 7cbb6fd541..85d8496dfd 100644 --- a/hugegraph-cassandra/src/main/java/com/baidu/hugegraph/backend/store/cassandra/CassandraSerializer.java +++ b/hugegraph-cassandra/src/main/java/com/baidu/hugegraph/backend/store/cassandra/CassandraSerializer.java @@ -34,6 +34,7 @@ import com.baidu.hugegraph.backend.serializer.TableBackendEntry; import com.baidu.hugegraph.backend.serializer.TableSerializer; import com.baidu.hugegraph.backend.store.BackendEntry; +import com.baidu.hugegraph.config.HugeConfig; import com.baidu.hugegraph.schema.PropertyKey; import com.baidu.hugegraph.schema.SchemaElement; import com.baidu.hugegraph.structure.HugeElement; @@ -51,6 +52,10 @@ public class CassandraSerializer extends TableSerializer { + public CassandraSerializer(HugeConfig config) { + super(config); + } + @Override public CassandraBackendEntry newBackendEntry(HugeType type, Id id) { return new CassandraBackendEntry(type, id); diff --git a/hugegraph-cassandra/src/main/java/com/baidu/hugegraph/backend/store/cassandra/CassandraStoreProvider.java b/hugegraph-cassandra/src/main/java/com/baidu/hugegraph/backend/store/cassandra/CassandraStoreProvider.java index feb8068643..da98fa602d 100644 --- a/hugegraph-cassandra/src/main/java/com/baidu/hugegraph/backend/store/cassandra/CassandraStoreProvider.java +++ b/hugegraph-cassandra/src/main/java/com/baidu/hugegraph/backend/store/cassandra/CassandraStoreProvider.java @@ -23,6 +23,7 @@ import com.baidu.hugegraph.backend.store.BackendStore; import com.baidu.hugegraph.backend.store.cassandra.CassandraStore.CassandraGraphStore; import com.baidu.hugegraph.backend.store.cassandra.CassandraStore.CassandraSchemaStore; +import com.baidu.hugegraph.config.HugeConfig; public class CassandraStoreProvider extends AbstractBackendStoreProvider { @@ -31,12 +32,12 @@ protected String keyspace() { } @Override - protected BackendStore newSchemaStore(String store) { + protected BackendStore newSchemaStore(HugeConfig config, String store) { return new CassandraSchemaStore(this, this.keyspace(), store); } @Override - protected BackendStore newGraphStore(String store) { + protected BackendStore newGraphStore(HugeConfig config, String store) { return new CassandraGraphStore(this, this.keyspace(), store); } diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/StandardHugeGraph.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/StandardHugeGraph.java index fe548149e9..1336fd98d4 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/StandardHugeGraph.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/StandardHugeGraph.java @@ -447,17 +447,17 @@ private void checkGraphNotClosed() { private BackendStore loadSchemaStore() { String name = this.configuration.get(CoreOptions.STORE_SCHEMA); - return this.storeProvider.loadSchemaStore(name); + return this.storeProvider.loadSchemaStore(this.configuration, name); } private BackendStore loadGraphStore() { String name = this.configuration.get(CoreOptions.STORE_GRAPH); - return this.storeProvider.loadGraphStore(name); + return this.storeProvider.loadGraphStore(this.configuration, name); } private BackendStore loadSystemStore() { String name = this.configuration.get(CoreOptions.STORE_SYSTEM); - return this.storeProvider.loadSystemStore(name); + return this.storeProvider.loadSystemStore(this.configuration, name); } @Watched @@ -498,7 +498,7 @@ private BackendStoreProvider loadStoreProvider() { private AbstractSerializer serializer() { String name = this.configuration.get(CoreOptions.SERIALIZER); LOG.debug("Loading serializer '{}' for graph '{}'", name, this.name); - AbstractSerializer serializer = SerializerFactory.serializer(name); + AbstractSerializer serializer = SerializerFactory.serializer(this.configuration, name); if (serializer == null) { throw new HugeException("Can't load serializer with name " + name); } diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/serializer/AbstractSerializer.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/serializer/AbstractSerializer.java index 1d3ea05601..56e75fc7ac 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/serializer/AbstractSerializer.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/serializer/AbstractSerializer.java @@ -25,11 +25,20 @@ import com.baidu.hugegraph.backend.query.IdQuery; import com.baidu.hugegraph.backend.query.Query; import com.baidu.hugegraph.backend.store.BackendEntry; +import com.baidu.hugegraph.config.HugeConfig; import com.baidu.hugegraph.type.HugeType; public abstract class AbstractSerializer implements GraphSerializer, SchemaSerializer { + public AbstractSerializer() { + // TODO: default constructor + } + + public AbstractSerializer(HugeConfig config) { + // TODO: use the config + } + protected BackendEntry convertEntry(BackendEntry entry) { return entry; } diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/serializer/BinaryBackendEntry.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/serializer/BinaryBackendEntry.java index 96e5d2026e..5344dc66f1 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/serializer/BinaryBackendEntry.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/serializer/BinaryBackendEntry.java @@ -45,7 +45,11 @@ public class BinaryBackendEntry implements BackendEntry { private boolean olap; public BinaryBackendEntry(HugeType type, byte[] bytes) { - this(type, BytesBuffer.wrap(bytes).parseId(type)); + this(type, BytesBuffer.wrap(bytes).parseId(type, false)); + } + + public BinaryBackendEntry(HugeType type, byte[] bytes, boolean enablePartition) { + this(type, BytesBuffer.wrap(bytes).parseId(type, enablePartition)); } public BinaryBackendEntry(HugeType type, BinaryId id) { diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/serializer/BinaryScatterSerializer.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/serializer/BinaryScatterSerializer.java index e9d8bbc12b..2b190f1b68 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/serializer/BinaryScatterSerializer.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/serializer/BinaryScatterSerializer.java @@ -24,6 +24,7 @@ import com.baidu.hugegraph.backend.id.IdGenerator; import com.baidu.hugegraph.backend.store.BackendEntry; import com.baidu.hugegraph.backend.store.BackendEntry.BackendColumn; +import com.baidu.hugegraph.config.HugeConfig; import com.baidu.hugegraph.schema.VertexLabel; import com.baidu.hugegraph.structure.HugeProperty; import com.baidu.hugegraph.structure.HugeVertex; @@ -32,8 +33,8 @@ public class BinaryScatterSerializer extends BinarySerializer { - public BinaryScatterSerializer() { - super(true, true); + public BinaryScatterSerializer(HugeConfig config) { + super(true, true, false); } @Override diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/serializer/BinarySerializer.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/serializer/BinarySerializer.java index 7b6f644482..2393f5a25f 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/serializer/BinarySerializer.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/serializer/BinarySerializer.java @@ -19,11 +19,9 @@ package com.baidu.hugegraph.backend.serializer; -import java.util.Collection; -import java.util.Iterator; -import java.util.List; -import java.util.Map; +import java.util.*; +import com.baidu.hugegraph.config.HugeConfig; import org.apache.commons.lang.NotImplementedException; import com.baidu.hugegraph.HugeGraph; @@ -81,29 +79,54 @@ public class BinarySerializer extends AbstractSerializer { */ private final boolean keyWithIdPrefix; private final boolean indexWithIdPrefix; + private final boolean enablePartition; public BinarySerializer() { - this(true, true); + this(true, true, false); + } + + public BinarySerializer(HugeConfig config) { + this(true, true, false); } public BinarySerializer(boolean keyWithIdPrefix, - boolean indexWithIdPrefix) { + boolean indexWithIdPrefix, + boolean enablePartition) { this.keyWithIdPrefix = keyWithIdPrefix; this.indexWithIdPrefix = indexWithIdPrefix; + this.enablePartition = enablePartition; } @Override protected BinaryBackendEntry newBackendEntry(HugeType type, Id id) { + if (type.isVertex()) { + BytesBuffer buffer = BytesBuffer.allocate(2 + 1 + id.length()); + writePartitionedId(HugeType.VERTEX, id, buffer); + return new BinaryBackendEntry(type, new BinaryId(buffer.bytes(), id)); + } + if (type.isEdge()) { E.checkState(id instanceof BinaryId, "Expect a BinaryId for BackendEntry with edge id"); return new BinaryBackendEntry(type, (BinaryId) id); } + if (type.isIndex()) { + if (this.enablePartition) { + if (type.isStringIndex()) { + // TODO: add string index partition + } + if (type.isNumericIndex()) { + // TODO: add numeric index partition + } + } + BytesBuffer buffer = BytesBuffer.allocate(1 + id.length()); + byte[] idBytes = buffer.writeIndexId(id, type).bytes(); + return new BinaryBackendEntry(type, new BinaryId(idBytes, id)); + } + BytesBuffer buffer = BytesBuffer.allocate(1 + id.length()); - byte[] idBytes = type.isIndex() ? - buffer.writeIndexId(id, type).bytes() : - buffer.writeId(id).bytes(); + byte[] idBytes = buffer.writeId(id).bytes(); return new BinaryBackendEntry(type, new BinaryId(idBytes, id)); } @@ -112,8 +135,7 @@ protected final BinaryBackendEntry newBackendEntry(HugeVertex vertex) { } protected final BinaryBackendEntry newBackendEntry(HugeEdge edge) { - BinaryId id = new BinaryId(formatEdgeName(edge), - edge.idWithDirection()); + BinaryId id = writeEdgeId(edge.idWithDirection()); return newBackendEntry(edge.type(), id); } @@ -224,12 +246,6 @@ protected void parseExpiredTime(BytesBuffer buffer, HugeElement element) { element.expiredTime(buffer.readVLong()); } - protected byte[] formatEdgeName(HugeEdge edge) { - // owner-vertex + dir + edge-label + sort-values + other-vertex - return BytesBuffer.allocate(BytesBuffer.BUF_EDGE_ID) - .writeEdgeId(edge.id()).bytes(); - } - protected byte[] formatEdgeValue(HugeEdge edge) { int propsCount = edge.sizeOfProperties(); BytesBuffer buffer = BytesBuffer.allocate(4 + 16 * propsCount); @@ -477,7 +493,8 @@ protected void parseVertexOlap(byte[] value, HugeVertex vertex) { public BackendEntry writeEdge(HugeEdge edge) { BinaryBackendEntry entry = newBackendEntry(edge); byte[] name = this.keyWithIdPrefix ? - this.formatEdgeName(edge) : EMPTY_BYTES; + entry.id().asBytes() : EMPTY_BYTES; + byte[] value = this.formatEdgeValue(edge); entry.column(name, value); @@ -571,6 +588,10 @@ public BackendEntry writeId(HugeType type, Id id) { protected Id writeQueryId(HugeType type, Id id) { if (type.isEdge()) { id = writeEdgeId(id); + } else if (type.isVertex()) { + BytesBuffer buffer = BytesBuffer.allocate(2 + 1 + id.length()); + writePartitionedId(HugeType.VERTEX, id, buffer); + id = new BinaryId(buffer.bytes(), id); } else { BytesBuffer buffer = BytesBuffer.allocate(1 + id.length()); id = new BinaryId(buffer.writeId(id).bytes(), id); @@ -600,13 +621,12 @@ private Query writeQueryEdgeRangeCondition(ConditionQuery cq) { } Id label = cq.condition(HugeKeys.LABEL); - int size = 1 + vertex.length() + 1 + label.length() + 16; - BytesBuffer start = BytesBuffer.allocate(size); - start.writeId(vertex); + BytesBuffer start = BytesBuffer.allocate(BytesBuffer.BUF_EDGE_ID); + writePartitionedId(HugeType.EDGE, vertex, start); start.write(direction.type().code()); start.writeId(label); - BytesBuffer end = BytesBuffer.allocate(size); + BytesBuffer end = BytesBuffer.allocate(BytesBuffer.BUF_EDGE_ID); end.copyFrom(start); RangeConditions range = new RangeConditions(sortValues); @@ -655,7 +675,7 @@ private Query writeQueryEdgePrefixCondition(ConditionQuery cq) { if (key == HugeKeys.OWNER_VERTEX || key == HugeKeys.OTHER_VERTEX) { - buffer.writeId((Id) value); + writePartitionedId(HugeType.EDGE, (Id) value, buffer); } else if (key == HugeKeys.DIRECTION) { byte t = ((Directions) value).type().code(); buffer.write(t); @@ -800,18 +820,58 @@ private BinaryBackendEntry formatILDeletion(HugeIndex index) { return entry; } - private static BinaryId writeEdgeId(Id id) { + private BinaryId writeEdgeId(Id id) { EdgeId edgeId; if (id instanceof EdgeId) { edgeId = (EdgeId) id; } else { edgeId = EdgeId.parse(id.asString()); } - BytesBuffer buffer = BytesBuffer.allocate(BytesBuffer.BUF_EDGE_ID) - .writeEdgeId(edgeId); + BytesBuffer buffer = BytesBuffer.allocate(BytesBuffer.BUF_EDGE_ID); + if (this.enablePartition) { + buffer.writeShort(getPartition(HugeType.EDGE, edgeId.ownerVertexId())); + buffer.writeEdgeId(edgeId); + } else { + buffer.writeEdgeId(edgeId); + } return new BinaryId(buffer.bytes(), id); } + private void writePartitionedId(HugeType type, Id id, BytesBuffer buffer) { + if (this.enablePartition) { + buffer.writeShort(getPartition(type, id)); + buffer.writeId(id); + } else { + buffer.writeId(id); + } + } + + protected short getPartition(HugeType type, Id id) { + return 0; + } + + public BackendEntry parse(BackendEntry originEntry) { + byte[] bytes = originEntry.id().asBytes(); + BinaryBackendEntry parsedEntry = new BinaryBackendEntry(originEntry.type(), + bytes, + this.enablePartition); + if (this.enablePartition) { + bytes = Arrays.copyOfRange(bytes, parsedEntry.id().length() + 2, bytes.length); + } else { + bytes = Arrays.copyOfRange(bytes, parsedEntry.id().length(), bytes.length); + } + BytesBuffer buffer = BytesBuffer.allocate(BytesBuffer.BUF_EDGE_ID); + buffer.write(parsedEntry.id().asBytes()); + buffer.write(bytes); + parsedEntry = new BinaryBackendEntry(originEntry.type(), + new BinaryId(buffer.bytes(), + BytesBuffer.wrap(buffer.bytes()).readEdgeId())); + for (BackendEntry.BackendColumn col : originEntry.columns()) { + parsedEntry.column(buffer.bytes(), col.value); + } + return parsedEntry; + } + private static Query prefixQuery(ConditionQuery query, Id prefix) { Query newQuery; if (query.paging() && !query.page().isEmpty()) { diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/serializer/BytesBuffer.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/serializer/BytesBuffer.java index abeea5f046..a93b3f007f 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/serializer/BytesBuffer.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/serializer/BytesBuffer.java @@ -707,6 +707,7 @@ public Id readId(boolean big) { } public BytesBuffer writeEdgeId(Id id) { + // owner-vertex + dir + edge-label + sort-values + other-vertex EdgeId edge = (EdgeId) id; this.writeId(edge.ownerVertexId()); this.write(edge.directionCode()); @@ -767,11 +768,14 @@ public BinaryId asId() { return new BinaryId(this.bytes(), null); } - public BinaryId parseId(HugeType type) { + public BinaryId parseId(HugeType type, boolean enablePartition) { if (type.isIndex()) { return this.readIndexId(type); } // Parse id from bytes + if ((type.isVertex() || type.isEdge()) && enablePartition) { + this.readShort(); + } int start = this.buffer.position(); /* * Since edge id in edges table doesn't prefix with leading 0x7e, diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/serializer/SerializerFactory.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/serializer/SerializerFactory.java index 86e1fedaa7..217719ead3 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/serializer/SerializerFactory.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/serializer/SerializerFactory.java @@ -19,10 +19,13 @@ package com.baidu.hugegraph.backend.serializer; +import java.lang.reflect.Constructor; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import com.baidu.hugegraph.backend.BackendException; +import com.baidu.hugegraph.config.HugeConfig; +import com.baidu.hugegraph.type.HugeType; public class SerializerFactory { @@ -32,15 +35,15 @@ public class SerializerFactory { serializers = new ConcurrentHashMap<>(); } - public static AbstractSerializer serializer(String name) { + public static AbstractSerializer serializer(HugeConfig config, String name) { name = name.toLowerCase(); switch (name) { case "binary": - return new BinarySerializer(); + return new BinarySerializer(config); case "binaryscatter": - return new BinaryScatterSerializer(); + return new BinaryScatterSerializer(config); case "text": - return new TextSerializer(); + return new TextSerializer(config); default: } @@ -51,7 +54,7 @@ public static AbstractSerializer serializer(String name) { assert AbstractSerializer.class.isAssignableFrom(clazz); try { - return clazz.getConstructor().newInstance(); + return clazz.getConstructor(HugeConfig.class).newInstance(config); } catch (Exception e) { throw new BackendException(e); } diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/serializer/TableSerializer.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/serializer/TableSerializer.java index 73a555e597..0079ba5191 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/serializer/TableSerializer.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/serializer/TableSerializer.java @@ -34,6 +34,7 @@ import com.baidu.hugegraph.backend.query.ConditionQuery; import com.baidu.hugegraph.backend.query.Query; import com.baidu.hugegraph.backend.store.BackendEntry; +import com.baidu.hugegraph.config.HugeConfig; import com.baidu.hugegraph.schema.EdgeLabel; import com.baidu.hugegraph.schema.IndexLabel; import com.baidu.hugegraph.schema.PropertyKey; @@ -64,6 +65,10 @@ public abstract class TableSerializer extends AbstractSerializer { + public TableSerializer(HugeConfig config) { + super(config); + } + @Override public TableBackendEntry newBackendEntry(HugeType type, Id id) { return new TableBackendEntry(type, id); diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/serializer/TextSerializer.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/serializer/TextSerializer.java index 3485172a3a..b90646b6fd 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/serializer/TextSerializer.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/serializer/TextSerializer.java @@ -25,6 +25,7 @@ import java.util.List; import java.util.Map; +import com.baidu.hugegraph.config.HugeConfig; import org.apache.commons.lang.NotImplementedException; import com.baidu.hugegraph.HugeException; @@ -78,6 +79,10 @@ public class TextSerializer extends AbstractSerializer { private static final String EDGE_OUT_TYPE = writeType(HugeType.EDGE_OUT); + public TextSerializer(HugeConfig config) { + super(config); + } + @Override public TextBackendEntry newBackendEntry(HugeType type, Id id) { return new TextBackendEntry(type, id); diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/AbstractBackendStoreProvider.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/AbstractBackendStoreProvider.java index d5eaf5a407..c9f0c6e996 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/AbstractBackendStoreProvider.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/AbstractBackendStoreProvider.java @@ -23,6 +23,7 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.Future; +import com.baidu.hugegraph.config.HugeConfig; import org.slf4j.Logger; import com.baidu.hugegraph.HugeGraph; @@ -61,9 +62,9 @@ protected final void checkOpened() { "The BackendStoreProvider has not been opened"); } - protected abstract BackendStore newSchemaStore(String store); + protected abstract BackendStore newSchemaStore(HugeConfig config, String store); - protected abstract BackendStore newGraphStore(String store); + protected abstract BackendStore newGraphStore(HugeConfig config, String store); @Override public void listen(EventListener listener) { @@ -170,13 +171,13 @@ public void resumeSnapshot() { } @Override - public BackendStore loadSchemaStore(final String name) { + public BackendStore loadSchemaStore(HugeConfig config, String name) { LOG.debug("The '{}' StoreProvider load SchemaStore '{}'", this.type(), name); this.checkOpened(); if (!this.stores.containsKey(name)) { - BackendStore s = this.newSchemaStore(name); + BackendStore s = this.newSchemaStore(config, name); this.stores.putIfAbsent(name, s); } @@ -186,13 +187,13 @@ public BackendStore loadSchemaStore(final String name) { } @Override - public BackendStore loadGraphStore(String name) { + public BackendStore loadGraphStore(HugeConfig config, String name) { LOG.debug("The '{}' StoreProvider load GraphStore '{}'", - this.type(), name); + this.type(), name); this.checkOpened(); if (!this.stores.containsKey(name)) { - BackendStore s = this.newGraphStore(name); + BackendStore s = this.newGraphStore(config, name); this.stores.putIfAbsent(name, s); } @@ -202,8 +203,8 @@ public BackendStore loadGraphStore(String name) { } @Override - public BackendStore loadSystemStore(String name) { - return this.loadGraphStore(name); + public BackendStore loadSystemStore(HugeConfig config, String name) { + return this.loadGraphStore(config, name); } @Override diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/BackendStoreProvider.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/BackendStoreProvider.java index 3a1d954c12..a4027e17ac 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/BackendStoreProvider.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/BackendStoreProvider.java @@ -35,11 +35,12 @@ public interface BackendStoreProvider { // Graph name (that's database name) public String graph(); - public BackendStore loadSystemStore(String name); + public BackendStore loadSystemStore(HugeConfig config, String name); - public BackendStore loadSchemaStore(String name); + public BackendStore loadSchemaStore(HugeConfig config, String name); + + public BackendStore loadGraphStore(HugeConfig config, String name); - public BackendStore loadGraphStore(String name); public void open(String name); diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/memory/InMemoryDBStoreProvider.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/memory/InMemoryDBStoreProvider.java index 86de96f74a..d0107423fe 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/memory/InMemoryDBStoreProvider.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/memory/InMemoryDBStoreProvider.java @@ -26,6 +26,7 @@ import com.baidu.hugegraph.backend.store.BackendStore; import com.baidu.hugegraph.backend.store.memory.InMemoryDBStore.InMemoryGraphStore; import com.baidu.hugegraph.backend.store.memory.InMemoryDBStore.InMemorySchemaStore; +import com.baidu.hugegraph.config.HugeConfig; import com.baidu.hugegraph.util.Events; public class InMemoryDBStoreProvider extends AbstractBackendStoreProvider { @@ -66,12 +67,12 @@ public void open(String graph) { } @Override - protected BackendStore newSchemaStore(String store) { + protected BackendStore newSchemaStore(HugeConfig config, String store) { return new InMemorySchemaStore(this, this.graph(), store); } @Override - protected BackendStore newGraphStore(String store) { + protected BackendStore newGraphStore(HugeConfig config, String store) { return new InMemoryGraphStore(this, this.graph(), store); } diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/raft/RaftBackendStoreProvider.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/raft/RaftBackendStoreProvider.java index 46e24f6731..73d8c200c0 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/raft/RaftBackendStoreProvider.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/raft/RaftBackendStoreProvider.java @@ -22,6 +22,7 @@ import java.util.Set; import java.util.concurrent.Future; +import com.baidu.hugegraph.config.HugeConfig; import org.slf4j.Logger; import com.baidu.hugegraph.HugeGraph; @@ -97,10 +98,10 @@ public String graph() { } @Override - public synchronized BackendStore loadSchemaStore(final String name) { + public synchronized BackendStore loadSchemaStore(HugeConfig config, String name) { if (this.schemaStore == null) { LOG.info("Init raft backend schema store"); - BackendStore store = this.provider.loadSchemaStore(name); + BackendStore store = this.provider.loadSchemaStore(config, name); this.checkNonSharedStore(store); this.schemaStore = new RaftBackendStore(store, this.context); this.context.addStore(StoreType.SCHEMA, this.schemaStore); @@ -109,10 +110,10 @@ public synchronized BackendStore loadSchemaStore(final String name) { } @Override - public synchronized BackendStore loadGraphStore(String name) { + public synchronized BackendStore loadGraphStore(HugeConfig config, String name) { if (this.graphStore == null) { LOG.info("Init raft backend graph store"); - BackendStore store = this.provider.loadGraphStore(name); + BackendStore store = this.provider.loadGraphStore(config, name); this.checkNonSharedStore(store); this.graphStore = new RaftBackendStore(store, this.context); this.context.addStore(StoreType.GRAPH, this.graphStore); @@ -121,10 +122,10 @@ public synchronized BackendStore loadGraphStore(String name) { } @Override - public synchronized BackendStore loadSystemStore(String name) { + public synchronized BackendStore loadSystemStore(HugeConfig config, String name) { if (this.systemStore == null) { LOG.info("Init raft backend system store"); - BackendStore store = this.provider.loadSystemStore(name); + BackendStore store = this.provider.loadSystemStore(config, name); this.checkNonSharedStore(store); this.systemStore = new RaftBackendStore(store, this.context); this.context.addStore(StoreType.SYSTEM, this.systemStore); diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/ram/RamTable.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/ram/RamTable.java index 8c6b4d8cb6..9fd2f14764 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/ram/RamTable.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/ram/RamTable.java @@ -200,6 +200,7 @@ private void loadFromDB() throws Exception { Id lastId = IdGenerator.ZERO; while (vertices.hasNext()) { Id vertex = (Id) vertices.next().id(); + LOG.info("scan from hbase {} loadfromDB", vertex); if (vertex.compareTo(lastId) < 0) { throw new HugeException("The ramtable feature is not " + "supported by %s backend", @@ -541,6 +542,7 @@ private void addVertex(Id vertex) { if (this.vertices.size() > 0) { lastId = this.vertices.get(this.vertices.size() - 1); } + LOG.info("scan from hbase source {} lastId value: {} compare {} size {}", vertex, lastId, vertex.compareTo(lastId), this.vertices.size()); if (vertex.compareTo(lastId) < 0) { throw new HugeException("The ramtable feature is not " + "supported by %s backend", diff --git a/hugegraph-dist/src/assembly/static/conf/graphs/hugegraph.properties b/hugegraph-dist/src/assembly/static/conf/graphs/hugegraph.properties index 7df076477e..9dc75f7244 100644 --- a/hugegraph-dist/src/assembly/static/conf/graphs/hugegraph.properties +++ b/hugegraph-dist/src/assembly/static/conf/graphs/hugegraph.properties @@ -66,6 +66,12 @@ cassandra.password= #hbase.port=2181 #hbase.znode_parent=/hbase #hbase.threads_max=64 +# IMPORTANT: recommend to modify the HBase partition number +# by the actual/env data amount & RS amount before init store +# It will influence the load speed a lot +#hbase.enable_partition=true +#hbase.vertex_partitions=10 +#hbase.edge_partitions=30 # mysql backend config #jdbc.driver=com.mysql.jdbc.Driver diff --git a/hugegraph-hbase/src/main/java/com/baidu/hugegraph/backend/store/hbase/HbaseOptions.java b/hugegraph-hbase/src/main/java/com/baidu/hugegraph/backend/store/hbase/HbaseOptions.java index d795bbb994..2fb3e39fbd 100644 --- a/hugegraph-hbase/src/main/java/com/baidu/hugegraph/backend/store/hbase/HbaseOptions.java +++ b/hugegraph-hbase/src/main/java/com/baidu/hugegraph/backend/store/hbase/HbaseOptions.java @@ -22,6 +22,7 @@ import static com.baidu.hugegraph.config.OptionChecker.disallowEmpty; import static com.baidu.hugegraph.config.OptionChecker.positiveInt; import static com.baidu.hugegraph.config.OptionChecker.rangeInt; +import static com.baidu.hugegraph.config.OptionChecker.nonNegativeInt; import com.baidu.hugegraph.config.ConfigOption; import com.baidu.hugegraph.config.OptionHolder; @@ -137,4 +138,28 @@ public static synchronized HbaseOptions instance() { null, "" ); + + public static final ConfigOption HBASE_ENABLE_PARTITION = + new ConfigOption<>( + "hbase.enable_partition", + "Is pre-split partitions enabled for HBase.", + disallowEmpty(), + true + ); + + public static final ConfigOption HBASE_VERTEX_PARTITION = + new ConfigOption<>( + "hbase.vertex_partitions", + "The number of partitions of the HBase vertex table", + nonNegativeInt(), + 10 + ); + + public static final ConfigOption HBASE_EDGE_PARTITION = + new ConfigOption<>( + "hbase.edge_partitions", + "The number of partitions of the HBase edge table", + nonNegativeInt(), + 30 + ); } diff --git a/hugegraph-hbase/src/main/java/com/baidu/hugegraph/backend/store/hbase/HbaseSerializer.java b/hugegraph-hbase/src/main/java/com/baidu/hugegraph/backend/store/hbase/HbaseSerializer.java index 71029d00ff..c7bf3de92b 100644 --- a/hugegraph-hbase/src/main/java/com/baidu/hugegraph/backend/store/hbase/HbaseSerializer.java +++ b/hugegraph-hbase/src/main/java/com/baidu/hugegraph/backend/store/hbase/HbaseSerializer.java @@ -19,11 +19,36 @@ package com.baidu.hugegraph.backend.store.hbase; +import com.baidu.hugegraph.backend.id.Id; import com.baidu.hugegraph.backend.serializer.BinarySerializer; +import com.baidu.hugegraph.config.HugeConfig; +import com.baidu.hugegraph.type.HugeType; +import com.baidu.hugegraph.util.Log; +import org.slf4j.Logger; +import java.util.Arrays; public class HbaseSerializer extends BinarySerializer { - public HbaseSerializer() { - super(false, true); + private static final Logger LOG = Log.logger(HbaseSerializer.class); + private final short vertexLogicPartitions; + private final short edgeLogicPartitions; + + public HbaseSerializer(HugeConfig config) { + super(false, true, config.get(HbaseOptions.HBASE_ENABLE_PARTITION).booleanValue()); + this.vertexLogicPartitions = config.get(HbaseOptions.HBASE_VERTEX_PARTITION).shortValue(); + this.edgeLogicPartitions = config.get(HbaseOptions.HBASE_EDGE_PARTITION).shortValue(); + LOG.debug("vertexLogicPartitions: " + vertexLogicPartitions); + } + + @Override + protected short getPartition(HugeType type, Id id) { + int hashcode = Arrays.hashCode(id.asBytes()); + short partition = 1; + if (type.isEdge()) { + partition = (short) (hashcode % this.edgeLogicPartitions); + } else if (type.isVertex()) { + partition = (short) (hashcode % this.vertexLogicPartitions); + } + return partition > 0 ? partition : (short) -partition; } } diff --git a/hugegraph-hbase/src/main/java/com/baidu/hugegraph/backend/store/hbase/HbaseSessions.java b/hugegraph-hbase/src/main/java/com/baidu/hugegraph/backend/store/hbase/HbaseSessions.java index 5037749c49..011026fd8b 100644 --- a/hugegraph-hbase/src/main/java/com/baidu/hugegraph/backend/store/hbase/HbaseSessions.java +++ b/hugegraph-hbase/src/main/java/com/baidu/hugegraph/backend/store/hbase/HbaseSessions.java @@ -227,6 +227,23 @@ public void createTable(String table, List cfs) throws IOException { } } + public void createPreSplitTable(String table, List cfs, short numOfPartitions) throws IOException { + TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder( + TableName.valueOf(this.namespace, table)); + for (byte[] cf : cfs) { + builder.setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(cf) + .build()); + } + byte[][] splits = new byte[numOfPartitions - 1][org.apache.hadoop.hbase.util.Bytes.SIZEOF_SHORT]; + for (short split = 1; split < numOfPartitions; split++) { + splits[split - 1] = org.apache.hadoop.hbase.util.Bytes.toBytes(split); + } + builder.setCoprocessor(COPROCESSOR_AGGR); + try (Admin admin = this.hbase.getAdmin()) { + admin.createTable(builder.build(), splits); + } + } + public void dropTable(String table) throws IOException { TableName tableName = TableName.valueOf(this.namespace, table); try (Admin admin = this.hbase.getAdmin()) { diff --git a/hugegraph-hbase/src/main/java/com/baidu/hugegraph/backend/store/hbase/HbaseStore.java b/hugegraph-hbase/src/main/java/com/baidu/hugegraph/backend/store/hbase/HbaseStore.java index 8b56ede21b..8e52d54b4e 100644 --- a/hugegraph-hbase/src/main/java/com/baidu/hugegraph/backend/store/hbase/HbaseStore.java +++ b/hugegraph-hbase/src/main/java/com/baidu/hugegraph/backend/store/hbase/HbaseStore.java @@ -61,6 +61,8 @@ public abstract class HbaseStore extends AbstractBackendStore { private final BackendStoreProvider provider; private final Map tables; + private short vertexLogicPartitions; + private short edgeLogicPartitions; private HbaseSessions sessions; @@ -141,6 +143,8 @@ public BackendFeatures features() { @Override public synchronized void open(HugeConfig config) { E.checkNotNull(config, "config"); + this.vertexLogicPartitions = config.get(HbaseOptions.HBASE_VERTEX_PARTITION).shortValue(); + this.edgeLogicPartitions = config.get(HbaseOptions.HBASE_EDGE_PARTITION).shortValue(); if (this.sessions == null) { this.sessions = new HbaseSessions(config, this.namespace, this.store); @@ -252,7 +256,14 @@ public void init() { // Create tables for (String table : this.tableNames()) { try { - this.sessions.createTable(table, HbaseTable.cfs()); + if (table.equals("g_oe") || table.equals("g_ie")) { + this.sessions.createPreSplitTable(table, HbaseTable.cfs(), this.edgeLogicPartitions); + } else if (table.equals("g_v")) { + this.sessions.createPreSplitTable(table, HbaseTable.cfs(), this.vertexLogicPartitions); + } else { + this.sessions.createTable(table, HbaseTable.cfs()); + } + } catch (TableExistsException ignored) { continue; } catch (IOException e) { @@ -477,18 +488,18 @@ public boolean isSchemaStore() { } public static class HbaseGraphStore extends HbaseStore { - - public HbaseGraphStore(BackendStoreProvider provider, + private boolean enablePartition; + public HbaseGraphStore(HugeConfig config, BackendStoreProvider provider, String namespace, String store) { super(provider, namespace, store); - + this.enablePartition = config.get(HbaseOptions.HBASE_ENABLE_PARTITION).booleanValue(); registerTableManager(HugeType.VERTEX, - new HbaseTables.Vertex(store)); + new HbaseTables.Vertex(store, enablePartition)); registerTableManager(HugeType.EDGE_OUT, - HbaseTables.Edge.out(store)); + HbaseTables.Edge.out(store, enablePartition)); registerTableManager(HugeType.EDGE_IN, - HbaseTables.Edge.in(store)); + HbaseTables.Edge.in(store, enablePartition)); registerTableManager(HugeType.SECONDARY_INDEX, new HbaseTables.SecondaryIndex(store)); diff --git a/hugegraph-hbase/src/main/java/com/baidu/hugegraph/backend/store/hbase/HbaseStoreProvider.java b/hugegraph-hbase/src/main/java/com/baidu/hugegraph/backend/store/hbase/HbaseStoreProvider.java index f1c6634d60..92a36f1737 100644 --- a/hugegraph-hbase/src/main/java/com/baidu/hugegraph/backend/store/hbase/HbaseStoreProvider.java +++ b/hugegraph-hbase/src/main/java/com/baidu/hugegraph/backend/store/hbase/HbaseStoreProvider.java @@ -23,6 +23,7 @@ import com.baidu.hugegraph.backend.store.BackendStore; import com.baidu.hugegraph.backend.store.hbase.HbaseStore.HbaseGraphStore; import com.baidu.hugegraph.backend.store.hbase.HbaseStore.HbaseSchemaStore; +import com.baidu.hugegraph.config.HugeConfig; public class HbaseStoreProvider extends AbstractBackendStoreProvider { @@ -31,13 +32,13 @@ protected String namespace() { } @Override - protected BackendStore newSchemaStore(String store) { + protected BackendStore newSchemaStore(HugeConfig config, String store) { return new HbaseSchemaStore(this, this.namespace(), store); } @Override - protected BackendStore newGraphStore(String store) { - return new HbaseGraphStore(this, this.namespace(), store); + protected BackendStore newGraphStore(HugeConfig config, String store) { + return new HbaseGraphStore(config, this, this.namespace(), store); } @Override diff --git a/hugegraph-hbase/src/main/java/com/baidu/hugegraph/backend/store/hbase/HbaseTable.java b/hugegraph-hbase/src/main/java/com/baidu/hugegraph/backend/store/hbase/HbaseTable.java index 43a5c008f5..5d14028c91 100644 --- a/hugegraph-hbase/src/main/java/com/baidu/hugegraph/backend/store/hbase/HbaseTable.java +++ b/hugegraph-hbase/src/main/java/com/baidu/hugegraph/backend/store/hbase/HbaseTable.java @@ -79,11 +79,18 @@ public class HbaseTable extends BackendTable { private final HbaseShardSplitter shardSplitter; - public HbaseTable(String table) { + private final boolean enablePartition; + + public HbaseTable(String table, boolean enablePartition) { super(table); + this.enablePartition = enablePartition; this.shardSplitter = new HbaseShardSplitter(this.table()); } + public HbaseTable(String table) { + this(table, false); + } + public static List cfs() { return ImmutableList.of(CF); } @@ -265,10 +272,10 @@ protected BackendEntryIterator newEntryIterator(Query query, if (entry == null || !Bytes.prefixWith(id, entry.id().asBytes())) { HugeType type = query.resultType(); // NOTE: only support BinaryBackendEntry currently - entry = new BinaryBackendEntry(type, id); + entry = new BinaryBackendEntry(type, id, this.enablePartition); } try { - this.parseRowColumns(row, entry, query); + this.parseRowColumns(row, entry, query, this.enablePartition); } catch (IOException e) { throw new BackendException("Failed to read HBase columns", e); } @@ -276,7 +283,7 @@ protected BackendEntryIterator newEntryIterator(Query query, }); } - protected void parseRowColumns(Result row, BackendEntry entry, Query query) + protected void parseRowColumns(Result row, BackendEntry entry, Query query, boolean enablePartition) throws IOException { CellScanner cellScanner = row.cellScanner(); while (cellScanner.advance()) { diff --git a/hugegraph-hbase/src/main/java/com/baidu/hugegraph/backend/store/hbase/HbaseTables.java b/hugegraph-hbase/src/main/java/com/baidu/hugegraph/backend/store/hbase/HbaseTables.java index cc029b2b74..658f7fd5f8 100644 --- a/hugegraph-hbase/src/main/java/com/baidu/hugegraph/backend/store/hbase/HbaseTables.java +++ b/hugegraph-hbase/src/main/java/com/baidu/hugegraph/backend/store/hbase/HbaseTables.java @@ -21,7 +21,11 @@ import java.io.IOException; import java.util.Arrays; +import java.util.Iterator; +import com.baidu.hugegraph.backend.id.EdgeId; +import com.baidu.hugegraph.backend.serializer.BytesBuffer; +import com.baidu.hugegraph.config.HugeConfig; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellScanner; import org.apache.hadoop.hbase.CellUtil; @@ -109,8 +113,8 @@ public static class Vertex extends HbaseTable { public static final String TABLE = HugeType.VERTEX.string(); - public Vertex(String store) { - super(joinTableName(store, TABLE)); + public Vertex(String store, boolean enablePartition) { + super(joinTableName(store, TABLE), enablePartition); } @Override @@ -130,8 +134,8 @@ public static class Edge extends HbaseTable { public static final String TABLE_SUFFIX = HugeType.EDGE.string(); - public Edge(String store, boolean out) { - super(joinTableName(store, table(out))); + public Edge(String store, boolean out, boolean enablePartition) { + super(joinTableName(store, table(out)), enablePartition); } private static String table(boolean out) { @@ -139,12 +143,12 @@ private static String table(boolean out) { return (out ? 'o' : 'i') + TABLE_SUFFIX; } - public static Edge out(String store) { - return new Edge(store, true); + public static Edge out(String store, boolean enablePartition) { + return new Edge(store, true, enablePartition); } - public static Edge in(String store) { - return new Edge(store, false); + public static Edge in(String store, boolean enablePartition) { + return new Edge(store, false, enablePartition); } @Override @@ -161,13 +165,17 @@ public void insert(Session session, BackendEntry entry) { @Override protected void parseRowColumns(Result row, BackendEntry entry, - Query query) throws IOException { + Query query, boolean enablePartition) throws IOException { /* * Collapse owner-vertex id from edge id, NOTE: unneeded to * collapse if BinarySerializer.keyWithIdPrefix set to true */ byte[] key = row.getRow(); - key = Arrays.copyOfRange(key, entry.id().length(), key.length); + if (enablePartition) { + key = Arrays.copyOfRange(key, entry.id().length() + 2, key.length); + } else { + key = Arrays.copyOfRange(key, entry.id().length(), key.length); + } long total = query.total(); CellScanner cellScanner = row.cellScanner(); diff --git a/hugegraph-mysql/src/main/java/com/baidu/hugegraph/backend/store/mysql/MysqlSerializer.java b/hugegraph-mysql/src/main/java/com/baidu/hugegraph/backend/store/mysql/MysqlSerializer.java index adb3bb41fe..8ab82eae08 100644 --- a/hugegraph-mysql/src/main/java/com/baidu/hugegraph/backend/store/mysql/MysqlSerializer.java +++ b/hugegraph-mysql/src/main/java/com/baidu/hugegraph/backend/store/mysql/MysqlSerializer.java @@ -24,6 +24,7 @@ import java.util.Map; import java.util.Set; +import com.baidu.hugegraph.config.HugeConfig; import org.apache.commons.lang.NotImplementedException; import com.baidu.hugegraph.backend.BackendException; @@ -43,6 +44,10 @@ public class MysqlSerializer extends TableSerializer { + public MysqlSerializer(HugeConfig config) { + super(config); + } + @Override public MysqlBackendEntry newBackendEntry(HugeType type, Id id) { return new MysqlBackendEntry(type, id); diff --git a/hugegraph-mysql/src/main/java/com/baidu/hugegraph/backend/store/mysql/MysqlStoreProvider.java b/hugegraph-mysql/src/main/java/com/baidu/hugegraph/backend/store/mysql/MysqlStoreProvider.java index bd6f2501b0..13357f7390 100644 --- a/hugegraph-mysql/src/main/java/com/baidu/hugegraph/backend/store/mysql/MysqlStoreProvider.java +++ b/hugegraph-mysql/src/main/java/com/baidu/hugegraph/backend/store/mysql/MysqlStoreProvider.java @@ -23,6 +23,7 @@ import com.baidu.hugegraph.backend.store.BackendStore; import com.baidu.hugegraph.backend.store.mysql.MysqlStore.MysqlGraphStore; import com.baidu.hugegraph.backend.store.mysql.MysqlStore.MysqlSchemaStore; +import com.baidu.hugegraph.config.HugeConfig; public class MysqlStoreProvider extends AbstractBackendStoreProvider { @@ -31,12 +32,12 @@ protected String database() { } @Override - protected BackendStore newSchemaStore(String store) { + protected BackendStore newSchemaStore(HugeConfig config, String store) { return new MysqlSchemaStore(this, this.database(), store); } @Override - protected BackendStore newGraphStore(String store) { + protected BackendStore newGraphStore(HugeConfig config, String store) { return new MysqlGraphStore(this, this.database(), store); } diff --git a/hugegraph-palo/src/main/java/com/baidu/hugegraph/backend/store/palo/PaloSerializer.java b/hugegraph-palo/src/main/java/com/baidu/hugegraph/backend/store/palo/PaloSerializer.java index 8175312bc5..d97c95f998 100644 --- a/hugegraph-palo/src/main/java/com/baidu/hugegraph/backend/store/palo/PaloSerializer.java +++ b/hugegraph-palo/src/main/java/com/baidu/hugegraph/backend/store/palo/PaloSerializer.java @@ -21,11 +21,16 @@ import com.baidu.hugegraph.backend.serializer.TableBackendEntry; import com.baidu.hugegraph.backend.store.mysql.MysqlSerializer; +import com.baidu.hugegraph.config.HugeConfig; import com.baidu.hugegraph.schema.SchemaLabel; import com.baidu.hugegraph.type.define.HugeKeys; public class PaloSerializer extends MysqlSerializer { + public PaloSerializer(HugeConfig config) { + super(config); + } + @Override protected void writeEnableLabelIndex(SchemaLabel schema, TableBackendEntry entry) { diff --git a/hugegraph-palo/src/main/java/com/baidu/hugegraph/backend/store/palo/PaloStoreProvider.java b/hugegraph-palo/src/main/java/com/baidu/hugegraph/backend/store/palo/PaloStoreProvider.java index 0a39f56aba..434cd2d40c 100644 --- a/hugegraph-palo/src/main/java/com/baidu/hugegraph/backend/store/palo/PaloStoreProvider.java +++ b/hugegraph-palo/src/main/java/com/baidu/hugegraph/backend/store/palo/PaloStoreProvider.java @@ -25,6 +25,7 @@ import com.baidu.hugegraph.backend.store.BackendStore; import com.baidu.hugegraph.backend.store.BackendStoreProvider; import com.baidu.hugegraph.backend.store.mysql.MysqlStoreProvider; +import com.baidu.hugegraph.config.HugeConfig; import com.baidu.hugegraph.type.HugeType; import com.baidu.hugegraph.type.define.Directions; @@ -33,12 +34,12 @@ public class PaloStoreProvider extends MysqlStoreProvider { private static final BackendFeatures FEATURES = new PaloFeatures(); @Override - protected BackendStore newSchemaStore(String store) { + protected BackendStore newSchemaStore(HugeConfig config, String store) { return new PaloSchemaStore(this, this.database(), store); } @Override - protected BackendStore newGraphStore(String store) { + protected BackendStore newGraphStore(HugeConfig config, String store) { return new PaloGraphStore(this, this.database(), store); } diff --git a/hugegraph-postgresql/src/main/java/com/baidu/hugegraph/backend/store/postgresql/PostgresqlSerializer.java b/hugegraph-postgresql/src/main/java/com/baidu/hugegraph/backend/store/postgresql/PostgresqlSerializer.java index 972e1dffdc..e28b0936fa 100644 --- a/hugegraph-postgresql/src/main/java/com/baidu/hugegraph/backend/store/postgresql/PostgresqlSerializer.java +++ b/hugegraph-postgresql/src/main/java/com/baidu/hugegraph/backend/store/postgresql/PostgresqlSerializer.java @@ -19,6 +19,7 @@ package com.baidu.hugegraph.backend.store.postgresql; +import com.baidu.hugegraph.config.HugeConfig; import org.apache.logging.log4j.util.Strings; import com.baidu.hugegraph.backend.id.IdUtil; @@ -30,6 +31,10 @@ public class PostgresqlSerializer extends MysqlSerializer { + public PostgresqlSerializer(HugeConfig config) { + super(config); + } + @Override public BackendEntry writeIndex(HugeIndex index) { TableBackendEntry entry = newBackendEntry(index); diff --git a/hugegraph-postgresql/src/main/java/com/baidu/hugegraph/backend/store/postgresql/PostgresqlStoreProvider.java b/hugegraph-postgresql/src/main/java/com/baidu/hugegraph/backend/store/postgresql/PostgresqlStoreProvider.java index b639894c45..955572a656 100644 --- a/hugegraph-postgresql/src/main/java/com/baidu/hugegraph/backend/store/postgresql/PostgresqlStoreProvider.java +++ b/hugegraph-postgresql/src/main/java/com/baidu/hugegraph/backend/store/postgresql/PostgresqlStoreProvider.java @@ -29,18 +29,19 @@ import com.baidu.hugegraph.backend.store.mysql.MysqlSessions; import com.baidu.hugegraph.backend.store.mysql.MysqlStoreProvider; import com.baidu.hugegraph.backend.store.mysql.MysqlTable; +import com.baidu.hugegraph.config.HugeConfig; import com.baidu.hugegraph.type.HugeType; import com.baidu.hugegraph.type.define.Directions; public class PostgresqlStoreProvider extends MysqlStoreProvider { @Override - protected BackendStore newSchemaStore(String store) { + protected BackendStore newSchemaStore(HugeConfig config, String store) { return new PostgresqlSchemaStore(this, this.database(), store); } @Override - protected BackendStore newGraphStore(String store) { + protected BackendStore newGraphStore(HugeConfig config, String store) { return new PostgresqlGraphStore(this, this.database(), store); } diff --git a/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBStoreProvider.java b/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBStoreProvider.java index ca8ae701d9..7f83f9cf85 100644 --- a/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBStoreProvider.java +++ b/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBStoreProvider.java @@ -35,12 +35,12 @@ protected String database() { } @Override - protected BackendStore newSchemaStore(String store) { + protected BackendStore newSchemaStore(HugeConfig config, String store) { return new RocksDBSchemaStore(this, this.database(), store); } @Override - protected BackendStore newGraphStore(String store) { + protected BackendStore newGraphStore(HugeConfig config, String store) { return new RocksDBGraphStore(this, this.database(), store); } diff --git a/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdbsst/RocksDBSstStoreProvider.java b/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdbsst/RocksDBSstStoreProvider.java index ea09b4e83f..69fa0a1931 100644 --- a/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdbsst/RocksDBSstStoreProvider.java +++ b/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdbsst/RocksDBSstStoreProvider.java @@ -22,11 +22,12 @@ import com.baidu.hugegraph.backend.store.BackendStore; import com.baidu.hugegraph.backend.store.rocksdb.RocksDBStoreProvider; import com.baidu.hugegraph.backend.store.rocksdbsst.RocksDBSstStore.RocksDBSstGraphStore; +import com.baidu.hugegraph.config.HugeConfig; public class RocksDBSstStoreProvider extends RocksDBStoreProvider { @Override - protected BackendStore newGraphStore(String store) { + protected BackendStore newGraphStore(HugeConfig config, String store) { return new RocksDBSstGraphStore(this, this.database(), store); } diff --git a/hugegraph-scylladb/src/main/java/com/baidu/hugegraph/backend/store/scylladb/ScyllaDBStoreProvider.java b/hugegraph-scylladb/src/main/java/com/baidu/hugegraph/backend/store/scylladb/ScyllaDBStoreProvider.java index 9e18b15633..0fe6d670e8 100644 --- a/hugegraph-scylladb/src/main/java/com/baidu/hugegraph/backend/store/scylladb/ScyllaDBStoreProvider.java +++ b/hugegraph-scylladb/src/main/java/com/baidu/hugegraph/backend/store/scylladb/ScyllaDBStoreProvider.java @@ -45,7 +45,7 @@ public String type() { } @Override - public BackendStore loadSchemaStore(String name) { + public BackendStore loadSchemaStore(HugeConfig config, String name) { LOG.debug("ScyllaDBStoreProvider load SchemaStore '{}'", name); if (!this.stores.containsKey(name)) { @@ -61,7 +61,7 @@ public BackendStore loadSchemaStore(String name) { } @Override - public BackendStore loadGraphStore(String name) { + public BackendStore loadGraphStore(HugeConfig config, String name) { LOG.debug("ScyllaDBStoreProvider load GraphStore '{}'", name); if (!this.stores.containsKey(name)) { diff --git a/hugegraph-test/src/main/java/com/baidu/hugegraph/unit/FakeObjects.java b/hugegraph-test/src/main/java/com/baidu/hugegraph/unit/FakeObjects.java index b988f1a52b..2b31de8c6f 100644 --- a/hugegraph-test/src/main/java/com/baidu/hugegraph/unit/FakeObjects.java +++ b/hugegraph-test/src/main/java/com/baidu/hugegraph/unit/FakeObjects.java @@ -172,4 +172,47 @@ public HugeEdge newEdge(long sourceVertexId, long targetVertexId) { return edge; } + + public HugeEdge newEdge(String sourceVertexId, String targetVertexId) { + PropertyKey name = this.newPropertyKey(IdGenerator.of(1), "name"); + PropertyKey age = this.newPropertyKey(IdGenerator.of(2), "age", + DataType.INT, + Cardinality.SINGLE); + PropertyKey city = this.newPropertyKey(IdGenerator.of(3), "city"); + PropertyKey date = this.newPropertyKey(IdGenerator.of(4), "date", + DataType.DATE); + PropertyKey weight = this.newPropertyKey(IdGenerator.of(5), + "weight", DataType.DOUBLE); + + VertexLabel vl = this.newVertexLabel(IdGenerator.of(1), "person", + IdStrategy.CUSTOMIZE_NUMBER, + name.id(), age.id(), city.id()); + + EdgeLabel el = this.newEdgeLabel(IdGenerator.of(1), "knows", + Frequency.SINGLE, vl.id(), vl.id(), + date.id(), weight.id()); + + HugeVertex source = new HugeVertex(this.graph(), + IdGenerator.of(sourceVertexId), vl); + source.addProperty(name, "tom"); + source.addProperty(age, 18); + source.addProperty(city, "Beijing"); + + HugeVertex target = new HugeVertex(this.graph(), + IdGenerator.of(targetVertexId), vl); + target.addProperty(name, "cat"); + target.addProperty(age, 20); + target.addProperty(city, "Shanghai"); + + Id id = EdgeId.parse("L123456>1>>L987654"); + HugeEdge edge = new HugeEdge(this.graph(), id, el); + + Whitebox.setInternalState(edge, "sourceVertex", source); + Whitebox.setInternalState(edge, "targetVertex", target); + edge.assignId(); + edge.addProperty(date, new Date()); + edge.addProperty(weight, 0.75); + + return edge; + } } diff --git a/hugegraph-test/src/main/java/com/baidu/hugegraph/unit/serializer/BinaryScatterSerializerTest.java b/hugegraph-test/src/main/java/com/baidu/hugegraph/unit/serializer/BinaryScatterSerializerTest.java index 80e473053c..7f4179893c 100644 --- a/hugegraph-test/src/main/java/com/baidu/hugegraph/unit/serializer/BinaryScatterSerializerTest.java +++ b/hugegraph-test/src/main/java/com/baidu/hugegraph/unit/serializer/BinaryScatterSerializerTest.java @@ -19,6 +19,7 @@ package com.baidu.hugegraph.unit.serializer; +import com.baidu.hugegraph.config.HugeConfig; import org.junit.Test; import com.baidu.hugegraph.backend.serializer.BinaryBackendEntry; @@ -35,7 +36,8 @@ public class BinaryScatterSerializerTest extends BaseUnitTest { @Test public void testVertex() { - BinaryScatterSerializer ser = new BinaryScatterSerializer(); + HugeConfig config = FakeObjects.newConfig(); + BinaryScatterSerializer ser = new BinaryScatterSerializer(config); HugeEdge edge = new FakeObjects().newEdge(123, 456); BackendEntry entry1 = ser.writeVertex(edge.sourceVertex()); @@ -60,7 +62,8 @@ public void testVertex() { @Test public void testEdge() { - BinaryScatterSerializer ser = new BinaryScatterSerializer(); + HugeConfig config = FakeObjects.newConfig(); + BinaryScatterSerializer ser = new BinaryScatterSerializer(config); FakeObjects objects = new FakeObjects(); HugeEdge edge1 = objects.newEdge(123, 456); diff --git a/hugegraph-test/src/main/java/com/baidu/hugegraph/unit/serializer/BinarySerializerTest.java b/hugegraph-test/src/main/java/com/baidu/hugegraph/unit/serializer/BinarySerializerTest.java index 9ca5900266..503255a567 100644 --- a/hugegraph-test/src/main/java/com/baidu/hugegraph/unit/serializer/BinarySerializerTest.java +++ b/hugegraph-test/src/main/java/com/baidu/hugegraph/unit/serializer/BinarySerializerTest.java @@ -19,6 +19,7 @@ package com.baidu.hugegraph.unit.serializer; +import com.baidu.hugegraph.config.HugeConfig; import org.junit.Test; import com.baidu.hugegraph.backend.serializer.BinarySerializer; @@ -34,7 +35,8 @@ public class BinarySerializerTest extends BaseUnitTest { @Test public void testVertex() { - BinarySerializer ser = new BinarySerializer(); + HugeConfig config = FakeObjects.newConfig(); + BinarySerializer ser = new BinarySerializer(config); HugeEdge edge = new FakeObjects().newEdge(123, 456); BackendEntry entry1 = ser.writeVertex(edge.sourceVertex()); @@ -59,7 +61,8 @@ public void testVertex() { @Test public void testEdge() { - BinarySerializer ser = new BinarySerializer(); + HugeConfig config = FakeObjects.newConfig(); + BinarySerializer ser = new BinarySerializer(config); FakeObjects objects = new FakeObjects(); HugeEdge edge1 = objects.newEdge(123, 456); @@ -79,4 +82,52 @@ public void testEdge() { Assert.assertEquals(edge2, edge); assertCollectionEquals(edge2.getProperties(), edge.getProperties()); } + + @Test + public void testVertexForPartition() { + BinarySerializer ser = new BinarySerializer(true, true, true); + HugeEdge edge = new FakeObjects().newEdge("123", "456"); + + BackendEntry entry1 = ser.writeVertex(edge.sourceVertex()); + HugeVertex vertex1 = ser.readVertex(edge.graph(), entry1); + Assert.assertEquals(edge.sourceVertex(), vertex1); + assertCollectionEquals(edge.sourceVertex().getProperties(), + vertex1.getProperties()); + + BackendEntry entry2 = ser.writeVertex(edge.targetVertex()); + HugeVertex vertex2 = ser.readVertex(edge.graph(), entry2); + Assert.assertEquals(edge.targetVertex(), vertex2); + assertCollectionEquals(edge.targetVertex().getProperties(), + vertex2.getProperties()); + + Whitebox.setInternalState(vertex2, "removed", true); + Assert.assertTrue(vertex2.removed()); + BackendEntry entry3 = ser.writeVertex(vertex2); + Assert.assertEquals(0, entry3.columnsSize()); + + Assert.assertNull(ser.readVertex(edge.graph(), null)); + } + + @Test + public void testEdgeForPartition() { + BinarySerializer ser = new BinarySerializer(true, true, true); + + FakeObjects objects = new FakeObjects(); + HugeEdge edge1 = objects.newEdge("123", "456"); + HugeEdge edge2 = objects.newEdge("147", "789"); + + BackendEntry entry1 = ser.writeEdge(edge1); + HugeVertex vertex1 = ser.readVertex(edge1.graph(), ser.parse(entry1)); + Assert.assertEquals(1, vertex1.getEdges().size()); + HugeEdge edge = vertex1.getEdges().iterator().next(); + Assert.assertEquals(edge1, edge); + assertCollectionEquals(edge1.getProperties(), edge.getProperties()); + + BackendEntry entry2 = ser.writeEdge(edge2); + HugeVertex vertex2 = ser.readVertex(edge1.graph(), ser.parse(entry2)); + Assert.assertEquals(1, vertex2.getEdges().size()); + edge = vertex2.getEdges().iterator().next(); + Assert.assertEquals(edge2, edge); + assertCollectionEquals(edge2.getProperties(), edge.getProperties()); + } } diff --git a/hugegraph-test/src/main/java/com/baidu/hugegraph/unit/serializer/SerializerFactoryTest.java b/hugegraph-test/src/main/java/com/baidu/hugegraph/unit/serializer/SerializerFactoryTest.java index 69db010585..b1d0e3464f 100644 --- a/hugegraph-test/src/main/java/com/baidu/hugegraph/unit/serializer/SerializerFactoryTest.java +++ b/hugegraph-test/src/main/java/com/baidu/hugegraph/unit/serializer/SerializerFactoryTest.java @@ -19,6 +19,9 @@ package com.baidu.hugegraph.unit.serializer; +import com.baidu.hugegraph.config.HugeConfig; +import com.baidu.hugegraph.type.HugeType; +import com.baidu.hugegraph.unit.FakeObjects; import org.junit.Test; import com.baidu.hugegraph.backend.BackendException; @@ -34,18 +37,19 @@ public class SerializerFactoryTest extends BaseUnitTest { @Test public void testSerializer() { - AbstractSerializer serializer = SerializerFactory.serializer("text"); + HugeConfig config = FakeObjects.newConfig(); + AbstractSerializer serializer = SerializerFactory.serializer(config,"text"); Assert.assertEquals(TextSerializer.class, serializer.getClass()); - serializer = SerializerFactory.serializer("binary"); + serializer = SerializerFactory.serializer(config, "binary"); Assert.assertEquals(BinarySerializer.class, serializer.getClass()); - serializer = SerializerFactory.serializer("binaryscatter"); + serializer = SerializerFactory.serializer(config, "binaryscatter"); Assert.assertEquals(BinaryScatterSerializer.class, serializer.getClass()); Assert.assertThrows(BackendException.class, () -> { - SerializerFactory.serializer("invalid"); + SerializerFactory.serializer(config, "invalid"); }, e -> { Assert.assertContains("Not exists serializer:", e.getMessage()); }); @@ -53,9 +57,10 @@ public void testSerializer() { @Test public void testRegister() { + HugeConfig config = FakeObjects.newConfig(); SerializerFactory.register("fake", FakeSerializer.class.getName()); Assert.assertEquals(FakeSerializer.class, - SerializerFactory.serializer("fake").getClass()); + SerializerFactory.serializer(config, "fake").getClass()); Assert.assertThrows(BackendException.class, () -> { // exist @@ -82,8 +87,12 @@ public void testRegister() { public static class FakeSerializer extends BinarySerializer { + public FakeSerializer(HugeConfig config){ + super(config); + } + public FakeSerializer() { - super(true, true); + super(true, true, false); } } } From e92d02f0f581c4113d57ddf0e37e2dd33d34be39 Mon Sep 17 00:00:00 2001 From: JackyYangPassion Date: Fri, 1 Apr 2022 16:37:57 +0800 Subject: [PATCH 2/3] change HbaseFeatures for enable partition --- .../hugegraph/backend/store/hbase/HbaseFeatures.java | 10 ++++++++-- .../hugegraph/backend/store/hbase/HbaseStore.java | 11 ++++++----- .../backend/store/hbase/HbaseStoreProvider.java | 2 +- 3 files changed, 15 insertions(+), 8 deletions(-) diff --git a/hugegraph-hbase/src/main/java/com/baidu/hugegraph/backend/store/hbase/HbaseFeatures.java b/hugegraph-hbase/src/main/java/com/baidu/hugegraph/backend/store/hbase/HbaseFeatures.java index d083f4523b..03734b6168 100644 --- a/hugegraph-hbase/src/main/java/com/baidu/hugegraph/backend/store/hbase/HbaseFeatures.java +++ b/hugegraph-hbase/src/main/java/com/baidu/hugegraph/backend/store/hbase/HbaseFeatures.java @@ -23,6 +23,12 @@ public class HbaseFeatures implements BackendFeatures { + private boolean enablePartition; + + public HbaseFeatures(boolean enablePartition) { + this.enablePartition = enablePartition; + } + @Override public boolean supportsScanToken() { return false; @@ -30,12 +36,12 @@ public boolean supportsScanToken() { @Override public boolean supportsScanKeyPrefix() { - return true; + return !enablePartition; } @Override public boolean supportsScanKeyRange() { - return true; + return !enablePartition; } @Override diff --git a/hugegraph-hbase/src/main/java/com/baidu/hugegraph/backend/store/hbase/HbaseStore.java b/hugegraph-hbase/src/main/java/com/baidu/hugegraph/backend/store/hbase/HbaseStore.java index 8e52d54b4e..c09549a5ae 100644 --- a/hugegraph-hbase/src/main/java/com/baidu/hugegraph/backend/store/hbase/HbaseStore.java +++ b/hugegraph-hbase/src/main/java/com/baidu/hugegraph/backend/store/hbase/HbaseStore.java @@ -54,7 +54,7 @@ public abstract class HbaseStore extends AbstractBackendStore { private static final Logger LOG = Log.logger(HbaseStore.class); - private static final BackendFeatures FEATURES = new HbaseFeatures(); + private static BackendFeatures FEATURES; private final String store; private final String namespace; @@ -67,13 +67,14 @@ public abstract class HbaseStore extends AbstractBackendStore { private HbaseSessions sessions; public HbaseStore(final BackendStoreProvider provider, - final String namespace, final String store) { + final String namespace, final String store, boolean enablePartition) { this.tables = new HashMap<>(); this.provider = provider; this.namespace = namespace; this.store = store; this.sessions = null; + this.FEATURES = new HbaseFeatures(enablePartition); this.registerMetaHandlers(); LOG.debug("Store loaded: {}", store); @@ -442,9 +443,9 @@ public static class HbaseSchemaStore extends HbaseStore { private final HbaseTables.Counters counters; - public HbaseSchemaStore(BackendStoreProvider provider, + public HbaseSchemaStore(HugeConfig config, BackendStoreProvider provider, String namespace, String store) { - super(provider, namespace, store); + super(provider, namespace, store, config.get(HbaseOptions.HBASE_ENABLE_PARTITION).booleanValue()); this.counters = new HbaseTables.Counters(); @@ -491,7 +492,7 @@ public static class HbaseGraphStore extends HbaseStore { private boolean enablePartition; public HbaseGraphStore(HugeConfig config, BackendStoreProvider provider, String namespace, String store) { - super(provider, namespace, store); + super(provider, namespace, store, config.get(HbaseOptions.HBASE_ENABLE_PARTITION).booleanValue()); this.enablePartition = config.get(HbaseOptions.HBASE_ENABLE_PARTITION).booleanValue(); registerTableManager(HugeType.VERTEX, new HbaseTables.Vertex(store, enablePartition)); diff --git a/hugegraph-hbase/src/main/java/com/baidu/hugegraph/backend/store/hbase/HbaseStoreProvider.java b/hugegraph-hbase/src/main/java/com/baidu/hugegraph/backend/store/hbase/HbaseStoreProvider.java index 92a36f1737..f8dd33a66d 100644 --- a/hugegraph-hbase/src/main/java/com/baidu/hugegraph/backend/store/hbase/HbaseStoreProvider.java +++ b/hugegraph-hbase/src/main/java/com/baidu/hugegraph/backend/store/hbase/HbaseStoreProvider.java @@ -33,7 +33,7 @@ protected String namespace() { @Override protected BackendStore newSchemaStore(HugeConfig config, String store) { - return new HbaseSchemaStore(this, this.namespace(), store); + return new HbaseSchemaStore(config, this, this.namespace(), store); } @Override From 6d5878aa0a496889a3f1ead3370d4d83f819c9ea Mon Sep 17 00:00:00 2001 From: JackyYangPassion Date: Tue, 5 Apr 2022 22:28:39 +0800 Subject: [PATCH 3/3] optimize code --- .../baidu/hugegraph/backend/store/hbase/HbaseFeatures.java | 4 ++-- .../com/baidu/hugegraph/backend/store/hbase/HbaseStore.java | 6 +++--- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/hugegraph-hbase/src/main/java/com/baidu/hugegraph/backend/store/hbase/HbaseFeatures.java b/hugegraph-hbase/src/main/java/com/baidu/hugegraph/backend/store/hbase/HbaseFeatures.java index 03734b6168..b977f92066 100644 --- a/hugegraph-hbase/src/main/java/com/baidu/hugegraph/backend/store/hbase/HbaseFeatures.java +++ b/hugegraph-hbase/src/main/java/com/baidu/hugegraph/backend/store/hbase/HbaseFeatures.java @@ -36,12 +36,12 @@ public boolean supportsScanToken() { @Override public boolean supportsScanKeyPrefix() { - return !enablePartition; + return !this.enablePartition; } @Override public boolean supportsScanKeyRange() { - return !enablePartition; + return !this.enablePartition; } @Override diff --git a/hugegraph-hbase/src/main/java/com/baidu/hugegraph/backend/store/hbase/HbaseStore.java b/hugegraph-hbase/src/main/java/com/baidu/hugegraph/backend/store/hbase/HbaseStore.java index c09549a5ae..6b3dbb0750 100644 --- a/hugegraph-hbase/src/main/java/com/baidu/hugegraph/backend/store/hbase/HbaseStore.java +++ b/hugegraph-hbase/src/main/java/com/baidu/hugegraph/backend/store/hbase/HbaseStore.java @@ -54,7 +54,7 @@ public abstract class HbaseStore extends AbstractBackendStore { private static final Logger LOG = Log.logger(HbaseStore.class); - private static BackendFeatures FEATURES; + private final BackendFeatures FEATURES; private final String store; private final String namespace; @@ -66,8 +66,8 @@ public abstract class HbaseStore extends AbstractBackendStore { private HbaseSessions sessions; - public HbaseStore(final BackendStoreProvider provider, - final String namespace, final String store, boolean enablePartition) { + public HbaseStore(BackendStoreProvider provider, + String namespace, String store, boolean enablePartition) { this.tables = new HashMap<>(); this.provider = provider;