From 171b33ad8ef61075e6c73798004693aef92ac68c Mon Sep 17 00:00:00 2001 From: Zhangmei Li Date: Fri, 21 Dec 2018 13:07:11 +0800 Subject: [PATCH] Support range sortKey feature implement #271 Change-Id: Ib618c45bc6169a0fb7632dbbf0fb29b7ee0a08e3 --- hugegraph-api/pom.xml | 2 +- hugegraph-cassandra/pom.xml | 2 +- .../cassandra/CassandraStoreProvider.java | 3 +- hugegraph-core/pom.xml | 6 +- .../hugegraph/backend/query/Condition.java | 10 + .../backend/query/ConditionQuery.java | 26 +- .../backend/query/IdPrefixQuery.java | 93 +++++++ .../hugegraph/backend/query/IdRangeQuery.java | 104 ++++++++ .../serializer/AbstractSerializer.java | 8 +- .../serializer/BinaryBackendEntry.java | 3 +- .../backend/serializer/BinarySerializer.java | 243 +++++++++++++----- .../backend/serializer/BytesBuffer.java | 4 + .../backend/serializer/TableSerializer.java | 2 +- .../backend/serializer/TextBackendEntry.java | 82 ++++-- .../backend/serializer/TextSerializer.java | 77 +++++- .../hugegraph/backend/store/BackendEntry.java | 63 ++++- .../hugegraph/backend/store/BackendTable.java | 9 +- .../store/memory/InMemoryDBStoreProvider.java | 3 +- .../backend/store/memory/InMemoryDBTable.java | 140 +++------- .../store/memory/InMemoryDBTables.java | 190 ++++++++++++++ .../backend/tx/GraphIndexTransaction.java | 15 +- .../backend/tx/GraphTransaction.java | 34 ++- .../baidu/hugegraph/schema/PropertyKey.java | 68 +++-- .../hugegraph/structure/HugeProperty.java | 6 +- .../baidu/hugegraph/util/StringEncoding.java | 6 + .../baidu/hugegraph/version/CoreVersion.java | 3 +- hugegraph-dist/pom.xml | 2 +- hugegraph-example/pom.xml | 2 +- .../com/baidu/hugegraph/example/Example1.java | 7 +- hugegraph-hbase/pom.xml | 2 +- .../backend/store/hbase/HbaseSerializer.java | 27 -- .../backend/store/hbase/HbaseSessions.java | 46 ++++ .../store/hbase/HbaseStoreProvider.java | 3 +- .../backend/store/hbase/HbaseTable.java | 30 ++- .../backend/store/hbase/HbaseTables.java | 52 ++-- hugegraph-mysql/pom.xml | 2 +- .../backend/store/mysql/MysqlSessions.java | 4 +- .../store/mysql/MysqlStoreProvider.java | 3 +- .../backend/store/mysql/MysqlTable.java | 2 +- hugegraph-palo/pom.xml | 2 +- .../backend/store/palo/PaloStoreProvider.java | 3 +- hugegraph-rocksdb/pom.xml | 2 +- .../store/rocksdb/RocksDBStdSessions.java | 48 ++-- .../store/rocksdb/RocksDBStoreProvider.java | 3 +- .../backend/store/rocksdb/RocksDBTable.java | 42 ++- .../backend/store/rocksdb/RocksDBTables.java | 15 +- hugegraph-scylladb/pom.xml | 2 +- hugegraph-test/pom.xml | 2 +- .../baidu/hugegraph/core/EdgeCoreTest.java | 79 ++++++ .../baidu/hugegraph/unit/UnitTestSuite.java | 2 + .../hugegraph/unit/core/VersionTest.java | 44 ++++ pom.xml | 2 +- 52 files changed, 1214 insertions(+), 416 deletions(-) create mode 100644 hugegraph-core/src/main/java/com/baidu/hugegraph/backend/query/IdPrefixQuery.java create mode 100644 hugegraph-core/src/main/java/com/baidu/hugegraph/backend/query/IdRangeQuery.java create mode 100644 hugegraph-test/src/main/java/com/baidu/hugegraph/unit/core/VersionTest.java diff --git a/hugegraph-api/pom.xml b/hugegraph-api/pom.xml index 2b20c21659..0ef7194e7e 100644 --- a/hugegraph-api/pom.xml +++ b/hugegraph-api/pom.xml @@ -5,7 +5,7 @@ hugegraph com.baidu.hugegraph - 0.9.0 + 0.9.1 4.0.0 diff --git a/hugegraph-cassandra/pom.xml b/hugegraph-cassandra/pom.xml index 4ba4f0bfc8..5f98a01252 100644 --- a/hugegraph-cassandra/pom.xml +++ b/hugegraph-cassandra/pom.xml @@ -5,7 +5,7 @@ hugegraph com.baidu.hugegraph - 0.9.0 + 0.9.1 4.0.0 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 70bfdd6074..402f7ab642 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 @@ -51,7 +51,8 @@ public String version() { * Versions history: * [1.0] HugeGraph-1328: supports backend table version checking * [1.1] HugeGraph-1322: add support for full-text search + * [1.2] #296: support range sortKey feature */ - return "1.1"; + return "1.2"; } } diff --git a/hugegraph-core/pom.xml b/hugegraph-core/pom.xml index f0012c997e..01ec1f1520 100644 --- a/hugegraph-core/pom.xml +++ b/hugegraph-core/pom.xml @@ -5,7 +5,7 @@ com.baidu.hugegraph hugegraph - 0.9.0 + 0.9.1 ../pom.xml hugegraph-core @@ -19,7 +19,7 @@ com.baidu.hugegraph hugegraph-common - 1.5.3 + 1.5.6 @@ -157,7 +157,7 @@ - 0.9.0.0 + 0.9.1.0 diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/query/Condition.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/query/Condition.java index d820198fc2..06edca2924 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/query/Condition.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/query/Condition.java @@ -204,6 +204,10 @@ public boolean isLogic() { this.type() == ConditionType.OR; } + public boolean isFlattened() { + return this.isRelation(); + } + public static Condition and(Condition left, Condition right) { return new And(left, right); } @@ -393,6 +397,12 @@ public boolean test(HugeElement element) { public Condition copy() { return new And(this.left().copy(), this.right().copy()); } + + @Override + public boolean isFlattened() { + // If this is flattened, its sub-condition should not be nested + return this.left().isRelation() && this.right().isRelation(); + } } public static class Or extends BinCondition { diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/query/ConditionQuery.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/query/ConditionQuery.java index 005451cc2c..a83a755659 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/query/ConditionQuery.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/query/ConditionQuery.java @@ -142,7 +142,7 @@ public Object condition(Object key) { for (Condition c : this.conditions) { if (c.isRelation()) { Condition.Relation r = (Condition.Relation) c; - if (r.key().equals(key)) { + if (r.key().equals(key) && r.relation() == RelationType.EQ) { values.add(r.value()); } } @@ -277,11 +277,11 @@ public Set userpropKeys() { /** * This method is only used for secondary index scenario, - * relation must be EQ + * its relation must be EQ * @param fields the user property fields - * @return the corresponding user property values of fileds + * @return the corresponding user property serial values of fields */ - public List userpropValues(List fields) { + public String userpropValuesString(List fields) { List values = new ArrayList<>(fields.size()); for (Id field : fields) { boolean got = false; @@ -290,9 +290,9 @@ public List userpropValues(List fields) { E.checkState(r.relation == RelationType.EQ, "Method userpropValues(List) only " + "used for secondary index, " + - "relation must be EQ, but got '%s'", + "relation must be EQ, but got %s", r.relation()); - values.add(r.value()); + values.add(r.serialValue()); got = true; } } @@ -302,18 +302,14 @@ public List userpropValues(List fields) { field, this); } } - return values; - } - - public String userpropValuesString(List fields) { - return SplicingIdGenerator.concatValues(this.userpropValues(fields)); + return SplicingIdGenerator.concatValues(values); } public Set userpropValues(Id field) { Set values = new HashSet<>(); for (Relation r : this.userpropRelations()) { if (r.key().equals(field)) { - values.add(r.value()); + values.add(r.serialValue()); } } return values; @@ -325,8 +321,8 @@ public Object userpropValue(Id field) { return null; } E.checkState(values.size() == 1, - "Expect one user-property value of field '%s', but got %s", - field, values.size()); + "Expect one user-property value of field '%s', " + + "but got '%s'", field, values.size()); return values.iterator().next(); } @@ -393,7 +389,7 @@ public boolean test(HugeElement element) { public void checkFlattened() { for (Condition condition : this.conditions) { - E.checkState(condition.isRelation(), + E.checkState(condition.isFlattened(), "Condition Query has none-flatten condition '%s'", condition); } diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/query/IdPrefixQuery.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/query/IdPrefixQuery.java new file mode 100644 index 0000000000..48cf77a9f9 --- /dev/null +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/query/IdPrefixQuery.java @@ -0,0 +1,93 @@ +/* + * Copyright 2017 HugeGraph Authors + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with this + * work for additional information regarding copyright ownership. The ASF + * licenses this file to You under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations + * under the License. + */ + +package com.baidu.hugegraph.backend.query; + +import com.baidu.hugegraph.backend.id.Id; +import com.baidu.hugegraph.structure.HugeElement; +import com.baidu.hugegraph.type.HugeType; +import com.baidu.hugegraph.util.Bytes; +import com.baidu.hugegraph.util.E; + +public class IdPrefixQuery extends Query { + + private final Id start; + private final boolean inclusiveStart; + private final Id prefix; + + public IdPrefixQuery(HugeType resultType, Id prefix) { + this(resultType, null, prefix, true, prefix); + } + + public IdPrefixQuery(Query originQuery, Id prefix) { + this(originQuery.resultType(), originQuery, prefix, true, prefix); + } + + public IdPrefixQuery(Query originQuery, + Id start, boolean inclusive, Id prefix) { + this(originQuery.resultType(), originQuery, start, inclusive, prefix); + } + + public IdPrefixQuery(HugeType resultType, Query originQuery, + Id start, boolean inclusive, Id prefix) { + super(resultType, originQuery); + E.checkArgumentNotNull(start, "The start parameter can't be null"); + this.start = start; + this.inclusiveStart = inclusive; + this.prefix = prefix; + } + + public Id start() { + return this.start; + } + + public boolean inclusiveStart() { + return this.inclusiveStart; + } + + public Id prefix() { + return this.prefix; + } + + @Override + public boolean empty() { + return false; + } + + @Override + public boolean test(HugeElement element) { + byte[] elem = element.id().asBytes(); + int cmp = Bytes.compare(elem, this.start.asBytes()); + boolean matchedStart = this.inclusiveStart ? cmp >= 0 : cmp > 0; + boolean matchedPrefix = Bytes.prefixWith(elem, this.prefix.asBytes()); + return matchedStart && matchedPrefix; + } + + @Override + public IdPrefixQuery copy() { + return (IdPrefixQuery) super.copy(); + } + + @Override + public String toString() { + return String.format("%s where id prefix with %s and start with %s(%s)", + super.toString(), this.prefix, this.start, + this.inclusiveStart ? "inclusive" : "exclusive"); + } +} diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/query/IdRangeQuery.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/query/IdRangeQuery.java new file mode 100644 index 0000000000..3c643282c8 --- /dev/null +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/query/IdRangeQuery.java @@ -0,0 +1,104 @@ +/* + * Copyright 2017 HugeGraph Authors + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with this + * work for additional information regarding copyright ownership. The ASF + * licenses this file to You under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations + * under the License. + */ + +package com.baidu.hugegraph.backend.query; + +import com.baidu.hugegraph.backend.id.Id; +import com.baidu.hugegraph.structure.HugeElement; +import com.baidu.hugegraph.type.HugeType; +import com.baidu.hugegraph.util.Bytes; +import com.baidu.hugegraph.util.E; + +public class IdRangeQuery extends Query { + + private final Id start; + private final Id end; + private final boolean inclusiveStart; + private final boolean inclusiveEnd; + + public IdRangeQuery(HugeType resultType, Id start, Id end) { + this(resultType, null, start, end); + } + + public IdRangeQuery(HugeType resultType, Query originQuery, + Id start, Id end) { + this(resultType, originQuery, start, true, end, false); + } + + public IdRangeQuery(Query originQuery, + Id start, boolean inclusiveStart, + Id end, boolean inclusiveEnd) { + this(originQuery.resultType(), originQuery, + start, inclusiveStart, end, inclusiveEnd); + } + + public IdRangeQuery(HugeType resultType, Query originQuery, + Id start, boolean inclusiveStart, + Id end, boolean inclusiveEnd) { + super(resultType, originQuery); + E.checkArgumentNotNull(start, "The start parameter can't be null"); + this.start = start; + this.end = end; + this.inclusiveStart = inclusiveStart; + this.inclusiveEnd = inclusiveEnd; + } + + public Id start() { + return this.start; + } + + public Id end() { + return this.end; + } + + public boolean inclusiveStart() { + return this.inclusiveStart; + } + + public boolean inclusiveEnd() { + return this.inclusiveEnd; + } + + @Override + public boolean empty() { + return false; + } + + @Override + public boolean test(HugeElement element) { + int cmp1 = Bytes.compare(element.id().asBytes(), this.start.asBytes()); + int cmp2 = Bytes.compare(element.id().asBytes(), this.end.asBytes()); + return (this.inclusiveStart ? cmp1 >= 0 : cmp1 > 0) && + (this.inclusiveEnd ? cmp2 <= 0 : cmp2 < 0); + } + + @Override + public IdRangeQuery copy() { + return (IdRangeQuery) super.copy(); + } + + @Override + public String toString() { + return String.format("%s where id in range %s%s,%s%s", + super.toString(), + this.inclusiveStart ? "[" : "(", + this.start, this.end, + this.inclusiveEnd ? "]" : ")"); + } +} 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 e51bbafffe..509507f876 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 @@ -38,7 +38,7 @@ protected BackendEntry convertEntry(BackendEntry entry) { protected abstract Id writeQueryId(HugeType type, Id id); - protected abstract Id writeQueryEdgeCondition(Query query); + protected abstract Query writeQueryEdgeCondition(Query query); protected abstract Query writeQueryCondition(Query query); @@ -53,9 +53,9 @@ public Query writeQuery(Query query) { "and by condition at the same time"); } - Id id = this.writeQueryEdgeCondition(query); - if (id != null) { - return new IdQuery(query, id); + Query result = this.writeQueryEdgeCondition(query); + if (result != null) { + return result; } } 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 b477b7b58c..703691c477 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 @@ -31,7 +31,6 @@ import com.baidu.hugegraph.type.HugeType; import com.baidu.hugegraph.util.Bytes; import com.baidu.hugegraph.util.E; -import com.baidu.hugegraph.util.StringEncoding; public class BinaryBackendEntry implements BackendEntry { @@ -229,7 +228,7 @@ public boolean equals(Object other) { @Override public String toString() { - return StringEncoding.decode(this.bytes); + return "0x" + Bytes.toHex(this.bytes); } } } 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 fb828d6929..0d4ebc8702 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 @@ -32,7 +32,9 @@ import com.baidu.hugegraph.backend.query.Condition; import com.baidu.hugegraph.backend.query.Condition.Relation; import com.baidu.hugegraph.backend.query.ConditionQuery; +import com.baidu.hugegraph.backend.query.IdPrefixQuery; import com.baidu.hugegraph.backend.query.IdQuery; +import com.baidu.hugegraph.backend.query.IdRangeQuery; import com.baidu.hugegraph.backend.query.Query; import com.baidu.hugegraph.backend.serializer.BinaryBackendEntry.BinaryId; import com.baidu.hugegraph.backend.store.BackendEntry; @@ -63,6 +65,10 @@ public class BinarySerializer extends AbstractSerializer { private static final byte[] EMPTY_BYTES = new byte[0]; + /* + * Id is stored in column name if keyWithIdPrefix=true like RocksDB, + * else stored in rowkey like HBase. + */ private final boolean keyWithIdPrefix; public BinarySerializer() { @@ -296,15 +302,24 @@ else if (type == HugeType.SYS_PROPERTY.code()) { } protected byte[] formatIndexName(HugeIndex index) { - Id indexId = index.id(); - if (indexIdLengthExceedLimit(indexId)) { - indexId = index.hashId(); - } Id elemId = index.elementId(); - BytesBuffer buffer = BytesBuffer.allocate(1 + indexId.length() + - 1 + elemId.length()); - // Write index-id + element-id - buffer.writeId(indexId); + int idLen = 1 + elemId.length(); + + BytesBuffer buffer; + if (!this.keyWithIdPrefix) { + buffer = BytesBuffer.allocate(idLen); + } else { + Id indexId = index.id(); + if (indexIdLengthExceedLimit(indexId)) { + indexId = index.hashId(); + } + // Write index-id + idLen += 1 + indexId.length(); + buffer = BytesBuffer.allocate(idLen); + buffer.writeId(indexId); + } + + // Write element-id buffer.writeId(elemId, true); return buffer.bytes(); @@ -314,11 +329,13 @@ protected void parseIndexName(BinaryBackendEntry entry, HugeIndex index, Object fieldValues) { for (BackendColumn col : entry.columns()) { if (indexFieldValuesUnmatched(col.value, fieldValues)) { - // Skip if field-values don't matched (just the same hash) + // Skip if field-values is not matched (just the same hash) continue; } BytesBuffer buffer = BytesBuffer.wrap(col.name); - buffer.readId(); + if (this.keyWithIdPrefix) { + buffer.readId(); + } index.elementIds(buffer.readId(true)); } } @@ -463,16 +480,83 @@ protected Id writeQueryId(HugeType type, Id id) { } @Override - protected Id writeQueryEdgeCondition(Query query) { + protected Query writeQueryEdgeCondition(Query query) { + ConditionQuery cq = (ConditionQuery) query; + if (cq.hasRangeCondition()) { + return this.writeQueryEdgeRangeCondition(cq); + } else { + return this.writeQueryEdgePrefixCondition(cq); + } + } + + private Query writeQueryEdgeRangeCondition(ConditionQuery cq) { + List sortValues = cq.syspropConditions(HugeKeys.SORT_VALUES); + E.checkArgument(sortValues.size() >= 1 && sortValues.size() <= 2, + "Edge range query must be with sort-values range"); + // Would ignore target vertex + Object vertex = cq.condition(HugeKeys.OWNER_VERTEX); + Object direction = cq.condition(HugeKeys.DIRECTION); + if (direction == null) { + direction = Directions.OUT; + } + Object label = cq.condition(HugeKeys.LABEL); + + BytesBuffer start = BytesBuffer.allocate(256); + start.writeId(HugeVertex.getIdValue(vertex)); + start.write(((Directions) direction).type().code()); + start.writeId((Id) label); + + BytesBuffer end = BytesBuffer.allocate(256); + end.copyFrom(start); + + int minEq = -1; + int maxEq = -1; + for (Condition sortValue : sortValues) { + Condition.Relation r = (Condition.Relation) sortValue; + switch (r.relation()) { + case GTE: + minEq = 1; + start.writeString((String) r.value()); + break; + case GT: + minEq = 0; + start.writeString((String) r.value()); + break; + case LTE: + maxEq = 1; + end.writeString((String) r.value()); + break; + case LT: + maxEq = 0; + end.writeString((String) r.value()); + break; + default: + E.checkArgument(false, "Unsupported relation '%s'", + r.relation()); + } + } + + // Sort-value will be empty if there is no start sort-value + Id startId = new BinaryId(start.bytes(), null); + // Set endId as prefix if there is no end sort-value + Id endId = new BinaryId(end.bytes(), null); + if (maxEq == -1) { + return new IdPrefixQuery(cq, startId, minEq == 1, endId); + } + return new IdRangeQuery(cq, startId, minEq == 1, endId, maxEq == 1); + } + + private Query writeQueryEdgePrefixCondition(ConditionQuery cq) { int count = 0; BytesBuffer buffer = BytesBuffer.allocate(256); for (HugeKeys key : EdgeId.KEYS) { - Object value = ((ConditionQuery) query).condition(key); + Object value = cq.condition(key); if (value != null) { count++; } else { if (key == HugeKeys.DIRECTION) { + // Direction is null, set to OUT value = Directions.OUT; } else { break; @@ -498,8 +582,8 @@ protected Id writeQueryEdgeCondition(Query query) { } if (count > 0) { - assert count == query.conditions().size(); - return new BinaryId(buffer.bytes(), null); + assert count == cq.conditions().size(); + return new IdPrefixQuery(cq, new BinaryId(buffer.bytes(), null)); } return null; @@ -528,49 +612,6 @@ protected Query writeQueryCondition(Query query) { return null; } - private BinaryBackendEntry formatILDeletion(HugeIndex index) { - Id id = index.indexLabel(); - BinaryBackendEntry entry = newBackendEntry(index.type(), id); - switch (index.type()) { - case SECONDARY_INDEX: - case SEARCH_INDEX: - String idString = id.asString(); - int idLength = idString.length(); - // TODO: to improve, use BytesBuffer to generate a mask - for (int i = idLength - 1; i < 128; i++) { - BytesBuffer buffer = BytesBuffer.allocate(idLength + 1); - /* - * Index id type is always non-number, that it will prefix - * with '0b1xxx xxxx' - */ - buffer.writeUInt8(i | 0x80); - buffer.write(IdGenerator.of(idString).asBytes()); - entry.column(buffer.bytes(), null); - } - break; - case RANGE_INDEX: - int il = (int) id.asLong(); - for (int i = 0; i < 4; i++) { - /* - * Field value(Number type) length is 1, 2, 4, 8. - * Index label id length is 4 - */ - int length = (int) Math.pow(2, i) + 4; - length -= 1; - BytesBuffer buffer = BytesBuffer.allocate(1 + 4); - buffer.writeUInt8(length | 0x80); - buffer.writeInt(il); - entry.column(buffer.bytes(), null); - } - break; - default: - throw new AssertionError(String.format( - "Index type must be Secondary or Range, " + - "but got '%s'", index.type())); - } - return entry; - } - private Query writeStringIndexQuery(ConditionQuery query) { E.checkArgument(query.allSysprop() && query.conditions().size() == 2, @@ -645,30 +686,68 @@ private Query writeRangeIndexQuery(ConditionQuery query) { keyMinEq = true; } - query = query.copy(); - query.resetConditions(); - Id min = formatIndexId(type, index, keyMin); - if (keyMinEq) { - query.gte(HugeKeys.ID, min); - } else { - query.gt(HugeKeys.ID, min); + if (!keyMinEq) { + /* + * Increase 1 to keyMin, index GT query is a scan with GT prefix, + * inclusiveStart=false will also match index started with keyMin + */ + increaseOne(min.asBytes()); + keyMinEq = true; } if (keyMax == null) { Id prefix = formatIndexId(type, index, null); // Reset the first byte to make same length-prefix prefix.asBytes()[0] = min.asBytes()[0]; - query.prefix(HugeKeys.ID, prefix); + return new IdPrefixQuery(query, min, keyMinEq, prefix); } else { Id max = formatIndexId(type, index, keyMax); - if (keyMaxEq) { - query.lte(HugeKeys.ID, max); - } else { - query.lt(HugeKeys.ID, max); - } + return new IdRangeQuery(query, min, keyMinEq, max, keyMaxEq); } - return query; + } + + private BinaryBackendEntry formatILDeletion(HugeIndex index) { + Id id = index.indexLabel(); + BinaryBackendEntry entry = newBackendEntry(index.type(), id); + switch (index.type()) { + case SECONDARY_INDEX: + case SEARCH_INDEX: + String idString = id.asString(); + int idLength = idString.length(); + // TODO: to improve, use BytesBuffer to generate a mask + for (int i = idLength - 1; i < 128; i++) { + BytesBuffer buffer = BytesBuffer.allocate(idLength + 1); + /* + * Index id type is always non-number, that it will prefix + * with '0b1xxx xxxx' + */ + buffer.writeUInt8(i | 0x80); + buffer.write(IdGenerator.of(idString).asBytes()); + entry.column(buffer.bytes(), null); + } + break; + case RANGE_INDEX: + int il = (int) id.asLong(); + for (int i = 0; i < 4; i++) { + /* + * Field value(Number type) length is 1, 2, 4, 8. + * Index label id length is 4 + */ + int length = (int) Math.pow(2, i) + 4; + length -= 1; + BytesBuffer buffer = BytesBuffer.allocate(1 + 4); + buffer.writeUInt8(length | 0x80); + buffer.writeInt(il); + entry.column(buffer.bytes(), null); + } + break; + default: + throw new AssertionError(String.format( + "Index type must be Secondary or Range, " + + "but got '%s'", index.type())); + } + return entry; } private static BinaryId writeEdgeId(Id id) { @@ -711,6 +790,28 @@ protected static boolean indexFieldValuesUnmatched(byte[] value, return false; } + public static final byte[] increaseOne(byte[] bytes) { + final byte BYTE_MAX_VALUE = (byte) 0xff; + assert bytes.length > 0; + byte last = bytes[bytes.length - 1]; + if (last != BYTE_MAX_VALUE) { + bytes[bytes.length - 1] += 0x01; + } else { + // Process overflow (like [1, 255] => [2, 0]) + int i = bytes.length - 1; + for (; i > 0 && bytes[i] == BYTE_MAX_VALUE; --i) { + bytes[i] += 0x01; + } + if (bytes[i] == BYTE_MAX_VALUE) { + assert i == 0; + throw new BackendException("Unable to increase bytes: %s", + Bytes.toHex(bytes)); + } + bytes[i] += 0x01; + } + return bytes; + } + // TODO: remove these methods when improving schema serialize private static String splitKeyId(byte[] bytes) { BytesBuffer buffer = BytesBuffer.wrap(bytes); 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 4d2c545ef7..8f96bdc108 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 @@ -101,6 +101,10 @@ public byte[] bytes() { } } + public BytesBuffer copyFrom(BytesBuffer other) { + return this.write(other.bytes()); + } + public int remaining() { return this.buffer.remaining(); } 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 f11f34fd6c..63a5047b12 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 @@ -331,7 +331,7 @@ protected Id writeQueryId(HugeType type, Id id) { } @Override - protected Id writeQueryEdgeCondition(Query query) { + protected Query writeQueryEdgeCondition(Query query) { ConditionQuery result = (ConditionQuery) query; for (Condition.Relation r : result.relations()) { Object value = r.value(); diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/serializer/TextBackendEntry.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/serializer/TextBackendEntry.java index 780277f184..9e3be1888d 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/serializer/TextBackendEntry.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/serializer/TextBackendEntry.java @@ -26,8 +26,9 @@ import java.util.List; import java.util.Map; import java.util.Map.Entry; +import java.util.NavigableMap; import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentSkipListMap; import com.baidu.hugegraph.backend.BackendException; import com.baidu.hugegraph.backend.id.Id; @@ -44,13 +45,13 @@ public class TextBackendEntry implements BackendEntry, Cloneable { private final HugeType type; private final Id id; private Id subId; - private Map columns; + private NavigableMap columns; public TextBackendEntry(HugeType type, Id id) { this.type = type; this.id = id; this.subId = null; - this.columns = new ConcurrentHashMap<>(); + this.resetColumns(); } @Override @@ -92,17 +93,63 @@ public String column(String column) { return this.columns.get(column); } + public BackendColumn columns(String column) { + String value = this.columns.get(column); + if (value == null) { + return null; + } + return BackendColumn.of(StringEncoding.encode(column), + StringEncoding.encode(value)); + } + + public Collection columnsWithPrefix(String prefix) { + return this.columnsWithPrefix(prefix, true, prefix); + } + + public Collection columnsWithPrefix(String start, + boolean inclusiveStart, + String prefix) { + List list = new ArrayList<>(); + Map map = this.columns.tailMap(start, inclusiveStart); + for (Map.Entry e : map.entrySet()) { + String key = e.getKey(); + String value = e.getValue(); + if (key.startsWith(prefix)) { + list.add(BackendColumn.of(StringEncoding.encode(key), + StringEncoding.encode(value))); + } + } + return list; + } + + public Collection columnsWithRange(String start, + boolean inclusiveStart, + String end, + boolean inclusiveEnd) { + List list = new ArrayList<>(); + Map map = this.columns.subMap(start, inclusiveStart, + end, inclusiveEnd); + for (Map.Entry e : map.entrySet()) { + String key = e.getKey(); + String value = e.getValue(); + list.add(BackendColumn.of(StringEncoding.encode(key), + StringEncoding.encode(value))); + } + return list; + } + public boolean contains(String column) { return this.columns.containsKey(column); } public boolean contains(String column, String value) { - return this.columns.containsKey(column) && - this.columns.get(column).equals(value); + String col = this.columns.get(column); + return col != null && col.equals(value); } public boolean containsPrefix(String column) { - for (String c : this.columns.keySet()) { + Map map = this.columns.tailMap(column, true); + for (String c : map.keySet()) { if (c.startsWith(column)) { return true; } @@ -114,19 +161,6 @@ public boolean containsValue(String value) { return this.columns.values().contains(value); } - public Collection columnsWithPrefix(String column) { - List list = new ArrayList<>(); - for (Map.Entry e : this.columns.entrySet()) { - String key = e.getKey(); - String value = e.getValue(); - if (key.startsWith(column)) { - list.add(BackendColumn.of(StringEncoding.encode(key), - StringEncoding.encode(value))); - } - } - return list; - } - public void append(TextBackendEntry entry) { for (Entry col : entry.columns.entrySet()) { String newValue = col.getValue(); @@ -232,10 +266,14 @@ public void clear() { this.columns.clear(); } + private void resetColumns() { + this.columns = new ConcurrentSkipListMap<>(); + } + public TextBackendEntry copy() { try { TextBackendEntry clone = (TextBackendEntry) this.clone(); - clone.columns = new ConcurrentHashMap<>(this.columns); + clone.columns = new ConcurrentSkipListMap<>(this.columns); return clone; } catch (CloneNotSupportedException e) { throw new BackendException(e); @@ -249,7 +287,7 @@ public TextBackendEntry copyLast(int count) { } catch (CloneNotSupportedException e) { throw new BackendException(e); } - clone.columns = new ConcurrentHashMap<>(); + clone.resetColumns(); // Copy the last count columns Iterator> it = this.columns.entrySet().iterator(); @@ -271,7 +309,7 @@ public TextBackendEntry copyHead(int count) { } catch (CloneNotSupportedException e) { throw new BackendException(e); } - clone.columns = new ConcurrentHashMap<>(); + clone.resetColumns(); // Copy the head count columns Iterator> it = this.columns.entrySet().iterator(); 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 06ce169ae7..4ba3b8eb20 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 @@ -36,6 +36,8 @@ import com.baidu.hugegraph.backend.id.SplicingIdGenerator; import com.baidu.hugegraph.backend.query.Condition; import com.baidu.hugegraph.backend.query.ConditionQuery; +import com.baidu.hugegraph.backend.query.IdPrefixQuery; +import com.baidu.hugegraph.backend.query.IdRangeQuery; import com.baidu.hugegraph.backend.query.Query; import com.baidu.hugegraph.backend.store.BackendEntry; import com.baidu.hugegraph.schema.EdgeLabel; @@ -384,12 +386,79 @@ protected Id writeQueryId(HugeType type, Id id) { } @Override - protected Id writeQueryEdgeCondition(Query query) { + protected Query writeQueryEdgeCondition(Query query) { + ConditionQuery cq = (ConditionQuery) query; + if (cq.hasRangeCondition()) { + return this.writeQueryEdgeRangeCondition(cq); + } else { + return this.writeQueryEdgePrefixCondition(cq); + } + } + + private Query writeQueryEdgeRangeCondition(ConditionQuery cq) { + List sortValues = cq.syspropConditions(HugeKeys.SORT_VALUES); + E.checkArgument(sortValues.size() >= 1 && sortValues.size() <= 2, + "Edge range query must be with sort-values range"); + // Would ignore target vertex + Object vertex = cq.condition(HugeKeys.OWNER_VERTEX); + Object direction = cq.condition(HugeKeys.DIRECTION); + if (direction == null) { + direction = Directions.OUT; + } + Object label = cq.condition(HugeKeys.LABEL); + + List start = new ArrayList<>(cq.conditions().size()); + start.add(writeEntryId((Id) vertex)); + start.add(writeType(((Directions) direction).type())); + start.add(writeId((Id) label)); + + List end = new ArrayList<>(start); + + int minEq = -1; + int maxEq = -1; + for (Condition sortValue : sortValues) { + Condition.Relation r = (Condition.Relation) sortValue; + switch (r.relation()) { + case GTE: + minEq = 1; + start.add((String) r.value()); + break; + case GT: + minEq = 0; + start.add((String) r.value()); + break; + case LTE: + maxEq = 1; + end.add((String) r.value()); + break; + case LT: + maxEq = 0; + end.add((String) r.value()); + break; + default: + E.checkArgument(false, "Unsupported relation '%s'", + r.relation()); + } + } + + // Sort-value will be empty if there is no start sort-value + String startId = EdgeId.concat(start.toArray(new String[0])); + // Set endId as prefix if there is no end sort-value + String endId = EdgeId.concat(end.toArray(new String[0])); + if (maxEq == -1) { + return new IdPrefixQuery(cq, IdGenerator.of(startId), minEq == 1, + IdGenerator.of(endId)); + } + return new IdRangeQuery(cq, IdGenerator.of(startId), minEq == 1, + IdGenerator.of(endId), maxEq == 1); + } + + private Query writeQueryEdgePrefixCondition(ConditionQuery cq) { // Convert query-by-condition to query-by-id - List condParts = new ArrayList<>(query.conditions().size()); + List condParts = new ArrayList<>(cq.conditions().size()); for (HugeKeys key : EdgeId.KEYS) { - Object value = ((ConditionQuery) query).condition(key); + Object value = cq.condition(key); if (value == null) { break; } @@ -408,7 +477,7 @@ protected Id writeQueryEdgeCondition(Query query) { if (condParts.size() > 0) { // Conditions to id String id = EdgeId.concat(condParts.toArray(new String[0])); - return IdGenerator.of(id); + return new IdPrefixQuery(cq, IdGenerator.of(id)); } return null; diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/BackendEntry.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/BackendEntry.java index adb9beb81a..bb02409dda 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/BackendEntry.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/BackendEntry.java @@ -19,6 +19,7 @@ package com.baidu.hugegraph.backend.store; +import java.util.Arrays; import java.util.Collection; import java.util.Iterator; import java.util.NoSuchElementException; @@ -54,7 +55,17 @@ public int compareTo(BackendColumn other) { if (other == null) { return 1; } - return Bytes.compare(name, other.name); + return Bytes.compare(this.name, other.name); + } + + @Override + public boolean equals(Object obj) { + if (!(obj instanceof BackendColumn)) { + return false; + } + BackendColumn other = (BackendColumn) obj; + return Bytes.equals(this.name, other.name) && + Bytes.equals(this.value, other.value); } } @@ -84,12 +95,16 @@ public interface BackendIterator extends Iterator { public byte[] position(); - @SuppressWarnings("unchecked") - public static BackendIterator empty() { - return (BackendIterator) EMPTY; + } + + public interface BackendColumnIterator + extends BackendIterator { + + public static BackendColumnIterator empty() { + return EMPTY; } - public final BackendIterator EMPTY = new BackendIterator() { + public final BackendColumnIterator EMPTY = new BackendColumnIterator() { @Override public boolean hasNext() { @@ -97,7 +112,7 @@ public boolean hasNext() { } @Override - public Object next() { + public BackendColumn next() { throw new NoSuchElementException(); } @@ -113,11 +128,37 @@ public byte[] position() { }; } - public interface BackendColumnIterator - extends BackendIterator { + public static class BackendColumnIteratorWrapper + implements BackendColumnIterator { - public static BackendColumnIterator empty() { - return (BackendColumnIterator) EMPTY; + private final Iterator itor; + + public BackendColumnIteratorWrapper(Iterator itor) { + this.itor = itor; } - }; + + public BackendColumnIteratorWrapper(BackendColumn... cols) { + this.itor = Arrays.asList(cols).iterator(); + } + + @Override + public boolean hasNext() { + return itor.hasNext(); + } + + @Override + public BackendColumn next() { + return itor.next(); + } + + @Override + public void close() { + // pass + } + + @Override + public byte[] position() { + return null; + } + } } diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/BackendTable.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/BackendTable.java index f9a6299fb6..cf2b60ca30 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/BackendTable.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/BackendTable.java @@ -24,7 +24,6 @@ import java.util.List; import com.baidu.hugegraph.backend.query.ConditionQuery; -import com.baidu.hugegraph.backend.query.IdQuery; import com.baidu.hugegraph.backend.query.Query; import com.baidu.hugegraph.backend.serializer.BytesBuffer; import com.baidu.hugegraph.type.HugeType; @@ -76,11 +75,11 @@ public static HugeType tableType(Query query) { // We assume query OUT edges type = HugeType.EDGE_OUT; - if (!query.ids().isEmpty() && query instanceof IdQuery && - query.originQuery() != null) { + while (!(query instanceof ConditionQuery || + query.originQuery() == null)) { /* - * Some backends may trans ConditionQuery to IdQuery like - * RocksDB, so we should get the origin query + * Some backends(like RocksDB) may trans ConditionQuery to + * IdQuery or IdPrefixQuery, so we should get the origin query. */ query = query.originQuery(); } 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 a0363dfa4f..0f24e6c9f3 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 @@ -86,7 +86,8 @@ public String version() { * Versions history: * [1.0] HugeGraph-1328: supports backend table version checking * [1.1] HugeGraph-1322: add support for full-text search + * [1.2] #296: support range sortKey feature */ - return "1.1"; + return "1.2"; } } diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/memory/InMemoryDBTable.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/memory/InMemoryDBTable.java index 8f72a7dfb3..0d8f3781c6 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/memory/InMemoryDBTable.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/memory/InMemoryDBTable.java @@ -20,20 +20,19 @@ package com.baidu.hugegraph.backend.store.memory; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Set; -import java.util.concurrent.ConcurrentSkipListMap; +import java.util.concurrent.ConcurrentHashMap; import com.baidu.hugegraph.backend.BackendException; -import com.baidu.hugegraph.backend.id.EdgeId; import com.baidu.hugegraph.backend.id.Id; -import com.baidu.hugegraph.backend.id.IdGenerator; import com.baidu.hugegraph.backend.query.Condition; +import com.baidu.hugegraph.backend.query.IdPrefixQuery; +import com.baidu.hugegraph.backend.query.IdRangeQuery; import com.baidu.hugegraph.backend.query.Query; import com.baidu.hugegraph.backend.serializer.TextBackendEntry; import com.baidu.hugegraph.backend.store.BackendEntry; @@ -41,7 +40,6 @@ import com.baidu.hugegraph.backend.store.BackendTable; import com.baidu.hugegraph.exception.NotSupportException; import com.baidu.hugegraph.type.HugeType; -import com.baidu.hugegraph.type.define.HugeKeys; import com.baidu.hugegraph.util.E; import com.baidu.hugegraph.util.InsertionOrderUtil; @@ -52,7 +50,7 @@ public class InMemoryDBTable extends BackendTable(); + this.store = new ConcurrentHashMap<>(); } public InMemoryDBTable(HugeType type, Map store) { @@ -120,27 +118,26 @@ public Iterator query(BackendSession session, Query query) { Map rs = this.store; + if (query instanceof IdPrefixQuery) { + IdPrefixQuery pq = (IdPrefixQuery) query; + rs = this.queryByIdPrefix(pq.start(), pq.inclusiveStart(), + pq.prefix(), rs); + } + + if (query instanceof IdRangeQuery) { + IdRangeQuery rq = (IdRangeQuery) query; + rs = this.queryByIdRange(rq.start(), rq.inclusiveStart(), + rq.end(), rq.inclusiveEnd(), rs); + } + // Query by id(s) if (!query.ids().isEmpty()) { - if (query.resultType().isEdge()) { - E.checkState(query.conditions().isEmpty(), - "Not support querying edge by %s", query); - // Query edge(in a vertex) by id (or v-id + column-name prefix) - // TODO: separate this method into table Edge - rs = this.queryEdgeById(query.ids(), rs); - } else { - rs = this.queryById(query.ids(), rs); - } + rs = this.queryById(query.ids(), rs); } // Query by condition(s) if (!query.conditions().isEmpty()) { - if (query.resultType().isEdge()) { - // TODO: separate this method into table Edge - rs = this.queryEdgeByFilter(query.conditions(), rs); - } else { - rs = this.queryByFilter(query.conditions(), rs); - } + rs = this.queryByFilter(query.conditions(), rs); } Iterator iterator = rs.values().iterator(); @@ -171,50 +168,19 @@ protected Map queryById(Set ids, return rs; } - protected Map queryEdgeById( - Set ids, - Map entries) { - assert ids.size() > 0; - Map rs = new HashMap<>(); - - for (Id id : ids) { - // TODO: improve id split - String[] parts = EdgeId.split(id); - Id entryId = IdGenerator.of(parts[0]); - - String column = null; - if (parts.length > 1) { - parts = Arrays.copyOfRange(parts, 1, parts.length); - column = EdgeId.concat(parts); - } else { - // All edges - assert parts.length == 1; - } - - if (entries.containsKey(entryId)) { - BackendEntry value = entries.get(entryId); - // TODO: Compatible with BackendEntry - TextBackendEntry entry = (TextBackendEntry) value; - if (column == null) { - // All edges in the vertex - rs.put(entryId, entry); - } else if (entry.containsPrefix(column)) { - // An edge in the vertex - BackendEntry edges = new TextBackendEntry(HugeType.VERTEX, - entryId); - edges.columns(entry.columnsWithPrefix(column)); - - BackendEntry result = rs.get(entryId); - if (result == null) { - rs.put(entryId, edges); - } else { - result.merge(edges); - } - } - } - } + protected Map queryByIdPrefix(Id start, + boolean inclusiveStart, + Id prefix, + Map rs) { + throw new BackendException("Unsupported prefix query: " + prefix); + } - return rs; + protected Map queryByIdRange(Id start, + boolean inclusiveStart, + Id end, + boolean inclusiveEnd, + Map rs) { + throw new BackendException("Unsupported range query: " + start); } protected Map queryByFilter( @@ -241,52 +207,6 @@ protected Map queryByFilter( return rs; } - protected Map queryEdgeByFilter( - Set conditions, - Map entries) { - if (conditions.isEmpty()) { - return entries; - } - - // Only support querying edge by label - E.checkState(conditions.size() == 1, - "Not support querying edge by %s", conditions); - Condition cond = conditions.iterator().next(); - E.checkState(cond.isRelation() && - ((Condition.Relation) cond).key().equals(HugeKeys.LABEL), - "Not support querying edge by %s", conditions); - Condition.Relation relation = (Condition.Relation) cond; - String label = (String) relation.serialValue(); - - Map rs = new HashMap<>(); - - for (BackendEntry value : entries.values()) { - // TODO: Compatible with BackendEntry - TextBackendEntry entry = (TextBackendEntry) value; - String out = EdgeId.concat(HugeType.EDGE_OUT.string(), label); - String in = EdgeId.concat(HugeType.EDGE_IN.string(), label); - if (entry.containsPrefix(out)) { - BackendEntry edges = new TextBackendEntry(HugeType.VERTEX, - entry.id()); - edges.columns(entry.columnsWithPrefix(out)); - rs.put(edges.id(), edges); - } - if (entry.containsPrefix(in)) { - BackendEntry edges = new TextBackendEntry(HugeType.VERTEX, - entry.id()); - edges.columns(entry.columnsWithPrefix(in)); - BackendEntry result = rs.get(edges.id()); - if (result == null) { - rs.put(edges.id(), edges); - } else { - result.merge(edges); - } - } - } - - return rs; - } - protected Iterator skipOffset(Iterator iterator, long offset) { // Skip offset (TODO: maybe we can improve when adding items to rs) diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/memory/InMemoryDBTables.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/memory/InMemoryDBTables.java index 5ebb9ca02e..c4e461806e 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/memory/InMemoryDBTables.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/memory/InMemoryDBTables.java @@ -20,6 +20,7 @@ package com.baidu.hugegraph.backend.store.memory; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.Iterator; @@ -39,12 +40,14 @@ import com.baidu.hugegraph.backend.query.Query; import com.baidu.hugegraph.backend.serializer.TextBackendEntry; import com.baidu.hugegraph.backend.store.BackendEntry; +import com.baidu.hugegraph.backend.store.BackendEntry.BackendColumn; import com.baidu.hugegraph.backend.store.BackendSession; import com.baidu.hugegraph.iterator.ExtendableIterator; import com.baidu.hugegraph.structure.HugeIndex; import com.baidu.hugegraph.type.HugeType; import com.baidu.hugegraph.type.define.HugeKeys; import com.baidu.hugegraph.util.E; +import com.baidu.hugegraph.util.InsertionOrderUtil; import com.google.common.collect.ImmutableList; public class InMemoryDBTables { @@ -97,6 +100,180 @@ public void eliminate(BackendSession session, TextBackendEntry entry) { throw new UnsupportedOperationException("Edge eliminate"); } + @Override + protected Map queryById( + Set ids, + Map entries) { + // Query edge(in a vertex) by id + return this.queryEdgeById(ids, false, entries); + } + + @Override + protected Map queryByIdPrefix( + Id start, + boolean inclusiveStart, + Id prefix, + Map entries) { + // Query edge(in a vertex) by v-id + column-name-prefix + BackendEntry value = this.getEntryById(start, entries); + if (value == null) { + return Collections.emptyMap(); + } + + Map rs = InsertionOrderUtil.newMap(); + + // TODO: Compatible with BackendEntry + TextBackendEntry entry = (TextBackendEntry) value; + // Prefix edges in the vertex + String startColumn = columnOfEdge(start); + String prefixColumn = columnOfEdge(prefix); + BackendEntry edges = new TextBackendEntry(HugeType.VERTEX, + entry.id()); + edges.columns(entry.columnsWithPrefix(startColumn, inclusiveStart, + prefixColumn)); + + BackendEntry result = rs.get(entry.id()); + if (result == null) { + rs.put(entry.id(), edges); + } else { + result.merge(edges); + } + + return rs; + } + + @Override + protected Map queryByIdRange( + Id start, + boolean inclusiveStart, + Id end, + boolean inclusiveEnd, + Map entries) { + BackendEntry value = this.getEntryById(start, entries); + if (value == null) { + return Collections.emptyMap(); + } + + Map rs = InsertionOrderUtil.newMap(); + + // TODO: Compatible with BackendEntry + TextBackendEntry entry = (TextBackendEntry) value; + // Range edges in the vertex + String startColumn = columnOfEdge(start); + String endColumn = columnOfEdge(end); + BackendEntry edges = new TextBackendEntry(HugeType.VERTEX, + entry.id()); + edges.columns(entry.columnsWithRange(startColumn, inclusiveStart, + endColumn, inclusiveEnd)); + + BackendEntry result = rs.get(entry.id()); + if (result == null) { + rs.put(entry.id(), edges); + } else { + result.merge(edges); + } + + return rs; + } + + private Map queryEdgeById( + Set ids, boolean prefix, + Map entries) { + assert ids.size() > 0; + Map rs = InsertionOrderUtil.newMap(); + + for (Id id : ids) { + BackendEntry value = this.getEntryById(id, entries); + if (value != null) { + // TODO: Compatible with BackendEntry + TextBackendEntry entry = (TextBackendEntry) value; + String column = columnOfEdge(id); + if (column == null) { + // All edges in the vertex + rs.put(entry.id(), entry); + } else if ((!prefix && entry.contains(column)) || + (prefix && entry.containsPrefix(column))) { + BackendEntry edges = new TextBackendEntry( + HugeType.VERTEX, entry.id()); + if (prefix) { + // Some edges with specified prefix in the vertex + edges.columns(entry.columnsWithPrefix(column)); + } else { + // An edge with specified id in the vertex + BackendColumn col = entry.columns(column); + if (col != null) { + edges.columns(col); + } + } + + BackendEntry result = rs.get(entry.id()); + if (result == null) { + rs.put(entry.id(), edges); + } else { + result.merge(edges); + } + } + } + } + + return rs; + } + + private BackendEntry getEntryById(Id id, + Map entries) { + // TODO: improve id split + Id entryId = IdGenerator.of(EdgeId.split(id)[0]); + return entries.get(entryId); + } + + @Override + protected Map queryByFilter( + Set conditions, + Map entries) { + if (conditions.isEmpty()) { + return entries; + } + + // Only support querying edge by label + E.checkState(conditions.size() == 1, + "Not support querying edge by %s", conditions); + Condition cond = conditions.iterator().next(); + E.checkState(cond.isRelation(), + "Not support querying edge by %s", conditions); + Condition.Relation relation = (Condition.Relation) cond; + E.checkState(relation.key().equals(HugeKeys.LABEL), + "Not support querying edge by %s", conditions); + String label = (String) relation.serialValue(); + + Map rs = InsertionOrderUtil.newMap(); + + for (BackendEntry value : entries.values()) { + // TODO: Compatible with BackendEntry + TextBackendEntry entry = (TextBackendEntry) value; + String out = EdgeId.concat(HugeType.EDGE_OUT.string(), label); + String in = EdgeId.concat(HugeType.EDGE_IN.string(), label); + if (entry.containsPrefix(out)) { + BackendEntry edges = new TextBackendEntry(HugeType.VERTEX, + entry.id()); + edges.columns(entry.columnsWithPrefix(out)); + rs.put(edges.id(), edges); + } + if (entry.containsPrefix(in)) { + BackendEntry edges = new TextBackendEntry(HugeType.VERTEX, + entry.id()); + edges.columns(entry.columnsWithPrefix(in)); + BackendEntry result = rs.get(edges.id()); + if (result == null) { + rs.put(edges.id(), edges); + } else { + result.merge(edges); + } + } + } + + return rs; + } + @Override protected Iterator skipOffset(Iterator itor, long offset) { @@ -155,6 +332,19 @@ private static Id vertexIdOfEdge(TextBackendEntry entry) { String vertexId = EdgeId.split(entry.id())[0]; return IdGenerator.of(vertexId); } + + private static String columnOfEdge(Id id) { + // TODO: improve id split + String[] parts = EdgeId.split(id); + if (parts.length > 1) { + parts = Arrays.copyOfRange(parts, 1, parts.length); + return EdgeId.concat(parts); + } else { + // All edges + assert parts.length == 1; + } + return null; + } } public static class SecondaryIndex extends InMemoryDBTable { diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/tx/GraphIndexTransaction.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/tx/GraphIndexTransaction.java index ede4b829ec..4b91527691 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/tx/GraphIndexTransaction.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/tx/GraphIndexTransaction.java @@ -430,8 +430,8 @@ private Set queryByUserpropWithSearchIndex(ConditionQuery query, continue; } Id indexField = il.indexField(); - Object fieldValue = query.userpropValue(indexField); - Set words = this.segmentWords(fieldValue.toString()); + String fieldValue = (String) query.userpropValue(indexField); + Set words = this.segmentWords(fieldValue); indexFields.add(indexField); query = query.copy(); @@ -447,7 +447,7 @@ private Set queryByUserpropWithSearchIndex(ConditionQuery query, // This is an index field of search index Id field = (Id) key; String propValue = elem.getPropertyValue(field); - String fvalue = originQuery.userpropValue(field).toString(); + String fvalue = (String) originQuery.userpropValue(field); if (this.matchSearchIndexWords(propValue, fvalue)) { continue; } @@ -882,6 +882,7 @@ private static ConditionQuery matchIndexLabel(ConditionQuery query, "Invalid index fields size for %s: %s", indexType, indexFields); Object fieldValue = query.userpropValue(indexFields.get(0)); + assert fieldValue instanceof String; // Query search index from SECONDARY_INDEX table indexQuery = new ConditionQuery(indexType.type(), query); indexQuery.eq(HugeKeys.INDEX_LABEL_ID, indexLabel.id()); @@ -911,10 +912,10 @@ private static ConditionQuery matchIndexLabel(ConditionQuery query, for (Condition condition : query.userpropConditions()) { assert condition instanceof Condition.Relation; Condition.Relation r = (Condition.Relation) condition; + Number value = NumericUtil.convertToNumber(r.value()); Condition.Relation sys = new Condition.SyspropRelation( - HugeKeys.FIELD_VALUES, - r.relation(), - NumericUtil.convertToNumber(r.value())); + HugeKeys.FIELD_VALUES, + r.relation(), value); condition = condition.replace(r, sys); indexQuery.query(condition); } @@ -1093,7 +1094,7 @@ public static IndexQueries of(IndexLabel il, ConditionQuery query) { public static enum OptimizedType { NONE, PRIMARY_KEY, - SORT_KEY, + SORT_KEYS, INDEX } } diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/tx/GraphTransaction.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/tx/GraphTransaction.java index e81d7946b5..867f784143 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/tx/GraphTransaction.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/tx/GraphTransaction.java @@ -952,9 +952,8 @@ protected Query optimizeQuery(ConditionQuery query) { String primaryValues = query.userpropValuesString(keys); LOG.debug("Query vertices by primaryKeys: {}", query); // Convert {vertex-label + primary-key} to vertex-id - Id id = SplicingIdGenerator.splicing( - vertexLabel.id().asString(), - primaryValues); + Id id = SplicingIdGenerator.splicing(label.asString(), + primaryValues); /* * Just query by primary-key(id), ignore other userprop(if * exists) that it will be filtered by queryVertices(Query) @@ -971,11 +970,34 @@ protected Query optimizeQuery(ConditionQuery query) { query.condition(HugeKeys.DIRECTION) != null && !keys.isEmpty() && query.matchUserpropKeys(keys)) { // Query edge by sourceVertex + direction + label + sort-values - query.optimized(OptimizedType.SORT_KEY.ordinal()); + query.optimized(OptimizedType.SORT_KEYS.ordinal()); query = query.copy(); - query.eq(HugeKeys.SORT_VALUES, - query.userpropValuesString(keys)); + // Serialize sort-values + for (Condition.Relation r : query.userpropRelations()) { + if (!keys.contains(r.key())) { + continue; + } + PropertyKey pk = this.graph().propertyKey((Id) r.key()); + r.serialValue(pk.serialValue(r.value())); + } + // Convert to sysprop condition {SORT_VALUES=value} + if (query.hasRangeCondition()) { + for (Condition condition : query.userpropConditions()) { + assert condition instanceof Condition.Relation; + Condition.Relation r = (Condition.Relation) condition; + Condition.Relation sys = new Condition.SyspropRelation( + HugeKeys.SORT_VALUES, + r.relation(), + r.serialValue()); + condition = condition.replace(r, sys); + query.query(condition); + } + } else { + query.eq(HugeKeys.SORT_VALUES, + query.userpropValuesString(keys)); + } query.resetUserpropConditions(); + LOG.debug("Query edges by sortKeys: {}", query); return query; } diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/schema/PropertyKey.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/schema/PropertyKey.java index 29e3720866..832c36dac9 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/schema/PropertyKey.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/schema/PropertyKey.java @@ -38,6 +38,7 @@ import com.baidu.hugegraph.type.define.Cardinality; import com.baidu.hugegraph.type.define.DataType; import com.baidu.hugegraph.util.E; +import com.baidu.hugegraph.util.LongEncoding; public class PropertyKey extends SchemaElement implements Propfiable { @@ -55,6 +56,34 @@ public HugeType type() { return HugeType.PROPERTY_KEY; } + public DataType dataType() { + return this.dataType; + } + + public void dataType(DataType dataType) { + this.dataType = dataType; + } + + public Cardinality cardinality() { + return this.cardinality; + } + + public void cardinality(Cardinality cardinality) { + this.cardinality = cardinality; + } + + @Override + public Set properties() { + return Collections.emptySet(); + } + + public PropertyKey properties(Id... properties) { + if (properties.length > 0) { + throw new NotSupportException("PropertyKey.properties(Id)"); + } + return this; + } + public Class clazz() { Class cls; switch (this.cardinality) { @@ -136,6 +165,17 @@ public boolean checkValue(V value) { return valid; } + public Object serialValue(V value) { + V validValue = this.validValue(value); + E.checkArgument(validValue != null, + "Invalid property value '%s' for key '%s'", + value, this.name()); + if (this.dataType.isNumber() || this.dataType.isDate()) { + return LongEncoding.encodeNumber(validValue); + } + return validValue; + } + public V validValue(V value) { return this.convValue(value, true); } @@ -192,34 +232,6 @@ private V convSingleValue(V value) { return value; } - public DataType dataType() { - return this.dataType; - } - - public void dataType(DataType dataType) { - this.dataType = dataType; - } - - public Cardinality cardinality() { - return this.cardinality; - } - - public void cardinality(Cardinality cardinality) { - this.cardinality = cardinality; - } - - @Override - public Set properties() { - return Collections.emptySet(); - } - - public PropertyKey properties(Id... properties) { - if (properties.length > 0) { - throw new NotSupportException("PropertyKey.properties(Id)"); - } - return this; - } - public interface Builder extends SchemaBuilder { Builder asText(); diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/structure/HugeProperty.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/structure/HugeProperty.java index 834371bc28..cdaa484b99 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/structure/HugeProperty.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/structure/HugeProperty.java @@ -19,7 +19,6 @@ package com.baidu.hugegraph.structure; -import java.util.Date; import java.util.NoSuchElementException; import org.apache.tinkerpop.gremlin.structure.Property; @@ -80,10 +79,7 @@ public V value() throws NoSuchElementException { } public Object serialValue() { - if (this.value instanceof Date) { - return ((Date) this.value).getTime(); - } - return this.value; + return this.pkey.serialValue(this.value); } @Override diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/util/StringEncoding.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/util/StringEncoding.java index aa82b936d7..d2184e63d2 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/util/StringEncoding.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/util/StringEncoding.java @@ -81,4 +81,10 @@ public static String decode(byte[] bytes) { throw new HugeException("Failed to decode string", e); } } + + public static String format(byte[] bytes) { + return String.format("%s[0x%s]", + StringEncoding.decode(bytes), + Bytes.toHex(bytes)); + } } diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/version/CoreVersion.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/version/CoreVersion.java index 2729c476a9..1135e52449 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/version/CoreVersion.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/version/CoreVersion.java @@ -32,7 +32,8 @@ public class CoreVersion { public static final String NAME = "hugegraph-core"; // The second parameter of Version.of() is for IDE running without JAR - public static final Version VERSION = Version.of(CoreVersion.class, "0.9.0"); + public static final Version VERSION = Version.of(CoreVersion.class, + "0.9.1"); public static final String GREMLIN_VERSION = "3.2.5"; diff --git a/hugegraph-dist/pom.xml b/hugegraph-dist/pom.xml index 532aa8894a..990022930e 100644 --- a/hugegraph-dist/pom.xml +++ b/hugegraph-dist/pom.xml @@ -5,7 +5,7 @@ hugegraph com.baidu.hugegraph - 0.9.0 + 0.9.1 4.0.0 hugegraph-dist diff --git a/hugegraph-example/pom.xml b/hugegraph-example/pom.xml index 889e6a157b..dece4997a1 100644 --- a/hugegraph-example/pom.xml +++ b/hugegraph-example/pom.xml @@ -5,7 +5,7 @@ hugegraph com.baidu.hugegraph - 0.9.0 + 0.9.1 4.0.0 diff --git a/hugegraph-example/src/main/java/com/baidu/hugegraph/example/Example1.java b/hugegraph-example/src/main/java/com/baidu/hugegraph/example/Example1.java index 10cd84c91c..74391e861a 100644 --- a/hugegraph-example/src/main/java/com/baidu/hugegraph/example/Example1.java +++ b/hugegraph-example/src/main/java/com/baidu/hugegraph/example/Example1.java @@ -175,9 +175,9 @@ public static void loadSchema(final HugeGraph graph) { .sortKeys("time") .create(); - schema.edgeLabel("look").multiTimes().properties("time") + schema.edgeLabel("look").multiTimes().properties("timestamp") .sourceLabel("person").targetLabel("book") - .sortKeys("time") + .sortKeys("timestamp") .create(); schema.edgeLabel("created").singleTime() @@ -407,7 +407,8 @@ public static void testRemove(final HugeGraph graph) { assert vertices.size() == 1; Vertex james = vertices.get(0); Vertex book6 = graph.addVertex(T.label, "book", "name", "java-6"); - james.addEdge("look", book6, "time", "2017-5-3"); + james.addEdge("look", book6, "timestamp", "2017-5-2 12:00:08.0"); + james.addEdge("look", book6, "timestamp", "2017-5-3 12:00:08.0"); graph.tx().commit(); assert graph.traversal().V(book6.id()).bothE().hasNext(); System.out.println(">>>> removing vertex: " + james); diff --git a/hugegraph-hbase/pom.xml b/hugegraph-hbase/pom.xml index e9bb9695ee..f4dfc2a1bc 100644 --- a/hugegraph-hbase/pom.xml +++ b/hugegraph-hbase/pom.xml @@ -5,7 +5,7 @@ hugegraph com.baidu.hugegraph - 0.9.0 + 0.9.1 4.0.0 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 5ee620936e..7e0d902922 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,38 +19,11 @@ package com.baidu.hugegraph.backend.store.hbase; -import com.baidu.hugegraph.backend.id.Id; -import com.baidu.hugegraph.backend.serializer.BinaryBackendEntry; import com.baidu.hugegraph.backend.serializer.BinarySerializer; -import com.baidu.hugegraph.backend.serializer.BytesBuffer; -import com.baidu.hugegraph.backend.store.BackendEntry.BackendColumn; -import com.baidu.hugegraph.structure.HugeIndex; public class HbaseSerializer extends BinarySerializer { public HbaseSerializer() { super(false); } - - @Override - protected byte[] formatIndexName(HugeIndex index) { - Id elemId = index.elementId(); - BytesBuffer buffer = BytesBuffer.allocate(1 + elemId.length()); - // Write element-id - buffer.writeId(elemId, true); - return buffer.bytes(); - } - - @Override - protected void parseIndexName(BinaryBackendEntry entry, HugeIndex index, - Object fieldValues) { - for (BackendColumn col : entry.columns()) { - if (indexFieldValuesUnmatched(col.value, fieldValues)) { - // Skip if field-values don't matched (just the same hash) - continue; - } - BytesBuffer buffer = BytesBuffer.wrap(col.name); - index.elementIds(buffer.readId(true)); - } - } } 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 a3407cb914..6e09cd35dd 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 @@ -33,6 +33,9 @@ import java.util.concurrent.Future; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellScanner; +import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.RegionMetrics; @@ -58,6 +61,7 @@ import org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange; import org.apache.hadoop.hbase.filter.PageFilter; import org.apache.hadoop.hbase.filter.PrefixFilter; +import org.apache.hadoop.hbase.util.VersionInfo; import com.baidu.hugegraph.backend.BackendException; import com.baidu.hugegraph.backend.store.BackendEntry.BackendColumn; @@ -65,7 +69,10 @@ import com.baidu.hugegraph.backend.store.BackendSession; import com.baidu.hugegraph.backend.store.BackendSessionPool; import com.baidu.hugegraph.config.HugeConfig; +import com.baidu.hugegraph.util.Bytes; import com.baidu.hugegraph.util.E; +import com.baidu.hugegraph.util.StringEncoding; +import com.baidu.hugegraph.util.VersionUtil; public class HbaseSessions extends BackendSessionPool { @@ -480,11 +487,27 @@ public RowIterator scan(String table, assert !this.hasChanges(); Scan scan = new Scan().withStartRow(startRow, inclusiveStart); if (stopRow != null) { + String version = VersionInfo.getVersion(); + if (inclusiveStop && !VersionUtil.gte(version, "2.0")) { + // The parameter stoprow-inclusive doesn't work before v2.0 + // https://issues.apache.org/jira/browse/HBASE-20675 + inclusiveStop = false; + // Add a trailing 0 byte to stopRow + stopRow = Arrays.copyOf(stopRow, stopRow.length + 1); + } + if (Bytes.equals(startRow, stopRow) && + inclusiveStart && !inclusiveStop) { + // Bug https://issues.apache.org/jira/browse/HBASE-21618 + return new RowIterator(); + } scan.withStopRow(stopRow, inclusiveStop); } return this.scan(table, scan); } + /** + * Inner scan: send scan request to HBase and get iterator + */ private RowIterator scan(String table, Scan scan) { try (Table htable = table(table)) { return new RowIterator(htable.getScanner(scan)); @@ -512,6 +535,29 @@ public long increase(String table, byte[] family, byte[] rowkey, public long storeSize(String table) throws IOException { return HbaseSessions.this.storeSize(table); } + + /** + * Just for debug + */ + @SuppressWarnings("unused") + private void dump(String table, Scan scan) throws IOException { + System.out.println(String.format(">>>> scan table %s with %s", + table, scan)); + RowIterator iterator = this.scan(table, scan); + while (iterator.hasNext()) { + Result row = iterator.next(); + System.out.println(StringEncoding.format(row.getRow())); + CellScanner cellScanner = row.cellScanner(); + while (cellScanner.advance()) { + Cell cell = cellScanner.current(); + byte[] key = CellUtil.cloneQualifier(cell); + byte[] val = CellUtil.cloneValue(cell); + System.out.println(String.format(" %s=%s", + StringEncoding.format(key), + StringEncoding.format(val))); + } + } + } } protected static class RowIterator implements BackendIterator { 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 282172a347..09763ee6f9 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 @@ -51,7 +51,8 @@ public String version() { * Versions history: * [1.0] HugeGraph-1328: supports backend table version checking * [1.1] HugeGraph-1322: add support for full-text search + * [1.2] #296: support range sortKey feature */ - return "1.1"; + return "1.2"; } } 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 6a62219634..43b1e5e1f6 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 @@ -35,6 +35,8 @@ import com.baidu.hugegraph.backend.id.Id; import com.baidu.hugegraph.backend.query.Condition.Relation; import com.baidu.hugegraph.backend.query.ConditionQuery; +import com.baidu.hugegraph.backend.query.IdPrefixQuery; +import com.baidu.hugegraph.backend.query.IdRangeQuery; import com.baidu.hugegraph.backend.query.Query; import com.baidu.hugegraph.backend.serializer.BinaryBackendEntry; import com.baidu.hugegraph.backend.serializer.BinaryEntryIterator; @@ -131,6 +133,18 @@ public Iterator query(Session session, Query query) { return newEntryIterator(this.queryAll(session, query), query); } + // Query by prefix + if (query instanceof IdPrefixQuery) { + IdPrefixQuery pq = (IdPrefixQuery) query; + return newEntryIterator(this.queryByPrefix(session, pq), query); + } + + // Query by range + if (query instanceof IdRangeQuery) { + IdRangeQuery rq = (IdRangeQuery) query; + return newEntryIterator(this.queryByRange(session, rq), query); + } + // Query by id if (query.conditions().isEmpty()) { assert !query.ids().isEmpty(); @@ -169,6 +183,18 @@ protected RowIterator queryByIds(Session session, Set ids) { return session.get(this.table(), null, rowkeys); } + protected RowIterator queryByPrefix(Session session, IdPrefixQuery query) { + return session.scan(this.table(), query.start().asBytes(), + query.inclusiveStart(), query.prefix().asBytes()); + } + + protected RowIterator queryByRange(Session session, IdRangeQuery query) { + byte[] start = query.start().asBytes(); + byte[] end = query.end() == null ? null : query.end().asBytes(); + return session.scan(this.table(), start, query.inclusiveStart(), + end, query.inclusiveEnd()); + } + protected RowIterator queryByCond(Session session, ConditionQuery query) { if (query.containsScanCondition()) { E.checkArgument(query.relations().size() == 1, @@ -180,10 +206,6 @@ protected RowIterator queryByCond(Session session, ConditionQuery query) { throw new NotSupportException("query: %s", query); } - protected RowIterator queryByRange(Session session, Id begin, Id end) { - return session.scan(this.table(), begin.asBytes(), end.asBytes()); - } - protected RowIterator queryByRange(Session session, Shard shard) { byte[] start = this.shardSpliter.position(shard.start()); byte[] end = this.shardSpliter.position(shard.end()); 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 7178aec355..c07550dff1 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 @@ -22,8 +22,6 @@ import java.io.IOException; import java.util.Arrays; import java.util.List; -import java.util.Set; -import java.util.stream.Collectors; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellScanner; @@ -141,27 +139,28 @@ public static Edge in(String store) { } @Override - protected RowIterator queryById(Session session, Id id) { - byte[] prefix = id.asBytes(); - return session.scan(this.table(), prefix); - } - - @Override - protected RowIterator queryByIds(Session session, Set ids) { - Set prefixs = ids.stream().map(Id::asBytes) - .collect(Collectors.toSet()); - return session.scan(this.table(), prefixs); + public void insert(Session session, BackendEntry entry) { + for (BackendColumn col : entry.columns()) { + E.checkArgument(col.name.length == 0, + "Expect empty column name, " + + "please ensure hbase serializer is used"); + } + super.insert(session, entry); } @Override protected void parseRowColumns(Result row, BackendEntry entry, Query query) throws IOException { + /* + * Collapse owner-vertex id from edge id, NOTE: unneeded to + * collapse if BinarySerializer.keyWithIdPrefix set to true + */ byte[] key = row.getRow(); - // Collapse owner vertex id key = Arrays.copyOfRange(key, entry.id().length(), key.length); + long total = query.total(); CellScanner cellScanner = row.cellScanner(); - while (cellScanner.advance()) { + while (cellScanner.advance() && total-- > 0) { Cell cell = cellScanner.current(); assert CellUtil.cloneQualifier(cell).length == 0; entry.columns(BackendColumn.of(key, CellUtil.cloneValue(cell))); @@ -207,15 +206,6 @@ public void delete(Session session, BackendEntry entry) { session.commit(); } } - } - - public static class SecondaryIndex extends IndexTable { - - public static final String TABLE = "si"; - - public SecondaryIndex(String store) { - super(joinTableName(store, TABLE)); - } @Override protected void parseRowColumns(Result row, BackendEntry entry, @@ -224,12 +214,9 @@ protected void parseRowColumns(Result row, BackendEntry entry, super.parseRowColumns(row, entry, query); return; } + long total = query.total(); CellScanner cellScanner = row.cellScanner(); - long total = query.offset() + query.limit(); - for (long i = 0; i < total; i++) { - if (!cellScanner.advance()) { - break; - } + while (cellScanner.advance() && total-- > 0) { Cell cell = cellScanner.current(); entry.columns(BackendColumn.of(CellUtil.cloneQualifier(cell), CellUtil.cloneValue(cell))); @@ -237,6 +224,15 @@ protected void parseRowColumns(Result row, BackendEntry entry, } } + public static class SecondaryIndex extends IndexTable { + + public static final String TABLE = "si"; + + public SecondaryIndex(String store) { + super(joinTableName(store, TABLE)); + } + } + public static class SearchIndex extends IndexTable { public static final String TABLE = "fi"; diff --git a/hugegraph-mysql/pom.xml b/hugegraph-mysql/pom.xml index 7c782c543b..86e66f4795 100644 --- a/hugegraph-mysql/pom.xml +++ b/hugegraph-mysql/pom.xml @@ -5,7 +5,7 @@ hugegraph com.baidu.hugegraph - 0.9.0 + 0.9.1 4.0.0 diff --git a/hugegraph-mysql/src/main/java/com/baidu/hugegraph/backend/store/mysql/MysqlSessions.java b/hugegraph-mysql/src/main/java/com/baidu/hugegraph/backend/store/mysql/MysqlSessions.java index 19007936ff..5b1c1256b8 100644 --- a/hugegraph-mysql/src/main/java/com/baidu/hugegraph/backend/store/mysql/MysqlSessions.java +++ b/hugegraph-mysql/src/main/java/com/baidu/hugegraph/backend/store/mysql/MysqlSessions.java @@ -154,8 +154,8 @@ public void createDatabase() { protected String buildCreateDatabase(String database) { return String.format("CREATE DATABASE IF NOT EXISTS %s " + - "DEFAULT CHARSET utf8 COLLATE " + - "utf8_general_ci;", database); + "DEFAULT CHARSET utf8 COLLATE utf8_bin;", + database); } public void dropDatabase() { 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 ad1c4f9026..f20323b958 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 @@ -69,7 +69,8 @@ public String version() { * Versions history: * [1.0] HugeGraph-1328: supports backend table version checking * [1.1] HugeGraph-1322: add support for full-text search + * [1.2] #296: support range sortKey feature */ - return "1.1"; + return "1.2"; } } diff --git a/hugegraph-mysql/src/main/java/com/baidu/hugegraph/backend/store/mysql/MysqlTable.java b/hugegraph-mysql/src/main/java/com/baidu/hugegraph/backend/store/mysql/MysqlTable.java index 4de25bc6ce..1edfb47fad 100644 --- a/hugegraph-mysql/src/main/java/com/baidu/hugegraph/backend/store/mysql/MysqlTable.java +++ b/hugegraph-mysql/src/main/java/com/baidu/hugegraph/backend/store/mysql/MysqlTable.java @@ -98,7 +98,7 @@ protected void createTable(Session session, TableDefine tableDefine) { } } - sql.append(")) ENGINE=InnoDB DEFAULT CHARSET=utf8;"); + sql.append(")) ENGINE=InnoDB;"); LOG.debug("Create table: {}", sql); try { diff --git a/hugegraph-palo/pom.xml b/hugegraph-palo/pom.xml index a6324e2496..7b6adde386 100644 --- a/hugegraph-palo/pom.xml +++ b/hugegraph-palo/pom.xml @@ -5,7 +5,7 @@ hugegraph com.baidu.hugegraph - 0.9.0 + 0.9.1 4.0.0 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 4db19d3614..6873d9d24a 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 @@ -53,8 +53,9 @@ public String version() { * Versions history: * [1.0] HugeGraph-1328: supports backend table version checking * [1.1] HugeGraph-1322: add support for full-text search + * [1.2] #296: support range sortKey feature */ - return "1.1"; + return "1.2"; } public static class PaloSchemaStore extends PaloStore { diff --git a/hugegraph-rocksdb/pom.xml b/hugegraph-rocksdb/pom.xml index a0275209f3..e56b9e7e62 100644 --- a/hugegraph-rocksdb/pom.xml +++ b/hugegraph-rocksdb/pom.xml @@ -5,7 +5,7 @@ hugegraph com.baidu.hugegraph - 0.9.0 + 0.9.1 4.0.0 diff --git a/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBStdSessions.java b/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBStdSessions.java index 127cfd18a3..4be55d8878 100644 --- a/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBStdSessions.java +++ b/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBStdSessions.java @@ -49,6 +49,7 @@ import org.rocksdb.WriteOptions; import com.baidu.hugegraph.backend.BackendException; +import com.baidu.hugegraph.backend.serializer.BinarySerializer; import com.baidu.hugegraph.backend.store.BackendEntry.BackendColumn; import com.baidu.hugegraph.backend.store.BackendEntry.BackendColumnIterator; import com.baidu.hugegraph.config.HugeConfig; @@ -350,28 +351,6 @@ public static final String decode(byte[] bytes) { return StringEncoding.decode(bytes); } - public static final byte[] increaseOne(byte[] bytes) { - final byte BYTE_MAX_VALUE = (byte) 0xff; - assert bytes.length > 0; - byte last = bytes[bytes.length - 1]; - if (last != BYTE_MAX_VALUE) { - bytes[bytes.length - 1] += 0x01; - } else { - // Process overflow (like [1, 255] => [2, 0]) - int i = bytes.length - 1; - for (; i > 0 && bytes[i] == BYTE_MAX_VALUE; --i) { - bytes[i] += 0x01; - } - if (bytes[i] == BYTE_MAX_VALUE) { - assert i == 0; - throw new BackendException("Unable to increase bytes: %s", - Bytes.toHex(bytes)); - } - bytes[i] += 0x01; - } - return bytes; - } - /** * StdSession implement for RocksDB */ @@ -511,7 +490,7 @@ public void remove(String table, byte[] key) { public void delete(String table, byte[] key) { byte[] keyFrom = key; byte[] keyTo = Arrays.copyOf(key, key.length); - keyTo = increaseOne(keyTo); + keyTo = BinarySerializer.increaseOne(keyTo); this.batch.deleteRange(cf(table), keyFrom, keyTo); } @@ -662,15 +641,15 @@ private boolean match(int expected) { */ @SuppressWarnings("unused") private void dump() { - this.itor.seekToFirst(); - System.out.println(">>>> seek from " + this.table + ": " + + this.seek(); + System.out.println(">>>> scan from " + this.table + ": " + (this.keyBegin == null ? "*" : - StringEncoding.decode(this.keyBegin)) + + StringEncoding.format(this.keyBegin)) + (this.itor.isValid() ? "" : " - No data")); for (; this.itor.isValid(); this.itor.next()) { - System.out.println(StringEncoding.decode(this.itor.key()) + - ": " + - StringEncoding.decode(this.itor.value())); + System.out.println(String.format("%s=%s", + StringEncoding.format(this.itor.key()), + StringEncoding.format(this.itor.value()))); } } @@ -715,9 +694,9 @@ private void seek() { // Skip `keyBegin` if set SCAN_GT_BEGIN (key > 'xx') if (this.match(Session.SCAN_GT_BEGIN) && !this.match(Session.SCAN_GTE_BEGIN)) { - while (this.hasNext() && !Bytes.equals(this.itor.key(), - this.keyBegin)) { - this.next(); + while (this.itor.isValid() && + Bytes.equals(this.itor.key(), this.keyBegin)) { + this.itor.next(); } } } @@ -752,7 +731,10 @@ private boolean filter(byte[] key) { return Bytes.compare(key, this.keyEnd) < 0; } } else { - assert this.match(Session.SCAN_ANY) : "Unknow scan type"; + assert this.match(Session.SCAN_ANY) || + this.match(Session.SCAN_GT_BEGIN) || + this.match(Session.SCAN_GTE_BEGIN) : + "Unknow scan type"; return true; } } 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 ee1654ea43..e4f0e6a62d 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 @@ -51,7 +51,8 @@ public String version() { * Versions history: * [1.0] HugeGraph-1328: supports backend table version checking * [1.1] HugeGraph-1322: add support for full-text search + * [1.2] #296: support range sortKey feature */ - return "1.1"; + return "1.2"; } } diff --git a/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBTable.java b/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBTable.java index 8f96f6e4bd..795dfe9d87 100644 --- a/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBTable.java +++ b/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBTable.java @@ -26,6 +26,8 @@ import com.baidu.hugegraph.backend.id.Id; import com.baidu.hugegraph.backend.query.Condition.Relation; import com.baidu.hugegraph.backend.query.ConditionQuery; +import com.baidu.hugegraph.backend.query.IdPrefixQuery; +import com.baidu.hugegraph.backend.query.IdRangeQuery; import com.baidu.hugegraph.backend.query.Query; import com.baidu.hugegraph.backend.serializer.BinaryBackendEntry; import com.baidu.hugegraph.backend.serializer.BinaryEntryIterator; @@ -120,6 +122,18 @@ public Iterator query(Session session, Query query) { return newEntryIterator(this.queryAll(session, query), query); } + // Query by prefix + if (query instanceof IdPrefixQuery) { + IdPrefixQuery pq = (IdPrefixQuery) query; + return newEntryIterator(this.queryByPrefix(session, pq), query); + } + + // Query by range + if (query instanceof IdRangeQuery) { + IdRangeQuery rq = (IdRangeQuery) query; + return newEntryIterator(this.queryByRange(session, rq), query); + } + // Query by id if (query.conditions().isEmpty()) { assert !query.ids().isEmpty(); @@ -146,9 +160,32 @@ protected BackendColumnIterator queryAll(Session session, Query query) { } protected BackendColumnIterator queryById(Session session, Id id) { + // TODO: change to get() after vertex and schema don't use id prefix return session.scan(this.table(), id.asBytes()); } + protected BackendColumnIterator queryByPrefix(Session session, + IdPrefixQuery query) { + int type = query.inclusiveStart() ? + Session.SCAN_GTE_BEGIN : Session.SCAN_GT_BEGIN; + type |= Session.SCAN_PREFIX_WITH_END; + return session.scan(this.table(), query.start().asBytes(), + query.prefix().asBytes(), type); + } + + protected BackendColumnIterator queryByRange(Session session, + IdRangeQuery query) { + byte[] start = query.start().asBytes(); + byte[] end = query.end() == null ? null : query.end().asBytes(); + int type = query.inclusiveStart() ? + Session.SCAN_GTE_BEGIN : Session.SCAN_GT_BEGIN; + if (end != null) { + type |= query.inclusiveEnd() ? + Session.SCAN_LTE_END : Session.SCAN_LT_END; + } + return session.scan(this.table(), start, end, type); + } + protected BackendColumnIterator queryByCond(Session session, ConditionQuery query) { if (query.containsScanCondition()) { @@ -161,11 +198,6 @@ protected BackendColumnIterator queryByCond(Session session, throw new NotSupportException("query: %s", query); } - protected BackendColumnIterator queryByRange(Session session, - Id begin, Id end) { - return session.scan(this.table(), begin.asBytes(), end.asBytes()); - } - protected BackendColumnIterator queryByRange(Session session, Shard shard) { byte[] start = this.shardSpliter.position(shard.start()); byte[] end = this.shardSpliter.position(shard.end()); diff --git a/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBTables.java b/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBTables.java index cad5028537..b1edac6105 100644 --- a/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBTables.java +++ b/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBTables.java @@ -27,8 +27,11 @@ import com.baidu.hugegraph.backend.query.Condition; import com.baidu.hugegraph.backend.query.Condition.Relation; import com.baidu.hugegraph.backend.query.ConditionQuery; +import com.baidu.hugegraph.backend.serializer.BinarySerializer; import com.baidu.hugegraph.backend.store.BackendEntry; +import com.baidu.hugegraph.backend.store.BackendEntry.BackendColumn; import com.baidu.hugegraph.backend.store.BackendEntry.BackendColumnIterator; +import com.baidu.hugegraph.backend.store.BackendEntry.BackendColumnIteratorWrapper; import com.baidu.hugegraph.backend.store.rocksdb.RocksDBSessions.Session; import com.baidu.hugegraph.type.HugeType; import com.baidu.hugegraph.type.define.HugeKeys; @@ -135,6 +138,16 @@ public static Edge out(String database) { public static Edge in(String database) { return new Edge(false, database); } + + @Override + protected BackendColumnIterator queryById(Session session, Id id) { + byte[] value = session.get(this.table(), id.asBytes()); + if (value == null) { + return BackendColumnIterator.empty(); + } + BackendColumn col = BackendColumn.of(id.asBytes(), value); + return new BackendColumnIteratorWrapper(col); + } } public static class IndexTable extends RocksDBTable { @@ -228,7 +241,7 @@ protected BackendColumnIterator queryByCond(Session session, E.checkArgumentNotNull(min, "Range index begin key is missing"); byte[] begin = min.asBytes(); if (!minEq) { - begin = RocksDBStdSessions.increaseOne(begin); + begin = BinarySerializer.increaseOne(begin); } if (max == null) { diff --git a/hugegraph-scylladb/pom.xml b/hugegraph-scylladb/pom.xml index 476a8902aa..229ad0a0c6 100644 --- a/hugegraph-scylladb/pom.xml +++ b/hugegraph-scylladb/pom.xml @@ -5,7 +5,7 @@ hugegraph com.baidu.hugegraph - 0.9.0 + 0.9.1 4.0.0 diff --git a/hugegraph-test/pom.xml b/hugegraph-test/pom.xml index 5f6a45139e..5412cb2c10 100644 --- a/hugegraph-test/pom.xml +++ b/hugegraph-test/pom.xml @@ -5,7 +5,7 @@ hugegraph com.baidu.hugegraph - 0.9.0 + 0.9.1 4.0.0 diff --git a/hugegraph-test/src/main/java/com/baidu/hugegraph/core/EdgeCoreTest.java b/hugegraph-test/src/main/java/com/baidu/hugegraph/core/EdgeCoreTest.java index 30302c34b1..e144713340 100644 --- a/hugegraph-test/src/main/java/com/baidu/hugegraph/core/EdgeCoreTest.java +++ b/hugegraph-test/src/main/java/com/baidu/hugegraph/core/EdgeCoreTest.java @@ -992,6 +992,85 @@ public void testQueryOutEdgesOfVertexBySortkeyAndProps() { Assert.assertEquals(0, edges.size()); } + @Test + public void testQueryOutEdgesOfVertexBySortkeyWithRange() { + HugeGraph graph = graph(); + + SchemaManager schema = graph.schema(); + schema.propertyKey("no").asText().create(); + schema.propertyKey("calltime").asDate().create(); + schema.vertexLabel("phone") + .properties("no") + .primaryKeys("no") + .enableLabelIndex(false) + .create(); + schema.edgeLabel("call").multiTimes().properties("calltime") + .sourceLabel("phone").targetLabel("phone") + .sortKeys("calltime") + .create(); + + Vertex v1 = graph.addVertex(T.label, "phone", "no", "13812345678"); + Vertex v2 = graph.addVertex(T.label, "phone", "no", "13866668888"); + Vertex v10086 = graph.addVertex(T.label, "phone", "no", "10086"); + + v1.addEdge("call", v2, "calltime", "2017-5-1 23:00:00"); + v1.addEdge("call", v2, "calltime", "2017-5-2 12:00:01"); + v1.addEdge("call", v2, "calltime", "2017-5-3 12:08:02"); + v1.addEdge("call", v2, "calltime", "2017-5-3 22:22:03"); + v1.addEdge("call", v2, "calltime", "2017-5-4 20:33:04"); + + v1.addEdge("call", v10086, "calltime", "2017-5-2 15:30:05"); + v1.addEdge("call", v10086, "calltime", "2017-5-3 14:56:06"); + v2.addEdge("call", v10086, "calltime", "2017-5-3 17:28:07"); + + graph.tx().commit(); + Assert.assertEquals(8, graph.traversal().E().toList().size()); + + List edges = graph.traversal().V(v1).outE("call") + .has("calltime", "2017-5-3 12:08:02") + .toList(); + Assert.assertEquals(1, edges.size()); + Assert.assertEquals(Utils.date("2017-5-3 12:08:02"), + edges.get(0).value("calltime")); + + edges = graph.traversal().V(v1).outE("call") + .has("calltime", P.lt("2017-5-2")) + .toList(); + Assert.assertEquals(1, edges.size()); + Assert.assertEquals(Utils.date("2017-5-1 23:00:00"), + edges.get(0).value("calltime")); + + edges = graph.traversal().V(v1).outE("call") + .has("calltime", P.gte("2017-5-3")) + .toList(); + Assert.assertEquals(4, edges.size()); + Assert.assertEquals(Utils.date("2017-5-3 12:08:02"), + edges.get(0).value("calltime")); + Assert.assertEquals(Utils.date("2017-5-3 14:56:06"), + edges.get(1).value("calltime")); + Assert.assertEquals(Utils.date("2017-5-3 22:22:03"), + edges.get(2).value("calltime")); + Assert.assertEquals(Utils.date("2017-5-4 20:33:04"), + edges.get(3).value("calltime")); + + edges = graph.traversal().V(v1).outE("call") + .has("calltime", P.gte("2017-5-3")) + .where(__.otherV().hasId(v2.id())) + .toList(); + Assert.assertEquals(3, edges.size()); + + edges = graph.traversal().V(v1).outE("call") + .has("calltime", P.between("2017-5-2","2017-5-4")) + .toList(); + Assert.assertEquals(5, edges.size()); + + edges = graph.traversal().V(v1).outE("call") + .has("calltime", P.between("2017-5-2","2017-5-4")) + .where(__.not(__.otherV().hasId((v10086.id())))) + .toList(); + Assert.assertEquals(3, edges.size()); + } + @Test public void testQueryOutVerticesOfVertexWithSortkey() { HugeGraph graph = graph(); diff --git a/hugegraph-test/src/main/java/com/baidu/hugegraph/unit/UnitTestSuite.java b/hugegraph-test/src/main/java/com/baidu/hugegraph/unit/UnitTestSuite.java index bf9b0aa7d4..eb7022121a 100644 --- a/hugegraph-test/src/main/java/com/baidu/hugegraph/unit/UnitTestSuite.java +++ b/hugegraph-test/src/main/java/com/baidu/hugegraph/unit/UnitTestSuite.java @@ -28,6 +28,7 @@ import com.baidu.hugegraph.unit.core.BackendMutationTest; import com.baidu.hugegraph.unit.core.ConditionQueryFlattenTest; import com.baidu.hugegraph.unit.core.EdgeIdTest; +import com.baidu.hugegraph.unit.core.VersionTest; import com.baidu.hugegraph.unit.rocksdb.RocksDBCountersTest; import com.baidu.hugegraph.unit.rocksdb.RocksDBSessionsTest; @@ -36,6 +37,7 @@ RamCacheTest.class, CacheManagerTest.class, + VersionTest.class, BackendMutationTest.class, ConditionQueryFlattenTest.class, EdgeIdTest.class, diff --git a/hugegraph-test/src/main/java/com/baidu/hugegraph/unit/core/VersionTest.java b/hugegraph-test/src/main/java/com/baidu/hugegraph/unit/core/VersionTest.java new file mode 100644 index 0000000000..943a9c65f1 --- /dev/null +++ b/hugegraph-test/src/main/java/com/baidu/hugegraph/unit/core/VersionTest.java @@ -0,0 +1,44 @@ +/* + * Copyright 2017 HugeGraph Authors + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with this + * work for additional information regarding copyright ownership. The ASF + * licenses this file to You under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations + * under the License. + */ + +package com.baidu.hugegraph.unit.core; + +import org.junit.Test; + +import com.baidu.hugegraph.testutil.Assert; +import com.baidu.hugegraph.util.VersionUtil; +import com.baidu.hugegraph.version.CoreVersion; + +public class VersionTest { + + @Test + public void testCoreVersionGet() { + String pomVersion = VersionUtil.getPomVersion(); + Assert.assertNotNull(pomVersion); + String version = CoreVersion.VERSION.get(); + Assert.assertNotNull(version); + Assert.assertEquals(pomVersion, version); + } + + @Test + public void testCoreVersionCheck() { + // Expect not throw exception + CoreVersion.check(); + } +} diff --git a/pom.xml b/pom.xml index 0cdf185b38..fa28d63ab6 100644 --- a/pom.xml +++ b/pom.xml @@ -4,7 +4,7 @@ 4.0.0 com.baidu.hugegraph hugegraph - 0.9.0 + 0.9.1 pom 3.3.9