diff --git a/hugegraph-cassandra/src/main/java/com/baidu/hugegraph/backend/store/cassandra/CassandraFeatures.java b/hugegraph-cassandra/src/main/java/com/baidu/hugegraph/backend/store/cassandra/CassandraFeatures.java index 139e052265..235872e5a1 100644 --- a/hugegraph-cassandra/src/main/java/com/baidu/hugegraph/backend/store/cassandra/CassandraFeatures.java +++ b/hugegraph-cassandra/src/main/java/com/baidu/hugegraph/backend/store/cassandra/CassandraFeatures.java @@ -75,6 +75,11 @@ public boolean supportsQueryByPage() { return true; } + @Override + public boolean supportsQuerySortByInputIds() { + return false; + } + @Override public boolean supportsDeleteEdgeByLabel() { return true; diff --git a/hugegraph-cassandra/src/main/java/com/baidu/hugegraph/backend/store/cassandra/CassandraTable.java b/hugegraph-cassandra/src/main/java/com/baidu/hugegraph/backend/store/cassandra/CassandraTable.java index 818282691f..66a0c4cf5c 100644 --- a/hugegraph-cassandra/src/main/java/com/baidu/hugegraph/backend/store/cassandra/CassandraTable.java +++ b/hugegraph-cassandra/src/main/java/com/baidu/hugegraph/backend/store/cassandra/CassandraTable.java @@ -40,6 +40,7 @@ import com.baidu.hugegraph.backend.store.BackendTable; import com.baidu.hugegraph.backend.store.Shard; import com.baidu.hugegraph.exception.NotFoundException; +import com.baidu.hugegraph.exception.NotSupportException; import com.baidu.hugegraph.iterator.ExtendableIterator; import com.baidu.hugegraph.type.HugeType; import com.baidu.hugegraph.type.define.HugeKeys; @@ -303,7 +304,7 @@ protected Clause relation2Cql(Relation relation) { // return QueryBuilder.like(key, value); case NEQ: default: - throw new AssertionError("Unsupported relation: " + relation); + throw new NotSupportException("relation '%s'", relation); } } diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/page/IdHolder.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/page/IdHolder.java index 9a065cd846..60e966cb46 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/page/IdHolder.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/page/IdHolder.java @@ -28,7 +28,7 @@ import com.baidu.hugegraph.util.InsertionOrderUtil; import com.google.common.collect.ImmutableSet; -public final class IdHolder { +public class IdHolder { private final ConditionQuery query; private final Function idsFetcher; @@ -67,6 +67,7 @@ public IdHolder(ConditionQuery query, } public void merge(Set ids) { + E.checkNotNull(this.ids, "ids"); this.ids.addAll(ids); } diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/page/IdHolderList.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/page/IdHolderList.java index 8e6db384e8..aa07bb531c 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/page/IdHolderList.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/page/IdHolderList.java @@ -24,7 +24,7 @@ import com.baidu.hugegraph.util.E; -public final class IdHolderList extends ArrayList { +public class IdHolderList extends ArrayList { private static final long serialVersionUID = -738694176552424990L; diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/page/PageEntryIterator.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/page/PageEntryIterator.java index 6c84537493..222c5b484f 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/page/PageEntryIterator.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/page/PageEntryIterator.java @@ -24,6 +24,7 @@ import com.baidu.hugegraph.backend.query.Query; import com.baidu.hugegraph.backend.store.BackendEntry; +import com.baidu.hugegraph.backend.tx.AbstractTransaction.QueryResults; import com.baidu.hugegraph.exception.NotSupportException; import com.baidu.hugegraph.iterator.Metadatable; import com.baidu.hugegraph.util.E; @@ -32,15 +33,19 @@ public class PageEntryIterator implements Iterator, Metadatable { private final QueryList queries; private final long pageSize; - private QueryList.PageIterator results; - private PageInfo pageInfo; + private final PageInfo pageInfo; + private final QueryResults queryResults; // for upper layer + + private QueryList.PageIterator pageResults; private long remaining; public PageEntryIterator(QueryList queries, long pageSize) { this.queries = queries; this.pageSize = pageSize; - this.results = QueryList.PageIterator.EMPTY; this.pageInfo = this.parsePageState(); + this.queryResults = new QueryResults(this); + + this.pageResults = QueryList.PageIterator.EMPTY; this.remaining = queries.parent().limit(); } @@ -55,7 +60,7 @@ private PageInfo parsePageState() { @Override public boolean hasNext() { - if (this.results.iterator().hasNext()) { + if (this.pageResults.get().hasNext()) { return true; } return this.fetch(); @@ -71,16 +76,17 @@ private boolean fetch() { if (this.remaining != Query.NO_LIMIT && this.remaining < pageSize) { pageSize = this.remaining; } - this.results = this.queries.fetchNext(this.pageInfo, pageSize); - assert this.results != null; + this.pageResults = this.queries.fetchNext(this.pageInfo, pageSize); + assert this.pageResults != null; + this.queryResults.setQuery(this.pageResults.query()); - if (this.results.iterator().hasNext()) { - if (!this.results.hasNextPage()) { + if (this.pageResults.get().hasNext()) { + if (!this.pageResults.hasNextPage()) { this.pageInfo.increase(); } else { - this.pageInfo.page(this.results.page()); + this.pageInfo.page(this.pageResults.page()); } - this.remaining -= this.results.total(); + this.remaining -= this.pageResults.total(); return true; } else { this.pageInfo.increase(); @@ -93,7 +99,7 @@ public BackendEntry next() { if (!this.hasNext()) { throw new NoSuchElementException(); } - return this.results.iterator().next(); + return this.pageResults.get().next(); } @Override @@ -106,4 +112,8 @@ public Object metadata(String meta, Object... args) { } throw new NotSupportException("Invalid meta '%s'", meta); } + + public QueryResults results() { + return this.queryResults; + } } diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/page/QueryList.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/page/QueryList.java index 3e6d56a08c..dd5631ce8e 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/page/QueryList.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/page/QueryList.java @@ -26,18 +26,17 @@ import java.util.Set; import java.util.function.Function; -import org.apache.tinkerpop.gremlin.util.iterator.IteratorUtils; - import com.baidu.hugegraph.HugeGraph; import com.baidu.hugegraph.backend.id.Id; import com.baidu.hugegraph.backend.query.IdQuery; import com.baidu.hugegraph.backend.query.Query; import com.baidu.hugegraph.backend.store.BackendEntry; +import com.baidu.hugegraph.backend.tx.AbstractTransaction.QueryResults; import com.baidu.hugegraph.config.CoreOptions; -import com.baidu.hugegraph.iterator.FlatMapperIterator; import com.baidu.hugegraph.util.Bytes; import com.baidu.hugegraph.util.CollectionUtil; import com.baidu.hugegraph.util.E; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; public final class QueryList { @@ -45,11 +44,11 @@ public final class QueryList { private final HugeGraph graph; private final Query parent; // The size of each page fetched by the inner page - private final Function> fetcher; + private final Function fetcher; private final List queries; public QueryList(HugeGraph graph, Query parent, - Function> fetcher) { + Function fetcher) { this.graph = graph; this.parent = parent; this.fetcher = fetcher; @@ -60,7 +59,7 @@ protected Query parent() { return this.parent; } - protected Function> fetcher() { + protected Function fetcher() { return this.fetcher; } @@ -93,21 +92,21 @@ public boolean empty() { return this.queries.isEmpty(); } - public Iterator fetch() { + public QueryResults fetch() { assert !this.queries.isEmpty(); if (this.parent.paging()) { int pageSize = this.graph.configuration() .get(CoreOptions.QUERY_PAGE_SIZE); - return new PageEntryIterator(this, pageSize); - } else { - return this.fetchAll(); + PageEntryIterator iterator = new PageEntryIterator(this, pageSize); + /* + * NOTE: PageEntryIterator query will change every fetch time. + * TODO: sort results by input ids in each page. + */ + return iterator.results(); } - } - protected Iterator fetchAll() { - return new FlatMapperIterator<>(this.queries.iterator(), q -> { - return q.iterator(); - }); + // Fetch all results once + return QueryResults.flatMap(this.queries.iterator(), q -> q.iterator()); } protected PageIterator fetchNext(PageInfo pageInfo, long pageSize) { @@ -157,7 +156,7 @@ private interface QueryHolder { * For non-paging situation * @return BackendEntry iterator */ - public Iterator iterator(); + public QueryResults iterator(); /** * For paging situation @@ -183,7 +182,7 @@ public OptimizedQuery(Query query) { } @Override - public Iterator iterator() { + public QueryResults iterator() { return fetcher().apply(this.query); } @@ -196,11 +195,11 @@ public PageIterator iterator(int index, String page, long pageSize) { if (this.query.limit() == Query.NO_LIMIT) { query.limit(pageSize); } - Iterator iterator = fetcher().apply(query); + QueryResults results = fetcher().apply(query); // Must iterate all entries before get the next page - List results = IteratorUtils.list(iterator); - return new PageIterator(results.iterator(), - PageInfo.pageState(iterator)); + return new PageIterator(results.list().iterator(), + results.queries(), + PageInfo.pageState(results.iterator())); } @Override @@ -222,23 +221,28 @@ public IndexQuery(List holders) { } @Override - public Iterator iterator() { - return new FlatMapperIterator<>(this.holders.iterator(), holder -> { - if (holder.ids().isEmpty()) { - return null; - } - Set ids = holder.ids(); - if (parent().limit() != Query.NO_LIMIT && - ids.size() > parent().limit()) { - /* - * Avoid too many ids in one time query, - * Assume it will get one result by each id - */ - ids = CollectionUtil.subSet(ids, 0, (int) parent().limit()); - } - IdQuery query = new IdQuery(parent(), ids); - return fetcher().apply(query); - }); + public QueryResults iterator() { + if (this.holders.size() == 1) { + return this.each(this.holders.get(0)); + } + return QueryResults.flatMap(this.holders.iterator(), this::each); + } + + private QueryResults each(IdHolder holder) { + Set ids = holder.ids(); + if (ids.isEmpty()) { + return null; + } + if (parent().limit() != Query.NO_LIMIT && + ids.size() > parent().limit()) { + /* + * Avoid too many ids in one time query, + * Assume it will get one result by each id + */ + ids = CollectionUtil.subSet(ids, 0, (int) parent().limit()); + } + IdQuery query = new IdQuery(parent(), ids); + return fetcher().apply(query); } @Override @@ -249,7 +253,8 @@ public PageIterator iterator(int index, String page, long pageSize) { return PageIterator.EMPTY; } IdQuery query = new IdQuery(parent(), pageIds.ids()); - return new PageIterator(fetcher().apply(query), + QueryResults results = fetcher().apply(query); + return new PageIterator(results.iterator(), results.queries(), pageIds.pageState()); } @@ -263,18 +268,21 @@ public static class PageIterator { public static final PageIterator EMPTY = new PageIterator( Collections.emptyIterator(), + ImmutableList.of(Query.NONE), PageState.EMPTY); private final Iterator iterator; + private final List queries; private final PageState pageState; public PageIterator(Iterator iterator, - PageState pageState) { + List queries, PageState pageState) { this.iterator = iterator; + this.queries = queries; this.pageState = pageState; } - public Iterator iterator() { + public Iterator get() { return this.iterator; } @@ -283,6 +291,12 @@ public boolean hasNextPage() { PageState.EMPTY_BYTES); } + public Query query() { + E.checkState(this.queries.size() == 1, + "Expect queries size 1, but got: %s", this.queries); + return this.queries.get(0); + } + public String page() { return this.pageState.toString(); } diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/page/SortByCountIdHolderList.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/page/SortByCountIdHolderList.java new file mode 100644 index 0000000000..77f292799f --- /dev/null +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/page/SortByCountIdHolderList.java @@ -0,0 +1,85 @@ +/* + * 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.page; + +import java.util.Map; +import java.util.Set; + +import org.apache.commons.lang.NotImplementedException; + +import com.baidu.hugegraph.backend.id.Id; +import com.baidu.hugegraph.util.CollectionUtil; +import com.baidu.hugegraph.util.InsertionOrderUtil; +import com.google.common.collect.ImmutableSet; + +public class SortByCountIdHolderList extends IdHolderList { + + private static final long serialVersionUID = -3702668078311531645L; + + public SortByCountIdHolderList(boolean paging) { + super(paging); + } + + @Override + public boolean add(IdHolder holder) { + if (!this.paging()) { + holder = new SortByCountIdHolder(holder); + } + return super.add(holder); + } + + private static class SortByCountIdHolder extends IdHolder { + + private final Map ids; + + public SortByCountIdHolder(IdHolder holder) { + super(ImmutableSet.of()); + this.ids = InsertionOrderUtil.newMap(); + this.merge(holder.ids()); + } + + @Override + public void merge(Set ids) { + for (Id id : ids) { + this.ids.compute(id, (k, v) -> v == null ? 1 : v + 1); + } + } + + @Override + public Set ids() { + return CollectionUtil.sortByValue(this.ids, false).keySet(); + } + + @Override + public int size() { + return this.ids.size(); + } + + @Override + public boolean paging() { + return false; + } + + @Override + public PageIds fetchNext(String page, long pageSize) { + throw new NotImplementedException("SortByCountIdHolder.fetchNext"); + } + } +} diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/query/IdQuery.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/query/IdQuery.java index 5eeef2f9b6..ec7ef1033a 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/query/IdQuery.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/query/IdQuery.java @@ -32,6 +32,7 @@ public class IdQuery extends Query { // The id(s) will be concated with `or` private Set ids = InsertionOrderUtil.newSet(); + private boolean mustSortByInput = true; public IdQuery(HugeType resultType) { super(resultType); @@ -61,6 +62,14 @@ public IdQuery(Query originQuery, Set ids) { this.query(ids); } + public boolean mustSortByInput() { + return this.mustSortByInput; + } + + public void mustSortByInput(boolean mustSortedByInput) { + this.mustSortByInput = mustSortedByInput; + } + @Override public Set ids() { return Collections.unmodifiableSet(this.ids); diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/query/Query.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/query/Query.java index 01d2be2958..0b5a89efde 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/query/Query.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/query/Query.java @@ -41,6 +41,10 @@ public class Query implements Cloneable { public static final long NO_CAPACITY = -1L; public static final long DEFAULT_CAPACITY = 800000L; // HugeGraph-777 + private static final ThreadLocal capacityContex = new ThreadLocal<>(); + + public static final Query NONE = new Query(HugeType.UNKNOWN); + private HugeType resultType; private Map orders; private long offset; @@ -52,8 +56,6 @@ public class Query implements Cloneable { private Query originQuery; - private static final ThreadLocal capacityContex = new ThreadLocal<>(); - public Query(HugeType resultType) { this(resultType, null); } @@ -305,7 +307,7 @@ public int hashCode() { public String toString() { Map pairs = InsertionOrderUtil.newMap(); if (this.page != null) { - pairs.put("page", this.page); + pairs.put("page", String.format("'%s'", this.page)); } if (this.offset != 0) { pairs.put("offset", this.offset); 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 010342570b..e1d6e488fb 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 @@ -364,7 +364,11 @@ protected void parseIndexName(BinaryBackendEntry entry, HugeIndex index, if (this.indexWithIdPrefix) { buffer.readIndexId(index.type()); } - index.elementIds(buffer.readId(true)); + Id elemId = buffer.readId(true); + if (index.indexLabel().queryType().isEdge()) { + elemId = EdgeId.parse(elemId.asString()); + } + index.elementIds(elemId); } } @@ -713,7 +717,7 @@ private Query writeRangeIndexQuery(ConditionQuery query) { } private BinaryBackendEntry formatILDeletion(HugeIndex index) { - Id id = index.indexLabel(); + Id id = index.indexLabelId(); BinaryId bid = new BinaryId(id.asBytes(), id); BinaryBackendEntry entry = new BinaryBackendEntry(index.type(), bid); if (index.type().isStringIndex()) { 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 cbdfb7964f..7822abe826 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 @@ -281,10 +281,10 @@ public BackendEntry writeIndex(HugeIndex index) { * meaningful for deletion of index data in secondary/range index. */ if (index.fieldValues() == null && index.elementIds().size() == 0) { - entry.column(HugeKeys.INDEX_LABEL_ID, index.indexLabel().asLong()); + entry.column(HugeKeys.INDEX_LABEL_ID, index.indexLabel().longId()); } else { entry.column(HugeKeys.FIELD_VALUES, index.fieldValues()); - entry.column(HugeKeys.INDEX_LABEL_ID, index.indexLabel().asLong()); + entry.column(HugeKeys.INDEX_LABEL_ID, index.indexLabel().longId()); entry.column(HugeKeys.ELEMENT_IDS, IdUtil.writeString(index.elementId())); entry.subId(index.elementId()); @@ -309,8 +309,12 @@ public HugeIndex readIndex(HugeGraph graph, ConditionQuery query, IndexLabel indexLabel = graph.indexLabel(this.toId(indexLabelId)); HugeIndex index = new HugeIndex(indexLabel); index.fieldValues(indexValues); - for (String id : elemIds) { - index.elementIds(IdUtil.readString(id)); + for (String elemId : elemIds) { + Id id = IdUtil.readString(elemId); + if (indexLabel.queryType().isEdge()) { + id = EdgeId.parse(id.asString()); + } + index.elementIds(id); } return index; } 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 197288ed28..520e8f9662 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 @@ -330,13 +330,13 @@ public BackendEntry writeIndex(HugeIndex index) { */ if (index.fieldValues() == null && index.elementIds().size() == 0) { entry.column(HugeKeys.INDEX_LABEL_ID, - writeId(index.indexLabel())); + writeId(index.indexLabelId())); } else { // TODO: field-values may be a number (range index) entry.column(formatSyspropName(HugeKeys.FIELD_VALUES), JsonUtil.toJson(index.fieldValues())); entry.column(formatSyspropName(HugeKeys.INDEX_LABEL_ID), - writeId(index.indexLabel())); + writeId(index.indexLabelId())); entry.column(formatSyspropName(HugeKeys.ELEMENT_IDS), writeIds(index.elementIds())); entry.subId(index.elementId()); @@ -357,13 +357,18 @@ public HugeIndex readIndex(HugeGraph graph, ConditionQuery query, formatSyspropName(HugeKeys.FIELD_VALUES)); String indexLabelId = entry.column( formatSyspropName(HugeKeys.INDEX_LABEL_ID)); - String elementIds = entry.column( + String elemIds = entry.column( formatSyspropName(HugeKeys.ELEMENT_IDS)); IndexLabel indexLabel = IndexLabel.label(graph, readId(indexLabelId)); HugeIndex index = new HugeIndex(indexLabel); index.fieldValues(JsonUtil.fromJson(indexValues, Object.class)); - index.elementIds(readIds(elementIds)); + for (Id elemId : readIds(elemIds)) { + if (indexLabel.queryType().isEdge()) { + elemId = EdgeId.parse(elemId.asString()); + } + index.elementIds(elemId); + } return index; } 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 21a32ded79..966cfe6264 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 @@ -26,10 +26,11 @@ import com.baidu.hugegraph.backend.id.Id; import com.baidu.hugegraph.type.HugeType; +import com.baidu.hugegraph.type.Idfiable; import com.baidu.hugegraph.util.Bytes; import com.baidu.hugegraph.util.StringEncoding; -public interface BackendEntry { +public interface BackendEntry extends Idfiable { public static class BackendColumn implements Comparable { diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/BackendFeatures.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/BackendFeatures.java index b9cb3f136b..78ab30e709 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/BackendFeatures.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/BackendFeatures.java @@ -45,6 +45,8 @@ public default boolean supportsPersistence() { public boolean supportsQueryByPage(); + public boolean supportsQuerySortByInputIds(); + public boolean supportsDeleteEdgeByLabel(); public boolean supportsUpdateVertexProperty(); diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/memory/InMemoryDBStore.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/memory/InMemoryDBStore.java index 3f2cf2d89a..0680719024 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/memory/InMemoryDBStore.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/store/memory/InMemoryDBStore.java @@ -381,6 +381,11 @@ public boolean supportsQueryByPage() { return false; } + @Override + public boolean supportsQuerySortByInputIds() { + return true; + } + @Override public boolean supportsDeleteEdgeByLabel() { return false; diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/tx/AbstractTransaction.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/tx/AbstractTransaction.java index 48674ac100..20a95bb28f 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/tx/AbstractTransaction.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/tx/AbstractTransaction.java @@ -19,8 +19,15 @@ package com.baidu.hugegraph.backend.tx; +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.function.Function; +import org.apache.tinkerpop.gremlin.util.iterator.IteratorUtils; import org.slf4j.Logger; import com.baidu.hugegraph.HugeGraph; @@ -34,10 +41,14 @@ import com.baidu.hugegraph.backend.store.BackendMutation; import com.baidu.hugegraph.backend.store.BackendStore; import com.baidu.hugegraph.exception.NotFoundException; +import com.baidu.hugegraph.iterator.FlatMapperIterator; +import com.baidu.hugegraph.iterator.MapperIterator; import com.baidu.hugegraph.perf.PerfUtil.Watched; import com.baidu.hugegraph.type.HugeType; +import com.baidu.hugegraph.type.Idfiable; import com.baidu.hugegraph.type.define.Action; import com.baidu.hugegraph.util.E; +import com.baidu.hugegraph.util.InsertionOrderUtil; import com.baidu.hugegraph.util.Log; import com.google.common.util.concurrent.RateLimiter; @@ -87,7 +98,7 @@ public R metadata(HugeType type, String meta, Object... args) { } @Watched(prefix = "tx") - public Iterator query(Query query) { + public QueryResults query(Query query) { LOG.debug("Transaction query: {}", query); /* * NOTE: it's dangerous if an IdQuery/ConditionQuery is empty @@ -97,11 +108,11 @@ public Iterator query(Query query) { throw new BackendException("Query without any id or condition"); } - query = this.serializer.writeQuery(query); + Query squery = this.serializer.writeQuery(query); this.beforeRead(); try { - return this.store.query(query); + return new QueryResults(this.store.query(squery), query); } finally { this.afterRead(); // TODO: not complete the iteration currently } @@ -110,7 +121,7 @@ public Iterator query(Query query) { @Watched(prefix = "tx") public BackendEntry query(HugeType type, Id id) { IdQuery q = new IdQuery(type, id); - Iterator results = this.query(q); + Iterator results = this.query(q).iterator(); if (results.hasNext()) { BackendEntry entry = results.next(); assert !results.hasNext(); @@ -308,12 +319,6 @@ protected void checkOwnerThread() { } } - protected void doAction(Action action, BackendEntry entry) { - LOG.debug("Transaction {} entry {}", action, entry); - E.checkNotNull(entry, "entry"); - this.mutation.add(entry, action); - } - @Watched(prefix = "tx") public void doInsert(BackendEntry entry) { this.doAction(Action.INSERT, entry); @@ -333,4 +338,144 @@ public void doEliminate(BackendEntry entry) { public void doRemove(BackendEntry entry) { this.doAction(Action.DELETE, entry); } + + protected void doAction(Action action, BackendEntry entry) { + LOG.debug("Transaction {} entry {}", action, entry); + E.checkNotNull(entry, "entry"); + this.mutation.add(entry, action); + } + + public static class QueryResults { + + private static final QueryResults EMPTY = new QueryResults( + Collections.emptyIterator(), + Query.NONE); + + private final Iterator results; + private final List queries; + + public QueryResults(Iterator results, Query query) { + this(results); + this.addQuery(query); + } + + public QueryResults(Iterator results) { + this.results = results; + this.queries = InsertionOrderUtil.newList(); + } + + public void setQuery(Query query) { + if (this.queries.size() > 0) { + this.queries.clear(); + } + this.addQuery(query); + } + + private void addQuery(Query query) { + E.checkNotNull(query, "query"); + this.queries.add(query); + } + + private void addQueries(List queries) { + for (Query query : queries) { + this.addQuery(query); + } + } + + public Iterator iterator() { + return this.results; + } + + public List list() { + return IteratorUtils.list(this.results); + } + + public List queries() { + return Collections.unmodifiableList(this.queries); + } + + protected Iterator keepInputOrderIfNeeded( + Iterator origin) { + if (!origin.hasNext()) { + // None result found + return origin; + } + Set ids; + if (this.paging() || !this.mustSortByInputIds() || + (ids = this.queryIds()).size() <= 1) { + /* + * Return the original iterator if it's paging query or if the + * query input is less than one id, or don't have to do sort. + */ + return origin; + } + + // Fill map with all elements + Map results = new HashMap<>(); + fillMap(origin, results); + + return new MapperIterator<>(ids.iterator(), id -> { + return results.get(id); + }); + } + + private boolean mustSortByInputIds() { + if (this.queries.size() == 1) { + Query query = this.queries.get(0); + if (query instanceof IdQuery) { + return ((IdQuery) query).mustSortByInput(); + } + } + return true; + } + + private boolean paging() { + for (Query query : this.queries) { + Query origin = query.originQuery(); + if (query.paging() || origin != null && origin.paging()) { + return true; + } + } + return false; + } + + private Set queryIds() { + if (this.queries.size() == 1) { + return this.queries.get(0).ids(); + } + + Set ids = InsertionOrderUtil.newSet(); + for (Query query : this.queries) { + ids.addAll(query.ids()); + } + return ids; + } + + public static void fillMap(Iterator iterator, + Map map) { + while (iterator.hasNext()) { + T result = iterator.next(); + assert result.id() != null; + map.put(result.id(), result); + } + } + + public static QueryResults empty() { + return EMPTY; + } + + public static QueryResults flatMap(Iterator iterator, + Function func) { + QueryResults[] qr = new QueryResults[1]; + qr[0] = new QueryResults(new FlatMapperIterator<>(iterator, i -> { + QueryResults results = func.apply(i); + if (results == null) { + return null; + } + qr[0].addQueries(results.queries()); + return results.iterator(); + })); + return qr[0]; + } + } } 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 61902313b7..c38aa5c660 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 @@ -47,6 +47,7 @@ import com.baidu.hugegraph.backend.page.PageIds; import com.baidu.hugegraph.backend.page.PageInfo; import com.baidu.hugegraph.backend.page.PageState; +import com.baidu.hugegraph.backend.page.SortByCountIdHolderList; import com.baidu.hugegraph.backend.query.Condition; import com.baidu.hugegraph.backend.query.Condition.RangeConditions; import com.baidu.hugegraph.backend.query.Condition.Relation; @@ -296,7 +297,7 @@ private boolean existUniqueValueInStore(IndexLabel indexLabel, ConditionQuery query = new ConditionQuery(HugeType.UNIQUE_INDEX); query.eq(HugeKeys.INDEX_LABEL_ID, indexLabel.id()); query.eq(HugeKeys.FIELD_VALUES, value); - Iterator iterator = this.query(query); + Iterator iterator = this.query(query).iterator(); boolean exist = iterator.hasNext(); if (exist) { HugeIndex index = this.serializer.readIndex(graph(), query, @@ -409,8 +410,8 @@ private List queryByUserprop(ConditionQuery query) { return ImmutableList.of(); } - boolean paging = query.paging(); // Do index query + boolean paging = query.paging(); IdHolderList holders = new IdHolderList(paging); long idsSize = 0; for (MatchedIndex index : indexes) { @@ -425,7 +426,7 @@ private List queryByUserprop(ConditionQuery query) { // Do secondary-index, range-index or shard-index query IndexQueries queries = index.constructIndexQueries(query); assert !paging || queries.size() <= 1; - IdHolder holder = this.doIndexQueries(queries); + IdHolder holder = this.doSingleOrJointIndex(queries); holders.add(holder); } @@ -437,14 +438,16 @@ private List queryByUserprop(ConditionQuery query) { return holders; } + @Watched(prefix = "index") private List doSearchIndex(ConditionQuery query, MatchedIndex index) { query = this.constructSearchQuery(query, index); - List holders = new IdHolderList(query.paging()); + List holders = new SortByCountIdHolderList(query.paging()); + // sorted by matched count for (ConditionQuery q : ConditionQueryFlatten.flatten(query)) { IndexQueries queries = index.constructIndexQueries(q); assert !query.paging() || queries.size() <= 1; - IdHolder holder = this.doIndexQueries(queries); + IdHolder holder = this.doSingleOrJointIndex(queries); // NOTE: ids will be merged into one IdHolder if not in paging holders.add(holder); } @@ -452,7 +455,7 @@ private List doSearchIndex(ConditionQuery query, } @Watched(prefix = "index") - private IdHolder doIndexQueries(IndexQueries queries) { + private IdHolder doSingleOrJointIndex(IndexQueries queries) { if (queries.size() == 1) { return this.doSingleOrCompositeIndex(queries); } else { @@ -486,63 +489,6 @@ private IdHolder doJointIndex(IndexQueries queries) { return new IdHolder(intersectIds); } - private ConditionQuery constructSearchQuery(ConditionQuery query, - MatchedIndex index) { - ConditionQuery originQuery = query; - Set indexFields = new HashSet<>(); - // Convert has(key, text) to has(key, textContainsAny(word1, word2)) - for (IndexLabel il : index.indexLabels()) { - if (il.indexType() != IndexType.SEARCH) { - continue; - } - Id indexField = il.indexField(); - String fieldValue = (String) query.userpropValue(indexField); - Set words = this.segmentWords(fieldValue); - indexFields.add(indexField); - - query = query.copy(); - query.unsetCondition(indexField); - query.query(Condition.textContainsAny(indexField, words)); - } - - // Register results filter - query.registerResultsFilter(elem -> { - for (Condition cond : originQuery.conditions()) { - Object key = cond.isRelation() ? ((Relation) cond).key() : null; - if (key instanceof Id && indexFields.contains(key)) { - // This is an index field of search index - Id field = (Id) key; - String propValue = elem.getPropertyValue(field); - String fvalue = (String) originQuery.userpropValue(field); - if (this.matchSearchIndexWords(propValue, fvalue)) { - continue; - } - return false; - } - if (!cond.test(elem)) { - return false; - } - } - return true; - }); - - return query; - } - - private boolean matchSearchIndexWords(String propValue, String fieldValue) { - Set propValues = this.segmentWords(propValue); - Set words = this.segmentWords(fieldValue); - return CollectionUtil.hasIntersection(propValues, words); - } - - private Set segmentWords(String text) { - return this.textAnalyzer.segment(text); - } - - private boolean needIndexForLabel() { - return !this.store().features().supportsQueryByLabel(); - } - @Watched(prefix = "index") private IdHolder doIndexQuery(IndexLabel indexLabel, ConditionQuery query) { if (!query.paging()) { @@ -564,7 +510,7 @@ private PageIds doIndexQueryOnce(IndexLabel indexLabel, locks.lockReads(LockUtil.INDEX_LABEL_REBUILD, indexLabel.id()); Set ids = InsertionOrderUtil.newSet(); - Iterator entries = super.query(query); + Iterator entries = super.query(query).iterator(); while(entries.hasNext()) { HugeIndex index = this.serializer.readIndex(graph(), query, entries.next()); @@ -664,6 +610,64 @@ private MatchedIndex collectMatchedIndex(SchemaLabel schemaLabel, return null; } + + private ConditionQuery constructSearchQuery(ConditionQuery query, + MatchedIndex index) { + ConditionQuery originQuery = query; + Set indexFields = new HashSet<>(); + // Convert has(key, text) to has(key, textContainsAny(word1, word2)) + for (IndexLabel il : index.indexLabels()) { + if (il.indexType() != IndexType.SEARCH) { + continue; + } + Id indexField = il.indexField(); + String fieldValue = (String) query.userpropValue(indexField); + Set words = this.segmentWords(fieldValue); + indexFields.add(indexField); + + query = query.copy(); + query.unsetCondition(indexField); + query.query(Condition.textContainsAny(indexField, words)); + } + + // Register results filter + query.registerResultsFilter(elem -> { + for (Condition cond : originQuery.conditions()) { + Object key = cond.isRelation() ? ((Relation) cond).key() : null; + if (key instanceof Id && indexFields.contains(key)) { + // This is an index field of search index + Id field = (Id) key; + String propValue = elem.getPropertyValue(field); + String fvalue = (String) originQuery.userpropValue(field); + if (this.matchSearchIndexWords(propValue, fvalue)) { + continue; + } + return false; + } + if (!cond.test(elem)) { + return false; + } + } + return true; + }); + + return query; + } + + private boolean matchSearchIndexWords(String propValue, String fieldValue) { + Set propValues = this.segmentWords(propValue); + Set words = this.segmentWords(fieldValue); + return CollectionUtil.hasIntersection(propValues, words); + } + + private Set segmentWords(String text) { + return this.textAnalyzer.segment(text); + } + + private boolean needIndexForLabel() { + return !this.store().features().supportsQueryByLabel(); + } + private static Set matchSingleOrCompositeIndex( ConditionQuery query, Set indexLabels) { @@ -1354,7 +1358,8 @@ private long processRangeIndexLeft(ConditionQuery query, continue; } // Query and delete index equals element id - for (Iterator it = tx.query(q); it.hasNext();) { + Iterator it = tx.query(q).iterator(); + while (it.hasNext()) { BackendEntry entry = it.next(); HugeIndex index = serializer.readIndex(graph(), q, entry); if (index.elementIds().contains(element.id())) { 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 bba3cb8a2d..0022324c41 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 @@ -23,6 +23,7 @@ import java.util.Arrays; import java.util.Collection; import java.util.Collections; +import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; import java.util.List; @@ -380,7 +381,7 @@ public void rollback() throws BackendException { } @Override - public Iterator query(Query query) { + public QueryResults query(Query query) { if (!(query instanceof ConditionQuery)) { return super.query(query); } @@ -402,7 +403,7 @@ public Iterator query(Query query) { } } - return !queries.empty() ? queries.fetch() : Collections.emptyIterator(); + return !queries.empty() ? queries.fetch() : QueryResults.empty(); } @Watched(prefix = "graph") @@ -503,7 +504,7 @@ public Iterator queryAdjacentVertices(Iterator edges) { public Iterator queryVertices(Object... vertexIds) { // NOTE: allowed duplicated vertices if query by duplicated ids List ids = InsertionOrderUtil.newList(); - Map vertices = InsertionOrderUtil.newMap(); + Map vertices = new HashMap<>(vertexIds.length); IdQuery query = new IdQuery(HugeType.VERTEX); for (Object vertexId : vertexIds) { @@ -517,7 +518,7 @@ public Iterator queryVertices(Object... vertexIds) { // Found from local tx vertices.put(vertex.id(), vertex); } else { - // Prepare query from backend store + // Prepare to query from backend store query.query(id); } ids.add(id); @@ -525,11 +526,16 @@ public Iterator queryVertices(Object... vertexIds) { if (!query.empty()) { // Query from backend store - Iterator it = this.queryVerticesFromBackend(query); - while (it.hasNext()) { - HugeVertex vertex = it.next(); - vertices.put(vertex.id(), vertex); + if (vertices.isEmpty() && query.ids().size() == ids.size()) { + // Sort at the lower layer and return directly + Iterator it = this.queryVerticesFromBackend(query); + @SuppressWarnings({ "unchecked", "rawtypes" }) + Iterator r = (Iterator) it; + return r; } + query.mustSortByInput(false); + Iterator it = this.queryVerticesFromBackend(query); + QueryResults.fillMap(it, vertices); } return new MapperIterator<>(ids.iterator(), id -> { @@ -579,13 +585,20 @@ public Iterator queryVertices(Query query) { protected Iterator queryVerticesFromBackend(Query query) { assert query.resultType().isVertex(); - Iterator entries = this.query(query); + QueryResults results = this.query(query); + Iterator entries = results.iterator(); - return new MapperIterator<>(entries, entry -> { + Iterator vertices = new MapperIterator<>(entries, entry -> { HugeVertex vertex = this.serializer.readVertex(graph(), entry); assert vertex != null; return vertex; }); + + if (!this.store().features().supportsQuerySortByInputIds()) { + // There is no id in BackendEntry, so sort after deserialization + vertices = results.keepInputOrderIfNeeded(vertices); + } + return vertices; } @Watched(prefix = "graph") @@ -638,7 +651,7 @@ public Iterator queryEdgesByVertex(Id id) { public Iterator queryEdges(Object... edgeIds) { // NOTE: allowed duplicated edges if query by duplicated ids List ids = InsertionOrderUtil.newList(); - Map edges = InsertionOrderUtil.newMap(); + Map edges = new HashMap<>(edgeIds.length); IdQuery query = new IdQuery(HugeType.EDGE); for (Object edgeId : edgeIds) { @@ -652,7 +665,7 @@ public Iterator queryEdges(Object... edgeIds) { // Found from local tx edges.put(edge.id(), edge); } else { - // Prepare query from backend store + // Prepare to query from backend store query.query(id); } ids.add(id); @@ -660,11 +673,16 @@ public Iterator queryEdges(Object... edgeIds) { if (!query.empty()) { // Query from backend store - Iterator it = this.queryEdgesFromBackend(query); - while (it.hasNext()) { - HugeEdge edge = it.next(); - edges.put(edge.id(), edge); + if (edges.isEmpty() && query.ids().size() == ids.size()) { + // Sort at the lower layer and return directly + Iterator it = this.queryEdgesFromBackend(query); + @SuppressWarnings({ "unchecked", "rawtypes" }) + Iterator r = (Iterator) it; + return r; } + query.mustSortByInput(false); + Iterator it = this.queryEdgesFromBackend(query); + QueryResults.fillMap(it, edges); } return new MapperIterator<>(ids.iterator(), id -> { @@ -731,9 +749,10 @@ public Iterator queryEdges(Query query) { protected Iterator queryEdgesFromBackend(Query query) { assert query.resultType().isEdge(); - Iterator entries = this.query(query); + QueryResults results = this.query(query); + Iterator entries = results.iterator(); - return new FlatMapperIterator<>(entries, entry -> { + Iterator edges = new FlatMapperIterator<>(entries, entry -> { // Edges are in a vertex HugeVertex vertex = this.serializer.readVertex(graph(), entry); assert vertex != null; @@ -743,6 +762,12 @@ protected Iterator queryEdgesFromBackend(Query query) { // Copy to avoid ConcurrentModificationException when removing edge return ImmutableList.copyOf(vertex.getEdges()).iterator(); }); + + if (!this.store().features().supportsQuerySortByInputIds()) { + // There is no id in BackendEntry, so sort after deserialization + edges = results.keepInputOrderIfNeeded(edges); + } + return edges; } @Watched(prefix = "graph") diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/tx/SchemaIndexTransaction.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/tx/SchemaIndexTransaction.java index ab39556f54..e9debbedc3 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/tx/SchemaIndexTransaction.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/tx/SchemaIndexTransaction.java @@ -19,7 +19,6 @@ package com.baidu.hugegraph.backend.tx; -import java.util.Collections; import java.util.Iterator; import com.baidu.hugegraph.HugeGraph; @@ -67,7 +66,7 @@ private boolean needIndexForName() { @Watched(prefix = "index") @Override - public Iterator query(Query query) { + public QueryResults query(Query query) { if (query instanceof ConditionQuery) { ConditionQuery q = (ConditionQuery) query; if (q.allSysprop() && q.conditions().size() == 1 && @@ -79,7 +78,7 @@ public Iterator query(Query query) { } @Watched(prefix = "index") - private Iterator queryByName(ConditionQuery query) { + private QueryResults queryByName(ConditionQuery query) { if (!this.needIndexForName()) { return super.query(query); } @@ -93,7 +92,7 @@ private Iterator queryByName(ConditionQuery query) { indexQuery.eq(HugeKeys.FIELD_VALUES, name); indexQuery.eq(HugeKeys.INDEX_LABEL_ID, il.id()); - Iterator entries = super.query(indexQuery); + Iterator entries = super.query(indexQuery).iterator(); IdQuery idQuery = new IdQuery(query.resultType(), query); while (entries.hasNext()) { HugeIndex index = this.serializer.readIndex(graph(), indexQuery, @@ -101,7 +100,7 @@ private Iterator queryByName(ConditionQuery query) { idQuery.query(index.elementIds()); } if (idQuery.ids().isEmpty()) { - return Collections.emptyIterator(); + return QueryResults.empty(); } assert idQuery.ids().size() == 1 : idQuery.ids(); return super.query(idQuery); diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/tx/SchemaTransaction.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/tx/SchemaTransaction.java index cf35fdc82a..a93708d188 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/tx/SchemaTransaction.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/backend/tx/SchemaTransaction.java @@ -307,7 +307,7 @@ protected T getSchema(HugeType type, this.beforeRead(); ConditionQuery query = new ConditionQuery(type); query.eq(HugeKeys.NAME, name); - Iterator iter = this.indexTx.query(query); + Iterator iter = this.indexTx.query(query).iterator(); this.afterRead(); if (iter.hasNext()) { T schema = this.deserialize(iter.next(), type); @@ -320,7 +320,7 @@ protected T getSchema(HugeType type, protected List getAllSchema(HugeType type) { Query query = new Query(type); - Iterator entries = this.query(query); + Iterator entries = this.query(query).iterator(); List result = new ArrayList<>(); while (entries.hasNext()) { diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/schema/IndexLabel.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/schema/IndexLabel.java index 3deb6062c7..d2ba544f68 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/schema/IndexLabel.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/schema/IndexLabel.java @@ -43,6 +43,8 @@ public class IndexLabel extends SchemaElement { public IndexLabel(final HugeGraph graph, Id id, String name) { super(graph, id, name); + this.baseType = HugeType.SYS_SCHEMA; + this.baseValue = null; this.indexType = IndexType.SECONDARY; this.indexFields = new ArrayList<>(); } @@ -86,6 +88,8 @@ public HugeType queryType() { return HugeType.VERTEX; case EDGE_LABEL: return HugeType.EDGE; + case SYS_SCHEMA: + return HugeType.SYS_SCHEMA; default: throw new AssertionError(String.format( "Query type of index label is either '%s' or '%s', " + diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/schema/SchemaElement.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/schema/SchemaElement.java index 3e46a1acf3..983bfd3f97 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/schema/SchemaElement.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/schema/SchemaElement.java @@ -67,6 +67,10 @@ public Id id() { return this.id; } + public long longId() { + return this.id.asLong(); + } + @Override public String name() { return this.name; diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/structure/HugeElement.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/structure/HugeElement.java index b124c10720..4e6a895381 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/structure/HugeElement.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/structure/HugeElement.java @@ -44,13 +44,14 @@ import com.baidu.hugegraph.schema.SchemaLabel; import com.baidu.hugegraph.schema.VertexLabel; import com.baidu.hugegraph.type.HugeType; +import com.baidu.hugegraph.type.Idfiable; import com.baidu.hugegraph.type.define.Cardinality; import com.baidu.hugegraph.type.define.HugeKeys; import com.baidu.hugegraph.util.CollectionUtil; import com.baidu.hugegraph.util.E; import com.google.common.collect.ImmutableMap; -public abstract class HugeElement implements Element, GraphType { +public abstract class HugeElement implements Element, GraphType, Idfiable { private static final Map> EMPTY = ImmutableMap.of(); private static final int MAX_PROPERTIES = BytesBuffer.UINT16_MAX; diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/structure/HugeIndex.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/structure/HugeIndex.java index 3c4afa7297..fec726f089 100644 --- a/hugegraph-core/src/main/java/com/baidu/hugegraph/structure/HugeIndex.java +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/structure/HugeIndex.java @@ -69,11 +69,11 @@ public HugeType type() { } public Id id() { - return formatIndexId(type(), this.indexLabel(), this.fieldValues()); + return formatIndexId(type(), this.indexLabelId(), this.fieldValues()); } public Id hashId() { - return formatIndexHashId(type(), this.indexLabel(), this.fieldValues()); + return formatIndexHashId(type(), this.indexLabelId(), this.fieldValues()); } public Object fieldValues() { @@ -84,10 +84,14 @@ public void fieldValues(Object fieldValues) { this.fieldValues = fieldValues; } - public Id indexLabel() { + public Id indexLabelId() { return this.indexLabel.id(); } + public IndexLabel indexLabel() { + return this.indexLabel; + } + public Id elementId() { E.checkState(this.elementIds.size() == 1, "Expect one element id, actual %s", diff --git a/hugegraph-core/src/main/java/com/baidu/hugegraph/type/Idfiable.java b/hugegraph-core/src/main/java/com/baidu/hugegraph/type/Idfiable.java new file mode 100644 index 0000000000..62e40c0879 --- /dev/null +++ b/hugegraph-core/src/main/java/com/baidu/hugegraph/type/Idfiable.java @@ -0,0 +1,27 @@ +/* + * 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.type; + +import com.baidu.hugegraph.backend.id.Id; + +public interface Idfiable { + + public Id id(); +} diff --git a/hugegraph-hbase/src/main/java/com/baidu/hugegraph/backend/store/hbase/HbaseFeatures.java b/hugegraph-hbase/src/main/java/com/baidu/hugegraph/backend/store/hbase/HbaseFeatures.java index 4ece477282..c4ad0958eb 100644 --- a/hugegraph-hbase/src/main/java/com/baidu/hugegraph/backend/store/hbase/HbaseFeatures.java +++ b/hugegraph-hbase/src/main/java/com/baidu/hugegraph/backend/store/hbase/HbaseFeatures.java @@ -55,6 +55,11 @@ public boolean supportsQueryWithRangeCondition() { return true; } + @Override + public boolean supportsQuerySortByInputIds() { + return true; + } + @Override public boolean supportsQueryWithOrderBy() { return true; diff --git a/hugegraph-mysql/src/main/java/com/baidu/hugegraph/backend/store/mysql/MysqlFeatures.java b/hugegraph-mysql/src/main/java/com/baidu/hugegraph/backend/store/mysql/MysqlFeatures.java index 15cfcbd1a7..ff7490925e 100644 --- a/hugegraph-mysql/src/main/java/com/baidu/hugegraph/backend/store/mysql/MysqlFeatures.java +++ b/hugegraph-mysql/src/main/java/com/baidu/hugegraph/backend/store/mysql/MysqlFeatures.java @@ -75,6 +75,11 @@ public boolean supportsQueryByPage() { return true; } + @Override + public boolean supportsQuerySortByInputIds() { + return false; + } + @Override public boolean supportsDeleteEdgeByLabel() { return true; 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 5c2bdbaa25..ece5f23e18 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 @@ -43,6 +43,7 @@ import com.baidu.hugegraph.backend.store.mysql.MysqlEntryIterator.PagePosition; import com.baidu.hugegraph.backend.store.mysql.MysqlSessions.Session; import com.baidu.hugegraph.exception.NotFoundException; +import com.baidu.hugegraph.exception.NotSupportException; import com.baidu.hugegraph.iterator.ExtendableIterator; import com.baidu.hugegraph.type.define.HugeKeys; import com.baidu.hugegraph.util.Log; @@ -480,7 +481,7 @@ protected StringBuilder relation2Sql(Condition.Relation relation) { case CONTAINS_KEY: case SCAN: default: - throw new AssertionError("Unsupported relation: " + relation); + throw new NotSupportException("relation '%s'", relation); } return sql; } diff --git a/hugegraph-palo/src/main/java/com/baidu/hugegraph/backend/store/palo/PaloFeatures.java b/hugegraph-palo/src/main/java/com/baidu/hugegraph/backend/store/palo/PaloFeatures.java index 9a9fe145e8..b0ef7ab4bc 100644 --- a/hugegraph-palo/src/main/java/com/baidu/hugegraph/backend/store/palo/PaloFeatures.java +++ b/hugegraph-palo/src/main/java/com/baidu/hugegraph/backend/store/palo/PaloFeatures.java @@ -59,6 +59,11 @@ public boolean supportsQueryWithRangeCondition() { return true; } + @Override + public boolean supportsQuerySortByInputIds() { + return false; + } + @Override public boolean supportsQueryWithContains() { return false; diff --git a/hugegraph-postgresql/src/main/java/com/baidu/hugegraph/backend/store/postgresql/PostgresqlSerializer.java b/hugegraph-postgresql/src/main/java/com/baidu/hugegraph/backend/store/postgresql/PostgresqlSerializer.java index d133ee7a31..6780ed32c2 100644 --- a/hugegraph-postgresql/src/main/java/com/baidu/hugegraph/backend/store/postgresql/PostgresqlSerializer.java +++ b/hugegraph-postgresql/src/main/java/com/baidu/hugegraph/backend/store/postgresql/PostgresqlSerializer.java @@ -38,14 +38,14 @@ public BackendEntry writeIndex(HugeIndex index) { * meaningful for deletion of index data in secondary/range index. */ if (index.fieldValues() == null && index.elementIds().size() == 0) { - entry.column(HugeKeys.INDEX_LABEL_ID, index.indexLabel().asLong()); + entry.column(HugeKeys.INDEX_LABEL_ID, index.indexLabel().longId()); } else { Object value = index.fieldValues(); if (value != null && value.equals("\u0000")) { value = Strings.EMPTY; } entry.column(HugeKeys.FIELD_VALUES, value); - entry.column(HugeKeys.INDEX_LABEL_ID, index.indexLabel().asLong()); + entry.column(HugeKeys.INDEX_LABEL_ID, index.indexLabel().longId()); entry.column(HugeKeys.ELEMENT_IDS, IdUtil.writeString(index.elementId())); entry.subId(index.elementId()); diff --git a/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBFeatures.java b/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBFeatures.java index af42362603..a0da7cea7b 100644 --- a/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBFeatures.java +++ b/hugegraph-rocksdb/src/main/java/com/baidu/hugegraph/backend/store/rocksdb/RocksDBFeatures.java @@ -77,6 +77,11 @@ public boolean supportsQueryByPage() { return true; } + @Override + public boolean supportsQuerySortByInputIds() { + return true; + } + @Override public boolean supportsDeleteEdgeByLabel() { // No index in RocksDB diff --git a/hugegraph-test/src/main/java/com/baidu/hugegraph/core/VertexCoreTest.java b/hugegraph-test/src/main/java/com/baidu/hugegraph/core/VertexCoreTest.java index fc5bd7ef3f..971539105e 100644 --- a/hugegraph-test/src/main/java/com/baidu/hugegraph/core/VertexCoreTest.java +++ b/hugegraph-test/src/main/java/com/baidu/hugegraph/core/VertexCoreTest.java @@ -2448,6 +2448,85 @@ public void testQueryByTextContainsAndExactMatchProperty() { Assert.assertEquals(0, vertices.size()); } + @Test + public void testQueryByTextContainsPropertyOrderByMatchedCount() { + HugeGraph graph = graph(); + + graph.schema().indexLabel("authorByLived").onV("author") + .search().by("lived").create(); + + graph.addVertex(T.label, "author", "id", 1, "name", "Tank", "age", 16, + "lived", "Beijing"); + graph.addVertex(T.label, "author", "id", 2, "name", "Dim", "age", 40, + "lived", "Shenzhen area"); + graph.addVertex(T.label, "author", "id", 3, "name", "Tom", "age", 19, + "lived", "New York Bay"); + graph.addVertex(T.label, "author", "id", 4, "name", "Jason", "age", 20, + "lived", "Tokyo Bay"); + graph.addVertex(T.label, "author", "id", 5, "name", "James", "age", 62, + "lived", "San Francisco Bay Area"); + graph.tx().commit(); + + List vertices = graph.traversal().V() + .hasLabel("author") + .has("lived", Text.contains("Bay Area")) + .toList(); + + Assert.assertEquals(4, vertices.size()); + Assert.assertEquals("James", vertices.get(0).value("name")); + Assert.assertEquals("Tom", vertices.get(1).value("name")); + Assert.assertEquals("Jason", vertices.get(2).value("name")); + Assert.assertEquals("Dim", vertices.get(3).value("name")); + assertContains(vertices, + T.label, "author", "id", 2, "name", "Dim", + "age", 40, "lived", "Shenzhen area"); + assertContains(vertices, + T.label, "author", "id", 3, "name", "Tom", + "age", 19, "lived", "New York Bay"); + assertContains(vertices, + T.label, "author", "id", 4, "name", "Jason", + "age", 20, "lived", "Tokyo Bay"); + assertContains(vertices, + T.label, "author", "id", 5, "name", "James", + "age", 62, "lived", "San Francisco Bay Area"); + } + + @Test + public void testQueryByTextContainsPropertyOrderByMatchedCountWithPaging() { + Assume.assumeTrue("Not support paging", + storeFeatures().supportsQueryByPage()); + + HugeGraph graph = graph(); + + graph.schema().indexLabel("authorByLived").onV("author") + .search().by("lived").create(); + + graph.addVertex(T.label, "author", "id", 1, "name", "Tank", "age", 16, + "lived", "Beijing"); + graph.addVertex(T.label, "author", "id", 2, "name", "Dim", "age", 40, + "lived", "Shenzhen area"); + graph.addVertex(T.label, "author", "id", 3, "name", "Tom", "age", 19, + "lived", "New York Bay"); + graph.addVertex(T.label, "author", "id", 4, "name", "Jason", "age", 20, + "lived", "Tokyo Bay"); + graph.addVertex(T.label, "author", "id", 5, "name", "James", "age", 62, + "lived", "San Francisco Bay Area"); + graph.tx().commit(); + + List vertices = graph.traversal().V() + .hasLabel("author") + .has("lived", Text.contains("Bay Area")) + .has("~page", "").limit(2) + .toList(); + Assert.assertEquals(2, vertices.size()); + assertContains(vertices, + T.label, "author", "id", 3, "name", "Tom", + "age", 19, "lived", "New York Bay"); + assertContains(vertices, + T.label, "author", "id", 4, "name", "Jason", + "age", 20, "lived", "Tokyo Bay"); + } + @Test public void testQueryByTextContainsPropertyWithLeftIndex() { HugeGraph graph = graph(); @@ -3364,9 +3443,10 @@ public void testAddVerticesWithUniqueIndexForNullableProperties() { schema.indexLabel("userByNameCityAge") .onV("user").by("name", "city", "age") .unique().create(); - Vertex v = graph().addVertex(T.label, "user", "name", "Tom", - "city", "Beijing", "age", 18); + graph().addVertex(T.label, "user", "name", "Tom", + "city", "Beijing", "age", 18); graph().tx().commit(); + // Nullable properties graph().addVertex(T.label, "user", "name", "Tom", "city", "Beijing"); graph().tx().commit(); diff --git a/hugegraph-test/src/main/java/com/baidu/hugegraph/unit/core/QueryTest.java b/hugegraph-test/src/main/java/com/baidu/hugegraph/unit/core/QueryTest.java index 3c3dccfff4..c23362a1ef 100644 --- a/hugegraph-test/src/main/java/com/baidu/hugegraph/unit/core/QueryTest.java +++ b/hugegraph-test/src/main/java/com/baidu/hugegraph/unit/core/QueryTest.java @@ -45,33 +45,33 @@ public void testToString() { Query query = new Query(HugeType.VERTEX); Assert.assertEquals("Query for VERTEX", query.toString()); - query.page("page1"); - Assert.assertEquals("Query for VERTEX page page1", query.toString()); + query.page("p1"); + Assert.assertEquals("Query for VERTEX page 'p1'", query.toString()); query = new Query(HugeType.VERTEX); query.limit(10L); Assert.assertEquals("Query for VERTEX limit 10", query.toString()); query = new Query(HugeType.VERTEX); - query.page("page2"); + query.page("p2"); query.limit(10L); - Assert.assertEquals("Query for VERTEX page page2, limit 10", + Assert.assertEquals("Query for VERTEX page 'p2', limit 10", query.toString()); query = new Query(HugeType.VERTEX); - query.page("page3"); + query.page("p3"); query.offset(100L); query.limit(10L); - Assert.assertEquals("Query for VERTEX page page3, offset 100, limit 10", + Assert.assertEquals("Query for VERTEX page 'p3', offset 100, limit 10", query.toString()); query = new Query(HugeType.VERTEX); - query.page("page4"); + query.page(""); query.offset(100L); query.limit(10L); query.order(HugeKeys.NAME, Order.ASC); query.order(HugeKeys.FIELDS, Order.DESC); - Assert.assertEquals("Query for VERTEX page page4, offset 100, " + + Assert.assertEquals("Query for VERTEX page '', offset 100, " + "limit 10, order by {NAME=ASC, FIELDS=DESC}", query.toString()); }