AMashenkov commented on a change in pull request #8490:
URL: https://github.com/apache/ignite/pull/8490#discussion_r572034957
##########
File path:
modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerIndexForceRebuildTest.java
##########
@@ -558,11 +558,11 @@ private void removeLogListener(IgniteEx ignite,
LogListener lsnr) {
/**
* Indexing that blocks index rebuild until status request is completed.
*/
- private static class BlockingIndexing extends IgniteH2Indexing {
+ private static class BlockingIndexing extends IndexesRebuildTask {
Review comment:
```suggestion
private static class BlockingIndexesRebuildTask extends
IndexesRebuildTask {
```
##########
File path:
modules/core/src/main/java/org/apache/ignite/cache/query/index/Index.java
##########
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.cache.query.index;
+
+import java.util.UUID;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Basic interface for Ignite indexes.
+ */
+public interface Index {
+ /**
+ * Unique ID.
+ */
+ public UUID id();
+
+ /**
+ * Index name.
+ */
+ public String name();
+
+ /**
+ * Checks whether index handles specified cache row.
+ *
+ * @param row Cache row.
+ * @return Whether index handles specified cache row
+ */
+ public boolean handlesRow(CacheDataRow row) throws IgniteCheckedException;
Review comment:
```suggestion
public boolean canHandle(CacheDataRow row) throws IgniteCheckedException;
```
##########
File path:
modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerIndexRebuildStatusTest.java
##########
@@ -240,18 +240,14 @@ private void checkResult(CommandHandler handler, UUID...
nodeIds) {
/**
* Indexing that blocks index rebuild until status request is completed.
*/
- private static class BlockingIndexing extends IgniteH2Indexing {
- /** {@inheritDoc} */
- @Override protected void rebuildIndexesFromHash0(
- GridCacheContext cctx,
- SchemaIndexCacheVisitorClosure clo,
- GridFutureAdapter<Void> rebuildIdxFut)
- {
+ private static class BlockingIndexing extends IndexesRebuildTask {
Review comment:
```suggestion
private static class BlockingIndexesRebuildTask extends
IndexesRebuildTask {
```
##########
File path:
modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/InlineIndexKeyType.java
##########
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.cache.query.index.sorted.inline;
+
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Basic interface for inlined index columns. It's not a generic to provide
opportunity compare different types.
+ */
+public interface InlineIndexKeyType {
+ /**
+ * Returns type of inlined column.
+ *
+ * @return Integer code of the column's value type.
+ */
+ public int type();
+
+ /**
+ * Returns size of inlined key. It contains system fields too (type,
length).
+ */
+ public int inlineSize();
+
+ /**
+ * Returns required inline size for specified key.
Review comment:
Javadoc looks unclear.
Is it suggested inline size for specified key?
Does system fields are taken into account?
##########
File path:
modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/InlineRecommender.java
##########
@@ -0,0 +1,145 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.cache.query.index.sorted.inline;
+
+import java.util.Arrays;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.IgniteSystemProperties;
+import org.apache.ignite.SystemProperty;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyDefinition;
+import
org.apache.ignite.internal.cache.query.index.sorted.SortedIndexDefinition;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.IndexRow;
+import
org.apache.ignite.internal.cache.query.index.sorted.inline.io.IndexSearchRowImpl;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ * Write to a log recommendation for inline size.
+ */
+public class InlineRecommender {
+ /** @see #IGNITE_THROTTLE_INLINE_SIZE_CALCULATION */
+ public static final int DFLT_THROTTLE_INLINE_SIZE_CALCULATION = 1_000;
+
+ /** */
+ @SystemProperty(value = "How often real invocation of inline size
calculation will be skipped.", type = Long.class,
+ defaults = "" + DFLT_THROTTLE_INLINE_SIZE_CALCULATION)
+ public static final String IGNITE_THROTTLE_INLINE_SIZE_CALCULATION =
"IGNITE_THROTTLE_INLINE_SIZE_CALCULATION";
+
+ /** Counter of inline size calculation for throttling real invocations. */
+ private final ThreadLocal<Long> inlineSizeCalculationCntr =
ThreadLocal.withInitial(() -> 0L);
+
+ /** How often real invocation of inline size calculation will be skipped.
*/
+ private final int inlineSizeThrottleThreshold =
+
IgniteSystemProperties.getInteger(IGNITE_THROTTLE_INLINE_SIZE_CALCULATION,
+ DFLT_THROTTLE_INLINE_SIZE_CALCULATION);
+
+ /** Keep max calculated inline size for current index. */
+ private final AtomicInteger maxCalculatedInlineSize = new AtomicInteger();
+
+ /** Ignite logger. */
+ private final IgniteLogger log;
+
+ /** Index definition. */
+ private final SortedIndexDefinition def;
+
+ /** Constructor. */
+ public InlineRecommender(GridCacheContext cctx, SortedIndexDefinition def)
{
+ log = cctx.kernalContext().indexing().getLogger();
+ this.def = def;
+ }
+
+ /**
+ * Calculate aggregate inline size for given indexes and log
recommendation in case calculated size more than
+ * current inline size.
+ */
+ @SuppressWarnings({"ConditionalBreakInInfiniteLoop", "IfMayBeConditional"})
+ public void recommend(IndexRow row, int currInlineSize) {
+ // Do the check only for put operations.
+ if (row instanceof IndexSearchRowImpl)
+ return;
+
+ Long invokeCnt = inlineSizeCalculationCntr.get();
+
+ inlineSizeCalculationCntr.set(++invokeCnt);
+
+ boolean throttle = invokeCnt % inlineSizeThrottleThreshold != 0;
+
+ if (throttle)
+ return;
Review comment:
As I understand InlineRecommender is created on per-Index basis.
If so, we will allocate thread-local slot in thead-local table for each of
index and will never free if when index is dropped.
Usually, it doesn't looks like a good idea to allocate thread-local for user
threads (that are not ignite pool thread).
May be we can use some timestamp based approach? E.g. every 5 minutes, a
thread that successfully CAS timestamp will log the recommendation.
Or may be try to CAS a shared AtomicLong field optimistically (with no loop)
and skip this step if CAS failed due to high load?
##########
File path:
modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/InlineIndexImpl.java
##########
@@ -0,0 +1,468 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.cache.query.index.sorted.inline;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.query.index.AbstractIndex;
+import org.apache.ignite.cache.query.index.Index;
+import org.apache.ignite.cache.query.index.SingleCursor;
+import org.apache.ignite.cache.query.index.sorted.IndexKey;
+import org.apache.ignite.failure.FailureContext;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyDefinition;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexValueCursor;
+import
org.apache.ignite.internal.cache.query.index.sorted.SortedIndexDefinition;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.IndexRow;
+import
org.apache.ignite.internal.cache.query.index.sorted.inline.io.IndexRowImpl;
+import
org.apache.ignite.internal.cache.query.index.sorted.inline.io.IndexSearchRow;
+import
org.apache.ignite.internal.cache.query.index.sorted.inline.io.ThreadLocalSchemaHolder;
+import org.apache.ignite.internal.metric.IoStatisticsHolderIndex;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import
org.apache.ignite.internal.processors.cache.persistence.metastorage.pendingtask.DurableBackgroundTask;
+import org.apache.ignite.internal.util.lang.GridCursor;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.spi.indexing.IndexingQueryCacheFilter;
+import org.apache.ignite.spi.indexing.IndexingQueryFilter;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.failure.FailureType.CRITICAL_ERROR;
+
+/**
+ * Sorted index implementation.
+ */
+public class InlineIndexImpl extends AbstractIndex implements InlineIndex {
+ /** Unique ID. */
+ private final UUID id = UUID.randomUUID();
+
+ /** Segments. */
+ private final InlineIndexTree[] segments;
+
+ /** Index function. */
+ private final SortedIndexDefinition def;
+
+ /** Name of underlying tree name. */
+ private final String treeName;
+
+ /** Cache context. */
+ private final GridCacheContext<?, ?> cctx;
+
+ /** */
+ private final IoStatisticsHolderIndex stats;
+
+ /** Constructor. */
+ public InlineIndexImpl(GridCacheContext<?, ?> cctx, SortedIndexDefinition
def, InlineIndexTree[] segments,
+ IoStatisticsHolderIndex stats) {
+ this.cctx = cctx;
+ this.segments = segments.clone();
+ this.def = def;
+ treeName = def.getTreeName();
+ this.stats = stats;
+ }
+
+ /** {@inheritDoc} */
+ @Override public GridCursor<IndexRow> find(IndexKey lower, IndexKey upper,
int segment) throws IgniteCheckedException {
+ return find(lower, upper, segment, null);
+ }
+
+ /** {@inheritDoc} */
+ @Override public GridCursor<IndexRow> find(IndexKey lower, IndexKey upper,
int segment, IndexingQueryFilter filter) throws IgniteCheckedException {
+ validateConditions(lower, upper);
+
+ InlineTreeFilterClosure closure = getFilterClosure(filter);
+
+ IndexSearchRow rlower = (IndexSearchRow) lower;
+ IndexSearchRow rupper = (IndexSearchRow) upper;
+
+ // If it is known that only one row will be returned an optimization
is employed
+ if (isSingleRowLookup(rlower, rupper)) {
+ try {
+ ThreadLocalSchemaHolder.setSchema(def.getSchema());
+
+ IndexRowImpl row = segments[segment].findOne(rlower, closure,
null);
+
+ if (row == null || isExpired(row))
+ return IndexValueCursor.EMPTY;
+
+ return new SingleCursor<>(row);
+
+ } finally {
+ ThreadLocalSchemaHolder.cleanSchema();
+ }
+ }
+
+ try {
+ ThreadLocalSchemaHolder.setSchema(def.getSchema());
+
+ return segments[segment].find(rlower, rupper, closure, null);
+
+ } finally {
+ ThreadLocalSchemaHolder.cleanSchema();
+ }
+ }
+
+ /** {@inheritDoc} */
+ @Override public long count(int segment) throws IgniteCheckedException {
+ return segments[segment].size();
+ }
+
+ /** {@inheritDoc} */
+ @Override public long count(int segment, IndexingQueryFilter filter)
throws IgniteCheckedException {
+ return segments[segment].size(getFilterClosure(filter));
+ }
+
+ /**
+ * Returns number of elements in the tree by scanning pages of the bottom
(leaf) level.
+ *
+ * @return Number of elements in the tree.
+ * @throws IgniteCheckedException If failed.
+ */
+ @Override public long totalCount() throws IgniteCheckedException {
+ long ret = 0;
+
+ for (int i = 0; i < segmentsCount(); i++)
+ ret += segments[i].size();
+
+ return ret;
+ }
+
+ /** */
+ private boolean isSingleRowLookup(IndexSearchRow lower, IndexSearchRow
upper) throws IgniteCheckedException {
+ return def.isPrimary() && lower != null && lower.isFullSchemaSearch()
&& checkRowsTheSame(lower, upper);
+ }
+
+ /**
+ * Checks both rows are the same. <p/>
+ * Primarly used to verify both search rows are the same and we can apply
+ * the single row lookup optimization.
Review comment:
```suggestion
* Checks both rows are the same.
* <p/>
* Primarily used to verify if the single-row-lookup optimization can be
applied.
```
##########
File path:
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
##########
@@ -2686,7 +2685,7 @@ private void finishUpdate(GridCacheContext cctx,
CacheDataRow newRow, @Nullable
GridCacheQueryManager qryMgr = cctx.queries();
- if (qryMgr.enabled())
+ if (qryMgr.enabled() || cctx.kernalContext().indexing().enabled())
Review comment:
Is it possible to have a single flag for this?
Seems, it make no sense having Indexing enabled, but qryMgr disabled.
##########
File path:
modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/InlineIndexTree.java
##########
@@ -0,0 +1,480 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.cache.query.index.sorted.inline;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteSystemProperties;
+import org.apache.ignite.cache.query.index.sorted.SortOrder;
+import org.apache.ignite.failure.FailureType;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyDefinition;
+import
org.apache.ignite.internal.cache.query.index.sorted.SortedIndexDefinition;
+import org.apache.ignite.internal.cache.query.index.sorted.SortedIndexSchema;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.IndexRow;
+import
org.apache.ignite.internal.cache.query.index.sorted.inline.io.IndexSearchRow;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.InnerIO;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.LeafIO;
+import
org.apache.ignite.internal.cache.query.index.sorted.inline.io.ThreadLocalSchemaHolder;
+import org.apache.ignite.internal.metric.IoStatisticsHolder;
+import org.apache.ignite.internal.pagemem.FullPageId;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageMemory;
+import org.apache.ignite.internal.pagemem.wal.record.PageSnapshot;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree;
+import
org.apache.ignite.internal.processors.cache.persistence.tree.CorruptedTreeException;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO;
+import
org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusInnerIO;
+import
org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusLeafIO;
+import
org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusMetaIO;
+import
org.apache.ignite.internal.processors.cache.persistence.tree.io.IOVersions;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import
org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIoResolver;
+import
org.apache.ignite.internal.processors.cache.persistence.tree.reuse.ReuseList;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+import static
org.apache.ignite.internal.cache.query.index.sorted.inline.keys.NullableInlineIndexKeyType.CANT_BE_COMPARE;
+import static
org.apache.ignite.internal.cache.query.index.sorted.inline.keys.NullableInlineIndexKeyType.COMPARE_UNSUPPORTED;
+
+/**
+ * BPlusTree where nodes stores inlined index keys.
+ */
+public class InlineIndexTree extends BPlusTree<IndexRow, IndexRow> {
+ /** Amount of bytes to store inlined index keys. */
+ private final int inlineSize;
+
+ /** Recommends change inline size if needed. */
+ private final InlineRecommender recommender;
+
+ /** Whether tree is created from scratch or reused from underlying store.
*/
+ private final boolean created;
+
+ /** Definition of index. */
+ private final SortedIndexDefinition def;
+
+ /** Cache context. */
+ private final GridCacheContext<?, ?> cctx;
+
+ /** Statistics holder used by underlying BPlusTree. */
+ private final IoStatisticsHolder stats;
+
+ /**
+ * Constructor.
+ */
+ public InlineIndexTree(
+ SortedIndexDefinition def,
+ GridCacheContext<?, ?> cctx,
+ String treeName,
+ IgniteCacheOffheapManager offheap,
+ ReuseList reuseList,
+ PageMemory pageMemory,
+ PageIoResolver pageIoResolver,
+ long metaPageId,
+ boolean initNew,
+ int configuredInlineSize,
+ IoStatisticsHolder stats,
+ InlineRecommender recommender) throws IgniteCheckedException {
+ super(
+ treeName,
+ cctx.groupId(),
+ cctx.group().name(),
+ pageMemory,
+ cctx.shared().wal(),
+ offheap.globalRemoveId(),
+ metaPageId,
+ reuseList,
+ PageIdAllocator.FLAG_IDX,
+ cctx.shared().kernalContext().failure(),
+ null,
+ pageIoResolver
+ );
+
+ this.stats = stats;
+
+ created = initNew;
+
+ this.def = def;
+
+ if (!initNew) {
+ // Init from metastore
+ // Page is ready - read meta information.
+ MetaPageInfo metaInfo = getMetaInfo();
+
+ this.def.setUseUnwrappedPk(metaInfo.useUnwrappedPk());
+
+ inlineSize = metaInfo.inlineSize();
+
+ boolean inlineObjSupported = inlineSize > 0 &&
metaInfo.inlineObjectSupported();
+
+ if (!metaInfo.flagsSupported())
+ upgradeMetaPage(inlineObjSupported);
+
+ } else {
+ this.def.setUseUnwrappedPk(true);
+
+ inlineSize = computeInlineSize(
+ def.getSchema().getKeyDefinitions(), configuredInlineSize,
cctx.config().getSqlIndexMaxInlineSize());
+ }
+
+ if (inlineSize == 0)
+ setIos(InnerIO.VERSIONS, LeafIO.VERSIONS);
+ else
+ setIos(
+ // -1 is required as payload starts with 1, and indexes in
list of IOs are with 0.
+ (IOVersions<BPlusInnerIO<IndexRow>>)
PageIO.getInnerVersions(inlineSize - 1, false),
+ (IOVersions<BPlusLeafIO<IndexRow>>)
PageIO.getLeafVersions(inlineSize - 1, false));
+
+ initTree(initNew, inlineSize);
+
+ this.recommender = recommender;
+
+ this.cctx = cctx;
+ }
+
+ /** {@inheritDoc} */
+ @Override protected int compare(BPlusIO<IndexRow> io, long pageAddr, int
idx, IndexRow row)
+ throws IgniteCheckedException {
+ IndexSearchRow r = (IndexSearchRow) row;
+
+ int searchKeysLength = r.getSearchKeysCount();
+
+ if (inlineSize == 0)
+ return compareFullRows(getRow(io, pageAddr, idx), row, 0,
searchKeysLength);
+
+ SortedIndexSchema schema = def.getSchema();
+
+ if ((schema.getKeyDefinitions().length != searchKeysLength) &&
r.isFullSchemaSearch())
+ throw new IgniteCheckedException("Find is configured for full
schema search.");
+
+ int fieldOff = 0;
+
+ // Use it when can't compare values (variable length, for example).
+ int lastIdxUsed = searchKeysLength;
+
+ for (int i = 0; i < searchKeysLength; i++) {
+ try {
+ // If a search key is null then skip other keys (consider that
null shows that we should get all
+ // possible keys for that comparison).
+ if (row.getKey(i) == null)
+ return 0;
+
+ // Other keys are not inlined. Should compare as rows.
+ if (i >= schema.getKeyDefinitions().length) {
Review comment:
Also, there is a check few lines below
" if (!InlineIndexKeyTypeRegistry.supportInline(keyDef.getIdxType())) break"
I think, it worth to cache a slice of array of key definitions which support
inlining.
##########
File path:
modules/core/src/main/java/org/apache/ignite/cache/query/index/AbstractIndex.java
##########
@@ -15,25 +15,31 @@
* limitations under the License.
*/
-package org.apache.ignite.internal.processors.query.h2.database.inlinecolumn;
+package org.apache.ignite.cache.query.index;
-import org.h2.table.Column;
-import org.h2.value.Value;
-import org.h2.value.ValueStringIgnoreCase;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static java.lang.Boolean.FALSE;
+import static java.lang.Boolean.TRUE;
/**
- * Inline index column implementation for inlining strings ignore case.
+ * Abstract class for all Index implementations.
*/
-public class StringIgnoreCaseInlineIndexColumn extends StringInlineIndexColumn
{
+public abstract class AbstractIndex implements Index {
+ /** Whether index is rebuilding now. */
+ private final AtomicBoolean rebuildInProgress = new AtomicBoolean(false);
+
/**
- * @param col Column.
+ * @param val Mark or unmark index to rebuild.
*/
- public StringIgnoreCaseInlineIndexColumn(Column col, boolean
useOptimizedCompare) {
- super(col, Value.STRING_IGNORECASE, useOptimizedCompare, true);
+ public void markIndexRebuild(boolean val) {
+ rebuildInProgress.compareAndSet(val ? FALSE : TRUE, val ? TRUE :
FALSE);
Review comment:
```suggestion
rebuildInProgress.compareAndSet(!val, val);
```
##########
File path:
modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/InlineIndexKeyType.java
##########
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.cache.query.index.sorted.inline;
+
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Basic interface for inlined index columns. It's not a generic to provide
opportunity compare different types.
+ */
+public interface InlineIndexKeyType {
+ /**
+ * Returns type of inlined column.
+ *
+ * @return Integer code of the column's value type.
+ */
+ public int type();
+
+ /**
+ * Returns size of inlined key. It contains system fields too (type,
length).
Review comment:
```suggestion
* Returns size of inlined key.
*
* Note: system fields (e.g. type, length) are taken into account as
well.
```
##########
File path:
modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/keys/StringInlineIndexKeyType.java
##########
@@ -15,62 +15,42 @@
* limitations under the License.
*/
-package org.apache.ignite.internal.processors.query.h2.database.inlinecolumn;
+package org.apache.ignite.internal.cache.query.index.sorted.inline.keys;
import java.nio.charset.Charset;
import java.nio.charset.StandardCharsets;
+import
org.apache.ignite.internal.cache.query.index.sorted.inline.IndexKeyTypes;
import org.apache.ignite.internal.pagemem.PageUtils;
import org.apache.ignite.internal.util.GridUnsafe;
-import org.h2.table.Column;
-import org.h2.value.Value;
-import org.h2.value.ValueString;
+import org.jetbrains.annotations.Nullable;
/**
- * Inline index column implementation for inlining {@link String} values.
+ * Inline index key implementation for inlining {@link String} values.
*/
-public class StringInlineIndexColumn extends AbstractInlineIndexColumn {
+public class StringInlineIndexKeyType extends
NullableInlineIndexKeyType<String> {
/** Default charset. */
protected static final Charset CHARSET = StandardCharsets.UTF_8;
+ // TODO: how to configure it? Looks like ignoreCase may be configured on
moment of Search row creation.
Review comment:
Unrelated change.
##########
File path:
modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/IndexKeyTypes.java
##########
@@ -0,0 +1,133 @@
+/*
+ * 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.
+ public static final int * = The ASF licenses this file to You under the
Apache License, Version 2;
+ * (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
+ *
+ public static final int * = Unless required by applicable law or agreed to in
writing, softwa;
+ public static final int * = distributed under the License is distributed on
an "AS IS" BASI;
+ public static final int * = WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND,
either express or implie;
Review comment:
Looks like unrelated change.
##########
File path:
modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/io/ThreadLocalSchemaHolder.java
##########
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.cache.query.index.sorted.inline.io;
+
+import org.apache.ignite.internal.cache.query.index.sorted.SortedIndexSchema;
+
+/**
+ * Holds an index schema during work session with an index tree.
+ */
+public class ThreadLocalSchemaHolder {
+ /** */
+ private static final ThreadLocal<SortedIndexSchema> holder = new
ThreadLocal<>();
Review comment:
When you use ThreadLocal to pass some temporary context and call
ThreadLocal.clear() on hot-path
it worth to init ThreadLocal with a Holder object and update the Holder
field directly rather then set-then-clear ThreadLocal itself, because
ThreadLocal initialization requires 2 lookups.
1-st lookup just to found it is not initialized (or with 'null' value), and
the 2-nd to initialize.
##########
File path:
modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/IndexKeyTypes.java
##########
@@ -0,0 +1,133 @@
+/*
+ * 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.
+ public static final int * = The ASF licenses this file to You under the
Apache License, Version 2;
Review comment:
Unrelated change.
##########
File path:
modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/keys/StringInlineIndexKeyType.java
##########
@@ -15,62 +15,42 @@
* limitations under the License.
*/
-package org.apache.ignite.internal.processors.query.h2.database.inlinecolumn;
+package org.apache.ignite.internal.cache.query.index.sorted.inline.keys;
import java.nio.charset.Charset;
import java.nio.charset.StandardCharsets;
+import
org.apache.ignite.internal.cache.query.index.sorted.inline.IndexKeyTypes;
import org.apache.ignite.internal.pagemem.PageUtils;
import org.apache.ignite.internal.util.GridUnsafe;
-import org.h2.table.Column;
-import org.h2.value.Value;
-import org.h2.value.ValueString;
+import org.jetbrains.annotations.Nullable;
/**
- * Inline index column implementation for inlining {@link String} values.
+ * Inline index key implementation for inlining {@link String} values.
*/
-public class StringInlineIndexColumn extends AbstractInlineIndexColumn {
+public class StringInlineIndexKeyType extends
NullableInlineIndexKeyType<String> {
/** Default charset. */
protected static final Charset CHARSET = StandardCharsets.UTF_8;
+ // TODO: how to configure it? Looks like ignoreCase may be configured on
moment of Search row creation.
/** Whether respect case or not while comparing. */
private final boolean compareIgnoreCase;
- /** Whether to use optimized comparison or not. */
- private final boolean useOptimizedCompare;
+ // TODO: what is it?
+// /** Whether to use optimized comparison or not. */
+// private final boolean useOptimizedCompare;
Review comment:
See h2.value.CompareMode. Flag useOptimizedCompare is false only if
CompareMode.OFF is set.
Seems, it is some kind of workaround when inlined Strings can't be compared
directly.
Let's either keep previous behavior of drop flag and remove TODO.
##########
File path: modules/platforms/cpp/odbc-test/CMakeLists.txt
##########
@@ -62,7 +62,7 @@ set(SOURCES src/teamcity/teamcity_boost.cpp
src/errors_test.cpp
src/odbc_test_suite.cpp
src/types_test.cpp
- src/transaction_test.cpp
+# src/transaction_test.cpp
Review comment:
Whats wrong with CPP tests?
Is there any ticket for fixing them?
##########
File path:
modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/IndexValueCursor.java
##########
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.cache.query.index.sorted;
+
+import java.util.function.Function;
+import org.apache.ignite.IgniteCheckedException;
+import
org.apache.ignite.internal.cache.query.index.sorted.inline.io.IndexSearchRow;
+import org.apache.ignite.internal.util.lang.GridCursor;
+
+/**
+ * Cursor over index values.
+ *
+ * @param <V> class represents of value stored in an index.
+ */
+public class IndexValueCursor<V> implements GridCursor<V> {
+ /**
+ * Empty cursor implementation.
+ */
+ public static final GridCursor EMPTY = new GridCursor() {
+ /** {@inheritDoc} */
+ @Override public boolean next() {
+ return false;
+ }
+
+ /** {@inheritDoc} */
+ @Override public Object get() {
+ return null;
Review comment:
Agree with @korlov42 , exception will be better here.
As for now, this is contract voilation and cursor.get() may return 'null' in
both cases
if there is a null-value and if there is no values.
@timoninmaxim , let's fix this and run tests or create a separate ticket if
there are any issues. WDYT?
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]